Merge branch 'master' into feature/async_rest_client

This commit is contained in:
javanna 2016-07-22 15:51:03 +02:00 committed by Luca Cavanna
commit db8beeba3b
184 changed files with 2743 additions and 1650 deletions

View File

@ -18,14 +18,22 @@
*/
package org.elasticsearch.action.admin.indices.analyze;
import org.elasticsearch.Version;
import org.elasticsearch.action.ActionRequestValidationException;
import org.elasticsearch.action.support.single.shard.SingleShardRequest;
import org.elasticsearch.common.Strings;
import org.elasticsearch.common.io.stream.StreamInput;
import org.elasticsearch.common.io.stream.StreamOutput;
import org.elasticsearch.common.io.stream.Writeable;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.xcontent.XContentBuilder;
import org.elasticsearch.common.xcontent.XContentFactory;
import org.elasticsearch.common.xcontent.XContentType;
import java.io.IOException;
import java.util.ArrayList;
import java.util.List;
import java.util.Map;
import java.util.Objects;
import static org.elasticsearch.action.ValidateActions.addValidationError;
@ -39,11 +47,11 @@ public class AnalyzeRequest extends SingleShardRequest<AnalyzeRequest> {
private String analyzer;
private String tokenizer;
private NameOrDefinition tokenizer;
private String[] tokenFilters = Strings.EMPTY_ARRAY;
private final List<NameOrDefinition> tokenFilters = new ArrayList<>();
private String[] charFilters = Strings.EMPTY_ARRAY;
private final List<NameOrDefinition> charFilters = new ArrayList<>();
private String field;
@ -51,6 +59,48 @@ public class AnalyzeRequest extends SingleShardRequest<AnalyzeRequest> {
private String[] attributes = Strings.EMPTY_ARRAY;
public static class NameOrDefinition implements Writeable {
// exactly one of these two members is not null
public final String name;
public final Settings definition;
NameOrDefinition(String name) {
this.name = Objects.requireNonNull(name);
this.definition = null;
}
NameOrDefinition(Map<String, ?> definition) {
this.name = null;
Objects.requireNonNull(definition);
try {
XContentBuilder builder = XContentFactory.contentBuilder(XContentType.JSON);
builder.map(definition);
this.definition = Settings.builder().loadFromSource(builder.string()).build();
} catch (IOException e) {
throw new IllegalArgumentException("Failed to parse [" + definition + "]", e);
}
}
NameOrDefinition(StreamInput in) throws IOException {
name = in.readOptionalString();
if (in.readBoolean()) {
definition = Settings.readSettingsFromStream(in);
} else {
definition = null;
}
}
@Override
public void writeTo(StreamOutput out) throws IOException {
out.writeOptionalString(name);
boolean isNotNullDefinition = this.definition != null;
out.writeBoolean(isNotNullDefinition);
if (isNotNullDefinition) {
Settings.writeSettingsToStream(definition, out);
}
}
}
public AnalyzeRequest() {
}
@ -82,35 +132,43 @@ public class AnalyzeRequest extends SingleShardRequest<AnalyzeRequest> {
}
public AnalyzeRequest tokenizer(String tokenizer) {
this.tokenizer = tokenizer;
this.tokenizer = new NameOrDefinition(tokenizer);
return this;
}
public String tokenizer() {
public AnalyzeRequest tokenizer(Map<String, ?> tokenizer) {
this.tokenizer = new NameOrDefinition(tokenizer);
return this;
}
public NameOrDefinition tokenizer() {
return this.tokenizer;
}
public AnalyzeRequest tokenFilters(String... tokenFilters) {
if (tokenFilters == null) {
throw new IllegalArgumentException("token filters must not be null");
}
this.tokenFilters = tokenFilters;
public AnalyzeRequest addTokenFilter(String tokenFilter) {
this.tokenFilters.add(new NameOrDefinition(tokenFilter));
return this;
}
public String[] tokenFilters() {
public AnalyzeRequest addTokenFilter(Map<String, ?> tokenFilter) {
this.tokenFilters.add(new NameOrDefinition(tokenFilter));
return this;
}
public List<NameOrDefinition> tokenFilters() {
return this.tokenFilters;
}
public AnalyzeRequest charFilters(String... charFilters) {
if (charFilters == null) {
throw new IllegalArgumentException("char filters must not be null");
}
this.charFilters = charFilters;
public AnalyzeRequest addCharFilter(Map<String, ?> charFilter) {
this.charFilters.add(new NameOrDefinition(charFilter));
return this;
}
public String[] charFilters() {
public AnalyzeRequest addCharFilter(String charFilter) {
this.charFilters.add(new NameOrDefinition(charFilter));
return this;
}
public List<NameOrDefinition> charFilters() {
return this.charFilters;
}
@ -158,14 +216,12 @@ public class AnalyzeRequest extends SingleShardRequest<AnalyzeRequest> {
super.readFrom(in);
text = in.readStringArray();
analyzer = in.readOptionalString();
tokenizer = in.readOptionalString();
tokenFilters = in.readStringArray();
charFilters = in.readStringArray();
tokenizer = in.readOptionalWriteable(NameOrDefinition::new);
tokenFilters.addAll(in.readList(NameOrDefinition::new));
charFilters.addAll(in.readList(NameOrDefinition::new));
field = in.readOptionalString();
if (in.getVersion().onOrAfter(Version.V_2_2_0)) {
explain = in.readBoolean();
attributes = in.readStringArray();
}
explain = in.readBoolean();
attributes = in.readStringArray();
}
@Override
@ -173,13 +229,11 @@ public class AnalyzeRequest extends SingleShardRequest<AnalyzeRequest> {
super.writeTo(out);
out.writeStringArray(text);
out.writeOptionalString(analyzer);
out.writeOptionalString(tokenizer);
out.writeStringArray(tokenFilters);
out.writeStringArray(charFilters);
out.writeOptionalWriteable(tokenizer);
out.writeList(tokenFilters);
out.writeList(charFilters);
out.writeOptionalString(field);
if (out.getVersion().onOrAfter(Version.V_2_2_0)) {
out.writeBoolean(explain);
out.writeStringArray(attributes);
}
out.writeBoolean(explain);
out.writeStringArray(attributes);
}
}

View File

@ -21,6 +21,8 @@ package org.elasticsearch.action.admin.indices.analyze;
import org.elasticsearch.action.support.single.shard.SingleShardOperationRequestBuilder;
import org.elasticsearch.client.ElasticsearchClient;
import java.util.Map;
/**
*
*/
@ -54,7 +56,7 @@ public class AnalyzeRequestBuilder extends SingleShardOperationRequestBuilder<An
}
/**
* Instead of setting the analyzer, sets the tokenizer that will be used as part of a custom
* Instead of setting the analyzer, sets the tokenizer as name that will be used as part of a custom
* analyzer.
*/
public AnalyzeRequestBuilder setTokenizer(String tokenizer) {
@ -63,18 +65,43 @@ public class AnalyzeRequestBuilder extends SingleShardOperationRequestBuilder<An
}
/**
* Sets token filters that will be used on top of a tokenizer provided.
* Instead of setting the analyzer, sets the tokenizer using custom settings that will be used as part of a custom
* analyzer.
*/
public AnalyzeRequestBuilder setTokenFilters(String... tokenFilters) {
request.tokenFilters(tokenFilters);
public AnalyzeRequestBuilder setTokenizer(Map<String, ?> tokenizer) {
request.tokenizer(tokenizer);
return this;
}
/**
* Sets char filters that will be used before the tokenizer.
* Add token filter setting that will be used on top of a tokenizer provided.
*/
public AnalyzeRequestBuilder setCharFilters(String... charFilters) {
request.charFilters(charFilters);
public AnalyzeRequestBuilder addTokenFilter(Map<String, ?> tokenFilter) {
request.addTokenFilter(tokenFilter);
return this;
}
/**
* Add a name of token filter that will be used on top of a tokenizer provided.
*/
public AnalyzeRequestBuilder addTokenFilter(String tokenFilter) {
request.addTokenFilter(tokenFilter);
return this;
}
/**
* Add char filter setting that will be used on top of a tokenizer provided.
*/
public AnalyzeRequestBuilder addCharFilter(Map<String, ?> charFilter) {
request.addCharFilter(charFilter);
return this;
}
/**
* Add a name of char filter that will be used before the tokenizer.
*/
public AnalyzeRequestBuilder addCharFilter(String tokenFilter) {
request.addCharFilter(tokenFilter);
return this;
}

View File

@ -25,23 +25,25 @@ import org.apache.lucene.analysis.tokenattributes.CharTermAttribute;
import org.apache.lucene.analysis.tokenattributes.OffsetAttribute;
import org.apache.lucene.analysis.tokenattributes.PositionIncrementAttribute;
import org.apache.lucene.analysis.tokenattributes.TypeAttribute;
import org.apache.lucene.util.Attribute;
import org.apache.lucene.util.AttributeReflector;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.IOUtils;
import org.elasticsearch.ElasticsearchException;
import org.elasticsearch.Version;
import org.elasticsearch.action.support.ActionFilters;
import org.elasticsearch.action.support.single.shard.TransportSingleShardAction;
import org.elasticsearch.cluster.ClusterState;
import org.elasticsearch.cluster.block.ClusterBlockException;
import org.elasticsearch.cluster.metadata.IndexMetaData;
import org.elasticsearch.cluster.metadata.IndexNameExpressionResolver;
import org.elasticsearch.cluster.routing.ShardsIterator;
import org.elasticsearch.cluster.service.ClusterService;
import org.elasticsearch.common.UUIDs;
import org.elasticsearch.common.inject.Inject;
import org.elasticsearch.common.io.FastStringReader;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.env.Environment;
import org.elasticsearch.index.IndexService;
import org.elasticsearch.index.IndexSettings;
import org.elasticsearch.index.analysis.AnalysisRegistry;
import org.elasticsearch.index.analysis.AnalysisService;
import org.elasticsearch.index.analysis.CharFilterFactory;
@ -167,65 +169,13 @@ public class TransportAnalyzeAction extends TransportSingleShardAction<AnalyzeRe
}
} else if (request.tokenizer() != null) {
TokenizerFactory tokenizerFactory;
if (analysisService == null) {
AnalysisModule.AnalysisProvider<TokenizerFactory> tokenizerFactoryFactory = analysisRegistry.getTokenizerProvider(request.tokenizer());
if (tokenizerFactoryFactory == null) {
throw new IllegalArgumentException("failed to find global tokenizer under [" + request.tokenizer() + "]");
}
tokenizerFactory = tokenizerFactoryFactory.get(environment, request.tokenizer());
} else {
tokenizerFactory = analysisService.tokenizer(request.tokenizer());
if (tokenizerFactory == null) {
throw new IllegalArgumentException("failed to find tokenizer under [" + request.tokenizer() + "]");
}
}
TokenizerFactory tokenizerFactory = parseTokenizerFactory(request, analysisService, analysisRegistry, environment);
TokenFilterFactory[] tokenFilterFactories = new TokenFilterFactory[0];
if (request.tokenFilters() != null && request.tokenFilters().length > 0) {
tokenFilterFactories = new TokenFilterFactory[request.tokenFilters().length];
for (int i = 0; i < request.tokenFilters().length; i++) {
String tokenFilterName = request.tokenFilters()[i];
if (analysisService == null) {
AnalysisModule.AnalysisProvider<TokenFilterFactory> tokenFilterFactoryFactory = analysisRegistry.getTokenFilterProvider(tokenFilterName);
if (tokenFilterFactoryFactory == null) {
throw new IllegalArgumentException("failed to find global token filter under [" + tokenFilterName + "]");
}
tokenFilterFactories[i] = tokenFilterFactoryFactory.get(environment, tokenFilterName);
} else {
tokenFilterFactories[i] = analysisService.tokenFilter(tokenFilterName);
if (tokenFilterFactories[i] == null) {
throw new IllegalArgumentException("failed to find token filter under [" + tokenFilterName + "]");
}
}
if (tokenFilterFactories[i] == null) {
throw new IllegalArgumentException("failed to find token filter under [" + tokenFilterName + "]");
}
}
}
tokenFilterFactories = getTokenFilterFactories(request, analysisService, analysisRegistry, environment, tokenFilterFactories);
CharFilterFactory[] charFilterFactories = new CharFilterFactory[0];
if (request.charFilters() != null && request.charFilters().length > 0) {
charFilterFactories = new CharFilterFactory[request.charFilters().length];
for (int i = 0; i < request.charFilters().length; i++) {
String charFilterName = request.charFilters()[i];
if (analysisService == null) {
AnalysisModule.AnalysisProvider<CharFilterFactory> charFilterFactoryFactory = analysisRegistry.getCharFilterProvider(charFilterName);
if (charFilterFactoryFactory == null) {
throw new IllegalArgumentException("failed to find global char filter under [" + charFilterName + "]");
}
charFilterFactories[i] = charFilterFactoryFactory.get(environment, charFilterName);
} else {
charFilterFactories[i] = analysisService.charFilter(charFilterName);
if (charFilterFactories[i] == null) {
throw new IllegalArgumentException("failed to find char filter under [" + charFilterName + "]");
}
}
if (charFilterFactories[i] == null) {
throw new IllegalArgumentException("failed to find char filter under [" + charFilterName + "]");
}
}
}
charFilterFactories = getCharFilterFactories(request, analysisService, analysisRegistry, environment, charFilterFactories);
analyzer = new CustomAnalyzer(tokenizerFactory, charFilterFactories, tokenFilterFactories);
closeAnalyzer = true;
@ -407,8 +357,9 @@ public class TransportAnalyzeAction extends TransportSingleShardAction<AnalyzeRe
} catch (IOException e) {
throw new ElasticsearchException("failed to analyze (charFiltering)", e);
}
if (len > 0)
if (len > 0) {
sb.append(buf, 0, len);
}
} while (len == BUFFER_SIZE);
return sb.toString();
}
@ -436,7 +387,7 @@ public class TransportAnalyzeAction extends TransportSingleShardAction<AnalyzeRe
lastPosition = lastPosition + increment;
}
tokens.add(new AnalyzeResponse.AnalyzeToken(term.toString(), lastPosition, lastOffset + offset.startOffset(),
lastOffset +offset.endOffset(), type.type(), extractExtendedAttributes(stream, includeAttributes)));
lastOffset + offset.endOffset(), type.type(), extractExtendedAttributes(stream, includeAttributes)));
}
stream.end();
@ -470,27 +421,164 @@ public class TransportAnalyzeAction extends TransportSingleShardAction<AnalyzeRe
private static Map<String, Object> extractExtendedAttributes(TokenStream stream, final Set<String> includeAttributes) {
final Map<String, Object> extendedAttributes = new TreeMap<>();
stream.reflectWith(new AttributeReflector() {
@Override
public void reflect(Class<? extends Attribute> attClass, String key, Object value) {
if (CharTermAttribute.class.isAssignableFrom(attClass))
return;
if (PositionIncrementAttribute.class.isAssignableFrom(attClass))
return;
if (OffsetAttribute.class.isAssignableFrom(attClass))
return;
if (TypeAttribute.class.isAssignableFrom(attClass))
return;
if (includeAttributes == null || includeAttributes.isEmpty() || includeAttributes.contains(key.toLowerCase(Locale.ROOT))) {
if (value instanceof BytesRef) {
final BytesRef p = (BytesRef) value;
value = p.toString();
}
extendedAttributes.put(key, value);
stream.reflectWith((attClass, key, value) -> {
if (CharTermAttribute.class.isAssignableFrom(attClass)) {
return;
}
if (PositionIncrementAttribute.class.isAssignableFrom(attClass)) {
return;
}
if (OffsetAttribute.class.isAssignableFrom(attClass)) {
return;
}
if (TypeAttribute.class.isAssignableFrom(attClass)) {
return;
}
if (includeAttributes == null || includeAttributes.isEmpty() || includeAttributes.contains(key.toLowerCase(Locale.ROOT))) {
if (value instanceof BytesRef) {
final BytesRef p = (BytesRef) value;
value = p.toString();
}
extendedAttributes.put(key, value);
}
});
return extendedAttributes;
}
private static CharFilterFactory[] getCharFilterFactories(AnalyzeRequest request, AnalysisService analysisService, AnalysisRegistry analysisRegistry,
Environment environment, CharFilterFactory[] charFilterFactories) throws IOException {
if (request.charFilters() != null && request.charFilters().size() > 0) {
charFilterFactories = new CharFilterFactory[request.charFilters().size()];
for (int i = 0; i < request.charFilters().size(); i++) {
final AnalyzeRequest.NameOrDefinition charFilter = request.charFilters().get(i);
// parse anonymous settings
if (charFilter.definition != null) {
Settings settings = getAnonymousSettings(charFilter.definition);
String charFilterTypeName = settings.get("type");
if (charFilterTypeName == null) {
throw new IllegalArgumentException("Missing [type] setting for anonymous char filter: " + charFilter.definition);
}
AnalysisModule.AnalysisProvider<CharFilterFactory> charFilterFactoryFactory =
analysisRegistry.getCharFilterProvider(charFilterTypeName);
if (charFilterFactoryFactory == null) {
throw new IllegalArgumentException("failed to find global char filter under [" + charFilterTypeName + "]");
}
// Need to set anonymous "name" of char_filter
charFilterFactories[i] = charFilterFactoryFactory.get(getNaIndexSettings(settings), environment, "_anonymous_charfilter_[" + i + "]", settings);
} else {
if (analysisService == null) {
AnalysisModule.AnalysisProvider<CharFilterFactory> 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) {
throw new IllegalArgumentException("failed to find char filter under [" + charFilter.name + "]");
}
}
}
if (charFilterFactories[i] == null) {
throw new IllegalArgumentException("failed to find char filter under [" + charFilter.name + "]");
}
}
}
return charFilterFactories;
}
private static TokenFilterFactory[] getTokenFilterFactories(AnalyzeRequest request, AnalysisService analysisService, AnalysisRegistry analysisRegistry,
Environment environment, TokenFilterFactory[] tokenFilterFactories) throws IOException {
if (request.tokenFilters() != null && request.tokenFilters().size() > 0) {
tokenFilterFactories = new TokenFilterFactory[request.tokenFilters().size()];
for (int i = 0; i < request.tokenFilters().size(); i++) {
final AnalyzeRequest.NameOrDefinition tokenFilter = request.tokenFilters().get(i);
// parse anonymous settings
if (tokenFilter.definition != null) {
Settings settings = getAnonymousSettings(tokenFilter.definition);
String filterTypeName = settings.get("type");
if (filterTypeName == null) {
throw new IllegalArgumentException("Missing [type] setting for anonymous token filter: " + tokenFilter.definition);
}
AnalysisModule.AnalysisProvider<TokenFilterFactory> tokenFilterFactoryFactory =
analysisRegistry.getTokenFilterProvider(filterTypeName);
if (tokenFilterFactoryFactory == null) {
throw new IllegalArgumentException("failed to find global token filter under [" + filterTypeName + "]");
}
// Need to set anonymous "name" of tokenfilter
tokenFilterFactories[i] = tokenFilterFactoryFactory.get(getNaIndexSettings(settings), environment, "_anonymous_tokenfilter_[" + i + "]", settings);
} else {
if (analysisService == null) {
AnalysisModule.AnalysisProvider<TokenFilterFactory> 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) {
throw new IllegalArgumentException("failed to find token filter under [" + tokenFilter.name + "]");
}
}
}
if (tokenFilterFactories[i] == null) {
throw new IllegalArgumentException("failed to find or create token filter under [" + tokenFilter.name + "]");
}
}
}
return tokenFilterFactories;
}
private static TokenizerFactory parseTokenizerFactory(AnalyzeRequest request, AnalysisService analysisService,
AnalysisRegistry analysisRegistry, Environment environment) throws IOException {
TokenizerFactory tokenizerFactory;
final AnalyzeRequest.NameOrDefinition tokenizer = request.tokenizer();
// parse anonymous settings
if (tokenizer.definition != null) {
Settings settings = getAnonymousSettings(tokenizer.definition);
String tokenizerTypeName = settings.get("type");
if (tokenizerTypeName == null) {
throw new IllegalArgumentException("Missing [type] setting for anonymous tokenizer: " + tokenizer.definition);
}
AnalysisModule.AnalysisProvider<TokenizerFactory> tokenizerFactoryFactory =
analysisRegistry.getTokenizerProvider(tokenizerTypeName);
if (tokenizerFactoryFactory == null) {
throw new IllegalArgumentException("failed to find global tokenizer under [" + tokenizerTypeName + "]");
}
// Need to set anonymous "name" of tokenizer
tokenizerFactory = tokenizerFactoryFactory.get(getNaIndexSettings(settings), environment, "_anonymous_tokenizer", settings);
} else {
if (analysisService == null) {
AnalysisModule.AnalysisProvider<TokenizerFactory> 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) {
throw new IllegalArgumentException("failed to find tokenizer under [" + tokenizer.name + "]");
}
}
}
return tokenizerFactory;
}
private static IndexSettings getNaIndexSettings(Settings settings) {
IndexMetaData metaData = IndexMetaData.builder(IndexMetaData.INDEX_UUID_NA_VALUE).settings(settings).build();
return new IndexSettings(metaData, Settings.EMPTY);
}
private static Settings getAnonymousSettings(Settings providerSetting) {
return Settings.builder().put(providerSetting)
// for _na_
.put(IndexMetaData.SETTING_VERSION_CREATED, Version.CURRENT)
.put(IndexMetaData.SETTING_NUMBER_OF_REPLICAS, 0)
.put(IndexMetaData.SETTING_NUMBER_OF_SHARDS, 1)
.put(IndexMetaData.SETTING_INDEX_UUID, UUIDs.randomBase64UUID())
.build();
}
}

View File

