Merge branch 'master' into feature/query-refactoring

This commit is contained in:
javanna 2015-08-31 13:28:48 +02:00 committed by Luca Cavanna
commit faf526dccb
169 changed files with 3280 additions and 1814 deletions

View File

@ -314,6 +314,7 @@
<include>org/elasticsearch/common/util/MockBigArrays.class</include> <include>org/elasticsearch/common/util/MockBigArrays.class</include>
<include>org/elasticsearch/common/util/MockBigArrays$*.class</include> <include>org/elasticsearch/common/util/MockBigArrays$*.class</include>
<include>org/elasticsearch/node/NodeMocksPlugin.class</include> <include>org/elasticsearch/node/NodeMocksPlugin.class</include>
<include>org/elasticsearch/node/MockNode.class</include>
</includes> </includes>
<excludes> <excludes>
<!-- unit tests for yaml suite parser & rest spec parser need to be excluded --> <!-- unit tests for yaml suite parser & rest spec parser need to be excluded -->

View File

@ -21,7 +21,6 @@ package org.elasticsearch.action.admin.cluster.stats;
import com.carrotsearch.hppc.ObjectObjectHashMap; import com.carrotsearch.hppc.ObjectObjectHashMap;
import com.carrotsearch.hppc.cursors.ObjectObjectCursor; import com.carrotsearch.hppc.cursors.ObjectObjectCursor;
import org.elasticsearch.action.admin.indices.stats.CommonStats; import org.elasticsearch.action.admin.indices.stats.CommonStats;
import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.io.stream.StreamInput;
import org.elasticsearch.common.io.stream.StreamOutput; import org.elasticsearch.common.io.stream.StreamOutput;
@ -67,10 +66,10 @@ public class ClusterStatsIndices implements ToXContent, Streamable {
for (ClusterStatsNodeResponse r : nodeResponses) { for (ClusterStatsNodeResponse r : nodeResponses) {
for (org.elasticsearch.action.admin.indices.stats.ShardStats shardStats : r.shardsStats()) { for (org.elasticsearch.action.admin.indices.stats.ShardStats shardStats : r.shardsStats()) {
ShardStats indexShardStats = countsPerIndex.get(shardStats.getIndex()); ShardStats indexShardStats = countsPerIndex.get(shardStats.getShardRouting().getIndex());
if (indexShardStats == null) { if (indexShardStats == null) {
indexShardStats = new ShardStats(); indexShardStats = new ShardStats();
countsPerIndex.put(shardStats.getIndex(), indexShardStats); countsPerIndex.put(shardStats.getShardRouting().getIndex(), indexShardStats);
} }
indexShardStats.total++; indexShardStats.total++;

View File

@ -23,6 +23,7 @@ import org.elasticsearch.action.admin.cluster.health.ClusterHealthStatus;
import org.elasticsearch.action.admin.cluster.health.ClusterIndexHealth; import org.elasticsearch.action.admin.cluster.health.ClusterIndexHealth;
import org.elasticsearch.action.admin.cluster.node.info.NodeInfo; import org.elasticsearch.action.admin.cluster.node.info.NodeInfo;
import org.elasticsearch.action.admin.cluster.node.stats.NodeStats; import org.elasticsearch.action.admin.cluster.node.stats.NodeStats;
import org.elasticsearch.action.admin.indices.stats.CommonStats;
import org.elasticsearch.action.admin.indices.stats.CommonStatsFlags; import org.elasticsearch.action.admin.indices.stats.CommonStatsFlags;
import org.elasticsearch.action.admin.indices.stats.ShardStats; import org.elasticsearch.action.admin.indices.stats.ShardStats;
import org.elasticsearch.action.support.ActionFilters; import org.elasticsearch.action.support.ActionFilters;
@ -106,7 +107,7 @@ public class TransportClusterStatsAction extends TransportNodesAction<ClusterSta
for (IndexShard indexShard : indexService) { for (IndexShard indexShard : indexService) {
if (indexShard.routingEntry() != null && indexShard.routingEntry().active()) { if (indexShard.routingEntry() != null && indexShard.routingEntry().active()) {
// only report on fully started shards // only report on fully started shards
shardsStats.add(new ShardStats(indexShard, SHARD_STATS_FLAGS)); shardsStats.add(new ShardStats(indexShard.routingEntry(), indexShard.shardPath(), new CommonStats(indexShard, SHARD_STATS_FLAGS), indexShard.commitStats()));
} }
} }
} }

View File

@ -1,92 +0,0 @@
/*
* Licensed to Elasticsearch under one or more contributor
* license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright
* ownership. Elasticsearch licenses this file to you under
* the Apache License, Version 2.0 (the "License"); you may
* not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing,
* software distributed under the License is distributed on an
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
* KIND, either express or implied. See the License for the
* specific language governing permissions and limitations
* under the License.
*/
package org.elasticsearch.action.admin.indices.cache.clear;
import org.elasticsearch.action.support.broadcast.BroadcastShardRequest;
import org.elasticsearch.common.io.stream.StreamInput;
import org.elasticsearch.common.io.stream.StreamOutput;
import org.elasticsearch.index.shard.ShardId;
import java.io.IOException;
/**
*
*/
class ShardClearIndicesCacheRequest extends BroadcastShardRequest {
private boolean queryCache = false;
private boolean fieldDataCache = false;
private boolean recycler;
private boolean requestCache = false;
private String[] fields = null;
ShardClearIndicesCacheRequest() {
}
ShardClearIndicesCacheRequest(ShardId shardId, ClearIndicesCacheRequest request) {
super(shardId, request);
queryCache = request.queryCache();
fieldDataCache = request.fieldDataCache();
fields = request.fields();
recycler = request.recycler();
requestCache = request.requestCache();
}
public boolean queryCache() {
return queryCache;
}
public boolean requestCache() {
return requestCache;
}
public boolean fieldDataCache() {
return this.fieldDataCache;
}
public boolean recycler() {
return this.recycler;
}
public String[] fields() {
return this.fields;
}
@Override
public void readFrom(StreamInput in) throws IOException {
super.readFrom(in);
queryCache = in.readBoolean();
fieldDataCache = in.readBoolean();
recycler = in.readBoolean();
fields = in.readStringArray();
requestCache = in.readBoolean();
}
@Override
public void writeTo(StreamOutput out) throws IOException {
super.writeTo(out);
out.writeBoolean(queryCache);
out.writeBoolean(fieldDataCache);
out.writeBoolean(recycler);
out.writeStringArrayNullable(fields);
out.writeBoolean(requestCache);
}
}

View File

@ -1,36 +0,0 @@
/*
* Licensed to Elasticsearch under one or more contributor
* license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright
* ownership. Elasticsearch licenses this file to you under
* the Apache License, Version 2.0 (the "License"); you may
* not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing,
* software distributed under the License is distributed on an
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
* KIND, either express or implied. See the License for the
* specific language governing permissions and limitations
* under the License.
*/
package org.elasticsearch.action.admin.indices.cache.clear;
import org.elasticsearch.action.support.broadcast.BroadcastShardResponse;
import org.elasticsearch.index.shard.ShardId;
/**
*
*/
class ShardClearIndicesCacheResponse extends BroadcastShardResponse {
ShardClearIndicesCacheResponse() {
}
ShardClearIndicesCacheResponse(ShardId shardId) {
super(shardId);
}
}

View File

@ -21,17 +21,16 @@ package org.elasticsearch.action.admin.indices.cache.clear;
import org.elasticsearch.action.ShardOperationFailedException; import org.elasticsearch.action.ShardOperationFailedException;
import org.elasticsearch.action.support.ActionFilters; import org.elasticsearch.action.support.ActionFilters;
import org.elasticsearch.action.support.DefaultShardOperationFailedException; import org.elasticsearch.action.support.broadcast.node.TransportBroadcastByNodeAction;
import org.elasticsearch.action.support.broadcast.BroadcastShardOperationFailedException;
import org.elasticsearch.action.support.broadcast.TransportBroadcastAction;
import org.elasticsearch.cluster.ClusterService; import org.elasticsearch.cluster.ClusterService;
import org.elasticsearch.cluster.ClusterState; import org.elasticsearch.cluster.ClusterState;
import org.elasticsearch.cluster.block.ClusterBlockException; import org.elasticsearch.cluster.block.ClusterBlockException;
import org.elasticsearch.cluster.block.ClusterBlockLevel; import org.elasticsearch.cluster.block.ClusterBlockLevel;
import org.elasticsearch.cluster.metadata.IndexNameExpressionResolver; import org.elasticsearch.cluster.metadata.IndexNameExpressionResolver;
import org.elasticsearch.cluster.routing.GroupShardsIterator;
import org.elasticsearch.cluster.routing.ShardRouting; import org.elasticsearch.cluster.routing.ShardRouting;
import org.elasticsearch.cluster.routing.ShardsIterator;
import org.elasticsearch.common.inject.Inject; import org.elasticsearch.common.inject.Inject;
import org.elasticsearch.common.io.stream.StreamInput;
import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.index.IndexService; import org.elasticsearch.index.IndexService;
import org.elasticsearch.index.shard.IndexShard; import org.elasticsearch.index.shard.IndexShard;
@ -40,14 +39,14 @@ import org.elasticsearch.indices.cache.request.IndicesRequestCache;
import org.elasticsearch.threadpool.ThreadPool; import org.elasticsearch.threadpool.ThreadPool;
import org.elasticsearch.transport.TransportService; import org.elasticsearch.transport.TransportService;
import java.util.ArrayList; import java.io.IOException;
import java.util.List; import java.util.List;
import java.util.concurrent.atomic.AtomicReferenceArray;
/** /**
* Indices clear cache action. * Indices clear cache action.
*/ */
public class TransportClearIndicesCacheAction extends TransportBroadcastAction<ClearIndicesCacheRequest, ClearIndicesCacheResponse, ShardClearIndicesCacheRequest, ShardClearIndicesCacheResponse> { public class TransportClearIndicesCacheAction extends TransportBroadcastByNodeAction<ClearIndicesCacheRequest, ClearIndicesCacheResponse, TransportBroadcastByNodeAction.EmptyResult> {
private final IndicesService indicesService; private final IndicesService indicesService;
private final IndicesRequestCache indicesRequestCache; private final IndicesRequestCache indicesRequestCache;
@ -58,48 +57,33 @@ public class TransportClearIndicesCacheAction extends TransportBroadcastAction<C
IndicesRequestCache indicesQueryCache, ActionFilters actionFilters, IndicesRequestCache indicesQueryCache, ActionFilters actionFilters,
IndexNameExpressionResolver indexNameExpressionResolver) { IndexNameExpressionResolver indexNameExpressionResolver) {
super(settings, ClearIndicesCacheAction.NAME, threadPool, clusterService, transportService, actionFilters, indexNameExpressionResolver, super(settings, ClearIndicesCacheAction.NAME, threadPool, clusterService, transportService, actionFilters, indexNameExpressionResolver,
ClearIndicesCacheRequest.class, ShardClearIndicesCacheRequest.class, ThreadPool.Names.MANAGEMENT); ClearIndicesCacheRequest.class, ThreadPool.Names.MANAGEMENT);
this.indicesService = indicesService; this.indicesService = indicesService;
this.indicesRequestCache = indicesQueryCache; this.indicesRequestCache = indicesQueryCache;
} }
@Override @Override
protected ClearIndicesCacheResponse newResponse(ClearIndicesCacheRequest request, AtomicReferenceArray shardsResponses, ClusterState clusterState) { protected EmptyResult readShardResult(StreamInput in) throws IOException {
int successfulShards = 0; return EmptyResult.readEmptyResultFrom(in);
int failedShards = 0;
List<ShardOperationFailedException> shardFailures = null;
for (int i = 0; i < shardsResponses.length(); i++) {
Object shardResponse = shardsResponses.get(i);
if (shardResponse == null) {
// simply ignore non active shards
} else if (shardResponse instanceof BroadcastShardOperationFailedException) {
failedShards++;
if (shardFailures == null) {
shardFailures = new ArrayList<>();
}
shardFailures.add(new DefaultShardOperationFailedException((BroadcastShardOperationFailedException) shardResponse));
} else {
successfulShards++;
}
}
return new ClearIndicesCacheResponse(shardsResponses.length(), successfulShards, failedShards, shardFailures);
} }
@Override @Override
protected ShardClearIndicesCacheRequest newShardRequest(int numShards, ShardRouting shard, ClearIndicesCacheRequest request) { protected ClearIndicesCacheResponse newResponse(ClearIndicesCacheRequest request, int totalShards, int successfulShards, int failedShards, List<EmptyResult> responses, List<ShardOperationFailedException> shardFailures, ClusterState clusterState) {
return new ShardClearIndicesCacheRequest(shard.shardId(), request); return new ClearIndicesCacheResponse(totalShards, successfulShards, failedShards, shardFailures);
} }
@Override @Override
protected ShardClearIndicesCacheResponse newShardResponse() { protected ClearIndicesCacheRequest readRequestFrom(StreamInput in) throws IOException {
return new ShardClearIndicesCacheResponse(); final ClearIndicesCacheRequest request = new ClearIndicesCacheRequest();
request.readFrom(in);
return request;
} }
@Override @Override
protected ShardClearIndicesCacheResponse shardOperation(ShardClearIndicesCacheRequest request) { protected EmptyResult shardOperation(ClearIndicesCacheRequest request, ShardRouting shardRouting) {
IndexService service = indicesService.indexService(request.shardId().getIndex()); IndexService service = indicesService.indexService(shardRouting.getIndex());
if (service != null) { if (service != null) {
IndexShard shard = service.shard(request.shardId().id()); IndexShard shard = service.shard(shardRouting.id());
boolean clearedAtLeastOne = false; boolean clearedAtLeastOne = false;
if (request.queryCache()) { if (request.queryCache()) {
clearedAtLeastOne = true; clearedAtLeastOne = true;
@ -137,15 +121,15 @@ public class TransportClearIndicesCacheAction extends TransportBroadcastAction<C
} }
} }
} }
return new ShardClearIndicesCacheResponse(request.shardId()); return EmptyResult.INSTANCE;
} }
/** /**
* The refresh request works against *all* shards. * The refresh request works against *all* shards.
*/ */
@Override @Override
protected GroupShardsIterator shards(ClusterState clusterState, ClearIndicesCacheRequest request, String[] concreteIndices) { protected ShardsIterator shards(ClusterState clusterState, ClearIndicesCacheRequest request, String[] concreteIndices) {
return clusterState.routingTable().allActiveShardsGrouped(concreteIndices, true); return clusterState.routingTable().allShards(concreteIndices);
} }
@Override @Override
@ -157,5 +141,4 @@ public class TransportClearIndicesCacheAction extends TransportBroadcastAction<C
protected ClusterBlockException checkRequestBlock(ClusterState state, ClearIndicesCacheRequest request, String[] concreteIndices) { protected ClusterBlockException checkRequestBlock(ClusterState state, ClearIndicesCacheRequest request, String[] concreteIndices) {
return state.blocks().indicesBlockedException(ClusterBlockLevel.METADATA_WRITE, concreteIndices); return state.blocks().indicesBlockedException(ClusterBlockLevel.METADATA_WRITE, concreteIndices);
} }
} }

View File

@ -1,60 +0,0 @@
/*
* Licensed to Elasticsearch under one or more contributor
* license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright
* ownership. Elasticsearch licenses this file to you under
* the Apache License, Version 2.0 (the "License"); you may
* not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing,
* software distributed under the License is distributed on an
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
* KIND, either express or implied. See the License for the
* specific language governing permissions and limitations
* under the License.
*/
package org.elasticsearch.action.admin.indices.optimize;
import org.elasticsearch.action.support.broadcast.BroadcastShardRequest;
import org.elasticsearch.common.io.stream.StreamInput;
import org.elasticsearch.common.io.stream.StreamOutput;
import org.elasticsearch.index.shard.ShardId;
import java.io.IOException;
/**
*
*/
final class ShardOptimizeRequest extends BroadcastShardRequest {
private OptimizeRequest request = new OptimizeRequest();
ShardOptimizeRequest() {
}
ShardOptimizeRequest(ShardId shardId, OptimizeRequest request) {
super(shardId, request);
this.request = request;
}
@Override
public void readFrom(StreamInput in) throws IOException {
super.readFrom(in);
request.readFrom(in);
}
@Override
public void writeTo(StreamOutput out) throws IOException {
super.writeTo(out);
request.writeTo(out);
}
public OptimizeRequest optimizeRequest() {
return this.request;
}
}

View File

@ -1,36 +0,0 @@
/*
* Licensed to Elasticsearch under one or more contributor
* license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright
* ownership. Elasticsearch licenses this file to you under
* the Apache License, Version 2.0 (the "License"); you may
* not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing,
* software distributed under the License is distributed on an
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
* KIND, either express or implied. See the License for the
* specific language governing permissions and limitations
* under the License.
*/
package org.elasticsearch.action.admin.indices.optimize;
import org.elasticsearch.action.support.broadcast.BroadcastShardResponse;
import org.elasticsearch.index.shard.ShardId;
/**
*
*/
class ShardOptimizeResponse extends BroadcastShardResponse {
ShardOptimizeResponse() {
}
ShardOptimizeResponse(ShardId shardId) {
super(shardId);
}
}

View File

@ -21,31 +21,29 @@ package org.elasticsearch.action.admin.indices.optimize;
import org.elasticsearch.action.ShardOperationFailedException; import org.elasticsearch.action.ShardOperationFailedException;
import org.elasticsearch.action.support.ActionFilters; import org.elasticsearch.action.support.ActionFilters;
import org.elasticsearch.action.support.DefaultShardOperationFailedException; import org.elasticsearch.action.support.broadcast.node.TransportBroadcastByNodeAction;
import org.elasticsearch.action.support.broadcast.BroadcastShardOperationFailedException;
import org.elasticsearch.action.support.broadcast.TransportBroadcastAction;
import org.elasticsearch.cluster.ClusterService; import org.elasticsearch.cluster.ClusterService;
import org.elasticsearch.cluster.ClusterState; import org.elasticsearch.cluster.ClusterState;
import org.elasticsearch.cluster.block.ClusterBlockException; import org.elasticsearch.cluster.block.ClusterBlockException;
import org.elasticsearch.cluster.block.ClusterBlockLevel; import org.elasticsearch.cluster.block.ClusterBlockLevel;
import org.elasticsearch.cluster.metadata.IndexNameExpressionResolver; import org.elasticsearch.cluster.metadata.IndexNameExpressionResolver;
import org.elasticsearch.cluster.routing.GroupShardsIterator;
import org.elasticsearch.cluster.routing.ShardRouting; import org.elasticsearch.cluster.routing.ShardRouting;
import org.elasticsearch.cluster.routing.ShardsIterator;
import org.elasticsearch.common.inject.Inject; import org.elasticsearch.common.inject.Inject;
import org.elasticsearch.common.io.stream.StreamInput;
import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.index.shard.IndexShard; import org.elasticsearch.index.shard.IndexShard;
import org.elasticsearch.indices.IndicesService; import org.elasticsearch.indices.IndicesService;
import org.elasticsearch.threadpool.ThreadPool; import org.elasticsearch.threadpool.ThreadPool;
import org.elasticsearch.transport.TransportService; import org.elasticsearch.transport.TransportService;
import java.util.ArrayList; import java.io.IOException;
import java.util.List; import java.util.List;
import java.util.concurrent.atomic.AtomicReferenceArray;
/** /**
* Optimize index/indices action. * Optimize index/indices action.
*/ */
public class TransportOptimizeAction extends TransportBroadcastAction<OptimizeRequest, OptimizeResponse, ShardOptimizeRequest, ShardOptimizeResponse> { public class TransportOptimizeAction extends TransportBroadcastByNodeAction<OptimizeRequest, OptimizeResponse, TransportBroadcastByNodeAction.EmptyResult> {
private final IndicesService indicesService; private final IndicesService indicesService;
@ -54,55 +52,40 @@ public class TransportOptimizeAction extends TransportBroadcastAction<OptimizeRe
TransportService transportService, IndicesService indicesService, TransportService transportService, IndicesService indicesService,
ActionFilters actionFilters, IndexNameExpressionResolver indexNameExpressionResolver) { ActionFilters actionFilters, IndexNameExpressionResolver indexNameExpressionResolver) {
super(settings, OptimizeAction.NAME, threadPool, clusterService, transportService, actionFilters, indexNameExpressionResolver, super(settings, OptimizeAction.NAME, threadPool, clusterService, transportService, actionFilters, indexNameExpressionResolver,
OptimizeRequest.class, ShardOptimizeRequest.class, ThreadPool.Names.OPTIMIZE); OptimizeRequest.class, ThreadPool.Names.OPTIMIZE);
this.indicesService = indicesService; this.indicesService = indicesService;
} }
@Override @Override
protected OptimizeResponse newResponse(OptimizeRequest request, AtomicReferenceArray shardsResponses, ClusterState clusterState) { protected EmptyResult readShardResult(StreamInput in) throws IOException {
int successfulShards = 0; return EmptyResult.readEmptyResultFrom(in);
int failedShards = 0;
List<ShardOperationFailedException> shardFailures = null;
for (int i = 0; i < shardsResponses.length(); i++) {
Object shardResponse = shardsResponses.get(i);
if (shardResponse == null) {
// a non active shard, ignore...
} else if (shardResponse instanceof BroadcastShardOperationFailedException) {
failedShards++;
if (shardFailures == null) {
shardFailures = new ArrayList<>();
}
shardFailures.add(new DefaultShardOperationFailedException((BroadcastShardOperationFailedException) shardResponse));
} else {
successfulShards++;
}
}
return new OptimizeResponse(shardsResponses.length(), successfulShards, failedShards, shardFailures);
} }
@Override @Override
protected ShardOptimizeRequest newShardRequest(int numShards, ShardRouting shard, OptimizeRequest request) { protected OptimizeResponse newResponse(OptimizeRequest request, int totalShards, int successfulShards, int failedShards, List<EmptyResult> responses, List<ShardOperationFailedException> shardFailures, ClusterState clusterState) {
return new ShardOptimizeRequest(shard.shardId(), request); return new OptimizeResponse(totalShards, successfulShards, failedShards, shardFailures);
} }
@Override @Override
protected ShardOptimizeResponse newShardResponse() { protected OptimizeRequest readRequestFrom(StreamInput in) throws IOException {
return new ShardOptimizeResponse(); final OptimizeRequest request = new OptimizeRequest();
request.readFrom(in);
return request;
} }
@Override @Override
protected ShardOptimizeResponse shardOperation(ShardOptimizeRequest request) { protected EmptyResult shardOperation(OptimizeRequest request, ShardRouting shardRouting) {
IndexShard indexShard = indicesService.indexServiceSafe(request.shardId().getIndex()).shardSafe(request.shardId().id()); IndexShard indexShard = indicesService.indexServiceSafe(shardRouting.shardId().getIndex()).shardSafe(shardRouting.shardId().id());
indexShard.optimize(request.optimizeRequest()); indexShard.optimize(request);
return new ShardOptimizeResponse(request.shardId()); return EmptyResult.INSTANCE;
} }
/** /**
* The refresh request works against *all* shards. * The refresh request works against *all* shards.
*/ */
@Override @Override
protected GroupShardsIterator shards(ClusterState clusterState, OptimizeRequest request, String[] concreteIndices) { protected ShardsIterator shards(ClusterState clusterState, OptimizeRequest request, String[] concreteIndices) {
return clusterState.routingTable().allActiveShardsGrouped(concreteIndices, true); return clusterState.routingTable().allShards(concreteIndices);
} }
@Override @Override

View File

@ -25,6 +25,7 @@ import org.elasticsearch.common.io.stream.StreamInput;
import org.elasticsearch.common.io.stream.StreamOutput; import org.elasticsearch.common.io.stream.StreamOutput;
import org.elasticsearch.common.xcontent.ToXContent; import org.elasticsearch.common.xcontent.ToXContent;
import org.elasticsearch.common.xcontent.XContentBuilder; import org.elasticsearch.common.xcontent.XContentBuilder;
import org.elasticsearch.indices.recovery.RecoveryState;
import java.io.IOException; import java.io.IOException;
import java.util.ArrayList; import java.util.ArrayList;
@ -38,7 +39,7 @@ import java.util.Map;
public class RecoveryResponse extends BroadcastResponse implements ToXContent { public class RecoveryResponse extends BroadcastResponse implements ToXContent {
private boolean detailed = false; private boolean detailed = false;
private Map<String, List<ShardRecoveryResponse>> shardResponses = new HashMap<>(); private Map<String, List<RecoveryState>> shardRecoveryStates = new HashMap<>();
public RecoveryResponse() { } public RecoveryResponse() { }
@ -50,18 +51,18 @@ public class RecoveryResponse extends BroadcastResponse implements ToXContent {
* @param successfulShards Count of shards successfully processed * @param successfulShards Count of shards successfully processed
* @param failedShards Count of shards which failed to process * @param failedShards Count of shards which failed to process
* @param detailed Display detailed metrics * @param detailed Display detailed metrics
* @param shardResponses Map of indices to shard recovery information * @param shardRecoveryStates Map of indices to shard recovery information
* @param shardFailures List of failures processing shards * @param shardFailures List of failures processing shards
*/ */
public RecoveryResponse(int totalShards, int successfulShards, int failedShards, boolean detailed, public RecoveryResponse(int totalShards, int successfulShards, int failedShards, boolean detailed,
Map<String, List<ShardRecoveryResponse>> shardResponses, List<ShardOperationFailedException> shardFailures) { Map<String, List<RecoveryState>> shardRecoveryStates, List<ShardOperationFailedException> shardFailures) {
super(totalShards, successfulShards, failedShards, shardFailures); super(totalShards, successfulShards, failedShards, shardFailures);
this.shardResponses = shardResponses; this.shardRecoveryStates = shardRecoveryStates;
this.detailed = detailed; this.detailed = detailed;
} }
public boolean hasRecoveries() { public boolean hasRecoveries() {
return shardResponses.size() > 0; return shardRecoveryStates.size() > 0;
} }
public boolean detailed() { public boolean detailed() {
@ -72,23 +73,23 @@ public class RecoveryResponse extends BroadcastResponse implements ToXContent {
this.detailed = detailed; this.detailed = detailed;
} }
public Map<String, List<ShardRecoveryResponse>> shardResponses() { public Map<String, List<RecoveryState>> shardRecoveryStates() {
return shardResponses; return shardRecoveryStates;
} }
@Override @Override
public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException { public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException {
if (hasRecoveries()) { if (hasRecoveries()) {
for (String index : shardResponses.keySet()) { for (String index : shardRecoveryStates.keySet()) {
List<ShardRecoveryResponse> responses = shardResponses.get(index); List<RecoveryState> recoveryStates = shardRecoveryStates.get(index);
if (responses == null || responses.size() == 0) { if (recoveryStates == null || recoveryStates.size() == 0) {
continue; continue;
} }
builder.startObject(index); builder.startObject(index);
builder.startArray("shards"); builder.startArray("shards");
for (ShardRecoveryResponse recoveryResponse : responses) { for (RecoveryState recoveryState : recoveryStates) {
builder.startObject(); builder.startObject();
recoveryResponse.toXContent(builder, params); recoveryState.toXContent(builder, params);
builder.endObject(); builder.endObject();
} }
builder.endArray(); builder.endArray();
@ -101,12 +102,12 @@ public class RecoveryResponse extends BroadcastResponse implements ToXContent {
@Override @Override
public void writeTo(StreamOutput out) throws IOException { public void writeTo(StreamOutput out) throws IOException {
super.writeTo(out); super.writeTo(out);
out.writeVInt(shardResponses.size()); out.writeVInt(shardRecoveryStates.size());
for (Map.Entry<String, List<ShardRecoveryResponse>> entry : shardResponses.entrySet()) { for (Map.Entry<String, List<RecoveryState>> entry : shardRecoveryStates.entrySet()) {
out.writeString(entry.getKey()); out.writeString(entry.getKey());
out.writeVInt(entry.getValue().size()); out.writeVInt(entry.getValue().size());
for (ShardRecoveryResponse recoveryResponse : entry.getValue()) { for (RecoveryState recoveryState : entry.getValue()) {
recoveryResponse.writeTo(out); recoveryState.writeTo(out);
} }
} }
} }
@ -118,11 +119,11 @@ public class RecoveryResponse extends BroadcastResponse implements ToXContent {
for (int i = 0; i < size; i++) { for (int i = 0; i < size; i++) {
String s = in.readString(); String s = in.readString();
int listSize = in.readVInt(); int listSize = in.readVInt();
List<ShardRecoveryResponse> list = new ArrayList<>(listSize); List<RecoveryState> list = new ArrayList<>(listSize);
for (int j = 0; j < listSize; j++) { for (int j = 0; j < listSize; j++) {
list.add(ShardRecoveryResponse.readShardRecoveryResponse(in)); list.add(RecoveryState.readRecoveryState(in));
} }
shardResponses.put(s, list); shardRecoveryStates.put(s, list);
} }
} }
} }

View File

@ -1,100 +0,0 @@
/*
* Licensed to Elasticsearch under one or more contributor
* license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright
* ownership. Elasticsearch licenses this file to you under
* the Apache License, Version 2.0 (the "License"); you may
* not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing,
* software distributed under the License is distributed on an
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
* KIND, either express or implied. See the License for the
* specific language governing permissions and limitations
* under the License.
*/
package org.elasticsearch.action.admin.indices.recovery;
import org.elasticsearch.action.support.broadcast.BroadcastShardResponse;
import org.elasticsearch.common.Nullable;
import org.elasticsearch.common.io.stream.StreamInput;
import org.elasticsearch.common.io.stream.StreamOutput;
import org.elasticsearch.common.xcontent.ToXContent;
import org.elasticsearch.common.xcontent.XContentBuilder;
import org.elasticsearch.index.shard.ShardId;
import org.elasticsearch.indices.recovery.RecoveryState;
import java.io.IOException;
/**
* Information regarding the recovery state of a shard.
*/
public class ShardRecoveryResponse extends BroadcastShardResponse implements ToXContent {
RecoveryState recoveryState;
public ShardRecoveryResponse() { }
/**
* Constructs shard recovery information for the given index and shard id.
*
* @param shardId Id of the shard
*/
ShardRecoveryResponse(ShardId shardId) {
super(shardId);
}
/**
* Sets the recovery state information for the shard.
*
* @param recoveryState Recovery state
*/
public void recoveryState(RecoveryState recoveryState) {
this.recoveryState = recoveryState;
}
/**
* Gets the recovery state information for the shard. Null if shard wasn't recovered / recovery didn't start yet.
*
* @return Recovery state
*/
@Nullable
public RecoveryState recoveryState() {
return recoveryState;
}
@Override
public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException {
recoveryState.toXContent(builder, params);
return builder;
}
@Override
public void writeTo(StreamOutput out) throws IOException {
super.writeTo(out);
recoveryState.writeTo(out);
}
@Override
public void readFrom(StreamInput in) throws IOException {
super.readFrom(in);
recoveryState = RecoveryState.readRecoveryState(in);
}
/**
* Builds a new ShardRecoveryResponse from the give input stream.
*
* @param in Input stream
* @return A new ShardRecoveryResponse
* @throws IOException
*/
public static ShardRecoveryResponse readShardRecoveryResponse(StreamInput in) throws IOException {
ShardRecoveryResponse response = new ShardRecoveryResponse();
response.readFrom(in);
return response;
}
}

View File

@ -19,40 +19,37 @@
package org.elasticsearch.action.admin.indices.recovery; package org.elasticsearch.action.admin.indices.recovery;
import com.google.common.collect.Maps;
import org.elasticsearch.action.ShardOperationFailedException; import org.elasticsearch.action.ShardOperationFailedException;
import org.elasticsearch.action.support.ActionFilters; import org.elasticsearch.action.support.ActionFilters;
import org.elasticsearch.action.support.DefaultShardOperationFailedException; import org.elasticsearch.action.support.broadcast.node.TransportBroadcastByNodeAction;
import org.elasticsearch.action.support.broadcast.BroadcastShardOperationFailedException;
import org.elasticsearch.action.support.broadcast.BroadcastShardRequest;
import org.elasticsearch.action.support.broadcast.TransportBroadcastAction;
import org.elasticsearch.cluster.ClusterService; import org.elasticsearch.cluster.ClusterService;
import org.elasticsearch.cluster.ClusterState; import org.elasticsearch.cluster.ClusterState;
import org.elasticsearch.cluster.block.ClusterBlockException; import org.elasticsearch.cluster.block.ClusterBlockException;
import org.elasticsearch.cluster.block.ClusterBlockLevel; import org.elasticsearch.cluster.block.ClusterBlockLevel;
import org.elasticsearch.cluster.metadata.IndexNameExpressionResolver; import org.elasticsearch.cluster.metadata.IndexNameExpressionResolver;
import org.elasticsearch.cluster.routing.GroupShardsIterator;
import org.elasticsearch.cluster.routing.ShardRouting; import org.elasticsearch.cluster.routing.ShardRouting;
import org.elasticsearch.cluster.routing.ShardsIterator;
import org.elasticsearch.common.inject.Inject; import org.elasticsearch.common.inject.Inject;
import org.elasticsearch.common.io.stream.StreamInput;
import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.index.IndexService; import org.elasticsearch.index.IndexService;
import org.elasticsearch.index.shard.IndexShard; import org.elasticsearch.index.shard.IndexShard;
import org.elasticsearch.index.shard.ShardId;
import org.elasticsearch.indices.IndicesService; import org.elasticsearch.indices.IndicesService;
import org.elasticsearch.indices.recovery.RecoveryState; import org.elasticsearch.indices.recovery.RecoveryState;
import org.elasticsearch.threadpool.ThreadPool; import org.elasticsearch.threadpool.ThreadPool;
import org.elasticsearch.transport.TransportService; import org.elasticsearch.transport.TransportService;
import java.io.IOException;
import java.util.ArrayList; import java.util.ArrayList;
import java.util.HashMap;
import java.util.List; import java.util.List;
import java.util.Map; import java.util.Map;
import java.util.concurrent.atomic.AtomicReferenceArray;
/** /**
* Transport action for shard recovery operation. This transport action does not actually * Transport action for shard recovery operation. This transport action does not actually
* perform shard recovery, it only reports on recoveries (both active and complete). * perform shard recovery, it only reports on recoveries (both active and complete).
*/ */
public class TransportRecoveryAction extends TransportBroadcastAction<RecoveryRequest, RecoveryResponse, TransportRecoveryAction.ShardRecoveryRequest, ShardRecoveryResponse> { public class TransportRecoveryAction extends TransportBroadcastByNodeAction<RecoveryRequest, RecoveryResponse, RecoveryState> {
private final IndicesService indicesService; private final IndicesService indicesService;
@ -61,84 +58,55 @@ public class TransportRecoveryAction extends TransportBroadcastAction<RecoveryRe
TransportService transportService, IndicesService indicesService, TransportService transportService, IndicesService indicesService,
ActionFilters actionFilters, IndexNameExpressionResolver indexNameExpressionResolver) { ActionFilters actionFilters, IndexNameExpressionResolver indexNameExpressionResolver) {
super(settings, RecoveryAction.NAME, threadPool, clusterService, transportService, actionFilters, indexNameExpressionResolver, super(settings, RecoveryAction.NAME, threadPool, clusterService, transportService, actionFilters, indexNameExpressionResolver,
RecoveryRequest.class, ShardRecoveryRequest.class, ThreadPool.Names.MANAGEMENT); RecoveryRequest.class, ThreadPool.Names.MANAGEMENT);
this.indicesService = indicesService; this.indicesService = indicesService;
} }
@Override @Override
protected RecoveryResponse newResponse(RecoveryRequest request, AtomicReferenceArray shardsResponses, ClusterState clusterState) { protected RecoveryState readShardResult(StreamInput in) throws IOException {
return RecoveryState.readRecoveryState(in);
}
int successfulShards = 0;
int failedShards = 0;
List<ShardOperationFailedException> shardFailures = null;
Map<String, List<ShardRecoveryResponse>> shardResponses = new HashMap<>();
for (int i = 0; i < shardsResponses.length(); i++) { @Override
Object shardResponse = shardsResponses.get(i); protected RecoveryResponse newResponse(RecoveryRequest request, int totalShards, int successfulShards, int failedShards, List<RecoveryState> responses, List<ShardOperationFailedException> shardFailures, ClusterState clusterState) {
if (shardResponse == null) { Map<String, List<RecoveryState>> shardResponses = Maps.newHashMap();
// simply ignore non active shards for (RecoveryState recoveryState : responses) {
} else if (shardResponse instanceof BroadcastShardOperationFailedException) { if (recoveryState == null) {
failedShards++; continue;
if (shardFailures == null) { }
shardFailures = new ArrayList<>(); String indexName = recoveryState.getShardId().getIndex();
if (!shardResponses.containsKey(indexName)) {
shardResponses.put(indexName, new ArrayList<RecoveryState>());
}
if (request.activeOnly()) {
if (recoveryState.getStage() != RecoveryState.Stage.DONE) {
shardResponses.get(indexName).add(recoveryState);
} }
shardFailures.add(new DefaultShardOperationFailedException((BroadcastShardOperationFailedException) shardResponse));
} else { } else {
ShardRecoveryResponse recoveryResponse = (ShardRecoveryResponse) shardResponse; shardResponses.get(indexName).add(recoveryState);
successfulShards++;
if (recoveryResponse.recoveryState() == null) {
// recovery not yet started
continue;
}
String indexName = recoveryResponse.getIndex();
List<ShardRecoveryResponse> responses = shardResponses.get(indexName);
if (responses == null) {
responses = new ArrayList<>();
shardResponses.put(indexName, responses);
}
if (request.activeOnly()) {
if (recoveryResponse.recoveryState().getStage() != RecoveryState.Stage.DONE) {
responses.add(recoveryResponse);
}
} else {
responses.add(recoveryResponse);
}
} }
} }
return new RecoveryResponse(totalShards, successfulShards, failedShards, request.detailed(), shardResponses, shardFailures);
return new RecoveryResponse(shardsResponses.length(), successfulShards,
failedShards, request.detailed(), shardResponses, shardFailures);
} }
@Override @Override
protected ShardRecoveryRequest newShardRequest(int numShards, ShardRouting shard, RecoveryRequest request) { protected RecoveryRequest readRequestFrom(StreamInput in) throws IOException {
return new ShardRecoveryRequest(shard.shardId(), request); final RecoveryRequest recoveryRequest = new RecoveryRequest();
recoveryRequest.readFrom(in);
return recoveryRequest;
} }
@Override @Override
protected ShardRecoveryResponse newShardResponse() { protected RecoveryState shardOperation(RecoveryRequest request, ShardRouting shardRouting) {
return new ShardRecoveryResponse(); IndexService indexService = indicesService.indexServiceSafe(shardRouting.shardId().getIndex());
IndexShard indexShard = indexService.shardSafe(shardRouting.shardId().id());
return indexShard.recoveryState();
} }
@Override @Override
protected ShardRecoveryResponse shardOperation(ShardRecoveryRequest request) { protected ShardsIterator shards(ClusterState state, RecoveryRequest request, String[] concreteIndices) {
return state.routingTable().allShardsIncludingRelocationTargets(concreteIndices);
IndexService indexService = indicesService.indexServiceSafe(request.shardId().getIndex());
IndexShard indexShard = indexService.shardSafe(request.shardId().id());
ShardRecoveryResponse shardRecoveryResponse = new ShardRecoveryResponse(request.shardId());
RecoveryState state = indexShard.recoveryState();
shardRecoveryResponse.recoveryState(state);
return shardRecoveryResponse;
}
@Override
protected GroupShardsIterator shards(ClusterState state, RecoveryRequest request, String[] concreteIndices) {
return state.routingTable().allAssignedShardsGrouped(concreteIndices, true, true);
} }
@Override @Override
@ -150,14 +118,4 @@ public class TransportRecoveryAction extends TransportBroadcastAction<RecoveryRe
protected ClusterBlockException checkRequestBlock(ClusterState state, RecoveryRequest request, String[] concreteIndices) { protected ClusterBlockException checkRequestBlock(ClusterState state, RecoveryRequest request, String[] concreteIndices) {
return state.blocks().indicesBlockedException(ClusterBlockLevel.READ, concreteIndices); return state.blocks().indicesBlockedException(ClusterBlockLevel.READ, concreteIndices);
} }
static class ShardRecoveryRequest extends BroadcastShardRequest {
ShardRecoveryRequest() {
}
ShardRecoveryRequest(ShardId shardId, RecoveryRequest request) {
super(shardId, request);
}
}
} }

View File

@ -24,7 +24,6 @@ import com.google.common.collect.Sets;
import org.apache.lucene.util.Accountable; import org.apache.lucene.util.Accountable;
import org.elasticsearch.action.ShardOperationFailedException; import org.elasticsearch.action.ShardOperationFailedException;
import org.elasticsearch.action.support.broadcast.BroadcastResponse; import org.elasticsearch.action.support.broadcast.BroadcastResponse;
import org.elasticsearch.cluster.ClusterState;
import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.io.stream.StreamInput;
import org.elasticsearch.common.io.stream.StreamOutput; import org.elasticsearch.common.io.stream.StreamOutput;
import org.elasticsearch.common.unit.ByteSizeValue; import org.elasticsearch.common.unit.ByteSizeValue;
@ -50,7 +49,7 @@ public class IndicesSegmentResponse extends BroadcastResponse implements ToXCont
} }
IndicesSegmentResponse(ShardSegments[] shards, ClusterState clusterState, int totalShards, int successfulShards, int failedShards, List<ShardOperationFailedException> shardFailures) { IndicesSegmentResponse(ShardSegments[] shards, int totalShards, int successfulShards, int failedShards, List<ShardOperationFailedException> shardFailures) {
super(totalShards, successfulShards, failedShards, shardFailures); super(totalShards, successfulShards, failedShards, shardFailures);
this.shards = shards; this.shards = shards;
} }
@ -63,7 +62,7 @@ public class IndicesSegmentResponse extends BroadcastResponse implements ToXCont
Set<String> indices = Sets.newHashSet(); Set<String> indices = Sets.newHashSet();
for (ShardSegments shard : shards) { for (ShardSegments shard : shards) {
indices.add(shard.getIndex()); indices.add(shard.getShardRouting().getIndex());
} }
for (String index : indices) { for (String index : indices) {

View File

@ -20,10 +20,10 @@
package org.elasticsearch.action.admin.indices.segments; package org.elasticsearch.action.admin.indices.segments;
import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableList;
import org.elasticsearch.action.support.broadcast.BroadcastShardResponse;
import org.elasticsearch.cluster.routing.ShardRouting; import org.elasticsearch.cluster.routing.ShardRouting;
import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.io.stream.StreamInput;
import org.elasticsearch.common.io.stream.StreamOutput; import org.elasticsearch.common.io.stream.StreamOutput;
import org.elasticsearch.common.io.stream.Streamable;
import org.elasticsearch.index.engine.Segment; import org.elasticsearch.index.engine.Segment;
import java.io.IOException; import java.io.IOException;
@ -33,7 +33,7 @@ import java.util.List;
import static org.elasticsearch.cluster.routing.ShardRouting.readShardRoutingEntry; import static org.elasticsearch.cluster.routing.ShardRouting.readShardRoutingEntry;
public class ShardSegments extends BroadcastShardResponse implements Iterable<Segment> { public class ShardSegments implements Streamable, Iterable<Segment> {
private ShardRouting shardRouting; private ShardRouting shardRouting;
@ -43,7 +43,6 @@ public class ShardSegments extends BroadcastShardResponse implements Iterable<Se
} }
ShardSegments(ShardRouting shardRouting, List<Segment> segments) { ShardSegments(ShardRouting shardRouting, List<Segment> segments) {
super(shardRouting.shardId());
this.shardRouting = shardRouting; this.shardRouting = shardRouting;
this.segments = segments; this.segments = segments;
} }
@ -89,7 +88,6 @@ public class ShardSegments extends BroadcastShardResponse implements Iterable<Se
@Override @Override
public void readFrom(StreamInput in) throws IOException { public void readFrom(StreamInput in) throws IOException {
super.readFrom(in);
shardRouting = readShardRoutingEntry(in); shardRouting = readShardRoutingEntry(in);
int size = in.readVInt(); int size = in.readVInt();
if (size == 0) { if (size == 0) {
@ -104,7 +102,6 @@ public class ShardSegments extends BroadcastShardResponse implements Iterable<Se
@Override @Override
public void writeTo(StreamOutput out) throws IOException { public void writeTo(StreamOutput out) throws IOException {
super.writeTo(out);
shardRouting.writeTo(out); shardRouting.writeTo(out);
out.writeVInt(segments.size()); out.writeVInt(segments.size());
for (Segment segment : segments) { for (Segment segment : segments) {

View File

@ -21,37 +21,30 @@ package org.elasticsearch.action.admin.indices.segments;
import org.elasticsearch.action.ShardOperationFailedException; import org.elasticsearch.action.ShardOperationFailedException;
import org.elasticsearch.action.support.ActionFilters; import org.elasticsearch.action.support.ActionFilters;
import org.elasticsearch.action.support.DefaultShardOperationFailedException; import org.elasticsearch.action.support.broadcast.node.TransportBroadcastByNodeAction;
import org.elasticsearch.action.support.broadcast.BroadcastShardOperationFailedException;
import org.elasticsearch.action.support.broadcast.BroadcastShardRequest;
import org.elasticsearch.action.support.broadcast.TransportBroadcastAction;
import org.elasticsearch.cluster.ClusterService; import org.elasticsearch.cluster.ClusterService;
import org.elasticsearch.cluster.ClusterState; import org.elasticsearch.cluster.ClusterState;
import org.elasticsearch.cluster.block.ClusterBlockException; import org.elasticsearch.cluster.block.ClusterBlockException;
import org.elasticsearch.cluster.block.ClusterBlockLevel; import org.elasticsearch.cluster.block.ClusterBlockLevel;
import org.elasticsearch.cluster.metadata.IndexNameExpressionResolver; import org.elasticsearch.cluster.metadata.IndexNameExpressionResolver;
import org.elasticsearch.cluster.routing.GroupShardsIterator;
import org.elasticsearch.cluster.routing.ShardRouting; import org.elasticsearch.cluster.routing.ShardRouting;
import org.elasticsearch.cluster.routing.ShardsIterator;
import org.elasticsearch.common.inject.Inject; import org.elasticsearch.common.inject.Inject;
import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.io.stream.StreamInput;
import org.elasticsearch.common.io.stream.StreamOutput;
import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.index.IndexService; import org.elasticsearch.index.IndexService;
import org.elasticsearch.index.shard.IndexShard; import org.elasticsearch.index.shard.IndexShard;
import org.elasticsearch.index.shard.ShardId;
import org.elasticsearch.indices.IndicesService; import org.elasticsearch.indices.IndicesService;
import org.elasticsearch.threadpool.ThreadPool; import org.elasticsearch.threadpool.ThreadPool;
import org.elasticsearch.transport.TransportService; import org.elasticsearch.transport.TransportService;
import java.io.IOException; import java.io.IOException;
import java.util.ArrayList;
import java.util.List; import java.util.List;
import java.util.concurrent.atomic.AtomicReferenceArray;
/** /**
* *
*/ */
public class TransportIndicesSegmentsAction extends TransportBroadcastAction<IndicesSegmentsRequest, IndicesSegmentResponse, TransportIndicesSegmentsAction.IndexShardSegmentRequest, ShardSegments> { public class TransportIndicesSegmentsAction extends TransportBroadcastByNodeAction<IndicesSegmentsRequest, IndicesSegmentResponse, ShardSegments> {
private final IndicesService indicesService; private final IndicesService indicesService;
@ -59,7 +52,7 @@ public class TransportIndicesSegmentsAction extends TransportBroadcastAction<Ind
public TransportIndicesSegmentsAction(Settings settings, ThreadPool threadPool, ClusterService clusterService, TransportService transportService, public TransportIndicesSegmentsAction(Settings settings, ThreadPool threadPool, ClusterService clusterService, TransportService transportService,
IndicesService indicesService, ActionFilters actionFilters, IndexNameExpressionResolver indexNameExpressionResolver) { IndicesService indicesService, ActionFilters actionFilters, IndexNameExpressionResolver indexNameExpressionResolver) {
super(settings, IndicesSegmentsAction.NAME, threadPool, clusterService, transportService, actionFilters, indexNameExpressionResolver, super(settings, IndicesSegmentsAction.NAME, threadPool, clusterService, transportService, actionFilters, indexNameExpressionResolver,
IndicesSegmentsRequest.class, TransportIndicesSegmentsAction.IndexShardSegmentRequest.class, ThreadPool.Names.MANAGEMENT); IndicesSegmentsRequest.class, ThreadPool.Names.MANAGEMENT);
this.indicesService = indicesService; this.indicesService = indicesService;
} }
@ -67,8 +60,8 @@ public class TransportIndicesSegmentsAction extends TransportBroadcastAction<Ind
* Segments goes across *all* active shards. * Segments goes across *all* active shards.
*/ */
@Override @Override
protected GroupShardsIterator shards(ClusterState clusterState, IndicesSegmentsRequest request, String[] concreteIndices) { protected ShardsIterator shards(ClusterState clusterState, IndicesSegmentsRequest request, String[] concreteIndices) {
return clusterState.routingTable().allActiveShardsGrouped(concreteIndices, true); return clusterState.routingTable().allShards(concreteIndices);
} }
@Override @Override
@ -82,68 +75,26 @@ public class TransportIndicesSegmentsAction extends TransportBroadcastAction<Ind
} }
@Override @Override
protected IndicesSegmentResponse newResponse(IndicesSegmentsRequest request, AtomicReferenceArray shardsResponses, ClusterState clusterState) { protected ShardSegments readShardResult(StreamInput in) throws IOException {
int successfulShards = 0; return ShardSegments.readShardSegments(in);
int failedShards = 0;
List<ShardOperationFailedException> shardFailures = null;
final List<ShardSegments> shards = new ArrayList<>();
for (int i = 0; i < shardsResponses.length(); i++) {
Object shardResponse = shardsResponses.get(i);
if (shardResponse == null) {
// simply ignore non active shards
} else if (shardResponse instanceof BroadcastShardOperationFailedException) {
failedShards++;
if (shardFailures == null) {
shardFailures = new ArrayList<>();
}
shardFailures.add(new DefaultShardOperationFailedException((BroadcastShardOperationFailedException) shardResponse));
} else {
shards.add((ShardSegments) shardResponse);
successfulShards++;
}
}
return new IndicesSegmentResponse(shards.toArray(new ShardSegments[shards.size()]), clusterState, shardsResponses.length(), successfulShards, failedShards, shardFailures);
} }
@Override @Override
protected IndexShardSegmentRequest newShardRequest(int numShards, ShardRouting shard, IndicesSegmentsRequest request) { protected IndicesSegmentResponse newResponse(IndicesSegmentsRequest request, int totalShards, int successfulShards, int failedShards, List<ShardSegments> results, List<ShardOperationFailedException> shardFailures, ClusterState clusterState) {
return new IndexShardSegmentRequest(shard.shardId(), request); return new IndicesSegmentResponse(results.toArray(new ShardSegments[results.size()]), totalShards, successfulShards, failedShards, shardFailures);
} }
@Override @Override
protected ShardSegments newShardResponse() { protected IndicesSegmentsRequest readRequestFrom(StreamInput in) throws IOException {
return new ShardSegments(); final IndicesSegmentsRequest request = new IndicesSegmentsRequest();
request.readFrom(in);
return request;
} }
@Override @Override
protected ShardSegments shardOperation(IndexShardSegmentRequest request) { protected ShardSegments shardOperation(IndicesSegmentsRequest request, ShardRouting shardRouting) {
IndexService indexService = indicesService.indexServiceSafe(request.shardId().getIndex()); IndexService indexService = indicesService.indexServiceSafe(shardRouting.getIndex());
IndexShard indexShard = indexService.shardSafe(request.shardId().id()); IndexShard indexShard = indexService.shardSafe(shardRouting.id());
return new ShardSegments(indexShard.routingEntry(), indexShard.engine().segments(request.verbose)); return new ShardSegments(indexShard.routingEntry(), indexShard.engine().segments(request.verbose()));
}
static class IndexShardSegmentRequest extends BroadcastShardRequest {
boolean verbose;
IndexShardSegmentRequest() {
verbose = false;
}
IndexShardSegmentRequest(ShardId shardId, IndicesSegmentsRequest request) {
super(shardId, request);
verbose = request.verbose();
}
@Override
public void writeTo(StreamOutput out) throws IOException {
super.writeTo(out);
out.writeBoolean(verbose);
}
@Override
public void readFrom(StreamInput in) throws IOException {
super.readFrom(in);
verbose = in.readBoolean();
}
} }
} }

View File

@ -24,7 +24,6 @@ import com.google.common.collect.Maps;
import com.google.common.collect.Sets; import com.google.common.collect.Sets;
import org.elasticsearch.action.ShardOperationFailedException; import org.elasticsearch.action.ShardOperationFailedException;
import org.elasticsearch.action.support.broadcast.BroadcastResponse; import org.elasticsearch.action.support.broadcast.BroadcastResponse;
import org.elasticsearch.cluster.ClusterState;
import org.elasticsearch.cluster.routing.ShardRouting; import org.elasticsearch.cluster.routing.ShardRouting;
import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.io.stream.StreamInput;
import org.elasticsearch.common.io.stream.StreamOutput; import org.elasticsearch.common.io.stream.StreamOutput;
@ -51,7 +50,7 @@ public class IndicesStatsResponse extends BroadcastResponse implements ToXConten
} }
IndicesStatsResponse(ShardStats[] shards, ClusterState clusterState, int totalShards, int successfulShards, int failedShards, List<ShardOperationFailedException> shardFailures) { IndicesStatsResponse(ShardStats[] shards, int totalShards, int successfulShards, int failedShards, List<ShardOperationFailedException> shardFailures) {
super(totalShards, successfulShards, failedShards, shardFailures); super(totalShards, successfulShards, failedShards, shardFailures);
this.shards = shards; this.shards = shards;
} }
@ -90,7 +89,7 @@ public class IndicesStatsResponse extends BroadcastResponse implements ToXConten
Set<String> indices = Sets.newHashSet(); Set<String> indices = Sets.newHashSet();
for (ShardStats shard : shards) { for (ShardStats shard : shards) {
indices.add(shard.getIndex()); indices.add(shard.getShardRouting().getIndex());
} }
for (String index : indices) { for (String index : indices) {

View File

@ -19,11 +19,11 @@
package org.elasticsearch.action.admin.indices.stats; package org.elasticsearch.action.admin.indices.stats;
import org.elasticsearch.action.support.broadcast.BroadcastShardResponse;
import org.elasticsearch.cluster.routing.ShardRouting; import org.elasticsearch.cluster.routing.ShardRouting;
import org.elasticsearch.common.Nullable; import org.elasticsearch.common.Nullable;
import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.io.stream.StreamInput;
import org.elasticsearch.common.io.stream.StreamOutput; import org.elasticsearch.common.io.stream.StreamOutput;
import org.elasticsearch.common.io.stream.Streamable;
import org.elasticsearch.common.xcontent.ToXContent; import org.elasticsearch.common.xcontent.ToXContent;
import org.elasticsearch.common.xcontent.XContentBuilder; import org.elasticsearch.common.xcontent.XContentBuilder;
import org.elasticsearch.common.xcontent.XContentBuilderString; import org.elasticsearch.common.xcontent.XContentBuilderString;
@ -37,7 +37,7 @@ import static org.elasticsearch.cluster.routing.ShardRouting.readShardRoutingEnt
/** /**
*/ */
public class ShardStats extends BroadcastShardResponse implements ToXContent { public class ShardStats implements Streamable, ToXContent {
private ShardRouting shardRouting; private ShardRouting shardRouting;
private CommonStats commonStats; private CommonStats commonStats;
@Nullable @Nullable
@ -49,14 +49,13 @@ public class ShardStats extends BroadcastShardResponse implements ToXContent {
ShardStats() { ShardStats() {
} }
public ShardStats(IndexShard indexShard, CommonStatsFlags flags) { public ShardStats(ShardRouting routing, ShardPath shardPath, CommonStats commonStats, CommitStats commitStats) {
super(indexShard.shardId()); this.shardRouting = routing;
this.shardRouting = indexShard.routingEntry(); this.dataPath = shardPath.getRootDataPath().toString();
this.dataPath = indexShard.shardPath().getRootDataPath().toString(); this.statePath = shardPath.getRootStatePath().toString();
this.statePath = indexShard.shardPath().getRootStatePath().toString(); this.isCustomDataPath = shardPath.isCustomDataPath();
this.isCustomDataPath = indexShard.shardPath().isCustomDataPath(); this.commitStats = commitStats;
this.commonStats = new CommonStats(indexShard, flags); this.commonStats = commonStats;
this.commitStats = indexShard.commitStats();
} }
/** /**
@ -94,7 +93,6 @@ public class ShardStats extends BroadcastShardResponse implements ToXContent {
@Override @Override
public void readFrom(StreamInput in) throws IOException { public void readFrom(StreamInput in) throws IOException {
super.readFrom(in);
shardRouting = readShardRoutingEntry(in); shardRouting = readShardRoutingEntry(in);
commonStats = CommonStats.readCommonStats(in); commonStats = CommonStats.readCommonStats(in);
commitStats = CommitStats.readOptionalCommitStatsFrom(in); commitStats = CommitStats.readOptionalCommitStatsFrom(in);
@ -105,7 +103,6 @@ public class ShardStats extends BroadcastShardResponse implements ToXContent {
@Override @Override
public void writeTo(StreamOutput out) throws IOException { public void writeTo(StreamOutput out) throws IOException {
super.writeTo(out);
shardRouting.writeTo(out); shardRouting.writeTo(out);
commonStats.writeTo(out); commonStats.writeTo(out);
out.writeOptionalStreamable(commitStats); out.writeOptionalStreamable(commitStats);
@ -146,5 +143,4 @@ public class ShardStats extends BroadcastShardResponse implements ToXContent {
static final XContentBuilderString NODE = new XContentBuilderString("node"); static final XContentBuilderString NODE = new XContentBuilderString("node");
static final XContentBuilderString RELOCATING_NODE = new XContentBuilderString("relocating_node"); static final XContentBuilderString RELOCATING_NODE = new XContentBuilderString("relocating_node");
} }
} }

View File

@ -21,37 +21,30 @@ package org.elasticsearch.action.admin.indices.stats;
import org.elasticsearch.action.ShardOperationFailedException; import org.elasticsearch.action.ShardOperationFailedException;
import org.elasticsearch.action.support.ActionFilters; import org.elasticsearch.action.support.ActionFilters;
import org.elasticsearch.action.support.DefaultShardOperationFailedException; import org.elasticsearch.action.support.broadcast.node.TransportBroadcastByNodeAction;
import org.elasticsearch.action.support.broadcast.BroadcastShardOperationFailedException;
import org.elasticsearch.action.support.broadcast.BroadcastShardRequest;
import org.elasticsearch.action.support.broadcast.TransportBroadcastAction;
import org.elasticsearch.cluster.ClusterService; import org.elasticsearch.cluster.ClusterService;
import org.elasticsearch.cluster.ClusterState; import org.elasticsearch.cluster.ClusterState;
import org.elasticsearch.cluster.block.ClusterBlockException; import org.elasticsearch.cluster.block.ClusterBlockException;
import org.elasticsearch.cluster.block.ClusterBlockLevel; import org.elasticsearch.cluster.block.ClusterBlockLevel;
import org.elasticsearch.cluster.metadata.IndexNameExpressionResolver; import org.elasticsearch.cluster.metadata.IndexNameExpressionResolver;
import org.elasticsearch.cluster.routing.GroupShardsIterator;
import org.elasticsearch.cluster.routing.ShardRouting; import org.elasticsearch.cluster.routing.ShardRouting;
import org.elasticsearch.cluster.routing.ShardsIterator;
import org.elasticsearch.common.inject.Inject; import org.elasticsearch.common.inject.Inject;
import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.io.stream.StreamInput;
import org.elasticsearch.common.io.stream.StreamOutput;
import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.index.IndexService; import org.elasticsearch.index.IndexService;
import org.elasticsearch.index.shard.IndexShard; import org.elasticsearch.index.shard.IndexShard;
import org.elasticsearch.index.shard.ShardId;
import org.elasticsearch.index.shard.ShardNotFoundException; import org.elasticsearch.index.shard.ShardNotFoundException;
import org.elasticsearch.indices.IndicesService; import org.elasticsearch.indices.IndicesService;
import org.elasticsearch.threadpool.ThreadPool; import org.elasticsearch.threadpool.ThreadPool;
import org.elasticsearch.transport.TransportService; import org.elasticsearch.transport.TransportService;
import java.io.IOException; import java.io.IOException;
import java.util.ArrayList;
import java.util.List; import java.util.List;
import java.util.concurrent.atomic.AtomicReferenceArray;
/** /**
*/ */
public class TransportIndicesStatsAction extends TransportBroadcastAction<IndicesStatsRequest, IndicesStatsResponse, TransportIndicesStatsAction.IndexShardStatsRequest, ShardStats> { public class TransportIndicesStatsAction extends TransportBroadcastByNodeAction<IndicesStatsRequest, IndicesStatsResponse, ShardStats> {
private final IndicesService indicesService; private final IndicesService indicesService;
@ -60,7 +53,7 @@ public class TransportIndicesStatsAction extends TransportBroadcastAction<Indice
TransportService transportService, IndicesService indicesService, TransportService transportService, IndicesService indicesService,
ActionFilters actionFilters, IndexNameExpressionResolver indexNameExpressionResolver) { ActionFilters actionFilters, IndexNameExpressionResolver indexNameExpressionResolver) {
super(settings, IndicesStatsAction.NAME, threadPool, clusterService, transportService, actionFilters, indexNameExpressionResolver, super(settings, IndicesStatsAction.NAME, threadPool, clusterService, transportService, actionFilters, indexNameExpressionResolver,
IndicesStatsRequest.class, IndexShardStatsRequest.class, ThreadPool.Names.MANAGEMENT); IndicesStatsRequest.class, ThreadPool.Names.MANAGEMENT);
this.indicesService = indicesService; this.indicesService = indicesService;
} }
@ -68,8 +61,8 @@ public class TransportIndicesStatsAction extends TransportBroadcastAction<Indice
* Status goes across *all* shards. * Status goes across *all* shards.
*/ */
@Override @Override
protected GroupShardsIterator shards(ClusterState clusterState, IndicesStatsRequest request, String[] concreteIndices) { protected ShardsIterator shards(ClusterState clusterState, IndicesStatsRequest request, String[] concreteIndices) {
return clusterState.routingTable().allAssignedShardsGrouped(concreteIndices, true); return clusterState.routingTable().allShards(concreteIndices);
} }
@Override @Override
@ -82,45 +75,27 @@ public class TransportIndicesStatsAction extends TransportBroadcastAction<Indice
return state.blocks().indicesBlockedException(ClusterBlockLevel.METADATA_READ, concreteIndices); return state.blocks().indicesBlockedException(ClusterBlockLevel.METADATA_READ, concreteIndices);
} }
@Override @Override
protected IndicesStatsResponse newResponse(IndicesStatsRequest request, AtomicReferenceArray shardsResponses, ClusterState clusterState) { protected ShardStats readShardResult(StreamInput in) throws IOException {
int successfulShards = 0; return ShardStats.readShardStats(in);
int failedShards = 0;
List<ShardOperationFailedException> shardFailures = null;
final List<ShardStats> shards = new ArrayList<>();
for (int i = 0; i < shardsResponses.length(); i++) {
Object shardResponse = shardsResponses.get(i);
if (shardResponse == null) {
// simply ignore non active shards
} else if (shardResponse instanceof BroadcastShardOperationFailedException) {
failedShards++;
if (shardFailures == null) {
shardFailures = new ArrayList<>();
}
shardFailures.add(new DefaultShardOperationFailedException((BroadcastShardOperationFailedException) shardResponse));
} else {
shards.add((ShardStats) shardResponse);
successfulShards++;
}
}
return new IndicesStatsResponse(shards.toArray(new ShardStats[shards.size()]), clusterState, shardsResponses.length(), successfulShards, failedShards, shardFailures);
} }
@Override @Override
protected IndexShardStatsRequest newShardRequest(int numShards, ShardRouting shard, IndicesStatsRequest request) { protected IndicesStatsResponse newResponse(IndicesStatsRequest request, int totalShards, int successfulShards, int failedShards, List<ShardStats> responses, List<ShardOperationFailedException> shardFailures, ClusterState clusterState) {
return new IndexShardStatsRequest(shard.shardId(), request); return new IndicesStatsResponse(responses.toArray(new ShardStats[responses.size()]), totalShards, successfulShards, failedShards, shardFailures);
} }
@Override @Override
protected ShardStats newShardResponse() { protected IndicesStatsRequest readRequestFrom(StreamInput in) throws IOException {
return new ShardStats(); IndicesStatsRequest request = new IndicesStatsRequest();
request.readFrom(in);
return request;
} }
@Override @Override
protected ShardStats shardOperation(IndexShardStatsRequest request) { protected ShardStats shardOperation(IndicesStatsRequest request, ShardRouting shardRouting) {
IndexService indexService = indicesService.indexServiceSafe(request.shardId().getIndex()); IndexService indexService = indicesService.indexServiceSafe(shardRouting.shardId().getIndex());
IndexShard indexShard = indexService.shardSafe(request.shardId().id()); IndexShard indexShard = indexService.shardSafe(shardRouting.shardId().id());
// if we don't have the routing entry yet, we need it stats wise, we treat it as if the shard is not ready yet // if we don't have the routing entry yet, we need it stats wise, we treat it as if the shard is not ready yet
if (indexShard.routingEntry() == null) { if (indexShard.routingEntry() == null) {
throw new ShardNotFoundException(indexShard.shardId()); throw new ShardNotFoundException(indexShard.shardId());
@ -128,92 +103,65 @@ public class TransportIndicesStatsAction extends TransportBroadcastAction<Indice
CommonStatsFlags flags = new CommonStatsFlags().clear(); CommonStatsFlags flags = new CommonStatsFlags().clear();
if (request.request.docs()) { if (request.docs()) {
flags.set(CommonStatsFlags.Flag.Docs); flags.set(CommonStatsFlags.Flag.Docs);
} }
if (request.request.store()) { if (request.store()) {
flags.set(CommonStatsFlags.Flag.Store); flags.set(CommonStatsFlags.Flag.Store);
} }
if (request.request.indexing()) { if (request.indexing()) {
flags.set(CommonStatsFlags.Flag.Indexing); flags.set(CommonStatsFlags.Flag.Indexing);
flags.types(request.request.types()); flags.types(request.types());
} }
if (request.request.get()) { if (request.get()) {
flags.set(CommonStatsFlags.Flag.Get); flags.set(CommonStatsFlags.Flag.Get);
} }
if (request.request.search()) { if (request.search()) {
flags.set(CommonStatsFlags.Flag.Search); flags.set(CommonStatsFlags.Flag.Search);
flags.groups(request.request.groups()); flags.groups(request.groups());
} }
if (request.request.merge()) { if (request.merge()) {
flags.set(CommonStatsFlags.Flag.Merge); flags.set(CommonStatsFlags.Flag.Merge);
} }
if (request.request.refresh()) { if (request.refresh()) {
flags.set(CommonStatsFlags.Flag.Refresh); flags.set(CommonStatsFlags.Flag.Refresh);
} }
if (request.request.flush()) { if (request.flush()) {
flags.set(CommonStatsFlags.Flag.Flush); flags.set(CommonStatsFlags.Flag.Flush);
} }
if (request.request.warmer()) { if (request.warmer()) {
flags.set(CommonStatsFlags.Flag.Warmer); flags.set(CommonStatsFlags.Flag.Warmer);
} }
if (request.request.queryCache()) { if (request.queryCache()) {
flags.set(CommonStatsFlags.Flag.QueryCache); flags.set(CommonStatsFlags.Flag.QueryCache);
} }
if (request.request.fieldData()) { if (request.fieldData()) {
flags.set(CommonStatsFlags.Flag.FieldData); flags.set(CommonStatsFlags.Flag.FieldData);
flags.fieldDataFields(request.request.fieldDataFields()); flags.fieldDataFields(request.fieldDataFields());
} }
if (request.request.percolate()) { if (request.percolate()) {
flags.set(CommonStatsFlags.Flag.Percolate); flags.set(CommonStatsFlags.Flag.Percolate);
} }
if (request.request.segments()) { if (request.segments()) {
flags.set(CommonStatsFlags.Flag.Segments); flags.set(CommonStatsFlags.Flag.Segments);
} }
if (request.request.completion()) { if (request.completion()) {
flags.set(CommonStatsFlags.Flag.Completion); flags.set(CommonStatsFlags.Flag.Completion);
flags.completionDataFields(request.request.completionFields()); flags.completionDataFields(request.completionFields());
} }
if (request.request.translog()) { if (request.translog()) {
flags.set(CommonStatsFlags.Flag.Translog); flags.set(CommonStatsFlags.Flag.Translog);
} }
if (request.request.suggest()) { if (request.suggest()) {
flags.set(CommonStatsFlags.Flag.Suggest); flags.set(CommonStatsFlags.Flag.Suggest);
} }
if (request.request.requestCache()) { if (request.requestCache()) {
flags.set(CommonStatsFlags.Flag.RequestCache); flags.set(CommonStatsFlags.Flag.RequestCache);
} }
if (request.request.recovery()) { if (request.recovery()) {
flags.set(CommonStatsFlags.Flag.Recovery); flags.set(CommonStatsFlags.Flag.Recovery);
} }
return new ShardStats(indexShard, flags); return new ShardStats(indexShard.routingEntry(), indexShard.shardPath(), new CommonStats(indexShard, flags), indexShard.commitStats());
}
static class IndexShardStatsRequest extends BroadcastShardRequest {
// TODO if there are many indices, the request might hold a large indices array..., we don't really need to serialize it
IndicesStatsRequest request;
IndexShardStatsRequest() {
}
IndexShardStatsRequest(ShardId shardId, IndicesStatsRequest request) {
super(shardId, request);
this.request = request;
}
@Override
public void readFrom(StreamInput in) throws IOException {
super.readFrom(in);
request = new IndicesStatsRequest();
request.readFrom(in);
}
@Override
public void writeTo(StreamOutput out) throws IOException {
super.writeTo(out);
request.writeTo(out);
}
} }
} }

View File

@ -22,35 +22,31 @@ package org.elasticsearch.action.admin.indices.upgrade.get;
import org.elasticsearch.Version; import org.elasticsearch.Version;
import org.elasticsearch.action.ShardOperationFailedException; import org.elasticsearch.action.ShardOperationFailedException;
import org.elasticsearch.action.support.ActionFilters; import org.elasticsearch.action.support.ActionFilters;
import org.elasticsearch.action.support.DefaultShardOperationFailedException; import org.elasticsearch.action.support.broadcast.node.TransportBroadcastByNodeAction;
import org.elasticsearch.action.support.broadcast.BroadcastShardOperationFailedException;
import org.elasticsearch.action.support.broadcast.BroadcastShardRequest;
import org.elasticsearch.action.support.broadcast.TransportBroadcastAction;
import org.elasticsearch.cluster.ClusterService; import org.elasticsearch.cluster.ClusterService;
import org.elasticsearch.cluster.ClusterState; import org.elasticsearch.cluster.ClusterState;
import org.elasticsearch.cluster.block.ClusterBlockException; import org.elasticsearch.cluster.block.ClusterBlockException;
import org.elasticsearch.cluster.block.ClusterBlockLevel; import org.elasticsearch.cluster.block.ClusterBlockLevel;
import org.elasticsearch.cluster.metadata.IndexNameExpressionResolver; import org.elasticsearch.cluster.metadata.IndexNameExpressionResolver;
import org.elasticsearch.cluster.routing.GroupShardsIterator;
import org.elasticsearch.cluster.routing.ShardRouting; import org.elasticsearch.cluster.routing.ShardRouting;
import org.elasticsearch.cluster.routing.ShardsIterator;
import org.elasticsearch.common.inject.Inject; import org.elasticsearch.common.inject.Inject;
import org.elasticsearch.common.io.stream.StreamInput;
import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.index.IndexService; import org.elasticsearch.index.IndexService;
import org.elasticsearch.index.engine.Segment; import org.elasticsearch.index.engine.Segment;
import org.elasticsearch.index.shard.IndexShard; import org.elasticsearch.index.shard.IndexShard;
import org.elasticsearch.index.shard.ShardId;
import org.elasticsearch.indices.IndicesService; import org.elasticsearch.indices.IndicesService;
import org.elasticsearch.threadpool.ThreadPool; import org.elasticsearch.threadpool.ThreadPool;
import org.elasticsearch.transport.TransportService; import org.elasticsearch.transport.TransportService;
import java.util.ArrayList; import java.io.IOException;
import java.util.List; import java.util.List;
import java.util.concurrent.atomic.AtomicReferenceArray;
/** /**
* *
*/ */
public class TransportUpgradeStatusAction extends TransportBroadcastAction<UpgradeStatusRequest, UpgradeStatusResponse, TransportUpgradeStatusAction.IndexShardUpgradeStatusRequest, ShardUpgradeStatus> { public class TransportUpgradeStatusAction extends TransportBroadcastByNodeAction<UpgradeStatusRequest, UpgradeStatusResponse, ShardUpgradeStatus> {
private final IndicesService indicesService; private final IndicesService indicesService;
@ -58,7 +54,7 @@ public class TransportUpgradeStatusAction extends TransportBroadcastAction<Upgra
public TransportUpgradeStatusAction(Settings settings, ThreadPool threadPool, ClusterService clusterService, TransportService transportService, public TransportUpgradeStatusAction(Settings settings, ThreadPool threadPool, ClusterService clusterService, TransportService transportService,
IndicesService indicesService, ActionFilters actionFilters, IndexNameExpressionResolver indexNameExpressionResolver) { IndicesService indicesService, ActionFilters actionFilters, IndexNameExpressionResolver indexNameExpressionResolver) {
super(settings, UpgradeStatusAction.NAME, threadPool, clusterService, transportService, actionFilters, indexNameExpressionResolver, super(settings, UpgradeStatusAction.NAME, threadPool, clusterService, transportService, actionFilters, indexNameExpressionResolver,
UpgradeStatusRequest.class, IndexShardUpgradeStatusRequest.class, ThreadPool.Names.MANAGEMENT); UpgradeStatusRequest.class, ThreadPool.Names.MANAGEMENT);
this.indicesService = indicesService; this.indicesService = indicesService;
} }
@ -66,8 +62,8 @@ public class TransportUpgradeStatusAction extends TransportBroadcastAction<Upgra
* Getting upgrade stats from *all* active shards. * Getting upgrade stats from *all* active shards.
*/ */
@Override @Override
protected GroupShardsIterator shards(ClusterState clusterState, UpgradeStatusRequest request, String[] concreteIndices) { protected ShardsIterator shards(ClusterState clusterState, UpgradeStatusRequest request, String[] concreteIndices) {
return clusterState.routingTable().allActiveShardsGrouped(concreteIndices, true); return clusterState.routingTable().allShards(concreteIndices);
} }
@Override @Override
@ -81,43 +77,26 @@ public class TransportUpgradeStatusAction extends TransportBroadcastAction<Upgra
} }
@Override @Override
protected UpgradeStatusResponse newResponse(UpgradeStatusRequest request, AtomicReferenceArray shardsResponses, ClusterState clusterState) { protected ShardUpgradeStatus readShardResult(StreamInput in) throws IOException {
int successfulShards = 0; return ShardUpgradeStatus.readShardUpgradeStatus(in);
int failedShards = 0;
List<ShardOperationFailedException> shardFailures = null;
final List<ShardUpgradeStatus> shards = new ArrayList<>();
for (int i = 0; i < shardsResponses.length(); i++) {
Object shardResponse = shardsResponses.get(i);
if (shardResponse == null) {
// simply ignore non active shards
} else if (shardResponse instanceof BroadcastShardOperationFailedException) {
failedShards++;
if (shardFailures == null) {
shardFailures = new ArrayList<>();
}
shardFailures.add(new DefaultShardOperationFailedException((BroadcastShardOperationFailedException) shardResponse));
} else {
shards.add((ShardUpgradeStatus) shardResponse);
successfulShards++;
}
}
return new UpgradeStatusResponse(shards.toArray(new ShardUpgradeStatus[shards.size()]), shardsResponses.length(), successfulShards, failedShards, shardFailures);
} }
@Override @Override
protected IndexShardUpgradeStatusRequest newShardRequest(int numShards, ShardRouting shard, UpgradeStatusRequest request) { protected UpgradeStatusResponse newResponse(UpgradeStatusRequest request, int totalShards, int successfulShards, int failedShards, List<ShardUpgradeStatus> responses, List<ShardOperationFailedException> shardFailures, ClusterState clusterState) {
return new IndexShardUpgradeStatusRequest(shard.shardId(), request); return new UpgradeStatusResponse(responses.toArray(new ShardUpgradeStatus[responses.size()]), totalShards, successfulShards, failedShards, shardFailures);
} }
@Override @Override
protected ShardUpgradeStatus newShardResponse() { protected UpgradeStatusRequest readRequestFrom(StreamInput in) throws IOException {
return new ShardUpgradeStatus(); UpgradeStatusRequest request = new UpgradeStatusRequest();
request.readFrom(in);
return request;
} }
@Override @Override
protected ShardUpgradeStatus shardOperation(IndexShardUpgradeStatusRequest request) { protected ShardUpgradeStatus shardOperation(UpgradeStatusRequest request, ShardRouting shardRouting) {
IndexService indexService = indicesService.indexServiceSafe(request.shardId().getIndex()); IndexService indexService = indicesService.indexServiceSafe(shardRouting.shardId().getIndex());
IndexShard indexShard = indexService.shardSafe(request.shardId().id()); IndexShard indexShard = indexService.shardSafe(shardRouting.shardId().id());
List<Segment> segments = indexShard.engine().segments(false); List<Segment> segments = indexShard.engine().segments(false);
long total_bytes = 0; long total_bytes = 0;
long to_upgrade_bytes = 0; long to_upgrade_bytes = 0;
@ -136,16 +115,4 @@ public class TransportUpgradeStatusAction extends TransportBroadcastAction<Upgra
return new ShardUpgradeStatus(indexShard.routingEntry(), total_bytes, to_upgrade_bytes, to_upgrade_bytes_ancient); return new ShardUpgradeStatus(indexShard.routingEntry(), total_bytes, to_upgrade_bytes, to_upgrade_bytes_ancient);
} }
static class IndexShardUpgradeStatusRequest extends BroadcastShardRequest {
IndexShardUpgradeStatusRequest() {
}
IndexShardUpgradeStatusRequest(ShardId shardId, UpgradeStatusRequest request) {
super(shardId, request);
}
}
} }

View File

@ -21,10 +21,6 @@ package org.elasticsearch.action.admin.indices.upgrade.get;
import org.elasticsearch.action.support.broadcast.BroadcastRequest; import org.elasticsearch.action.support.broadcast.BroadcastRequest;
import org.elasticsearch.common.Strings; import org.elasticsearch.common.Strings;
import org.elasticsearch.common.io.stream.StreamInput;
import org.elasticsearch.common.io.stream.StreamOutput;
import java.io.IOException;
public class UpgradeStatusRequest extends BroadcastRequest<UpgradeStatusRequest> { public class UpgradeStatusRequest extends BroadcastRequest<UpgradeStatusRequest> {

View File

@ -36,14 +36,11 @@ import java.util.Map;
import java.util.Set; import java.util.Set;
public class UpgradeStatusResponse extends BroadcastResponse implements ToXContent { public class UpgradeStatusResponse extends BroadcastResponse implements ToXContent {
private ShardUpgradeStatus[] shards; private ShardUpgradeStatus[] shards;
private Map<String, IndexUpgradeStatus> indicesUpgradeStatus; private Map<String, IndexUpgradeStatus> indicesUpgradeStatus;
UpgradeStatusResponse() { UpgradeStatusResponse() {
} }
UpgradeStatusResponse(ShardUpgradeStatus[] shards, int totalShards, int successfulShards, int failedShards, List<ShardOperationFailedException> shardFailures) { UpgradeStatusResponse(ShardUpgradeStatus[] shards, int totalShards, int successfulShards, int failedShards, List<ShardOperationFailedException> shardFailures) {
@ -75,7 +72,6 @@ public class UpgradeStatusResponse extends BroadcastResponse implements ToXConte
return indicesUpgradeStats; return indicesUpgradeStats;
} }
@Override @Override
public void readFrom(StreamInput in) throws IOException { public void readFrom(StreamInput in) throws IOException {
super.readFrom(in); super.readFrom(in);
@ -120,8 +116,6 @@ public class UpgradeStatusResponse extends BroadcastResponse implements ToXConte
@Override @Override
public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException { public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException {
builder.byteSizeField(Fields.SIZE_IN_BYTES, Fields.SIZE, getTotalBytes()); builder.byteSizeField(Fields.SIZE_IN_BYTES, Fields.SIZE, getTotalBytes());
builder.byteSizeField(Fields.SIZE_TO_UPGRADE_IN_BYTES, Fields.SIZE_TO_UPGRADE, getToUpgradeBytes()); builder.byteSizeField(Fields.SIZE_TO_UPGRADE_IN_BYTES, Fields.SIZE_TO_UPGRADE, getToUpgradeBytes());
builder.byteSizeField(Fields.SIZE_TO_UPGRADE_ANCIENT_IN_BYTES, Fields.SIZE_TO_UPGRADE_ANCIENT, getToUpgradeBytesAncient()); builder.byteSizeField(Fields.SIZE_TO_UPGRADE_ANCIENT_IN_BYTES, Fields.SIZE_TO_UPGRADE_ANCIENT, getToUpgradeBytesAncient());
@ -163,10 +157,8 @@ public class UpgradeStatusResponse extends BroadcastResponse implements ToXConte
} }
builder.endObject(); builder.endObject();
} }
builder.endObject(); builder.endObject();
} }
builder.endObject(); builder.endObject();
} }
return builder; return builder;
@ -186,6 +178,5 @@ public class UpgradeStatusResponse extends BroadcastResponse implements ToXConte
static final XContentBuilderString SIZE_TO_UPGRADE_ANCIENT = new XContentBuilderString("size_to_upgrade_ancient"); static final XContentBuilderString SIZE_TO_UPGRADE_ANCIENT = new XContentBuilderString("size_to_upgrade_ancient");
static final XContentBuilderString SIZE_TO_UPGRADE_IN_BYTES = new XContentBuilderString("size_to_upgrade_in_bytes"); static final XContentBuilderString SIZE_TO_UPGRADE_IN_BYTES = new XContentBuilderString("size_to_upgrade_in_bytes");
static final XContentBuilderString SIZE_TO_UPGRADE_ANCIENT_IN_BYTES = new XContentBuilderString("size_to_upgrade_ancient_in_bytes"); static final XContentBuilderString SIZE_TO_UPGRADE_ANCIENT_IN_BYTES = new XContentBuilderString("size_to_upgrade_ancient_in_bytes");
} }
} }

View File

@ -20,9 +20,9 @@
package org.elasticsearch.action.admin.indices.upgrade.post; package org.elasticsearch.action.admin.indices.upgrade.post;
import org.elasticsearch.Version; import org.elasticsearch.Version;
import org.elasticsearch.action.support.broadcast.BroadcastShardResponse;
import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.io.stream.StreamInput;
import org.elasticsearch.common.io.stream.StreamOutput; import org.elasticsearch.common.io.stream.StreamOutput;
import org.elasticsearch.common.io.stream.Streamable;
import org.elasticsearch.index.shard.ShardId; import org.elasticsearch.index.shard.ShardId;
import java.io.IOException; import java.io.IOException;
@ -31,7 +31,9 @@ import java.text.ParseException;
/** /**
* *
*/ */
class ShardUpgradeResponse extends BroadcastShardResponse { class ShardUpgradeResult implements Streamable {
private ShardId shardId;
private org.apache.lucene.util.Version oldestLuceneSegment; private org.apache.lucene.util.Version oldestLuceneSegment;
@ -40,16 +42,20 @@ class ShardUpgradeResponse extends BroadcastShardResponse {
private boolean primary; private boolean primary;
ShardUpgradeResponse() { ShardUpgradeResult() {
} }
ShardUpgradeResponse(ShardId shardId, boolean primary, Version upgradeVersion, org.apache.lucene.util.Version oldestLuceneSegment) { ShardUpgradeResult(ShardId shardId, boolean primary, Version upgradeVersion, org.apache.lucene.util.Version oldestLuceneSegment) {
super(shardId); this.shardId = shardId;
this.primary = primary; this.primary = primary;
this.upgradeVersion = upgradeVersion; this.upgradeVersion = upgradeVersion;
this.oldestLuceneSegment = oldestLuceneSegment; this.oldestLuceneSegment = oldestLuceneSegment;
} }
public ShardId getShardId() {
return shardId;
}
public org.apache.lucene.util.Version oldestLuceneSegment() { public org.apache.lucene.util.Version oldestLuceneSegment() {
return this.oldestLuceneSegment; return this.oldestLuceneSegment;
} }
@ -65,7 +71,7 @@ class ShardUpgradeResponse extends BroadcastShardResponse {
@Override @Override
public void readFrom(StreamInput in) throws IOException { public void readFrom(StreamInput in) throws IOException {
super.readFrom(in); shardId = ShardId.readShardId(in);
primary = in.readBoolean(); primary = in.readBoolean();
upgradeVersion = Version.readVersion(in); upgradeVersion = Version.readVersion(in);
try { try {
@ -78,10 +84,9 @@ class ShardUpgradeResponse extends BroadcastShardResponse {
@Override @Override
public void writeTo(StreamOutput out) throws IOException { public void writeTo(StreamOutput out) throws IOException {
super.writeTo(out); shardId.writeTo(out);
out.writeBoolean(primary); out.writeBoolean(primary);
Version.writeVersion(upgradeVersion, out); Version.writeVersion(upgradeVersion, out);
out.writeString(oldestLuceneSegment.toString()); out.writeString(oldestLuceneSegment.toString());
} }
} }

View File

@ -24,32 +24,30 @@ import org.elasticsearch.action.ActionListener;
import org.elasticsearch.action.PrimaryMissingActionException; import org.elasticsearch.action.PrimaryMissingActionException;
import org.elasticsearch.action.ShardOperationFailedException; import org.elasticsearch.action.ShardOperationFailedException;
import org.elasticsearch.action.support.ActionFilters; import org.elasticsearch.action.support.ActionFilters;
import org.elasticsearch.action.support.DefaultShardOperationFailedException; import org.elasticsearch.action.support.broadcast.node.TransportBroadcastByNodeAction;
import org.elasticsearch.action.support.broadcast.BroadcastShardOperationFailedException;
import org.elasticsearch.action.support.broadcast.TransportBroadcastAction;
import org.elasticsearch.cluster.ClusterService; import org.elasticsearch.cluster.ClusterService;
import org.elasticsearch.cluster.ClusterState; import org.elasticsearch.cluster.ClusterState;
import org.elasticsearch.cluster.block.ClusterBlockException; import org.elasticsearch.cluster.block.ClusterBlockException;
import org.elasticsearch.cluster.block.ClusterBlockLevel; import org.elasticsearch.cluster.block.ClusterBlockLevel;
import org.elasticsearch.cluster.metadata.IndexNameExpressionResolver; import org.elasticsearch.cluster.metadata.IndexNameExpressionResolver;
import org.elasticsearch.cluster.metadata.MetaData; import org.elasticsearch.cluster.metadata.MetaData;
import org.elasticsearch.cluster.routing.GroupShardsIterator;
import org.elasticsearch.cluster.routing.IndexRoutingTable; import org.elasticsearch.cluster.routing.IndexRoutingTable;
import org.elasticsearch.cluster.routing.RoutingTable; import org.elasticsearch.cluster.routing.RoutingTable;
import org.elasticsearch.cluster.routing.ShardRouting; import org.elasticsearch.cluster.routing.ShardRouting;
import org.elasticsearch.cluster.routing.ShardsIterator;
import org.elasticsearch.common.collect.Tuple; import org.elasticsearch.common.collect.Tuple;
import org.elasticsearch.common.inject.Inject; import org.elasticsearch.common.inject.Inject;
import org.elasticsearch.common.io.stream.StreamInput;
import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.index.shard.IndexShard; import org.elasticsearch.index.shard.IndexShard;
import org.elasticsearch.indices.IndicesService; import org.elasticsearch.indices.IndicesService;
import org.elasticsearch.threadpool.ThreadPool; import org.elasticsearch.threadpool.ThreadPool;
import org.elasticsearch.transport.TransportService; import org.elasticsearch.transport.TransportService;
import java.util.ArrayList; import java.io.IOException;
import java.util.List; import java.util.List;
import java.util.Map; import java.util.Map;
import java.util.Set; import java.util.Set;
import java.util.concurrent.atomic.AtomicReferenceArray;
import static com.google.common.collect.Maps.newHashMap; import static com.google.common.collect.Maps.newHashMap;
import static com.google.common.collect.Sets.newHashSet; import static com.google.common.collect.Sets.newHashSet;
@ -57,7 +55,7 @@ import static com.google.common.collect.Sets.newHashSet;
/** /**
* Upgrade index/indices action. * Upgrade index/indices action.
*/ */
public class TransportUpgradeAction extends TransportBroadcastAction<UpgradeRequest, UpgradeResponse, ShardUpgradeRequest, ShardUpgradeResponse> { public class TransportUpgradeAction extends TransportBroadcastByNodeAction<UpgradeRequest, UpgradeResponse, ShardUpgradeResult> {
private final IndicesService indicesService; private final IndicesService indicesService;
@ -67,56 +65,40 @@ public class TransportUpgradeAction extends TransportBroadcastAction<UpgradeRequ
public TransportUpgradeAction(Settings settings, ThreadPool threadPool, ClusterService clusterService, public TransportUpgradeAction(Settings settings, ThreadPool threadPool, ClusterService clusterService,
TransportService transportService, IndicesService indicesService, ActionFilters actionFilters, TransportService transportService, IndicesService indicesService, ActionFilters actionFilters,
IndexNameExpressionResolver indexNameExpressionResolver, TransportUpgradeSettingsAction upgradeSettingsAction) { IndexNameExpressionResolver indexNameExpressionResolver, TransportUpgradeSettingsAction upgradeSettingsAction) {
super(settings, UpgradeAction.NAME, threadPool, clusterService, transportService, actionFilters, indexNameExpressionResolver, super(settings, UpgradeAction.NAME, threadPool, clusterService, transportService, actionFilters, indexNameExpressionResolver, UpgradeRequest.class, ThreadPool.Names.OPTIMIZE);
UpgradeRequest.class, ShardUpgradeRequest.class, ThreadPool.Names.OPTIMIZE);
this.indicesService = indicesService; this.indicesService = indicesService;
this.upgradeSettingsAction = upgradeSettingsAction; this.upgradeSettingsAction = upgradeSettingsAction;
} }
@Override @Override
protected UpgradeResponse newResponse(UpgradeRequest request, AtomicReferenceArray shardsResponses, ClusterState clusterState) { protected UpgradeResponse newResponse(UpgradeRequest request, int totalShards, int successfulShards, int failedShards, List<ShardUpgradeResult> shardUpgradeResults, List<ShardOperationFailedException> shardFailures, ClusterState clusterState) {
int successfulShards = 0;
int failedShards = 0;
List<ShardOperationFailedException> shardFailures = null;
Map<String, Integer> successfulPrimaryShards = newHashMap(); Map<String, Integer> successfulPrimaryShards = newHashMap();
Map<String, Tuple<Version, org.apache.lucene.util.Version>> versions = newHashMap(); Map<String, Tuple<Version, org.apache.lucene.util.Version>> versions = newHashMap();
for (int i = 0; i < shardsResponses.length(); i++) { for (ShardUpgradeResult result : shardUpgradeResults) {
Object shardResponse = shardsResponses.get(i); successfulShards++;
if (shardResponse == null) { String index = result.getShardId().getIndex();
// a non active shard, ignore... if (result.primary()) {
} else if (shardResponse instanceof BroadcastShardOperationFailedException) { Integer count = successfulPrimaryShards.get(index);
failedShards++; successfulPrimaryShards.put(index, count == null ? 1 : count + 1);
if (shardFailures == null) { }
shardFailures = new ArrayList<>(); Tuple<Version, org.apache.lucene.util.Version> versionTuple = versions.get(index);
} if (versionTuple == null) {
shardFailures.add(new DefaultShardOperationFailedException((BroadcastShardOperationFailedException) shardResponse)); versions.put(index, new Tuple<>(result.upgradeVersion(), result.oldestLuceneSegment()));
} else { } else {
successfulShards++; // We already have versions for this index - let's see if we need to update them based on the current shard
ShardUpgradeResponse shardUpgradeResponse = (ShardUpgradeResponse) shardResponse; Version version = versionTuple.v1();
String index = shardUpgradeResponse.getIndex(); org.apache.lucene.util.Version luceneVersion = versionTuple.v2();
if (shardUpgradeResponse.primary()) { // For the metadata we are interested in the _latest_ Elasticsearch version that was processing the metadata
Integer count = successfulPrimaryShards.get(index); // Since we rewrite the mapping during upgrade the metadata is always rewritten by the latest version
successfulPrimaryShards.put(index, count == null ? 1 : count + 1); if (result.upgradeVersion().after(versionTuple.v1())) {
version = result.upgradeVersion();
} }
Tuple<Version, org.apache.lucene.util.Version> versionTuple = versions.get(index); // For the lucene version we are interested in the _oldest_ lucene version since it determines the
if (versionTuple == null) { // oldest version that we need to support
versions.put(index, new Tuple<>(shardUpgradeResponse.upgradeVersion(), shardUpgradeResponse.oldestLuceneSegment())); if (result.oldestLuceneSegment().onOrAfter(versionTuple.v2()) == false) {
} else { luceneVersion = result.oldestLuceneSegment();
// We already have versions for this index - let's see if we need to update them based on the current shard
Version version = versionTuple.v1();
org.apache.lucene.util.Version luceneVersion = versionTuple.v2();
// For the metadata we are interested in the _latest_ elasticsearch version that was processing the metadata
// Since we rewrite the mapping during upgrade the metadata is always rewritten by the latest version
if (shardUpgradeResponse.upgradeVersion().after(versionTuple.v1())) {
version = shardUpgradeResponse.upgradeVersion();
}
// For the lucene version we are interested in the _oldest_ lucene version since it determines the
// oldest version that we need to support
if (shardUpgradeResponse.oldestLuceneSegment().onOrAfter(versionTuple.v2()) == false) {
luceneVersion = shardUpgradeResponse.oldestLuceneSegment();
}
versions.put(index, new Tuple<>(version, luceneVersion));
} }
versions.put(index, new Tuple<>(version, luceneVersion));
} }
} }
Map<String, Tuple<org.elasticsearch.Version, String>> updatedVersions = newHashMap(); Map<String, Tuple<org.elasticsearch.Version, String>> updatedVersions = newHashMap();
@ -133,33 +115,37 @@ public class TransportUpgradeAction extends TransportBroadcastAction<UpgradeRequ
} }
} }
return new UpgradeResponse(updatedVersions, shardsResponses.length(), successfulShards, failedShards, shardFailures); return new UpgradeResponse(updatedVersions, totalShards, successfulShards, failedShards, shardFailures);
} }
@Override @Override
protected ShardUpgradeRequest newShardRequest(int numShards, ShardRouting shard, UpgradeRequest request) { protected ShardUpgradeResult shardOperation(UpgradeRequest request, ShardRouting shardRouting) {
return new ShardUpgradeRequest(shard.shardId(), request); IndexShard indexShard = indicesService.indexServiceSafe(shardRouting.shardId().getIndex()).shardSafe(shardRouting.shardId().id());
org.apache.lucene.util.Version oldestLuceneSegment = indexShard.upgrade(request);
// We are using the current version of Elasticsearch as upgrade version since we update mapping to match the current version
return new ShardUpgradeResult(shardRouting.shardId(), indexShard.routingEntry().primary(), Version.CURRENT, oldestLuceneSegment);
} }
@Override @Override
protected ShardUpgradeResponse newShardResponse() { protected ShardUpgradeResult readShardResult(StreamInput in) throws IOException {
return new ShardUpgradeResponse(); ShardUpgradeResult result = new ShardUpgradeResult();
result.readFrom(in);
return result;
} }
@Override @Override
protected ShardUpgradeResponse shardOperation(ShardUpgradeRequest request) { protected UpgradeRequest readRequestFrom(StreamInput in) throws IOException {
IndexShard indexShard = indicesService.indexServiceSafe(request.shardId().getIndex()).shardSafe(request.shardId().id()); UpgradeRequest request = new UpgradeRequest();
org.apache.lucene.util.Version oldestLuceneSegment = indexShard.upgrade(request.upgradeRequest()); request.readFrom(in);
// We are using the current version of elasticsearch as upgrade version since we update mapping to match the current version return request;
return new ShardUpgradeResponse(request.shardId(), indexShard.routingEntry().primary(), Version.CURRENT, oldestLuceneSegment);
} }
/** /**
* The upgrade request works against *all* shards. * The upgrade request works against *all* shards.
*/ */
@Override @Override
protected GroupShardsIterator shards(ClusterState clusterState, UpgradeRequest request, String[] concreteIndices) { protected ShardsIterator shards(ClusterState clusterState, UpgradeRequest request, String[] concreteIndices) {
GroupShardsIterator iterator = clusterState.routingTable().allActiveShardsGrouped(concreteIndices, true); ShardsIterator iterator = clusterState.routingTable().allShards(concreteIndices);
Set<String> indicesWithMissingPrimaries = indicesWithMissingPrimaries(clusterState, concreteIndices); Set<String> indicesWithMissingPrimaries = indicesWithMissingPrimaries(clusterState, concreteIndices);
if (indicesWithMissingPrimaries.isEmpty()) { if (indicesWithMissingPrimaries.isEmpty()) {
return iterator; return iterator;
@ -231,5 +217,4 @@ public class TransportUpgradeAction extends TransportBroadcastAction<UpgradeRequ
} }
}); });
} }
} }

View File

@ -0,0 +1,565 @@
/*
* 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.action.support.broadcast.node;
import com.google.common.collect.Maps;
import org.elasticsearch.action.ActionListener;
import org.elasticsearch.action.FailedNodeException;
import org.elasticsearch.action.IndicesRequest;
import org.elasticsearch.action.NoShardAvailableActionException;
import org.elasticsearch.action.ShardOperationFailedException;
import org.elasticsearch.action.support.ActionFilters;
import org.elasticsearch.action.support.DefaultShardOperationFailedException;
import org.elasticsearch.action.support.HandledTransportAction;
import org.elasticsearch.action.support.IndicesOptions;
import org.elasticsearch.action.support.TransportActions;
import org.elasticsearch.action.support.broadcast.BroadcastRequest;
import org.elasticsearch.action.support.broadcast.BroadcastResponse;
import org.elasticsearch.action.support.broadcast.BroadcastShardOperationFailedException;
import org.elasticsearch.cluster.ClusterService;
import org.elasticsearch.cluster.ClusterState;
import org.elasticsearch.cluster.block.ClusterBlockException;
import org.elasticsearch.cluster.metadata.IndexNameExpressionResolver;
import org.elasticsearch.cluster.node.DiscoveryNode;
import org.elasticsearch.cluster.node.DiscoveryNodes;
import org.elasticsearch.cluster.routing.ShardRouting;
import org.elasticsearch.cluster.routing.ShardsIterator;
import org.elasticsearch.common.io.stream.StreamInput;
import org.elasticsearch.common.io.stream.StreamOutput;
import org.elasticsearch.common.io.stream.Streamable;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.threadpool.ThreadPool;
import org.elasticsearch.transport.BaseTransportResponseHandler;
import org.elasticsearch.transport.NodeShouldNotConnectException;
import org.elasticsearch.transport.TransportChannel;
import org.elasticsearch.transport.TransportException;
import org.elasticsearch.transport.TransportRequest;
import org.elasticsearch.transport.TransportRequestHandler;
import org.elasticsearch.transport.TransportResponse;
import org.elasticsearch.transport.TransportService;
import java.io.IOException;
import java.util.ArrayList;
import java.util.List;
import java.util.Map;
import java.util.concurrent.Callable;
import java.util.concurrent.atomic.AtomicInteger;
import java.util.concurrent.atomic.AtomicReferenceArray;
/**
* Abstraction for transporting aggregated shard-level operations in a single request (NodeRequest) per-node
* and executing the shard-level operations serially on the receiving node. Each shard-level operation can produce a
* result (ShardOperationResult), these per-node shard-level results are aggregated into a single result
* (BroadcastByNodeResponse) to the coordinating node. These per-node results are aggregated into a single result (Result)
* to the client.
*
* @param <Request> the underlying client request
* @param <Response> the response to the client request
* @param <ShardOperationResult> per-shard operation results
*/
public abstract class TransportBroadcastByNodeAction<Request extends BroadcastRequest,
Response extends BroadcastResponse,
ShardOperationResult extends Streamable> extends HandledTransportAction<Request, Response> {
private final ClusterService clusterService;
private final TransportService transportService;
final String transportNodeBroadcastAction;
public TransportBroadcastByNodeAction(
Settings settings,
String actionName,
ThreadPool threadPool,
ClusterService clusterService,
TransportService transportService,
ActionFilters actionFilters,
IndexNameExpressionResolver indexNameExpressionResolver,
Class<Request> request,
String executor) {
super(settings, actionName, threadPool, transportService, actionFilters, indexNameExpressionResolver, request);
this.clusterService = clusterService;
this.transportService = transportService;
transportNodeBroadcastAction = actionName + "[n]";
transportService.registerRequestHandler(transportNodeBroadcastAction, new Callable<NodeRequest>() {
@Override
public NodeRequest call() throws Exception {
return new NodeRequest();
}
}, executor, new BroadcastByNodeTransportRequestHandler());
}
private final Response newResponse(
Request request,
AtomicReferenceArray responses,
List<NoShardAvailableActionException> unavailableShardExceptions,
Map<String, List<ShardRouting>> nodes,
ClusterState clusterState) {
int totalShards = 0;
int successfulShards = 0;
List<ShardOperationResult> broadcastByNodeResponses = new ArrayList<>();
List<ShardOperationFailedException> exceptions = new ArrayList<>();
for (int i = 0; i < responses.length(); i++) {
if (responses.get(i) instanceof FailedNodeException) {
FailedNodeException exception = (FailedNodeException) responses.get(i);
totalShards += nodes.get(exception.nodeId()).size();
for (ShardRouting shard : nodes.get(exception.nodeId())) {
exceptions.add(new DefaultShardOperationFailedException(shard.getIndex(), shard.getId(), exception));
}
} else {
NodeResponse response = (NodeResponse) responses.get(i);
broadcastByNodeResponses.addAll(response.results);
totalShards += response.getTotalShards();
successfulShards += response.getSuccessfulShards();
for (BroadcastShardOperationFailedException throwable : response.getExceptions()) {
if (!TransportActions.isShardNotAvailableException(throwable)) {
exceptions.add(new DefaultShardOperationFailedException(throwable.getIndex(), throwable.getShardId().getId(), throwable));
}
}
}
}
totalShards += unavailableShardExceptions.size();
int failedShards = exceptions.size();
return newResponse(request, totalShards, successfulShards, failedShards, broadcastByNodeResponses, exceptions, clusterState);
}
/**
* Deserialize a shard-level result from an input stream
*
* @param in input stream
* @return a deserialized shard-level result
* @throws IOException
*/
protected abstract ShardOperationResult readShardResult(StreamInput in) throws IOException;
/**
* Creates a new response to the underlying request.
*
* @param request the underlying request
* @param totalShards the total number of shards considered for execution of the operation
* @param successfulShards the total number of shards for which execution of the operation was successful
* @param failedShards the total number of shards for which execution of the operation failed
* @param results the per-node aggregated shard-level results
* @param shardFailures the exceptions corresponding to shard operationa failures
* @param clusterState the cluster state
* @return the response
*/
protected abstract Response newResponse(Request request, int totalShards, int successfulShards, int failedShards, List<ShardOperationResult> results, List<ShardOperationFailedException> shardFailures, ClusterState clusterState);
/**
* Deserialize a request from an input stream
*
* @param in input stream
* @return a de-serialized request
* @throws IOException
*/
protected abstract Request readRequestFrom(StreamInput in) throws IOException;
/**
* Executes the shard-level operation. This method is called once per shard serially on the receiving node.
*
* @param request the node-level request
* @param shardRouting the shard on which to execute the operation
* @return the result of the shard-level operation for the shard
*/
protected abstract ShardOperationResult shardOperation(Request request, ShardRouting shardRouting);
/**
* Determines the shards on which this operation will be executed on. The operation is executed once per shard.
*
* @param clusterState the cluster state
* @param request the underlying request
* @param concreteIndices the concrete indices on which to execute the operation
* @return the shards on which to execute the operation
*/
protected abstract ShardsIterator shards(ClusterState clusterState, Request request, String[] concreteIndices);
/**
* Executes a global block check before polling the cluster state.
*
* @param state the cluster state
* @param request the underlying request
* @return a non-null exception if the operation is blocked
*/
protected abstract ClusterBlockException checkGlobalBlock(ClusterState state, Request request);
/**
* Executes a global request-level check before polling the cluster state.
*
* @param state the cluster state
* @param request the underlying request
* @param concreteIndices the concrete indices on which to execute the operation
* @return a non-null exception if the operation if blocked
*/
protected abstract ClusterBlockException checkRequestBlock(ClusterState state, Request request, String[] concreteIndices);
@Override
protected void doExecute(Request request, ActionListener<Response> listener) {
new AsyncAction(request, listener).start();
}
protected class AsyncAction {
private final Request request;
private final ActionListener<Response> listener;
private final ClusterState clusterState;
private final DiscoveryNodes nodes;
private final Map<String, List<ShardRouting>> nodeIds;
private final AtomicReferenceArray<Object> responses;
private final AtomicInteger counter = new AtomicInteger();
private List<NoShardAvailableActionException> unavailableShardExceptions = new ArrayList<>();
protected AsyncAction(Request request, ActionListener<Response> listener) {
this.request = request;
this.listener = listener;
clusterState = clusterService.state();
nodes = clusterState.nodes();
ClusterBlockException globalBlockException = checkGlobalBlock(clusterState, request);
if (globalBlockException != null) {
throw globalBlockException;
}
String[] concreteIndices = indexNameExpressionResolver.concreteIndices(clusterState, request);
ClusterBlockException requestBlockException = checkRequestBlock(clusterState, request, concreteIndices);
if (requestBlockException != null) {
throw requestBlockException;
}
logger.trace("resolving shards for [{}] based on cluster state version [{}]", actionName, clusterState.version());
ShardsIterator shardIt = shards(clusterState, request, concreteIndices);
nodeIds = Maps.newHashMap();
for (ShardRouting shard : shardIt.asUnordered()) {
if (shard.assignedToNode()) {
String nodeId = shard.currentNodeId();
if (!nodeIds.containsKey(nodeId)) {
nodeIds.put(nodeId, new ArrayList<ShardRouting>());
}
nodeIds.get(nodeId).add(shard);
} else {
unavailableShardExceptions.add(
new NoShardAvailableActionException(
shard.shardId(),
" no shards available for shard " + shard.toString() + " while executing " + actionName
)
);
}
}
responses = new AtomicReferenceArray<>(nodeIds.size());
}
public void start() {
if (nodeIds.size() == 0) {
try {
onCompletion();
} catch (Throwable e) {
listener.onFailure(e);
}
} else {
int nodeIndex = -1;
for (Map.Entry<String, List<ShardRouting>> entry : nodeIds.entrySet()) {
nodeIndex++;
DiscoveryNode node = nodes.get(entry.getKey());
sendNodeRequest(node, entry.getValue(), nodeIndex);
}
}
}
private void sendNodeRequest(final DiscoveryNode node, List<ShardRouting> shards, final int nodeIndex) {
try {
NodeRequest nodeRequest = new NodeRequest(node.getId(), request, shards);
transportService.sendRequest(node, transportNodeBroadcastAction, nodeRequest, new BaseTransportResponseHandler<NodeResponse>() {
@Override
public NodeResponse newInstance() {
return new NodeResponse();
}
@Override
public void handleResponse(NodeResponse response) {
onNodeResponse(node, nodeIndex, response);
}
@Override
public void handleException(TransportException exp) {
onNodeFailure(node, nodeIndex, exp);
}
@Override
public String executor() {
return ThreadPool.Names.SAME;
}
});
} catch (Throwable e) {
onNodeFailure(node, nodeIndex, e);
}
}
protected void onNodeResponse(DiscoveryNode node, int nodeIndex, NodeResponse response) {
logger.trace("received response for [{}] from node [{}]", actionName, node.id());
// this is defensive to protect against the possibility of double invocation
// the current implementation of TransportService#sendRequest guards against this
// but concurrency is hard, safety is important, and the small performance loss here does not matter
if (responses.compareAndSet(nodeIndex, null, response)) {
if (counter.incrementAndGet() == responses.length()) {
onCompletion();
}
}
}
protected void onNodeFailure(DiscoveryNode node, int nodeIndex, Throwable t) {
String nodeId = node.id();
if (logger.isDebugEnabled() && !(t instanceof NodeShouldNotConnectException)) {
logger.debug("failed to execute [{}] on node [{}]", t, actionName, nodeId);
}
// this is defensive to protect against the possibility of double invocation
// the current implementation of TransportService#sendRequest guards against this
// but concurrency is hard, safety is important, and the small performance loss here does not matter
if (responses.compareAndSet(nodeIndex, null, new FailedNodeException(nodeId, "Failed node [" + nodeId + "]", t))) {
if (counter.incrementAndGet() == responses.length()) {
onCompletion();
}
}
}
protected void onCompletion() {
Response response = null;
try {
response = newResponse(request, responses, unavailableShardExceptions, nodeIds, clusterState);
} catch (Throwable t) {
logger.debug("failed to combine responses from nodes", t);
listener.onFailure(t);
}
if (response != null) {
try {
listener.onResponse(response);
} catch (Throwable t) {
listener.onFailure(t);
}
}
}
}
class BroadcastByNodeTransportRequestHandler implements TransportRequestHandler<NodeRequest> {
@Override
public void messageReceived(final NodeRequest request, TransportChannel channel) throws Exception {
List<ShardRouting> shards = request.getShards();
final int totalShards = shards.size();
logger.trace("[{}] executing operation on [{}] shards", actionName, totalShards);
final Object[] shardResultOrExceptions = new Object[totalShards];
int shardIndex = -1;
for (final ShardRouting shardRouting : shards) {
shardIndex++;
onShardOperation(request, shardResultOrExceptions, shardIndex, shardRouting);
}
List<BroadcastShardOperationFailedException> accumulatedExceptions = new ArrayList<>();
List<ShardOperationResult> results = new ArrayList<>();
for (int i = 0; i < totalShards; i++) {
if (shardResultOrExceptions[i] instanceof BroadcastShardOperationFailedException) {
accumulatedExceptions.add((BroadcastShardOperationFailedException) shardResultOrExceptions[i]);
} else {
results.add((ShardOperationResult) shardResultOrExceptions[i]);
}
}
channel.sendResponse(new NodeResponse(request.getNodeId(), totalShards, results, accumulatedExceptions));
}
private void onShardOperation(final NodeRequest request, final Object[] shardResults, final int shardIndex, final ShardRouting shardRouting) {
try {
logger.trace("[{}] executing operation for shard [{}]", actionName, shardRouting.shortSummary());
ShardOperationResult result = shardOperation(request.indicesLevelRequest, shardRouting);
shardResults[shardIndex] = result;
logger.trace("[{}] completed operation for shard [{}]", actionName, shardRouting.shortSummary());
} catch (Throwable t) {
BroadcastShardOperationFailedException e = new BroadcastShardOperationFailedException(shardRouting.shardId(), "operation " + actionName + " failed", t);
e.setIndex(shardRouting.getIndex());
e.setShard(shardRouting.shardId());
shardResults[shardIndex] = e;
logger.debug("[{}] failed to execute operation for shard [{}]", e, actionName, shardRouting.shortSummary());
}
}
}
protected class NodeRequest extends TransportRequest implements IndicesRequest {
private String nodeId;
private List<ShardRouting> shards;
protected Request indicesLevelRequest;
protected NodeRequest() {
}
public NodeRequest(String nodeId, Request request, List<ShardRouting> shards) {
super(request);
this.indicesLevelRequest = request;
this.shards = shards;
this.nodeId = nodeId;
}
public List<ShardRouting> getShards() {
return shards;
}
public String getNodeId() {
return nodeId;
}
public String[] indices() {
return indicesLevelRequest.indices();
}
public IndicesOptions indicesOptions() {
return indicesLevelRequest.indicesOptions();
}
@Override
public void readFrom(StreamInput in) throws IOException {
super.readFrom(in);
indicesLevelRequest = readRequestFrom(in);
int size = in.readVInt();
shards = new ArrayList<>(size);
for (int i = 0; i < size; i++) {
shards.add(ShardRouting.readShardRoutingEntry(in));
}
nodeId = in.readString();
}
@Override
public void writeTo(StreamOutput out) throws IOException {
super.writeTo(out);
indicesLevelRequest.writeTo(out);
int size = shards.size();
out.writeVInt(size);
for (int i = 0; i < size; i++) {
shards.get(i).writeTo(out);
}
out.writeString(nodeId);
}
}
class NodeResponse extends TransportResponse {
protected String nodeId;
protected int totalShards;
protected List<BroadcastShardOperationFailedException> exceptions;
protected List<ShardOperationResult> results;
public NodeResponse() {
}
public NodeResponse(String nodeId,
int totalShards,
List<ShardOperationResult> results,
List<BroadcastShardOperationFailedException> exceptions) {
this.nodeId = nodeId;
this.totalShards = totalShards;
this.results = results;
this.exceptions = exceptions;
}
public String getNodeId() {
return nodeId;
}
public int getTotalShards() {
return totalShards;
}
public int getSuccessfulShards() {
return results.size();
}
public List<BroadcastShardOperationFailedException> getExceptions() {
return exceptions;
}
@Override
public void readFrom(StreamInput in) throws IOException {
super.readFrom(in);
nodeId = in.readString();
totalShards = in.readVInt();
int resultsSize = in.readVInt();
results = new ArrayList<>(resultsSize);
for (; resultsSize > 0; resultsSize--) {
final ShardOperationResult result = in.readBoolean() ? readShardResult(in) : null;
results.add(result);
}
if (in.readBoolean()) {
int failureShards = in.readVInt();
exceptions = new ArrayList<>(failureShards);
for (int i = 0; i < failureShards; i++) {
exceptions.add(new BroadcastShardOperationFailedException(in));
}
} else {
exceptions = null;
}
}
@Override
public void writeTo(StreamOutput out) throws IOException {
super.writeTo(out);
out.writeString(nodeId);
out.writeVInt(totalShards);
out.writeVInt(results.size());
for (ShardOperationResult result : results) {
out.writeOptionalStreamable(result);
}
out.writeBoolean(exceptions != null);
if (exceptions != null) {
int failureShards = exceptions.size();
out.writeVInt(failureShards);
for (int i = 0; i < failureShards; i++) {
exceptions.get(i).writeTo(out);
}
}
}
}
/**
* Can be used for implementations of {@link #shardOperation(BroadcastRequest, ShardRouting) shardOperation} for
* which there is no shard-level return value.
*/
public final static class EmptyResult implements Streamable {
public static EmptyResult INSTANCE = new EmptyResult();
private EmptyResult() {
}
@Override
public void readFrom(StreamInput in) throws IOException {
}
@Override
public void writeTo(StreamOutput out) throws IOException {
}
public static EmptyResult readEmptyResultFrom(StreamInput in) {
return INSTANCE;
}
}
}

View File

@ -97,7 +97,7 @@ public abstract class TransportMasterNodeAction<Request extends MasterNodeReques
listener.onFailure(blockException); listener.onFailure(blockException);
return; return;
} }
logger.trace("can't execute due to a cluster block: [{}], retrying", blockException); logger.trace("can't execute due to a cluster block, retrying", blockException);
observer.waitForNextChange( observer.waitForNextChange(
new ClusterStateObserver.Listener() { new ClusterStateObserver.Listener() {
@Override @Override

View File

@ -46,6 +46,7 @@ import org.elasticsearch.env.EnvironmentModule;
import org.elasticsearch.indices.breaker.CircuitBreakerModule; import org.elasticsearch.indices.breaker.CircuitBreakerModule;
import org.elasticsearch.monitor.MonitorService; import org.elasticsearch.monitor.MonitorService;
import org.elasticsearch.node.internal.InternalSettingsPreparer; import org.elasticsearch.node.internal.InternalSettingsPreparer;
import org.elasticsearch.plugins.Plugin;
import org.elasticsearch.plugins.PluginsModule; import org.elasticsearch.plugins.PluginsModule;
import org.elasticsearch.plugins.PluginsService; import org.elasticsearch.plugins.PluginsService;
import org.elasticsearch.search.SearchModule; import org.elasticsearch.search.SearchModule;
@ -55,6 +56,7 @@ import org.elasticsearch.transport.TransportModule;
import org.elasticsearch.transport.TransportService; import org.elasticsearch.transport.TransportService;
import org.elasticsearch.transport.netty.NettyTransport; import org.elasticsearch.transport.netty.NettyTransport;
import java.util.ArrayList;
import java.util.List; import java.util.List;
import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeUnit;
@ -82,6 +84,7 @@ public class TransportClient extends AbstractClient {
public static class Builder { public static class Builder {
private Settings settings = Settings.EMPTY; private Settings settings = Settings.EMPTY;
private List<Class<? extends Plugin>> pluginClasses = new ArrayList<>();
private boolean loadConfigSettings = true; private boolean loadConfigSettings = true;
/** /**
@ -108,6 +111,14 @@ public class TransportClient extends AbstractClient {
return this; return this;
} }
/**
* Add the given plugin to the client when it is created.
*/
public Builder addPlugin(Class<? extends Plugin> pluginClass) {
pluginClasses.add(pluginClass);
return this;
}
/** /**
* Builds a new instance of the transport client. * Builds a new instance of the transport client.
*/ */
@ -122,7 +133,7 @@ public class TransportClient extends AbstractClient {
.build(); .build();
Environment environment = tuple.v2(); Environment environment = tuple.v2();
PluginsService pluginsService = new PluginsService(settings, tuple.v2()); PluginsService pluginsService = new PluginsService(settings, tuple.v2(), pluginClasses);
this.settings = pluginsService.updatedSettings(); this.settings = pluginsService.updatedSettings();
Version version = Version.CURRENT; Version version = Version.CURRENT;

View File

@ -32,28 +32,63 @@ import java.util.Map;
*/ */
public class ClusterInfo { public class ClusterInfo {
private final Map<String, DiskUsage> usages; private final Map<String, DiskUsage> leastAvailableSpaceUsage;
private final Map<String, DiskUsage> mostAvailableSpaceUsage;
final Map<String, Long> shardSizes; final Map<String, Long> shardSizes;
public static final ClusterInfo EMPTY = new ClusterInfo(); public static final ClusterInfo EMPTY = new ClusterInfo();
private final Map<ShardRouting, String> routingToDataPath;
private ClusterInfo() { protected ClusterInfo() {
this.usages = Collections.emptyMap(); this(Collections.EMPTY_MAP, Collections.EMPTY_MAP, Collections.EMPTY_MAP, Collections.EMPTY_MAP);
this.shardSizes = Collections.emptyMap();
} }
public ClusterInfo(Map<String, DiskUsage> usages, Map<String, Long> shardSizes) { /**
this.usages = usages; * Creates a new ClusterInfo instance.
*
* @param leastAvailableSpaceUsage a node id to disk usage mapping for the path that has the least available space on the node.
* @param mostAvailableSpaceUsage a node id to disk usage mapping for the path that has the most available space on the node.
* @param shardSizes a shardkey to size in bytes mapping per shard.
* @param routingToDataPath the shard routing to datapath mapping
* @see #shardIdentifierFromRouting
*/
public ClusterInfo(final Map<String, DiskUsage> leastAvailableSpaceUsage, final Map<String, DiskUsage> mostAvailableSpaceUsage, final Map<String, Long> shardSizes, Map<ShardRouting, String> routingToDataPath) {
this.leastAvailableSpaceUsage = leastAvailableSpaceUsage;
this.shardSizes = shardSizes; this.shardSizes = shardSizes;
this.mostAvailableSpaceUsage = mostAvailableSpaceUsage;
this.routingToDataPath = routingToDataPath;
} }
public Map<String, DiskUsage> getNodeDiskUsages() { /**
return this.usages; * Returns a node id to disk usage mapping for the path that has the least available space on the node.
*/
public Map<String, DiskUsage> getNodeLeastAvailableDiskUsages() {
return this.leastAvailableSpaceUsage;
} }
/**
* Returns a node id to disk usage mapping for the path that has the most available space on the node.
*/
public Map<String, DiskUsage> getNodeMostAvailableDiskUsages() {
return this.mostAvailableSpaceUsage;
}
/**
* Returns the shard size for the given shard routing or <code>null</code> it that metric is not available.
*/
public Long getShardSize(ShardRouting shardRouting) { public Long getShardSize(ShardRouting shardRouting) {
return shardSizes.get(shardIdentifierFromRouting(shardRouting)); return shardSizes.get(shardIdentifierFromRouting(shardRouting));
} }
/**
* Returns the nodes absolute data-path the given shard is allocated on or <code>null</code> if the information is not available.
*/
public String getDataPath(ShardRouting shardRouting) {
return routingToDataPath.get(shardRouting);
}
/**
* Returns the shard size for the given shard routing or <code>defaultValue</code> it that metric is not available.
*/
public long getShardSize(ShardRouting shardRouting, long defaultValue) { public long getShardSize(ShardRouting shardRouting, long defaultValue) {
Long shardSize = getShardSize(shardRouting); Long shardSize = getShardSize(shardRouting);
return shardSize == null ? defaultValue : shardSize; return shardSize == null ? defaultValue : shardSize;

View File

@ -28,6 +28,7 @@ import org.elasticsearch.common.unit.ByteSizeValue;
public class DiskUsage { public class DiskUsage {
final String nodeId; final String nodeId;
final String nodeName; final String nodeName;
final String path;
final long totalBytes; final long totalBytes;
final long freeBytes; final long freeBytes;
@ -35,11 +36,12 @@ public class DiskUsage {
* Create a new DiskUsage, if {@code totalBytes} is 0, {@get getFreeDiskAsPercentage} * Create a new DiskUsage, if {@code totalBytes} is 0, {@get getFreeDiskAsPercentage}
* will always return 100.0% free * will always return 100.0% free
*/ */
public DiskUsage(String nodeId, String nodeName, long totalBytes, long freeBytes) { public DiskUsage(String nodeId, String nodeName, String path, long totalBytes, long freeBytes) {
this.nodeId = nodeId; this.nodeId = nodeId;
this.nodeName = nodeName; this.nodeName = nodeName;
this.freeBytes = freeBytes; this.freeBytes = freeBytes;
this.totalBytes = totalBytes; this.totalBytes = totalBytes;
this.path = path;
} }
public String getNodeId() { public String getNodeId() {
@ -50,6 +52,10 @@ public class DiskUsage {
return nodeName; return nodeName;
} }
public String getPath() {
return path;
}
public double getFreeDiskAsPercentage() { public double getFreeDiskAsPercentage() {
// We return 100.0% in order to fail "open", in that if we have invalid // We return 100.0% in order to fail "open", in that if we have invalid
// numbers for the total bytes, it's as if we don't know disk usage. // numbers for the total bytes, it's as if we don't know disk usage.
@ -77,7 +83,7 @@ public class DiskUsage {
@Override @Override
public String toString() { public String toString() {
return "[" + nodeId + "][" + nodeName + "] free: " + new ByteSizeValue(getFreeBytes()) + return "[" + nodeId + "][" + nodeName + "][" + path + "] free: " + new ByteSizeValue(getFreeBytes()) +
"[" + Strings.format1Decimals(getFreeDiskAsPercentage(), "%") + "]"; "[" + Strings.format1Decimals(getFreeDiskAsPercentage(), "%") + "]";
} }
} }

View File

@ -19,7 +19,6 @@
package org.elasticsearch.cluster; package org.elasticsearch.cluster;
import org.elasticsearch.ElasticsearchException;
import org.elasticsearch.action.ActionListener; import org.elasticsearch.action.ActionListener;
import org.elasticsearch.action.LatchedActionListener; import org.elasticsearch.action.LatchedActionListener;
import org.elasticsearch.action.admin.cluster.node.stats.NodeStats; import org.elasticsearch.action.admin.cluster.node.stats.NodeStats;
@ -32,12 +31,13 @@ import org.elasticsearch.action.admin.indices.stats.ShardStats;
import org.elasticsearch.action.admin.indices.stats.TransportIndicesStatsAction; import org.elasticsearch.action.admin.indices.stats.TransportIndicesStatsAction;
import org.elasticsearch.cluster.block.ClusterBlockException; import org.elasticsearch.cluster.block.ClusterBlockException;
import org.elasticsearch.cluster.node.DiscoveryNode; import org.elasticsearch.cluster.node.DiscoveryNode;
import org.elasticsearch.cluster.routing.ShardRouting;
import org.elasticsearch.cluster.routing.allocation.decider.DiskThresholdDecider; import org.elasticsearch.cluster.routing.allocation.decider.DiskThresholdDecider;
import org.elasticsearch.common.component.AbstractComponent; import org.elasticsearch.common.component.AbstractComponent;
import org.elasticsearch.common.inject.Inject; import org.elasticsearch.common.inject.Inject;
import org.elasticsearch.common.logging.ESLogger;
import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.unit.TimeValue; import org.elasticsearch.common.unit.TimeValue;
import org.elasticsearch.common.util.concurrent.AbstractRunnable;
import org.elasticsearch.common.util.concurrent.EsRejectedExecutionException; import org.elasticsearch.common.util.concurrent.EsRejectedExecutionException;
import org.elasticsearch.monitor.fs.FsInfo; import org.elasticsearch.monitor.fs.FsInfo;
import org.elasticsearch.node.settings.NodeSettingsService; import org.elasticsearch.node.settings.NodeSettingsService;
@ -47,7 +47,6 @@ import org.elasticsearch.transport.ReceiveTimeoutTransportException;
import java.util.*; import java.util.*;
import java.util.concurrent.CountDownLatch; import java.util.concurrent.CountDownLatch;
import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicReference;
/** /**
* InternalClusterInfoService provides the ClusterInfoService interface, * InternalClusterInfoService provides the ClusterInfoService interface,
@ -67,7 +66,9 @@ public class InternalClusterInfoService extends AbstractComponent implements Clu
private volatile TimeValue updateFrequency; private volatile TimeValue updateFrequency;
private volatile Map<String, DiskUsage> usages; private volatile Map<String, DiskUsage> leastAvailableSpaceUsages;
private volatile Map<String, DiskUsage> mostAvailableSpaceUsages;
private volatile Map<ShardRouting, String> shardRoutingToDataPath;
private volatile Map<String, Long> shardSizes; private volatile Map<String, Long> shardSizes;
private volatile boolean isMaster = false; private volatile boolean isMaster = false;
private volatile boolean enabled; private volatile boolean enabled;
@ -84,7 +85,9 @@ public class InternalClusterInfoService extends AbstractComponent implements Clu
TransportIndicesStatsAction transportIndicesStatsAction, ClusterService clusterService, TransportIndicesStatsAction transportIndicesStatsAction, ClusterService clusterService,
ThreadPool threadPool) { ThreadPool threadPool) {
super(settings); super(settings);
this.usages = Collections.emptyMap(); this.leastAvailableSpaceUsages = Collections.emptyMap();
this.mostAvailableSpaceUsages = Collections.emptyMap();
this.shardRoutingToDataPath = Collections.emptyMap();
this.shardSizes = Collections.emptyMap(); this.shardSizes = Collections.emptyMap();
this.transportNodesStatsAction = transportNodesStatsAction; this.transportNodesStatsAction = transportNodesStatsAction;
this.transportIndicesStatsAction = transportIndicesStatsAction; this.transportIndicesStatsAction = transportIndicesStatsAction;
@ -200,9 +203,16 @@ public class InternalClusterInfoService extends AbstractComponent implements Clu
if (logger.isTraceEnabled()) { if (logger.isTraceEnabled()) {
logger.trace("Removing node from cluster info: {}", removedNode.getId()); logger.trace("Removing node from cluster info: {}", removedNode.getId());
} }
Map<String, DiskUsage> newUsages = new HashMap<>(usages); if (leastAvailableSpaceUsages.containsKey(removedNode.getId())) {
newUsages.remove(removedNode.getId()); Map<String, DiskUsage> newMaxUsages = new HashMap<>(leastAvailableSpaceUsages);
usages = Collections.unmodifiableMap(newUsages); newMaxUsages.remove(removedNode.getId());
leastAvailableSpaceUsages = Collections.unmodifiableMap(newMaxUsages);
}
if (mostAvailableSpaceUsages.containsKey(removedNode.getId())) {
Map<String, DiskUsage> newMinUsages = new HashMap<>(mostAvailableSpaceUsages);
newMinUsages.remove(removedNode.getId());
mostAvailableSpaceUsages = Collections.unmodifiableMap(newMinUsages);
}
} }
} }
} }
@ -210,7 +220,7 @@ public class InternalClusterInfoService extends AbstractComponent implements Clu
@Override @Override
public ClusterInfo getClusterInfo() { public ClusterInfo getClusterInfo() {
return new ClusterInfo(usages, shardSizes); return new ClusterInfo(leastAvailableSpaceUsages, mostAvailableSpaceUsages, shardSizes, shardRoutingToDataPath);
} }
@Override @Override
@ -313,27 +323,11 @@ public class InternalClusterInfoService extends AbstractComponent implements Clu
CountDownLatch nodeLatch = updateNodeStats(new ActionListener<NodesStatsResponse>() { CountDownLatch nodeLatch = updateNodeStats(new ActionListener<NodesStatsResponse>() {
@Override @Override
public void onResponse(NodesStatsResponse nodeStatses) { public void onResponse(NodesStatsResponse nodeStatses) {
Map<String, DiskUsage> newUsages = new HashMap<>(); Map<String, DiskUsage> newLeastAvaiableUsages = new HashMap<>();
for (NodeStats nodeStats : nodeStatses.getNodes()) { Map<String, DiskUsage> newMostAvaiableUsages = new HashMap<>();
if (nodeStats.getFs() == null) { fillDiskUsagePerNode(logger, nodeStatses.getNodes(), newLeastAvaiableUsages, newMostAvaiableUsages);
logger.warn("Unable to retrieve node FS stats for {}", nodeStats.getNode().name()); leastAvailableSpaceUsages = Collections.unmodifiableMap(newLeastAvaiableUsages);
} else { mostAvailableSpaceUsages = Collections.unmodifiableMap(newMostAvaiableUsages);
long available = 0;
long total = 0;
for (FsInfo.Path info : nodeStats.getFs()) {
available += info.getAvailable().bytes();
total += info.getTotal().bytes();
}
String nodeId = nodeStats.getNode().id();
String nodeName = nodeStats.getNode().getName();
if (logger.isTraceEnabled()) {
logger.trace("node: [{}], total disk: {}, available disk: {}", nodeId, total, available);
}
newUsages.put(nodeId, new DiskUsage(nodeId, nodeName, total, available));
}
}
usages = Collections.unmodifiableMap(newUsages);
} }
@Override @Override
@ -349,7 +343,8 @@ public class InternalClusterInfoService extends AbstractComponent implements Clu
logger.warn("Failed to execute NodeStatsAction for ClusterInfoUpdateJob", e); logger.warn("Failed to execute NodeStatsAction for ClusterInfoUpdateJob", e);
} }
// we empty the usages list, to be safe - we don't know what's going on. // we empty the usages list, to be safe - we don't know what's going on.
usages = Collections.emptyMap(); leastAvailableSpaceUsages = Collections.emptyMap();
mostAvailableSpaceUsages = Collections.emptyMap();
} }
} }
}); });
@ -358,16 +353,11 @@ public class InternalClusterInfoService extends AbstractComponent implements Clu
@Override @Override
public void onResponse(IndicesStatsResponse indicesStatsResponse) { public void onResponse(IndicesStatsResponse indicesStatsResponse) {
ShardStats[] stats = indicesStatsResponse.getShards(); ShardStats[] stats = indicesStatsResponse.getShards();
HashMap<String, Long> newShardSizes = new HashMap<>(); final HashMap<String, Long> newShardSizes = new HashMap<>();
for (ShardStats s : stats) { final HashMap<ShardRouting, String> newShardRoutingToDataPath = new HashMap<>();
long size = s.getStats().getStore().sizeInBytes(); buildShardLevelInfo(logger, stats, newShardSizes, newShardRoutingToDataPath);
String sid = ClusterInfo.shardIdentifierFromRouting(s.getShardRouting());
if (logger.isTraceEnabled()) {
logger.trace("shard: {} size: {}", sid, size);
}
newShardSizes.put(sid, size);
}
shardSizes = Collections.unmodifiableMap(newShardSizes); shardSizes = Collections.unmodifiableMap(newShardSizes);
shardRoutingToDataPath = Collections.unmodifiableMap(newShardRoutingToDataPath);
} }
@Override @Override
@ -384,6 +374,7 @@ public class InternalClusterInfoService extends AbstractComponent implements Clu
} }
// we empty the usages list, to be safe - we don't know what's going on. // we empty the usages list, to be safe - we don't know what's going on.
shardSizes = Collections.emptyMap(); shardSizes = Collections.emptyMap();
shardRoutingToDataPath = Collections.emptyMap();
} }
} }
}); });
@ -412,5 +403,46 @@ public class InternalClusterInfoService extends AbstractComponent implements Clu
} }
} }
static void buildShardLevelInfo(ESLogger logger, ShardStats[] stats, HashMap<String, Long> newShardSizes, HashMap<ShardRouting, String> newShardRoutingToDataPath) {
for (ShardStats s : stats) {
newShardRoutingToDataPath.put(s.getShardRouting(), s.getDataPath());
long size = s.getStats().getStore().sizeInBytes();
String sid = ClusterInfo.shardIdentifierFromRouting(s.getShardRouting());
if (logger.isTraceEnabled()) {
logger.trace("shard: {} size: {}", sid, size);
}
newShardSizes.put(sid, size);
}
}
static void fillDiskUsagePerNode(ESLogger logger, NodeStats[] nodeStatsArray, Map<String, DiskUsage> newLeastAvaiableUsages, Map<String, DiskUsage> newMostAvaiableUsages) {
for (NodeStats nodeStats : nodeStatsArray) {
if (nodeStats.getFs() == null) {
logger.warn("Unable to retrieve node FS stats for {}", nodeStats.getNode().name());
} else {
FsInfo.Path leastAvailablePath = null;
FsInfo.Path mostAvailablePath = null;
for (FsInfo.Path info : nodeStats.getFs()) {
if (leastAvailablePath == null) {
assert mostAvailablePath == null;
mostAvailablePath = leastAvailablePath = info;
} else if (leastAvailablePath.getAvailable().bytes() > info.getAvailable().bytes()){
leastAvailablePath = info;
} else if (mostAvailablePath.getAvailable().bytes() < info.getAvailable().bytes()) {
mostAvailablePath = info;
}
}
String nodeId = nodeStats.getNode().id();
String nodeName = nodeStats.getNode().getName();
if (logger.isTraceEnabled()) {
logger.trace("node: [{}], most available: total disk: {}, available disk: {} / least available: total disk: {}, available disk: {}", nodeId, mostAvailablePath.getTotal(), leastAvailablePath.getAvailable(), leastAvailablePath.getTotal(), leastAvailablePath.getAvailable());
}
newLeastAvaiableUsages.put(nodeId, new DiskUsage(nodeId, nodeName, leastAvailablePath.getPath(), leastAvailablePath.getTotal().bytes(), leastAvailablePath.getAvailable().bytes()));
newMostAvaiableUsages.put(nodeId, new DiskUsage(nodeId, nodeName, mostAvailablePath.getPath(), mostAvailablePath.getTotal().bytes(), mostAvailablePath.getAvailable().bytes()));
}
}
}
} }

View File

@ -21,6 +21,7 @@ package org.elasticsearch.cluster.routing;
import com.carrotsearch.hppc.IntSet; import com.carrotsearch.hppc.IntSet;
import com.google.common.base.Predicate; import com.google.common.base.Predicate;
import com.google.common.base.Predicates;
import com.google.common.collect.ImmutableMap; import com.google.common.collect.ImmutableMap;
import com.google.common.collect.Iterables; import com.google.common.collect.Iterables;
import com.google.common.collect.UnmodifiableIterator; import com.google.common.collect.UnmodifiableIterator;
@ -223,6 +224,38 @@ public class RoutingTable implements Iterable<IndexRoutingTable>, Diffable<Routi
return new GroupShardsIterator(set); return new GroupShardsIterator(set);
} }
public ShardsIterator allShards(String[] indices) {
return allShardsSatisfyingPredicate(indices, Predicates.<ShardRouting>alwaysTrue(), false);
}
public ShardsIterator allShardsIncludingRelocationTargets(String[] indices) {
return allShardsSatisfyingPredicate(indices, Predicates.<ShardRouting>alwaysTrue(), true);
}
// TODO: replace with JDK 8 native java.util.function.Predicate
private ShardsIterator allShardsSatisfyingPredicate(String[] indices, Predicate<ShardRouting> predicate, boolean includeRelocationTargets) {
// use list here since we need to maintain identity across shards
List<ShardRouting> shards = new ArrayList<>();
for (String index : indices) {
IndexRoutingTable indexRoutingTable = index(index);
if (indexRoutingTable == null) {
continue;
// we simply ignore indices that don't exists (make sense for operations that use it currently)
}
for (IndexShardRoutingTable indexShardRoutingTable : indexRoutingTable) {
for (ShardRouting shardRouting : indexShardRoutingTable) {
if (predicate.apply(shardRouting)) {
shards.add(shardRouting);
if (includeRelocationTargets && shardRouting.relocating()) {
shards.add(shardRouting.buildTargetRelocatingShard());
}
}
}
}
}
return new PlainShardsIterator(shards);
}
/** /**
* All the *active* primary shards for the provided indices grouped (each group is a single element, consisting * All the *active* primary shards for the provided indices grouped (each group is a single element, consisting
* of the primary shard). This is handy for components that expect to get group iterators, but still want in some * of the primary shard). This is handy for components that expect to get group iterators, but still want in some

View File

@ -164,7 +164,7 @@ public class DiskThresholdDecider extends AllocationDecider {
@Override @Override
public void onNewInfo(ClusterInfo info) { public void onNewInfo(ClusterInfo info) {
Map<String, DiskUsage> usages = info.getNodeDiskUsages(); Map<String, DiskUsage> usages = info.getNodeLeastAvailableDiskUsages();
if (usages != null) { if (usages != null) {
boolean reroute = false; boolean reroute = false;
String explanation = ""; String explanation = "";
@ -313,13 +313,16 @@ public class DiskThresholdDecider extends AllocationDecider {
* If subtractShardsMovingAway is set then the size of shards moving away is subtracted from the total size * If subtractShardsMovingAway is set then the size of shards moving away is subtracted from the total size
* of all shards * of all shards
*/ */
public static long sizeOfRelocatingShards(RoutingNode node, ClusterInfo clusterInfo, boolean subtractShardsMovingAway) { public static long sizeOfRelocatingShards(RoutingNode node, ClusterInfo clusterInfo, boolean subtractShardsMovingAway, String dataPath) {
long totalSize = 0; long totalSize = 0;
for (ShardRouting routing : node.shardsWithState(ShardRoutingState.RELOCATING, ShardRoutingState.INITIALIZING)) { for (ShardRouting routing : node.shardsWithState(ShardRoutingState.RELOCATING, ShardRoutingState.INITIALIZING)) {
if (routing.initializing() && routing.relocatingNodeId() != null) { String actualPath = clusterInfo.getDataPath(routing);
totalSize += getShardSize(routing, clusterInfo); if (dataPath.equals(actualPath)) {
} else if (subtractShardsMovingAway && routing.relocating()) { if (routing.initializing() && routing.relocatingNodeId() != null) {
totalSize -= getShardSize(routing, clusterInfo); totalSize += getShardSize(routing, clusterInfo);
} else if (subtractShardsMovingAway && routing.relocating()) {
totalSize -= getShardSize(routing, clusterInfo);
}
} }
} }
return totalSize; return totalSize;
@ -339,7 +342,9 @@ public class DiskThresholdDecider extends AllocationDecider {
final double usedDiskThresholdLow = 100.0 - DiskThresholdDecider.this.freeDiskThresholdLow; final double usedDiskThresholdLow = 100.0 - DiskThresholdDecider.this.freeDiskThresholdLow;
final double usedDiskThresholdHigh = 100.0 - DiskThresholdDecider.this.freeDiskThresholdHigh; final double usedDiskThresholdHigh = 100.0 - DiskThresholdDecider.this.freeDiskThresholdHigh;
DiskUsage usage = getDiskUsage(node, allocation); ClusterInfo clusterInfo = allocation.clusterInfo();
Map<String, DiskUsage> usages = clusterInfo.getNodeMostAvailableDiskUsages();
DiskUsage usage = getDiskUsage(node, allocation, usages);
// First, check that the node currently over the low watermark // First, check that the node currently over the low watermark
double freeDiskPercentage = usage.getFreeDiskAsPercentage(); double freeDiskPercentage = usage.getFreeDiskAsPercentage();
// Cache the used disk percentage for displaying disk percentages consistent with documentation // Cache the used disk percentage for displaying disk percentages consistent with documentation
@ -441,17 +446,26 @@ public class DiskThresholdDecider extends AllocationDecider {
@Override @Override
public Decision canRemain(ShardRouting shardRouting, RoutingNode node, RoutingAllocation allocation) { public Decision canRemain(ShardRouting shardRouting, RoutingNode node, RoutingAllocation allocation) {
if (shardRouting.currentNodeId().equals(node.nodeId()) == false) {
throw new IllegalArgumentException("Shard [" + shardRouting + "] is not allocated on node: [" + node.nodeId() + "]");
}
final Decision decision = earlyTerminate(allocation); final Decision decision = earlyTerminate(allocation);
if (decision != null) { if (decision != null) {
return decision; return decision;
} }
DiskUsage usage = getDiskUsage(node, allocation); final ClusterInfo clusterInfo = allocation.clusterInfo();
final Map<String, DiskUsage> usages = clusterInfo.getNodeLeastAvailableDiskUsages();
final DiskUsage usage = getDiskUsage(node, allocation, usages);
final String dataPath = clusterInfo.getDataPath(shardRouting);
// If this node is already above the high threshold, the shard cannot remain (get it off!) // If this node is already above the high threshold, the shard cannot remain (get it off!)
double freeDiskPercentage = usage.getFreeDiskAsPercentage(); final double freeDiskPercentage = usage.getFreeDiskAsPercentage();
long freeBytes = usage.getFreeBytes(); final long freeBytes = usage.getFreeBytes();
if (logger.isDebugEnabled()) { if (logger.isDebugEnabled()) {
logger.debug("node [{}] has {}% free disk ({} bytes)", node.nodeId(), freeDiskPercentage, freeBytes); logger.debug("node [{}] has {}% free disk ({} bytes)", node.nodeId(), freeDiskPercentage, freeBytes);
} }
if (dataPath == null || usage.getPath().equals(dataPath) == false) {
return allocation.decision(Decision.YES, NAME, "shard is not allocated on the most utilized disk");
}
if (freeBytes < freeBytesThresholdHigh.bytes()) { if (freeBytes < freeBytesThresholdHigh.bytes()) {
if (logger.isDebugEnabled()) { if (logger.isDebugEnabled()) {
logger.debug("less than the required {} free bytes threshold ({} bytes free) on node {}, shard cannot remain", logger.debug("less than the required {} free bytes threshold ({} bytes free) on node {}, shard cannot remain",
@ -472,9 +486,8 @@ public class DiskThresholdDecider extends AllocationDecider {
return allocation.decision(Decision.YES, NAME, "enough disk for shard to remain on node, free: [%s]", new ByteSizeValue(freeBytes)); return allocation.decision(Decision.YES, NAME, "enough disk for shard to remain on node, free: [%s]", new ByteSizeValue(freeBytes));
} }
private DiskUsage getDiskUsage(RoutingNode node, RoutingAllocation allocation) { private DiskUsage getDiskUsage(RoutingNode node, RoutingAllocation allocation, Map<String, DiskUsage> usages) {
ClusterInfo clusterInfo = allocation.clusterInfo(); ClusterInfo clusterInfo = allocation.clusterInfo();
Map<String, DiskUsage> usages = clusterInfo.getNodeDiskUsages();
DiskUsage usage = usages.get(node.nodeId()); DiskUsage usage = usages.get(node.nodeId());
if (usage == null) { if (usage == null) {
// If there is no usage, and we have other nodes in the cluster, // If there is no usage, and we have other nodes in the cluster,
@ -487,8 +500,8 @@ public class DiskThresholdDecider extends AllocationDecider {
} }
if (includeRelocations) { if (includeRelocations) {
long relocatingShardsSize = sizeOfRelocatingShards(node, clusterInfo, true); long relocatingShardsSize = sizeOfRelocatingShards(node, clusterInfo, true, usage.getPath());
DiskUsage usageIncludingRelocations = new DiskUsage(node.nodeId(), node.node().name(), DiskUsage usageIncludingRelocations = new DiskUsage(node.nodeId(), node.node().name(), usage.getPath(),
usage.getTotalBytes(), usage.getFreeBytes() - relocatingShardsSize); usage.getTotalBytes(), usage.getFreeBytes() - relocatingShardsSize);
if (logger.isTraceEnabled()) { if (logger.isTraceEnabled()) {
logger.trace("usage without relocations: {}", usage); logger.trace("usage without relocations: {}", usage);
@ -508,7 +521,7 @@ public class DiskThresholdDecider extends AllocationDecider {
*/ */
public DiskUsage averageUsage(RoutingNode node, Map<String, DiskUsage> usages) { public DiskUsage averageUsage(RoutingNode node, Map<String, DiskUsage> usages) {
if (usages.size() == 0) { if (usages.size() == 0) {
return new DiskUsage(node.nodeId(), node.node().name(), 0, 0); return new DiskUsage(node.nodeId(), node.node().name(), "_na_", 0, 0);
} }
long totalBytes = 0; long totalBytes = 0;
long freeBytes = 0; long freeBytes = 0;
@ -516,7 +529,7 @@ public class DiskThresholdDecider extends AllocationDecider {
totalBytes += du.getTotalBytes(); totalBytes += du.getTotalBytes();
freeBytes += du.getFreeBytes(); freeBytes += du.getFreeBytes();
} }
return new DiskUsage(node.nodeId(), node.node().name(), totalBytes / usages.size(), freeBytes / usages.size()); return new DiskUsage(node.nodeId(), node.node().name(), "_na_", totalBytes / usages.size(), freeBytes / usages.size());
} }
/** /**
@ -528,8 +541,8 @@ public class DiskThresholdDecider extends AllocationDecider {
*/ */
public double freeDiskPercentageAfterShardAssigned(DiskUsage usage, Long shardSize) { public double freeDiskPercentageAfterShardAssigned(DiskUsage usage, Long shardSize) {
shardSize = (shardSize == null) ? 0 : shardSize; shardSize = (shardSize == null) ? 0 : shardSize;
DiskUsage newUsage = new DiskUsage(usage.getNodeId(), usage.getNodeName(), DiskUsage newUsage = new DiskUsage(usage.getNodeId(), usage.getNodeName(), usage.getPath(),
usage.getTotalBytes(), usage.getFreeBytes() - shardSize); usage.getTotalBytes(), usage.getFreeBytes() - shardSize);
return newUsage.getFreeDiskAsPercentage(); return newUsage.getFreeDiskAsPercentage();
} }
@ -600,7 +613,7 @@ public class DiskThresholdDecider extends AllocationDecider {
return allocation.decision(Decision.YES, NAME, "cluster info unavailable"); return allocation.decision(Decision.YES, NAME, "cluster info unavailable");
} }
final Map<String, DiskUsage> usages = clusterInfo.getNodeDiskUsages(); final Map<String, DiskUsage> usages = clusterInfo.getNodeLeastAvailableDiskUsages();
// Fail open if there are no disk usages available // Fail open if there are no disk usages available
if (usages.isEmpty()) { if (usages.isEmpty()) {
if (logger.isTraceEnabled()) { if (logger.isTraceEnabled()) {

View File

@ -46,8 +46,7 @@ public class LZFCompressor implements Compressor {
public LZFCompressor() { public LZFCompressor() {
this.decoder = ChunkDecoderFactory.safeInstance(); this.decoder = ChunkDecoderFactory.safeInstance();
Loggers.getLogger(LZFCompressor.class).debug("using encoder [{}] and decoder[{}] ", Loggers.getLogger(LZFCompressor.class).debug("using decoder[{}] ", this.decoder.getClass().getSimpleName());
this.decoder.getClass().getSimpleName());
} }
@Override @Override

View File

@ -27,7 +27,6 @@ import org.elasticsearch.*;
import org.elasticsearch.common.Base64; import org.elasticsearch.common.Base64;
import org.elasticsearch.common.Nullable; import org.elasticsearch.common.Nullable;
import org.elasticsearch.common.unit.TimeValue; import org.elasticsearch.common.unit.TimeValue;
import org.elasticsearch.common.util.ByteArray;
import java.io.*; import java.io.*;
import java.net.HttpURLConnection; import java.net.HttpURLConnection;
@ -37,9 +36,7 @@ import java.nio.file.Files;
import java.nio.file.NoSuchFileException; import java.nio.file.NoSuchFileException;
import java.nio.file.Path; import java.nio.file.Path;
import java.nio.file.attribute.FileTime; import java.nio.file.attribute.FileTime;
import java.util.Arrays;
import java.util.List; import java.util.List;
import java.util.concurrent.Callable;
/** /**
* *
@ -152,12 +149,6 @@ public class HttpDownloadHelper {
} catch (FileNotFoundException | NoSuchFileException e) { } catch (FileNotFoundException | NoSuchFileException e) {
// checksum file doesn't exist // checksum file doesn't exist
return false; return false;
} catch (IOException e) {
if (ExceptionsHelper.unwrapCause(e) instanceof FileNotFoundException) {
// checksum file didn't exist
return false;
}
throw e;
} finally { } finally {
IOUtils.deleteFilesIgnoringExceptions(checksumFile); IOUtils.deleteFilesIgnoringExceptions(checksumFile);
} }
@ -378,9 +369,6 @@ public class HttpDownloadHelper {
responseCode == HttpURLConnection.HTTP_MOVED_TEMP || responseCode == HttpURLConnection.HTTP_MOVED_TEMP ||
responseCode == HttpURLConnection.HTTP_SEE_OTHER) { responseCode == HttpURLConnection.HTTP_SEE_OTHER) {
String newLocation = httpConnection.getHeaderField("Location"); String newLocation = httpConnection.getHeaderField("Location");
String message = aSource
+ (responseCode == HttpURLConnection.HTTP_MOVED_PERM ? " permanently"
: "") + " moved to " + newLocation;
URL newURL = new URL(newLocation); URL newURL = new URL(newLocation);
if (!redirectionAllowed(aSource, newURL)) { if (!redirectionAllowed(aSource, newURL)) {
return null; return null;
@ -426,7 +414,7 @@ public class HttpDownloadHelper {
} }
} }
if (is == null) { if (is == null) {
throw new IOException("Can't get " + source + " to " + dest, lastEx); throw lastEx;
} }
os = Files.newOutputStream(dest); os = Files.newOutputStream(dest);

View File

@ -22,6 +22,7 @@ package org.elasticsearch.index;
import com.google.common.base.Function; import com.google.common.base.Function;
import com.google.common.collect.ImmutableMap; import com.google.common.collect.ImmutableMap;
import com.google.common.collect.Iterators; import com.google.common.collect.Iterators;
import org.apache.lucene.util.IOUtils; import org.apache.lucene.util.IOUtils;
import org.elasticsearch.ElasticsearchException; import org.elasticsearch.ElasticsearchException;
import org.elasticsearch.cluster.metadata.IndexMetaData; import org.elasticsearch.cluster.metadata.IndexMetaData;
@ -56,8 +57,10 @@ import org.elasticsearch.plugins.PluginsService;
import java.io.Closeable; import java.io.Closeable;
import java.io.IOException; import java.io.IOException;
import java.nio.file.Path;
import java.util.HashMap; import java.util.HashMap;
import java.util.Iterator; import java.util.Iterator;
import java.util.Map;
import java.util.Set; import java.util.Set;
import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicBoolean;
@ -315,8 +318,23 @@ public class IndexService extends AbstractIndexComponent implements IndexCompone
throw t; throw t;
} }
} }
if (path == null) { if (path == null) {
path = ShardPath.selectNewPathForShard(nodeEnv, shardId, indexSettings, routing.getExpectedShardSize() == ShardRouting.UNAVAILABLE_EXPECTED_SHARD_SIZE ? getAvgShardSizeInBytes() : routing.getExpectedShardSize(), this); // TODO: we should, instead, hold a "bytes reserved" of how large we anticipate this shard will be, e.g. for a shard
// that's being relocated/replicated we know how large it will become once it's done copying:
// Count up how many shards are currently on each data path:
Map<Path,Integer> dataPathToShardCount = new HashMap<>();
for(IndexShard shard : this) {
Path dataPath = shard.shardPath().getRootStatePath();
Integer curCount = dataPathToShardCount.get(dataPath);
if (curCount == null) {
curCount = 0;
}
dataPathToShardCount.put(dataPath, curCount+1);
}
path = ShardPath.selectNewPathForShard(nodeEnv, shardId, indexSettings, routing.getExpectedShardSize() == ShardRouting.UNAVAILABLE_EXPECTED_SHARD_SIZE ? getAvgShardSizeInBytes() : routing.getExpectedShardSize(),
dataPathToShardCount);
logger.debug("{} creating using a new path [{}]", shardId, path); logger.debug("{} creating using a new path [{}]", shardId, path);
} else { } else {
logger.debug("{} creating using an existing path [{}]", shardId, path); logger.debug("{} creating using an existing path [{}]", shardId, path);

View File

@ -104,8 +104,9 @@ class DocumentParser implements Closeable {
if (token != XContentParser.Token.START_OBJECT) { if (token != XContentParser.Token.START_OBJECT) {
throw new MapperParsingException("Malformed content, must start with an object"); throw new MapperParsingException("Malformed content, must start with an object");
} }
boolean emptyDoc = false;
if (mapping.root.isEnabled()) { if (mapping.root.isEnabled()) {
boolean emptyDoc = false;
token = parser.nextToken(); token = parser.nextToken();
if (token == XContentParser.Token.END_OBJECT) { if (token == XContentParser.Token.END_OBJECT) {
// empty doc, we can handle it... // empty doc, we can handle it...
@ -113,23 +114,24 @@ class DocumentParser implements Closeable {
} else if (token != XContentParser.Token.FIELD_NAME) { } else if (token != XContentParser.Token.FIELD_NAME) {
throw new MapperParsingException("Malformed content, after first object, either the type field or the actual properties should exist"); throw new MapperParsingException("Malformed content, after first object, either the type field or the actual properties should exist");
} }
}
for (MetadataFieldMapper metadataMapper : mapping.metadataMappers) { for (MetadataFieldMapper metadataMapper : mapping.metadataMappers) {
metadataMapper.preParse(context); metadataMapper.preParse(context);
} }
if (emptyDoc == false) {
Mapper update = parseObject(context, mapping.root);
if (update != null) {
context.addDynamicMappingsUpdate(update);
}
}
for (MetadataFieldMapper metadataMapper : mapping.metadataMappers) {
metadataMapper.postParse(context);
}
} else { if (mapping.root.isEnabled() == false) {
// entire type is disabled // entire type is disabled
parser.skipChildren(); parser.skipChildren();
} else if (emptyDoc == false) {
Mapper update = parseObject(context, mapping.root);
if (update != null) {
context.addDynamicMappingsUpdate(update);
}
}
for (MetadataFieldMapper metadataMapper : mapping.metadataMappers) {
metadataMapper.postParse(context);
} }
// try to parse the next token, this should be null if the object is ended properly // try to parse the next token, this should be null if the object is ended properly

View File

@ -199,19 +199,27 @@ public final class ShardPath {
} }
public static ShardPath selectNewPathForShard(NodeEnvironment env, ShardId shardId, @IndexSettings Settings indexSettings, public static ShardPath selectNewPathForShard(NodeEnvironment env, ShardId shardId, @IndexSettings Settings indexSettings,
long avgShardSizeInBytes, Iterable<IndexShard> shards) throws IOException { long avgShardSizeInBytes, Map<Path,Integer> dataPathToShardCount) throws IOException {
final Path dataPath; final Path dataPath;
final Path statePath; final Path statePath;
final String indexUUID = indexSettings.get(IndexMetaData.SETTING_INDEX_UUID, IndexMetaData.INDEX_UUID_NA_VALUE);
if (NodeEnvironment.hasCustomDataPath(indexSettings)) { if (NodeEnvironment.hasCustomDataPath(indexSettings)) {
dataPath = env.resolveCustomLocation(indexSettings, shardId); dataPath = env.resolveCustomLocation(indexSettings, shardId);
statePath = env.nodePaths()[0].resolve(shardId); statePath = env.nodePaths()[0].resolve(shardId);
} else { } else {
Map<Path,Long> estReservedBytes = getEstimatedReservedBytes(env, avgShardSizeInBytes, shards); long totFreeSpace = 0;
for (NodeEnvironment.NodePath nodePath : env.nodePaths()) {
totFreeSpace += nodePath.fileStore.getUsableSpace();
}
// TODO: this is a hack!! We should instead keep track of incoming (relocated) shards since we know
// how large they will be once they're done copying, instead of a silly guess for such cases:
// Very rough heurisic of how much disk space we expect the shard will use over its lifetime, the max of current average
// shard size across the cluster and 5% of the total available free space on this node:
long estShardSizeInBytes = Math.max(avgShardSizeInBytes, (long) (totFreeSpace/20.0));
// TODO - do we need something more extensible? Yet, this does the job for now... // TODO - do we need something more extensible? Yet, this does the job for now...
final NodeEnvironment.NodePath[] paths = env.nodePaths(); final NodeEnvironment.NodePath[] paths = env.nodePaths();
@ -220,10 +228,11 @@ public final class ShardPath {
for (NodeEnvironment.NodePath nodePath : paths) { for (NodeEnvironment.NodePath nodePath : paths) {
FileStore fileStore = nodePath.fileStore; FileStore fileStore = nodePath.fileStore;
long usableBytes = fileStore.getUsableSpace(); long usableBytes = fileStore.getUsableSpace();
Long reservedBytes = estReservedBytes.get(nodePath.path);
if (reservedBytes != null) { // Deduct estimated reserved bytes from usable space:
// Deduct estimated reserved bytes from usable space: Integer count = dataPathToShardCount.get(nodePath.path);
usableBytes -= reservedBytes; if (count != null) {
usableBytes -= estShardSizeInBytes * count;
} }
if (usableBytes > maxUsableBytes) { if (usableBytes > maxUsableBytes) {
maxUsableBytes = usableBytes; maxUsableBytes = usableBytes;
@ -235,6 +244,8 @@ public final class ShardPath {
dataPath = statePath; dataPath = statePath;
} }
final String indexUUID = indexSettings.get(IndexMetaData.SETTING_INDEX_UUID, IndexMetaData.INDEX_UUID_NA_VALUE);
return new ShardPath(NodeEnvironment.hasCustomDataPath(indexSettings), dataPath, statePath, indexUUID, shardId); return new ShardPath(NodeEnvironment.hasCustomDataPath(indexSettings), dataPath, statePath, indexUUID, shardId);
} }

View File

@ -254,7 +254,7 @@ public class IndicesService extends AbstractLifecycleComponent<IndicesService> i
if (indexShard.routingEntry() == null) { if (indexShard.routingEntry() == null) {
continue; continue;
} }
IndexShardStats indexShardStats = new IndexShardStats(indexShard.shardId(), new ShardStats[] { new ShardStats(indexShard, flags) }); IndexShardStats indexShardStats = new IndexShardStats(indexShard.shardId(), new ShardStats[] { new ShardStats(indexShard.routingEntry(), indexShard.shardPath(), new CommonStats(indexShard, flags), indexShard.commitStats()) });
if (!statsByShard.containsKey(indexService.index())) { if (!statsByShard.containsKey(indexService.index())) {
statsByShard.put(indexService.index(), arrayAsArrayList(indexShardStats)); statsByShard.put(indexService.index(), arrayAsArrayList(indexShardStats));
} else { } else {

View File

@ -73,6 +73,7 @@ import org.elasticsearch.node.internal.InternalSettingsPreparer;
import org.elasticsearch.node.settings.NodeSettingsService; import org.elasticsearch.node.settings.NodeSettingsService;
import org.elasticsearch.percolator.PercolatorModule; import org.elasticsearch.percolator.PercolatorModule;
import org.elasticsearch.percolator.PercolatorService; import org.elasticsearch.percolator.PercolatorService;
import org.elasticsearch.plugins.Plugin;
import org.elasticsearch.plugins.PluginsModule; import org.elasticsearch.plugins.PluginsModule;
import org.elasticsearch.plugins.PluginsService; import org.elasticsearch.plugins.PluginsService;
import org.elasticsearch.repositories.RepositoriesModule; import org.elasticsearch.repositories.RepositoriesModule;
@ -95,6 +96,8 @@ import org.elasticsearch.watcher.ResourceWatcherService;
import java.io.IOException; import java.io.IOException;
import java.util.Arrays; import java.util.Arrays;
import java.util.Collection;
import java.util.Collections;
import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeUnit;
import static org.elasticsearch.common.settings.Settings.settingsBuilder; import static org.elasticsearch.common.settings.Settings.settingsBuilder;
@ -117,19 +120,22 @@ public class Node implements Releasable {
private final PluginsService pluginsService; private final PluginsService pluginsService;
private final Client client; private final Client client;
public Node() { /**
this(Settings.Builder.EMPTY_SETTINGS, true); * Constructs a node with the given settings.
*
* @param preparedSettings Base settings to configure the node with
* @param loadConfigSettings true if settings should also be loaded and merged from configuration files
*/
public Node(Settings preparedSettings, boolean loadConfigSettings) {
this(preparedSettings, loadConfigSettings, Version.CURRENT, Collections.<Class<? extends Plugin>>emptyList());
} }
public Node(Settings preparedSettings, boolean loadConfigSettings) { Node(Settings preparedSettings, boolean loadConfigSettings, Version version, Collection<Class<? extends Plugin>> classpathPlugins) {
final Settings pSettings = settingsBuilder().put(preparedSettings) final Settings pSettings = settingsBuilder().put(preparedSettings)
.put(Client.CLIENT_TYPE_SETTING, CLIENT_TYPE).build(); .put(Client.CLIENT_TYPE_SETTING, CLIENT_TYPE).build();
Tuple<Settings, Environment> tuple = InternalSettingsPreparer.prepareSettings(pSettings, loadConfigSettings); Tuple<Settings, Environment> tuple = InternalSettingsPreparer.prepareSettings(pSettings, loadConfigSettings);
tuple = new Tuple<>(TribeService.processSettings(tuple.v1()), tuple.v2()); tuple = new Tuple<>(TribeService.processSettings(tuple.v1()), tuple.v2());
// The only place we can actually fake the version a node is running on:
Version version = pSettings.getAsVersion("tests.mock.version", Version.CURRENT);
ESLogger logger = Loggers.getLogger(Node.class, tuple.v1().get("name")); ESLogger logger = Loggers.getLogger(Node.class, tuple.v1().get("name"));
logger.info("version[{}], pid[{}], build[{}/{}]", version, JvmInfo.jvmInfo().pid(), Build.CURRENT.hashShort(), Build.CURRENT.timestamp()); logger.info("version[{}], pid[{}], build[{}/{}]", version, JvmInfo.jvmInfo().pid(), Build.CURRENT.hashShort(), Build.CURRENT.timestamp());
@ -141,7 +147,7 @@ public class Node implements Releasable {
env.configFile(), Arrays.toString(env.dataFiles()), env.logsFile(), env.pluginsFile()); env.configFile(), Arrays.toString(env.dataFiles()), env.logsFile(), env.pluginsFile());
} }
this.pluginsService = new PluginsService(tuple.v1(), tuple.v2()); this.pluginsService = new PluginsService(tuple.v1(), tuple.v2(), classpathPlugins);
this.settings = pluginsService.updatedSettings(); this.settings = pluginsService.updatedSettings();
// create the environment based on the finalized (processed) view of the settings // create the environment based on the finalized (processed) view of the settings
this.environment = new Environment(this.settings()); this.environment = new Environment(this.settings());
@ -421,15 +427,4 @@ public class Node implements Releasable {
public Injector injector() { public Injector injector() {
return this.injector; return this.injector;
} }
public static void main(String[] args) throws Exception {
final Node node = new Node();
node.start();
Runtime.getRuntime().addShutdownHook(new Thread() {
@Override
public void run() {
node.close();
}
});
}
} }

View File

@ -463,17 +463,15 @@ public class PluginManager {
if (version != null) { if (version != null) {
// Elasticsearch new download service uses groupId org.elasticsearch.plugin from 2.0.0 // Elasticsearch new download service uses groupId org.elasticsearch.plugin from 2.0.0
if (user == null) { if (user == null) {
// TODO Update to https
if (!Strings.isNullOrEmpty(System.getProperty(PROPERTY_SUPPORT_STAGING_URLS))) { if (!Strings.isNullOrEmpty(System.getProperty(PROPERTY_SUPPORT_STAGING_URLS))) {
addUrl(urls, String.format(Locale.ROOT, "http://download.elastic.co/elasticsearch/staging/%s-%s/org/elasticsearch/plugin/%s/%s/%s-%s.zip", version, Build.CURRENT.hashShort(), name, version, name, version)); addUrl(urls, String.format(Locale.ROOT, "https://download.elastic.co/elasticsearch/staging/%s-%s/org/elasticsearch/plugin/%s/%s/%s-%s.zip", version, Build.CURRENT.hashShort(), name, version, name, version));
} }
addUrl(urls, String.format(Locale.ROOT, "http://download.elastic.co/elasticsearch/release/org/elasticsearch/plugin/%s/%s/%s-%s.zip", name, version, name, version)); addUrl(urls, String.format(Locale.ROOT, "https://download.elastic.co/elasticsearch/release/org/elasticsearch/plugin/%s/%s/%s-%s.zip", name, version, name, version));
} else { } else {
// Elasticsearch old download service // Elasticsearch old download service
// TODO Update to https addUrl(urls, String.format(Locale.ROOT, "https://download.elastic.co/%1$s/%2$s/%2$s-%3$s.zip", user, name, version));
addUrl(urls, String.format(Locale.ROOT, "http://download.elastic.co/%1$s/%2$s/%2$s-%3$s.zip", user, name, version));
// Maven central repository // Maven central repository
addUrl(urls, String.format(Locale.ROOT, "http://search.maven.org/remotecontent?filepath=%1$s/%2$s/%3$s/%2$s-%3$s.zip", user.replace('.', '/'), name, version)); addUrl(urls, String.format(Locale.ROOT, "https://search.maven.org/remotecontent?filepath=%1$s/%2$s/%3$s/%2$s-%3$s.zip", user.replace('.', '/'), name, version));
// Sonatype repository // Sonatype repository
addUrl(urls, String.format(Locale.ROOT, "https://oss.sonatype.org/service/local/repositories/releases/content/%1$s/%2$s/%3$s/%2$s-%3$s.zip", user.replace('.', '/'), name, version)); addUrl(urls, String.format(Locale.ROOT, "https://oss.sonatype.org/service/local/repositories/releases/content/%1$s/%2$s/%3$s/%2$s-%3$s.zip", user.replace('.', '/'), name, version));
// Github repository // Github repository

View File

@ -70,10 +70,10 @@ public class PluginsService extends AbstractComponent {
/** /**
* We keep around a list of plugins * We keep around a list of plugins
*/ */
private final ImmutableList<Tuple<PluginInfo, Plugin>> plugins; private final List<Tuple<PluginInfo, Plugin>> plugins;
private final PluginsInfo info; private final PluginsInfo info;
private final ImmutableMap<Plugin, List<OnModuleReference>> onModuleReferences; private final Map<Plugin, List<OnModuleReference>> onModuleReferences;
static class OnModuleReference { static class OnModuleReference {
public final Class<? extends Module> moduleClass; public final Class<? extends Module> moduleClass;
@ -89,20 +89,19 @@ public class PluginsService extends AbstractComponent {
* Constructs a new PluginService * Constructs a new PluginService
* @param settings The settings of the system * @param settings The settings of the system
* @param environment The environment of the system * @param environment The environment of the system
* @param classpathPlugins Plugins that exist in the classpath which should be loaded
*/ */
public PluginsService(Settings settings, Environment environment) { public PluginsService(Settings settings, Environment environment, Collection<Class<? extends Plugin>> classpathPlugins) {
super(settings); super(settings);
ImmutableList.Builder<Tuple<PluginInfo, Plugin>> tupleBuilder = ImmutableList.builder(); List<Tuple<PluginInfo, Plugin>> tupleBuilder = new ArrayList<>();
// first we load specified plugins via 'plugin.types' settings parameter. // first we load plugins that are on the classpath. this is for tests and transport clients
// this is a hack for what is between unit and integration tests... for (Class<? extends Plugin> pluginClass : classpathPlugins) {
String[] defaultPluginsClasses = settings.getAsArray("plugin.types"); Plugin plugin = loadPlugin(pluginClass, settings);
for (String pluginClass : defaultPluginsClasses) { PluginInfo pluginInfo = new PluginInfo(plugin.name(), plugin.description(), false, "NA", true, pluginClass.getName(), false);
Plugin plugin = loadPlugin(pluginClass, settings, getClass().getClassLoader());
PluginInfo pluginInfo = new PluginInfo(plugin.name(), plugin.description(), false, "NA", true, pluginClass, false);
if (logger.isTraceEnabled()) { if (logger.isTraceEnabled()) {
logger.trace("plugin loaded from settings [{}]", pluginInfo); logger.trace("plugin loaded from classpath [{}]", pluginInfo);
} }
tupleBuilder.add(new Tuple<>(pluginInfo, plugin)); tupleBuilder.add(new Tuple<>(pluginInfo, plugin));
} }
@ -115,7 +114,7 @@ public class PluginsService extends AbstractComponent {
throw new IllegalStateException("Unable to initialize plugins", ex); throw new IllegalStateException("Unable to initialize plugins", ex);
} }
plugins = tupleBuilder.build(); plugins = Collections.unmodifiableList(tupleBuilder);
info = new PluginsInfo(); info = new PluginsInfo();
for (Tuple<PluginInfo, Plugin> tuple : plugins) { for (Tuple<PluginInfo, Plugin> tuple : plugins) {
info.add(tuple.v1()); info.add(tuple.v1());
@ -128,7 +127,7 @@ public class PluginsService extends AbstractComponent {
for (Tuple<PluginInfo, Plugin> tuple : plugins) { for (Tuple<PluginInfo, Plugin> tuple : plugins) {
PluginInfo info = tuple.v1(); PluginInfo info = tuple.v1();
if (info.isJvm()) { if (info.isJvm()) {
jvmPlugins.put(tuple.v2().name(), tuple.v2()); jvmPlugins.put(info.getName(), tuple.v2());
} }
if (info.isSite()) { if (info.isSite()) {
sitePlugins.add(info.getName()); sitePlugins.add(info.getName());
@ -151,7 +150,7 @@ public class PluginsService extends AbstractComponent {
logger.info("loaded {}, sites {}", jvmPlugins.keySet(), sitePlugins); logger.info("loaded {}, sites {}", jvmPlugins.keySet(), sitePlugins);
MapBuilder<Plugin, List<OnModuleReference>> onModuleReferences = MapBuilder.newMapBuilder(); Map<Plugin, List<OnModuleReference>> onModuleReferences = new HashMap<>();
for (Plugin plugin : jvmPlugins.values()) { for (Plugin plugin : jvmPlugins.values()) {
List<OnModuleReference> list = new ArrayList<>(); List<OnModuleReference> list = new ArrayList<>();
for (Method method : plugin.getClass().getMethods()) { for (Method method : plugin.getClass().getMethods()) {
@ -173,10 +172,10 @@ public class PluginsService extends AbstractComponent {
onModuleReferences.put(plugin, list); onModuleReferences.put(plugin, list);
} }
} }
this.onModuleReferences = onModuleReferences.immutableMap(); this.onModuleReferences = Collections.unmodifiableMap(onModuleReferences);
} }
public ImmutableList<Tuple<PluginInfo, Plugin>> plugins() { public List<Tuple<PluginInfo, Plugin>> plugins() {
return plugins; return plugins;
} }
@ -355,7 +354,8 @@ public class PluginsService extends AbstractComponent {
if (pluginInfo.isJvm()) { if (pluginInfo.isJvm()) {
// reload lucene SPI with any new services from the plugin // reload lucene SPI with any new services from the plugin
reloadLuceneSPI(loader); reloadLuceneSPI(loader);
plugin = loadPlugin(pluginInfo.getClassname(), settings, loader); Class<? extends Plugin> pluginClass = loadPluginClass(pluginInfo.getClassname(), loader);
plugin = loadPlugin(pluginClass, settings);
} else { } else {
plugin = new SitePlugin(pluginInfo.getName(), pluginInfo.getDescription()); plugin = new SitePlugin(pluginInfo.getName(), pluginInfo.getDescription());
} }
@ -384,10 +384,16 @@ public class PluginsService extends AbstractComponent {
TokenizerFactory.reloadTokenizers(loader); TokenizerFactory.reloadTokenizers(loader);
} }
private Plugin loadPlugin(String className, Settings settings, ClassLoader loader) { private Class<? extends Plugin> loadPluginClass(String className, ClassLoader loader) {
try { try {
Class<? extends Plugin> pluginClass = loader.loadClass(className).asSubclass(Plugin.class); return loader.loadClass(className).asSubclass(Plugin.class);
} catch (ClassNotFoundException e) {
throw new ElasticsearchException("Could not find plugin class [" + className + "]", e);
}
}
private Plugin loadPlugin(Class<? extends Plugin> pluginClass, Settings settings) {
try {
try { try {
return pluginClass.getConstructor(Settings.class).newInstance(settings); return pluginClass.getConstructor(Settings.class).newInstance(settings);
} catch (NoSuchMethodException e) { } catch (NoSuchMethodException e) {
@ -395,13 +401,12 @@ public class PluginsService extends AbstractComponent {
return pluginClass.getConstructor().newInstance(); return pluginClass.getConstructor().newInstance();
} catch (NoSuchMethodException e1) { } catch (NoSuchMethodException e1) {
throw new ElasticsearchException("No constructor for [" + pluginClass + "]. A plugin class must " + throw new ElasticsearchException("No constructor for [" + pluginClass + "]. A plugin class must " +
"have either an empty default constructor or a single argument constructor accepting a " + "have either an empty default constructor or a single argument constructor accepting a " +
"Settings instance"); "Settings instance");
} }
} }
} catch (Throwable e) { } catch (Throwable e) {
throw new ElasticsearchException("Failed to load plugin class [" + className + "]", e); throw new ElasticsearchException("Failed to load plugin class [" + pluginClass.getName() + "]", e);
} }
} }
} }

View File

@ -22,7 +22,6 @@ package org.elasticsearch.rest.action.cat;
import org.apache.lucene.util.CollectionUtil; import org.apache.lucene.util.CollectionUtil;
import org.elasticsearch.action.admin.indices.recovery.RecoveryRequest; import org.elasticsearch.action.admin.indices.recovery.RecoveryRequest;
import org.elasticsearch.action.admin.indices.recovery.RecoveryResponse; import org.elasticsearch.action.admin.indices.recovery.RecoveryResponse;
import org.elasticsearch.action.admin.indices.recovery.ShardRecoveryResponse;
import org.elasticsearch.action.support.IndicesOptions; import org.elasticsearch.action.support.IndicesOptions;
import org.elasticsearch.client.Client; import org.elasticsearch.client.Client;
import org.elasticsearch.common.Strings; import org.elasticsearch.common.Strings;
@ -116,19 +115,19 @@ public class RestRecoveryAction extends AbstractCatAction {
Table t = getTableWithHeader(request); Table t = getTableWithHeader(request);
for (String index : response.shardResponses().keySet()) { for (String index : response.shardRecoveryStates().keySet()) {
List<ShardRecoveryResponse> shardRecoveryResponses = response.shardResponses().get(index); List<RecoveryState> shardRecoveryStates = response.shardRecoveryStates().get(index);
if (shardRecoveryResponses.size() == 0) { if (shardRecoveryStates.size() == 0) {
continue; continue;
} }
// Sort ascending by shard id for readability // Sort ascending by shard id for readability
CollectionUtil.introSort(shardRecoveryResponses, new Comparator<ShardRecoveryResponse>() { CollectionUtil.introSort(shardRecoveryStates, new Comparator<RecoveryState>() {
@Override @Override
public int compare(ShardRecoveryResponse o1, ShardRecoveryResponse o2) { public int compare(RecoveryState o1, RecoveryState o2) {
int id1 = o1.recoveryState().getShardId().id(); int id1 = o1.getShardId().id();
int id2 = o2.recoveryState().getShardId().id(); int id2 = o2.getShardId().id();
if (id1 < id2) { if (id1 < id2) {
return -1; return -1;
} else if (id1 > id2) { } else if (id1 > id2) {
@ -139,12 +138,10 @@ public class RestRecoveryAction extends AbstractCatAction {
} }
}); });
for (ShardRecoveryResponse shardResponse : shardRecoveryResponses) { for (RecoveryState state: shardRecoveryStates) {
RecoveryState state = shardResponse.recoveryState();
t.startRow(); t.startRow();
t.addCell(index); t.addCell(index);
t.addCell(shardResponse.getShardId()); t.addCell(state.getShardId().id());
t.addCell(state.getTimer().time()); t.addCell(state.getTimer().time());
t.addCell(state.getType().toString().toLowerCase(Locale.ROOT)); t.addCell(state.getType().toString().toLowerCase(Locale.ROOT));
t.addCell(state.getStage().toString().toLowerCase(Locale.ROOT)); t.addCell(state.getStage().toString().toLowerCase(Locale.ROOT));

View File

@ -21,7 +21,11 @@ package org.elasticsearch.rest.action.cat;
import org.elasticsearch.action.admin.cluster.state.ClusterStateRequest; import org.elasticsearch.action.admin.cluster.state.ClusterStateRequest;
import org.elasticsearch.action.admin.cluster.state.ClusterStateResponse; import org.elasticsearch.action.admin.cluster.state.ClusterStateResponse;
import org.elasticsearch.action.admin.indices.segments.*; import org.elasticsearch.action.admin.indices.segments.IndexSegments;
import org.elasticsearch.action.admin.indices.segments.IndexShardSegments;
import org.elasticsearch.action.admin.indices.segments.IndicesSegmentResponse;
import org.elasticsearch.action.admin.indices.segments.IndicesSegmentsRequest;
import org.elasticsearch.action.admin.indices.segments.ShardSegments;
import org.elasticsearch.client.Client; import org.elasticsearch.client.Client;
import org.elasticsearch.cluster.node.DiscoveryNodes; import org.elasticsearch.cluster.node.DiscoveryNodes;
import org.elasticsearch.common.Strings; import org.elasticsearch.common.Strings;
@ -29,7 +33,10 @@ import org.elasticsearch.common.Table;
import org.elasticsearch.common.inject.Inject; import org.elasticsearch.common.inject.Inject;
import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.index.engine.Segment; import org.elasticsearch.index.engine.Segment;
import org.elasticsearch.rest.*; import org.elasticsearch.rest.RestChannel;
import org.elasticsearch.rest.RestController;
import org.elasticsearch.rest.RestRequest;
import org.elasticsearch.rest.RestResponse;
import org.elasticsearch.rest.action.support.RestActionListener; import org.elasticsearch.rest.action.support.RestActionListener;
import org.elasticsearch.rest.action.support.RestResponseListener; import org.elasticsearch.rest.action.support.RestResponseListener;
import org.elasticsearch.rest.action.support.RestTable; import org.elasticsearch.rest.action.support.RestTable;
@ -120,8 +127,8 @@ public class RestSegmentsAction extends AbstractCatAction {
for (Segment segment : segments) { for (Segment segment : segments) {
table.startRow(); table.startRow();
table.addCell(shardSegment.getIndex()); table.addCell(shardSegment.getShardRouting().getIndex());
table.addCell(shardSegment.getShardId()); table.addCell(shardSegment.getShardRouting().getId());
table.addCell(shardSegment.getShardRouting().primary() ? "p" : "r"); table.addCell(shardSegment.getShardRouting().primary() ? "p" : "r");
table.addCell(nodes.get(shardSegment.getShardRouting().currentNodeId()).getHostAddress()); table.addCell(nodes.get(shardSegment.getShardRouting().currentNodeId()).getHostAddress());
table.addCell(shardSegment.getShardRouting().currentNodeId()); table.addCell(shardSegment.getShardRouting().currentNodeId());

View File

@ -68,7 +68,7 @@ public class RestTable {
public static RestResponse buildTextPlainResponse(Table table, RestChannel channel) throws IOException { public static RestResponse buildTextPlainResponse(Table table, RestChannel channel) throws IOException {
RestRequest request = channel.request(); RestRequest request = channel.request();
boolean verbose = request.paramAsBoolean("v", true); boolean verbose = request.paramAsBoolean("v", false);
List<DisplayHeader> headers = buildDisplayHeaders(table, request); List<DisplayHeader> headers = buildDisplayHeaders(table, request);
int[] width = buildWidths(table, request, verbose, headers); int[] width = buildWidths(table, request, verbose, headers);

View File

@ -21,6 +21,7 @@ package org.elasticsearch.transport;
import java.lang.reflect.Constructor; import java.lang.reflect.Constructor;
import java.util.concurrent.Callable;
/** /**
* *
@ -28,20 +29,19 @@ import java.lang.reflect.Constructor;
public class RequestHandlerRegistry<Request extends TransportRequest> { public class RequestHandlerRegistry<Request extends TransportRequest> {
private final String action; private final String action;
private final Constructor<Request> requestConstructor;
private final TransportRequestHandler<Request> handler; private final TransportRequestHandler<Request> handler;
private final boolean forceExecution; private final boolean forceExecution;
private final String executor; private final String executor;
private final Callable<Request> requestFactory;
RequestHandlerRegistry(String action, Class<Request> request, TransportRequestHandler<Request> handler, RequestHandlerRegistry(String action, Class<Request> request, TransportRequestHandler<Request> handler,
String executor, boolean forceExecution) { String executor, boolean forceExecution) {
this(action, new ReflectionFactory<>(request), handler, executor, forceExecution);
}
public RequestHandlerRegistry(String action, Callable<Request> requestFactory, TransportRequestHandler<Request> handler, String executor, boolean forceExecution) {
this.action = action; this.action = action;
try { this.requestFactory = requestFactory;
this.requestConstructor = request.getDeclaredConstructor();
} catch (NoSuchMethodException e) {
throw new IllegalStateException("failed to create constructor (does it have a default constructor?) for request " + request, e);
}
this.requestConstructor.setAccessible(true);
assert newRequest() != null; assert newRequest() != null;
this.handler = handler; this.handler = handler;
this.forceExecution = forceExecution; this.forceExecution = forceExecution;
@ -54,7 +54,7 @@ public class RequestHandlerRegistry<Request extends TransportRequest> {
public Request newRequest() { public Request newRequest() {
try { try {
return requestConstructor.newInstance(); return requestFactory.call();
} catch (Exception e) { } catch (Exception e) {
throw new IllegalStateException("failed to instantiate request ", e); throw new IllegalStateException("failed to instantiate request ", e);
} }
@ -71,4 +71,22 @@ public class RequestHandlerRegistry<Request extends TransportRequest> {
public String getExecutor() { public String getExecutor() {
return executor; return executor;
} }
private final static class ReflectionFactory<Request> implements Callable<Request> {
private final Constructor<Request> requestConstructor;
public ReflectionFactory(Class<Request> request) {
try {
this.requestConstructor = request.getDeclaredConstructor();
} catch (NoSuchMethodException e) {
throw new IllegalStateException("failed to create constructor (does it have a default constructor?) for request " + request, e);
}
this.requestConstructor.setAccessible(true);
}
@Override
public Request call() throws Exception {
return requestConstructor.newInstance();
}
}
} }

View File

@ -85,7 +85,7 @@ public class TransportModule extends AbstractModule {
bind(TransportService.class).asEagerSingleton(); bind(TransportService.class).asEagerSingleton();
} else { } else {
if (transportServices.containsKey(typeName) == false) { if (transportServices.containsKey(typeName) == false) {
throw new IllegalArgumentException("Unknown TransportService [" + typeName + "]"); throw new IllegalArgumentException("Unknown TransportService type [" + typeName + "], known types are: " + transportServices.keySet());
} }
bind(TransportService.class).to(transportServices.get(typeName)).asEagerSingleton(); bind(TransportService.class).to(transportServices.get(typeName)).asEagerSingleton();
} }

View File

@ -22,8 +22,6 @@ package org.elasticsearch.transport;
import com.google.common.collect.ImmutableMap; import com.google.common.collect.ImmutableMap;
import org.elasticsearch.action.admin.cluster.node.liveness.TransportLivenessAction; import org.elasticsearch.action.admin.cluster.node.liveness.TransportLivenessAction;
import org.elasticsearch.cluster.node.DiscoveryNode; import org.elasticsearch.cluster.node.DiscoveryNode;
import org.elasticsearch.cluster.settings.ClusterDynamicSettings;
import org.elasticsearch.cluster.settings.DynamicSettings;
import org.elasticsearch.common.Strings; import org.elasticsearch.common.Strings;
import org.elasticsearch.common.collect.MapBuilder; import org.elasticsearch.common.collect.MapBuilder;
import org.elasticsearch.common.component.AbstractLifecycleComponent; import org.elasticsearch.common.component.AbstractLifecycleComponent;
@ -35,12 +33,21 @@ import org.elasticsearch.common.regex.Regex;
import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.transport.BoundTransportAddress; import org.elasticsearch.common.transport.BoundTransportAddress;
import org.elasticsearch.common.transport.TransportAddress; import org.elasticsearch.common.transport.TransportAddress;
import org.elasticsearch.common.util.concurrent.*; import org.elasticsearch.common.util.concurrent.AbstractRunnable;
import org.elasticsearch.common.util.concurrent.ConcurrentCollections;
import org.elasticsearch.common.util.concurrent.ConcurrentMapLong;
import org.elasticsearch.common.util.concurrent.EsRejectedExecutionException;
import org.elasticsearch.common.util.concurrent.FutureUtils;
import org.elasticsearch.node.settings.NodeSettingsService; import org.elasticsearch.node.settings.NodeSettingsService;
import org.elasticsearch.threadpool.ThreadPool; import org.elasticsearch.threadpool.ThreadPool;
import java.io.IOException; import java.io.IOException;
import java.util.*; import java.util.Arrays;
import java.util.Collections;
import java.util.LinkedHashMap;
import java.util.List;
import java.util.Map;
import java.util.concurrent.Callable;
import java.util.concurrent.CopyOnWriteArrayList; import java.util.concurrent.CopyOnWriteArrayList;
import java.util.concurrent.ScheduledFuture; import java.util.concurrent.ScheduledFuture;
import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicBoolean;
@ -399,6 +406,18 @@ public class TransportService extends AbstractLifecycleComponent<TransportServic
registerRequestHandler(action, request, executor, false, handler); registerRequestHandler(action, request, executor, false, handler);
} }
/**
* Registers a new request handler
* @param action The action the request handler is associated with
* @param requestFactory a callable to be used construct new instances for streaming
* @param executor The executor the request handling will be executed on
* @param handler The handler itself that implements the request handling
*/
public <Request extends TransportRequest> void registerRequestHandler(String action, Callable<Request> requestFactory, String executor, TransportRequestHandler<Request> handler) {
RequestHandlerRegistry<Request> reg = new RequestHandlerRegistry<>(action, requestFactory, handler, executor, false);
registerRequestHandler(reg);
}
/** /**
* Registers a new request handler * Registers a new request handler
* @param action The action the request handler is associated with * @param action The action the request handler is associated with
@ -408,8 +427,12 @@ public class TransportService extends AbstractLifecycleComponent<TransportServic
* @param handler The handler itself that implements the request handling * @param handler The handler itself that implements the request handling
*/ */
public <Request extends TransportRequest> void registerRequestHandler(String action, Class<Request> request, String executor, boolean forceExecution, TransportRequestHandler<Request> handler) { public <Request extends TransportRequest> void registerRequestHandler(String action, Class<Request> request, String executor, boolean forceExecution, TransportRequestHandler<Request> handler) {
RequestHandlerRegistry<Request> reg = new RequestHandlerRegistry<>(action, request, handler, executor, forceExecution);
registerRequestHandler(reg);
}
protected <Request extends TransportRequest> void registerRequestHandler(RequestHandlerRegistry<Request> reg) {
synchronized (requestHandlerMutex) { synchronized (requestHandlerMutex) {
RequestHandlerRegistry<Request> reg = new RequestHandlerRegistry<>(action, request, handler, executor, forceExecution);
RequestHandlerRegistry replaced = requestHandlers.get(reg.getAction()); RequestHandlerRegistry replaced = requestHandlers.get(reg.getAction());
requestHandlers = MapBuilder.newMapBuilder(requestHandlers).put(reg.getAction(), reg).immutableMap(); requestHandlers = MapBuilder.newMapBuilder(requestHandlers).put(reg.getAction(), reg).immutableMap();
if (replaced != null) { if (replaced != null) {

View File

@ -95,6 +95,7 @@ import org.elasticsearch.search.action.SearchServiceTransportAction;
import org.elasticsearch.test.ESIntegTestCase; import org.elasticsearch.test.ESIntegTestCase;
import org.elasticsearch.test.ESIntegTestCase.ClusterScope; import org.elasticsearch.test.ESIntegTestCase.ClusterScope;
import org.elasticsearch.test.ESIntegTestCase.Scope; import org.elasticsearch.test.ESIntegTestCase.Scope;
import org.elasticsearch.test.transport.MockTransportService;
import org.elasticsearch.threadpool.ThreadPool; import org.elasticsearch.threadpool.ThreadPool;
import org.elasticsearch.transport.Transport; import org.elasticsearch.transport.Transport;
import org.elasticsearch.transport.TransportChannel; import org.elasticsearch.transport.TransportChannel;
@ -107,12 +108,14 @@ import org.junit.Before;
import org.junit.Test; import org.junit.Test;
import java.util.ArrayList; import java.util.ArrayList;
import java.util.Collection;
import java.util.Collections; import java.util.Collections;
import java.util.HashMap; import java.util.HashMap;
import java.util.HashSet; import java.util.HashSet;
import java.util.List; import java.util.List;
import java.util.Map; import java.util.Map;
import java.util.Set; import java.util.Set;
import java.util.concurrent.Callable;
import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertAcked; import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertAcked;
import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertNoFailures; import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertNoFailures;
@ -141,11 +144,15 @@ public class IndicesRequestIT extends ESIntegTestCase {
} }
@Override @Override
protected Settings nodeSettings(int nodeOrdinal) { protected Settings nodeSettings(int ordinal) {
return Settings.settingsBuilder() // must set this independently of the plugin so it overrides MockTransportService
.put(super.nodeSettings(nodeOrdinal)) return Settings.builder().put(super.nodeSettings(ordinal))
.extendArray("plugin.types", InterceptingTransportService.TestPlugin.class.getName()) .put(TransportModule.TRANSPORT_SERVICE_TYPE_KEY, "intercepting").build();
.build(); }
@Override
protected Collection<Class<? extends Plugin>> nodePlugins() {
return pluginList(InterceptingTransportService.TestPlugin.class);
} }
@Before @Before
@ -395,7 +402,7 @@ public class IndicesRequestIT extends ESIntegTestCase {
@Test @Test
public void testOptimize() { public void testOptimize() {
String optimizeShardAction = OptimizeAction.NAME + "[s]"; String optimizeShardAction = OptimizeAction.NAME + "[n]";
interceptTransportActions(optimizeShardAction); interceptTransportActions(optimizeShardAction);
OptimizeRequest optimizeRequest = new OptimizeRequest(randomIndicesOrAliases()); OptimizeRequest optimizeRequest = new OptimizeRequest(randomIndicesOrAliases());
@ -419,7 +426,7 @@ public class IndicesRequestIT extends ESIntegTestCase {
@Test @Test
public void testClearCache() { public void testClearCache() {
String clearCacheAction = ClearIndicesCacheAction.NAME + "[s]"; String clearCacheAction = ClearIndicesCacheAction.NAME + "[n]";
interceptTransportActions(clearCacheAction); interceptTransportActions(clearCacheAction);
ClearIndicesCacheRequest clearIndicesCacheRequest = new ClearIndicesCacheRequest(randomIndicesOrAliases()); ClearIndicesCacheRequest clearIndicesCacheRequest = new ClearIndicesCacheRequest(randomIndicesOrAliases());
@ -431,7 +438,7 @@ public class IndicesRequestIT extends ESIntegTestCase {
@Test @Test
public void testRecovery() { public void testRecovery() {
String recoveryAction = RecoveryAction.NAME + "[s]"; String recoveryAction = RecoveryAction.NAME + "[n]";
interceptTransportActions(recoveryAction); interceptTransportActions(recoveryAction);
RecoveryRequest recoveryRequest = new RecoveryRequest(randomIndicesOrAliases()); RecoveryRequest recoveryRequest = new RecoveryRequest(randomIndicesOrAliases());
@ -443,7 +450,7 @@ public class IndicesRequestIT extends ESIntegTestCase {
@Test @Test
public void testSegments() { public void testSegments() {
String segmentsAction = IndicesSegmentsAction.NAME + "[s]"; String segmentsAction = IndicesSegmentsAction.NAME + "[n]";
interceptTransportActions(segmentsAction); interceptTransportActions(segmentsAction);
IndicesSegmentsRequest segmentsRequest = new IndicesSegmentsRequest(randomIndicesOrAliases()); IndicesSegmentsRequest segmentsRequest = new IndicesSegmentsRequest(randomIndicesOrAliases());
@ -455,7 +462,7 @@ public class IndicesRequestIT extends ESIntegTestCase {
@Test @Test
public void testIndicesStats() { public void testIndicesStats() {
String indicesStats = IndicesStatsAction.NAME + "[s]"; String indicesStats = IndicesStatsAction.NAME + "[n]";
interceptTransportActions(indicesStats); interceptTransportActions(indicesStats);
IndicesStatsRequest indicesStatsRequest = new IndicesStatsRequest().indices(randomIndicesOrAliases()); IndicesStatsRequest indicesStatsRequest = new IndicesStatsRequest().indices(randomIndicesOrAliases());
@ -856,10 +863,6 @@ public class IndicesRequestIT extends ESIntegTestCase {
public void onModule(TransportModule transportModule) { public void onModule(TransportModule transportModule) {
transportModule.addTransportService("intercepting", InterceptingTransportService.class); transportModule.addTransportService("intercepting", InterceptingTransportService.class);
} }
@Override
public Settings additionalSettings() {
return Settings.builder().put(TransportModule.TRANSPORT_SERVICE_TYPE_KEY, "intercepting").build();
}
} }
private final Set<String> actions = new HashSet<>(); private final Set<String> actions = new HashSet<>();
@ -888,6 +891,11 @@ public class IndicesRequestIT extends ESIntegTestCase {
super.registerRequestHandler(action, request, executor, forceExecution, new InterceptingRequestHandler(action, handler)); super.registerRequestHandler(action, request, executor, forceExecution, new InterceptingRequestHandler(action, handler));
} }
@Override
public <Request extends TransportRequest> void registerRequestHandler(String action, Callable<Request> requestFactory, String executor, TransportRequestHandler<Request> handler) {
super.registerRequestHandler(action, requestFactory, executor, new InterceptingRequestHandler(action, handler));
}
private class InterceptingRequestHandler implements TransportRequestHandler { private class InterceptingRequestHandler implements TransportRequestHandler {
private final TransportRequestHandler requestHandler; private final TransportRequestHandler requestHandler;

View File

@ -0,0 +1,422 @@
/*
* 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.action.support.broadcast.node;
import org.elasticsearch.ElasticsearchException;
import org.elasticsearch.Version;
import org.elasticsearch.action.IndicesRequest;
import org.elasticsearch.action.ShardOperationFailedException;
import org.elasticsearch.action.support.ActionFilter;
import org.elasticsearch.action.support.ActionFilters;
import org.elasticsearch.action.support.PlainActionFuture;
import org.elasticsearch.action.support.broadcast.BroadcastRequest;
import org.elasticsearch.action.support.broadcast.BroadcastResponse;
import org.elasticsearch.action.support.broadcast.BroadcastShardOperationFailedException;
import org.elasticsearch.cluster.ClusterName;
import org.elasticsearch.cluster.ClusterState;
import org.elasticsearch.cluster.block.ClusterBlock;
import org.elasticsearch.cluster.block.ClusterBlockException;
import org.elasticsearch.cluster.block.ClusterBlockLevel;
import org.elasticsearch.cluster.block.ClusterBlocks;
import org.elasticsearch.cluster.metadata.IndexNameExpressionResolver;
import org.elasticsearch.cluster.node.DiscoveryNode;
import org.elasticsearch.cluster.node.DiscoveryNodes;
import org.elasticsearch.cluster.routing.IndexRoutingTable;
import org.elasticsearch.cluster.routing.IndexShardRoutingTable;
import org.elasticsearch.cluster.routing.RoutingTable;
import org.elasticsearch.cluster.routing.ShardRouting;
import org.elasticsearch.cluster.routing.ShardRoutingState;
import org.elasticsearch.cluster.routing.ShardsIterator;
import org.elasticsearch.cluster.routing.TestShardRouting;
import org.elasticsearch.common.io.stream.StreamInput;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.transport.DummyTransportAddress;
import org.elasticsearch.index.shard.ShardId;
import org.elasticsearch.rest.RestStatus;
import org.elasticsearch.test.ESTestCase;
import org.elasticsearch.test.cluster.TestClusterService;
import org.elasticsearch.test.transport.CapturingTransport;
import org.elasticsearch.threadpool.ThreadPool;
import org.elasticsearch.transport.TransportChannel;
import org.elasticsearch.transport.TransportResponse;
import org.elasticsearch.transport.TransportResponseOptions;
import org.elasticsearch.transport.TransportService;
import org.junit.AfterClass;
import org.junit.Before;
import org.junit.BeforeClass;
import java.io.IOException;
import java.util.ArrayList;
import java.util.HashMap;
import java.util.HashSet;
import java.util.List;
import java.util.Map;
import java.util.Set;
import java.util.concurrent.ExecutionException;
import java.util.concurrent.TimeUnit;
import static org.hamcrest.CoreMatchers.containsString;
import static org.hamcrest.CoreMatchers.is;
import static org.hamcrest.object.HasToString.hasToString;
public class TransportBroadcastByNodeActionTests extends ESTestCase {
private static final String TEST_INDEX = "test-index";
private static final String TEST_CLUSTER = "test-cluster";
private static ThreadPool THREAD_POOL;
private TestClusterService clusterService;
private CapturingTransport transport;
private TransportService transportService;
private TestTransportBroadcastByNodeAction action;
public static class Request extends BroadcastRequest<Request> {
public Request() {
}
public Request(String[] indices) {
super(indices);
}
}
public static class Response extends BroadcastResponse {
public Response() {
}
public Response(int totalShards, int successfulShards, int failedShards, List<ShardOperationFailedException> shardFailures) {
super(totalShards, successfulShards, failedShards, shardFailures);
}
}
class TestTransportBroadcastByNodeAction extends TransportBroadcastByNodeAction<Request, Response, TransportBroadcastByNodeAction.EmptyResult> {
private final Map<ShardRouting, Object> shards = new HashMap<>();
public TestTransportBroadcastByNodeAction(Settings settings, TransportService transportService, ActionFilters actionFilters, IndexNameExpressionResolver indexNameExpressionResolver, Class<Request> request, String executor) {
super(settings, "indices:admin/test", THREAD_POOL, TransportBroadcastByNodeActionTests.this.clusterService, transportService, actionFilters, indexNameExpressionResolver, request, executor);
}
@Override
protected EmptyResult readShardResult(StreamInput in) throws IOException {
return EmptyResult.readEmptyResultFrom(in);
}
@Override
protected Response newResponse(Request request, int totalShards, int successfulShards, int failedShards, List<EmptyResult> emptyResults, List<ShardOperationFailedException> shardFailures, ClusterState clusterState) {
return new Response(totalShards, successfulShards, failedShards, shardFailures);
}
@Override
protected Request readRequestFrom(StreamInput in) throws IOException {
final Request request = new Request();
request.readFrom(in);
return request;
}
@Override
protected EmptyResult shardOperation(Request request, ShardRouting shardRouting) {
if (rarely()) {
shards.put(shardRouting, Boolean.TRUE);
return EmptyResult.INSTANCE;
} else {
ElasticsearchException e = new ElasticsearchException("operation failed");
shards.put(shardRouting, e);
throw e;
}
}
@Override
protected ShardsIterator shards(ClusterState clusterState, Request request, String[] concreteIndices) {
return clusterState.routingTable().allShards(new String[]{TEST_INDEX});
}
@Override
protected ClusterBlockException checkGlobalBlock(ClusterState state, Request request) {
return state.blocks().globalBlockedException(ClusterBlockLevel.METADATA_WRITE);
}
@Override
protected ClusterBlockException checkRequestBlock(ClusterState state, Request request, String[] concreteIndices) {
return state.blocks().indicesBlockedException(ClusterBlockLevel.METADATA_WRITE, concreteIndices);
}
public Map<ShardRouting, Object> getResults() {
return shards;
}
}
class MyResolver extends IndexNameExpressionResolver {
public MyResolver() {
super(Settings.EMPTY);
}
@Override
public String[] concreteIndices(ClusterState state, IndicesRequest request) {
return request.indices();
}
}
@BeforeClass
public static void startThreadPool() {
THREAD_POOL = new ThreadPool(TransportBroadcastByNodeActionTests.class.getSimpleName());
}
@Before
public void setUp() throws Exception {
super.setUp();
transport = new CapturingTransport();
clusterService = new TestClusterService(THREAD_POOL);
transportService = new TransportService(transport, THREAD_POOL);
transportService.start();
setClusterState(clusterService, TEST_INDEX);
action = new TestTransportBroadcastByNodeAction(
Settings.EMPTY,
transportService,
new ActionFilters(new HashSet<ActionFilter>()),
new MyResolver(),
Request.class,
ThreadPool.Names.SAME
);
}
void setClusterState(TestClusterService clusterService, String index) {
int numberOfNodes = randomIntBetween(3, 5);
DiscoveryNodes.Builder discoBuilder = DiscoveryNodes.builder();
IndexRoutingTable.Builder indexRoutingTable = IndexRoutingTable.builder(index);
int shardIndex = -1;
for (int i = 0; i < numberOfNodes; i++) {
final DiscoveryNode node = newNode(i);
discoBuilder = discoBuilder.put(node);
int numberOfShards = randomIntBetween(0, 10);
for (int j = 0; j < numberOfShards; j++) {
final ShardId shardId = new ShardId(index, ++shardIndex);
ShardRouting shard = TestShardRouting.newShardRouting(index, shardId.getId(), node.id(), true, ShardRoutingState.STARTED, 1);
IndexShardRoutingTable.Builder indexShard = new IndexShardRoutingTable.Builder(shardId);
indexShard.addShard(shard);
indexRoutingTable.addIndexShard(indexShard.build());
}
}
discoBuilder.localNodeId(newNode(0).id());
discoBuilder.masterNodeId(newNode(numberOfNodes - 1).id());
ClusterState.Builder stateBuilder = ClusterState.builder(new ClusterName(TEST_CLUSTER));
stateBuilder.nodes(discoBuilder);
stateBuilder.routingTable(RoutingTable.builder().add(indexRoutingTable.build()).build());
ClusterState clusterState = stateBuilder.build();
clusterService.setState(clusterState);
}
static DiscoveryNode newNode(int nodeId) {
return new DiscoveryNode("node_" + nodeId, DummyTransportAddress.INSTANCE, Version.CURRENT);
}
@AfterClass
public static void destroyThreadPool() {
ThreadPool.terminate(THREAD_POOL, 30, TimeUnit.SECONDS);
// since static must set to null to be eligible for collection
THREAD_POOL = null;
}
public void testGlobalBlock() {
Request request = new Request(new String[]{TEST_INDEX});
PlainActionFuture<Response> listener = new PlainActionFuture<>();
ClusterBlocks.Builder block = ClusterBlocks.builder()
.addGlobalBlock(new ClusterBlock(1, "", false, true, RestStatus.SERVICE_UNAVAILABLE, ClusterBlockLevel.ALL));
clusterService.setState(ClusterState.builder(clusterService.state()).blocks(block));
try {
action.new AsyncAction(request, listener).start();
fail("expected ClusterBlockException");
} catch (ClusterBlockException expected) {
}
}
public void testRequestBlock() {
Request request = new Request(new String[]{TEST_INDEX});
PlainActionFuture<Response> listener = new PlainActionFuture<>();
ClusterBlocks.Builder block = ClusterBlocks.builder()
.addIndexBlock(TEST_INDEX, new ClusterBlock(1, "test-block", false, true, RestStatus.SERVICE_UNAVAILABLE, ClusterBlockLevel.ALL));
clusterService.setState(ClusterState.builder(clusterService.state()).blocks(block));
try {
action.new AsyncAction(request, listener).start();
fail("expected ClusterBlockException");
} catch (ClusterBlockException expected) {
}
}
public void testOneRequestIsSentToEachNodeHoldingAShard() {
Request request = new Request(new String[]{TEST_INDEX});
PlainActionFuture<Response> listener = new PlainActionFuture<>();
action.new AsyncAction(request, listener).start();
Map<String, List<CapturingTransport.CapturedRequest>> capturedRequests = transport.capturedRequestsByTargetNode();
ShardsIterator shardIt = clusterService.state().routingTable().allShards(new String[]{TEST_INDEX});
Set<String> set = new HashSet<>();
for (ShardRouting shard : shardIt.asUnordered()) {
set.add(shard.currentNodeId());
}
// check a request was sent to the right number of nodes
assertEquals(set.size(), capturedRequests.size());
// check requests were sent to the right nodes
assertEquals(set, capturedRequests.keySet());
for (Map.Entry<String, List<CapturingTransport.CapturedRequest>> entry : capturedRequests.entrySet()) {
// check one request was sent to each node
assertEquals(1, entry.getValue().size());
}
}
public void testOperationExecution() throws Exception {
ShardsIterator shardIt = clusterService.state().routingTable().allShards(new String[]{TEST_INDEX});
Set<ShardRouting> shards = new HashSet<>();
String nodeId = shardIt.asUnordered().iterator().next().currentNodeId();
for (ShardRouting shard : shardIt.asUnordered()) {
if (nodeId.equals(shard.currentNodeId())) {
shards.add(shard);
}
}
final TransportBroadcastByNodeAction.BroadcastByNodeTransportRequestHandler handler =
action.new BroadcastByNodeTransportRequestHandler();
TestTransportChannel channel = new TestTransportChannel();
handler.messageReceived(action.new NodeRequest(nodeId, new Request(), new ArrayList<>(shards)), channel);
// check the operation was executed only on the expected shards
assertEquals(shards, action.getResults().keySet());
TransportResponse response = channel.getCapturedResponse();
assertTrue(response instanceof TransportBroadcastByNodeAction.NodeResponse);
TransportBroadcastByNodeAction.NodeResponse nodeResponse = (TransportBroadcastByNodeAction.NodeResponse)response;
// check the operation was executed on the correct node
assertEquals("node id", nodeId, nodeResponse.getNodeId());
int successfulShards = 0;
int failedShards = 0;
for (Object result : action.getResults().values()) {
if (!(result instanceof ElasticsearchException)) {
successfulShards++;
} else {
failedShards++;
}
}
// check the operation results
assertEquals("successful shards", successfulShards, nodeResponse.getSuccessfulShards());
assertEquals("total shards", action.getResults().size(), nodeResponse.getTotalShards());
assertEquals("failed shards", failedShards, nodeResponse.getExceptions().size());
List<BroadcastShardOperationFailedException> exceptions = nodeResponse.getExceptions();
for (BroadcastShardOperationFailedException exception : exceptions) {
assertThat(exception.getMessage(), is("operation indices:admin/test failed"));
assertThat(exception, hasToString(containsString("operation failed")));
}
}
public void testResultAggregation() throws ExecutionException, InterruptedException {
Request request = new Request(new String[]{TEST_INDEX});
PlainActionFuture<Response> listener = new PlainActionFuture<>();
action.new AsyncAction(request, listener).start();
Map<String, List<CapturingTransport.CapturedRequest>> capturedRequests = transport.capturedRequestsByTargetNode();
transport.clear();
ShardsIterator shardIt = clusterService.state().getRoutingTable().allShards(new String[]{TEST_INDEX});
Map<String, List<ShardRouting>> map = new HashMap<>();
for (ShardRouting shard : shardIt.asUnordered()) {
if (!map.containsKey(shard.currentNodeId())) {
map.put(shard.currentNodeId(), new ArrayList<ShardRouting>());
}
map.get(shard.currentNodeId()).add(shard);
}
int totalShards = 0;
int totalSuccessfulShards = 0;
int totalFailedShards = 0;
for (Map.Entry<String, List<CapturingTransport.CapturedRequest>> entry : capturedRequests.entrySet()) {
List<BroadcastShardOperationFailedException> exceptions = new ArrayList<>();
long requestId = entry.getValue().get(0).requestId;
if (rarely()) {
// simulate node failure
totalShards += map.get(entry.getKey()).size();
totalFailedShards += map.get(entry.getKey()).size();
transport.handleResponse(requestId, new Exception());
} else {
List<ShardRouting> shards = map.get(entry.getKey());
List<TransportBroadcastByNodeAction.EmptyResult> shardResults = new ArrayList<>();
for (ShardRouting shard : shards) {
totalShards++;
if (rarely()) {
// simulate operation failure
totalFailedShards++;
exceptions.add(new BroadcastShardOperationFailedException(shard.shardId(), "operation indices:admin/test failed"));
} else {
shardResults.add(TransportBroadcastByNodeAction.EmptyResult.INSTANCE);
}
}
totalSuccessfulShards += shardResults.size();
TransportBroadcastByNodeAction.NodeResponse nodeResponse = action.new NodeResponse(entry.getKey(), shards.size(), shardResults, exceptions);
transport.handleResponse(requestId, nodeResponse);
}
}
Response response = listener.get();
assertEquals("total shards", totalShards, response.getTotalShards());
assertEquals("successful shards", totalSuccessfulShards, response.getSuccessfulShards());
assertEquals("failed shards", totalFailedShards, response.getFailedShards());
assertEquals("accumulated exceptions", totalFailedShards, response.getShardFailures().length);
}
public class TestTransportChannel implements TransportChannel {
private TransportResponse capturedResponse;
public TransportResponse getCapturedResponse() {
return capturedResponse;
}
@Override
public String action() {
return null;
}
@Override
public String getProfileName() {
return "";
}
@Override
public void sendResponse(TransportResponse response) throws IOException {
capturedResponse = response;
}
@Override
public void sendResponse(TransportResponse response, TransportResponseOptions options) throws IOException {
}
@Override
public void sendResponse(Throwable error) throws IOException {
}
}
}

View File

@ -19,7 +19,6 @@
package org.elasticsearch.benchmark.recovery; package org.elasticsearch.benchmark.recovery;
import org.elasticsearch.action.admin.indices.recovery.RecoveryResponse; import org.elasticsearch.action.admin.indices.recovery.RecoveryResponse;
import org.elasticsearch.action.admin.indices.recovery.ShardRecoveryResponse;
import org.elasticsearch.bootstrap.BootstrapForTesting; import org.elasticsearch.bootstrap.BootstrapForTesting;
import org.elasticsearch.client.Client; import org.elasticsearch.client.Client;
import org.elasticsearch.cluster.metadata.IndexMetaData; import org.elasticsearch.cluster.metadata.IndexMetaData;
@ -30,6 +29,7 @@ import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.unit.SizeValue; import org.elasticsearch.common.unit.SizeValue;
import org.elasticsearch.common.unit.TimeValue; import org.elasticsearch.common.unit.TimeValue;
import org.elasticsearch.index.IndexNotFoundException; import org.elasticsearch.index.IndexNotFoundException;
import org.elasticsearch.indices.recovery.RecoveryState;
import org.elasticsearch.node.Node; import org.elasticsearch.node.Node;
import org.elasticsearch.test.BackgroundIndexer; import org.elasticsearch.test.BackgroundIndexer;
import org.elasticsearch.transport.TransportModule; import org.elasticsearch.transport.TransportModule;
@ -128,12 +128,12 @@ public class ReplicaRecoveryBenchmark {
long currentTime = System.currentTimeMillis(); long currentTime = System.currentTimeMillis();
long currentDocs = indexer.totalIndexedDocs(); long currentDocs = indexer.totalIndexedDocs();
RecoveryResponse recoveryResponse = client1.admin().indices().prepareRecoveries(INDEX_NAME).setActiveOnly(true).get(); RecoveryResponse recoveryResponse = client1.admin().indices().prepareRecoveries(INDEX_NAME).setActiveOnly(true).get();
List<ShardRecoveryResponse> indexRecoveries = recoveryResponse.shardResponses().get(INDEX_NAME); List<RecoveryState> indexRecoveries = recoveryResponse.shardRecoveryStates().get(INDEX_NAME);
long translogOps; long translogOps;
long bytes; long bytes;
if (indexRecoveries.size() > 0) { if (indexRecoveries.size() > 0) {
translogOps = indexRecoveries.get(0).recoveryState().getTranslog().recoveredOperations(); translogOps = indexRecoveries.get(0).getTranslog().recoveredOperations();
bytes = recoveryResponse.shardResponses().get(INDEX_NAME).get(0).recoveryState().getIndex().recoveredBytes(); bytes = recoveryResponse.shardRecoveryStates().get(INDEX_NAME).get(0).getIndex().recoveredBytes();
} else { } else {
bytes = lastBytes = 0; bytes = lastBytes = 0;
translogOps = lastTranslogOps = 0; translogOps = lastTranslogOps = 0;

View File

@ -20,6 +20,7 @@
package org.elasticsearch.benchmark.scripts.expression; package org.elasticsearch.benchmark.scripts.expression;
import org.elasticsearch.ElasticsearchException; import org.elasticsearch.ElasticsearchException;
import org.elasticsearch.Version;
import org.elasticsearch.action.bulk.BulkRequestBuilder; import org.elasticsearch.action.bulk.BulkRequestBuilder;
import org.elasticsearch.action.search.SearchRequestBuilder; import org.elasticsearch.action.search.SearchRequestBuilder;
import org.elasticsearch.client.Client; import org.elasticsearch.client.Client;
@ -28,13 +29,18 @@ import org.elasticsearch.common.StopWatch;
import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.unit.TimeValue; import org.elasticsearch.common.unit.TimeValue;
import org.elasticsearch.index.query.QueryBuilders; import org.elasticsearch.index.query.QueryBuilders;
import org.elasticsearch.node.MockNode;
import org.elasticsearch.node.Node; import org.elasticsearch.node.Node;
import org.elasticsearch.plugins.Plugin;
import org.elasticsearch.script.Script; import org.elasticsearch.script.Script;
import org.elasticsearch.script.ScriptService.ScriptType; import org.elasticsearch.script.ScriptService.ScriptType;
import org.elasticsearch.search.sort.ScriptSortBuilder; import org.elasticsearch.search.sort.ScriptSortBuilder;
import org.elasticsearch.search.sort.SortBuilders; import org.elasticsearch.search.sort.SortBuilders;
import org.joda.time.PeriodType; import org.joda.time.PeriodType;
import java.util.Arrays;
import java.util.Collection;
import java.util.Collections;
import java.util.Random; import java.util.Random;
import static org.elasticsearch.common.settings.Settings.settingsBuilder; import static org.elasticsearch.common.settings.Settings.settingsBuilder;
@ -102,10 +108,11 @@ public class ScriptComparisonBenchmark {
static Client setupIndex() throws Exception { static Client setupIndex() throws Exception {
// create cluster // create cluster
Settings settings = settingsBuilder().put("plugin.types", NativeScriptPlugin.class.getName()) Settings settings = settingsBuilder().put("name", "node1")
.put("name", "node1") .put("cluster.name", clusterName).build();
.build(); Collection<Class<? extends Plugin>> plugins = Collections.<Class<? extends Plugin>>singletonList(NativeScriptPlugin.class);
Node node1 = nodeBuilder().clusterName(clusterName).settings(settings).node(); Node node1 = new MockNode(settings, true, Version.CURRENT, plugins);
node1.start();
Client client = node1.client(); Client client = node1.client();
client.admin().cluster().prepareHealth(indexName).setWaitForGreenStatus().setTimeout("10s").execute().actionGet(); client.admin().cluster().prepareHealth(indexName).setWaitForGreenStatus().setTimeout("10s").execute().actionGet();

View File

@ -18,14 +18,19 @@
*/ */
package org.elasticsearch.benchmark.scripts.score; package org.elasticsearch.benchmark.scripts.score;
import org.elasticsearch.Version;
import org.elasticsearch.benchmark.scripts.score.plugin.NativeScriptExamplesPlugin; import org.elasticsearch.benchmark.scripts.score.plugin.NativeScriptExamplesPlugin;
import org.elasticsearch.benchmark.scripts.score.script.NativeConstantForLoopScoreScript; import org.elasticsearch.benchmark.scripts.score.script.NativeConstantForLoopScoreScript;
import org.elasticsearch.benchmark.scripts.score.script.NativeConstantScoreScript; import org.elasticsearch.benchmark.scripts.score.script.NativeConstantScoreScript;
import org.elasticsearch.client.Client; import org.elasticsearch.client.Client;
import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.node.MockNode;
import org.elasticsearch.node.Node; import org.elasticsearch.node.Node;
import org.elasticsearch.plugins.Plugin;
import java.util.ArrayList; import java.util.ArrayList;
import java.util.Collection;
import java.util.Collections;
import java.util.List; import java.util.List;
import java.util.Map.Entry; import java.util.Map.Entry;
@ -46,10 +51,13 @@ public class ScriptsConstantScoreBenchmark extends BasicScriptBenchmark {
init(maxTerms); init(maxTerms);
List<Results> allResults = new ArrayList<>(); List<Results> allResults = new ArrayList<>();
Settings settings = settingsBuilder().put("plugin.types", NativeScriptExamplesPlugin.class.getName()).build();
String clusterName = ScriptsConstantScoreBenchmark.class.getSimpleName(); String clusterName = ScriptsConstantScoreBenchmark.class.getSimpleName();
Node node1 = nodeBuilder().clusterName(clusterName).settings(settingsBuilder().put(settings).put("name", "node1")).node(); Settings settings = settingsBuilder().put("name", "node1")
.put("cluster.name", clusterName).build();
Collection<Class<? extends Plugin>> plugins = Collections.<Class<? extends Plugin>>singletonList(NativeScriptExamplesPlugin.class);
Node node1 = new MockNode(settings, true, Version.CURRENT, plugins);
node1.start();
Client client = node1.client(); Client client = node1.client();
client.admin().cluster().prepareHealth("test").setWaitForGreenStatus().setTimeout("10s").execute().actionGet(); client.admin().cluster().prepareHealth("test").setWaitForGreenStatus().setTimeout("10s").execute().actionGet();

View File

@ -18,13 +18,18 @@
*/ */
package org.elasticsearch.benchmark.scripts.score; package org.elasticsearch.benchmark.scripts.score;
import org.elasticsearch.Version;
import org.elasticsearch.benchmark.scripts.score.plugin.NativeScriptExamplesPlugin; import org.elasticsearch.benchmark.scripts.score.plugin.NativeScriptExamplesPlugin;
import org.elasticsearch.benchmark.scripts.score.script.NativeNaiveTFIDFScoreScript; import org.elasticsearch.benchmark.scripts.score.script.NativeNaiveTFIDFScoreScript;
import org.elasticsearch.client.Client; import org.elasticsearch.client.Client;
import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.node.MockNode;
import org.elasticsearch.node.Node; import org.elasticsearch.node.Node;
import org.elasticsearch.plugins.Plugin;
import java.util.ArrayList; import java.util.ArrayList;
import java.util.Collection;
import java.util.Collections;
import java.util.List; import java.util.List;
import java.util.Map.Entry; import java.util.Map.Entry;
@ -46,10 +51,12 @@ public class ScriptsScoreBenchmark extends BasicScriptBenchmark {
boolean runMVEL = false; boolean runMVEL = false;
init(maxTerms); init(maxTerms);
List<Results> allResults = new ArrayList<>(); List<Results> allResults = new ArrayList<>();
Settings settings = settingsBuilder().put("plugin.types", NativeScriptExamplesPlugin.class.getName()).build();
String clusterName = ScriptsScoreBenchmark.class.getSimpleName(); String clusterName = ScriptsScoreBenchmark.class.getSimpleName();
Node node1 = nodeBuilder().clusterName(clusterName).settings(settingsBuilder().put(settings).put("name", "node1")).node(); Settings settings = settingsBuilder().put("name", "node1")
.put("cluster.name", clusterName).build();
Collection<Class<? extends Plugin>> plugins = Collections.<Class<? extends Plugin>>singletonList(NativeScriptExamplesPlugin.class);
Node node1 = new MockNode(settings, true, Version.CURRENT, plugins);
node1.start();
Client client = node1.client(); Client client = node1.client();
client.admin().cluster().prepareHealth("test").setWaitForGreenStatus().setTimeout("10s").execute().actionGet(); client.admin().cluster().prepareHealth("test").setWaitForGreenStatus().setTimeout("10s").execute().actionGet();

View File

@ -18,14 +18,19 @@
*/ */
package org.elasticsearch.benchmark.scripts.score; package org.elasticsearch.benchmark.scripts.score;
import org.elasticsearch.Version;
import org.elasticsearch.benchmark.scripts.score.plugin.NativeScriptExamplesPlugin; import org.elasticsearch.benchmark.scripts.score.plugin.NativeScriptExamplesPlugin;
import org.elasticsearch.benchmark.scripts.score.script.NativePayloadSumNoRecordScoreScript; import org.elasticsearch.benchmark.scripts.score.script.NativePayloadSumNoRecordScoreScript;
import org.elasticsearch.benchmark.scripts.score.script.NativePayloadSumScoreScript; import org.elasticsearch.benchmark.scripts.score.script.NativePayloadSumScoreScript;
import org.elasticsearch.client.Client; import org.elasticsearch.client.Client;
import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.node.MockNode;
import org.elasticsearch.node.Node; import org.elasticsearch.node.Node;
import org.elasticsearch.plugins.Plugin;
import java.util.ArrayList; import java.util.ArrayList;
import java.util.Collection;
import java.util.Collections;
import java.util.List; import java.util.List;
import java.util.Map.Entry; import java.util.Map.Entry;
@ -46,10 +51,12 @@ public class ScriptsScorePayloadSumBenchmark extends BasicScriptBenchmark {
init(maxTerms); init(maxTerms);
List<Results> allResults = new ArrayList<>(); List<Results> allResults = new ArrayList<>();
Settings settings = settingsBuilder().put("plugin.types", NativeScriptExamplesPlugin.class.getName()).build();
String clusterName = ScriptsScoreBenchmark.class.getSimpleName(); String clusterName = ScriptsScoreBenchmark.class.getSimpleName();
Node node1 = nodeBuilder().clusterName(clusterName).settings(settingsBuilder().put(settings).put("name", "node1")).node(); Settings settings = settingsBuilder().put("name", "node1")
.put("cluster.name", clusterName).build();
Collection<Class<? extends Plugin>> plugins = Collections.<Class<? extends Plugin>>singletonList(NativeScriptExamplesPlugin.class);
Node node1 = new MockNode(settings, true, Version.CURRENT, plugins);
node1.start();
Client client = node1.client(); Client client = node1.client();
client.admin().cluster().prepareHealth("test").setWaitForGreenStatus().setTimeout("10s").execute().actionGet(); client.admin().cluster().prepareHealth("test").setWaitForGreenStatus().setTimeout("10s").execute().actionGet();

View File

@ -48,6 +48,7 @@ import org.elasticsearch.transport.TransportResponseHandler;
import org.elasticsearch.transport.TransportService; import org.elasticsearch.transport.TransportService;
import org.junit.Test; import org.junit.Test;
import java.util.Collection;
import java.util.concurrent.CountDownLatch; import java.util.concurrent.CountDownLatch;
import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeUnit;
@ -63,12 +64,13 @@ public class TransportClientHeadersTests extends AbstractClientHeadersTests {
@Override @Override
protected Client buildClient(Settings headersSettings, GenericAction[] testedActions) { protected Client buildClient(Settings headersSettings, GenericAction[] testedActions) {
TransportClient client = TransportClient.builder().settings(Settings.builder() TransportClient client = TransportClient.builder()
.settings(Settings.builder()
.put("client.transport.sniff", false) .put("client.transport.sniff", false)
.put("node.name", "transport_client_" + this.getTestName()) .put("node.name", "transport_client_" + this.getTestName())
.put("plugin.types", InternalTransportService.TestPlugin.class.getName())
.put(headersSettings) .put(headersSettings)
.build()).build(); .build())
.addPlugin(InternalTransportService.TestPlugin.class).build();
client.addTransportAddress(address); client.addTransportAddress(address);
return client; return client;
@ -76,15 +78,17 @@ public class TransportClientHeadersTests extends AbstractClientHeadersTests {
@Test @Test
public void testWithSniffing() throws Exception { public void testWithSniffing() throws Exception {
TransportClient client = TransportClient.builder().settings(Settings.builder() TransportClient client = TransportClient.builder()
.settings(Settings.builder()
.put("client.transport.sniff", true) .put("client.transport.sniff", true)
.put("cluster.name", "cluster1") .put("cluster.name", "cluster1")
.put("node.name", "transport_client_" + this.getTestName() + "_1") .put("node.name", "transport_client_" + this.getTestName() + "_1")
.put("client.transport.nodes_sampler_interval", "1s") .put("client.transport.nodes_sampler_interval", "1s")
.put("plugin.types", InternalTransportService.TestPlugin.class.getName())
.put(HEADER_SETTINGS) .put(HEADER_SETTINGS)
.put("path.home", createTempDir().toString()) .put("path.home", createTempDir().toString()).build())
.build()).build(); .addPlugin(InternalTransportService.TestPlugin.class)
.build();
try { try {
client.addTransportAddress(address); client.addTransportAddress(address);

View File

@ -31,10 +31,16 @@ import org.elasticsearch.action.support.ActionFilter;
import org.elasticsearch.action.support.ActionFilters; import org.elasticsearch.action.support.ActionFilters;
import org.elasticsearch.cluster.metadata.IndexMetaData; import org.elasticsearch.cluster.metadata.IndexMetaData;
import org.elasticsearch.cluster.node.DiscoveryNode; import org.elasticsearch.cluster.node.DiscoveryNode;
import org.elasticsearch.cluster.routing.RoutingNodes;
import org.elasticsearch.cluster.routing.ShardRouting;
import org.elasticsearch.cluster.routing.allocation.decider.EnableAllocationDecider;
import org.elasticsearch.common.Strings; import org.elasticsearch.common.Strings;
import org.elasticsearch.common.inject.Inject; import org.elasticsearch.common.inject.Inject;
import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.index.IndexService;
import org.elasticsearch.index.shard.IndexShard;
import org.elasticsearch.index.store.Store; import org.elasticsearch.index.store.Store;
import org.elasticsearch.indices.IndicesService;
import org.elasticsearch.plugins.Plugin; import org.elasticsearch.plugins.Plugin;
import org.elasticsearch.test.ESIntegTestCase; import org.elasticsearch.test.ESIntegTestCase;
import org.elasticsearch.test.InternalTestCluster; import org.elasticsearch.test.InternalTestCluster;
@ -47,6 +53,7 @@ import org.hamcrest.Matchers;
import org.junit.Test; import org.junit.Test;
import java.io.IOException; import java.io.IOException;
import java.util.Collection;
import java.util.Map; import java.util.Map;
import java.util.Set; import java.util.Set;
import java.util.concurrent.CountDownLatch; import java.util.concurrent.CountDownLatch;
@ -146,16 +153,23 @@ public class ClusterInfoServiceIT extends ESIntegTestCase {
return Settings.builder() return Settings.builder()
// manual collection or upon cluster forming. // manual collection or upon cluster forming.
.put(InternalClusterInfoService.INTERNAL_CLUSTER_INFO_TIMEOUT, "1s") .put(InternalClusterInfoService.INTERNAL_CLUSTER_INFO_TIMEOUT, "1s")
.putArray("plugin.types", TestPlugin.class.getName(), MockTransportService.TestPlugin.class.getName())
.build(); .build();
} }
@Override
protected Collection<Class<? extends Plugin>> nodePlugins() {
return pluginList(TestPlugin.class,
MockTransportService.TestPlugin.class);
}
@Test @Test
public void testClusterInfoServiceCollectsInformation() throws Exception { public void testClusterInfoServiceCollectsInformation() throws Exception {
internalCluster().startNodesAsync(2, internalCluster().startNodesAsync(2,
Settings.builder().put(InternalClusterInfoService.INTERNAL_CLUSTER_INFO_UPDATE_INTERVAL, "200ms").build()) Settings.builder().put(InternalClusterInfoService.INTERNAL_CLUSTER_INFO_UPDATE_INTERVAL, "200ms").build())
.get(); .get();
assertAcked(prepareCreate("test").setSettings(settingsBuilder().put(Store.INDEX_STORE_STATS_REFRESH_INTERVAL, 0).build())); assertAcked(prepareCreate("test").setSettings(settingsBuilder()
.put(Store.INDEX_STORE_STATS_REFRESH_INTERVAL, 0)
.put(EnableAllocationDecider.INDEX_ROUTING_REBALANCE_ENABLE, EnableAllocationDecider.Rebalance.NONE).build()));
ensureGreen("test"); ensureGreen("test");
InternalTestCluster internalTestCluster = internalCluster(); InternalTestCluster internalTestCluster = internalCluster();
// Get the cluster info service on the master node // Get the cluster info service on the master node
@ -164,13 +178,18 @@ public class ClusterInfoServiceIT extends ESIntegTestCase {
infoService.addListener(listener); infoService.addListener(listener);
ClusterInfo info = listener.get(); ClusterInfo info = listener.get();
assertNotNull("info should not be null", info); assertNotNull("info should not be null", info);
Map<String, DiskUsage> usages = info.getNodeDiskUsages(); final Map<String, DiskUsage> leastUsages = info.getNodeLeastAvailableDiskUsages();
Map<String, Long> shardSizes = info.shardSizes; final Map<String, DiskUsage> mostUsages = info.getNodeMostAvailableDiskUsages();
assertNotNull(usages); final Map<String, Long> shardSizes = info.shardSizes;
assertNotNull(leastUsages);
assertNotNull(shardSizes); assertNotNull(shardSizes);
assertThat("some usages are populated", usages.values().size(), Matchers.equalTo(2)); assertThat("some usages are populated", leastUsages.values().size(), Matchers.equalTo(2));
assertThat("some shard sizes are populated", shardSizes.values().size(), greaterThan(0)); assertThat("some shard sizes are populated", shardSizes.values().size(), greaterThan(0));
for (DiskUsage usage : usages.values()) { for (DiskUsage usage : leastUsages.values()) {
logger.info("--> usage: {}", usage);
assertThat("usage has be retrieved", usage.getFreeBytes(), greaterThan(0L));
}
for (DiskUsage usage : mostUsages.values()) {
logger.info("--> usage: {}", usage); logger.info("--> usage: {}", usage);
assertThat("usage has be retrieved", usage.getFreeBytes(), greaterThan(0L)); assertThat("usage has be retrieved", usage.getFreeBytes(), greaterThan(0L));
} }
@ -178,6 +197,21 @@ public class ClusterInfoServiceIT extends ESIntegTestCase {
logger.info("--> shard size: {}", size); logger.info("--> shard size: {}", size);
assertThat("shard size is greater than 0", size, greaterThan(0L)); assertThat("shard size is greater than 0", size, greaterThan(0L));
} }
ClusterService clusterService = internalTestCluster.getInstance(ClusterService.class, internalTestCluster.getMasterName());
ClusterState state = clusterService.state();
RoutingNodes routingNodes = state.getRoutingNodes();
for (ShardRouting shard : routingNodes.getRoutingTable().allShards()) {
String dataPath = info.getDataPath(shard);
assertNotNull(dataPath);
String nodeId = shard.currentNodeId();
DiscoveryNode discoveryNode = state.getNodes().get(nodeId);
IndicesService indicesService = internalTestCluster.getInstance(IndicesService.class, discoveryNode.getName());
IndexService indexService = indicesService.indexService(shard.index());
IndexShard indexShard = indexService.shard(shard.id());
assertEquals(indexShard.shardPath().getRootDataPath().toString(), dataPath);
}
} }
@Test @Test
@ -197,14 +231,14 @@ public class ClusterInfoServiceIT extends ESIntegTestCase {
infoService.updateOnce(); infoService.updateOnce();
ClusterInfo info = listener.get(); ClusterInfo info = listener.get();
assertNotNull("failed to collect info", info); assertNotNull("failed to collect info", info);
assertThat("some usages are populated", info.getNodeDiskUsages().size(), Matchers.equalTo(2)); assertThat("some usages are populated", info.getNodeLeastAvailableDiskUsages().size(), Matchers.equalTo(2));
assertThat("some shard sizes are populated", info.shardSizes.size(), greaterThan(0)); assertThat("some shard sizes are populated", info.shardSizes.size(), greaterThan(0));
MockTransportService mockTransportService = (MockTransportService) internalCluster().getInstance(TransportService.class, internalTestCluster.getMasterName()); MockTransportService mockTransportService = (MockTransportService) internalCluster().getInstance(TransportService.class, internalTestCluster.getMasterName());
final AtomicBoolean timeout = new AtomicBoolean(false); final AtomicBoolean timeout = new AtomicBoolean(false);
final Set<String> blockedActions = ImmutableSet.of(NodesStatsAction.NAME, NodesStatsAction.NAME + "[n]", IndicesStatsAction.NAME, IndicesStatsAction.NAME + "[s]"); final Set<String> blockedActions = ImmutableSet.of(NodesStatsAction.NAME, NodesStatsAction.NAME + "[n]", IndicesStatsAction.NAME, IndicesStatsAction.NAME + "[n]");
// drop all outgoing stats requests to force a timeout. // drop all outgoing stats requests to force a timeout.
for (DiscoveryNode node : internalTestCluster.clusterService().state().getNodes()) { for (DiscoveryNode node : internalTestCluster.clusterService().state().getNodes()) {
mockTransportService.addDelegate(node, new MockTransportService.DelegateTransport(mockTransportService.original()) { mockTransportService.addDelegate(node, new MockTransportService.DelegateTransport(mockTransportService.original()) {
@ -231,7 +265,8 @@ public class ClusterInfoServiceIT extends ESIntegTestCase {
// node info will time out both on the request level on the count down latch. this means // node info will time out both on the request level on the count down latch. this means
// it is likely to update the node disk usage based on the one response that came be from local // it is likely to update the node disk usage based on the one response that came be from local
// node. // node.
assertThat(info.getNodeDiskUsages().size(), greaterThanOrEqualTo(1)); assertThat(info.getNodeLeastAvailableDiskUsages().size(), greaterThanOrEqualTo(1));
assertThat(info.getNodeMostAvailableDiskUsages().size(), greaterThanOrEqualTo(1));
// indices is guaranteed to time out on the latch, not updating anything. // indices is guaranteed to time out on the latch, not updating anything.
assertThat(info.shardSizes.size(), greaterThan(1)); assertThat(info.shardSizes.size(), greaterThan(1));
@ -252,7 +287,8 @@ public class ClusterInfoServiceIT extends ESIntegTestCase {
infoService.updateOnce(); infoService.updateOnce();
info = listener.get(); info = listener.get();
assertNotNull("info should not be null", info); assertNotNull("info should not be null", info);
assertThat(info.getNodeDiskUsages().size(), equalTo(0)); assertThat(info.getNodeLeastAvailableDiskUsages().size(), equalTo(0));
assertThat(info.getNodeMostAvailableDiskUsages().size(), equalTo(0));
assertThat(info.shardSizes.size(), equalTo(0)); assertThat(info.shardSizes.size(), equalTo(0));
// check we recover // check we recover
@ -261,7 +297,8 @@ public class ClusterInfoServiceIT extends ESIntegTestCase {
infoService.updateOnce(); infoService.updateOnce();
info = listener.get(); info = listener.get();
assertNotNull("info should not be null", info); assertNotNull("info should not be null", info);
assertThat(info.getNodeDiskUsages().size(), equalTo(2)); assertThat(info.getNodeLeastAvailableDiskUsages().size(), equalTo(2));
assertThat(info.getNodeMostAvailableDiskUsages().size(), equalTo(2));
assertThat(info.shardSizes.size(), greaterThan(0)); assertThat(info.shardSizes.size(), greaterThan(0));
} }

View File

@ -69,6 +69,11 @@ import static org.hamcrest.Matchers.notNullValue;
@ESIntegTestCase.SuppressLocalMode @ESIntegTestCase.SuppressLocalMode
public class ClusterServiceIT extends ESIntegTestCase { public class ClusterServiceIT extends ESIntegTestCase {
@Override
protected Collection<Class<? extends Plugin>> nodePlugins() {
return pluginList(TestPlugin.class);
}
@Test @Test
public void testTimeoutUpdateTask() throws Exception { public void testTimeoutUpdateTask() throws Exception {
Settings settings = settingsBuilder() Settings settings = settingsBuilder()
@ -637,7 +642,6 @@ public class ClusterServiceIT extends ESIntegTestCase {
.put("discovery.zen.minimum_master_nodes", 1) .put("discovery.zen.minimum_master_nodes", 1)
.put("discovery.zen.ping_timeout", "400ms") .put("discovery.zen.ping_timeout", "400ms")
.put("discovery.initial_state_timeout", "500ms") .put("discovery.initial_state_timeout", "500ms")
.put("plugin.types", TestPlugin.class.getName())
.build(); .build();
String node_0 = internalCluster().startNode(settings); String node_0 = internalCluster().startNode(settings);

View File

@ -19,16 +19,33 @@
package org.elasticsearch.cluster; package org.elasticsearch.cluster;
import org.elasticsearch.Version;
import org.elasticsearch.action.admin.cluster.node.stats.NodeStats;
import org.elasticsearch.action.admin.cluster.node.stats.NodesStatsResponse;
import org.elasticsearch.action.admin.indices.stats.CommonStats;
import org.elasticsearch.action.admin.indices.stats.ShardStats;
import org.elasticsearch.cluster.node.DiscoveryNode;
import org.elasticsearch.cluster.routing.ShardRouting;
import org.elasticsearch.cluster.routing.ShardRoutingHelper;
import org.elasticsearch.cluster.routing.UnassignedInfo;
import org.elasticsearch.common.transport.DummyTransportAddress;
import org.elasticsearch.index.shard.ShardPath;
import org.elasticsearch.index.store.StoreStats;
import org.elasticsearch.monitor.fs.FsInfo;
import org.elasticsearch.test.ESTestCase; import org.elasticsearch.test.ESTestCase;
import org.junit.Test; import org.junit.Test;
import java.nio.file.Path;
import java.util.HashMap;
import java.util.Map;
import static org.hamcrest.Matchers.equalTo; import static org.hamcrest.Matchers.equalTo;
public class DiskUsageTests extends ESTestCase { public class DiskUsageTests extends ESTestCase {
@Test @Test
public void diskUsageCalcTest() { public void diskUsageCalcTest() {
DiskUsage du = new DiskUsage("node1", "n1", 100, 40); DiskUsage du = new DiskUsage("node1", "n1", "random", 100, 40);
assertThat(du.getFreeDiskAsPercentage(), equalTo(40.0)); assertThat(du.getFreeDiskAsPercentage(), equalTo(40.0));
assertThat(du.getUsedDiskAsPercentage(), equalTo(100.0 - 40.0)); assertThat(du.getUsedDiskAsPercentage(), equalTo(100.0 - 40.0));
assertThat(du.getFreeBytes(), equalTo(40L)); assertThat(du.getFreeBytes(), equalTo(40L));
@ -37,19 +54,19 @@ public class DiskUsageTests extends ESTestCase {
// Test that DiskUsage handles invalid numbers, as reported by some // Test that DiskUsage handles invalid numbers, as reported by some
// filesystems (ZFS & NTFS) // filesystems (ZFS & NTFS)
DiskUsage du2 = new DiskUsage("node1", "n1", 100, 101); DiskUsage du2 = new DiskUsage("node1", "n1","random", 100, 101);
assertThat(du2.getFreeDiskAsPercentage(), equalTo(101.0)); assertThat(du2.getFreeDiskAsPercentage(), equalTo(101.0));
assertThat(du2.getFreeBytes(), equalTo(101L)); assertThat(du2.getFreeBytes(), equalTo(101L));
assertThat(du2.getUsedBytes(), equalTo(-1L)); assertThat(du2.getUsedBytes(), equalTo(-1L));
assertThat(du2.getTotalBytes(), equalTo(100L)); assertThat(du2.getTotalBytes(), equalTo(100L));
DiskUsage du3 = new DiskUsage("node1", "n1", -1, -1); DiskUsage du3 = new DiskUsage("node1", "n1", "random",-1, -1);
assertThat(du3.getFreeDiskAsPercentage(), equalTo(100.0)); assertThat(du3.getFreeDiskAsPercentage(), equalTo(100.0));
assertThat(du3.getFreeBytes(), equalTo(-1L)); assertThat(du3.getFreeBytes(), equalTo(-1L));
assertThat(du3.getUsedBytes(), equalTo(0L)); assertThat(du3.getUsedBytes(), equalTo(0L));
assertThat(du3.getTotalBytes(), equalTo(-1L)); assertThat(du3.getTotalBytes(), equalTo(-1L));
DiskUsage du4 = new DiskUsage("node1", "n1", 0, 0); DiskUsage du4 = new DiskUsage("node1", "n1","random", 0, 0);
assertThat(du4.getFreeDiskAsPercentage(), equalTo(100.0)); assertThat(du4.getFreeDiskAsPercentage(), equalTo(100.0));
assertThat(du4.getFreeBytes(), equalTo(0L)); assertThat(du4.getFreeBytes(), equalTo(0L));
assertThat(du4.getUsedBytes(), equalTo(0L)); assertThat(du4.getUsedBytes(), equalTo(0L));
@ -62,7 +79,7 @@ public class DiskUsageTests extends ESTestCase {
for (int i = 1; i < iters; i++) { for (int i = 1; i < iters; i++) {
long total = between(Integer.MIN_VALUE, Integer.MAX_VALUE); long total = between(Integer.MIN_VALUE, Integer.MAX_VALUE);
long free = between(Integer.MIN_VALUE, Integer.MAX_VALUE); long free = between(Integer.MIN_VALUE, Integer.MAX_VALUE);
DiskUsage du = new DiskUsage("random", "random", total, free); DiskUsage du = new DiskUsage("random", "random", "random", total, free);
if (total == 0) { if (total == 0) {
assertThat(du.getFreeBytes(), equalTo(free)); assertThat(du.getFreeBytes(), equalTo(free));
assertThat(du.getTotalBytes(), equalTo(0L)); assertThat(du.getTotalBytes(), equalTo(0L));
@ -78,4 +95,85 @@ public class DiskUsageTests extends ESTestCase {
} }
} }
} }
public void testFillShardLevelInfo() {
ShardRouting test_0 = ShardRouting.newUnassigned("test", 0, null, false, new UnassignedInfo(UnassignedInfo.Reason.INDEX_CREATED, "foo"));
ShardRoutingHelper.initialize(test_0, "node1");
ShardRoutingHelper.moveToStarted(test_0);
Path test0Path = createTempDir().resolve("indices").resolve("test").resolve("0");
CommonStats commonStats0 = new CommonStats();
commonStats0.store = new StoreStats(100, 1);
ShardRouting test_1 = ShardRouting.newUnassigned("test", 1, null, false, new UnassignedInfo(UnassignedInfo.Reason.INDEX_CREATED, "foo"));
ShardRoutingHelper.initialize(test_1, "node2");
ShardRoutingHelper.moveToStarted(test_1);
Path test1Path = createTempDir().resolve("indices").resolve("test").resolve("1");
CommonStats commonStats1 = new CommonStats();
commonStats1.store = new StoreStats(1000, 1);
ShardStats[] stats = new ShardStats[] {
new ShardStats(test_0, new ShardPath(false, test0Path, test0Path, "0xdeadbeef", test_0.shardId()), commonStats0 , null),
new ShardStats(test_1, new ShardPath(false, test1Path, test1Path, "0xdeadbeef", test_1.shardId()), commonStats1 , null)
};
HashMap<String, Long> shardSizes = new HashMap<>();
HashMap<ShardRouting, String> routingToPath = new HashMap<>();
InternalClusterInfoService.buildShardLevelInfo(logger, stats, shardSizes, routingToPath);
assertEquals(2, shardSizes.size());
assertTrue(shardSizes.containsKey(ClusterInfo.shardIdentifierFromRouting(test_0)));
assertTrue(shardSizes.containsKey(ClusterInfo.shardIdentifierFromRouting(test_1)));
assertEquals(100l, shardSizes.get(ClusterInfo.shardIdentifierFromRouting(test_0)).longValue());
assertEquals(1000l, shardSizes.get(ClusterInfo.shardIdentifierFromRouting(test_1)).longValue());
assertEquals(2, routingToPath.size());
assertTrue(routingToPath.containsKey(test_0));
assertTrue(routingToPath.containsKey(test_1));
assertEquals(test0Path.getParent().getParent().getParent().toAbsolutePath().toString(), routingToPath.get(test_0));
assertEquals(test1Path.getParent().getParent().getParent().toAbsolutePath().toString(), routingToPath.get(test_1));
}
public void testFillDiskUsage() {
Map<String, DiskUsage> newLeastAvaiableUsages = new HashMap<>();
Map<String, DiskUsage> newMostAvaiableUsages = new HashMap<>();
FsInfo.Path[] node1FSInfo = new FsInfo.Path[] {
new FsInfo.Path("/middle", "/dev/sda", 100, 90, 80),
new FsInfo.Path("/least", "/dev/sdb", 200, 190, 70),
new FsInfo.Path("/most", "/dev/sdc", 300, 290, 280),
};
FsInfo.Path[] node2FSInfo = new FsInfo.Path[] {
new FsInfo.Path("/least_most", "/dev/sda", 100, 90, 80),
};
FsInfo.Path[] node3FSInfo = new FsInfo.Path[] {
new FsInfo.Path("/least", "/dev/sda", 100, 90, 70),
new FsInfo.Path("/most", "/dev/sda", 100, 90, 80),
};
NodeStats[] nodeStats = new NodeStats[] {
new NodeStats(new DiscoveryNode("node_1", DummyTransportAddress.INSTANCE, Version.CURRENT), 0,
null,null,null,null,null,new FsInfo(0, node1FSInfo), null,null,null,null),
new NodeStats(new DiscoveryNode("node_2", DummyTransportAddress.INSTANCE, Version.CURRENT), 0,
null,null,null,null,null, new FsInfo(0, node2FSInfo), null,null,null,null),
new NodeStats(new DiscoveryNode("node_3", DummyTransportAddress.INSTANCE, Version.CURRENT), 0,
null,null,null,null,null, new FsInfo(0, node3FSInfo), null,null,null,null)
};
InternalClusterInfoService.fillDiskUsagePerNode(logger, nodeStats, newLeastAvaiableUsages, newMostAvaiableUsages);
DiskUsage leastNode_1 = newLeastAvaiableUsages.get("node_1");
DiskUsage mostNode_1 = newMostAvaiableUsages.get("node_1");
assertDiskUsage(mostNode_1, node1FSInfo[2]);
assertDiskUsage(leastNode_1, node1FSInfo[1]);
DiskUsage leastNode_2 = newLeastAvaiableUsages.get("node_2");
DiskUsage mostNode_2 = newMostAvaiableUsages.get("node_2");
assertDiskUsage(leastNode_2, node2FSInfo[0]);
assertDiskUsage(mostNode_2, node2FSInfo[0]);
DiskUsage leastNode_3 = newLeastAvaiableUsages.get("node_3");
DiskUsage mostNode_3 = newMostAvaiableUsages.get("node_3");
assertDiskUsage(leastNode_3, node3FSInfo[0]);
assertDiskUsage(mostNode_3, node3FSInfo[1]);
}
private void assertDiskUsage(DiskUsage usage, FsInfo.Path path) {
assertEquals(usage.toString(), usage.getPath(), path.getPath());
assertEquals(usage.toString(), usage.getTotalBytes(), path.getTotal().bytes());
assertEquals(usage.toString(), usage.getFreeBytes(), path.getAvailable().bytes());
}
} }

View File

@ -24,6 +24,7 @@ import org.elasticsearch.action.admin.cluster.node.stats.NodesStatsResponse;
import org.elasticsearch.action.admin.cluster.node.stats.TransportNodesStatsAction; import org.elasticsearch.action.admin.cluster.node.stats.TransportNodesStatsAction;
import org.elasticsearch.action.admin.indices.stats.IndicesStatsResponse; import org.elasticsearch.action.admin.indices.stats.IndicesStatsResponse;
import org.elasticsearch.action.admin.indices.stats.TransportIndicesStatsAction; import org.elasticsearch.action.admin.indices.stats.TransportIndicesStatsAction;
import org.elasticsearch.cluster.routing.allocation.decider.DiskThresholdDeciderTests;
import org.elasticsearch.cluster.routing.allocation.decider.MockDiskUsagesIT; import org.elasticsearch.cluster.routing.allocation.decider.MockDiskUsagesIT;
import org.elasticsearch.common.inject.Inject; import org.elasticsearch.common.inject.Inject;
import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.settings.Settings;
@ -63,9 +64,9 @@ public class MockInternalClusterInfoService extends InternalClusterInfoService {
ClusterService clusterService, ThreadPool threadPool) { ClusterService clusterService, ThreadPool threadPool) {
super(settings, nodeSettingsService, transportNodesStatsAction, transportIndicesStatsAction, clusterService, threadPool); super(settings, nodeSettingsService, transportNodesStatsAction, transportIndicesStatsAction, clusterService, threadPool);
this.clusterName = ClusterName.clusterNameFromSettings(settings); this.clusterName = ClusterName.clusterNameFromSettings(settings);
stats[0] = MockDiskUsagesIT.makeStats("node_t1", new DiskUsage("node_t1", "n1", 100, 100)); stats[0] = MockDiskUsagesIT.makeStats("node_t1", new DiskUsage("node_t1", "n1", "/dev/null", 100, 100));
stats[1] = MockDiskUsagesIT.makeStats("node_t2", new DiskUsage("node_t2", "n2", 100, 100)); stats[1] = MockDiskUsagesIT.makeStats("node_t2", new DiskUsage("node_t2", "n2", "/dev/null", 100, 100));
stats[2] = MockDiskUsagesIT.makeStats("node_t3", new DiskUsage("node_t3", "n3", 100, 100)); stats[2] = MockDiskUsagesIT.makeStats("node_t3", new DiskUsage("node_t3", "n3", "/dev/null", 100, 100));
} }
public void setN1Usage(String nodeName, DiskUsage newUsage) { public void setN1Usage(String nodeName, DiskUsage newUsage) {
@ -92,4 +93,9 @@ public class MockInternalClusterInfoService extends InternalClusterInfoService {
// Not used, so noop // Not used, so noop
return new CountDownLatch(0); return new CountDownLatch(0);
} }
public ClusterInfo getClusterInfo() {
ClusterInfo clusterInfo = super.getClusterInfo();
return new ClusterInfo(clusterInfo.getNodeLeastAvailableDiskUsages(), clusterInfo.getNodeMostAvailableDiskUsages(), clusterInfo.shardSizes, DiskThresholdDeciderTests.DEV_NULL_MAP);
}
} }

View File

@ -243,4 +243,25 @@ public class RoutingTableTest extends ESAllocationTestCase {
fail("Calling with non-existing index should be ignored at the moment"); fail("Calling with non-existing index should be ignored at the moment");
} }
} }
public void testAllShardsForMultipleIndices() {
assertThat(this.emptyRoutingTable.allShards(new String[0]).size(), is(0));
assertThat(this.testRoutingTable.allShards(new String[]{TEST_INDEX_1}).size(), is(this.shardsPerIndex));
initPrimaries();
assertThat(this.testRoutingTable.allShards(new String[]{TEST_INDEX_1}).size(), is(this.shardsPerIndex));
startInitializingShards(TEST_INDEX_1);
assertThat(this.testRoutingTable.allShards(new String[]{TEST_INDEX_1}).size(), is(this.shardsPerIndex));
startInitializingShards(TEST_INDEX_2);
assertThat(this.testRoutingTable.allShards(new String[]{TEST_INDEX_1, TEST_INDEX_2}).size(), is(this.totalNumberOfShards));
try {
this.testRoutingTable.allShards(new String[]{TEST_INDEX_1, "not_exists"});
} catch (IndexNotFoundException e) {
fail("Calling with non-existing index should be ignored at the moment");
}
}
} }

View File

@ -59,7 +59,7 @@ public class ExpectedShardSizeAllocationTests extends ESAllocationTestCase {
AllocationService strategy = createAllocationService(Settings.EMPTY, new ClusterInfoService() { AllocationService strategy = createAllocationService(Settings.EMPTY, new ClusterInfoService() {
@Override @Override
public ClusterInfo getClusterInfo() { public ClusterInfo getClusterInfo() {
return new ClusterInfo(Collections.EMPTY_MAP, Collections.EMPTY_MAP) { return new ClusterInfo() {
@Override @Override
public Long getShardSize(ShardRouting shardRouting) { public Long getShardSize(ShardRouting shardRouting) {
if (shardRouting.index().equals("test") && shardRouting.shardId().getId() == 0) { if (shardRouting.index().equals("test") && shardRouting.shardId().getId() == 0) {
@ -118,7 +118,7 @@ public class ExpectedShardSizeAllocationTests extends ESAllocationTestCase {
final AllocationService allocation = createAllocationService(Settings.EMPTY, new ClusterInfoService() { final AllocationService allocation = createAllocationService(Settings.EMPTY, new ClusterInfoService() {
@Override @Override
public ClusterInfo getClusterInfo() { public ClusterInfo getClusterInfo() {
return new ClusterInfo(Collections.EMPTY_MAP, Collections.EMPTY_MAP) { return new ClusterInfo() {
@Override @Override
public Long getShardSize(ShardRouting shardRouting) { public Long getShardSize(ShardRouting shardRouting) {
if (shardRouting.index().equals("test") && shardRouting.shardId().getId() == 0) { if (shardRouting.index().equals("test") && shardRouting.shardId().getId() == 0) {

View File

@ -66,7 +66,7 @@ public class RebalanceAfterActiveTests extends ESAllocationTestCase {
new ClusterInfoService() { new ClusterInfoService() {
@Override @Override
public ClusterInfo getClusterInfo() { public ClusterInfo getClusterInfo() {
return new ClusterInfo(Collections.EMPTY_MAP, Collections.EMPTY_MAP) { return new ClusterInfo() {
@Override @Override
public Long getShardSize(ShardRouting shardRouting) { public Long getShardSize(ShardRouting shardRouting) {
if (shardRouting.index().equals("test")) { if (shardRouting.index().equals("test")) {

View File

@ -65,15 +65,15 @@ public class DiskThresholdDeciderTests extends ESAllocationTestCase {
.put(DiskThresholdDecider.CLUSTER_ROUTING_ALLOCATION_HIGH_DISK_WATERMARK, 0.8).build(); .put(DiskThresholdDecider.CLUSTER_ROUTING_ALLOCATION_HIGH_DISK_WATERMARK, 0.8).build();
Map<String, DiskUsage> usages = new HashMap<>(); Map<String, DiskUsage> usages = new HashMap<>();
usages.put("node1", new DiskUsage("node1", "node1", 100, 10)); // 90% used usages.put("node1", new DiskUsage("node1", "node1", "/dev/null", 100, 10)); // 90% used
usages.put("node2", new DiskUsage("node2", "node2", 100, 35)); // 65% used usages.put("node2", new DiskUsage("node2", "node2", "/dev/null", 100, 35)); // 65% used
usages.put("node3", new DiskUsage("node3", "node3", 100, 60)); // 40% used usages.put("node3", new DiskUsage("node3", "node3", "/dev/null", 100, 60)); // 40% used
usages.put("node4", new DiskUsage("node4", "node4", 100, 80)); // 20% used usages.put("node4", new DiskUsage("node4", "node4", "/dev/null", 100, 80)); // 20% used
Map<String, Long> shardSizes = new HashMap<>(); Map<String, Long> shardSizes = new HashMap<>();
shardSizes.put("[test][0][p]", 10L); // 10 bytes shardSizes.put("[test][0][p]", 10L); // 10 bytes
shardSizes.put("[test][0][r]", 10L); shardSizes.put("[test][0][r]", 10L);
final ClusterInfo clusterInfo = new ClusterInfo(Collections.unmodifiableMap(usages), Collections.unmodifiableMap(shardSizes)); final ClusterInfo clusterInfo = new ClusterInfo(Collections.unmodifiableMap(usages), Collections.unmodifiableMap(usages), Collections.unmodifiableMap(shardSizes), DEV_NULL_MAP);
AllocationDeciders deciders = new AllocationDeciders(Settings.EMPTY, AllocationDeciders deciders = new AllocationDeciders(Settings.EMPTY,
new HashSet<>(Arrays.asList( new HashSet<>(Arrays.asList(
@ -92,7 +92,6 @@ public class DiskThresholdDeciderTests extends ESAllocationTestCase {
// noop // noop
} }
}; };
AllocationService strategy = new AllocationService(settingsBuilder() AllocationService strategy = new AllocationService(settingsBuilder()
.put("cluster.routing.allocation.concurrent_recoveries", 10) .put("cluster.routing.allocation.concurrent_recoveries", 10)
.put(ClusterRebalanceAllocationDecider.CLUSTER_ROUTING_ALLOCATION_ALLOW_REBALANCE, "always") .put(ClusterRebalanceAllocationDecider.CLUSTER_ROUTING_ALLOCATION_ALLOW_REBALANCE, "always")
@ -259,16 +258,16 @@ public class DiskThresholdDeciderTests extends ESAllocationTestCase {
.put(DiskThresholdDecider.CLUSTER_ROUTING_ALLOCATION_HIGH_DISK_WATERMARK, "9b").build(); .put(DiskThresholdDecider.CLUSTER_ROUTING_ALLOCATION_HIGH_DISK_WATERMARK, "9b").build();
Map<String, DiskUsage> usages = new HashMap<>(); Map<String, DiskUsage> usages = new HashMap<>();
usages.put("node1", new DiskUsage("node1", "n1", 100, 10)); // 90% used usages.put("node1", new DiskUsage("node1", "n1", "/dev/null", 100, 10)); // 90% used
usages.put("node2", new DiskUsage("node2", "n2", 100, 10)); // 90% used usages.put("node2", new DiskUsage("node2", "n2", "/dev/null", 100, 10)); // 90% used
usages.put("node3", new DiskUsage("node3", "n3", 100, 60)); // 40% used usages.put("node3", new DiskUsage("node3", "n3", "/dev/null", 100, 60)); // 40% used
usages.put("node4", new DiskUsage("node4", "n4", 100, 80)); // 20% used usages.put("node4", new DiskUsage("node4", "n4", "/dev/null", 100, 80)); // 20% used
usages.put("node5", new DiskUsage("node5", "n5", 100, 85)); // 15% used usages.put("node5", new DiskUsage("node5", "n5", "/dev/null", 100, 85)); // 15% used
Map<String, Long> shardSizes = new HashMap<>(); Map<String, Long> shardSizes = new HashMap<>();
shardSizes.put("[test][0][p]", 10L); // 10 bytes shardSizes.put("[test][0][p]", 10L); // 10 bytes
shardSizes.put("[test][0][r]", 10L); shardSizes.put("[test][0][r]", 10L);
final ClusterInfo clusterInfo = new ClusterInfo(Collections.unmodifiableMap(usages), Collections.unmodifiableMap(shardSizes)); final ClusterInfo clusterInfo = new ClusterInfo(Collections.unmodifiableMap(usages), Collections.unmodifiableMap(usages), Collections.unmodifiableMap(shardSizes), DEV_NULL_MAP);
AllocationDeciders deciders = new AllocationDeciders(Settings.EMPTY, AllocationDeciders deciders = new AllocationDeciders(Settings.EMPTY,
new HashSet<>(Arrays.asList( new HashSet<>(Arrays.asList(
@ -329,8 +328,8 @@ public class DiskThresholdDeciderTests extends ESAllocationTestCase {
logger.info("--> nodeWithoutPrimary: {}", nodeWithoutPrimary); logger.info("--> nodeWithoutPrimary: {}", nodeWithoutPrimary);
// Make node without the primary now habitable to replicas // Make node without the primary now habitable to replicas
usages.put(nodeWithoutPrimary, new DiskUsage(nodeWithoutPrimary, "", 100, 35)); // 65% used usages.put(nodeWithoutPrimary, new DiskUsage(nodeWithoutPrimary, "", "/dev/null", 100, 35)); // 65% used
final ClusterInfo clusterInfo2 = new ClusterInfo(Collections.unmodifiableMap(usages), Collections.unmodifiableMap(shardSizes)); final ClusterInfo clusterInfo2 = new ClusterInfo(Collections.unmodifiableMap(usages), Collections.unmodifiableMap(usages), Collections.unmodifiableMap(shardSizes), DEV_NULL_MAP);
cis = new ClusterInfoService() { cis = new ClusterInfoService() {
@Override @Override
public ClusterInfo getClusterInfo() { public ClusterInfo getClusterInfo() {
@ -524,12 +523,12 @@ public class DiskThresholdDeciderTests extends ESAllocationTestCase {
.put(DiskThresholdDecider.CLUSTER_ROUTING_ALLOCATION_HIGH_DISK_WATERMARK, "71%").build(); .put(DiskThresholdDecider.CLUSTER_ROUTING_ALLOCATION_HIGH_DISK_WATERMARK, "71%").build();
Map<String, DiskUsage> usages = new HashMap<>(); Map<String, DiskUsage> usages = new HashMap<>();
usages.put("node1", new DiskUsage("node1", "n1", 100, 31)); // 69% used usages.put("node1", new DiskUsage("node1", "n1", "/dev/null", 100, 31)); // 69% used
usages.put("node2", new DiskUsage("node2", "n2", 100, 1)); // 99% used usages.put("node2", new DiskUsage("node2", "n2", "/dev/null", 100, 1)); // 99% used
Map<String, Long> shardSizes = new HashMap<>(); Map<String, Long> shardSizes = new HashMap<>();
shardSizes.put("[test][0][p]", 10L); // 10 bytes shardSizes.put("[test][0][p]", 10L); // 10 bytes
final ClusterInfo clusterInfo = new ClusterInfo(Collections.unmodifiableMap(usages), Collections.unmodifiableMap(shardSizes)); final ClusterInfo clusterInfo = new ClusterInfo(Collections.unmodifiableMap(usages), Collections.unmodifiableMap(usages), Collections.unmodifiableMap(shardSizes), DEV_NULL_MAP);
AllocationDeciders deciders = new AllocationDeciders(Settings.EMPTY, AllocationDeciders deciders = new AllocationDeciders(Settings.EMPTY,
new HashSet<>(Arrays.asList( new HashSet<>(Arrays.asList(
@ -590,13 +589,13 @@ public class DiskThresholdDeciderTests extends ESAllocationTestCase {
.put(DiskThresholdDecider.CLUSTER_ROUTING_ALLOCATION_HIGH_DISK_WATERMARK, 0.85).build(); .put(DiskThresholdDecider.CLUSTER_ROUTING_ALLOCATION_HIGH_DISK_WATERMARK, 0.85).build();
Map<String, DiskUsage> usages = new HashMap<>(); Map<String, DiskUsage> usages = new HashMap<>();
usages.put("node2", new DiskUsage("node2", "node2", 100, 50)); // 50% used usages.put("node2", new DiskUsage("node2", "node2", "/dev/null", 100, 50)); // 50% used
usages.put("node3", new DiskUsage("node3", "node3", 100, 0)); // 100% used usages.put("node3", new DiskUsage("node3", "node3", "/dev/null", 100, 0)); // 100% used
Map<String, Long> shardSizes = new HashMap<>(); Map<String, Long> shardSizes = new HashMap<>();
shardSizes.put("[test][0][p]", 10L); // 10 bytes shardSizes.put("[test][0][p]", 10L); // 10 bytes
shardSizes.put("[test][0][r]", 10L); // 10 bytes shardSizes.put("[test][0][r]", 10L); // 10 bytes
final ClusterInfo clusterInfo = new ClusterInfo(Collections.unmodifiableMap(usages), Collections.unmodifiableMap(shardSizes)); final ClusterInfo clusterInfo = new ClusterInfo(Collections.unmodifiableMap(usages), Collections.unmodifiableMap(usages), Collections.unmodifiableMap(shardSizes), DEV_NULL_MAP);
AllocationDeciders deciders = new AllocationDeciders(Settings.EMPTY, AllocationDeciders deciders = new AllocationDeciders(Settings.EMPTY,
new HashSet<>(Arrays.asList( new HashSet<>(Arrays.asList(
@ -661,8 +660,8 @@ public class DiskThresholdDeciderTests extends ESAllocationTestCase {
DiskThresholdDecider decider = new DiskThresholdDecider(Settings.EMPTY); DiskThresholdDecider decider = new DiskThresholdDecider(Settings.EMPTY);
Map<String, DiskUsage> usages = new HashMap<>(); Map<String, DiskUsage> usages = new HashMap<>();
usages.put("node2", new DiskUsage("node2", "n2", 100, 50)); // 50% used usages.put("node2", new DiskUsage("node2", "n2", "/dev/null", 100, 50)); // 50% used
usages.put("node3", new DiskUsage("node3", "n3", 100, 0)); // 100% used usages.put("node3", new DiskUsage("node3", "n3", "/dev/null", 100, 0)); // 100% used
DiskUsage node1Usage = decider.averageUsage(rn, usages); DiskUsage node1Usage = decider.averageUsage(rn, usages);
assertThat(node1Usage.getTotalBytes(), equalTo(100L)); assertThat(node1Usage.getTotalBytes(), equalTo(100L));
@ -675,10 +674,10 @@ public class DiskThresholdDeciderTests extends ESAllocationTestCase {
DiskThresholdDecider decider = new DiskThresholdDecider(Settings.EMPTY); DiskThresholdDecider decider = new DiskThresholdDecider(Settings.EMPTY);
Map<String, DiskUsage> usages = new HashMap<>(); Map<String, DiskUsage> usages = new HashMap<>();
usages.put("node2", new DiskUsage("node2", "n2", 100, 50)); // 50% used usages.put("node2", new DiskUsage("node2", "n2", "/dev/null", 100, 50)); // 50% used
usages.put("node3", new DiskUsage("node3", "n3", 100, 0)); // 100% used usages.put("node3", new DiskUsage("node3", "n3", "/dev/null", 100, 0)); // 100% used
Double after = decider.freeDiskPercentageAfterShardAssigned(new DiskUsage("node2", "n2", 100, 30), 11L); Double after = decider.freeDiskPercentageAfterShardAssigned(new DiskUsage("node2", "n2", "/dev/null", 100, 30), 11L);
assertThat(after, equalTo(19.0)); assertThat(after, equalTo(19.0));
} }
@ -691,16 +690,16 @@ public class DiskThresholdDeciderTests extends ESAllocationTestCase {
.put(DiskThresholdDecider.CLUSTER_ROUTING_ALLOCATION_HIGH_DISK_WATERMARK, 0.8).build(); .put(DiskThresholdDecider.CLUSTER_ROUTING_ALLOCATION_HIGH_DISK_WATERMARK, 0.8).build();
Map<String, DiskUsage> usages = new HashMap<>(); Map<String, DiskUsage> usages = new HashMap<>();
usages.put("node1", new DiskUsage("node1", "n1", 100, 40)); // 60% used usages.put("node1", new DiskUsage("node1", "n1", "/dev/null", 100, 40)); // 60% used
usages.put("node2", new DiskUsage("node2", "n2", 100, 40)); // 60% used usages.put("node2", new DiskUsage("node2", "n2", "/dev/null", 100, 40)); // 60% used
usages.put("node2", new DiskUsage("node3", "n3", 100, 40)); // 60% used usages.put("node3", new DiskUsage("node3", "n3", "/dev/null", 100, 40)); // 60% used
Map<String, Long> shardSizes = new HashMap<>(); Map<String, Long> shardSizes = new HashMap<>();
shardSizes.put("[test][0][p]", 14L); // 14 bytes shardSizes.put("[test][0][p]", 14L); // 14 bytes
shardSizes.put("[test][0][r]", 14L); shardSizes.put("[test][0][r]", 14L);
shardSizes.put("[test2][0][p]", 1L); // 1 bytes shardSizes.put("[test2][0][p]", 1L); // 1 bytes
shardSizes.put("[test2][0][r]", 1L); shardSizes.put("[test2][0][r]", 1L);
final ClusterInfo clusterInfo = new ClusterInfo(Collections.unmodifiableMap(usages), Collections.unmodifiableMap(shardSizes)); final ClusterInfo clusterInfo = new ClusterInfo(Collections.unmodifiableMap(usages), Collections.unmodifiableMap(usages), Collections.unmodifiableMap(shardSizes), DEV_NULL_MAP);
AllocationDeciders deciders = new AllocationDeciders(Settings.EMPTY, AllocationDeciders deciders = new AllocationDeciders(Settings.EMPTY,
new HashSet<>(Arrays.asList( new HashSet<>(Arrays.asList(
@ -797,13 +796,13 @@ public class DiskThresholdDeciderTests extends ESAllocationTestCase {
// We have an index with 2 primary shards each taking 40 bytes. Each node has 100 bytes available // We have an index with 2 primary shards each taking 40 bytes. Each node has 100 bytes available
Map<String, DiskUsage> usages = new HashMap<>(); Map<String, DiskUsage> usages = new HashMap<>();
usages.put("node1", new DiskUsage("node1", "n1", 100, 20)); // 80% used usages.put("node1", new DiskUsage("node1", "n1", "/dev/null", 100, 20)); // 80% used
usages.put("node2", new DiskUsage("node2", "n2", 100, 100)); // 0% used usages.put("node2", new DiskUsage("node2", "n2", "/dev/null", 100, 100)); // 0% used
Map<String, Long> shardSizes = new HashMap<>(); Map<String, Long> shardSizes = new HashMap<>();
shardSizes.put("[test][0][p]", 40L); shardSizes.put("[test][0][p]", 40L);
shardSizes.put("[test][1][p]", 40L); shardSizes.put("[test][1][p]", 40L);
final ClusterInfo clusterInfo = new ClusterInfo(Collections.unmodifiableMap(usages), Collections.unmodifiableMap(shardSizes)); final ClusterInfo clusterInfo = new ClusterInfo(Collections.unmodifiableMap(usages), Collections.unmodifiableMap(usages), Collections.unmodifiableMap(shardSizes), DEV_NULL_MAP);
DiskThresholdDecider diskThresholdDecider = new DiskThresholdDecider(diskSettings); DiskThresholdDecider diskThresholdDecider = new DiskThresholdDecider(diskSettings);
MetaData metaData = MetaData.builder() MetaData metaData = MetaData.builder()
@ -916,4 +915,26 @@ public class DiskThresholdDeciderTests extends ESAllocationTestCase {
rn.shardsWithState(RELOCATING), rn.shardsWithState(RELOCATING),
rn.shardsWithState(STARTED)); rn.shardsWithState(STARTED));
} }
public static final Map<ShardRouting, String> DEV_NULL_MAP = Collections.unmodifiableMap(new StaticValueMap("/dev/null"));
// a test only map that always returns the same value no matter what key is passed
private static final class StaticValueMap extends AbstractMap<ShardRouting, String> {
private final String value;
private StaticValueMap(String value) {
this.value = value;
}
@Override
public String get(Object key) {
return value;
}
@Override
public Set<Entry<ShardRouting, String>> entrySet() {
throw new UnsupportedOperationException("this is a test-only map that only supports #get(Object key)");
}
}
} }

View File

@ -20,15 +20,15 @@
package org.elasticsearch.cluster.routing.allocation.decider; package org.elasticsearch.cluster.routing.allocation.decider;
import org.elasticsearch.Version; import org.elasticsearch.Version;
import org.elasticsearch.cluster.ClusterInfo; import org.elasticsearch.cluster.*;
import org.elasticsearch.cluster.ClusterInfoService; import org.elasticsearch.cluster.metadata.IndexMetaData;
import org.elasticsearch.cluster.EmptyClusterInfoService; import org.elasticsearch.cluster.metadata.MetaData;
import org.elasticsearch.cluster.node.DiscoveryNode; import org.elasticsearch.cluster.node.DiscoveryNode;
import org.elasticsearch.cluster.routing.RoutingNode; import org.elasticsearch.cluster.node.DiscoveryNodes;
import org.elasticsearch.cluster.routing.ShardRouting; import org.elasticsearch.cluster.routing.*;
import org.elasticsearch.cluster.routing.ShardRoutingHelper; import org.elasticsearch.cluster.routing.allocation.RoutingAllocation;
import org.elasticsearch.cluster.routing.UnassignedInfo;
import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.transport.DummyTransportAddress;
import org.elasticsearch.common.transport.LocalTransportAddress; import org.elasticsearch.common.transport.LocalTransportAddress;
import org.elasticsearch.common.unit.ByteSizeValue; import org.elasticsearch.common.unit.ByteSizeValue;
import org.elasticsearch.node.settings.NodeSettingsService; import org.elasticsearch.node.settings.NodeSettingsService;
@ -76,11 +76,11 @@ public class DiskThresholdDeciderUnitTests extends ESTestCase {
applySettings.onRefreshSettings(newSettings); applySettings.onRefreshSettings(newSettings);
assertThat("high threshold bytes should be unset", assertThat("high threshold bytes should be unset",
decider.getFreeBytesThresholdHigh(), equalTo(ByteSizeValue.parseBytesSizeValue("0b", "test"))); decider.getFreeBytesThresholdHigh(), equalTo(ByteSizeValue.parseBytesSizeValue("0b", "test")));
assertThat("high threshold percentage should be changed", assertThat("high threshold percentage should be changed",
decider.getFreeDiskThresholdHigh(), equalTo(30.0d)); decider.getFreeDiskThresholdHigh(), equalTo(30.0d));
assertThat("low threshold bytes should be set to 500mb", assertThat("low threshold bytes should be set to 500mb",
decider.getFreeBytesThresholdLow(), equalTo(ByteSizeValue.parseBytesSizeValue("500mb", "test"))); decider.getFreeBytesThresholdLow(), equalTo(ByteSizeValue.parseBytesSizeValue("500mb", "test")));
assertThat("low threshold bytes should be unset", assertThat("low threshold bytes should be unset",
decider.getFreeDiskThresholdLow(), equalTo(0.0d)); decider.getFreeDiskThresholdLow(), equalTo(0.0d));
assertThat("reroute interval should be changed to 30 seconds", assertThat("reroute interval should be changed to 30 seconds",
@ -89,13 +89,133 @@ public class DiskThresholdDeciderUnitTests extends ESTestCase {
assertFalse("relocations should now be disabled", decider.isIncludeRelocations()); assertFalse("relocations should now be disabled", decider.isIncludeRelocations());
} }
public void testCanAllocateUsesMaxAvailableSpace() {
NodeSettingsService nss = new NodeSettingsService(Settings.EMPTY);
ClusterInfoService cis = EmptyClusterInfoService.INSTANCE;
DiskThresholdDecider decider = new DiskThresholdDecider(Settings.EMPTY, nss, cis, null);
ShardRouting test_0 = ShardRouting.newUnassigned("test", 0, null, true, new UnassignedInfo(UnassignedInfo.Reason.INDEX_CREATED, "foo"));
DiscoveryNode node_0 = new DiscoveryNode("node_0", DummyTransportAddress.INSTANCE, Version.CURRENT);
DiscoveryNode node_1 = new DiscoveryNode("node_1", DummyTransportAddress.INSTANCE, Version.CURRENT);
MetaData metaData = MetaData.builder()
.put(IndexMetaData.builder("test").settings(settings(Version.CURRENT)).numberOfShards(1).numberOfReplicas(1))
.build();
RoutingTable routingTable = RoutingTable.builder()
.addAsNew(metaData.index("test"))
.build();
ClusterState clusterState = ClusterState.builder(org.elasticsearch.cluster.ClusterName.DEFAULT).metaData(metaData).routingTable(routingTable).build();
clusterState = ClusterState.builder(clusterState).nodes(DiscoveryNodes.builder()
.put(node_0)
.put(node_1)
).build();
// actual test -- after all that bloat :)
Map<String, DiskUsage> leastAvailableUsages = new HashMap<>();
leastAvailableUsages.put("node_0", new DiskUsage("node_0", "node_0", "_na_", 100, 0)); // all full
leastAvailableUsages.put("node_1", new DiskUsage("node_1", "node_1", "_na_", 100, 0)); // all full
Map<String, DiskUsage> mostAvailableUsage = new HashMap<>();
mostAvailableUsage.put("node_0", new DiskUsage("node_0", "node_0", "_na_", 100, randomIntBetween(20, 100))); // 20 - 99 percent since after allocation there must be at least 10% left and shard is 10byte
mostAvailableUsage.put("node_1", new DiskUsage("node_1", "node_1", "_na_", 100, randomIntBetween(0, 10))); // this is weird and smells like a bug! it should be up to 20%?
Map<String, Long> shardSizes = new HashMap<>();
shardSizes.put("[test][0][p]", 10L); // 10 bytes
final ClusterInfo clusterInfo = new ClusterInfo(Collections.unmodifiableMap(leastAvailableUsages), Collections.unmodifiableMap(mostAvailableUsage), Collections.unmodifiableMap(shardSizes), Collections.EMPTY_MAP);
RoutingAllocation allocation = new RoutingAllocation(new AllocationDeciders(Settings.EMPTY, new AllocationDecider[]{decider}), clusterState.getRoutingNodes(), clusterState.nodes(), clusterInfo);
assertEquals(mostAvailableUsage.toString(), Decision.YES, decider.canAllocate(test_0, new RoutingNode("node_0", node_0), allocation));
assertEquals(mostAvailableUsage.toString(), Decision.NO, decider.canAllocate(test_0, new RoutingNode("node_1", node_1), allocation));
}
public void testCanRemainUsesLeastAvailableSpace() {
NodeSettingsService nss = new NodeSettingsService(Settings.EMPTY);
ClusterInfoService cis = EmptyClusterInfoService.INSTANCE;
DiskThresholdDecider decider = new DiskThresholdDecider(Settings.EMPTY, nss, cis, null);
Map<ShardRouting, String> shardRoutingMap = new HashMap<>();
DiscoveryNode node_0 = new DiscoveryNode("node_0", DummyTransportAddress.INSTANCE, Version.CURRENT);
DiscoveryNode node_1 = new DiscoveryNode("node_1", DummyTransportAddress.INSTANCE, Version.CURRENT);
ShardRouting test_0 = ShardRouting.newUnassigned("test", 0, null, true, new UnassignedInfo(UnassignedInfo.Reason.INDEX_CREATED, "foo"));
ShardRoutingHelper.initialize(test_0, node_0.getId());
ShardRoutingHelper.moveToStarted(test_0);
shardRoutingMap.put(test_0, "/node0/least");
ShardRouting test_1 = ShardRouting.newUnassigned("test", 1, null, true, new UnassignedInfo(UnassignedInfo.Reason.INDEX_CREATED, "foo"));
ShardRoutingHelper.initialize(test_1, node_1.getId());
ShardRoutingHelper.moveToStarted(test_1);
shardRoutingMap.put(test_1, "/node1/least");
MetaData metaData = MetaData.builder()
.put(IndexMetaData.builder("test").settings(settings(Version.CURRENT)).numberOfShards(1).numberOfReplicas(1))
.build();
RoutingTable routingTable = RoutingTable.builder()
.addAsNew(metaData.index("test"))
.build();
ClusterState clusterState = ClusterState.builder(org.elasticsearch.cluster.ClusterName.DEFAULT).metaData(metaData).routingTable(routingTable).build();
logger.info("--> adding two nodes");
clusterState = ClusterState.builder(clusterState).nodes(DiscoveryNodes.builder()
.put(node_0)
.put(node_1)
).build();
// actual test -- after all that bloat :)
Map<String, DiskUsage> leastAvailableUsages = new HashMap<>();
leastAvailableUsages.put("node_0", new DiskUsage("node_0", "node_0", "/node0/least", 100, 10)); // 90% used
leastAvailableUsages.put("node_1", new DiskUsage("node_1", "node_1", "/node1/least", 100, 9)); // 91% used
Map<String, DiskUsage> mostAvailableUsage = new HashMap<>();
mostAvailableUsage.put("node_0", new DiskUsage("node_0", "node_0", "/node0/most", 100, 90)); // 10% used
mostAvailableUsage.put("node_1", new DiskUsage("node_1", "node_1", "/node1/most", 100, 90)); // 10% used
Map<String, Long> shardSizes = new HashMap<>();
shardSizes.put("[test][0][p]", 10L); // 10 bytes
shardSizes.put("[test][1][p]", 10L);
shardSizes.put("[test][2][p]", 10L);
final ClusterInfo clusterInfo = new ClusterInfo(Collections.unmodifiableMap(leastAvailableUsages), Collections.unmodifiableMap(mostAvailableUsage), Collections.unmodifiableMap(shardSizes), shardRoutingMap);
RoutingAllocation allocation = new RoutingAllocation(new AllocationDeciders(Settings.EMPTY, new AllocationDecider[]{decider}), clusterState.getRoutingNodes(), clusterState.nodes(), clusterInfo);
assertEquals(Decision.YES, decider.canRemain(test_0, new RoutingNode("node_0", node_0), allocation));
assertEquals(Decision.NO, decider.canRemain(test_1, new RoutingNode("node_1", node_1), allocation));
try {
decider.canRemain(test_0, new RoutingNode("node_1", node_1), allocation);
fail("not allocated on this node");
} catch (IllegalArgumentException ex) {
// not allocated on that node
}
try {
decider.canRemain(test_1, new RoutingNode("node_0", node_0), allocation);
fail("not allocated on this node");
} catch (IllegalArgumentException ex) {
// not allocated on that node
}
ShardRouting test_2 = ShardRouting.newUnassigned("test", 2, null, true, new UnassignedInfo(UnassignedInfo.Reason.INDEX_CREATED, "foo"));
ShardRoutingHelper.initialize(test_2, node_1.getId());
ShardRoutingHelper.moveToStarted(test_2);
shardRoutingMap.put(test_2, "/node1/most");
assertEquals("can stay since allocated on a different path with enough space", Decision.YES, decider.canRemain(test_2, new RoutingNode("node_1", node_1), allocation));
ShardRouting test_3 = ShardRouting.newUnassigned("test", 3, null, true, new UnassignedInfo(UnassignedInfo.Reason.INDEX_CREATED, "foo"));
ShardRoutingHelper.initialize(test_3, node_1.getId());
ShardRoutingHelper.moveToStarted(test_3);
assertEquals("can stay since we don't have information about this shard", Decision.YES, decider.canRemain(test_2, new RoutingNode("node_1", node_1), allocation));
}
public void testShardSizeAndRelocatingSize() { public void testShardSizeAndRelocatingSize() {
Map<String, Long> shardSizes = new HashMap<>(); Map<String, Long> shardSizes = new HashMap<>();
shardSizes.put("[test][0][r]", 10L); shardSizes.put("[test][0][r]", 10L);
shardSizes.put("[test][1][r]", 100L); shardSizes.put("[test][1][r]", 100L);
shardSizes.put("[test][2][r]", 1000L); shardSizes.put("[test][2][r]", 1000L);
shardSizes.put("[other][0][p]", 10000L); shardSizes.put("[other][0][p]", 10000L);
ClusterInfo info = new ClusterInfo(Collections.EMPTY_MAP, shardSizes); ClusterInfo info = new ClusterInfo(Collections.EMPTY_MAP, Collections.EMPTY_MAP, shardSizes, DiskThresholdDeciderTests.DEV_NULL_MAP);
ShardRouting test_0 = ShardRouting.newUnassigned("test", 0, null, false, new UnassignedInfo(UnassignedInfo.Reason.INDEX_CREATED, "foo")); ShardRouting test_0 = ShardRouting.newUnassigned("test", 0, null, false, new UnassignedInfo(UnassignedInfo.Reason.INDEX_CREATED, "foo"));
ShardRoutingHelper.initialize(test_0, "node1"); ShardRoutingHelper.initialize(test_0, "node1");
ShardRoutingHelper.moveToStarted(test_0); ShardRoutingHelper.moveToStarted(test_0);
@ -115,8 +235,10 @@ public class DiskThresholdDeciderUnitTests extends ESTestCase {
assertEquals(10l, DiskThresholdDecider.getShardSize(test_0, info)); assertEquals(10l, DiskThresholdDecider.getShardSize(test_0, info));
RoutingNode node = new RoutingNode("node1", new DiscoveryNode("node1", LocalTransportAddress.PROTO, Version.CURRENT), Arrays.asList(test_0, test_1.buildTargetRelocatingShard(), test_2)); RoutingNode node = new RoutingNode("node1", new DiscoveryNode("node1", LocalTransportAddress.PROTO, Version.CURRENT), Arrays.asList(test_0, test_1.buildTargetRelocatingShard(), test_2));
assertEquals(100l, DiskThresholdDecider.sizeOfRelocatingShards(node, info, false)); assertEquals(100l, DiskThresholdDecider.sizeOfRelocatingShards(node, info, false, "/dev/null"));
assertEquals(90l, DiskThresholdDecider.sizeOfRelocatingShards(node, info, true)); assertEquals(90l, DiskThresholdDecider.sizeOfRelocatingShards(node, info, true, "/dev/null"));
assertEquals(0l, DiskThresholdDecider.sizeOfRelocatingShards(node, info, true, "/dev/some/other/dev"));
assertEquals(0l, DiskThresholdDecider.sizeOfRelocatingShards(node, info, true, "/dev/some/other/dev"));
ShardRouting test_3 = ShardRouting.newUnassigned("test", 3, null, false, new UnassignedInfo(UnassignedInfo.Reason.INDEX_CREATED, "foo")); ShardRouting test_3 = ShardRouting.newUnassigned("test", 3, null, false, new UnassignedInfo(UnassignedInfo.Reason.INDEX_CREATED, "foo"));
ShardRoutingHelper.initialize(test_3, "node1"); ShardRoutingHelper.initialize(test_3, "node1");
@ -132,11 +254,11 @@ public class DiskThresholdDeciderUnitTests extends ESTestCase {
node = new RoutingNode("node1", new DiscoveryNode("node1", LocalTransportAddress.PROTO, Version.CURRENT), Arrays.asList(test_0, test_1.buildTargetRelocatingShard(), test_2, other_0.buildTargetRelocatingShard())); node = new RoutingNode("node1", new DiscoveryNode("node1", LocalTransportAddress.PROTO, Version.CURRENT), Arrays.asList(test_0, test_1.buildTargetRelocatingShard(), test_2, other_0.buildTargetRelocatingShard()));
if (other_0.primary()) { if (other_0.primary()) {
assertEquals(10100l, DiskThresholdDecider.sizeOfRelocatingShards(node, info, false)); assertEquals(10100l, DiskThresholdDecider.sizeOfRelocatingShards(node, info, false, "/dev/null"));
assertEquals(10090l, DiskThresholdDecider.sizeOfRelocatingShards(node, info, true)); assertEquals(10090l, DiskThresholdDecider.sizeOfRelocatingShards(node, info, true, "/dev/null"));
} else { } else {
assertEquals(100l, DiskThresholdDecider.sizeOfRelocatingShards(node, info, false)); assertEquals(100l, DiskThresholdDecider.sizeOfRelocatingShards(node, info, false, "/dev/null"));
assertEquals(90l, DiskThresholdDecider.sizeOfRelocatingShards(node, info, true)); assertEquals(90l, DiskThresholdDecider.sizeOfRelocatingShards(node, info, true, "/dev/null"));
} }
} }

View File

@ -33,9 +33,11 @@ import org.elasticsearch.cluster.routing.RoutingNode;
import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.transport.DummyTransportAddress; import org.elasticsearch.common.transport.DummyTransportAddress;
import org.elasticsearch.monitor.fs.FsInfo; import org.elasticsearch.monitor.fs.FsInfo;
import org.elasticsearch.plugins.Plugin;
import org.elasticsearch.test.ESIntegTestCase; import org.elasticsearch.test.ESIntegTestCase;
import org.junit.Test; import org.junit.Test;
import java.util.Collection;
import java.util.ArrayList; import java.util.ArrayList;
import java.util.Iterator; import java.util.Iterator;
import java.util.List; import java.util.List;
@ -52,13 +54,17 @@ public class MockDiskUsagesIT extends ESIntegTestCase {
protected Settings nodeSettings(int nodeOrdinal) { protected Settings nodeSettings(int nodeOrdinal) {
return Settings.builder() return Settings.builder()
.put(super.nodeSettings(nodeOrdinal)) .put(super.nodeSettings(nodeOrdinal))
// Use the mock internal cluster info service, which has fake-able disk usages
.extendArray("plugin.types", MockInternalClusterInfoService.TestPlugin.class.getName())
// Update more frequently // Update more frequently
.put(InternalClusterInfoService.INTERNAL_CLUSTER_INFO_UPDATE_INTERVAL, "1s") .put(InternalClusterInfoService.INTERNAL_CLUSTER_INFO_UPDATE_INTERVAL, "1s")
.build(); .build();
} }
@Override
protected Collection<Class<? extends Plugin>> nodePlugins() {
// Use the mock internal cluster info service, which has fake-able disk usages
return pluginList(MockInternalClusterInfoService.TestPlugin.class);
}
@Test @Test
//@TestLogging("org.elasticsearch.cluster:TRACE,org.elasticsearch.cluster.routing.allocation.decider:TRACE") //@TestLogging("org.elasticsearch.cluster:TRACE,org.elasticsearch.cluster.routing.allocation.decider:TRACE")
public void testRerouteOccursOnDiskPassingHighWatermark() throws Exception { public void testRerouteOccursOnDiskPassingHighWatermark() throws Exception {
@ -76,9 +82,9 @@ public class MockDiskUsagesIT extends ESIntegTestCase {
// Start with all nodes at 50% usage // Start with all nodes at 50% usage
final MockInternalClusterInfoService cis = (MockInternalClusterInfoService) final MockInternalClusterInfoService cis = (MockInternalClusterInfoService)
internalCluster().getInstance(ClusterInfoService.class, internalCluster().getMasterName()); internalCluster().getInstance(ClusterInfoService.class, internalCluster().getMasterName());
cis.setN1Usage(nodes.get(0), new DiskUsage(nodes.get(0), "n1", 100, 50)); cis.setN1Usage(nodes.get(0), new DiskUsage(nodes.get(0), "n1", "/dev/null", 100, 50));
cis.setN2Usage(nodes.get(1), new DiskUsage(nodes.get(1), "n2", 100, 50)); cis.setN2Usage(nodes.get(1), new DiskUsage(nodes.get(1), "n2", "/dev/null", 100, 50));
cis.setN3Usage(nodes.get(2), new DiskUsage(nodes.get(2), "n3", 100, 50)); cis.setN3Usage(nodes.get(2), new DiskUsage(nodes.get(2), "n3", "/dev/null", 100, 50));
client().admin().cluster().prepareUpdateSettings().setTransientSettings(settingsBuilder() client().admin().cluster().prepareUpdateSettings().setTransientSettings(settingsBuilder()
.put(DiskThresholdDecider.CLUSTER_ROUTING_ALLOCATION_LOW_DISK_WATERMARK, randomFrom("20b", "80%")) .put(DiskThresholdDecider.CLUSTER_ROUTING_ALLOCATION_LOW_DISK_WATERMARK, randomFrom("20b", "80%"))
@ -97,8 +103,8 @@ public class MockDiskUsagesIT extends ESIntegTestCase {
@Override @Override
public void run() { public void run() {
ClusterInfo info = cis.getClusterInfo(); ClusterInfo info = cis.getClusterInfo();
logger.info("--> got: {} nodes", info.getNodeDiskUsages().size()); logger.info("--> got: {} nodes", info.getNodeLeastAvailableDiskUsages().size());
assertThat(info.getNodeDiskUsages().size(), greaterThan(0)); assertThat(info.getNodeLeastAvailableDiskUsages().size(), greaterThan(0));
} }
}); });
@ -113,9 +119,9 @@ public class MockDiskUsagesIT extends ESIntegTestCase {
} }
// Update the disk usages so one node has now passed the high watermark // Update the disk usages so one node has now passed the high watermark
cis.setN1Usage(realNodeNames.get(0), new DiskUsage(nodes.get(0), "n1", 100, 50)); cis.setN1Usage(realNodeNames.get(0), new DiskUsage(nodes.get(0), "n1", "_na_", 100, 50));
cis.setN2Usage(realNodeNames.get(1), new DiskUsage(nodes.get(1), "n2", 100, 50)); cis.setN2Usage(realNodeNames.get(1), new DiskUsage(nodes.get(1), "n2", "_na_", 100, 50));
cis.setN3Usage(realNodeNames.get(2), new DiskUsage(nodes.get(2), "n3", 100, 0)); // nothing free on node3 cis.setN3Usage(realNodeNames.get(2), new DiskUsage(nodes.get(2), "n3", "_na_", 100, 0)); // nothing free on node3
// Retrieve the count of shards on each node // Retrieve the count of shards on each node
final Map<String, Integer> nodesToShardCount = newHashMap(); final Map<String, Integer> nodesToShardCount = newHashMap();
@ -138,9 +144,9 @@ public class MockDiskUsagesIT extends ESIntegTestCase {
}); });
// Update the disk usages so one node is now back under the high watermark // Update the disk usages so one node is now back under the high watermark
cis.setN1Usage(realNodeNames.get(0), new DiskUsage(nodes.get(0), "n1", 100, 50)); cis.setN1Usage(realNodeNames.get(0), new DiskUsage(nodes.get(0), "n1", "_na_", 100, 50));
cis.setN2Usage(realNodeNames.get(1), new DiskUsage(nodes.get(1), "n2", 100, 50)); cis.setN2Usage(realNodeNames.get(1), new DiskUsage(nodes.get(1), "n2", "_na_", 100, 50));
cis.setN3Usage(realNodeNames.get(2), new DiskUsage(nodes.get(2), "n3", 100, 50)); // node3 has free space now cis.setN3Usage(realNodeNames.get(2), new DiskUsage(nodes.get(2), "n3", "_na_", 100, 50)); // node3 has free space now
// Retrieve the count of shards on each node // Retrieve the count of shards on each node
nodesToShardCount.clear(); nodesToShardCount.clear();
@ -166,7 +172,7 @@ public class MockDiskUsagesIT extends ESIntegTestCase {
/** Create a fake NodeStats for the given node and usage */ /** Create a fake NodeStats for the given node and usage */
public static NodeStats makeStats(String nodeName, DiskUsage usage) { public static NodeStats makeStats(String nodeName, DiskUsage usage) {
FsInfo.Path[] paths = new FsInfo.Path[1]; FsInfo.Path[] paths = new FsInfo.Path[1];
FsInfo.Path path = new FsInfo.Path("/path.data", null, FsInfo.Path path = new FsInfo.Path("/dev/null", null,
usage.getTotalBytes(), usage.getFreeBytes(), usage.getFreeBytes()); usage.getTotalBytes(), usage.getFreeBytes(), usage.getFreeBytes());
paths[0] = path; paths[0] = path;
FsInfo fsInfo = new FsInfo(System.currentTimeMillis(), paths); FsInfo fsInfo = new FsInfo(System.currentTimeMillis(), paths);

View File

@ -42,11 +42,8 @@ import static org.hamcrest.Matchers.nullValue;
public class SettingsFilteringIT extends ESIntegTestCase { public class SettingsFilteringIT extends ESIntegTestCase {
@Override @Override
protected Settings nodeSettings(int nodeOrdinal) { protected Collection<Class<? extends Plugin>> nodePlugins() {
return Settings.settingsBuilder() return pluginList(SettingsFilteringPlugin.class);
.put(super.nodeSettings(nodeOrdinal))
.put("plugin.types", SettingsFilteringPlugin.class.getName())
.build();
} }
public static class SettingsFilteringPlugin extends Plugin { public static class SettingsFilteringPlugin extends Plugin {

View File

@ -48,6 +48,7 @@ import org.elasticsearch.discovery.zen.ping.ZenPing;
import org.elasticsearch.discovery.zen.ping.ZenPingService; import org.elasticsearch.discovery.zen.ping.ZenPingService;
import org.elasticsearch.discovery.zen.ping.unicast.UnicastZenPing; import org.elasticsearch.discovery.zen.ping.unicast.UnicastZenPing;
import org.elasticsearch.discovery.zen.publish.PublishClusterStateAction; import org.elasticsearch.discovery.zen.publish.PublishClusterStateAction;
import org.elasticsearch.plugins.Plugin;
import org.elasticsearch.test.ESIntegTestCase; import org.elasticsearch.test.ESIntegTestCase;
import org.elasticsearch.test.InternalTestCluster; import org.elasticsearch.test.InternalTestCluster;
import org.elasticsearch.test.discovery.ClusterDiscoveryConfiguration; import org.elasticsearch.test.discovery.ClusterDiscoveryConfiguration;
@ -82,7 +83,7 @@ public class DiscoveryWithServiceDisruptionsIT extends ESIntegTestCase {
@Override @Override
protected Settings nodeSettings(int nodeOrdinal) { protected Settings nodeSettings(int nodeOrdinal) {
return discoveryConfig.node(nodeOrdinal); return discoveryConfig.nodeSettings(nodeOrdinal);
} }
@Before @Before
@ -134,9 +135,13 @@ public class DiscoveryWithServiceDisruptionsIT extends ESIntegTestCase {
.put("transport.bind_host", "127.0.0.1") .put("transport.bind_host", "127.0.0.1")
.put("transport.publish_host", "127.0.0.1") .put("transport.publish_host", "127.0.0.1")
.put("gateway.local.list_timeout", "10s") // still long to induce failures but to long so test won't time out .put("gateway.local.list_timeout", "10s") // still long to induce failures but to long so test won't time out
.put("plugin.types", MockTransportService.TestPlugin.class.getName())
.build(); .build();
@Override
protected Collection<Class<? extends Plugin>> nodePlugins() {
return pluginList(MockTransportService.TestPlugin.class);
}
private void configureUnicastCluster(int numberOfNodes, @Nullable int[] unicastHostsOrdinals, int minimumMasterNode) throws ExecutionException, InterruptedException { private void configureUnicastCluster(int numberOfNodes, @Nullable int[] unicastHostsOrdinals, int minimumMasterNode) throws ExecutionException, InterruptedException {
if (minimumMasterNode < 0) { if (minimumMasterNode < 0) {
minimumMasterNode = numberOfNodes / 2 + 1; minimumMasterNode = numberOfNodes / 2 + 1;

View File

@ -43,7 +43,7 @@ public class ZenUnicastDiscoveryIT extends ESIntegTestCase {
@Override @Override
protected Settings nodeSettings(int nodeOrdinal) { protected Settings nodeSettings(int nodeOrdinal) {
return discoveryConfig.node(nodeOrdinal); return discoveryConfig.nodeSettings(nodeOrdinal);
} }
@Before @Before

View File

@ -42,7 +42,11 @@ import org.elasticsearch.discovery.zen.publish.PublishClusterStateAction;
import org.elasticsearch.test.ESIntegTestCase; import org.elasticsearch.test.ESIntegTestCase;
import org.elasticsearch.test.junit.annotations.TestLogging; import org.elasticsearch.test.junit.annotations.TestLogging;
import org.elasticsearch.threadpool.ThreadPool; import org.elasticsearch.threadpool.ThreadPool;
import org.elasticsearch.transport.*; import org.elasticsearch.transport.BytesTransportRequest;
import org.elasticsearch.transport.EmptyTransportResponseHandler;
import org.elasticsearch.transport.TransportException;
import org.elasticsearch.transport.TransportResponse;
import org.elasticsearch.transport.TransportService;
import org.hamcrest.Matchers; import org.hamcrest.Matchers;
import org.junit.Test; import org.junit.Test;
@ -112,7 +116,7 @@ public class ZenDiscoveryIT extends ESIntegTestCase {
createIndex("test"); createIndex("test");
ensureSearchable("test"); ensureSearchable("test");
RecoveryResponse r = client().admin().indices().prepareRecoveries("test").get(); RecoveryResponse r = client().admin().indices().prepareRecoveries("test").get();
int numRecoveriesBeforeNewMaster = r.shardResponses().get("test").size(); int numRecoveriesBeforeNewMaster = r.shardRecoveryStates().get("test").size();
final String oldMaster = internalCluster().getMasterName(); final String oldMaster = internalCluster().getMasterName();
internalCluster().stopCurrentMasterNode(); internalCluster().stopCurrentMasterNode();
@ -127,7 +131,7 @@ public class ZenDiscoveryIT extends ESIntegTestCase {
ensureSearchable("test"); ensureSearchable("test");
r = client().admin().indices().prepareRecoveries("test").get(); r = client().admin().indices().prepareRecoveries("test").get();
int numRecoveriesAfterNewMaster = r.shardResponses().get("test").size(); int numRecoveriesAfterNewMaster = r.shardRecoveryStates().get("test").size();
assertThat(numRecoveriesAfterNewMaster, equalTo(numRecoveriesBeforeNewMaster)); assertThat(numRecoveriesAfterNewMaster, equalTo(numRecoveriesBeforeNewMaster));
} }

View File

@ -35,7 +35,10 @@ import org.elasticsearch.test.ESIntegTestCase;
import org.junit.Test; import org.junit.Test;
import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertAcked; import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertAcked;
import static org.hamcrest.Matchers.*; import static org.hamcrest.Matchers.equalTo;
import static org.hamcrest.Matchers.greaterThanOrEqualTo;
import static org.hamcrest.Matchers.not;
import static org.hamcrest.Matchers.nullValue;
/** /**
*/ */
@ -146,7 +149,7 @@ public class ShardInfoIT extends ESIntegTestCase {
RecoveryResponse recoveryResponse = client().admin().indices().prepareRecoveries("idx") RecoveryResponse recoveryResponse = client().admin().indices().prepareRecoveries("idx")
.setActiveOnly(true) .setActiveOnly(true)
.get(); .get();
assertThat(recoveryResponse.shardResponses().get("idx").size(), equalTo(0)); assertThat(recoveryResponse.shardRecoveryStates().get("idx").size(), equalTo(0));
} }
}); });
} }

View File

@ -19,7 +19,6 @@
package org.elasticsearch.gateway; package org.elasticsearch.gateway;
import org.elasticsearch.action.admin.indices.recovery.RecoveryResponse; import org.elasticsearch.action.admin.indices.recovery.RecoveryResponse;
import org.elasticsearch.action.admin.indices.recovery.ShardRecoveryResponse;
import org.elasticsearch.action.count.CountResponse; import org.elasticsearch.action.count.CountResponse;
import org.elasticsearch.action.index.IndexRequestBuilder; import org.elasticsearch.action.index.IndexRequestBuilder;
import org.elasticsearch.cluster.metadata.IndexMetaData; import org.elasticsearch.cluster.metadata.IndexMetaData;
@ -99,8 +98,7 @@ public class RecoveryBackwardsCompatibilityIT extends ESBackcompatTestCase {
HashMap<String, String> map = new HashMap<>(); HashMap<String, String> map = new HashMap<>();
map.put("details", "true"); map.put("details", "true");
final ToXContent.Params params = new ToXContent.MapParams(map); final ToXContent.Params params = new ToXContent.MapParams(map);
for (ShardRecoveryResponse response : recoveryResponse.shardResponses().get("test")) { for (RecoveryState recoveryState : recoveryResponse.shardRecoveryStates().get("test")) {
RecoveryState recoveryState = response.recoveryState();
final String recoverStateAsJSON = XContentHelper.toString(recoveryState, params); final String recoverStateAsJSON = XContentHelper.toString(recoveryState, params);
if (!recoveryState.getPrimary()) { if (!recoveryState.getPrimary()) {
RecoveryState.Index index = recoveryState.getIndex(); RecoveryState.Index index = recoveryState.getIndex();

View File

@ -20,7 +20,6 @@
package org.elasticsearch.gateway; package org.elasticsearch.gateway;
import org.elasticsearch.action.admin.indices.recovery.RecoveryResponse; import org.elasticsearch.action.admin.indices.recovery.RecoveryResponse;
import org.elasticsearch.action.admin.indices.recovery.ShardRecoveryResponse;
import org.elasticsearch.action.admin.indices.stats.IndexStats; import org.elasticsearch.action.admin.indices.stats.IndexStats;
import org.elasticsearch.action.admin.indices.stats.ShardStats; import org.elasticsearch.action.admin.indices.stats.ShardStats;
import org.elasticsearch.client.Client; import org.elasticsearch.client.Client;
@ -400,8 +399,7 @@ public class RecoveryFromGatewayIT extends ESIntegTestCase {
assertSyncIdsNotNull(); assertSyncIdsNotNull();
} }
RecoveryResponse recoveryResponse = client().admin().indices().prepareRecoveries("test").get(); RecoveryResponse recoveryResponse = client().admin().indices().prepareRecoveries("test").get();
for (ShardRecoveryResponse response : recoveryResponse.shardResponses().get("test")) { for (RecoveryState recoveryState : recoveryResponse.shardRecoveryStates().get("test")) {
RecoveryState recoveryState = response.recoveryState();
long recovered = 0; long recovered = 0;
for (RecoveryState.File file : recoveryState.getIndex().fileDetails()) { for (RecoveryState.File file : recoveryState.getIndex().fileDetails()) {
if (file.name().startsWith("segments")) { if (file.name().startsWith("segments")) {
@ -410,7 +408,7 @@ public class RecoveryFromGatewayIT extends ESIntegTestCase {
} }
if (!recoveryState.getPrimary() && (useSyncIds == false)) { if (!recoveryState.getPrimary() && (useSyncIds == false)) {
logger.info("--> replica shard {} recovered from {} to {}, recovered {}, reuse {}", logger.info("--> replica shard {} recovered from {} to {}, recovered {}, reuse {}",
response.getShardId(), recoveryState.getSourceNode().name(), recoveryState.getTargetNode().name(), recoveryState.getShardId().getId(), recoveryState.getSourceNode().name(), recoveryState.getTargetNode().name(),
recoveryState.getIndex().recoveredBytes(), recoveryState.getIndex().reusedBytes()); recoveryState.getIndex().recoveredBytes(), recoveryState.getIndex().reusedBytes());
assertThat("no bytes should be recovered", recoveryState.getIndex().recoveredBytes(), equalTo(recovered)); assertThat("no bytes should be recovered", recoveryState.getIndex().recoveredBytes(), equalTo(recovered));
assertThat("data should have been reused", recoveryState.getIndex().reusedBytes(), greaterThan(0l)); assertThat("data should have been reused", recoveryState.getIndex().reusedBytes(), greaterThan(0l));
@ -422,7 +420,7 @@ public class RecoveryFromGatewayIT extends ESIntegTestCase {
} else { } else {
if (useSyncIds && !recoveryState.getPrimary()) { if (useSyncIds && !recoveryState.getPrimary()) {
logger.info("--> replica shard {} recovered from {} to {} using sync id, recovered {}, reuse {}", logger.info("--> replica shard {} recovered from {} to {} using sync id, recovered {}, reuse {}",
response.getShardId(), recoveryState.getSourceNode().name(), recoveryState.getTargetNode().name(), recoveryState.getShardId().getId(), recoveryState.getSourceNode().name(), recoveryState.getTargetNode().name(),
recoveryState.getIndex().recoveredBytes(), recoveryState.getIndex().reusedBytes()); recoveryState.getIndex().recoveredBytes(), recoveryState.getIndex().reusedBytes());
} }
assertThat(recoveryState.getIndex().recoveredBytes(), equalTo(0l)); assertThat(recoveryState.getIndex().recoveredBytes(), equalTo(0l));

View File

@ -38,6 +38,7 @@ import org.elasticsearch.index.shard.IndexShard;
import org.elasticsearch.index.shard.ShadowIndexShard; import org.elasticsearch.index.shard.ShadowIndexShard;
import org.elasticsearch.indices.IndicesService; import org.elasticsearch.indices.IndicesService;
import org.elasticsearch.indices.recovery.RecoveryTarget; import org.elasticsearch.indices.recovery.RecoveryTarget;
import org.elasticsearch.plugins.Plugin;
import org.elasticsearch.search.SearchHit; import org.elasticsearch.search.SearchHit;
import org.elasticsearch.search.sort.SortOrder; import org.elasticsearch.search.sort.SortOrder;
import org.elasticsearch.snapshots.SnapshotState; import org.elasticsearch.snapshots.SnapshotState;
@ -52,6 +53,7 @@ import org.junit.Test;
import java.io.IOException; import java.io.IOException;
import java.nio.file.Path; import java.nio.file.Path;
import java.util.Collection;
import java.util.ArrayList; import java.util.ArrayList;
import java.util.List; import java.util.List;
import java.util.concurrent.CopyOnWriteArrayList; import java.util.concurrent.CopyOnWriteArrayList;
@ -83,6 +85,11 @@ public class IndexWithShadowReplicasIT extends ESIntegTestCase {
.build(); .build();
} }
@Override
protected Collection<Class<? extends Plugin>> nodePlugins() {
return pluginList(MockTransportService.TestPlugin.class);
}
public void testCannotCreateWithBadPath() throws Exception { public void testCannotCreateWithBadPath() throws Exception {
Settings nodeSettings = nodeSettings("/badpath"); Settings nodeSettings = nodeSettings("/badpath");
internalCluster().startNodesAsync(1, nodeSettings).get(); internalCluster().startNodesAsync(1, nodeSettings).get();
@ -419,7 +426,6 @@ public class IndexWithShadowReplicasIT extends ESIntegTestCase {
Path dataPath = createTempDir(); Path dataPath = createTempDir();
Settings nodeSettings = Settings.builder() Settings nodeSettings = Settings.builder()
.put("node.add_id_to_custom_path", false) .put("node.add_id_to_custom_path", false)
.put("plugin.types", MockTransportService.TestPlugin.class.getName())
.put("path.shared_data", dataPath) .put("path.shared_data", dataPath)
.build(); .build();

View File

@ -33,12 +33,14 @@ import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.discovery.Discovery; import org.elasticsearch.discovery.Discovery;
import org.elasticsearch.discovery.DiscoverySettings; import org.elasticsearch.discovery.DiscoverySettings;
import org.elasticsearch.discovery.zen.fd.FaultDetection; import org.elasticsearch.discovery.zen.fd.FaultDetection;
import org.elasticsearch.plugins.Plugin;
import org.elasticsearch.test.ESIntegTestCase; import org.elasticsearch.test.ESIntegTestCase;
import org.elasticsearch.test.transport.MockTransportService; import org.elasticsearch.test.transport.MockTransportService;
import org.elasticsearch.transport.TransportModule; import org.elasticsearch.transport.TransportModule;
import org.elasticsearch.transport.TransportService; import org.elasticsearch.transport.TransportService;
import org.junit.Test; import org.junit.Test;
import java.util.Collection;
import java.util.List; import java.util.List;
import static org.elasticsearch.cluster.routing.ShardRoutingState.*; import static org.elasticsearch.cluster.routing.ShardRoutingState.*;
@ -57,9 +59,13 @@ public class TransportIndexFailuresIT extends ESIntegTestCase {
.put(FaultDetection.SETTING_PING_RETRIES, "1") // <-- for hitting simulated network failures quickly .put(FaultDetection.SETTING_PING_RETRIES, "1") // <-- for hitting simulated network failures quickly
.put(DiscoverySettings.PUBLISH_TIMEOUT, "1s") // <-- for hitting simulated network failures quickly .put(DiscoverySettings.PUBLISH_TIMEOUT, "1s") // <-- for hitting simulated network failures quickly
.put("discovery.zen.minimum_master_nodes", 1) .put("discovery.zen.minimum_master_nodes", 1)
.put("plugin.types", MockTransportService.TestPlugin.class.getName())
.build(); .build();
@Override
protected Collection<Class<? extends Plugin>> nodePlugins() {
return pluginList(MockTransportService.TestPlugin.class);
}
@Override @Override
protected int numberOfShards() { protected int numberOfShards() {
return 1; return 1;

View File

@ -19,12 +19,9 @@
package org.elasticsearch.index.mapper; package org.elasticsearch.index.mapper;
import org.elasticsearch.common.bytes.BytesArray;
import org.elasticsearch.common.bytes.BytesReference; import org.elasticsearch.common.bytes.BytesReference;
import org.elasticsearch.common.xcontent.XContentFactory; import org.elasticsearch.common.xcontent.XContentFactory;
import org.elasticsearch.common.xcontent.XContentParser; import org.elasticsearch.index.mapper.internal.UidFieldMapper;
import org.elasticsearch.common.xcontent.json.JsonXContent;
import org.elasticsearch.common.xcontent.json.JsonXContentParser;
import org.elasticsearch.test.ESSingleNodeTestCase; import org.elasticsearch.test.ESSingleNodeTestCase;
// TODO: make this a real unit test // TODO: make this a real unit test
@ -37,11 +34,12 @@ public class DocumentParserTests extends ESSingleNodeTestCase {
DocumentMapper mapper = mapperParser.parse(mapping); DocumentMapper mapper = mapperParser.parse(mapping);
BytesReference bytes = XContentFactory.jsonBuilder() BytesReference bytes = XContentFactory.jsonBuilder()
.startObject() .startObject().startObject("foo")
.field("field", "1234") .field("field", "1234")
.endObject().bytes(); .endObject().endObject().bytes();
ParsedDocument doc = mapper.parse("test", "type", "1", bytes); ParsedDocument doc = mapper.parse("test", "type", "1", bytes);
assertNull(doc.rootDoc().getField("field")); assertNull(doc.rootDoc().getField("field"));
assertNotNull(doc.rootDoc().getField(UidFieldMapper.NAME));
} }
public void testFieldDisabled() throws Exception { public void testFieldDisabled() throws Exception {
@ -60,5 +58,6 @@ public class DocumentParserTests extends ESSingleNodeTestCase {
ParsedDocument doc = mapper.parse("test", "type", "1", bytes); ParsedDocument doc = mapper.parse("test", "type", "1", bytes);
assertNull(doc.rootDoc().getField("foo")); assertNull(doc.rootDoc().getField("foo"));
assertNotNull(doc.rootDoc().getField("bar")); assertNotNull(doc.rootDoc().getField("bar"));
assertNotNull(doc.rootDoc().getField(UidFieldMapper.NAME));
} }
} }

View File

@ -25,19 +25,20 @@ import org.elasticsearch.common.geo.builders.ShapeBuilder;
import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.xcontent.XContentFactory; import org.elasticsearch.common.xcontent.XContentFactory;
import org.elasticsearch.index.query.QueryBuilders; import org.elasticsearch.index.query.QueryBuilders;
import org.elasticsearch.plugins.Plugin;
import org.elasticsearch.test.ESIntegTestCase; import org.elasticsearch.test.ESIntegTestCase;
import org.junit.Test; import org.junit.Test;
import java.util.Arrays;
import java.util.Collection;
import static org.hamcrest.Matchers.equalTo; import static org.hamcrest.Matchers.equalTo;
public class ExternalValuesMapperIntegrationIT extends ESIntegTestCase { public class ExternalValuesMapperIntegrationIT extends ESIntegTestCase {
@Override @Override
protected Settings nodeSettings(int nodeOrdinal) { protected Collection<Class<? extends Plugin>> nodePlugins() {
return Settings.settingsBuilder() return pluginList(ExternalMapperPlugin.class);
.put(super.nodeSettings(nodeOrdinal))
.put("plugin.types", ExternalMapperPlugin.class.getName())
.build();
} }
@Test @Test

View File

@ -27,10 +27,13 @@ import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.index.query.BoolQueryBuilder; import org.elasticsearch.index.query.BoolQueryBuilder;
import org.elasticsearch.index.query.IndexQueryParserService; import org.elasticsearch.index.query.IndexQueryParserService;
import org.elasticsearch.indices.IndicesService; import org.elasticsearch.indices.IndicesService;
import org.elasticsearch.plugins.Plugin;
import org.elasticsearch.test.ESIntegTestCase; import org.elasticsearch.test.ESIntegTestCase;
import org.junit.Before; import org.junit.Before;
import org.junit.Test; import org.junit.Test;
import java.util.Collection;
import static org.elasticsearch.index.query.QueryBuilders.boolQuery; import static org.elasticsearch.index.query.QueryBuilders.boolQuery;
import static org.elasticsearch.index.query.QueryBuilders.constantScoreQuery; import static org.elasticsearch.index.query.QueryBuilders.constantScoreQuery;
import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertHitCount; import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertHitCount;
@ -39,9 +42,8 @@ import static org.hamcrest.Matchers.instanceOf;
public class CustomQueryParserIT extends ESIntegTestCase { public class CustomQueryParserIT extends ESIntegTestCase {
@Override @Override
protected Settings nodeSettings(int nodeOrdinal) { protected Collection<Class<? extends Plugin>> nodePlugins() {
return Settings.builder().put(super.nodeSettings(nodeOrdinal)) return pluginList(DummyQueryParserPlugin.class);
.put("plugin.types", DummyQueryParserPlugin.class.getName()).build();
} }
@Before @Before

View File

@ -23,6 +23,7 @@ import org.apache.lucene.store.LockObtainFailedException;
import org.apache.lucene.util.Constants; import org.apache.lucene.util.Constants;
import org.apache.lucene.util.IOUtils; import org.apache.lucene.util.IOUtils;
import org.elasticsearch.Version; import org.elasticsearch.Version;
import org.elasticsearch.action.admin.indices.stats.CommonStats;
import org.elasticsearch.action.admin.indices.stats.CommonStatsFlags; import org.elasticsearch.action.admin.indices.stats.CommonStatsFlags;
import org.elasticsearch.action.admin.indices.stats.IndexStats; import org.elasticsearch.action.admin.indices.stats.IndexStats;
import org.elasticsearch.action.admin.indices.stats.ShardStats; import org.elasticsearch.action.admin.indices.stats.ShardStats;
@ -67,7 +68,10 @@ import java.util.HashSet;
import java.util.Set; import java.util.Set;
import java.util.concurrent.ExecutionException; import java.util.concurrent.ExecutionException;
import static org.elasticsearch.cluster.metadata.IndexMetaData.*; import static org.elasticsearch.cluster.metadata.IndexMetaData.EMPTY_PARAMS;
import static org.elasticsearch.cluster.metadata.IndexMetaData.SETTING_NUMBER_OF_REPLICAS;
import static org.elasticsearch.cluster.metadata.IndexMetaData.SETTING_NUMBER_OF_SHARDS;
import static org.elasticsearch.cluster.metadata.IndexMetaData.SETTING_VERSION_CREATED;
import static org.elasticsearch.common.settings.Settings.settingsBuilder; import static org.elasticsearch.common.settings.Settings.settingsBuilder;
import static org.elasticsearch.common.xcontent.ToXContent.EMPTY_PARAMS; import static org.elasticsearch.common.xcontent.ToXContent.EMPTY_PARAMS;
import static org.elasticsearch.index.query.QueryBuilders.matchAllQuery; import static org.elasticsearch.index.query.QueryBuilders.matchAllQuery;
@ -555,7 +559,7 @@ public class IndexShardTests extends ESSingleNodeTestCase {
IndicesService indicesService = getInstanceFromNode(IndicesService.class); IndicesService indicesService = getInstanceFromNode(IndicesService.class);
IndexService test = indicesService.indexService("test"); IndexService test = indicesService.indexService("test");
IndexShard shard = test.shard(0); IndexShard shard = test.shard(0);
ShardStats stats = new ShardStats(shard, new CommonStatsFlags()); ShardStats stats = new ShardStats(shard.routingEntry(), shard.shardPath(), new CommonStats(shard, new CommonStatsFlags()), shard.commitStats());
assertEquals(shard.shardPath().getRootDataPath().toString(), stats.getDataPath()); assertEquals(shard.shardPath().getRootDataPath().toString(), stats.getDataPath());
assertEquals(shard.shardPath().getRootStatePath().toString(), stats.getStatePath()); assertEquals(shard.shardPath().getRootStatePath().toString(), stats.getStatePath());
assertEquals(shard.shardPath().isCustomDataPath(), stats.isCustomDataPath()); assertEquals(shard.shardPath().isCustomDataPath(), stats.isCustomDataPath());

View File

@ -0,0 +1,236 @@
/*
* 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.index.shard;
import com.carrotsearch.randomizedtesting.annotations.Repeat;
import org.apache.lucene.mockfile.FilterFileSystem;
import org.apache.lucene.mockfile.FilterFileSystemProvider;
import org.apache.lucene.mockfile.FilterPath;
import org.apache.lucene.util.Constants;
import org.elasticsearch.cluster.metadata.IndexMetaData;
import org.elasticsearch.common.SuppressForbidden;
import org.elasticsearch.common.io.PathUtils;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.env.Environment;
import org.elasticsearch.env.NodeEnvironment.NodePath;
import org.elasticsearch.env.NodeEnvironment;
import org.elasticsearch.test.ESTestCase;
import org.junit.AfterClass;
import org.junit.BeforeClass;
import org.junit.Test;
import java.io.File;
import java.io.IOException;
import java.lang.reflect.Field;
import java.nio.file.FileStore;
import java.nio.file.FileSystem;
import java.nio.file.FileSystems;
import java.nio.file.Files;
import java.nio.file.Path;
import java.nio.file.attribute.FileAttributeView;
import java.nio.file.attribute.FileStoreAttributeView;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collections;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.Set;
import static org.elasticsearch.common.settings.Settings.settingsBuilder;
/** Separate test class from ShardPathTests because we need static (BeforeClass) setup to install mock filesystems... */
@SuppressForbidden(reason = "ProviderMismatchException if I try to use PathUtils.getDefault instead")
public class NewPathForShardTest extends ESTestCase {
// Sneakiness to install mock file stores so we can pretend how much free space we have on each path.data:
private static MockFileStore aFileStore = new MockFileStore("mocka");
private static MockFileStore bFileStore = new MockFileStore("mockb");
private static FileSystem origFileSystem;
private static String aPathPart = File.separator + 'a' + File.separator;
private static String bPathPart = File.separator + 'b' + File.separator;
@BeforeClass
public static void installMockUsableSpaceFS() throws Exception {
// Necessary so when Environment.clinit runs, to gather all FileStores, it sees ours:
origFileSystem = FileSystems.getDefault();
Field field = PathUtils.class.getDeclaredField("DEFAULT");
field.setAccessible(true);
FileSystem mock = new MockUsableSpaceFileSystemProvider().getFileSystem(getBaseTempDirForTestClass().toUri());
field.set(null, mock);
assertEquals(mock, PathUtils.getDefaultFileSystem());
}
@AfterClass
public static void removeMockUsableSpaceFS() throws Exception {
Field field = PathUtils.class.getDeclaredField("DEFAULT");
field.setAccessible(true);
field.set(null, origFileSystem);
origFileSystem = null;
aFileStore = null;
bFileStore = null;
}
/** Mock file system that fakes usable space for each FileStore */
@SuppressForbidden(reason = "ProviderMismatchException if I try to use PathUtils.getDefault instead")
static class MockUsableSpaceFileSystemProvider extends FilterFileSystemProvider {
public MockUsableSpaceFileSystemProvider() {
super("mockusablespace://", FileSystems.getDefault());
final List<FileStore> fileStores = new ArrayList<>();
fileStores.add(aFileStore);
fileStores.add(bFileStore);
fileSystem = new FilterFileSystem(this, origFileSystem) {
@Override
public Iterable<FileStore> getFileStores() {
return fileStores;
}
};
}
@Override
public FileStore getFileStore(Path path) throws IOException {
if (path.toString().contains(aPathPart)) {
return aFileStore;
} else {
return bFileStore;
}
}
}
static class MockFileStore extends FileStore {
public long usableSpace;
private final String desc;
public MockFileStore(String desc) {
this.desc = desc;
}
@Override
public String type() {
return "mock";
}
@Override
public String name() {
return desc;
}
@Override
public String toString() {
return desc;
}
@Override
public boolean isReadOnly() {
return false;
}
@Override
public long getTotalSpace() throws IOException {
return usableSpace*3;
}
@Override
public long getUsableSpace() throws IOException {
return usableSpace;
}
@Override
public long getUnallocatedSpace() throws IOException {
return usableSpace*2;
}
@Override
public boolean supportsFileAttributeView(Class<? extends FileAttributeView> type) {
return false;
}
@Override
public boolean supportsFileAttributeView(String name) {
return false;
}
@Override
public <V extends FileStoreAttributeView> V getFileStoreAttributeView(Class<V> type) {
return null;
}
@Override
public Object getAttribute(String attribute) throws IOException {
return null;
}
}
public void testSelectNewPathForShard() throws Exception {
assumeFalse("Consistenty fails on windows ('could not remove the following files')", Constants.WINDOWS);
Path path = PathUtils.get(createTempDir().toString());
// Use 2 data paths:
String[] paths = new String[] {path.resolve("a").toString(),
path.resolve("b").toString()};
Settings settings = Settings.builder()
.put("path.home", path)
.putArray("path.data", paths).build();
NodeEnvironment nodeEnv = new NodeEnvironment(settings, new Environment(settings));
// Make sure all our mocking above actually worked:
NodePath[] nodePaths = nodeEnv.nodePaths();
assertEquals(2, nodePaths.length);
assertEquals("mocka", nodePaths[0].fileStore.name());
assertEquals("mockb", nodePaths[1].fileStore.name());
// Path a has lots of free space, but b has little, so new shard should go to a:
aFileStore.usableSpace = 100000;
bFileStore.usableSpace = 1000;
ShardId shardId = new ShardId("index", 0);
ShardPath result = ShardPath.selectNewPathForShard(nodeEnv, shardId, Settings.EMPTY, 100, Collections.<Path,Integer>emptyMap());
assertTrue(result.getDataPath().toString().contains(aPathPart));
// Test the reverse: b has lots of free space, but a has little, so new shard should go to b:
aFileStore.usableSpace = 1000;
bFileStore.usableSpace = 100000;
shardId = new ShardId("index", 0);
result = ShardPath.selectNewPathForShard(nodeEnv, shardId, Settings.EMPTY, 100, Collections.<Path,Integer>emptyMap());
assertTrue(result.getDataPath().toString().contains(bPathPart));
// Now a and be have equal usable space; we allocate two shards to the node, and each should go to different paths:
aFileStore.usableSpace = 100000;
bFileStore.usableSpace = 100000;
Map<Path,Integer> dataPathToShardCount = new HashMap<>();
ShardPath result1 = ShardPath.selectNewPathForShard(nodeEnv, shardId, Settings.EMPTY, 100, dataPathToShardCount);
dataPathToShardCount.put(NodeEnvironment.shardStatePathToDataPath(result1.getDataPath()), 1);
ShardPath result2 = ShardPath.selectNewPathForShard(nodeEnv, shardId, Settings.EMPTY, 100, dataPathToShardCount);
// #11122: this was the original failure: on a node with 2 disks that have nearly equal
// free space, we would always allocate all N incoming shards to the one path that
// had the most free space, never using the other drive unless new shards arrive
// after the first shards started using storage:
assertNotEquals(result1.getDataPath(), result2.getDataPath());
}
}

View File

@ -70,6 +70,7 @@ import org.elasticsearch.indices.recovery.RecoveryFileChunkRequest;
import org.elasticsearch.indices.recovery.RecoverySettings; import org.elasticsearch.indices.recovery.RecoverySettings;
import org.elasticsearch.indices.recovery.RecoveryTarget; import org.elasticsearch.indices.recovery.RecoveryTarget;
import org.elasticsearch.monitor.fs.FsInfo; import org.elasticsearch.monitor.fs.FsInfo;
import org.elasticsearch.plugins.Plugin;
import org.elasticsearch.snapshots.SnapshotState; import org.elasticsearch.snapshots.SnapshotState;
import org.elasticsearch.test.ESIntegTestCase; import org.elasticsearch.test.ESIntegTestCase;
import org.elasticsearch.test.InternalTestCluster; import org.elasticsearch.test.InternalTestCluster;
@ -91,6 +92,7 @@ import java.nio.file.Files;
import java.nio.file.Path; import java.nio.file.Path;
import java.nio.file.StandardOpenOption; import java.nio.file.StandardOpenOption;
import java.util.ArrayList; import java.util.ArrayList;
import java.util.Collection;
import java.util.Collections; import java.util.Collections;
import java.util.HashMap; import java.util.HashMap;
import java.util.List; import java.util.List;
@ -117,7 +119,6 @@ public class CorruptedFileIT extends ESIntegTestCase {
// we really need local GW here since this also checks for corruption etc. // we really need local GW here since this also checks for corruption etc.
// and we need to make sure primaries are not just trashed if we don't have replicas // and we need to make sure primaries are not just trashed if we don't have replicas
.put(super.nodeSettings(nodeOrdinal)) .put(super.nodeSettings(nodeOrdinal))
.extendArray("plugin.types", MockTransportService.TestPlugin.class.getName())
// speed up recoveries // speed up recoveries
.put(RecoverySettings.INDICES_RECOVERY_CONCURRENT_STREAMS, 10) .put(RecoverySettings.INDICES_RECOVERY_CONCURRENT_STREAMS, 10)
.put(RecoverySettings.INDICES_RECOVERY_CONCURRENT_SMALL_FILE_STREAMS, 10) .put(RecoverySettings.INDICES_RECOVERY_CONCURRENT_SMALL_FILE_STREAMS, 10)
@ -125,6 +126,11 @@ public class CorruptedFileIT extends ESIntegTestCase {
.build(); .build();
} }
@Override
protected Collection<Class<? extends Plugin>> nodePlugins() {
return pluginList(MockTransportService.TestPlugin.class);
}
/** /**
* Tests that we can actually recover from a corruption on the primary given that we have replica shards around. * Tests that we can actually recover from a corruption on the primary given that we have replica shards around.
*/ */

View File

@ -32,6 +32,7 @@ import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.index.shard.IndexShard; import org.elasticsearch.index.shard.IndexShard;
import org.elasticsearch.index.translog.TranslogConfig; import org.elasticsearch.index.translog.TranslogConfig;
import org.elasticsearch.monitor.fs.FsInfo; import org.elasticsearch.monitor.fs.FsInfo;
import org.elasticsearch.plugins.Plugin;
import org.elasticsearch.test.ESIntegTestCase; import org.elasticsearch.test.ESIntegTestCase;
import org.elasticsearch.test.engine.MockEngineSupport; import org.elasticsearch.test.engine.MockEngineSupport;
import org.elasticsearch.test.junit.annotations.TestLogging; import org.elasticsearch.test.junit.annotations.TestLogging;
@ -46,6 +47,7 @@ import java.nio.file.Files;
import java.nio.file.Path; import java.nio.file.Path;
import java.nio.file.StandardOpenOption; import java.nio.file.StandardOpenOption;
import java.util.Arrays; import java.util.Arrays;
import java.util.Collection;
import java.util.List; import java.util.List;
import java.util.Set; import java.util.Set;
import java.util.TreeSet; import java.util.TreeSet;
@ -62,12 +64,10 @@ import static org.hamcrest.Matchers.notNullValue;
public class CorruptedTranslogIT extends ESIntegTestCase { public class CorruptedTranslogIT extends ESIntegTestCase {
@Override @Override
protected Settings nodeSettings(int nodeOrdinal) { protected Collection<Class<? extends Plugin>> nodePlugins() {
return Settings.builder() // we really need local GW here since this also checks for corruption etc.
// we really need local GW here since this also checks for corruption etc. // and we need to make sure primaries are not just trashed if we don't have replicas
// and we need to make sure primaries are not just trashed if we don't have replicas return pluginList(MockTransportService.TestPlugin.class);
.put(super.nodeSettings(nodeOrdinal))
.extendArray("plugin.types", MockTransportService.TestPlugin.class.getName()).build();
} }
@Test @Test

View File

@ -29,6 +29,7 @@ import org.elasticsearch.cluster.node.DiscoveryNode;
import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.xcontent.XContentBuilder; import org.elasticsearch.common.xcontent.XContentBuilder;
import org.elasticsearch.discovery.Discovery; import org.elasticsearch.discovery.Discovery;
import org.elasticsearch.plugins.Plugin;
import org.elasticsearch.search.SearchHit; import org.elasticsearch.search.SearchHit;
import org.elasticsearch.test.ESIntegTestCase; import org.elasticsearch.test.ESIntegTestCase;
import org.elasticsearch.test.transport.MockTransportService; import org.elasticsearch.test.transport.MockTransportService;
@ -36,6 +37,7 @@ import org.elasticsearch.transport.*;
import org.junit.Test; import org.junit.Test;
import java.io.IOException; import java.io.IOException;
import java.util.Collection;
import java.util.HashSet; import java.util.HashSet;
import java.util.Set; import java.util.Set;
import java.util.concurrent.ExecutionException; import java.util.concurrent.ExecutionException;
@ -51,11 +53,8 @@ import static org.hamcrest.Matchers.greaterThan;
public class ExceptionRetryIT extends ESIntegTestCase { public class ExceptionRetryIT extends ESIntegTestCase {
@Override @Override
protected Settings nodeSettings(int nodeOrdinal) { protected Collection<Class<? extends Plugin>> nodePlugins() {
return Settings.builder() return pluginList(MockTransportService.TestPlugin.class);
.put(super.nodeSettings(nodeOrdinal))
.extendArray("plugin.types", MockTransportService.TestPlugin.class.getName())
.build();
} }
@Override @Override

View File

@ -24,11 +24,13 @@ import org.apache.lucene.analysis.Analyzer;
import org.elasticsearch.Version; import org.elasticsearch.Version;
import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.xcontent.XContentBuilder; import org.elasticsearch.common.xcontent.XContentBuilder;
import org.elasticsearch.plugins.Plugin;
import org.elasticsearch.test.ESBackcompatTestCase; import org.elasticsearch.test.ESBackcompatTestCase;
import org.elasticsearch.test.ESIntegTestCase; import org.elasticsearch.test.ESIntegTestCase;
import org.junit.Test; import org.junit.Test;
import java.lang.reflect.Field; import java.lang.reflect.Field;
import java.util.Collection;
import java.util.ArrayList; import java.util.ArrayList;
import java.util.List; import java.util.List;
import java.util.Locale; import java.util.Locale;
@ -44,11 +46,8 @@ import static org.hamcrest.Matchers.notNullValue;
public class PreBuiltAnalyzerIntegrationIT extends ESIntegTestCase { public class PreBuiltAnalyzerIntegrationIT extends ESIntegTestCase {
@Override @Override
protected Settings nodeSettings(int nodeOrdinal) { protected Collection<Class<? extends Plugin>> nodePlugins() {
return Settings.settingsBuilder() return pluginList(DummyAnalysisPlugin.class);
.put(super.nodeSettings(nodeOrdinal))
.put("plugin.types", DummyAnalysisPlugin.class.getName())
.build();
} }
@Test @Test

View File

@ -45,6 +45,7 @@ import org.junit.Test;
import java.io.IOException; import java.io.IOException;
import java.util.Arrays; import java.util.Arrays;
import java.util.Collection;
import java.util.Random; import java.util.Random;
import java.util.concurrent.ExecutionException; import java.util.concurrent.ExecutionException;
@ -57,6 +58,11 @@ import static org.hamcrest.Matchers.equalTo;
*/ */
public class RandomExceptionCircuitBreakerIT extends ESIntegTestCase { public class RandomExceptionCircuitBreakerIT extends ESIntegTestCase {
@Override
protected Collection<Class<? extends Plugin>> nodePlugins() {
return pluginList(RandomExceptionDirectoryReaderWrapper.TestPlugin.class);
}
@Test @Test
public void testBreakerWithRandomExceptions() throws IOException, InterruptedException, ExecutionException { public void testBreakerWithRandomExceptions() throws IOException, InterruptedException, ExecutionException {
for (NodeStats node : client().admin().cluster().prepareNodesStats() for (NodeStats node : client().admin().cluster().prepareNodesStats()
@ -107,7 +113,6 @@ public class RandomExceptionCircuitBreakerIT extends ESIntegTestCase {
Settings.Builder settings = settingsBuilder() Settings.Builder settings = settingsBuilder()
.put(indexSettings()) .put(indexSettings())
.extendArray("plugin.types", RandomExceptionDirectoryReaderWrapper.TestPlugin.class.getName())
.put(EXCEPTION_TOP_LEVEL_RATIO_KEY, topLevelRate) .put(EXCEPTION_TOP_LEVEL_RATIO_KEY, topLevelRate)
.put(EXCEPTION_LOW_LEVEL_RATIO_KEY, lowLevelRate) .put(EXCEPTION_LOW_LEVEL_RATIO_KEY, lowLevelRate)
.put(MockEngineSupport.WRAP_READER_RATIO, 1.0d); .put(MockEngineSupport.WRAP_READER_RATIO, 1.0d);

View File

@ -27,7 +27,6 @@ import org.elasticsearch.action.admin.cluster.snapshots.create.CreateSnapshotRes
import org.elasticsearch.action.admin.cluster.snapshots.restore.RestoreSnapshotResponse; import org.elasticsearch.action.admin.cluster.snapshots.restore.RestoreSnapshotResponse;
import org.elasticsearch.action.admin.cluster.state.ClusterStateResponse; import org.elasticsearch.action.admin.cluster.state.ClusterStateResponse;
import org.elasticsearch.action.admin.indices.recovery.RecoveryResponse; import org.elasticsearch.action.admin.indices.recovery.RecoveryResponse;
import org.elasticsearch.action.admin.indices.recovery.ShardRecoveryResponse;
import org.elasticsearch.action.admin.indices.stats.CommonStatsFlags; import org.elasticsearch.action.admin.indices.stats.CommonStatsFlags;
import org.elasticsearch.action.admin.indices.stats.IndicesStatsResponse; import org.elasticsearch.action.admin.indices.stats.IndicesStatsResponse;
import org.elasticsearch.action.index.IndexRequestBuilder; import org.elasticsearch.action.index.IndexRequestBuilder;
@ -47,17 +46,24 @@ import org.elasticsearch.index.store.Store;
import org.elasticsearch.indices.IndicesService; import org.elasticsearch.indices.IndicesService;
import org.elasticsearch.indices.recovery.RecoveryState.Stage; import org.elasticsearch.indices.recovery.RecoveryState.Stage;
import org.elasticsearch.indices.recovery.RecoveryState.Type; import org.elasticsearch.indices.recovery.RecoveryState.Type;
import org.elasticsearch.plugins.Plugin;
import org.elasticsearch.snapshots.SnapshotState; import org.elasticsearch.snapshots.SnapshotState;
import org.elasticsearch.test.ESIntegTestCase; import org.elasticsearch.test.ESIntegTestCase;
import org.elasticsearch.test.ESIntegTestCase.ClusterScope; import org.elasticsearch.test.ESIntegTestCase.ClusterScope;
import org.elasticsearch.test.junit.annotations.TestLogging; import org.elasticsearch.test.junit.annotations.TestLogging;
import org.elasticsearch.test.store.MockFSDirectoryService; import org.elasticsearch.test.store.MockFSDirectoryService;
import org.elasticsearch.test.transport.MockTransportService; import org.elasticsearch.test.transport.MockTransportService;
import org.elasticsearch.transport.*; import org.elasticsearch.transport.ConnectTransportException;
import org.elasticsearch.transport.Transport;
import org.elasticsearch.transport.TransportException;
import org.elasticsearch.transport.TransportRequest;
import org.elasticsearch.transport.TransportRequestOptions;
import org.elasticsearch.transport.TransportService;
import org.junit.Test; import org.junit.Test;
import java.io.IOException; import java.io.IOException;
import java.util.ArrayList; import java.util.ArrayList;
import java.util.Collection;
import java.util.List; import java.util.List;
import java.util.Map; import java.util.Map;
import java.util.concurrent.CountDownLatch; import java.util.concurrent.CountDownLatch;
@ -67,7 +73,13 @@ import static org.elasticsearch.common.settings.Settings.settingsBuilder;
import static org.elasticsearch.test.ESIntegTestCase.Scope; import static org.elasticsearch.test.ESIntegTestCase.Scope;
import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertAcked; import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertAcked;
import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertHitCount; import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertHitCount;
import static org.hamcrest.Matchers.*; import static org.hamcrest.Matchers.arrayWithSize;
import static org.hamcrest.Matchers.equalTo;
import static org.hamcrest.Matchers.greaterThan;
import static org.hamcrest.Matchers.greaterThanOrEqualTo;
import static org.hamcrest.Matchers.is;
import static org.hamcrest.Matchers.lessThanOrEqualTo;
import static org.hamcrest.Matchers.not;
/** /**
* *
@ -85,6 +97,10 @@ public class IndexRecoveryIT extends ESIntegTestCase {
private static final int SHARD_COUNT = 1; private static final int SHARD_COUNT = 1;
private static final int REPLICA_COUNT = 0; private static final int REPLICA_COUNT = 0;
@Override
protected Collection<Class<? extends Plugin>> nodePlugins() {
return pluginList(MockTransportService.TestPlugin.class);
}
private void assertRecoveryStateWithoutStage(RecoveryState state, int shardId, Type type, private void assertRecoveryStateWithoutStage(RecoveryState state, int shardId, Type type,
String sourceNode, String targetNode, boolean hasRestoreSource) { String sourceNode, String targetNode, boolean hasRestoreSource) {
@ -155,18 +171,17 @@ public class IndexRecoveryIT extends ESIntegTestCase {
logger.info("--> request recoveries"); logger.info("--> request recoveries");
RecoveryResponse response = client().admin().indices().prepareRecoveries(INDEX_NAME).execute().actionGet(); RecoveryResponse response = client().admin().indices().prepareRecoveries(INDEX_NAME).execute().actionGet();
assertThat(response.shardResponses().size(), equalTo(SHARD_COUNT)); assertThat(response.shardRecoveryStates().size(), equalTo(SHARD_COUNT));
assertThat(response.shardResponses().get(INDEX_NAME).size(), equalTo(1)); assertThat(response.shardRecoveryStates().get(INDEX_NAME).size(), equalTo(1));
List<ShardRecoveryResponse> shardResponses = response.shardResponses().get(INDEX_NAME); List<RecoveryState> recoveryStates = response.shardRecoveryStates().get(INDEX_NAME);
assertThat(shardResponses.size(), equalTo(1)); assertThat(recoveryStates.size(), equalTo(1));
ShardRecoveryResponse shardResponse = shardResponses.get(0); RecoveryState recoveryState = recoveryStates.get(0);
RecoveryState state = shardResponse.recoveryState();
assertRecoveryState(state, 0, Type.STORE, Stage.DONE, node, node, false); assertRecoveryState(recoveryState, 0, Type.STORE, Stage.DONE, node, node, false);
validateIndexRecoveryState(state.getIndex()); validateIndexRecoveryState(recoveryState.getIndex());
} }
@Test @Test
@ -183,8 +198,8 @@ public class IndexRecoveryIT extends ESIntegTestCase {
logger.info("--> request recoveries"); logger.info("--> request recoveries");
RecoveryResponse response = client().admin().indices().prepareRecoveries(INDEX_NAME).setActiveOnly(true).execute().actionGet(); RecoveryResponse response = client().admin().indices().prepareRecoveries(INDEX_NAME).setActiveOnly(true).execute().actionGet();
List<ShardRecoveryResponse> shardResponses = response.shardResponses().get(INDEX_NAME); List<RecoveryState> recoveryStates = response.shardRecoveryStates().get(INDEX_NAME);
assertThat(shardResponses.size(), equalTo(0)); // Should not expect any responses back assertThat(recoveryStates.size(), equalTo(0)); // Should not expect any responses back
} }
@Test @Test
@ -209,23 +224,23 @@ public class IndexRecoveryIT extends ESIntegTestCase {
RecoveryResponse response = client().admin().indices().prepareRecoveries(INDEX_NAME).execute().actionGet(); RecoveryResponse response = client().admin().indices().prepareRecoveries(INDEX_NAME).execute().actionGet();
// we should now have two total shards, one primary and one replica // we should now have two total shards, one primary and one replica
List<ShardRecoveryResponse> shardResponses = response.shardResponses().get(INDEX_NAME); List<RecoveryState> recoveryStates = response.shardRecoveryStates().get(INDEX_NAME);
assertThat(shardResponses.size(), equalTo(2)); assertThat(recoveryStates.size(), equalTo(2));
List<ShardRecoveryResponse> nodeAResponses = findRecoveriesForTargetNode(nodeA, shardResponses); List<RecoveryState> nodeAResponses = findRecoveriesForTargetNode(nodeA, recoveryStates);
assertThat(nodeAResponses.size(), equalTo(1)); assertThat(nodeAResponses.size(), equalTo(1));
List<ShardRecoveryResponse> nodeBResponses = findRecoveriesForTargetNode(nodeB, shardResponses); List<RecoveryState> nodeBResponses = findRecoveriesForTargetNode(nodeB, recoveryStates);
assertThat(nodeBResponses.size(), equalTo(1)); assertThat(nodeBResponses.size(), equalTo(1));
// validate node A recovery // validate node A recovery
ShardRecoveryResponse nodeAShardResponse = nodeAResponses.get(0); RecoveryState nodeARecoveryState = nodeAResponses.get(0);
assertRecoveryState(nodeAShardResponse.recoveryState(), 0, Type.STORE, Stage.DONE, nodeA, nodeA, false); assertRecoveryState(nodeARecoveryState, 0, Type.STORE, Stage.DONE, nodeA, nodeA, false);
validateIndexRecoveryState(nodeAShardResponse.recoveryState().getIndex()); validateIndexRecoveryState(nodeARecoveryState.getIndex());
// validate node B recovery // validate node B recovery
ShardRecoveryResponse nodeBShardResponse = nodeBResponses.get(0); RecoveryState nodeBRecoveryState = nodeBResponses.get(0);
assertRecoveryState(nodeBShardResponse.recoveryState(), 0, Type.REPLICA, Stage.DONE, nodeA, nodeB, false); assertRecoveryState(nodeBRecoveryState, 0, Type.REPLICA, Stage.DONE, nodeA, nodeB, false);
validateIndexRecoveryState(nodeBShardResponse.recoveryState().getIndex()); validateIndexRecoveryState(nodeBRecoveryState.getIndex());
} }
@Test @Test
@ -266,17 +281,17 @@ public class IndexRecoveryIT extends ESIntegTestCase {
logger.info("--> request recoveries"); logger.info("--> request recoveries");
RecoveryResponse response = client().admin().indices().prepareRecoveries(INDEX_NAME).execute().actionGet(); RecoveryResponse response = client().admin().indices().prepareRecoveries(INDEX_NAME).execute().actionGet();
List<ShardRecoveryResponse> shardResponses = response.shardResponses().get(INDEX_NAME); List<RecoveryState> recoveryStates = response.shardRecoveryStates().get(INDEX_NAME);
List<ShardRecoveryResponse> nodeAResponses = findRecoveriesForTargetNode(nodeA, shardResponses); List<RecoveryState> nodeARecoveryStates = findRecoveriesForTargetNode(nodeA, recoveryStates);
assertThat(nodeAResponses.size(), equalTo(1)); assertThat(nodeARecoveryStates.size(), equalTo(1));
List<ShardRecoveryResponse> nodeBResponses = findRecoveriesForTargetNode(nodeB, shardResponses); List<RecoveryState> nodeBRecoveryStates = findRecoveriesForTargetNode(nodeB, recoveryStates);
assertThat(nodeBResponses.size(), equalTo(1)); assertThat(nodeBRecoveryStates.size(), equalTo(1));
assertRecoveryState(nodeAResponses.get(0).recoveryState(), 0, Type.STORE, Stage.DONE, nodeA, nodeA, false); assertRecoveryState(nodeARecoveryStates.get(0), 0, Type.STORE, Stage.DONE, nodeA, nodeA, false);
validateIndexRecoveryState(nodeAResponses.get(0).recoveryState().getIndex()); validateIndexRecoveryState(nodeARecoveryStates.get(0).getIndex());
assertOnGoingRecoveryState(nodeBResponses.get(0).recoveryState(), 0, Type.RELOCATION, nodeA, nodeB, false); assertOnGoingRecoveryState(nodeBRecoveryStates.get(0), 0, Type.RELOCATION, nodeA, nodeB, false);
validateIndexRecoveryState(nodeBResponses.get(0).recoveryState().getIndex()); validateIndexRecoveryState(nodeBRecoveryStates.get(0).getIndex());
logger.info("--> request node recovery stats"); logger.info("--> request node recovery stats");
NodesStatsResponse statsResponse = client().admin().cluster().prepareNodesStats().clear().setIndices(new CommonStatsFlags(CommonStatsFlags.Flag.Recovery)).get(); NodesStatsResponse statsResponse = client().admin().cluster().prepareNodesStats().clear().setIndices(new CommonStatsFlags(CommonStatsFlags.Flag.Recovery)).get();
@ -325,11 +340,11 @@ public class IndexRecoveryIT extends ESIntegTestCase {
response = client().admin().indices().prepareRecoveries(INDEX_NAME).execute().actionGet(); response = client().admin().indices().prepareRecoveries(INDEX_NAME).execute().actionGet();
shardResponses = response.shardResponses().get(INDEX_NAME); recoveryStates = response.shardRecoveryStates().get(INDEX_NAME);
assertThat(shardResponses.size(), equalTo(1)); assertThat(recoveryStates.size(), equalTo(1));
assertRecoveryState(shardResponses.get(0).recoveryState(), 0, Type.RELOCATION, Stage.DONE, nodeA, nodeB, false); assertRecoveryState(recoveryStates.get(0), 0, Type.RELOCATION, Stage.DONE, nodeA, nodeB, false);
validateIndexRecoveryState(shardResponses.get(0).recoveryState().getIndex()); validateIndexRecoveryState(recoveryStates.get(0).getIndex());
statsResponse = client().admin().cluster().prepareNodesStats().clear().setIndices(new CommonStatsFlags(CommonStatsFlags.Flag.Recovery)).get(); statsResponse = client().admin().cluster().prepareNodesStats().clear().setIndices(new CommonStatsFlags(CommonStatsFlags.Flag.Recovery)).get();
assertThat(statsResponse.getNodes(), arrayWithSize(2)); assertThat(statsResponse.getNodes(), arrayWithSize(2));
@ -377,45 +392,45 @@ public class IndexRecoveryIT extends ESIntegTestCase {
.execute().actionGet().getState(); .execute().actionGet().getState();
response = client().admin().indices().prepareRecoveries(INDEX_NAME).execute().actionGet(); response = client().admin().indices().prepareRecoveries(INDEX_NAME).execute().actionGet();
shardResponses = response.shardResponses().get(INDEX_NAME); recoveryStates = response.shardRecoveryStates().get(INDEX_NAME);
nodeAResponses = findRecoveriesForTargetNode(nodeA, shardResponses); nodeARecoveryStates = findRecoveriesForTargetNode(nodeA, recoveryStates);
assertThat(nodeAResponses.size(), equalTo(1)); assertThat(nodeARecoveryStates.size(), equalTo(1));
nodeBResponses = findRecoveriesForTargetNode(nodeB, shardResponses); nodeBRecoveryStates = findRecoveriesForTargetNode(nodeB, recoveryStates);
assertThat(nodeBResponses.size(), equalTo(1)); assertThat(nodeBRecoveryStates.size(), equalTo(1));
List<ShardRecoveryResponse> nodeCResponses = findRecoveriesForTargetNode(nodeC, shardResponses); List<RecoveryState> nodeCRecoveryStates = findRecoveriesForTargetNode(nodeC, recoveryStates);
assertThat(nodeCResponses.size(), equalTo(1)); assertThat(nodeCRecoveryStates.size(), equalTo(1));
assertRecoveryState(nodeAResponses.get(0).recoveryState(), 0, Type.REPLICA, Stage.DONE, nodeB, nodeA, false); assertRecoveryState(nodeARecoveryStates.get(0), 0, Type.REPLICA, Stage.DONE, nodeB, nodeA, false);
validateIndexRecoveryState(nodeAResponses.get(0).recoveryState().getIndex()); validateIndexRecoveryState(nodeARecoveryStates.get(0).getIndex());
assertRecoveryState(nodeBResponses.get(0).recoveryState(), 0, Type.RELOCATION, Stage.DONE, nodeA, nodeB, false); assertRecoveryState(nodeBRecoveryStates.get(0), 0, Type.RELOCATION, Stage.DONE, nodeA, nodeB, false);
validateIndexRecoveryState(nodeBResponses.get(0).recoveryState().getIndex()); validateIndexRecoveryState(nodeBRecoveryStates.get(0).getIndex());
// relocations of replicas are marked as REPLICA and the source node is the node holding the primary (B) // relocations of replicas are marked as REPLICA and the source node is the node holding the primary (B)
assertOnGoingRecoveryState(nodeCResponses.get(0).recoveryState(), 0, Type.REPLICA, nodeB, nodeC, false); assertOnGoingRecoveryState(nodeCRecoveryStates.get(0), 0, Type.REPLICA, nodeB, nodeC, false);
validateIndexRecoveryState(nodeCResponses.get(0).recoveryState().getIndex()); validateIndexRecoveryState(nodeCRecoveryStates.get(0).getIndex());
logger.info("--> speeding up recoveries"); logger.info("--> speeding up recoveries");
restoreRecoverySpeed(); restoreRecoverySpeed();
ensureGreen(); ensureGreen();
response = client().admin().indices().prepareRecoveries(INDEX_NAME).execute().actionGet(); response = client().admin().indices().prepareRecoveries(INDEX_NAME).execute().actionGet();
shardResponses = response.shardResponses().get(INDEX_NAME); recoveryStates = response.shardRecoveryStates().get(INDEX_NAME);
nodeAResponses = findRecoveriesForTargetNode(nodeA, shardResponses); nodeARecoveryStates = findRecoveriesForTargetNode(nodeA, recoveryStates);
assertThat(nodeAResponses.size(), equalTo(0)); assertThat(nodeARecoveryStates.size(), equalTo(0));
nodeBResponses = findRecoveriesForTargetNode(nodeB, shardResponses); nodeBRecoveryStates = findRecoveriesForTargetNode(nodeB, recoveryStates);
assertThat(nodeBResponses.size(), equalTo(1)); assertThat(nodeBRecoveryStates.size(), equalTo(1));
nodeCResponses = findRecoveriesForTargetNode(nodeC, shardResponses); nodeCRecoveryStates = findRecoveriesForTargetNode(nodeC, recoveryStates);
assertThat(nodeCResponses.size(), equalTo(1)); assertThat(nodeCRecoveryStates.size(), equalTo(1));
assertRecoveryState(nodeBResponses.get(0).recoveryState(), 0, Type.RELOCATION, Stage.DONE, nodeA, nodeB, false); assertRecoveryState(nodeBRecoveryStates.get(0), 0, Type.RELOCATION, Stage.DONE, nodeA, nodeB, false);
validateIndexRecoveryState(nodeBResponses.get(0).recoveryState().getIndex()); validateIndexRecoveryState(nodeBRecoveryStates.get(0).getIndex());
// relocations of replicas are marked as REPLICA and the source node is the node holding the primary (B) // relocations of replicas are marked as REPLICA and the source node is the node holding the primary (B)
assertRecoveryState(nodeCResponses.get(0).recoveryState(), 0, Type.REPLICA, Stage.DONE, nodeB, nodeC, false); assertRecoveryState(nodeCRecoveryStates.get(0), 0, Type.REPLICA, Stage.DONE, nodeB, nodeC, false);
validateIndexRecoveryState(nodeCResponses.get(0).recoveryState().getIndex()); validateIndexRecoveryState(nodeCRecoveryStates.get(0).getIndex());
} }
@Test @Test
@ -457,24 +472,24 @@ public class IndexRecoveryIT extends ESIntegTestCase {
logger.info("--> request recoveries"); logger.info("--> request recoveries");
RecoveryResponse response = client().admin().indices().prepareRecoveries(INDEX_NAME).execute().actionGet(); RecoveryResponse response = client().admin().indices().prepareRecoveries(INDEX_NAME).execute().actionGet();
for (Map.Entry<String, List<ShardRecoveryResponse>> shardRecoveryResponse : response.shardResponses().entrySet()) { for (Map.Entry<String, List<RecoveryState>> indexRecoveryStates : response.shardRecoveryStates().entrySet()) {
assertThat(shardRecoveryResponse.getKey(), equalTo(INDEX_NAME)); assertThat(indexRecoveryStates.getKey(), equalTo(INDEX_NAME));
List<ShardRecoveryResponse> shardRecoveryResponses = shardRecoveryResponse.getValue(); List<RecoveryState> recoveryStates = indexRecoveryStates.getValue();
assertThat(shardRecoveryResponses.size(), equalTo(totalShards)); assertThat(recoveryStates.size(), equalTo(totalShards));
for (ShardRecoveryResponse shardResponse : shardRecoveryResponses) { for (RecoveryState recoveryState : recoveryStates) {
assertRecoveryState(shardResponse.recoveryState(), 0, Type.SNAPSHOT, Stage.DONE, null, nodeA, true); assertRecoveryState(recoveryState, 0, Type.SNAPSHOT, Stage.DONE, null, nodeA, true);
validateIndexRecoveryState(shardResponse.recoveryState().getIndex()); validateIndexRecoveryState(recoveryState.getIndex());
} }
} }
} }
private List<ShardRecoveryResponse> findRecoveriesForTargetNode(String nodeName, List<ShardRecoveryResponse> responses) { private List<RecoveryState> findRecoveriesForTargetNode(String nodeName, List<RecoveryState> recoveryStates) {
List<ShardRecoveryResponse> nodeResponses = new ArrayList<>(); List<RecoveryState> nodeResponses = new ArrayList<>();
for (ShardRecoveryResponse response : responses) { for (RecoveryState recoveryState : recoveryStates) {
if (response.recoveryState().getTargetNode().getName().equals(nodeName)) { if (recoveryState.getTargetNode().getName().equals(nodeName)) {
nodeResponses.add(response); nodeResponses.add(recoveryState);
} }
} }
return nodeResponses; return nodeResponses;
@ -519,7 +534,6 @@ public class IndexRecoveryIT extends ESIntegTestCase {
final Settings nodeSettings = Settings.builder() final Settings nodeSettings = Settings.builder()
.put(RecoverySettings.INDICES_RECOVERY_RETRY_DELAY_NETWORK, "100ms") .put(RecoverySettings.INDICES_RECOVERY_RETRY_DELAY_NETWORK, "100ms")
.put(RecoverySettings.INDICES_RECOVERY_INTERNAL_ACTION_TIMEOUT, "1s") .put(RecoverySettings.INDICES_RECOVERY_INTERNAL_ACTION_TIMEOUT, "1s")
.put("plugin.types", MockTransportService.TestPlugin.class.getName())
.put(MockFSDirectoryService.RANDOM_PREVENT_DOUBLE_WRITE, false) // restarted recoveries will delete temp files and write them again .put(MockFSDirectoryService.RANDOM_PREVENT_DOUBLE_WRITE, false) // restarted recoveries will delete temp files and write them again
.build(); .build();
// start a master node // start a master node

View File

@ -47,6 +47,7 @@ import org.elasticsearch.index.Index;
import org.elasticsearch.index.shard.ShardId; import org.elasticsearch.index.shard.ShardId;
import org.elasticsearch.indices.IndicesService; import org.elasticsearch.indices.IndicesService;
import org.elasticsearch.indices.recovery.RecoverySource; import org.elasticsearch.indices.recovery.RecoverySource;
import org.elasticsearch.plugins.Plugin;
import org.elasticsearch.test.ESIntegTestCase; import org.elasticsearch.test.ESIntegTestCase;
import org.elasticsearch.test.ESIntegTestCase.ClusterScope; import org.elasticsearch.test.ESIntegTestCase.ClusterScope;
import org.elasticsearch.test.InternalTestCluster; import org.elasticsearch.test.InternalTestCluster;
@ -63,6 +64,7 @@ import java.io.IOException;
import java.nio.file.Files; import java.nio.file.Files;
import java.nio.file.Path; import java.nio.file.Path;
import java.util.Arrays; import java.util.Arrays;
import java.util.Collection;
import java.util.List; import java.util.List;
import java.util.concurrent.CountDownLatch; import java.util.concurrent.CountDownLatch;
import java.util.concurrent.Future; import java.util.concurrent.Future;
@ -87,10 +89,14 @@ public class IndicesStoreIntegrationIT extends ESIntegTestCase {
// which is between 1 and 2 sec can cause each of the shard deletion requests to timeout. // which is between 1 and 2 sec can cause each of the shard deletion requests to timeout.
// to prevent this we are setting the timeout here to something highish ie. the default in practice // to prevent this we are setting the timeout here to something highish ie. the default in practice
.put(IndicesStore.INDICES_STORE_DELETE_SHARD_TIMEOUT, new TimeValue(30, TimeUnit.SECONDS)) .put(IndicesStore.INDICES_STORE_DELETE_SHARD_TIMEOUT, new TimeValue(30, TimeUnit.SECONDS))
.extendArray("plugin.types", MockTransportService.TestPlugin.class.getName())
.build(); .build();
} }
@Override
protected Collection<Class<? extends Plugin>> nodePlugins() {
return pluginList(MockTransportService.TestPlugin.class);
}
@Override @Override
protected void ensureClusterStateConsistency() throws IOException { protected void ensureClusterStateConsistency() throws IOException {
// testShardActiveElseWhere might change the state of a non-master node // testShardActiveElseWhere might change the state of a non-master node

View File

@ -33,6 +33,8 @@ import org.elasticsearch.test.ESIntegTestCase.ClusterScope;
import org.elasticsearch.test.ESIntegTestCase.Scope; import org.elasticsearch.test.ESIntegTestCase.Scope;
import org.junit.Test; import org.junit.Test;
import java.util.Collection;
import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertAcked; import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertAcked;
import static org.hamcrest.Matchers.is; import static org.hamcrest.Matchers.is;
import static org.hamcrest.core.IsNull.notNullValue; import static org.hamcrest.core.IsNull.notNullValue;
@ -41,11 +43,8 @@ import static org.hamcrest.core.IsNull.notNullValue;
public class IndexTemplateFilteringIT extends ESIntegTestCase { public class IndexTemplateFilteringIT extends ESIntegTestCase {
@Override @Override
protected Settings nodeSettings(int nodeOrdinal) { protected Collection<Class<? extends Plugin>> nodePlugins() {
return Settings.builder() return pluginList(TestPlugin.class);
.put(super.nodeSettings(nodeOrdinal))
.put("plugin.types", TestPlugin.class.getName())
.build();
} }
@Test @Test

View File

@ -21,7 +21,6 @@ package org.elasticsearch.indices.warmer;
import com.carrotsearch.hppc.cursors.ObjectObjectCursor; import com.carrotsearch.hppc.cursors.ObjectObjectCursor;
import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableList;
import org.elasticsearch.action.admin.indices.create.CreateIndexRequestBuilder; import org.elasticsearch.action.admin.indices.create.CreateIndexRequestBuilder;
import org.elasticsearch.action.admin.indices.segments.IndexSegments; import org.elasticsearch.action.admin.indices.segments.IndexSegments;
import org.elasticsearch.action.admin.indices.segments.IndexShardSegments; import org.elasticsearch.action.admin.indices.segments.IndexShardSegments;
@ -49,7 +48,10 @@ import org.junit.Test;
import java.util.Locale; import java.util.Locale;
import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertAcked; import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertAcked;
import static org.hamcrest.Matchers.*; import static org.hamcrest.Matchers.equalTo;
import static org.hamcrest.Matchers.greaterThan;
import static org.hamcrest.Matchers.greaterThanOrEqualTo;
import static org.hamcrest.Matchers.is;
public class SimpleIndicesWarmerIT extends ESIntegTestCase { public class SimpleIndicesWarmerIT extends ESIntegTestCase {
@ -272,7 +274,7 @@ public class SimpleIndicesWarmerIT extends ESIntegTestCase {
for (IndexShardSegments indexShardSegments : indicesSegments) { for (IndexShardSegments indexShardSegments : indicesSegments) {
for (ShardSegments shardSegments : indexShardSegments) { for (ShardSegments shardSegments : indexShardSegments) {
for (Segment segment : shardSegments) { for (Segment segment : shardSegments) {
logger.debug("+=" + segment.memoryInBytes + " " + indexShardSegments.getShardId() + " " + shardSegments.getIndex()); logger.debug("+=" + segment.memoryInBytes + " " + indexShardSegments.getShardId() + " " + shardSegments.getShardRouting().getIndex());
total += segment.memoryInBytes; total += segment.memoryInBytes;
} }
} }

View File

@ -0,0 +1,54 @@
/*
* Licensed to Elasticsearch under one or more contributor
* license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright
* ownership. Elasticsearch licenses this file to you under
* the Apache License, Version 2.0 (the "License"); you may
* not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing,
* software distributed under the License is distributed on an
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
* KIND, either express or implied. See the License for the
* specific language governing permissions and limitations
* under the License.
*/
package org.elasticsearch.node;
import org.elasticsearch.Version;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.plugins.Plugin;
import java.util.Collection;
/**
* A node for testing which allows:
* <ul>
* <li>Overriding Version.CURRENT</li>
* <li>Adding test plugins that exist on the classpath</li>
* </ul>
*/
public class MockNode extends Node {
// these are kept here so a copy of this MockNode can be created, since Node does not store them
private Version version;
private Collection<Class<? extends Plugin>> plugins;
public MockNode(Settings settings, boolean loadConfigSettings, Version version, Collection<Class<? extends Plugin>> classpathPlugins) {
super(settings, loadConfigSettings, version, classpathPlugins);
this.version = version;
this.plugins = classpathPlugins;
}
public Collection<Class<? extends Plugin>> getPlugins() {
return plugins;
}
public Version getVersion() {
return version;
}
}

View File

@ -22,7 +22,7 @@ package org.elasticsearch.nodesinfo;
import org.elasticsearch.action.admin.cluster.health.ClusterHealthResponse; import org.elasticsearch.action.admin.cluster.health.ClusterHealthResponse;
import org.elasticsearch.action.admin.cluster.node.info.NodesInfoResponse; import org.elasticsearch.action.admin.cluster.node.info.NodesInfoResponse;
import org.elasticsearch.cluster.ClusterService; import org.elasticsearch.cluster.ClusterService;
import org.elasticsearch.plugins.PluginTestCase; import org.elasticsearch.test.ESIntegTestCase;
import org.elasticsearch.test.ESIntegTestCase.ClusterScope; import org.elasticsearch.test.ESIntegTestCase.ClusterScope;
import org.junit.Test; import org.junit.Test;
@ -36,7 +36,7 @@ import static org.hamcrest.Matchers.*;
* *
*/ */
@ClusterScope(scope= Scope.TEST, numDataNodes =0) @ClusterScope(scope= Scope.TEST, numDataNodes =0)
public class SimpleNodesInfoIT extends PluginTestCase { public class SimpleNodesInfoIT extends ESIntegTestCase {
static final class Fields { static final class Fields {
static final String SITE_PLUGIN = "dummy"; static final String SITE_PLUGIN = "dummy";

View File

@ -31,6 +31,7 @@ import org.elasticsearch.transport.*;
import org.junit.Test; import org.junit.Test;
import java.io.IOException; import java.io.IOException;
import java.util.Collection;
import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicInteger;
import static org.elasticsearch.common.settings.Settings.settingsBuilder; import static org.elasticsearch.common.settings.Settings.settingsBuilder;
@ -51,16 +52,17 @@ public class PluggableTransportModuleIT extends ESIntegTestCase {
return settingsBuilder() return settingsBuilder()
.put(super.nodeSettings(nodeOrdinal)) .put(super.nodeSettings(nodeOrdinal))
.put(DiscoveryModule.DISCOVERY_TYPE_KEY, "local") .put(DiscoveryModule.DISCOVERY_TYPE_KEY, "local")
.put("plugin.types", CountingSentRequestsPlugin.class.getName())
.build(); .build();
} }
@Override @Override
protected Settings transportClientSettings() { protected Collection<Class<? extends Plugin>> nodePlugins() {
return settingsBuilder() return pluginList(CountingSentRequestsPlugin.class);
.put("plugin.types", CountingSentRequestsPlugin.class.getName()) }
.put(super.transportClientSettings())
.build(); @Override
protected Collection<Class<? extends Plugin>> transportClientPlugins() {
return pluginList(CountingSentRequestsPlugin.class);
} }
@Test @Test

View File

@ -82,12 +82,12 @@ public class PluginManagerUnitTests extends ESTestCase {
Iterator<URL> iterator = handle.urls().iterator(); Iterator<URL> iterator = handle.urls().iterator();
if (supportStagingUrls) { if (supportStagingUrls) {
String expectedStagingURL = String.format(Locale.ROOT, "http://download.elastic.co/elasticsearch/staging/%s-%s/org/elasticsearch/plugin/%s/%s/%s-%s.zip", String expectedStagingURL = String.format(Locale.ROOT, "https://download.elastic.co/elasticsearch/staging/%s-%s/org/elasticsearch/plugin/%s/%s/%s-%s.zip",
Version.CURRENT.number(), Build.CURRENT.hashShort(), pluginName, Version.CURRENT.number(), pluginName, Version.CURRENT.number()); Version.CURRENT.number(), Build.CURRENT.hashShort(), pluginName, Version.CURRENT.number(), pluginName, Version.CURRENT.number());
assertThat(iterator.next().toExternalForm(), is(expectedStagingURL)); assertThat(iterator.next().toExternalForm(), is(expectedStagingURL));
} }
URL expected = new URL("http", "download.elastic.co", "/elasticsearch/release/org/elasticsearch/plugin/" + pluginName + "/" + Version.CURRENT.number() + "/" + URL expected = new URL("https", "download.elastic.co", "/elasticsearch/release/org/elasticsearch/plugin/" + pluginName + "/" + Version.CURRENT.number() + "/" +
pluginName + "-" + Version.CURRENT.number() + ".zip"); pluginName + "-" + Version.CURRENT.number() + ".zip");
assertThat(iterator.next().toExternalForm(), is(expected.toExternalForm())); assertThat(iterator.next().toExternalForm(), is(expected.toExternalForm()));
@ -108,12 +108,12 @@ public class PluginManagerUnitTests extends ESTestCase {
Iterator<URL> iterator = handle.urls().iterator(); Iterator<URL> iterator = handle.urls().iterator();
if (supportStagingUrls) { if (supportStagingUrls) {
String expectedStagingUrl = String.format(Locale.ROOT, "http://download.elastic.co/elasticsearch/staging/%s-%s/org/elasticsearch/plugin/%s/%s/%s-%s.zip", String expectedStagingUrl = String.format(Locale.ROOT, "https://download.elastic.co/elasticsearch/staging/%s-%s/org/elasticsearch/plugin/%s/%s/%s-%s.zip",
Version.CURRENT.number(), Build.CURRENT.hashShort(), randomPluginName, Version.CURRENT.number(), randomPluginName, Version.CURRENT.number()); Version.CURRENT.number(), Build.CURRENT.hashShort(), randomPluginName, Version.CURRENT.number(), randomPluginName, Version.CURRENT.number());
assertThat(iterator.next().toExternalForm(), is(expectedStagingUrl)); assertThat(iterator.next().toExternalForm(), is(expectedStagingUrl));
} }
String releaseUrl = String.format(Locale.ROOT, "http://download.elastic.co/elasticsearch/release/org/elasticsearch/plugin/%s/%s/%s-%s.zip", String releaseUrl = String.format(Locale.ROOT, "https://download.elastic.co/elasticsearch/release/org/elasticsearch/plugin/%s/%s/%s-%s.zip",
randomPluginName, Version.CURRENT.number(), randomPluginName, Version.CURRENT.number()); randomPluginName, Version.CURRENT.number(), randomPluginName, Version.CURRENT.number());
assertThat(iterator.next().toExternalForm(), is(releaseUrl)); assertThat(iterator.next().toExternalForm(), is(releaseUrl));

View File

@ -1,56 +0,0 @@
/*
* Licensed to Elasticsearch under one or more contributor
* license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright
* ownership. Elasticsearch licenses this file to you under
* the Apache License, Version 2.0 (the "License"); you may
* not use this file except in compliance with the License.
* You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing,
* software distributed under the License is distributed on an
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
* KIND, either express or implied. See the License for the
* specific language governing permissions and limitations
* under the License.
*/
package org.elasticsearch.plugins;
import org.elasticsearch.cluster.ClusterService;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.test.ESIntegTestCase;
import java.net.URISyntaxException;
import java.net.URL;
import static org.elasticsearch.client.Requests.clusterHealthRequest;
import static org.elasticsearch.common.settings.Settings.settingsBuilder;
/**
* Base class that lets you start a node with plugins.
*/
public abstract class PluginTestCase extends ESIntegTestCase {
public String startNodeWithPlugins(Settings nodeSettings, String pluginDir, String ... pluginClassNames) throws URISyntaxException {
URL resource = getClass().getResource(pluginDir);
Settings.Builder settings = settingsBuilder();
settings.put(nodeSettings);
if (resource != null) {
settings.put("path.plugins", getDataPath(pluginDir).toAbsolutePath());
}
if (pluginClassNames.length > 0) {
settings.putArray("plugin.types", pluginClassNames);
}
String nodeName = internalCluster().startNode(settings);
// We wait for a Green status
client().admin().cluster().health(clusterHealthRequest().waitForGreenStatus()).actionGet();
return internalCluster().getInstance(ClusterService.class, nodeName).state().nodes().localNodeId();
}
}

View File

@ -24,6 +24,8 @@ import org.elasticsearch.env.Environment;
import org.elasticsearch.index.store.IndexStoreModule; import org.elasticsearch.index.store.IndexStoreModule;
import org.elasticsearch.test.ESTestCase; import org.elasticsearch.test.ESTestCase;
import java.util.Arrays;
public class PluginsServiceTests extends ESTestCase { public class PluginsServiceTests extends ESTestCase {
public static class AdditionalSettingsPlugin1 extends Plugin { public static class AdditionalSettingsPlugin1 extends Plugin {
@Override @Override
@ -54,13 +56,16 @@ public class PluginsServiceTests extends ESTestCase {
} }
} }
static PluginsService newPluginsService(Settings settings, Class<? extends Plugin>... classpathPlugins) {
return new PluginsService(settings, new Environment(settings), Arrays.asList(classpathPlugins));
}
public void testAdditionalSettings() { public void testAdditionalSettings() {
Settings settings = Settings.builder() Settings settings = Settings.builder()
.put("path.home", createTempDir()) .put("path.home", createTempDir())
.put("my.setting", "test") .put("my.setting", "test")
.put(IndexStoreModule.STORE_TYPE, IndexStoreModule.Type.SIMPLEFS.getSettingsKey()) .put(IndexStoreModule.STORE_TYPE, IndexStoreModule.Type.SIMPLEFS.getSettingsKey()).build();
.putArray("plugin.types", AdditionalSettingsPlugin1.class.getName()).build(); PluginsService service = newPluginsService(settings, AdditionalSettingsPlugin1.class);
PluginsService service = new PluginsService(settings, new Environment(settings));
Settings newSettings = service.updatedSettings(); Settings newSettings = service.updatedSettings();
assertEquals("test", newSettings.get("my.setting")); // previous settings still exist assertEquals("test", newSettings.get("my.setting")); // previous settings still exist
assertEquals("1", newSettings.get("foo.bar")); // added setting exists assertEquals("1", newSettings.get("foo.bar")); // added setting exists
@ -69,9 +74,8 @@ public class PluginsServiceTests extends ESTestCase {
public void testAdditionalSettingsClash() { public void testAdditionalSettingsClash() {
Settings settings = Settings.builder() Settings settings = Settings.builder()
.put("path.home", createTempDir()) .put("path.home", createTempDir()).build();
.putArray("plugin.types", AdditionalSettingsPlugin1.class.getName(), AdditionalSettingsPlugin2.class.getName()).build(); PluginsService service = newPluginsService(settings, AdditionalSettingsPlugin1.class, AdditionalSettingsPlugin2.class);
PluginsService service = new PluginsService(settings, new Environment(settings));
try { try {
service.updatedSettings(); service.updatedSettings();
fail("Expected exception when building updated settings"); fail("Expected exception when building updated settings");

View File

@ -25,6 +25,8 @@ import org.elasticsearch.test.ESIntegTestCase.ClusterScope;
import org.elasticsearch.test.rest.client.http.HttpResponse; import org.elasticsearch.test.rest.client.http.HttpResponse;
import org.junit.Test; import org.junit.Test;
import java.util.Collection;
import static org.elasticsearch.rest.RestStatus.OK; import static org.elasticsearch.rest.RestStatus.OK;
import static org.elasticsearch.rest.RestStatus.UNAUTHORIZED; import static org.elasticsearch.rest.RestStatus.UNAUTHORIZED;
import static org.elasticsearch.test.ESIntegTestCase.Scope; import static org.elasticsearch.test.ESIntegTestCase.Scope;
@ -41,11 +43,15 @@ public class ResponseHeaderPluginIT extends ESIntegTestCase {
protected Settings nodeSettings(int nodeOrdinal) { protected Settings nodeSettings(int nodeOrdinal) {
return Settings.settingsBuilder() return Settings.settingsBuilder()
.put(super.nodeSettings(nodeOrdinal)) .put(super.nodeSettings(nodeOrdinal))
.put("plugin.types", TestResponseHeaderPlugin.class.getName())
.put("force.http.enabled", true) .put("force.http.enabled", true)
.build(); .build();
} }
@Override
protected Collection<Class<? extends Plugin>> nodePlugins() {
return pluginList(TestResponseHeaderPlugin.class);
}
@Test @Test
public void testThatSettingHeadersWorks() throws Exception { public void testThatSettingHeadersWorks() throws Exception {
ensureGreen(); ensureGreen();

View File

@ -284,7 +284,7 @@ public class RecoveryWhileUnderLoadIT extends ESIntegTestCase {
IndicesStatsResponse indicesStatsResponse = client().admin().indices().prepareStats().get(); IndicesStatsResponse indicesStatsResponse = client().admin().indices().prepareStats().get();
for (ShardStats shardStats : indicesStatsResponse.getShards()) { for (ShardStats shardStats : indicesStatsResponse.getShards()) {
DocsStats docsStats = shardStats.getStats().docs; DocsStats docsStats = shardStats.getStats().docs;
logger.info("shard [{}] - count {}, primary {}", shardStats.getShardId(), docsStats.getCount(), shardStats.getShardRouting().primary()); logger.info("shard [{}] - count {}, primary {}", shardStats.getShardRouting().id(), docsStats.getCount(), shardStats.getShardRouting().primary());
} }
//if there was an error we try to wait and see if at some point it'll get fixed //if there was an error we try to wait and see if at some point it'll get fixed

View File

@ -54,6 +54,7 @@ import org.elasticsearch.indices.IndicesLifecycle;
import org.elasticsearch.indices.recovery.RecoveryFileChunkRequest; import org.elasticsearch.indices.recovery.RecoveryFileChunkRequest;
import org.elasticsearch.indices.recovery.RecoverySettings; import org.elasticsearch.indices.recovery.RecoverySettings;
import org.elasticsearch.indices.recovery.RecoveryTarget; import org.elasticsearch.indices.recovery.RecoveryTarget;
import org.elasticsearch.plugins.Plugin;
import org.elasticsearch.search.SearchHit; import org.elasticsearch.search.SearchHit;
import org.elasticsearch.search.SearchHits; import org.elasticsearch.search.SearchHits;
import org.elasticsearch.test.BackgroundIndexer; import org.elasticsearch.test.BackgroundIndexer;
@ -63,7 +64,6 @@ import org.elasticsearch.test.junit.annotations.TestLogging;
import org.elasticsearch.test.transport.MockTransportService; import org.elasticsearch.test.transport.MockTransportService;
import org.elasticsearch.transport.Transport; import org.elasticsearch.transport.Transport;
import org.elasticsearch.transport.TransportException; import org.elasticsearch.transport.TransportException;
import org.elasticsearch.transport.TransportModule;
import org.elasticsearch.transport.TransportRequest; import org.elasticsearch.transport.TransportRequest;
import org.elasticsearch.transport.TransportRequestOptions; import org.elasticsearch.transport.TransportRequestOptions;
import org.elasticsearch.transport.TransportService; import org.elasticsearch.transport.TransportService;
@ -76,6 +76,7 @@ import java.nio.file.Path;
import java.nio.file.SimpleFileVisitor; import java.nio.file.SimpleFileVisitor;
import java.nio.file.attribute.BasicFileAttributes; import java.nio.file.attribute.BasicFileAttributes;
import java.util.ArrayList; import java.util.ArrayList;
import java.util.Collection;
import java.util.List; import java.util.List;
import java.util.concurrent.CountDownLatch; import java.util.concurrent.CountDownLatch;
import java.util.concurrent.Semaphore; import java.util.concurrent.Semaphore;
@ -100,12 +101,10 @@ public class RelocationIT extends ESIntegTestCase {
private final TimeValue ACCEPTABLE_RELOCATION_TIME = new TimeValue(5, TimeUnit.MINUTES); private final TimeValue ACCEPTABLE_RELOCATION_TIME = new TimeValue(5, TimeUnit.MINUTES);
@Override @Override
protected Settings nodeSettings(int nodeOrdinal) { protected Collection<Class<? extends Plugin>> nodePlugins() {
return Settings.builder() return pluginList(MockTransportService.TestPlugin.class);
.put("plugin.types", MockTransportService.TestPlugin.class.getName()).build();
} }
@Test @Test
public void testSimpleRelocationNoIndexing() { public void testSimpleRelocationNoIndexing() {
logger.info("--> starting [node1] ..."); logger.info("--> starting [node1] ...");
@ -422,7 +421,7 @@ public class RelocationIT extends ESIntegTestCase {
public boolean apply(Object input) { public boolean apply(Object input) {
RecoveryResponse recoveryResponse = internalCluster().client(redNodeName).admin().indices().prepareRecoveries(indexName) RecoveryResponse recoveryResponse = internalCluster().client(redNodeName).admin().indices().prepareRecoveries(indexName)
.get(); .get();
return !recoveryResponse.shardResponses().get(indexName).isEmpty(); return !recoveryResponse.shardRecoveryStates().get(indexName).isEmpty();
} }
} }
); );

View File

@ -34,6 +34,7 @@ import org.elasticsearch.index.query.QueryBuilders;
import org.elasticsearch.indices.recovery.RecoveryFileChunkRequest; import org.elasticsearch.indices.recovery.RecoveryFileChunkRequest;
import org.elasticsearch.indices.recovery.RecoverySettings; import org.elasticsearch.indices.recovery.RecoverySettings;
import org.elasticsearch.indices.recovery.RecoveryTarget; import org.elasticsearch.indices.recovery.RecoveryTarget;
import org.elasticsearch.plugins.Plugin;
import org.elasticsearch.test.ESIntegTestCase; import org.elasticsearch.test.ESIntegTestCase;
import org.elasticsearch.test.transport.MockTransportService; import org.elasticsearch.test.transport.MockTransportService;
import org.elasticsearch.transport.*; import org.elasticsearch.transport.*;
@ -41,6 +42,7 @@ import org.junit.Test;
import java.io.IOException; import java.io.IOException;
import java.util.ArrayList; import java.util.ArrayList;
import java.util.Collection;
import java.util.Collections; import java.util.Collections;
import java.util.List; import java.util.List;
import java.util.concurrent.CountDownLatch; import java.util.concurrent.CountDownLatch;
@ -55,14 +57,19 @@ import static org.hamcrest.Matchers.greaterThanOrEqualTo;
@SuppressCodecs("*") // test relies on exact file extensions @SuppressCodecs("*") // test relies on exact file extensions
public class TruncatedRecoveryIT extends ESIntegTestCase { public class TruncatedRecoveryIT extends ESIntegTestCase {
@Override
protected Settings nodeSettings(int nodeOrdinal) { protected Settings nodeSettings(int nodeOrdinal) {
Settings.Builder builder = Settings.builder() Settings.Builder builder = Settings.builder()
.put(super.nodeSettings(nodeOrdinal)) .put(super.nodeSettings(nodeOrdinal))
.extendArray("plugin.types", MockTransportService.TestPlugin.class.getName())
.put(RecoverySettings.INDICES_RECOVERY_FILE_CHUNK_SIZE, new ByteSizeValue(randomIntBetween(50, 300), ByteSizeUnit.BYTES)); .put(RecoverySettings.INDICES_RECOVERY_FILE_CHUNK_SIZE, new ByteSizeValue(randomIntBetween(50, 300), ByteSizeUnit.BYTES));
return builder.build(); return builder.build();
} }
@Override
protected Collection<Class<? extends Plugin>> nodePlugins() {
return pluginList(MockTransportService.TestPlugin.class);
}
/** /**
* This test tries to truncate some of larger files in the index to trigger leftovers on the recovery * This test tries to truncate some of larger files in the index to trigger leftovers on the recovery
* target. This happens during recovery when the last chunk of the file is transferred to the replica * target. This happens during recovery when the last chunk of the file is transferred to the replica

View File

@ -28,6 +28,8 @@ import org.elasticsearch.script.mustache.MustacheScriptEngineService;
import org.elasticsearch.test.ESIntegTestCase; import org.elasticsearch.test.ESIntegTestCase;
import org.junit.Test; import org.junit.Test;
import java.util.Collection;
import static org.hamcrest.CoreMatchers.containsString; import static org.hamcrest.CoreMatchers.containsString;
import static org.hamcrest.CoreMatchers.notNullValue; import static org.hamcrest.CoreMatchers.notNullValue;
@ -40,12 +42,16 @@ public class CustomScriptContextIT extends ESIntegTestCase {
@Override @Override
protected Settings nodeSettings(int nodeOrdinal) { protected Settings nodeSettings(int nodeOrdinal) {
return Settings.builder().put(super.nodeSettings(nodeOrdinal)) return Settings.builder().put(super.nodeSettings(nodeOrdinal))
.put("plugin.types", CustomScriptContextPlugin.class.getName())
.put("script." + PLUGIN_NAME + "_custom_globally_disabled_op", "off") .put("script." + PLUGIN_NAME + "_custom_globally_disabled_op", "off")
.put("script.engine.expression.inline." + PLUGIN_NAME + "_custom_exp_disabled_op", "off") .put("script.engine.expression.inline." + PLUGIN_NAME + "_custom_exp_disabled_op", "off")
.build(); .build();
} }
@Override
protected Collection<Class<? extends Plugin>> nodePlugins() {
return pluginList(CustomScriptContextPlugin.class);
}
@Test @Test
public void testCustomScriptContextsSettings() { public void testCustomScriptContextsSettings() {
ScriptService scriptService = internalCluster().getInstance(ScriptService.class); ScriptService scriptService = internalCluster().getInstance(ScriptService.class);

View File

@ -30,6 +30,7 @@ import org.elasticsearch.test.ESIntegTestCase;
import org.elasticsearch.test.ESIntegTestCase.ClusterScope; import org.elasticsearch.test.ESIntegTestCase.ClusterScope;
import org.elasticsearch.test.ESIntegTestCase.Scope; import org.elasticsearch.test.ESIntegTestCase.Scope;
import java.util.Collection;
import java.util.Map; import java.util.Map;
import java.util.concurrent.ExecutionException; import java.util.concurrent.ExecutionException;
@ -40,8 +41,8 @@ import static org.hamcrest.Matchers.equalTo;
public class ScriptFieldIT extends ESIntegTestCase { public class ScriptFieldIT extends ESIntegTestCase {
@Override @Override
protected Settings nodeSettings(int nodeOrdinal) { protected Collection<Class<? extends Plugin>> nodePlugins() {
return settingsBuilder().put(super.nodeSettings(nodeOrdinal)).put("plugin.types", CustomScriptPlugin.class.getName()).build(); return pluginList(CustomScriptPlugin.class);
} }
static int[] intArray = { Integer.MAX_VALUE, Integer.MIN_VALUE, 3 }; static int[] intArray = { Integer.MAX_VALUE, Integer.MIN_VALUE, 3 };

View File

@ -93,11 +93,15 @@ public class SignificantTermsSignificanceScoreIT extends ESIntegTestCase {
public Settings nodeSettings(int nodeOrdinal) { public Settings nodeSettings(int nodeOrdinal) {
return settingsBuilder() return settingsBuilder()
.put(super.nodeSettings(nodeOrdinal)) .put(super.nodeSettings(nodeOrdinal))
.put("plugin.types", CustomSignificanceHeuristicPlugin.class.getName())
.put("path.conf", this.getDataPath("config")) .put("path.conf", this.getDataPath("config"))
.build(); .build();
} }
@Override
protected Collection<Class<? extends Plugin>> nodePlugins() {
return pluginList(CustomSignificanceHeuristicPlugin.class);
}
public String randomExecutionHint() { public String randomExecutionHint() {
return randomBoolean() ? null : randomFrom(SignificantTermsAggregatorFactory.ExecutionMode.values()).toString(); return randomBoolean() ? null : randomFrom(SignificantTermsAggregatorFactory.ExecutionMode.values()).toString();
} }

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