Convert upgrade action to broadcast by node

Several shard-level operations that previously broadcasted a request
per shard were converted to broadcast a request per node. This commit
converts upgrade action to this new model as well.

Closes #13204
This commit is contained in:
Jason Tedor 2015-08-30 20:40:34 -04:00
parent aa26b66e96
commit d1223b7369
10 changed files with 75 additions and 79 deletions

View File

@ -68,7 +68,7 @@ public class TransportClearIndicesCacheAction extends TransportBroadcastByNodeAc
} }
@Override @Override
protected ClearIndicesCacheResponse newResponse(ClearIndicesCacheRequest request, int totalShards, int successfulShards, int failedShards, List<EmptyResult> responses, List<ShardOperationFailedException> shardFailures) { protected ClearIndicesCacheResponse newResponse(ClearIndicesCacheRequest request, int totalShards, int successfulShards, int failedShards, List<EmptyResult> responses, List<ShardOperationFailedException> shardFailures, ClusterState clusterState) {
return new ClearIndicesCacheResponse(totalShards, successfulShards, failedShards, shardFailures); return new ClearIndicesCacheResponse(totalShards, successfulShards, failedShards, shardFailures);
} }

View File

@ -62,7 +62,7 @@ public class TransportOptimizeAction extends TransportBroadcastByNodeAction<Opti
} }
@Override @Override
protected OptimizeResponse newResponse(OptimizeRequest request, int totalShards, int successfulShards, int failedShards, List<EmptyResult> responses, List<ShardOperationFailedException> shardFailures) { protected OptimizeResponse newResponse(OptimizeRequest request, int totalShards, int successfulShards, int failedShards, List<EmptyResult> responses, List<ShardOperationFailedException> shardFailures, ClusterState clusterState) {
return new OptimizeResponse(totalShards, successfulShards, failedShards, shardFailures); return new OptimizeResponse(totalShards, successfulShards, failedShards, shardFailures);
} }

View File

@ -69,7 +69,7 @@ public class TransportRecoveryAction extends TransportBroadcastByNodeAction<Reco
@Override @Override
protected RecoveryResponse newResponse(RecoveryRequest request, int totalShards, int successfulShards, int failedShards, List<RecoveryState> responses, List<ShardOperationFailedException> shardFailures) { protected RecoveryResponse newResponse(RecoveryRequest request, int totalShards, int successfulShards, int failedShards, List<RecoveryState> responses, List<ShardOperationFailedException> shardFailures, ClusterState clusterState) {
Map<String, List<RecoveryState>> shardResponses = Maps.newHashMap(); Map<String, List<RecoveryState>> shardResponses = Maps.newHashMap();
for (RecoveryState recoveryState : responses) { for (RecoveryState recoveryState : responses) {
if (recoveryState == null) { if (recoveryState == null) {

View File

@ -80,7 +80,7 @@ public class TransportIndicesSegmentsAction extends TransportBroadcastByNodeActi
} }
@Override @Override
protected IndicesSegmentResponse newResponse(IndicesSegmentsRequest request, int totalShards, int successfulShards, int failedShards, List<ShardSegments> results, List<ShardOperationFailedException> shardFailures) { protected IndicesSegmentResponse newResponse(IndicesSegmentsRequest request, int totalShards, int successfulShards, int failedShards, List<ShardSegments> results, List<ShardOperationFailedException> shardFailures, ClusterState clusterState) {
return new IndicesSegmentResponse(results.toArray(new ShardSegments[results.size()]), totalShards, successfulShards, failedShards, shardFailures); return new IndicesSegmentResponse(results.toArray(new ShardSegments[results.size()]), totalShards, successfulShards, failedShards, shardFailures);
} }

View File

@ -81,7 +81,7 @@ public class TransportIndicesStatsAction extends TransportBroadcastByNodeAction<
} }
@Override @Override
protected IndicesStatsResponse newResponse(IndicesStatsRequest request, int totalShards, int successfulShards, int failedShards, List<ShardStats> responses, List<ShardOperationFailedException> shardFailures) { protected IndicesStatsResponse newResponse(IndicesStatsRequest request, int totalShards, int successfulShards, int failedShards, List<ShardStats> responses, List<ShardOperationFailedException> shardFailures, ClusterState clusterState) {
return new IndicesStatsResponse(responses.toArray(new ShardStats[responses.size()]), totalShards, successfulShards, failedShards, shardFailures); return new IndicesStatsResponse(responses.toArray(new ShardStats[responses.size()]), totalShards, successfulShards, failedShards, shardFailures);
} }

View File

@ -82,7 +82,7 @@ public class TransportUpgradeStatusAction extends TransportBroadcastByNodeAction
} }
@Override @Override
protected UpgradeStatusResponse newResponse(UpgradeStatusRequest request, int totalShards, int successfulShards, int failedShards, List<ShardUpgradeStatus> responses, List<ShardOperationFailedException> shardFailures) { protected UpgradeStatusResponse newResponse(UpgradeStatusRequest request, int totalShards, int successfulShards, int failedShards, List<ShardUpgradeStatus> responses, List<ShardOperationFailedException> shardFailures, ClusterState clusterState) {
return new UpgradeStatusResponse(responses.toArray(new ShardUpgradeStatus[responses.size()]), totalShards, successfulShards, failedShards, shardFailures); return new UpgradeStatusResponse(responses.toArray(new ShardUpgradeStatus[responses.size()]), totalShards, successfulShards, failedShards, shardFailures);
} }

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

