Merge branch 'master' into feature/rank-eval
This commit is contained in:
commit
ff17b9b38f
|
@ -60,8 +60,8 @@ public final class Allocators {
|
|||
}
|
||||
|
||||
@Override
|
||||
public boolean allocateUnassigned(RoutingAllocation allocation) {
|
||||
return false;
|
||||
public void allocateUnassigned(RoutingAllocation allocation) {
|
||||
// noop
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -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<TaskInfo> tasks, List<TaskOperationFailure> taskFailures, List<? extends FailedNodeException>
|
||||
nodeFailures, DiscoveryNodes discoveryNodes) {
|
||||
super(tasks, taskFailures, nodeFailures, discoveryNodes);
|
||||
nodeFailures) {
|
||||
super(tasks, taskFailures, nodeFailures);
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
@ -66,7 +66,7 @@ public class TransportCancelTasksAction extends TransportTasksAction<Cancellable
|
|||
TransportService transportService, ActionFilters actionFilters, IndexNameExpressionResolver
|
||||
indexNameExpressionResolver) {
|
||||
super(settings, CancelTasksAction.NAME, threadPool, clusterService, transportService, actionFilters,
|
||||
indexNameExpressionResolver, CancelTasksRequest::new, () -> 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<Cancellable
|
|||
@Override
|
||||
protected CancelTasksResponse newResponse(CancelTasksRequest request, List<TaskInfo> tasks, List<TaskOperationFailure>
|
||||
taskOperationFailures, List<FailedNodeException> failedNodeExceptions) {
|
||||
return new CancelTasksResponse(tasks, taskOperationFailures, failedNodeExceptions, clusterService.state().nodes());
|
||||
return new CancelTasksResponse(tasks, taskOperationFailures, failedNodeExceptions);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -51,21 +51,14 @@ public class ListTasksResponse extends BaseTasksResponse implements ToXContent {
|
|||
|
||||
private List<TaskGroup> 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<TaskInfo> tasks, List<TaskOperationFailure> taskFailures,
|
||||
List<? extends FailedNodeException> nodeFailures, DiscoveryNodes discoveryNodes) {
|
||||
List<? extends FailedNodeException> 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<TaskGroup> 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<TaskInfo> 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<String, List<TaskInfo>> 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<String, String> 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<String, List<TaskInfo>> 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<String, String> 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
|
||||
|
|
|
@ -56,15 +56,14 @@ public class TransportListTasksAction extends TransportTasksAction<Task, ListTas
|
|||
@Inject
|
||||
public TransportListTasksAction(Settings settings, ThreadPool threadPool, ClusterService clusterService,
|
||||
TransportService transportService, ActionFilters actionFilters, IndexNameExpressionResolver indexNameExpressionResolver) {
|
||||
super(settings, ListTasksAction.NAME, threadPool, clusterService, transportService, actionFilters,
|
||||
indexNameExpressionResolver, ListTasksRequest::new, () -> 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<TaskInfo> tasks,
|
||||
List<TaskOperationFailure> taskOperationFailures, List<FailedNodeException> failedNodeExceptions) {
|
||||
return new ListTasksResponse(tasks, taskOperationFailures, failedNodeExceptions, clusterService.state().nodes());
|
||||
return new ListTasksResponse(tasks, taskOperationFailures, failedNodeExceptions);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -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<IndexMetaData>, 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];
|
||||
|
|
|
@ -460,6 +460,11 @@ public class MetaData implements Iterable<IndexMetaData>, Diffable<MetaData>, 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
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
|
@ -409,7 +409,7 @@ public class RoutingNodes implements Iterable<RoutingNode> {
|
|||
* @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<RoutingNode> {
|
|||
}
|
||||
addRecovery(initializedShard);
|
||||
assignedShardsAdd(initializedShard);
|
||||
routingChangesObserver.shardInitialized(unassignedShard);
|
||||
return initializedShard;
|
||||
}
|
||||
|
||||
|
@ -429,7 +430,8 @@ public class RoutingNodes implements Iterable<RoutingNode> {
|
|||
*
|
||||
* @return pair of source relocating and target initializing shards.
|
||||
*/
|
||||
public Tuple<ShardRouting,ShardRouting> relocateShard(ShardRouting startedShard, String nodeId, long expectedShardSize) {
|
||||
public Tuple<ShardRouting,ShardRouting> 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<RoutingNode> {
|
|||
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<RoutingNode> {
|
|||
*
|
||||
* @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<RoutingNode> {
|
|||
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<RoutingNode> {
|
|||
* - 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<RoutingNode> {
|
|||
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<RoutingNode> {
|
|||
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<RoutingNode> {
|
|||
cancelRelocation(sourceShard);
|
||||
remove(failedShard);
|
||||
}
|
||||
routingChangesObserver.shardFailed(failedShard, unassignedInfo);
|
||||
} else {
|
||||
assert failedShard.active();
|
||||
if (failedShard.primary()) {
|
||||
|
@ -555,8 +564,10 @@ public class RoutingNodes implements Iterable<RoutingNode> {
|
|||
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<RoutingNode> {
|
|||
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<RoutingNode> {
|
|||
* 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<RoutingNode> {
|
|||
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<ShardRouting> {
|
||||
|
@ -854,10 +864,11 @@ public class RoutingNodes implements Iterable<RoutingNode> {
|
|||
*
|
||||
* @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<RoutingNode> {
|
|||
* 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<RoutingNode> {
|
|||
* @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<RoutingNode> {
|
|||
|
||||
/**
|
||||
* Returns <code>true</code> 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();
|
||||
|
|
|
@ -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<ShardRouting> 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<String> 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<String> 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<FailedRerouteAllocation.FailedShard> 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<RoutingNode> 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<ShardRouting> 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());
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -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<ShardId, Updates> 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<Index, List<Map.Entry<ShardId, Updates>>> changesGroupedByIndex =
|
||||
shardChanges.entrySet().stream().collect(Collectors.groupingBy(e -> e.getKey().getIndex()));
|
||||
|
||||
MetaData.Builder metaDataBuilder = null;
|
||||
for (Map.Entry<Index, List<Map.Entry<ShardId, Updates>>> 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<ShardId, Updates> 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<String> 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<String> addedAllocationIds = new HashSet<>(); // allocation ids that should be added to the in-sync set
|
||||
private Set<String> removedAllocationIds = new HashSet<>(); // allocation ids that should be removed from the in-sync set
|
||||
}
|
||||
}
|
|
@ -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
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
}
|
|
@ -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 <code>true</code> if the current configuration has been
|
||||
* changed, otherwise <code>false</code>
|
||||
*/
|
||||
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<DiscoveryNode, Float> weighShard(ShardRouting shard) {
|
||||
|
@ -340,12 +336,8 @@ public class BalancedShardsAllocator extends AbstractComponent implements Shards
|
|||
* the threshold is set to <tt>1.0</tt> to enforce gaining relocation
|
||||
* only, or in other words relocations that move the weight delta closer
|
||||
* to <tt>0.0</tt>
|
||||
*
|
||||
* @return <code>true</code> if the current configuration has been
|
||||
* changed, otherwise <code>false</code>
|
||||
*/
|
||||
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 <code>true</code> if the allocation has changed, otherwise <code>false</code>
|
||||
*/
|
||||
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<ShardRouting> 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 <code>true</code> if the shard was moved successfully, otherwise <code>false</code>
|
||||
*/
|
||||
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<ShardRouting, ShardRouting> relocatingShards = routingNodes.relocateShard(shardRouting, target.nodeId(), allocation.clusterInfo().getShardSize(shardRouting, ShardRouting.UNAVAILABLE_EXPECTED_SHARD_SIZE));
|
||||
Tuple<ShardRouting, ShardRouting> 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 <code>true</code> if the current configuration has been
|
||||
* changed, otherwise <code>false</code>
|
||||
*/
|
||||
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;
|
||||
|
|
|
@ -40,9 +40,8 @@ public interface ShardsAllocator {
|
|||
* - relocate shards to find a good shard balance in the cluster
|
||||
*
|
||||
* @param allocation current node allocation
|
||||
* @return <code>true</code> if the allocation has changed, otherwise <code>false</code>
|
||||
*/
|
||||
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.
|
||||
|
|
|
@ -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";
|
||||
|
|
|
@ -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"));
|
||||
}
|
||||
|
|
|
@ -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) {
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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();
|
||||
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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<TransportService> transportServiceTypes = new ExtensionPoint.SelectedType<>("transport_service", TransportService.class);
|
||||
private final ExtensionPoint.SelectedType<Transport> transportTypes = new ExtensionPoint.SelectedType<>("transport", Transport.class);
|
||||
private final ExtensionPoint.SelectedType<HttpServerTransport> httpTransportTypes = new ExtensionPoint.SelectedType<>("http_transport", HttpServerTransport.class);
|
||||
private final List<Entry> namedWriteables = new ArrayList<>();
|
||||
private final List<NamedWriteableRegistry.Entry> 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<? extends Task.Status> reader) {
|
||||
namedWriteables.add(new Entry(Task.Status.class, name, reader));
|
||||
}
|
||||
|
||||
/**
|
||||
* Register an allocation command.
|
||||
* <p>
|
||||
|
|
|
@ -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<ShardLock> 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<ShardLock> lockAllForIndex(Index index, IndexSettings settings, long lockTimeoutMS) throws IOException {
|
||||
public List<ShardLock> 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);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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();
|
||||
}
|
||||
|
|
48
core/src/main/java/org/elasticsearch/env/ShardLockObtainFailedException.java
vendored
Normal file
48
core/src/main/java/org/elasticsearch/env/ShardLockObtainFailedException.java
vendored
Normal file
|
@ -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();
|
||||
}
|
||||
}
|
|
@ -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<T extends BaseNodeResponse> extends AsyncShardFetch<T> {
|
||||
|
|
|
@ -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 <code>metaData</code> if no upgrade is needed or an upgraded metaData
|
||||
*/
|
||||
private void upgradeMetaData() throws Exception {
|
||||
MetaData metaData = loadMetaState();
|
||||
List<IndexMetaData> 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<String, MetaData.Custom> existingCustoms = new HashMap<>();
|
||||
for (ObjectObjectCursor<String, MetaData.Custom> customCursor : metaData.customs()) {
|
||||
existingCustoms.put(customCursor.key, customCursor.value);
|
||||
}
|
||||
// upgrade global custom meta data
|
||||
Map<String, MetaData.Custom> upgradedCustoms = metaDataUpgrader.customMetaDataUpgraders.apply(existingCustoms);
|
||||
if (upgradedCustoms.equals(existingCustoms) == false) {
|
||||
existingCustoms.keySet().forEach(upgradedMetaData::removeCustom);
|
||||
for (Map.Entry<String, MetaData.Custom> 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
|
||||
|
|
|
@ -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<NodeGatewayStartedShards> nodeShardStates) {
|
||||
private NodesToAllocate buildNodesToAllocate(RoutingAllocation allocation,
|
||||
List<NodeGatewayStartedShards> nodeShardStates,
|
||||
ShardRouting shardRouting,
|
||||
boolean forceAllocate) {
|
||||
List<NodeGatewayStartedShards> yesNodeShards = new ArrayList<>();
|
||||
List<NodeGatewayStartedShards> throttledNodeShards = new ArrayList<>();
|
||||
List<NodeGatewayStartedShards> 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) {
|
||||
|
|
|
@ -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<Runnable> 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;
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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<String> 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 + "]");
|
||||
}
|
||||
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -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<ShardLock> shardLocks = nodeEnv.lockAllForIndex(index, indexSettings, timeout.millis());
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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<Object> 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<UnaryOperator<Map<String, MetaData.Custom>>> 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<? extends SearchService> 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<? extends SearchService> pickSearchServiceImplementation() {
|
||||
return SearchService.class;
|
||||
}
|
||||
|
||||
/**
|
||||
* Get Custom Name Resolvers list based on a Discovery Plugins list
|
||||
* @param discoveryPlugins Discovery plugins list
|
||||
|
|
|
@ -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<Map<String, MetaData.Custom>> customMetaDataUpgraders;
|
||||
|
||||
public MetaDataUpgrader(Collection<UnaryOperator<Map<String, MetaData.Custom>>> customMetaDataUpgraders) {
|
||||
this.customMetaDataUpgraders = customs -> {
|
||||
Map<String, MetaData.Custom> upgradedCustoms = new HashMap<>(customs);
|
||||
for (UnaryOperator<Map<String, MetaData.Custom>> customMetaDataUpgrader : customMetaDataUpgraders) {
|
||||
upgradedCustoms = customMetaDataUpgrader.apply(upgradedCustoms);
|
||||
}
|
||||
return upgradedCustoms;
|
||||
};
|
||||
}
|
||||
}
|
|
@ -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.
|
||||
* <p>
|
||||
|
@ -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<Object> 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<String> getSettingsFilter() { return Collections.emptyList(); }
|
||||
|
||||
/**
|
||||
* Provides a function to modify global custom meta data on startup.
|
||||
* <p>
|
||||
* 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<Map<String, MetaData.Custom>> 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.
|
||||
|
|
|
@ -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<QuerySpec<?>> getQueries() {
|
||||
return emptyList();
|
||||
}
|
||||
/**
|
||||
* The new {@link Aggregation}s added by this plugin.
|
||||
*/
|
||||
default List<AggregationSpec> getAggregations() {
|
||||
return emptyList();
|
||||
}
|
||||
/**
|
||||
* The new {@link PipelineAggregator}s added by this plugin.
|
||||
*/
|
||||
default List<PipelineAggregationSpec> getPipelineAggregations() {
|
||||
return emptyList();
|
||||
}
|
||||
|
||||
/**
|
||||
* Specification of custom {@link ScoreFunction}.
|
||||
*/
|
||||
class ScoreFunctionSpec<T extends ScoreFunctionBuilder<T>> extends SearchExtensionSpec<T, ScoreFunctionParser<T>> {
|
||||
public ScoreFunctionSpec(ParseField name, Reader<T> reader, ScoreFunctionParser<T> parser) {
|
||||
public ScoreFunctionSpec(ParseField name, Writeable.Reader<T> reader, ScoreFunctionParser<T> parser) {
|
||||
super(name, reader, parser);
|
||||
}
|
||||
|
||||
public ScoreFunctionSpec(String name, Reader<T> reader, ScoreFunctionParser<T> parser) {
|
||||
public ScoreFunctionSpec(String name, Writeable.Reader<T> reader, ScoreFunctionParser<T> 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<T> reader, QueryParser<T> parser) {
|
||||
public QuerySpec(ParseField name, Writeable.Reader<T> reader, QueryParser<T> 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<T> reader, QueryParser<T> parser) {
|
||||
public QuerySpec(String name, Writeable.Reader<T> reader, QueryParser<T> parser) {
|
||||
super(name, reader, parser);
|
||||
}
|
||||
}
|
||||
/**
|
||||
* Specification for an {@link Aggregation}.
|
||||
*/
|
||||
public static class AggregationSpec extends SearchExtensionSpec<AggregationBuilder, Aggregator.Parser> {
|
||||
private final Map<String, Writeable.Reader<? extends InternalAggregation>> 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<? extends AggregationBuilder> 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<? extends AggregationBuilder> 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<? extends InternalAggregation> resultReader) {
|
||||
return addResultReader(getName().getPreferredName(), resultReader);
|
||||
}
|
||||
|
||||
/**
|
||||
* Add a reader for the shard level results of the aggregation.
|
||||
*/
|
||||
public AggregationSpec addResultReader(String writeableName, Writeable.Reader<? extends InternalAggregation> resultReader) {
|
||||
resultReaders.put(writeableName, resultReader);
|
||||
return this;
|
||||
}
|
||||
|
||||
/**
|
||||
* Get the readers that must be registered for this aggregation's results.
|
||||
*/
|
||||
public Map<String, Writeable.Reader<? extends InternalAggregation>> getResultReaders() {
|
||||
return resultReaders;
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Specification for a {@link PipelineAggregator}.
|
||||
*/
|
||||
public static class PipelineAggregationSpec extends SearchExtensionSpec<PipelineAggregationBuilder, PipelineAggregator.Parser> {
|
||||
private final Map<String, Writeable.Reader<? extends InternalAggregation>> resultReaders = new TreeMap<>();
|
||||
private final Writeable.Reader<? extends PipelineAggregator> 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<? extends PipelineAggregationBuilder> builderReader,
|
||||
Writeable.Reader<? extends PipelineAggregator> 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<? extends PipelineAggregationBuilder> builderReader,
|
||||
Writeable.Reader<? extends PipelineAggregator> 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<? extends InternalAggregation> resultReader) {
|
||||
return addResultReader(getName().getPreferredName(), resultReader);
|
||||
}
|
||||
|
||||
/**
|
||||
* Add a reader for the shard level results of the aggregation.
|
||||
*/
|
||||
public PipelineAggregationSpec addResultReader(String writeableName, Writeable.Reader<? extends InternalAggregation> resultReader) {
|
||||
resultReaders.put(writeableName, resultReader);
|
||||
return this;
|
||||
}
|
||||
|
||||
/**
|
||||
* The reader for the {@link PipelineAggregator}.
|
||||
*/
|
||||
public Writeable.Reader<? extends PipelineAggregator> getAggregatorReader() {
|
||||
return aggregatorReader;
|
||||
}
|
||||
|
||||
/**
|
||||
* Get the readers that must be registered for this aggregation's results.
|
||||
*/
|
||||
public Map<String, Writeable.Reader<? extends InternalAggregation>> 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<W extends NamedWriteable, P> {
|
||||
private final ParseField name;
|
||||
private final Writeable.Reader<W> reader;
|
||||
private final Writeable.Reader<? extends W> 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<W> reader, P parser) {
|
||||
public SearchExtensionSpec(ParseField name, Writeable.Reader<? extends W> 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<W> reader, P parser) {
|
||||
public SearchExtensionSpec(String name, Writeable.Reader<? extends W> 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<W> getReader() {
|
||||
public Writeable.Reader<? extends W> getReader() {
|
||||
return reader;
|
||||
}
|
||||
|
||||
|
|
|
@ -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<CancelTasksResponse> listener = nodeSettingListener(clusterService, new RestToXContentListener<>(channel));
|
||||
client.admin().cluster().cancelTasks(cancelTasksRequest, listener);
|
||||
client.admin().cluster().cancelTasks(cancelTasksRequest, listTasksResponseListener(clusterService, channel));
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -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<ListTasksResponse> 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 <T extends ListTasksResponse> ActionListener<T> nodeSettingListener(ClusterService clusterService,
|
||||
ActionListener<T> channelListener) {
|
||||
return new ActionListener<T>() {
|
||||
@Override
|
||||
public void onResponse(T response) {
|
||||
channelListener.onResponse(response);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void onFailure(Exception e) {
|
||||
channelListener.onFailure(e);
|
||||
}
|
||||
};
|
||||
public static <T extends ListTasksResponse> ActionListener<T> listTasksResponseListener(ClusterService clusterService,
|
||||
RestChannel channel) {
|
||||
String groupBy = channel.request().param("group_by", "nodes");
|
||||
if ("nodes".equals(groupBy)) {
|
||||
return new RestBuilderListener<T>(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
|
||||
|
|
|
@ -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"})
|
||||
|
|
|
@ -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);
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -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");
|
||||
|
|
|
@ -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<Integer> 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<String, Highlighter> highlighters;
|
||||
|
@ -309,11 +309,7 @@ public class SearchModule extends AbstractModule {
|
|||
|
||||
private final Settings settings;
|
||||
private final List<Entry> namedWriteables = new ArrayList<>();
|
||||
public static final Setting<Integer> 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<? extends SearchService> searchServiceImpl = SearchService.class;
|
||||
private final SearchRequestParsers searchRequestParsers;
|
||||
|
||||
public SearchModule(Settings settings, boolean transportClient, List<SearchPlugin> 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<Entry> 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<String, Writeable.Reader<? extends InternalAggregation>> t : spec.resultReaders.entrySet()) {
|
||||
}
|
||||
|
||||
private void registerAggregations(List<SearchPlugin> 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<String, Writeable.Reader<? extends InternalAggregation>> t : spec.getResultReaders().entrySet()) {
|
||||
String writeableName = t.getKey();
|
||||
Writeable.Reader<? extends InternalAggregation> internalReader = t.getValue();
|
||||
namedWriteables.add(new Entry(InternalAggregation.class, writeableName, internalReader));
|
||||
}
|
||||
}
|
||||
|
||||
public static class AggregationSpec {
|
||||
private final Map<String, Writeable.Reader<? extends InternalAggregation>> resultReaders = new TreeMap<>();
|
||||
private final Writeable.Reader<? extends AggregationBuilder> 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<? extends AggregationBuilder> 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<? extends InternalAggregation> resultReader) {
|
||||
return addResultReader(name.getPreferredName(), resultReader);
|
||||
}
|
||||
|
||||
/**
|
||||
* Add a reader for the shard level results of the aggregation.
|
||||
*/
|
||||
public AggregationSpec addResultReader(String writeableName, Writeable.Reader<? extends InternalAggregation> 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<String, Writeable.Reader<? extends InternalAggregation>> resultReader : spec.resultReaders.entrySet()) {
|
||||
namedWriteables.add(new Entry(InternalAggregation.class, resultReader.getKey(), resultReader.getValue()));
|
||||
}
|
||||
}
|
||||
|
||||
public static class PipelineAggregationSpec {
|
||||
private final Map<String, Writeable.Reader<? extends InternalAggregation>> resultReaders = new TreeMap<>();
|
||||
private final Writeable.Reader<? extends PipelineAggregationBuilder> builderReader;
|
||||
private final Writeable.Reader<? extends PipelineAggregator> aggregatorReader;
|
||||
private final PipelineAggregator.Parser parser;
|
||||
private final ParseField name;
|
||||
|
||||
/**
|
||||
* Register a pipeline aggregation.
|
||||
*
|
||||
* @param builderReader reads the {@link PipelineAggregationBuilder} from a stream
|
||||
* @param aggregatorReader reads the {@link PipelineAggregator} from a stream
|
||||
* @param parser reads the aggregation builder from XContent
|
||||
* @param name names by which the aggregation may be parsed. The first name is special because it is the name that the reader is
|
||||
* registered under.
|
||||
*/
|
||||
public PipelineAggregationSpec(Reader<? extends PipelineAggregationBuilder> builderReader,
|
||||
Writeable.Reader<? extends PipelineAggregator> aggregatorReader,
|
||||
PipelineAggregator.Parser parser, ParseField name) {
|
||||
this.builderReader = builderReader;
|
||||
this.aggregatorReader = aggregatorReader;
|
||||
this.parser = parser;
|
||||
this.name = name;
|
||||
}
|
||||
|
||||
/**
|
||||
* Add a reader for the shard level results of the aggregation with {@linkplain #name}'s {@link ParseField#getPreferredName()} as
|
||||
* the {@link NamedWriteable#getWriteableName()}.
|
||||
*/
|
||||
public PipelineAggregationSpec addResultReader(Writeable.Reader<? extends InternalAggregation> resultReader) {
|
||||
return addResultReader(name.getPreferredName(), resultReader);
|
||||
}
|
||||
|
||||
/**
|
||||
* Add a reader for the shard level results of the aggregation.
|
||||
*/
|
||||
public PipelineAggregationSpec addResultReader(String writeableName, Writeable.Reader<? extends InternalAggregation> 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<SearchPlugin> 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<String, Writeable.Reader<? extends InternalAggregation>> 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() {
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
}
|
|
@ -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<ParentChild, ChildrenAggregationBuilder> {
|
||||
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;
|
||||
|
|
|
@ -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<FilterAggregationBuilder> {
|
||||
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;
|
||||
|
||||
|
|
|
@ -46,7 +46,6 @@ import static org.elasticsearch.index.query.QueryBuilders.matchAllQuery;
|
|||
public class FiltersAggregationBuilder extends AbstractAggregationBuilder<FiltersAggregationBuilder> {
|
||||
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");
|
||||
|
|
|
@ -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<ValuesSource.GeoPoint, GeoGridAggregationBuilder> {
|
||||
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;
|
||||
|
|
|
@ -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<GlobalAggregationBuilder> {
|
||||
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);
|
||||
|
|
|
@ -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<ValuesSource.Numeric, DateHistogramAggregationBuilder> {
|
||||
public static final String NAME = InternalDateHistogram.TYPE.name();
|
||||
public static final ParseField AGGREGATION_NAME_FIELD = new ParseField(NAME);
|
||||
|
||||
private long interval;
|
||||
private DateHistogramInterval dateHistogramInterval;
|
||||
|
|
|
@ -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<ValuesSource.Numeric, HistogramAggregationBuilder> {
|
||||
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;
|
||||
|
|
|
@ -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<ValuesSource, MissingAggregationBuilder> {
|
||||
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);
|
||||
|
|
|
@ -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<NestedAggregationBuilder> {
|
||||
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;
|
||||
|
||||
|
|
|
@ -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<ReverseNestedAggregationBuilder> {
|
||||
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;
|
||||
|
||||
|
|
|
@ -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<RangeAggregationBuilder, Range> {
|
||||
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);
|
||||
|
|
|
@ -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<DateRangeAggregationBuilder, RangeAggregator.Range> {
|
||||
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);
|
||||
|
|
|
@ -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<ValuesSource.GeoPoint, GeoDistanceAggregationBuilder> {
|
||||
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<Range> ranges = new ArrayList<>();
|
||||
|
|
|
@ -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<ValuesSource.Bytes, IpRangeAggregationBuilder> {
|
||||
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 {
|
||||
|
|
|
@ -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<ValuesSource, DiversifiedAggregationBuilder> {
|
||||
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;
|
||||
|
|
|
@ -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<SamplerAggregationBuilder> {
|
||||
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;
|
||||
|
||||
|
|
|
@ -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<ValuesSource, SignificantTermsAggregationBuilder> {
|
||||
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");
|
||||
|
|
|
@ -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<ValuesSource, TermsAggregationBuilder> {
|
||||
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");
|
||||
|
|
|
@ -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<ValuesSource.Numeric, AvgAggregationBuilder> {
|
||||
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);
|
||||
|
|
|
@ -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");
|
||||
|
||||
|
|
|
@ -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<ValuesSource.GeoPoint, GeoBoundsAggregationBuilder> {
|
||||
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;
|
||||
|
||||
|
|
|
@ -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<ValuesSource.GeoPoint, GeoCentroidAggregationBuilder> {
|
||||
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);
|
||||
|
|
|
@ -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<ValuesSource.Numeric, MaxAggregationBuilder> {
|
||||
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);
|
||||
|
|
|
@ -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<ValuesSource.Numeric, MinAggregationBuilder> {
|
||||
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);
|
||||
|
|
|
@ -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<ValuesSource.Numeric, PercentileRanksAggregationBuilder> {
|
||||
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;
|
||||
|
|
|
@ -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<ValuesSource.Numeric, PercentilesAggregationBuilder> {
|
||||
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;
|
||||
|
|
|
@ -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");
|
||||
|
|
|
@ -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<ValuesSource.Numeric, StatsAggregationBuilder> {
|
||||
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);
|
||||
|
|
|
@ -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<ValuesSource.Numeric, ExtendedStatsAggregationBuilder> {
|
||||
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;
|
||||
|
||||
|
|
|
@ -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<ValuesSource.Numeric, SumAggregationBuilder> {
|
||||
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);
|
||||
|
|
|
@ -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<TopHitsAggregationBuilder> {
|
||||
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;
|
||||
|
|
|
@ -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<ValuesSource, ValueCountAggregationBuilder> {
|
||||
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);
|
||||
|
|
|
@ -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<AvgBucketPipelineAggregationBuilder> {
|
||||
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 });
|
||||
|
|
|
@ -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<MaxBucketPipelineAggregationBuilder> {
|
||||
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 });
|
||||
|
|
|
@ -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<MinBucketPipelineAggregationBuilder> {
|
||||
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 });
|
||||
|
|
|
@ -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<PercentilesBucketPipelineAggregationBuilder> {
|
||||
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");
|
||||
|
||||
|
|
|
@ -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<StatsBucketPipelineAggregationBuilder> {
|
||||
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 });
|
||||
|
|
|
@ -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<ExtendedStatsBucketPipelineAggregationBuilder> {
|
||||
public static final String NAME = "extended_stats_bucket";
|
||||
public static final ParseField AGGREGATION_NAME_FIELD = new ParseField(NAME);
|
||||
|
||||
private double sigma = 2.0;
|
||||
|
||||
|
|
|
@ -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<SumBucketPipelineAggregationBuilder> {
|
||||
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 });
|
||||
|
|
|
@ -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<BucketScriptPipelineAggregationBuilder> {
|
||||
public static final String NAME = "bucket_script";
|
||||
public static final ParseField AGGREGATION_NAME_FIELD = new ParseField(NAME);
|
||||
|
||||
private final Script script;
|
||||
private final Map<String, String> bucketsPathsMap;
|
||||
|
|
|
@ -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<BucketSelectorPipelineAggregationBuilder> {
|
||||
public static final String NAME = "bucket_selector";
|
||||
public static final ParseField AGGREGATION_NAME_FIELD = new ParseField(NAME);
|
||||
|
||||
private final Map<String, String> bucketsPathsMap;
|
||||
private Script script;
|
||||
|
|
|
@ -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<CumulativeSumPipelineAggregationBuilder> {
|
||||
public static final String NAME = "cumulative_sum";
|
||||
public static final ParseField AGGREGATION_NAME_FIELD = new ParseField(NAME);
|
||||
|
||||
private String format;
|
||||
|
||||
|
|
|
@ -47,7 +47,6 @@ import java.util.Objects;
|
|||
|
||||
public class DerivativePipelineAggregationBuilder extends AbstractPipelineAggregationBuilder<DerivativePipelineAggregationBuilder> {
|
||||
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");
|
||||
|
|
|
@ -52,7 +52,6 @@ import static org.elasticsearch.search.aggregations.pipeline.PipelineAggregator.
|
|||
|
||||
public class MovAvgPipelineAggregationBuilder extends AbstractPipelineAggregationBuilder<MovAvgPipelineAggregationBuilder> {
|
||||
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");
|
||||
|
|
|
@ -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<String, Object> byNodes = serialize(response, new ToXContent.MapParams(Collections.singletonMap("group_by", "nodes")));
|
||||
Map<String, Object> byNodes = serialize(response, true);
|
||||
byNodes = (Map<String, Object>) 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<String, Object> byParent = serialize(response, new ToXContent.MapParams(Collections.singletonMap("group_by", "parents")));
|
||||
Map<String, Object> byParent = serialize(response, false);
|
||||
byParent = (Map<String, Object>) 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<String, Object> serialize(ToXContent response, ToXContent.Params params) throws IOException {
|
||||
private Map<String, Object> 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());
|
||||
|
|
|
@ -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<MatchAllQueryBuilder> parser = MatchAllQueryBuilder::fromXContent;
|
||||
registry.register(parser, MatchAllQueryBuilder.NAME);
|
||||
return registry;
|
||||
return new SearchRequestParsers(registry, null, null);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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());
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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();
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
});
|
||||
|
||||
|
|
|
@ -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)));
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
@ -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();
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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());
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
Some files were not shown because too many files have changed in this diff Show More
Loading…
Reference in New Issue