Merge branch 'master' into feature/multi_cluster_search

This commit is contained in:
Simon Willnauer 2017-01-11 09:28:00 +01:00
commit 6d2d878068
280 changed files with 3775 additions and 8514 deletions

View File

@ -430,15 +430,33 @@ cd $BATS_ARCHIVES
sudo -E bats $BATS_TESTS/*.bats
-------------------------------------------------
Note: Starting vagrant VM outside of the elasticsearch folder requires to
indicates the folder that contains the Vagrantfile using the VAGRANT_CWD
environment variable:
You can also use Gradle to prepare the test environment and then starts a single VM:
-------------------------------------------------
gradle vagrantSetUp
VAGRANT_CWD=/path/to/elasticsearch vagrant up centos-7 --provider virtualbox
gradle vagrantFedora24#up
-------------------------------------------------
Or any of vagrantCentos6#up, vagrantDebian8#up, vagrantFedora24#up, vagrantOel6#up,
vagrantOel7#up, vagrantOpensuse13#up, vagrantSles12#up, vagrantUbuntu1204#up,
vagrantUbuntu1604#up.
Once up, you can then connect to the VM using SSH from the elasticsearch directory:
-------------------------------------------------
vagrant ssh fedora-24
-------------------------------------------------
Or from another directory:
-------------------------------------------------
VAGRANT_CWD=/path/to/elasticsearch vagrant ssh fedora-24
-------------------------------------------------
Note: Starting vagrant VM outside of the elasticsearch folder requires to
indicates the folder that contains the Vagrantfile using the VAGRANT_CWD
environment variable.
== Coverage analysis
Tests can be run instrumented with jacoco to produce a coverage report in

View File

@ -39,7 +39,7 @@ public class RestTestPlugin implements Plugin<Project> {
if (false == REQUIRED_PLUGINS.any {project.pluginManager.hasPlugin(it)}) {
throw new InvalidUserDataException('elasticsearch.rest-test '
+ 'requires either elasticsearch.build or '
+ 'elasticsearch.standalone-test')
+ 'elasticsearch.standalone-rest-test')
}
RestIntegTestTask integTest = project.tasks.create('integTest', RestIntegTestTask.class)

View File

@ -40,9 +40,9 @@ public class StandaloneRestTestPlugin implements Plugin<Project> {
@Override
public void apply(Project project) {
if (project.pluginManager.hasPlugin('elasticsearch.build')) {
throw new InvalidUserDataException('elasticsearch.standalone-test, '
+ 'elasticsearch.standalone-test, and elasticsearch.build are '
+ 'mutually exclusive')
throw new InvalidUserDataException('elasticsearch.standalone-test '
+ 'elasticsearch.standalone-rest-test, and elasticsearch.build '
+ 'are mutually exclusive')
}
project.pluginManager.apply(JavaBasePlugin)
project.pluginManager.apply(RandomizedTestingPlugin)

View File

@ -404,10 +404,6 @@ class VagrantTestPlugin implements Plugin<Project> {
args 'halt', box
}
stop.dependsOn(halt)
if (project.extensions.esvagrant.boxes.contains(box) == false) {
// we only need a halt task if this box was not specified
continue;
}
Task update = project.tasks.create("vagrant${boxTask}#update", VagrantCommandTask) {
boxName box
@ -435,6 +431,11 @@ class VagrantTestPlugin implements Plugin<Project> {
dependsOn update
}
if (project.extensions.esvagrant.boxes.contains(box) == false) {
// we d'ont need tests tasks if this box was not specified
continue;
}
Task smoke = project.tasks.create("vagrant${boxTask}#smoketest", Exec) {
environment vagrantEnvVars
dependsOn up

View File

@ -443,7 +443,6 @@
<suppress files="core[/\\]src[/\\]main[/\\]java[/\\]org[/\\]elasticsearch[/\\]repositories[/\\]fs[/\\]FsRepository.java" checks="LineLength" />
<suppress files="core[/\\]src[/\\]main[/\\]java[/\\]org[/\\]elasticsearch[/\\]repositories[/\\]uri[/\\]URLIndexShardRepository.java" checks="LineLength" />
<suppress files="core[/\\]src[/\\]main[/\\]java[/\\]org[/\\]elasticsearch[/\\]repositories[/\\]uri[/\\]URLRepository.java" checks="LineLength" />
<suppress files="core[/\\]src[/\\]main[/\\]java[/\\]org[/\\]elasticsearch[/\\]rest[/\\]BytesRestResponse.java" checks="LineLength" />
<suppress files="core[/\\]src[/\\]main[/\\]java[/\\]org[/\\]elasticsearch[/\\]rest[/\\]RestController.java" checks="LineLength" />
<suppress files="core[/\\]src[/\\]main[/\\]java[/\\]org[/\\]elasticsearch[/\\]rest[/\\]action[/\\]cat[/\\]RestCountAction.java" checks="LineLength" />
<suppress files="core[/\\]src[/\\]main[/\\]java[/\\]org[/\\]elasticsearch[/\\]rest[/\\]action[/\\]cat[/\\]RestIndicesAction.java" checks="LineLength" />
@ -468,7 +467,6 @@
<suppress files="core[/\\]src[/\\]main[/\\]java[/\\]org[/\\]elasticsearch[/\\]search[/\\]aggregations[/\\]bucket[/\\]filters[/\\]InternalFilters.java" checks="LineLength" />
<suppress files="core[/\\]src[/\\]main[/\\]java[/\\]org[/\\]elasticsearch[/\\]search[/\\]aggregations[/\\]bucket[/\\]geogrid[/\\]GeoHashGridAggregator.java" checks="LineLength" />
<suppress files="core[/\\]src[/\\]main[/\\]java[/\\]org[/\\]elasticsearch[/\\]search[/\\]aggregations[/\\]bucket[/\\]global[/\\]GlobalAggregator.java" checks="LineLength" />
<suppress files="core[/\\]src[/\\]main[/\\]java[/\\]org[/\\]elasticsearch[/\\]search[/\\]aggregations[/\\]bucket[/\\]global[/\\]InternalGlobal.java" checks="LineLength" />
<suppress files="core[/\\]src[/\\]main[/\\]java[/\\]org[/\\]elasticsearch[/\\]search[/\\]aggregations[/\\]bucket[/\\]histogram[/\\]HistogramAggregator.java" checks="LineLength" />
<suppress files="core[/\\]src[/\\]main[/\\]java[/\\]org[/\\]elasticsearch[/\\]search[/\\]aggregations[/\\]bucket[/\\]missing[/\\]InternalMissing.java" checks="LineLength" />
<suppress files="core[/\\]src[/\\]main[/\\]java[/\\]org[/\\]elasticsearch[/\\]search[/\\]aggregations[/\\]bucket[/\\]missing[/\\]MissingAggregator.java" checks="LineLength" />

View File

@ -24,9 +24,12 @@ import org.elasticsearch.action.support.WriteRequest.RefreshPolicy;
import org.elasticsearch.action.support.WriteResponse;
import org.elasticsearch.action.support.replication.ReplicationResponse;
import org.elasticsearch.common.Nullable;
import org.elasticsearch.common.ParseField;
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.xcontent.ConstructingObjectParser;
import org.elasticsearch.common.xcontent.ObjectParser;
import org.elasticsearch.common.xcontent.StatusToXContentObject;
import org.elasticsearch.common.xcontent.XContentBuilder;
import org.elasticsearch.index.IndexSettings;
@ -39,11 +42,23 @@ import java.net.URI;
import java.net.URISyntaxException;
import java.util.Locale;
import static org.elasticsearch.common.xcontent.ConstructingObjectParser.constructorArg;
import static org.elasticsearch.common.xcontent.ConstructingObjectParser.optionalConstructorArg;
/**
* A base class for the response of a write operation that involves a single doc
*/
public abstract class DocWriteResponse extends ReplicationResponse implements WriteResponse, StatusToXContentObject {
private static final String _SHARDS = "_shards";
private static final String _INDEX = "_index";
private static final String _TYPE = "_type";
private static final String _ID = "_id";
private static final String _VERSION = "_version";
private static final String _SEQ_NO = "_seq_no";
private static final String RESULT = "result";
private static final String FORCED_REFRESH = "forced_refresh";
/**
* An enum that represents the the results of CRUD operations, primarily used to communicate the type of
* operation that occurred.
@ -253,18 +268,32 @@ public abstract class DocWriteResponse extends ReplicationResponse implements Wr
public XContentBuilder innerToXContent(XContentBuilder builder, Params params) throws IOException {
ReplicationResponse.ShardInfo shardInfo = getShardInfo();
builder.field("_index", shardId.getIndexName())
.field("_type", type)
.field("_id", id)
.field("_version", version)
.field("result", getResult().getLowercase());
builder.field(_INDEX, shardId.getIndexName())
.field(_TYPE, type)
.field(_ID, id)
.field(_VERSION, version)
.field(RESULT, getResult().getLowercase());
if (forcedRefresh) {
builder.field("forced_refresh", true);
builder.field(FORCED_REFRESH, true);
}
shardInfo.toXContent(builder, params);
builder.field(_SHARDS, shardInfo);
if (getSeqNo() >= 0) {
builder.field("_seq_no", getSeqNo());
builder.field(_SEQ_NO, getSeqNo());
}
return builder;
}
/**
* Declare the {@link ObjectParser} fields to use when parsing a {@link DocWriteResponse}
*/
protected static void declareParserFields(ConstructingObjectParser<? extends DocWriteResponse, Void> objParser) {
objParser.declareString(constructorArg(), new ParseField(_INDEX));
objParser.declareString(constructorArg(), new ParseField(_TYPE));
objParser.declareString(constructorArg(), new ParseField(_ID));
objParser.declareLong(constructorArg(), new ParseField(_VERSION));
objParser.declareString(constructorArg(), new ParseField(RESULT));
objParser.declareLong(optionalConstructorArg(), new ParseField(_SEQ_NO));
objParser.declareBoolean(DocWriteResponse::setForcedRefresh, new ParseField(FORCED_REFRESH));
objParser.declareObject(DocWriteResponse::setShardInfo, (p, c) -> ShardInfo.fromXContent(p), new ParseField(_SHARDS));
}
}

View File