@ -108,7 +108,12 @@ public abstract class TransportBroadcastByNodeAction<Request extends BroadcastRe
}, executor, new BroadcastByNodeTransportRequestHandler()); }, executor, new BroadcastByNodeTransportRequestHandler());
} }
private final Response newResponse(Request request, AtomicReferenceArray responses, List<NoShardAvailableActionException> unavailableShardExceptions, Map<String, List<ShardRouting>> nodes) { private final Response newResponse(
Request request,
AtomicReferenceArray responses,
List<NoShardAvailableActionException> unavailableShardExceptions,
Map<String, List<ShardRouting>> nodes,
ClusterState clusterState) {
int totalShards = 0; int totalShards = 0;
int successfulShards = 0; int successfulShards = 0;
List<ShardOperationResult> broadcastByNodeResponses = new ArrayList<>(); List<ShardOperationResult> broadcastByNodeResponses = new ArrayList<>();
@ -134,7 +139,7 @@ public abstract class TransportBroadcastByNodeAction<Request extends BroadcastRe
} }
totalShards += unavailableShardExceptions.size(); totalShards += unavailableShardExceptions.size();
int failedShards = exceptions.size(); int failedShards = exceptions.size();
return newResponse(request, totalShards, successfulShards, failedShards, broadcastByNodeResponses, exceptions); return newResponse(request, totalShards, successfulShards, failedShards, broadcastByNodeResponses, exceptions, clusterState);
} }
/** /**
@ -155,9 +160,10 @@ public abstract class TransportBroadcastByNodeAction<Request extends BroadcastRe
* @param failedShards the total number of shards for which execution of the operation failed * @param failedShards the total number of shards for which execution of the operation failed
* @param results the per-node aggregated shard-level results * @param results the per-node aggregated shard-level results
* @param shardFailures the exceptions corresponding to shard operationa failures * @param shardFailures the exceptions corresponding to shard operationa failures
* @param clusterState the cluster state
* @return the response * @return the response
*/ */
protected abstract Response newResponse(Request request, int totalShards, int successfulShards, int failedShards, List<ShardOperationResult> results, List<ShardOperationFailedException> shardFailures); 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 * Deserialize a request from an input stream
@ -341,7 +347,7 @@ public abstract class TransportBroadcastByNodeAction<Request extends BroadcastRe
protected void onCompletion() { protected void onCompletion() {
Response response = null; Response response = null;
try { try {
response = newResponse(request, responses, unavailableShardExceptions, nodeIds); response = newResponse(request, responses, unavailableShardExceptions, nodeIds, clusterState);
} catch (Throwable t) { } catch (Throwable t) {
logger.debug("failed to combine responses from nodes", t); logger.debug("failed to combine responses from nodes", t);
listener.onFailure(t); listener.onFailure(t);

View File

@ -119,7 +119,7 @@ public class TransportBroadcastByNodeActionTests extends ESTestCase {
} }
@Override @Override
protected Response newResponse(Request request, int totalShards, int successfulShards, int failedShards, List<EmptyResult> emptyResults, List<ShardOperationFailedException> shardFailures) { 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); return new Response(totalShards, successfulShards, failedShards, shardFailures);
} }