@ -43,7 +43,7 @@ public class ClusterInfo implements ToXContent, Writeable {
private final ImmutableOpenMap<String, DiskUsage> mostAvailableSpaceUsage;
final ImmutableOpenMap<String, Long> shardSizes;
public static final ClusterInfo EMPTY = new ClusterInfo();
private final ImmutableOpenMap<ShardRouting, String> routingToDataPath;
final ImmutableOpenMap<ShardRouting, String> routingToDataPath;
protected ClusterInfo() {
this(ImmutableOpenMap.of(), ImmutableOpenMap.of(), ImmutableOpenMap.of(), ImmutableOpenMap.of());
@ -68,29 +68,10 @@ public class ClusterInfo implements ToXContent, Writeable {
}
public ClusterInfo(StreamInput in) throws IOException {
int size = in.readInt();
Map<String, DiskUsage> leastMap = new HashMap<>(size);
for (int i = 0; i < size; i++) {
leastMap.put(in.readString(), new DiskUsage(in));
}
size = in.readInt();
Map<String, DiskUsage> mostMap = new HashMap<>(size);
for (int i = 0; i < size; i++) {
mostMap.put(in.readString(), new DiskUsage(in));
}
size = in.readInt();
Map<String, Long> sizeMap = new HashMap<>(size);
for (int i = 0; i < size; i++) {
sizeMap.put(in.readString(), in.readLong());
}
size = in.readInt();
Map<ShardRouting, String> routingMap = new HashMap<>(size);
for (int i = 0; i < size; i++) {
routingMap.put(new ShardRouting(in), in.readString());
}
Map<String, DiskUsage> leastMap = in.readMap(StreamInput::readString, DiskUsage::new);
Map<String, DiskUsage> mostMap = in.readMap(StreamInput::readString, DiskUsage::new);
Map<String, Long> sizeMap = in.readMap(StreamInput::readString, StreamInput::readLong);
Map<ShardRouting, String> routingMap = in.readMap(ShardRouting::new, StreamInput::readString);
ImmutableOpenMap.Builder<String, DiskUsage> leastBuilder = ImmutableOpenMap.builder();
this.leastAvailableSpaceUsage = leastBuilder.putAll(leastMap).build();

View File

@ -25,10 +25,11 @@ import org.elasticsearch.common.io.stream.StreamOutput;
import org.elasticsearch.common.io.stream.Writeable;
import org.elasticsearch.common.unit.ByteSizeValue;
import org.elasticsearch.common.xcontent.ToXContent;
import org.elasticsearch.common.xcontent.ToXContent.Params;
import org.elasticsearch.common.xcontent.XContentBuilder;
import java.io.IOException;
import java.util.Objects;
/**
* Encapsulation class used to represent the amount of disk used on a node.
*/
@ -126,6 +127,24 @@ public class DiskUsage implements ToXContent, Writeable {
return getTotalBytes() - getFreeBytes();
}
@Override
public boolean equals(Object o) {
if (this == o) return true;
if (o == null || getClass() != o.getClass()) return false;
DiskUsage other = (DiskUsage) o;
return Objects.equals(nodeId, other.nodeId) &&
Objects.equals(nodeName, other.nodeName) &&
Objects.equals(totalBytes, other.totalBytes) &&
Objects.equals(freeBytes, other.freeBytes);
}
@Override
public int hashCode() {
return Objects.hash(nodeId, nodeName, path, totalBytes, freeBytes);
}
@Override
public String toString() {
return "[" + nodeId + "][" + nodeName + "][" + path + "] free: " + new ByteSizeValue(getFreeBytes()) +

View File

@ -411,7 +411,8 @@ public class ClusterService extends AbstractLifecycleComponent {
List<UpdateTask> existingTasks = updateTasksPerExecutor.computeIfAbsent(executor, k -> new ArrayList<>());
for (@SuppressWarnings("unchecked") UpdateTask<T> existing : existingTasks) {
if (tasksIdentity.containsKey(existing.task)) {
throw new IllegalStateException("task [" + existing.task + "] with source [" + source + "] is already queued");
throw new IllegalStateException("task [" + executor.describeTasks(Collections.singletonList(existing.task)) +
"] with source [" + source + "] is already queued");
}
}
existingTasks.addAll(updateTasks);
@ -517,11 +518,11 @@ public class ClusterService extends AbstractLifecycleComponent {
if (pending != null) {
for (UpdateTask<T> task : pending) {
if (task.processed.getAndSet(true) == false) {
logger.trace("will process [{}[{}]]", task.source, task.task);
logger.trace("will process {}", task.toString(executor));
toExecute.add(task);
processTasksBySource.computeIfAbsent(task.source, s -> new ArrayList<>()).add(task.task);
} else {
logger.trace("skipping [{}[{}]], already processed", task.source, task.task);
logger.trace("skipping {}, already processed", task.toString(executor));
}
}
}
@ -571,7 +572,8 @@ public class ClusterService extends AbstractLifecycleComponent {
assert (assertsEnabled = true);
if (assertsEnabled) {
for (UpdateTask<T> updateTask : toExecute) {
assert batchResult.executionResults.containsKey(updateTask.task) : "missing task result for [" + updateTask.task + "]";
assert batchResult.executionResults.containsKey(updateTask.task) :
"missing task result for " + updateTask.toString(executor);
}
}
@ -579,13 +581,13 @@ public class ClusterService extends AbstractLifecycleComponent {
final ArrayList<UpdateTask<T>> proccessedListeners = new ArrayList<>();
// fail all tasks that have failed and extract those that are waiting for results
for (UpdateTask<T> updateTask : toExecute) {
assert batchResult.executionResults.containsKey(updateTask.task) : "missing " + updateTask.task.toString();
assert batchResult.executionResults.containsKey(updateTask.task) : "missing " + updateTask.toString(executor);
final ClusterStateTaskExecutor.TaskResult executionResult =
batchResult.executionResults.get(updateTask.task);
executionResult.handle(
() -> proccessedListeners.add(updateTask),
ex -> {
logger.debug("cluster state update task [{}] failed", ex, updateTask.source);
logger.debug("cluster state update task {} failed", ex, updateTask.toString(executor));
updateTask.listener.onFailure(updateTask.source, ex);
}
);
@ -854,6 +856,15 @@ public class ClusterService extends AbstractLifecycleComponent {
public void run() {
runTasksForExecutor(executor);
}
public String toString(ClusterStateTaskExecutor<T> executor) {
String taskDescription = executor.describeTasks(Collections.singletonList(task));
if (taskDescription.isEmpty()) {
return "[" + source + "]";
} else {
return "[" + source + "[" + taskDescription + "]]";
}
}
}
private void warnAboutSlowTaskIfNeeded(TimeValue executionTime, String source) {

View File

@ -420,6 +420,17 @@ public abstract class StreamInput extends InputStream {
return null;
}
public <K, V> Map<K, V> readMap(Writeable.Reader<K> keyReader, Writeable.Reader<V> valueReader) throws IOException {
int size = readVInt();
Map<K, V> map = new HashMap<>(size);
for (int i = 0; i < size; i++) {
K key = keyReader.read(this);
V value = valueReader.read(this);
map.put(key, value);
}
return map;
}
@Nullable
@SuppressWarnings("unchecked")
public Map<String, Object> readMap() throws IOException {
@ -825,6 +836,18 @@ public abstract class StreamInput extends InputStream {
return builder;
}
/**
* Reads a list of {@link NamedWriteable}s.
*/
public <T extends NamedWriteable> List<T> readNamedWriteableList(Class<T> categoryClass) throws IOException {
int count = readVInt();
List<T> builder = new ArrayList<>(count);
for (int i=0; i<count; i++) {
builder.add(readNamedWriteable(categoryClass));
}
return builder;
}
public static StreamInput wrap(byte[] bytes) {
return wrap(bytes, 0, bytes.length);
}

View File

@ -835,4 +835,14 @@ public abstract class StreamOutput extends OutputStream {
obj.writeTo(this);
}
}
/**
* Writes a list of {@link NamedWriteable} objects.
*/
public void writeNamedWriteableList(List<? extends NamedWriteable> list) throws IOException {
writeVInt(list.size());
for (NamedWriteable obj: list) {
writeNamedWriteable(obj);
}
}
}

View File

@ -55,6 +55,9 @@ public class NetworkExceptionHelper {
if (e.getMessage().contains("Connection timed out")) {
return true;
}
if (e.getMessage().equals("Socket is closed")) {
return true;
}
}
return false;
}

View File

@ -20,6 +20,7 @@
package org.elasticsearch.common.util.concurrent;
import org.elasticsearch.common.Nullable;
import org.elasticsearch.threadpool.ThreadPool;
import org.elasticsearch.transport.Transports;
import java.util.Objects;
@ -32,6 +33,8 @@ import java.util.concurrent.locks.AbstractQueuedSynchronizer;
public abstract class BaseFuture<V> implements Future<V> {
private static final String BLOCKING_OP_REASON = "Blocking operation";
/**
* Synchronization control for AbstractFutures.
*/
@ -56,7 +59,8 @@ public abstract class BaseFuture<V> implements Future<V> {
@Override
public V get(long timeout, TimeUnit unit) throws InterruptedException,
TimeoutException, ExecutionException {
assert timeout <= 0 || Transports.assertNotTransportThread("Blocking operation");
assert timeout <= 0 ||
(Transports.assertNotTransportThread(BLOCKING_OP_REASON) && ThreadPool.assertNotScheduleThread(BLOCKING_OP_REASON));
return sync.get(unit.toNanos(timeout));
}
@ -78,7 +82,7 @@ public abstract class BaseFuture<V> implements Future<V> {
*/
@Override
public V get() throws InterruptedException, ExecutionException {
assert Transports.assertNotTransportThread("Blocking operation");
assert Transports.assertNotTransportThread(BLOCKING_OP_REASON) && ThreadPool.assertNotScheduleThread(BLOCKING_OP_REASON);
return sync.get();
}

View File

@ -50,6 +50,13 @@ public class ParseFieldRegistry<T> {
return registry.keySet();
}
/**
* Register a parser.
*/
public void register(T value, String name) {
register(value, new ParseField(name));
}
/**
* Register a parser.
*/

View File

@ -28,8 +28,11 @@ import org.elasticsearch.index.IndexSettings;
import org.elasticsearch.index.mapper.core.TextFieldMapper;
import java.io.Closeable;
import java.util.Arrays;
import java.util.HashMap;
import java.util.HashSet;
import java.util.Map;
import java.util.Set;
import static java.util.Collections.unmodifiableMap;
@ -58,69 +61,34 @@ public class AnalysisService extends AbstractIndexComponent implements Closeable
this.tokenFilters = unmodifiableMap(tokenFilterFactoryFactories);
analyzerProviders = new HashMap<>(analyzerProviders);
if (!analyzerProviders.containsKey("default")) {
analyzerProviders.put("default", new StandardAnalyzerProvider(indexSettings, null, "default", Settings.Builder.EMPTY_SETTINGS));
}
if (!analyzerProviders.containsKey("default_search")) {
analyzerProviders.put("default_search", analyzerProviders.get("default"));
}
if (!analyzerProviders.containsKey("default_search_quoted")) {
analyzerProviders.put("default_search_quoted", analyzerProviders.get("default_search"));
}
Map<String, NamedAnalyzer> analyzerAliases = new HashMap<>();
Map<String, NamedAnalyzer> analyzers = new HashMap<>();
for (Map.Entry<String, AnalyzerProvider<?>> entry : analyzerProviders.entrySet()) {
AnalyzerProvider<?> analyzerFactory = entry.getValue();
String name = entry.getKey();
/*
* Lucene defaults positionIncrementGap to 0 in all analyzers but
* Elasticsearch defaults them to 0 only before version 2.0
* and 100 afterwards so we override the positionIncrementGap if it
* doesn't match here.
*/
int overridePositionIncrementGap = TextFieldMapper.Defaults.POSITION_INCREMENT_GAP;
if (analyzerFactory instanceof CustomAnalyzerProvider) {
((CustomAnalyzerProvider) analyzerFactory).build(this);
/*
* Custom analyzers already default to the correct, version
* dependent positionIncrementGap and the user is be able to
* configure the positionIncrementGap directly on the analyzer so
* we disable overriding the positionIncrementGap to preserve the
* user's setting.
*/
overridePositionIncrementGap = Integer.MIN_VALUE;
}
Analyzer analyzerF = analyzerFactory.get();
if (analyzerF == null) {
throw new IllegalArgumentException("analyzer [" + analyzerFactory.name() + "] created null analyzer");
}
NamedAnalyzer analyzer;
if (analyzerF instanceof NamedAnalyzer) {
// if we got a named analyzer back, use it...
analyzer = (NamedAnalyzer) analyzerF;
if (overridePositionIncrementGap >= 0 && analyzer.getPositionIncrementGap(analyzer.name()) != overridePositionIncrementGap) {
// unless the positionIncrementGap needs to be overridden
analyzer = new NamedAnalyzer(analyzer, overridePositionIncrementGap);
}
processAnalyzerFactory(entry.getKey(), entry.getValue(), analyzerAliases, analyzers);
}
for (Map.Entry<String, NamedAnalyzer> entry : analyzerAliases.entrySet()) {
String key = entry.getKey();
if (analyzers.containsKey(key) &&
("default".equals(key) || "default_search".equals(key) || "default_search_quoted".equals(key)) == false) {
throw new IllegalStateException("already registered analyzer with name: " + key);
} else {
analyzer = new NamedAnalyzer(name, analyzerFactory.scope(), analyzerF, overridePositionIncrementGap);
}
if (analyzers.containsKey(name)) {
throw new IllegalStateException("already registered analyzer with name: " + name);
}
analyzers.put(name, analyzer);
String strAliases = this.indexSettings.getSettings().get("index.analysis.analyzer." + analyzerFactory.name() + ".alias");
if (strAliases != null) {
for (String alias : Strings.commaDelimitedListToStringArray(strAliases)) {
analyzers.put(alias, analyzer);
}
}
String[] aliases = this.indexSettings.getSettings().getAsArray("index.analysis.analyzer." + analyzerFactory.name() + ".alias");
for (String alias : aliases) {
analyzers.put(alias, analyzer);
NamedAnalyzer configured = entry.getValue();
analyzers.put(key, configured);
}
}
if (!analyzers.containsKey("default")) {
processAnalyzerFactory("default", new StandardAnalyzerProvider(indexSettings, null, "default", Settings.Builder.EMPTY_SETTINGS),
analyzerAliases, analyzers);
}
if (!analyzers.containsKey("default_search")) {
analyzers.put("default_search", analyzers.get("default"));
}
if (!analyzers.containsKey("default_search_quoted")) {
analyzers.put("default_search_quoted", analyzers.get("default_search"));
}
NamedAnalyzer defaultAnalyzer = analyzers.get("default");
if (defaultAnalyzer == null) {
throw new IllegalArgumentException("no default analyzer configured");
@ -145,6 +113,58 @@ public class AnalysisService extends AbstractIndexComponent implements Closeable
this.analyzers = unmodifiableMap(analyzers);
}
private void processAnalyzerFactory(String name, AnalyzerProvider<?> analyzerFactory, Map<String, NamedAnalyzer> analyzerAliases, Map<String, NamedAnalyzer> analyzers) {
/*
* Lucene defaults positionIncrementGap to 0 in all analyzers but
* Elasticsearch defaults them to 0 only before version 2.0
* and 100 afterwards so we override the positionIncrementGap if it
* doesn't match here.
*/
int overridePositionIncrementGap = TextFieldMapper.Defaults.POSITION_INCREMENT_GAP;
if (analyzerFactory instanceof CustomAnalyzerProvider) {
((CustomAnalyzerProvider) analyzerFactory).build(this);
/*
* Custom analyzers already default to the correct, version
* dependent positionIncrementGap and the user is be able to
* configure the positionIncrementGap directly on the analyzer so
* we disable overriding the positionIncrementGap to preserve the
* user's setting.
*/
overridePositionIncrementGap = Integer.MIN_VALUE;
}
Analyzer analyzerF = analyzerFactory.get();
if (analyzerF == null) {
throw new IllegalArgumentException("analyzer [" + analyzerFactory.name() + "] created null analyzer");
}
NamedAnalyzer analyzer;
if (analyzerF instanceof NamedAnalyzer) {
// if we got a named analyzer back, use it...
analyzer = (NamedAnalyzer) analyzerF;
if (overridePositionIncrementGap >= 0 && analyzer.getPositionIncrementGap(analyzer.name()) != overridePositionIncrementGap) {
// unless the positionIncrementGap needs to be overridden
analyzer = new NamedAnalyzer(analyzer, overridePositionIncrementGap);
}
} else {
analyzer = new NamedAnalyzer(name, analyzerFactory.scope(), analyzerF, overridePositionIncrementGap);
}
if (analyzers.containsKey(name)) {
throw new IllegalStateException("already registered analyzer with name: " + name);
}
analyzers.put(name, analyzer);
String strAliases = this.indexSettings.getSettings().get("index.analysis.analyzer." + analyzerFactory.name() + ".alias");
Set<String> aliases = new HashSet<>();
if (strAliases != null) {
aliases.addAll(Strings.commaDelimitedListToSet(strAliases));
}
aliases.addAll(Arrays.asList(this.indexSettings.getSettings()
.getAsArray("index.analysis.analyzer." + analyzerFactory.name() + ".alias")));
for (String alias : aliases) {
if (analyzerAliases.putIfAbsent(alias, analyzer) != null) {
throw new IllegalStateException("alias [" + alias + "] is already used by [" + analyzerAliases.get(alias).name() + "]");
}
}
}
@Override
public void close() {
for (NamedAnalyzer analyzer : analyzers.values()) {

View File

@ -43,6 +43,7 @@ import org.elasticsearch.index.mapper.core.TextFieldMapper.TextFieldType;
import org.elasticsearch.index.mapper.internal.TypeFieldMapper;
import org.elasticsearch.index.mapper.internal.UidFieldMapper;
import org.elasticsearch.index.mapper.object.ArrayValueMapperParser;
import org.elasticsearch.index.mapper.object.DynamicTemplate.XContentFieldType;
import org.elasticsearch.index.mapper.object.ObjectMapper;
import java.io.IOException;
@ -471,7 +472,7 @@ final class DocumentParser {
if (dynamic == ObjectMapper.Dynamic.STRICT) {
throw new StrictDynamicMappingException(mapper.fullPath(), currentFieldName);
} else if (dynamic == ObjectMapper.Dynamic.TRUE) {
Mapper.Builder builder = context.root().findTemplateBuilder(context, currentFieldName, "object");
Mapper.Builder builder = context.root().findTemplateBuilder(context, currentFieldName, XContentFieldType.OBJECT);
if (builder == null) {
builder = new ObjectMapper.Builder(currentFieldName).enabled(true);
}
@ -516,7 +517,7 @@ final class DocumentParser {
if (dynamic == ObjectMapper.Dynamic.STRICT) {
throw new StrictDynamicMappingException(parentMapper.fullPath(), arrayFieldName);
} else if (dynamic == ObjectMapper.Dynamic.TRUE) {
Mapper.Builder builder = context.root().findTemplateBuilder(context, arrayFieldName, "object");
Mapper.Builder builder = context.root().findTemplateBuilder(context, arrayFieldName, XContentFieldType.OBJECT);
if (builder == null) {
parseNonDynamicArray(context, parentMapper, lastFieldName, arrayFieldName);
} else {
@ -596,34 +597,34 @@ final class DocumentParser {
private static Mapper.Builder<?,?> createBuilderFromFieldType(final ParseContext context, MappedFieldType fieldType, String currentFieldName) {
Mapper.Builder builder = null;
if (fieldType instanceof StringFieldType) {
builder = context.root().findTemplateBuilder(context, currentFieldName, "string", "string");
builder = context.root().findTemplateBuilder(context, currentFieldName, "string", XContentFieldType.STRING);
} else if (fieldType instanceof TextFieldType) {
builder = context.root().findTemplateBuilder(context, currentFieldName, "text", "string");
builder = context.root().findTemplateBuilder(context, currentFieldName, "text", XContentFieldType.STRING);
if (builder == null) {
builder = new TextFieldMapper.Builder(currentFieldName)
.addMultiField(new KeywordFieldMapper.Builder("keyword").ignoreAbove(256));
}
} else if (fieldType instanceof KeywordFieldType) {
builder = context.root().findTemplateBuilder(context, currentFieldName, "keyword", "string");
builder = context.root().findTemplateBuilder(context, currentFieldName, "keyword", XContentFieldType.STRING);
} else {
switch (fieldType.typeName()) {
case DateFieldMapper.CONTENT_TYPE:
builder = context.root().findTemplateBuilder(context, currentFieldName, "date");
builder = context.root().findTemplateBuilder(context, currentFieldName, XContentFieldType.DATE);
break;
case "long":
builder = context.root().findTemplateBuilder(context, currentFieldName, "long");
builder = context.root().findTemplateBuilder(context, currentFieldName, "long", XContentFieldType.LONG);
break;
case "double":
builder = context.root().findTemplateBuilder(context, currentFieldName, "double");
builder = context.root().findTemplateBuilder(context, currentFieldName, "double", XContentFieldType.DOUBLE);
break;
case "integer":
builder = context.root().findTemplateBuilder(context, currentFieldName, "integer");
builder = context.root().findTemplateBuilder(context, currentFieldName, "integer", XContentFieldType.LONG);
break;
case "float":
builder = context.root().findTemplateBuilder(context, currentFieldName, "float");
builder = context.root().findTemplateBuilder(context, currentFieldName, "float", XContentFieldType.DOUBLE);
break;
case BooleanFieldMapper.CONTENT_TYPE:
builder = context.root().findTemplateBuilder(context, currentFieldName, "boolean");
builder = context.root().findTemplateBuilder(context, currentFieldName, "boolean", XContentFieldType.BOOLEAN);
break;
default:
break;
@ -682,7 +683,7 @@ final class DocumentParser {
for (FormatDateTimeFormatter dateTimeFormatter : context.root().dynamicDateTimeFormatters()) {
try {
dateTimeFormatter.parser().parseMillis(text);
Mapper.Builder builder = context.root().findTemplateBuilder(context, currentFieldName, "date");
Mapper.Builder builder = context.root().findTemplateBuilder(context, currentFieldName, XContentFieldType.DATE);
if (builder == null) {
builder = newDateBuilder(currentFieldName, dateTimeFormatter, Version.indexCreated(context.indexSettings()));
}
@ -697,7 +698,7 @@ final class DocumentParser {
String text = context.parser().text();
try {
Long.parseLong(text);
Mapper.Builder builder = context.root().findTemplateBuilder(context, currentFieldName, "long");
Mapper.Builder builder = context.root().findTemplateBuilder(context, currentFieldName, XContentFieldType.LONG);
if (builder == null) {
builder = newLongBuilder(currentFieldName, Version.indexCreated(context.indexSettings()));
}
@ -707,7 +708,7 @@ final class DocumentParser {
}
try {
Double.parseDouble(text);
Mapper.Builder builder = context.root().findTemplateBuilder(context, currentFieldName, "double");
Mapper.Builder builder = context.root().findTemplateBuilder(context, currentFieldName, XContentFieldType.DOUBLE);
if (builder == null) {
builder = newFloatBuilder(currentFieldName, Version.indexCreated(context.indexSettings()));
}
@ -716,7 +717,7 @@ final class DocumentParser {
// not a long number
}
}
Mapper.Builder builder = context.root().findTemplateBuilder(context, currentFieldName, "string");
Mapper.Builder builder = context.root().findTemplateBuilder(context, currentFieldName, XContentFieldType.STRING);
if (builder == null) {
builder = new TextFieldMapper.Builder(currentFieldName)
.addMultiField(new KeywordFieldMapper.Builder("keyword").ignoreAbove(256));
@ -725,13 +726,13 @@ final class DocumentParser {
} else if (token == XContentParser.Token.VALUE_NUMBER) {
XContentParser.NumberType numberType = context.parser().numberType();
if (numberType == XContentParser.NumberType.INT || numberType == XContentParser.NumberType.LONG) {
Mapper.Builder builder = context.root().findTemplateBuilder(context, currentFieldName, "long");
Mapper.Builder builder = context.root().findTemplateBuilder(context, currentFieldName, XContentFieldType.LONG);
if (builder == null) {
builder = newLongBuilder(currentFieldName, Version.indexCreated(context.indexSettings()));
}
return builder;
} else if (numberType == XContentParser.NumberType.FLOAT || numberType == XContentParser.NumberType.DOUBLE) {
Mapper.Builder builder = context.root().findTemplateBuilder(context, currentFieldName, "double");
Mapper.Builder builder = context.root().findTemplateBuilder(context, currentFieldName, XContentFieldType.DOUBLE);
if (builder == null) {
// no templates are defined, we use float by default instead of double
// since this is much more space-efficient and should be enough most of
@ -741,19 +742,19 @@ final class DocumentParser {
return builder;
}
} else if (token == XContentParser.Token.VALUE_BOOLEAN) {
Mapper.Builder builder = context.root().findTemplateBuilder(context, currentFieldName, "boolean");
Mapper.Builder builder = context.root().findTemplateBuilder(context, currentFieldName, XContentFieldType.BOOLEAN);
if (builder == null) {
builder = new BooleanFieldMapper.Builder(currentFieldName);
}
return builder;
} else if (token == XContentParser.Token.VALUE_EMBEDDED_OBJECT) {
Mapper.Builder builder = context.root().findTemplateBuilder(context, currentFieldName, "binary");
Mapper.Builder builder = context.root().findTemplateBuilder(context, currentFieldName, XContentFieldType.BINARY);
if (builder == null) {
builder = new BinaryFieldMapper.Builder(currentFieldName);
}
return builder;
} else {
Mapper.Builder builder = context.root().findTemplateBuilder(context, currentFieldName, null);
Mapper.Builder builder = context.root().findTemplateBuilder(context, currentFieldName, XContentFieldType.STRING);
if (builder != null) {
return builder;
}
@ -858,7 +859,7 @@ final class DocumentParser {
case STRICT:
throw new StrictDynamicMappingException(parent.fullPath(), paths[i]);
case TRUE:
Mapper.Builder builder = context.root().findTemplateBuilder(context, paths[i], "object");
Mapper.Builder builder = context.root().findTemplateBuilder(context, paths[i], XContentFieldType.OBJECT);
if (builder == null) {
builder = new ObjectMapper.Builder(paths[i]).enabled(true);
}

View File

@ -435,9 +435,9 @@ public abstract class FieldMapper extends Mapper implements Cloneable {
boolean hasDifferentSearchQuoteAnalyzer = fieldType().searchAnalyzer().name().equals(fieldType().searchQuoteAnalyzer().name()) == false;
if (includeDefaults || hasDefaultIndexAnalyzer == false || hasDifferentSearchAnalyzer || hasDifferentSearchQuoteAnalyzer) {
builder.field("analyzer", fieldType().indexAnalyzer().name());
if (hasDifferentSearchAnalyzer || hasDifferentSearchQuoteAnalyzer) {
if (includeDefaults || hasDifferentSearchAnalyzer || hasDifferentSearchQuoteAnalyzer) {
builder.field("search_analyzer", fieldType().searchAnalyzer().name());
if (hasDifferentSearchQuoteAnalyzer) {
if (includeDefaults || hasDifferentSearchQuoteAnalyzer) {
builder.field("search_quote_analyzer", fieldType().searchQuoteAnalyzer().name());
}
}

View File

@ -163,11 +163,6 @@ public class StringFieldMapper extends FieldMapper implements AllFieldMapper.Inc
@Override
public StringFieldMapper build(BuilderContext context) {
if (positionIncrementGap != POSITION_INCREMENT_GAP_USE_ANALYZER) {
fieldType.setIndexAnalyzer(new NamedAnalyzer(fieldType.indexAnalyzer(), positionIncrementGap));
fieldType.setSearchAnalyzer(new NamedAnalyzer(fieldType.searchAnalyzer(), positionIncrementGap));
fieldType.setSearchQuoteAnalyzer(new NamedAnalyzer(fieldType.searchQuoteAnalyzer(), positionIncrementGap));
}
// if the field is not analyzed, then by default, we should omit norms and have docs only
// index options, as probably what the user really wants
// if they are set explicitly, we will use those values
@ -183,6 +178,15 @@ public class StringFieldMapper extends FieldMapper implements AllFieldMapper.Inc
fieldType.setIndexOptions(IndexOptions.DOCS);
}
}
if (positionIncrementGap != POSITION_INCREMENT_GAP_USE_ANALYZER) {
if (fieldType.indexOptions().compareTo(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS) < 0) {
throw new IllegalArgumentException("Cannot set position_increment_gap on field ["
+ name + "] without positions enabled");
}
fieldType.setIndexAnalyzer(new NamedAnalyzer(fieldType.indexAnalyzer(), positionIncrementGap));
fieldType.setSearchAnalyzer(new NamedAnalyzer(fieldType.searchAnalyzer(), positionIncrementGap));
fieldType.setSearchQuoteAnalyzer(new NamedAnalyzer(fieldType.searchQuoteAnalyzer(), positionIncrementGap));
}
setupFieldType(context);
StringFieldMapper fieldMapper = new StringFieldMapper(
name, fieldType(), defaultFieldType, positionIncrementGap, ignoreAbove,

View File

@ -119,6 +119,10 @@ public class TextFieldMapper extends FieldMapper implements AllFieldMapper.Inclu
@Override
public TextFieldMapper build(BuilderContext context) {
if (positionIncrementGap != POSITION_INCREMENT_GAP_USE_ANALYZER) {
if (fieldType.indexOptions().compareTo(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS) < 0) {
throw new IllegalArgumentException("Cannot set position_increment_gap on field ["
+ name + "] without positions enabled");
}
fieldType.setIndexAnalyzer(new NamedAnalyzer(fieldType.indexAnalyzer(), positionIncrementGap));
fieldType.setSearchAnalyzer(new NamedAnalyzer(fieldType.searchAnalyzer(), positionIncrementGap));
fieldType.setSearchQuoteAnalyzer(new NamedAnalyzer(fieldType.searchQuoteAnalyzer(), positionIncrementGap));

View File

@ -20,15 +20,21 @@
package org.elasticsearch.index.mapper.object;
import org.elasticsearch.Version;
import org.elasticsearch.common.Strings;
import org.elasticsearch.common.logging.DeprecationLogger;
import org.elasticsearch.common.logging.Loggers;
import org.elasticsearch.common.regex.Regex;
import org.elasticsearch.common.xcontent.ToXContent;
import org.elasticsearch.common.xcontent.XContentBuilder;
import org.elasticsearch.index.mapper.ContentPath;
import org.elasticsearch.index.mapper.MapperParsingException;
import org.elasticsearch.index.mapper.core.BinaryFieldMapper;
import org.elasticsearch.index.mapper.core.BooleanFieldMapper;
import org.elasticsearch.index.mapper.core.DateFieldMapper;
import org.elasticsearch.index.mapper.core.NumberFieldMapper;
import org.elasticsearch.index.mapper.core.TextFieldMapper;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
@ -39,6 +45,8 @@ import java.util.TreeMap;
*/
public class DynamicTemplate implements ToXContent {
private static final DeprecationLogger DEPRECATION_LOGGER = new DeprecationLogger(Loggers.getLogger(DynamicTemplate.class));
public static enum MatchType {
SIMPLE {
@Override
@ -74,6 +82,93 @@ public class DynamicTemplate implements ToXContent {
public abstract boolean matches(String regex, String value);
}
/** The type of a field as detected while parsing a json document. */
public enum XContentFieldType {
OBJECT {
@Override
public String defaultMappingType() {
return ObjectMapper.CONTENT_TYPE;
}
@Override
public String toString() {
return "object";
}
},
STRING {
@Override
public String defaultMappingType() {
return TextFieldMapper.CONTENT_TYPE;
}
@Override
public String toString() {
return "string";
}
},
LONG {
@Override
public String defaultMappingType() {
return NumberFieldMapper.NumberType.LONG.typeName();
}
@Override
public String toString() {
return "long";
}
},
DOUBLE {
@Override
public String defaultMappingType() {
return NumberFieldMapper.NumberType.FLOAT.typeName();
}
@Override
public String toString() {
return "double";
}
},
BOOLEAN {
@Override
public String defaultMappingType() {
return BooleanFieldMapper.CONTENT_TYPE;
}
@Override
public String toString() {
return "boolean";
}
},
DATE {
@Override
public String defaultMappingType() {
return DateFieldMapper.CONTENT_TYPE;
}
@Override
public String toString() {
return "date";
}
},
BINARY {
@Override
public String defaultMappingType() {
return BinaryFieldMapper.CONTENT_TYPE;
}
@Override
public String toString() {
return "binary";
}
};
public static XContentFieldType fromString(String value) {
for (XContentFieldType v : values()) {
if (v.toString().equals(value)) {
return v;
}
}
throw new IllegalArgumentException("No xcontent type matched on [" + value + "], possible values are "
+ Arrays.toString(values()));
}
/** The default mapping type to use for fields of this {@link XContentFieldType}. */
public abstract String defaultMappingType();
}
public static DynamicTemplate parse(String name, Map<String, Object> conf,
Version indexVersionCreated) throws MapperParsingException {
String match = null;
@ -107,7 +202,30 @@ public class DynamicTemplate implements ToXContent {
}
}
return new DynamicTemplate(name, pathMatch, pathUnmatch, match, unmatch, matchMappingType, MatchType.fromString(matchPattern), mapping);
if (match == null && pathMatch == null && matchMappingType == null) {
throw new MapperParsingException("template must have match, path_match or match_mapping_type set " + conf.toString());
}
if (mapping == null) {
throw new MapperParsingException("template must have mapping set");
}
XContentFieldType xcontentFieldType = null;
if (matchMappingType != null && matchMappingType.equals("*") == false) {
try {
xcontentFieldType = XContentFieldType.fromString(matchMappingType);
} catch (IllegalArgumentException e) {
// TODO: do this in 6.0
/*if (indexVersionCreated.onOrAfter(Version.V_6_0_0)) {
throw e;
}*/
DEPRECATION_LOGGER.deprecated("Ignoring unrecognized match_mapping_type: [" + matchMappingType + "]");
// this template is on an unknown type so it will never match anything
// null indicates that the template should be ignored
return null;
}
}
return new DynamicTemplate(name, pathMatch, pathUnmatch, match, unmatch, xcontentFieldType, MatchType.fromString(matchPattern), mapping);
}
private final String name;
@ -122,24 +240,19 @@ public class DynamicTemplate implements ToXContent {
private final MatchType matchType;
private final String matchMappingType;
private final XContentFieldType xcontentFieldType;
private final Map<String, Object> mapping;
public DynamicTemplate(String name, String pathMatch, String pathUnmatch, String match, String unmatch, String matchMappingType, MatchType matchType, Map<String, Object> mapping) {
if (match == null && pathMatch == null && matchMappingType == null) {
throw new MapperParsingException("template must have match, path_match or match_mapping_type set");
}
if (mapping == null) {
throw new MapperParsingException("template must have mapping set");
}
private DynamicTemplate(String name, String pathMatch, String pathUnmatch, String match, String unmatch,
XContentFieldType xcontentFieldType, MatchType matchType, Map<String, Object> mapping) {
this.name = name;
this.pathMatch = pathMatch;
this.pathUnmatch = pathUnmatch;
this.match = match;
this.unmatch = unmatch;
this.matchType = matchType;
this.matchMappingType = matchMappingType;
this.xcontentFieldType = xcontentFieldType;
this.mapping = mapping;
}
@ -147,26 +260,21 @@ public class DynamicTemplate implements ToXContent {
return this.name;
}
public boolean match(ContentPath path, String name, String dynamicType) {
if (pathMatch != null && !matchType.matches(pathMatch, path.pathAsText(name))) {
public boolean match(String path, String name, XContentFieldType xcontentFieldType) {
if (pathMatch != null && !matchType.matches(pathMatch, path)) {
return false;
}
if (match != null && !matchType.matches(match, name)) {
return false;
}
if (pathUnmatch != null && matchType.matches(pathUnmatch, path.pathAsText(name))) {
if (pathUnmatch != null && matchType.matches(pathUnmatch, path)) {
return false;
}
if (unmatch != null && matchType.matches(unmatch, name)) {
return false;
}
if (matchMappingType != null) {
if (dynamicType == null) {
return false;
}
if (!matchType.matches(matchMappingType, dynamicType)) {
return false;
}
if (this.xcontentFieldType != null && this.xcontentFieldType != xcontentFieldType) {
return false;
}
return true;
}
@ -248,8 +356,10 @@ public class DynamicTemplate implements ToXContent {
if (pathUnmatch != null) {
builder.field("path_unmatch", pathUnmatch);
}
if (matchMappingType != null) {
builder.field("match_mapping_type", matchMappingType);
if (xcontentFieldType != null) {
builder.field("match_mapping_type", xcontentFieldType);
} else if (match == null && pathMatch == null) {
builder.field("match_mapping_type", "*");
}
if (matchType != MatchType.SIMPLE) {
builder.field("match_pattern", matchType);

View File

@ -21,7 +21,6 @@ package org.elasticsearch.index.mapper.object;
import org.elasticsearch.Version;
import org.elasticsearch.common.Nullable;
import org.elasticsearch.common.Strings;
import org.elasticsearch.common.joda.FormatDateTimeFormatter;
import org.elasticsearch.common.joda.Joda;
import org.elasticsearch.common.settings.Settings;
@ -33,6 +32,7 @@ import org.elasticsearch.index.mapper.Mapper;
import org.elasticsearch.index.mapper.MapperParsingException;
import org.elasticsearch.index.mapper.ParseContext;
import org.elasticsearch.index.mapper.core.DateFieldMapper;
import org.elasticsearch.index.mapper.object.DynamicTemplate.XContentFieldType;
import java.io.IOException;
import java.util.ArrayList;
@ -190,7 +190,9 @@ public class RootObjectMapper extends ObjectMapper {
String templateName = entry.getKey();
Map<String, Object> templateParams = (Map<String, Object>) entry.getValue();
DynamicTemplate template = DynamicTemplate.parse(templateName, templateParams, indexVersionCreated);
((Builder) builder).add(template);
if (template != null) {
((Builder) builder).add(template);
}
}
return true;
} else if (fieldName.equals("date_detection")) {
@ -240,21 +242,8 @@ public class RootObjectMapper extends ObjectMapper {
return dynamicDateTimeFormatters;
}
public Mapper.Builder findTemplateBuilder(ParseContext context, String name, String matchType) {
final String dynamicType;
switch (matchType) {
case "string":
// string is a corner case since a json string can either map to a
// text or keyword field in elasticsearch. For now we use text when
// unspecified. For other types, the mapping type matches the json
// type so we are fine
dynamicType = "text";
break;
default:
dynamicType = matchType;
break;
}
return findTemplateBuilder(context, name, dynamicType, matchType);
public Mapper.Builder findTemplateBuilder(ParseContext context, String name, XContentFieldType matchType) {
return findTemplateBuilder(context, name, matchType.defaultMappingType(), matchType);
}
/**
@ -264,7 +253,7 @@ public class RootObjectMapper extends ObjectMapper {
* @param matchType the type of the field in the json document or null if unknown
* @return a mapper builder, or null if there is no template for such a field
*/
public Mapper.Builder findTemplateBuilder(ParseContext context, String name, String dynamicType, String matchType) {
public Mapper.Builder findTemplateBuilder(ParseContext context, String name, String dynamicType, XContentFieldType matchType) {
DynamicTemplate dynamicTemplate = findTemplate(context.path(), name, matchType);
if (dynamicTemplate == null) {
return null;
@ -278,9 +267,10 @@ public class RootObjectMapper extends ObjectMapper {
return typeParser.parse(name, dynamicTemplate.mappingForName(name, dynamicType), parserContext);
}
private DynamicTemplate findTemplate(ContentPath path, String name, String matchType) {
public DynamicTemplate findTemplate(ContentPath path, String name, XContentFieldType matchType) {
final String pathAsString = path.pathAsText(name);
for (DynamicTemplate dynamicTemplate : dynamicTemplates) {
if (dynamicTemplate.match(path, name, matchType)) {
if (dynamicTemplate.match(pathAsString, name, matchType)) {
return dynamicTemplate;
}
}

View File

@ -46,9 +46,7 @@ import static org.elasticsearch.common.lucene.search.Queries.fixNegativeQueryIfN
* A Query that matches documents matching boolean combinations of other queries.
*/
public class BoolQueryBuilder extends AbstractQueryBuilder<BoolQueryBuilder> {
public static final String NAME = "bool";
public static final ParseField QUERY_NAME_FIELD = new ParseField(BoolQueryBuilder.NAME);
public static final boolean ADJUST_PURE_NEGATIVE_DEFAULT = true;
public static final boolean DISABLE_COORD_DEFAULT = false;

View File

@ -46,9 +46,7 @@ import java.util.Optional;
* demoting effect
*/
public class BoostingQueryBuilder extends AbstractQueryBuilder<BoostingQueryBuilder> {
public static final String NAME = "boosting";
public static final ParseField QUERY_NAME_FIELD = new ParseField(NAME);
private static final ParseField POSITIVE_FIELD = new ParseField("positive");
private static final ParseField NEGATIVE_FIELD = new ParseField("negative");

View File

@ -60,7 +60,6 @@ import java.util.Optional;
public class CommonTermsQueryBuilder extends AbstractQueryBuilder<CommonTermsQueryBuilder> {
public static final String NAME = "common";
public static final ParseField QUERY_NAME_FIELD = new ParseField(NAME);
public static final float DEFAULT_CUTOFF_FREQ = 0.01f;
public static final Operator DEFAULT_HIGH_FREQ_OCCUR = Operator.OR;

View File

@ -38,9 +38,7 @@ import java.util.Optional;
* query boost for every document in the filter.
*/
public class ConstantScoreQueryBuilder extends AbstractQueryBuilder<ConstantScoreQueryBuilder> {
public static final String NAME = "constant_score";
public static final ParseField QUERY_NAME_FIELD = new ParseField(NAME);
private static final ParseField INNER_QUERY_FIELD = new ParseField("filter", "query");

View File

@ -42,9 +42,7 @@ import java.util.Optional;
* additional matching sub-queries.
*/
public class DisMaxQueryBuilder extends AbstractQueryBuilder<DisMaxQueryBuilder> {
public static final String NAME = "dis_max";
public static final ParseField QUERY_NAME_FIELD = new ParseField(NAME);
/** Default multiplication factor for breaking ties in document scores.*/
public static final float DEFAULT_TIE_BREAKER = 0.0f;

View File

@ -43,9 +43,7 @@ import java.util.Optional;
* Constructs a query that only match on documents that the field has a value in them.
*/
public class ExistsQueryBuilder extends AbstractQueryBuilder<ExistsQueryBuilder> {
public static final String NAME = "exists";
public static final ParseField QUERY_NAME_FIELD = new ParseField(NAME);
public static final ParseField FIELD_FIELD = new ParseField("field");

View File

@ -35,11 +35,8 @@ import java.io.IOException;
import java.util.Objects;
import java.util.Optional;
public class FieldMaskingSpanQueryBuilder extends AbstractQueryBuilder<FieldMaskingSpanQueryBuilder>
implements SpanQueryBuilder {
public class FieldMaskingSpanQueryBuilder extends AbstractQueryBuilder<FieldMaskingSpanQueryBuilder> implements SpanQueryBuilder {
public static final String NAME = "field_masking_span";
public static final ParseField QUERY_NAME_FIELD = new ParseField(NAME);
private static final ParseField FIELD_FIELD = new ParseField("field");
private static final ParseField QUERY_FIELD = new ParseField("query");

View File

@ -47,9 +47,7 @@ import java.util.Optional;
*/
@Deprecated
public class FuzzyQueryBuilder extends AbstractQueryBuilder<FuzzyQueryBuilder> implements MultiTermQueryBuilder {
public static final String NAME = "fuzzy";
public static final ParseField QUERY_NAME_FIELD = new ParseField(NAME);
/** Default maximum edit distance. Defaults to AUTO. */
public static final Fuzziness DEFAULT_FUZZINESS = Fuzziness.AUTO;

View File

@ -55,7 +55,6 @@ import java.util.Optional;
* enabled.
* */
public class GeoBoundingBoxQueryBuilder extends AbstractQueryBuilder<GeoBoundingBoxQueryBuilder> {
/** Name of the query. */
public static final String NAME = "geo_bounding_box";
public static final ParseField QUERY_NAME_FIELD = new ParseField(NAME, "geo_bbox");

View File

@ -50,12 +50,9 @@ import java.util.Optional;
/**
* Filter results of a query to include only those within a specific distance to some
* geo point.
* */
*/
public class GeoDistanceQueryBuilder extends AbstractQueryBuilder<GeoDistanceQueryBuilder> {
/** Name of the query in the query dsl. */
public static final String NAME = "geo_distance";
public static final ParseField QUERY_NAME_FIELD = new ParseField(NAME);
/** Default for latitude normalization (as of this writing true).*/
public static final boolean DEFAULT_NORMALIZE_LAT = true;

View File

@ -48,9 +48,7 @@ import java.util.Objects;
import java.util.Optional;
public class GeoDistanceRangeQueryBuilder extends AbstractQueryBuilder<GeoDistanceRangeQueryBuilder> {
public static final String NAME = "geo_distance_range";
public static final ParseField QUERY_NAME_FIELD = new ParseField(NAME);
public static final boolean DEFAULT_INCLUDE_LOWER = true;
public static final boolean DEFAULT_INCLUDE_UPPER = true;

View File

@ -46,9 +46,7 @@ import java.util.Objects;
import java.util.Optional;
public class GeoPolygonQueryBuilder extends AbstractQueryBuilder<GeoPolygonQueryBuilder> {
public static final String NAME = "geo_polygon";
public static final ParseField QUERY_NAME_FIELD = new ParseField(NAME);
/**
* The default value for ignore_unmapped.

View File

@ -53,9 +53,7 @@ import java.util.Optional;
* {@link QueryBuilder} that builds a GeoShape Query
*/
public class GeoShapeQueryBuilder extends AbstractQueryBuilder<GeoShapeQueryBuilder> {
public static final String NAME = "geo_shape";
public static final ParseField QUERY_NAME_FIELD = new ParseField(NAME);
public static final String DEFAULT_SHAPE_INDEX_NAME = "shapes";
public static final String DEFAULT_SHAPE_FIELD_NAME = "shape";

View File

@ -60,9 +60,7 @@ import java.util.Optional;
* </pre>
*/
public class GeohashCellQuery {
public static final String NAME = "geohash_cell";
public static final ParseField QUERY_NAME_FIELD = new ParseField(NAME);
public static final boolean DEFAULT_NEIGHBORS = false;

View File

@ -49,12 +49,7 @@ import java.util.Optional;
* A query builder for <tt>has_child</tt> query.
*/
public class HasChildQueryBuilder extends AbstractQueryBuilder<HasChildQueryBuilder> {
/**
* The queries name
*/
public static final String NAME = "has_child";
public static final ParseField QUERY_NAME_FIELD = new ParseField(NAME);
/**
* The default maximum number of children that are required to match for the parent to be considered a match.

View File

@ -45,9 +45,7 @@ import java.util.Set;
* Builder for the 'has_parent' query.
*/
public class HasParentQueryBuilder extends AbstractQueryBuilder<HasParentQueryBuilder> {
public static final String NAME = "has_parent";
public static final ParseField QUERY_NAME_FIELD = new ParseField(NAME);
/**
* The default value for ignore_unmapped.

View File

@ -47,9 +47,7 @@ import java.util.Set;
* A query that will return only documents matching specific ids (and a type).
*/
public class IdsQueryBuilder extends AbstractQueryBuilder<IdsQueryBuilder> {
public static final String NAME = "ids";
public static final ParseField QUERY_NAME_FIELD = new ParseField(NAME);
private static final ParseField TYPE_FIELD = new ParseField("type", "types", "_type");
private static final ParseField VALUES_FIELD = new ParseField("values");

View File

@ -47,7 +47,6 @@ import java.util.Optional;
public class IndicesQueryBuilder extends AbstractQueryBuilder<IndicesQueryBuilder> {
public static final String NAME = "indices";
public static final ParseField QUERY_NAME_FIELD = new ParseField(NAME);
private static final ParseField QUERY_FIELD = new ParseField("query");
private static final ParseField NO_MATCH_QUERY = new ParseField("no_match_query");

View File

@ -35,9 +35,7 @@ import java.util.Optional;
* A query that matches on all documents.
*/
public class MatchAllQueryBuilder extends AbstractQueryBuilder<MatchAllQueryBuilder> {
public static final String NAME = "match_all";
public static final ParseField QUERY_NAME_FIELD = new ParseField(NAME);
public MatchAllQueryBuilder() {
}

View File

@ -35,9 +35,7 @@ import java.util.Optional;
* A query that matches no document.
*/
public class MatchNoneQueryBuilder extends AbstractQueryBuilder<MatchNoneQueryBuilder> {
public static final String NAME = "match_none";
public static final ParseField QUERY_NAME_FIELD = new ParseField(NAME);
public MatchNoneQueryBuilder() {
}

View File

@ -38,9 +38,7 @@ import java.util.Optional;
* query as the result of the analysis.
*/
public class MatchPhrasePrefixQueryBuilder extends AbstractQueryBuilder<MatchPhrasePrefixQueryBuilder> {
public static final String NAME = "match_phrase_prefix";
public static final ParseField QUERY_NAME_FIELD = new ParseField(NAME);
public static final ParseField MAX_EXPANSIONS_FIELD = new ParseField("max_expansions");
private final String fieldName;

View File

@ -37,9 +37,7 @@ import java.util.Optional;
* as the result of the analysis.
*/
public class MatchPhraseQueryBuilder extends AbstractQueryBuilder<MatchPhraseQueryBuilder> {
public static final String NAME = "match_phrase";
public static final ParseField QUERY_NAME_FIELD = new ParseField(NAME);
public static final ParseField SLOP_FIELD = new ParseField("slop", "phrase_slop");
private final String fieldName;

View File

@ -79,7 +79,6 @@ import static org.elasticsearch.index.mapper.Uid.createUidAsBytes;
* The documents are provided as a set of strings and/or a list of {@link Item}.
*/
public class MoreLikeThisQueryBuilder extends AbstractQueryBuilder<MoreLikeThisQueryBuilder> {
public static final String NAME = "more_like_this";
public static final ParseField QUERY_NAME_FIELD = new ParseField(NAME, "mlt");

View File

@ -60,7 +60,6 @@ public class MultiMatchQueryBuilder extends AbstractQueryBuilder<MultiMatchQuery
public static final boolean DEFAULT_LENIENCY = MatchQuery.DEFAULT_LENIENCY;
public static final MatchQuery.ZeroTermsQuery DEFAULT_ZERO_TERMS_QUERY = MatchQuery.DEFAULT_ZERO_TERMS_QUERY;
public static final ParseField QUERY_NAME_FIELD = new ParseField(NAME);
private static final ParseField SLOP_FIELD = new ParseField("slop", "phrase_slop");
private static final ParseField ZERO_TERMS_QUERY_FIELD = new ParseField("zero_terms_query");
private static final ParseField LENIENT_FIELD = new ParseField("lenient");

View File

@ -39,12 +39,7 @@ import java.util.Objects;
import java.util.Optional;
public class NestedQueryBuilder extends AbstractQueryBuilder<NestedQueryBuilder> {
/**
* The queries name used while parsing
*/
public static final String NAME = "nested";
public static final ParseField QUERY_NAME_FIELD = new ParseField(NAME);
/**
* The default value for ignore_unmapped.
*/

View File

@ -41,9 +41,7 @@ import java.util.Objects;
import java.util.Optional;
public final class ParentIdQueryBuilder extends AbstractQueryBuilder<ParentIdQueryBuilder> {
public static final String NAME = "parent_id";
public static final ParseField QUERY_NAME_FIELD = new ParseField(NAME);
/**
* The default value for ignore_unmapped.

View File

@ -42,9 +42,7 @@ import java.util.Optional;
* A Query that matches documents containing terms with a specified prefix.
*/
public class PrefixQueryBuilder extends AbstractQueryBuilder<PrefixQueryBuilder> implements MultiTermQueryBuilder {
public static final String NAME = "prefix";
public static final ParseField QUERY_NAME_FIELD = new ParseField(NAME);
private static final ParseField PREFIX_FIELD = new ParseField("value", "prefix");
private static final ParseField REWRITE_FIELD = new ParseField("rewrite");

View File

@ -57,9 +57,7 @@ import java.util.TreeMap;
* them either using DisMax or a plain boolean query (see {@link #useDisMax(boolean)}).
*/
public class QueryStringQueryBuilder extends AbstractQueryBuilder<QueryStringQueryBuilder> {
public static final String NAME = "query_string";
public static final ParseField QUERY_NAME_FIELD = new ParseField(NAME);
public static final boolean DEFAULT_AUTO_GENERATE_PHRASE_QUERIES = false;
public static final int DEFAULT_MAX_DETERMINED_STATES = Operations.DEFAULT_MAX_DETERMINIZED_STATES;
@ -82,7 +80,7 @@ public class QueryStringQueryBuilder extends AbstractQueryBuilder<QueryStringQue
private static final ParseField ANALYZER_FIELD = new ParseField("analyzer");
private static final ParseField QUOTE_ANALYZER_FIELD = new ParseField("quote_analyzer");
private static final ParseField ALLOW_LEADING_WILDCARD_FIELD = new ParseField("allow_leading_wildcard");
private static final ParseField AUTO_GENERATED_PHRASE_QUERIES_FIELD = new ParseField("auto_generated_phrase_queries");
private static final ParseField AUTO_GENERATE_PHRASE_QUERIES_FIELD = new ParseField("auto_generate_phrase_queries");
private static final ParseField MAX_DETERMINED_STATES_FIELD = new ParseField("max_determined_states");
private static final ParseField LOWERCASE_EXPANDED_TERMS_FIELD = new ParseField("lowercase_expanded_terms");
private static final ParseField ENABLE_POSITION_INCREMENTS_FIELD = new ParseField("enable_position_increment");
@ -594,7 +592,7 @@ public class QueryStringQueryBuilder extends AbstractQueryBuilder<QueryStringQue
if (this.quoteAnalyzer != null) {
builder.field(QUOTE_ANALYZER_FIELD.getPreferredName(), this.quoteAnalyzer);
}
builder.field(AUTO_GENERATED_PHRASE_QUERIES_FIELD.getPreferredName(), this.autoGeneratePhraseQueries);
builder.field(AUTO_GENERATE_PHRASE_QUERIES_FIELD.getPreferredName(), this.autoGeneratePhraseQueries);
builder.field(MAX_DETERMINED_STATES_FIELD.getPreferredName(), this.maxDeterminizedStates);
if (this.allowLeadingWildcard != null) {
builder.field(ALLOW_LEADING_WILDCARD_FIELD.getPreferredName(), this.allowLeadingWildcard);
@ -704,7 +702,7 @@ public class QueryStringQueryBuilder extends AbstractQueryBuilder<QueryStringQue
quoteAnalyzer = parser.text();
} else if (parseContext.getParseFieldMatcher().match(currentFieldName, ALLOW_LEADING_WILDCARD_FIELD)) {
allowLeadingWildcard = parser.booleanValue();
} else if (parseContext.getParseFieldMatcher().match(currentFieldName, AUTO_GENERATED_PHRASE_QUERIES_FIELD)) {
} else if (parseContext.getParseFieldMatcher().match(currentFieldName, AUTO_GENERATE_PHRASE_QUERIES_FIELD)) {
autoGeneratePhraseQueries = parser.booleanValue();
} else if (parseContext.getParseFieldMatcher().match(currentFieldName, MAX_DETERMINED_STATES_FIELD)) {
maxDeterminizedStates = parser.intValue();

View File

@ -49,7 +49,6 @@ import java.util.Optional;
*/
public class RangeQueryBuilder extends AbstractQueryBuilder<RangeQueryBuilder> implements MultiTermQueryBuilder {
public static final String NAME = "range";
public static final ParseField QUERY_NAME_FIELD = new ParseField(NAME);
public static final boolean DEFAULT_INCLUDE_UPPER = true;
public static final boolean DEFAULT_INCLUDE_LOWER = true;

View File

@ -43,9 +43,7 @@ import java.util.Optional;
* A Query that does fuzzy matching for a specific value.
*/
public class RegexpQueryBuilder extends AbstractQueryBuilder<RegexpQueryBuilder> implements MultiTermQueryBuilder {
public static final String NAME = "regexp";
public static final ParseField QUERY_NAME_FIELD = new ParseField(NAME);
public static final int DEFAULT_FLAGS_VALUE = RegexpFlag.ALL.value();
public static final int DEFAULT_MAX_DETERMINIZED_STATES = Operations.DEFAULT_MAX_DETERMINIZED_STATES;

View File

@ -46,9 +46,7 @@ import java.util.Objects;
import java.util.Optional;
public class ScriptQueryBuilder extends AbstractQueryBuilder<ScriptQueryBuilder> {
public static final String NAME = "script";
public static final ParseField QUERY_NAME_FIELD = new ParseField(NAME);
private static final ParseField PARAMS_FIELD = new ParseField("params");

View File

@ -93,7 +93,6 @@ public class SimpleQueryStringBuilder extends AbstractQueryBuilder<SimpleQuerySt
/** Name for (de-)serialization. */
public static final String NAME = "simple_query_string";
public static final ParseField QUERY_NAME_FIELD = new ParseField(NAME);
private static final ParseField MINIMUM_SHOULD_MATCH_FIELD = new ParseField("minimum_should_match");
private static final ParseField ANALYZE_WILDCARD_FIELD = new ParseField("analyze_wildcard");

View File

@ -36,11 +36,8 @@ import java.util.Optional;
/**
* Builder for {@link org.apache.lucene.search.spans.SpanContainingQuery}.
*/
public class SpanContainingQueryBuilder extends AbstractQueryBuilder<SpanContainingQueryBuilder>
implements SpanQueryBuilder {
public class SpanContainingQueryBuilder extends AbstractQueryBuilder<SpanContainingQueryBuilder> implements SpanQueryBuilder {
public static final String NAME = "span_containing";
public static final ParseField QUERY_NAME_FIELD = new ParseField(NAME);
private static final ParseField BIG_FIELD = new ParseField("big");
private static final ParseField LITTLE_FIELD = new ParseField("little");

View File

@ -34,9 +34,7 @@ import java.util.Objects;
import java.util.Optional;
public class SpanFirstQueryBuilder extends AbstractQueryBuilder<SpanFirstQueryBuilder> implements SpanQueryBuilder {
public static final String NAME = "span_first";
public static final ParseField QUERY_NAME_FIELD = new ParseField(NAME);
private static final ParseField MATCH_FIELD = new ParseField("match");
private static final ParseField END_FIELD = new ParseField("end");

View File

@ -43,7 +43,6 @@ public class SpanMultiTermQueryBuilder extends AbstractQueryBuilder<SpanMultiTer
implements SpanQueryBuilder {
public static final String NAME = "span_multi";
public static final ParseField QUERY_NAME_FIELD = new ParseField(NAME);
private static final ParseField MATCH_FIELD = new ParseField("match");

View File

@ -42,9 +42,7 @@ import java.util.Optional;
* The span near query maps to Lucene {@link SpanNearQuery}.
*/
public class SpanNearQueryBuilder extends AbstractQueryBuilder<SpanNearQueryBuilder> implements SpanQueryBuilder {
public static final String NAME = "span_near";
public static final ParseField QUERY_NAME_FIELD = new ParseField(NAME);
/** Default for flag controlling whether matches are required to be in-order */
public static boolean DEFAULT_IN_ORDER = true;

View File

@ -34,9 +34,7 @@ import java.util.Objects;
import java.util.Optional;
public class SpanNotQueryBuilder extends AbstractQueryBuilder<SpanNotQueryBuilder> implements SpanQueryBuilder {
public static final String NAME = "span_not";
public static final ParseField QUERY_NAME_FIELD = new ParseField(NAME);
/** the default pre parameter size */
public static final int DEFAULT_PRE = 0;

View File

@ -40,9 +40,7 @@ import java.util.Optional;
* Span query that matches the union of its clauses. Maps to {@link SpanOrQuery}.
*/
public class SpanOrQueryBuilder extends AbstractQueryBuilder<SpanOrQueryBuilder> implements SpanQueryBuilder {
public static final String NAME = "span_or";
public static final ParseField QUERY_NAME_FIELD = new ParseField(NAME);
private static final ParseField CLAUSES_FIELD = new ParseField("clauses");

View File

@ -38,9 +38,7 @@ import java.util.Optional;
* @see SpanTermQuery
*/
public class SpanTermQueryBuilder extends BaseTermQueryBuilder<SpanTermQueryBuilder> implements SpanQueryBuilder {
public static final String NAME = "span_term";
public static final ParseField QUERY_NAME_FIELD = new ParseField(NAME);
private static final ParseField TERM_FIELD = new ParseField("term");

View File

@ -36,11 +36,8 @@ import java.util.Optional;
/**
* Builder for {@link org.apache.lucene.search.spans.SpanWithinQuery}.
*/
public class SpanWithinQueryBuilder extends AbstractQueryBuilder<SpanWithinQueryBuilder>
implements SpanQueryBuilder {
public class SpanWithinQueryBuilder extends AbstractQueryBuilder<SpanWithinQueryBuilder> implements SpanQueryBuilder {
public static final String NAME = "span_within";
public static final ParseField QUERY_NAME_FIELD = new ParseField(NAME);
private static final ParseField BIG_FIELD = new ParseField("big");
private static final ParseField LITTLE_FIELD = new ParseField("little");

View File

@ -36,9 +36,7 @@ import java.util.Optional;
* A Query that matches documents containing a term.
*/
public class TermQueryBuilder extends BaseTermQueryBuilder<TermQueryBuilder> {
public static final String NAME = "term";
public static final ParseField QUERY_NAME_FIELD = new ParseField(NAME);
private static final ParseField TERM_FIELD = new ParseField("term");
private static final ParseField VALUE_FIELD = new ParseField("value");

View File

@ -55,7 +55,6 @@ import java.util.stream.IntStream;
* A filter for a field based on several terms matching on any of them.
*/
public class TermsQueryBuilder extends AbstractQueryBuilder<TermsQueryBuilder> {
public static final String NAME = "terms";
public static final ParseField QUERY_NAME_FIELD = new ParseField(NAME, "in");

View File

@ -37,7 +37,6 @@ import java.util.Optional;
public class TypeQueryBuilder extends AbstractQueryBuilder<TypeQueryBuilder> {
public static final String NAME = "type";
public static final ParseField QUERY_NAME_FIELD = new ParseField(NAME);
private static final ParseField VALUE_FIELD = new ParseField("value");

View File

@ -46,11 +46,8 @@ import java.util.Optional;
* a Wildcard term should not start with one of the wildcards <tt>*</tt> or
* <tt>?</tt>.
*/
public class WildcardQueryBuilder extends AbstractQueryBuilder<WildcardQueryBuilder>
implements MultiTermQueryBuilder {
public class WildcardQueryBuilder extends AbstractQueryBuilder<WildcardQueryBuilder> implements MultiTermQueryBuilder {
public static final String NAME = "wildcard";
public static final ParseField QUERY_NAME_FIELD = new ParseField(NAME);
private static final ParseField WILDCARD_FIELD = new ParseField("wildcard");
private static final ParseField VALUE_FIELD = new ParseField("value");

View File

@ -51,9 +51,7 @@ import java.util.Optional;
* </pre>
*/
public class WrapperQueryBuilder extends AbstractQueryBuilder<WrapperQueryBuilder> {
public static final String NAME = "wrapper";
public static final ParseField QUERY_NAME_FIELD = new ParseField(NAME);
private static final ParseField QUERY_FIELD = new ParseField("query");

View File

@ -60,7 +60,6 @@ import java.util.Optional;
*/
public class FunctionScoreQueryBuilder extends AbstractQueryBuilder<FunctionScoreQueryBuilder> {
public static final String NAME = "function_score";
public static final ParseField QUERY_NAME_FIELD = new ParseField(NAME);
// For better readability of error message
static final String MISPLACED_FUNCTION_MESSAGE_PREFIX = "you can either define [functions] array or a single function, not both. ";

View File

@ -27,7 +27,6 @@ import org.elasticsearch.common.unit.ByteSizeUnit;
import org.elasticsearch.common.unit.ByteSizeValue;
import org.elasticsearch.common.unit.TimeValue;
import org.elasticsearch.common.util.concurrent.AbstractRunnable;
import org.elasticsearch.common.util.concurrent.FutureUtils;
import org.elasticsearch.index.engine.Engine;
import org.elasticsearch.index.engine.EngineClosedException;
import org.elasticsearch.index.engine.FlushNotAllowedEngineException;
@ -35,6 +34,8 @@ import org.elasticsearch.index.shard.IndexShard;
import org.elasticsearch.index.shard.IndexShardState;
import org.elasticsearch.index.shard.IndexingOperationListener;
import org.elasticsearch.threadpool.ThreadPool;
import org.elasticsearch.threadpool.ThreadPool.Cancellable;
import org.elasticsearch.threadpool.ThreadPool.Names;
import java.io.Closeable;
import java.util.ArrayList;
@ -43,7 +44,6 @@ import java.util.HashSet;
import java.util.List;
import java.util.PriorityQueue;
import java.util.Set;
import java.util.concurrent.ScheduledFuture;
import java.util.concurrent.atomic.AtomicLong;
import java.util.concurrent.locks.ReentrantLock;
@ -84,7 +84,7 @@ public class IndexingMemoryController extends AbstractComponent implements Index
/** Contains shards currently being throttled because we can't write segments quickly enough */
private final Set<IndexShard> throttled = new HashSet<>();
private final ScheduledFuture scheduler;
private final Cancellable scheduler;
private static final EnumSet<IndexShardState> CAN_WRITE_INDEX_BUFFER_STATES = EnumSet.of(
IndexShardState.RECOVERING, IndexShardState.POST_RECOVERY, IndexShardState.STARTED, IndexShardState.RELOCATED);
@ -128,14 +128,14 @@ public class IndexingMemoryController extends AbstractComponent implements Index
this.threadPool = threadPool;
}
protected ScheduledFuture<?> scheduleTask(ThreadPool threadPool) {
protected Cancellable scheduleTask(ThreadPool threadPool) {
// it's fine to run it on the scheduler thread, no busy work
return threadPool.scheduleWithFixedDelay(statusChecker, interval);
return threadPool.scheduleWithFixedDelay(statusChecker, interval, Names.SAME);
}
@Override
public void close() {
FutureUtils.cancel(scheduler);
scheduler.cancel();
}
/**

View File

@ -109,6 +109,7 @@ public class CompoundProcessor implements Processor {
throw compoundProcessorException;
} else {
executeOnFailure(ingestDocument, compoundProcessorException);
break;
}
}
}

View File

@ -19,7 +19,9 @@
package org.elasticsearch.ingest;
import org.elasticsearch.ElasticsearchException;
import org.elasticsearch.ElasticsearchParseException;
import org.elasticsearch.ExceptionsHelper;
import java.util.ArrayList;
import java.util.Arrays;
@ -221,19 +223,17 @@ public final class ConfigurationUtils {
return value;
}
public static ElasticsearchParseException newConfigurationException(String processorType, String processorTag,
public static ElasticsearchException newConfigurationException(String processorType, String processorTag,
String propertyName, String reason) {
ElasticsearchParseException exception = new ElasticsearchParseException("[" + propertyName + "] " + reason);
addHeadersToException(exception, processorType, processorTag, propertyName);
return exception;
}
if (processorType != null) {
exception.addHeader("processor_type", processorType);
}
if (processorTag != null) {
exception.addHeader("processor_tag", processorTag);
}
if (propertyName != null) {
exception.addHeader("property_name", propertyName);
}
public static ElasticsearchException newConfigurationException(String processorType, String processorTag,
String propertyName, Exception cause) {
ElasticsearchException exception = ExceptionsHelper.convertToElastic(cause);
addHeadersToException(exception, processorType, processorTag, propertyName);
return exception;
}
@ -251,6 +251,28 @@ public final class ConfigurationUtils {
return processors;
}
public static TemplateService.Template compileTemplate(String processorType, String processorTag, String propertyName,
String propertyValue, TemplateService templateService) {
try {
return templateService.compile(propertyValue);
} catch (Exception e) {
throw ConfigurationUtils.newConfigurationException(processorType, processorTag, propertyName, e);
}
}
private static void addHeadersToException(ElasticsearchException exception, String processorType,
String processorTag, String propertyName) {
if (processorType != null) {
exception.addHeader("processor_type", processorType);
}
if (processorTag != null) {
exception.addHeader("processor_tag", processorTag);
}
if (propertyName != null) {
exception.addHeader("property_name", propertyName);
}
}
public static Processor readProcessor(Map<String, Processor.Factory> processorFactories,
String type, Map<String, Object> config) throws Exception {
Processor.Factory factory = processorFactories.get(type);
@ -261,20 +283,25 @@ public final class ConfigurationUtils {
List<Processor> onFailureProcessors = readProcessorConfigs(onFailureProcessorConfigs, processorFactories);
String tag = ConfigurationUtils.readOptionalStringProperty(null, null, config, TAG_KEY);
Processor processor = factory.create(processorFactories, tag, config);
if (onFailureProcessorConfigs != null && onFailureProcessors.isEmpty()) {
throw newConfigurationException(processor.getType(), processor.getTag(), Pipeline.ON_FAILURE_KEY,
throw newConfigurationException(type, tag, Pipeline.ON_FAILURE_KEY,
"processors list cannot be empty");
}
if (config.isEmpty() == false) {
throw new ElasticsearchParseException("processor [{}] doesn't support one or more provided configuration parameters {}",
type, Arrays.toString(config.keySet().toArray()));
}
if (onFailureProcessors.size() > 0 || ignoreFailure) {
return new CompoundProcessor(ignoreFailure, Collections.singletonList(processor), onFailureProcessors);
} else {
return processor;
try {
Processor processor = factory.create(processorFactories, tag, config);
if (config.isEmpty() == false) {
throw new ElasticsearchParseException("processor [{}] doesn't support one or more provided configuration parameters {}",
type, Arrays.toString(config.keySet().toArray()));
}
if (onFailureProcessors.size() > 0 || ignoreFailure) {
return new CompoundProcessor(ignoreFailure, Collections.singletonList(processor), onFailureProcessors);
} else {
return processor;
}
} catch (Exception e) {
throw newConfigurationException(type, tag, null, e);
}
}
throw new ElasticsearchParseException("No processor type exists with name [" + type + "]");

View File

@ -29,6 +29,8 @@ import org.elasticsearch.common.unit.TimeValue;
import org.elasticsearch.common.util.concurrent.FutureUtils;
import org.elasticsearch.monitor.jvm.JvmStats.GarbageCollector;
import org.elasticsearch.threadpool.ThreadPool;
import org.elasticsearch.threadpool.ThreadPool.Cancellable;
import org.elasticsearch.threadpool.ThreadPool.Names;
import java.util.HashMap;
import java.util.Locale;
@ -48,7 +50,7 @@ public class JvmGcMonitorService extends AbstractLifecycleComponent {
private final Map<String, GcThreshold> gcThresholds;
private final GcOverheadThreshold gcOverheadThreshold;
private volatile ScheduledFuture scheduledFuture;
private volatile Cancellable scheduledFuture;
public static final Setting<Boolean> ENABLED_SETTING =
Setting.boolSetting("monitor.jvm.gc.enabled", true, Property.NodeScope);
@ -198,7 +200,7 @@ public class JvmGcMonitorService extends AbstractLifecycleComponent {
void onGcOverhead(final Threshold threshold, final long current, final long elapsed, final long seq) {
logGcOverhead(logger, threshold, current, elapsed, seq);
}
}, interval);
}, interval, Names.SAME);
}
private static final String SLOW_GC_LOG_MESSAGE =
@ -334,7 +336,7 @@ public class JvmGcMonitorService extends AbstractLifecycleComponent {
if (!enabled) {
return;
}
FutureUtils.cancel(scheduledFuture);
scheduledFuture.cancel();
}
@Override

View File

@ -19,12 +19,16 @@
package org.elasticsearch.plugins;
import org.apache.lucene.search.Query;
import org.elasticsearch.common.ParseField;
import org.elasticsearch.common.io.stream.NamedWriteable;
import org.elasticsearch.common.io.stream.StreamInput;
import org.elasticsearch.common.io.stream.Writeable;
import org.elasticsearch.common.io.stream.Writeable.Reader;
import org.elasticsearch.common.lucene.search.function.ScoreFunction;
import org.elasticsearch.common.xcontent.XContent;
import org.elasticsearch.index.query.QueryBuilder;
import org.elasticsearch.index.query.QueryParser;
import org.elasticsearch.index.query.functionscore.ScoreFunctionBuilder;
import org.elasticsearch.index.query.functionscore.ScoreFunctionParser;
import org.elasticsearch.search.aggregations.bucket.significant.SignificantTerms;
@ -56,14 +60,14 @@ public interface SearchPlugin {
* The new {@link SignificanceHeuristic}s defined by this plugin. {@linkplain SignificanceHeuristic}s are used by the
* {@link SignificantTerms} aggregation to pick which terms are significant for a given query.
*/
default List<SearchPluginSpec<SignificanceHeuristic, SignificanceHeuristicParser>> getSignificanceHeuristics() {
default List<SearchExtensionSpec<SignificanceHeuristic, SignificanceHeuristicParser>> getSignificanceHeuristics() {
return emptyList();
}
/**
* The new {@link MovAvgModel}s defined by this plugin. {@linkplain MovAvgModel}s are used by the {@link MovAvgPipelineAggregator} to
* model trends in data.
*/
default List<SearchPluginSpec<MovAvgModel, MovAvgModel.AbstractModelParser>> getMovingAverageModels() {
default List<SearchExtensionSpec<MovAvgModel, MovAvgModel.AbstractModelParser>> getMovingAverageModels() {
return emptyList();
}
/**
@ -84,11 +88,17 @@ public interface SearchPlugin {
default Map<String, Suggester<?>> getSuggesters() {
return emptyMap();
}
/**
* The new {@link Query}s defined by this plugin.
*/
default List<QuerySpec<?>> getQueries() {
return emptyList();
}
/**
* Specification of custom {@link ScoreFunction}.
*/
public class ScoreFunctionSpec<T extends ScoreFunctionBuilder<T>> extends SearchPluginSpec<T, ScoreFunctionParser<T>> {
public class ScoreFunctionSpec<T extends ScoreFunctionBuilder<T>> extends SearchExtensionSpec<T, ScoreFunctionParser<T>> {
public ScoreFunctionSpec(ParseField name, Reader<T> reader, ScoreFunctionParser<T> parser) {
super(name, reader, parser);
}
@ -98,6 +108,38 @@ public interface SearchPlugin {
}
}
/**
* Specification of custom {@link Query}.
*/
public class QuerySpec<T extends QueryBuilder> extends SearchExtensionSpec<T, QueryParser<T>> {
/**
* Specification of custom {@link Query}.
*
* @param name holds the names by which this query might be parsed. The {@link ParseField#getPreferredName()} is special as it
* is the name by under which the reader is registered. So it is the name that the query should use as its
* {@link NamedWriteable#getWriteableName()} too.
* @param reader the reader registered for this query's builder. Typically a reference to a constructor that takes a
* {@link StreamInput}
* @param parser the parser the reads the query builder from xcontent
*/
public QuerySpec(ParseField name, Reader<T> reader, QueryParser<T> parser) {
super(name, reader, parser);
}
/**
* Specification of custom {@link Query}.
*
* @param name the name by which this query might be parsed or deserialized. Make sure that the query builder returns this name for
* {@link NamedWriteable#getWriteableName()}.
* @param reader the reader registered for this query's builder. Typically a reference to a constructor that takes a
* {@link StreamInput}
* @param parser the parser the reads the query builder from xcontent
*/
public QuerySpec(String name, Reader<T> reader, QueryParser<T> parser) {
super(name, reader, parser);
}
}
/**
* Specification of search time behavior extension like a custom {@link MovAvgModel} or {@link ScoreFunction}.
*
@ -106,7 +148,7 @@ public interface SearchPlugin {
* @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
*/
public class SearchPluginSpec<W extends NamedWriteable, P> {
public class SearchExtensionSpec<W extends NamedWriteable, P> {
private final ParseField name;
private final Writeable.Reader<W> reader;
private final P parser;
@ -120,7 +162,7 @@ public interface SearchPlugin {
* @param reader reader that reads the behavior from the internode protocol
* @param parser parser that read the behavior from a REST request
*/
public SearchPluginSpec(ParseField name, Writeable.Reader<W> reader, P parser) {
public SearchExtensionSpec(ParseField name, Writeable.Reader<W> reader, P parser) {
this.name = name;
this.reader = reader;
this.parser = parser;
@ -134,7 +176,7 @@ public interface SearchPlugin {
* @param reader reader that reads the behavior from the internode protocol
* @param parser parser that read the behavior from a REST request
*/
public SearchPluginSpec(String name, Writeable.Reader<W> reader, P parser) {
public SearchExtensionSpec(String name, Writeable.Reader<W> reader, P parser) {
this(new ParseField(name), reader, parser);
}

View File

@ -23,9 +23,11 @@ import org.elasticsearch.action.admin.indices.analyze.AnalyzeResponse;
import org.elasticsearch.client.node.NodeClient;
import org.elasticsearch.common.ParseField;
import org.elasticsearch.common.ParseFieldMatcher;
import org.elasticsearch.common.Strings;
import org.elasticsearch.common.bytes.BytesReference;
import org.elasticsearch.common.inject.Inject;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.xcontent.XContentFactory;
import org.elasticsearch.common.xcontent.XContentHelper;
import org.elasticsearch.common.xcontent.XContentParser;
import org.elasticsearch.common.xcontent.XContentType;
@ -39,6 +41,7 @@ import org.elasticsearch.rest.action.support.RestToXContentListener;
import java.io.IOException;
import java.util.ArrayList;
import java.util.List;
import java.util.function.Consumer;
import static org.elasticsearch.rest.RestRequest.Method.GET;
import static org.elasticsearch.rest.RestRequest.Method.POST;
@ -77,9 +80,15 @@ public class RestAnalyzeAction extends BaseRestHandler {
analyzeRequest.text(texts);
analyzeRequest.analyzer(request.param("analyzer"));
analyzeRequest.field(request.param("field"));
analyzeRequest.tokenizer(request.param("tokenizer"));
analyzeRequest.tokenFilters(request.paramAsStringArray("filter", request.paramAsStringArray("token_filter", analyzeRequest.tokenFilters())));
analyzeRequest.charFilters(request.paramAsStringArray("char_filter", analyzeRequest.charFilters()));
if (request.hasParam("tokenizer")) {
analyzeRequest.tokenizer(request.param("tokenizer"));
}
for (String filter : request.paramAsStringArray("filter", request.paramAsStringArray("token_filter", Strings.EMPTY_ARRAY))) {
analyzeRequest.addTokenFilter(filter);
}
for (String charFilter : request.paramAsStringArray("char_filter", Strings.EMPTY_ARRAY)) {
analyzeRequest.addTokenFilter(charFilter);
}
analyzeRequest.explain(request.paramAsBoolean("explain", false));
analyzeRequest.attributes(request.paramAsStringArray("attributes", analyzeRequest.attributes()));
@ -96,7 +105,7 @@ public class RestAnalyzeAction extends BaseRestHandler {
}
}
client.admin().indices().analyze(analyzeRequest, new RestToXContentListener<AnalyzeResponse>(channel));
client.admin().indices().analyze(analyzeRequest, new RestToXContentListener<>(channel));
}
public static void buildFromContent(BytesReference content, AnalyzeRequest analyzeRequest, ParseFieldMatcher parseFieldMatcher) {
@ -124,33 +133,41 @@ public class RestAnalyzeAction extends BaseRestHandler {
analyzeRequest.analyzer(parser.text());
} else if (parseFieldMatcher.match(currentFieldName, Fields.FIELD) && token == XContentParser.Token.VALUE_STRING) {
analyzeRequest.field(parser.text());
} else if (parseFieldMatcher.match(currentFieldName, Fields.TOKENIZER) && token == XContentParser.Token.VALUE_STRING) {
analyzeRequest.tokenizer(parser.text());
} else if (parseFieldMatcher.match(currentFieldName, Fields.TOKENIZER)) {
if (token == XContentParser.Token.VALUE_STRING) {
analyzeRequest.tokenizer(parser.text());
} else if (token == XContentParser.Token.START_OBJECT) {
analyzeRequest.tokenizer(parser.map());
} else {
throw new IllegalArgumentException(currentFieldName + " should be tokenizer's name or setting");
}
} else if (parseFieldMatcher.match(currentFieldName, Fields.TOKEN_FILTERS) && token == XContentParser.Token.START_ARRAY) {
List<String> filters = new ArrayList<>();
while ((token = parser.nextToken()) != XContentParser.Token.END_ARRAY) {
if (token.isValue() == false) {
throw new IllegalArgumentException(currentFieldName + " array element should only contain token filter's name");
if (token == XContentParser.Token.VALUE_STRING) {
analyzeRequest.addTokenFilter(parser.text());
} else if (token == XContentParser.Token.START_OBJECT) {
analyzeRequest.addTokenFilter(parser.map());
} else {
throw new IllegalArgumentException(currentFieldName + " array element should contain token_filter's name or setting");
}
filters.add(parser.text());
}
analyzeRequest.tokenFilters(filters.toArray(new String[filters.size()]));
} else if (parseFieldMatcher.match(currentFieldName, Fields.CHAR_FILTERS) && token == XContentParser.Token.START_ARRAY) {
List<String> charFilters = new ArrayList<>();
while ((token = parser.nextToken()) != XContentParser.Token.END_ARRAY) {
if (token.isValue() == false) {
throw new IllegalArgumentException(currentFieldName + " array element should only contain char filter's name");
if (token == XContentParser.Token.VALUE_STRING) {
analyzeRequest.addCharFilter(parser.text());
} else if (token == XContentParser.Token.START_OBJECT) {
analyzeRequest.addCharFilter(parser.map());
} else {
throw new IllegalArgumentException(currentFieldName + " array element should contain char filter's name or setting");
}
charFilters.add(parser.text());
}
analyzeRequest.charFilters(charFilters.toArray(new String[charFilters.size()]));
} else if (parseFieldMatcher.match(currentFieldName, Fields.EXPLAIN)) {
if (parser.isBooleanValue()) {
analyzeRequest.explain(parser.booleanValue());
} else {
throw new IllegalArgumentException(currentFieldName + " must be either 'true' or 'false'");
}
} else if (parseFieldMatcher.match(currentFieldName, Fields.ATTRIBUTES) && token == XContentParser.Token.START_ARRAY){
} else if (parseFieldMatcher.match(currentFieldName, Fields.ATTRIBUTES) && token == XContentParser.Token.START_ARRAY) {
List<String> attributes = new ArrayList<>();
while ((token = parser.nextToken()) != XContentParser.Token.END_ARRAY) {
if (token.isValue() == false) {

View File

@ -61,7 +61,6 @@ import org.elasticsearch.index.query.NestedQueryBuilder;
import org.elasticsearch.index.query.ParentIdQueryBuilder;
import org.elasticsearch.index.query.PrefixQueryBuilder;
import org.elasticsearch.index.query.QueryBuilder;
import org.elasticsearch.index.query.QueryParser;
import org.elasticsearch.index.query.QueryStringQueryBuilder;
import org.elasticsearch.index.query.RangeQueryBuilder;
import org.elasticsearch.index.query.RegexpQueryBuilder;
@ -93,8 +92,9 @@ import org.elasticsearch.index.query.functionscore.WeightBuilder;
import org.elasticsearch.indices.query.IndicesQueriesRegistry;
import org.elasticsearch.plugins.SearchPlugin;
import org.elasticsearch.plugins.SearchPlugin.FetchPhaseConstructionContext;
import org.elasticsearch.plugins.SearchPlugin.QuerySpec;
import org.elasticsearch.plugins.SearchPlugin.ScoreFunctionSpec;
import org.elasticsearch.plugins.SearchPlugin.SearchPluginSpec;
import org.elasticsearch.plugins.SearchPlugin.SearchExtensionSpec;
import org.elasticsearch.search.action.SearchTransportService;
import org.elasticsearch.search.aggregations.AggregationBuilder;
import org.elasticsearch.search.aggregations.Aggregator;
@ -211,6 +211,7 @@ import org.elasticsearch.search.aggregations.pipeline.bucketmetrics.max.MaxBucke
import org.elasticsearch.search.aggregations.pipeline.bucketmetrics.max.MaxBucketPipelineAggregator;
import org.elasticsearch.search.aggregations.pipeline.bucketmetrics.min.MinBucketPipelineAggregationBuilder;
import org.elasticsearch.search.aggregations.pipeline.bucketmetrics.min.MinBucketPipelineAggregator;
import org.elasticsearch.search.aggregations.pipeline.bucketmetrics.percentile.InternalPercentilesBucket;
import org.elasticsearch.search.aggregations.pipeline.bucketmetrics.percentile.PercentilesBucketPipelineAggregationBuilder;
import org.elasticsearch.search.aggregations.pipeline.bucketmetrics.percentile.PercentilesBucketPipelineAggregator;
import org.elasticsearch.search.aggregations.pipeline.bucketmetrics.stats.InternalStatsBucket;
@ -321,7 +322,7 @@ public class SearchModule extends AbstractModule {
suggesters = setupSuggesters(plugins);
highlighters = setupHighlighters(settings, plugins);
registerScoreFunctions(plugins);
registerBuiltinQueryParsers();
registerQueryParsers(plugins);
registerRescorers();
registerSorts();
registerValueFormats();
@ -331,22 +332,6 @@ public class SearchModule extends AbstractModule {
registerFetchSubPhases(plugins);
}
/**
* Register a query.
*
* @param reader the reader registered for this query's builder. Typically a reference to a constructor that takes a
* {@link org.elasticsearch.common.io.stream.StreamInput}
* @param queryParser the parser the reads the query builder from xcontent
* @param queryName holds the names by which this query might be parsed. The {@link ParseField#getPreferredName()} is special as it
* is the name by under which the reader is registered. So it is the name that the query should use as its
* {@link NamedWriteable#getWriteableName()} too.
*/
public <QB extends QueryBuilder> void registerQuery(Writeable.Reader<QB> reader, QueryParser<QB> queryParser,
ParseField queryName) {
queryParserRegistry.register(queryParser, queryName);
namedWriteableRegistry.register(QueryBuilder.class, queryName.getPreferredName(), reader);
}
public Suggesters getSuggesters() {
return new Suggesters(suggesters);
}
@ -437,18 +422,16 @@ public class SearchModule extends AbstractModule {
pipelineAggregationParserRegistry.register(spec.parser, spec.name);
}
namedWriteableRegistry.register(PipelineAggregationBuilder.class, spec.name.getPreferredName(), spec.builderReader);
for (Map.Entry<String, Writeable.Reader<? extends PipelineAggregator>> resultReader : spec.resultReaders.entrySet()) {
namedWriteableRegistry.register(PipelineAggregator.class, resultReader.getKey(), resultReader.getValue());
}
for (Map.Entry<String, Writeable.Reader<? extends InternalAggregation>> bucketReaders : spec.bucketReaders.entrySet()) {
namedWriteableRegistry.register(InternalAggregation.class, bucketReaders.getKey(), bucketReaders.getValue());
namedWriteableRegistry.register(PipelineAggregator.class, spec.name.getPreferredName(), spec.aggregatorReader);
for (Map.Entry<String, Writeable.Reader<? extends InternalAggregation>> resultReader : spec.resultReaders.entrySet()) {
namedWriteableRegistry.register(InternalAggregation.class, resultReader.getKey(), resultReader.getValue());
}
}
public static class PipelineAggregationSpec {
private final Map<String, Writeable.Reader<? extends PipelineAggregator>> resultReaders = new TreeMap<>();
private final Map<String, Writeable.Reader<? extends InternalAggregation>> bucketReaders = new TreeMap<>();
private final Map<String, Writeable.Reader<? extends InternalAggregation>> resultReaders = new TreeMap<>();
private final Writeable.Reader<? extends PipelineAggregationBuilder> builderReader;
private final Writeable.Reader<? extends PipelineAggregator> aggregatorReader;
private final PipelineAggregator.Parser parser;
private final ParseField name;
@ -456,13 +439,16 @@ public class SearchModule extends AbstractModule {
* Register a pipeline aggregation.
*
* @param builderReader reads the {@link PipelineAggregationBuilder} from a stream
* @param aggregatorReader reads the {@link PipelineAggregator} from a stream
* @param parser reads the aggregation builder from XContent
* @param name names by which the aggregation may be parsed. The first name is special because it is the name that the reader is
* registered under.
*/
public PipelineAggregationSpec(Reader<? extends PipelineAggregationBuilder> builderReader,
Writeable.Reader<? extends PipelineAggregator> aggregatorReader,
PipelineAggregator.Parser parser, ParseField name) {
this.builderReader = builderReader;
this.aggregatorReader = aggregatorReader;
this.parser = parser;
this.name = name;
}
@ -471,43 +457,19 @@ public class SearchModule extends AbstractModule {
* Add a reader for the shard level results of the aggregation with {@linkplain #name}'s {@link ParseField#getPreferredName()} as
* the {@link NamedWriteable#getWriteableName()}.
*/
public PipelineAggregationSpec addResultReader(Writeable.Reader<? extends PipelineAggregator> resultReader) {
public PipelineAggregationSpec addResultReader(Writeable.Reader<? extends InternalAggregation> resultReader) {
return addResultReader(name.getPreferredName(), resultReader);
}
/**
* Add a reader for the shard level results of the aggregation.
*/
public PipelineAggregationSpec addResultReader(String writeableName, Writeable.Reader<? extends PipelineAggregator> resultReader) {
public PipelineAggregationSpec addResultReader(String writeableName, Writeable.Reader<? extends InternalAggregation> resultReader) {
resultReaders.put(writeableName, resultReader);
return this;
}
/**
* Add a reader for the shard level bucket results of the aggregation with {@linkplain name}'s {@link ParseField#getPreferredName()}
* as the {@link NamedWriteable#getWriteableName()}.
*/
public PipelineAggregationSpec addBucketReader(Writeable.Reader<? extends InternalAggregation> resultReader) {
return addBucketReader(name.getPreferredName(), resultReader);
}
/**
* Add a reader for the shard level results of the aggregation.
*/
public PipelineAggregationSpec addBucketReader(String writeableName, Writeable.Reader<? extends InternalAggregation> resultReader) {
bucketReaders.put(writeableName, resultReader);
return this;
}
}
public void registerPipelineAggregation(Writeable.Reader<? extends PipelineAggregationBuilder> reader,
PipelineAggregator.Parser aggregationParser, ParseField aggregationName) {
// NORELEASE remove me in favor of the above method
pipelineAggregationParserRegistry.register(aggregationParser, aggregationName);
namedWriteableRegistry.register(PipelineAggregationBuilder.class, aggregationName.getPreferredName(), reader);
}
@Override
protected void configure() {
if (false == transportClient) {
@ -606,49 +568,80 @@ public class SearchModule extends AbstractModule {
registerPipelineAggregation(new PipelineAggregationSpec(
DerivativePipelineAggregationBuilder::new,
DerivativePipelineAggregator::new,
DerivativePipelineAggregationBuilder::parse,
DerivativePipelineAggregationBuilder.AGGREGATION_NAME_FIELD)
.addResultReader(DerivativePipelineAggregator::new)
.addBucketReader(InternalDerivative::new));
registerPipelineAggregation(MaxBucketPipelineAggregationBuilder::new, MaxBucketPipelineAggregationBuilder.PARSER,
MaxBucketPipelineAggregationBuilder.AGGREGATION_NAME_FIELD);
registerPipelineAggregation(MinBucketPipelineAggregationBuilder::new, MinBucketPipelineAggregationBuilder.PARSER,
MinBucketPipelineAggregationBuilder.AGGREGATION_FIELD_NAME);
registerPipelineAggregation(AvgBucketPipelineAggregationBuilder::new, AvgBucketPipelineAggregationBuilder.PARSER,
AvgBucketPipelineAggregationBuilder.AGGREGATION_NAME_FIELD);
registerPipelineAggregation(SumBucketPipelineAggregationBuilder::new, SumBucketPipelineAggregationBuilder.PARSER,
SumBucketPipelineAggregationBuilder.AGGREGATION_NAME_FIELD);
.addResultReader(InternalDerivative::new));
registerPipelineAggregation(new PipelineAggregationSpec(
MaxBucketPipelineAggregationBuilder::new,
MaxBucketPipelineAggregator::new,
MaxBucketPipelineAggregationBuilder.PARSER,
MaxBucketPipelineAggregationBuilder.AGGREGATION_NAME_FIELD)
// This bucket is used by many pipeline aggreations.
.addResultReader(InternalBucketMetricValue.NAME, InternalBucketMetricValue::new));
registerPipelineAggregation(new PipelineAggregationSpec(
MinBucketPipelineAggregationBuilder::new,
MinBucketPipelineAggregator::new,
MinBucketPipelineAggregationBuilder.PARSER,
MinBucketPipelineAggregationBuilder.AGGREGATION_FIELD_NAME)
/* Uses InternalBucketMetricValue */);
registerPipelineAggregation(new PipelineAggregationSpec(
AvgBucketPipelineAggregationBuilder::new,
AvgBucketPipelineAggregator::new,
AvgBucketPipelineAggregationBuilder.PARSER,
AvgBucketPipelineAggregationBuilder.AGGREGATION_NAME_FIELD)
// This bucket is used by many pipeline aggreations.
.addResultReader(InternalSimpleValue.NAME, InternalSimpleValue::new));
registerPipelineAggregation(new PipelineAggregationSpec(
SumBucketPipelineAggregationBuilder::new,
SumBucketPipelineAggregator::new,
SumBucketPipelineAggregationBuilder.PARSER,
SumBucketPipelineAggregationBuilder.AGGREGATION_NAME_FIELD)
/* Uses InternalSimpleValue */);
registerPipelineAggregation(new PipelineAggregationSpec(
StatsBucketPipelineAggregationBuilder::new,
StatsBucketPipelineAggregator::new,
StatsBucketPipelineAggregationBuilder.PARSER,
StatsBucketPipelineAggregationBuilder.AGGREGATION_NAME_FIELD)
.addResultReader(StatsBucketPipelineAggregator::new)
.addBucketReader(InternalStatsBucket::new));
.addResultReader(InternalStatsBucket::new));
registerPipelineAggregation(new PipelineAggregationSpec(
ExtendedStatsBucketPipelineAggregationBuilder::new,
ExtendedStatsBucketPipelineAggregator::new,
new ExtendedStatsBucketParser(),
ExtendedStatsBucketPipelineAggregationBuilder.AGGREGATION_NAME_FIELD)
.addResultReader(ExtendedStatsBucketPipelineAggregator::new)
.addBucketReader(InternalExtendedStatsBucket::new));
registerPipelineAggregation(PercentilesBucketPipelineAggregationBuilder::new, PercentilesBucketPipelineAggregationBuilder.PARSER,
PercentilesBucketPipelineAggregationBuilder.AGGREGATION_NAME_FIELD);
.addResultReader(InternalExtendedStatsBucket::new));
registerPipelineAggregation(new PipelineAggregationSpec(
PercentilesBucketPipelineAggregationBuilder::new,
PercentilesBucketPipelineAggregator::new,
PercentilesBucketPipelineAggregationBuilder.PARSER,
PercentilesBucketPipelineAggregationBuilder.AGGREGATION_NAME_FIELD)
.addResultReader(InternalPercentilesBucket::new));
registerPipelineAggregation(new PipelineAggregationSpec(
MovAvgPipelineAggregationBuilder::new,
MovAvgPipelineAggregator::new,
(n, c) -> MovAvgPipelineAggregationBuilder.parse(movingAverageModelParserRegistry, n, c),
MovAvgPipelineAggregationBuilder.AGGREGATION_FIELD_NAME)
.addResultReader(MovAvgPipelineAggregator::new)
/* Uses InternalHistogram for buckets */);
registerPipelineAggregation(CumulativeSumPipelineAggregationBuilder::new, CumulativeSumPipelineAggregationBuilder::parse,
CumulativeSumPipelineAggregationBuilder.AGGREGATION_NAME_FIELD);
registerPipelineAggregation(BucketScriptPipelineAggregationBuilder::new, BucketScriptPipelineAggregationBuilder::parse,
BucketScriptPipelineAggregationBuilder.AGGREGATION_NAME_FIELD);
registerPipelineAggregation(BucketSelectorPipelineAggregationBuilder::new, BucketSelectorPipelineAggregationBuilder::parse,
BucketSelectorPipelineAggregationBuilder.AGGREGATION_NAME_FIELD);
registerPipelineAggregation(new PipelineAggregationSpec(
CumulativeSumPipelineAggregationBuilder::new,
CumulativeSumPipelineAggregator::new,
CumulativeSumPipelineAggregationBuilder::parse,
CumulativeSumPipelineAggregationBuilder.AGGREGATION_NAME_FIELD));
registerPipelineAggregation(new PipelineAggregationSpec(
BucketScriptPipelineAggregationBuilder::new,
BucketScriptPipelineAggregator::new,
BucketScriptPipelineAggregationBuilder::parse,
BucketScriptPipelineAggregationBuilder.AGGREGATION_NAME_FIELD));
registerPipelineAggregation(new PipelineAggregationSpec(
BucketSelectorPipelineAggregationBuilder::new,
BucketSelectorPipelineAggregator::new,
BucketSelectorPipelineAggregationBuilder::parse,
BucketSelectorPipelineAggregationBuilder.AGGREGATION_NAME_FIELD));
registerPipelineAggregation(new PipelineAggregationSpec(
SerialDiffPipelineAggregationBuilder::new,
SerialDiffPipelineAggregator::new,
SerialDiffPipelineAggregationBuilder::parse,
SerialDiffPipelineAggregationBuilder.AGGREGATION_NAME_FIELD)
.addResultReader(SerialDiffPipelineAggregator::new));
SerialDiffPipelineAggregationBuilder.AGGREGATION_NAME_FIELD));
}
protected void configureSearch() {
@ -767,32 +760,32 @@ public class SearchModule extends AbstractModule {
}
private void registerSignificanceHeuristics(List<SearchPlugin> plugins) {
registerSignificanceHeuristic(new SearchPluginSpec<>(ChiSquare.NAME, ChiSquare::new, ChiSquare.PARSER));
registerSignificanceHeuristic(new SearchPluginSpec<>(GND.NAME, GND::new, GND.PARSER));
registerSignificanceHeuristic(new SearchPluginSpec<>(JLHScore.NAME, JLHScore::new, JLHScore::parse));
registerSignificanceHeuristic(new SearchPluginSpec<>(MutualInformation.NAME, MutualInformation::new, MutualInformation.PARSER));
registerSignificanceHeuristic(new SearchPluginSpec<>(PercentageScore.NAME, PercentageScore::new, PercentageScore::parse));
registerSignificanceHeuristic(new SearchPluginSpec<>(ScriptHeuristic.NAME, ScriptHeuristic::new, ScriptHeuristic::parse));
registerSignificanceHeuristic(new SearchExtensionSpec<>(ChiSquare.NAME, ChiSquare::new, ChiSquare.PARSER));
registerSignificanceHeuristic(new SearchExtensionSpec<>(GND.NAME, GND::new, GND.PARSER));
registerSignificanceHeuristic(new SearchExtensionSpec<>(JLHScore.NAME, JLHScore::new, JLHScore::parse));
registerSignificanceHeuristic(new SearchExtensionSpec<>(MutualInformation.NAME, MutualInformation::new, MutualInformation.PARSER));
registerSignificanceHeuristic(new SearchExtensionSpec<>(PercentageScore.NAME, PercentageScore::new, PercentageScore::parse));
registerSignificanceHeuristic(new SearchExtensionSpec<>(ScriptHeuristic.NAME, ScriptHeuristic::new, ScriptHeuristic::parse));
registerFromPlugin(plugins, SearchPlugin::getSignificanceHeuristics, this::registerSignificanceHeuristic);
}
private void registerSignificanceHeuristic(SearchPluginSpec<SignificanceHeuristic, SignificanceHeuristicParser> heuristic) {
private void registerSignificanceHeuristic(SearchExtensionSpec<SignificanceHeuristic, SignificanceHeuristicParser> heuristic) {
significanceHeuristicParserRegistry.register(heuristic.getParser(), heuristic.getName());
namedWriteableRegistry.register(SignificanceHeuristic.class, heuristic.getName().getPreferredName(), heuristic.getReader());
}
private void registerMovingAverageModels(List<SearchPlugin> plugins) {
registerMovingAverageModel(new SearchPluginSpec<>(SimpleModel.NAME, SimpleModel::new, SimpleModel.PARSER));
registerMovingAverageModel(new SearchPluginSpec<>(LinearModel.NAME, LinearModel::new, LinearModel.PARSER));
registerMovingAverageModel(new SearchPluginSpec<>(EwmaModel.NAME, EwmaModel::new, EwmaModel.PARSER));
registerMovingAverageModel(new SearchPluginSpec<>(HoltLinearModel.NAME, HoltLinearModel::new, HoltLinearModel.PARSER));
registerMovingAverageModel(new SearchPluginSpec<>(HoltWintersModel.NAME, HoltWintersModel::new, HoltWintersModel.PARSER));
registerMovingAverageModel(new SearchExtensionSpec<>(SimpleModel.NAME, SimpleModel::new, SimpleModel.PARSER));
registerMovingAverageModel(new SearchExtensionSpec<>(LinearModel.NAME, LinearModel::new, LinearModel.PARSER));
registerMovingAverageModel(new SearchExtensionSpec<>(EwmaModel.NAME, EwmaModel::new, EwmaModel.PARSER));
registerMovingAverageModel(new SearchExtensionSpec<>(HoltLinearModel.NAME, HoltLinearModel::new, HoltLinearModel.PARSER));
registerMovingAverageModel(new SearchExtensionSpec<>(HoltWintersModel.NAME, HoltWintersModel::new, HoltWintersModel.PARSER));
registerFromPlugin(plugins, SearchPlugin::getMovingAverageModels, this::registerMovingAverageModel);
}
private void registerMovingAverageModel(SearchPluginSpec<MovAvgModel, MovAvgModel.AbstractModelParser> movAvgModel) {
private void registerMovingAverageModel(SearchExtensionSpec<MovAvgModel, MovAvgModel.AbstractModelParser> movAvgModel) {
movingAverageModelParserRegistry.register(movAvgModel.getParser(), movAvgModel.getName());
namedWriteableRegistry.register(MovAvgModel.class, movAvgModel.getName().getPreferredName(), movAvgModel.getReader());
}
@ -819,78 +812,75 @@ public class SearchModule extends AbstractModule {
fetchSubPhases.add(requireNonNull(subPhase, "FetchSubPhase must not be null"));
}
private void registerBuiltinQueryParsers() {
registerQuery(MatchQueryBuilder::new, MatchQueryBuilder::fromXContent, MatchQueryBuilder.QUERY_NAME_FIELD);
registerQuery(MatchPhraseQueryBuilder::new, MatchPhraseQueryBuilder::fromXContent, MatchPhraseQueryBuilder.QUERY_NAME_FIELD);
registerQuery(MatchPhrasePrefixQueryBuilder::new, MatchPhrasePrefixQueryBuilder::fromXContent,
MatchPhrasePrefixQueryBuilder.QUERY_NAME_FIELD);
registerQuery(MultiMatchQueryBuilder::new, MultiMatchQueryBuilder::fromXContent, MultiMatchQueryBuilder.QUERY_NAME_FIELD);
registerQuery(NestedQueryBuilder::new, NestedQueryBuilder::fromXContent, NestedQueryBuilder.QUERY_NAME_FIELD);
registerQuery(HasChildQueryBuilder::new, HasChildQueryBuilder::fromXContent, HasChildQueryBuilder.QUERY_NAME_FIELD);
registerQuery(HasParentQueryBuilder::new, HasParentQueryBuilder::fromXContent, HasParentQueryBuilder.QUERY_NAME_FIELD);
registerQuery(DisMaxQueryBuilder::new, DisMaxQueryBuilder::fromXContent, DisMaxQueryBuilder.QUERY_NAME_FIELD);
registerQuery(IdsQueryBuilder::new, IdsQueryBuilder::fromXContent, IdsQueryBuilder.QUERY_NAME_FIELD);
registerQuery(MatchAllQueryBuilder::new, MatchAllQueryBuilder::fromXContent, MatchAllQueryBuilder.QUERY_NAME_FIELD);
registerQuery(QueryStringQueryBuilder::new, QueryStringQueryBuilder::fromXContent, QueryStringQueryBuilder.QUERY_NAME_FIELD);
registerQuery(BoostingQueryBuilder::new, BoostingQueryBuilder::fromXContent, BoostingQueryBuilder.QUERY_NAME_FIELD);
private void registerQueryParsers(List<SearchPlugin> plugins) {
registerQuery(new QuerySpec<>(MatchQueryBuilder.QUERY_NAME_FIELD, MatchQueryBuilder::new, MatchQueryBuilder::fromXContent));
registerQuery(new QuerySpec<>(MatchPhraseQueryBuilder.NAME, MatchPhraseQueryBuilder::new, MatchPhraseQueryBuilder::fromXContent));
registerQuery(new QuerySpec<>(MatchPhrasePrefixQueryBuilder.NAME, MatchPhrasePrefixQueryBuilder::new,
MatchPhrasePrefixQueryBuilder::fromXContent));
registerQuery(new QuerySpec<>(MultiMatchQueryBuilder.NAME, MultiMatchQueryBuilder::new, MultiMatchQueryBuilder::fromXContent));
registerQuery(new QuerySpec<>(NestedQueryBuilder.NAME, NestedQueryBuilder::new, NestedQueryBuilder::fromXContent));
registerQuery(new QuerySpec<>(HasChildQueryBuilder.NAME, HasChildQueryBuilder::new, HasChildQueryBuilder::fromXContent));
registerQuery(new QuerySpec<>(HasParentQueryBuilder.NAME, HasParentQueryBuilder::new, HasParentQueryBuilder::fromXContent));
registerQuery(new QuerySpec<>(DisMaxQueryBuilder.NAME, DisMaxQueryBuilder::new, DisMaxQueryBuilder::fromXContent));
registerQuery(new QuerySpec<>(IdsQueryBuilder.NAME, IdsQueryBuilder::new, IdsQueryBuilder::fromXContent));
registerQuery(new QuerySpec<>(MatchAllQueryBuilder.NAME, MatchAllQueryBuilder::new, MatchAllQueryBuilder::fromXContent));
registerQuery(new QuerySpec<>(QueryStringQueryBuilder.NAME, QueryStringQueryBuilder::new, QueryStringQueryBuilder::fromXContent));
registerQuery(new QuerySpec<>(BoostingQueryBuilder.NAME, BoostingQueryBuilder::new, BoostingQueryBuilder::fromXContent));
BooleanQuery.setMaxClauseCount(INDICES_MAX_CLAUSE_COUNT_SETTING.get(settings));
registerQuery(BoolQueryBuilder::new, BoolQueryBuilder::fromXContent, BoolQueryBuilder.QUERY_NAME_FIELD);
registerQuery(TermQueryBuilder::new, TermQueryBuilder::fromXContent, TermQueryBuilder.QUERY_NAME_FIELD);
registerQuery(TermsQueryBuilder::new, TermsQueryBuilder::fromXContent, TermsQueryBuilder.QUERY_NAME_FIELD);
registerQuery(FuzzyQueryBuilder::new, FuzzyQueryBuilder::fromXContent, FuzzyQueryBuilder.QUERY_NAME_FIELD);
registerQuery(RegexpQueryBuilder::new, RegexpQueryBuilder::fromXContent, RegexpQueryBuilder.QUERY_NAME_FIELD);
registerQuery(RangeQueryBuilder::new, RangeQueryBuilder::fromXContent, RangeQueryBuilder.QUERY_NAME_FIELD);
registerQuery(PrefixQueryBuilder::new, PrefixQueryBuilder::fromXContent, PrefixQueryBuilder.QUERY_NAME_FIELD);
registerQuery(WildcardQueryBuilder::new, WildcardQueryBuilder::fromXContent, WildcardQueryBuilder.QUERY_NAME_FIELD);
registerQuery(ConstantScoreQueryBuilder::new, ConstantScoreQueryBuilder::fromXContent, ConstantScoreQueryBuilder.QUERY_NAME_FIELD);
registerQuery(SpanTermQueryBuilder::new, SpanTermQueryBuilder::fromXContent, SpanTermQueryBuilder.QUERY_NAME_FIELD);
registerQuery(SpanNotQueryBuilder::new, SpanNotQueryBuilder::fromXContent, SpanNotQueryBuilder.QUERY_NAME_FIELD);
registerQuery(SpanWithinQueryBuilder::new, SpanWithinQueryBuilder::fromXContent, SpanWithinQueryBuilder.QUERY_NAME_FIELD);
registerQuery(SpanContainingQueryBuilder::new, SpanContainingQueryBuilder::fromXContent,
SpanContainingQueryBuilder.QUERY_NAME_FIELD);
registerQuery(FieldMaskingSpanQueryBuilder::new, FieldMaskingSpanQueryBuilder::fromXContent,
FieldMaskingSpanQueryBuilder.QUERY_NAME_FIELD);
registerQuery(SpanFirstQueryBuilder::new, SpanFirstQueryBuilder::fromXContent, SpanFirstQueryBuilder.QUERY_NAME_FIELD);
registerQuery(SpanNearQueryBuilder::new, SpanNearQueryBuilder::fromXContent, SpanNearQueryBuilder.QUERY_NAME_FIELD);
registerQuery(SpanOrQueryBuilder::new, SpanOrQueryBuilder::fromXContent, SpanOrQueryBuilder.QUERY_NAME_FIELD);
registerQuery(MoreLikeThisQueryBuilder::new, MoreLikeThisQueryBuilder::fromXContent, MoreLikeThisQueryBuilder.QUERY_NAME_FIELD);
registerQuery(WrapperQueryBuilder::new, WrapperQueryBuilder::fromXContent, WrapperQueryBuilder.QUERY_NAME_FIELD);
registerQuery(new QuerySpec<>(BoolQueryBuilder.NAME, BoolQueryBuilder::new, BoolQueryBuilder::fromXContent));
registerQuery(new QuerySpec<>(TermQueryBuilder.NAME, TermQueryBuilder::new, TermQueryBuilder::fromXContent));
registerQuery(new QuerySpec<>(TermsQueryBuilder.QUERY_NAME_FIELD, TermsQueryBuilder::new, TermsQueryBuilder::fromXContent));
registerQuery(new QuerySpec<>(FuzzyQueryBuilder.NAME, FuzzyQueryBuilder::new, FuzzyQueryBuilder::fromXContent));
registerQuery(new QuerySpec<>(RegexpQueryBuilder.NAME, RegexpQueryBuilder::new, RegexpQueryBuilder::fromXContent));
registerQuery(new QuerySpec<>(RangeQueryBuilder.NAME, RangeQueryBuilder::new, RangeQueryBuilder::fromXContent));
registerQuery(new QuerySpec<>(PrefixQueryBuilder.NAME, PrefixQueryBuilder::new, PrefixQueryBuilder::fromXContent));
registerQuery(new QuerySpec<>(WildcardQueryBuilder.NAME, WildcardQueryBuilder::new, WildcardQueryBuilder::fromXContent));
registerQuery(
new QuerySpec<>(ConstantScoreQueryBuilder.NAME, ConstantScoreQueryBuilder::new, ConstantScoreQueryBuilder::fromXContent));
registerQuery(new QuerySpec<>(SpanTermQueryBuilder.NAME, SpanTermQueryBuilder::new, SpanTermQueryBuilder::fromXContent));
registerQuery(new QuerySpec<>(SpanNotQueryBuilder.NAME, SpanNotQueryBuilder::new, SpanNotQueryBuilder::fromXContent));
registerQuery(new QuerySpec<>(SpanWithinQueryBuilder.NAME, SpanWithinQueryBuilder::new, SpanWithinQueryBuilder::fromXContent));
registerQuery(new QuerySpec<>(SpanContainingQueryBuilder.NAME, SpanContainingQueryBuilder::new,
SpanContainingQueryBuilder::fromXContent));
registerQuery(new QuerySpec<>(FieldMaskingSpanQueryBuilder.NAME, FieldMaskingSpanQueryBuilder::new,
FieldMaskingSpanQueryBuilder::fromXContent));
registerQuery(new QuerySpec<>(SpanFirstQueryBuilder.NAME, SpanFirstQueryBuilder::new, SpanFirstQueryBuilder::fromXContent));
registerQuery(new QuerySpec<>(SpanNearQueryBuilder.NAME, SpanNearQueryBuilder::new, SpanNearQueryBuilder::fromXContent));
registerQuery(new QuerySpec<>(SpanOrQueryBuilder.NAME, SpanOrQueryBuilder::new, SpanOrQueryBuilder::fromXContent));
registerQuery(new QuerySpec<>(MoreLikeThisQueryBuilder.QUERY_NAME_FIELD, MoreLikeThisQueryBuilder::new,
MoreLikeThisQueryBuilder::fromXContent));
registerQuery(new QuerySpec<>(WrapperQueryBuilder.NAME, WrapperQueryBuilder::new, WrapperQueryBuilder::fromXContent));
// TODO Remove IndicesQuery in 6.0
registerQuery(IndicesQueryBuilder::new, IndicesQueryBuilder::fromXContent, IndicesQueryBuilder.QUERY_NAME_FIELD);
registerQuery(CommonTermsQueryBuilder::new, CommonTermsQueryBuilder::fromXContent, CommonTermsQueryBuilder.QUERY_NAME_FIELD);
registerQuery(SpanMultiTermQueryBuilder::new, SpanMultiTermQueryBuilder::fromXContent, SpanMultiTermQueryBuilder.QUERY_NAME_FIELD);
registerQuery(FunctionScoreQueryBuilder::new, c -> FunctionScoreQueryBuilder.fromXContent(scoreFunctionParserRegistry, c),
FunctionScoreQueryBuilder.QUERY_NAME_FIELD);
registerQuery(SimpleQueryStringBuilder::new, SimpleQueryStringBuilder::fromXContent, SimpleQueryStringBuilder.QUERY_NAME_FIELD);
registerQuery(TypeQueryBuilder::new, TypeQueryBuilder::fromXContent, TypeQueryBuilder.QUERY_NAME_FIELD);
registerQuery(ScriptQueryBuilder::new, ScriptQueryBuilder::fromXContent, ScriptQueryBuilder.QUERY_NAME_FIELD);
registerQuery(GeoDistanceQueryBuilder::new, GeoDistanceQueryBuilder::fromXContent, GeoDistanceQueryBuilder.QUERY_NAME_FIELD);
registerQuery(GeoDistanceRangeQueryBuilder::new, GeoDistanceRangeQueryBuilder::fromXContent,
GeoDistanceRangeQueryBuilder.QUERY_NAME_FIELD);
registerQuery(GeoBoundingBoxQueryBuilder::new, GeoBoundingBoxQueryBuilder::fromXContent,
GeoBoundingBoxQueryBuilder.QUERY_NAME_FIELD);
registerQuery(GeohashCellQuery.Builder::new, GeohashCellQuery.Builder::fromXContent, GeohashCellQuery.QUERY_NAME_FIELD);
registerQuery(GeoPolygonQueryBuilder::new, GeoPolygonQueryBuilder::fromXContent, GeoPolygonQueryBuilder.QUERY_NAME_FIELD);
registerQuery(ExistsQueryBuilder::new, ExistsQueryBuilder::fromXContent, ExistsQueryBuilder.QUERY_NAME_FIELD);
registerQuery(MatchNoneQueryBuilder::new, MatchNoneQueryBuilder::fromXContent, MatchNoneQueryBuilder.QUERY_NAME_FIELD);
registerQuery(ParentIdQueryBuilder::new, ParentIdQueryBuilder::fromXContent, ParentIdQueryBuilder.QUERY_NAME_FIELD);
registerQuery(new QuerySpec<>(IndicesQueryBuilder.NAME, IndicesQueryBuilder::new, IndicesQueryBuilder::fromXContent));
registerQuery(new QuerySpec<>(CommonTermsQueryBuilder.NAME, CommonTermsQueryBuilder::new, CommonTermsQueryBuilder::fromXContent));
registerQuery(
new QuerySpec<>(SpanMultiTermQueryBuilder.NAME, SpanMultiTermQueryBuilder::new, SpanMultiTermQueryBuilder::fromXContent));
registerQuery(new QuerySpec<>(FunctionScoreQueryBuilder.NAME, FunctionScoreQueryBuilder::new,
c -> FunctionScoreQueryBuilder.fromXContent(scoreFunctionParserRegistry, c)));
registerQuery(
new QuerySpec<>(SimpleQueryStringBuilder.NAME, SimpleQueryStringBuilder::new, SimpleQueryStringBuilder::fromXContent));
registerQuery(new QuerySpec<>(TypeQueryBuilder.NAME, TypeQueryBuilder::new, TypeQueryBuilder::fromXContent));
registerQuery(new QuerySpec<>(ScriptQueryBuilder.NAME, ScriptQueryBuilder::new, ScriptQueryBuilder::fromXContent));
registerQuery(new QuerySpec<>(GeoDistanceQueryBuilder.NAME, GeoDistanceQueryBuilder::new, GeoDistanceQueryBuilder::fromXContent));
registerQuery(new QuerySpec<>(GeoDistanceRangeQueryBuilder.NAME, GeoDistanceRangeQueryBuilder::new,
GeoDistanceRangeQueryBuilder::fromXContent));
registerQuery(new QuerySpec<>(GeoBoundingBoxQueryBuilder.QUERY_NAME_FIELD, GeoBoundingBoxQueryBuilder::new,
GeoBoundingBoxQueryBuilder::fromXContent));
registerQuery(new QuerySpec<>(GeohashCellQuery.NAME, GeohashCellQuery.Builder::new, GeohashCellQuery.Builder::fromXContent));
registerQuery(new QuerySpec<>(GeoPolygonQueryBuilder.NAME, GeoPolygonQueryBuilder::new, GeoPolygonQueryBuilder::fromXContent));
registerQuery(new QuerySpec<>(ExistsQueryBuilder.NAME, ExistsQueryBuilder::new, ExistsQueryBuilder::fromXContent));
registerQuery(new QuerySpec<>(MatchNoneQueryBuilder.NAME, MatchNoneQueryBuilder::new, MatchNoneQueryBuilder::fromXContent));
registerQuery(new QuerySpec<>(ParentIdQueryBuilder.NAME, ParentIdQueryBuilder::new, ParentIdQueryBuilder::fromXContent));
if (ShapesAvailability.JTS_AVAILABLE && ShapesAvailability.SPATIAL4J_AVAILABLE) {
registerQuery(GeoShapeQueryBuilder::new, GeoShapeQueryBuilder::fromXContent, GeoShapeQueryBuilder.QUERY_NAME_FIELD);
registerQuery(new QuerySpec<>(GeoShapeQueryBuilder.NAME, GeoShapeQueryBuilder::new, GeoShapeQueryBuilder::fromXContent));
}
registerFromPlugin(plugins, SearchPlugin::getQueries, this::registerQuery);
}
static {
// Pipeline Aggregations
InternalSimpleValue.registerStreams();
InternalBucketMetricValue.registerStreams();
MaxBucketPipelineAggregator.registerStreams();
MinBucketPipelineAggregator.registerStreams();
AvgBucketPipelineAggregator.registerStreams();
SumBucketPipelineAggregator.registerStreams();
PercentilesBucketPipelineAggregator.registerStreams();
CumulativeSumPipelineAggregator.registerStreams();
BucketScriptPipelineAggregator.registerStreams();
BucketSelectorPipelineAggregator.registerStreams();
private void registerQuery(QuerySpec<?> spec) {
queryParserRegistry.register(spec.getParser(), spec.getName());
namedWriteableRegistry.register(QueryBuilder.class, spec.getName().getPreferredName(), spec.getReader());
}
}

View File

@ -89,6 +89,8 @@ import org.elasticsearch.search.searchafter.SearchAfterBuilder;
import org.elasticsearch.search.sort.SortAndFormats;
import org.elasticsearch.search.sort.SortBuilder;
import org.elasticsearch.threadpool.ThreadPool;
import org.elasticsearch.threadpool.ThreadPool.Cancellable;
import org.elasticsearch.threadpool.ThreadPool.Names;
import java.io.IOException;
import java.util.Collections;
@ -96,7 +98,6 @@ import java.util.HashMap;
import java.util.Map;
import java.util.Optional;
import java.util.concurrent.ExecutionException;
import java.util.concurrent.ScheduledFuture;
import java.util.concurrent.atomic.AtomicLong;
import static java.util.Collections.unmodifiableMap;
@ -139,7 +140,7 @@ public class SearchService extends AbstractLifecycleComponent implements IndexEv
private volatile TimeValue defaultSearchTimeout;
private final ScheduledFuture<?> keepAliveReaper;
private final Cancellable keepAliveReaper;
private final AtomicLong idGenerator = new AtomicLong();
@ -171,7 +172,7 @@ public class SearchService extends AbstractLifecycleComponent implements IndexEv
elementParsers.putAll(fetchPhase.parseElements());
this.elementParsers = unmodifiableMap(elementParsers);
this.keepAliveReaper = threadPool.scheduleWithFixedDelay(new Reaper(), keepAliveInterval);
this.keepAliveReaper = threadPool.scheduleWithFixedDelay(new Reaper(), keepAliveInterval, Names.SAME);
defaultSearchTimeout = DEFAULT_SEARCH_TIMEOUT_SETTING.get(settings);
clusterSettings.addSettingsUpdateConsumer(DEFAULT_SEARCH_TIMEOUT_SETTING, this::setDefaultSearchTimeout);
@ -224,7 +225,7 @@ public class SearchService extends AbstractLifecycleComponent implements IndexEv
@Override
protected void doClose() {
doStop();
FutureUtils.cancel(keepAliveReaper);
keepAliveReaper.cancel();
}
public DfsSearchResult executeDfsPhase(ShardSearchRequest request) throws IOException {

View File

@ -154,8 +154,8 @@ public class AggregationPhase implements SearchPhase {
siblingPipelineAggregators.add((SiblingPipelineAggregator) pipelineAggregator);
} else {
throw new AggregationExecutionException("Invalid pipeline aggregation named [" + pipelineAggregator.name()
+ "] of type [" + pipelineAggregator.type().name()
+ "]. Only sibling pipeline aggregations are allowed at the top level");
+ "] of type [" + pipelineAggregator.getWriteableName() + "]. Only sibling pipeline aggregations are "
+ "allowed at the top level");
}
}
context.queryResult().pipelineAggregators(siblingPipelineAggregators);

View File

@ -1,69 +0,0 @@
/*
* Licensed to Elasticsearch under one or more contributor
* license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright
* ownership. Elasticsearch licenses this file to you under
* the Apache License, Version 2.0 (the "License"); you may
* not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing,
* software distributed under the License is distributed on an
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
* KIND, either express or implied. See the License for the
* specific language governing permissions and limitations
* under the License.
*/
package org.elasticsearch.search.aggregations;
import org.elasticsearch.common.bytes.BytesReference;
import org.elasticsearch.common.io.stream.StreamInput;
import java.io.IOException;
import java.util.HashMap;
import java.util.Map;
import static java.util.Collections.emptyMap;
import static java.util.Collections.unmodifiableMap;
/**
* A registry for all the dedicated streams in the aggregation module. This is to support dynamic addAggregation that
* know how to stream themselves.
*/
public class AggregationStreams {
private static Map<BytesReference, Stream> streams = emptyMap();
/**
* A stream that knows how to read an aggregation from the input.
*/
public interface Stream {
InternalAggregation readResult(StreamInput in) throws IOException;
}
/**
* Registers the given stream and associate it with the given types.
*
* @param stream The streams to register
* @param types The types associated with the streams
*/
public static synchronized void registerStream(Stream stream, BytesReference... types) {
Map<BytesReference, Stream> newStreams = new HashMap<>(streams);
for (BytesReference type : types) {
newStreams.put(type, stream);
}
streams = unmodifiableMap(newStreams);
}
/**
* Returns the stream that is registered for the given type
*
* @param type The given type
* @return The associated stream
*/
public static Stream stream(BytesReference type) {
return streams.get(type);
}
}

View File

@ -20,55 +20,35 @@ package org.elasticsearch.search.aggregations;
import org.elasticsearch.cluster.ClusterState;
import org.elasticsearch.common.ParseField;
import org.elasticsearch.common.bytes.BytesArray;
import org.elasticsearch.common.bytes.BytesReference;
import org.elasticsearch.common.io.stream.NamedWriteable;
import org.elasticsearch.common.io.stream.StreamInput;
import org.elasticsearch.common.io.stream.StreamOutput;
import org.elasticsearch.common.io.stream.Streamable;
import org.elasticsearch.common.util.BigArrays;
import org.elasticsearch.common.xcontent.ToXContent;
import org.elasticsearch.common.xcontent.XContentBuilder;
import org.elasticsearch.script.ScriptService;
import org.elasticsearch.search.aggregations.pipeline.PipelineAggregator;
import org.elasticsearch.search.aggregations.pipeline.PipelineAggregatorStreams;
import org.elasticsearch.search.aggregations.support.AggregationPath;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Collections;
import java.util.List;
import java.util.Map;
/**
* An internal implementation of {@link Aggregation}. Serves as a base class for all aggregation implementations.
*/
public abstract class InternalAggregation implements Aggregation, ToXContent, Streamable, NamedWriteable {
// NORELEASE remove Streamable
public abstract class InternalAggregation implements Aggregation, ToXContent, NamedWriteable {
/**
* The aggregation type that holds all the string types that are associated with an aggregation:
* <ul>
* <li>name - used as the parser type</li>
* <li>stream - used as the stream type</li>
* </ul>
*/
public static class Type {
private String name;
private BytesReference stream;
private final String name;
public Type(String name) {
this(name, new BytesArray(name));
}
public Type(String name, String stream) {
this(name, new BytesArray(stream));
}
public Type(String name, BytesReference stream) {
this.name = name;
this.stream = stream;
}
/**
@ -79,14 +59,6 @@ public abstract class InternalAggregation implements Aggregation, ToXContent, St
return name;
}
/**
* @return The name of the stream type (used for registering the aggregation stream
* (see {@link AggregationStreams#registerStream(AggregationStreams.Stream, BytesReference...)}).
*/
public BytesReference stream() {
return stream;
}
@Override
public String toString() {
return name;
@ -118,15 +90,11 @@ public abstract class InternalAggregation implements Aggregation, ToXContent, St
}
}
protected final String name;
protected String name;
protected final Map<String, Object> metaData;
protected Map<String, Object> metaData;
private List<PipelineAggregator> pipelineAggregators;
/** Constructs an un initialized addAggregation (used for serialization) **/
protected InternalAggregation() {} // NORELEASE remove when removing Streamable
private final List<PipelineAggregator> pipelineAggregators;
/**
* Constructs an get with a given name.
@ -145,96 +113,25 @@ public abstract class InternalAggregation implements Aggregation, ToXContent, St
protected InternalAggregation(StreamInput in) throws IOException {
name = in.readString();
metaData = in.readMap();
int size = in.readVInt();
if (size == 0) {
pipelineAggregators = Collections.emptyList();
} else {
pipelineAggregators = new ArrayList<>(size);
for (int i = 0; i < size; i++) {
if (in.readBoolean()) {
pipelineAggregators.add(in.readNamedWriteable(PipelineAggregator.class));
} else {
BytesReference type = in.readBytesReference();
PipelineAggregator pipelineAggregator = PipelineAggregatorStreams.stream(type).readResult(in);
pipelineAggregators.add(pipelineAggregator);
}
}
}
}
@Override
public final void readFrom(StreamInput in) throws IOException {
try {
getWriteableName(); // Throws UnsupportedOperationException if this aggregation should be read using old style Streams
assert false : "Used reading constructor instead";
} catch (UnsupportedOperationException e) {
// OK
}
name = in.readString();
metaData = in.readMap();
int size = in.readVInt();
if (size == 0) {
pipelineAggregators = Collections.emptyList();
} else {
pipelineAggregators = new ArrayList<>(size);
for (int i = 0; i < size; i++) {
if (in.readBoolean()) {
pipelineAggregators.add(in.readNamedWriteable(PipelineAggregator.class));
} else {
BytesReference type = in.readBytesReference();
PipelineAggregator pipelineAggregator = PipelineAggregatorStreams.stream(type).readResult(in);
pipelineAggregators.add(pipelineAggregator);
}
}
}
doReadFrom(in);
}
protected void doReadFrom(StreamInput in) throws IOException {
throw new UnsupportedOperationException("Use reading constructor instead"); // NORELEASE remove when we remove Streamable
pipelineAggregators = in.readNamedWriteableList(PipelineAggregator.class);
}
@Override
public final void writeTo(StreamOutput out) throws IOException {
out.writeString(name);
out.writeGenericValue(metaData);
out.writeVInt(pipelineAggregators.size());
for (PipelineAggregator pipelineAggregator : pipelineAggregators) {
// NORELEASE temporary hack to support old style streams and new style NamedWriteable
try {
pipelineAggregator.getWriteableName(); // Throws UnsupportedOperationException if we should use old style streams.
out.writeBoolean(true);
out.writeNamedWriteable(pipelineAggregator);
} catch (UnsupportedOperationException e) {
out.writeBoolean(false);
out.writeBytesReference(pipelineAggregator.type().stream());
pipelineAggregator.writeTo(out);
}
}
out.writeNamedWriteableList(pipelineAggregators);
doWriteTo(out);
}
protected abstract void doWriteTo(StreamOutput out) throws IOException;
@Override
public String getWriteableName() {
// NORELEASE remove me when all InternalAggregations override it
throw new UnsupportedOperationException("Override on every class");
}
@Override
public String getName() {
return name;
}
/**
* @return The {@link Type} of this aggregation
*/
public Type type() {
// NORELEASE remove this method
throw new UnsupportedOperationException(getClass().getName() + " used type but should Use getWriteableName instead");
}
/**
* Reduces the given aggregations to a single one and returns it. In <b>most</b> cases, the assumption will be the all given
* aggregations are of the same type (the same type as this aggregation). For best efficiency, when implementing,

View File

@ -18,7 +18,6 @@
*/
package org.elasticsearch.search.aggregations;
import org.elasticsearch.common.bytes.BytesReference;
import org.elasticsearch.common.io.stream.StreamInput;
import org.elasticsearch.common.io.stream.StreamOutput;
import org.elasticsearch.common.io.stream.Streamable;
@ -198,41 +197,15 @@ public class InternalAggregations implements Aggregations, ToXContent, Streamabl
@Override
public void readFrom(StreamInput in) throws IOException {
int size = in.readVInt();
if (size == 0) {
aggregations = Collections.emptyList();
aggregations = in.readList(stream -> in.readNamedWriteable(InternalAggregation.class));
if (aggregations.isEmpty()) {
aggregationsAsMap = emptyMap();
} else {
aggregations = new ArrayList<>(size);
for (int i = 0; i < size; i++) {
// NORELEASE temporary hack to support old style streams and new style NamedWriteable at the same time
if (in.readBoolean()) {
aggregations.add(in.readNamedWriteable(InternalAggregation.class));
} else {
BytesReference type = in.readBytesReference();
InternalAggregation aggregation = AggregationStreams.stream(type).readResult(in);
aggregations.add(aggregation);
}
}
}
}
@Override
public void writeTo(StreamOutput out) throws IOException {
out.writeVInt(aggregations.size());
for (Aggregation aggregation : aggregations) {
InternalAggregation internal = (InternalAggregation) aggregation;
// NORELEASE Temporary hack to support old style streams and new style NamedWriteable at the same time
try {
internal.getWriteableName(); // Throws UnsupportedOperationException if we should use old style streams.
out.writeBoolean(true);
out.writeNamedWriteable(internal);
} catch (UnsupportedOperationException e) {
out.writeBoolean(false);
out.writeBytesReference(internal.type().stream());
internal.writeTo(out);
}
}
out.writeNamedWriteableList(aggregations);
}
}

View File

@ -29,10 +29,6 @@ import java.util.Map;
public abstract class InternalMultiBucketAggregation<A extends InternalMultiBucketAggregation, B extends InternalMultiBucketAggregation.InternalBucket>
extends InternalAggregation implements MultiBucketsAggregation {
public InternalMultiBucketAggregation() {
}
public InternalMultiBucketAggregation(String name, List<PipelineAggregator> pipelineAggregators, Map<String, Object> metaData) {
super(name, pipelineAggregators, metaData);
}

View File

@ -39,8 +39,6 @@ public abstract class InternalSingleBucketAggregation extends InternalAggregatio
private long docCount;
private InternalAggregations aggregations;
protected InternalSingleBucketAggregation() {} // for serialization
/**
* Creates a single bucket aggregation.
*
@ -63,12 +61,6 @@ public abstract class InternalSingleBucketAggregation extends InternalAggregatio
aggregations = InternalAggregations.readAggregations(in);
}
@Override
protected void doReadFrom(StreamInput in) throws IOException {
docCount = in.readVLong();
aggregations = InternalAggregations.readAggregations(in);
}
@Override
protected void doWriteTo(StreamOutput out) throws IOException {
out.writeVLong(docCount);

View File

@ -40,9 +40,7 @@ public interface MultiBucketsAggregation extends Aggregation {
* A bucket represents a criteria to which all documents that fall in it adhere to. It is also uniquely identified
* by a key, and can potentially hold sub-aggregations computed over all documents in it.
*/
public interface Bucket extends HasAggregations, ToXContent, Streamable, Writeable {
// NORELEASE remove Streamable
public interface Bucket extends HasAggregations, ToXContent, Writeable {
/**
* @return The key associated with the bucket
*/
@ -66,12 +64,6 @@ public interface MultiBucketsAggregation extends Aggregation {
Object getProperty(String containingAggName, List<String> path);
@Override
default void readFrom(StreamInput in) throws IOException {
// NORELEASE remove me when no Buckets override it
throw new UnsupportedOperationException("Prefer the Writeable interface");
}
static class SubAggregationComparator<B extends Bucket> implements java.util.Comparator<B> {
private final AggregationPath path;

View File

@ -35,7 +35,7 @@ import java.io.IOException;
public class InternalDateHistogram {
public static final Factory HISTOGRAM_FACTORY = new Factory();
static final Type TYPE = new Type("date_histogram", "dhisto");
static final Type TYPE = new Type("date_histogram");
static class Bucket extends InternalHistogram.Bucket {
Bucket(long key, long docCount, InternalAggregations aggregations, boolean keyed, DocValueFormat formatter,

View File

@ -302,11 +302,6 @@ public class InternalHistogram<B extends InternalHistogram.Bucket> extends Inter
return HistogramAggregationBuilder.NAME;
}
@Override
public Type type() {
return TYPE;
}
@Override
public List<B> getBuckets() {
return buckets;

View File

@ -172,14 +172,8 @@ public class InternalRange<B extends InternalRange.Bucket, R extends InternalRan
return sb.toString();
}
@Override
public void readFrom(StreamInput in) throws IOException {
}
@Override
public void writeTo(StreamOutput out) throws IOException {
}
}

View File

@ -241,11 +241,6 @@ public class GlobalOrdinalsStringTermsAggregator extends AbstractStringTermsAggr
throw new UnsupportedOperationException();
}
@Override
public void readFrom(StreamInput in) throws IOException {
throw new UnsupportedOperationException();
}
@Override
protected void writeTermTo(StreamOutput out) throws IOException {
throw new UnsupportedOperationException();

View File

@ -196,13 +196,9 @@ public abstract class InternalTerms<A extends InternalTerms<A, B>, B extends Int
!terms.getClass().equals(UnmappedTerms.class)) {
// control gets into this loop when the same field name against which the query is executed
// is of different types in different indices.
throw new AggregationExecutionException("Merging/Reducing the aggregations failed " +
"when computing the aggregation [ Name: " +
referenceTerms.getName() + ", Type: " +
referenceTerms.type() + " ]" + " because: " +
"the field you gave in the aggregation query " +
"existed as two different types " +
"in two different indices");
throw new AggregationExecutionException("Merging/Reducing the aggregations failed when computing the aggregation ["
+ referenceTerms.getName() + "] because the field you gave in the aggregation query existed as two different "
+ "types in two different indices");
}
otherDocCount += terms.getSumOfOtherDocCounts();
final long thisAggDocCountError;

View File

@ -28,9 +28,6 @@ import java.util.List;
import java.util.Map;
public abstract class InternalMetricsAggregation extends InternalAggregation {
protected InternalMetricsAggregation() {} // NORELEASE remove when we remove streamable
protected InternalMetricsAggregation(String name, List<PipelineAggregator> pipelineAggregators, Map<String, Object> metaData) {
super(name, pipelineAggregators, metaData);
}

View File

@ -36,9 +36,6 @@ public abstract class InternalNumericMetricsAggregation extends InternalMetricsA
protected DocValueFormat format = DEFAULT_FORMAT;
public abstract static class SingleValue extends InternalNumericMetricsAggregation implements NumericMetricsAggregation.SingleValue {
protected SingleValue() {} // NORELEASE remove when we remove Streamable
protected SingleValue(String name, List<PipelineAggregator> pipelineAggregators, Map<String, Object> metaData) {
super(name, pipelineAggregators, metaData);
}
@ -69,9 +66,6 @@ public abstract class InternalNumericMetricsAggregation extends InternalMetricsA
}
public abstract static class MultiValue extends InternalNumericMetricsAggregation implements NumericMetricsAggregation.MultiValue {
protected MultiValue() {}
protected MultiValue(String name, List<PipelineAggregator> pipelineAggregators, Map<String, Object> metaData) {
super(name, pipelineAggregators, metaData);
}
@ -101,8 +95,6 @@ public abstract class InternalNumericMetricsAggregation extends InternalMetricsA
}
}
private InternalNumericMetricsAggregation() {} // NORELEASE remove when we remove Streamable
private InternalNumericMetricsAggregation(String name, List<PipelineAggregator> pipelineAggregators, Map<String, Object> metaData) {
super(name, pipelineAggregators, metaData);
}

View File

@ -24,7 +24,6 @@ import org.elasticsearch.common.io.stream.StreamOutput;
import org.elasticsearch.common.util.BigArrays;
import org.elasticsearch.common.xcontent.XContentBuilder;
import org.elasticsearch.search.DocValueFormat;
import org.elasticsearch.search.aggregations.AggregationStreams;
import org.elasticsearch.search.aggregations.InternalAggregation;
import org.elasticsearch.search.aggregations.metrics.InternalNumericMetricsAggregation;
import org.elasticsearch.search.aggregations.pipeline.PipelineAggregator;

View File

@ -23,7 +23,6 @@ import org.elasticsearch.common.io.stream.StreamInput;
import org.elasticsearch.common.io.stream.StreamOutput;
import org.elasticsearch.common.xcontent.XContentBuilder;
import org.elasticsearch.search.DocValueFormat;
import org.elasticsearch.search.aggregations.AggregationStreams;
import org.elasticsearch.search.aggregations.InternalAggregation;
import org.elasticsearch.search.aggregations.metrics.InternalNumericMetricsAggregation;
import org.elasticsearch.search.aggregations.metrics.max.InternalMax;
@ -33,26 +32,8 @@ import java.util.List;
import java.util.Map;
public class InternalSimpleValue extends InternalNumericMetricsAggregation.SingleValue implements SimpleValue {
public static final Type TYPE = new Type("simple_value");
public static final AggregationStreams.Stream STREAM = new AggregationStreams.Stream() {
@Override
public InternalSimpleValue readResult(StreamInput in) throws IOException {
InternalSimpleValue result = new InternalSimpleValue();
result.readFrom(in);
return result;
}
};
public static void registerStreams() {
AggregationStreams.registerStream(STREAM, TYPE.stream());
}
private double value;
protected InternalSimpleValue() { // NORELEASE remove and make value final if possible
} // for serialization
public static final String NAME = "simple_value";
private final double value;
public InternalSimpleValue(String name, double value, DocValueFormat formatter, List<PipelineAggregator> pipelineAggregators,
Map<String, Object> metaData) {
@ -70,18 +51,17 @@ public class InternalSimpleValue extends InternalNumericMetricsAggregation.Singl
value = in.readDouble();
}
@Override
protected void doReadFrom(StreamInput in) throws IOException {
format = in.readNamedWriteable(DocValueFormat.class);
value = in.readDouble();
}
@Override
protected void doWriteTo(StreamOutput out) throws IOException {
out.writeNamedWriteable(format);
out.writeDouble(value);
}
@Override
public String getWriteableName() {
return NAME;
}
@Override
public double value() {
return value;
@ -91,11 +71,6 @@ public class InternalSimpleValue extends InternalNumericMetricsAggregation.Singl
return value;
}
@Override
public Type type() {
return TYPE;
}
@Override
public InternalMax doReduce(List<InternalAggregation> aggregations, ReduceContext reduceContext) {
throw new UnsupportedOperationException("Not supported");

View File

@ -24,19 +24,15 @@ import org.elasticsearch.common.ParseField;
import org.elasticsearch.common.io.stream.NamedWriteable;
import org.elasticsearch.common.io.stream.StreamInput;
import org.elasticsearch.common.io.stream.StreamOutput;
import org.elasticsearch.common.io.stream.Streamable;
import org.elasticsearch.index.query.QueryParseContext;
import org.elasticsearch.search.aggregations.InternalAggregation;
import org.elasticsearch.search.aggregations.InternalAggregation.ReduceContext;
import org.elasticsearch.search.aggregations.InternalAggregation.Type;
import org.elasticsearch.search.aggregations.PipelineAggregationBuilder;
import java.io.IOException;
import java.util.Map;
public abstract class PipelineAggregator implements Streamable, NamedWriteable {
// NORELEASE remove Streamable
public abstract class PipelineAggregator implements NamedWriteable {
/**
* Parse the {@link PipelineAggregationBuilder} from a {@link QueryParseContext}.
*/
@ -66,9 +62,6 @@ public abstract class PipelineAggregator implements Streamable, NamedWriteable {
private String[] bucketsPaths;
private Map<String, Object> metaData;
protected PipelineAggregator() { // for Serialisation
}
protected PipelineAggregator(String name, String[] bucketsPaths, Map<String, Object> metaData) {
this.name = name;
this.bucketsPaths = bucketsPaths;
@ -84,27 +77,9 @@ public abstract class PipelineAggregator implements Streamable, NamedWriteable {
metaData = in.readMap();
}
@Override
public final void readFrom(StreamInput in) throws IOException {
try {
getWriteableName(); // Throws UnsupportedOperationException if this aggregation should be read using old style Streams
assert false : "Used reading constructor instead";
} catch (UnsupportedOperationException e) {
// OK
}
name = in.readString();
bucketsPaths = in.readStringArray();
metaData = in.readMap();
doReadFrom(in);
}
protected void doReadFrom(StreamInput in) throws IOException {
throw new UnsupportedOperationException("Use reading constructor instead"); // NORELEASE remove when we remove Streamable
}
@Override
public final void writeTo(StreamOutput out) throws IOException {
out.writeString(name); // NORELEASE remote writing the name - it is automatically handled with writeNamedWriteable
out.writeString(name);
out.writeStringArray(bucketsPaths);
out.writeMap(metaData);
doWriteTo(out);
@ -112,12 +87,6 @@ public abstract class PipelineAggregator implements Streamable, NamedWriteable {
protected abstract void doWriteTo(StreamOutput out) throws IOException;
@Override
public String getWriteableName() {
// NORELEASE remove me when all InternalAggregations override it
throw new UnsupportedOperationException("Override on every class");
}
public String name() {
return name;
}
@ -130,10 +99,5 @@ public abstract class PipelineAggregator implements Streamable, NamedWriteable {
return metaData;
}
public Type type() {
// NORELEASE remove this method
throw new UnsupportedOperationException(getClass().getName() + " used type but should Use getWriteableName instead");
}
public abstract InternalAggregation reduce(InternalAggregation aggregation, ReduceContext reduceContext);
}

View File

@ -1,69 +0,0 @@
/*
* Licensed to Elasticsearch under one or more contributor
* license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright
* ownership. Elasticsearch licenses this file to you under
* the Apache License, Version 2.0 (the "License"); you may
* not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing,
* software distributed under the License is distributed on an
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
* KIND, either express or implied. See the License for the
* specific language governing permissions and limitations
* under the License.
*/
package org.elasticsearch.search.aggregations.pipeline;
import org.elasticsearch.common.bytes.BytesReference;
import org.elasticsearch.common.io.stream.StreamInput;
import java.io.IOException;
import java.util.HashMap;
import java.util.Map;
import static java.util.Collections.emptyMap;
import static java.util.Collections.unmodifiableMap;
/**
* A registry for all the dedicated streams in the aggregation module. This is to support dynamic addAggregation that
* know how to stream themselves.
*/
public class PipelineAggregatorStreams {
private static Map<BytesReference, Stream> streams = emptyMap();
/**
* A stream that knows how to read an aggregation from the input.
*/
public interface Stream {
PipelineAggregator readResult(StreamInput in) throws IOException;
}
/**
* Registers the given stream and associate it with the given types.
*
* @param stream The streams to register
* @param types The types associated with the streams
*/
public static synchronized void registerStream(Stream stream, BytesReference... types) {
Map<BytesReference, Stream> newStreams = new HashMap<>(streams);
for (BytesReference type : types) {
newStreams.put(type, stream);
}
streams = unmodifiableMap(newStreams);
}
/**
* Returns the stream that is registered for the given type
*
* @param type The given type
* @return The associated stream
*/
public static Stream stream(BytesReference type) {
return streams.get(type);
}
}

View File

@ -36,11 +36,6 @@ import java.util.stream.Collectors;
import java.util.stream.StreamSupport;
public abstract class SiblingPipelineAggregator extends PipelineAggregator {
protected SiblingPipelineAggregator() { // for Serialisation
super();
}
protected SiblingPipelineAggregator(String name, String[] bucketsPaths, Map<String, Object> metaData) {
super(name, bucketsPaths, metaData);
}
@ -83,7 +78,7 @@ public abstract class SiblingPipelineAggregator extends PipelineAggregator {
return singleBucketAgg.create(new InternalAggregations(aggs));
} else {
throw new IllegalStateException("Aggregation [" + aggregation.getName() + "] must be a bucket aggregation ["
+ aggregation.type().name() + "]");
+ aggregation.getWriteableName() + "]");
}
}

View File

@ -45,12 +45,8 @@ import java.util.Map;
*/
public abstract class BucketMetricsPipelineAggregator extends SiblingPipelineAggregator {
protected DocValueFormat format;
protected GapPolicy gapPolicy;
public BucketMetricsPipelineAggregator() {
super();
}
protected final DocValueFormat format;
protected final GapPolicy gapPolicy;
protected BucketMetricsPipelineAggregator(String name, String[] bucketsPaths, GapPolicy gapPolicy, DocValueFormat format,
Map<String, Object> metaData) {
@ -68,16 +64,6 @@ public abstract class BucketMetricsPipelineAggregator extends SiblingPipelineAgg
gapPolicy = GapPolicy.readFrom(in);
}
@Override
public final void doReadFrom(StreamInput in) throws IOException {
format = in.readNamedWriteable(DocValueFormat.class);
gapPolicy = GapPolicy.readFrom(in);
innerReadFrom(in);
}
protected void innerReadFrom(StreamInput in) throws IOException {
}
@Override
public final void doWriteTo(StreamOutput out) throws IOException {
out.writeNamedWriteable(format);

View File

@ -23,7 +23,6 @@ import org.elasticsearch.common.io.stream.StreamInput;
import org.elasticsearch.common.io.stream.StreamOutput;
import org.elasticsearch.common.xcontent.XContentBuilder;
import org.elasticsearch.search.DocValueFormat;
import org.elasticsearch.search.aggregations.AggregationStreams;
import org.elasticsearch.search.aggregations.InternalAggregation;
import org.elasticsearch.search.aggregations.metrics.InternalNumericMetricsAggregation;
import org.elasticsearch.search.aggregations.pipeline.PipelineAggregator;
@ -33,30 +32,11 @@ import java.util.List;
import java.util.Map;
public class InternalBucketMetricValue extends InternalNumericMetricsAggregation.SingleValue {
public static final Type TYPE = new Type("bucket_metric_value");
public static final AggregationStreams.Stream STREAM = new AggregationStreams.Stream() {
@Override
public InternalBucketMetricValue readResult(StreamInput in) throws IOException {
InternalBucketMetricValue result = new InternalBucketMetricValue();
result.readFrom(in);
return result;
}
};
public static void registerStreams() {
AggregationStreams.registerStream(STREAM, TYPE.stream());
}
public static final String NAME = "bucket_metric_value";
private double value;
private String[] keys;
protected InternalBucketMetricValue() {
super();
}
public InternalBucketMetricValue(String name, String[] keys, double value, DocValueFormat formatter,
List<PipelineAggregator> pipelineAggregators, Map<String, Object> metaData) {
super(name, pipelineAggregators, metaData);
@ -65,9 +45,26 @@ public class InternalBucketMetricValue extends InternalNumericMetricsAggregation
this.format = formatter;
}
/**
* Read from a stream.
*/
public InternalBucketMetricValue(StreamInput in) throws IOException {
super(in);
format = in.readNamedWriteable(DocValueFormat.class);
value = in.readDouble();
keys = in.readStringArray();
}
@Override
public Type type() {
return TYPE;
protected void doWriteTo(StreamOutput out) throws IOException {
out.writeNamedWriteable(format);
out.writeDouble(value);
out.writeStringArray(keys);
}
@Override
public String getWriteableName() {
return NAME;
}
@Override
@ -97,20 +94,6 @@ public class InternalBucketMetricValue extends InternalNumericMetricsAggregation
}
}
@Override
protected void doReadFrom(StreamInput in) throws IOException {
format = in.readNamedWriteable(DocValueFormat.class);
value = in.readDouble();
keys = in.readStringArray();
}
@Override
protected void doWriteTo(StreamOutput out) throws IOException {
out.writeNamedWriteable(format);
out.writeDouble(value);
out.writeStringArray(keys);
}
@Override
public XContentBuilder doXContentBody(XContentBuilder builder, Params params) throws IOException {
boolean hasValue = !Double.isInfinite(value);

View File

@ -34,11 +34,11 @@ import java.util.List;
import java.util.Map;
public class AvgBucketPipelineAggregationBuilder extends BucketMetricsPipelineAggregationBuilder<AvgBucketPipelineAggregationBuilder> {
public static final String NAME = AvgBucketPipelineAggregator.TYPE.name();
public static final String NAME = "avg_bucket";
public static final ParseField AGGREGATION_NAME_FIELD = new ParseField(NAME);
public AvgBucketPipelineAggregationBuilder(String name, String bucketsPath) {
super(name, AvgBucketPipelineAggregator.TYPE.name(), new String[] { bucketsPath });
super(name, NAME, new String[] { bucketsPath });
}
/**

View File

@ -22,11 +22,9 @@ package org.elasticsearch.search.aggregations.pipeline.bucketmetrics.avg;
import org.elasticsearch.common.io.stream.StreamInput;
import org.elasticsearch.search.DocValueFormat;
import org.elasticsearch.search.aggregations.InternalAggregation;
import org.elasticsearch.search.aggregations.InternalAggregation.Type;
import org.elasticsearch.search.aggregations.pipeline.BucketHelpers.GapPolicy;
import org.elasticsearch.search.aggregations.pipeline.InternalSimpleValue;
import org.elasticsearch.search.aggregations.pipeline.PipelineAggregator;
import org.elasticsearch.search.aggregations.pipeline.PipelineAggregatorStreams;
import org.elasticsearch.search.aggregations.pipeline.bucketmetrics.BucketMetricsPipelineAggregator;
import java.io.IOException;
@ -34,36 +32,24 @@ import java.util.List;
import java.util.Map;
public class AvgBucketPipelineAggregator extends BucketMetricsPipelineAggregator {
public static final Type TYPE = new Type("avg_bucket");
public static final PipelineAggregatorStreams.Stream STREAM = new PipelineAggregatorStreams.Stream() {
@Override
public AvgBucketPipelineAggregator readResult(StreamInput in) throws IOException {
AvgBucketPipelineAggregator result = new AvgBucketPipelineAggregator();
result.readFrom(in);
return result;
}
};
public static void registerStreams() {
PipelineAggregatorStreams.registerStream(STREAM, TYPE.stream());
}
private int count = 0;
private double sum = 0;
private AvgBucketPipelineAggregator() {
}
protected AvgBucketPipelineAggregator(String name, String[] bucketsPaths, GapPolicy gapPolicy, DocValueFormat format,
Map<String, Object> metaData) {
super(name, bucketsPaths, gapPolicy, format, metaData);
}
/**
* Read from a stream.
*/
public AvgBucketPipelineAggregator(StreamInput in) throws IOException {
super(in);
}
@Override
public Type type() {
return TYPE;
public String getWriteableName() {
return AvgBucketPipelineAggregationBuilder.NAME;
}
@Override

View File

@ -34,11 +34,11 @@ import java.util.List;
import java.util.Map;
public class MaxBucketPipelineAggregationBuilder extends BucketMetricsPipelineAggregationBuilder<MaxBucketPipelineAggregationBuilder> {
public static final String NAME = MaxBucketPipelineAggregator.TYPE.name();
public static final String NAME = "max_bucket";
public static final ParseField AGGREGATION_NAME_FIELD = new ParseField(NAME);
public MaxBucketPipelineAggregationBuilder(String name, String bucketsPath) {
super(name, MaxBucketPipelineAggregator.TYPE.name(), new String[] { bucketsPath });
super(name, NAME, new String[] { bucketsPath });
}
/**

View File

@ -22,10 +22,8 @@ package org.elasticsearch.search.aggregations.pipeline.bucketmetrics.max;
import org.elasticsearch.common.io.stream.StreamInput;
import org.elasticsearch.search.DocValueFormat;
import org.elasticsearch.search.aggregations.InternalAggregation;
import org.elasticsearch.search.aggregations.InternalAggregation.Type;
import org.elasticsearch.search.aggregations.pipeline.BucketHelpers.GapPolicy;
import org.elasticsearch.search.aggregations.pipeline.PipelineAggregator;
import org.elasticsearch.search.aggregations.pipeline.PipelineAggregatorStreams;
import org.elasticsearch.search.aggregations.pipeline.bucketmetrics.BucketMetricsPipelineAggregator;
import org.elasticsearch.search.aggregations.pipeline.bucketmetrics.InternalBucketMetricValue;
@ -36,36 +34,24 @@ import java.util.List;
import java.util.Map;
public class MaxBucketPipelineAggregator extends BucketMetricsPipelineAggregator {
public static final Type TYPE = new Type("max_bucket");
public static final PipelineAggregatorStreams.Stream STREAM = new PipelineAggregatorStreams.Stream() {
@Override
public MaxBucketPipelineAggregator readResult(StreamInput in) throws IOException {
MaxBucketPipelineAggregator result = new MaxBucketPipelineAggregator();
result.readFrom(in);
return result;
}
};
public static void registerStreams() {
PipelineAggregatorStreams.registerStream(STREAM, TYPE.stream());
}
private List<String> maxBucketKeys;
private double maxValue;
private MaxBucketPipelineAggregator() {
}
protected MaxBucketPipelineAggregator(String name, String[] bucketsPaths, GapPolicy gapPolicy, DocValueFormat formatter,
Map<String, Object> metaData) {
super(name, bucketsPaths, gapPolicy, formatter, metaData);
}
/**
* Read from a stream.
*/
public MaxBucketPipelineAggregator(StreamInput in) throws IOException {
super(in);
}
@Override
public Type type() {
return TYPE;
public String getWriteableName() {
return MaxBucketPipelineAggregationBuilder.NAME;
}
@Override

View File

@ -34,11 +34,11 @@ import java.util.List;
import java.util.Map;
public class MinBucketPipelineAggregationBuilder extends BucketMetricsPipelineAggregationBuilder<MinBucketPipelineAggregationBuilder> {
public static final String NAME = MinBucketPipelineAggregator.TYPE.name();
public static final String NAME = "min_bucket";
public static final ParseField AGGREGATION_FIELD_NAME = new ParseField(NAME);
public MinBucketPipelineAggregationBuilder(String name, String bucketsPath) {
super(name, MinBucketPipelineAggregator.TYPE.name(), new String[] { bucketsPath });
super(name, NAME, new String[] { bucketsPath });
}
/**

View File

@ -22,10 +22,8 @@ package org.elasticsearch.search.aggregations.pipeline.bucketmetrics.min;
import org.elasticsearch.common.io.stream.StreamInput;
import org.elasticsearch.search.DocValueFormat;
import org.elasticsearch.search.aggregations.InternalAggregation;
import org.elasticsearch.search.aggregations.InternalAggregation.Type;
import org.elasticsearch.search.aggregations.pipeline.BucketHelpers.GapPolicy;
import org.elasticsearch.search.aggregations.pipeline.PipelineAggregator;
import org.elasticsearch.search.aggregations.pipeline.PipelineAggregatorStreams;
import org.elasticsearch.search.aggregations.pipeline.bucketmetrics.BucketMetricsPipelineAggregator;
import org.elasticsearch.search.aggregations.pipeline.bucketmetrics.InternalBucketMetricValue;
@ -36,36 +34,24 @@ import java.util.List;
import java.util.Map;
public class MinBucketPipelineAggregator extends BucketMetricsPipelineAggregator {
public static final Type TYPE = new Type("min_bucket");
public static final PipelineAggregatorStreams.Stream STREAM = new PipelineAggregatorStreams.Stream() {
@Override
public MinBucketPipelineAggregator readResult(StreamInput in) throws IOException {
MinBucketPipelineAggregator result = new MinBucketPipelineAggregator();
result.readFrom(in);
return result;
}
};
public static void registerStreams() {
PipelineAggregatorStreams.registerStream(STREAM, TYPE.stream());
}
private List<String> minBucketKeys;
private double minValue;
private MinBucketPipelineAggregator() {
}
protected MinBucketPipelineAggregator(String name, String[] bucketsPaths, GapPolicy gapPolicy, DocValueFormat formatter,
Map<String, Object> metaData) {
super(name, bucketsPaths, gapPolicy, formatter, metaData);
}
/**
* Read from a stream.
*/
public MinBucketPipelineAggregator(StreamInput in) throws IOException {
super(in);
}
@Override
public Type type() {
return TYPE;
public String getWriteableName() {
return MinBucketPipelineAggregationBuilder.NAME;
}
@Override
@ -86,8 +72,7 @@ public class MinBucketPipelineAggregator extends BucketMetricsPipelineAggregator
}
@Override
protected InternalAggregation buildAggregation(java.util.List<PipelineAggregator> pipelineAggregators,
java.util.Map<String, Object> metadata) {
protected InternalAggregation buildAggregation(List<PipelineAggregator> pipelineAggregators, Map<String, Object> metadata) {
String[] keys = minBucketKeys.toArray(new String[minBucketKeys.size()]);
return new InternalBucketMetricValue(name(), keys, minValue, format, Collections.emptyList(), metaData());
}

View File

@ -23,7 +23,6 @@ import org.elasticsearch.common.io.stream.StreamInput;
import org.elasticsearch.common.io.stream.StreamOutput;
import org.elasticsearch.common.xcontent.XContentBuilder;
import org.elasticsearch.search.DocValueFormat;
import org.elasticsearch.search.aggregations.AggregationStreams;
import org.elasticsearch.search.aggregations.InternalAggregation;
import org.elasticsearch.search.aggregations.metrics.InternalNumericMetricsAggregation;
import org.elasticsearch.search.aggregations.metrics.max.InternalMax;
@ -38,28 +37,9 @@ import java.util.List;
import java.util.Map;
public class InternalPercentilesBucket extends InternalNumericMetricsAggregation.MultiValue implements PercentilesBucket {
public static final Type TYPE = new Type("percentiles_bucket");
public static final AggregationStreams.Stream STREAM = new AggregationStreams.Stream() {
@Override
public InternalPercentilesBucket readResult(StreamInput in) throws IOException {
InternalPercentilesBucket result = new InternalPercentilesBucket();
result.readFrom(in);
return result;
}
};
public static void registerStreams() {
AggregationStreams.registerStream(STREAM, TYPE.stream());
}
private double[] percentiles;
private double[] percents;
protected InternalPercentilesBucket() {
} // for serialization
public InternalPercentilesBucket(String name, double[] percents, double[] percentiles,
DocValueFormat formatter, List<PipelineAggregator> pipelineAggregators,
Map<String, Object> metaData) {
@ -69,6 +49,28 @@ public class InternalPercentilesBucket extends InternalNumericMetricsAggregation
this.percents = percents;
}
/**
* Read from a stream.
*/
public InternalPercentilesBucket(StreamInput in) throws IOException {
super(in);
format = in.readNamedWriteable(DocValueFormat.class);
percentiles = in.readDoubleArray();
percents = in.readDoubleArray();
}
@Override
protected void doWriteTo(StreamOutput out) throws IOException {
out.writeNamedWriteable(format);
out.writeDoubleArray(percentiles);
out.writeDoubleArray(percents);
}
@Override
public String getWriteableName() {
return PercentilesBucketPipelineAggregationBuilder.NAME;
}
@Override
public double percentile(double percent) throws IllegalArgumentException {
int index = Arrays.binarySearch(percents, percent);
@ -94,30 +96,11 @@ public class InternalPercentilesBucket extends InternalNumericMetricsAggregation
return percentile(Double.parseDouble(name));
}
@Override
public Type type() {
return TYPE;
}
@Override
public InternalMax doReduce(List<InternalAggregation> aggregations, ReduceContext reduceContext) {
throw new UnsupportedOperationException("Not supported");
}
@Override
protected void doReadFrom(StreamInput in) throws IOException {
format = in.readNamedWriteable(DocValueFormat.class);
percentiles = in.readDoubleArray();
percents = in.readDoubleArray();
}
@Override
protected void doWriteTo(StreamOutput out) throws IOException {
out.writeNamedWriteable(format);
out.writeDoubleArray(percentiles);
out.writeDoubleArray(percents);
}
@Override
public XContentBuilder doXContentBody(XContentBuilder builder, Params params) throws IOException {
builder.startObject("values");

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