@ -20,13 +20,21 @@
package org.elasticsearch.action.index;
import org.elasticsearch.action.DocWriteResponse;
import org.elasticsearch.cluster.metadata.IndexMetaData;
import org.elasticsearch.common.ParseField;
import org.elasticsearch.common.Strings;
import org.elasticsearch.common.xcontent.ConstructingObjectParser;
import org.elasticsearch.common.xcontent.XContentBuilder;
import org.elasticsearch.common.xcontent.XContentParser;
import org.elasticsearch.index.Index;
import org.elasticsearch.index.seqno.SequenceNumbersService;
import org.elasticsearch.index.shard.ShardId;
import org.elasticsearch.rest.RestStatus;
import java.io.IOException;
import static org.elasticsearch.common.xcontent.ConstructingObjectParser.constructorArg;
/**
* A response of an index operation,
*
@ -35,6 +43,8 @@ import java.io.IOException;
*/
public class IndexResponse extends DocWriteResponse {
private static final String CREATED = "created";
public IndexResponse() {
}
@ -64,7 +74,34 @@ public class IndexResponse extends DocWriteResponse {
@Override
public XContentBuilder innerToXContent(XContentBuilder builder, Params params) throws IOException {
super.innerToXContent(builder, params);
builder.field("created", result == Result.CREATED);
builder.field(CREATED, result == Result.CREATED);
return builder;
}
/**
* ConstructingObjectParser used to parse the {@link IndexResponse}. We use a ObjectParser here
* because most fields are parsed by the parent abstract class {@link DocWriteResponse} and it's
* not easy to parse part of the fields in the parent class and other fields in the children class
* using the usual streamed parsing method.
*/
private static final ConstructingObjectParser<IndexResponse, Void> PARSER;
static {
PARSER = new ConstructingObjectParser<>(IndexResponse.class.getName(),
args -> {
// index uuid and shard id are unknown and can't be parsed back for now.
ShardId shardId = new ShardId(new Index((String) args[0], IndexMetaData.INDEX_UUID_NA_VALUE), -1);
String type = (String) args[1];
String id = (String) args[2];
long version = (long) args[3];
long seqNo = (args[5] != null) ? (long) args[5] : SequenceNumbersService.UNASSIGNED_SEQ_NO;
boolean created = (boolean) args[6];
return new IndexResponse(shardId, type, id, seqNo, version, created);
});
DocWriteResponse.declareParserFields(PARSER);
PARSER.declareBoolean(constructorArg(), new ParseField(CREATED));
}
public static IndexResponse fromXContent(XContentParser parser) throws IOException {
return PARSER.apply(parser, null);
}
}

View File

@ -73,7 +73,7 @@ public class ReplicationResponse extends ActionResponse {
this.shardInfo = shardInfo;
}
public static class ShardInfo implements Streamable, ToXContent {
public static class ShardInfo implements Streamable, ToXContentObject {
private static final String _SHARDS = "_shards";
private static final String TOTAL = "total";
@ -179,7 +179,7 @@ public class ReplicationResponse extends ActionResponse {
@Override
public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException {
builder.startObject(_SHARDS);
builder.startObject();
builder.field(TOTAL, total);
builder.field(SUCCESSFUL, successful);
builder.field(FAILED, getFailed());
@ -195,18 +195,12 @@ public class ReplicationResponse extends ActionResponse {
}
public static ShardInfo fromXContent(XContentParser parser) throws IOException {
XContentParser.Token token = parser.nextToken();
ensureExpectedToken(XContentParser.Token.FIELD_NAME, token, parser::getTokenLocation);
String currentFieldName = parser.currentName();
if (_SHARDS.equals(currentFieldName) == false) {
throwUnknownField(currentFieldName, parser.getTokenLocation());
}
token = parser.nextToken();
XContentParser.Token token = parser.currentToken();
ensureExpectedToken(XContentParser.Token.START_OBJECT, token, parser::getTokenLocation);
int total = 0, successful = 0;
List<Failure> failuresList = null;
String currentFieldName = null;
while ((token = parser.nextToken()) != XContentParser.Token.END_OBJECT) {
if (token == XContentParser.Token.FIELD_NAME) {
currentFieldName = parser.currentName();

View File

@ -28,12 +28,14 @@ import org.elasticsearch.action.GenericAction;
import org.elasticsearch.action.support.TransportAction;
import org.elasticsearch.client.Client;
import org.elasticsearch.client.support.AbstractClient;
import org.elasticsearch.cluster.node.DiscoveryNode;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.tasks.Task;
import org.elasticsearch.tasks.TaskListener;
import org.elasticsearch.threadpool.ThreadPool;
import java.util.Map;
import java.util.function.Supplier;
/**
* Client that executes actions on the local node.
@ -41,13 +43,19 @@ import java.util.Map;
public class NodeClient extends AbstractClient {
private Map<GenericAction, TransportAction> actions;
/**
* The id of the local {@link DiscoveryNode}. Useful for generating task ids from tasks returned by
* {@link #executeLocally(GenericAction, ActionRequest, TaskListener)}.
*/
private Supplier<String> localNodeId;
public NodeClient(Settings settings, ThreadPool threadPool) {
super(settings, threadPool);
}
public void initialize(Map<GenericAction, TransportAction> actions) {
public void initialize(Map<GenericAction, TransportAction> actions, Supplier<String> localNodeId) {
this.actions = actions;
this.localNodeId = localNodeId;
}
@Override
@ -85,6 +93,14 @@ public class NodeClient extends AbstractClient {
return transportAction(action).execute(request, listener);
}
/**
* The id of the local {@link DiscoveryNode}. Useful for generating task ids from tasks returned by
* {@link #executeLocally(GenericAction, ActionRequest, TaskListener)}.
*/
public String getLocalNodeId() {
return localNodeId.get();
}
/**
* Get the {@link TransportAction} for an {@link Action}, throwing exceptions if the action isn't available.
*/

View File

@ -36,7 +36,9 @@ import org.elasticsearch.discovery.zen.NodesFaultDetection;
import org.elasticsearch.threadpool.ThreadPool;
import org.elasticsearch.transport.TransportService;
import java.util.HashSet;
import java.util.List;
import java.util.Set;
import java.util.concurrent.ConcurrentMap;
import java.util.concurrent.ScheduledFuture;
@ -76,20 +78,26 @@ public class NodeConnectionsService extends AbstractLifecycleComponent {
this.reconnectInterval = NodeConnectionsService.CLUSTER_NODE_RECONNECT_INTERVAL_SETTING.get(settings);
}
public void connectToNodes(List<DiscoveryNode> addedNodes) {
public void connectToNodes(Iterable<DiscoveryNode> discoveryNodes) {
// TODO: do this in parallel (and wait)
for (final DiscoveryNode node : addedNodes) {
for (final DiscoveryNode node : discoveryNodes) {
try (Releasable ignored = nodeLocks.acquire(node)) {
Integer current = nodes.put(node, 0);
assert current == null : "node " + node + " was added in event but already in internal nodes";
nodes.putIfAbsent(node, 0);
validateNodeConnected(node);
}
}
}
public void disconnectFromNodes(List<DiscoveryNode> removedNodes) {
for (final DiscoveryNode node : removedNodes) {
/**
* Disconnects from all nodes except the ones provided as parameter
*/
public void disconnectFromNodesExcept(Iterable<DiscoveryNode> nodesToKeep) {
Set<DiscoveryNode> currentNodes = new HashSet<>(nodes.keySet());
for (DiscoveryNode node : nodesToKeep) {
currentNodes.remove(node);
}
for (final DiscoveryNode node : currentNodes) {
try (Releasable ignored = nodeLocks.acquire(node)) {
Integer current = nodes.remove(node);
assert current != null : "node " + node + " was removed in event but not in internal nodes";

View File

@ -1,31 +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.cluster.routing.allocation.command;
import org.elasticsearch.common.xcontent.ParseFieldRegistry;
/**
* Registry of allocation commands. This is it's own class just to make Guice happy.
*/
public class AllocationCommandRegistry extends ParseFieldRegistry<AllocationCommand.Parser<?>> {
public AllocationCommandRegistry() {
super("allocation_command");
}
}

View File

@ -125,11 +125,10 @@ public class AllocationCommands extends ToXContentToBytes {
* }
* </pre>
* @param parser {@link XContentParser} to read the commands from
* @param registry of allocation command parsers
* @return {@link AllocationCommands} read
* @throws IOException if something bad happens while reading the stream
*/
public static AllocationCommands fromXContent(XContentParser parser, AllocationCommandRegistry registry) throws IOException {
public static AllocationCommands fromXContent(XContentParser parser) throws IOException {
AllocationCommands commands = new AllocationCommands();
XContentParser.Token token = parser.currentToken();
@ -158,7 +157,7 @@ public class AllocationCommands extends ToXContentToBytes {
token = parser.nextToken();
String commandName = parser.currentName();
token = parser.nextToken();
commands.add(registry.lookup(commandName, parser.getTokenLocation()).fromXContent(parser));
commands.add(parser.namedObject(AllocationCommand.class, commandName, null));
// move to the end object one
if (parser.nextToken() != XContentParser.Token.END_OBJECT) {
throw new ElasticsearchParseException("allocation command is malformed, done parsing a command, but didn't get END_OBJECT, got [{}] instead", token);

View File

@ -772,7 +772,7 @@ public class ClusterService extends AbstractLifecycleComponent {
taskOutputs.createAckListener(threadPool, newClusterState) :
null;
nodeConnectionsService.connectToNodes(clusterChangedEvent.nodesDelta().addedNodes());
nodeConnectionsService.connectToNodes(newClusterState.nodes());
// if we are the master, publish the new state to all nodes
// we publish here before we send a notification to all the listeners, since if it fails
@ -788,7 +788,8 @@ public class ClusterService extends AbstractLifecycleComponent {
"failing [{}]: failed to commit cluster state version [{}]", taskInputs.summary, version),
t);
// ensure that list of connected nodes in NodeConnectionsService is in-sync with the nodes of the current cluster state
nodeConnectionsService.disconnectFromNodes(clusterChangedEvent.nodesDelta().addedNodes());
nodeConnectionsService.connectToNodes(previousClusterState.nodes());
nodeConnectionsService.disconnectFromNodesExcept(previousClusterState.nodes());
taskOutputs.publishingFailed(t);
return;
}
@ -808,7 +809,7 @@ public class ClusterService extends AbstractLifecycleComponent {
logger.debug("set local cluster state to version {}", newClusterState.version());
callClusterStateAppliers(newClusterState, clusterChangedEvent);
nodeConnectionsService.disconnectFromNodes(clusterChangedEvent.nodesDelta().removedNodes());
nodeConnectionsService.disconnectFromNodesExcept(newClusterState.nodes());
updateState(css -> newClusterState);

View File

@ -28,6 +28,7 @@ import org.elasticsearch.common.xcontent.XContentFactory;
import org.elasticsearch.common.xcontent.XContentType;
import java.io.IOException;
import java.util.Objects;
public class CompressorFactory {
@ -68,9 +69,10 @@ public class CompressorFactory {
/**
* Uncompress the provided data, data can be detected as compressed using {@link #isCompressed(BytesReference)}.
* @throws NullPointerException a NullPointerException will be thrown when bytes is null
*/
public static BytesReference uncompressIfNeeded(BytesReference bytes) throws IOException {
Compressor compressor = compressor(bytes);
Compressor compressor = compressor(Objects.requireNonNull(bytes, "the BytesReference must not be null"));
BytesReference uncompressed;
if (compressor != null) {
uncompressed = uncompress(bytes, compressor);

View File

@ -38,8 +38,8 @@ public final class ESLoggerFactory {
public static final Setting<Level> LOG_DEFAULT_LEVEL_SETTING =
new Setting<>("logger.level", Level.INFO.name(), Level::valueOf, Property.NodeScope);
public static final Setting<Level> LOG_LEVEL_SETTING =
Setting.prefixKeySetting("logger.", Level.INFO.name(), Level::valueOf,
Property.Dynamic, Property.NodeScope);
Setting.prefixKeySetting("logger.", (key) -> new Setting<>(key, Level.INFO.name(), Level::valueOf, Property.Dynamic,
Property.NodeScope));
public static Logger getLogger(String prefix, String name) {
return getLogger(prefix, LogManager.getLogger(name));

View File

@ -24,18 +24,17 @@ import org.elasticsearch.cluster.routing.allocation.command.AllocateEmptyPrimary
import org.elasticsearch.cluster.routing.allocation.command.AllocateReplicaAllocationCommand;
import org.elasticsearch.cluster.routing.allocation.command.AllocateStalePrimaryAllocationCommand;
import org.elasticsearch.cluster.routing.allocation.command.AllocationCommand;
import org.elasticsearch.cluster.routing.allocation.command.AllocationCommandRegistry;
import org.elasticsearch.cluster.routing.allocation.command.CancelAllocationCommand;
import org.elasticsearch.cluster.routing.allocation.command.MoveAllocationCommand;
import org.elasticsearch.common.ParseField;
import org.elasticsearch.common.io.stream.NamedWriteableRegistry;
import org.elasticsearch.common.io.stream.NamedWriteableRegistry.Entry;
import org.elasticsearch.common.io.stream.Writeable;
import org.elasticsearch.common.settings.Setting;
import org.elasticsearch.common.settings.Setting.Property;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.util.BigArrays;
import org.elasticsearch.common.xcontent.NamedXContentRegistry;
import org.elasticsearch.common.xcontent.NamedXContentRegistry.FromXContent;
import org.elasticsearch.http.HttpServerTransport;
import org.elasticsearch.indices.breaker.CircuitBreakerService;
import org.elasticsearch.plugins.NetworkPlugin;
@ -76,12 +75,8 @@ public final class NetworkModule {
private final Settings settings;
private final boolean transportClient;
private static final AllocationCommandRegistry allocationCommandRegistry = new AllocationCommandRegistry();
private static final List<NamedWriteableRegistry.Entry> namedWriteables = new ArrayList<>();
private final Map<String, Supplier<Transport>> transportFactories = new HashMap<>();
private final Map<String, Supplier<HttpServerTransport>> transportHttpFactories = new HashMap<>();
private final List<TransportInterceptor> transportIntercetors = new ArrayList<>();
private static final List<NamedXContentRegistry.Entry> namedXContents = new ArrayList<>();
static {
registerAllocationCommand(CancelAllocationCommand::new, CancelAllocationCommand::fromXContent,
@ -99,6 +94,11 @@ public final class NetworkModule {
namedWriteables.add(
new NamedWriteableRegistry.Entry(Task.Status.class, RawTaskStatus.NAME, RawTaskStatus::new));
}
private final Map<String, Supplier<Transport>> transportFactories = new HashMap<>();
private final Map<String, Supplier<HttpServerTransport>> transportHttpFactories = new HashMap<>();
private final List<TransportInterceptor> transportIntercetors = new ArrayList<>();
/**
* Creates a network module that custom networking classes can be plugged into.
* @param settings The settings for the node
@ -165,22 +165,19 @@ public final class NetworkModule {
* it is the name under which the command's reader is registered.
*/
private static <T extends AllocationCommand> void registerAllocationCommand(Writeable.Reader<T> reader,
AllocationCommand.Parser<T> parser, ParseField commandName) {
allocationCommandRegistry.register(parser, commandName);
namedWriteables.add(new Entry(AllocationCommand.class, commandName.getPreferredName(), reader));
FromXContent<T> parser, ParseField commandName) {
namedXContents.add(new NamedXContentRegistry.Entry(AllocationCommand.class, commandName, parser));
namedWriteables.add(new NamedWriteableRegistry.Entry(AllocationCommand.class, commandName.getPreferredName(), reader));
}
/**
* The registry of allocation command parsers.
*/
public static AllocationCommandRegistry getAllocationCommandRegistry() {
return allocationCommandRegistry;
}
public static List<Entry> getNamedWriteables() {
public static List<NamedWriteableRegistry.Entry> getNamedWriteables() {
return Collections.unmodifiableList(namedWriteables);
}
public static List<NamedXContentRegistry.Entry> getNamedXContents() {
return Collections.unmodifiableList(namedXContents);
}
public Supplier<HttpServerTransport> getHttpServerTransportSupplier() {
final String name;
if (HTTP_TYPE_SETTING.exists(settings)) {

View File

@ -195,6 +195,19 @@ public abstract class AbstractScopedSettings extends AbstractComponent {
addSettingsUpdater(setting.newUpdater(consumer, logger, validator));
}
/**
* Adds a settings consumer for affix settings. Affix settings have a namespace associated to it that needs to be available to the
* consumer in order to be processed correctly.
*/
public synchronized <T> void addAffixUpdateConsumer(Setting.AffixSetting<T> setting, BiConsumer<String, T> consumer,
BiConsumer<String, T> validator) {
final Setting<?> registeredSetting = this.complexMatchers.get(setting.getKey());
if (setting != registeredSetting) {
throw new IllegalArgumentException("Setting is not registered for key [" + setting.getKey() + "]");
}
addSettingsUpdater(setting.newAffixUpdater(consumer, logger, validator));
}
synchronized void addSettingsUpdater(SettingUpdater<?> updater) {
this.settingUpdaters.add(updater);
}

View File

@ -42,14 +42,17 @@ import java.io.IOException;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.EnumSet;
import java.util.IdentityHashMap;
import java.util.List;
import java.util.Map;
import java.util.Objects;
import java.util.function.BiConsumer;
import java.util.function.Consumer;
import java.util.function.Function;
import java.util.regex.Matcher;
import java.util.regex.Pattern;
import java.util.stream.Collectors;
import java.util.stream.Stream;
/**
* A setting. Encapsulates typical stuff like default value, parsing, and scope.
@ -410,8 +413,8 @@ public class Setting<T> extends ToXContentToBytes {
}
/**
* Updates settings that depend on eachother. See {@link AbstractScopedSettings#addSettingsUpdateConsumer(Setting, Setting, BiConsumer)}
* and its usage for details.
* Updates settings that depend on each other.
* See {@link AbstractScopedSettings#addSettingsUpdateConsumer(Setting, Setting, BiConsumer)} and its usage for details.
*/
static <A, B> AbstractScopedSettings.SettingUpdater<Tuple<A, B>> compoundUpdater(final BiConsumer<A, B> consumer,
final Setting<A> aSetting, final Setting<B> bSetting, Logger logger) {
@ -446,6 +449,76 @@ public class Setting<T> extends ToXContentToBytes {
};
}
public static class AffixSetting<T> extends Setting<T> {
private final AffixKey key;
private final Function<String, Setting<T>> delegateFactory;
public AffixSetting(AffixKey key, Setting<T> delegate, Function<String, Setting<T>> delegateFactory) {
super(key, delegate.defaultValue, delegate.parser, delegate.properties.toArray(new Property[0]));
this.key = key;
this.delegateFactory = delegateFactory;
}
boolean isGroupSetting() {
return true;
}
private Stream<String> matchStream(Settings settings) {
return settings.getAsMap().keySet().stream().filter((key) -> match(key)).map(settingKey -> key.getConcreteString(settingKey));
}
AbstractScopedSettings.SettingUpdater<Map<AbstractScopedSettings.SettingUpdater<T>, T>> newAffixUpdater(
BiConsumer<String, T> consumer, Logger logger, BiConsumer<String, T> validator) {
return new AbstractScopedSettings.SettingUpdater<Map<AbstractScopedSettings.SettingUpdater<T>, T>>() {
@Override
public boolean hasChanged(Settings current, Settings previous) {
return Stream.concat(matchStream(current), matchStream(previous)).findAny().isPresent();
}
@Override
public Map<AbstractScopedSettings.SettingUpdater<T>, T> getValue(Settings current, Settings previous) {
// we collect all concrete keys and then delegate to the actual setting for validation and settings extraction
final Map<AbstractScopedSettings.SettingUpdater<T>, T> result = new IdentityHashMap<>();
Stream.concat(matchStream(current), matchStream(previous)).forEach(aKey -> {
String namespace = key.getNamespace(aKey);
AbstractScopedSettings.SettingUpdater<T> updater =
getConcreteSetting(aKey).newUpdater((v) -> consumer.accept(namespace, v), logger,
(v) -> validator.accept(namespace, v));
if (updater.hasChanged(current, previous)) {
// only the ones that have changed otherwise we might get too many updates
// the hasChanged above checks only if there are any changes
T value = updater.getValue(current, previous);
result.put(updater, value);
}
});
return result;
}
@Override
public void apply(Map<AbstractScopedSettings.SettingUpdater<T>, T> value, Settings current, Settings previous) {
for (Map.Entry<AbstractScopedSettings.SettingUpdater<T>, T> entry : value.entrySet()) {
entry.getKey().apply(entry.getValue(), current, previous);
}
}
};
}
@Override
public Setting<T> getConcreteSetting(String key) {
if (match(key)) {
return delegateFactory.apply(key);
} else {
throw new IllegalArgumentException("key [" + key + "] must match [" + getKey() + "] but didn't.");
}
}
@Override
public void diff(Settings.Builder builder, Settings source, Settings defaultSettings) {
matchStream(defaultSettings).forEach((key) -> getConcreteSetting(key).diff(builder, source, defaultSettings));
}
}
private final class Updater implements AbstractScopedSettings.SettingUpdater<T> {
private final Consumer<T> consumer;
@ -727,7 +800,6 @@ public class Setting<T> extends ToXContentToBytes {
}
}
private static String arrayToParsableString(String[] array) {
try {
XContentBuilder builder = XContentBuilder.builder(XContentType.JSON.xContent());
@ -741,9 +813,11 @@ public class Setting<T> extends ToXContentToBytes {
throw new ElasticsearchException(ex);
}
}
public static Setting<Settings> groupSetting(String key, Property... properties) {
return groupSetting(key, (s) -> {}, properties);
}
public static Setting<Settings> groupSetting(String key, Consumer<Settings> validator, Property... properties) {
return new Setting<Settings>(new GroupKey(key), (s) -> "", (s) -> null, properties) {
@Override
@ -894,59 +968,24 @@ public class Setting<T> extends ToXContentToBytes {
* can easily be added with this setting. Yet, prefix key settings don't support updaters out of the box unless
* {@link #getConcreteSetting(String)} is used to pull the updater.
*/
public static <T> Setting<T> prefixKeySetting(String prefix, String defaultValue, Function<String, T> parser,
Property... properties) {
return affixKeySetting(AffixKey.withPrefix(prefix), (s) -> defaultValue, parser, properties);
public static <T> AffixSetting<T> prefixKeySetting(String prefix, Function<String, Setting<T>> delegateFactory) {
return affixKeySetting(new AffixKey(prefix), delegateFactory);
}
/**
* This setting type allows to validate settings that have the same type and a common prefix and suffix. For instance
* storage.${backend}.enable=[true|false] can easily be added with this setting. Yet, adfix key settings don't support updaters
* storage.${backend}.enable=[true|false] can easily be added with this setting. Yet, affix key settings don't support updaters
* out of the box unless {@link #getConcreteSetting(String)} is used to pull the updater.
*/
public static <T> Setting<T> affixKeySetting(String prefix, String suffix, Function<Settings, String> defaultValue,
Function<String, T> parser, Property... properties) {
return affixKeySetting(AffixKey.withAffix(prefix, suffix), defaultValue, parser, properties);
public static <T> AffixSetting<T> affixKeySetting(String prefix, String suffix, Function<String, Setting<T>> delegateFactory) {
return affixKeySetting(new AffixKey(prefix, suffix), delegateFactory);
}
public static <T> Setting<T> affixKeySetting(String prefix, String suffix, String defaultValue, Function<String, T> parser,
Property... properties) {
return affixKeySetting(prefix, suffix, (s) -> defaultValue, parser, properties);
}
private static <T> AffixSetting<T> affixKeySetting(AffixKey key, Function<String, Setting<T>> delegateFactory) {
Setting<T> delegate = delegateFactory.apply("_na_");
return new AffixSetting<>(key, delegate, delegateFactory);
};
public static <T> Setting<T> affixKeySetting(AffixKey key, Function<Settings, String> defaultValue, Function<String, T> parser,
Property... properties) {
return new Setting<T>(key, defaultValue, parser, properties) {
@Override
boolean isGroupSetting() {
return true;
}
@Override
AbstractScopedSettings.SettingUpdater<T> newUpdater(Consumer<T> consumer, Logger logger, Consumer<T> validator) {
throw new UnsupportedOperationException("Affix settings can't be updated. Use #getConcreteSetting for updating.");
}
@Override
public Setting<T> getConcreteSetting(String key) {
if (match(key)) {
return new Setting<>(key, defaultValue, parser, properties);
} else {
throw new IllegalArgumentException("key [" + key + "] must match [" + getKey() + "] but didn't.");
}
}
@Override
public void diff(Settings.Builder builder, Settings source, Settings defaultSettings) {
for (Map.Entry<String, String> entry : defaultSettings.getAsMap().entrySet()) {
if (match(entry.getKey())) {
getConcreteSetting(entry.getKey()).diff(builder, source, defaultSettings);
}
}
}
};
}
public interface Key {
@ -1012,37 +1051,60 @@ public class Setting<T> extends ToXContentToBytes {
}
}
/**
* A key that allows for static pre and suffix. This is used for settings
* that have dynamic namespaces like for different accounts etc.
*/
public static final class AffixKey implements Key {
public static AffixKey withPrefix(String prefix) {
return new AffixKey(prefix, null);
}
public static AffixKey withAffix(String prefix, String suffix) {
return new AffixKey(prefix, suffix);
}
private final Pattern pattern;
private final String prefix;
private final String suffix;
public AffixKey(String prefix, String suffix) {
AffixKey(String prefix) {
this(prefix, null);
}
AffixKey(String prefix, String suffix) {
assert prefix != null || suffix != null: "Either prefix or suffix must be non-null";
this.prefix = prefix;
if (prefix.endsWith(".") == false) {
throw new IllegalArgumentException("prefix must end with a '.'");
}
this.suffix = suffix;
if (suffix == null) {
pattern = Pattern.compile("(" + Pattern.quote(prefix) + "((?:[-\\w]+[.])*[-\\w]+$))");
} else {
// the last part of this regexp is for lists since they are represented as x.${namespace}.y.1, x.${namespace}.y.2
pattern = Pattern.compile("(" + Pattern.quote(prefix) + "([-\\w]+)\\." + Pattern.quote(suffix) + ")(?:\\.\\d+)?");
}
}
@Override
public boolean match(String key) {
boolean match = true;
if (prefix != null) {
match = key.startsWith(prefix);
return pattern.matcher(key).matches();
}
/**
* Returns a string representation of the concrete setting key
*/
String getConcreteString(String key) {
Matcher matcher = pattern.matcher(key);
if (matcher.matches() == false) {
throw new IllegalStateException("can't get concrete string for key " + key + " key doesn't match");
}
if (suffix != null) {
match = match && key.endsWith(suffix);
return matcher.group(1);
}
/**
* Returns a string representation of the concrete setting key
*/
String getNamespace(String key) {
Matcher matcher = pattern.matcher(key);
if (matcher.matches() == false) {
throw new IllegalStateException("can't get concrete string for key " + key + " key doesn't match");
}
return match;
return matcher.group(2);
}
public SimpleKey toConcreteKey(String missingPart) {

View File

@ -20,6 +20,7 @@
package org.elasticsearch.common.xcontent;
import org.elasticsearch.common.ParsingException;
import org.elasticsearch.common.bytes.BytesArray;
import org.elasticsearch.common.xcontent.XContentParser.Token;
import java.io.IOException;
@ -56,6 +57,14 @@ public final class XContentParserUtils {
throw new ParsingException(location, String.format(Locale.ROOT, message, field));
}
/**
* @throws ParsingException with a "unknown token found" reason
*/
public static void throwUnknownToken(XContentParser.Token token, XContentLocation location) {
String message = "Failed to parse object: unexpected token [%s] found";
throw new ParsingException(location, String.format(Locale.ROOT, message, token));
}
/**
* Makes sure that provided token is of the expected type
*
@ -67,4 +76,35 @@ public final class XContentParserUtils {
throw new ParsingException(location.get(), String.format(Locale.ROOT, message, expected, actual));
}
}
/**
* Parse the current token depending on its token type. The following token types will be
* parsed by the corresponding parser methods:
* <ul>
* <li>XContentParser.Token.VALUE_STRING: parser.text()</li>
* <li>XContentParser.Token.VALUE_NUMBER: parser.numberValue()</li>
* <li>XContentParser.Token.VALUE_BOOLEAN: parser.booleanValue()</li>
* <li>XContentParser.Token.VALUE_EMBEDDED_OBJECT: parser.binaryValue()</li>
* </ul>
*
* @throws ParsingException if the token none of the allowed values
*/
public static Object parseStoredFieldsValue(XContentParser parser) throws IOException {
XContentParser.Token token = parser.currentToken();
Object value = null;
if (token == XContentParser.Token.VALUE_STRING) {
//binary values will be parsed back and returned as base64 strings when reading from json and yaml
value = parser.text();
} else if (token == XContentParser.Token.VALUE_NUMBER) {
value = parser.numberValue();
} else if (token == XContentParser.Token.VALUE_BOOLEAN) {
value = parser.booleanValue();
} else if (token == XContentParser.Token.VALUE_EMBEDDED_OBJECT) {
//binary values will be parsed back and returned as BytesArray when reading from cbor and smile
value = new BytesArray(parser.binaryValue());
} else {
throwUnknownToken(token, parser.getTokenLocation());
}
return value;
}
}

View File

@ -19,8 +19,6 @@
package org.elasticsearch.index.get;
import org.elasticsearch.common.ParsingException;
import org.elasticsearch.common.bytes.BytesArray;
import org.elasticsearch.common.io.stream.StreamInput;
import org.elasticsearch.common.io.stream.StreamOutput;
import org.elasticsearch.common.io.stream.Streamable;
@ -36,6 +34,7 @@ import java.util.List;
import java.util.Objects;
import static org.elasticsearch.common.xcontent.XContentParserUtils.ensureExpectedToken;
import static org.elasticsearch.common.xcontent.XContentParserUtils.parseStoredFieldsValue;
public class GetField implements Streamable, ToXContent, Iterable<Object> {
@ -119,21 +118,7 @@ public class GetField implements Streamable, ToXContent, Iterable<Object> {
ensureExpectedToken(XContentParser.Token.START_ARRAY, token, parser::getTokenLocation);
List<Object> values = new ArrayList<>();
while((token = parser.nextToken()) != XContentParser.Token.END_ARRAY) {
Object value;
if (token == XContentParser.Token.VALUE_STRING) {
//binary values will be parsed back and returned as base64 strings when reading from json and yaml
value = parser.text();
} else if (token == XContentParser.Token.VALUE_NUMBER) {
value = parser.numberValue();
} else if (token == XContentParser.Token.VALUE_BOOLEAN) {
value = parser.booleanValue();
} else if (token == XContentParser.Token.VALUE_EMBEDDED_OBJECT) {
//binary values will be parsed back and returned as BytesArray when reading from cbor and smile
value = new BytesArray(parser.binaryValue());
} else {
throw new ParsingException(parser.getTokenLocation(), "Failed to parse object: unsupported token found [" + token + "]");
}
values.add(value);
values.add(parseStoredFieldsValue(parser));
}
return new GetField(fieldName, values);
}

View File

@ -54,7 +54,7 @@ public class GetResult implements Streamable, Iterable<GetField>, ToXContentObje
private static final String _VERSION = "_version";
private static final String FOUND = "found";
private static final String FIELDS = "fields";
private String index;
private String type;
private String id;
@ -135,6 +135,10 @@ public class GetResult implements Streamable, Iterable<GetField>, ToXContentObje
* Returns bytes reference, also un compress the source if needed.
*/
public BytesReference sourceRef() {
if (source == null) {
return null;
}
try {
this.source = CompressorFactory.uncompressIfNeeded(this.source);
return this.source;

View File

@ -473,7 +473,10 @@ public class HasChildQueryBuilder extends AbstractQueryBuilder<HasChildQueryBuil
QueryBuilder rewrittenQuery = query.rewrite(queryRewriteContext);
if (rewrittenQuery != query) {
InnerHitBuilder rewrittenInnerHit = InnerHitBuilder.rewrite(innerHitBuilder, rewrittenQuery);
return new HasChildQueryBuilder(type, rewrittenQuery, minChildren, maxChildren, scoreMode, rewrittenInnerHit);
HasChildQueryBuilder hasChildQueryBuilder =
new HasChildQueryBuilder(type, rewrittenQuery, minChildren, maxChildren, scoreMode, rewrittenInnerHit);
hasChildQueryBuilder.ignoreUnmapped(ignoreUnmapped);
return hasChildQueryBuilder;
}
return this;
}

View File

@ -305,7 +305,9 @@ public class HasParentQueryBuilder extends AbstractQueryBuilder<HasParentQueryBu
QueryBuilder rewrittenQuery = query.rewrite(queryShardContext);
if (rewrittenQuery != query) {
InnerHitBuilder rewrittenInnerHit = InnerHitBuilder.rewrite(innerHit, rewrittenQuery);
return new HasParentQueryBuilder(type, rewrittenQuery, score, rewrittenInnerHit);
HasParentQueryBuilder hasParentQueryBuilder = new HasParentQueryBuilder(type, rewrittenQuery, score, rewrittenInnerHit);
hasParentQueryBuilder.ignoreUnmapped(ignoreUnmapped);
return hasParentQueryBuilder;
}
return this;
}

View File

@ -255,7 +255,9 @@ public class NestedQueryBuilder extends AbstractQueryBuilder<NestedQueryBuilder>
QueryBuilder rewrittenQuery = query.rewrite(queryRewriteContext);
if (rewrittenQuery != query) {
InnerHitBuilder rewrittenInnerHit = InnerHitBuilder.rewrite(innerHitBuilder, rewrittenQuery);
return new NestedQueryBuilder(path, rewrittenQuery, scoreMode, rewrittenInnerHit);
NestedQueryBuilder nestedQuery = new NestedQueryBuilder(path, rewrittenQuery, scoreMode, rewrittenInnerHit);
nestedQuery.ignoreUnmapped(ignoreUnmapped);
return nestedQuery;
}
return this;
}

View File

@ -27,6 +27,7 @@ import org.apache.lucene.search.BooleanClause.Occur;
import org.apache.lucene.search.BooleanQuery;
import org.apache.lucene.search.BoostQuery;
import org.apache.lucene.search.FuzzyQuery;
import org.apache.lucene.search.GraphQuery;
import org.apache.lucene.search.MultiPhraseQuery;
import org.apache.lucene.search.MultiTermQuery;
import org.apache.lucene.search.PhraseQuery;
@ -48,6 +49,7 @@ import org.elasticsearch.index.query.QueryShardContext;
import org.elasticsearch.index.query.support.QueryParsers;
import java.io.IOException;
import java.util.List;
public class MatchQuery {
@ -316,6 +318,21 @@ public class MatchQuery {
public Query createPhrasePrefixQuery(String field, String queryText, int phraseSlop, int maxExpansions) {
final Query query = createFieldQuery(getAnalyzer(), Occur.MUST, field, queryText, true, phraseSlop);
if (query instanceof GraphQuery) {
// we have a graph query, convert inner queries to multi phrase prefix queries
List<Query> oldQueries = ((GraphQuery) query).getQueries();
Query[] queries = new Query[oldQueries.size()];
for (int i = 0; i < queries.length; i++) {
queries[i] = toMultiPhrasePrefix(oldQueries.get(i), phraseSlop, maxExpansions);
}
return new GraphQuery(queries);
}
return toMultiPhrasePrefix(query, phraseSlop, maxExpansions);
}
private Query toMultiPhrasePrefix(final Query query, int phraseSlop, int maxExpansions) {
float boost = 1;
Query innerQuery = query;
while (innerQuery instanceof BoostQuery) {
@ -357,18 +374,38 @@ public class MatchQuery {
Query booleanQuery = createBooleanQuery(field, queryText, lowFreqOccur);
if (booleanQuery != null && booleanQuery instanceof BooleanQuery) {
BooleanQuery bq = (BooleanQuery) booleanQuery;
ExtendedCommonTermsQuery query = new ExtendedCommonTermsQuery(highFreqOccur, lowFreqOccur, maxTermFrequency, (
(BooleanQuery) booleanQuery).isCoordDisabled(), fieldType);
for (BooleanClause clause : bq.clauses()) {
if (!(clause.getQuery() instanceof TermQuery)) {
return booleanQuery;
return boolToExtendedCommonTermsQuery(bq, highFreqOccur, lowFreqOccur, maxTermFrequency, fieldType);
} else if (booleanQuery != null && booleanQuery instanceof GraphQuery && ((GraphQuery) booleanQuery).hasBoolean()) {
// we have a graph query that has at least one boolean sub-query
// re-build and use extended common terms
List<Query> oldQueries = ((GraphQuery) booleanQuery).getQueries();
Query[] queries = new Query[oldQueries.size()];
for (int i = 0; i < queries.length; i++) {
Query oldQuery = oldQueries.get(i);
if (oldQuery instanceof BooleanQuery) {
queries[i] = boolToExtendedCommonTermsQuery((BooleanQuery) oldQuery, highFreqOccur, lowFreqOccur, maxTermFrequency, fieldType);
} else {
queries[i] = oldQuery;
}
query.add(((TermQuery) clause.getQuery()).getTerm());
}
return query;
}
return booleanQuery;
return new GraphQuery(queries);
}
return booleanQuery;
}
private Query boolToExtendedCommonTermsQuery(BooleanQuery bq, Occur highFreqOccur, Occur lowFreqOccur, float
maxTermFrequency, MappedFieldType fieldType) {
ExtendedCommonTermsQuery query = new ExtendedCommonTermsQuery(highFreqOccur, lowFreqOccur, maxTermFrequency,
bq.isCoordDisabled(), fieldType);
for (BooleanClause clause : bq.clauses()) {
if (!(clause.getQuery() instanceof TermQuery)) {
return bq;
}
query.add(((TermQuery) clause.getQuery()).getTerm());
}
return query;
}
}

View File

@ -19,6 +19,7 @@
package org.elasticsearch.index.shard;
import org.elasticsearch.cluster.metadata.IndexMetaData;
import org.elasticsearch.common.io.stream.StreamInput;
import org.elasticsearch.common.io.stream.StreamOutput;
import org.elasticsearch.common.io.stream.Streamable;
@ -71,6 +72,22 @@ public class ShardId implements Streamable, Comparable<ShardId> {
return "[" + index.getName() + "][" + shardId + "]";
}
/**
* Parse the string representation of this shardId back to an object.
* We lose index uuid information here, but since we use toString in
* rest responses, this is the best we can do to reconstruct the object
* on the client side.
*/
public static ShardId fromString(String shardIdString) {
int splitPosition = shardIdString.indexOf("][");
if (splitPosition <= 0 || shardIdString.charAt(0) != '[' || shardIdString.charAt(shardIdString.length() - 1) != ']') {
throw new IllegalArgumentException("Unexpected shardId string format, expected [indexName][shardId] but got " + shardIdString);
}
String indexName = shardIdString.substring(1, splitPosition);
int shardId = Integer.parseInt(shardIdString.substring(splitPosition + 2, shardIdString.length() - 1));
return new ShardId(new Index(indexName, IndexMetaData.INDEX_UUID_NA_VALUE), shardId);
}
@Override
public boolean equals(Object o) {
if (this == o) return true;

View File

@ -222,7 +222,7 @@ public class RecoverySourceHandler {
final long numDocsSource = recoverySourceMetadata.getNumDocs();
if (numDocsTarget != numDocsSource) {
throw new IllegalStateException("try to recover " + request.shardId() + " from primary shard with sync id but number " +
"of docs differ: " + numDocsTarget + " (" + request.sourceNode().getName() + ", primary) vs " + numDocsSource
"of docs differ: " + numDocsSource + " (" + request.sourceNode().getName() + ", primary) vs " + numDocsTarget
+ "(" + request.targetNode().getName() + ")");
}
// we shortcut recovery here because we have nothing to copy. but we must still start the engine on the target.

View File

@ -47,7 +47,6 @@ import org.elasticsearch.cluster.metadata.MetaDataIndexUpgradeService;
import org.elasticsearch.cluster.node.DiscoveryNode;
import org.elasticsearch.cluster.routing.RoutingService;
import org.elasticsearch.cluster.routing.allocation.AllocationService;
import org.elasticsearch.cluster.routing.allocation.command.AllocationCommandRegistry;
import org.elasticsearch.cluster.service.ClusterService;
import org.elasticsearch.common.StopWatch;
import org.elasticsearch.common.component.Lifecycle;
@ -119,7 +118,6 @@ import org.elasticsearch.plugins.SearchPlugin;
import org.elasticsearch.repositories.RepositoriesModule;
import org.elasticsearch.script.ScriptModule;
import org.elasticsearch.script.ScriptService;
import org.elasticsearch.search.SearchExtRegistry;
import org.elasticsearch.search.SearchModule;
import org.elasticsearch.search.SearchRequestParsers;
import org.elasticsearch.search.SearchService;
@ -362,6 +360,7 @@ public class Node implements Closeable {
.flatMap(Function.identity()).collect(Collectors.toList());
final NamedWriteableRegistry namedWriteableRegistry = new NamedWriteableRegistry(namedWriteables);
NamedXContentRegistry xContentRegistry = new NamedXContentRegistry(Stream.of(
NetworkModule.getNamedXContents().stream(),
searchModule.getNamedXContents().stream(),
pluginsService.filterPlugins(Plugin.class).stream()
.flatMap(p -> p.getNamedXContent().stream()),
@ -412,7 +411,6 @@ public class Node implements Closeable {
namedWriteableRegistry, networkService, clusterService, pluginsService.filterPlugins(DiscoveryPlugin.class));
modules.add(b -> {
b.bind(SearchRequestParsers.class).toInstance(searchModule.getSearchRequestParsers());
b.bind(SearchExtRegistry.class).toInstance(searchModule.getSearchExtRegistry());
b.bind(NamedXContentRegistry.class).toInstance(xContentRegistry);
b.bind(PluginsService.class).toInstance(pluginsService);
b.bind(Client.class).toInstance(client);
@ -440,7 +438,6 @@ public class Node implements Closeable {
b.bind(Transport.class).toInstance(transport);
b.bind(TransportService.class).toInstance(transportService);
b.bind(NetworkService.class).toInstance(networkService);
b.bind(AllocationCommandRegistry.class).toInstance(NetworkModule.getAllocationCommandRegistry());
b.bind(UpdateHelper.class).toInstance(new UpdateHelper(settings, scriptModule.getScriptService()));
b.bind(MetaDataIndexUpgradeService.class).toInstance(new MetaDataIndexUpgradeService(settings,
xContentRegistry, indicesModule.getMapperRegistry(), settingsModule.getIndexScopedSettings()));
@ -467,7 +464,8 @@ public class Node implements Closeable {
.map(injector::getInstance).collect(Collectors.toList()));
resourcesToClose.addAll(pluginLifecycleComponents);
this.pluginLifecycleComponents = Collections.unmodifiableList(pluginLifecycleComponents);
client.initialize(injector.getInstance(new Key<Map<GenericAction, TransportAction>>() {}));
client.initialize(injector.getInstance(new Key<Map<GenericAction, TransportAction>>() {}),
() -> clusterService.localNode().getId());
logger.info("initialized");

View File

@ -25,13 +25,13 @@ 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.lucene.search.function.ScoreFunction;
import org.elasticsearch.common.xcontent.AbstractObjectParser.NoContextParser;
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.SearchExtBuilder;
import org.elasticsearch.search.SearchExtParser;
import org.elasticsearch.search.aggregations.Aggregation;
import org.elasticsearch.search.aggregations.AggregationBuilder;
import org.elasticsearch.search.aggregations.Aggregator;
@ -46,6 +46,7 @@ import org.elasticsearch.search.aggregations.pipeline.movavg.models.MovAvgModel;
import org.elasticsearch.search.fetch.FetchSubPhase;
import org.elasticsearch.search.fetch.subphase.highlight.Highlighter;
import org.elasticsearch.search.suggest.Suggester;
import org.elasticsearch.search.suggest.SuggestionBuilder;
import java.util.List;
import java.util.Map;
@ -85,7 +86,7 @@ public interface SearchPlugin {
return emptyList();
}
/**
* The new {@link SearchExtParser}s defined by this plugin.
* The new {@link SearchExtBuilder}s defined by this plugin.
*/
default List<SearchExtSpec<?>> getSearchExts() {
return emptyList();
@ -99,8 +100,8 @@ public interface SearchPlugin {
/**
* The new {@link Suggester}s defined by this plugin.
*/
default Map<String, Suggester<?>> getSuggesters() {
return emptyMap();
default List<SuggesterSpec<?>> getSuggesters() {
return emptyList();
}
/**
* The new {@link Query}s defined by this plugin.
@ -134,6 +135,38 @@ public interface SearchPlugin {
}
}
/**
* Specification for a {@link Suggester}.
*/
class SuggesterSpec<T extends SuggestionBuilder<T>> extends SearchExtensionSpec<T, NoContextParser<T>> {
/**
* Specification of custom {@link Suggester}.
*
* @param name holds the names by which this suggester 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 suggester's builder. Typically a reference to a constructor that takes a
* {@link StreamInput}
* @param parser the parser the reads the query suggester from xcontent
*/
public SuggesterSpec(ParseField name, Writeable.Reader<T> reader, NoContextParser<T> parser) {
super(name, reader, parser);
}
/**
* Specification of custom {@link Suggester}.
*
* @param name the name by which this suggester 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 suggester's builder. Typically a reference to a constructor that takes a
* {@link StreamInput}
* @param parser the parser the reads the suggester builder from xcontent
*/
public SuggesterSpec(String name, Writeable.Reader<T> reader, NoContextParser<T> parser) {
super(name, reader, parser);
}
}
/**
* Specification of custom {@link Query}.
*/
@ -302,12 +335,12 @@ public interface SearchPlugin {
* Specification for a {@link SearchExtBuilder} which represents an additional section that can be
* parsed in a search request (within the ext element).
*/
class SearchExtSpec<T extends SearchExtBuilder> extends SearchExtensionSpec<T, SearchExtParser<T>> {
public SearchExtSpec(ParseField name, Writeable.Reader<? extends T> reader, SearchExtParser<T> parser) {
class SearchExtSpec<T extends SearchExtBuilder> extends SearchExtensionSpec<T, NoContextParser<T>> {
public SearchExtSpec(ParseField name, Writeable.Reader<? extends T> reader, NoContextParser<T> parser) {
super(name, reader, parser);
}
public SearchExtSpec(String name, Writeable.Reader<? extends T> reader, SearchExtParser<T> parser) {
public SearchExtSpec(String name, Writeable.Reader<? extends T> reader, NoContextParser<T> parser) {
super(name, reader, parser);
}
}

View File

@ -176,8 +176,6 @@ public abstract class BlobStoreRepository extends AbstractLifecycleComponent imp
private static final String SNAPSHOT_CODEC = "snapshot";
static final String SNAPSHOTS_FILE = "index"; // package private for unit testing
private static final String INDEX_FILE_PREFIX = "index-";
private static final String INDEX_LATEST_BLOB = "index.latest";
@ -373,7 +371,7 @@ public abstract class BlobStoreRepository extends AbstractLifecycleComponent imp
BlobPath indexPath = basePath().add("indices").add(indexId.getId());
BlobContainer indexMetaDataBlobContainer = blobStore().blobContainer(indexPath);
try {
indexMetaDataFormat(snapshot.version()).delete(indexMetaDataBlobContainer, snapshotId.getUUID());
indexMetaDataFormat.delete(indexMetaDataBlobContainer, snapshotId.getUUID());
} catch (IOException ex) {
logger.warn((Supplier<?>) () -> new ParameterizedMessage("[{}] failed to delete metadata for index [{}]", snapshotId, index), ex);
}
@ -421,7 +419,7 @@ public abstract class BlobStoreRepository extends AbstractLifecycleComponent imp
if (snapshotInfo != null) {
// we know the version the snapshot was created with
try {
snapshotFormat(snapshotInfo.version()).delete(snapshotsBlobContainer, blobId);
snapshotFormat.delete(snapshotsBlobContainer, blobId);
} catch (IOException e) {
logger.warn((Supplier<?>) () -> new ParameterizedMessage("[{}] Unable to delete snapshot file [{}]", snapshotInfo.snapshotId(), blobId), e);
}
@ -439,7 +437,7 @@ public abstract class BlobStoreRepository extends AbstractLifecycleComponent imp
if (snapshotInfo != null) {
// we know the version the snapshot was created with
try {
globalMetaDataFormat(snapshotInfo.version()).delete(snapshotsBlobContainer, blobId);
globalMetaDataFormat.delete(snapshotsBlobContainer, blobId);
} catch (IOException e) {
logger.warn((Supplier<?>) () -> new ParameterizedMessage("[{}] Unable to delete global metadata file [{}]", snapshotInfo.snapshotId(), blobId), e);
}
@ -522,7 +520,7 @@ public abstract class BlobStoreRepository extends AbstractLifecycleComponent imp
}
}
try {
metaData = globalMetaDataFormat(snapshotVersion).read(snapshotsBlobContainer, snapshotId.getUUID());
metaData = globalMetaDataFormat.read(snapshotsBlobContainer, snapshotId.getUUID());
} catch (NoSuchFileException ex) {
throw new SnapshotMissingException(metadata.name(), snapshotId, ex);
} catch (IOException ex) {
@ -533,7 +531,7 @@ public abstract class BlobStoreRepository extends AbstractLifecycleComponent imp
BlobPath indexPath = basePath().add("indices").add(index.getId());
BlobContainer indexMetaDataBlobContainer = blobStore().blobContainer(indexPath);
try {
metaDataBuilder.put(indexMetaDataFormat(snapshotVersion).read(indexMetaDataBlobContainer, snapshotId.getUUID()), false);
metaDataBuilder.put(indexMetaDataFormat.read(indexMetaDataBlobContainer, snapshotId.getUUID()), false);
} catch (ElasticsearchParseException | IOException ex) {
if (ignoreIndexErrors) {
logger.warn((Supplier<?>) () -> new ParameterizedMessage("[{}] [{}] failed to read metadata for index", snapshotId, index.getName()), ex);
@ -563,27 +561,6 @@ public abstract class BlobStoreRepository extends AbstractLifecycleComponent imp
}
}
/**
* Returns appropriate global metadata format based on the provided version of the snapshot
*/
private BlobStoreFormat<MetaData> globalMetaDataFormat(Version version) {
return globalMetaDataFormat;
}
/**
* Returns appropriate snapshot format based on the provided version of the snapshot
*/
private BlobStoreFormat<SnapshotInfo> snapshotFormat(Version version) {
return snapshotFormat;
}
/**
* Returns appropriate index metadata format based on the provided version of the snapshot
*/
private BlobStoreFormat<IndexMetaData> indexMetaDataFormat(Version version) {
return indexMetaDataFormat;
}
@Override
public long getSnapshotThrottleTimeInNanos() {
return snapshotRateLimitingTimeInNanos.count();

View File

@ -123,29 +123,21 @@ public class BytesRestResponse extends RestResponse {
} else if (channel.detailedErrorsEnabled()) {
final ToXContent.Params params;
if (channel.request().paramAsBoolean("error_trace", !ElasticsearchException.REST_EXCEPTION_SKIP_STACK_TRACE_DEFAULT)) {
params = new ToXContent.DelegatingMapParams(Collections.singletonMap(ElasticsearchException.REST_EXCEPTION_SKIP_STACK_TRACE, "false"), channel.request());
params = new ToXContent.DelegatingMapParams(
Collections.singletonMap(ElasticsearchException.REST_EXCEPTION_SKIP_STACK_TRACE, "false"), channel.request());
} else {
if (status.getStatus() < 500) {
SUPPRESSED_ERROR_LOGGER.debug((Supplier<?>) () -> new ParameterizedMessage("path: {}, params: {}", channel.request().rawPath(), channel.request().params()), e);
SUPPRESSED_ERROR_LOGGER.debug(
(Supplier<?>) () -> new ParameterizedMessage("path: {}, params: {}",
channel.request().rawPath(), channel.request().params()), e);
} else {
SUPPRESSED_ERROR_LOGGER.warn((Supplier<?>) () -> new ParameterizedMessage("path: {}, params: {}", channel.request().rawPath(), channel.request().params()), e);
SUPPRESSED_ERROR_LOGGER.warn(
(Supplier<?>) () -> new ParameterizedMessage("path: {}, params: {}",
channel.request().rawPath(), channel.request().params()), e);
}
params = channel.request();
}
builder.field("error");
builder.startObject();
final ElasticsearchException[] rootCauses = ElasticsearchException.guessRootCauses(e);
builder.field("root_cause");
builder.startArray();
for (ElasticsearchException rootCause : rootCauses){
builder.startObject();
rootCause.toXContent(builder, new ToXContent.DelegatingMapParams(Collections.singletonMap(ElasticsearchException.REST_EXCEPTION_SKIP_CAUSE, "true"), params));
builder.endObject();
}
builder.endArray();
ElasticsearchException.toXContent(builder, params, e);
builder.endObject();
ElasticsearchException.renderException(builder, params, e);
} else {
builder.field("error", simpleMessage(e));
}

View File

@ -24,11 +24,8 @@ import org.elasticsearch.action.admin.cluster.reroute.ClusterRerouteResponse;
import org.elasticsearch.client.Requests;
import org.elasticsearch.client.node.NodeClient;
import org.elasticsearch.cluster.ClusterState;
import org.elasticsearch.cluster.routing.allocation.command.AllocationCommandRegistry;
import org.elasticsearch.cluster.routing.allocation.command.AllocationCommands;
import org.elasticsearch.common.ParseField;
import org.elasticsearch.common.ParseFieldMatcher;
import org.elasticsearch.common.ParseFieldMatcherSupplier;
import org.elasticsearch.common.Strings;
import org.elasticsearch.common.inject.Inject;
import org.elasticsearch.common.settings.Settings;
@ -49,10 +46,10 @@ import java.util.HashSet;
import java.util.Set;
public class RestClusterRerouteAction extends BaseRestHandler {
private static final ObjectParser<ClusterRerouteRequest, ParseContext> PARSER = new ObjectParser<>("cluster_reroute");
private static final ObjectParser<ClusterRerouteRequest, Void> PARSER = new ObjectParser<>("cluster_reroute");
static {
PARSER.declareField((p, v, c) -> v.commands(AllocationCommands.fromXContent(p, c.registry)),
new ParseField("commands"), ValueType.OBJECT_ARRAY);
PARSER.declareField((p, v, c) -> v.commands(AllocationCommands.fromXContent(p)), new ParseField("commands"),
ValueType.OBJECT_ARRAY);
PARSER.declareBoolean(ClusterRerouteRequest::dryRun, new ParseField("dry_run"));
}
@ -60,20 +57,17 @@ public class RestClusterRerouteAction extends BaseRestHandler {
.arrayToCommaDelimitedString(EnumSet.complementOf(EnumSet.of(ClusterState.Metric.METADATA)).toArray());
private final SettingsFilter settingsFilter;
private final AllocationCommandRegistry registry;
@Inject
public RestClusterRerouteAction(Settings settings, RestController controller, SettingsFilter settingsFilter,
AllocationCommandRegistry registry) {
public RestClusterRerouteAction(Settings settings, RestController controller, SettingsFilter settingsFilter) {
super(settings);
this.settingsFilter = settingsFilter;
this.registry = registry;
controller.registerHandler(RestRequest.Method.POST, "/_cluster/reroute", this);
}
@Override
public RestChannelConsumer prepareRequest(final RestRequest request, final NodeClient client) throws IOException {
ClusterRerouteRequest clusterRerouteRequest = createRequest(request, registry, parseFieldMatcher);
ClusterRerouteRequest clusterRerouteRequest = createRequest(request);
// by default, return everything but metadata
final String metric = request.param("metric");
@ -111,31 +105,14 @@ public class RestClusterRerouteAction extends BaseRestHandler {
return RESPONSE_PARAMS;
}
public static ClusterRerouteRequest createRequest(RestRequest request, AllocationCommandRegistry registry,
ParseFieldMatcher parseFieldMatcher) throws IOException {
public static ClusterRerouteRequest createRequest(RestRequest request) throws IOException {
ClusterRerouteRequest clusterRerouteRequest = Requests.clusterRerouteRequest();
clusterRerouteRequest.dryRun(request.paramAsBoolean("dry_run", clusterRerouteRequest.dryRun()));
clusterRerouteRequest.explain(request.paramAsBoolean("explain", clusterRerouteRequest.explain()));
clusterRerouteRequest.timeout(request.paramAsTime("timeout", clusterRerouteRequest.timeout()));
clusterRerouteRequest.setRetryFailed(request.paramAsBoolean("retry_failed", clusterRerouteRequest.isRetryFailed()));
clusterRerouteRequest.masterNodeTimeout(request.paramAsTime("master_timeout", clusterRerouteRequest.masterNodeTimeout()));
request.applyContentParser(parser -> PARSER.parse(parser, clusterRerouteRequest, new ParseContext(registry, parseFieldMatcher)));
request.applyContentParser(parser -> PARSER.parse(parser, clusterRerouteRequest, null));
return clusterRerouteRequest;
}
private static class ParseContext implements ParseFieldMatcherSupplier {
private final AllocationCommandRegistry registry;
private final ParseFieldMatcher parseFieldMatcher;
private ParseContext(AllocationCommandRegistry registry, ParseFieldMatcher parseFieldMatcher) {
this.registry = registry;
this.parseFieldMatcher = parseFieldMatcher;
}
@Override
public ParseFieldMatcher getParseFieldMatcher() {
return parseFieldMatcher;
}
}
}

View File

@ -90,8 +90,7 @@ public class RestMultiSearchAction extends BaseRestHandler {
parseMultiLineRequest(restRequest, multiRequest.indicesOptions(), allowExplicitIndex, (searchRequest, parser) -> {
try {
final QueryParseContext queryParseContext = new QueryParseContext(parser, parseFieldMatcher);
searchRequest.source(SearchSourceBuilder.fromXContent(queryParseContext,
searchRequestParsers.aggParsers, searchRequestParsers.suggesters, searchRequestParsers.searchExtParsers));
searchRequest.source(SearchSourceBuilder.fromXContent(queryParseContext));
multiRequest.add(searchRequest);
} catch (IOException e) {
throw new ElasticsearchParseException("Exception when parsing search request", e);

View File

@ -93,8 +93,7 @@ public class RestSearchAction extends BaseRestHandler {
searchRequest.indices(Strings.splitStringByCommaToArray(request.param("index")));
if (requestContentParser != null) {
QueryParseContext context = new QueryParseContext(requestContentParser, parseFieldMatcher);
searchRequest.source().parseXContent(context, searchRequestParsers.aggParsers, searchRequestParsers.suggesters,
searchRequestParsers.searchExtParsers);
searchRequest.source().parseXContent(context);
}
// do not allow 'query_and_fetch' or 'dfs_query_and_fetch' search types

View File

@ -24,6 +24,7 @@ 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.xcontent.ToXContent;
import org.elasticsearch.common.xcontent.AbstractObjectParser.NoContextParser;
import org.elasticsearch.plugins.SearchPlugin;
import org.elasticsearch.plugins.SearchPlugin.SearchExtSpec;
@ -34,13 +35,12 @@ import org.elasticsearch.plugins.SearchPlugin.SearchExtSpec;
* read from the incoming stream, usually done adding a constructor that takes {@link StreamInput} as
* an argument.
*
* Registration happens through {@link SearchPlugin#getSearchExts()}, which also needs a {@link SearchExtParser} that's able to parse
* Registration happens through {@link SearchPlugin#getSearchExts()}, which also needs a {@link NoContextParser} that's able to parse
* the incoming request from the REST layer into the proper {@link SearchExtBuilder} subclass.
*
* {@link #getWriteableName()} must return the same name as the one used for the registration
* of the {@link SearchExtSpec}.
*
* @see SearchExtParser
* @see SearchExtSpec
*/
public abstract class SearchExtBuilder implements NamedWriteable, ToXContent {

View File

@ -1,32 +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;
import org.elasticsearch.common.xcontent.ParseFieldRegistry;
/**
* Extensions to ParseFieldRegistry to make Guice happy.
*/
public class SearchExtRegistry extends ParseFieldRegistry<SearchExtParser> {
public SearchExtRegistry() {
super("ext");
}
}

View File

@ -24,7 +24,7 @@ import org.elasticsearch.ElasticsearchParseException;
import org.elasticsearch.common.bytes.BytesReference;
import org.elasticsearch.common.io.stream.Streamable;
import org.elasticsearch.common.text.Text;
import org.elasticsearch.common.xcontent.ToXContent;
import org.elasticsearch.common.xcontent.ToXContentObject;
import org.elasticsearch.search.fetch.subphase.highlight.HighlightField;
import java.util.Map;
@ -34,7 +34,7 @@ import java.util.Map;
*
* @see SearchHits
*/
public interface SearchHit extends Streamable, ToXContent, Iterable<SearchHitField> {
public interface SearchHit extends Streamable, ToXContentObject, Iterable<SearchHitField> {
/**
* The score.

View File

@ -93,9 +93,10 @@ import org.elasticsearch.plugins.SearchPlugin.QuerySpec;
import org.elasticsearch.plugins.SearchPlugin.ScoreFunctionSpec;
import org.elasticsearch.plugins.SearchPlugin.SearchExtSpec;
import org.elasticsearch.plugins.SearchPlugin.SearchExtensionSpec;
import org.elasticsearch.plugins.SearchPlugin.SuggesterSpec;
import org.elasticsearch.search.aggregations.AggregationBuilder;
import org.elasticsearch.search.aggregations.Aggregator;
import org.elasticsearch.search.aggregations.AggregatorParsers;
import org.elasticsearch.search.aggregations.AggregatorFactories;
import org.elasticsearch.search.aggregations.BaseAggregationBuilder;
import org.elasticsearch.search.aggregations.InternalAggregation;
import org.elasticsearch.search.aggregations.PipelineAggregationBuilder;
import org.elasticsearch.search.aggregations.bucket.children.ChildrenAggregationBuilder;
@ -238,16 +239,14 @@ import org.elasticsearch.search.sort.GeoDistanceSortBuilder;
import org.elasticsearch.search.sort.ScoreSortBuilder;
import org.elasticsearch.search.sort.ScriptSortBuilder;
import org.elasticsearch.search.sort.SortBuilder;
import org.elasticsearch.search.suggest.Suggester;
import org.elasticsearch.search.suggest.Suggesters;
import org.elasticsearch.search.suggest.SuggestionBuilder;
import org.elasticsearch.search.suggest.completion.CompletionSuggester;
import org.elasticsearch.search.suggest.completion.CompletionSuggestionBuilder;
import org.elasticsearch.search.suggest.phrase.Laplace;
import org.elasticsearch.search.suggest.phrase.LinearInterpolation;
import org.elasticsearch.search.suggest.phrase.PhraseSuggester;
import org.elasticsearch.search.suggest.phrase.PhraseSuggestionBuilder;
import org.elasticsearch.search.suggest.phrase.SmoothingModel;
import org.elasticsearch.search.suggest.phrase.StupidBackoff;
import org.elasticsearch.search.suggest.term.TermSuggester;
import org.elasticsearch.search.suggest.term.TermSuggestionBuilder;
import java.util.ArrayList;
import java.util.List;
@ -267,18 +266,12 @@ public class SearchModule {
private final boolean transportClient;
private final Map<String, Highlighter> highlighters;
private final Map<String, Suggester<?>> suggesters;
private final ParseFieldRegistry<Aggregator.Parser> aggregationParserRegistry = new ParseFieldRegistry<>("aggregation");
private final ParseFieldRegistry<PipelineAggregator.Parser> pipelineAggregationParserRegistry = new ParseFieldRegistry<>(
"pipline_aggregation");
private final AggregatorParsers aggregatorParsers = new AggregatorParsers(aggregationParserRegistry, pipelineAggregationParserRegistry);
private final ParseFieldRegistry<SignificanceHeuristicParser> significanceHeuristicParserRegistry = new ParseFieldRegistry<>(
"significance_heuristic");
private final ParseFieldRegistry<MovAvgModel.AbstractModelParser> movingAverageModelParserRegistry = new ParseFieldRegistry<>(
"moving_avg_model");
private final List<FetchSubPhase> fetchSubPhases = new ArrayList<>();
private final SearchExtRegistry searchExtParserRegistry = new SearchExtRegistry();
private final Settings settings;
private final List<NamedWriteableRegistry.Entry> namedWriteables = new ArrayList<>();
@ -288,7 +281,7 @@ public class SearchModule {
public SearchModule(Settings settings, boolean transportClient, List<SearchPlugin> plugins) {
this.settings = settings;
this.transportClient = transportClient;
suggesters = setupSuggesters(plugins);
registerSuggesters(plugins);
highlighters = setupHighlighters(settings, plugins);
registerScoreFunctions(plugins);
registerQueryParsers(plugins);
@ -302,7 +295,7 @@ public class SearchModule {
registerFetchSubPhases(plugins);
registerSearchExts(plugins);
registerShapes();
searchRequestParsers = new SearchRequestParsers(aggregatorParsers, getSuggesters(), searchExtParserRegistry);
searchRequestParsers = new SearchRequestParsers();
}
public List<NamedWriteableRegistry.Entry> getNamedWriteables() {
@ -313,10 +306,6 @@ public class SearchModule {
return namedXContents;
}
public Suggesters getSuggesters() {
return new Suggesters(suggesters);
}
public SearchRequestParsers getSearchRequestParsers() {
return searchRequestParsers;
}
@ -342,13 +331,6 @@ public class SearchModule {
return movingAverageModelParserRegistry;
}
/**
* Parsers for {@link AggregationBuilder}s and {@link PipelineAggregationBuilder}s.
*/
public AggregatorParsers getAggregatorParsers() {
return aggregatorParsers;
}
private void registerAggregations(List<SearchPlugin> plugins) {
registerAggregation(new AggregationSpec(AvgAggregationBuilder.NAME, AvgAggregationBuilder::new, AvgAggregationBuilder::parse)
.addResultReader(InternalAvg::new));
@ -434,7 +416,10 @@ public class SearchModule {
private void registerAggregation(AggregationSpec spec) {
if (false == transportClient) {
aggregationParserRegistry.register(spec.getParser(), spec.getName());
namedXContents.add(new NamedXContentRegistry.Entry(BaseAggregationBuilder.class, spec.getName(), (p, c) -> {
AggregatorFactories.AggParseContext context = (AggregatorFactories.AggParseContext) c;
return spec.getParser().parse(context.name, context.queryParseContext);
}));
}
namedWriteables.add(
new NamedWriteableRegistry.Entry(AggregationBuilder.class, spec.getName().getPreferredName(), spec.getReader()));
@ -528,7 +513,10 @@ public class SearchModule {
private void registerPipelineAggregation(PipelineAggregationSpec spec) {
if (false == transportClient) {
pipelineAggregationParserRegistry.register(spec.getParser(), spec.getName());
namedXContents.add(new NamedXContentRegistry.Entry(BaseAggregationBuilder.class, spec.getName(), (p, c) -> {
AggregatorFactories.AggParseContext context = (AggregatorFactories.AggParseContext) c;
return spec.getParser().parse(context.name, context.queryParseContext);
}));
}
namedWriteables.add(
new NamedWriteableRegistry.Entry(PipelineAggregationBuilder.class, spec.getName().getPreferredName(), spec.getReader()));
@ -571,23 +559,22 @@ public class SearchModule {
namedWriteables.add(new NamedWriteableRegistry.Entry(SmoothingModel.class, StupidBackoff.NAME, StupidBackoff::new));
}
private Map<String, Suggester<?>> setupSuggesters(List<SearchPlugin> plugins) {
private void registerSuggesters(List<SearchPlugin> plugins) {
registerSmoothingModels(namedWriteables);
// Suggester<?> is weird - it is both a Parser and a reader....
NamedRegistry<Suggester<?>> suggesters = new NamedRegistry<Suggester<?>>("suggester") {
@Override
public void register(String name, Suggester<?> t) {
super.register(name, t);
namedWriteables.add(new NamedWriteableRegistry.Entry(SuggestionBuilder.class, name, t));
}
};
suggesters.register("phrase", PhraseSuggester.INSTANCE);
suggesters.register("term", TermSuggester.INSTANCE);
suggesters.register("completion", CompletionSuggester.INSTANCE);
registerSuggester(new SuggesterSpec<>("term", TermSuggestionBuilder::new, TermSuggestionBuilder::fromXContent));
registerSuggester(new SuggesterSpec<>("phrase", PhraseSuggestionBuilder::new, PhraseSuggestionBuilder::fromXContent));
registerSuggester(new SuggesterSpec<>("completion", CompletionSuggestionBuilder::new, CompletionSuggestionBuilder::fromXContent));
suggesters.extractAndRegister(plugins, SearchPlugin::getSuggesters);
return unmodifiableMap(suggesters.getRegistry());
registerFromPlugin(plugins, SearchPlugin::getSuggesters, this::registerSuggester);
}
private void registerSuggester(SuggesterSpec<?> suggester) {
namedWriteables.add(new NamedWriteableRegistry.Entry(
SuggestionBuilder.class, suggester.getName().getPreferredName(), suggester.getReader()));
// TODO Merge NoContextParser and FromXContent
namedXContents.add(new NamedXContentRegistry.Entry(SuggestionBuilder.class, suggester.getName(),
p -> suggester.getParser().parse(p)));
}
private Map<String, Highlighter> setupHighlighters(Settings settings, List<SearchPlugin> plugins) {
@ -699,7 +686,8 @@ public class SearchModule {
}
private void registerSearchExt(SearchExtSpec<?> spec) {
searchExtParserRegistry.register(spec.getParser(), spec.getName());
// TODO merge NoContextParser and ToXContent
namedXContents.add(new NamedXContentRegistry.Entry(SearchExtBuilder.class, spec.getName(), p -> spec.getParser().parse(p)));
namedWriteables.add(new NamedWriteableRegistry.Entry(SearchExtBuilder.class, spec.getName().getPreferredName(), spec.getReader()));
}
@ -782,8 +770,4 @@ public class SearchModule {
public FetchPhase getFetchPhase() {
return new FetchPhase(fetchSubPhases);
}
public SearchExtRegistry getSearchExtRegistry() {
return searchExtParserRegistry;
}
}

View File

@ -19,44 +19,11 @@
package org.elasticsearch.search;
import org.elasticsearch.index.query.QueryParseContext;
import org.elasticsearch.search.aggregations.AggregatorParsers;
import org.elasticsearch.search.suggest.Suggesters;
/**
* A container for all parsers used to parse
* {@link org.elasticsearch.action.search.SearchRequest} objects from a rest request.
*/
public class SearchRequestParsers {
// TODO: this class should be renamed to SearchRequestParser, and all the parse
// methods split across RestSearchAction and SearchSourceBuilder should be moved here
// TODO: make all members private once parsing functions are moved here
// TODO: AggregatorParsers should be removed and the underlying maps of agg
// and pipeline agg parsers should be here
/**
* Agg and pipeline agg parsers that may be used in search requests.
* @see org.elasticsearch.search.builder.SearchSourceBuilder#fromXContent(QueryParseContext, AggregatorParsers,
* Suggesters, SearchExtRegistry)
*/
public final AggregatorParsers aggParsers;
// TODO: Suggesters should be removed and the underlying map moved here
/**
* Suggesters that may be used in search requests.
* @see org.elasticsearch.search.builder.SearchSourceBuilder#fromXContent(QueryParseContext, AggregatorParsers,
* Suggesters, SearchExtRegistry)
*/
public final Suggesters suggesters;
/**
* Pluggable section that can be parsed out of a search section, within the ext element
*/
public final SearchExtRegistry searchExtParsers;
public SearchRequestParsers(AggregatorParsers aggParsers, Suggesters suggesters, SearchExtRegistry searchExtParsers) {
this.aggParsers = aggParsers;
this.suggesters = suggesters;
this.searchExtParsers = searchExtParsers;
public SearchRequestParsers() {
}
}

View File

@ -21,7 +21,6 @@ package org.elasticsearch.search.aggregations;
import org.elasticsearch.common.io.stream.StreamInput;
import org.elasticsearch.common.io.stream.StreamOutput;
import org.elasticsearch.common.xcontent.XContentBuilder;
import org.elasticsearch.search.aggregations.InternalAggregation.Type;
import org.elasticsearch.search.internal.SearchContext;
import java.io.IOException;
@ -40,17 +39,16 @@ public abstract class AbstractAggregationBuilder<AB extends AbstractAggregationB
* Constructs a new aggregation builder.
*
* @param name The aggregation name
* @param type The aggregation type
*/
public AbstractAggregationBuilder(String name, Type type) {
super(name, type);
public AbstractAggregationBuilder(String name) {
super(name);
}
/**
* Read from a stream.
*/
protected AbstractAggregationBuilder(StreamInput in, Type type) throws IOException {
super(in.readString(), type);
protected AbstractAggregationBuilder(StreamInput in) throws IOException {
super(in.readString());
factoriesBuilder = new AggregatorFactories.Builder(in);
metaData = in.readMap();
}
@ -117,8 +115,10 @@ public abstract class AbstractAggregationBuilder<AB extends AbstractAggregationB
return (AB) this;
}
public String getType() {
return type.name();
@Override
public final String getWriteableName() {
// We always use the type of the aggregation as the writeable name
return getType();
}
@Override
@ -137,7 +137,7 @@ public abstract class AbstractAggregationBuilder<AB extends AbstractAggregationB
if (this.metaData != null) {
builder.field("meta", this.metaData);
}
builder.field(type.name());
builder.field(getType());
internalXContent(builder, params);
if (factoriesBuilder != null && (factoriesBuilder.count()) > 0) {
@ -153,7 +153,7 @@ public abstract class AbstractAggregationBuilder<AB extends AbstractAggregationB
@Override
public int hashCode() {
return Objects.hash(factoriesBuilder, metaData, name, type, doHashCode());
return Objects.hash(factoriesBuilder, metaData, name, doHashCode());
}
protected abstract int doHashCode();
@ -168,8 +168,6 @@ public abstract class AbstractAggregationBuilder<AB extends AbstractAggregationB
AbstractAggregationBuilder<AB> other = (AbstractAggregationBuilder<AB>) obj;
if (!Objects.equals(name, other.name))
return false;
if (!Objects.equals(type, other.type))
return false;
if (!Objects.equals(metaData, other.metaData))
return false;
if (!Objects.equals(factoriesBuilder, other.factoriesBuilder))

View File

@ -23,7 +23,7 @@ import org.elasticsearch.action.support.ToXContentToBytes;
import org.elasticsearch.common.ParseField;
import org.elasticsearch.common.io.stream.NamedWriteable;
import org.elasticsearch.common.xcontent.ToXContent;
import org.elasticsearch.search.aggregations.InternalAggregation.Type;
import org.elasticsearch.index.query.QueryParseContext;
import org.elasticsearch.search.internal.SearchContext;
import java.io.IOException;
@ -34,27 +34,21 @@ import java.util.Map;
*/
public abstract class AggregationBuilder
extends ToXContentToBytes
implements NamedWriteable, ToXContent {
implements NamedWriteable, ToXContent, BaseAggregationBuilder {
protected final String name;
protected final Type type;
protected AggregatorFactories.Builder factoriesBuilder = AggregatorFactories.builder();
/**
* Constructs a new aggregation builder.
*
* @param name The aggregation name
* @param type The aggregation type
*/
protected AggregationBuilder(String name, Type type) {
protected AggregationBuilder(String name) {
if (name == null) {
throw new IllegalArgumentException("[name] must not be null: [" + name + "]");
}
if (type == null) {
throw new IllegalArgumentException("[type] must not be null: [" + name + "]");
}
this.name = name;
this.type = type;
}
/** Return this aggregation's name. */
@ -66,6 +60,7 @@ public abstract class AggregationBuilder
protected abstract AggregatorFactory<?> build(SearchContext context, AggregatorFactory<?> parent) throws IOException;
/** Associate metadata with this {@link AggregationBuilder}. */
@Override
public abstract AggregationBuilder setMetaData(Map<String, Object> metaData);
/** Add a sub aggregation to this builder. */
@ -77,13 +72,14 @@ public abstract class AggregationBuilder
/**
* Internal: Registers sub-factories with this factory. The sub-factory will be
* responsible for the creation of sub-aggregators under the aggregator
* created by this factory. This is only for use by {@link AggregatorParsers}.
* created by this factory. This is only for use by {@link AggregatorFactories#parseAggregators(QueryParseContext)}.
*
* @param subFactories
* The sub-factories
* @return this factory (fluent interface)
*/
protected abstract AggregationBuilder subAggregations(AggregatorFactories.Builder subFactories);
@Override
public abstract AggregationBuilder subAggregations(AggregatorFactories.Builder subFactories);
/** Common xcontent fields shared among aggregator builders */
public static final class CommonFields extends ParseField.CommonFields {

View File

@ -19,10 +19,13 @@
package org.elasticsearch.search.aggregations;
import org.elasticsearch.action.support.ToXContentToBytes;
import org.elasticsearch.common.ParsingException;
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.xcontent.XContentBuilder;
import org.elasticsearch.common.xcontent.XContentParser;
import org.elasticsearch.index.query.QueryParseContext;
import org.elasticsearch.search.aggregations.pipeline.PipelineAggregator;
import org.elasticsearch.search.aggregations.support.AggregationPath;
import org.elasticsearch.search.aggregations.support.AggregationPath.PathElement;
@ -40,8 +43,126 @@ import java.util.List;
import java.util.Map;
import java.util.Objects;
import java.util.Set;
import java.util.regex.Matcher;
import java.util.regex.Pattern;
public class AggregatorFactories {
public static final Pattern VALID_AGG_NAME = Pattern.compile("[^\\[\\]>]+");
/**
* Parses the aggregation request recursively generating aggregator factories in turn.
*
* @param parseContext The parse context.
*
* @return The parsed aggregator factories.
*
* @throws IOException When parsing fails for unknown reasons.
*/
public static AggregatorFactories.Builder parseAggregators(QueryParseContext parseContext) throws IOException {
return parseAggregators(parseContext, 0);
}
private static AggregatorFactories.Builder parseAggregators(QueryParseContext parseContext, int level) throws IOException {
Matcher validAggMatcher = VALID_AGG_NAME.matcher("");
AggregatorFactories.Builder factories = new AggregatorFactories.Builder();
XContentParser.Token token = null;
XContentParser parser = parseContext.parser();
while ((token = parser.nextToken()) != XContentParser.Token.END_OBJECT) {
if (token != XContentParser.Token.FIELD_NAME) {
throw new ParsingException(parser.getTokenLocation(),
"Unexpected token " + token + " in [aggs]: aggregations definitions must start with the name of the aggregation.");
}
final String aggregationName = parser.currentName();
if (!validAggMatcher.reset(aggregationName).matches()) {
throw new ParsingException(parser.getTokenLocation(), "Invalid aggregation name [" + aggregationName
+ "]. Aggregation names must be alpha-numeric and can only contain '_' and '-'");
}
token = parser.nextToken();
if (token != XContentParser.Token.START_OBJECT) {
throw new ParsingException(parser.getTokenLocation(), "Aggregation definition for [" + aggregationName + " starts with a ["
+ token + "], expected a [" + XContentParser.Token.START_OBJECT + "].");
}
BaseAggregationBuilder aggBuilder = null;
AggregatorFactories.Builder subFactories = null;
Map<String, Object> metaData = null;
while ((token = parser.nextToken()) != XContentParser.Token.END_OBJECT) {
if (token != XContentParser.Token.FIELD_NAME) {
throw new ParsingException(
parser.getTokenLocation(), "Expected [" + XContentParser.Token.FIELD_NAME + "] under a ["
+ XContentParser.Token.START_OBJECT + "], but got a [" + token + "] in [" + aggregationName + "]",
parser.getTokenLocation());
}
final String fieldName = parser.currentName();
token = parser.nextToken();
if (token == XContentParser.Token.START_OBJECT) {
switch (fieldName) {
case "meta":
metaData = parser.map();
break;
case "aggregations":
case "aggs":
if (subFactories != null) {
throw new ParsingException(parser.getTokenLocation(),
"Found two sub aggregation definitions under [" + aggregationName + "]");
}
subFactories = parseAggregators(parseContext, level + 1);
break;
default:
if (aggBuilder != null) {
throw new ParsingException(parser.getTokenLocation(), "Found two aggregation type definitions in ["
+ aggregationName + "]: [" + aggBuilder.getType() + "] and [" + fieldName + "]");
}
aggBuilder = parser.namedObject(BaseAggregationBuilder.class, fieldName,
new AggParseContext(aggregationName, parseContext));
}
} else {
throw new ParsingException(parser.getTokenLocation(), "Expected [" + XContentParser.Token.START_OBJECT + "] under ["
+ fieldName + "], but got a [" + token + "] in [" + aggregationName + "]");
}
}
if (aggBuilder == null) {
throw new ParsingException(parser.getTokenLocation(), "Missing definition for aggregation [" + aggregationName + "]",
parser.getTokenLocation());
} else {
if (metaData != null) {
aggBuilder.setMetaData(metaData);
}
if (subFactories != null) {
aggBuilder.subAggregations(subFactories);
}
if (aggBuilder instanceof AggregationBuilder) {
factories.addAggregator((AggregationBuilder) aggBuilder);
} else {
factories.addPipelineAggregator((PipelineAggregationBuilder) aggBuilder);
}
}
}
return factories;
}
/**
* Context to parse and aggregation. This should eventually be removed and replaced with a String.
*/
public static final class AggParseContext {
public final String name;
public final QueryParseContext queryParseContext;
public AggParseContext(String name, QueryParseContext queryParseContext) {
this.name = name;
this.queryParseContext = queryParseContext;
}
}
public static final AggregatorFactories EMPTY = new AggregatorFactories(null, new AggregatorFactory<?>[0],
new ArrayList<PipelineAggregationBuilder>());

View File

@ -24,7 +24,6 @@ import org.apache.lucene.search.Scorer;
import org.elasticsearch.common.lease.Releasables;
import org.elasticsearch.common.util.BigArrays;
import org.elasticsearch.common.util.ObjectArray;
import org.elasticsearch.search.aggregations.InternalAggregation.Type;
import org.elasticsearch.search.aggregations.pipeline.PipelineAggregator;
import org.elasticsearch.search.internal.SearchContext;
import org.elasticsearch.search.internal.SearchContext.Lifetime;
@ -163,7 +162,6 @@ public abstract class AggregatorFactory<AF extends AggregatorFactory<AF>> {
}
protected final String name;
protected final Type type;
protected final AggregatorFactory<?> parent;
protected final AggregatorFactories factories;
protected final Map<String, Object> metaData;
@ -174,15 +172,12 @@ public abstract class AggregatorFactory<AF extends AggregatorFactory<AF>> {
*
* @param name
* The aggregation name
* @param type
* The aggregation type
* @throws IOException
* if an error occurs creating the factory
*/
public AggregatorFactory(String name, Type type, SearchContext context, AggregatorFactory<?> parent,
public AggregatorFactory(String name, SearchContext context, AggregatorFactory<?> parent,
AggregatorFactories.Builder subFactoriesBuilder, Map<String, Object> metaData) throws IOException {
this.name = name;
this.type = type;
this.context = context;
this.parent = parent;
this.factories = subFactoriesBuilder.build(context, this);
@ -226,10 +221,6 @@ public abstract class AggregatorFactory<AF extends AggregatorFactory<AF>> {
return createInternal(parent, collectsFromSingleBucket, this.factories.createPipelineAggregators(), this.metaData);
}
public String getType() {
return type.name();
}
public AggregatorFactory<?> getParent() {
return parent;
}

View File

@ -1,192 +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.ParsingException;
import org.elasticsearch.common.xcontent.ParseFieldRegistry;
import org.elasticsearch.common.xcontent.XContentParser;
import org.elasticsearch.index.query.QueryParseContext;
import org.elasticsearch.search.aggregations.pipeline.PipelineAggregator;
import java.io.IOException;
import java.util.Map;
import java.util.regex.Matcher;
import java.util.regex.Pattern;
/**
* A registry for all the aggregator parser, also servers as the main parser for the aggregations module
*/
public class AggregatorParsers {
public static final Pattern VALID_AGG_NAME = Pattern.compile("[^\\[\\]>]+");
private final ParseFieldRegistry<Aggregator.Parser> aggregationParserRegistry;
private final ParseFieldRegistry<PipelineAggregator.Parser> pipelineAggregationParserRegistry;
public AggregatorParsers(ParseFieldRegistry<Aggregator.Parser> aggregationParserRegistry,
ParseFieldRegistry<PipelineAggregator.Parser> pipelineAggregationParserRegistry) {
this.aggregationParserRegistry = aggregationParserRegistry;
this.pipelineAggregationParserRegistry = pipelineAggregationParserRegistry;
}
/**
* Returns the parser that is registered under the given aggregation type.
*
* @param type The aggregation type
* @return The parser associated with the given aggregation type or null if it wasn't found.
*/
public Aggregator.Parser parser(String type) {
return aggregationParserRegistry.lookupReturningNullIfNotFound(type);
}
/**
* Returns the parser that is registered under the given pipeline aggregator type.
*
* @param type The pipeline aggregator type
* @return The parser associated with the given pipeline aggregator type or null if it wasn't found.
*/
public PipelineAggregator.Parser pipelineParser(String type) {
return pipelineAggregationParserRegistry.lookupReturningNullIfNotFound(type);
}
/**
* Parses the aggregation request recursively generating aggregator factories in turn.
*
* @param parseContext The parse context.
*
* @return The parsed aggregator factories.
*
* @throws IOException When parsing fails for unknown reasons.
*/
public AggregatorFactories.Builder parseAggregators(QueryParseContext parseContext) throws IOException {
return parseAggregators(parseContext, 0);
}
private AggregatorFactories.Builder parseAggregators(QueryParseContext parseContext, int level) throws IOException {
Matcher validAggMatcher = VALID_AGG_NAME.matcher("");
AggregatorFactories.Builder factories = new AggregatorFactories.Builder();
XContentParser.Token token = null;
XContentParser parser = parseContext.parser();
while ((token = parser.nextToken()) != XContentParser.Token.END_OBJECT) {
if (token != XContentParser.Token.FIELD_NAME) {
throw new ParsingException(parser.getTokenLocation(),
"Unexpected token " + token + " in [aggs]: aggregations definitions must start with the name of the aggregation.");
}
final String aggregationName = parser.currentName();
if (!validAggMatcher.reset(aggregationName).matches()) {
throw new ParsingException(parser.getTokenLocation(), "Invalid aggregation name [" + aggregationName
+ "]. Aggregation names must be alpha-numeric and can only contain '_' and '-'");
}
token = parser.nextToken();
if (token != XContentParser.Token.START_OBJECT) {
throw new ParsingException(parser.getTokenLocation(), "Aggregation definition for [" + aggregationName + " starts with a ["
+ token + "], expected a [" + XContentParser.Token.START_OBJECT + "].");
}
AggregationBuilder aggFactory = null;
PipelineAggregationBuilder pipelineAggregatorFactory = null;
AggregatorFactories.Builder subFactories = null;
Map<String, Object> metaData = null;
while ((token = parser.nextToken()) != XContentParser.Token.END_OBJECT) {
if (token != XContentParser.Token.FIELD_NAME) {
throw new ParsingException(
parser.getTokenLocation(), "Expected [" + XContentParser.Token.FIELD_NAME + "] under a ["
+ XContentParser.Token.START_OBJECT + "], but got a [" + token + "] in [" + aggregationName + "]",
parser.getTokenLocation());
}
final String fieldName = parser.currentName();
token = parser.nextToken();
if (token == XContentParser.Token.START_OBJECT) {
switch (fieldName) {
case "meta":
metaData = parser.map();
break;
case "aggregations":
case "aggs":
if (subFactories != null) {
throw new ParsingException(parser.getTokenLocation(),
"Found two sub aggregation definitions under [" + aggregationName + "]");
}
subFactories = parseAggregators(parseContext, level + 1);
break;
default:
if (aggFactory != null) {
throw new ParsingException(parser.getTokenLocation(), "Found two aggregation type definitions in ["
+ aggregationName + "]: [" + aggFactory.type + "] and [" + fieldName + "]");
}
if (pipelineAggregatorFactory != null) {
throw new ParsingException(parser.getTokenLocation(), "Found two aggregation type definitions in ["
+ aggregationName + "]: [" + pipelineAggregatorFactory + "] and [" + fieldName + "]");
}
Aggregator.Parser aggregatorParser = parser(fieldName);
if (aggregatorParser == null) {
PipelineAggregator.Parser pipelineAggregatorParser = pipelineParser(fieldName);
if (pipelineAggregatorParser == null) {
throw new ParsingException(parser.getTokenLocation(),
"Could not find aggregator type [" + fieldName + "] in [" + aggregationName + "]");
} else {
pipelineAggregatorFactory = pipelineAggregatorParser.parse(aggregationName, parseContext);
}
} else {
aggFactory = aggregatorParser.parse(aggregationName, parseContext);
}
}
} else {
throw new ParsingException(parser.getTokenLocation(), "Expected [" + XContentParser.Token.START_OBJECT + "] under ["
+ fieldName + "], but got a [" + token + "] in [" + aggregationName + "]");
}
}
if (aggFactory == null && pipelineAggregatorFactory == null) {
throw new ParsingException(parser.getTokenLocation(), "Missing definition for aggregation [" + aggregationName + "]",
parser.getTokenLocation());
} else if (aggFactory != null) {
assert pipelineAggregatorFactory == null;
if (metaData != null) {
aggFactory.setMetaData(metaData);
}
if (subFactories != null) {
aggFactory.subAggregations(subFactories);
}
factories.addAggregator(aggFactory);
} else {
assert pipelineAggregatorFactory != null;
if (subFactories != null) {
throw new ParsingException(parser.getTokenLocation(),
"Aggregation [" + aggregationName + "] cannot define sub-aggregations",
parser.getTokenLocation());
}
if (metaData != null) {
pipelineAggregatorFactory.setMetaData(metaData);
}
factories.addPipelineAggregator(pipelineAggregatorFactory);
}
}
return factories;
}
}

View File

@ -17,27 +17,30 @@
* under the License.
*/
package org.elasticsearch.search;
package org.elasticsearch.search.aggregations;
import org.elasticsearch.common.xcontent.XContentParser;
import org.elasticsearch.search.aggregations.AggregatorFactories.Builder;
import java.io.IOException;
import java.util.Map;
/**
* Defines a parser that is able to parse {@link org.elasticsearch.search.SearchExtBuilder}s
* from {@link org.elasticsearch.common.xcontent.XContent}.
*
* Registration happens through {@link org.elasticsearch.plugins.SearchPlugin#getSearchExts()}, which also needs a {@link SearchExtBuilder}
* implementation which is the object that this parser returns when reading an incoming request form the REST layer.
*
* @see SearchExtBuilder
* @see org.elasticsearch.plugins.SearchPlugin.SearchExtSpec
* Interface shared by {@link AggregationBuilder} and {@link PipelineAggregationBuilder} so they can conveniently share the same namespace
* for {@link XContentParser#namedObject(Class, String, Object)}.
*/
@FunctionalInterface
public interface SearchExtParser<T extends SearchExtBuilder> {
public interface BaseAggregationBuilder {
/**
* The name of the type of aggregation built by this builder.
*/
String getType();
/**
* Parses the supported element placed within the ext section of a search request
* Set the aggregation's metadata. Returns {@code this} for chaining.
*/
T fromXContent(XContentParser parser) throws IOException;
BaseAggregationBuilder setMetaData(Map<String, Object> metaData);
/**
* Set the sub aggregations if this aggregation supports sub aggregations. Returns {@code this} for chaining.
*/
BaseAggregationBuilder subAggregations(Builder subFactories);
}

View File

@ -38,33 +38,6 @@ import java.util.Objects;
* An internal implementation of {@link Aggregation}. Serves as a base class for all aggregation implementations.
*/
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>
* </ul>
*/
public static class Type {
private final String name;
public Type(String name) {
this.name = name;
}
/**
* @return The name of the type of aggregation. This is the key for parsing the aggregation from XContent and is the name of the
* aggregation's builder when serialized.
*/
public String name() {
return name;
}
@Override
public String toString() {
return name;
}
}
public static class ReduceContext {
private final BigArrays bigArrays;

View File

@ -20,7 +20,7 @@ package org.elasticsearch.search.aggregations;
import org.elasticsearch.action.support.ToXContentToBytes;
import org.elasticsearch.common.io.stream.NamedWriteable;
import org.elasticsearch.search.aggregations.AggregatorFactory;
import org.elasticsearch.search.aggregations.AggregatorFactories.Builder;
import org.elasticsearch.search.aggregations.pipeline.PipelineAggregator;
import java.io.IOException;
@ -32,7 +32,7 @@ import java.util.Map;
* specific type.
*/
public abstract class PipelineAggregationBuilder extends ToXContentToBytes
implements NamedWriteable {
implements NamedWriteable, BaseAggregationBuilder {
protected final String name;
protected final String[] bucketsPaths;
@ -79,6 +79,11 @@ public abstract class PipelineAggregationBuilder extends ToXContentToBytes
protected abstract PipelineAggregator create() throws IOException;
/** Associate metadata with this {@link PipelineAggregationBuilder}. */
@Override
public abstract PipelineAggregationBuilder setMetaData(Map<String, Object> metaData);
@Override
public PipelineAggregationBuilder subAggregations(Builder subFactories) {
throw new IllegalArgumentException("Aggregation [" + name + "] cannot define sub-aggregations");
}
}

View File

@ -31,7 +31,6 @@ import org.elasticsearch.index.mapper.ParentFieldMapper;
import org.elasticsearch.index.query.QueryParseContext;
import org.elasticsearch.search.aggregations.AggregatorFactories.Builder;
import org.elasticsearch.search.aggregations.AggregatorFactory;
import org.elasticsearch.search.aggregations.InternalAggregation.Type;
import org.elasticsearch.search.aggregations.support.FieldContext;
import org.elasticsearch.search.aggregations.support.ValueType;
import org.elasticsearch.search.aggregations.support.ValuesSource.Bytes.ParentChild;
@ -46,7 +45,6 @@ import java.util.Objects;
public class ChildrenAggregationBuilder extends ValuesSourceAggregationBuilder<ParentChild, ChildrenAggregationBuilder> {
public static final String NAME = "children";
private static final Type TYPE = new Type(NAME);
private String parentType;
private final String childType;
@ -60,7 +58,7 @@ public class ChildrenAggregationBuilder extends ValuesSourceAggregationBuilder<P
* the type of children documents
*/
public ChildrenAggregationBuilder(String name, String childType) {
super(name, TYPE, ValuesSourceType.BYTES, ValueType.STRING);
super(name, ValuesSourceType.BYTES, ValueType.STRING);
if (childType == null) {
throw new IllegalArgumentException("[childType] must not be null: [" + name + "]");
}
@ -71,7 +69,7 @@ public class ChildrenAggregationBuilder extends ValuesSourceAggregationBuilder<P
* Read from a stream.
*/
public ChildrenAggregationBuilder(StreamInput in) throws IOException {
super(in, TYPE, ValuesSourceType.BYTES, ValueType.STRING);
super(in, ValuesSourceType.BYTES, ValueType.STRING);
childType = in.readString();
}
@ -83,7 +81,7 @@ public class ChildrenAggregationBuilder extends ValuesSourceAggregationBuilder<P
@Override
protected ValuesSourceAggregatorFactory<ParentChild, ?> innerBuild(SearchContext context,
ValuesSourceConfig<ParentChild> config, AggregatorFactory<?> parent, Builder subFactoriesBuilder) throws IOException {
return new ChildrenAggregatorFactory(name, type, config, parentType, childFilter, parentFilter, context, parent,
return new ChildrenAggregatorFactory(name, config, parentType, childFilter, parentFilter, context, parent,
subFactoriesBuilder, metaData);
}
@ -163,7 +161,7 @@ public class ChildrenAggregationBuilder extends ValuesSourceAggregationBuilder<P
}
@Override
public String getWriteableName() {
public String getType() {
return NAME;
}
}

View File

@ -24,7 +24,6 @@ import org.elasticsearch.search.aggregations.Aggregator;
import org.elasticsearch.search.aggregations.AggregatorFactories;
import org.elasticsearch.search.aggregations.AggregatorFactory;
import org.elasticsearch.search.aggregations.InternalAggregation;
import org.elasticsearch.search.aggregations.InternalAggregation.Type;
import org.elasticsearch.search.aggregations.NonCollectingAggregator;
import org.elasticsearch.search.aggregations.pipeline.PipelineAggregator;
import org.elasticsearch.search.aggregations.support.ValuesSource;
@ -44,10 +43,10 @@ public class ChildrenAggregatorFactory
private final Query parentFilter;
private final Query childFilter;
public ChildrenAggregatorFactory(String name, Type type, ValuesSourceConfig<ParentChild> config, String parentType, Query childFilter,
public ChildrenAggregatorFactory(String name, ValuesSourceConfig<ParentChild> config, String parentType, Query childFilter,
Query parentFilter, SearchContext context, AggregatorFactory<?> parent, AggregatorFactories.Builder subFactoriesBuilder,
Map<String, Object> metaData) throws IOException {
super(name, type, config, context, parent, subFactoriesBuilder, metaData);
super(name, config, context, parent, subFactoriesBuilder, metaData);
this.parentType = parentType;
this.childFilter = childFilter;
this.parentFilter = parentFilter;

View File

@ -27,7 +27,6 @@ import org.elasticsearch.index.query.QueryParseContext;
import org.elasticsearch.search.aggregations.AbstractAggregationBuilder;
import org.elasticsearch.search.aggregations.AggregatorFactories;
import org.elasticsearch.search.aggregations.AggregatorFactory;
import org.elasticsearch.search.aggregations.InternalAggregation.Type;
import org.elasticsearch.search.internal.SearchContext;
import java.io.IOException;
@ -35,7 +34,6 @@ import java.util.Objects;
public class FilterAggregationBuilder extends AbstractAggregationBuilder<FilterAggregationBuilder> {
public static final String NAME = "filter";
private static final Type TYPE = new Type(NAME);
private final QueryBuilder filter;
@ -48,7 +46,7 @@ public class FilterAggregationBuilder extends AbstractAggregationBuilder<FilterA
* {@link Filter} aggregation.
*/
public FilterAggregationBuilder(String name, QueryBuilder filter) {
super(name, TYPE);
super(name);
if (filter == null) {
throw new IllegalArgumentException("[filter] must not be null: [" + name + "]");
}
@ -59,7 +57,7 @@ public class FilterAggregationBuilder extends AbstractAggregationBuilder<FilterA
* Read from a stream.
*/
public FilterAggregationBuilder(StreamInput in) throws IOException {
super(in, TYPE);
super(in);
filter = in.readNamedWriteable(QueryBuilder.class);
}
@ -73,7 +71,7 @@ public class FilterAggregationBuilder extends AbstractAggregationBuilder<FilterA
AggregatorFactories.Builder subFactoriesBuilder) throws IOException {
// TODO this sucks we need a rewrite phase for aggregations too
final QueryBuilder rewrittenFilter = QueryBuilder.rewriteQuery(filter, context.getQueryShardContext());
return new FilterAggregatorFactory(name, type, rewrittenFilter, context, parent, subFactoriesBuilder, metaData);
return new FilterAggregatorFactory(name, rewrittenFilter, context, parent, subFactoriesBuilder, metaData);
}
@Override
@ -101,7 +99,7 @@ public class FilterAggregationBuilder extends AbstractAggregationBuilder<FilterA
}
@Override
public String getWriteableName() {
public String getType() {
return NAME;
}
}

View File

@ -26,7 +26,6 @@ import org.elasticsearch.index.query.QueryBuilder;
import org.elasticsearch.search.aggregations.Aggregator;
import org.elasticsearch.search.aggregations.AggregatorFactories;
import org.elasticsearch.search.aggregations.AggregatorFactory;
import org.elasticsearch.search.aggregations.InternalAggregation.Type;
import org.elasticsearch.search.aggregations.pipeline.PipelineAggregator;
import org.elasticsearch.search.internal.SearchContext;
@ -38,9 +37,9 @@ public class FilterAggregatorFactory extends AggregatorFactory<FilterAggregatorF
private final Weight weight;
public FilterAggregatorFactory(String name, Type type, QueryBuilder filterBuilder, SearchContext context,
public FilterAggregatorFactory(String name, QueryBuilder filterBuilder, SearchContext context,
AggregatorFactory<?> parent, AggregatorFactories.Builder subFactoriesBuilder, Map<String, Object> metaData) throws IOException {
super(name, type, context, parent, subFactoriesBuilder, metaData);
super(name, context, parent, subFactoriesBuilder, metaData);
IndexSearcher contextSearcher = context.searcher();
Query filter = filterBuilder.toQuery(context.getQueryShardContext());
weight = contextSearcher.createNormalizedWeight(filter, false);

View File

@ -30,7 +30,6 @@ import org.elasticsearch.index.query.QueryParseContext;
import org.elasticsearch.search.aggregations.AbstractAggregationBuilder;
import org.elasticsearch.search.aggregations.AggregatorFactories.Builder;
import org.elasticsearch.search.aggregations.AggregatorFactory;
import org.elasticsearch.search.aggregations.InternalAggregation.Type;
import org.elasticsearch.search.aggregations.bucket.filters.FiltersAggregator.KeyedFilter;
import org.elasticsearch.search.internal.SearchContext;
@ -43,7 +42,6 @@ import java.util.Objects;
public class FiltersAggregationBuilder extends AbstractAggregationBuilder<FiltersAggregationBuilder> {
public static final String NAME = "filters";
private static final Type TYPE = new Type(NAME);
private static final ParseField FILTERS_FIELD = new ParseField("filters");
private static final ParseField OTHER_BUCKET_FIELD = new ParseField("other_bucket");
@ -65,7 +63,7 @@ public class FiltersAggregationBuilder extends AbstractAggregationBuilder<Filter
}
private FiltersAggregationBuilder(String name, List<KeyedFilter> filters) {
super(name, TYPE);
super(name);
// internally we want to have a fixed order of filters, regardless of the order of the filters in the request
this.filters = new ArrayList<>(filters);
Collections.sort(this.filters, (KeyedFilter kf1, KeyedFilter kf2) -> kf1.key().compareTo(kf2.key()));
@ -79,7 +77,7 @@ public class FiltersAggregationBuilder extends AbstractAggregationBuilder<Filter
* the filters to use with this aggregation
*/
public FiltersAggregationBuilder(String name, QueryBuilder... filters) {
super(name, TYPE);
super(name);
List<KeyedFilter> keyedFilters = new ArrayList<>(filters.length);
for (int i = 0; i < filters.length; i++) {
keyedFilters.add(new KeyedFilter(String.valueOf(i), filters[i]));
@ -92,7 +90,7 @@ public class FiltersAggregationBuilder extends AbstractAggregationBuilder<Filter
* Read from a stream.
*/
public FiltersAggregationBuilder(StreamInput in) throws IOException {
super(in, TYPE);
super(in);
keyed = in.readBoolean();
int filtersSize = in.readVInt();
filters = new ArrayList<>(filtersSize);
@ -176,7 +174,7 @@ public class FiltersAggregationBuilder extends AbstractAggregationBuilder<Filter
rewrittenFilters.add(new KeyedFilter(kf.key(), QueryBuilder.rewriteQuery(kf.filter(),
context.getQueryShardContext())));
}
return new FiltersAggregatorFactory(name, type, rewrittenFilters, keyed, otherBucket, otherBucketKey, context, parent,
return new FiltersAggregatorFactory(name, rewrittenFilters, keyed, otherBucket, otherBucketKey, context, parent,
subFactoriesBuilder, metaData);
}
@ -300,7 +298,7 @@ public class FiltersAggregationBuilder extends AbstractAggregationBuilder<Filter
}
@Override
public String getWriteableName() {
public String getType() {
return NAME;
}
}

View File

@ -25,7 +25,6 @@ import org.apache.lucene.search.Weight;
import org.elasticsearch.search.aggregations.Aggregator;
import org.elasticsearch.search.aggregations.AggregatorFactories;
import org.elasticsearch.search.aggregations.AggregatorFactory;
import org.elasticsearch.search.aggregations.InternalAggregation.Type;
import org.elasticsearch.search.aggregations.bucket.filters.FiltersAggregator.KeyedFilter;
import org.elasticsearch.search.aggregations.pipeline.PipelineAggregator;
import org.elasticsearch.search.internal.SearchContext;
@ -42,10 +41,10 @@ public class FiltersAggregatorFactory extends AggregatorFactory<FiltersAggregato
private final boolean otherBucket;
private final String otherBucketKey;
public FiltersAggregatorFactory(String name, Type type, List<KeyedFilter> filters, boolean keyed, boolean otherBucket,
public FiltersAggregatorFactory(String name, List<KeyedFilter> filters, boolean keyed, boolean otherBucket,
String otherBucketKey, SearchContext context, AggregatorFactory<?> parent, AggregatorFactories.Builder subFactories,
Map<String, Object> metaData) throws IOException {
super(name, type, context, parent, subFactories, metaData);
super(name, context, parent, subFactories, metaData);
this.keyed = keyed;
this.otherBucket = otherBucket;
this.otherBucketKey = otherBucketKey;

View File

@ -35,7 +35,6 @@ import org.elasticsearch.index.fielddata.SortingNumericDocValues;
import org.elasticsearch.index.query.QueryParseContext;
import org.elasticsearch.search.aggregations.AggregatorFactories.Builder;
import org.elasticsearch.search.aggregations.AggregatorFactory;
import org.elasticsearch.search.aggregations.InternalAggregation.Type;
import org.elasticsearch.search.aggregations.bucket.BucketUtils;
import org.elasticsearch.search.aggregations.support.ValueType;
import org.elasticsearch.search.aggregations.support.ValuesSource;
@ -51,7 +50,6 @@ import java.util.Objects;
public class GeoGridAggregationBuilder extends ValuesSourceAggregationBuilder<ValuesSource.GeoPoint, GeoGridAggregationBuilder> {
public static final String NAME = "geohash_grid";
private static final Type TYPE = new Type(NAME);
public static final int DEFAULT_PRECISION = 5;
public static final int DEFAULT_MAX_NUM_CELLS = 10000;
@ -73,14 +71,14 @@ public class GeoGridAggregationBuilder extends ValuesSourceAggregationBuilder<Va
private int shardSize = -1;
public GeoGridAggregationBuilder(String name) {
super(name, TYPE, ValuesSourceType.GEOPOINT, ValueType.GEOPOINT);
super(name, ValuesSourceType.GEOPOINT, ValueType.GEOPOINT);
}
/**
* Read from a stream.
*/
public GeoGridAggregationBuilder(StreamInput in) throws IOException {
super(in, TYPE, ValuesSourceType.GEOPOINT, ValueType.GEOPOINT);
super(in, ValuesSourceType.GEOPOINT, ValueType.GEOPOINT);
precision = in.readVInt();
requiredSize = in.readVInt();
shardSize = in.readVInt();
@ -150,7 +148,7 @@ public class GeoGridAggregationBuilder extends ValuesSourceAggregationBuilder<Va
if (shardSize < requiredSize) {
shardSize = requiredSize;
}
return new GeoHashGridAggregatorFactory(name, type, config, precision, requiredSize, shardSize, context, parent,
return new GeoHashGridAggregatorFactory(name, config, precision, requiredSize, shardSize, context, parent,
subFactoriesBuilder, metaData);
}
@ -185,7 +183,7 @@ public class GeoGridAggregationBuilder extends ValuesSourceAggregationBuilder<Va
}
@Override
public String getWriteableName() {
public String getType() {
return NAME;
}

View File

@ -23,7 +23,6 @@ import org.elasticsearch.search.aggregations.Aggregator;
import org.elasticsearch.search.aggregations.AggregatorFactories;
import org.elasticsearch.search.aggregations.AggregatorFactory;
import org.elasticsearch.search.aggregations.InternalAggregation;
import org.elasticsearch.search.aggregations.InternalAggregation.Type;
import org.elasticsearch.search.aggregations.NonCollectingAggregator;
import org.elasticsearch.search.aggregations.bucket.geogrid.GeoGridAggregationBuilder.CellIdSource;
import org.elasticsearch.search.aggregations.pipeline.PipelineAggregator;
@ -44,10 +43,10 @@ public class GeoHashGridAggregatorFactory extends ValuesSourceAggregatorFactory<
private final int requiredSize;
private final int shardSize;
public GeoHashGridAggregatorFactory(String name, Type type, ValuesSourceConfig<GeoPoint> config, int precision, int requiredSize,
public GeoHashGridAggregatorFactory(String name, ValuesSourceConfig<GeoPoint> config, int precision, int requiredSize,
int shardSize, SearchContext context, AggregatorFactory<?> parent, AggregatorFactories.Builder subFactoriesBuilder,
Map<String, Object> metaData) throws IOException {
super(name, type, config, context, parent, subFactoriesBuilder, metaData);
super(name, config, context, parent, subFactoriesBuilder, metaData);
this.precision = precision;
this.requiredSize = requiredSize;
this.shardSize = shardSize;

View File

@ -26,24 +26,22 @@ import org.elasticsearch.index.query.QueryParseContext;
import org.elasticsearch.search.aggregations.AbstractAggregationBuilder;
import org.elasticsearch.search.aggregations.AggregatorFactories.Builder;
import org.elasticsearch.search.aggregations.AggregatorFactory;
import org.elasticsearch.search.aggregations.InternalAggregation.Type;
import org.elasticsearch.search.internal.SearchContext;
import java.io.IOException;
public class GlobalAggregationBuilder extends AbstractAggregationBuilder<GlobalAggregationBuilder> {
public static final String NAME = "global";
private static final Type TYPE = new Type(NAME);
public GlobalAggregationBuilder(String name) {
super(name, TYPE);
super(name);
}
/**
* Read from a stream.
*/
public GlobalAggregationBuilder(StreamInput in) throws IOException {
super(in, TYPE);
super(in);
}
@Override
@ -54,7 +52,7 @@ public class GlobalAggregationBuilder extends AbstractAggregationBuilder<GlobalA
@Override
protected AggregatorFactory<?> doBuild(SearchContext context, AggregatorFactory<?> parent, Builder subFactoriesBuilder)
throws IOException {
return new GlobalAggregatorFactory(name, type, context, parent, subFactoriesBuilder, metaData);
return new GlobalAggregatorFactory(name, context, parent, subFactoriesBuilder, metaData);
}
@Override
@ -80,7 +78,7 @@ public class GlobalAggregationBuilder extends AbstractAggregationBuilder<GlobalA
}
@Override
public String getWriteableName() {
public String getType() {
return NAME;
}
}

View File

@ -23,7 +23,6 @@ import org.elasticsearch.search.aggregations.AggregationExecutionException;
import org.elasticsearch.search.aggregations.Aggregator;
import org.elasticsearch.search.aggregations.AggregatorFactories;
import org.elasticsearch.search.aggregations.AggregatorFactory;
import org.elasticsearch.search.aggregations.InternalAggregation.Type;
import org.elasticsearch.search.aggregations.pipeline.PipelineAggregator;
import org.elasticsearch.search.internal.SearchContext;
@ -33,9 +32,9 @@ import java.util.Map;
public class GlobalAggregatorFactory extends AggregatorFactory<GlobalAggregatorFactory> {
public GlobalAggregatorFactory(String name, Type type, SearchContext context, AggregatorFactory<?> parent,
public GlobalAggregatorFactory(String name, SearchContext context, AggregatorFactory<?> parent,
AggregatorFactories.Builder subFactories, Map<String, Object> metaData) throws IOException {
super(name, type, context, parent, subFactories, metaData);
super(name, context, parent, subFactories, metaData);
}
@Override

View File

@ -32,7 +32,8 @@ import java.util.Map;
* regardless the query.
*/
public class InternalGlobal extends InternalSingleBucketAggregation implements Global {
InternalGlobal(String name, long docCount, InternalAggregations aggregations, List<PipelineAggregator> pipelineAggregators, Map<String, Object> metaData) {
InternalGlobal(String name, long docCount, InternalAggregations aggregations, List<PipelineAggregator> pipelineAggregators,
Map<String, Object> metaData) {
super(name, docCount, aggregations, pipelineAggregators, metaData);
}

View File

@ -54,7 +54,7 @@ import static java.util.Collections.unmodifiableMap;
*/
public class DateHistogramAggregationBuilder
extends ValuesSourceAggregationBuilder<ValuesSource.Numeric, DateHistogramAggregationBuilder> {
public static final String NAME = InternalDateHistogram.TYPE.name();
public static final String NAME = "date_histogram";
public static final Map<String, DateTimeUnit> DATE_FIELD_UNITS;
@ -131,12 +131,12 @@ public class DateHistogramAggregationBuilder
/** Create a new builder with the given name. */
public DateHistogramAggregationBuilder(String name) {
super(name, InternalDateHistogram.TYPE, ValuesSourceType.NUMERIC, ValueType.DATE);
super(name, ValuesSourceType.NUMERIC, ValueType.DATE);
}
/** Read from a stream, for internal use only. */
public DateHistogramAggregationBuilder(StreamInput in) throws IOException {
super(in, InternalDateHistogram.TYPE, ValuesSourceType.NUMERIC, ValueType.DATE);
super(in, ValuesSourceType.NUMERIC, ValueType.DATE);
if (in.readBoolean()) {
order = InternalOrder.Streams.readOrder(in);
}
@ -315,7 +315,7 @@ public class DateHistogramAggregationBuilder
}
@Override
public String getWriteableName() {
public String getType() {
return NAME;
}
@ -328,7 +328,7 @@ public class DateHistogramAggregationBuilder
// parse any string bounds to longs and round
roundedBounds = this.extendedBounds.parseAndValidate(name, context, config.format()).round(rounding);
}
return new DateHistogramAggregatorFactory(name, type, config, interval, dateHistogramInterval, offset, order, keyed, minDocCount,
return new DateHistogramAggregatorFactory(name, config, interval, dateHistogramInterval, offset, order, keyed, minDocCount,
rounding, roundedBounds, context, parent, subFactoriesBuilder, metaData);
}

View File

@ -23,7 +23,6 @@ import org.elasticsearch.common.rounding.Rounding;
import org.elasticsearch.search.aggregations.Aggregator;
import org.elasticsearch.search.aggregations.AggregatorFactories;
import org.elasticsearch.search.aggregations.AggregatorFactory;
import org.elasticsearch.search.aggregations.InternalAggregation.Type;
import org.elasticsearch.search.aggregations.pipeline.PipelineAggregator;
import org.elasticsearch.search.aggregations.support.ValuesSource;
import org.elasticsearch.search.aggregations.support.ValuesSource.Numeric;
@ -47,11 +46,11 @@ public final class DateHistogramAggregatorFactory
private final ExtendedBounds extendedBounds;
private Rounding rounding;
public DateHistogramAggregatorFactory(String name, Type type, ValuesSourceConfig<Numeric> config, long interval,
public DateHistogramAggregatorFactory(String name, ValuesSourceConfig<Numeric> config, long interval,
DateHistogramInterval dateHistogramInterval, long offset, InternalOrder order, boolean keyed, long minDocCount,
Rounding rounding, ExtendedBounds extendedBounds, SearchContext context, AggregatorFactory<?> parent,
AggregatorFactories.Builder subFactoriesBuilder, Map<String, Object> metaData) throws IOException {
super(name, type, config, context, parent, subFactoriesBuilder, metaData);
super(name, config, context, parent, subFactoriesBuilder, metaData);
this.interval = interval;
this.dateHistogramInterval = dateHistogramInterval;
this.offset = offset;

View File

@ -49,7 +49,7 @@ import java.util.Objects;
*/
public class HistogramAggregationBuilder
extends ValuesSourceAggregationBuilder<ValuesSource.Numeric, HistogramAggregationBuilder> {
public static final String NAME = InternalHistogram.TYPE.name();
public static final String NAME = "histogram";
private static final ObjectParser<double[], ParseFieldMatcherSupplier> EXTENDED_BOUNDS_PARSER = new ObjectParser<>(
Histogram.EXTENDED_BOUNDS_FIELD.getPreferredName(),
@ -94,12 +94,12 @@ public class HistogramAggregationBuilder
/** Create a new builder with the given name. */
public HistogramAggregationBuilder(String name) {
super(name, InternalHistogram.TYPE, ValuesSourceType.NUMERIC, ValueType.DOUBLE);
super(name, ValuesSourceType.NUMERIC, ValueType.DOUBLE);
}
/** Read from a stream, for internal use only. */
public HistogramAggregationBuilder(StreamInput in) throws IOException {
super(in, InternalHistogram.TYPE, ValuesSourceType.NUMERIC, ValueType.DOUBLE);
super(in, ValuesSourceType.NUMERIC, ValueType.DOUBLE);
if (in.readBoolean()) {
order = InternalOrder.Streams.readOrder(in);
}
@ -260,14 +260,14 @@ public class HistogramAggregationBuilder
}
@Override
public String getWriteableName() {
return InternalHistogram.TYPE.name();
public String getType() {
return NAME;
}
@Override
protected ValuesSourceAggregatorFactory<Numeric, ?> innerBuild(SearchContext context, ValuesSourceConfig<Numeric> config,
AggregatorFactory<?> parent, Builder subFactoriesBuilder) throws IOException {
return new HistogramAggregatorFactory(name, type, config, interval, offset, order, keyed, minDocCount, minBound, maxBound,
return new HistogramAggregatorFactory(name, config, interval, offset, order, keyed, minDocCount, minBound, maxBound,
context, parent, subFactoriesBuilder, metaData);
}

View File

@ -22,7 +22,6 @@ package org.elasticsearch.search.aggregations.bucket.histogram;
import org.elasticsearch.search.aggregations.Aggregator;
import org.elasticsearch.search.aggregations.AggregatorFactories;
import org.elasticsearch.search.aggregations.AggregatorFactory;
import org.elasticsearch.search.aggregations.InternalAggregation.Type;
import org.elasticsearch.search.aggregations.pipeline.PipelineAggregator;
import org.elasticsearch.search.aggregations.support.ValuesSource;
import org.elasticsearch.search.aggregations.support.ValuesSource.Numeric;
@ -42,11 +41,11 @@ public final class HistogramAggregatorFactory extends ValuesSourceAggregatorFact
private final long minDocCount;
private final double minBound, maxBound;
HistogramAggregatorFactory(String name, Type type, ValuesSourceConfig<Numeric> config, double interval, double offset,
HistogramAggregatorFactory(String name, ValuesSourceConfig<Numeric> config, double interval, double offset,
InternalOrder order, boolean keyed, long minDocCount, double minBound, double maxBound,
SearchContext context, AggregatorFactory<?> parent,
AggregatorFactories.Builder subFactoriesBuilder, Map<String, Object> metaData) throws IOException {
super(name, type, config, context, parent, subFactoriesBuilder, metaData);
super(name, config, context, parent, subFactoriesBuilder, metaData);
this.interval = interval;
this.offset = offset;
this.order = order;

View File

@ -43,13 +43,11 @@ import java.util.ListIterator;
import java.util.Map;
/**
* Imelementation of {@link Histogram}.
* Implementation of {@link Histogram}.
*/
public final class InternalDateHistogram extends InternalMultiBucketAggregation<InternalDateHistogram, InternalDateHistogram.Bucket>
implements Histogram, HistogramFactory {
static final Type TYPE = new Type("date_histogram");
public static class Bucket extends InternalMultiBucketAggregation.InternalBucket implements Histogram.Bucket {
final long key;

View File

@ -40,13 +40,10 @@ import java.util.ListIterator;
import java.util.Map;
/**
* Imelementation of {@link Histogram}.
* Implementation of {@link Histogram}.
*/
public final class InternalHistogram extends InternalMultiBucketAggregation<InternalHistogram, InternalHistogram.Bucket>
implements Histogram, HistogramFactory {
static final Type TYPE = new Type("histogram");
public static class Bucket extends InternalMultiBucketAggregation.InternalBucket implements Histogram.Bucket {
final double key;

View File

@ -26,7 +26,6 @@ import org.elasticsearch.common.xcontent.XContentBuilder;
import org.elasticsearch.index.query.QueryParseContext;
import org.elasticsearch.search.aggregations.AggregatorFactories.Builder;
import org.elasticsearch.search.aggregations.AggregatorFactory;
import org.elasticsearch.search.aggregations.InternalAggregation.Type;
import org.elasticsearch.search.aggregations.support.ValueType;
import org.elasticsearch.search.aggregations.support.ValuesSource;
import org.elasticsearch.search.aggregations.support.ValuesSourceAggregationBuilder;
@ -40,7 +39,6 @@ import java.io.IOException;
public class MissingAggregationBuilder extends ValuesSourceAggregationBuilder<ValuesSource, MissingAggregationBuilder> {
public static final String NAME = "missing";
public static final Type TYPE = new Type(NAME);
private static final ObjectParser<MissingAggregationBuilder, QueryParseContext> PARSER;
static {
@ -53,14 +51,14 @@ public class MissingAggregationBuilder extends ValuesSourceAggregationBuilder<Va
}
public MissingAggregationBuilder(String name, ValueType targetValueType) {
super(name, TYPE, ValuesSourceType.ANY, targetValueType);
super(name, ValuesSourceType.ANY, targetValueType);
}
/**
* Read from a stream.
*/
public MissingAggregationBuilder(StreamInput in) throws IOException {
super(in, TYPE, ValuesSourceType.ANY);
super(in, ValuesSourceType.ANY);
}
@Override
@ -76,7 +74,7 @@ public class MissingAggregationBuilder extends ValuesSourceAggregationBuilder<Va
@Override
protected ValuesSourceAggregatorFactory<ValuesSource, ?> innerBuild(SearchContext context,
ValuesSourceConfig<ValuesSource> config, AggregatorFactory<?> parent, Builder subFactoriesBuilder) throws IOException {
return new MissingAggregatorFactory(name, type, config, context, parent, subFactoriesBuilder, metaData);
return new MissingAggregatorFactory(name, config, context, parent, subFactoriesBuilder, metaData);
}
@Override
@ -95,7 +93,7 @@ public class MissingAggregationBuilder extends ValuesSourceAggregationBuilder<Va
}
@Override
public String getWriteableName() {
public String getType() {
return NAME;
}
}

View File

@ -22,7 +22,6 @@ package org.elasticsearch.search.aggregations.bucket.missing;
import org.elasticsearch.search.aggregations.Aggregator;
import org.elasticsearch.search.aggregations.AggregatorFactories;
import org.elasticsearch.search.aggregations.AggregatorFactory;
import org.elasticsearch.search.aggregations.InternalAggregation.Type;
import org.elasticsearch.search.aggregations.pipeline.PipelineAggregator;
import org.elasticsearch.search.aggregations.support.ValuesSource;
import org.elasticsearch.search.aggregations.support.ValuesSourceAggregatorFactory;
@ -35,9 +34,9 @@ import java.util.Map;
public class MissingAggregatorFactory extends ValuesSourceAggregatorFactory<ValuesSource, MissingAggregatorFactory> {
public MissingAggregatorFactory(String name, Type type, ValuesSourceConfig<ValuesSource> config, SearchContext context,
public MissingAggregatorFactory(String name, ValuesSourceConfig<ValuesSource> config, SearchContext context,
AggregatorFactory<?> parent, AggregatorFactories.Builder subFactoriesBuilder, Map<String, Object> metaData) throws IOException {
super(name, type, config, context, parent, subFactoriesBuilder, metaData);
super(name, config, context, parent, subFactoriesBuilder, metaData);
}
@Override

View File

@ -30,7 +30,6 @@ import org.elasticsearch.search.aggregations.AbstractAggregationBuilder;
import org.elasticsearch.search.aggregations.AggregationExecutionException;
import org.elasticsearch.search.aggregations.AggregatorFactories.Builder;
import org.elasticsearch.search.aggregations.AggregatorFactory;
import org.elasticsearch.search.aggregations.InternalAggregation.Type;
import org.elasticsearch.search.internal.SearchContext;
import java.io.IOException;
@ -38,7 +37,6 @@ import java.util.Objects;
public class NestedAggregationBuilder extends AbstractAggregationBuilder<NestedAggregationBuilder> {
public static final String NAME = "nested";
private static final Type TYPE = new Type(NAME);
private final String path;
@ -50,7 +48,7 @@ public class NestedAggregationBuilder extends AbstractAggregationBuilder<NestedA
* match the path to a nested object in the mappings.
*/
public NestedAggregationBuilder(String name, String path) {
super(name, TYPE);
super(name);
if (path == null) {
throw new IllegalArgumentException("[path] must not be null: [" + name + "]");
}
@ -61,7 +59,7 @@ public class NestedAggregationBuilder extends AbstractAggregationBuilder<NestedA
* Read from a stream.
*/
public NestedAggregationBuilder(StreamInput in) throws IOException {
super(in, TYPE);
super(in);
path = in.readString();
}
@ -83,7 +81,7 @@ public class NestedAggregationBuilder extends AbstractAggregationBuilder<NestedA
ObjectMapper childObjectMapper = context.getObjectMapper(path);
if (childObjectMapper == null) {
// in case the path has been unmapped:
return new NestedAggregatorFactory(name, type, null, null, context, parent, subFactoriesBuilder, metaData);
return new NestedAggregatorFactory(name, null, null, context, parent, subFactoriesBuilder, metaData);
}
if (childObjectMapper.nested().isNested() == false) {
@ -91,7 +89,7 @@ public class NestedAggregationBuilder extends AbstractAggregationBuilder<NestedA
}
try {
ObjectMapper parentObjectMapper = context.getQueryShardContext().nestedScope().nextLevel(childObjectMapper);
return new NestedAggregatorFactory(name, type, parentObjectMapper, childObjectMapper, context, parent, subFactoriesBuilder,
return new NestedAggregatorFactory(name, parentObjectMapper, childObjectMapper, context, parent, subFactoriesBuilder,
metaData);
} finally {
context.getQueryShardContext().nestedScope().previousLevel();
@ -148,7 +146,7 @@ public class NestedAggregationBuilder extends AbstractAggregationBuilder<NestedA
}
@Override
public String getWriteableName() {
public String getType() {
return NAME;
}
}

View File

@ -24,7 +24,6 @@ import org.elasticsearch.search.aggregations.Aggregator;
import org.elasticsearch.search.aggregations.AggregatorFactories;
import org.elasticsearch.search.aggregations.AggregatorFactory;
import org.elasticsearch.search.aggregations.InternalAggregation;
import org.elasticsearch.search.aggregations.InternalAggregation.Type;
import org.elasticsearch.search.aggregations.NonCollectingAggregator;
import org.elasticsearch.search.aggregations.pipeline.PipelineAggregator;
import org.elasticsearch.search.internal.SearchContext;
@ -38,10 +37,10 @@ public class NestedAggregatorFactory extends AggregatorFactory<NestedAggregatorF
private final ObjectMapper parentObjectMapper;
private final ObjectMapper childObjectMapper;
public NestedAggregatorFactory(String name, Type type, ObjectMapper parentObjectMapper, ObjectMapper childObjectMapper,
public NestedAggregatorFactory(String name, ObjectMapper parentObjectMapper, ObjectMapper childObjectMapper,
SearchContext context, AggregatorFactory<?> parent, AggregatorFactories.Builder subFactories,
Map<String, Object> metaData) throws IOException {
super(name, type, context, parent, subFactories, metaData);
super(name, context, parent, subFactories, metaData);
this.parentObjectMapper = parentObjectMapper;
this.childObjectMapper = childObjectMapper;
}

View File

@ -32,7 +32,6 @@ import org.elasticsearch.search.aggregations.AbstractAggregationBuilder;
import org.elasticsearch.search.aggregations.AggregationExecutionException;
import org.elasticsearch.search.aggregations.AggregatorFactories.Builder;
import org.elasticsearch.search.aggregations.AggregatorFactory;
import org.elasticsearch.search.aggregations.InternalAggregation.Type;
import org.elasticsearch.search.internal.SearchContext;
import java.io.IOException;
@ -40,19 +39,18 @@ import java.util.Objects;
public class ReverseNestedAggregationBuilder extends AbstractAggregationBuilder<ReverseNestedAggregationBuilder> {
public static final String NAME = "reverse_nested";
private static final Type TYPE = new Type(NAME);
private String path;
public ReverseNestedAggregationBuilder(String name) {
super(name, TYPE);
super(name);
}
/**
* Read from a stream.
*/
public ReverseNestedAggregationBuilder(StreamInput in) throws IOException {
super(in, TYPE);
super(in);
path = in.readOptionalString();
}
@ -93,7 +91,7 @@ public class ReverseNestedAggregationBuilder extends AbstractAggregationBuilder<
if (path != null) {
parentObjectMapper = context.getObjectMapper(path);
if (parentObjectMapper == null) {
return new ReverseNestedAggregatorFactory(name, type, true, null, context, parent, subFactoriesBuilder, metaData);
return new ReverseNestedAggregatorFactory(name, true, null, context, parent, subFactoriesBuilder, metaData);
}
if (parentObjectMapper.nested().isNested() == false) {
throw new AggregationExecutionException("[reverse_nested] nested path [" + path + "] is not nested");
@ -103,7 +101,7 @@ public class ReverseNestedAggregationBuilder extends AbstractAggregationBuilder<
NestedScope nestedScope = context.getQueryShardContext().nestedScope();
try {
nestedScope.nextLevel(parentObjectMapper);
return new ReverseNestedAggregatorFactory(name, type, false, parentObjectMapper, context, parent, subFactoriesBuilder,
return new ReverseNestedAggregatorFactory(name, false, parentObjectMapper, context, parent, subFactoriesBuilder,
metaData);
} finally {
nestedScope.previousLevel();
@ -172,7 +170,7 @@ public class ReverseNestedAggregationBuilder extends AbstractAggregationBuilder<
}
@Override
public String getWriteableName() {
public String getType() {
return NAME;
}
}

View File

@ -24,7 +24,6 @@ import org.elasticsearch.search.aggregations.Aggregator;
import org.elasticsearch.search.aggregations.AggregatorFactories;
import org.elasticsearch.search.aggregations.AggregatorFactory;
import org.elasticsearch.search.aggregations.InternalAggregation;
import org.elasticsearch.search.aggregations.InternalAggregation.Type;
import org.elasticsearch.search.aggregations.NonCollectingAggregator;
import org.elasticsearch.search.aggregations.pipeline.PipelineAggregator;
import org.elasticsearch.search.internal.SearchContext;
@ -38,11 +37,11 @@ public class ReverseNestedAggregatorFactory extends AggregatorFactory<ReverseNes
private final boolean unmapped;
private final ObjectMapper parentObjectMapper;
public ReverseNestedAggregatorFactory(String name, Type type, boolean unmapped, ObjectMapper parentObjectMapper,
public ReverseNestedAggregatorFactory(String name, boolean unmapped, ObjectMapper parentObjectMapper,
SearchContext context, AggregatorFactory<?> parent,
AggregatorFactories.Builder subFactories,
Map<String, Object> metaData) throws IOException {
super(name, type, context, parent, subFactories, metaData);
super(name, context, parent, subFactories, metaData);
this.unmapped = unmapped;
this.parentObjectMapper = parentObjectMapper;
}

View File

@ -22,7 +22,6 @@ package org.elasticsearch.search.aggregations.bucket.range;
import org.elasticsearch.search.aggregations.Aggregator;
import org.elasticsearch.search.aggregations.AggregatorFactories;
import org.elasticsearch.search.aggregations.AggregatorFactory;
import org.elasticsearch.search.aggregations.InternalAggregation.Type;
import org.elasticsearch.search.aggregations.bucket.range.RangeAggregator.Range;
import org.elasticsearch.search.aggregations.bucket.range.RangeAggregator.Unmapped;
import org.elasticsearch.search.aggregations.pipeline.PipelineAggregator;
@ -43,10 +42,10 @@ public class AbstractRangeAggregatorFactory<AF extends AbstractRangeAggregatorFa
private final R[] ranges;
private final boolean keyed;
public AbstractRangeAggregatorFactory(String name, Type type, ValuesSourceConfig<Numeric> config, R[] ranges, boolean keyed,
public AbstractRangeAggregatorFactory(String name, ValuesSourceConfig<Numeric> config, R[] ranges, boolean keyed,
InternalRange.Factory<?, ?> rangeFactory, SearchContext context, AggregatorFactory<?> parent,
AggregatorFactories.Builder subFactoriesBuilder, Map<String, Object> metaData) throws IOException {
super(name, type, config, context, parent, subFactoriesBuilder, metaData);
super(name, config, context, parent, subFactoriesBuilder, metaData);
this.ranges = ranges;
this.keyed = keyed;
this.rangeFactory = rangeFactory;

View File

@ -44,7 +44,7 @@ public abstract class AbstractRangeBuilder<AB extends AbstractRangeBuilder<AB, R
protected boolean keyed = false;
protected AbstractRangeBuilder(String name, InternalRange.Factory<?, ?> rangeFactory) {
super(name, rangeFactory.type(), rangeFactory.getValueSourceType(), rangeFactory.getValueType());
super(name, rangeFactory.getValueSourceType(), rangeFactory.getValueType());
this.rangeFactory = rangeFactory;
}
@ -53,7 +53,7 @@ public abstract class AbstractRangeBuilder<AB extends AbstractRangeBuilder<AB, R
*/
protected AbstractRangeBuilder(StreamInput in, InternalRange.Factory<?, ?> rangeFactory, Writeable.Reader<R> rangeReader)
throws IOException {
super(in, rangeFactory.type(), rangeFactory.getValueSourceType(), rangeFactory.getValueType());
super(in, rangeFactory.getValueSourceType(), rangeFactory.getValueType());
this.rangeFactory = rangeFactory;
ranges = in.readList(rangeReader);
keyed = in.readBoolean();

View File

@ -18,33 +18,32 @@
*/
package org.elasticsearch.search.aggregations.bucket.range;
import java.io.IOException;
import java.util.List;
import java.util.Map;
import org.elasticsearch.search.aggregations.Aggregator;
import org.elasticsearch.search.aggregations.AggregatorFactories.Builder;
import org.elasticsearch.search.aggregations.AggregatorFactory;
import org.elasticsearch.search.aggregations.InternalAggregation.Type;
import org.elasticsearch.search.aggregations.pipeline.PipelineAggregator;
import org.elasticsearch.search.aggregations.support.ValuesSource;
import org.elasticsearch.search.aggregations.support.ValuesSourceAggregatorFactory;
import org.elasticsearch.search.aggregations.support.ValuesSourceConfig;
import org.elasticsearch.search.internal.SearchContext;
import java.io.IOException;
import java.util.List;
import java.util.Map;
public class BinaryRangeAggregatorFactory
extends ValuesSourceAggregatorFactory<ValuesSource.Bytes, BinaryRangeAggregatorFactory> {
private final List<BinaryRangeAggregator.Range> ranges;
private final boolean keyed;
public BinaryRangeAggregatorFactory(String name, Type type,
public BinaryRangeAggregatorFactory(String name,
ValuesSourceConfig<ValuesSource.Bytes> config,
List<BinaryRangeAggregator.Range> ranges, boolean keyed,
SearchContext context,
AggregatorFactory<?> parent, Builder subFactoriesBuilder,
Map<String, Object> metaData) throws IOException {
super(name, type, config, context, parent, subFactoriesBuilder, metaData);
super(name, config, context, parent, subFactoriesBuilder, metaData);
this.ranges = ranges;
this.keyed = keyed;
}

View File

@ -175,10 +175,6 @@ public class InternalRange<B extends InternalRange.Bucket, R extends InternalRan
}
public static class Factory<B extends Bucket, R extends InternalRange<B, R>> {
public Type type() {
return RangeAggregationBuilder.TYPE;
}
public ValuesSourceType getValueSourceType() {
return ValuesSourceType.NUMERIC;
}

View File

@ -23,10 +23,9 @@ import org.elasticsearch.common.io.stream.StreamInput;
import org.elasticsearch.common.xcontent.ObjectParser;
import org.elasticsearch.common.xcontent.XContentParser;
import org.elasticsearch.index.query.QueryParseContext;
import org.elasticsearch.search.aggregations.AggregatorFactories.Builder;
import org.elasticsearch.search.aggregations.AggregationBuilder;
import org.elasticsearch.search.aggregations.AggregatorFactories.Builder;
import org.elasticsearch.search.aggregations.AggregatorFactory;
import org.elasticsearch.search.aggregations.InternalAggregation.Type;
import org.elasticsearch.search.aggregations.bucket.range.RangeAggregator.Range;
import org.elasticsearch.search.aggregations.support.ValuesSource.Numeric;
import org.elasticsearch.search.aggregations.support.ValuesSourceConfig;
@ -37,7 +36,6 @@ import java.io.IOException;
public class RangeAggregationBuilder extends AbstractRangeBuilder<RangeAggregationBuilder, Range> {
public static final String NAME = "range";
static final Type TYPE = new Type(NAME);
private static final ObjectParser<RangeAggregationBuilder, QueryParseContext> PARSER;
static {
@ -142,12 +140,12 @@ public class RangeAggregationBuilder extends AbstractRangeBuilder<RangeAggregati
AggregatorFactory<?> parent, Builder subFactoriesBuilder) throws IOException {
// We need to call processRanges here so they are parsed before we make the decision of whether to cache the request
Range[] ranges = processRanges(context, config);
return new RangeAggregatorFactory(name, type, config, ranges, keyed, rangeFactory, context, parent, subFactoriesBuilder,
return new RangeAggregatorFactory(name, config, ranges, keyed, rangeFactory, context, parent, subFactoriesBuilder,
metaData);
}
@Override
public String getWriteableName() {
public String getType() {
return NAME;
}
}

View File

@ -21,7 +21,6 @@ package org.elasticsearch.search.aggregations.bucket.range;
import org.elasticsearch.search.aggregations.AggregatorFactories;
import org.elasticsearch.search.aggregations.AggregatorFactory;
import org.elasticsearch.search.aggregations.InternalAggregation.Type;
import org.elasticsearch.search.aggregations.bucket.range.InternalRange.Factory;
import org.elasticsearch.search.aggregations.bucket.range.RangeAggregator.Range;
import org.elasticsearch.search.aggregations.support.ValuesSource.Numeric;
@ -33,10 +32,10 @@ import java.util.Map;
public class RangeAggregatorFactory extends AbstractRangeAggregatorFactory<RangeAggregatorFactory, RangeAggregator.Range> {
public RangeAggregatorFactory(String name, Type type, ValuesSourceConfig<Numeric> config, Range[] ranges, boolean keyed,
public RangeAggregatorFactory(String name, ValuesSourceConfig<Numeric> config, Range[] ranges, boolean keyed,
Factory<?, ?> rangeFactory, SearchContext context, AggregatorFactory<?> parent,
AggregatorFactories.Builder subFactoriesBuilder, Map<String, Object> metaData) throws IOException {
super(name, type, config, ranges, keyed, rangeFactory, context, parent, subFactoriesBuilder, metaData);
super(name, config, ranges, keyed, rangeFactory, context, parent, subFactoriesBuilder, metaData);
}
}

View File

@ -23,10 +23,9 @@ import org.elasticsearch.common.io.stream.StreamInput;
import org.elasticsearch.common.xcontent.ObjectParser;
import org.elasticsearch.common.xcontent.XContentParser;
import org.elasticsearch.index.query.QueryParseContext;
import org.elasticsearch.search.aggregations.AggregatorFactories.Builder;
import org.elasticsearch.search.aggregations.AggregationBuilder;
import org.elasticsearch.search.aggregations.AggregatorFactories.Builder;
import org.elasticsearch.search.aggregations.AggregatorFactory;
import org.elasticsearch.search.aggregations.InternalAggregation.Type;
import org.elasticsearch.search.aggregations.bucket.range.AbstractRangeBuilder;
import org.elasticsearch.search.aggregations.bucket.range.RangeAggregator;
import org.elasticsearch.search.aggregations.bucket.range.RangeAggregator.Range;
@ -40,7 +39,6 @@ import java.io.IOException;
public class DateRangeAggregationBuilder extends AbstractRangeBuilder<DateRangeAggregationBuilder, RangeAggregator.Range> {
public static final String NAME = "date_range";
static final Type TYPE = new Type(NAME);
private static final ObjectParser<DateRangeAggregationBuilder, QueryParseContext> PARSER;
static {
@ -75,7 +73,7 @@ public class DateRangeAggregationBuilder extends AbstractRangeBuilder<DateRangeA
}
@Override
public String getWriteableName() {
public String getType() {
return NAME;
}
@ -288,7 +286,7 @@ public class DateRangeAggregationBuilder extends AbstractRangeBuilder<DateRangeA
// We need to call processRanges here so they are parsed and we know whether `now` has been used before we make
// the decision of whether to cache the request
Range[] ranges = processRanges(context, config);
return new DateRangeAggregatorFactory(name, type, config, ranges, keyed, rangeFactory, context, parent, subFactoriesBuilder,
return new DateRangeAggregatorFactory(name, config, ranges, keyed, rangeFactory, context, parent, subFactoriesBuilder,
metaData);
}
}

View File

@ -21,7 +21,6 @@ package org.elasticsearch.search.aggregations.bucket.range.date;
import org.elasticsearch.search.aggregations.AggregatorFactories;
import org.elasticsearch.search.aggregations.AggregatorFactory;
import org.elasticsearch.search.aggregations.InternalAggregation.Type;
import org.elasticsearch.search.aggregations.bucket.range.AbstractRangeAggregatorFactory;
import org.elasticsearch.search.aggregations.bucket.range.InternalRange.Factory;
import org.elasticsearch.search.aggregations.bucket.range.RangeAggregator.Range;
@ -34,10 +33,10 @@ import java.util.Map;
public class DateRangeAggregatorFactory extends AbstractRangeAggregatorFactory<DateRangeAggregatorFactory, Range> {
public DateRangeAggregatorFactory(String name, Type type, ValuesSourceConfig<Numeric> config, Range[] ranges, boolean keyed,
public DateRangeAggregatorFactory(String name, ValuesSourceConfig<Numeric> config, Range[] ranges, boolean keyed,
Factory<?, ?> rangeFactory, SearchContext context, AggregatorFactory<?> parent,
AggregatorFactories.Builder subFactoriesBuilder, Map<String, Object> metaData) throws IOException {
super(name, type, config, ranges, keyed, rangeFactory, context, parent, subFactoriesBuilder, metaData);
super(name, config, ranges, keyed, rangeFactory, context, parent, subFactoriesBuilder, metaData);
}
}

View File

@ -84,11 +84,6 @@ public class InternalDateRange extends InternalRange<InternalDateRange.Bucket, I
}
public static class Factory extends InternalRange.Factory<InternalDateRange.Bucket, InternalDateRange> {
@Override
public Type type() {
return DateRangeAggregationBuilder.TYPE;
}
@Override
public ValueType getValueType() {
return ValueType.DATE;

View File

@ -34,7 +34,6 @@ import org.elasticsearch.index.query.QueryParseContext;
import org.elasticsearch.search.aggregations.AggregationBuilder;
import org.elasticsearch.search.aggregations.AggregatorFactories.Builder;
import org.elasticsearch.search.aggregations.AggregatorFactory;
import org.elasticsearch.search.aggregations.InternalAggregation.Type;
import org.elasticsearch.search.aggregations.bucket.range.InternalRange;
import org.elasticsearch.search.aggregations.bucket.range.RangeAggregator;
import org.elasticsearch.search.aggregations.support.ValuesSource;
@ -51,7 +50,6 @@ import java.util.Objects;
public class GeoDistanceAggregationBuilder extends ValuesSourceAggregationBuilder<ValuesSource.GeoPoint, GeoDistanceAggregationBuilder> {
public static final String NAME = "geo_distance";
public static final Type TYPE = new Type(NAME);
static final ParseField ORIGIN_FIELD = new ParseField("origin", "center", "point", "por");
static final ParseField UNIT_FIELD = new ParseField("unit");
static final ParseField DISTANCE_TYPE_FIELD = new ParseField("distance_type");
@ -215,7 +213,7 @@ public class GeoDistanceAggregationBuilder extends ValuesSourceAggregationBuilde
private GeoDistanceAggregationBuilder(String name, GeoPoint origin,
InternalRange.Factory<InternalGeoDistance.Bucket, InternalGeoDistance> rangeFactory) {
super(name, rangeFactory.type(), rangeFactory.getValueSourceType(), rangeFactory.getValueType());
super(name, rangeFactory.getValueSourceType(), rangeFactory.getValueType());
this.origin = origin;
}
@ -223,8 +221,7 @@ public class GeoDistanceAggregationBuilder extends ValuesSourceAggregationBuilde
* Read from a stream.
*/
public GeoDistanceAggregationBuilder(StreamInput in) throws IOException {
super(in, InternalGeoDistance.FACTORY.type(), InternalGeoDistance.FACTORY.getValueSourceType(),
InternalGeoDistance.FACTORY.getValueType());
super(in, InternalGeoDistance.FACTORY.getValueSourceType(), InternalGeoDistance.FACTORY.getValueType());
origin = new GeoPoint(in.readDouble(), in.readDouble());
int size = in.readVInt();
ranges = new ArrayList<>(size);
@ -345,7 +342,7 @@ public class GeoDistanceAggregationBuilder extends ValuesSourceAggregationBuilde
}
@Override
public String getWriteableName() {
public String getType() {
return NAME;
}
@ -387,7 +384,7 @@ public class GeoDistanceAggregationBuilder extends ValuesSourceAggregationBuilde
ValuesSourceConfig<ValuesSource.GeoPoint> config, AggregatorFactory<?> parent, Builder subFactoriesBuilder)
throws IOException {
Range[] ranges = this.ranges.toArray(new Range[this.range().size()]);
return new GeoDistanceRangeAggregatorFactory(name, type, config, origin, ranges, unit, distanceType, keyed, context, parent,
return new GeoDistanceRangeAggregatorFactory(name, config, origin, ranges, unit, distanceType, keyed, context, parent,
subFactoriesBuilder, metaData);
}

View File

@ -31,7 +31,6 @@ import org.elasticsearch.index.fielddata.SortedNumericDoubleValues;
import org.elasticsearch.search.aggregations.Aggregator;
import org.elasticsearch.search.aggregations.AggregatorFactories;
import org.elasticsearch.search.aggregations.AggregatorFactory;
import org.elasticsearch.search.aggregations.InternalAggregation.Type;
import org.elasticsearch.search.aggregations.bucket.range.InternalRange;
import org.elasticsearch.search.aggregations.bucket.range.RangeAggregator;
import org.elasticsearch.search.aggregations.bucket.range.RangeAggregator.Unmapped;
@ -56,10 +55,10 @@ public class GeoDistanceRangeAggregatorFactory
private final GeoDistance distanceType;
private final boolean keyed;
public GeoDistanceRangeAggregatorFactory(String name, Type type, ValuesSourceConfig<ValuesSource.GeoPoint> config, GeoPoint origin,
public GeoDistanceRangeAggregatorFactory(String name, ValuesSourceConfig<ValuesSource.GeoPoint> config, GeoPoint origin,
Range[] ranges, DistanceUnit unit, GeoDistance distanceType, boolean keyed, SearchContext context,
AggregatorFactory<?> parent, AggregatorFactories.Builder subFactoriesBuilder, Map<String, Object> metaData) throws IOException {
super(name, type, config, context, parent, subFactoriesBuilder, metaData);
super(name, config, context, parent, subFactoriesBuilder, metaData);
this.origin = origin;
this.ranges = ranges;
this.unit = unit;

View File

@ -59,11 +59,6 @@ public class InternalGeoDistance extends InternalRange<InternalGeoDistance.Bucke
}
public static class Factory extends InternalRange.Factory<InternalGeoDistance.Bucket, InternalGeoDistance> {
@Override
public Type type() {
return GeoDistanceAggregationBuilder.TYPE;
}
@Override
public ValuesSourceType getValueSourceType() {
return ValuesSourceType.GEOPOINT;

View File

@ -35,7 +35,6 @@ import org.elasticsearch.script.Script;
import org.elasticsearch.search.aggregations.AggregationBuilder;
import org.elasticsearch.search.aggregations.AggregatorFactories.Builder;
import org.elasticsearch.search.aggregations.AggregatorFactory;
import org.elasticsearch.search.aggregations.InternalAggregation;
import org.elasticsearch.search.aggregations.bucket.range.BinaryRangeAggregator;
import org.elasticsearch.search.aggregations.bucket.range.BinaryRangeAggregatorFactory;
import org.elasticsearch.search.aggregations.bucket.range.RangeAggregator;
@ -60,7 +59,6 @@ import java.util.Objects;
public final class IpRangeAggregationBuilder
extends ValuesSourceAggregationBuilder<ValuesSource.Bytes, IpRangeAggregationBuilder> {
public static final String NAME = "ip_range";
private static final InternalAggregation.Type TYPE = new InternalAggregation.Type(NAME);
private static final ParseField MASK_FIELD = new ParseField("mask");
private static final ObjectParser<IpRangeAggregationBuilder, QueryParseContext> PARSER;
@ -233,11 +231,11 @@ public final class IpRangeAggregationBuilder
private List<Range> ranges = new ArrayList<>();
public IpRangeAggregationBuilder(String name) {
super(name, TYPE, ValuesSourceType.BYTES, ValueType.IP);
super(name, ValuesSourceType.BYTES, ValueType.IP);
}
@Override
public String getWriteableName() {
public String getType() {
return NAME;
}
@ -339,7 +337,7 @@ public final class IpRangeAggregationBuilder
}
public IpRangeAggregationBuilder(StreamInput in) throws IOException {
super(in, TYPE, ValuesSourceType.BYTES, ValueType.IP);
super(in, ValuesSourceType.BYTES, ValueType.IP);
final int numRanges = in.readVInt();
for (int i = 0; i < numRanges; ++i) {
addRange(new Range(in));
@ -374,7 +372,7 @@ public final class IpRangeAggregationBuilder
for (Range range : this.ranges) {
ranges.add(new BinaryRangeAggregator.Range(range.key, toBytesRef(range.from), toBytesRef(range.to)));
}
return new BinaryRangeAggregatorFactory(name, TYPE, config, ranges,
return new BinaryRangeAggregatorFactory(name, config, ranges,
keyed, context, parent, subFactoriesBuilder, metaData);
}

View File

@ -24,10 +24,9 @@ import org.elasticsearch.common.io.stream.StreamOutput;
import org.elasticsearch.common.xcontent.ObjectParser;
import org.elasticsearch.common.xcontent.XContentBuilder;
import org.elasticsearch.index.query.QueryParseContext;
import org.elasticsearch.search.aggregations.AggregatorFactories.Builder;
import org.elasticsearch.search.aggregations.AggregationBuilder;
import org.elasticsearch.search.aggregations.AggregatorFactories.Builder;
import org.elasticsearch.search.aggregations.AggregatorFactory;
import org.elasticsearch.search.aggregations.InternalAggregation.Type;
import org.elasticsearch.search.aggregations.support.ValuesSource;
import org.elasticsearch.search.aggregations.support.ValuesSourceAggregationBuilder;
import org.elasticsearch.search.aggregations.support.ValuesSourceAggregatorFactory;
@ -41,7 +40,6 @@ import java.util.Objects;
public class DiversifiedAggregationBuilder extends ValuesSourceAggregationBuilder<ValuesSource, DiversifiedAggregationBuilder> {
public static final String NAME = "diversified_sampler";
public static final Type TYPE = new Type(NAME);
public static final int MAX_DOCS_PER_VALUE_DEFAULT = 1;
@ -63,14 +61,14 @@ public class DiversifiedAggregationBuilder extends ValuesSourceAggregationBuilde
private String executionHint = null;
public DiversifiedAggregationBuilder(String name) {
super(name, TYPE, ValuesSourceType.ANY, null);
super(name, ValuesSourceType.ANY, null);
}
/**
* Read from a stream.
*/
public DiversifiedAggregationBuilder(StreamInput in) throws IOException {
super(in, TYPE, ValuesSourceType.ANY, null);
super(in, ValuesSourceType.ANY, null);
shardSize = in.readVInt();
maxDocsPerValue = in.readVInt();
executionHint = in.readOptionalString();
@ -139,7 +137,7 @@ public class DiversifiedAggregationBuilder extends ValuesSourceAggregationBuilde
@Override
protected ValuesSourceAggregatorFactory<ValuesSource, ?> innerBuild(SearchContext context,
ValuesSourceConfig<ValuesSource> config, AggregatorFactory<?> parent, Builder subFactoriesBuilder) throws IOException {
return new DiversifiedAggregatorFactory(name, TYPE, config, shardSize, maxDocsPerValue, executionHint, context, parent,
return new DiversifiedAggregatorFactory(name, config, shardSize, maxDocsPerValue, executionHint, context, parent,
subFactoriesBuilder, metaData);
}
@ -167,7 +165,7 @@ public class DiversifiedAggregationBuilder extends ValuesSourceAggregationBuilde
}
@Override
public String getWriteableName() {
public String getType() {
return NAME;
}
}

View File

@ -24,7 +24,6 @@ import org.elasticsearch.search.aggregations.Aggregator;
import org.elasticsearch.search.aggregations.AggregatorFactories;
import org.elasticsearch.search.aggregations.AggregatorFactory;
import org.elasticsearch.search.aggregations.InternalAggregation;
import org.elasticsearch.search.aggregations.InternalAggregation.Type;
import org.elasticsearch.search.aggregations.NonCollectingAggregator;
import org.elasticsearch.search.aggregations.bucket.sampler.SamplerAggregator.ExecutionMode;
import org.elasticsearch.search.aggregations.pipeline.PipelineAggregator;
@ -44,10 +43,10 @@ public class DiversifiedAggregatorFactory extends ValuesSourceAggregatorFactory<
private final int maxDocsPerValue;
private final String executionHint;
public DiversifiedAggregatorFactory(String name, Type type, ValuesSourceConfig<ValuesSource> config, int shardSize, int maxDocsPerValue,
public DiversifiedAggregatorFactory(String name, ValuesSourceConfig<ValuesSource> config, int shardSize, int maxDocsPerValue,
String executionHint, SearchContext context, AggregatorFactory<?> parent, AggregatorFactories.Builder subFactoriesBuilder,
Map<String, Object> metaData) throws IOException {
super(name, type, config, context, parent, subFactoriesBuilder, metaData);
super(name, config, context, parent, subFactoriesBuilder, metaData);
this.shardSize = shardSize;
this.maxDocsPerValue = maxDocsPerValue;
this.executionHint = executionHint;

View File

@ -28,7 +28,6 @@ import org.elasticsearch.index.query.QueryParseContext;
import org.elasticsearch.search.aggregations.AbstractAggregationBuilder;
import org.elasticsearch.search.aggregations.AggregatorFactories.Builder;
import org.elasticsearch.search.aggregations.AggregatorFactory;
import org.elasticsearch.search.aggregations.InternalAggregation.Type;
import org.elasticsearch.search.internal.SearchContext;
import java.io.IOException;
@ -36,21 +35,20 @@ import java.util.Objects;
public class SamplerAggregationBuilder extends AbstractAggregationBuilder<SamplerAggregationBuilder> {
public static final String NAME = "sampler";
private static final Type TYPE = new Type(NAME);
public static final int DEFAULT_SHARD_SAMPLE_SIZE = 100;
private int shardSize = DEFAULT_SHARD_SAMPLE_SIZE;
public SamplerAggregationBuilder(String name) {
super(name, TYPE);
super(name);
}
/**
* Read from a stream.
*/
public SamplerAggregationBuilder(StreamInput in) throws IOException {
super(in, TYPE);
super(in);
shardSize = in.readVInt();
}
@ -77,7 +75,7 @@ public class SamplerAggregationBuilder extends AbstractAggregationBuilder<Sample
@Override
protected SamplerAggregatorFactory doBuild(SearchContext context, AggregatorFactory<?> parent, Builder subFactoriesBuilder)
throws IOException {
return new SamplerAggregatorFactory(name, type, shardSize, context, parent, subFactoriesBuilder, metaData);
return new SamplerAggregatorFactory(name, shardSize, context, parent, subFactoriesBuilder, metaData);
}
@Override
@ -129,7 +127,7 @@ public class SamplerAggregationBuilder extends AbstractAggregationBuilder<Sample
}
@Override
public String getWriteableName() {
public String getType() {
return NAME;
}
}

View File

@ -22,7 +22,6 @@ package org.elasticsearch.search.aggregations.bucket.sampler;
import org.elasticsearch.search.aggregations.Aggregator;
import org.elasticsearch.search.aggregations.AggregatorFactories;
import org.elasticsearch.search.aggregations.AggregatorFactory;
import org.elasticsearch.search.aggregations.InternalAggregation.Type;
import org.elasticsearch.search.aggregations.pipeline.PipelineAggregator;
import org.elasticsearch.search.internal.SearchContext;
@ -34,9 +33,9 @@ public class SamplerAggregatorFactory extends AggregatorFactory<SamplerAggregato
private final int shardSize;
public SamplerAggregatorFactory(String name, Type type, int shardSize, SearchContext context, AggregatorFactory<?> parent,
public SamplerAggregatorFactory(String name, int shardSize, SearchContext context, AggregatorFactory<?> parent,
AggregatorFactories.Builder subFactories, Map<String, Object> metaData) throws IOException {
super(name, type, context, parent, subFactories, metaData);
super(name, context, parent, subFactories, metaData);
this.shardSize = shardSize;
}

View File

@ -26,12 +26,10 @@ import org.elasticsearch.common.xcontent.ParseFieldRegistry;
import org.elasticsearch.common.xcontent.XContentBuilder;
import org.elasticsearch.index.query.QueryBuilder;
import org.elasticsearch.index.query.QueryParseContext;
import org.elasticsearch.search.aggregations.AggregatorFactories.Builder;
import org.elasticsearch.search.aggregations.AggregationBuilder;
import org.elasticsearch.search.aggregations.Aggregator;
import org.elasticsearch.search.aggregations.AggregatorFactories.Builder;
import org.elasticsearch.search.aggregations.AggregatorFactory;
import org.elasticsearch.search.aggregations.InternalAggregation;
import org.elasticsearch.search.aggregations.InternalAggregation.Type;
import org.elasticsearch.search.aggregations.bucket.significant.heuristics.JLHScore;
import org.elasticsearch.search.aggregations.bucket.significant.heuristics.SignificanceHeuristic;
import org.elasticsearch.search.aggregations.bucket.significant.heuristics.SignificanceHeuristicParser;
@ -53,7 +51,6 @@ import java.util.Objects;
public class SignificantTermsAggregationBuilder extends ValuesSourceAggregationBuilder<ValuesSource, SignificantTermsAggregationBuilder> {
public static final String NAME = "significant_terms";
public static final InternalAggregation.Type TYPE = new Type(NAME);
static final ParseField BACKGROUND_FILTER = new ParseField("background_filter");
static final ParseField HEURISTIC = new ParseField("significance_heuristic");
@ -111,14 +108,14 @@ public class SignificantTermsAggregationBuilder extends ValuesSourceAggregationB
private SignificanceHeuristic significanceHeuristic = DEFAULT_SIGNIFICANCE_HEURISTIC;
public SignificantTermsAggregationBuilder(String name, ValueType valueType) {
super(name, TYPE, ValuesSourceType.ANY, valueType);
super(name, ValuesSourceType.ANY, valueType);
}
/**
* Read from a Stream.
*/
public SignificantTermsAggregationBuilder(StreamInput in) throws IOException {
super(in, TYPE, ValuesSourceType.ANY);
super(in, ValuesSourceType.ANY);
bucketCountThresholds = new BucketCountThresholds(in);
executionHint = in.readOptionalString();
filterBuilder = in.readOptionalNamedWriteable(QueryBuilder.class);
@ -267,7 +264,7 @@ public class SignificantTermsAggregationBuilder extends ValuesSourceAggregationB
protected ValuesSourceAggregatorFactory<ValuesSource, ?> innerBuild(SearchContext context, ValuesSourceConfig<ValuesSource> config,
AggregatorFactory<?> parent, Builder subFactoriesBuilder) throws IOException {
SignificanceHeuristic executionHeuristic = this.significanceHeuristic.rewrite(context);
return new SignificantTermsAggregatorFactory(name, type, config, includeExclude, executionHint, filterBuilder,
return new SignificantTermsAggregatorFactory(name, config, includeExclude, executionHint, filterBuilder,
bucketCountThresholds, executionHeuristic, context, parent, subFactoriesBuilder, metaData);
}
@ -303,7 +300,7 @@ public class SignificantTermsAggregationBuilder extends ValuesSourceAggregationB
}
@Override
public String getWriteableName() {
public String getType() {
return NAME;
}
}

View File

@ -40,7 +40,6 @@ import org.elasticsearch.search.aggregations.Aggregator;
import org.elasticsearch.search.aggregations.AggregatorFactories;
import org.elasticsearch.search.aggregations.AggregatorFactory;
import org.elasticsearch.search.aggregations.InternalAggregation;
import org.elasticsearch.search.aggregations.InternalAggregation.Type;
import org.elasticsearch.search.aggregations.NonCollectingAggregator;
import org.elasticsearch.search.aggregations.bucket.BucketUtils;
import org.elasticsearch.search.aggregations.bucket.significant.heuristics.SignificanceHeuristic;
@ -71,11 +70,11 @@ public class SignificantTermsAggregatorFactory extends ValuesSourceAggregatorFac
private final TermsAggregator.BucketCountThresholds bucketCountThresholds;
private final SignificanceHeuristic significanceHeuristic;
public SignificantTermsAggregatorFactory(String name, Type type, ValuesSourceConfig<ValuesSource> config, IncludeExclude includeExclude,
public SignificantTermsAggregatorFactory(String name, ValuesSourceConfig<ValuesSource> config, IncludeExclude includeExclude,
String executionHint, QueryBuilder filterBuilder, TermsAggregator.BucketCountThresholds bucketCountThresholds,
SignificanceHeuristic significanceHeuristic, SearchContext context, AggregatorFactory<?> parent,
AggregatorFactories.Builder subFactoriesBuilder, Map<String, Object> metaData) throws IOException {
super(name, type, config, context, parent, subFactoriesBuilder, metaData);
super(name, config, context, parent, subFactoriesBuilder, metaData);
this.includeExclude = includeExclude;
this.executionHint = executionHint;
this.filter = filterBuilder == null

View File

@ -22,10 +22,12 @@ 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.InternalAggregation;
import org.elasticsearch.search.aggregations.InternalAggregations;
import org.elasticsearch.search.aggregations.pipeline.PipelineAggregator;
import java.io.IOException;
import java.util.ArrayList;
import java.util.List;
import java.util.Map;
@ -152,4 +154,32 @@ public class DoubleTerms extends InternalMappedTerms<DoubleTerms, DoubleTerms.Bu
protected Bucket[] createBucketsArray(int size) {
return new Bucket[size];
}
@Override
public InternalAggregation doReduce(List<InternalAggregation> aggregations, ReduceContext reduceContext) {
boolean promoteToDouble = false;
for (InternalAggregation agg : aggregations) {
if (agg instanceof LongTerms && ((LongTerms) agg).format == DocValueFormat.RAW) {
/**
* this terms agg mixes longs and doubles, we must promote longs to doubles to make the internal aggs
* compatible
*/
promoteToDouble = true;
break;
}
}
if (promoteToDouble == false) {
return super.doReduce(aggregations, reduceContext);
}
List<InternalAggregation> newAggs = new ArrayList<>();
for (InternalAggregation agg : aggregations) {
if (agg instanceof LongTerms) {
DoubleTerms dTerms = LongTerms.convertLongTermsToDouble((LongTerms) agg, format);
newAggs.add(dTerms);
} else {
newAggs.add(agg);
}
}
return newAggs.get(0).doReduce(newAggs, reduceContext);
}
}

View File

@ -22,10 +22,12 @@ 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.InternalAggregation;
import org.elasticsearch.search.aggregations.InternalAggregations;
import org.elasticsearch.search.aggregations.pipeline.PipelineAggregator;
import java.io.IOException;
import java.util.ArrayList;
import java.util.List;
import java.util.Map;
@ -152,4 +154,32 @@ public class LongTerms extends InternalMappedTerms<LongTerms, LongTerms.Bucket>
protected Bucket[] createBucketsArray(int size) {
return new Bucket[size];
}
@Override
public InternalAggregation doReduce(List<InternalAggregation> aggregations, ReduceContext reduceContext) {
for (InternalAggregation agg : aggregations) {
if (agg instanceof DoubleTerms) {
return agg.doReduce(aggregations, reduceContext);
}
}
return super.doReduce(aggregations, reduceContext);
}
/**
* Converts a {@link LongTerms} into a {@link DoubleTerms}, returning the value of the specified long terms as doubles.
*/
static DoubleTerms convertLongTermsToDouble(LongTerms longTerms, DocValueFormat decimalFormat) {
List<Terms.Bucket> buckets = longTerms.getBuckets();
List<DoubleTerms.Bucket> newBuckets = new ArrayList<>();
for (Terms.Bucket bucket : buckets) {
newBuckets.add(new DoubleTerms.Bucket(bucket.getKeyAsNumber().doubleValue(),
bucket.getDocCount(), (InternalAggregations) bucket.getAggregations(), longTerms.showTermDocCountError,
longTerms.showTermDocCountError ? bucket.getDocCountError() : 0, decimalFormat));
}
return new DoubleTerms(longTerms.getName(), longTerms.order, longTerms.requiredSize,
longTerms.minDocCount, longTerms.pipelineAggregators(),
longTerms.metaData, longTerms.format, longTerms.shardSize,
longTerms.showTermDocCountError, longTerms.otherDocCount,
newBuckets, longTerms.docCountError);
}
}

View File

@ -26,12 +26,10 @@ import org.elasticsearch.common.xcontent.ObjectParser;
import org.elasticsearch.common.xcontent.XContentBuilder;
import org.elasticsearch.common.xcontent.XContentParser;
import org.elasticsearch.index.query.QueryParseContext;
import org.elasticsearch.search.aggregations.AggregationBuilder;
import org.elasticsearch.search.aggregations.Aggregator.SubAggCollectionMode;
import org.elasticsearch.search.aggregations.AggregatorFactories.Builder;
import org.elasticsearch.search.aggregations.AggregationBuilder;
import org.elasticsearch.search.aggregations.AggregatorFactory;
import org.elasticsearch.search.aggregations.InternalAggregation;
import org.elasticsearch.search.aggregations.InternalAggregation.Type;
import org.elasticsearch.search.aggregations.bucket.terms.Terms.Order;
import org.elasticsearch.search.aggregations.bucket.terms.TermsAggregator.BucketCountThresholds;
import org.elasticsearch.search.aggregations.bucket.terms.support.IncludeExclude;
@ -50,7 +48,6 @@ import java.util.Objects;
public class TermsAggregationBuilder extends ValuesSourceAggregationBuilder<ValuesSource, TermsAggregationBuilder> {
public static final String NAME = "terms";
private static final InternalAggregation.Type TYPE = new Type("terms");
public static final ParseField EXECUTION_HINT_FIELD_NAME = new ParseField("execution_hint");
public static final ParseField SHARD_SIZE_FIELD_NAME = new ParseField("shard_size");
@ -108,14 +105,14 @@ public class TermsAggregationBuilder extends ValuesSourceAggregationBuilder<Valu
private boolean showTermDocCountError = false;
public TermsAggregationBuilder(String name, ValueType valueType) {
super(name, TYPE, ValuesSourceType.ANY, valueType);
super(name, ValuesSourceType.ANY, valueType);
}
/**
* Read from a stream.
*/
public TermsAggregationBuilder(StreamInput in) throws IOException {
super(in, TYPE, ValuesSourceType.ANY);
super(in, ValuesSourceType.ANY);
bucketCountThresholds = new BucketCountThresholds(in);
collectMode = in.readOptionalWriteable(SubAggCollectionMode::readFromStream);
executionHint = in.readOptionalString();
@ -287,7 +284,7 @@ public class TermsAggregationBuilder extends ValuesSourceAggregationBuilder<Valu
@Override
protected ValuesSourceAggregatorFactory<ValuesSource, ?> innerBuild(SearchContext context, ValuesSourceConfig<ValuesSource> config,
AggregatorFactory<?> parent, Builder subFactoriesBuilder) throws IOException {
return new TermsAggregatorFactory(name, type, config, order, includeExclude, executionHint, collectMode,
return new TermsAggregatorFactory(name, config, order, includeExclude, executionHint, collectMode,
bucketCountThresholds, showTermDocCountError, context, parent, subFactoriesBuilder, metaData);
}
@ -326,7 +323,7 @@ public class TermsAggregationBuilder extends ValuesSourceAggregationBuilder<Valu
}
@Override
public String getWriteableName() {
public String getType() {
return NAME;
}

View File

@ -28,7 +28,6 @@ import org.elasticsearch.search.aggregations.Aggregator.SubAggCollectionMode;
import org.elasticsearch.search.aggregations.AggregatorFactories;
import org.elasticsearch.search.aggregations.AggregatorFactory;
import org.elasticsearch.search.aggregations.InternalAggregation;
import org.elasticsearch.search.aggregations.InternalAggregation.Type;
import org.elasticsearch.search.aggregations.NonCollectingAggregator;
import org.elasticsearch.search.aggregations.bucket.BucketUtils;
import org.elasticsearch.search.aggregations.bucket.terms.TermsAggregator.BucketCountThresholds;
@ -52,11 +51,11 @@ public class TermsAggregatorFactory extends ValuesSourceAggregatorFactory<Values
private final TermsAggregator.BucketCountThresholds bucketCountThresholds;
private boolean showTermDocCountError;
public TermsAggregatorFactory(String name, Type type, ValuesSourceConfig<ValuesSource> config, Terms.Order order,
public TermsAggregatorFactory(String name, ValuesSourceConfig<ValuesSource> config, Terms.Order order,
IncludeExclude includeExclude, String executionHint, SubAggCollectionMode collectMode,
TermsAggregator.BucketCountThresholds bucketCountThresholds, boolean showTermDocCountError, SearchContext context,
AggregatorFactory<?> parent, AggregatorFactories.Builder subFactoriesBuilder, Map<String, Object> metaData) throws IOException {
super(name, type, config, context, parent, subFactoriesBuilder, metaData);
super(name, config, context, parent, subFactoriesBuilder, metaData);
this.order = order;
this.includeExclude = includeExclude;
this.executionHint = executionHint;

View File

@ -24,10 +24,9 @@ import org.elasticsearch.common.io.stream.StreamOutput;
import org.elasticsearch.common.xcontent.ObjectParser;
import org.elasticsearch.common.xcontent.XContentBuilder;
import org.elasticsearch.index.query.QueryParseContext;
import org.elasticsearch.search.aggregations.AggregatorFactories.Builder;
import org.elasticsearch.search.aggregations.AggregationBuilder;
import org.elasticsearch.search.aggregations.AggregatorFactories.Builder;
import org.elasticsearch.search.aggregations.AggregatorFactory;
import org.elasticsearch.search.aggregations.InternalAggregation.Type;
import org.elasticsearch.search.aggregations.support.ValueType;
import org.elasticsearch.search.aggregations.support.ValuesSource;
import org.elasticsearch.search.aggregations.support.ValuesSource.Numeric;
@ -41,7 +40,6 @@ import java.io.IOException;
public class AvgAggregationBuilder extends ValuesSourceAggregationBuilder.LeafOnly<ValuesSource.Numeric, AvgAggregationBuilder> {
public static final String NAME = "avg";
private static final Type TYPE = new Type(NAME);
private static final ObjectParser<AvgAggregationBuilder, QueryParseContext> PARSER;
static {
@ -54,14 +52,14 @@ public class AvgAggregationBuilder extends ValuesSourceAggregationBuilder.LeafOn
}
public AvgAggregationBuilder(String name) {
super(name, TYPE, ValuesSourceType.NUMERIC, ValueType.NUMERIC);
super(name, ValuesSourceType.NUMERIC, ValueType.NUMERIC);
}
/**
* Read from a stream.
*/
public AvgAggregationBuilder(StreamInput in) throws IOException {
super(in, TYPE, ValuesSourceType.NUMERIC, ValueType.NUMERIC);
super(in, ValuesSourceType.NUMERIC, ValueType.NUMERIC);
}
@Override
@ -72,7 +70,7 @@ public class AvgAggregationBuilder extends ValuesSourceAggregationBuilder.LeafOn
@Override
protected AvgAggregatorFactory innerBuild(SearchContext context, ValuesSourceConfig<Numeric> config,
AggregatorFactory<?> parent, Builder subFactoriesBuilder) throws IOException {
return new AvgAggregatorFactory(name, type, config, context, parent, subFactoriesBuilder, metaData);
return new AvgAggregatorFactory(name, config, context, parent, subFactoriesBuilder, metaData);
}
@Override
@ -91,7 +89,7 @@ public class AvgAggregationBuilder extends ValuesSourceAggregationBuilder.LeafOn
}
@Override
public String getWriteableName() {
public String getType() {
return NAME;
}
}

View File

@ -22,7 +22,6 @@ package org.elasticsearch.search.aggregations.metrics.avg;
import org.elasticsearch.search.aggregations.Aggregator;
import org.elasticsearch.search.aggregations.AggregatorFactories;
import org.elasticsearch.search.aggregations.AggregatorFactory;
import org.elasticsearch.search.aggregations.InternalAggregation.Type;
import org.elasticsearch.search.aggregations.pipeline.PipelineAggregator;
import org.elasticsearch.search.aggregations.support.ValuesSource;
import org.elasticsearch.search.aggregations.support.ValuesSource.Numeric;
@ -36,9 +35,9 @@ import java.util.Map;
public class AvgAggregatorFactory extends ValuesSourceAggregatorFactory<ValuesSource.Numeric, AvgAggregatorFactory> {
public AvgAggregatorFactory(String name, Type type, ValuesSourceConfig<Numeric> config, SearchContext context,
public AvgAggregatorFactory(String name, ValuesSourceConfig<Numeric> config, SearchContext context,
AggregatorFactory<?> parent, AggregatorFactories.Builder subFactoriesBuilder, Map<String, Object> metaData) throws IOException {
super(name, type, config, context, parent, subFactoriesBuilder, metaData);
super(name, config, context, parent, subFactoriesBuilder, metaData);
}
@Override

View File

@ -25,10 +25,9 @@ import org.elasticsearch.common.io.stream.StreamOutput;
import org.elasticsearch.common.xcontent.ObjectParser;
import org.elasticsearch.common.xcontent.XContentBuilder;
import org.elasticsearch.index.query.QueryParseContext;
import org.elasticsearch.search.aggregations.AggregatorFactories.Builder;
import org.elasticsearch.search.aggregations.AggregationBuilder;
import org.elasticsearch.search.aggregations.AggregatorFactories.Builder;
import org.elasticsearch.search.aggregations.AggregatorFactory;
import org.elasticsearch.search.aggregations.InternalAggregation.Type;
import org.elasticsearch.search.aggregations.support.ValueType;
import org.elasticsearch.search.aggregations.support.ValuesSource;
import org.elasticsearch.search.aggregations.support.ValuesSourceAggregationBuilder;
@ -44,7 +43,6 @@ public final class CardinalityAggregationBuilder
extends ValuesSourceAggregationBuilder.LeafOnly<ValuesSource, CardinalityAggregationBuilder> {
public static final String NAME = "cardinality";
private static final Type TYPE = new Type(NAME);
private static final ParseField REHASH = new ParseField("rehash").withAllDeprecated("no replacement - values will always be rehashed");
public static final ParseField PRECISION_THRESHOLD_FIELD = new ParseField("precision_threshold");
@ -64,14 +62,14 @@ public final class CardinalityAggregationBuilder
private Long precisionThreshold = null;
public CardinalityAggregationBuilder(String name, ValueType targetValueType) {
super(name, TYPE, ValuesSourceType.ANY, targetValueType);
super(name, ValuesSourceType.ANY, targetValueType);
}
/**
* Read from a stream.
*/
public CardinalityAggregationBuilder(StreamInput in) throws IOException {
super(in, TYPE, ValuesSourceType.ANY);
super(in, ValuesSourceType.ANY);
if (in.readBoolean()) {
precisionThreshold = in.readLong();
}
@ -124,7 +122,7 @@ public final class CardinalityAggregationBuilder
@Override
protected CardinalityAggregatorFactory innerBuild(SearchContext context, ValuesSourceConfig<ValuesSource> config,
AggregatorFactory<?> parent, Builder subFactoriesBuilder) throws IOException {
return new CardinalityAggregatorFactory(name, type, config, precisionThreshold, context, parent, subFactoriesBuilder, metaData);
return new CardinalityAggregatorFactory(name, config, precisionThreshold, context, parent, subFactoriesBuilder, metaData);
}
@Override
@ -147,7 +145,7 @@ public final class CardinalityAggregationBuilder
}
@Override
public String getWriteableName() {
public String getType() {
return NAME;
}
}

View File

@ -22,7 +22,6 @@ package org.elasticsearch.search.aggregations.metrics.cardinality;
import org.elasticsearch.search.aggregations.Aggregator;
import org.elasticsearch.search.aggregations.AggregatorFactories;
import org.elasticsearch.search.aggregations.AggregatorFactory;
import org.elasticsearch.search.aggregations.InternalAggregation.Type;
import org.elasticsearch.search.aggregations.pipeline.PipelineAggregator;
import org.elasticsearch.search.aggregations.support.ValuesSource;
import org.elasticsearch.search.aggregations.support.ValuesSourceAggregatorFactory;
@ -37,10 +36,10 @@ public class CardinalityAggregatorFactory extends ValuesSourceAggregatorFactory<
private final Long precisionThreshold;
public CardinalityAggregatorFactory(String name, Type type, ValuesSourceConfig<ValuesSource> config, Long precisionThreshold,
public CardinalityAggregatorFactory(String name, ValuesSourceConfig<ValuesSource> config, Long precisionThreshold,
SearchContext context, AggregatorFactory<?> parent, AggregatorFactories.Builder subFactoriesBuilder,
Map<String, Object> metaData) throws IOException {
super(name, type, config, context, parent, subFactoriesBuilder, metaData);
super(name, config, context, parent, subFactoriesBuilder, metaData);
this.precisionThreshold = precisionThreshold;
}

View File

@ -24,10 +24,9 @@ import org.elasticsearch.common.io.stream.StreamOutput;
import org.elasticsearch.common.xcontent.ObjectParser;
import org.elasticsearch.common.xcontent.XContentBuilder;
import org.elasticsearch.index.query.QueryParseContext;
import org.elasticsearch.search.aggregations.AggregatorFactories.Builder;
import org.elasticsearch.search.aggregations.AggregationBuilder;
import org.elasticsearch.search.aggregations.AggregatorFactories.Builder;
import org.elasticsearch.search.aggregations.AggregatorFactory;
import org.elasticsearch.search.aggregations.InternalAggregation.Type;
import org.elasticsearch.search.aggregations.support.ValueType;
import org.elasticsearch.search.aggregations.support.ValuesSource;
import org.elasticsearch.search.aggregations.support.ValuesSourceAggregationBuilder;
@ -41,7 +40,6 @@ import java.util.Objects;
public class GeoBoundsAggregationBuilder extends ValuesSourceAggregationBuilder<ValuesSource.GeoPoint, GeoBoundsAggregationBuilder> {
public static final String NAME = "geo_bounds";
private static final Type TYPE = new Type(NAME);
private static final ObjectParser<GeoBoundsAggregationBuilder, QueryParseContext> PARSER;
static {
@ -57,14 +55,14 @@ public class GeoBoundsAggregationBuilder extends ValuesSourceAggregationBuilder<
private boolean wrapLongitude = true;
public GeoBoundsAggregationBuilder(String name) {
super(name, TYPE, ValuesSourceType.GEOPOINT, ValueType.GEOPOINT);
super(name, ValuesSourceType.GEOPOINT, ValueType.GEOPOINT);
}
/**
* Read from a stream.
*/
public GeoBoundsAggregationBuilder(StreamInput in) throws IOException {
super(in, TYPE, ValuesSourceType.GEOPOINT, ValueType.GEOPOINT);
super(in, ValuesSourceType.GEOPOINT, ValueType.GEOPOINT);
wrapLongitude = in.readBoolean();
}
@ -91,7 +89,7 @@ public class GeoBoundsAggregationBuilder extends ValuesSourceAggregationBuilder<
@Override
protected GeoBoundsAggregatorFactory innerBuild(SearchContext context, ValuesSourceConfig<ValuesSource.GeoPoint> config,
AggregatorFactory<?> parent, Builder subFactoriesBuilder) throws IOException {
return new GeoBoundsAggregatorFactory(name, type, config, wrapLongitude, context, parent, subFactoriesBuilder, metaData);
return new GeoBoundsAggregatorFactory(name, config, wrapLongitude, context, parent, subFactoriesBuilder, metaData);
}
@Override
@ -112,7 +110,7 @@ public class GeoBoundsAggregationBuilder extends ValuesSourceAggregationBuilder<
}
@Override
public String getWriteableName() {
public String getType() {
return NAME;
}
}

View File

@ -22,7 +22,6 @@ package org.elasticsearch.search.aggregations.metrics.geobounds;
import org.elasticsearch.search.aggregations.Aggregator;
import org.elasticsearch.search.aggregations.AggregatorFactories;
import org.elasticsearch.search.aggregations.AggregatorFactory;
import org.elasticsearch.search.aggregations.InternalAggregation.Type;
import org.elasticsearch.search.aggregations.pipeline.PipelineAggregator;
import org.elasticsearch.search.aggregations.support.ValuesSource;
import org.elasticsearch.search.aggregations.support.ValuesSourceAggregatorFactory;
@ -37,10 +36,10 @@ public class GeoBoundsAggregatorFactory extends ValuesSourceAggregatorFactory<Va
private final boolean wrapLongitude;
public GeoBoundsAggregatorFactory(String name, Type type, ValuesSourceConfig<ValuesSource.GeoPoint> config, boolean wrapLongitude,
public GeoBoundsAggregatorFactory(String name, ValuesSourceConfig<ValuesSource.GeoPoint> config, boolean wrapLongitude,
SearchContext context, AggregatorFactory<?> parent, AggregatorFactories.Builder subFactoriesBuilder,
Map<String, Object> metaData) throws IOException {
super(name, type, config, context, parent, subFactoriesBuilder, metaData);
super(name, config, context, parent, subFactoriesBuilder, metaData);
this.wrapLongitude = wrapLongitude;
}

View File

@ -24,10 +24,9 @@ import org.elasticsearch.common.io.stream.StreamOutput;
import org.elasticsearch.common.xcontent.ObjectParser;
import org.elasticsearch.common.xcontent.XContentBuilder;
import org.elasticsearch.index.query.QueryParseContext;
import org.elasticsearch.search.aggregations.AggregatorFactories.Builder;
import org.elasticsearch.search.aggregations.AggregationBuilder;
import org.elasticsearch.search.aggregations.AggregatorFactories.Builder;
import org.elasticsearch.search.aggregations.AggregatorFactory;
import org.elasticsearch.search.aggregations.InternalAggregation.Type;
import org.elasticsearch.search.aggregations.support.ValueType;
import org.elasticsearch.search.aggregations.support.ValuesSource;
import org.elasticsearch.search.aggregations.support.ValuesSourceAggregationBuilder;
@ -41,7 +40,6 @@ import java.io.IOException;
public class GeoCentroidAggregationBuilder
extends ValuesSourceAggregationBuilder.LeafOnly<ValuesSource.GeoPoint, GeoCentroidAggregationBuilder> {
public static final String NAME = "geo_centroid";
public static final Type TYPE = new Type(NAME);
private static final ObjectParser<GeoCentroidAggregationBuilder, QueryParseContext> PARSER;
static {
@ -54,14 +52,14 @@ public class GeoCentroidAggregationBuilder
}
public GeoCentroidAggregationBuilder(String name) {
super(name, TYPE, ValuesSourceType.GEOPOINT, ValueType.GEOPOINT);
super(name, ValuesSourceType.GEOPOINT, ValueType.GEOPOINT);
}
/**
* Read from a stream.
*/
public GeoCentroidAggregationBuilder(StreamInput in) throws IOException {
super(in, TYPE, ValuesSourceType.GEOPOINT, ValueType.GEOPOINT);
super(in, ValuesSourceType.GEOPOINT, ValueType.GEOPOINT);
}
@Override
@ -72,7 +70,7 @@ public class GeoCentroidAggregationBuilder
@Override
protected GeoCentroidAggregatorFactory innerBuild(SearchContext context, ValuesSourceConfig<ValuesSource.GeoPoint> config,
AggregatorFactory<?> parent, Builder subFactoriesBuilder) throws IOException {
return new GeoCentroidAggregatorFactory(name, type, config, context, parent, subFactoriesBuilder, metaData);
return new GeoCentroidAggregatorFactory(name, config, context, parent, subFactoriesBuilder, metaData);
}
@Override
@ -91,7 +89,7 @@ public class GeoCentroidAggregationBuilder
}
@Override
public String getWriteableName() {
public String getType() {
return NAME;
}
}

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