Merge branch 'master' into feature/rank-eval

This commit is contained in:
Christoph Büscher 2016-08-17 11:06:32 +02:00
commit ff17b9b38f
143 changed files with 2562 additions and 1197 deletions

View File

@ -60,8 +60,8 @@ public final class Allocators {
}
@Override
public boolean allocateUnassigned(RoutingAllocation allocation) {
return false;
public void allocateUnassigned(RoutingAllocation allocation) {
// noop
}
}

View File

@ -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);
}
}

View File

@ -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

View File

@ -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

View File

@ -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

View File

@ -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];

View File

@ -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

View File

@ -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);
}
}
}
}

View File

@ -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();

View File

@ -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());
}
}

View File

@ -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
}
}

View File

@ -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

View File

@ -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;
}
}

View File

@ -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;

View File

@ -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.

View File

@ -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";

View File

@ -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"));
}

View File

@ -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) {

View File

@ -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;
}
}
}

View File

@ -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;
}
}

View File

@ -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();

View File

@ -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);
}
}

View File

@ -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>

View File

@ -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);
}
}
}

View File

@ -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();
}

View 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();
}
}

View File

@ -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> {

View File

@ -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

View File

@ -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) {

View File

@ -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;
}
/**

View File

@ -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);
}
}

View File

@ -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 + "]");
}

View File

@ -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);
}
}

View File

@ -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());

View File

@ -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;
}
}

View File

@ -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

View File

@ -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;
};
}
}

View File

@ -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.

View File

@ -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;
}

View File

@ -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

View File

@ -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

View File

@ -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"})

View File

@ -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);

View File

@ -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);
}
}

View File

@ -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");

View File

@ -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() {

View File

@ -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;
}
}

View File

@ -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;

View File

@ -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;

View File

@ -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");

View File

@ -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;

View File

@ -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);

View File

@ -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;

View File

@ -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;

View File

@ -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);

View File

@ -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;

View File

@ -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;

View File

@ -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);

View File

@ -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);

View File

@ -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<>();

View File

@ -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 {

View File

@ -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;

View File

@ -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;

View File

@ -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");

View File

@ -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");

View File

@ -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);

View File

@ -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");

View File

@ -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;

View File

@ -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);

View File

@ -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);

View File

@ -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);

View File

@ -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;

View File

@ -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;

View File

@ -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");

View File

@ -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);

View File

@ -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;

View File

@ -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);

View File

@ -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;

View File

@ -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);

View File

@ -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 });

View File

@ -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 });

View File

@ -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 });

View File

@ -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");

View File

@ -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 });

View File

@ -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;

View File

@ -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 });

View File

@ -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;

View File

@ -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;

View File

@ -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;

View File

@ -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");

View File

@ -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");

View File

@ -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());

View File

@ -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);
}
}

View File

@ -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

View File

@ -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());
}
}

View File

@ -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();

View File

@ -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);
}
});

View File

@ -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)));
}
}

View File

@ -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();
}
}
}

View File

@ -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