diff --git a/benchmarks/src/main/java/org/elasticsearch/benchmark/routing/allocation/Allocators.java b/benchmarks/src/main/java/org/elasticsearch/benchmark/routing/allocation/Allocators.java index 97fbda80dc6..9b1cfaabf93 100644 --- a/benchmarks/src/main/java/org/elasticsearch/benchmark/routing/allocation/Allocators.java +++ b/benchmarks/src/main/java/org/elasticsearch/benchmark/routing/allocation/Allocators.java @@ -60,8 +60,8 @@ public final class Allocators { } @Override - public boolean allocateUnassigned(RoutingAllocation allocation) { - return false; + public void allocateUnassigned(RoutingAllocation allocation) { + // noop } } diff --git a/core/src/main/java/org/elasticsearch/action/admin/cluster/node/tasks/cancel/CancelTasksResponse.java b/core/src/main/java/org/elasticsearch/action/admin/cluster/node/tasks/cancel/CancelTasksResponse.java index 34daee1d14d..5e7c2c0f97d 100644 --- a/core/src/main/java/org/elasticsearch/action/admin/cluster/node/tasks/cancel/CancelTasksResponse.java +++ b/core/src/main/java/org/elasticsearch/action/admin/cluster/node/tasks/cancel/CancelTasksResponse.java @@ -22,7 +22,6 @@ package org.elasticsearch.action.admin.cluster.node.tasks.cancel; import org.elasticsearch.action.FailedNodeException; import org.elasticsearch.action.TaskOperationFailure; import org.elasticsearch.action.admin.cluster.node.tasks.list.ListTasksResponse; -import org.elasticsearch.cluster.node.DiscoveryNodes; import org.elasticsearch.tasks.TaskInfo; import java.util.List; @@ -35,13 +34,8 @@ public class CancelTasksResponse extends ListTasksResponse { public CancelTasksResponse() { } - public CancelTasksResponse(DiscoveryNodes discoveryNodes) { - super(discoveryNodes); - } - public CancelTasksResponse(List tasks, List taskFailures, List - nodeFailures, DiscoveryNodes discoveryNodes) { - super(tasks, taskFailures, nodeFailures, discoveryNodes); + nodeFailures) { + super(tasks, taskFailures, nodeFailures); } - } diff --git a/core/src/main/java/org/elasticsearch/action/admin/cluster/node/tasks/cancel/TransportCancelTasksAction.java b/core/src/main/java/org/elasticsearch/action/admin/cluster/node/tasks/cancel/TransportCancelTasksAction.java index 195cdd86f5c..dc52e4fd508 100644 --- a/core/src/main/java/org/elasticsearch/action/admin/cluster/node/tasks/cancel/TransportCancelTasksAction.java +++ b/core/src/main/java/org/elasticsearch/action/admin/cluster/node/tasks/cancel/TransportCancelTasksAction.java @@ -66,7 +66,7 @@ public class TransportCancelTasksAction extends TransportTasksAction new CancelTasksResponse(clusterService.state().nodes()), + indexNameExpressionResolver, CancelTasksRequest::new, CancelTasksResponse::new, ThreadPool.Names.MANAGEMENT); transportService.registerRequestHandler(BAN_PARENT_ACTION_NAME, BanParentTaskRequest::new, ThreadPool.Names.SAME, new BanParentRequestHandler()); @@ -75,7 +75,7 @@ public class TransportCancelTasksAction extends TransportTasksAction tasks, List taskOperationFailures, List failedNodeExceptions) { - return new CancelTasksResponse(tasks, taskOperationFailures, failedNodeExceptions, clusterService.state().nodes()); + return new CancelTasksResponse(tasks, taskOperationFailures, failedNodeExceptions); } @Override diff --git a/core/src/main/java/org/elasticsearch/action/admin/cluster/node/tasks/list/ListTasksResponse.java b/core/src/main/java/org/elasticsearch/action/admin/cluster/node/tasks/list/ListTasksResponse.java index ae6f0d0855f..b33226b973b 100644 --- a/core/src/main/java/org/elasticsearch/action/admin/cluster/node/tasks/list/ListTasksResponse.java +++ b/core/src/main/java/org/elasticsearch/action/admin/cluster/node/tasks/list/ListTasksResponse.java @@ -51,21 +51,14 @@ public class ListTasksResponse extends BaseTasksResponse implements ToXContent { private List groups; - private final DiscoveryNodes discoveryNodes; - public ListTasksResponse() { - this(null, null, null, null); - } - - public ListTasksResponse(DiscoveryNodes discoveryNodes) { - this(null, null, null, discoveryNodes); + this(null, null, null); } public ListTasksResponse(List tasks, List taskFailures, - List nodeFailures, DiscoveryNodes discoveryNodes) { + List nodeFailures) { super(taskFailures, nodeFailures); this.tasks = tasks == null ? Collections.emptyList() : Collections.unmodifiableList(new ArrayList<>(tasks)); - this.discoveryNodes = discoveryNodes; } @Override @@ -90,6 +83,9 @@ public class ListTasksResponse extends BaseTasksResponse implements ToXContent { return perNodeTasks; } + /** + * Get the tasks found by this request grouped by parent tasks. + */ public List getTaskGroups() { if (groups == null) { buildTaskGroups(); @@ -125,12 +121,76 @@ public class ListTasksResponse extends BaseTasksResponse implements ToXContent { this.groups = Collections.unmodifiableList(topLevelTasks.stream().map(TaskGroup.Builder::build).collect(Collectors.toList())); } + /** + * Get the tasks found by this request. + */ public List getTasks() { return tasks; } + /** + * Convert this task response to XContent grouping by executing nodes. + */ + public XContentBuilder toXContentGroupedByNode(XContentBuilder builder, Params params, DiscoveryNodes discoveryNodes) + throws IOException { + toXContentCommon(builder, params); + builder.startObject("nodes"); + for (Map.Entry> entry : getPerNodeTasks().entrySet()) { + DiscoveryNode node = discoveryNodes.get(entry.getKey()); + builder.startObject(entry.getKey()); + if (node != null) { + // If the node is no longer part of the cluster, oh well, we'll just skip it's useful information. + builder.field("name", node.getName()); + builder.field("transport_address", node.getAddress().toString()); + builder.field("host", node.getHostName()); + builder.field("ip", node.getAddress()); + + builder.startArray("roles"); + for (DiscoveryNode.Role role : node.getRoles()) { + builder.value(role.getRoleName()); + } + builder.endArray(); + + if (!node.getAttributes().isEmpty()) { + builder.startObject("attributes"); + for (Map.Entry attrEntry : node.getAttributes().entrySet()) { + builder.field(attrEntry.getKey(), attrEntry.getValue()); + } + builder.endObject(); + } + } + builder.startObject("tasks"); + for(TaskInfo task : entry.getValue()) { + builder.field(task.getTaskId().toString()); + task.toXContent(builder, params); + } + builder.endObject(); + builder.endObject(); + } + builder.endObject(); + return builder; + } + + /** + * Convert this response to XContent grouping by parent tasks. + */ + public XContentBuilder toXContentGroupedByParents(XContentBuilder builder, Params params) throws IOException { + toXContentCommon(builder, params); + builder.startObject("tasks"); + for (TaskGroup group : getTaskGroups()) { + builder.field(group.getTaskInfo().getTaskId().toString()); + group.toXContent(builder, params); + } + builder.endObject(); + return builder; + } + @Override public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException { + return toXContentGroupedByParents(builder, params); + } + + private void toXContentCommon(XContentBuilder builder, Params params) throws IOException { if (getTaskFailures() != null && getTaskFailures().size() > 0) { builder.startArray("task_failures"); for (TaskOperationFailure ex : getTaskFailures()){ @@ -150,51 +210,6 @@ public class ListTasksResponse extends BaseTasksResponse implements ToXContent { } builder.endArray(); } - String groupBy = params.param("group_by", "nodes"); - if ("nodes".equals(groupBy)) { - builder.startObject("nodes"); - for (Map.Entry> entry : getPerNodeTasks().entrySet()) { - DiscoveryNode node = discoveryNodes.get(entry.getKey()); - builder.startObject(entry.getKey()); - if (node != null) { - // If the node is no longer part of the cluster, oh well, we'll just skip it's useful information. - builder.field("name", node.getName()); - builder.field("transport_address", node.getAddress().toString()); - builder.field("host", node.getHostName()); - builder.field("ip", node.getAddress()); - - builder.startArray("roles"); - for (DiscoveryNode.Role role : node.getRoles()) { - builder.value(role.getRoleName()); - } - builder.endArray(); - - if (!node.getAttributes().isEmpty()) { - builder.startObject("attributes"); - for (Map.Entry attrEntry : node.getAttributes().entrySet()) { - builder.field(attrEntry.getKey(), attrEntry.getValue()); - } - builder.endObject(); - } - } - builder.startObject("tasks"); - for(TaskInfo task : entry.getValue()) { - builder.field(task.getTaskId().toString()); - task.toXContent(builder, params); - } - builder.endObject(); - builder.endObject(); - } - builder.endObject(); - } else if ("parents".equals(groupBy)) { - builder.startObject("tasks"); - for (TaskGroup group : getTaskGroups()) { - builder.field(group.getTaskInfo().getTaskId().toString()); - group.toXContent(builder, params); - } - builder.endObject(); - } - return builder; } @Override diff --git a/core/src/main/java/org/elasticsearch/action/admin/cluster/node/tasks/list/TransportListTasksAction.java b/core/src/main/java/org/elasticsearch/action/admin/cluster/node/tasks/list/TransportListTasksAction.java index a1f84f87f9d..32c0c3c1845 100644 --- a/core/src/main/java/org/elasticsearch/action/admin/cluster/node/tasks/list/TransportListTasksAction.java +++ b/core/src/main/java/org/elasticsearch/action/admin/cluster/node/tasks/list/TransportListTasksAction.java @@ -56,15 +56,14 @@ public class TransportListTasksAction extends TransportTasksAction new ListTasksResponse(clusterService.state().nodes()), - ThreadPool.Names.MANAGEMENT); + super(settings, ListTasksAction.NAME, threadPool, clusterService, transportService, actionFilters, indexNameExpressionResolver, + ListTasksRequest::new, ListTasksResponse::new, ThreadPool.Names.MANAGEMENT); } @Override protected ListTasksResponse newResponse(ListTasksRequest request, List tasks, List taskOperationFailures, List failedNodeExceptions) { - return new ListTasksResponse(tasks, taskOperationFailures, failedNodeExceptions, clusterService.state().nodes()); + return new ListTasksResponse(tasks, taskOperationFailures, failedNodeExceptions); } @Override diff --git a/core/src/main/java/org/elasticsearch/cluster/metadata/IndexMetaData.java b/core/src/main/java/org/elasticsearch/cluster/metadata/IndexMetaData.java index 0af860953f0..4f51fa9818f 100644 --- a/core/src/main/java/org/elasticsearch/cluster/metadata/IndexMetaData.java +++ b/core/src/main/java/org/elasticsearch/cluster/metadata/IndexMetaData.java @@ -23,7 +23,6 @@ import com.carrotsearch.hppc.LongArrayList; import com.carrotsearch.hppc.cursors.IntObjectCursor; import com.carrotsearch.hppc.cursors.ObjectCursor; import com.carrotsearch.hppc.cursors.ObjectObjectCursor; - import org.elasticsearch.Version; import org.elasticsearch.action.support.ActiveShardCount; import org.elasticsearch.cluster.Diff; @@ -32,8 +31,7 @@ import org.elasticsearch.cluster.DiffableUtils; import org.elasticsearch.cluster.block.ClusterBlock; import org.elasticsearch.cluster.block.ClusterBlockLevel; import org.elasticsearch.cluster.node.DiscoveryNodeFilters; -import org.elasticsearch.cluster.routing.RoutingTable; -import org.elasticsearch.cluster.routing.allocation.AllocationService; +import org.elasticsearch.cluster.routing.allocation.IndexMetaDataUpdater; import org.elasticsearch.common.Nullable; import org.elasticsearch.common.ParseFieldMatcher; import org.elasticsearch.common.collect.ImmutableOpenIntMap; @@ -342,8 +340,7 @@ public class IndexMetaData implements Diffable, FromXContentBuild * a primary shard is assigned after a full cluster restart or a replica shard is promoted to a primary. * * Note: since we increment the term every time a shard is assigned, the term for any operational shard (i.e., a shard - * that can be indexed into) is larger than 0. - * See {@link AllocationService#updateMetaDataWithRoutingTable(MetaData, RoutingTable, RoutingTable)}. + * that can be indexed into) is larger than 0. See {@link IndexMetaDataUpdater#applyChanges(MetaData)}. **/ public long primaryTerm(int shardId) { return this.primaryTerms[shardId]; diff --git a/core/src/main/java/org/elasticsearch/cluster/metadata/MetaData.java b/core/src/main/java/org/elasticsearch/cluster/metadata/MetaData.java index 6fdecf542f5..1bfd775ecce 100644 --- a/core/src/main/java/org/elasticsearch/cluster/metadata/MetaData.java +++ b/core/src/main/java/org/elasticsearch/cluster/metadata/MetaData.java @@ -460,6 +460,11 @@ public class MetaData implements Iterable, Diffable, Fr return null; } + /** Returns true iff existing index has the same {@link IndexMetaData} instance */ + public boolean hasIndexMetaData(final IndexMetaData indexMetaData) { + return indices.get(indexMetaData.getIndex().getName()) == indexMetaData; + } + /** * Returns the {@link IndexMetaData} for this index. * @throws IndexNotFoundException if no metadata for this index is found diff --git a/core/src/main/java/org/elasticsearch/cluster/routing/RoutingChangesObserver.java b/core/src/main/java/org/elasticsearch/cluster/routing/RoutingChangesObserver.java new file mode 100644 index 00000000000..d54df7e0801 --- /dev/null +++ b/core/src/main/java/org/elasticsearch/cluster/routing/RoutingChangesObserver.java @@ -0,0 +1,196 @@ +/* + * 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; + +/** + * Records changes made to {@link RoutingNodes} during an allocation round. + */ +public interface RoutingChangesObserver { + /** + * Called when unassigned shard is initialized. Does not include initializing relocation target shards. + */ + void shardInitialized(ShardRouting unassignedShard); + + /** + * Called when an initializing shard is started. + */ + void shardStarted(ShardRouting initializingShard, ShardRouting startedShard); + + /** + * Called when relocation of a started shard is initiated. + */ + void relocationStarted(ShardRouting startedShard, ShardRouting targetRelocatingShard); + + /** + * Called when an unassigned shard's unassigned information was updated + */ + void unassignedInfoUpdated(ShardRouting unassignedShard, UnassignedInfo newUnassignedInfo); + + /** + * Called when a shard is failed or cancelled. + */ + void shardFailed(ShardRouting failedShard, UnassignedInfo unassignedInfo); + + /** + * Called on relocation source when relocation completes after relocation target is started. + */ + void relocationCompleted(ShardRouting removedRelocationSource); + + /** + * Called on replica relocation target when replica relocation source fails. Promotes the replica relocation target to ordinary + * initializing shard. + */ + void relocationSourceRemoved(ShardRouting removedReplicaRelocationSource); + + /** + * Called on started primary shard after it has been promoted from replica to primary and is reinitialized due to shadow replicas. + */ + void startedPrimaryReinitialized(ShardRouting startedPrimaryShard, ShardRouting initializedShard); + + /** + * Called when started replica is promoted to primary. + */ + void replicaPromoted(ShardRouting replicaShard); + + + /** + * Abstract implementation of {@link RoutingChangesObserver} that does not take any action. Useful for subclasses that only override + * certain methods. + */ + class AbstractRoutingChangesObserver implements RoutingChangesObserver { + + @Override + public void shardInitialized(ShardRouting unassignedShard) { + + } + + @Override + public void shardStarted(ShardRouting initializingShard, ShardRouting startedShard) { + + } + + @Override + public void relocationStarted(ShardRouting startedShard, ShardRouting targetRelocatingShard) { + + } + + @Override + public void unassignedInfoUpdated(ShardRouting unassignedShard, UnassignedInfo newUnassignedInfo) { + + } + + @Override + public void shardFailed(ShardRouting activeShard, UnassignedInfo unassignedInfo) { + + } + + @Override + public void relocationCompleted(ShardRouting removedRelocationSource) { + + } + + @Override + public void relocationSourceRemoved(ShardRouting removedReplicaRelocationSource) { + + } + + @Override + public void startedPrimaryReinitialized(ShardRouting startedPrimaryShard, ShardRouting initializedShard) { + + } + + @Override + public void replicaPromoted(ShardRouting replicaShard) { + + } + } + + class DelegatingRoutingChangesObserver implements RoutingChangesObserver { + + private final RoutingChangesObserver[] routingChangesObservers; + + public DelegatingRoutingChangesObserver(RoutingChangesObserver... routingChangesObservers) { + this.routingChangesObservers = routingChangesObservers; + } + + @Override + public void shardInitialized(ShardRouting unassignedShard) { + for (RoutingChangesObserver routingChangesObserver : routingChangesObservers) { + routingChangesObserver.shardInitialized(unassignedShard); + } + } + + @Override + public void shardStarted(ShardRouting initializingShard, ShardRouting startedShard) { + for (RoutingChangesObserver routingChangesObserver : routingChangesObservers) { + routingChangesObserver.shardStarted(initializingShard, startedShard); + } + } + + @Override + public void relocationStarted(ShardRouting startedShard, ShardRouting targetRelocatingShard) { + for (RoutingChangesObserver routingChangesObserver : routingChangesObservers) { + routingChangesObserver.relocationStarted(startedShard, targetRelocatingShard); + } + } + + @Override + public void unassignedInfoUpdated(ShardRouting unassignedShard, UnassignedInfo newUnassignedInfo) { + for (RoutingChangesObserver routingChangesObserver : routingChangesObservers) { + routingChangesObserver.unassignedInfoUpdated(unassignedShard, newUnassignedInfo); + } + } + + @Override + public void shardFailed(ShardRouting activeShard, UnassignedInfo unassignedInfo) { + for (RoutingChangesObserver routingChangesObserver : routingChangesObservers) { + routingChangesObserver.shardFailed(activeShard, unassignedInfo); + } + } + + @Override + public void relocationCompleted(ShardRouting removedRelocationSource) { + for (RoutingChangesObserver routingChangesObserver : routingChangesObservers) { + routingChangesObserver.relocationCompleted(removedRelocationSource); + } + } + + @Override + public void relocationSourceRemoved(ShardRouting removedReplicaRelocationSource) { + for (RoutingChangesObserver routingChangesObserver : routingChangesObservers) { + routingChangesObserver.relocationSourceRemoved(removedReplicaRelocationSource); + } + } + + @Override + public void startedPrimaryReinitialized(ShardRouting startedPrimaryShard, ShardRouting initializedShard) { + for (RoutingChangesObserver routingChangesObserver : routingChangesObservers) { + routingChangesObserver.startedPrimaryReinitialized(startedPrimaryShard, initializedShard); + } + } + + @Override + public void replicaPromoted(ShardRouting replicaShard) { + for (RoutingChangesObserver routingChangesObserver : routingChangesObservers) { + routingChangesObserver.replicaPromoted(replicaShard); + } + } + } +} diff --git a/core/src/main/java/org/elasticsearch/cluster/routing/RoutingNodes.java b/core/src/main/java/org/elasticsearch/cluster/routing/RoutingNodes.java index eb9a18228f3..2ac9fcc8dd0 100644 --- a/core/src/main/java/org/elasticsearch/cluster/routing/RoutingNodes.java +++ b/core/src/main/java/org/elasticsearch/cluster/routing/RoutingNodes.java @@ -409,7 +409,7 @@ public class RoutingNodes implements Iterable { * @return the initialized shard */ public ShardRouting initializeShard(ShardRouting unassignedShard, String nodeId, @Nullable String existingAllocationId, - long expectedSize) { + long expectedSize, RoutingChangesObserver routingChangesObserver) { ensureMutable(); assert unassignedShard.unassigned() : "expected an unassigned shard " + unassignedShard; ShardRouting initializedShard = unassignedShard.initialize(nodeId, existingAllocationId, expectedSize); @@ -420,6 +420,7 @@ public class RoutingNodes implements Iterable { } addRecovery(initializedShard); assignedShardsAdd(initializedShard); + routingChangesObserver.shardInitialized(unassignedShard); return initializedShard; } @@ -429,7 +430,8 @@ public class RoutingNodes implements Iterable { * * @return pair of source relocating and target initializing shards. */ - public Tuple relocateShard(ShardRouting startedShard, String nodeId, long expectedShardSize) { + public Tuple relocateShard(ShardRouting startedShard, String nodeId, long expectedShardSize, + RoutingChangesObserver changes) { ensureMutable(); relocatingShards++; ShardRouting source = startedShard.relocate(nodeId, expectedShardSize); @@ -438,6 +440,7 @@ public class RoutingNodes implements Iterable { node(target.currentNodeId()).add(target); assignedShardsAdd(target); addRecovery(target); + changes.relocationStarted(startedShard, target); return Tuple.tuple(source, target); } @@ -448,10 +451,11 @@ public class RoutingNodes implements Iterable { * * @return the started shard */ - public ShardRouting startShard(ESLogger logger, ShardRouting initializingShard) { + public ShardRouting startShard(ESLogger logger, ShardRouting initializingShard, RoutingChangesObserver routingChangesObserver) { ensureMutable(); ShardRouting startedShard = started(initializingShard); logger.trace("{} marked shard as started (routing: {})", initializingShard.shardId(), initializingShard); + routingChangesObserver.shardStarted(initializingShard, startedShard); if (initializingShard.relocatingNodeId() != null) { // relocation target has been started, remove relocation source @@ -461,6 +465,7 @@ public class RoutingNodes implements Iterable { assert relocationSourceShard.getTargetRelocatingShard() == initializingShard : "relocation target mismatch, expected: " + initializingShard + " but was: " + relocationSourceShard.getTargetRelocatingShard(); remove(relocationSourceShard); + routingChangesObserver.relocationCompleted(relocationSourceShard); } return startedShard; } @@ -478,7 +483,8 @@ public class RoutingNodes implements Iterable { * - If shard is a (primary or replica) relocation target, this also clears the relocation information on the source shard. * */ - public void failShard(ESLogger logger, ShardRouting failedShard, UnassignedInfo unassignedInfo, IndexMetaData indexMetaData) { + public void failShard(ESLogger logger, ShardRouting failedShard, UnassignedInfo unassignedInfo, IndexMetaData indexMetaData, + RoutingChangesObserver routingChangesObserver) { ensureMutable(); assert failedShard.assignedToNode() : "only assigned shards can be failed"; assert indexMetaData.getIndex().equals(failedShard.index()) : @@ -502,7 +508,7 @@ public class RoutingNodes implements Iterable { UnassignedInfo primaryFailedUnassignedInfo = new UnassignedInfo(UnassignedInfo.Reason.PRIMARY_FAILED, "primary failed while replica initializing", null, 0, unassignedInfo.getUnassignedTimeInNanos(), unassignedInfo.getUnassignedTimeInMillis(), false, AllocationStatus.NO_ATTEMPT); - failShard(logger, replicaShard, primaryFailedUnassignedInfo, indexMetaData); + failShard(logger, replicaShard, primaryFailedUnassignedInfo, indexMetaData, routingChangesObserver); } } } @@ -516,11 +522,13 @@ public class RoutingNodes implements Iterable { logger.trace("{} is removed due to the failure/cancellation of the source shard", targetShard); // cancel and remove target shard remove(targetShard); + routingChangesObserver.shardFailed(targetShard, unassignedInfo); } else { logger.trace("{}, relocation source failed / cancelled, mark as initializing without relocation source", targetShard); // promote to initializing shard without relocation source and ensure that removed relocation source // is not added back as unassigned shard removeRelocationSource(targetShard); + routingChangesObserver.relocationSourceRemoved(targetShard); } } @@ -542,6 +550,7 @@ public class RoutingNodes implements Iterable { cancelRelocation(sourceShard); remove(failedShard); } + routingChangesObserver.shardFailed(failedShard, unassignedInfo); } else { assert failedShard.active(); if (failedShard.primary()) { @@ -555,8 +564,10 @@ public class RoutingNodes implements Iterable { assert activeReplica.started() : "replica relocation should have been cancelled: " + activeReplica; movePrimaryToUnassignedAndDemoteToReplica(failedShard, unassignedInfo); ShardRouting primarySwappedCandidate = promoteActiveReplicaShardToPrimary(activeReplica); + routingChangesObserver.replicaPromoted(activeReplica); if (IndexMetaData.isIndexUsingShadowReplicas(indexMetaData.getSettings())) { - reinitShadowPrimary(primarySwappedCandidate); + ShardRouting initializedShard = reinitShadowPrimary(primarySwappedCandidate); + routingChangesObserver.startedPrimaryReinitialized(primarySwappedCandidate, initializedShard); } } } else { @@ -567,6 +578,7 @@ public class RoutingNodes implements Iterable { moveToUnassigned(failedShard, unassignedInfo); } } + routingChangesObserver.shardFailed(failedShard, unassignedInfo); } assert node(failedShard.currentNodeId()).getByShardId(failedShard.shardId()) == null : "failedShard " + failedShard + " was matched but wasn't removed"; @@ -806,13 +818,11 @@ public class RoutingNodes implements Iterable { * Should be used with caution, typically, * the correct usage is to removeAndIgnore from the iterator. * @see #ignored() - * @see UnassignedIterator#removeAndIgnore(AllocationStatus) + * @see UnassignedIterator#removeAndIgnore(AllocationStatus, RoutingChangesObserver) * @see #isIgnoredEmpty() - * @return true iff the decision caused a change to the unassigned info */ - public boolean ignoreShard(ShardRouting shard, AllocationStatus allocationStatus) { + public void ignoreShard(ShardRouting shard, AllocationStatus allocationStatus, RoutingChangesObserver changes) { nodes.ensureMutable(); - boolean changed = false; if (shard.primary()) { ignoredPrimaries++; UnassignedInfo currInfo = shard.unassignedInfo(); @@ -822,12 +832,12 @@ public class RoutingNodes implements Iterable { currInfo.getNumFailedAllocations(), currInfo.getUnassignedTimeInNanos(), currInfo.getUnassignedTimeInMillis(), currInfo.isDelayed(), allocationStatus); - shard = shard.updateUnassignedInfo(newInfo); - changed = true; + ShardRouting updatedShard = shard.updateUnassignedInfo(newInfo); + changes.unassignedInfoUpdated(shard, newInfo); + shard = updatedShard; } } ignored.add(shard); - return changed; } public class UnassignedIterator implements Iterator { @@ -854,10 +864,11 @@ public class RoutingNodes implements Iterable { * * @param existingAllocationId allocation id to use. If null, a fresh allocation id is generated. */ - public ShardRouting initialize(String nodeId, @Nullable String existingAllocationId, long expectedShardSize) { + public ShardRouting initialize(String nodeId, @Nullable String existingAllocationId, long expectedShardSize, + RoutingChangesObserver routingChangesObserver) { nodes.ensureMutable(); innerRemove(); - return nodes.initializeShard(current, nodeId, existingAllocationId, expectedShardSize); + return nodes.initializeShard(current, nodeId, existingAllocationId, expectedShardSize, routingChangesObserver); } /** @@ -867,12 +878,11 @@ public class RoutingNodes implements Iterable { * that subsequent consumers of this API won't try to allocate this shard again. * * @param attempt the result of the allocation attempt - * @return true iff the decision caused an update to the unassigned info */ - public boolean removeAndIgnore(AllocationStatus attempt) { + public void removeAndIgnore(AllocationStatus attempt, RoutingChangesObserver changes) { nodes.ensureMutable(); innerRemove(); - return ignoreShard(current, attempt); + ignoreShard(current, attempt, changes); } private void updateShardRouting(ShardRouting shardRouting) { @@ -886,16 +896,17 @@ public class RoutingNodes implements Iterable { * @param unassignedInfo the new unassigned info to use * @return the shard with unassigned info updated */ - public ShardRouting updateUnassignedInfo(UnassignedInfo unassignedInfo) { + public ShardRouting updateUnassignedInfo(UnassignedInfo unassignedInfo, RoutingChangesObserver changes) { nodes.ensureMutable(); ShardRouting updatedShardRouting = current.updateUnassignedInfo(unassignedInfo); + changes.unassignedInfoUpdated(current, unassignedInfo); updateShardRouting(updatedShardRouting); return updatedShardRouting; } /** - * Unsupported operation, just there for the interface. Use {@link #removeAndIgnore(AllocationStatus)} or - * {@link #initialize(String, String, long)}. + * Unsupported operation, just there for the interface. Use {@link #removeAndIgnore(AllocationStatus, RoutingChangesObserver)} or + * {@link #initialize(String, String, long, RoutingChangesObserver)}. */ @Override public void remove() { @@ -919,8 +930,8 @@ public class RoutingNodes implements Iterable { /** * Returns true iff any unassigned shards are marked as temporarily ignored. - * @see UnassignedShards#ignoreShard(ShardRouting, AllocationStatus) - * @see UnassignedIterator#removeAndIgnore(AllocationStatus) + * @see UnassignedShards#ignoreShard(ShardRouting, AllocationStatus, RoutingChangesObserver) + * @see UnassignedIterator#removeAndIgnore(AllocationStatus, RoutingChangesObserver) */ public boolean isIgnoredEmpty() { return ignored.isEmpty(); diff --git a/core/src/main/java/org/elasticsearch/cluster/routing/allocation/AllocationService.java b/core/src/main/java/org/elasticsearch/cluster/routing/allocation/AllocationService.java index 8401b5f48e1..0b9e43c5c84 100644 --- a/core/src/main/java/org/elasticsearch/cluster/routing/allocation/AllocationService.java +++ b/core/src/main/java/org/elasticsearch/cluster/routing/allocation/AllocationService.java @@ -26,9 +26,6 @@ import org.elasticsearch.cluster.health.ClusterHealthStatus; import org.elasticsearch.cluster.health.ClusterStateHealth; import org.elasticsearch.cluster.metadata.IndexMetaData; import org.elasticsearch.cluster.metadata.MetaData; -import org.elasticsearch.cluster.routing.AllocationId; -import org.elasticsearch.cluster.routing.IndexRoutingTable; -import org.elasticsearch.cluster.routing.IndexShardRoutingTable; import org.elasticsearch.cluster.routing.RoutingNode; import org.elasticsearch.cluster.routing.RoutingNodes; import org.elasticsearch.cluster.routing.RoutingTable; @@ -43,13 +40,10 @@ import org.elasticsearch.common.component.AbstractComponent; import org.elasticsearch.common.inject.Inject; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.gateway.GatewayAllocator; -import org.elasticsearch.index.shard.ShardId; import java.util.Collections; import java.util.Iterator; import java.util.List; -import java.util.Objects; -import java.util.Set; import java.util.function.Function; import java.util.stream.Collectors; @@ -93,7 +87,7 @@ public class AllocationService extends AbstractComponent { public Result applyStartedShards(ClusterState clusterState, List startedShards, boolean withReroute) { if (startedShards.isEmpty()) { - return new Result(false, clusterState.routingTable(), clusterState.metaData()); + return Result.unchanged(clusterState); } RoutingNodes routingNodes = getMutableRoutingNodes(clusterState); // shuffle the unassigned nodes, just so we won't have things like poison failed shards @@ -111,103 +105,22 @@ public class AllocationService extends AbstractComponent { protected Result buildResultAndLogHealthChange(RoutingAllocation allocation, String reason) { return buildResultAndLogHealthChange(allocation, reason, new RoutingExplanations()); - } protected Result buildResultAndLogHealthChange(RoutingAllocation allocation, String reason, RoutingExplanations explanations) { - MetaData oldMetaData = allocation.metaData(); RoutingTable oldRoutingTable = allocation.routingTable(); RoutingNodes newRoutingNodes = allocation.routingNodes(); final RoutingTable newRoutingTable = new RoutingTable.Builder().updateNodes(oldRoutingTable.version(), newRoutingNodes).build(); - MetaData newMetaData = updateMetaDataWithRoutingTable(oldMetaData, oldRoutingTable, newRoutingTable); + MetaData newMetaData = allocation.updateMetaDataWithRoutingChanges(); assert newRoutingTable.validate(newMetaData); // validates the routing table is coherent with the cluster state metadata logClusterHealthStateChange( new ClusterStateHealth(ClusterState.builder(clusterName). - metaData(allocation.metaData()).routingTable(allocation.routingTable()).build()), + metaData(allocation.metaData()).routingTable(oldRoutingTable).build()), new ClusterStateHealth(ClusterState.builder(clusterName). metaData(newMetaData).routingTable(newRoutingTable).build()), reason ); - return new Result(true, newRoutingTable, newMetaData, explanations); - } - - /** - * Updates the current {@link MetaData} based on the newly created {@link RoutingTable}. Specifically - * we update {@link IndexMetaData#getActiveAllocationIds()} and {@link IndexMetaData#primaryTerm(int)} based on - * the changes made during this allocation. - * - * @param oldMetaData {@link MetaData} object from before the routing table was changed. - * @param oldRoutingTable {@link RoutingTable} from before the change. - * @param newRoutingTable new {@link RoutingTable} created by the allocation change - * @return adapted {@link MetaData}, potentially the original one if no change was needed. - */ - static MetaData updateMetaDataWithRoutingTable(MetaData oldMetaData, RoutingTable oldRoutingTable, RoutingTable newRoutingTable) { - MetaData.Builder metaDataBuilder = null; - for (IndexRoutingTable newIndexTable : newRoutingTable) { - final IndexMetaData oldIndexMetaData = oldMetaData.index(newIndexTable.getIndex()); - if (oldIndexMetaData == null) { - throw new IllegalStateException("no metadata found for index " + newIndexTable.getIndex().getName()); - } - IndexMetaData.Builder indexMetaDataBuilder = null; - for (IndexShardRoutingTable newShardTable : newIndexTable) { - final ShardId shardId = newShardTable.shardId(); - - // update activeAllocationIds - Set activeAllocationIds = newShardTable.activeShards().stream() - .map(ShardRouting::allocationId) - .filter(Objects::nonNull) - .map(AllocationId::getId) - .collect(Collectors.toSet()); - // only update active allocation ids if there is an active shard - if (activeAllocationIds.isEmpty() == false) { - // get currently stored allocation ids - Set storedAllocationIds = oldIndexMetaData.activeAllocationIds(shardId.id()); - if (activeAllocationIds.equals(storedAllocationIds) == false) { - if (indexMetaDataBuilder == null) { - indexMetaDataBuilder = IndexMetaData.builder(oldIndexMetaData); - } - indexMetaDataBuilder.putActiveAllocationIds(shardId.id(), activeAllocationIds); - } - } - - // update primary terms - final ShardRouting newPrimary = newShardTable.primaryShard(); - if (newPrimary == null) { - throw new IllegalStateException("missing primary shard for " + newShardTable.shardId()); - } - final ShardRouting oldPrimary = oldRoutingTable.shardRoutingTable(shardId).primaryShard(); - if (oldPrimary == null) { - throw new IllegalStateException("missing primary shard for " + newShardTable.shardId()); - } - // we update the primary term on initial assignment or when a replica is promoted. Most notably we do *not* - // update them when a primary relocates - if (newPrimary.unassigned() || - newPrimary.isSameAllocation(oldPrimary) || - // we do not use newPrimary.isTargetRelocationOf(oldPrimary) because that one enforces newPrimary to - // be initializing. However, when the target shard is activated, we still want the primary term to staty - // the same - (oldPrimary.relocating() && newPrimary.isSameAllocation(oldPrimary.getTargetRelocatingShard()))) { - // do nothing - } else { - // incrementing the primary term - if (indexMetaDataBuilder == null) { - indexMetaDataBuilder = IndexMetaData.builder(oldIndexMetaData); - } - indexMetaDataBuilder.primaryTerm(shardId.id(), oldIndexMetaData.primaryTerm(shardId.id()) + 1); - } - } - if (indexMetaDataBuilder != null) { - if (metaDataBuilder == null) { - metaDataBuilder = MetaData.builder(oldMetaData); - } - metaDataBuilder.put(indexMetaDataBuilder); - } - } - if (metaDataBuilder != null) { - return metaDataBuilder.build(); - } else { - return oldMetaData; - } + return Result.changed(newRoutingTable, newMetaData, explanations); } public Result applyFailedShard(ClusterState clusterState, ShardRouting failedShard) { @@ -223,7 +136,7 @@ public class AllocationService extends AbstractComponent { */ public Result applyFailedShards(ClusterState clusterState, List failedShards) { if (failedShards.isEmpty()) { - return new Result(false, clusterState.routingTable(), clusterState.metaData()); + return Result.unchanged(clusterState); } RoutingNodes routingNodes = getMutableRoutingNodes(clusterState); // shuffle the unassigned nodes, just so we won't have things like poison failed shards @@ -247,7 +160,7 @@ public class AllocationService extends AbstractComponent { UnassignedInfo unassignedInfo = new UnassignedInfo(UnassignedInfo.Reason.ALLOCATION_FAILED, failedShardEntry.message, failedShardEntry.failure, failedAllocations + 1, currentNanoTime, System.currentTimeMillis(), false, AllocationStatus.NO_ATTEMPT); - routingNodes.failShard(logger, failedShard, unassignedInfo, indexMetaData); + routingNodes.failShard(logger, failedShard, unassignedInfo, indexMetaData, allocation.changes()); } else { logger.trace("{} shard routing failed in an earlier iteration (routing: {})", shardToFail.shardId(), shardToFail); } @@ -271,25 +184,24 @@ public class AllocationService extends AbstractComponent { clusterInfoService.getClusterInfo(), currentNanoTime(), false); // first, clear from the shards any node id they used to belong to that is now dead - boolean changed = deassociateDeadNodes(allocation); + deassociateDeadNodes(allocation); if (reroute) { - changed |= reroute(allocation); + reroute(allocation); } - if (!changed) { - return new RoutingAllocation.Result(false, clusterState.routingTable(), clusterState.metaData()); + if (allocation.routingNodesChanged() == false) { + return Result.unchanged(clusterState); } return buildResultAndLogHealthChange(allocation, reason); } /** - * Removes delay markers from unassigned shards based on current time stamp. Returns true if markers were removed. + * Removes delay markers from unassigned shards based on current time stamp. */ - private boolean removeDelayMarkers(RoutingAllocation allocation) { + private void removeDelayMarkers(RoutingAllocation allocation) { final RoutingNodes.UnassignedShards.UnassignedIterator unassignedIterator = allocation.routingNodes().unassigned().iterator(); final MetaData metaData = allocation.metaData(); - boolean changed = false; while (unassignedIterator.hasNext()) { ShardRouting shardRouting = unassignedIterator.next(); UnassignedInfo unassignedInfo = shardRouting.unassignedInfo(); @@ -297,14 +209,12 @@ public class AllocationService extends AbstractComponent { final long newComputedLeftDelayNanos = unassignedInfo.getRemainingDelay(allocation.getCurrentNanoTime(), metaData.getIndexSafe(shardRouting.index()).getSettings()); if (newComputedLeftDelayNanos == 0) { - changed = true; unassignedIterator.updateUnassignedInfo(new UnassignedInfo(unassignedInfo.getReason(), unassignedInfo.getMessage(), unassignedInfo.getFailure(), unassignedInfo.getNumFailedAllocations(), unassignedInfo.getUnassignedTimeInNanos(), - unassignedInfo.getUnassignedTimeInMillis(), false, unassignedInfo.getLastAllocationStatus())); + unassignedInfo.getUnassignedTimeInMillis(), false, unassignedInfo.getLastAllocationStatus()), allocation.changes()); } } } - return changed; } /** @@ -366,8 +276,9 @@ public class AllocationService extends AbstractComponent { RoutingAllocation allocation = new RoutingAllocation(allocationDeciders, routingNodes, clusterState, clusterInfoService.getClusterInfo(), currentNanoTime(), false); allocation.debugDecision(debug); - if (!reroute(allocation)) { - return new Result(false, clusterState.routingTable(), clusterState.metaData()); + reroute(allocation); + if (allocation.routingNodesChanged() == false) { + return Result.unchanged(clusterState); } return buildResultAndLogHealthChange(allocation, reason); } @@ -380,43 +291,47 @@ public class AllocationService extends AbstractComponent { } } - private boolean reroute(RoutingAllocation allocation) { - assert deassociateDeadNodes(allocation) == false : "dead nodes should be explicitly cleaned up. See deassociateDeadNodes"; - - boolean changed = false; - // now allocate all the unassigned to available nodes - if (allocation.routingNodes().unassigned().size() > 0) { - changed |= removeDelayMarkers(allocation); - changed |= gatewayAllocator.allocateUnassigned(allocation); + private boolean hasDeadNodes(RoutingAllocation allocation) { + for (RoutingNode routingNode : allocation.routingNodes()) { + if (allocation.nodes().getDataNodes().containsKey(routingNode.nodeId()) == false) { + return true; + } } - - changed |= shardsAllocator.allocate(allocation); - assert RoutingNodes.assertShardStats(allocation.routingNodes()); - return changed; + return false; } - private boolean deassociateDeadNodes(RoutingAllocation allocation) { - boolean changed = false; + private void reroute(RoutingAllocation allocation) { + assert hasDeadNodes(allocation) == false : "dead nodes should be explicitly cleaned up. See deassociateDeadNodes"; + + // now allocate all the unassigned to available nodes + if (allocation.routingNodes().unassigned().size() > 0) { + removeDelayMarkers(allocation); + gatewayAllocator.allocateUnassigned(allocation); + } + + shardsAllocator.allocate(allocation); + assert RoutingNodes.assertShardStats(allocation.routingNodes()); + } + + private void deassociateDeadNodes(RoutingAllocation allocation) { for (Iterator it = allocation.routingNodes().mutableIterator(); it.hasNext(); ) { RoutingNode node = it.next(); if (allocation.nodes().getDataNodes().containsKey(node.nodeId())) { // its a live node, continue continue; } - changed = true; // now, go over all the shards routing on the node, and fail them for (ShardRouting shardRouting : node.copyShards()) { final IndexMetaData indexMetaData = allocation.metaData().getIndexSafe(shardRouting.index()); boolean delayed = INDEX_DELAYED_NODE_LEFT_TIMEOUT_SETTING.get(indexMetaData.getSettings()).nanos() > 0; UnassignedInfo unassignedInfo = new UnassignedInfo(UnassignedInfo.Reason.NODE_LEFT, "node_left[" + node.nodeId() + "]", null, 0, allocation.getCurrentNanoTime(), System.currentTimeMillis(), delayed, AllocationStatus.NO_ATTEMPT); - allocation.routingNodes().failShard(logger, shardRouting, unassignedInfo, indexMetaData); + allocation.routingNodes().failShard(logger, shardRouting, unassignedInfo, indexMetaData, allocation.changes()); } // its a dead node, remove it, note, its important to remove it *after* we apply failed shard // since it relies on the fact that the RoutingNode exists in the list of nodes it.remove(); } - return changed; } private void applyStartedShards(RoutingAllocation routingAllocation, List startedShardEntries) { @@ -430,7 +345,7 @@ public class AllocationService extends AbstractComponent { "shard routing to start does not exist in routing table, expected: " + startedShard + " but was: " + routingNodes.getByAllocationId(startedShard.shardId(), startedShard.allocationId().getId()); - routingNodes.startShard(logger, startedShard); + routingNodes.startShard(logger, startedShard, routingAllocation.changes()); } } diff --git a/core/src/main/java/org/elasticsearch/cluster/routing/allocation/IndexMetaDataUpdater.java b/core/src/main/java/org/elasticsearch/cluster/routing/allocation/IndexMetaDataUpdater.java new file mode 100644 index 00000000000..1f232cd8ac5 --- /dev/null +++ b/core/src/main/java/org/elasticsearch/cluster/routing/allocation/IndexMetaDataUpdater.java @@ -0,0 +1,178 @@ +/* + * 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; + +import org.elasticsearch.cluster.metadata.IndexMetaData; +import org.elasticsearch.cluster.metadata.MetaData; +import org.elasticsearch.cluster.routing.RoutingChangesObserver; +import org.elasticsearch.cluster.routing.ShardRouting; +import org.elasticsearch.cluster.routing.UnassignedInfo; +import org.elasticsearch.common.util.set.Sets; +import org.elasticsearch.index.Index; +import org.elasticsearch.index.shard.ShardId; + +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.stream.Collectors; + +/** + * Observer that tracks changes made to RoutingNodes in order to update the primary terms and in-sync allocation ids in + * {@link IndexMetaData} once the allocation round has completed. + * + * Primary terms are updated on primary initialization or primary promotion. + * + * Allocation ids are added for shards that become active and removed for shards that stop being active. + */ +public class IndexMetaDataUpdater extends RoutingChangesObserver.AbstractRoutingChangesObserver { + private final Map shardChanges = new HashMap<>(); + + @Override + public void shardInitialized(ShardRouting unassignedShard) { + if (unassignedShard.primary()) { + increasePrimaryTerm(unassignedShard); + } + } + + @Override + public void replicaPromoted(ShardRouting replicaShard) { + increasePrimaryTerm(replicaShard); + } + + @Override + public void shardStarted(ShardRouting initializingShard, ShardRouting startedShard) { + addAllocationId(startedShard); + } + + @Override + public void shardFailed(ShardRouting failedShard, UnassignedInfo unassignedInfo) { + if (failedShard.active()) { + removeAllocationId(failedShard); + } + } + + @Override + public void relocationCompleted(ShardRouting removedRelocationSource) { + removeAllocationId(removedRelocationSource); + } + + @Override + public void startedPrimaryReinitialized(ShardRouting startedPrimaryShard, ShardRouting initializedShard) { + removeAllocationId(startedPrimaryShard); + } + + /** + * Updates the current {@link MetaData} based on the changes of this RoutingChangesObserver. Specifically + * we update {@link IndexMetaData#getActiveAllocationIds()} and {@link IndexMetaData#primaryTerm(int)} based on + * the changes made during this allocation. + * + * @param oldMetaData {@link MetaData} object from before the routing nodes was changed. + * @return adapted {@link MetaData}, potentially the original one if no change was needed. + */ + public MetaData applyChanges(MetaData oldMetaData) { + Map>> changesGroupedByIndex = + shardChanges.entrySet().stream().collect(Collectors.groupingBy(e -> e.getKey().getIndex())); + + MetaData.Builder metaDataBuilder = null; + for (Map.Entry>> indexChanges : changesGroupedByIndex.entrySet()) { + Index index = indexChanges.getKey(); + final IndexMetaData oldIndexMetaData = oldMetaData.index(index); + if (oldIndexMetaData == null) { + throw new IllegalStateException("no metadata found for index " + index); + } + IndexMetaData.Builder indexMetaDataBuilder = null; + for (Map.Entry shardEntry : indexChanges.getValue()) { + ShardId shardId = shardEntry.getKey(); + Updates updates = shardEntry.getValue(); + + assert Sets.haveEmptyIntersection(updates.addedAllocationIds, updates.removedAllocationIds) : + "Allocation ids cannot be both added and removed in the same allocation round, added ids: " + + updates.addedAllocationIds + ", removed ids: " + updates.removedAllocationIds; + + Set activeAllocationIds = new HashSet<>(oldIndexMetaData.activeAllocationIds(shardId.id())); + activeAllocationIds.addAll(updates.addedAllocationIds); + activeAllocationIds.removeAll(updates.removedAllocationIds); + // only update active allocation ids if there is an active shard + if (activeAllocationIds.isEmpty() == false) { + if (indexMetaDataBuilder == null) { + indexMetaDataBuilder = IndexMetaData.builder(oldIndexMetaData); + } + indexMetaDataBuilder.putActiveAllocationIds(shardId.id(), activeAllocationIds); + } + + if (updates.increaseTerm) { + if (indexMetaDataBuilder == null) { + indexMetaDataBuilder = IndexMetaData.builder(oldIndexMetaData); + } + indexMetaDataBuilder.primaryTerm(shardId.id(), oldIndexMetaData.primaryTerm(shardId.id()) + 1); + } + } + + if (indexMetaDataBuilder != null) { + if (metaDataBuilder == null) { + metaDataBuilder = MetaData.builder(oldMetaData); + } + metaDataBuilder.put(indexMetaDataBuilder); + } + } + + if (metaDataBuilder != null) { + return metaDataBuilder.build(); + } else { + return oldMetaData; + } + } + + /** + * Helper method that creates update entry for the given shard id if such an entry does not exist yet. + */ + private Updates changes(ShardId shardId) { + return shardChanges.computeIfAbsent(shardId, k -> new Updates()); + } + + /** + * Remove allocation id of this shard from the set of in-sync shard copies + */ + private void removeAllocationId(ShardRouting shardRouting) { + changes(shardRouting.shardId()).removedAllocationIds.add(shardRouting.allocationId().getId()); + } + + /** + * Add allocation id of this shard to the set of in-sync shard copies + */ + private void addAllocationId(ShardRouting shardRouting) { + changes(shardRouting.shardId()).addedAllocationIds.add(shardRouting.allocationId().getId()); + } + + /** + * Increase primary term for this shard id + */ + private void increasePrimaryTerm(ShardRouting shardRouting) { + changes(shardRouting.shardId()).increaseTerm = true; + } + + private static class Updates { + private boolean increaseTerm; // whether primary term should be increased + private Set addedAllocationIds = new HashSet<>(); // allocation ids that should be added to the in-sync set + private Set removedAllocationIds = new HashSet<>(); // allocation ids that should be removed from the in-sync set + } +} diff --git a/core/src/main/java/org/elasticsearch/cluster/routing/allocation/RoutingAllocation.java b/core/src/main/java/org/elasticsearch/cluster/routing/allocation/RoutingAllocation.java index d26b976e6be..0794c6d828e 100644 --- a/core/src/main/java/org/elasticsearch/cluster/routing/allocation/RoutingAllocation.java +++ b/core/src/main/java/org/elasticsearch/cluster/routing/allocation/RoutingAllocation.java @@ -23,6 +23,7 @@ import org.elasticsearch.cluster.ClusterInfo; import org.elasticsearch.cluster.ClusterState; import org.elasticsearch.cluster.metadata.MetaData; import org.elasticsearch.cluster.node.DiscoveryNodes; +import org.elasticsearch.cluster.routing.RoutingChangesObserver; import org.elasticsearch.cluster.routing.RoutingNodes; import org.elasticsearch.cluster.routing.RoutingTable; import org.elasticsearch.cluster.routing.allocation.decider.AllocationDeciders; @@ -59,13 +60,21 @@ public class RoutingAllocation { private final RoutingExplanations explanations; /** - * Creates a new {@link RoutingAllocation.Result} - * @param changed a flag to determine whether the actual {@link RoutingTable} has been changed + * Creates a new {@link RoutingAllocation.Result} where no change to the routing table was made. + * @param clusterState the unchanged {@link ClusterState} + */ + public static Result unchanged(ClusterState clusterState) { + return new Result(false, clusterState.routingTable(), clusterState.metaData(), new RoutingExplanations()); + } + + /** + * Creates a new {@link RoutingAllocation.Result} where changes were made to the routing table. * @param routingTable the {@link RoutingTable} this Result references * @param metaData the {@link MetaData} this Result references + * @param explanations Explanation for the reroute actions */ - public Result(boolean changed, RoutingTable routingTable, MetaData metaData) { - this(changed, routingTable, metaData, new RoutingExplanations()); + public static Result changed(RoutingTable routingTable, MetaData metaData, RoutingExplanations explanations) { + return new Result(true, routingTable, metaData, explanations); } /** @@ -75,7 +84,7 @@ public class RoutingAllocation { * @param metaData the {@link MetaData} this Result references * @param explanations Explanation for the reroute actions */ - public Result(boolean changed, RoutingTable routingTable, MetaData metaData, RoutingExplanations explanations) { + private Result(boolean changed, RoutingTable routingTable, MetaData metaData, RoutingExplanations explanations) { this.changed = changed; this.routingTable = routingTable; this.metaData = metaData; @@ -142,6 +151,12 @@ public class RoutingAllocation { private final long currentNanoTime; + private final IndexMetaDataUpdater indexMetaDataUpdater = new IndexMetaDataUpdater(); + private final RoutingNodesChangedObserver nodesChangedObserver = new RoutingNodesChangedObserver(); + private final RoutingChangesObserver routingChangesObserver = new RoutingChangesObserver.DelegatingRoutingChangesObserver( + nodesChangedObserver, indexMetaDataUpdater + ); + /** * Creates a new {@link RoutingAllocation} @@ -252,6 +267,17 @@ public class RoutingAllocation { nodes.add(nodeId); } + /** + * Returns whether the given node id should be ignored from consideration when {@link AllocationDeciders} + * is deciding whether to allocate the specified shard id to that node. The node will be ignored if + * the specified shard failed on that node, triggering the current round of allocation. Since the shard + * just failed on that node, we don't want to try to reassign it there, if the node is still a part + * of the cluster. + * + * @param shardId the shard id to be allocated + * @param nodeId the node id to check against + * @return true if the node id should be ignored in allocation decisions, false otherwise + */ public boolean shouldIgnoreShardForNode(ShardId shardId, String nodeId) { if (ignoredShardToNodes == null) { return false; @@ -271,6 +297,27 @@ public class RoutingAllocation { return unmodifiableSet(new HashSet<>(ignore)); } + /** + * Returns observer to use for changes made to the routing nodes + */ + public RoutingChangesObserver changes() { + return routingChangesObserver; + } + + /** + * Returns updated {@link MetaData} based on the changes that were made to the routing nodes + */ + public MetaData updateMetaDataWithRoutingChanges() { + return indexMetaDataUpdater.applyChanges(metaData); + } + + /** + * Returns true iff changes were made to the routing nodes + */ + public boolean routingNodesChanged() { + return nodesChangedObserver.isChanged(); + } + /** * Create a routing decision, including the reason if the debug flag is * turned on diff --git a/core/src/main/java/org/elasticsearch/cluster/routing/allocation/RoutingNodesChangedObserver.java b/core/src/main/java/org/elasticsearch/cluster/routing/allocation/RoutingNodesChangedObserver.java new file mode 100644 index 00000000000..c971687234f --- /dev/null +++ b/core/src/main/java/org/elasticsearch/cluster/routing/allocation/RoutingNodesChangedObserver.java @@ -0,0 +1,104 @@ +/* + * 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; + +import org.elasticsearch.cluster.routing.RoutingChangesObserver; +import org.elasticsearch.cluster.routing.RoutingNodes; +import org.elasticsearch.cluster.routing.ShardRouting; +import org.elasticsearch.cluster.routing.UnassignedInfo; + +/** + * Records if changes were made to {@link RoutingNodes} during an allocation round. + */ +public class RoutingNodesChangedObserver implements RoutingChangesObserver { + private boolean changed; + + /** + * Returns whether changes were made + */ + public boolean isChanged() { + return changed; + } + + @Override + public void shardInitialized(ShardRouting unassignedShard) { + assert unassignedShard.unassigned() : "expected unassigned shard " + unassignedShard; + setChanged(); + } + + @Override + public void shardStarted(ShardRouting initializingShard, ShardRouting startedShard) { + assert initializingShard.initializing() : "expected initializing shard " + initializingShard; + assert startedShard.started() : "expected started shard " + startedShard; + setChanged(); + } + + @Override + public void relocationStarted(ShardRouting startedShard, ShardRouting targetRelocatingShard) { + assert startedShard.started() : "expected started shard " + startedShard; + assert targetRelocatingShard.isRelocationTarget() : "expected relocation target shard " + targetRelocatingShard; + setChanged(); + } + + @Override + public void unassignedInfoUpdated(ShardRouting unassignedShard, UnassignedInfo newUnassignedInfo) { + assert unassignedShard.unassigned() : "expected unassigned shard " + unassignedShard; + setChanged(); + } + + @Override + public void shardFailed(ShardRouting failedShard, UnassignedInfo unassignedInfo) { + assert failedShard.assignedToNode() : "expected assigned shard " + failedShard; + setChanged(); + } + + @Override + public void relocationCompleted(ShardRouting removedRelocationSource) { + assert removedRelocationSource.relocating() : "expected relocating shard " + removedRelocationSource; + setChanged(); + } + + @Override + public void relocationSourceRemoved(ShardRouting removedReplicaRelocationSource) { + assert removedReplicaRelocationSource.primary() == false && removedReplicaRelocationSource.isRelocationTarget() : + "expected replica relocation target shard " + removedReplicaRelocationSource; + setChanged(); + } + + @Override + public void startedPrimaryReinitialized(ShardRouting startedPrimaryShard, ShardRouting initializedShard) { + assert startedPrimaryShard.primary() && startedPrimaryShard.started() : "expected started primary shard " + startedPrimaryShard; + assert initializedShard.primary() && initializedShard.initializing(): "expected initializing primary shard " + initializedShard; + setChanged(); + } + + @Override + public void replicaPromoted(ShardRouting replicaShard) { + assert replicaShard.started() && replicaShard.primary() == false : "expected started replica shard " + replicaShard; + setChanged(); + } + + /** + * Marks the allocation as changed. + */ + private void setChanged() { + changed = true; + } +} diff --git a/core/src/main/java/org/elasticsearch/cluster/routing/allocation/allocator/BalancedShardsAllocator.java b/core/src/main/java/org/elasticsearch/cluster/routing/allocation/allocator/BalancedShardsAllocator.java index ed856f44e68..c86e256bd6f 100644 --- a/core/src/main/java/org/elasticsearch/cluster/routing/allocation/allocator/BalancedShardsAllocator.java +++ b/core/src/main/java/org/elasticsearch/cluster/routing/allocation/allocator/BalancedShardsAllocator.java @@ -111,16 +111,15 @@ public class BalancedShardsAllocator extends AbstractComponent implements Shards } @Override - public boolean allocate(RoutingAllocation allocation) { + public void allocate(RoutingAllocation allocation) { if (allocation.routingNodes().size() == 0) { /* with no nodes this is pointless */ - return false; + return; } final Balancer balancer = new Balancer(logger, allocation, weightFunction, threshold); - boolean changed = balancer.allocateUnassigned(); - changed |= balancer.moveShards(); - changed |= balancer.balance(); - return changed; + balancer.allocateUnassigned(); + balancer.moveShards(); + balancer.balance(); } /** @@ -277,11 +276,8 @@ public class BalancedShardsAllocator extends AbstractComponent implements Shards /** * Balances the nodes on the cluster model according to the weight function. * The actual balancing is delegated to {@link #balanceByWeights()} - * - * @return true if the current configuration has been - * changed, otherwise false */ - private boolean balance() { + private void balance() { if (logger.isTraceEnabled()) { logger.trace("Start balancing cluster"); } @@ -294,17 +290,17 @@ public class BalancedShardsAllocator extends AbstractComponent implements Shards * Therefore we only do a rebalance if we have fetched all information. */ logger.debug("skipping rebalance due to in-flight shard/store fetches"); - return false; + return; } if (allocation.deciders().canRebalance(allocation).type() != Type.YES) { logger.trace("skipping rebalance as it is disabled"); - return false; + return; } if (nodes.size() < 2) { /* skip if we only have one node */ logger.trace("skipping rebalance as single node only"); - return false; + return; } - return balanceByWeights(); + balanceByWeights(); } public Map weighShard(ShardRouting shard) { @@ -340,12 +336,8 @@ public class BalancedShardsAllocator extends AbstractComponent implements Shards * the threshold is set to 1.0 to enforce gaining relocation * only, or in other words relocations that move the weight delta closer * to 0.0 - * - * @return true if the current configuration has been - * changed, otherwise false */ - private boolean balanceByWeights() { - boolean changed = false; + private void balanceByWeights() { final NodeSorter sorter = newNodeSorter(); final AllocationDeciders deciders = allocation.deciders(); final ModelNode[] modelNodes = sorter.modelNodes; @@ -419,7 +411,6 @@ public class BalancedShardsAllocator extends AbstractComponent implements Shards sorter.sort(0, relevantNodes); lowIdx = 0; highIdx = relevantNodes - 1; - changed = true; continue; } } @@ -441,7 +432,6 @@ public class BalancedShardsAllocator extends AbstractComponent implements Shards } } } - return changed; } /** @@ -506,14 +496,11 @@ public class BalancedShardsAllocator extends AbstractComponent implements Shards * {@link ShardRoutingState#RELOCATING} and a shadow instance of this * shard is created with an incremented version in the state * {@link ShardRoutingState#INITIALIZING}. - * - * @return true if the allocation has changed, otherwise false */ - public boolean moveShards() { + public void moveShards() { // Iterate over the started shards interleaving between nodes, and check if they can remain. In the presence of throttling // shard movements, the goal of this iteration order is to achieve a fairer movement of shards from the nodes that are // offloading the shards. - boolean changed = false; final NodeSorter sorter = newNodeSorter(); for (Iterator it = allocation.routingNodes().nodeInterleavedShardIterator(); it.hasNext(); ) { ShardRouting shardRouting = it.next(); @@ -524,20 +511,16 @@ public class BalancedShardsAllocator extends AbstractComponent implements Shards RoutingNode routingNode = sourceNode.getRoutingNode(); Decision decision = allocation.deciders().canRemain(shardRouting, routingNode, allocation); if (decision.type() == Decision.Type.NO) { - changed |= moveShard(sorter, shardRouting, sourceNode, routingNode); + moveShard(sorter, shardRouting, sourceNode, routingNode); } } } - - return changed; } /** * Move started shard to the minimal eligible node with respect to the weight function - * - * @return true if the shard was moved successfully, otherwise false */ - private boolean moveShard(NodeSorter sorter, ShardRouting shardRouting, ModelNode sourceNode, RoutingNode routingNode) { + private void moveShard(NodeSorter sorter, ShardRouting shardRouting, ModelNode sourceNode, RoutingNode routingNode) { logger.debug("[{}][{}] allocated on [{}], but can no longer be allocated on it, moving...", shardRouting.index(), shardRouting.id(), routingNode.node()); sorter.reset(shardRouting.getIndexName()); /* @@ -553,17 +536,16 @@ public class BalancedShardsAllocator extends AbstractComponent implements Shards Decision allocationDecision = allocation.deciders().canAllocate(shardRouting, target, allocation); if (allocationDecision.type() == Type.YES) { // TODO maybe we can respect throttling here too? sourceNode.removeShard(shardRouting); - Tuple relocatingShards = routingNodes.relocateShard(shardRouting, target.nodeId(), allocation.clusterInfo().getShardSize(shardRouting, ShardRouting.UNAVAILABLE_EXPECTED_SHARD_SIZE)); + Tuple relocatingShards = routingNodes.relocateShard(shardRouting, target.nodeId(), allocation.clusterInfo().getShardSize(shardRouting, ShardRouting.UNAVAILABLE_EXPECTED_SHARD_SIZE), allocation.changes()); currentNode.addShard(relocatingShards.v2()); if (logger.isTraceEnabled()) { logger.trace("Moved shard [{}] to node [{}]", shardRouting, routingNode.node()); } - return true; + return; } } } logger.debug("[{}][{}] can't move", shardRouting.index(), shardRouting.id()); - return false; } /** @@ -595,19 +577,16 @@ public class BalancedShardsAllocator extends AbstractComponent implements Shards /** * Allocates all given shards on the minimal eligible node for the shards index * with respect to the weight function. All given shards must be unassigned. - * @return true if the current configuration has been - * changed, otherwise false */ - private boolean allocateUnassigned() { + private void allocateUnassigned() { RoutingNodes.UnassignedShards unassigned = routingNodes.unassigned(); assert !nodes.isEmpty(); if (logger.isTraceEnabled()) { logger.trace("Start allocating unassigned shards"); } if (unassigned.isEmpty()) { - return false; + return; } - boolean changed = false; /* * TODO: We could be smarter here and group the shards by index and then @@ -651,9 +630,9 @@ public class BalancedShardsAllocator extends AbstractComponent implements Shards final Decision decision = deciders.canAllocate(shard, allocation); if (decision.type() == Type.NO) { UnassignedInfo.AllocationStatus allocationStatus = UnassignedInfo.AllocationStatus.fromDecision(decision); - changed |= unassigned.ignoreShard(shard, allocationStatus); + unassigned.ignoreShard(shard, allocationStatus, allocation.changes()); while(i < primaryLength-1 && comparator.compare(primary[i], primary[i+1]) == 0) { - changed |= unassigned.ignoreShard(primary[++i], allocationStatus); + unassigned.ignoreShard(primary[++i], allocationStatus, allocation.changes()); } continue; } else { @@ -728,9 +707,8 @@ public class BalancedShardsAllocator extends AbstractComponent implements Shards logger.trace("Assigned shard [{}] to [{}]", shard, minNode.getNodeId()); } - shard = routingNodes.initializeShard(shard, minNode.getNodeId(), null, shardSize); + shard = routingNodes.initializeShard(shard, minNode.getNodeId(), null, shardSize, allocation.changes()); minNode.addShard(shard); - changed = true; continue; // don't add to ignoreUnassigned } else { minNode.addShard(shard.initialize(minNode.getNodeId(), null, shardSize)); @@ -754,10 +732,10 @@ public class BalancedShardsAllocator extends AbstractComponent implements Shards UnassignedInfo.AllocationStatus allocationStatus = decision == null ? UnassignedInfo.AllocationStatus.DECIDERS_NO : UnassignedInfo.AllocationStatus.fromDecision(decision); - changed |= unassigned.ignoreShard(shard, allocationStatus); + unassigned.ignoreShard(shard, allocationStatus, allocation.changes()); if (!shard.primary()) { // we could not allocate it and we are a replica - check if we can ignore the other replicas while(secondaryLength > 0 && comparator.compare(shard, secondary[secondaryLength-1]) == 0) { - changed |= unassigned.ignoreShard(secondary[--secondaryLength], allocationStatus); + unassigned.ignoreShard(secondary[--secondaryLength], allocationStatus, allocation.changes()); } } } @@ -768,7 +746,6 @@ public class BalancedShardsAllocator extends AbstractComponent implements Shards secondaryLength = 0; } while (primaryLength > 0); // clear everything we have either added it or moved to ignoreUnassigned - return changed; } /** @@ -820,7 +797,7 @@ public class BalancedShardsAllocator extends AbstractComponent implements Shards minNode.getNodeId()); } /* now allocate on the cluster */ - minNode.addShard(routingNodes.relocateShard(candidate, minNode.getNodeId(), shardSize).v1()); + minNode.addShard(routingNodes.relocateShard(candidate, minNode.getNodeId(), shardSize, allocation.changes()).v1()); return true; } else { assert decision.type() == Type.THROTTLE; diff --git a/core/src/main/java/org/elasticsearch/cluster/routing/allocation/allocator/ShardsAllocator.java b/core/src/main/java/org/elasticsearch/cluster/routing/allocation/allocator/ShardsAllocator.java index aa59e7788f3..35f3b265418 100644 --- a/core/src/main/java/org/elasticsearch/cluster/routing/allocation/allocator/ShardsAllocator.java +++ b/core/src/main/java/org/elasticsearch/cluster/routing/allocation/allocator/ShardsAllocator.java @@ -40,9 +40,8 @@ public interface ShardsAllocator { * - relocate shards to find a good shard balance in the cluster * * @param allocation current node allocation - * @return true if the allocation has changed, otherwise false */ - boolean allocate(RoutingAllocation allocation); + void allocate(RoutingAllocation allocation); /** * Returns a map of node to a float "weight" of where the allocator would like to place the shard. diff --git a/core/src/main/java/org/elasticsearch/cluster/routing/allocation/command/AbstractAllocateAllocationCommand.java b/core/src/main/java/org/elasticsearch/cluster/routing/allocation/command/AbstractAllocateAllocationCommand.java index 2c5aa670f4f..f9b5a398ba7 100644 --- a/core/src/main/java/org/elasticsearch/cluster/routing/allocation/command/AbstractAllocateAllocationCommand.java +++ b/core/src/main/java/org/elasticsearch/cluster/routing/allocation/command/AbstractAllocateAllocationCommand.java @@ -207,9 +207,9 @@ public abstract class AbstractAllocateAllocationCommand implements AllocationCom continue; } if (unassignedInfo != null) { - unassigned = it.updateUnassignedInfo(unassignedInfo); + unassigned = it.updateUnassignedInfo(unassignedInfo, allocation.changes()); } - it.initialize(routingNode.nodeId(), null, allocation.clusterInfo().getShardSize(unassigned, ShardRouting.UNAVAILABLE_EXPECTED_SHARD_SIZE)); + it.initialize(routingNode.nodeId(), null, allocation.clusterInfo().getShardSize(unassigned, ShardRouting.UNAVAILABLE_EXPECTED_SHARD_SIZE), allocation.changes()); return; } assert false : "shard to initialize not found in list of unassigned shards"; diff --git a/core/src/main/java/org/elasticsearch/cluster/routing/allocation/command/CancelAllocationCommand.java b/core/src/main/java/org/elasticsearch/cluster/routing/allocation/command/CancelAllocationCommand.java index 349df2d7af7..eae4739c127 100644 --- a/core/src/main/java/org/elasticsearch/cluster/routing/allocation/command/CancelAllocationCommand.java +++ b/core/src/main/java/org/elasticsearch/cluster/routing/allocation/command/CancelAllocationCommand.java @@ -26,7 +26,6 @@ import org.elasticsearch.cluster.routing.RoutingNode; import org.elasticsearch.cluster.routing.RoutingNodes; import org.elasticsearch.cluster.routing.ShardRouting; import org.elasticsearch.cluster.routing.UnassignedInfo; -import org.elasticsearch.cluster.routing.allocation.AllocationService; import org.elasticsearch.cluster.routing.allocation.RerouteExplanation; import org.elasticsearch.cluster.routing.allocation.RoutingAllocation; import org.elasticsearch.cluster.routing.allocation.decider.Decision; @@ -156,7 +155,7 @@ public class CancelAllocationCommand implements AllocationCommand { } } routingNodes.failShard(Loggers.getLogger(CancelAllocationCommand.class), shardRouting, - new UnassignedInfo(UnassignedInfo.Reason.REROUTE_CANCELLED, null), indexMetaData); + new UnassignedInfo(UnassignedInfo.Reason.REROUTE_CANCELLED, null), indexMetaData, allocation.changes()); return new RerouteExplanation(this, allocation.decision(Decision.YES, "cancel_allocation_command", "shard " + shardId + " on node " + discoNode + " can be cancelled")); } diff --git a/core/src/main/java/org/elasticsearch/cluster/routing/allocation/command/MoveAllocationCommand.java b/core/src/main/java/org/elasticsearch/cluster/routing/allocation/command/MoveAllocationCommand.java index dbd345a81a0..6e302d458ba 100644 --- a/core/src/main/java/org/elasticsearch/cluster/routing/allocation/command/MoveAllocationCommand.java +++ b/core/src/main/java/org/elasticsearch/cluster/routing/allocation/command/MoveAllocationCommand.java @@ -132,7 +132,7 @@ public class MoveAllocationCommand implements AllocationCommand { if (decision.type() == Decision.Type.THROTTLE) { // its being throttled, maybe have a flag to take it into account and fail? for now, just do it since the "user" wants it... } - allocation.routingNodes().relocateShard(shardRouting, toRoutingNode.nodeId(), allocation.clusterInfo().getShardSize(shardRouting, ShardRouting.UNAVAILABLE_EXPECTED_SHARD_SIZE)); + allocation.routingNodes().relocateShard(shardRouting, toRoutingNode.nodeId(), allocation.clusterInfo().getShardSize(shardRouting, ShardRouting.UNAVAILABLE_EXPECTED_SHARD_SIZE), allocation.changes()); } if (!found) { diff --git a/core/src/main/java/org/elasticsearch/cluster/routing/allocation/decider/AllocationDecider.java b/core/src/main/java/org/elasticsearch/cluster/routing/allocation/decider/AllocationDecider.java index 3bd4069ac73..011b6c97737 100644 --- a/core/src/main/java/org/elasticsearch/cluster/routing/allocation/decider/AllocationDecider.java +++ b/core/src/main/java/org/elasticsearch/cluster/routing/allocation/decider/AllocationDecider.java @@ -23,6 +23,7 @@ import org.elasticsearch.cluster.metadata.IndexMetaData; import org.elasticsearch.cluster.routing.RoutingNode; import org.elasticsearch.cluster.routing.ShardRouting; import org.elasticsearch.cluster.routing.allocation.RoutingAllocation; +import org.elasticsearch.cluster.routing.allocation.decider.Decision.Type; import org.elasticsearch.common.component.AbstractComponent; import org.elasticsearch.common.settings.Settings; @@ -98,4 +99,30 @@ public abstract class AllocationDecider extends AbstractComponent { public Decision canRebalance(RoutingAllocation allocation) { return Decision.ALWAYS; } + + /** + * Returns a {@link Decision} whether the given primary shard can be + * forcibly allocated on the given node. This method should only be called + * for unassigned primary shards where the node has a shard copy on disk. + * + * Note: all implementations that override this behavior should take into account + * the results of {@link #canAllocate(ShardRouting, RoutingNode, RoutingAllocation)} + * before making a decision on force allocation, because force allocation should only + * be considered if all deciders return {@link Decision#NO}. + */ + public Decision canForceAllocatePrimary(ShardRouting shardRouting, RoutingNode node, RoutingAllocation allocation) { + assert shardRouting.primary() : "must not call canForceAllocatePrimary on a non-primary shard " + shardRouting; + assert shardRouting.unassigned() : "must not call canForceAllocatePrimary on an assigned shard " + shardRouting; + Decision decision = canAllocate(shardRouting, node, allocation); + if (decision.type() == Type.NO) { + // On a NO decision, by default, we allow force allocating the primary. + return allocation.decision(Decision.YES, + decision.label(), + "primary shard [{}] allowed to force allocate on node [{}]", + shardRouting.shardId(), node.nodeId()); + } else { + // On a THROTTLE/YES decision, we use the same decision instead of forcing allocation + return decision; + } + } } diff --git a/core/src/main/java/org/elasticsearch/cluster/routing/allocation/decider/AllocationDeciders.java b/core/src/main/java/org/elasticsearch/cluster/routing/allocation/decider/AllocationDeciders.java index 059748c3f62..841b374e87d 100644 --- a/core/src/main/java/org/elasticsearch/cluster/routing/allocation/decider/AllocationDeciders.java +++ b/core/src/main/java/org/elasticsearch/cluster/routing/allocation/decider/AllocationDeciders.java @@ -196,4 +196,32 @@ public class AllocationDeciders extends AllocationDecider { } return ret; } + + @Override + public Decision canForceAllocatePrimary(ShardRouting shardRouting, RoutingNode node, RoutingAllocation allocation) { + assert shardRouting.primary() : "must not call canForceAllocatePrimary on a non-primary shard routing " + shardRouting; + + if (allocation.shouldIgnoreShardForNode(shardRouting.shardId(), node.nodeId())) { + return Decision.NO; + } + Decision.Multi ret = new Decision.Multi(); + for (AllocationDecider decider : allocations) { + Decision decision = decider.canForceAllocatePrimary(shardRouting, node, allocation); + // short track if a NO is returned. + if (decision == Decision.NO) { + if (logger.isTraceEnabled()) { + logger.trace("Shard [{}] can not be forcefully allocated to node [{}] due to [{}].", + shardRouting.shardId(), node.nodeId(), decider.getClass().getSimpleName()); + } + if (!allocation.debugDecision()) { + return decision; + } else { + ret.add(decision); + } + } else if (decision != Decision.ALWAYS) { + ret.add(decision); + } + } + return ret; + } } diff --git a/core/src/main/java/org/elasticsearch/cluster/routing/allocation/decider/Decision.java b/core/src/main/java/org/elasticsearch/cluster/routing/allocation/decider/Decision.java index 02b3978d06e..3792f536f2f 100644 --- a/core/src/main/java/org/elasticsearch/cluster/routing/allocation/decider/Decision.java +++ b/core/src/main/java/org/elasticsearch/cluster/routing/allocation/decider/Decision.java @@ -53,7 +53,7 @@ public abstract class Decision implements ToXContent { * @param explanationParams additional parameters for the decision * @return new {@link Decision} instance */ - public static Decision single(Type type, String label, String explanation, Object... explanationParams) { + public static Decision single(Type type, @Nullable String label, @Nullable String explanation, @Nullable Object... explanationParams) { return new Single(type, label, explanation, explanationParams); } @@ -146,6 +146,9 @@ public abstract class Decision implements ToXContent { */ public abstract Type type(); + /** + * Get the description label for this decision. + */ @Nullable public abstract String label(); diff --git a/core/src/main/java/org/elasticsearch/cluster/routing/allocation/decider/MaxRetryAllocationDecider.java b/core/src/main/java/org/elasticsearch/cluster/routing/allocation/decider/MaxRetryAllocationDecider.java index 6a8a0ccc5fa..74db79d36ba 100644 --- a/core/src/main/java/org/elasticsearch/cluster/routing/allocation/decider/MaxRetryAllocationDecider.java +++ b/core/src/main/java/org/elasticsearch/cluster/routing/allocation/decider/MaxRetryAllocationDecider.java @@ -80,4 +80,12 @@ public class MaxRetryAllocationDecider extends AllocationDecider { public Decision canAllocate(ShardRouting shardRouting, RoutingNode node, RoutingAllocation allocation) { return canAllocate(shardRouting, allocation); } + + @Override + public Decision canForceAllocatePrimary(ShardRouting shardRouting, RoutingNode node, RoutingAllocation allocation) { + assert shardRouting.primary() : "must not call canForceAllocatePrimary on a non-primary shard " + shardRouting; + // check if we have passed the maximum retry threshold through canAllocate, + // if so, we don't want to force the primary allocation here + return canAllocate(shardRouting, node, allocation); + } } diff --git a/core/src/main/java/org/elasticsearch/common/network/NetworkModule.java b/core/src/main/java/org/elasticsearch/common/network/NetworkModule.java index f6c2ef326f2..bb4a4bd3b30 100644 --- a/core/src/main/java/org/elasticsearch/common/network/NetworkModule.java +++ b/core/src/main/java/org/elasticsearch/common/network/NetworkModule.java @@ -19,9 +19,6 @@ package org.elasticsearch.common.network; -import java.util.ArrayList; -import java.util.List; - import org.elasticsearch.action.support.replication.ReplicationTask; import org.elasticsearch.cluster.routing.allocation.command.AllocateEmptyPrimaryAllocationCommand; import org.elasticsearch.cluster.routing.allocation.command.AllocateReplicaAllocationCommand; @@ -33,6 +30,7 @@ import org.elasticsearch.cluster.routing.allocation.command.MoveAllocationComman import org.elasticsearch.common.ParseField; import org.elasticsearch.common.inject.AbstractModule; import org.elasticsearch.common.inject.util.Providers; +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; @@ -47,6 +45,9 @@ import org.elasticsearch.transport.Transport; import org.elasticsearch.transport.TransportService; import org.elasticsearch.transport.local.LocalTransport; +import java.util.ArrayList; +import java.util.List; + /** * A module to handle registering and binding all network related classes. */ @@ -76,11 +77,11 @@ public class NetworkModule extends AbstractModule { private final ExtensionPoint.SelectedType transportServiceTypes = new ExtensionPoint.SelectedType<>("transport_service", TransportService.class); private final ExtensionPoint.SelectedType transportTypes = new ExtensionPoint.SelectedType<>("transport", Transport.class); private final ExtensionPoint.SelectedType httpTransportTypes = new ExtensionPoint.SelectedType<>("http_transport", HttpServerTransport.class); - private final List namedWriteables = new ArrayList<>(); + private final List namedWriteables = new ArrayList<>(); /** * Creates a network module that custom networking classes can be plugged into. - * @param networkService A constructed network service object to bind. + * @param networkService A constructed network service object to bind. * @param settings The settings for the node * @param transportClient True if only transport classes should be allowed to be registered, false otherwise. */ @@ -90,8 +91,8 @@ public class NetworkModule extends AbstractModule { this.transportClient = transportClient; registerTransportService("default", TransportService.class); registerTransport(LOCAL_TRANSPORT, LocalTransport.class); - registerTaskStatus(ReplicationTask.Status.NAME, ReplicationTask.Status::new); - registerTaskStatus(RawTaskStatus.NAME, RawTaskStatus::new); + namedWriteables.add(new NamedWriteableRegistry.Entry(Task.Status.class, ReplicationTask.Status.NAME, ReplicationTask.Status::new)); + namedWriteables.add(new NamedWriteableRegistry.Entry(Task.Status.class, RawTaskStatus.NAME, RawTaskStatus::new)); registerBuiltinAllocationCommands(); } @@ -118,10 +119,6 @@ public class NetworkModule extends AbstractModule { httpTransportTypes.registerExtension(name, clazz); } - public void registerTaskStatus(String name, Writeable.Reader reader) { - namedWriteables.add(new Entry(Task.Status.class, name, reader)); - } - /** * Register an allocation command. *

diff --git a/core/src/main/java/org/elasticsearch/env/NodeEnvironment.java b/core/src/main/java/org/elasticsearch/env/NodeEnvironment.java index 59ef122760b..4ddf0e38b75 100644 --- a/core/src/main/java/org/elasticsearch/env/NodeEnvironment.java +++ b/core/src/main/java/org/elasticsearch/env/NodeEnvironment.java @@ -433,7 +433,7 @@ public final class NodeEnvironment implements Closeable { * @param shardId the id of the shard to delete to delete * @throws IOException if an IOException occurs */ - public void deleteShardDirectorySafe(ShardId shardId, IndexSettings indexSettings) throws IOException { + public void deleteShardDirectorySafe(ShardId shardId, IndexSettings indexSettings) throws IOException, ShardLockObtainFailedException { final Path[] paths = availableShardPaths(shardId); logger.trace("deleting shard {} directory, paths: [{}]", shardId, paths); try (ShardLock lock = shardLock(shardId)) { @@ -462,7 +462,7 @@ public final class NodeEnvironment implements Closeable { locks[i] = dirs[i].obtainLock(IndexWriter.WRITE_LOCK_NAME); } catch (IOException ex) { throw new LockObtainFailedException("unable to acquire " + - IndexWriter.WRITE_LOCK_NAME + " for " + p); + IndexWriter.WRITE_LOCK_NAME + " for " + p, ex); } } } finally { @@ -504,7 +504,7 @@ public final class NodeEnvironment implements Closeable { try { shardLock(id, 0).close(); return false; - } catch (IOException ex) { + } catch (ShardLockObtainFailedException ex) { return true; } } @@ -519,7 +519,8 @@ public final class NodeEnvironment implements Closeable { * @param indexSettings settings for the index being deleted * @throws IOException if any of the shards data directories can't be locked or deleted */ - public void deleteIndexDirectorySafe(Index index, long lockTimeoutMS, IndexSettings indexSettings) throws IOException { + public void deleteIndexDirectorySafe(Index index, long lockTimeoutMS, IndexSettings indexSettings) + throws IOException, ShardLockObtainFailedException { final List locks = lockAllForIndex(index, indexSettings, lockTimeoutMS); try { deleteIndexDirectoryUnderLock(index, indexSettings); @@ -549,14 +550,15 @@ public final class NodeEnvironment implements Closeable { /** * Tries to lock all local shards for the given index. If any of the shard locks can't be acquired - * an {@link LockObtainFailedException} is thrown and all previously acquired locks are released. + * a {@link ShardLockObtainFailedException} is thrown and all previously acquired locks are released. * * @param index the index to lock shards for * @param lockTimeoutMS how long to wait for acquiring the indices shard locks * @return the {@link ShardLock} instances for this index. * @throws IOException if an IOException occurs. */ - public List lockAllForIndex(Index index, IndexSettings settings, long lockTimeoutMS) throws IOException { + public List lockAllForIndex(Index index, IndexSettings settings, long lockTimeoutMS) + throws IOException, ShardLockObtainFailedException { final int numShards = settings.getNumberOfShards(); if (numShards <= 0) { throw new IllegalArgumentException("settings must contain a non-null > 0 number of shards"); @@ -584,15 +586,14 @@ public final class NodeEnvironment implements Closeable { * Tries to lock the given shards ID. A shard lock is required to perform any kind of * write operation on a shards data directory like deleting files, creating a new index writer * or recover from a different shard instance into it. If the shard lock can not be acquired - * an {@link LockObtainFailedException} is thrown. + * a {@link ShardLockObtainFailedException} is thrown. * * Note: this method will return immediately if the lock can't be acquired. * * @param id the shard ID to lock * @return the shard lock. Call {@link ShardLock#close()} to release the lock - * @throws IOException if an IOException occurs. */ - public ShardLock shardLock(ShardId id) throws IOException { + public ShardLock shardLock(ShardId id) throws ShardLockObtainFailedException { return shardLock(id, 0); } @@ -600,13 +601,12 @@ public final class NodeEnvironment implements Closeable { * Tries to lock the given shards ID. A shard lock is required to perform any kind of * write operation on a shards data directory like deleting files, creating a new index writer * or recover from a different shard instance into it. If the shard lock can not be acquired - * an {@link org.apache.lucene.store.LockObtainFailedException} is thrown + * a {@link ShardLockObtainFailedException} is thrown * @param shardId the shard ID to lock * @param lockTimeoutMS the lock timeout in milliseconds * @return the shard lock. Call {@link ShardLock#close()} to release the lock - * @throws IOException if an IOException occurs. */ - public ShardLock shardLock(final ShardId shardId, long lockTimeoutMS) throws IOException { + public ShardLock shardLock(final ShardId shardId, long lockTimeoutMS) throws ShardLockObtainFailedException { logger.trace("acquiring node shardlock on [{}], timeout [{}]", shardId, lockTimeoutMS); final InternalShardLock shardLock; final boolean acquired; @@ -647,8 +647,7 @@ public final class NodeEnvironment implements Closeable { */ @FunctionalInterface public interface ShardLocker { - - ShardLock lock(ShardId shardId, long lockTimeoutMS) throws IOException; + ShardLock lock(ShardId shardId, long lockTimeoutMS) throws ShardLockObtainFailedException; } /** @@ -703,14 +702,15 @@ public final class NodeEnvironment implements Closeable { } } - void acquire(long timeoutInMillis) throws LockObtainFailedException{ + void acquire(long timeoutInMillis) throws ShardLockObtainFailedException { try { if (mutex.tryAcquire(timeoutInMillis, TimeUnit.MILLISECONDS) == false) { - throw new LockObtainFailedException("Can't lock shard " + shardId + ", timed out after " + timeoutInMillis + "ms"); + throw new ShardLockObtainFailedException(shardId, + "obtaining shard lock timed out after " + timeoutInMillis + "ms"); } } catch (InterruptedException e) { Thread.currentThread().interrupt(); - throw new LockObtainFailedException("Can't lock shard " + shardId + ", interrupted", e); + throw new ShardLockObtainFailedException(shardId, "thread interrupted while trying to obtain shard lock", e); } } } diff --git a/core/src/main/java/org/elasticsearch/env/ShardLock.java b/core/src/main/java/org/elasticsearch/env/ShardLock.java index 4ff1237ba20..99dd973b4ce 100644 --- a/core/src/main/java/org/elasticsearch/env/ShardLock.java +++ b/core/src/main/java/org/elasticsearch/env/ShardLock.java @@ -49,7 +49,7 @@ public abstract class ShardLock implements Closeable { } @Override - public final void close() throws IOException { + public final void close() { if (this.closed.compareAndSet(false, true)) { closeInternal(); } diff --git a/core/src/main/java/org/elasticsearch/env/ShardLockObtainFailedException.java b/core/src/main/java/org/elasticsearch/env/ShardLockObtainFailedException.java new file mode 100644 index 00000000000..1c113e1abea --- /dev/null +++ b/core/src/main/java/org/elasticsearch/env/ShardLockObtainFailedException.java @@ -0,0 +1,48 @@ +/* + * 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.env; + +import org.elasticsearch.index.shard.ShardId; + +/** + * Exception used when the in-memory lock for a shard cannot be obtained + */ +public class ShardLockObtainFailedException extends Exception { + private final ShardId shardId; + + public ShardLockObtainFailedException(ShardId shardId, String message) { + super(message); + this.shardId = shardId; + } + + public ShardLockObtainFailedException(ShardId shardId, String message, Throwable cause) { + super(message, cause); + this.shardId = shardId; + } + + @Override + public String getMessage() { + StringBuffer sb = new StringBuffer(); + sb.append(shardId.toString()); + sb.append(": "); + sb.append(super.getMessage()); + return sb.toString(); + } +} diff --git a/core/src/main/java/org/elasticsearch/gateway/GatewayAllocator.java b/core/src/main/java/org/elasticsearch/gateway/GatewayAllocator.java index ab6f6ae3ed2..a9fbe0ac82e 100644 --- a/core/src/main/java/org/elasticsearch/gateway/GatewayAllocator.java +++ b/core/src/main/java/org/elasticsearch/gateway/GatewayAllocator.java @@ -129,16 +129,13 @@ public class GatewayAllocator extends AbstractComponent { } } - public boolean allocateUnassigned(final RoutingAllocation allocation) { - boolean changed = false; - + public void allocateUnassigned(final RoutingAllocation allocation) { RoutingNodes.UnassignedShards unassigned = allocation.routingNodes().unassigned(); unassigned.sort(PriorityComparator.getAllocationComparator(allocation)); // sort for priority ordering - changed |= primaryShardAllocator.allocateUnassigned(allocation); - changed |= replicaShardAllocator.processExistingRecoveries(allocation); - changed |= replicaShardAllocator.allocateUnassigned(allocation); - return changed; + primaryShardAllocator.allocateUnassigned(allocation); + replicaShardAllocator.processExistingRecoveries(allocation); + replicaShardAllocator.allocateUnassigned(allocation); } class InternalAsyncFetch extends AsyncShardFetch { diff --git a/core/src/main/java/org/elasticsearch/gateway/GatewayMetaState.java b/core/src/main/java/org/elasticsearch/gateway/GatewayMetaState.java index a96b03ee560..a05e85299a8 100644 --- a/core/src/main/java/org/elasticsearch/gateway/GatewayMetaState.java +++ b/core/src/main/java/org/elasticsearch/gateway/GatewayMetaState.java @@ -19,6 +19,7 @@ package org.elasticsearch.gateway; +import com.carrotsearch.hppc.cursors.ObjectObjectCursor; import org.elasticsearch.Version; import org.elasticsearch.cluster.ClusterChangedEvent; import org.elasticsearch.cluster.ClusterState; @@ -37,14 +38,17 @@ import org.elasticsearch.common.unit.TimeValue; import org.elasticsearch.common.util.IndexFolderUpgrader; import org.elasticsearch.env.NodeEnvironment; import org.elasticsearch.index.Index; +import org.elasticsearch.plugins.MetaDataUpgrader; import java.nio.file.DirectoryStream; import java.nio.file.Files; import java.nio.file.Path; import java.util.ArrayList; import java.util.Collections; +import java.util.HashMap; import java.util.HashSet; import java.util.List; +import java.util.Map; import java.util.Set; import static java.util.Collections.emptySet; @@ -55,7 +59,6 @@ public class GatewayMetaState extends AbstractComponent implements ClusterStateL private final NodeEnvironment nodeEnv; private final MetaStateService metaStateService; private final DanglingIndicesState danglingIndicesState; - private final MetaDataIndexUpgradeService metaDataIndexUpgradeService; @Nullable private volatile MetaData previousMetaData; @@ -65,12 +68,12 @@ public class GatewayMetaState extends AbstractComponent implements ClusterStateL @Inject public GatewayMetaState(Settings settings, NodeEnvironment nodeEnv, MetaStateService metaStateService, DanglingIndicesState danglingIndicesState, TransportNodesListGatewayMetaState nodesListGatewayMetaState, - MetaDataIndexUpgradeService metaDataIndexUpgradeService) throws Exception { + MetaDataIndexUpgradeService metaDataIndexUpgradeService, MetaDataUpgrader metaDataUpgrader) + throws Exception { super(settings); this.nodeEnv = nodeEnv; this.metaStateService = metaStateService; this.danglingIndicesState = danglingIndicesState; - this.metaDataIndexUpgradeService = metaDataIndexUpgradeService; nodesListGatewayMetaState.init(this); if (DiscoveryNode.isDataNode(settings)) { @@ -84,7 +87,21 @@ public class GatewayMetaState extends AbstractComponent implements ClusterStateL try { ensureNoPre019State(); IndexFolderUpgrader.upgradeIndicesIfNeeded(settings, nodeEnv); - upgradeMetaData(); + final MetaData metaData = metaStateService.loadFullState(); + final MetaData upgradedMetaData = upgradeMetaData(metaData, metaDataIndexUpgradeService, metaDataUpgrader); + // We finished global state validation and successfully checked all indices for backward compatibility + // and found no non-upgradable indices, which means the upgrade can continue. + // Now it's safe to overwrite global and index metadata. + if (metaData != upgradedMetaData) { + if (MetaData.isGlobalStateEquals(metaData, upgradedMetaData) == false) { + metaStateService.writeGlobalState("upgrade", upgradedMetaData); + } + for (IndexMetaData indexMetaData : upgradedMetaData) { + if (metaData.hasIndexMetaData(indexMetaData) == false) { + metaStateService.writeIndex("upgrade", indexMetaData); + } + } + } long startNS = System.nanoTime(); metaStateService.loadFullState(); logger.debug("took {} to load state", TimeValue.timeValueMillis(TimeValue.nsecToMSec(System.nanoTime() - startNS))); @@ -216,24 +233,37 @@ public class GatewayMetaState extends AbstractComponent implements ClusterStateL /** * Elasticsearch 2.0 removed several deprecated features and as well as support for Lucene 3.x. This method calls * {@link MetaDataIndexUpgradeService} to makes sure that indices are compatible with the current version. The - * MetaDataIndexUpgradeService might also update obsolete settings if needed. When this happens we rewrite - * index metadata with new settings. + * MetaDataIndexUpgradeService might also update obsolete settings if needed. + * Allows upgrading global custom meta data via {@link MetaDataUpgrader#customMetaDataUpgraders} + * + * @return input metaData if no upgrade is needed or an upgraded metaData */ - private void upgradeMetaData() throws Exception { - MetaData metaData = loadMetaState(); - List updateIndexMetaData = new ArrayList<>(); + static MetaData upgradeMetaData(MetaData metaData, + MetaDataIndexUpgradeService metaDataIndexUpgradeService, + MetaDataUpgrader metaDataUpgrader) throws Exception { + // upgrade index meta data + boolean changed = false; + final MetaData.Builder upgradedMetaData = MetaData.builder(metaData); for (IndexMetaData indexMetaData : metaData) { IndexMetaData newMetaData = metaDataIndexUpgradeService.upgradeIndexMetaData(indexMetaData); - if (indexMetaData != newMetaData) { - updateIndexMetaData.add(newMetaData); + changed |= indexMetaData != newMetaData; + upgradedMetaData.put(newMetaData, false); + } + // collect current customs + Map existingCustoms = new HashMap<>(); + for (ObjectObjectCursor customCursor : metaData.customs()) { + existingCustoms.put(customCursor.key, customCursor.value); + } + // upgrade global custom meta data + Map upgradedCustoms = metaDataUpgrader.customMetaDataUpgraders.apply(existingCustoms); + if (upgradedCustoms.equals(existingCustoms) == false) { + existingCustoms.keySet().forEach(upgradedMetaData::removeCustom); + for (Map.Entry upgradedCustomEntry : upgradedCustoms.entrySet()) { + upgradedMetaData.putCustom(upgradedCustomEntry.getKey(), upgradedCustomEntry.getValue()); } + changed = true; } - // We successfully checked all indices for backward compatibility and found no non-upgradable indices, which - // means the upgrade can continue. Now it's safe to overwrite index metadata with the new version. - for (IndexMetaData indexMetaData : updateIndexMetaData) { - // since we upgraded the index folders already, write index state in the upgraded index folder - metaStateService.writeIndex("upgrade", indexMetaData); - } + return changed ? upgradedMetaData.build() : metaData; } // shard state BWC diff --git a/core/src/main/java/org/elasticsearch/gateway/PrimaryShardAllocator.java b/core/src/main/java/org/elasticsearch/gateway/PrimaryShardAllocator.java index f8ed1cc7334..600e2c5e404 100644 --- a/core/src/main/java/org/elasticsearch/gateway/PrimaryShardAllocator.java +++ b/core/src/main/java/org/elasticsearch/gateway/PrimaryShardAllocator.java @@ -27,10 +27,12 @@ import org.elasticsearch.cluster.node.DiscoveryNode; import org.elasticsearch.cluster.routing.RoutingNode; import org.elasticsearch.cluster.routing.RoutingNodes; import org.elasticsearch.cluster.routing.ShardRouting; +import org.elasticsearch.cluster.routing.UnassignedInfo; import org.elasticsearch.cluster.routing.UnassignedInfo.AllocationStatus; import org.elasticsearch.cluster.routing.allocation.RoutingAllocation; import org.elasticsearch.cluster.routing.allocation.decider.Decision; import org.elasticsearch.common.component.AbstractComponent; +import org.elasticsearch.common.logging.ESLogger; import org.elasticsearch.common.settings.Setting; import org.elasticsearch.common.settings.Setting.Property; import org.elasticsearch.common.settings.Settings; @@ -47,8 +49,17 @@ import java.util.function.Function; import java.util.stream.Collectors; /** - * The primary shard allocator allocates primary shard that were not created as - * a result of an API to a node that held them last to be recovered. + * The primary shard allocator allocates unassigned primary shards to nodes that hold + * valid copies of the unassigned primaries. It does this by iterating over all unassigned + * primary shards in the routing table and fetching shard metadata from each node in the cluster + * that holds a copy of the shard. The shard metadata from each node is compared against the + * set of valid allocation IDs and for all valid shard copies (if any), the primary shard allocator + * executes the allocation deciders to chose a copy to assign the primary shard to. + * + * Note that the PrimaryShardAllocator does *not* allocate primaries on index creation + * (see {@link org.elasticsearch.cluster.routing.allocation.allocator.BalancedShardsAllocator}), + * nor does it allocate primaries when a primary shard failed and there is a valid replica + * copy that can immediately be promoted to primary, as this takes place in {@link RoutingNodes#failShard}. */ public abstract class PrimaryShardAllocator extends AbstractComponent { @@ -82,8 +93,7 @@ public abstract class PrimaryShardAllocator extends AbstractComponent { logger.debug("using initial_shards [{}]", NODE_INITIAL_SHARDS_SETTING.get(settings)); } - public boolean allocateUnassigned(RoutingAllocation allocation) { - boolean changed = false; + public void allocateUnassigned(RoutingAllocation allocation) { final RoutingNodes routingNodes = allocation.routingNodes(); final MetaData metaData = allocation.metaData(); @@ -107,7 +117,7 @@ public abstract class PrimaryShardAllocator extends AbstractComponent { if (shardState.hasData() == false) { logger.trace("{}: ignoring allocation, still fetching shard started state", shard); allocation.setHasPendingAsyncFetch(); - changed |= unassignedIterator.removeAndIgnore(AllocationStatus.FETCHING_SHARD_DATA); + unassignedIterator.removeAndIgnore(AllocationStatus.FETCHING_SHARD_DATA, allocation.changes()); continue; } @@ -148,30 +158,45 @@ public abstract class PrimaryShardAllocator extends AbstractComponent { logger.debug("[{}][{}]: missing local data, recover from any node", shard.index(), shard.id()); } else { // we can't really allocate, so ignore it and continue - changed |= unassignedIterator.removeAndIgnore(AllocationStatus.NO_VALID_SHARD_COPY); + unassignedIterator.removeAndIgnore(AllocationStatus.NO_VALID_SHARD_COPY, allocation.changes()); logger.debug("[{}][{}]: not allocating, number_of_allocated_shards_found [{}]", shard.index(), shard.id(), nodeShardsResult.allocationsFound); } continue; } - final NodesToAllocate nodesToAllocate = buildNodesToAllocate(shard, allocation, nodeShardsResult.orderedAllocationCandidates); + final NodesToAllocate nodesToAllocate = buildNodesToAllocate( + allocation, nodeShardsResult.orderedAllocationCandidates, shard, false + ); if (nodesToAllocate.yesNodeShards.isEmpty() == false) { NodeGatewayStartedShards nodeShardState = nodesToAllocate.yesNodeShards.get(0); logger.debug("[{}][{}]: allocating [{}] to [{}] on primary allocation", shard.index(), shard.id(), shard, nodeShardState.getNode()); - changed = true; - unassignedIterator.initialize(nodeShardState.getNode().getId(), nodeShardState.allocationId(), ShardRouting.UNAVAILABLE_EXPECTED_SHARD_SIZE); + unassignedIterator.initialize(nodeShardState.getNode().getId(), nodeShardState.allocationId(), ShardRouting.UNAVAILABLE_EXPECTED_SHARD_SIZE, allocation.changes()); } else if (nodesToAllocate.throttleNodeShards.isEmpty() == true && nodesToAllocate.noNodeShards.isEmpty() == false) { - NodeGatewayStartedShards nodeShardState = nodesToAllocate.noNodeShards.get(0); - logger.debug("[{}][{}]: forcing allocating [{}] to [{}] on primary allocation", shard.index(), shard.id(), shard, nodeShardState.getNode()); - changed = true; - unassignedIterator.initialize(nodeShardState.getNode().getId(), nodeShardState.allocationId(), ShardRouting.UNAVAILABLE_EXPECTED_SHARD_SIZE); + // The deciders returned a NO decision for all nodes with shard copies, so we check if primary shard + // can be force-allocated to one of the nodes. + final NodesToAllocate nodesToForceAllocate = buildNodesToAllocate( + allocation, nodeShardsResult.orderedAllocationCandidates, shard, true + ); + if (nodesToForceAllocate.yesNodeShards.isEmpty() == false) { + NodeGatewayStartedShards nodeShardState = nodesToForceAllocate.yesNodeShards.get(0); + logger.debug("[{}][{}]: allocating [{}] to [{}] on forced primary allocation", + shard.index(), shard.id(), shard, nodeShardState.getNode()); + unassignedIterator.initialize(nodeShardState.getNode().getId(), nodeShardState.allocationId(), + ShardRouting.UNAVAILABLE_EXPECTED_SHARD_SIZE, allocation.changes()); + } else if (nodesToForceAllocate.throttleNodeShards.isEmpty() == false) { + logger.debug("[{}][{}]: throttling allocation [{}] to [{}] on forced primary allocation", + shard.index(), shard.id(), shard, nodesToForceAllocate.throttleNodeShards); + unassignedIterator.removeAndIgnore(AllocationStatus.DECIDERS_THROTTLED, allocation.changes()); + } else { + logger.debug("[{}][{}]: forced primary allocation denied [{}]", shard.index(), shard.id(), shard); + unassignedIterator.removeAndIgnore(AllocationStatus.DECIDERS_NO, allocation.changes()); + } } else { // we are throttling this, but we have enough to allocate to this node, ignore it for now logger.debug("[{}][{}]: throttling allocation [{}] to [{}] on primary allocation", shard.index(), shard.id(), shard, nodesToAllocate.throttleNodeShards); - changed |= unassignedIterator.removeAndIgnore(AllocationStatus.DECIDERS_THROTTLED); + unassignedIterator.removeAndIgnore(AllocationStatus.DECIDERS_THROTTLED, allocation.changes()); } } - return changed; } /** @@ -268,7 +293,10 @@ public abstract class PrimaryShardAllocator extends AbstractComponent { /** * Split the list of node shard states into groups yes/no/throttle based on allocation deciders */ - private NodesToAllocate buildNodesToAllocate(ShardRouting shard, RoutingAllocation allocation, List nodeShardStates) { + private NodesToAllocate buildNodesToAllocate(RoutingAllocation allocation, + List nodeShardStates, + ShardRouting shardRouting, + boolean forceAllocate) { List yesNodeShards = new ArrayList<>(); List throttledNodeShards = new ArrayList<>(); List noNodeShards = new ArrayList<>(); @@ -278,7 +306,8 @@ public abstract class PrimaryShardAllocator extends AbstractComponent { continue; } - Decision decision = allocation.deciders().canAllocate(shard, node, allocation); + Decision decision = forceAllocate ? allocation.deciders().canForceAllocatePrimary(shardRouting, node, allocation) : + allocation.deciders().canAllocate(shardRouting, node, allocation); if (decision.type() == Decision.Type.THROTTLE) { throttledNodeShards.add(nodeShardState); } else if (decision.type() == Decision.Type.NO) { diff --git a/core/src/main/java/org/elasticsearch/gateway/ReplicaShardAllocator.java b/core/src/main/java/org/elasticsearch/gateway/ReplicaShardAllocator.java index 1cb555347cc..75a8a43fabd 100644 --- a/core/src/main/java/org/elasticsearch/gateway/ReplicaShardAllocator.java +++ b/core/src/main/java/org/elasticsearch/gateway/ReplicaShardAllocator.java @@ -31,11 +31,10 @@ import org.elasticsearch.cluster.routing.RoutingNodes; import org.elasticsearch.cluster.routing.ShardRouting; import org.elasticsearch.cluster.routing.UnassignedInfo; import org.elasticsearch.cluster.routing.UnassignedInfo.AllocationStatus; -import org.elasticsearch.cluster.routing.allocation.AllocationService; +import org.elasticsearch.cluster.routing.RoutingChangesObserver; import org.elasticsearch.cluster.routing.allocation.RoutingAllocation; import org.elasticsearch.cluster.routing.allocation.decider.Decision; import org.elasticsearch.common.Nullable; -import org.elasticsearch.common.collect.Tuple; import org.elasticsearch.common.component.AbstractComponent; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.unit.ByteSizeValue; @@ -60,8 +59,7 @@ public abstract class ReplicaShardAllocator extends AbstractComponent { * match. Today, a better match is one that has full sync id match compared to not having one in * the previous recovery. */ - public boolean processExistingRecoveries(RoutingAllocation allocation) { - boolean changed = false; + public void processExistingRecoveries(RoutingAllocation allocation) { MetaData metaData = allocation.metaData(); RoutingNodes routingNodes = allocation.routingNodes(); List shardCancellationActions = new ArrayList<>(); @@ -121,8 +119,7 @@ public abstract class ReplicaShardAllocator extends AbstractComponent { "existing allocation of replica to [" + currentNode + "] cancelled, sync id match found on node ["+ nodeWithHighestMatch + "]", null, 0, allocation.getCurrentNanoTime(), System.currentTimeMillis(), false, UnassignedInfo.AllocationStatus.NO_ATTEMPT); // don't cancel shard in the loop as it will cause a ConcurrentModificationException - shardCancellationActions.add(() -> routingNodes.failShard(logger, shard, unassignedInfo, indexMetaData)); - changed = true; + shardCancellationActions.add(() -> routingNodes.failShard(logger, shard, unassignedInfo, indexMetaData, allocation.changes())); } } } @@ -130,11 +127,9 @@ public abstract class ReplicaShardAllocator extends AbstractComponent { for (Runnable action : shardCancellationActions) { action.run(); } - return changed; } - public boolean allocateUnassigned(RoutingAllocation allocation) { - boolean changed = false; + public void allocateUnassigned(RoutingAllocation allocation) { final RoutingNodes routingNodes = allocation.routingNodes(); final RoutingNodes.UnassignedShards.UnassignedIterator unassignedIterator = routingNodes.unassigned().iterator(); MetaData metaData = allocation.metaData(); @@ -154,7 +149,7 @@ public abstract class ReplicaShardAllocator extends AbstractComponent { Decision decision = canBeAllocatedToAtLeastOneNode(shard, allocation); if (decision.type() != Decision.Type.YES) { logger.trace("{}: ignoring allocation, can't be allocated on any node", shard); - changed |= unassignedIterator.removeAndIgnore(UnassignedInfo.AllocationStatus.fromDecision(decision)); + unassignedIterator.removeAndIgnore(UnassignedInfo.AllocationStatus.fromDecision(decision), allocation.changes()); continue; } @@ -162,7 +157,7 @@ public abstract class ReplicaShardAllocator extends AbstractComponent { if (shardStores.hasData() == false) { logger.trace("{}: ignoring allocation, still fetching shard stores", shard); allocation.setHasPendingAsyncFetch(); - changed |= unassignedIterator.removeAndIgnore(AllocationStatus.FETCHING_SHARD_DATA); + unassignedIterator.removeAndIgnore(AllocationStatus.FETCHING_SHARD_DATA, allocation.changes()); continue; // still fetching } @@ -187,19 +182,17 @@ public abstract class ReplicaShardAllocator extends AbstractComponent { if (decision.type() == Decision.Type.THROTTLE) { logger.debug("[{}][{}]: throttling allocation [{}] to [{}] in order to reuse its unallocated persistent store", shard.index(), shard.id(), shard, nodeWithHighestMatch.node()); // we are throttling this, but we have enough to allocate to this node, ignore it for now - changed |= unassignedIterator.removeAndIgnore(UnassignedInfo.AllocationStatus.fromDecision(decision)); + unassignedIterator.removeAndIgnore(UnassignedInfo.AllocationStatus.fromDecision(decision), allocation.changes()); } else { logger.debug("[{}][{}]: allocating [{}] to [{}] in order to reuse its unallocated persistent store", shard.index(), shard.id(), shard, nodeWithHighestMatch.node()); // we found a match - changed = true; - unassignedIterator.initialize(nodeWithHighestMatch.nodeId(), null, allocation.clusterInfo().getShardSize(shard, ShardRouting.UNAVAILABLE_EXPECTED_SHARD_SIZE)); + unassignedIterator.initialize(nodeWithHighestMatch.nodeId(), null, allocation.clusterInfo().getShardSize(shard, ShardRouting.UNAVAILABLE_EXPECTED_SHARD_SIZE), allocation.changes()); } } else if (matchingNodes.hasAnyData() == false) { // if we didn't manage to find *any* data (regardless of matching sizes), check if the allocation of the replica shard needs to be delayed - changed |= ignoreUnassignedIfDelayed(unassignedIterator, shard); + ignoreUnassignedIfDelayed(unassignedIterator, shard, allocation.changes()); } } - return changed; } /** @@ -212,14 +205,12 @@ public abstract class ReplicaShardAllocator extends AbstractComponent { * * @param unassignedIterator iterator over unassigned shards * @param shard the shard which might be delayed - * @return true iff there was a change to the unassigned info */ - public boolean ignoreUnassignedIfDelayed(RoutingNodes.UnassignedShards.UnassignedIterator unassignedIterator, ShardRouting shard) { + public void ignoreUnassignedIfDelayed(RoutingNodes.UnassignedShards.UnassignedIterator unassignedIterator, ShardRouting shard, RoutingChangesObserver changes) { if (shard.unassignedInfo().isDelayed()) { logger.debug("{}: allocation of [{}] is delayed", shard.shardId(), shard); - return unassignedIterator.removeAndIgnore(AllocationStatus.DELAYED_ALLOCATION); + unassignedIterator.removeAndIgnore(AllocationStatus.DELAYED_ALLOCATION, changes); } - return false; } /** diff --git a/core/src/main/java/org/elasticsearch/index/IndexService.java b/core/src/main/java/org/elasticsearch/index/IndexService.java index 0ba37418792..b9f93bf2ac4 100644 --- a/core/src/main/java/org/elasticsearch/index/IndexService.java +++ b/core/src/main/java/org/elasticsearch/index/IndexService.java @@ -39,6 +39,7 @@ import org.elasticsearch.common.xcontent.XContentFactory; import org.elasticsearch.common.xcontent.XContentParser; import org.elasticsearch.env.NodeEnvironment; import org.elasticsearch.env.ShardLock; +import org.elasticsearch.env.ShardLockObtainFailedException; import org.elasticsearch.index.analysis.AnalysisRegistry; import org.elasticsearch.index.analysis.AnalysisService; import org.elasticsearch.index.cache.IndexCache; @@ -279,8 +280,9 @@ public class IndexService extends AbstractIndexComponent implements IndicesClust boolean success = false; Store store = null; IndexShard indexShard = null; - final ShardLock lock = nodeEnv.shardLock(shardId, TimeUnit.SECONDS.toMillis(5)); + ShardLock lock = null; try { + lock = nodeEnv.shardLock(shardId, TimeUnit.SECONDS.toMillis(5)); eventListener.beforeIndexShardCreated(shardId, indexSettings); ShardPath path; try { @@ -349,9 +351,13 @@ public class IndexService extends AbstractIndexComponent implements IndicesClust shards = newMapBuilder(shards).put(shardId.id(), indexShard).immutableMap(); success = true; return indexShard; + } catch (ShardLockObtainFailedException e) { + throw new IOException("failed to obtain in-memory shard lock", e); } finally { if (success == false) { - IOUtils.closeWhileHandlingException(lock); + if (lock != null) { + IOUtils.closeWhileHandlingException(lock); + } closeShard("initialization failed", shardId, indexShard, store, eventListener); } } diff --git a/core/src/main/java/org/elasticsearch/index/mapper/StringFieldMapper.java b/core/src/main/java/org/elasticsearch/index/mapper/StringFieldMapper.java index 6f529c82ea1..887a40fe70b 100644 --- a/core/src/main/java/org/elasticsearch/index/mapper/StringFieldMapper.java +++ b/core/src/main/java/org/elasticsearch/index/mapper/StringFieldMapper.java @@ -22,13 +22,9 @@ package org.elasticsearch.index.mapper; import org.apache.lucene.document.Field; import org.apache.lucene.document.SortedSetDocValuesField; import org.apache.lucene.index.IndexOptions; -import org.apache.lucene.index.Term; -import org.apache.lucene.search.MultiTermQuery; import org.apache.lucene.search.Query; -import org.apache.lucene.search.RegexpQuery; import org.apache.lucene.util.BytesRef; import org.elasticsearch.Version; -import org.elasticsearch.common.Nullable; import org.elasticsearch.common.logging.DeprecationLogger; import org.elasticsearch.common.logging.ESLogger; import org.elasticsearch.common.logging.Loggers; @@ -40,7 +36,6 @@ import org.elasticsearch.index.analysis.NamedAnalyzer; import org.elasticsearch.index.fielddata.IndexFieldData; import org.elasticsearch.index.fielddata.plain.DocValuesIndexFieldData; import org.elasticsearch.index.fielddata.plain.PagedBytesIndexFieldData; -import org.elasticsearch.index.query.QueryShardContext; import java.io.IOException; import java.util.Arrays; @@ -71,7 +66,8 @@ public class StringFieldMapper extends FieldMapper implements AllFieldMapper.Inc "type", // common text parameters, for which the upgrade is straightforward "index", "store", "doc_values", "omit_norms", "norms", "fields", "copy_to", - "fielddata", "include_in_all", "analyzer", "search_analyzer", "search_quote_analyzer")); + "fielddata", "include_in_all", "analyzer", "search_analyzer", "search_quote_analyzer", + "index_options", "position_increment_gap")); public static class Defaults { public static double FIELDDATA_MIN_FREQUENCY = 0; @@ -259,7 +255,10 @@ public class StringFieldMapper extends FieldMapper implements AllFieldMapper.Inc } } - throw new IllegalArgumentException("The [string] type is removed in 5.0. You should now use either a [text] " + Set unsupportedParameters = new HashSet<>(node.keySet()); + unsupportedParameters.removeAll(autoUpgradeParameters); + throw new IllegalArgumentException("The [string] type is removed in 5.0 and automatic upgrade failed because parameters " + + unsupportedParameters + " are not supported for automatic upgrades. You should now use either a [text] " + "or [keyword] field instead for field [" + fieldName + "]"); } diff --git a/core/src/main/java/org/elasticsearch/index/store/Store.java b/core/src/main/java/org/elasticsearch/index/store/Store.java index 659b230edab..e714d8db8b6 100644 --- a/core/src/main/java/org/elasticsearch/index/store/Store.java +++ b/core/src/main/java/org/elasticsearch/index/store/Store.java @@ -70,6 +70,7 @@ import org.elasticsearch.common.util.concurrent.RefCounted; import org.elasticsearch.common.util.iterable.Iterables; import org.elasticsearch.env.NodeEnvironment; import org.elasticsearch.env.ShardLock; +import org.elasticsearch.env.ShardLockObtainFailedException; import org.elasticsearch.index.IndexSettings; import org.elasticsearch.index.engine.Engine; import org.elasticsearch.index.shard.AbstractIndexShardComponent; @@ -388,6 +389,8 @@ public class Store extends AbstractIndexShardComponent implements Closeable, Ref // that's fine - happens all the time no need to log } catch (FileNotFoundException | NoSuchFileException ex) { logger.info("Failed to open / find files while reading metadata snapshot"); + } catch (ShardLockObtainFailedException ex) { + logger.info("{}: failed to obtain shard lock", ex, shardId); } return MetadataSnapshot.EMPTY; } @@ -418,6 +421,9 @@ public class Store extends AbstractIndexShardComponent implements Closeable, Ref failIfCorrupted(dir, shardId); SegmentInfos segInfo = Lucene.readSegmentInfos(dir); logger.trace("{} loaded segment info [{}]", shardId, segInfo); + } catch (ShardLockObtainFailedException ex) { + logger.error("{} unable to acquire shard lock", ex, shardId); + throw new IOException(ex); } } diff --git a/core/src/main/java/org/elasticsearch/indices/IndicesService.java b/core/src/main/java/org/elasticsearch/indices/IndicesService.java index d5a0da1be4b..7519494c39c 100644 --- a/core/src/main/java/org/elasticsearch/indices/IndicesService.java +++ b/core/src/main/java/org/elasticsearch/indices/IndicesService.java @@ -62,6 +62,7 @@ import org.elasticsearch.common.util.concurrent.EsExecutors; import org.elasticsearch.common.util.iterable.Iterables; import org.elasticsearch.env.NodeEnvironment; import org.elasticsearch.env.ShardLock; +import org.elasticsearch.env.ShardLockObtainFailedException; import org.elasticsearch.gateway.MetaDataStateFormat; import org.elasticsearch.gateway.MetaStateService; import org.elasticsearch.index.Index; @@ -676,7 +677,8 @@ public class IndicesService extends AbstractLifecycleComponent * @param clusterState . This is required to access the indexes settings etc. * @throws IOException if an IOException occurs */ - public void deleteShardStore(String reason, ShardId shardId, ClusterState clusterState) throws IOException { + public void deleteShardStore(String reason, ShardId shardId, ClusterState clusterState) + throws IOException, ShardLockObtainFailedException { final IndexMetaData metaData = clusterState.getMetaData().indices().get(shardId.getIndexName()); final IndexSettings indexSettings = buildIndexSettings(metaData); @@ -891,7 +893,8 @@ public class IndicesService extends AbstractLifecycleComponent * @param timeout the timeout used for processing pending deletes */ @Override - public void processPendingDeletes(Index index, IndexSettings indexSettings, TimeValue timeout) throws IOException, InterruptedException { + public void processPendingDeletes(Index index, IndexSettings indexSettings, TimeValue timeout) + throws IOException, InterruptedException, ShardLockObtainFailedException { logger.debug("{} processing pending deletes", index); final long startTimeNS = System.nanoTime(); final List shardLocks = nodeEnv.lockAllForIndex(index, indexSettings, timeout.millis()); diff --git a/core/src/main/java/org/elasticsearch/indices/cluster/IndicesClusterStateService.java b/core/src/main/java/org/elasticsearch/indices/cluster/IndicesClusterStateService.java index f01a09d4608..fd77722f86b 100644 --- a/core/src/main/java/org/elasticsearch/indices/cluster/IndicesClusterStateService.java +++ b/core/src/main/java/org/elasticsearch/indices/cluster/IndicesClusterStateService.java @@ -42,6 +42,7 @@ import org.elasticsearch.common.unit.TimeValue; import org.elasticsearch.common.util.Callback; import org.elasticsearch.common.util.concurrent.AbstractRunnable; import org.elasticsearch.common.util.concurrent.ConcurrentCollections; +import org.elasticsearch.env.ShardLockObtainFailedException; import org.elasticsearch.gateway.GatewayService; import org.elasticsearch.index.Index; import org.elasticsearch.index.IndexComponent; @@ -835,6 +836,7 @@ public class IndicesClusterStateService extends AbstractLifecycleComponent imple return null; } - void processPendingDeletes(Index index, IndexSettings indexSettings, TimeValue timeValue) throws IOException, InterruptedException; + void processPendingDeletes(Index index, IndexSettings indexSettings, TimeValue timeValue) + throws IOException, InterruptedException, ShardLockObtainFailedException; } } diff --git a/core/src/main/java/org/elasticsearch/node/Node.java b/core/src/main/java/org/elasticsearch/node/Node.java index 955676b0166..7b098323f79 100644 --- a/core/src/main/java/org/elasticsearch/node/Node.java +++ b/core/src/main/java/org/elasticsearch/node/Node.java @@ -36,6 +36,7 @@ import org.elasticsearch.cluster.ClusterStateObserver; import org.elasticsearch.cluster.MasterNodeChangePredicate; import org.elasticsearch.cluster.NodeConnectionsService; import org.elasticsearch.cluster.action.index.MappingUpdatedAction; +import org.elasticsearch.cluster.metadata.MetaData; import org.elasticsearch.cluster.node.DiscoveryNode; import org.elasticsearch.cluster.routing.RoutingService; import org.elasticsearch.cluster.routing.allocation.AllocationService; @@ -96,6 +97,7 @@ import org.elasticsearch.plugins.DiscoveryPlugin; import org.elasticsearch.plugins.IngestPlugin; import org.elasticsearch.plugins.MapperPlugin; import org.elasticsearch.plugins.Plugin; +import org.elasticsearch.plugins.MetaDataUpgrader; import org.elasticsearch.plugins.PluginsService; import org.elasticsearch.plugins.RepositoryPlugin; import org.elasticsearch.plugins.ScriptPlugin; @@ -134,6 +136,7 @@ import java.util.Map; import java.util.concurrent.CountDownLatch; import java.util.concurrent.TimeUnit; import java.util.function.Function; +import java.util.function.UnaryOperator; import java.util.stream.Collectors; import java.util.stream.Stream; @@ -347,8 +350,13 @@ public class Node implements Closeable { client = new NodeClient(settings, threadPool); Collection pluginComponents = pluginsService.filterPlugins(Plugin.class).stream() .flatMap(p -> p.createComponents(client, clusterService, threadPool, resourceWatcherService, - scriptModule.getScriptService()).stream()) + scriptModule.getScriptService(), searchModule.getSearchRequestParsers()).stream()) .collect(Collectors.toList()); + Collection>> customMetaDataUpgraders = + pluginsService.filterPlugins(Plugin.class).stream() + .map(Plugin::getCustomMetaDataUpgrader) + .collect(Collectors.toList()); + final MetaDataUpgrader metaDataUpgrader = new MetaDataUpgrader(customMetaDataUpgraders); modules.add(b -> { b.bind(PluginsService.class).toInstance(pluginsService); b.bind(Client.class).toInstance(client); @@ -364,7 +372,14 @@ public class Node implements Closeable { b.bind(AnalysisRegistry.class).toInstance(analysisModule.getAnalysisRegistry()); b.bind(IngestService.class).toInstance(ingestService); b.bind(NamedWriteableRegistry.class).toInstance(namedWriteableRegistry); + Class searchServiceImpl = pickSearchServiceImplementation(); + if (searchServiceImpl == SearchService.class) { + b.bind(SearchService.class).asEagerSingleton(); + } else { + b.bind(SearchService.class).to(searchServiceImpl).asEagerSingleton(); + } pluginComponents.stream().forEach(p -> b.bind((Class) p.getClass()).toInstance(p)); + b.bind(MetaDataUpgrader.class).toInstance(metaDataUpgrader); } ); injector = modules.createInjector(); @@ -716,6 +731,13 @@ public class Node implements Closeable { } } + /** + * The {@link PluginsService} used to build this node's components. + */ + protected PluginsService getPluginsService() { + return pluginsService; + } + /** * Creates a new {@link CircuitBreakerService} based on the settings provided. * @see #BREAKER_TYPE_KEY @@ -739,6 +761,13 @@ public class Node implements Closeable { return new BigArrays(settings, circuitBreakerService); } + /** + * Select the search service implementation. Overrided by tests. + */ + protected Class pickSearchServiceImplementation() { + return SearchService.class; + } + /** * Get Custom Name Resolvers list based on a Discovery Plugins list * @param discoveryPlugins Discovery plugins list diff --git a/core/src/main/java/org/elasticsearch/plugins/MetaDataUpgrader.java b/core/src/main/java/org/elasticsearch/plugins/MetaDataUpgrader.java new file mode 100644 index 00000000000..aaeddbec119 --- /dev/null +++ b/core/src/main/java/org/elasticsearch/plugins/MetaDataUpgrader.java @@ -0,0 +1,44 @@ +/* + * 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.plugins; + +import org.elasticsearch.cluster.metadata.MetaData; + +import java.util.Collection; +import java.util.HashMap; +import java.util.Map; +import java.util.function.UnaryOperator; + +/** + * Upgrades {@link MetaData} on startup on behalf of installed {@link Plugin}s + */ +public class MetaDataUpgrader { + public final UnaryOperator> customMetaDataUpgraders; + + public MetaDataUpgrader(Collection>> customMetaDataUpgraders) { + this.customMetaDataUpgraders = customs -> { + Map upgradedCustoms = new HashMap<>(customs); + for (UnaryOperator> customMetaDataUpgrader : customMetaDataUpgraders) { + upgradedCustoms = customMetaDataUpgrader.apply(upgradedCustoms); + } + return upgradedCustoms; + }; + } +} diff --git a/core/src/main/java/org/elasticsearch/plugins/Plugin.java b/core/src/main/java/org/elasticsearch/plugins/Plugin.java index b7223541644..1c79986e18f 100644 --- a/core/src/main/java/org/elasticsearch/plugins/Plugin.java +++ b/core/src/main/java/org/elasticsearch/plugins/Plugin.java @@ -22,16 +22,15 @@ package org.elasticsearch.plugins; import java.util.Collection; import java.util.Collections; import java.util.List; -import java.util.Map; import org.elasticsearch.action.ActionModule; import org.elasticsearch.client.Client; +import org.elasticsearch.cluster.metadata.MetaData; import org.elasticsearch.cluster.service.ClusterService; import org.elasticsearch.common.component.LifecycleComponent; import org.elasticsearch.common.inject.Module; import org.elasticsearch.common.io.stream.NamedWriteable; import org.elasticsearch.common.io.stream.NamedWriteableRegistry; -import org.elasticsearch.common.io.stream.Writeable; import org.elasticsearch.common.settings.Setting; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.settings.SettingsModule; @@ -39,10 +38,15 @@ import org.elasticsearch.index.IndexModule; import org.elasticsearch.indices.analysis.AnalysisModule; import org.elasticsearch.script.ScriptModule; import org.elasticsearch.script.ScriptService; +import org.elasticsearch.search.SearchModule; +import org.elasticsearch.search.SearchRequestParsers; import org.elasticsearch.threadpool.ExecutorBuilder; import org.elasticsearch.threadpool.ThreadPool; import org.elasticsearch.watcher.ResourceWatcherService; +import java.util.Map; +import java.util.function.UnaryOperator; + /** * An extension point allowing to plug in custom functionality. *

@@ -84,9 +88,11 @@ public abstract class Plugin { * @param threadPool A service to allow retrieving an executor to run an async action * @param resourceWatcherService A service to watch for changes to node local files * @param scriptService A service to allow running scripts on the local node + * @param searchRequestParsers Parsers for search requests which may be used to templatize search requests */ public Collection createComponents(Client client, ClusterService clusterService, ThreadPool threadPool, - ResourceWatcherService resourceWatcherService, ScriptService scriptService) { + ResourceWatcherService resourceWatcherService, ScriptService scriptService, + SearchRequestParsers searchRequestParsers) { return Collections.emptyList(); } @@ -122,6 +128,18 @@ public abstract class Plugin { */ public List getSettingsFilter() { return Collections.emptyList(); } + /** + * Provides a function to modify global custom meta data on startup. + *

+ * Plugins should return the input custom map via {@link UnaryOperator#identity()} if no upgrade is required. + * @return Never {@code null}. The same or upgraded {@code MetaData.Custom} map. + * @throws IllegalStateException if the node should not start because at least one {@code MetaData.Custom} + * is unsupported + */ + public UnaryOperator> getCustomMetaDataUpgrader() { + return UnaryOperator.identity(); + } + /** * Old-style guice index level extension point. * @@ -163,6 +181,14 @@ public abstract class Plugin { @Deprecated public final void onModule(ActionModule module) {} + /** + * Old-style action extension point. + * + * @deprecated implement {@link SearchPlugin} instead + */ + @Deprecated + public final void onModule(SearchModule module) {} + /** * Provides the list of this plugin's custom thread pools, empty if * none. diff --git a/core/src/main/java/org/elasticsearch/plugins/SearchPlugin.java b/core/src/main/java/org/elasticsearch/plugins/SearchPlugin.java index 861f61eb396..97ee715ef6a 100644 --- a/core/src/main/java/org/elasticsearch/plugins/SearchPlugin.java +++ b/core/src/main/java/org/elasticsearch/plugins/SearchPlugin.java @@ -24,16 +24,21 @@ import org.elasticsearch.common.ParseField; import org.elasticsearch.common.io.stream.NamedWriteable; import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.io.stream.Writeable; -import org.elasticsearch.common.io.stream.Writeable.Reader; import org.elasticsearch.common.lucene.search.function.ScoreFunction; import org.elasticsearch.common.xcontent.XContent; import org.elasticsearch.index.query.QueryBuilder; import org.elasticsearch.index.query.QueryParser; import org.elasticsearch.index.query.functionscore.ScoreFunctionBuilder; import org.elasticsearch.index.query.functionscore.ScoreFunctionParser; +import org.elasticsearch.search.aggregations.Aggregation; +import org.elasticsearch.search.aggregations.AggregationBuilder; +import org.elasticsearch.search.aggregations.Aggregator; +import org.elasticsearch.search.aggregations.InternalAggregation; +import org.elasticsearch.search.aggregations.PipelineAggregationBuilder; import org.elasticsearch.search.aggregations.bucket.significant.SignificantTerms; import org.elasticsearch.search.aggregations.bucket.significant.heuristics.SignificanceHeuristic; import org.elasticsearch.search.aggregations.bucket.significant.heuristics.SignificanceHeuristicParser; +import org.elasticsearch.search.aggregations.pipeline.PipelineAggregator; import org.elasticsearch.search.aggregations.pipeline.movavg.MovAvgPipelineAggregator; import org.elasticsearch.search.aggregations.pipeline.movavg.models.MovAvgModel; import org.elasticsearch.search.fetch.FetchSubPhase; @@ -42,6 +47,7 @@ import org.elasticsearch.search.suggest.Suggester; import java.util.List; import java.util.Map; +import java.util.TreeMap; import static java.util.Collections.emptyList; import static java.util.Collections.emptyMap; @@ -94,16 +100,28 @@ public interface SearchPlugin { default List> getQueries() { return emptyList(); } + /** + * The new {@link Aggregation}s added by this plugin. + */ + default List getAggregations() { + return emptyList(); + } + /** + * The new {@link PipelineAggregator}s added by this plugin. + */ + default List getPipelineAggregations() { + return emptyList(); + } /** * Specification of custom {@link ScoreFunction}. */ class ScoreFunctionSpec> extends SearchExtensionSpec> { - public ScoreFunctionSpec(ParseField name, Reader reader, ScoreFunctionParser parser) { + public ScoreFunctionSpec(ParseField name, Writeable.Reader reader, ScoreFunctionParser parser) { super(name, reader, parser); } - public ScoreFunctionSpec(String name, Reader reader, ScoreFunctionParser parser) { + public ScoreFunctionSpec(String name, Writeable.Reader reader, ScoreFunctionParser parser) { super(name, reader, parser); } } @@ -122,7 +140,7 @@ public interface SearchPlugin { * {@link StreamInput} * @param parser the parser the reads the query builder from xcontent */ - public QuerySpec(ParseField name, Reader reader, QueryParser parser) { + public QuerySpec(ParseField name, Writeable.Reader reader, QueryParser parser) { super(name, reader, parser); } @@ -135,10 +153,143 @@ public interface SearchPlugin { * {@link StreamInput} * @param parser the parser the reads the query builder from xcontent */ - public QuerySpec(String name, Reader reader, QueryParser parser) { + public QuerySpec(String name, Writeable.Reader reader, QueryParser parser) { super(name, reader, parser); } } + /** + * Specification for an {@link Aggregation}. + */ + public static class AggregationSpec extends SearchExtensionSpec { + private final Map> resultReaders = new TreeMap<>(); + + /** + * Specification for an {@link Aggregation}. + * + * @param name holds the names by which this aggregation 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 {@link AggregationBuilder} should return + * from {@link NamedWriteable#getWriteableName()}. + * @param reader the reader registered for this aggregation's builder. Typically a reference to a constructor that takes a + * {@link StreamInput} + * @param parser the parser the reads the aggregation builder from xcontent + */ + public AggregationSpec(ParseField name, Writeable.Reader reader, Aggregator.Parser parser) { + super(name, reader, parser); + } + + /** + * Specification for an {@link Aggregation}. + * + * @param name the name by which this aggregation might be parsed or deserialized. Make sure that the {@link AggregationBuilder} + * returns this from {@link NamedWriteable#getWriteableName()}. + * @param reader the reader registered for this aggregation's builder. Typically a reference to a constructor that takes a + * {@link StreamInput} + * @param parser the parser the reads the aggregation builder from xcontent + */ + public AggregationSpec(String name, Writeable.Reader reader, Aggregator.Parser parser) { + super(name, reader, parser); + } + + /** + * Add a reader for the shard level results of the aggregation with {@linkplain #getName}'s {@link ParseField#getPreferredName()} as + * the {@link NamedWriteable#getWriteableName()}. + */ + public AggregationSpec addResultReader(Writeable.Reader resultReader) { + return addResultReader(getName().getPreferredName(), resultReader); + } + + /** + * Add a reader for the shard level results of the aggregation. + */ + public AggregationSpec addResultReader(String writeableName, Writeable.Reader resultReader) { + resultReaders.put(writeableName, resultReader); + return this; + } + + /** + * Get the readers that must be registered for this aggregation's results. + */ + public Map> getResultReaders() { + return resultReaders; + } + } + + /** + * Specification for a {@link PipelineAggregator}. + */ + public static class PipelineAggregationSpec extends SearchExtensionSpec { + private final Map> resultReaders = new TreeMap<>(); + private final Writeable.Reader aggregatorReader; + + /** + * Specification of a {@link PipelineAggregator}. + * + * @param name holds the names by which this aggregation might be parsed. The {@link ParseField#getPreferredName()} is special as it + * is the name by under which the readers are registered. So it is the name that the {@link PipelineAggregationBuilder} and + * {@link PipelineAggregator} should return from {@link NamedWriteable#getWriteableName()}. + * @param builderReader the reader registered for this aggregation's builder. Typically a reference to a constructor that takes a + * {@link StreamInput} + * @param aggregatorReader reads the {@link PipelineAggregator} from a stream + * @param parser reads the aggregation builder from XContent + */ + public PipelineAggregationSpec(ParseField name, + Writeable.Reader builderReader, + Writeable.Reader aggregatorReader, + PipelineAggregator.Parser parser) { + super(name, builderReader, parser); + this.aggregatorReader = aggregatorReader; + } + + /** + * Specification of a {@link PipelineAggregator}. + * + * @param name name by which this aggregation might be parsed or deserialized. Make sure it is the name that the + * {@link PipelineAggregationBuilder} and {@link PipelineAggregator} should return from + * {@link NamedWriteable#getWriteableName()}. + * @param builderReader the reader registered for this aggregation's builder. Typically a reference to a constructor that takes a + * {@link StreamInput} + * @param aggregatorReader reads the {@link PipelineAggregator} from a stream + * @param parser reads the aggregation builder from XContent + */ + public PipelineAggregationSpec(String name, + Writeable.Reader builderReader, + Writeable.Reader aggregatorReader, + PipelineAggregator.Parser parser) { + super(name, builderReader, parser); + this.aggregatorReader = aggregatorReader; + } + + /** + * Add a reader for the shard level results of the aggregation with {@linkplain #getName()}'s {@link ParseField#getPreferredName()} + * as the {@link NamedWriteable#getWriteableName()}. + */ + public PipelineAggregationSpec addResultReader(Writeable.Reader resultReader) { + return addResultReader(getName().getPreferredName(), resultReader); + } + + /** + * Add a reader for the shard level results of the aggregation. + */ + public PipelineAggregationSpec addResultReader(String writeableName, Writeable.Reader resultReader) { + resultReaders.put(writeableName, resultReader); + return this; + } + + /** + * The reader for the {@link PipelineAggregator}. + */ + public Writeable.Reader getAggregatorReader() { + return aggregatorReader; + } + + /** + * Get the readers that must be registered for this aggregation's results. + */ + public Map> getResultReaders() { + return resultReaders; + } + } + /** * Specification of search time behavior extension like a custom {@link MovAvgModel} or {@link ScoreFunction}. @@ -150,7 +301,7 @@ public interface SearchPlugin { */ class SearchExtensionSpec { private final ParseField name; - private final Writeable.Reader reader; + private final Writeable.Reader reader; private final P parser; /** @@ -162,7 +313,7 @@ public interface SearchPlugin { * @param reader reader that reads the behavior from the internode protocol * @param parser parser that read the behavior from a REST request */ - public SearchExtensionSpec(ParseField name, Writeable.Reader reader, P parser) { + public SearchExtensionSpec(ParseField name, Writeable.Reader reader, P parser) { this.name = name; this.reader = reader; this.parser = parser; @@ -176,7 +327,7 @@ public interface SearchPlugin { * @param reader reader that reads the behavior from the internode protocol * @param parser parser that read the behavior from a REST request */ - public SearchExtensionSpec(String name, Writeable.Reader reader, P parser) { + public SearchExtensionSpec(String name, Writeable.Reader reader, P parser) { this(new ParseField(name), reader, parser); } @@ -190,7 +341,7 @@ public interface SearchPlugin { /** * The reader responsible for reading the behavior from the internode protocol. */ - public Writeable.Reader getReader() { + public Writeable.Reader getReader() { return reader; } diff --git a/core/src/main/java/org/elasticsearch/rest/action/admin/cluster/RestCancelTasksAction.java b/core/src/main/java/org/elasticsearch/rest/action/admin/cluster/RestCancelTasksAction.java index 65786accd81..3c558fba937 100644 --- a/core/src/main/java/org/elasticsearch/rest/action/admin/cluster/RestCancelTasksAction.java +++ b/core/src/main/java/org/elasticsearch/rest/action/admin/cluster/RestCancelTasksAction.java @@ -19,9 +19,7 @@ package org.elasticsearch.rest.action.admin.cluster; -import org.elasticsearch.action.ActionListener; import org.elasticsearch.action.admin.cluster.node.tasks.cancel.CancelTasksRequest; -import org.elasticsearch.action.admin.cluster.node.tasks.cancel.CancelTasksResponse; import org.elasticsearch.client.node.NodeClient; import org.elasticsearch.cluster.service.ClusterService; import org.elasticsearch.common.Strings; @@ -31,11 +29,10 @@ import org.elasticsearch.rest.BaseRestHandler; import org.elasticsearch.rest.RestChannel; import org.elasticsearch.rest.RestController; import org.elasticsearch.rest.RestRequest; -import org.elasticsearch.rest.action.RestToXContentListener; import org.elasticsearch.tasks.TaskId; import static org.elasticsearch.rest.RestRequest.Method.POST; -import static org.elasticsearch.rest.action.admin.cluster.RestListTasksAction.nodeSettingListener; +import static org.elasticsearch.rest.action.admin.cluster.RestListTasksAction.listTasksResponseListener; public class RestCancelTasksAction extends BaseRestHandler { @@ -61,8 +58,7 @@ public class RestCancelTasksAction extends BaseRestHandler { cancelTasksRequest.setNodesIds(nodesIds); cancelTasksRequest.setActions(actions); cancelTasksRequest.setParentTaskId(parentTaskId); - ActionListener listener = nodeSettingListener(clusterService, new RestToXContentListener<>(channel)); - client.admin().cluster().cancelTasks(cancelTasksRequest, listener); + client.admin().cluster().cancelTasks(cancelTasksRequest, listTasksResponseListener(clusterService, channel)); } @Override diff --git a/core/src/main/java/org/elasticsearch/rest/action/admin/cluster/RestListTasksAction.java b/core/src/main/java/org/elasticsearch/rest/action/admin/cluster/RestListTasksAction.java index 13b97cb0942..d5ff427e3d0 100644 --- a/core/src/main/java/org/elasticsearch/rest/action/admin/cluster/RestListTasksAction.java +++ b/core/src/main/java/org/elasticsearch/rest/action/admin/cluster/RestListTasksAction.java @@ -28,10 +28,15 @@ import org.elasticsearch.common.Strings; import org.elasticsearch.common.inject.Inject; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.unit.TimeValue; +import org.elasticsearch.common.xcontent.XContentBuilder; import org.elasticsearch.rest.BaseRestHandler; +import org.elasticsearch.rest.BytesRestResponse; import org.elasticsearch.rest.RestChannel; import org.elasticsearch.rest.RestController; import org.elasticsearch.rest.RestRequest; +import org.elasticsearch.rest.RestResponse; +import org.elasticsearch.rest.RestStatus; +import org.elasticsearch.rest.action.RestBuilderListener; import org.elasticsearch.rest.action.RestToXContentListener; import org.elasticsearch.tasks.TaskId; @@ -68,27 +73,30 @@ public class RestListTasksAction extends BaseRestHandler { @Override public void handleRequest(final RestRequest request, final RestChannel channel, final NodeClient client) { - ActionListener listener = nodeSettingListener(clusterService, new RestToXContentListener<>(channel)); - client.admin().cluster().listTasks(generateListTasksRequest(request), listener); + client.admin().cluster().listTasks(generateListTasksRequest(request), listTasksResponseListener(clusterService, channel)); } /** - * Wrap the normal channel listener in one that sets the discovery nodes on the response so we can support all of it's toXContent - * formats. + * Standard listener for extensions of {@link ListTasksResponse} that supports {@code group_by=nodes}. */ - public static ActionListener nodeSettingListener(ClusterService clusterService, - ActionListener channelListener) { - return new ActionListener() { - @Override - public void onResponse(T response) { - channelListener.onResponse(response); - } - - @Override - public void onFailure(Exception e) { - channelListener.onFailure(e); - } - }; + public static ActionListener listTasksResponseListener(ClusterService clusterService, + RestChannel channel) { + String groupBy = channel.request().param("group_by", "nodes"); + if ("nodes".equals(groupBy)) { + return new RestBuilderListener(channel) { + @Override + public RestResponse buildResponse(T response, XContentBuilder builder) throws Exception { + builder.startObject(); + response.toXContentGroupedByNode(builder, channel.request(), clusterService.state().nodes()); + builder.endObject(); + return new BytesRestResponse(RestStatus.OK, builder); + } + }; + } else if ("parents".equals(groupBy)) { + return new RestToXContentListener<>(channel); + } else { + throw new IllegalArgumentException("[group_by] must be one of [nodes] or [parents] but was [" + groupBy + "]"); + } } @Override diff --git a/core/src/main/java/org/elasticsearch/rest/action/admin/indices/RestCreateIndexAction.java b/core/src/main/java/org/elasticsearch/rest/action/admin/indices/RestCreateIndexAction.java index 524cd5feb26..2a7f2a629a7 100644 --- a/core/src/main/java/org/elasticsearch/rest/action/admin/indices/RestCreateIndexAction.java +++ b/core/src/main/java/org/elasticsearch/rest/action/admin/indices/RestCreateIndexAction.java @@ -43,7 +43,6 @@ public class RestCreateIndexAction extends BaseRestHandler { public RestCreateIndexAction(Settings settings, RestController controller) { super(settings); controller.registerHandler(RestRequest.Method.PUT, "/{index}", this); - controller.registerHandler(RestRequest.Method.POST, "/{index}", this); } @SuppressWarnings({"unchecked"}) diff --git a/core/src/main/java/org/elasticsearch/rest/action/search/RestMultiSearchAction.java b/core/src/main/java/org/elasticsearch/rest/action/search/RestMultiSearchAction.java index a2cc4ce1b7f..ae320bccac2 100644 --- a/core/src/main/java/org/elasticsearch/rest/action/search/RestMultiSearchAction.java +++ b/core/src/main/java/org/elasticsearch/rest/action/search/RestMultiSearchAction.java @@ -19,6 +19,10 @@ package org.elasticsearch.rest.action.search; +import java.io.IOException; +import java.util.Map; +import java.util.function.BiConsumer; + import org.elasticsearch.ElasticsearchParseException; import org.elasticsearch.action.search.MultiSearchRequest; import org.elasticsearch.action.search.SearchRequest; @@ -33,20 +37,14 @@ import org.elasticsearch.common.xcontent.XContent; import org.elasticsearch.common.xcontent.XContentFactory; import org.elasticsearch.common.xcontent.XContentParser; import org.elasticsearch.index.query.QueryParseContext; -import org.elasticsearch.indices.query.IndicesQueriesRegistry; import org.elasticsearch.rest.BaseRestHandler; import org.elasticsearch.rest.RestChannel; import org.elasticsearch.rest.RestController; -import org.elasticsearch.rest.RestRequest; +import org.elasticsearch.rest.RestRequest; import org.elasticsearch.rest.action.RestActions; import org.elasticsearch.rest.action.RestToXContentListener; -import org.elasticsearch.search.aggregations.AggregatorParsers; +import org.elasticsearch.search.SearchRequestParsers; import org.elasticsearch.search.builder.SearchSourceBuilder; -import org.elasticsearch.search.suggest.Suggesters; - -import java.io.IOException; -import java.util.Map; -import java.util.function.BiConsumer; import static org.elasticsearch.common.xcontent.support.XContentMapValues.lenientNodeBooleanValue; import static org.elasticsearch.common.xcontent.support.XContentMapValues.nodeStringArrayValue; @@ -59,16 +57,12 @@ import static org.elasticsearch.rest.RestRequest.Method.POST; public class RestMultiSearchAction extends BaseRestHandler { private final boolean allowExplicitIndex; - private final IndicesQueriesRegistry indicesQueriesRegistry; - private final AggregatorParsers aggParsers; - private final Suggesters suggesters; + private final SearchRequestParsers searchRequestParsers; @Inject - public RestMultiSearchAction(Settings settings, RestController controller, IndicesQueriesRegistry indicesQueriesRegistry, - AggregatorParsers aggParsers, Suggesters suggesters) { + public RestMultiSearchAction(Settings settings, RestController controller, SearchRequestParsers searchRequestParsers) { super(settings); - this.aggParsers = aggParsers; - this.suggesters = suggesters; + this.searchRequestParsers = searchRequestParsers; controller.registerHandler(GET, "/_msearch", this); controller.registerHandler(POST, "/_msearch", this); @@ -78,13 +72,11 @@ public class RestMultiSearchAction extends BaseRestHandler { controller.registerHandler(POST, "/{index}/{type}/_msearch", this); this.allowExplicitIndex = MULTI_ALLOW_EXPLICIT_INDEX.get(settings); - this.indicesQueriesRegistry = indicesQueriesRegistry; } @Override public void handleRequest(final RestRequest request, final RestChannel channel, final NodeClient client) throws Exception { - MultiSearchRequest multiSearchRequest = parseRequest(request, allowExplicitIndex, indicesQueriesRegistry, parseFieldMatcher, - aggParsers, suggesters); + MultiSearchRequest multiSearchRequest = parseRequest(request, allowExplicitIndex, searchRequestParsers, parseFieldMatcher); client.multiSearch(multiSearchRequest, new RestToXContentListener<>(channel)); } @@ -92,8 +84,8 @@ public class RestMultiSearchAction extends BaseRestHandler { * Parses a {@link RestRequest} body and returns a {@link MultiSearchRequest} */ public static MultiSearchRequest parseRequest(RestRequest restRequest, boolean allowExplicitIndex, - IndicesQueriesRegistry queriesRegistry, ParseFieldMatcher parseFieldMatcher, - AggregatorParsers aggParsers, Suggesters suggesters) throws IOException { + SearchRequestParsers searchRequestParsers, + ParseFieldMatcher parseFieldMatcher) throws IOException { MultiSearchRequest multiRequest = new MultiSearchRequest(); if (restRequest.hasParam("max_concurrent_searches")) { @@ -102,8 +94,10 @@ public class RestMultiSearchAction extends BaseRestHandler { parseMultiLineRequest(restRequest, multiRequest.indicesOptions(), allowExplicitIndex, (searchRequest, bytes) -> { try (XContentParser requestParser = XContentFactory.xContent(bytes).createParser(bytes)) { - final QueryParseContext queryParseContext = new QueryParseContext(queriesRegistry, requestParser, parseFieldMatcher); - searchRequest.source(SearchSourceBuilder.fromXContent(queryParseContext, aggParsers, suggesters)); + final QueryParseContext queryParseContext = new QueryParseContext(searchRequestParsers.queryParsers, + requestParser, parseFieldMatcher); + searchRequest.source(SearchSourceBuilder.fromXContent(queryParseContext, + searchRequestParsers.aggParsers, searchRequestParsers.suggesters)); multiRequest.add(searchRequest); } catch (IOException e) { throw new ElasticsearchParseException("Exception when parsing search request", e); diff --git a/core/src/main/java/org/elasticsearch/rest/action/search/RestSearchAction.java b/core/src/main/java/org/elasticsearch/rest/action/search/RestSearchAction.java index 989e6ed4408..f75eccbd302 100644 --- a/core/src/main/java/org/elasticsearch/rest/action/search/RestSearchAction.java +++ b/core/src/main/java/org/elasticsearch/rest/action/search/RestSearchAction.java @@ -41,6 +41,7 @@ import org.elasticsearch.rest.RestRequest; import org.elasticsearch.rest.action.RestActions; import org.elasticsearch.rest.action.RestStatusToXContentListener; import org.elasticsearch.search.Scroll; +import org.elasticsearch.search.SearchRequestParsers; import org.elasticsearch.search.aggregations.AggregatorParsers; import org.elasticsearch.search.builder.SearchSourceBuilder; import org.elasticsearch.search.fetch.subphase.FetchSourceContext; @@ -63,17 +64,12 @@ import static org.elasticsearch.search.suggest.SuggestBuilders.termSuggestion; */ public class RestSearchAction extends BaseRestHandler { - private final IndicesQueriesRegistry queryRegistry; - private final AggregatorParsers aggParsers; - private final Suggesters suggesters; + private final SearchRequestParsers searchRequestParsers; @Inject - public RestSearchAction(Settings settings, RestController controller, IndicesQueriesRegistry queryRegistry, - AggregatorParsers aggParsers, Suggesters suggesters) { + public RestSearchAction(Settings settings, RestController controller, SearchRequestParsers searchRequestParsers) { super(settings); - this.queryRegistry = queryRegistry; - this.aggParsers = aggParsers; - this.suggesters = suggesters; + this.searchRequestParsers = searchRequestParsers; controller.registerHandler(GET, "/_search", this); controller.registerHandler(POST, "/_search", this); controller.registerHandler(GET, "/{index}/_search", this); @@ -86,7 +82,7 @@ public class RestSearchAction extends BaseRestHandler { public void handleRequest(final RestRequest request, final RestChannel channel, final NodeClient client) throws IOException { SearchRequest searchRequest = new SearchRequest(); BytesReference restContent = RestActions.hasBodyContent(request) ? RestActions.getRestContent(request) : null; - parseSearchRequest(searchRequest, queryRegistry, request, parseFieldMatcher, aggParsers, suggesters, restContent); + parseSearchRequest(searchRequest, request, searchRequestParsers, parseFieldMatcher, restContent); client.search(searchRequest, new RestStatusToXContentListener<>(channel)); } @@ -99,9 +95,8 @@ public class RestSearchAction extends BaseRestHandler { * content is read from the request using * RestAction.hasBodyContent. */ - public static void parseSearchRequest(SearchRequest searchRequest, IndicesQueriesRegistry indicesQueriesRegistry, RestRequest request, - ParseFieldMatcher parseFieldMatcher, AggregatorParsers aggParsers, Suggesters suggesters, BytesReference restContent) - throws IOException { + public static void parseSearchRequest(SearchRequest searchRequest, RestRequest request, SearchRequestParsers searchRequestParsers, + ParseFieldMatcher parseFieldMatcher, BytesReference restContent) throws IOException { if (searchRequest.source() == null) { searchRequest.source(new SearchSourceBuilder()); @@ -109,8 +104,8 @@ public class RestSearchAction extends BaseRestHandler { searchRequest.indices(Strings.splitStringByCommaToArray(request.param("index"))); if (restContent != null) { try (XContentParser parser = XContentFactory.xContent(restContent).createParser(restContent)) { - QueryParseContext context = new QueryParseContext(indicesQueriesRegistry, parser, parseFieldMatcher); - searchRequest.source().parseXContent(context, aggParsers, suggesters); + QueryParseContext context = new QueryParseContext(searchRequestParsers.queryParsers, parser, parseFieldMatcher); + searchRequest.source().parseXContent(context, searchRequestParsers.aggParsers, searchRequestParsers.suggesters); } } diff --git a/core/src/main/java/org/elasticsearch/rest/action/search/RestSuggestAction.java b/core/src/main/java/org/elasticsearch/rest/action/search/RestSuggestAction.java index 62db8c3786c..b55a1590f3d 100644 --- a/core/src/main/java/org/elasticsearch/rest/action/search/RestSuggestAction.java +++ b/core/src/main/java/org/elasticsearch/rest/action/search/RestSuggestAction.java @@ -19,6 +19,8 @@ package org.elasticsearch.rest.action.search; +import java.io.IOException; + import org.elasticsearch.action.search.SearchRequest; import org.elasticsearch.action.search.SearchResponse; import org.elasticsearch.action.support.IndicesOptions; @@ -31,7 +33,6 @@ import org.elasticsearch.common.xcontent.XContentBuilder; import org.elasticsearch.common.xcontent.XContentFactory; import org.elasticsearch.common.xcontent.XContentParser; import org.elasticsearch.index.query.QueryParseContext; -import org.elasticsearch.indices.query.IndicesQueriesRegistry; import org.elasticsearch.rest.BaseRestHandler; import org.elasticsearch.rest.BytesRestResponse; import org.elasticsearch.rest.RestChannel; @@ -41,12 +42,10 @@ import org.elasticsearch.rest.RestResponse; import org.elasticsearch.rest.RestStatus; import org.elasticsearch.rest.action.RestActions; import org.elasticsearch.rest.action.RestBuilderListener; +import org.elasticsearch.search.SearchRequestParsers; import org.elasticsearch.search.builder.SearchSourceBuilder; import org.elasticsearch.search.suggest.Suggest; import org.elasticsearch.search.suggest.SuggestBuilder; -import org.elasticsearch.search.suggest.Suggesters; - -import java.io.IOException; import static org.elasticsearch.rest.RestRequest.Method.GET; import static org.elasticsearch.rest.RestRequest.Method.POST; @@ -54,15 +53,13 @@ import static org.elasticsearch.rest.action.RestActions.buildBroadcastShardsHead public class RestSuggestAction extends BaseRestHandler { - private final IndicesQueriesRegistry queryRegistry; - private final Suggesters suggesters; + private final SearchRequestParsers searchRequestParsers; @Inject public RestSuggestAction(Settings settings, RestController controller, - IndicesQueriesRegistry queryRegistry, Suggesters suggesters) { + SearchRequestParsers searchRequestParsers) { super(settings); - this.queryRegistry = queryRegistry; - this.suggesters = suggesters; + this.searchRequestParsers = searchRequestParsers; controller.registerHandler(POST, "/_suggest", this); controller.registerHandler(GET, "/_suggest", this); controller.registerHandler(POST, "/{index}/_suggest", this); @@ -77,8 +74,8 @@ public class RestSuggestAction extends BaseRestHandler { if (RestActions.hasBodyContent(request)) { final BytesReference sourceBytes = RestActions.getRestContent(request); try (XContentParser parser = XContentFactory.xContent(sourceBytes).createParser(sourceBytes)) { - final QueryParseContext context = new QueryParseContext(queryRegistry, parser, parseFieldMatcher); - searchRequest.source().suggest(SuggestBuilder.fromXContent(context, suggesters)); + final QueryParseContext context = new QueryParseContext(searchRequestParsers.queryParsers, parser, parseFieldMatcher); + searchRequest.source().suggest(SuggestBuilder.fromXContent(context, searchRequestParsers.suggesters)); } } else { throw new IllegalArgumentException("no content or source provided to execute suggestion"); diff --git a/core/src/main/java/org/elasticsearch/search/SearchModule.java b/core/src/main/java/org/elasticsearch/search/SearchModule.java index 5a5137ea571..ab36f03639c 100644 --- a/core/src/main/java/org/elasticsearch/search/SearchModule.java +++ b/core/src/main/java/org/elasticsearch/search/SearchModule.java @@ -21,14 +21,11 @@ package org.elasticsearch.search; import org.apache.lucene.search.BooleanQuery; import org.elasticsearch.common.NamedRegistry; -import org.elasticsearch.common.ParseField; import org.elasticsearch.common.geo.ShapesAvailability; import org.elasticsearch.common.geo.builders.ShapeBuilders; import org.elasticsearch.common.inject.AbstractModule; -import org.elasticsearch.common.io.stream.NamedWriteable; import org.elasticsearch.common.io.stream.NamedWriteableRegistry.Entry; import org.elasticsearch.common.io.stream.Writeable; -import org.elasticsearch.common.io.stream.Writeable.Reader; import org.elasticsearch.common.settings.Setting; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.xcontent.ParseFieldRegistry; @@ -91,7 +88,9 @@ import org.elasticsearch.index.query.functionscore.ScriptScoreFunctionBuilder; import org.elasticsearch.index.query.functionscore.WeightBuilder; import org.elasticsearch.indices.query.IndicesQueriesRegistry; import org.elasticsearch.plugins.SearchPlugin; +import org.elasticsearch.plugins.SearchPlugin.AggregationSpec; import org.elasticsearch.plugins.SearchPlugin.FetchPhaseConstructionContext; +import org.elasticsearch.plugins.SearchPlugin.PipelineAggregationSpec; import org.elasticsearch.plugins.SearchPlugin.QuerySpec; import org.elasticsearch.plugins.SearchPlugin.ScoreFunctionSpec; import org.elasticsearch.plugins.SearchPlugin.SearchExtensionSpec; @@ -279,7 +278,6 @@ import org.elasticsearch.search.suggest.term.TermSuggester; import java.util.ArrayList; import java.util.List; import java.util.Map; -import java.util.TreeMap; import java.util.function.Consumer; import java.util.function.Function; @@ -290,6 +288,8 @@ import static java.util.Objects.requireNonNull; * Sets up things that can be done at search time like queries, aggregations, and suggesters. */ public class SearchModule extends AbstractModule { + public static final Setting INDICES_MAX_CLAUSE_COUNT_SETTING = Setting.intSetting("indices.query.bool.max_clause_count", + 1024, 1, Integer.MAX_VALUE, Setting.Property.NodeScope); private final boolean transportClient; private final Map highlighters; @@ -309,11 +309,7 @@ public class SearchModule extends AbstractModule { private final Settings settings; private final List namedWriteables = new ArrayList<>(); - public static final Setting INDICES_MAX_CLAUSE_COUNT_SETTING = Setting.intSetting("indices.query.bool.max_clause_count", - 1024, 1, Integer.MAX_VALUE, Setting.Property.NodeScope); - - // pkg private so tests can mock - Class searchServiceImpl = SearchService.class; + private final SearchRequestParsers searchRequestParsers; public SearchModule(Settings settings, boolean transportClient, List plugins) { this.settings = settings; @@ -327,9 +323,11 @@ public class SearchModule extends AbstractModule { registerValueFormats(); registerSignificanceHeuristics(plugins); registerMovingAverageModels(plugins); - registerBuiltinAggregations(); + registerAggregations(plugins); + registerPipelineAggregations(plugins); registerFetchSubPhases(plugins); registerShapes(); + searchRequestParsers = new SearchRequestParsers(queryParserRegistry, aggregatorParsers, getSuggesters()); } public List getNamedWriteables() { @@ -344,6 +342,10 @@ public class SearchModule extends AbstractModule { return queryParserRegistry; } + public SearchRequestParsers getSearchRequestParsers() { + return searchRequestParsers; + } + /** * Returns the {@link Highlighter} registry */ @@ -366,285 +368,206 @@ public class SearchModule extends AbstractModule { } /** - * Register an aggregation. + * Parsers for {@link AggregationBuilder}s and {@link PipelineAggregationBuilder}s. */ - public void registerAggregation(AggregationSpec spec) { + public AggregatorParsers getAggregatorParsers() { + return aggregatorParsers; + } + + + @Override + protected void configure() { if (false == transportClient) { - aggregationParserRegistry.register(spec.parser, spec.name); + bind(IndicesQueriesRegistry.class).toInstance(queryParserRegistry); + bind(SearchRequestParsers.class).toInstance(searchRequestParsers); + configureSearch(); } - namedWriteables.add(new Entry(AggregationBuilder.class, spec.name.getPreferredName(), spec.builderReader)); - for (Map.Entry> t : spec.resultReaders.entrySet()) { + } + + private void registerAggregations(List plugins) { + registerAggregation(new AggregationSpec(AvgAggregationBuilder.NAME, AvgAggregationBuilder::new, new AvgParser()) + .addResultReader(InternalAvg::new)); + registerAggregation(new AggregationSpec(SumAggregationBuilder.NAME, SumAggregationBuilder::new, new SumParser()) + .addResultReader(InternalSum::new)); + registerAggregation(new AggregationSpec(MinAggregationBuilder.NAME, MinAggregationBuilder::new, new MinParser()) + .addResultReader(InternalMin::new)); + registerAggregation(new AggregationSpec(MaxAggregationBuilder.NAME, MaxAggregationBuilder::new, new MaxParser()) + .addResultReader(InternalMax::new)); + registerAggregation(new AggregationSpec(StatsAggregationBuilder.NAME, StatsAggregationBuilder::new, new StatsParser()) + .addResultReader(InternalStats::new)); + registerAggregation(new AggregationSpec(ExtendedStatsAggregationBuilder.NAME, ExtendedStatsAggregationBuilder::new, + new ExtendedStatsParser()).addResultReader(InternalExtendedStats::new)); + registerAggregation(new AggregationSpec(ValueCountAggregationBuilder.NAME, ValueCountAggregationBuilder::new, + new ValueCountParser()).addResultReader(InternalValueCount::new)); + registerAggregation(new AggregationSpec(PercentilesAggregationBuilder.NAME, PercentilesAggregationBuilder::new, + new PercentilesParser()) + .addResultReader(InternalTDigestPercentiles.NAME, InternalTDigestPercentiles::new) + .addResultReader(InternalHDRPercentiles.NAME, InternalHDRPercentiles::new)); + registerAggregation(new AggregationSpec(PercentileRanksAggregationBuilder.NAME, PercentileRanksAggregationBuilder::new, + new PercentileRanksParser()) + .addResultReader(InternalTDigestPercentileRanks.NAME, InternalTDigestPercentileRanks::new) + .addResultReader(InternalHDRPercentileRanks.NAME, InternalHDRPercentileRanks::new)); + registerAggregation(new AggregationSpec(CardinalityAggregationBuilder.NAME, CardinalityAggregationBuilder::new, + new CardinalityParser()).addResultReader(InternalCardinality::new)); + registerAggregation(new AggregationSpec(GlobalAggregationBuilder.NAME, GlobalAggregationBuilder::new, + GlobalAggregationBuilder::parse).addResultReader(InternalGlobal::new)); + registerAggregation(new AggregationSpec(MissingAggregationBuilder.NAME, MissingAggregationBuilder::new, new MissingParser()) + .addResultReader(InternalMissing::new)); + registerAggregation(new AggregationSpec(FilterAggregationBuilder.NAME, FilterAggregationBuilder::new, + FilterAggregationBuilder::parse).addResultReader(InternalFilter::new)); + registerAggregation(new AggregationSpec(FiltersAggregationBuilder.NAME, FiltersAggregationBuilder::new, + FiltersAggregationBuilder::parse).addResultReader(InternalFilters::new)); + registerAggregation(new AggregationSpec(SamplerAggregationBuilder.NAME, SamplerAggregationBuilder::new, + SamplerAggregationBuilder::parse) + .addResultReader(InternalSampler.NAME, InternalSampler::new) + .addResultReader(UnmappedSampler.NAME, UnmappedSampler::new)); + registerAggregation(new AggregationSpec(DiversifiedAggregationBuilder.NAME, DiversifiedAggregationBuilder::new, + new DiversifiedSamplerParser()) + /* Reuses result readers from SamplerAggregator*/); + registerAggregation(new AggregationSpec(TermsAggregationBuilder.NAME, TermsAggregationBuilder::new, new TermsParser()) + .addResultReader(StringTerms.NAME, StringTerms::new) + .addResultReader(UnmappedTerms.NAME, UnmappedTerms::new) + .addResultReader(LongTerms.NAME, LongTerms::new) + .addResultReader(DoubleTerms.NAME, DoubleTerms::new)); + registerAggregation(new AggregationSpec(SignificantTermsAggregationBuilder.NAME, SignificantTermsAggregationBuilder::new, + new SignificantTermsParser(significanceHeuristicParserRegistry, queryParserRegistry)) + .addResultReader(SignificantStringTerms.NAME, SignificantStringTerms::new) + .addResultReader(SignificantLongTerms.NAME, SignificantLongTerms::new) + .addResultReader(UnmappedSignificantTerms.NAME, UnmappedSignificantTerms::new)); + registerAggregation(new AggregationSpec(RangeAggregationBuilder.NAME, RangeAggregationBuilder::new, + new RangeParser()).addResultReader(InternalRange::new)); + registerAggregation(new AggregationSpec(DateRangeAggregationBuilder.NAME, DateRangeAggregationBuilder::new, new DateRangeParser()) + .addResultReader(InternalDateRange::new)); + registerAggregation(new AggregationSpec(IpRangeAggregationBuilder.NAME, IpRangeAggregationBuilder::new, new IpRangeParser()) + .addResultReader(InternalBinaryRange::new)); + registerAggregation(new AggregationSpec(HistogramAggregationBuilder.NAME, HistogramAggregationBuilder::new, new HistogramParser()) + .addResultReader(InternalHistogram::new)); + registerAggregation(new AggregationSpec(DateHistogramAggregationBuilder.NAME, DateHistogramAggregationBuilder::new, + new DateHistogramParser()).addResultReader(InternalDateHistogram::new)); + registerAggregation(new AggregationSpec(GeoDistanceAggregationBuilder.NAME, GeoDistanceAggregationBuilder::new, + new GeoDistanceParser()).addResultReader(InternalGeoDistance::new)); + registerAggregation(new AggregationSpec(GeoGridAggregationBuilder.NAME, GeoGridAggregationBuilder::new, new GeoHashGridParser()) + .addResultReader(InternalGeoHashGrid::new)); + registerAggregation(new AggregationSpec(NestedAggregationBuilder.NAME, NestedAggregationBuilder::new, + NestedAggregationBuilder::parse).addResultReader(InternalNested::new)); + registerAggregation(new AggregationSpec(ReverseNestedAggregationBuilder.NAME, ReverseNestedAggregationBuilder::new, + ReverseNestedAggregationBuilder::parse).addResultReader(InternalReverseNested::new)); + registerAggregation(new AggregationSpec(TopHitsAggregationBuilder.NAME, TopHitsAggregationBuilder::new, + TopHitsAggregationBuilder::parse).addResultReader(InternalTopHits::new)); + registerAggregation(new AggregationSpec(GeoBoundsAggregationBuilder.NAME, GeoBoundsAggregationBuilder::new, new GeoBoundsParser()) + .addResultReader(InternalGeoBounds::new)); + registerAggregation(new AggregationSpec(GeoCentroidAggregationBuilder.NAME, GeoCentroidAggregationBuilder::new, + new GeoCentroidParser()).addResultReader(InternalGeoCentroid::new)); + registerAggregation(new AggregationSpec(ScriptedMetricAggregationBuilder.NAME, ScriptedMetricAggregationBuilder::new, + ScriptedMetricAggregationBuilder::parse).addResultReader(InternalScriptedMetric::new)); + registerAggregation(new AggregationSpec(ChildrenAggregationBuilder.NAME, ChildrenAggregationBuilder::new, + ChildrenAggregationBuilder::parse).addResultReader(InternalChildren::new)); + + registerFromPlugin(plugins, SearchPlugin::getAggregations, this::registerAggregation); + } + + private void registerAggregation(AggregationSpec spec) { + if (false == transportClient) { + aggregationParserRegistry.register(spec.getParser(), spec.getName()); + } + namedWriteables.add(new Entry(AggregationBuilder.class, spec.getName().getPreferredName(), spec.getReader())); + for (Map.Entry> t : spec.getResultReaders().entrySet()) { String writeableName = t.getKey(); Writeable.Reader internalReader = t.getValue(); namedWriteables.add(new Entry(InternalAggregation.class, writeableName, internalReader)); } } - public static class AggregationSpec { - private final Map> resultReaders = new TreeMap<>(); - private final Writeable.Reader builderReader; - private final Aggregator.Parser parser; - private final ParseField name; - - /** - * Register an aggregation. - * - * @param builderReader reads the {@link AggregationBuilder} from a stream - * @param parser reads the aggregation builder from XContent - * @param name names by which the aggregation may be parsed. The first name is special because it is the name that the reader is - * registered under. - */ - public AggregationSpec(Reader builderReader, Aggregator.Parser parser, - ParseField name) { - this.builderReader = builderReader; - this.parser = parser; - this.name = name; - } - - /** - * Add a reader for the shard level results of the aggregation with {@linkplain #name}'s {@link ParseField#getPreferredName()} as - * the {@link NamedWriteable#getWriteableName()}. - */ - public AggregationSpec addResultReader(Writeable.Reader resultReader) { - return addResultReader(name.getPreferredName(), resultReader); - } - - /** - * Add a reader for the shard level results of the aggregation. - */ - public AggregationSpec addResultReader(String writeableName, Writeable.Reader resultReader) { - resultReaders.put(writeableName, resultReader); - return this; - } - } - - /** - * Register a pipeline aggregation. - */ - public void registerPipelineAggregation(PipelineAggregationSpec spec) { - if (false == transportClient) { - pipelineAggregationParserRegistry.register(spec.parser, spec.name); - } - namedWriteables.add(new Entry(PipelineAggregationBuilder.class, spec.name.getPreferredName(), spec.builderReader)); - namedWriteables.add(new Entry(PipelineAggregator.class, spec.name.getPreferredName(), spec.aggregatorReader)); - for (Map.Entry> resultReader : spec.resultReaders.entrySet()) { - namedWriteables.add(new Entry(InternalAggregation.class, resultReader.getKey(), resultReader.getValue())); - } - } - - public static class PipelineAggregationSpec { - private final Map> resultReaders = new TreeMap<>(); - private final Writeable.Reader builderReader; - private final Writeable.Reader aggregatorReader; - private final PipelineAggregator.Parser parser; - private final ParseField name; - - /** - * Register a pipeline aggregation. - * - * @param builderReader reads the {@link PipelineAggregationBuilder} from a stream - * @param aggregatorReader reads the {@link PipelineAggregator} from a stream - * @param parser reads the aggregation builder from XContent - * @param name names by which the aggregation may be parsed. The first name is special because it is the name that the reader is - * registered under. - */ - public PipelineAggregationSpec(Reader builderReader, - Writeable.Reader aggregatorReader, - PipelineAggregator.Parser parser, ParseField name) { - this.builderReader = builderReader; - this.aggregatorReader = aggregatorReader; - this.parser = parser; - this.name = name; - } - - /** - * Add a reader for the shard level results of the aggregation with {@linkplain #name}'s {@link ParseField#getPreferredName()} as - * the {@link NamedWriteable#getWriteableName()}. - */ - public PipelineAggregationSpec addResultReader(Writeable.Reader resultReader) { - return addResultReader(name.getPreferredName(), resultReader); - } - - /** - * Add a reader for the shard level results of the aggregation. - */ - public PipelineAggregationSpec addResultReader(String writeableName, Writeable.Reader resultReader) { - resultReaders.put(writeableName, resultReader); - return this; - } - } - - @Override - protected void configure() { - if (false == transportClient) { - /* - * Nothing is bound for transport client *but* SearchModule is still responsible for settings up the things like the - * NamedWriteableRegistry. - */ - bind(IndicesQueriesRegistry.class).toInstance(queryParserRegistry); - bind(Suggesters.class).toInstance(getSuggesters()); - configureSearch(); - bind(AggregatorParsers.class).toInstance(aggregatorParsers); - } - } - - private void registerBuiltinAggregations() { - registerAggregation(new AggregationSpec(AvgAggregationBuilder::new, new AvgParser(), AvgAggregationBuilder.AGGREGATION_NAME_FIELD) - .addResultReader(InternalAvg::new)); - registerAggregation(new AggregationSpec(SumAggregationBuilder::new, new SumParser(), SumAggregationBuilder.AGGREGATION_NAME_FIELD) - .addResultReader(InternalSum::new)); - registerAggregation(new AggregationSpec(MinAggregationBuilder::new, new MinParser(), MinAggregationBuilder.AGGREGATION_NAME_FIELD) - .addResultReader(InternalMin::new)); - registerAggregation(new AggregationSpec(MaxAggregationBuilder::new, new MaxParser(), MaxAggregationBuilder.AGGREGATION_NAME_FIELD) - .addResultReader(InternalMax::new)); - registerAggregation(new AggregationSpec(StatsAggregationBuilder::new, new StatsParser(), - StatsAggregationBuilder.AGGREGATION_NAME_FIELD).addResultReader(InternalStats::new)); - registerAggregation(new AggregationSpec(ExtendedStatsAggregationBuilder::new, new ExtendedStatsParser(), - ExtendedStatsAggregationBuilder.AGGREGATION_NAME_FIELD).addResultReader(InternalExtendedStats::new)); - registerAggregation(new AggregationSpec(ValueCountAggregationBuilder::new, new ValueCountParser(), - ValueCountAggregationBuilder.AGGREGATION_NAME_FIELD).addResultReader(InternalValueCount::new)); - registerAggregation(new AggregationSpec(PercentilesAggregationBuilder::new, new PercentilesParser(), - PercentilesAggregationBuilder.AGGREGATION_NAME_FIELD) - .addResultReader(InternalTDigestPercentiles.NAME, InternalTDigestPercentiles::new) - .addResultReader(InternalHDRPercentiles.NAME, InternalHDRPercentiles::new)); - registerAggregation(new AggregationSpec(PercentileRanksAggregationBuilder::new, new PercentileRanksParser(), - PercentileRanksAggregationBuilder.AGGREGATION_NAME_FIELD) - .addResultReader(InternalTDigestPercentileRanks.NAME, InternalTDigestPercentileRanks::new) - .addResultReader(InternalHDRPercentileRanks.NAME, InternalHDRPercentileRanks::new)); - registerAggregation(new AggregationSpec(CardinalityAggregationBuilder::new, new CardinalityParser(), - CardinalityAggregationBuilder.AGGREGATION_NAME_FIELD).addResultReader(InternalCardinality::new)); - registerAggregation(new AggregationSpec(GlobalAggregationBuilder::new, GlobalAggregationBuilder::parse, - GlobalAggregationBuilder.AGGREGATION_NAME_FIELD).addResultReader(InternalGlobal::new)); - registerAggregation( - new AggregationSpec(MissingAggregationBuilder::new, new MissingParser(), MissingAggregationBuilder.AGGREGATION_NAME_FIELD) - .addResultReader(InternalMissing::new)); - registerAggregation(new AggregationSpec(FilterAggregationBuilder::new, FilterAggregationBuilder::parse, - FilterAggregationBuilder.AGGREGATION_NAME_FIELD).addResultReader(InternalFilter::new)); - registerAggregation(new AggregationSpec(FiltersAggregationBuilder::new, FiltersAggregationBuilder::parse, - FiltersAggregationBuilder.AGGREGATION_NAME_FIELD).addResultReader(InternalFilters::new)); - registerAggregation(new AggregationSpec(SamplerAggregationBuilder::new, SamplerAggregationBuilder::parse, - SamplerAggregationBuilder.AGGREGATION_NAME_FIELD).addResultReader(InternalSampler.NAME, InternalSampler::new) - .addResultReader(UnmappedSampler.NAME, UnmappedSampler::new)); - registerAggregation(new AggregationSpec(DiversifiedAggregationBuilder::new, new DiversifiedSamplerParser(), - DiversifiedAggregationBuilder.AGGREGATION_NAME_FIELD)); - registerAggregation( - new AggregationSpec(TermsAggregationBuilder::new, new TermsParser(), TermsAggregationBuilder.AGGREGATION_NAME_FIELD) - .addResultReader(StringTerms.NAME, StringTerms::new) - .addResultReader(UnmappedTerms.NAME, UnmappedTerms::new) - .addResultReader(LongTerms.NAME, LongTerms::new) - .addResultReader(DoubleTerms.NAME, DoubleTerms::new)); - registerAggregation(new AggregationSpec(SignificantTermsAggregationBuilder::new, - new SignificantTermsParser(significanceHeuristicParserRegistry, queryParserRegistry), - SignificantTermsAggregationBuilder.AGGREGATION_NAME_FIELD) - .addResultReader(SignificantStringTerms.NAME, SignificantStringTerms::new) - .addResultReader(SignificantLongTerms.NAME, SignificantLongTerms::new) - .addResultReader(UnmappedSignificantTerms.NAME, UnmappedSignificantTerms::new)); - registerAggregation(new AggregationSpec(RangeAggregationBuilder::new, new RangeParser(), - RangeAggregationBuilder.AGGREGATION_NAME_FIELD).addResultReader(InternalRange::new)); - registerAggregation(new AggregationSpec(DateRangeAggregationBuilder::new, new DateRangeParser(), - DateRangeAggregationBuilder.AGGREGATION_NAME_FIELD).addResultReader(InternalDateRange::new)); - registerAggregation( - new AggregationSpec(IpRangeAggregationBuilder::new, new IpRangeParser(), IpRangeAggregationBuilder.AGGREGATION_NAME_FIELD) - .addResultReader(InternalBinaryRange::new)); - registerAggregation(new AggregationSpec(HistogramAggregationBuilder::new, new HistogramParser(), - HistogramAggregationBuilder.AGGREGATION_NAME_FIELD).addResultReader(InternalHistogram::new)); - registerAggregation(new AggregationSpec(DateHistogramAggregationBuilder::new, new DateHistogramParser(), - DateHistogramAggregationBuilder.AGGREGATION_NAME_FIELD).addResultReader(InternalDateHistogram::new)); - registerAggregation(new AggregationSpec(GeoDistanceAggregationBuilder::new, new GeoDistanceParser(), - GeoDistanceAggregationBuilder.AGGREGATION_NAME_FIELD).addResultReader(InternalGeoDistance::new)); - registerAggregation(new AggregationSpec(GeoGridAggregationBuilder::new, new GeoHashGridParser(), - GeoGridAggregationBuilder.AGGREGATION_NAME_FIELD).addResultReader(InternalGeoHashGrid::new)); - registerAggregation(new AggregationSpec(NestedAggregationBuilder::new, NestedAggregationBuilder::parse, - NestedAggregationBuilder.AGGREGATION_FIELD_NAME).addResultReader(InternalNested::new)); - registerAggregation(new AggregationSpec(ReverseNestedAggregationBuilder::new, ReverseNestedAggregationBuilder::parse, - ReverseNestedAggregationBuilder.AGGREGATION_NAME_FIELD).addResultReader(InternalReverseNested::new)); - registerAggregation(new AggregationSpec(TopHitsAggregationBuilder::new, TopHitsAggregationBuilder::parse, - TopHitsAggregationBuilder.AGGREGATION_NAME_FIELD).addResultReader(InternalTopHits::new)); - registerAggregation(new AggregationSpec(GeoBoundsAggregationBuilder::new, new GeoBoundsParser(), - GeoBoundsAggregationBuilder.AGGREGATION_NAME_FIED).addResultReader(InternalGeoBounds::new)); - registerAggregation(new AggregationSpec(GeoCentroidAggregationBuilder::new, new GeoCentroidParser(), - GeoCentroidAggregationBuilder.AGGREGATION_NAME_FIELD).addResultReader(InternalGeoCentroid::new)); - registerAggregation(new AggregationSpec(ScriptedMetricAggregationBuilder::new, ScriptedMetricAggregationBuilder::parse, - ScriptedMetricAggregationBuilder.AGGREGATION_NAME_FIELD).addResultReader(InternalScriptedMetric::new)); - registerAggregation(new AggregationSpec(ChildrenAggregationBuilder::new, ChildrenAggregationBuilder::parse, - ChildrenAggregationBuilder.AGGREGATION_NAME_FIELD).addResultReader(InternalChildren::new)); - + private void registerPipelineAggregations(List plugins) { registerPipelineAggregation(new PipelineAggregationSpec( + DerivativePipelineAggregationBuilder.NAME, DerivativePipelineAggregationBuilder::new, DerivativePipelineAggregator::new, - DerivativePipelineAggregationBuilder::parse, - DerivativePipelineAggregationBuilder.AGGREGATION_NAME_FIELD) + DerivativePipelineAggregationBuilder::parse) .addResultReader(InternalDerivative::new)); registerPipelineAggregation(new PipelineAggregationSpec( + MaxBucketPipelineAggregationBuilder.NAME, MaxBucketPipelineAggregationBuilder::new, MaxBucketPipelineAggregator::new, - MaxBucketPipelineAggregationBuilder.PARSER, - MaxBucketPipelineAggregationBuilder.AGGREGATION_NAME_FIELD) + MaxBucketPipelineAggregationBuilder.PARSER) // This bucket is used by many pipeline aggreations. .addResultReader(InternalBucketMetricValue.NAME, InternalBucketMetricValue::new)); registerPipelineAggregation(new PipelineAggregationSpec( + MinBucketPipelineAggregationBuilder.NAME, MinBucketPipelineAggregationBuilder::new, MinBucketPipelineAggregator::new, - MinBucketPipelineAggregationBuilder.PARSER, - MinBucketPipelineAggregationBuilder.AGGREGATION_FIELD_NAME) + MinBucketPipelineAggregationBuilder.PARSER) /* Uses InternalBucketMetricValue */); registerPipelineAggregation(new PipelineAggregationSpec( + AvgBucketPipelineAggregationBuilder.NAME, AvgBucketPipelineAggregationBuilder::new, AvgBucketPipelineAggregator::new, - AvgBucketPipelineAggregationBuilder.PARSER, - AvgBucketPipelineAggregationBuilder.AGGREGATION_NAME_FIELD) + AvgBucketPipelineAggregationBuilder.PARSER) // This bucket is used by many pipeline aggreations. .addResultReader(InternalSimpleValue.NAME, InternalSimpleValue::new)); registerPipelineAggregation(new PipelineAggregationSpec( + SumBucketPipelineAggregationBuilder.NAME, SumBucketPipelineAggregationBuilder::new, SumBucketPipelineAggregator::new, - SumBucketPipelineAggregationBuilder.PARSER, - SumBucketPipelineAggregationBuilder.AGGREGATION_NAME_FIELD) + SumBucketPipelineAggregationBuilder.PARSER) /* Uses InternalSimpleValue */); registerPipelineAggregation(new PipelineAggregationSpec( + StatsBucketPipelineAggregationBuilder.NAME, StatsBucketPipelineAggregationBuilder::new, StatsBucketPipelineAggregator::new, - StatsBucketPipelineAggregationBuilder.PARSER, - StatsBucketPipelineAggregationBuilder.AGGREGATION_NAME_FIELD) + StatsBucketPipelineAggregationBuilder.PARSER) .addResultReader(InternalStatsBucket::new)); registerPipelineAggregation(new PipelineAggregationSpec( + ExtendedStatsBucketPipelineAggregationBuilder.NAME, ExtendedStatsBucketPipelineAggregationBuilder::new, ExtendedStatsBucketPipelineAggregator::new, - new ExtendedStatsBucketParser(), - ExtendedStatsBucketPipelineAggregationBuilder.AGGREGATION_NAME_FIELD) + new ExtendedStatsBucketParser()) .addResultReader(InternalExtendedStatsBucket::new)); registerPipelineAggregation(new PipelineAggregationSpec( + PercentilesBucketPipelineAggregationBuilder.NAME, PercentilesBucketPipelineAggregationBuilder::new, PercentilesBucketPipelineAggregator::new, - PercentilesBucketPipelineAggregationBuilder.PARSER, - PercentilesBucketPipelineAggregationBuilder.AGGREGATION_NAME_FIELD) + PercentilesBucketPipelineAggregationBuilder.PARSER) .addResultReader(InternalPercentilesBucket::new)); registerPipelineAggregation(new PipelineAggregationSpec( + MovAvgPipelineAggregationBuilder.NAME, MovAvgPipelineAggregationBuilder::new, MovAvgPipelineAggregator::new, - (n, c) -> MovAvgPipelineAggregationBuilder.parse(movingAverageModelParserRegistry, n, c), - MovAvgPipelineAggregationBuilder.AGGREGATION_FIELD_NAME) + (n, c) -> MovAvgPipelineAggregationBuilder.parse(movingAverageModelParserRegistry, n, c)) /* Uses InternalHistogram for buckets */); registerPipelineAggregation(new PipelineAggregationSpec( + CumulativeSumPipelineAggregationBuilder.NAME, CumulativeSumPipelineAggregationBuilder::new, CumulativeSumPipelineAggregator::new, - CumulativeSumPipelineAggregationBuilder::parse, - CumulativeSumPipelineAggregationBuilder.AGGREGATION_NAME_FIELD)); + CumulativeSumPipelineAggregationBuilder::parse)); registerPipelineAggregation(new PipelineAggregationSpec( + BucketScriptPipelineAggregationBuilder.NAME, BucketScriptPipelineAggregationBuilder::new, BucketScriptPipelineAggregator::new, - BucketScriptPipelineAggregationBuilder::parse, - BucketScriptPipelineAggregationBuilder.AGGREGATION_NAME_FIELD)); + BucketScriptPipelineAggregationBuilder::parse)); registerPipelineAggregation(new PipelineAggregationSpec( + BucketSelectorPipelineAggregationBuilder.NAME, BucketSelectorPipelineAggregationBuilder::new, BucketSelectorPipelineAggregator::new, - BucketSelectorPipelineAggregationBuilder::parse, - BucketSelectorPipelineAggregationBuilder.AGGREGATION_NAME_FIELD)); + BucketSelectorPipelineAggregationBuilder::parse)); registerPipelineAggregation(new PipelineAggregationSpec( + SerialDiffPipelineAggregationBuilder.NAME, SerialDiffPipelineAggregationBuilder::new, SerialDiffPipelineAggregator::new, - SerialDiffPipelineAggregationBuilder::parse, - SerialDiffPipelineAggregationBuilder.AGGREGATION_NAME_FIELD)); + SerialDiffPipelineAggregationBuilder::parse)); + + registerFromPlugin(plugins, SearchPlugin::getPipelineAggregations, this::registerPipelineAggregation); + } + + private void registerPipelineAggregation(PipelineAggregationSpec spec) { + if (false == transportClient) { + pipelineAggregationParserRegistry.register(spec.getParser(), spec.getName()); + } + namedWriteables.add(new Entry(PipelineAggregationBuilder.class, spec.getName().getPreferredName(), spec.getReader())); + namedWriteables.add(new Entry(PipelineAggregator.class, spec.getName().getPreferredName(), spec.getAggregatorReader())); + for (Map.Entry> resultReader : spec.getResultReaders().entrySet()) { + namedWriteables.add(new Entry(InternalAggregation.class, resultReader.getKey(), resultReader.getValue())); + } } protected void configureSearch() { @@ -652,11 +575,6 @@ public class SearchModule extends AbstractModule { bind(SearchPhaseController.class).asEagerSingleton(); bind(FetchPhase.class).toInstance(new FetchPhase(fetchSubPhases)); bind(SearchTransportService.class).asEagerSingleton(); - if (searchServiceImpl == SearchService.class) { - bind(SearchService.class).asEagerSingleton(); - } else { - bind(SearchService.class).to(searchServiceImpl).asEagerSingleton(); - } } private void registerShapes() { diff --git a/core/src/main/java/org/elasticsearch/search/SearchRequestParsers.java b/core/src/main/java/org/elasticsearch/search/SearchRequestParsers.java new file mode 100644 index 00000000000..83eebd125d8 --- /dev/null +++ b/core/src/main/java/org/elasticsearch/search/SearchRequestParsers.java @@ -0,0 +1,64 @@ +/* + * 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.index.query.QueryParseContext; +import org.elasticsearch.indices.query.IndicesQueriesRegistry; +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: IndicesQueriesRegistry should be removed and just have the map of query parsers here + /** + * Query parsers that may be used in search requests. + * @see org.elasticsearch.index.query.QueryParseContext + * @see org.elasticsearch.search.builder.SearchSourceBuilder#fromXContent(QueryParseContext, AggregatorParsers, Suggesters) + */ + public final IndicesQueriesRegistry queryParsers; + + // 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) + */ + 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) + */ + public final Suggesters suggesters; + + public SearchRequestParsers(IndicesQueriesRegistry queryParsers, AggregatorParsers aggParsers, Suggesters suggesters) { + this.queryParsers = queryParsers; + this.aggParsers = aggParsers; + this.suggesters = suggesters; + } +} diff --git a/core/src/main/java/org/elasticsearch/search/aggregations/bucket/children/ChildrenAggregationBuilder.java b/core/src/main/java/org/elasticsearch/search/aggregations/bucket/children/ChildrenAggregationBuilder.java index 5e4b2e2aa79..a05dda207e3 100644 --- a/core/src/main/java/org/elasticsearch/search/aggregations/bucket/children/ChildrenAggregationBuilder.java +++ b/core/src/main/java/org/elasticsearch/search/aggregations/bucket/children/ChildrenAggregationBuilder.java @@ -20,7 +20,6 @@ package org.elasticsearch.search.aggregations.bucket.children; import org.apache.lucene.search.Query; -import org.elasticsearch.common.ParseField; import org.elasticsearch.common.ParsingException; import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.io.stream.StreamOutput; @@ -31,8 +30,8 @@ import org.elasticsearch.index.mapper.DocumentMapper; import org.elasticsearch.index.mapper.ParentFieldMapper; import org.elasticsearch.index.query.QueryParseContext; import org.elasticsearch.search.aggregations.AggregatorFactories.Builder; -import org.elasticsearch.search.aggregations.InternalAggregation.Type; import org.elasticsearch.search.aggregations.AggregatorFactory; +import org.elasticsearch.search.aggregations.InternalAggregation.Type; import org.elasticsearch.search.aggregations.support.AggregationContext; import org.elasticsearch.search.aggregations.support.FieldContext; import org.elasticsearch.search.aggregations.support.ValueType; @@ -48,7 +47,6 @@ import java.util.Objects; public class ChildrenAggregationBuilder extends ValuesSourceAggregationBuilder { public static final String NAME = "children"; private static final Type TYPE = new Type(NAME); - public static final ParseField AGGREGATION_NAME_FIELD = new ParseField(NAME); private String parentType; private final String childType; diff --git a/core/src/main/java/org/elasticsearch/search/aggregations/bucket/filter/FilterAggregationBuilder.java b/core/src/main/java/org/elasticsearch/search/aggregations/bucket/filter/FilterAggregationBuilder.java index 40334fc75eb..48be5365bb1 100644 --- a/core/src/main/java/org/elasticsearch/search/aggregations/bucket/filter/FilterAggregationBuilder.java +++ b/core/src/main/java/org/elasticsearch/search/aggregations/bucket/filter/FilterAggregationBuilder.java @@ -19,7 +19,6 @@ package org.elasticsearch.search.aggregations.bucket.filter; -import org.elasticsearch.common.ParseField; import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.io.stream.StreamOutput; import org.elasticsearch.common.xcontent.XContentBuilder; @@ -38,7 +37,6 @@ import java.util.Objects; public class FilterAggregationBuilder extends AbstractAggregationBuilder { public static final String NAME = "filter"; private static final Type TYPE = new Type(NAME); - public static final ParseField AGGREGATION_NAME_FIELD = new ParseField(NAME); private final QueryBuilder filter; diff --git a/core/src/main/java/org/elasticsearch/search/aggregations/bucket/filters/FiltersAggregationBuilder.java b/core/src/main/java/org/elasticsearch/search/aggregations/bucket/filters/FiltersAggregationBuilder.java index 018963a2ca7..2cd4f508ccb 100644 --- a/core/src/main/java/org/elasticsearch/search/aggregations/bucket/filters/FiltersAggregationBuilder.java +++ b/core/src/main/java/org/elasticsearch/search/aggregations/bucket/filters/FiltersAggregationBuilder.java @@ -46,7 +46,6 @@ import static org.elasticsearch.index.query.QueryBuilders.matchAllQuery; public class FiltersAggregationBuilder extends AbstractAggregationBuilder { public static final String NAME = "filters"; private static final Type TYPE = new Type(NAME); - public static final ParseField AGGREGATION_NAME_FIELD = new ParseField(NAME); private static final ParseField FILTERS_FIELD = new ParseField("filters"); private static final ParseField OTHER_BUCKET_FIELD = new ParseField("other_bucket"); diff --git a/core/src/main/java/org/elasticsearch/search/aggregations/bucket/geogrid/GeoGridAggregationBuilder.java b/core/src/main/java/org/elasticsearch/search/aggregations/bucket/geogrid/GeoGridAggregationBuilder.java index 8aa4f6ab598..24d5999e698 100644 --- a/core/src/main/java/org/elasticsearch/search/aggregations/bucket/geogrid/GeoGridAggregationBuilder.java +++ b/core/src/main/java/org/elasticsearch/search/aggregations/bucket/geogrid/GeoGridAggregationBuilder.java @@ -22,7 +22,6 @@ package org.elasticsearch.search.aggregations.bucket.geogrid; import org.apache.lucene.index.LeafReaderContext; import org.apache.lucene.index.SortedNumericDocValues; import org.elasticsearch.ElasticsearchException; -import org.elasticsearch.common.ParseField; import org.elasticsearch.common.geo.GeoHashUtils; import org.elasticsearch.common.geo.GeoPoint; import org.elasticsearch.common.io.stream.StreamInput; @@ -33,8 +32,8 @@ import org.elasticsearch.index.fielddata.SortedBinaryDocValues; import org.elasticsearch.index.fielddata.SortedNumericDoubleValues; import org.elasticsearch.index.fielddata.SortingNumericDocValues; import org.elasticsearch.search.aggregations.AggregatorFactories.Builder; -import org.elasticsearch.search.aggregations.InternalAggregation.Type; 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.AggregationContext; import org.elasticsearch.search.aggregations.support.ValueType; @@ -50,7 +49,6 @@ import java.util.Objects; public class GeoGridAggregationBuilder extends ValuesSourceAggregationBuilder { public static final String NAME = "geohash_grid"; private static final Type TYPE = new Type(NAME); - public static final ParseField AGGREGATION_NAME_FIELD = new ParseField(NAME); private int precision = GeoHashGridParser.DEFAULT_PRECISION; private int requiredSize = GeoHashGridParser.DEFAULT_MAX_NUM_CELLS; diff --git a/core/src/main/java/org/elasticsearch/search/aggregations/bucket/global/GlobalAggregationBuilder.java b/core/src/main/java/org/elasticsearch/search/aggregations/bucket/global/GlobalAggregationBuilder.java index 189eb9bebfc..94211f39dd0 100644 --- a/core/src/main/java/org/elasticsearch/search/aggregations/bucket/global/GlobalAggregationBuilder.java +++ b/core/src/main/java/org/elasticsearch/search/aggregations/bucket/global/GlobalAggregationBuilder.java @@ -19,15 +19,14 @@ package org.elasticsearch.search.aggregations.bucket.global; -import org.elasticsearch.common.ParseField; import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.io.stream.StreamOutput; import org.elasticsearch.common.xcontent.XContentBuilder; import org.elasticsearch.index.query.QueryParseContext; import org.elasticsearch.search.aggregations.AbstractAggregationBuilder; import org.elasticsearch.search.aggregations.AggregatorFactories.Builder; -import org.elasticsearch.search.aggregations.InternalAggregation.Type; import org.elasticsearch.search.aggregations.AggregatorFactory; +import org.elasticsearch.search.aggregations.InternalAggregation.Type; import org.elasticsearch.search.aggregations.support.AggregationContext; import java.io.IOException; @@ -35,7 +34,6 @@ import java.io.IOException; public class GlobalAggregationBuilder extends AbstractAggregationBuilder { public static final String NAME = "global"; private static final Type TYPE = new Type(NAME); - public static final ParseField AGGREGATION_NAME_FIELD = new ParseField(NAME); public GlobalAggregationBuilder(String name) { super(name, TYPE); diff --git a/core/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/DateHistogramAggregationBuilder.java b/core/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/DateHistogramAggregationBuilder.java index 8cfbefe1ba3..d3b8857ccab 100644 --- a/core/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/DateHistogramAggregationBuilder.java +++ b/core/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/DateHistogramAggregationBuilder.java @@ -19,7 +19,6 @@ package org.elasticsearch.search.aggregations.bucket.histogram; -import org.elasticsearch.common.ParseField; import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.io.stream.StreamOutput; import org.elasticsearch.common.unit.TimeValue; @@ -29,8 +28,8 @@ import org.elasticsearch.search.aggregations.AggregatorFactory; import org.elasticsearch.search.aggregations.support.AggregationContext; import org.elasticsearch.search.aggregations.support.ValueType; import org.elasticsearch.search.aggregations.support.ValuesSource; -import org.elasticsearch.search.aggregations.support.ValuesSourceAggregationBuilder; import org.elasticsearch.search.aggregations.support.ValuesSource.Numeric; +import org.elasticsearch.search.aggregations.support.ValuesSourceAggregationBuilder; import org.elasticsearch.search.aggregations.support.ValuesSourceAggregatorFactory; import org.elasticsearch.search.aggregations.support.ValuesSourceConfig; import org.elasticsearch.search.aggregations.support.ValuesSourceType; @@ -44,7 +43,6 @@ import java.util.Objects; public class DateHistogramAggregationBuilder extends ValuesSourceAggregationBuilder { public static final String NAME = InternalDateHistogram.TYPE.name(); - public static final ParseField AGGREGATION_NAME_FIELD = new ParseField(NAME); private long interval; private DateHistogramInterval dateHistogramInterval; diff --git a/core/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/HistogramAggregationBuilder.java b/core/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/HistogramAggregationBuilder.java index 1226284529c..03ff1ee935a 100644 --- a/core/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/HistogramAggregationBuilder.java +++ b/core/src/main/java/org/elasticsearch/search/aggregations/bucket/histogram/HistogramAggregationBuilder.java @@ -19,7 +19,6 @@ package org.elasticsearch.search.aggregations.bucket.histogram; -import org.elasticsearch.common.ParseField; import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.io.stream.StreamOutput; import org.elasticsearch.common.xcontent.XContentBuilder; @@ -28,8 +27,8 @@ import org.elasticsearch.search.aggregations.AggregatorFactory; import org.elasticsearch.search.aggregations.support.AggregationContext; import org.elasticsearch.search.aggregations.support.ValueType; import org.elasticsearch.search.aggregations.support.ValuesSource; -import org.elasticsearch.search.aggregations.support.ValuesSourceAggregationBuilder; import org.elasticsearch.search.aggregations.support.ValuesSource.Numeric; +import org.elasticsearch.search.aggregations.support.ValuesSourceAggregationBuilder; import org.elasticsearch.search.aggregations.support.ValuesSourceAggregatorFactory; import org.elasticsearch.search.aggregations.support.ValuesSourceConfig; import org.elasticsearch.search.aggregations.support.ValuesSourceType; @@ -43,7 +42,6 @@ import java.util.Objects; public class HistogramAggregationBuilder extends ValuesSourceAggregationBuilder { public static final String NAME = InternalHistogram.TYPE.name(); - public static final ParseField AGGREGATION_NAME_FIELD = new ParseField(NAME); private double interval; private double offset = 0; diff --git a/core/src/main/java/org/elasticsearch/search/aggregations/bucket/missing/MissingAggregationBuilder.java b/core/src/main/java/org/elasticsearch/search/aggregations/bucket/missing/MissingAggregationBuilder.java index 05868f96df2..09916acfdf8 100644 --- a/core/src/main/java/org/elasticsearch/search/aggregations/bucket/missing/MissingAggregationBuilder.java +++ b/core/src/main/java/org/elasticsearch/search/aggregations/bucket/missing/MissingAggregationBuilder.java @@ -19,13 +19,12 @@ package org.elasticsearch.search.aggregations.bucket.missing; -import org.elasticsearch.common.ParseField; 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.AggregatorFactories.Builder; -import org.elasticsearch.search.aggregations.InternalAggregation.Type; import org.elasticsearch.search.aggregations.AggregatorFactory; +import org.elasticsearch.search.aggregations.InternalAggregation.Type; import org.elasticsearch.search.aggregations.support.AggregationContext; import org.elasticsearch.search.aggregations.support.ValueType; import org.elasticsearch.search.aggregations.support.ValuesSource; @@ -39,7 +38,6 @@ import java.io.IOException; public class MissingAggregationBuilder extends ValuesSourceAggregationBuilder { public static final String NAME = "missing"; public static final Type TYPE = new Type(NAME); - public static final ParseField AGGREGATION_NAME_FIELD = new ParseField(NAME); public MissingAggregationBuilder(String name, ValueType targetValueType) { super(name, TYPE, ValuesSourceType.ANY, targetValueType); diff --git a/core/src/main/java/org/elasticsearch/search/aggregations/bucket/nested/NestedAggregationBuilder.java b/core/src/main/java/org/elasticsearch/search/aggregations/bucket/nested/NestedAggregationBuilder.java index e3dd04b62fb..c2bf201e48f 100644 --- a/core/src/main/java/org/elasticsearch/search/aggregations/bucket/nested/NestedAggregationBuilder.java +++ b/core/src/main/java/org/elasticsearch/search/aggregations/bucket/nested/NestedAggregationBuilder.java @@ -19,7 +19,6 @@ package org.elasticsearch.search.aggregations.bucket.nested; -import org.elasticsearch.common.ParseField; import org.elasticsearch.common.ParsingException; import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.io.stream.StreamOutput; @@ -30,8 +29,8 @@ import org.elasticsearch.index.query.QueryParseContext; import org.elasticsearch.search.aggregations.AbstractAggregationBuilder; import org.elasticsearch.search.aggregations.AggregationExecutionException; import org.elasticsearch.search.aggregations.AggregatorFactories.Builder; -import org.elasticsearch.search.aggregations.InternalAggregation.Type; import org.elasticsearch.search.aggregations.AggregatorFactory; +import org.elasticsearch.search.aggregations.InternalAggregation.Type; import org.elasticsearch.search.aggregations.support.AggregationContext; import java.io.IOException; @@ -40,7 +39,6 @@ import java.util.Objects; public class NestedAggregationBuilder extends AbstractAggregationBuilder { public static final String NAME = "nested"; private static final Type TYPE = new Type(NAME); - public static final ParseField AGGREGATION_FIELD_NAME = new ParseField(NAME); private final String path; diff --git a/core/src/main/java/org/elasticsearch/search/aggregations/bucket/nested/ReverseNestedAggregationBuilder.java b/core/src/main/java/org/elasticsearch/search/aggregations/bucket/nested/ReverseNestedAggregationBuilder.java index 999ad0fc32f..707d4633845 100644 --- a/core/src/main/java/org/elasticsearch/search/aggregations/bucket/nested/ReverseNestedAggregationBuilder.java +++ b/core/src/main/java/org/elasticsearch/search/aggregations/bucket/nested/ReverseNestedAggregationBuilder.java @@ -19,7 +19,6 @@ package org.elasticsearch.search.aggregations.bucket.nested; -import org.elasticsearch.common.ParseField; import org.elasticsearch.common.ParsingException; import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.io.stream.StreamOutput; @@ -32,19 +31,16 @@ import org.elasticsearch.search.SearchParseException; import org.elasticsearch.search.aggregations.AbstractAggregationBuilder; import org.elasticsearch.search.aggregations.AggregationExecutionException; import org.elasticsearch.search.aggregations.AggregatorFactories.Builder; -import org.elasticsearch.search.aggregations.InternalAggregation.Type; import org.elasticsearch.search.aggregations.AggregatorFactory; +import org.elasticsearch.search.aggregations.InternalAggregation.Type; import org.elasticsearch.search.aggregations.support.AggregationContext; import java.io.IOException; -import java.util.Deque; -import java.util.LinkedList; import java.util.Objects; public class ReverseNestedAggregationBuilder extends AbstractAggregationBuilder { public static final String NAME = "reverse_nested"; private static final Type TYPE = new Type(NAME); - public static final ParseField AGGREGATION_NAME_FIELD = new ParseField(NAME); private String path; diff --git a/core/src/main/java/org/elasticsearch/search/aggregations/bucket/range/RangeAggregationBuilder.java b/core/src/main/java/org/elasticsearch/search/aggregations/bucket/range/RangeAggregationBuilder.java index ccf5f5f56cc..c815ae9d3cf 100644 --- a/core/src/main/java/org/elasticsearch/search/aggregations/bucket/range/RangeAggregationBuilder.java +++ b/core/src/main/java/org/elasticsearch/search/aggregations/bucket/range/RangeAggregationBuilder.java @@ -19,7 +19,6 @@ package org.elasticsearch.search.aggregations.bucket.range; -import org.elasticsearch.common.ParseField; import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.search.aggregations.AggregatorFactories.Builder; import org.elasticsearch.search.aggregations.AggregatorFactory; @@ -34,7 +33,6 @@ import java.io.IOException; public class RangeAggregationBuilder extends AbstractRangeBuilder { public static final String NAME = "range"; static final Type TYPE = new Type(NAME); - public static final ParseField AGGREGATION_NAME_FIELD = new ParseField(NAME); public RangeAggregationBuilder(String name) { super(name, InternalRange.FACTORY); diff --git a/core/src/main/java/org/elasticsearch/search/aggregations/bucket/range/date/DateRangeAggregationBuilder.java b/core/src/main/java/org/elasticsearch/search/aggregations/bucket/range/date/DateRangeAggregationBuilder.java index ad0f890aeaf..a75b071569c 100644 --- a/core/src/main/java/org/elasticsearch/search/aggregations/bucket/range/date/DateRangeAggregationBuilder.java +++ b/core/src/main/java/org/elasticsearch/search/aggregations/bucket/range/date/DateRangeAggregationBuilder.java @@ -19,7 +19,6 @@ package org.elasticsearch.search.aggregations.bucket.range.date; -import org.elasticsearch.common.ParseField; import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.search.aggregations.AggregatorFactories.Builder; import org.elasticsearch.search.aggregations.AggregatorFactory; @@ -37,7 +36,6 @@ import java.io.IOException; public class DateRangeAggregationBuilder extends AbstractRangeBuilder { public static final String NAME = "date_range"; static final Type TYPE = new Type(NAME); - public static final ParseField AGGREGATION_NAME_FIELD = new ParseField(NAME); public DateRangeAggregationBuilder(String name) { super(name, InternalDateRange.FACTORY); diff --git a/core/src/main/java/org/elasticsearch/search/aggregations/bucket/range/geodistance/GeoDistanceAggregationBuilder.java b/core/src/main/java/org/elasticsearch/search/aggregations/bucket/range/geodistance/GeoDistanceAggregationBuilder.java index a60899a590d..4a4cab2affa 100644 --- a/core/src/main/java/org/elasticsearch/search/aggregations/bucket/range/geodistance/GeoDistanceAggregationBuilder.java +++ b/core/src/main/java/org/elasticsearch/search/aggregations/bucket/range/geodistance/GeoDistanceAggregationBuilder.java @@ -19,7 +19,6 @@ package org.elasticsearch.search.aggregations.bucket.range.geodistance; -import org.elasticsearch.common.ParseField; import org.elasticsearch.common.geo.GeoDistance; import org.elasticsearch.common.geo.GeoPoint; import org.elasticsearch.common.io.stream.StreamInput; @@ -27,8 +26,8 @@ import org.elasticsearch.common.io.stream.StreamOutput; import org.elasticsearch.common.unit.DistanceUnit; import org.elasticsearch.common.xcontent.XContentBuilder; import org.elasticsearch.search.aggregations.AggregatorFactories.Builder; -import org.elasticsearch.search.aggregations.InternalAggregation.Type; 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.geodistance.GeoDistanceParser.Range; @@ -46,7 +45,6 @@ import java.util.Objects; public class GeoDistanceAggregationBuilder extends ValuesSourceAggregationBuilder { public static final String NAME = "geo_distance"; public static final Type TYPE = new Type(NAME); - public static final ParseField AGGREGATION_NAME_FIELD = new ParseField(NAME); private final GeoPoint origin; private List ranges = new ArrayList<>(); diff --git a/core/src/main/java/org/elasticsearch/search/aggregations/bucket/range/ip/IpRangeAggregationBuilder.java b/core/src/main/java/org/elasticsearch/search/aggregations/bucket/range/ip/IpRangeAggregationBuilder.java index 11b7e90de09..bd2353b5099 100644 --- a/core/src/main/java/org/elasticsearch/search/aggregations/bucket/range/ip/IpRangeAggregationBuilder.java +++ b/core/src/main/java/org/elasticsearch/search/aggregations/bucket/range/ip/IpRangeAggregationBuilder.java @@ -18,17 +18,8 @@ */ package org.elasticsearch.search.aggregations.bucket.range.ip; -import java.io.IOException; -import java.net.InetAddress; -import java.net.UnknownHostException; -import java.util.ArrayList; -import java.util.Collections; -import java.util.List; -import java.util.Objects; - import org.apache.lucene.document.InetAddressPoint; import org.apache.lucene.util.BytesRef; -import org.elasticsearch.common.ParseField; import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.io.stream.StreamOutput; import org.elasticsearch.common.network.InetAddresses; @@ -49,11 +40,18 @@ import org.elasticsearch.search.aggregations.support.ValuesSourceAggregatorFacto import org.elasticsearch.search.aggregations.support.ValuesSourceConfig; import org.elasticsearch.search.aggregations.support.ValuesSourceType; +import java.io.IOException; +import java.net.InetAddress; +import java.net.UnknownHostException; +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; +import java.util.Objects; + public final class IpRangeAggregationBuilder extends ValuesSourceAggregationBuilder { public static final String NAME = "ip_range"; - public static final ParseField AGGREGATION_NAME_FIELD = new ParseField(NAME); private static final InternalAggregation.Type TYPE = new InternalAggregation.Type(NAME); public static class Range implements ToXContent { diff --git a/core/src/main/java/org/elasticsearch/search/aggregations/bucket/sampler/DiversifiedAggregationBuilder.java b/core/src/main/java/org/elasticsearch/search/aggregations/bucket/sampler/DiversifiedAggregationBuilder.java index 804574eea10..8a83fabcab2 100644 --- a/core/src/main/java/org/elasticsearch/search/aggregations/bucket/sampler/DiversifiedAggregationBuilder.java +++ b/core/src/main/java/org/elasticsearch/search/aggregations/bucket/sampler/DiversifiedAggregationBuilder.java @@ -19,7 +19,6 @@ package org.elasticsearch.search.aggregations.bucket.sampler; -import org.elasticsearch.common.ParseField; import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.io.stream.StreamOutput; import org.elasticsearch.common.xcontent.XContentBuilder; @@ -38,7 +37,6 @@ import java.util.Objects; public class DiversifiedAggregationBuilder extends ValuesSourceAggregationBuilder { public static final String NAME = "diversified_sampler"; - public static final ParseField AGGREGATION_NAME_FIELD = new ParseField(NAME); public static final Type TYPE = new Type(NAME); public static final int MAX_DOCS_PER_VALUE_DEFAULT = 1; diff --git a/core/src/main/java/org/elasticsearch/search/aggregations/bucket/sampler/SamplerAggregationBuilder.java b/core/src/main/java/org/elasticsearch/search/aggregations/bucket/sampler/SamplerAggregationBuilder.java index 1f34d4987e5..2230408b033 100644 --- a/core/src/main/java/org/elasticsearch/search/aggregations/bucket/sampler/SamplerAggregationBuilder.java +++ b/core/src/main/java/org/elasticsearch/search/aggregations/bucket/sampler/SamplerAggregationBuilder.java @@ -19,7 +19,6 @@ package org.elasticsearch.search.aggregations.bucket.sampler; -import org.elasticsearch.common.ParseField; import org.elasticsearch.common.ParsingException; import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.io.stream.StreamOutput; @@ -38,7 +37,6 @@ import java.util.Objects; public class SamplerAggregationBuilder extends AbstractAggregationBuilder { public static final String NAME = "sampler"; private static final Type TYPE = new Type(NAME); - public static final ParseField AGGREGATION_NAME_FIELD = new ParseField(NAME); public static final int DEFAULT_SHARD_SAMPLE_SIZE = 100; diff --git a/core/src/main/java/org/elasticsearch/search/aggregations/bucket/significant/SignificantTermsAggregationBuilder.java b/core/src/main/java/org/elasticsearch/search/aggregations/bucket/significant/SignificantTermsAggregationBuilder.java index cdc3fcab8ce..245297d72cc 100644 --- a/core/src/main/java/org/elasticsearch/search/aggregations/bucket/significant/SignificantTermsAggregationBuilder.java +++ b/core/src/main/java/org/elasticsearch/search/aggregations/bucket/significant/SignificantTermsAggregationBuilder.java @@ -24,14 +24,14 @@ import org.elasticsearch.common.io.stream.StreamOutput; import org.elasticsearch.common.xcontent.XContentBuilder; import org.elasticsearch.index.query.QueryBuilder; import org.elasticsearch.search.aggregations.AggregatorFactories.Builder; -import org.elasticsearch.search.aggregations.InternalAggregation.Type; 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.terms.TermsAggregationBuilder; import org.elasticsearch.search.aggregations.bucket.terms.TermsAggregator; import org.elasticsearch.search.aggregations.bucket.terms.TermsAggregator.BucketCountThresholds; -import org.elasticsearch.search.aggregations.bucket.terms.TermsAggregationBuilder; import org.elasticsearch.search.aggregations.bucket.terms.support.IncludeExclude; import org.elasticsearch.search.aggregations.support.AggregationContext; import org.elasticsearch.search.aggregations.support.ValueType; @@ -50,7 +50,6 @@ import java.util.Objects; public class SignificantTermsAggregationBuilder extends ValuesSourceAggregationBuilder { public static final String NAME = "significant_terms"; public static final InternalAggregation.Type TYPE = new Type(NAME); - public static final ParseField AGGREGATION_NAME_FIELD = new ParseField(NAME); static final ParseField BACKGROUND_FILTER = new ParseField("background_filter"); static final ParseField HEURISTIC = new ParseField("significance_heuristic"); diff --git a/core/src/main/java/org/elasticsearch/search/aggregations/bucket/terms/TermsAggregationBuilder.java b/core/src/main/java/org/elasticsearch/search/aggregations/bucket/terms/TermsAggregationBuilder.java index d1a260bc157..e34c3bcbed4 100644 --- a/core/src/main/java/org/elasticsearch/search/aggregations/bucket/terms/TermsAggregationBuilder.java +++ b/core/src/main/java/org/elasticsearch/search/aggregations/bucket/terms/TermsAggregationBuilder.java @@ -22,10 +22,10 @@ import org.elasticsearch.common.ParseField; 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.AggregatorFactory; -import org.elasticsearch.search.aggregations.InternalAggregation; import org.elasticsearch.search.aggregations.Aggregator.SubAggCollectionMode; 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.terms.TermsAggregator.BucketCountThresholds; import org.elasticsearch.search.aggregations.bucket.terms.support.IncludeExclude; @@ -36,6 +36,7 @@ import org.elasticsearch.search.aggregations.support.ValuesSourceAggregationBuil import org.elasticsearch.search.aggregations.support.ValuesSourceAggregatorFactory; import org.elasticsearch.search.aggregations.support.ValuesSourceConfig; import org.elasticsearch.search.aggregations.support.ValuesSourceType; + import java.io.IOException; import java.util.List; import java.util.Objects; @@ -43,7 +44,6 @@ import java.util.Objects; public class TermsAggregationBuilder extends ValuesSourceAggregationBuilder { public static final String NAME = "terms"; private static final InternalAggregation.Type TYPE = new Type("terms"); - public static final ParseField AGGREGATION_NAME_FIELD = new ParseField(NAME); public static final ParseField EXECUTION_HINT_FIELD_NAME = new ParseField("execution_hint"); public static final ParseField SHARD_SIZE_FIELD_NAME = new ParseField("shard_size"); diff --git a/core/src/main/java/org/elasticsearch/search/aggregations/metrics/avg/AvgAggregationBuilder.java b/core/src/main/java/org/elasticsearch/search/aggregations/metrics/avg/AvgAggregationBuilder.java index 287b437f136..37d67887822 100644 --- a/core/src/main/java/org/elasticsearch/search/aggregations/metrics/avg/AvgAggregationBuilder.java +++ b/core/src/main/java/org/elasticsearch/search/aggregations/metrics/avg/AvgAggregationBuilder.java @@ -19,13 +19,12 @@ package org.elasticsearch.search.aggregations.metrics.avg; -import org.elasticsearch.common.ParseField; 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.AggregatorFactories.Builder; -import org.elasticsearch.search.aggregations.InternalAggregation.Type; import org.elasticsearch.search.aggregations.AggregatorFactory; +import org.elasticsearch.search.aggregations.InternalAggregation.Type; import org.elasticsearch.search.aggregations.support.AggregationContext; import org.elasticsearch.search.aggregations.support.ValueType; import org.elasticsearch.search.aggregations.support.ValuesSource; @@ -39,7 +38,6 @@ import java.io.IOException; public class AvgAggregationBuilder extends ValuesSourceAggregationBuilder.LeafOnly { public static final String NAME = "avg"; private static final Type TYPE = new Type(NAME); - public static final ParseField AGGREGATION_NAME_FIELD = new ParseField(NAME); public AvgAggregationBuilder(String name) { super(name, TYPE, ValuesSourceType.NUMERIC, ValueType.NUMERIC); diff --git a/core/src/main/java/org/elasticsearch/search/aggregations/metrics/cardinality/CardinalityAggregationBuilder.java b/core/src/main/java/org/elasticsearch/search/aggregations/metrics/cardinality/CardinalityAggregationBuilder.java index 2d573c48e8e..d545cfce23e 100644 --- a/core/src/main/java/org/elasticsearch/search/aggregations/metrics/cardinality/CardinalityAggregationBuilder.java +++ b/core/src/main/java/org/elasticsearch/search/aggregations/metrics/cardinality/CardinalityAggregationBuilder.java @@ -24,8 +24,8 @@ 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.AggregatorFactories.Builder; -import org.elasticsearch.search.aggregations.InternalAggregation.Type; import org.elasticsearch.search.aggregations.AggregatorFactory; +import org.elasticsearch.search.aggregations.InternalAggregation.Type; import org.elasticsearch.search.aggregations.support.AggregationContext; import org.elasticsearch.search.aggregations.support.ValueType; import org.elasticsearch.search.aggregations.support.ValuesSource; @@ -41,7 +41,6 @@ public final class CardinalityAggregationBuilder public static final String NAME = "cardinality"; private static final Type TYPE = new Type(NAME); - public static final ParseField AGGREGATION_NAME_FIELD = new ParseField(NAME); public static final ParseField PRECISION_THRESHOLD_FIELD = new ParseField("precision_threshold"); diff --git a/core/src/main/java/org/elasticsearch/search/aggregations/metrics/geobounds/GeoBoundsAggregationBuilder.java b/core/src/main/java/org/elasticsearch/search/aggregations/metrics/geobounds/GeoBoundsAggregationBuilder.java index 833787738e5..c3565d794a8 100644 --- a/core/src/main/java/org/elasticsearch/search/aggregations/metrics/geobounds/GeoBoundsAggregationBuilder.java +++ b/core/src/main/java/org/elasticsearch/search/aggregations/metrics/geobounds/GeoBoundsAggregationBuilder.java @@ -19,13 +19,12 @@ package org.elasticsearch.search.aggregations.metrics.geobounds; -import org.elasticsearch.common.ParseField; 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.AggregatorFactories.Builder; -import org.elasticsearch.search.aggregations.InternalAggregation.Type; import org.elasticsearch.search.aggregations.AggregatorFactory; +import org.elasticsearch.search.aggregations.InternalAggregation.Type; import org.elasticsearch.search.aggregations.support.AggregationContext; import org.elasticsearch.search.aggregations.support.ValueType; import org.elasticsearch.search.aggregations.support.ValuesSource; @@ -39,7 +38,6 @@ import java.util.Objects; public class GeoBoundsAggregationBuilder extends ValuesSourceAggregationBuilder { public static final String NAME = "geo_bounds"; private static final Type TYPE = new Type(NAME); - public static final ParseField AGGREGATION_NAME_FIED = new ParseField(NAME); private boolean wrapLongitude = true; diff --git a/core/src/main/java/org/elasticsearch/search/aggregations/metrics/geocentroid/GeoCentroidAggregationBuilder.java b/core/src/main/java/org/elasticsearch/search/aggregations/metrics/geocentroid/GeoCentroidAggregationBuilder.java index 3e61f34eef6..3c5208f8738 100644 --- a/core/src/main/java/org/elasticsearch/search/aggregations/metrics/geocentroid/GeoCentroidAggregationBuilder.java +++ b/core/src/main/java/org/elasticsearch/search/aggregations/metrics/geocentroid/GeoCentroidAggregationBuilder.java @@ -19,13 +19,12 @@ package org.elasticsearch.search.aggregations.metrics.geocentroid; -import org.elasticsearch.common.ParseField; 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.AggregatorFactories.Builder; -import org.elasticsearch.search.aggregations.InternalAggregation.Type; import org.elasticsearch.search.aggregations.AggregatorFactory; +import org.elasticsearch.search.aggregations.InternalAggregation.Type; import org.elasticsearch.search.aggregations.support.AggregationContext; import org.elasticsearch.search.aggregations.support.ValueType; import org.elasticsearch.search.aggregations.support.ValuesSource; @@ -39,7 +38,6 @@ public class GeoCentroidAggregationBuilder extends ValuesSourceAggregationBuilder.LeafOnly { public static final String NAME = "geo_centroid"; public static final Type TYPE = new Type(NAME); - public static final ParseField AGGREGATION_NAME_FIELD = new ParseField(NAME); public GeoCentroidAggregationBuilder(String name) { super(name, TYPE, ValuesSourceType.GEOPOINT, ValueType.GEOPOINT); diff --git a/core/src/main/java/org/elasticsearch/search/aggregations/metrics/max/MaxAggregationBuilder.java b/core/src/main/java/org/elasticsearch/search/aggregations/metrics/max/MaxAggregationBuilder.java index cb45e73db41..117b360992a 100644 --- a/core/src/main/java/org/elasticsearch/search/aggregations/metrics/max/MaxAggregationBuilder.java +++ b/core/src/main/java/org/elasticsearch/search/aggregations/metrics/max/MaxAggregationBuilder.java @@ -19,13 +19,12 @@ package org.elasticsearch.search.aggregations.metrics.max; -import org.elasticsearch.common.ParseField; 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.AggregatorFactories.Builder; -import org.elasticsearch.search.aggregations.InternalAggregation.Type; import org.elasticsearch.search.aggregations.AggregatorFactory; +import org.elasticsearch.search.aggregations.InternalAggregation.Type; import org.elasticsearch.search.aggregations.support.AggregationContext; import org.elasticsearch.search.aggregations.support.ValueType; import org.elasticsearch.search.aggregations.support.ValuesSource; @@ -39,7 +38,6 @@ import java.io.IOException; public class MaxAggregationBuilder extends ValuesSourceAggregationBuilder.LeafOnly { public static final String NAME = "max"; public static final Type TYPE = new Type(NAME); - public static final ParseField AGGREGATION_NAME_FIELD = new ParseField(NAME); public MaxAggregationBuilder(String name) { super(name, TYPE, ValuesSourceType.NUMERIC, ValueType.NUMERIC); diff --git a/core/src/main/java/org/elasticsearch/search/aggregations/metrics/min/MinAggregationBuilder.java b/core/src/main/java/org/elasticsearch/search/aggregations/metrics/min/MinAggregationBuilder.java index 10edbe9d3eb..248b83db0c9 100644 --- a/core/src/main/java/org/elasticsearch/search/aggregations/metrics/min/MinAggregationBuilder.java +++ b/core/src/main/java/org/elasticsearch/search/aggregations/metrics/min/MinAggregationBuilder.java @@ -19,13 +19,12 @@ package org.elasticsearch.search.aggregations.metrics.min; -import org.elasticsearch.common.ParseField; 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.AggregatorFactories.Builder; -import org.elasticsearch.search.aggregations.InternalAggregation.Type; import org.elasticsearch.search.aggregations.AggregatorFactory; +import org.elasticsearch.search.aggregations.InternalAggregation.Type; import org.elasticsearch.search.aggregations.support.AggregationContext; import org.elasticsearch.search.aggregations.support.ValueType; import org.elasticsearch.search.aggregations.support.ValuesSource; @@ -39,7 +38,6 @@ import java.io.IOException; public class MinAggregationBuilder extends ValuesSourceAggregationBuilder.LeafOnly { public static final String NAME = "min"; private static final Type TYPE = new Type(NAME); - public static final ParseField AGGREGATION_NAME_FIELD = new ParseField(NAME); public MinAggregationBuilder(String name) { super(name, TYPE, ValuesSourceType.NUMERIC, ValueType.NUMERIC); diff --git a/core/src/main/java/org/elasticsearch/search/aggregations/metrics/percentiles/PercentileRanksAggregationBuilder.java b/core/src/main/java/org/elasticsearch/search/aggregations/metrics/percentiles/PercentileRanksAggregationBuilder.java index 576406ad2c1..8320774da0f 100644 --- a/core/src/main/java/org/elasticsearch/search/aggregations/metrics/percentiles/PercentileRanksAggregationBuilder.java +++ b/core/src/main/java/org/elasticsearch/search/aggregations/metrics/percentiles/PercentileRanksAggregationBuilder.java @@ -19,7 +19,6 @@ package org.elasticsearch.search.aggregations.metrics.percentiles; -import org.elasticsearch.common.ParseField; import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.io.stream.StreamOutput; import org.elasticsearch.common.xcontent.XContentBuilder; @@ -44,7 +43,6 @@ import java.util.Objects; public class PercentileRanksAggregationBuilder extends LeafOnly { public static final String NAME = PercentileRanks.TYPE_NAME; public static final Type TYPE = new Type(NAME); - public static final ParseField AGGREGATION_NAME_FIELD = new ParseField(NAME); private double[] values; private PercentilesMethod method = PercentilesMethod.TDIGEST; diff --git a/core/src/main/java/org/elasticsearch/search/aggregations/metrics/percentiles/PercentilesAggregationBuilder.java b/core/src/main/java/org/elasticsearch/search/aggregations/metrics/percentiles/PercentilesAggregationBuilder.java index ca7a877c373..f3fd9ad744b 100644 --- a/core/src/main/java/org/elasticsearch/search/aggregations/metrics/percentiles/PercentilesAggregationBuilder.java +++ b/core/src/main/java/org/elasticsearch/search/aggregations/metrics/percentiles/PercentilesAggregationBuilder.java @@ -19,7 +19,6 @@ package org.elasticsearch.search.aggregations.metrics.percentiles; -import org.elasticsearch.common.ParseField; import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.io.stream.StreamOutput; import org.elasticsearch.common.xcontent.XContentBuilder; @@ -44,7 +43,6 @@ import java.util.Objects; public class PercentilesAggregationBuilder extends LeafOnly { public static final String NAME = Percentiles.TYPE_NAME; public static final Type TYPE = new Type(NAME); - public static final ParseField AGGREGATION_NAME_FIELD = new ParseField(NAME); private double[] percents = PercentilesParser.DEFAULT_PERCENTS; private PercentilesMethod method = PercentilesMethod.TDIGEST; diff --git a/core/src/main/java/org/elasticsearch/search/aggregations/metrics/scripted/ScriptedMetricAggregationBuilder.java b/core/src/main/java/org/elasticsearch/search/aggregations/metrics/scripted/ScriptedMetricAggregationBuilder.java index fd692e300aa..ec0b2aef613 100644 --- a/core/src/main/java/org/elasticsearch/search/aggregations/metrics/scripted/ScriptedMetricAggregationBuilder.java +++ b/core/src/main/java/org/elasticsearch/search/aggregations/metrics/scripted/ScriptedMetricAggregationBuilder.java @@ -43,7 +43,6 @@ public class ScriptedMetricAggregationBuilder extends AbstractAggregationBuilder public static final String NAME = "scripted_metric"; private static final Type TYPE = new Type(NAME); - public static final ParseField AGGREGATION_NAME_FIELD = new ParseField(NAME); private static final ParseField INIT_SCRIPT_FIELD = new ParseField("init_script"); private static final ParseField MAP_SCRIPT_FIELD = new ParseField("map_script"); diff --git a/core/src/main/java/org/elasticsearch/search/aggregations/metrics/stats/StatsAggregationBuilder.java b/core/src/main/java/org/elasticsearch/search/aggregations/metrics/stats/StatsAggregationBuilder.java index e1ecc8d4f7b..e26edbcc290 100644 --- a/core/src/main/java/org/elasticsearch/search/aggregations/metrics/stats/StatsAggregationBuilder.java +++ b/core/src/main/java/org/elasticsearch/search/aggregations/metrics/stats/StatsAggregationBuilder.java @@ -19,13 +19,12 @@ package org.elasticsearch.search.aggregations.metrics.stats; -import org.elasticsearch.common.ParseField; 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.AggregatorFactories.Builder; -import org.elasticsearch.search.aggregations.InternalAggregation.Type; import org.elasticsearch.search.aggregations.AggregatorFactory; +import org.elasticsearch.search.aggregations.InternalAggregation.Type; import org.elasticsearch.search.aggregations.support.AggregationContext; import org.elasticsearch.search.aggregations.support.ValueType; import org.elasticsearch.search.aggregations.support.ValuesSource; @@ -39,7 +38,6 @@ import java.io.IOException; public class StatsAggregationBuilder extends ValuesSourceAggregationBuilder.LeafOnly { public static final String NAME = "stats"; private static final Type TYPE = new Type(NAME); - public static final ParseField AGGREGATION_NAME_FIELD = new ParseField(NAME); public StatsAggregationBuilder(String name) { super(name, TYPE, ValuesSourceType.NUMERIC, ValueType.NUMERIC); diff --git a/core/src/main/java/org/elasticsearch/search/aggregations/metrics/stats/extended/ExtendedStatsAggregationBuilder.java b/core/src/main/java/org/elasticsearch/search/aggregations/metrics/stats/extended/ExtendedStatsAggregationBuilder.java index ca90fe17f5e..a20b8fa6764 100644 --- a/core/src/main/java/org/elasticsearch/search/aggregations/metrics/stats/extended/ExtendedStatsAggregationBuilder.java +++ b/core/src/main/java/org/elasticsearch/search/aggregations/metrics/stats/extended/ExtendedStatsAggregationBuilder.java @@ -19,13 +19,12 @@ package org.elasticsearch.search.aggregations.metrics.stats.extended; -import org.elasticsearch.common.ParseField; 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.AggregatorFactories.Builder; -import org.elasticsearch.search.aggregations.InternalAggregation.Type; import org.elasticsearch.search.aggregations.AggregatorFactory; +import org.elasticsearch.search.aggregations.InternalAggregation.Type; import org.elasticsearch.search.aggregations.support.AggregationContext; import org.elasticsearch.search.aggregations.support.ValueType; import org.elasticsearch.search.aggregations.support.ValuesSource; @@ -41,7 +40,6 @@ public class ExtendedStatsAggregationBuilder extends ValuesSourceAggregationBuilder.LeafOnly { public static final String NAME = "extended_stats"; public static final Type TYPE = new Type(NAME); - public static final ParseField AGGREGATION_NAME_FIELD = new ParseField(NAME); private double sigma = 2.0; diff --git a/core/src/main/java/org/elasticsearch/search/aggregations/metrics/sum/SumAggregationBuilder.java b/core/src/main/java/org/elasticsearch/search/aggregations/metrics/sum/SumAggregationBuilder.java index 04422ed8137..2f285b146c1 100644 --- a/core/src/main/java/org/elasticsearch/search/aggregations/metrics/sum/SumAggregationBuilder.java +++ b/core/src/main/java/org/elasticsearch/search/aggregations/metrics/sum/SumAggregationBuilder.java @@ -19,13 +19,12 @@ package org.elasticsearch.search.aggregations.metrics.sum; -import org.elasticsearch.common.ParseField; 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.AggregatorFactories.Builder; -import org.elasticsearch.search.aggregations.InternalAggregation.Type; import org.elasticsearch.search.aggregations.AggregatorFactory; +import org.elasticsearch.search.aggregations.InternalAggregation.Type; import org.elasticsearch.search.aggregations.support.AggregationContext; import org.elasticsearch.search.aggregations.support.ValueType; import org.elasticsearch.search.aggregations.support.ValuesSource; @@ -39,7 +38,6 @@ import java.io.IOException; public class SumAggregationBuilder extends ValuesSourceAggregationBuilder.LeafOnly { public static final String NAME = "sum"; private static final Type TYPE = new Type(NAME); - public static final ParseField AGGREGATION_NAME_FIELD = new ParseField(NAME); public SumAggregationBuilder(String name) { super(name, TYPE, ValuesSourceType.NUMERIC, ValueType.NUMERIC); diff --git a/core/src/main/java/org/elasticsearch/search/aggregations/metrics/tophits/TopHitsAggregationBuilder.java b/core/src/main/java/org/elasticsearch/search/aggregations/metrics/tophits/TopHitsAggregationBuilder.java index b95f4154cb7..c72f0d3eb97 100644 --- a/core/src/main/java/org/elasticsearch/search/aggregations/metrics/tophits/TopHitsAggregationBuilder.java +++ b/core/src/main/java/org/elasticsearch/search/aggregations/metrics/tophits/TopHitsAggregationBuilder.java @@ -20,7 +20,6 @@ package org.elasticsearch.search.aggregations.metrics.tophits; import org.elasticsearch.common.Nullable; -import org.elasticsearch.common.ParseField; import org.elasticsearch.common.ParsingException; import org.elasticsearch.common.Strings; import org.elasticsearch.common.io.stream.StreamInput; @@ -32,9 +31,9 @@ import org.elasticsearch.script.Script; import org.elasticsearch.search.aggregations.AbstractAggregationBuilder; import org.elasticsearch.search.aggregations.AggregationInitializationException; import org.elasticsearch.search.aggregations.AggregatorFactories.Builder; -import org.elasticsearch.search.aggregations.InternalAggregation.Type; import org.elasticsearch.search.aggregations.AggregatorFactory; import org.elasticsearch.search.aggregations.InternalAggregation; +import org.elasticsearch.search.aggregations.InternalAggregation.Type; import org.elasticsearch.search.aggregations.support.AggregationContext; import org.elasticsearch.search.builder.SearchSourceBuilder; import org.elasticsearch.search.builder.SearchSourceBuilder.ScriptField; @@ -56,7 +55,6 @@ import java.util.Set; public class TopHitsAggregationBuilder extends AbstractAggregationBuilder { public static final String NAME = "top_hits"; private static final InternalAggregation.Type TYPE = new Type(NAME); - public static final ParseField AGGREGATION_NAME_FIELD = new ParseField(NAME); private int from = 0; private int size = 3; diff --git a/core/src/main/java/org/elasticsearch/search/aggregations/metrics/valuecount/ValueCountAggregationBuilder.java b/core/src/main/java/org/elasticsearch/search/aggregations/metrics/valuecount/ValueCountAggregationBuilder.java index 067d16db9f8..ce0c1fd3e5b 100644 --- a/core/src/main/java/org/elasticsearch/search/aggregations/metrics/valuecount/ValueCountAggregationBuilder.java +++ b/core/src/main/java/org/elasticsearch/search/aggregations/metrics/valuecount/ValueCountAggregationBuilder.java @@ -19,7 +19,6 @@ package org.elasticsearch.search.aggregations.metrics.valuecount; -import org.elasticsearch.common.ParseField; import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.io.stream.StreamOutput; import org.elasticsearch.common.xcontent.XContentBuilder; @@ -38,7 +37,6 @@ import java.io.IOException; public class ValueCountAggregationBuilder extends ValuesSourceAggregationBuilder.LeafOnly { public static final String NAME = "value_count"; public static final Type TYPE = new Type(NAME); - public static final ParseField AGGREGATION_NAME_FIELD = new ParseField(NAME); public ValueCountAggregationBuilder(String name, ValueType targetValueType) { super(name, TYPE, ValuesSourceType.ANY, targetValueType); diff --git a/core/src/main/java/org/elasticsearch/search/aggregations/pipeline/bucketmetrics/avg/AvgBucketPipelineAggregationBuilder.java b/core/src/main/java/org/elasticsearch/search/aggregations/pipeline/bucketmetrics/avg/AvgBucketPipelineAggregationBuilder.java index 95a62d388a4..28280edb872 100644 --- a/core/src/main/java/org/elasticsearch/search/aggregations/pipeline/bucketmetrics/avg/AvgBucketPipelineAggregationBuilder.java +++ b/core/src/main/java/org/elasticsearch/search/aggregations/pipeline/bucketmetrics/avg/AvgBucketPipelineAggregationBuilder.java @@ -19,7 +19,6 @@ package org.elasticsearch.search.aggregations.pipeline.bucketmetrics.avg; -import org.elasticsearch.common.ParseField; import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.io.stream.StreamOutput; import org.elasticsearch.common.xcontent.XContentBuilder; @@ -35,7 +34,6 @@ import java.util.Map; public class AvgBucketPipelineAggregationBuilder extends BucketMetricsPipelineAggregationBuilder { public static final String NAME = "avg_bucket"; - public static final ParseField AGGREGATION_NAME_FIELD = new ParseField(NAME); public AvgBucketPipelineAggregationBuilder(String name, String bucketsPath) { super(name, NAME, new String[] { bucketsPath }); diff --git a/core/src/main/java/org/elasticsearch/search/aggregations/pipeline/bucketmetrics/max/MaxBucketPipelineAggregationBuilder.java b/core/src/main/java/org/elasticsearch/search/aggregations/pipeline/bucketmetrics/max/MaxBucketPipelineAggregationBuilder.java index 7f517a6dc1b..6044fdea14d 100644 --- a/core/src/main/java/org/elasticsearch/search/aggregations/pipeline/bucketmetrics/max/MaxBucketPipelineAggregationBuilder.java +++ b/core/src/main/java/org/elasticsearch/search/aggregations/pipeline/bucketmetrics/max/MaxBucketPipelineAggregationBuilder.java @@ -19,7 +19,6 @@ package org.elasticsearch.search.aggregations.pipeline.bucketmetrics.max; -import org.elasticsearch.common.ParseField; import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.io.stream.StreamOutput; import org.elasticsearch.common.xcontent.XContentBuilder; @@ -35,7 +34,6 @@ import java.util.Map; public class MaxBucketPipelineAggregationBuilder extends BucketMetricsPipelineAggregationBuilder { public static final String NAME = "max_bucket"; - public static final ParseField AGGREGATION_NAME_FIELD = new ParseField(NAME); public MaxBucketPipelineAggregationBuilder(String name, String bucketsPath) { super(name, NAME, new String[] { bucketsPath }); diff --git a/core/src/main/java/org/elasticsearch/search/aggregations/pipeline/bucketmetrics/min/MinBucketPipelineAggregationBuilder.java b/core/src/main/java/org/elasticsearch/search/aggregations/pipeline/bucketmetrics/min/MinBucketPipelineAggregationBuilder.java index 5c45c737074..f6533a65dba 100644 --- a/core/src/main/java/org/elasticsearch/search/aggregations/pipeline/bucketmetrics/min/MinBucketPipelineAggregationBuilder.java +++ b/core/src/main/java/org/elasticsearch/search/aggregations/pipeline/bucketmetrics/min/MinBucketPipelineAggregationBuilder.java @@ -19,7 +19,6 @@ package org.elasticsearch.search.aggregations.pipeline.bucketmetrics.min; -import org.elasticsearch.common.ParseField; import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.io.stream.StreamOutput; import org.elasticsearch.common.xcontent.XContentBuilder; @@ -35,7 +34,6 @@ import java.util.Map; public class MinBucketPipelineAggregationBuilder extends BucketMetricsPipelineAggregationBuilder { public static final String NAME = "min_bucket"; - public static final ParseField AGGREGATION_FIELD_NAME = new ParseField(NAME); public MinBucketPipelineAggregationBuilder(String name, String bucketsPath) { super(name, NAME, new String[] { bucketsPath }); diff --git a/core/src/main/java/org/elasticsearch/search/aggregations/pipeline/bucketmetrics/percentile/PercentilesBucketPipelineAggregationBuilder.java b/core/src/main/java/org/elasticsearch/search/aggregations/pipeline/bucketmetrics/percentile/PercentilesBucketPipelineAggregationBuilder.java index d0f0e29bdbd..435d0239cbe 100644 --- a/core/src/main/java/org/elasticsearch/search/aggregations/pipeline/bucketmetrics/percentile/PercentilesBucketPipelineAggregationBuilder.java +++ b/core/src/main/java/org/elasticsearch/search/aggregations/pipeline/bucketmetrics/percentile/PercentilesBucketPipelineAggregationBuilder.java @@ -20,6 +20,7 @@ package org.elasticsearch.search.aggregations.pipeline.bucketmetrics.percentile; import com.carrotsearch.hppc.DoubleArrayList; + import org.elasticsearch.common.ParseField; import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.io.stream.StreamOutput; @@ -41,7 +42,6 @@ import java.util.Objects; public class PercentilesBucketPipelineAggregationBuilder extends BucketMetricsPipelineAggregationBuilder { public static final String NAME = "percentiles_bucket"; - public static final ParseField AGGREGATION_NAME_FIELD = new ParseField(NAME); private static final ParseField PERCENTS_FIELD = new ParseField("percents"); diff --git a/core/src/main/java/org/elasticsearch/search/aggregations/pipeline/bucketmetrics/stats/StatsBucketPipelineAggregationBuilder.java b/core/src/main/java/org/elasticsearch/search/aggregations/pipeline/bucketmetrics/stats/StatsBucketPipelineAggregationBuilder.java index 9e0432b2883..ef0852f30af 100644 --- a/core/src/main/java/org/elasticsearch/search/aggregations/pipeline/bucketmetrics/stats/StatsBucketPipelineAggregationBuilder.java +++ b/core/src/main/java/org/elasticsearch/search/aggregations/pipeline/bucketmetrics/stats/StatsBucketPipelineAggregationBuilder.java @@ -19,7 +19,6 @@ package org.elasticsearch.search.aggregations.pipeline.bucketmetrics.stats; -import org.elasticsearch.common.ParseField; import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.io.stream.StreamOutput; import org.elasticsearch.common.xcontent.XContentBuilder; @@ -36,7 +35,6 @@ import java.util.Map; public class StatsBucketPipelineAggregationBuilder extends BucketMetricsPipelineAggregationBuilder { public static final String NAME = "stats_bucket"; - public static final ParseField AGGREGATION_NAME_FIELD = new ParseField(NAME); public StatsBucketPipelineAggregationBuilder(String name, String bucketsPath) { super(name, NAME, new String[] { bucketsPath }); diff --git a/core/src/main/java/org/elasticsearch/search/aggregations/pipeline/bucketmetrics/stats/extended/ExtendedStatsBucketPipelineAggregationBuilder.java b/core/src/main/java/org/elasticsearch/search/aggregations/pipeline/bucketmetrics/stats/extended/ExtendedStatsBucketPipelineAggregationBuilder.java index 82920be6cf1..fb18c918b0a 100644 --- a/core/src/main/java/org/elasticsearch/search/aggregations/pipeline/bucketmetrics/stats/extended/ExtendedStatsBucketPipelineAggregationBuilder.java +++ b/core/src/main/java/org/elasticsearch/search/aggregations/pipeline/bucketmetrics/stats/extended/ExtendedStatsBucketPipelineAggregationBuilder.java @@ -19,7 +19,6 @@ package org.elasticsearch.search.aggregations.pipeline.bucketmetrics.stats.extended; -import org.elasticsearch.common.ParseField; import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.io.stream.StreamOutput; import org.elasticsearch.common.xcontent.XContentBuilder; @@ -37,7 +36,6 @@ import java.util.Objects; public class ExtendedStatsBucketPipelineAggregationBuilder extends BucketMetricsPipelineAggregationBuilder { public static final String NAME = "extended_stats_bucket"; - public static final ParseField AGGREGATION_NAME_FIELD = new ParseField(NAME); private double sigma = 2.0; diff --git a/core/src/main/java/org/elasticsearch/search/aggregations/pipeline/bucketmetrics/sum/SumBucketPipelineAggregationBuilder.java b/core/src/main/java/org/elasticsearch/search/aggregations/pipeline/bucketmetrics/sum/SumBucketPipelineAggregationBuilder.java index 669700cbf71..604cba05056 100644 --- a/core/src/main/java/org/elasticsearch/search/aggregations/pipeline/bucketmetrics/sum/SumBucketPipelineAggregationBuilder.java +++ b/core/src/main/java/org/elasticsearch/search/aggregations/pipeline/bucketmetrics/sum/SumBucketPipelineAggregationBuilder.java @@ -19,7 +19,6 @@ package org.elasticsearch.search.aggregations.pipeline.bucketmetrics.sum; -import org.elasticsearch.common.ParseField; import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.io.stream.StreamOutput; import org.elasticsearch.common.xcontent.XContentBuilder; @@ -35,7 +34,6 @@ import java.util.Map; public class SumBucketPipelineAggregationBuilder extends BucketMetricsPipelineAggregationBuilder { public static final String NAME = "sum_bucket"; - public static final ParseField AGGREGATION_NAME_FIELD = new ParseField(NAME); public SumBucketPipelineAggregationBuilder(String name, String bucketsPath) { super(name, NAME, new String[] { bucketsPath }); diff --git a/core/src/main/java/org/elasticsearch/search/aggregations/pipeline/bucketscript/BucketScriptPipelineAggregationBuilder.java b/core/src/main/java/org/elasticsearch/search/aggregations/pipeline/bucketscript/BucketScriptPipelineAggregationBuilder.java index 4925de9c774..cee17076e5d 100644 --- a/core/src/main/java/org/elasticsearch/search/aggregations/pipeline/bucketscript/BucketScriptPipelineAggregationBuilder.java +++ b/core/src/main/java/org/elasticsearch/search/aggregations/pipeline/bucketscript/BucketScriptPipelineAggregationBuilder.java @@ -19,7 +19,6 @@ package org.elasticsearch.search.aggregations.pipeline.bucketscript; -import org.elasticsearch.common.ParseField; import org.elasticsearch.common.ParsingException; import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.io.stream.StreamOutput; @@ -48,7 +47,6 @@ import static org.elasticsearch.search.aggregations.pipeline.PipelineAggregator. public class BucketScriptPipelineAggregationBuilder extends AbstractPipelineAggregationBuilder { public static final String NAME = "bucket_script"; - public static final ParseField AGGREGATION_NAME_FIELD = new ParseField(NAME); private final Script script; private final Map bucketsPathsMap; diff --git a/core/src/main/java/org/elasticsearch/search/aggregations/pipeline/bucketselector/BucketSelectorPipelineAggregationBuilder.java b/core/src/main/java/org/elasticsearch/search/aggregations/pipeline/bucketselector/BucketSelectorPipelineAggregationBuilder.java index c9b423eedc3..97cf02d69a1 100644 --- a/core/src/main/java/org/elasticsearch/search/aggregations/pipeline/bucketselector/BucketSelectorPipelineAggregationBuilder.java +++ b/core/src/main/java/org/elasticsearch/search/aggregations/pipeline/bucketselector/BucketSelectorPipelineAggregationBuilder.java @@ -19,7 +19,6 @@ package org.elasticsearch.search.aggregations.pipeline.bucketselector; -import org.elasticsearch.common.ParseField; import org.elasticsearch.common.ParsingException; import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.io.stream.StreamOutput; @@ -46,7 +45,6 @@ import static org.elasticsearch.search.aggregations.pipeline.PipelineAggregator. public class BucketSelectorPipelineAggregationBuilder extends AbstractPipelineAggregationBuilder { public static final String NAME = "bucket_selector"; - public static final ParseField AGGREGATION_NAME_FIELD = new ParseField(NAME); private final Map bucketsPathsMap; private Script script; diff --git a/core/src/main/java/org/elasticsearch/search/aggregations/pipeline/cumulativesum/CumulativeSumPipelineAggregationBuilder.java b/core/src/main/java/org/elasticsearch/search/aggregations/pipeline/cumulativesum/CumulativeSumPipelineAggregationBuilder.java index 7bc54832612..a83787f1365 100644 --- a/core/src/main/java/org/elasticsearch/search/aggregations/pipeline/cumulativesum/CumulativeSumPipelineAggregationBuilder.java +++ b/core/src/main/java/org/elasticsearch/search/aggregations/pipeline/cumulativesum/CumulativeSumPipelineAggregationBuilder.java @@ -19,7 +19,6 @@ package org.elasticsearch.search.aggregations.pipeline.cumulativesum; -import org.elasticsearch.common.ParseField; import org.elasticsearch.common.ParsingException; import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.io.stream.StreamOutput; @@ -46,7 +45,6 @@ import static org.elasticsearch.search.aggregations.pipeline.PipelineAggregator. public class CumulativeSumPipelineAggregationBuilder extends AbstractPipelineAggregationBuilder { public static final String NAME = "cumulative_sum"; - public static final ParseField AGGREGATION_NAME_FIELD = new ParseField(NAME); private String format; diff --git a/core/src/main/java/org/elasticsearch/search/aggregations/pipeline/derivative/DerivativePipelineAggregationBuilder.java b/core/src/main/java/org/elasticsearch/search/aggregations/pipeline/derivative/DerivativePipelineAggregationBuilder.java index e7b6b89d042..5ffc77669b8 100644 --- a/core/src/main/java/org/elasticsearch/search/aggregations/pipeline/derivative/DerivativePipelineAggregationBuilder.java +++ b/core/src/main/java/org/elasticsearch/search/aggregations/pipeline/derivative/DerivativePipelineAggregationBuilder.java @@ -47,7 +47,6 @@ import java.util.Objects; public class DerivativePipelineAggregationBuilder extends AbstractPipelineAggregationBuilder { public static final String NAME = "derivative"; - public static final ParseField AGGREGATION_NAME_FIELD = new ParseField(NAME); private static final ParseField FORMAT_FIELD = new ParseField("format"); private static final ParseField GAP_POLICY_FIELD = new ParseField("gap_policy"); diff --git a/core/src/main/java/org/elasticsearch/search/aggregations/pipeline/movavg/MovAvgPipelineAggregationBuilder.java b/core/src/main/java/org/elasticsearch/search/aggregations/pipeline/movavg/MovAvgPipelineAggregationBuilder.java index 89bc721e74a..f0aa1f81126 100644 --- a/core/src/main/java/org/elasticsearch/search/aggregations/pipeline/movavg/MovAvgPipelineAggregationBuilder.java +++ b/core/src/main/java/org/elasticsearch/search/aggregations/pipeline/movavg/MovAvgPipelineAggregationBuilder.java @@ -52,7 +52,6 @@ import static org.elasticsearch.search.aggregations.pipeline.PipelineAggregator. public class MovAvgPipelineAggregationBuilder extends AbstractPipelineAggregationBuilder { public static final String NAME = "moving_avg"; - public static final ParseField AGGREGATION_FIELD_NAME = new ParseField(NAME); public static final ParseField MODEL = new ParseField("model"); private static final ParseField WINDOW = new ParseField("window"); diff --git a/core/src/test/java/org/elasticsearch/action/admin/cluster/node/tasks/TransportTasksActionTests.java b/core/src/test/java/org/elasticsearch/action/admin/cluster/node/tasks/TransportTasksActionTests.java index 17ca7e7268b..f964eeba9f7 100644 --- a/core/src/test/java/org/elasticsearch/action/admin/cluster/node/tasks/TransportTasksActionTests.java +++ b/core/src/test/java/org/elasticsearch/action/admin/cluster/node/tasks/TransportTasksActionTests.java @@ -47,6 +47,7 @@ import org.elasticsearch.common.xcontent.XContentBuilder; import org.elasticsearch.common.xcontent.XContentFactory; import org.elasticsearch.common.xcontent.XContentHelper; import org.elasticsearch.common.xcontent.XContentType; +import org.elasticsearch.rest.action.admin.cluster.RestListTasksAction; import org.elasticsearch.tasks.Task; import org.elasticsearch.tasks.TaskId; import org.elasticsearch.tasks.TaskInfo; @@ -65,6 +66,7 @@ import java.util.concurrent.CountDownLatch; import java.util.concurrent.ExecutionException; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicReference; +import java.util.function.Consumer; import static org.elasticsearch.action.support.PlainActionFuture.newFuture; import static org.hamcrest.Matchers.containsString; @@ -736,7 +738,7 @@ public class TransportTasksActionTests extends TaskManagerTestCase { ListTasksResponse response = testNodes[0].transportListTasksAction.execute(listTasksRequest).get(); assertEquals(testNodes.length + 1, response.getTasks().size()); - Map byNodes = serialize(response, new ToXContent.MapParams(Collections.singletonMap("group_by", "nodes"))); + Map byNodes = serialize(response, true); byNodes = (Map) byNodes.get("nodes"); // One element on the top level assertEquals(testNodes.length, byNodes.size()); @@ -750,7 +752,7 @@ public class TransportTasksActionTests extends TaskManagerTestCase { } // Group by parents - Map byParent = serialize(response, new ToXContent.MapParams(Collections.singletonMap("group_by", "parents"))); + Map byParent = serialize(response, false); byParent = (Map) byParent.get("tasks"); // One element on the top level assertEquals(1, byParent.size()); // Only one top level task @@ -763,10 +765,15 @@ public class TransportTasksActionTests extends TaskManagerTestCase { } } - private Map serialize(ToXContent response, ToXContent.Params params) throws IOException { + private Map serialize(ListTasksResponse response, boolean byParents) throws IOException { XContentBuilder builder = XContentFactory.contentBuilder(XContentType.JSON); builder.startObject(); - response.toXContent(builder, params); + if (byParents) { + DiscoveryNodes nodes = testNodes[0].clusterService.state().nodes(); + response.toXContentGroupedByNode(builder, ToXContent.EMPTY_PARAMS, nodes); + } else { + response.toXContentGroupedByParents(builder, ToXContent.EMPTY_PARAMS); + } builder.endObject(); builder.flush(); logger.info(builder.string()); diff --git a/core/src/test/java/org/elasticsearch/action/search/MultiSearchRequestTests.java b/core/src/test/java/org/elasticsearch/action/search/MultiSearchRequestTests.java index 2eab8674bdd..3c9acb4104b 100644 --- a/core/src/test/java/org/elasticsearch/action/search/MultiSearchRequestTests.java +++ b/core/src/test/java/org/elasticsearch/action/search/MultiSearchRequestTests.java @@ -31,6 +31,7 @@ import org.elasticsearch.index.query.QueryParser; import org.elasticsearch.indices.query.IndicesQueriesRegistry; import org.elasticsearch.rest.RestRequest; import org.elasticsearch.rest.action.search.RestMultiSearchAction; +import org.elasticsearch.search.SearchRequestParsers; import org.elasticsearch.test.ESTestCase; import org.elasticsearch.test.StreamsUtils; import org.elasticsearch.test.rest.FakeRestRequest; @@ -167,13 +168,13 @@ public class MultiSearchRequestTests extends ESTestCase { private MultiSearchRequest parseMultiSearchRequest(String sample) throws IOException { byte[] data = StreamsUtils.copyToBytesFromClasspath(sample); RestRequest restRequest = new FakeRestRequest.Builder().withContent(new BytesArray(data)).build(); - return RestMultiSearchAction.parseRequest(restRequest, true, registry(), ParseFieldMatcher.EMPTY, null, null); + return RestMultiSearchAction.parseRequest(restRequest, true, parsers(), ParseFieldMatcher.EMPTY); } - private IndicesQueriesRegistry registry() { + private SearchRequestParsers parsers() { IndicesQueriesRegistry registry = new IndicesQueriesRegistry(); QueryParser parser = MatchAllQueryBuilder::fromXContent; registry.register(parser, MatchAllQueryBuilder.NAME); - return registry; + return new SearchRequestParsers(registry, null, null); } } diff --git a/core/src/test/java/org/elasticsearch/cluster/ClusterModuleTests.java b/core/src/test/java/org/elasticsearch/cluster/ClusterModuleTests.java index 514968d1643..98404a22754 100644 --- a/core/src/test/java/org/elasticsearch/cluster/ClusterModuleTests.java +++ b/core/src/test/java/org/elasticsearch/cluster/ClusterModuleTests.java @@ -51,8 +51,8 @@ public class ClusterModuleTests extends ModuleTestCase { static class FakeShardsAllocator implements ShardsAllocator { @Override - public boolean allocate(RoutingAllocation allocation) { - return false; + public void allocate(RoutingAllocation allocation) { + // noop } @Override diff --git a/core/src/test/java/org/elasticsearch/cluster/routing/PrimaryAllocationIT.java b/core/src/test/java/org/elasticsearch/cluster/routing/PrimaryAllocationIT.java index 10c71e47d24..9077f9923ea 100644 --- a/core/src/test/java/org/elasticsearch/cluster/routing/PrimaryAllocationIT.java +++ b/core/src/test/java/org/elasticsearch/cluster/routing/PrimaryAllocationIT.java @@ -24,6 +24,7 @@ import org.elasticsearch.action.admin.cluster.reroute.ClusterRerouteRequestBuild import org.elasticsearch.action.admin.indices.shards.IndicesShardStoresResponse; import org.elasticsearch.action.support.ActiveShardCount; import org.elasticsearch.cluster.ClusterState; +import org.elasticsearch.cluster.metadata.IndexMetaData; import org.elasticsearch.cluster.routing.allocation.command.AllocateEmptyPrimaryAllocationCommand; import org.elasticsearch.cluster.routing.allocation.command.AllocateStalePrimaryAllocationCommand; import org.elasticsearch.common.collect.ImmutableOpenIntMap; @@ -219,4 +220,32 @@ public class PrimaryAllocationIT extends ESIntegTestCase { ensureYellow("test"); assertHitCount(client().prepareSearch().setSize(0).setQuery(matchAllQuery()).get(), 1L); } + + /** + * This test ensures that for an unassigned primary shard that has a valid shard copy on at least one node, + * we will force allocate the primary shard to one of those nodes, even if the allocation deciders all return + * a NO decision to allocate. + */ + public void testForceAllocatePrimaryOnNoDecision() throws Exception { + logger.info("--> starting 1 node"); + final String node = internalCluster().startNodeAsync().get(); + logger.info("--> creating index with 1 primary and 0 replicas"); + final String indexName = "test-idx"; + assertAcked(client().admin().indices() + .prepareCreate(indexName) + .setSettings(Settings.builder().put(IndexMetaData.INDEX_NUMBER_OF_SHARDS_SETTING.getKey(), 1) + .put(IndexMetaData.INDEX_NUMBER_OF_REPLICAS_SETTING.getKey(), 0)) + .get()); + logger.info("--> update the settings to prevent allocation to the data node"); + assertTrue(client().admin().indices().prepareUpdateSettings(indexName) + .setSettings(Settings.builder().put(IndexMetaData.INDEX_ROUTING_EXCLUDE_GROUP_SETTING.getKey() + "_name", node)) + .get() + .isAcknowledged()); + logger.info("--> full cluster restart"); + internalCluster().fullRestart(); + logger.info("--> checking that the primary shard is force allocated to the data node despite being blocked by the exclude filter"); + ensureGreen(indexName); + assertEquals(1, client().admin().cluster().prepareState().get().getState() + .routingTable().index(indexName).shardsWithState(ShardRoutingState.STARTED).size()); + } } diff --git a/core/src/test/java/org/elasticsearch/cluster/routing/allocation/BalanceConfigurationTests.java b/core/src/test/java/org/elasticsearch/cluster/routing/allocation/BalanceConfigurationTests.java index 46376ac3afa..12b37a32156 100644 --- a/core/src/test/java/org/elasticsearch/cluster/routing/allocation/BalanceConfigurationTests.java +++ b/core/src/test/java/org/elasticsearch/cluster/routing/allocation/BalanceConfigurationTests.java @@ -326,52 +326,50 @@ public class BalanceConfigurationTests extends ESAllocationTestCase { --------[test][3], node[3], [P], s[STARTED] ---- unassigned */ - public boolean allocate(RoutingAllocation allocation) { + public void allocate(RoutingAllocation allocation) { RoutingNodes.UnassignedShards unassigned = allocation.routingNodes().unassigned(); - boolean changed = !unassigned.isEmpty(); ShardRouting[] drain = unassigned.drain(); ArrayUtil.timSort(drain, (a, b) -> { return a.primary() ? -1 : 1; }); // we have to allocate primaries first for (ShardRouting sr : drain) { switch (sr.id()) { case 0: if (sr.primary()) { - allocation.routingNodes().initializeShard(sr, "node1", null, -1); + allocation.routingNodes().initializeShard(sr, "node1", null, -1, allocation.changes()); } else { - allocation.routingNodes().initializeShard(sr, "node0", null, -1); + allocation.routingNodes().initializeShard(sr, "node0", null, -1, allocation.changes()); } break; case 1: if (sr.primary()) { - allocation.routingNodes().initializeShard(sr, "node1", null, -1); + allocation.routingNodes().initializeShard(sr, "node1", null, -1, allocation.changes()); } else { - allocation.routingNodes().initializeShard(sr, "node2", null, -1); + allocation.routingNodes().initializeShard(sr, "node2", null, -1, allocation.changes()); } break; case 2: if (sr.primary()) { - allocation.routingNodes().initializeShard(sr, "node3", null, -1); + allocation.routingNodes().initializeShard(sr, "node3", null, -1, allocation.changes()); } else { - allocation.routingNodes().initializeShard(sr, "node2", null, -1); + allocation.routingNodes().initializeShard(sr, "node2", null, -1, allocation.changes()); } break; case 3: if (sr.primary()) { - allocation.routingNodes().initializeShard(sr, "node3", null, -1); + allocation.routingNodes().initializeShard(sr, "node3", null, -1, allocation.changes()); } else { - allocation.routingNodes().initializeShard(sr, "node1", null, -1); + allocation.routingNodes().initializeShard(sr, "node1", null, -1, allocation.changes()); } break; case 4: if (sr.primary()) { - allocation.routingNodes().initializeShard(sr, "node2", null, -1); + allocation.routingNodes().initializeShard(sr, "node2", null, -1, allocation.changes()); } else { - allocation.routingNodes().initializeShard(sr, "node0", null, -1); + allocation.routingNodes().initializeShard(sr, "node0", null, -1, allocation.changes()); } break; } } - return changed; } }, EmptyClusterInfoService.INSTANCE); MetaData.Builder metaDataBuilder = MetaData.builder(); diff --git a/core/src/test/java/org/elasticsearch/cluster/routing/allocation/ClusterRebalanceRoutingTests.java b/core/src/test/java/org/elasticsearch/cluster/routing/allocation/ClusterRebalanceRoutingTests.java index e52738fcc4d..a4227ea7e4b 100644 --- a/core/src/test/java/org/elasticsearch/cluster/routing/allocation/ClusterRebalanceRoutingTests.java +++ b/core/src/test/java/org/elasticsearch/cluster/routing/allocation/ClusterRebalanceRoutingTests.java @@ -636,19 +636,19 @@ public class ClusterRebalanceRoutingTests extends ESAllocationTestCase { AllocationService strategy = createAllocationService(Settings.EMPTY, new NoopGatewayAllocator() { @Override - public boolean allocateUnassigned(RoutingAllocation allocation) { + public void allocateUnassigned(RoutingAllocation allocation) { if (allocateTest1.get() == false) { RoutingNodes.UnassignedShards unassigned = allocation.routingNodes().unassigned(); RoutingNodes.UnassignedShards.UnassignedIterator iterator = unassigned.iterator(); while (iterator.hasNext()) { ShardRouting next = iterator.next(); if ("test1".equals(next.index().getName())) { - iterator.removeAndIgnore(UnassignedInfo.AllocationStatus.NO_ATTEMPT); + iterator.removeAndIgnore(UnassignedInfo.AllocationStatus.NO_ATTEMPT, allocation.changes()); } } } - return super.allocateUnassigned(allocation); + super.allocateUnassigned(allocation); } }); @@ -742,11 +742,11 @@ public class ClusterRebalanceRoutingTests extends ESAllocationTestCase { AllocationService strategy = createAllocationService(Settings.builder().put(ClusterRebalanceAllocationDecider.CLUSTER_ROUTING_ALLOCATION_ALLOW_REBALANCE_SETTING.getKey(), ClusterRebalanceAllocationDecider.ClusterRebalanceType.ALWAYS.toString()).build(), new NoopGatewayAllocator() { @Override - public boolean allocateUnassigned(RoutingAllocation allocation) { + public void allocateUnassigned(RoutingAllocation allocation) { if (hasFetches.get()) { allocation.setHasPendingAsyncFetch(); } - return super.allocateUnassigned(allocation); + super.allocateUnassigned(allocation); } }); diff --git a/core/src/test/java/org/elasticsearch/cluster/routing/allocation/MaxRetryAllocationDeciderTests.java b/core/src/test/java/org/elasticsearch/cluster/routing/allocation/MaxRetryAllocationDeciderTests.java index 133da3c005f..df4e155032e 100644 --- a/core/src/test/java/org/elasticsearch/cluster/routing/allocation/MaxRetryAllocationDeciderTests.java +++ b/core/src/test/java/org/elasticsearch/cluster/routing/allocation/MaxRetryAllocationDeciderTests.java @@ -27,9 +27,11 @@ import org.elasticsearch.cluster.metadata.IndexMetaData; import org.elasticsearch.cluster.metadata.MetaData; import org.elasticsearch.cluster.node.DiscoveryNodes; import org.elasticsearch.cluster.routing.RoutingTable; +import org.elasticsearch.cluster.routing.ShardRouting; import org.elasticsearch.cluster.routing.allocation.allocator.BalancedShardsAllocator; import org.elasticsearch.cluster.routing.allocation.command.AllocationCommands; import org.elasticsearch.cluster.routing.allocation.decider.AllocationDeciders; +import org.elasticsearch.cluster.routing.allocation.decider.Decision; import org.elasticsearch.cluster.routing.allocation.decider.MaxRetryAllocationDecider; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.test.ESAllocationTestCase; @@ -147,9 +149,13 @@ public class MaxRetryAllocationDeciderTests extends ESAllocationTestCase { routingTable = result.routingTable(); clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build(); assertEquals(routingTable.index("idx").shards().size(), 1); - assertEquals(routingTable.index("idx").shard(0).shards().get(0).state(), INITIALIZING); - assertEquals(routingTable.index("idx").shard(0).shards().get(0).unassignedInfo().getNumFailedAllocations(), i+1); - assertEquals(routingTable.index("idx").shard(0).shards().get(0).unassignedInfo().getMessage(), "boom" + i); + ShardRouting unassignedPrimary = routingTable.index("idx").shard(0).shards().get(0); + assertEquals(unassignedPrimary.state(), INITIALIZING); + assertEquals(unassignedPrimary.unassignedInfo().getNumFailedAllocations(), i+1); + assertEquals(unassignedPrimary.unassignedInfo().getMessage(), "boom" + i); + // MaxRetryAllocationDecider#canForceAllocatePrimary should return YES decisions because canAllocate returns YES here + assertEquals(Decision.YES, new MaxRetryAllocationDecider(Settings.EMPTY).canForceAllocatePrimary( + unassignedPrimary, null, new RoutingAllocation(null, null, clusterState, null, 0, false))); } // now we go and check that we are actually stick to unassigned on the next failure { @@ -161,9 +167,13 @@ public class MaxRetryAllocationDeciderTests extends ESAllocationTestCase { routingTable = result.routingTable(); clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build(); assertEquals(routingTable.index("idx").shards().size(), 1); - assertEquals(routingTable.index("idx").shard(0).shards().get(0).unassignedInfo().getNumFailedAllocations(), retries); - assertEquals(routingTable.index("idx").shard(0).shards().get(0).state(), UNASSIGNED); - assertEquals(routingTable.index("idx").shard(0).shards().get(0).unassignedInfo().getMessage(), "boom"); + ShardRouting unassignedPrimary = routingTable.index("idx").shard(0).shards().get(0); + assertEquals(unassignedPrimary.unassignedInfo().getNumFailedAllocations(), retries); + assertEquals(unassignedPrimary.state(), UNASSIGNED); + assertEquals(unassignedPrimary.unassignedInfo().getMessage(), "boom"); + // MaxRetryAllocationDecider#canForceAllocatePrimary should return a NO decision because canAllocate returns NO here + assertEquals(Decision.NO, new MaxRetryAllocationDecider(Settings.EMPTY).canForceAllocatePrimary( + unassignedPrimary, null, new RoutingAllocation(null, null, clusterState, null, 0, false))); } // change the settings and ensure we can do another round of allocation for that index. @@ -179,13 +189,17 @@ public class MaxRetryAllocationDeciderTests extends ESAllocationTestCase { clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build(); // good we are initializing and we are maintaining failure information assertEquals(routingTable.index("idx").shards().size(), 1); - assertEquals(routingTable.index("idx").shard(0).shards().get(0).unassignedInfo().getNumFailedAllocations(), retries); - assertEquals(routingTable.index("idx").shard(0).shards().get(0).state(), INITIALIZING); - assertEquals(routingTable.index("idx").shard(0).shards().get(0).unassignedInfo().getMessage(), "boom"); + ShardRouting unassignedPrimary = routingTable.index("idx").shard(0).shards().get(0); + assertEquals(unassignedPrimary.unassignedInfo().getNumFailedAllocations(), retries); + assertEquals(unassignedPrimary.state(), INITIALIZING); + assertEquals(unassignedPrimary.unassignedInfo().getMessage(), "boom"); + // bumped up the max retry count, so canForceAllocatePrimary should return a YES decision + assertEquals(Decision.YES, new MaxRetryAllocationDecider(Settings.EMPTY).canForceAllocatePrimary( + routingTable.index("idx").shard(0).shards().get(0), null, new RoutingAllocation(null, null, clusterState, null, 0, false))); // now we start the shard - routingTable = strategy.applyStartedShards(clusterState, Collections.singletonList(routingTable.index("idx") - .shard(0).shards().get(0))).routingTable(); + routingTable = strategy.applyStartedShards(clusterState, Collections.singletonList( + routingTable.index("idx").shard(0).shards().get(0))).routingTable(); clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build(); // all counters have been reset to 0 ie. no unassigned info @@ -202,8 +216,13 @@ public class MaxRetryAllocationDeciderTests extends ESAllocationTestCase { routingTable = result.routingTable(); clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build(); assertEquals(routingTable.index("idx").shards().size(), 1); - assertEquals(routingTable.index("idx").shard(0).shards().get(0).unassignedInfo().getNumFailedAllocations(), 1); - assertEquals(routingTable.index("idx").shard(0).shards().get(0).state(), INITIALIZING); - assertEquals(routingTable.index("idx").shard(0).shards().get(0).unassignedInfo().getMessage(), "ZOOOMG"); + unassignedPrimary = routingTable.index("idx").shard(0).shards().get(0); + assertEquals(unassignedPrimary.unassignedInfo().getNumFailedAllocations(), 1); + assertEquals(unassignedPrimary.state(), INITIALIZING); + assertEquals(unassignedPrimary.unassignedInfo().getMessage(), "ZOOOMG"); + // Counter reset, so MaxRetryAllocationDecider#canForceAllocatePrimary should return a YES decision + assertEquals(Decision.YES, new MaxRetryAllocationDecider(Settings.EMPTY).canForceAllocatePrimary( + unassignedPrimary, null, new RoutingAllocation(null, null, clusterState, null, 0, false))); } + } diff --git a/core/src/test/java/org/elasticsearch/common/network/NetworkModuleTests.java b/core/src/test/java/org/elasticsearch/common/network/NetworkModuleTests.java index babc8bde346..734068347b3 100644 --- a/core/src/test/java/org/elasticsearch/common/network/NetworkModuleTests.java +++ b/core/src/test/java/org/elasticsearch/common/network/NetworkModuleTests.java @@ -19,20 +19,12 @@ package org.elasticsearch.common.network; -import java.io.IOException; -import java.util.Collections; - -import org.elasticsearch.action.support.replication.ReplicationTask; import org.elasticsearch.client.node.NodeClient; import org.elasticsearch.common.Table; import org.elasticsearch.common.component.AbstractLifecycleComponent; import org.elasticsearch.common.inject.ModuleTestCase; -import org.elasticsearch.common.io.stream.NamedWriteableRegistry; -import org.elasticsearch.common.io.stream.StreamInput; -import org.elasticsearch.common.io.stream.StreamOutput; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.transport.BoundTransportAddress; -import org.elasticsearch.common.xcontent.XContentBuilder; import org.elasticsearch.http.HttpInfo; import org.elasticsearch.http.HttpServerAdapter; import org.elasticsearch.http.HttpServerTransport; @@ -41,11 +33,12 @@ import org.elasticsearch.rest.BaseRestHandler; import org.elasticsearch.rest.RestChannel; import org.elasticsearch.rest.RestRequest; import org.elasticsearch.rest.action.cat.AbstractCatAction; -import org.elasticsearch.tasks.Task; import org.elasticsearch.test.transport.AssertingLocalTransport; import org.elasticsearch.transport.Transport; import org.elasticsearch.transport.TransportService; +import java.util.Collections; + public class NetworkModuleTests extends ModuleTestCase { static class FakeTransportService extends TransportService { @@ -168,40 +161,4 @@ public class NetworkModuleTests extends ModuleTestCase { assertNotBound(module, HttpServerTransport.class); assertFalse(module.isTransportClient()); } - - public void testRegisterTaskStatus() { - Settings settings = Settings.EMPTY; - NetworkModule module = new NetworkModule(new NetworkService(settings, Collections.emptyList()), settings, false); - NamedWriteableRegistry registry = new NamedWriteableRegistry(module.getNamedWriteables()); - assertFalse(module.isTransportClient()); - - // Builtin reader comes back - assertNotNull(registry.getReader(Task.Status.class, ReplicationTask.Status.NAME)); - - module.registerTaskStatus(DummyTaskStatus.NAME, DummyTaskStatus::new); - assertTrue(module.getNamedWriteables().stream().anyMatch(x -> x.name.equals(DummyTaskStatus.NAME))); - } - - private class DummyTaskStatus implements Task.Status { - public static final String NAME = "dummy"; - - public DummyTaskStatus(StreamInput in) { - throw new UnsupportedOperationException("test"); - } - - @Override - public String getWriteableName() { - return NAME; - } - - @Override - public void writeTo(StreamOutput out) throws IOException { - throw new UnsupportedOperationException(); - } - - @Override - public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException { - throw new UnsupportedOperationException(); - } - } } diff --git a/core/src/test/java/org/elasticsearch/env/NodeEnvironmentTests.java b/core/src/test/java/org/elasticsearch/env/NodeEnvironmentTests.java index ee403bfe910..e5c1c53dad7 100644 --- a/core/src/test/java/org/elasticsearch/env/NodeEnvironmentTests.java +++ b/core/src/test/java/org/elasticsearch/env/NodeEnvironmentTests.java @@ -19,7 +19,6 @@ package org.elasticsearch.env; import org.apache.lucene.index.SegmentInfos; -import org.apache.lucene.store.LockObtainFailedException; import org.apache.lucene.util.IOUtils; import org.apache.lucene.util.LuceneTestCase; import org.elasticsearch.cluster.metadata.IndexMetaData; @@ -129,7 +128,7 @@ public class NodeEnvironmentTests extends ESTestCase { IOUtils.close(first, second); } - public void testShardLock() throws IOException { + public void testShardLock() throws Exception { final NodeEnvironment env = newNodeEnvironment(); Index index = new Index("foo", "fooUUID"); @@ -139,7 +138,7 @@ public class NodeEnvironmentTests extends ESTestCase { try { env.shardLock(new ShardId(index, 0)); fail("shard is locked"); - } catch (LockObtainFailedException ex) { + } catch (ShardLockObtainFailedException ex) { // expected } for (Path path : env.indexPaths(index)) { @@ -149,7 +148,7 @@ public class NodeEnvironmentTests extends ESTestCase { try { env.lockAllForIndex(index, idxSettings, randomIntBetween(0, 10)); fail("shard 0 is locked"); - } catch (LockObtainFailedException ex) { + } catch (ShardLockObtainFailedException ex) { // expected } @@ -161,7 +160,7 @@ public class NodeEnvironmentTests extends ESTestCase { try { env.shardLock(new ShardId(index, 0)); fail("shard is locked"); - } catch (LockObtainFailedException ex) { + } catch (ShardLockObtainFailedException ex) { // expected } IOUtils.close(locks); @@ -213,13 +212,12 @@ public class NodeEnvironmentTests extends ESTestCase { env.close(); } - public void testDeleteSafe() throws IOException, InterruptedException { + public void testDeleteSafe() throws Exception { final NodeEnvironment env = newNodeEnvironment(); final Index index = new Index("foo", "fooUUID"); ShardLock fooLock = env.shardLock(new ShardId(index, 0)); assertEquals(new ShardId(index, 0), fooLock.getShardId()); - for (Path path : env.indexPaths(index)) { Files.createDirectories(path.resolve("0")); Files.createDirectories(path.resolve("1")); @@ -228,14 +226,13 @@ public class NodeEnvironmentTests extends ESTestCase { try { env.deleteShardDirectorySafe(new ShardId(index, 0), idxSettings); fail("shard is locked"); - } catch (LockObtainFailedException ex) { + } catch (ShardLockObtainFailedException ex) { // expected } for (Path path : env.indexPaths(index)) { assertTrue(Files.exists(path.resolve("0"))); assertTrue(Files.exists(path.resolve("1"))); - } env.deleteShardDirectorySafe(new ShardId(index, 1), idxSettings); @@ -248,7 +245,7 @@ public class NodeEnvironmentTests extends ESTestCase { try { env.deleteIndexDirectorySafe(index, randomIntBetween(0, 10), idxSettings); fail("shard is locked"); - } catch (LockObtainFailedException ex) { + } catch (ShardLockObtainFailedException ex) { // expected } fooLock.close(); @@ -338,10 +335,8 @@ public class NodeEnvironmentTests extends ESTestCase { assertEquals(flipFlop[shard].incrementAndGet(), 1); assertEquals(flipFlop[shard].decrementAndGet(), 0); } - } catch (LockObtainFailedException ex) { + } catch (ShardLockObtainFailedException ex) { // ok - } catch (IOException ex) { - fail(ex.toString()); } } } diff --git a/core/src/test/java/org/elasticsearch/gateway/GatewayMetaStateTests.java b/core/src/test/java/org/elasticsearch/gateway/GatewayMetaStateTests.java index a0e813663b8..a3bb21a64b8 100644 --- a/core/src/test/java/org/elasticsearch/gateway/GatewayMetaStateTests.java +++ b/core/src/test/java/org/elasticsearch/gateway/GatewayMetaStateTests.java @@ -24,6 +24,7 @@ import org.elasticsearch.cluster.ClusterChangedEvent; import org.elasticsearch.cluster.ClusterState; import org.elasticsearch.cluster.metadata.IndexMetaData; import org.elasticsearch.cluster.metadata.MetaData; +import org.elasticsearch.cluster.metadata.MetaDataIndexUpgradeService; import org.elasticsearch.cluster.node.DiscoveryNode; import org.elasticsearch.cluster.node.DiscoveryNodes; import org.elasticsearch.cluster.routing.RoutingTable; @@ -31,9 +32,15 @@ import org.elasticsearch.cluster.routing.allocation.AllocationService; import org.elasticsearch.cluster.routing.allocation.decider.ClusterRebalanceAllocationDecider; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.index.Index; +import org.elasticsearch.plugins.MetaDataUpgrader; import org.elasticsearch.test.ESAllocationTestCase; +import org.elasticsearch.test.TestCustomMetaData; +import org.junit.Before; +import java.util.Arrays; import java.util.Collections; +import java.util.EnumSet; +import java.util.HashMap; import java.util.Iterator; import java.util.Set; @@ -54,6 +61,12 @@ import static org.hamcrest.Matchers.equalTo; */ public class GatewayMetaStateTests extends ESAllocationTestCase { + @Before + public void setup() { + MetaData.registerPrototype(CustomMetaData1.TYPE, new CustomMetaData1("")); + MetaData.registerPrototype(CustomMetaData2.TYPE, new CustomMetaData2("")); + } + ClusterChangedEvent generateEvent(boolean initializing, boolean versionChanged, boolean masterEligible) { //ridiculous settings to make sure we don't run into uninitialized because fo default AllocationService strategy = createAllocationService(Settings.builder() @@ -245,4 +258,208 @@ public class GatewayMetaStateTests extends ESAllocationTestCase { ClusterChangedEvent event = generateCloseEvent(masterEligible); assertState(event, stateInMemory, expectMetaData); } + + public void testAddCustomMetaDataOnUpgrade() throws Exception { + MetaData metaData = randomMetaData(); + MetaDataUpgrader metaDataUpgrader = new MetaDataUpgrader( + Collections.singletonList(customs -> { + customs.put(CustomMetaData1.TYPE, new CustomMetaData1("modified_data1")); + return customs; + }) + ); + MetaData upgrade = GatewayMetaState.upgradeMetaData(metaData, new MockMetaDataIndexUpgradeService(false), metaDataUpgrader); + assertTrue(upgrade != metaData); + assertFalse(MetaData.isGlobalStateEquals(upgrade, metaData)); + assertNotNull(upgrade.custom(CustomMetaData1.TYPE)); + assertThat(((TestCustomMetaData) upgrade.custom(CustomMetaData1.TYPE)).getData(), equalTo("modified_data1")); + } + + public void testRemoveCustomMetaDataOnUpgrade() throws Exception { + MetaData metaData = randomMetaData(new CustomMetaData1("data")); + MetaDataUpgrader metaDataUpgrader = new MetaDataUpgrader( + Collections.singletonList(customs -> { + customs.remove(CustomMetaData1.TYPE); + return customs; + }) + ); + MetaData upgrade = GatewayMetaState.upgradeMetaData(metaData, new MockMetaDataIndexUpgradeService(false), metaDataUpgrader); + assertTrue(upgrade != metaData); + assertFalse(MetaData.isGlobalStateEquals(upgrade, metaData)); + assertNull(upgrade.custom(CustomMetaData1.TYPE)); + } + + public void testUpdateCustomMetaDataOnUpgrade() throws Exception { + MetaData metaData = randomMetaData(new CustomMetaData1("data")); + MetaDataUpgrader metaDataUpgrader = new MetaDataUpgrader( + Collections.singletonList(customs -> { + customs.put(CustomMetaData1.TYPE, new CustomMetaData1("modified_data1")); + return customs; + }) + ); + + MetaData upgrade = GatewayMetaState.upgradeMetaData(metaData, new MockMetaDataIndexUpgradeService(false), metaDataUpgrader); + assertTrue(upgrade != metaData); + assertFalse(MetaData.isGlobalStateEquals(upgrade, metaData)); + assertNotNull(upgrade.custom(CustomMetaData1.TYPE)); + assertThat(((TestCustomMetaData) upgrade.custom(CustomMetaData1.TYPE)).getData(), equalTo("modified_data1")); + } + + public void testNoMetaDataUpgrade() throws Exception { + MetaData metaData = randomMetaData(new CustomMetaData1("data")); + MetaDataUpgrader metaDataUpgrader = new MetaDataUpgrader(Collections.emptyList()); + MetaData upgrade = GatewayMetaState.upgradeMetaData(metaData, new MockMetaDataIndexUpgradeService(false), metaDataUpgrader); + assertTrue(upgrade == metaData); + assertTrue(MetaData.isGlobalStateEquals(upgrade, metaData)); + for (IndexMetaData indexMetaData : upgrade) { + assertTrue(metaData.hasIndexMetaData(indexMetaData)); + } + } + + public void testCustomMetaDataValidation() throws Exception { + MetaData metaData = randomMetaData(new CustomMetaData1("data")); + MetaDataUpgrader metaDataUpgrader = new MetaDataUpgrader(Collections.singletonList( + customs -> { + throw new IllegalStateException("custom meta data too old"); + } + )); + try { + GatewayMetaState.upgradeMetaData(metaData, new MockMetaDataIndexUpgradeService(false), metaDataUpgrader); + } catch (IllegalStateException e) { + assertThat(e.getMessage(), equalTo("custom meta data too old")); + } + } + + public void testMultipleCustomMetaDataUpgrade() throws Exception { + final MetaData metaData; + switch (randomIntBetween(0, 2)) { + case 0: + metaData = randomMetaData(new CustomMetaData1("data1"), new CustomMetaData2("data2")); + break; + case 1: + metaData = randomMetaData(randomBoolean() ? new CustomMetaData1("data1") : new CustomMetaData2("data2")); + break; + case 2: + metaData = randomMetaData(); + break; + default: throw new IllegalStateException("should never happen"); + } + MetaDataUpgrader metaDataUpgrader = new MetaDataUpgrader( + Arrays.asList( + customs -> { + customs.put(CustomMetaData1.TYPE, new CustomMetaData1("modified_data1")); + return customs; + }, + customs -> { + customs.put(CustomMetaData2.TYPE, new CustomMetaData1("modified_data2")); + return customs; + }) + ); + MetaData upgrade = GatewayMetaState.upgradeMetaData(metaData, new MockMetaDataIndexUpgradeService(false), metaDataUpgrader); + assertTrue(upgrade != metaData); + assertFalse(MetaData.isGlobalStateEquals(upgrade, metaData)); + assertNotNull(upgrade.custom(CustomMetaData1.TYPE)); + assertThat(((TestCustomMetaData) upgrade.custom(CustomMetaData1.TYPE)).getData(), equalTo("modified_data1")); + assertNotNull(upgrade.custom(CustomMetaData2.TYPE)); + assertThat(((TestCustomMetaData) upgrade.custom(CustomMetaData2.TYPE)).getData(), equalTo("modified_data2")); + for (IndexMetaData indexMetaData : upgrade) { + assertTrue(metaData.hasIndexMetaData(indexMetaData)); + } + } + + public void testIndexMetaDataUpgrade() throws Exception { + MetaData metaData = randomMetaData(); + MetaDataUpgrader metaDataUpgrader = new MetaDataUpgrader(Collections.emptyList()); + MetaData upgrade = GatewayMetaState.upgradeMetaData(metaData, new MockMetaDataIndexUpgradeService(true), metaDataUpgrader); + assertTrue(upgrade != metaData); + assertTrue(MetaData.isGlobalStateEquals(upgrade, metaData)); + for (IndexMetaData indexMetaData : upgrade) { + assertFalse(metaData.hasIndexMetaData(indexMetaData)); + } + } + + public void testCustomMetaDataNoChange() throws Exception { + MetaData metaData = randomMetaData(new CustomMetaData1("data")); + MetaDataUpgrader metaDataUpgrader = new MetaDataUpgrader(Collections.singletonList(HashMap::new)); + MetaData upgrade = GatewayMetaState.upgradeMetaData(metaData, new MockMetaDataIndexUpgradeService(false), metaDataUpgrader); + assertTrue(upgrade == metaData); + assertTrue(MetaData.isGlobalStateEquals(upgrade, metaData)); + for (IndexMetaData indexMetaData : upgrade) { + assertTrue(metaData.hasIndexMetaData(indexMetaData)); + } + } + + private static class MockMetaDataIndexUpgradeService extends MetaDataIndexUpgradeService { + private final boolean upgrade; + + public MockMetaDataIndexUpgradeService(boolean upgrade) { + super(Settings.EMPTY, null, null); + this.upgrade = upgrade; + } + @Override + public IndexMetaData upgradeIndexMetaData(IndexMetaData indexMetaData) { + return upgrade ? IndexMetaData.builder(indexMetaData).build() : indexMetaData; + } + } + + private static class CustomMetaData1 extends TestCustomMetaData { + public static final String TYPE = "custom_md_1"; + + protected CustomMetaData1(String data) { + super(data); + } + + @Override + protected TestCustomMetaData newTestCustomMetaData(String data) { + return new CustomMetaData1(data); + } + + @Override + public String type() { + return TYPE; + } + + @Override + public EnumSet context() { + return EnumSet.of(MetaData.XContentContext.GATEWAY); + } + } + + private static class CustomMetaData2 extends TestCustomMetaData { + public static final String TYPE = "custom_md_2"; + + protected CustomMetaData2(String data) { + super(data); + } + + @Override + protected TestCustomMetaData newTestCustomMetaData(String data) { + return new CustomMetaData2(data); + } + + @Override + public String type() { + return TYPE; + } + + @Override + public EnumSet context() { + return EnumSet.of(MetaData.XContentContext.GATEWAY); + } + } + + private static MetaData randomMetaData(TestCustomMetaData... customMetaDatas) { + MetaData.Builder builder = MetaData.builder(); + for (TestCustomMetaData customMetaData : customMetaDatas) { + builder.putCustom(customMetaData.type(), customMetaData); + } + for (int i = 0; i < randomIntBetween(1, 5); i++) { + builder.put( + IndexMetaData.builder(randomAsciiOfLength(10)) + .settings(settings(Version.CURRENT)) + .numberOfReplicas(randomIntBetween(0, 3)) + .numberOfShards(randomIntBetween(1, 5)) + ); + } + return builder.build(); + } } diff --git a/core/src/test/java/org/elasticsearch/gateway/PrimaryShardAllocatorTests.java b/core/src/test/java/org/elasticsearch/gateway/PrimaryShardAllocatorTests.java index f893f73433e..aeb4ff7b697 100644 --- a/core/src/test/java/org/elasticsearch/gateway/PrimaryShardAllocatorTests.java +++ b/core/src/test/java/org/elasticsearch/gateway/PrimaryShardAllocatorTests.java @@ -30,12 +30,16 @@ import org.elasticsearch.cluster.metadata.MetaData; import org.elasticsearch.cluster.node.DiscoveryNode; import org.elasticsearch.cluster.node.DiscoveryNodes; import org.elasticsearch.cluster.routing.RestoreSource; +import org.elasticsearch.cluster.routing.RoutingNode; import org.elasticsearch.cluster.routing.RoutingNodes; import org.elasticsearch.cluster.routing.RoutingTable; import org.elasticsearch.cluster.routing.ShardRouting; import org.elasticsearch.cluster.routing.ShardRoutingState; +import org.elasticsearch.cluster.routing.UnassignedInfo.AllocationStatus; import org.elasticsearch.cluster.routing.allocation.RoutingAllocation; +import org.elasticsearch.cluster.routing.allocation.decider.AllocationDecider; import org.elasticsearch.cluster.routing.allocation.decider.AllocationDeciders; +import org.elasticsearch.cluster.routing.allocation.decider.Decision; import org.elasticsearch.common.Nullable; import org.elasticsearch.common.UUIDs; import org.elasticsearch.common.settings.Settings; @@ -49,6 +53,7 @@ import org.junit.Before; import java.util.Collections; import java.util.HashMap; +import java.util.List; import java.util.Map; import static org.hamcrest.Matchers.anyOf; @@ -77,8 +82,8 @@ public class PrimaryShardAllocatorTests extends ESAllocationTestCase { } else { allocation = routingAllocationWithOnePrimaryNoReplicas(yesAllocationDeciders(), true, Version.V_2_1_0); } - boolean changed = testAllocator.allocateUnassigned(allocation); - assertThat(changed, equalTo(false)); + testAllocator.allocateUnassigned(allocation); + assertThat(allocation.routingNodesChanged(), equalTo(false)); assertThat(allocation.routingNodes().unassigned().size(), equalTo(1)); assertThat(allocation.routingNodes().unassigned().iterator().next().shardId(), equalTo(shardId)); assertClusterHealthStatus(allocation, ClusterHealthStatus.YELLOW); @@ -94,8 +99,8 @@ public class PrimaryShardAllocatorTests extends ESAllocationTestCase { } else { allocation = routingAllocationWithOnePrimaryNoReplicas(yesAllocationDeciders(), false, Version.V_2_1_0); } - boolean changed = testAllocator.allocateUnassigned(allocation); - assertThat(changed, equalTo(true)); + testAllocator.allocateUnassigned(allocation); + assertThat(allocation.routingNodesChanged(), equalTo(true)); assertThat(allocation.routingNodes().unassigned().ignored().size(), equalTo(1)); assertThat(allocation.routingNodes().unassigned().ignored().get(0).shardId(), equalTo(shardId)); assertClusterHealthStatus(allocation, ClusterHealthStatus.YELLOW); @@ -113,8 +118,8 @@ public class PrimaryShardAllocatorTests extends ESAllocationTestCase { allocation = routingAllocationWithOnePrimaryNoReplicas(yesAllocationDeciders(), false, Version.V_2_1_0); } testAllocator.addData(node1, ShardStateMetaData.NO_VERSION, null, randomBoolean()); - boolean changed = testAllocator.allocateUnassigned(allocation); - assertThat(changed, equalTo(true)); + testAllocator.allocateUnassigned(allocation); + assertThat(allocation.routingNodesChanged(), equalTo(true)); assertThat(allocation.routingNodes().unassigned().ignored().size(), equalTo(1)); assertThat(allocation.routingNodes().unassigned().ignored().get(0).shardId(), equalTo(shardId)); assertClusterHealthStatus(allocation, ClusterHealthStatus.YELLOW); @@ -126,8 +131,8 @@ public class PrimaryShardAllocatorTests extends ESAllocationTestCase { public void testNoMatchingAllocationIdFound() { RoutingAllocation allocation = routingAllocationWithOnePrimaryNoReplicas(yesAllocationDeciders(), false, Version.CURRENT, "id2"); testAllocator.addData(node1, ShardStateMetaData.NO_VERSION, "id1", randomBoolean()); - boolean changed = testAllocator.allocateUnassigned(allocation); - assertThat(changed, equalTo(true)); + testAllocator.allocateUnassigned(allocation); + assertThat(allocation.routingNodesChanged(), equalTo(true)); assertThat(allocation.routingNodes().unassigned().ignored().size(), equalTo(1)); assertThat(allocation.routingNodes().unassigned().ignored().get(0).shardId(), equalTo(shardId)); assertClusterHealthStatus(allocation, ClusterHealthStatus.YELLOW); @@ -140,8 +145,8 @@ public class PrimaryShardAllocatorTests extends ESAllocationTestCase { public void testNoActiveAllocationIds() { RoutingAllocation allocation = routingAllocationWithOnePrimaryNoReplicas(yesAllocationDeciders(), false, Version.V_2_1_1); testAllocator.addData(node1, 1, null, randomBoolean()); - boolean changed = testAllocator.allocateUnassigned(allocation); - assertThat(changed, equalTo(true)); + testAllocator.allocateUnassigned(allocation); + assertThat(allocation.routingNodesChanged(), equalTo(true)); assertThat(allocation.routingNodes().unassigned().ignored().isEmpty(), equalTo(true)); assertThat(allocation.routingNodes().shardsWithState(ShardRoutingState.INITIALIZING).size(), equalTo(1)); assertThat(allocation.routingNodes().shardsWithState(ShardRoutingState.INITIALIZING).get(0).currentNodeId(), equalTo(node1.getId())); @@ -160,8 +165,8 @@ public class PrimaryShardAllocatorTests extends ESAllocationTestCase { allocation = routingAllocationWithOnePrimaryNoReplicas(yesAllocationDeciders(), false, Version.V_2_1_1); testAllocator.addData(node1, 3, null, randomBoolean(), new CorruptIndexException("test", "test")); } - boolean changed = testAllocator.allocateUnassigned(allocation); - assertThat(changed, equalTo(true)); + testAllocator.allocateUnassigned(allocation); + assertThat(allocation.routingNodesChanged(), equalTo(true)); assertThat(allocation.routingNodes().unassigned().ignored().size(), equalTo(1)); assertThat(allocation.routingNodes().unassigned().ignored().get(0).shardId(), equalTo(shardId)); assertClusterHealthStatus(allocation, ClusterHealthStatus.YELLOW); @@ -180,8 +185,8 @@ public class PrimaryShardAllocatorTests extends ESAllocationTestCase { allocation = routingAllocationWithOnePrimaryNoReplicas(yesAllocationDeciders(), false, Version.V_2_2_0); testAllocator.addData(node1, 3, null, randomBoolean()); } - boolean changed = testAllocator.allocateUnassigned(allocation); - assertThat(changed, equalTo(true)); + testAllocator.allocateUnassigned(allocation); + assertThat(allocation.routingNodesChanged(), equalTo(true)); assertThat(allocation.routingNodes().unassigned().ignored().isEmpty(), equalTo(true)); assertThat(allocation.routingNodes().shardsWithState(ShardRoutingState.INITIALIZING).size(), equalTo(1)); assertThat(allocation.routingNodes().shardsWithState(ShardRoutingState.INITIALIZING).get(0).currentNodeId(), equalTo(node1.getId())); @@ -192,6 +197,74 @@ public class PrimaryShardAllocatorTests extends ESAllocationTestCase { assertClusterHealthStatus(allocation, ClusterHealthStatus.YELLOW); } + /** + * Tests that when the nodes with prior copies of the given shard all return a decision of NO, but + * {@link AllocationDecider#canForceAllocatePrimary(ShardRouting, RoutingNode, RoutingAllocation)} + * returns a YES decision for at least one of those NO nodes, then we force allocate to one of them + */ + public void testForceAllocatePrimary() { + testAllocator.addData(node1, ShardStateMetaData.NO_VERSION, "allocId1", randomBoolean()); + AllocationDeciders deciders = new AllocationDeciders(Settings.EMPTY, new AllocationDecider[] { + // since the deciders return a NO decision for allocating a shard (due to the guaranteed NO decision from the second decider), + // the allocator will see if it can force assign the primary, where the decision will be YES + new TestAllocateDecision(randomBoolean() ? Decision.YES : Decision.NO), getNoDeciderThatAllowsForceAllocate() + }); + RoutingAllocation allocation = routingAllocationWithOnePrimaryNoReplicas(deciders, false, Version.CURRENT, "allocId1"); + testAllocator.allocateUnassigned(allocation); + assertThat(allocation.routingNodesChanged(), equalTo(true)); + assertTrue(allocation.routingNodes().unassigned().ignored().isEmpty()); + assertEquals(allocation.routingNodes().shardsWithState(ShardRoutingState.INITIALIZING).size(), 1); + assertEquals(allocation.routingNodes().shardsWithState(ShardRoutingState.INITIALIZING).get(0).currentNodeId(), node1.getId()); + } + + /** + * Tests that when the nodes with prior copies of the given shard all return a decision of NO, and + * {@link AllocationDecider#canForceAllocatePrimary(ShardRouting, RoutingNode, RoutingAllocation)} + * returns a NO or THROTTLE decision for a node, then we do not force allocate to that node. + */ + public void testDontAllocateOnNoOrThrottleForceAllocationDecision() { + testAllocator.addData(node1, ShardStateMetaData.NO_VERSION, "allocId1", randomBoolean()); + boolean forceDecisionNo = randomBoolean(); + AllocationDeciders deciders = new AllocationDeciders(Settings.EMPTY, new AllocationDecider[] { + // since both deciders here return a NO decision for allocating a shard, + // the allocator will see if it can force assign the primary, where the decision will be either NO or THROTTLE, + // so the shard will remain un-initialized + new TestAllocateDecision(Decision.NO), forceDecisionNo ? getNoDeciderThatDeniesForceAllocate() : + getNoDeciderThatThrottlesForceAllocate() + }); + RoutingAllocation allocation = routingAllocationWithOnePrimaryNoReplicas(deciders, false, Version.CURRENT, "allocId1"); + testAllocator.allocateUnassigned(allocation); + assertThat(allocation.routingNodesChanged(), equalTo(true)); + List ignored = allocation.routingNodes().unassigned().ignored(); + assertEquals(ignored.size(), 1); + assertEquals(ignored.get(0).unassignedInfo().getLastAllocationStatus(), + forceDecisionNo ? AllocationStatus.DECIDERS_NO : AllocationStatus.DECIDERS_THROTTLED); + assertTrue(allocation.routingNodes().shardsWithState(ShardRoutingState.INITIALIZING).isEmpty()); + } + + /** + * Tests that when the nodes with prior copies of the given shard return a THROTTLE decision, + * then we do not force allocate to that node but instead throttle. + */ + public void testDontForceAllocateOnThrottleDecision() { + testAllocator.addData(node1, ShardStateMetaData.NO_VERSION, "allocId1", randomBoolean()); + AllocationDeciders deciders = new AllocationDeciders(Settings.EMPTY, new AllocationDecider[] { + // since we have a NO decision for allocating a shard (because the second decider returns a NO decision), + // the allocator will see if it can force assign the primary, and in this case, + // the TestAllocateDecision's decision for force allocating is to THROTTLE (using + // the default behavior) so despite the other decider's decision to return YES for + // force allocating the shard, we still THROTTLE due to the decision from TestAllocateDecision + new TestAllocateDecision(Decision.THROTTLE), getNoDeciderThatAllowsForceAllocate() + }); + RoutingAllocation allocation = routingAllocationWithOnePrimaryNoReplicas(deciders, false, Version.CURRENT, "allocId1"); + testAllocator.allocateUnassigned(allocation); + assertThat(allocation.routingNodesChanged(), equalTo(true)); + List ignored = allocation.routingNodes().unassigned().ignored(); + assertEquals(ignored.size(), 1); + assertEquals(ignored.get(0).unassignedInfo().getLastAllocationStatus(), AllocationStatus.DECIDERS_THROTTLED); + assertTrue(allocation.routingNodes().shardsWithState(ShardRoutingState.INITIALIZING).isEmpty()); + } + /** * Tests that when there was a node that previously had the primary, it will be allocated to that same node again. */ @@ -202,8 +275,8 @@ public class PrimaryShardAllocatorTests extends ESAllocationTestCase { boolean node1HasPrimaryShard = randomBoolean(); testAllocator.addData(node1, ShardStateMetaData.NO_VERSION, node1HasPrimaryShard ? primaryAllocId : replicaAllocId, node1HasPrimaryShard); testAllocator.addData(node2, ShardStateMetaData.NO_VERSION, node1HasPrimaryShard ? replicaAllocId : primaryAllocId, !node1HasPrimaryShard); - boolean changed = testAllocator.allocateUnassigned(allocation); - assertThat(changed, equalTo(true)); + testAllocator.allocateUnassigned(allocation); + assertThat(allocation.routingNodesChanged(), equalTo(true)); assertThat(allocation.routingNodes().unassigned().ignored().isEmpty(), equalTo(true)); assertThat(allocation.routingNodes().shardsWithState(ShardRoutingState.INITIALIZING).size(), equalTo(1)); DiscoveryNode allocatedNode = node1HasPrimaryShard ? node1 : node2; @@ -224,8 +297,8 @@ public class PrimaryShardAllocatorTests extends ESAllocationTestCase { allocation = routingAllocationWithOnePrimaryNoReplicas(throttleAllocationDeciders(), false, Version.V_2_2_0); testAllocator.addData(node1, 3, null, randomBoolean()); } - boolean changed = testAllocator.allocateUnassigned(allocation); - assertThat(changed, equalTo(true)); + testAllocator.allocateUnassigned(allocation); + assertThat(allocation.routingNodesChanged(), equalTo(true)); assertThat(allocation.routingNodes().unassigned().ignored().size(), equalTo(1)); assertThat(allocation.routingNodes().unassigned().ignored().get(0).shardId(), equalTo(shardId)); assertClusterHealthStatus(allocation, ClusterHealthStatus.YELLOW); @@ -244,8 +317,8 @@ public class PrimaryShardAllocatorTests extends ESAllocationTestCase { allocation = routingAllocationWithOnePrimaryNoReplicas(noAllocationDeciders(), false, Version.V_2_0_0); testAllocator.addData(node1, 3, null, randomBoolean()); } - boolean changed = testAllocator.allocateUnassigned(allocation); - assertThat(changed, equalTo(true)); + testAllocator.allocateUnassigned(allocation); + assertThat(allocation.routingNodesChanged(), equalTo(true)); assertThat(allocation.routingNodes().unassigned().ignored().isEmpty(), equalTo(true)); assertThat(allocation.routingNodes().shardsWithState(ShardRoutingState.INITIALIZING).size(), equalTo(1)); assertThat(allocation.routingNodes().shardsWithState(ShardRoutingState.INITIALIZING).get(0).currentNodeId(), equalTo(node1.getId())); @@ -258,8 +331,8 @@ public class PrimaryShardAllocatorTests extends ESAllocationTestCase { public void testAllocateToTheHighestVersionOnLegacyIndex() { RoutingAllocation allocation = routingAllocationWithOnePrimaryNoReplicas(yesAllocationDeciders(), false, Version.V_2_0_0); testAllocator.addData(node1, 10, null, randomBoolean()).addData(node2, 12, null, randomBoolean()); - boolean changed = testAllocator.allocateUnassigned(allocation); - assertThat(changed, equalTo(true)); + testAllocator.allocateUnassigned(allocation); + assertThat(allocation.routingNodesChanged(), equalTo(true)); assertThat(allocation.routingNodes().unassigned().ignored().isEmpty(), equalTo(true)); assertThat(allocation.routingNodes().shardsWithState(ShardRoutingState.INITIALIZING).size(), equalTo(1)); assertThat(allocation.routingNodes().shardsWithState(ShardRoutingState.INITIALIZING).get(0).currentNodeId(), equalTo(node2.getId())); @@ -277,8 +350,8 @@ public class PrimaryShardAllocatorTests extends ESAllocationTestCase { testAllocator.addData(node1, 10, null, randomBoolean()); testAllocator.addData(node2, ShardStateMetaData.NO_VERSION, "some allocId", randomBoolean()); testAllocator.addData(node3, 12, null, randomBoolean()); - boolean changed = testAllocator.allocateUnassigned(allocation); - assertThat(changed, equalTo(true)); + testAllocator.allocateUnassigned(allocation); + assertThat(allocation.routingNodesChanged(), equalTo(true)); assertThat(allocation.routingNodes().unassigned().ignored().isEmpty(), equalTo(true)); assertThat(allocation.routingNodes().shardsWithState(ShardRoutingState.INITIALIZING).size(), equalTo(1)); assertThat(allocation.routingNodes().shardsWithState(ShardRoutingState.INITIALIZING).get(0).currentNodeId(), equalTo(node2.getId())); @@ -297,8 +370,8 @@ public class PrimaryShardAllocatorTests extends ESAllocationTestCase { boolean clusterHasActiveAllocationIds = shardStateHasAllocationId ? randomBoolean() : false; RoutingAllocation allocation = getRestoreRoutingAllocation(yesAllocationDeciders(), clusterHasActiveAllocationIds); testAllocator.addData(node1, legacyVersion, allocationId, randomBoolean()); - boolean changed = testAllocator.allocateUnassigned(allocation); - assertThat(changed, equalTo(true)); + testAllocator.allocateUnassigned(allocation); + assertThat(allocation.routingNodesChanged(), equalTo(true)); assertThat(allocation.routingNodes().unassigned().ignored().isEmpty(), equalTo(true)); assertThat(allocation.routingNodes().shardsWithState(ShardRoutingState.INITIALIZING).size(), equalTo(1)); assertClusterHealthStatus(allocation, ClusterHealthStatus.YELLOW); @@ -315,8 +388,8 @@ public class PrimaryShardAllocatorTests extends ESAllocationTestCase { boolean clusterHasActiveAllocationIds = shardStateHasAllocationId ? randomBoolean() : false; RoutingAllocation allocation = getRestoreRoutingAllocation(throttleAllocationDeciders(), clusterHasActiveAllocationIds); testAllocator.addData(node1, legacyVersion, allocationId, randomBoolean()); - boolean changed = testAllocator.allocateUnassigned(allocation); - assertThat(changed, equalTo(true)); + testAllocator.allocateUnassigned(allocation); + assertThat(allocation.routingNodesChanged(), equalTo(true)); assertThat(allocation.routingNodes().unassigned().ignored().isEmpty(), equalTo(false)); assertClusterHealthStatus(allocation, ClusterHealthStatus.YELLOW); } @@ -332,8 +405,8 @@ public class PrimaryShardAllocatorTests extends ESAllocationTestCase { boolean clusterHasActiveAllocationIds = shardStateHasAllocationId ? randomBoolean() : false; RoutingAllocation allocation = getRestoreRoutingAllocation(noAllocationDeciders(), clusterHasActiveAllocationIds); testAllocator.addData(node1, legacyVersion, allocationId, randomBoolean()); - boolean changed = testAllocator.allocateUnassigned(allocation); - assertThat(changed, equalTo(true)); + testAllocator.allocateUnassigned(allocation); + assertThat(allocation.routingNodesChanged(), equalTo(true)); assertThat(allocation.routingNodes().unassigned().ignored().isEmpty(), equalTo(true)); assertThat(allocation.routingNodes().shardsWithState(ShardRoutingState.INITIALIZING).size(), equalTo(1)); assertClusterHealthStatus(allocation, ClusterHealthStatus.YELLOW); @@ -346,8 +419,8 @@ public class PrimaryShardAllocatorTests extends ESAllocationTestCase { public void testRestoreDoesNotAssignIfNoShardAvailable() { RoutingAllocation allocation = getRestoreRoutingAllocation(yesAllocationDeciders(), randomBoolean()); testAllocator.addData(node1, ShardStateMetaData.NO_VERSION, null, false); - boolean changed = testAllocator.allocateUnassigned(allocation); - assertThat(changed, equalTo(false)); + testAllocator.allocateUnassigned(allocation); + assertThat(allocation.routingNodesChanged(), equalTo(false)); assertThat(allocation.routingNodes().unassigned().ignored().isEmpty(), equalTo(true)); assertThat(allocation.routingNodes().unassigned().size(), equalTo(1)); assertClusterHealthStatus(allocation, ClusterHealthStatus.YELLOW); @@ -381,8 +454,8 @@ public class PrimaryShardAllocatorTests extends ESAllocationTestCase { long legacyVersion = hasActiveAllocation ? ShardStateMetaData.NO_VERSION : 1; RoutingAllocation allocation = getRecoverOnAnyNodeRoutingAllocation(yesAllocationDeciders(), hasActiveAllocation); testAllocator.addData(node1, legacyVersion, allocationId, randomBoolean()); - boolean changed = testAllocator.allocateUnassigned(allocation); - assertThat(changed, equalTo(true)); + testAllocator.allocateUnassigned(allocation); + assertThat(allocation.routingNodesChanged(), equalTo(true)); assertThat(allocation.routingNodes().unassigned().ignored().isEmpty(), equalTo(true)); assertThat(allocation.routingNodes().shardsWithState(ShardRoutingState.INITIALIZING).size(), equalTo(1)); assertClusterHealthStatus(allocation, ClusterHealthStatus.RED); @@ -398,8 +471,8 @@ public class PrimaryShardAllocatorTests extends ESAllocationTestCase { long legacyVersion = hasActiveAllocation ? ShardStateMetaData.NO_VERSION : 1; RoutingAllocation allocation = getRecoverOnAnyNodeRoutingAllocation(throttleAllocationDeciders(), hasActiveAllocation); testAllocator.addData(node1, legacyVersion, allocationId, randomBoolean()); - boolean changed = testAllocator.allocateUnassigned(allocation); - assertThat(changed, equalTo(true)); + testAllocator.allocateUnassigned(allocation); + assertThat(allocation.routingNodesChanged(), equalTo(true)); assertThat(allocation.routingNodes().unassigned().ignored().isEmpty(), equalTo(false)); assertClusterHealthStatus(allocation, ClusterHealthStatus.YELLOW); } @@ -414,8 +487,8 @@ public class PrimaryShardAllocatorTests extends ESAllocationTestCase { long legacyVersion = hasActiveAllocation ? ShardStateMetaData.NO_VERSION : 1; RoutingAllocation allocation = getRecoverOnAnyNodeRoutingAllocation(noAllocationDeciders(), hasActiveAllocation); testAllocator.addData(node1, legacyVersion, allocationId, randomBoolean()); - boolean changed = testAllocator.allocateUnassigned(allocation); - assertThat(changed, equalTo(true)); + testAllocator.allocateUnassigned(allocation); + assertThat(allocation.routingNodesChanged(), equalTo(true)); assertThat(allocation.routingNodes().unassigned().ignored().isEmpty(), equalTo(true)); assertThat(allocation.routingNodes().shardsWithState(ShardRoutingState.INITIALIZING).size(), equalTo(1)); assertClusterHealthStatus(allocation, ClusterHealthStatus.RED); @@ -428,8 +501,8 @@ public class PrimaryShardAllocatorTests extends ESAllocationTestCase { public void testRecoverOnAnyNodeDoesNotAssignIfNoShardAvailable() { RoutingAllocation allocation = getRecoverOnAnyNodeRoutingAllocation(yesAllocationDeciders(), randomBoolean()); testAllocator.addData(node1, ShardStateMetaData.NO_VERSION, null, randomBoolean()); - boolean changed = testAllocator.allocateUnassigned(allocation); - assertThat(changed, equalTo(false)); + testAllocator.allocateUnassigned(allocation); + assertThat(allocation.routingNodesChanged(), equalTo(false)); assertThat(allocation.routingNodes().unassigned().ignored().isEmpty(), equalTo(true)); assertThat(allocation.routingNodes().unassigned().size(), equalTo(1)); assertClusterHealthStatus(allocation, ClusterHealthStatus.YELLOW); @@ -471,8 +544,8 @@ public class PrimaryShardAllocatorTests extends ESAllocationTestCase { .nodes(DiscoveryNodes.builder().add(node1).add(node2).add(node3)).build(); RoutingAllocation allocation = new RoutingAllocation(yesAllocationDeciders(), new RoutingNodes(state, false), state, null, System.nanoTime(), false); - boolean changed = testAllocator.allocateUnassigned(allocation); - assertThat(changed, equalTo(true)); + testAllocator.allocateUnassigned(allocation); + assertThat(allocation.routingNodesChanged(), equalTo(true)); assertThat(allocation.routingNodes().unassigned().ignored().size(), equalTo(1)); assertThat(allocation.routingNodes().unassigned().ignored().get(0).shardId(), equalTo(shardId)); assertThat(allocation.routingNodes().shardsWithState(ShardRoutingState.UNASSIGNED).size(), equalTo(2)); // replicas @@ -480,8 +553,8 @@ public class PrimaryShardAllocatorTests extends ESAllocationTestCase { testAllocator.addData(node1, 1, null, randomBoolean()); allocation = new RoutingAllocation(yesAllocationDeciders(), new RoutingNodes(state, false), state, null, System.nanoTime(), false); - changed = testAllocator.allocateUnassigned(allocation); - assertThat(changed, equalTo(true)); + testAllocator.allocateUnassigned(allocation); + assertThat(allocation.routingNodesChanged(), equalTo(true)); assertThat(allocation.routingNodes().unassigned().ignored().size(), equalTo(1)); assertThat(allocation.routingNodes().unassigned().ignored().get(0).shardId(), equalTo(shardId)); assertThat(allocation.routingNodes().shardsWithState(ShardRoutingState.UNASSIGNED).size(), equalTo(2)); // replicas @@ -489,8 +562,8 @@ public class PrimaryShardAllocatorTests extends ESAllocationTestCase { testAllocator.addData(node2, 1, null, randomBoolean()); allocation = new RoutingAllocation(yesAllocationDeciders(), new RoutingNodes(state, false), state, null, System.nanoTime(), false); - changed = testAllocator.allocateUnassigned(allocation); - assertThat(changed, equalTo(true)); + testAllocator.allocateUnassigned(allocation); + assertThat(allocation.routingNodesChanged(), equalTo(true)); assertThat(allocation.routingNodes().unassigned().ignored().size(), equalTo(0)); assertThat(allocation.routingNodes().shardsWithState(ShardRoutingState.UNASSIGNED).size(), equalTo(2)); // replicas assertThat(allocation.routingNodes().shardsWithState(ShardRoutingState.INITIALIZING).size(), equalTo(1)); @@ -515,8 +588,8 @@ public class PrimaryShardAllocatorTests extends ESAllocationTestCase { .nodes(DiscoveryNodes.builder().add(node1).add(node2).add(node3)).build(); RoutingAllocation allocation = new RoutingAllocation(yesAllocationDeciders(), new RoutingNodes(state, false), state, null, System.nanoTime(), false); - boolean changed = testAllocator.allocateUnassigned(allocation); - assertThat(changed, equalTo(true)); + testAllocator.allocateUnassigned(allocation); + assertThat(allocation.routingNodesChanged(), equalTo(true)); assertThat(allocation.routingNodes().unassigned().ignored().size(), equalTo(1)); assertThat(allocation.routingNodes().unassigned().ignored().get(0).shardId(), equalTo(shardId)); assertThat(allocation.routingNodes().shardsWithState(ShardRoutingState.UNASSIGNED).size(), equalTo(2)); // replicas @@ -524,8 +597,8 @@ public class PrimaryShardAllocatorTests extends ESAllocationTestCase { testAllocator.addData(node1, 1, null, randomBoolean()); allocation = new RoutingAllocation(yesAllocationDeciders(), new RoutingNodes(state, false), state, null, System.nanoTime(), false); - changed = testAllocator.allocateUnassigned(allocation); - assertThat(changed, equalTo(true)); + testAllocator.allocateUnassigned(allocation); + assertThat(allocation.routingNodesChanged(), equalTo(true)); assertThat(allocation.routingNodes().unassigned().ignored().size(), equalTo(1)); assertThat(allocation.routingNodes().unassigned().ignored().get(0).shardId(), equalTo(shardId)); assertThat(allocation.routingNodes().shardsWithState(ShardRoutingState.UNASSIGNED).size(), equalTo(2)); // replicas @@ -533,8 +606,8 @@ public class PrimaryShardAllocatorTests extends ESAllocationTestCase { testAllocator.addData(node2, 2, null, randomBoolean()); allocation = new RoutingAllocation(yesAllocationDeciders(), new RoutingNodes(state, false), state, null, System.nanoTime(), false); - changed = testAllocator.allocateUnassigned(allocation); - assertThat(changed, equalTo(true)); + testAllocator.allocateUnassigned(allocation); + assertThat(allocation.routingNodesChanged(), equalTo(true)); assertThat(allocation.routingNodes().unassigned().ignored().size(), equalTo(0)); assertThat(allocation.routingNodes().shardsWithState(ShardRoutingState.UNASSIGNED).size(), equalTo(2)); // replicas assertThat(allocation.routingNodes().shardsWithState(ShardRoutingState.INITIALIZING).size(), equalTo(1)); @@ -542,11 +615,12 @@ public class PrimaryShardAllocatorTests extends ESAllocationTestCase { assertClusterHealthStatus(allocation, ClusterHealthStatus.RED); } - private RoutingAllocation routingAllocationWithOnePrimaryNoReplicas(AllocationDeciders deciders, boolean asNew, Version version, String... activeAllocationIds) { + private RoutingAllocation routingAllocationWithOnePrimaryNoReplicas(AllocationDeciders deciders, boolean asNew, Version version, + String... activeAllocationIds) { MetaData metaData = MetaData.builder() .put(IndexMetaData.builder(shardId.getIndexName()).settings(settings(version)) - .numberOfShards(1).numberOfReplicas(0).putActiveAllocationIds(0, Sets.newHashSet(activeAllocationIds))) - .build(); + .numberOfShards(1).numberOfReplicas(0).putActiveAllocationIds(shardId.id(), Sets.newHashSet(activeAllocationIds))) + .build(); RoutingTable.Builder routingTableBuilder = RoutingTable.builder(); if (asNew) { routingTableBuilder.addAsNew(metaData.index(shardId.getIndex())); @@ -573,6 +647,28 @@ public class PrimaryShardAllocatorTests extends ESAllocationTestCase { assertThat(clusterStateHealth.getStatus().ordinal(), lessThanOrEqualTo(expectedStatus.ordinal())); } + private AllocationDecider getNoDeciderThatAllowsForceAllocate() { + return getNoDeciderWithForceAllocate(Decision.YES); + } + + private AllocationDecider getNoDeciderThatThrottlesForceAllocate() { + return getNoDeciderWithForceAllocate(Decision.THROTTLE); + } + + private AllocationDecider getNoDeciderThatDeniesForceAllocate() { + return getNoDeciderWithForceAllocate(Decision.NO); + } + + private AllocationDecider getNoDeciderWithForceAllocate(final Decision forceAllocateDecision) { + return new TestAllocateDecision(Decision.NO) { + @Override + public Decision canForceAllocatePrimary(ShardRouting shardRouting, RoutingNode node, RoutingAllocation allocation) { + assert shardRouting.primary() : "cannot force allocate a non-primary shard " + shardRouting; + return forceAllocateDecision; + } + }; + } + class TestAllocator extends PrimaryShardAllocator { private Map data; diff --git a/core/src/test/java/org/elasticsearch/gateway/ReplicaShardAllocatorTests.java b/core/src/test/java/org/elasticsearch/gateway/ReplicaShardAllocatorTests.java index 549b4e786cb..2570df3a561 100644 --- a/core/src/test/java/org/elasticsearch/gateway/ReplicaShardAllocatorTests.java +++ b/core/src/test/java/org/elasticsearch/gateway/ReplicaShardAllocatorTests.java @@ -234,16 +234,16 @@ public class ReplicaShardAllocatorTests extends ESAllocationTestCase { // we sometime return empty list of files, make sure we test this as well testAllocator.addData(node2, null); } - boolean changed = testAllocator.allocateUnassigned(allocation); - assertThat(changed, equalTo(false)); + testAllocator.allocateUnassigned(allocation); + assertThat(allocation.routingNodesChanged(), equalTo(false)); assertThat(allocation.routingNodes().unassigned().ignored().size(), equalTo(1)); assertThat(allocation.routingNodes().unassigned().ignored().get(0).shardId(), equalTo(shardId)); allocation = onePrimaryOnNode1And1Replica(yesAllocationDeciders(), Settings.builder().put(UnassignedInfo.INDEX_DELAYED_NODE_LEFT_TIMEOUT_SETTING.getKey(), TimeValue.timeValueHours(1)).build(), UnassignedInfo.Reason.NODE_LEFT); testAllocator.addData(node2, "MATCH", new StoreFileMetaData("file1", 10, "MATCH_CHECKSUM")); - changed = testAllocator.allocateUnassigned(allocation); - assertThat(changed, equalTo(true)); + testAllocator.allocateUnassigned(allocation); + assertThat(allocation.routingNodesChanged(), equalTo(true)); assertThat(allocation.routingNodes().shardsWithState(ShardRoutingState.INITIALIZING).size(), equalTo(1)); assertThat(allocation.routingNodes().shardsWithState(ShardRoutingState.INITIALIZING).get(0).currentNodeId(), equalTo(node2.getId())); } @@ -253,8 +253,8 @@ public class ReplicaShardAllocatorTests extends ESAllocationTestCase { testAllocator.addData(node1, "MATCH", new StoreFileMetaData("file1", 10, "MATCH_CHECKSUM")) .addData(node2, "NO_MATCH", new StoreFileMetaData("file1", 10, "MATCH_CHECKSUM")) .addData(node3, "MATCH", new StoreFileMetaData("file1", 10, "MATCH_CHECKSUM")); - boolean changed = testAllocator.processExistingRecoveries(allocation); - assertThat(changed, equalTo(true)); + testAllocator.processExistingRecoveries(allocation); + assertThat(allocation.routingNodesChanged(), equalTo(true)); assertThat(allocation.routingNodes().shardsWithState(ShardRoutingState.UNASSIGNED).size(), equalTo(1)); assertThat(allocation.routingNodes().shardsWithState(ShardRoutingState.UNASSIGNED).get(0).shardId(), equalTo(shardId)); } @@ -264,8 +264,8 @@ public class ReplicaShardAllocatorTests extends ESAllocationTestCase { testAllocator.addData(node1, "MATCH", new StoreFileMetaData("file1", 10, "MATCH_CHECKSUM")) .addData(node2, "MATCH", new StoreFileMetaData("file1", 10, "MATCH_CHECKSUM")) .addData(node3, randomBoolean() ? "MATCH" : "NO_MATCH", new StoreFileMetaData("file1", 10, "MATCH_CHECKSUM")); - boolean changed = testAllocator.processExistingRecoveries(allocation); - assertThat(changed, equalTo(false)); + testAllocator.processExistingRecoveries(allocation); + assertThat(allocation.routingNodesChanged(), equalTo(false)); assertThat(allocation.routingNodes().shardsWithState(ShardRoutingState.UNASSIGNED).size(), equalTo(0)); } @@ -273,8 +273,8 @@ public class ReplicaShardAllocatorTests extends ESAllocationTestCase { RoutingAllocation allocation = onePrimaryOnNode1And1ReplicaRecovering(yesAllocationDeciders()); testAllocator.addData(node1, "MATCH", new StoreFileMetaData("file1", 10, "MATCH_CHECKSUM")) .addData(node2, "MATCH", new StoreFileMetaData("file1", 10, "MATCH_CHECKSUM")); - boolean changed = testAllocator.processExistingRecoveries(allocation); - assertThat(changed, equalTo(false)); + testAllocator.processExistingRecoveries(allocation); + assertThat(allocation.routingNodesChanged(), equalTo(false)); assertThat(allocation.routingNodes().shardsWithState(ShardRoutingState.UNASSIGNED).size(), equalTo(0)); } diff --git a/core/src/test/java/org/elasticsearch/index/mapper/StringMappingUpgradeTests.java b/core/src/test/java/org/elasticsearch/index/mapper/StringMappingUpgradeTests.java index 41077493761..311bf0205ed 100644 --- a/core/src/test/java/org/elasticsearch/index/mapper/StringMappingUpgradeTests.java +++ b/core/src/test/java/org/elasticsearch/index/mapper/StringMappingUpgradeTests.java @@ -106,6 +106,32 @@ public class StringMappingUpgradeTests extends ESSingleNodeTestCase { assertEquals(IndexOptions.NONE, field.fieldType().indexOptions()); } + public void testUpgradeIndexOptions() throws IOException { + IndexService indexService = createIndex("test"); + DocumentMapperParser parser = indexService.mapperService().documentMapperParser(); + String mapping = XContentFactory.jsonBuilder().startObject().startObject("type") + .startObject("properties").startObject("field").field("type", "string") + .field("index_options", "offsets").endObject().endObject() + .endObject().endObject().string(); + DocumentMapper mapper = parser.parse("type", new CompressedXContent(mapping)); + FieldMapper field = mapper.mappers().getMapper("field"); + assertThat(field, instanceOf(TextFieldMapper.class)); + assertEquals(IndexOptions.DOCS_AND_FREQS_AND_POSITIONS_AND_OFFSETS, field.fieldType().indexOptions()); + } + + public void testUpgradePositionGap() throws IOException { + IndexService indexService = createIndex("test"); + DocumentMapperParser parser = indexService.mapperService().documentMapperParser(); + String mapping = XContentFactory.jsonBuilder().startObject().startObject("type") + .startObject("properties").startObject("field").field("type", "string") + .field("position_increment_gap", 42).endObject().endObject() + .endObject().endObject().string(); + DocumentMapper mapper = parser.parse("type", new CompressedXContent(mapping)); + FieldMapper field = mapper.mappers().getMapper("field"); + assertThat(field, instanceOf(TextFieldMapper.class)); + assertEquals(42, field.fieldType().indexAnalyzer().getPositionIncrementGap("field")); + } + public void testIllegalIndexValue() throws IOException { IndexService indexService = createIndex("test"); DocumentMapperParser parser = indexService.mapperService().documentMapperParser(); @@ -297,7 +323,11 @@ public class StringMappingUpgradeTests extends ESSingleNodeTestCase { } if (randomBoolean()) { // this option is not upgraded automatically - mapping.field("index_options", "docs"); + if (keyword) { + mapping.field("index_options", "docs"); + } else { + mapping.field("ignore_above", 30); + } shouldUpgrade = false; } mapping.endObject().endObject().endObject().endObject(); diff --git a/core/src/test/java/org/elasticsearch/indices/IndicesServiceTests.java b/core/src/test/java/org/elasticsearch/indices/IndicesServiceTests.java index 27bb2cccff4..216ddf76f6e 100644 --- a/core/src/test/java/org/elasticsearch/indices/IndicesServiceTests.java +++ b/core/src/test/java/org/elasticsearch/indices/IndicesServiceTests.java @@ -33,6 +33,7 @@ import org.elasticsearch.common.io.FileSystemUtils; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.unit.TimeValue; import org.elasticsearch.env.NodeEnvironment; +import org.elasticsearch.env.ShardLockObtainFailedException; import org.elasticsearch.gateway.GatewayMetaState; import org.elasticsearch.gateway.LocalAllocateDangledIndices; import org.elasticsearch.gateway.MetaStateService; @@ -172,7 +173,7 @@ public class IndicesServiceTests extends ESSingleNodeTestCase { try { indicesService.processPendingDeletes(test.index(), test.getIndexSettings(), new TimeValue(0, TimeUnit.MILLISECONDS)); fail("can't get lock"); - } catch (LockObtainFailedException ex) { + } catch (ShardLockObtainFailedException ex) { } assertTrue(path.exists()); diff --git a/core/src/test/java/org/elasticsearch/search/SearchModuleTests.java b/core/src/test/java/org/elasticsearch/search/SearchModuleTests.java index d7bc1645c33..c3324955fe6 100644 --- a/core/src/test/java/org/elasticsearch/search/SearchModuleTests.java +++ b/core/src/test/java/org/elasticsearch/search/SearchModuleTests.java @@ -21,20 +21,40 @@ package org.elasticsearch.search; import org.elasticsearch.common.ParseFieldMatcher; import org.elasticsearch.common.bytes.BytesArray; import org.elasticsearch.common.inject.ModuleTestCase; -import org.elasticsearch.common.io.stream.NamedWriteableRegistry; +import org.elasticsearch.common.io.stream.StreamInput; +import org.elasticsearch.common.io.stream.StreamOutput; import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.common.xcontent.XContentBuilder; import org.elasticsearch.common.xcontent.XContentHelper; import org.elasticsearch.common.xcontent.XContentParser; +import org.elasticsearch.index.query.QueryParseContext; import org.elasticsearch.index.query.QueryParser; import org.elasticsearch.index.query.TermQueryBuilder; import org.elasticsearch.index.query.functionscore.GaussDecayFunctionBuilder; import org.elasticsearch.indices.query.IndicesQueriesRegistry; import org.elasticsearch.plugins.SearchPlugin; +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.InternalAggregation.ReduceContext; import org.elasticsearch.search.aggregations.bucket.significant.heuristics.ChiSquare; import org.elasticsearch.search.aggregations.bucket.significant.heuristics.SignificanceHeuristic; import org.elasticsearch.search.aggregations.bucket.significant.heuristics.SignificanceHeuristicParser; +import org.elasticsearch.search.aggregations.bucket.terms.TermsAggregationBuilder; +import org.elasticsearch.search.aggregations.bucket.terms.TermsParser; +import org.elasticsearch.search.aggregations.pipeline.AbstractPipelineAggregationBuilder; +import org.elasticsearch.search.aggregations.pipeline.PipelineAggregator; +import org.elasticsearch.search.aggregations.pipeline.derivative.DerivativePipelineAggregationBuilder; +import org.elasticsearch.search.aggregations.pipeline.derivative.DerivativePipelineAggregator; +import org.elasticsearch.search.aggregations.pipeline.derivative.InternalDerivative; import org.elasticsearch.search.aggregations.pipeline.movavg.models.MovAvgModel; import org.elasticsearch.search.aggregations.pipeline.movavg.models.SimpleModel; +import org.elasticsearch.search.aggregations.support.AggregationContext; +import org.elasticsearch.search.aggregations.support.ValuesSource; +import org.elasticsearch.search.aggregations.support.ValuesSourceAggregationBuilder; +import org.elasticsearch.search.aggregations.support.ValuesSourceAggregatorFactory; +import org.elasticsearch.search.aggregations.support.ValuesSourceConfig; import org.elasticsearch.search.fetch.FetchSubPhase; import org.elasticsearch.search.fetch.subphase.ExplainFetchSubPhase; import org.elasticsearch.search.fetch.subphase.highlight.CustomHighlighter; @@ -119,13 +139,34 @@ public class SearchModuleTests extends ModuleTestCase { expectThrows(IllegalArgumentException.class, () -> new SearchModule(Settings.EMPTY, false, singletonList(registersDupeFetchSubPhase))); - SearchPlugin registersDupeFetchQuery = new SearchPlugin() { + SearchPlugin registersDupeQuery = new SearchPlugin() { public List> getQueries() { return singletonList(new QuerySpec<>(TermQueryBuilder.NAME, TermQueryBuilder::new, TermQueryBuilder::fromXContent)); } }; expectThrows(IllegalArgumentException.class, () -> new SearchModule(Settings.EMPTY, false, - singletonList(registersDupeFetchQuery))); + singletonList(registersDupeQuery))); + + SearchPlugin registersDupeAggregation = new SearchPlugin() { + public List getAggregations() { + return singletonList(new AggregationSpec(TermsAggregationBuilder.NAME, TermsAggregationBuilder::new, new TermsParser())); + } + }; + expectThrows(IllegalArgumentException.class, () -> new SearchModule(Settings.EMPTY, false, + singletonList(registersDupeAggregation))); + + SearchPlugin registersDupePipelineAggregation = new SearchPlugin() { + public List getPipelineAggregations() { + return singletonList(new PipelineAggregationSpec( + DerivativePipelineAggregationBuilder.NAME, + DerivativePipelineAggregationBuilder::new, + DerivativePipelineAggregator::new, + DerivativePipelineAggregationBuilder::parse) + .addResultReader(InternalDerivative::new)); + } + }; + expectThrows(IllegalArgumentException.class, () -> new SearchModule(Settings.EMPTY, false, + singletonList(registersDupePipelineAggregation))); } public void testRegisterSuggester() { @@ -185,6 +226,27 @@ public class SearchModuleTests extends ModuleTestCase { } } + public void testRegisterAggregation() { + SearchModule module = new SearchModule(Settings.EMPTY, false, singletonList(new SearchPlugin() { + public List getAggregations() { + return singletonList(new AggregationSpec("test", TestAggregationBuilder::new, TestAggregationBuilder::fromXContent)); + } + })); + + assertNotNull(module.getAggregatorParsers().parser("test", ParseFieldMatcher.STRICT)); + } + + public void testRegisterPipelineAggregation() { + SearchModule module = new SearchModule(Settings.EMPTY, false, singletonList(new SearchPlugin() { + public List getPipelineAggregations() { + return singletonList(new PipelineAggregationSpec("test", + TestPipelineAggregationBuilder::new, TestPipelineAggregator::new, TestPipelineAggregationBuilder::fromXContent)); + } + })); + + assertNotNull(module.getAggregatorParsers().pipelineParser("test", ParseFieldMatcher.STRICT)); + } + private static final String[] NON_DEPRECATED_QUERIES = new String[] { "bool", "boosting", @@ -242,4 +304,120 @@ public class SearchModuleTests extends ModuleTestCase { "match_fuzzy", "mlt" }; + + /** + * Dummy test {@link AggregationBuilder} used to test registering aggregation builders. + */ + private static class TestAggregationBuilder extends ValuesSourceAggregationBuilder { + /** + * Read from a stream. + */ + protected TestAggregationBuilder(StreamInput in) throws IOException { + super(in, null, null); + } + + @Override + public String getWriteableName() { + return "test"; + } + + @Override + protected void innerWriteTo(StreamOutput out) throws IOException { + } + + @Override + protected ValuesSourceAggregatorFactory innerBuild(AggregationContext context, + ValuesSourceConfig config, AggregatorFactory parent, Builder subFactoriesBuilder) throws IOException { + return null; + } + + @Override + protected XContentBuilder doXContentBody(XContentBuilder builder, Params params) throws IOException { + return null; + } + + @Override + protected int innerHashCode() { + return 0; + } + + @Override + protected boolean innerEquals(Object obj) { + return false; + } + + private static TestAggregationBuilder fromXContent(String name, QueryParseContext c) { + return null; + } + } + + /** + * Dummy test {@link PipelineAggregator} used to test registering aggregation builders. + */ + private static class TestPipelineAggregationBuilder extends AbstractPipelineAggregationBuilder { + /** + * Read from a stream. + */ + public TestPipelineAggregationBuilder(StreamInput in) throws IOException { + super(in, "test"); + } + + @Override + public String getWriteableName() { + return "test"; + } + + @Override + protected void doWriteTo(StreamOutput out) throws IOException { + } + + @Override + protected PipelineAggregator createInternal(Map metaData) throws IOException { + return null; + } + + @Override + protected XContentBuilder internalXContent(XContentBuilder builder, Params params) throws IOException { + return null; + } + + @Override + protected int doHashCode() { + return 0; + } + + @Override + protected boolean doEquals(Object obj) { + return false; + } + + private static TestPipelineAggregationBuilder fromXContent(String name, QueryParseContext c) { + return null; + } + } + + /** + * Dummy test {@link PipelineAggregator} used to test registering aggregation builders. + */ + private static class TestPipelineAggregator extends PipelineAggregator { + /** + * Read from a stream. + */ + public TestPipelineAggregator(StreamInput in) throws IOException { + super(in); + } + @Override + public String getWriteableName() { + return "test"; + } + + @Override + protected void doWriteTo(StreamOutput out) throws IOException { + } + + @Override + public InternalAggregation reduce(InternalAggregation aggregation, ReduceContext reduceContext) { + return null; + } + } } diff --git a/core/src/test/java/org/elasticsearch/search/aggregations/AggregationCollectorTests.java b/core/src/test/java/org/elasticsearch/search/aggregations/AggregationCollectorTests.java index f2b16c3e3ee..44f1adccc8b 100644 --- a/core/src/test/java/org/elasticsearch/search/aggregations/AggregationCollectorTests.java +++ b/core/src/test/java/org/elasticsearch/search/aggregations/AggregationCollectorTests.java @@ -25,6 +25,7 @@ import org.elasticsearch.common.xcontent.json.JsonXContent; import org.elasticsearch.index.IndexService; import org.elasticsearch.index.query.QueryParseContext; import org.elasticsearch.indices.query.IndicesQueriesRegistry; +import org.elasticsearch.search.SearchRequestParsers; import org.elasticsearch.search.aggregations.support.AggregationContext; import org.elasticsearch.search.internal.SearchContext; import org.elasticsearch.test.ESSingleNodeTestCase; @@ -60,7 +61,7 @@ public class AggregationCollectorTests extends ESSingleNodeTestCase { } private boolean needsScores(IndexService index, String agg) throws IOException { - AggregatorParsers parser = getInstanceFromNode(AggregatorParsers.class); + AggregatorParsers parser = getInstanceFromNode(SearchRequestParsers.class).aggParsers; IndicesQueriesRegistry queriesRegistry = getInstanceFromNode(IndicesQueriesRegistry.class); XContentParser aggParser = JsonXContent.jsonXContent.createParser(agg); QueryParseContext parseContext = new QueryParseContext(queriesRegistry, aggParser, ParseFieldMatcher.STRICT); diff --git a/core/src/test/java/org/elasticsearch/search/aggregations/AggregatorParsingTests.java b/core/src/test/java/org/elasticsearch/search/aggregations/AggregatorParsingTests.java index c3581fb6622..a7476381b8e 100644 --- a/core/src/test/java/org/elasticsearch/search/aggregations/AggregatorParsingTests.java +++ b/core/src/test/java/org/elasticsearch/search/aggregations/AggregatorParsingTests.java @@ -38,6 +38,7 @@ import org.elasticsearch.common.xcontent.XContentParser; import org.elasticsearch.common.xcontent.json.JsonXContent; import org.elasticsearch.env.Environment; import org.elasticsearch.index.Index; +import org.elasticsearch.search.SearchRequestParsers; import org.elasticsearch.test.AbstractQueryTestCase; import org.elasticsearch.index.query.QueryParseContext; import org.elasticsearch.indices.IndicesModule; @@ -144,7 +145,7 @@ public class AggregatorParsingTests extends ESTestCase { bind(NamedWriteableRegistry.class).toInstance(namedWriteableRegistry); } }).createInjector(); - aggParsers = injector.getInstance(AggregatorParsers.class); + aggParsers = injector.getInstance(SearchRequestParsers.class).aggParsers; // create some random type with some default field, those types will // stick around for all of the subclasses currentTypes = new String[randomIntBetween(0, 5)]; diff --git a/core/src/test/java/org/elasticsearch/search/aggregations/BaseAggregationTestCase.java b/core/src/test/java/org/elasticsearch/search/aggregations/BaseAggregationTestCase.java index 076f6c5fb66..b9c0e5f09c1 100644 --- a/core/src/test/java/org/elasticsearch/search/aggregations/BaseAggregationTestCase.java +++ b/core/src/test/java/org/elasticsearch/search/aggregations/BaseAggregationTestCase.java @@ -42,6 +42,7 @@ import org.elasticsearch.common.xcontent.XContentParser; import org.elasticsearch.common.xcontent.XContentType; import org.elasticsearch.env.Environment; import org.elasticsearch.index.Index; +import org.elasticsearch.search.SearchRequestParsers; import org.elasticsearch.test.AbstractQueryTestCase; import org.elasticsearch.index.query.QueryParseContext; import org.elasticsearch.indices.IndicesModule; @@ -106,7 +107,7 @@ public abstract class BaseAggregationTestCase SearchSourceBuilder.fromXContent(createParseContext(parser), aggParsers, suggesters)); + () -> SearchSourceBuilder.fromXContent(createParseContext(parser), + searchRequestParsers.aggParsers, searchRequestParsers.suggesters)); assertThat(e, hasToString(containsString("unit is missing or unrecognized"))); } } diff --git a/core/src/test/java/org/elasticsearch/tasks/ListTasksResponseTests.java b/core/src/test/java/org/elasticsearch/tasks/ListTasksResponseTests.java index 433f3bdd16a..6643a71b096 100644 --- a/core/src/test/java/org/elasticsearch/tasks/ListTasksResponseTests.java +++ b/core/src/test/java/org/elasticsearch/tasks/ListTasksResponseTests.java @@ -20,29 +20,23 @@ package org.elasticsearch.tasks; import org.elasticsearch.action.admin.cluster.node.tasks.list.ListTasksResponse; -import org.elasticsearch.cluster.node.DiscoveryNodes; import org.elasticsearch.test.ESTestCase; -import org.hamcrest.Matchers; -import java.util.Collections; +import static java.util.Collections.emptyList; +import static java.util.Collections.singletonList; public class ListTasksResponseTests extends ESTestCase { - public void testToStringNoTask() { - ListTasksResponse tasksResponse = new ListTasksResponse(); - String string = tasksResponse.toString(); - assertThat(string, Matchers.containsString("nodes")); + public void testEmptyToString() { + assertEquals("{\"tasks\":{}}", new ListTasksResponse().toString()); } - public void testToString() { + public void testNonEmptyToString() { TaskInfo info = new TaskInfo( new TaskId("node1", 1), "dummy-type", "dummy-action", "dummy-description", null, 0, 1, true, new TaskId("node1", 0)); - - DiscoveryNodes nodes = DiscoveryNodes.builder().build(); - ListTasksResponse tasksResponse = new ListTasksResponse(Collections.singletonList(info), Collections.emptyList(), - Collections.emptyList(), nodes); - - String string = tasksResponse.toString(); - assertThat(string, Matchers.containsString("\"type\":\"dummy-type\"")); + ListTasksResponse tasksResponse = new ListTasksResponse(singletonList(info), emptyList(), emptyList()); + assertEquals("{\"tasks\":{\"node1:1\":{\"node\":\"node1\",\"id\":1,\"type\":\"dummy-type\",\"action\":\"dummy-action\"," + + "\"description\":\"dummy-description\",\"start_time_in_millis\":0,\"running_time_in_nanos\":1,\"cancellable\":true," + + "\"parent_task_id\":\"node1:0\"}}}", tasksResponse.toString()); } } diff --git a/docs/plugins/mapper-attachments.asciidoc b/docs/plugins/mapper-attachments.asciidoc index a35d9efe614..b5ceef39ae8 100644 --- a/docs/plugins/mapper-attachments.asciidoc +++ b/docs/plugins/mapper-attachments.asciidoc @@ -45,7 +45,7 @@ Create a property mapping using the new type `attachment`: [source,js] -------------------------- -POST /trying-out-mapper-attachments +PUT /trying-out-mapper-attachments { "mappings": { "person": { diff --git a/docs/reference/indices/aliases.asciidoc b/docs/reference/indices/aliases.asciidoc index 0a54be7cd8d..294bbd32a54 100644 --- a/docs/reference/indices/aliases.asciidoc +++ b/docs/reference/indices/aliases.asciidoc @@ -22,7 +22,7 @@ POST /_aliases } -------------------------------------------------- // CONSOLE -// TEST[s/^/PUT test1\n/] +// TEST[s/^/PUT test1\nPUT test2\n/] An alias can also be removed, for example: @@ -48,7 +48,7 @@ POST /_aliases { "actions" : [ { "remove" : { "index" : "test1", "alias" : "alias1" } }, - { "add" : { "index" : "test1", "alias" : "alias2" } } + { "add" : { "index" : "test2", "alias" : "alias1" } } ] } -------------------------------------------------- diff --git a/docs/reference/indices/create-index.asciidoc b/docs/reference/indices/create-index.asciidoc index bfd2ba2b791..a04fe0ca3f6 100644 --- a/docs/reference/indices/create-index.asciidoc +++ b/docs/reference/indices/create-index.asciidoc @@ -14,15 +14,17 @@ associated with it. [source,js] -------------------------------------------------- -$ curl -XPUT 'http://localhost:9200/twitter/' -d '{ +PUT twitter +{ "settings" : { "index" : { "number_of_shards" : 3, <1> "number_of_replicas" : 2 <2> } } -}' +} -------------------------------------------------- +// CONSOLE <1> Default for `number_of_shards` is 5 <2> Default for `number_of_replicas` is 1 (ie one replica for each primary shard) @@ -33,27 +35,31 @@ index settings can also be defined with http://www.json.org[JSON]: [source,js] -------------------------------------------------- -$ curl -XPUT 'http://localhost:9200/twitter/' -d '{ +PUT twitter +{ "settings" : { "index" : { "number_of_shards" : 3, "number_of_replicas" : 2 } } -}' +} -------------------------------------------------- +// CONSOLE or more simplified [source,js] -------------------------------------------------- -$ curl -XPUT 'http://localhost:9200/twitter/' -d '{ +PUT twitter +{ "settings" : { "number_of_shards" : 3, "number_of_replicas" : 2 } -}' +} -------------------------------------------------- +// CONSOLE [NOTE] You do not have to explicitly specify `index` section inside the @@ -72,7 +78,8 @@ The create index API allows to provide a set of one or more mappings: [source,js] -------------------------------------------------- -curl -XPOST localhost:9200/test -d '{ +PUT test +{ "settings" : { "number_of_shards" : 1 }, @@ -83,8 +90,9 @@ curl -XPOST localhost:9200/test -d '{ } } } -}' +} -------------------------------------------------- +// CONSOLE [float] [[create-index-aliases]] @@ -94,7 +102,8 @@ The create index API allows also to provide a set of <> [source,js] -------------------------------------------------- -curl -XPUT localhost:9200/test -d '{ +PUT test +{ "aliases" : { "alias_1" : {}, "alias_2" : { @@ -104,24 +113,9 @@ curl -XPUT localhost:9200/test -d '{ "routing" : "kimchy" } } -}' +} -------------------------------------------------- - -[float] -=== Creation Date - -When an index is created, a timestamp is stored in the index metadata for the creation date. By -default this is automatically generated but it can also be specified using the -`creation_date` parameter on the create index API: - -[source,js] --------------------------------------------------- -curl -XPUT localhost:9200/test -d '{ - "creation_date" : 1407751337000 <1> -}' --------------------------------------------------- - -<1> `creation_date` is set using epoch time in milliseconds. +// CONSOLE [float] [[create-index-wait-for-active-shards]] @@ -138,6 +132,7 @@ what happened: "shards_acknowledged": true } -------------------------------------------------- +// TESTRESPONSE `acknowledged` indicates whether the index was successfully created in the cluster, while `shards_acknowledged` indices whether the requisite number of shard copies were started for @@ -156,19 +151,24 @@ the `wait_for_active_shards` value on all subsequent write operations): [source,js] -------------------------------------------------- -curl -XPUT localhost:9200/test -d '{ +PUT test +{ "settings": { "index.write.wait_for_active_shards": "2" } } -------------------------------------------------- +// CONSOLE +// TEST[skip:requires two nodes] or through the request parameter `wait_for_active_shards`: [source,js] -------------------------------------------------- -curl -XPUT localhost:9200/test?wait_for_active_shards=2 +PUT test?wait_for_active_shards=2 -------------------------------------------------- +// CONSOLE +// TEST[skip:requires two nodes] A detailed explanation of `wait_for_active_shards` and its possible values can be found <>. diff --git a/docs/reference/migration/migrate_5_0/rest.asciidoc b/docs/reference/migration/migrate_5_0/rest.asciidoc index 9d135c4d5bf..36f4071ce15 100644 --- a/docs/reference/migration/migrate_5_0/rest.asciidoc +++ b/docs/reference/migration/migrate_5_0/rest.asciidoc @@ -15,6 +15,11 @@ endpoint should be used in lieu of optimize. The `GET` HTTP verb for `/_forcemerge` is no longer supported, please use the `POST` HTTP verb. +==== Index creation endpoint only accepts `PUT` + +It used to be possible to create an index by either calling `PUT index_name` +or `POST index_name`. Only the former is now supported. + ==== Removed `mem` section from `/_cluster/stats` response The `mem` section contained only one value, the total memory available diff --git a/docs/reference/search/suggesters/phrase-suggest.asciidoc b/docs/reference/search/suggesters/phrase-suggest.asciidoc index 487075a5677..03cbb5af35c 100644 --- a/docs/reference/search/suggesters/phrase-suggest.asciidoc +++ b/docs/reference/search/suggesters/phrase-suggest.asciidoc @@ -23,7 +23,7 @@ work. The `reverse` analyzer is used only in the last example. [source,js] -------------------------------------------------- -POST test +PUT test { "settings": { "index": { diff --git a/modules/aggs-matrix-stats/src/main/java/org/elasticsearch/search/aggregations/matrix/MatrixAggregationPlugin.java b/modules/aggs-matrix-stats/src/main/java/org/elasticsearch/search/aggregations/matrix/MatrixAggregationPlugin.java index 0b636b7abe2..a712371fa10 100644 --- a/modules/aggs-matrix-stats/src/main/java/org/elasticsearch/search/aggregations/matrix/MatrixAggregationPlugin.java +++ b/modules/aggs-matrix-stats/src/main/java/org/elasticsearch/search/aggregations/matrix/MatrixAggregationPlugin.java @@ -20,15 +20,19 @@ package org.elasticsearch.search.aggregations.matrix; import org.elasticsearch.plugins.Plugin; -import org.elasticsearch.search.SearchModule; -import org.elasticsearch.search.SearchModule.AggregationSpec; +import org.elasticsearch.plugins.SearchPlugin; import org.elasticsearch.search.aggregations.matrix.stats.InternalMatrixStats; import org.elasticsearch.search.aggregations.matrix.stats.MatrixStatsAggregationBuilder; import org.elasticsearch.search.aggregations.matrix.stats.MatrixStatsParser; -public class MatrixAggregationPlugin extends Plugin { - public void onModule(SearchModule searchModule) { - searchModule.registerAggregation(new AggregationSpec(MatrixStatsAggregationBuilder::new, new MatrixStatsParser(), - MatrixStatsAggregationBuilder.AGGREGATION_NAME_FIELD).addResultReader(InternalMatrixStats::new)); +import java.util.List; + +import static java.util.Collections.singletonList; + +public class MatrixAggregationPlugin extends Plugin implements SearchPlugin { + @Override + public List getAggregations() { + return singletonList(new AggregationSpec(MatrixStatsAggregationBuilder.NAME, MatrixStatsAggregationBuilder::new, + new MatrixStatsParser()).addResultReader(InternalMatrixStats::new)); } } diff --git a/modules/aggs-matrix-stats/src/main/java/org/elasticsearch/search/aggregations/matrix/stats/MatrixStatsAggregationBuilder.java b/modules/aggs-matrix-stats/src/main/java/org/elasticsearch/search/aggregations/matrix/stats/MatrixStatsAggregationBuilder.java index e32c7d3ad73..867bcdbdfa5 100644 --- a/modules/aggs-matrix-stats/src/main/java/org/elasticsearch/search/aggregations/matrix/stats/MatrixStatsAggregationBuilder.java +++ b/modules/aggs-matrix-stats/src/main/java/org/elasticsearch/search/aggregations/matrix/stats/MatrixStatsAggregationBuilder.java @@ -18,7 +18,6 @@ */ package org.elasticsearch.search.aggregations.matrix.stats; -import org.elasticsearch.common.ParseField; import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.io.stream.StreamOutput; import org.elasticsearch.common.xcontent.ToXContent; @@ -29,11 +28,11 @@ import org.elasticsearch.search.aggregations.AggregatorFactory; import org.elasticsearch.search.aggregations.InternalAggregation.Type; import org.elasticsearch.search.aggregations.support.AggregationContext; import org.elasticsearch.search.aggregations.support.MultiValuesSourceAggregationBuilder; +import org.elasticsearch.search.aggregations.support.ValueType; import org.elasticsearch.search.aggregations.support.ValuesSource; +import org.elasticsearch.search.aggregations.support.ValuesSource.Numeric; import org.elasticsearch.search.aggregations.support.ValuesSourceConfig; import org.elasticsearch.search.aggregations.support.ValuesSourceType; -import org.elasticsearch.search.aggregations.support.ValueType; -import org.elasticsearch.search.aggregations.support.ValuesSource.Numeric; import java.io.IOException; import java.util.Map; @@ -42,7 +41,6 @@ public class MatrixStatsAggregationBuilder extends MultiValuesSourceAggregationBuilder.LeafOnly { public static final String NAME = "matrix_stats"; public static final Type TYPE = new Type(NAME); - public static final ParseField AGGREGATION_NAME_FIELD = new ParseField(NAME); private MultiValueMode multiValueMode = MultiValueMode.AVG; diff --git a/modules/lang-mustache/src/main/java/org/elasticsearch/script/mustache/RestSearchTemplateAction.java b/modules/lang-mustache/src/main/java/org/elasticsearch/script/mustache/RestSearchTemplateAction.java index cbac185deeb..18637b15e84 100644 --- a/modules/lang-mustache/src/main/java/org/elasticsearch/script/mustache/RestSearchTemplateAction.java +++ b/modules/lang-mustache/src/main/java/org/elasticsearch/script/mustache/RestSearchTemplateAction.java @@ -43,6 +43,7 @@ import org.elasticsearch.rest.action.RestActions; import org.elasticsearch.rest.action.RestStatusToXContentListener; import org.elasticsearch.rest.action.search.RestSearchAction; import org.elasticsearch.script.ScriptService; +import org.elasticsearch.search.SearchRequestParsers; import org.elasticsearch.search.aggregations.AggregatorParsers; import org.elasticsearch.search.suggest.Suggesters; @@ -81,17 +82,12 @@ public class RestSearchTemplateAction extends BaseRestHandler { }, new ParseField("inline", "template"), ObjectParser.ValueType.OBJECT_OR_STRING); } - private final IndicesQueriesRegistry queryRegistry; - private final AggregatorParsers aggParsers; - private final Suggesters suggesters; + private final SearchRequestParsers searchRequestParsers; @Inject - public RestSearchTemplateAction(Settings settings, RestController controller, IndicesQueriesRegistry queryRegistry, - AggregatorParsers aggregatorParsers, Suggesters suggesters) { + public RestSearchTemplateAction(Settings settings, RestController controller, SearchRequestParsers searchRequestParsers) { super(settings); - this.queryRegistry = queryRegistry; - this.aggParsers = aggregatorParsers; - this.suggesters = suggesters; + this.searchRequestParsers = searchRequestParsers; controller.registerHandler(GET, "/_search/template", this); controller.registerHandler(POST, "/_search/template", this); @@ -109,7 +105,7 @@ public class RestSearchTemplateAction extends BaseRestHandler { // Creates the search request with all required params SearchRequest searchRequest = new SearchRequest(); - RestSearchAction.parseSearchRequest(searchRequest, queryRegistry, request, parseFieldMatcher, aggParsers, suggesters, null); + RestSearchAction.parseSearchRequest(searchRequest, request, searchRequestParsers, parseFieldMatcher, null); // Creates the search template request SearchTemplateRequest searchTemplateRequest = parse(RestActions.getRestContent(request)); diff --git a/modules/lang-mustache/src/main/java/org/elasticsearch/script/mustache/TransportSearchTemplateAction.java b/modules/lang-mustache/src/main/java/org/elasticsearch/script/mustache/TransportSearchTemplateAction.java index ded396deb12..473e287c9a2 100644 --- a/modules/lang-mustache/src/main/java/org/elasticsearch/script/mustache/TransportSearchTemplateAction.java +++ b/modules/lang-mustache/src/main/java/org/elasticsearch/script/mustache/TransportSearchTemplateAction.java @@ -36,6 +36,7 @@ import org.elasticsearch.indices.query.IndicesQueriesRegistry; import org.elasticsearch.script.ExecutableScript; import org.elasticsearch.script.Script; import org.elasticsearch.script.ScriptService; +import org.elasticsearch.search.SearchRequestParsers; import org.elasticsearch.search.aggregations.AggregatorParsers; import org.elasticsearch.search.builder.SearchSourceBuilder; import org.elasticsearch.search.suggest.Suggesters; @@ -51,22 +52,17 @@ public class TransportSearchTemplateAction extends HandledTransportAction() { diff --git a/modules/percolator/src/main/java/org/elasticsearch/percolator/TransportMultiPercolateAction.java b/modules/percolator/src/main/java/org/elasticsearch/percolator/TransportMultiPercolateAction.java index 0bd8b15bfb7..44914e140b1 100644 --- a/modules/percolator/src/main/java/org/elasticsearch/percolator/TransportMultiPercolateAction.java +++ b/modules/percolator/src/main/java/org/elasticsearch/percolator/TransportMultiPercolateAction.java @@ -38,6 +38,7 @@ import org.elasticsearch.common.collect.Tuple; import org.elasticsearch.common.inject.Inject; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.indices.query.IndicesQueriesRegistry; +import org.elasticsearch.search.SearchRequestParsers; import org.elasticsearch.search.aggregations.AggregatorParsers; import org.elasticsearch.threadpool.ThreadPool; import org.elasticsearch.transport.TransportService; @@ -54,18 +55,17 @@ public class TransportMultiPercolateAction extends HandledTransportAction listener) { SearchRequest searchRequest; try { - searchRequest = createSearchRequest(request, docSource, queryRegistry, aggParsers, parseFieldMatcher); + searchRequest = createSearchRequest(request, docSource, searchRequestParsers.queryParsers, + searchRequestParsers.aggParsers, parseFieldMatcher); } catch (IOException e) { listener.onFailure(e); return; diff --git a/modules/reindex/src/main/java/org/elasticsearch/index/reindex/AbstractBaseReindexRestHandler.java b/modules/reindex/src/main/java/org/elasticsearch/index/reindex/AbstractBaseReindexRestHandler.java index ef14f6fa66b..faa2607ce20 100644 --- a/modules/reindex/src/main/java/org/elasticsearch/index/reindex/AbstractBaseReindexRestHandler.java +++ b/modules/reindex/src/main/java/org/elasticsearch/index/reindex/AbstractBaseReindexRestHandler.java @@ -32,6 +32,7 @@ import org.elasticsearch.rest.BytesRestResponse; import org.elasticsearch.rest.RestChannel; import org.elasticsearch.rest.RestRequest; import org.elasticsearch.rest.RestStatus; +import org.elasticsearch.search.SearchRequestParsers; import org.elasticsearch.search.aggregations.AggregatorParsers; import org.elasticsearch.search.suggest.Suggesters; import org.elasticsearch.tasks.LoggingTaskListener; @@ -46,19 +47,14 @@ public abstract class AbstractBaseReindexRestHandler< A extends GenericAction > extends BaseRestHandler { - protected final IndicesQueriesRegistry indicesQueriesRegistry; - protected final AggregatorParsers aggParsers; - protected final Suggesters suggesters; + protected final SearchRequestParsers searchRequestParsers; private final ClusterService clusterService; private final A action; - protected AbstractBaseReindexRestHandler(Settings settings, IndicesQueriesRegistry indicesQueriesRegistry, - AggregatorParsers aggParsers, Suggesters suggesters, + protected AbstractBaseReindexRestHandler(Settings settings, SearchRequestParsers searchRequestParsers, ClusterService clusterService, A action) { super(settings); - this.indicesQueriesRegistry = indicesQueriesRegistry; - this.aggParsers = aggParsers; - this.suggesters = suggesters; + this.searchRequestParsers = searchRequestParsers; this.clusterService = clusterService; this.action = action; } diff --git a/modules/reindex/src/main/java/org/elasticsearch/index/reindex/AbstractBulkByQueryRestHandler.java b/modules/reindex/src/main/java/org/elasticsearch/index/reindex/AbstractBulkByQueryRestHandler.java index d81b5e054f4..2db5f4123cf 100644 --- a/modules/reindex/src/main/java/org/elasticsearch/index/reindex/AbstractBulkByQueryRestHandler.java +++ b/modules/reindex/src/main/java/org/elasticsearch/index/reindex/AbstractBulkByQueryRestHandler.java @@ -19,6 +19,10 @@ package org.elasticsearch.index.reindex; +import java.io.IOException; +import java.util.Map; +import java.util.function.Consumer; + import org.elasticsearch.action.GenericAction; import org.elasticsearch.action.search.SearchRequest; import org.elasticsearch.cluster.service.ClusterService; @@ -29,16 +33,10 @@ import org.elasticsearch.common.xcontent.XContentBuilder; import org.elasticsearch.common.xcontent.XContentFactory; import org.elasticsearch.common.xcontent.XContentHelper; import org.elasticsearch.common.xcontent.XContentType; -import org.elasticsearch.indices.query.IndicesQueriesRegistry; import org.elasticsearch.rest.RestRequest; import org.elasticsearch.rest.action.RestActions; import org.elasticsearch.rest.action.search.RestSearchAction; -import org.elasticsearch.search.aggregations.AggregatorParsers; -import org.elasticsearch.search.suggest.Suggesters; - -import java.io.IOException; -import java.util.Map; -import java.util.function.Consumer; +import org.elasticsearch.search.SearchRequestParsers; import static org.elasticsearch.index.reindex.AbstractBulkByScrollRequest.SIZE_ALL_MATCHES; @@ -49,10 +47,9 @@ public abstract class AbstractBulkByQueryRestHandler< Request extends AbstractBulkByScrollRequest, A extends GenericAction> extends AbstractBaseReindexRestHandler { - protected AbstractBulkByQueryRestHandler(Settings settings, IndicesQueriesRegistry indicesQueriesRegistry, - AggregatorParsers aggParsers, Suggesters suggesters, ClusterService clusterService, - A action) { - super(settings, indicesQueriesRegistry, aggParsers, suggesters, clusterService, action); + protected AbstractBulkByQueryRestHandler(Settings settings, SearchRequestParsers searchRequestParsers, + ClusterService clusterService, A action) { + super(settings, searchRequestParsers, clusterService, action); } protected void parseInternalRequest(Request internal, RestRequest restRequest, @@ -111,7 +108,6 @@ public abstract class AbstractBulkByQueryRestHandler< } } - RestSearchAction.parseSearchRequest(searchRequest, indicesQueriesRegistry, restRequest, parseFieldMatcher, aggParsers, - suggesters, content); + RestSearchAction.parseSearchRequest(searchRequest, restRequest, searchRequestParsers, parseFieldMatcher, content); } } diff --git a/modules/reindex/src/main/java/org/elasticsearch/index/reindex/ReindexPlugin.java b/modules/reindex/src/main/java/org/elasticsearch/index/reindex/ReindexPlugin.java index 4f2cb2578ac..fe7bcb1f85b 100644 --- a/modules/reindex/src/main/java/org/elasticsearch/index/reindex/ReindexPlugin.java +++ b/modules/reindex/src/main/java/org/elasticsearch/index/reindex/ReindexPlugin.java @@ -21,11 +21,12 @@ package org.elasticsearch.index.reindex; import org.elasticsearch.action.ActionRequest; import org.elasticsearch.action.ActionResponse; -import org.elasticsearch.common.network.NetworkModule; -import org.elasticsearch.plugins.ActionPlugin; +import org.elasticsearch.common.io.stream.NamedWriteableRegistry; import org.elasticsearch.common.settings.Setting; +import org.elasticsearch.plugins.ActionPlugin; import org.elasticsearch.plugins.Plugin; import org.elasticsearch.rest.RestHandler; +import org.elasticsearch.tasks.Task; import java.util.Arrays; import java.util.List; @@ -43,16 +44,18 @@ public class ReindexPlugin extends Plugin implements ActionPlugin { new ActionHandler<>(RethrottleAction.INSTANCE, TransportRethrottleAction.class)); } + @Override + public List getNamedWriteables() { + return singletonList( + new NamedWriteableRegistry.Entry(Task.Status.class, BulkByScrollTask.Status.NAME, BulkByScrollTask.Status::new)); + } + @Override public List> getRestHandlers() { return Arrays.asList(RestReindexAction.class, RestUpdateByQueryAction.class, RestDeleteByQueryAction.class, RestRethrottleAction.class); } - public void onModule(NetworkModule networkModule) { - networkModule.registerTaskStatus(BulkByScrollTask.Status.NAME, BulkByScrollTask.Status::new); - } - @Override public List> getSettings() { return singletonList(TransportReindexAction.REMOTE_CLUSTER_WHITELIST); diff --git a/modules/reindex/src/main/java/org/elasticsearch/index/reindex/RestDeleteByQueryAction.java b/modules/reindex/src/main/java/org/elasticsearch/index/reindex/RestDeleteByQueryAction.java index 704bcf2dbaf..92a96a880eb 100644 --- a/modules/reindex/src/main/java/org/elasticsearch/index/reindex/RestDeleteByQueryAction.java +++ b/modules/reindex/src/main/java/org/elasticsearch/index/reindex/RestDeleteByQueryAction.java @@ -29,6 +29,7 @@ import org.elasticsearch.indices.query.IndicesQueriesRegistry; import org.elasticsearch.rest.RestChannel; import org.elasticsearch.rest.RestController; import org.elasticsearch.rest.RestRequest; +import org.elasticsearch.search.SearchRequestParsers; import org.elasticsearch.search.aggregations.AggregatorParsers; import org.elasticsearch.search.suggest.Suggesters; @@ -43,9 +44,8 @@ public class RestDeleteByQueryAction extends AbstractBulkByQueryRestHandler listener = nodeSettingListener(clusterService, new RestToXContentListener<>(channel)); - client.execute(RethrottleAction.INSTANCE, internalRequest, listener); + client.execute(RethrottleAction.INSTANCE, internalRequest, listTasksResponseListener(clusterService, channel)); } } diff --git a/modules/reindex/src/main/java/org/elasticsearch/index/reindex/RestUpdateByQueryAction.java b/modules/reindex/src/main/java/org/elasticsearch/index/reindex/RestUpdateByQueryAction.java index a858986ccab..06adb76df78 100644 --- a/modules/reindex/src/main/java/org/elasticsearch/index/reindex/RestUpdateByQueryAction.java +++ b/modules/reindex/src/main/java/org/elasticsearch/index/reindex/RestUpdateByQueryAction.java @@ -32,6 +32,7 @@ import org.elasticsearch.rest.RestController; import org.elasticsearch.rest.RestRequest; import org.elasticsearch.script.Script; import org.elasticsearch.script.ScriptService; +import org.elasticsearch.search.SearchRequestParsers; import org.elasticsearch.search.aggregations.AggregatorParsers; import org.elasticsearch.search.suggest.Suggesters; @@ -48,9 +49,8 @@ public class RestUpdateByQueryAction extends AbstractBulkByQueryRestHandler new ListTasksResponse(clusterService.state().nodes()), - ThreadPool.Names.MANAGEMENT); + super(settings, RethrottleAction.NAME, threadPool, clusterService, transportService, actionFilters, indexNameExpressionResolver, + RethrottleRequest::new, ListTasksResponse::new, ThreadPool.Names.MANAGEMENT); } @Override @@ -60,7 +59,7 @@ public class TransportRethrottleAction extends TransportTasksAction tasks, List taskOperationFailures, List failedNodeExceptions) { - return new ListTasksResponse(tasks, taskOperationFailures, failedNodeExceptions, clusterService.state().nodes()); + return new ListTasksResponse(tasks, taskOperationFailures, failedNodeExceptions); } @Override diff --git a/modules/reindex/src/main/java/org/elasticsearch/index/reindex/UpdateByQueryRequest.java b/modules/reindex/src/main/java/org/elasticsearch/index/reindex/UpdateByQueryRequest.java index aa4b4fc0ac5..3401ce4582b 100644 --- a/modules/reindex/src/main/java/org/elasticsearch/index/reindex/UpdateByQueryRequest.java +++ b/modules/reindex/src/main/java/org/elasticsearch/index/reindex/UpdateByQueryRequest.java @@ -22,6 +22,10 @@ package org.elasticsearch.index.reindex; import org.elasticsearch.action.IndicesRequest; import org.elasticsearch.action.search.SearchRequest; import org.elasticsearch.action.support.IndicesOptions; +import org.elasticsearch.common.io.stream.StreamInput; +import org.elasticsearch.common.io.stream.StreamOutput; + +import java.io.IOException; /** * Request to update some documents. That means you can't change their type, id, index, or anything like that. This implements @@ -89,4 +93,16 @@ public class UpdateByQueryRequest extends AbstractBulkIndexByScrollRequest> nodePlugins() { return Arrays.asList(ReindexPlugin.class); } + @Override + protected Collection> transportClientPlugins() { + return Arrays.asList(ReindexPlugin.class); + } + protected ReindexRequestBuilder reindex() { return ReindexAction.INSTANCE.newRequestBuilder(client()); } diff --git a/modules/reindex/src/test/java/org/elasticsearch/index/reindex/RestReindexActionTests.java b/modules/reindex/src/test/java/org/elasticsearch/index/reindex/RestReindexActionTests.java index b27ecfa3eb1..f8cbee17329 100644 --- a/modules/reindex/src/test/java/org/elasticsearch/index/reindex/RestReindexActionTests.java +++ b/modules/reindex/src/test/java/org/elasticsearch/index/reindex/RestReindexActionTests.java @@ -29,6 +29,7 @@ import org.elasticsearch.common.xcontent.json.JsonXContent; import org.elasticsearch.index.reindex.RestReindexAction.ReindexParseContext; import org.elasticsearch.index.reindex.remote.RemoteInfo; import org.elasticsearch.indices.query.IndicesQueriesRegistry; +import org.elasticsearch.search.SearchRequestParsers; import org.elasticsearch.test.ESTestCase; import java.io.IOException; @@ -109,8 +110,8 @@ public class RestReindexActionTests extends ESTestCase { } try (XContentParser p = JsonXContent.jsonXContent.createParser(request)) { ReindexRequest r = new ReindexRequest(new SearchRequest(), new IndexRequest()); - RestReindexAction.PARSER.parse(p, r, - new ReindexParseContext(new IndicesQueriesRegistry(), null, null, ParseFieldMatcher.STRICT)); + SearchRequestParsers searchParsers = new SearchRequestParsers(new IndicesQueriesRegistry(), null, null); + RestReindexAction.PARSER.parse(p, r, new ReindexParseContext(searchParsers, ParseFieldMatcher.STRICT)); assertEquals("localhost", r.getRemoteInfo().getHost()); assertArrayEquals(new String[] {"source"}, r.getSearchRequest().indices()); } diff --git a/modules/reindex/src/test/java/org/elasticsearch/index/reindex/RoundTripTests.java b/modules/reindex/src/test/java/org/elasticsearch/index/reindex/RoundTripTests.java index 93f730d3b53..b0fc9b428ba 100644 --- a/modules/reindex/src/test/java/org/elasticsearch/index/reindex/RoundTripTests.java +++ b/modules/reindex/src/test/java/org/elasticsearch/index/reindex/RoundTripTests.java @@ -91,9 +91,13 @@ public class RoundTripTests extends ESTestCase { public void testUpdateByQueryRequest() throws IOException { UpdateByQueryRequest update = new UpdateByQueryRequest(new SearchRequest()); randomRequest(update); + if (randomBoolean()) { + update.setPipeline(randomAsciiOfLength(5)); + } UpdateByQueryRequest tripped = new UpdateByQueryRequest(); roundTrip(update, tripped); assertRequestEquals(update, tripped); + assertEquals(update.getPipeline(), tripped.getPipeline()); } private void randomRequest(AbstractBulkIndexByScrollRequest request) { diff --git a/rest-api-spec/src/main/resources/rest-api-spec/api/indices.create.json b/rest-api-spec/src/main/resources/rest-api-spec/api/indices.create.json index b0267eae3a3..1433c893e25 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/api/indices.create.json +++ b/rest-api-spec/src/main/resources/rest-api-spec/api/indices.create.json @@ -1,7 +1,7 @@ { "indices.create": { "documentation": "http://www.elastic.co/guide/en/elasticsearch/reference/master/indices-create-index.html", - "methods": ["PUT", "POST"], + "methods": ["PUT"], "url": { "path": "/{index}", "paths": ["/{index}"], diff --git a/test/framework/src/main/java/org/elasticsearch/node/MockNode.java b/test/framework/src/main/java/org/elasticsearch/node/MockNode.java index 33077d5ffbd..37da93b8257 100644 --- a/test/framework/src/main/java/org/elasticsearch/node/MockNode.java +++ b/test/framework/src/main/java/org/elasticsearch/node/MockNode.java @@ -25,8 +25,12 @@ import org.elasticsearch.common.util.MockBigArrays; import org.elasticsearch.indices.breaker.CircuitBreakerService; import org.elasticsearch.node.internal.InternalSettingsPreparer; import org.elasticsearch.plugins.Plugin; +import org.elasticsearch.plugins.SearchPlugin; +import org.elasticsearch.search.MockSearchService; +import org.elasticsearch.search.SearchService; import java.util.Collection; +import java.util.List; /** * A node for testing which allows: @@ -36,26 +40,33 @@ import java.util.Collection; * */ public class MockNode extends Node { - - private final boolean mockBigArrays; - private Collection> plugins; + private final Collection> classpathPlugins; public MockNode(Settings settings, Collection> classpathPlugins) { super(InternalSettingsPreparer.prepareEnvironment(settings, null), classpathPlugins); - this.plugins = classpathPlugins; - this.mockBigArrays = classpathPlugins.contains(NodeMocksPlugin.class); // if this plugin is present we mock bigarrays :) + this.classpathPlugins = classpathPlugins; } - public Collection> getPlugins() { - return plugins; + /** + * The classpath plugins this node was constructed with. + */ + public Collection> getClasspathPlugins() { + return classpathPlugins; } @Override protected BigArrays createBigArrays(Settings settings, CircuitBreakerService circuitBreakerService) { - if (mockBigArrays) { - return new MockBigArrays(settings, circuitBreakerService); - } else { + if (getPluginsService().filterPlugins(NodeMocksPlugin.class).isEmpty()) { return super.createBigArrays(settings, circuitBreakerService); } + return new MockBigArrays(settings, circuitBreakerService); + } + + @Override + protected Class pickSearchServiceImplementation() { + if (getPluginsService().filterPlugins(MockSearchService.TestPlugin.class).isEmpty()) { + return super.pickSearchServiceImplementation(); + } + return MockSearchService.class; } } diff --git a/test/framework/src/main/java/org/elasticsearch/search/MockSearchService.java b/test/framework/src/main/java/org/elasticsearch/search/MockSearchService.java index 9125fd47e30..cae5b2ff95b 100644 --- a/test/framework/src/main/java/org/elasticsearch/search/MockSearchService.java +++ b/test/framework/src/main/java/org/elasticsearch/search/MockSearchService.java @@ -25,14 +25,11 @@ import org.elasticsearch.common.settings.ClusterSettings; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.util.BigArrays; import org.elasticsearch.indices.IndicesService; +import org.elasticsearch.node.MockNode; import org.elasticsearch.plugins.Plugin; import org.elasticsearch.script.ScriptService; -import org.elasticsearch.search.aggregations.AggregatorParsers; -import org.elasticsearch.search.dfs.DfsPhase; import org.elasticsearch.search.fetch.FetchPhase; import org.elasticsearch.search.internal.SearchContext; -import org.elasticsearch.search.query.QueryPhase; -import org.elasticsearch.search.suggest.Suggesters; import org.elasticsearch.threadpool.ThreadPool; import java.util.HashMap; @@ -40,11 +37,10 @@ import java.util.Map; import java.util.concurrent.ConcurrentHashMap; public class MockSearchService extends SearchService { - public static class TestPlugin extends Plugin { - public void onModule(SearchModule module) { - module.searchServiceImpl = MockSearchService.class; - } - } + /** + * Marker plugin used by {@link MockNode} to enable {@link MockSearchService}. + */ + public static class TestPlugin extends Plugin {} private static final Map ACTIVE_SEARCH_CONTEXTS = new ConcurrentHashMap<>(); diff --git a/test/framework/src/main/java/org/elasticsearch/test/ESAllocationTestCase.java b/test/framework/src/main/java/org/elasticsearch/test/ESAllocationTestCase.java index 1aa0428454e..482d7c22c81 100644 --- a/test/framework/src/main/java/org/elasticsearch/test/ESAllocationTestCase.java +++ b/test/framework/src/main/java/org/elasticsearch/test/ESAllocationTestCase.java @@ -239,7 +239,7 @@ public abstract class ESAllocationTestCase extends ESTestCase { public void applyFailedShards(FailedRerouteAllocation allocation) {} @Override - public boolean allocateUnassigned(RoutingAllocation allocation) { + public void allocateUnassigned(RoutingAllocation allocation) { final RoutingNodes.UnassignedShards.UnassignedIterator unassignedIterator = allocation.routingNodes().unassigned().iterator(); while (unassignedIterator.hasNext()) { ShardRouting shard = unassignedIterator.next(); @@ -247,9 +247,8 @@ public abstract class ESAllocationTestCase extends ESTestCase { if (shard.primary() || shard.allocatedPostIndexCreate(indexMetaData) == false) { continue; } - replicaShardAllocator.ignoreUnassignedIfDelayed(unassignedIterator, shard); + replicaShardAllocator.ignoreUnassignedIfDelayed(unassignedIterator, shard, allocation.changes()); } - return false; } } } diff --git a/test/framework/src/main/java/org/elasticsearch/test/InternalTestCluster.java b/test/framework/src/main/java/org/elasticsearch/test/InternalTestCluster.java index cc78afb987f..d76fea08911 100644 --- a/test/framework/src/main/java/org/elasticsearch/test/InternalTestCluster.java +++ b/test/framework/src/main/java/org/elasticsearch/test/InternalTestCluster.java @@ -67,6 +67,7 @@ import org.elasticsearch.common.util.concurrent.ThreadContext; import org.elasticsearch.discovery.DiscoverySettings; import org.elasticsearch.env.Environment; import org.elasticsearch.env.NodeEnvironment; +import org.elasticsearch.env.ShardLockObtainFailedException; import org.elasticsearch.http.HttpServerTransport; import org.elasticsearch.index.Index; import org.elasticsearch.index.IndexService; @@ -863,7 +864,7 @@ public final class InternalTestCluster extends TestCluster { private void createNewNode(final Settings newSettings) { final long newIdSeed = NodeEnvironment.NODE_ID_SEED_SETTING.get(node.settings()) + 1; // use a new seed to make sure we have new node id Settings finalSettings = Settings.builder().put(node.settings()).put(newSettings).put(NodeEnvironment.NODE_ID_SEED_SETTING.getKey(), newIdSeed).build(); - Collection> plugins = node.getPlugins(); + Collection> plugins = node.getClasspathPlugins(); node = new MockNode(finalSettings, plugins); markNodeDataDirsAsNotEligableForWipe(node); } @@ -1912,7 +1913,7 @@ public final class InternalTestCluster extends TestCluster { for (ShardId id : shardIds) { try { env.shardLock(id, TimeUnit.SECONDS.toMillis(5)).close(); - } catch (IOException ex) { + } catch (ShardLockObtainFailedException ex) { fail("Shard " + id + " is still locked after 5 sec waiting"); } } diff --git a/test/framework/src/main/java/org/elasticsearch/test/gateway/NoopGatewayAllocator.java b/test/framework/src/main/java/org/elasticsearch/test/gateway/NoopGatewayAllocator.java index 825b203022d..e321a98f371 100644 --- a/test/framework/src/main/java/org/elasticsearch/test/gateway/NoopGatewayAllocator.java +++ b/test/framework/src/main/java/org/elasticsearch/test/gateway/NoopGatewayAllocator.java @@ -47,7 +47,7 @@ public class NoopGatewayAllocator extends GatewayAllocator { } @Override - public boolean allocateUnassigned(RoutingAllocation allocation) { - return false; + public void allocateUnassigned(RoutingAllocation allocation) { + // noop } } diff --git a/test/framework/src/test/java/org/elasticsearch/node/MockNodeTests.java b/test/framework/src/test/java/org/elasticsearch/node/MockNodeTests.java new file mode 100644 index 00000000000..51a90282b4f --- /dev/null +++ b/test/framework/src/test/java/org/elasticsearch/node/MockNodeTests.java @@ -0,0 +1,70 @@ +/* + * 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.node; + +import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.common.util.BigArrays; +import org.elasticsearch.common.util.MockBigArrays; +import org.elasticsearch.env.Environment; +import org.elasticsearch.plugins.Plugin; +import org.elasticsearch.search.MockSearchService; +import org.elasticsearch.search.SearchService; +import org.elasticsearch.test.ESTestCase; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; + +public class MockNodeTests extends ESTestCase { + /** + * Test that we add the appropriate mock services when their plugins are added. This is a very heavy test for a testing component but + * we've broken it in the past so it is important. + */ + public void testComponentsMockedByMarkerPlugins() throws IOException { + Settings settings = Settings.builder() // All these are required or MockNode will fail to build. + .put(Environment.PATH_HOME_SETTING.getKey(), createTempDir()) + .put("transport.type", "local") + .put("http.enabled", false) + .build(); + List> plugins = new ArrayList<>(); + boolean useMockBigArrays = randomBoolean(); + boolean useMockSearchService = randomBoolean(); + if (useMockBigArrays) { + plugins.add(NodeMocksPlugin.class); + } + if (useMockSearchService) { + plugins.add(MockSearchService.TestPlugin.class); + } + try (MockNode node = new MockNode(settings, plugins)) { + BigArrays bigArrays = node.injector().getInstance(BigArrays.class); + SearchService searchService = node.injector().getInstance(SearchService.class); + if (useMockBigArrays) { + assertSame(bigArrays.getClass(), MockBigArrays.class); + } else { + assertSame(bigArrays.getClass(), BigArrays.class); + } + if (useMockSearchService) { + assertSame(searchService.getClass(), MockSearchService.class); + } else { + assertSame(searchService.getClass(), SearchService.class); + } + } + } +}