Merge branch 'master' into painless_method_references

This commit is contained in:
Robert Muir 2016-06-06 22:39:28 -04:00
commit 950b093fc7
252 changed files with 5475 additions and 2279 deletions

View File

@ -170,6 +170,13 @@ public class RestTestsFromSnippetsTask extends SnippetsTask {
current.println('---')
current.println("setup:")
body(setup)
// always wait for yellow before anything is executed
current.println(
" - do:\n" +
" raw:\n" +
" method: GET\n" +
" path: \"_cluster/health\"\n" +
" wait_for_status: \"yellow\"")
}
private void body(Snippet snippet) {

View File

@ -19,7 +19,6 @@
<suppress files="core[/\\]src[/\\]main[/\\]java[/\\]org[/\\]elasticsearch[/\\]action[/\\]Action.java" checks="LineLength" />
<suppress files="core[/\\]src[/\\]main[/\\]java[/\\]org[/\\]elasticsearch[/\\]action[/\\]ActionModule.java" checks="LineLength" />
<suppress files="core[/\\]src[/\\]main[/\\]java[/\\]org[/\\]elasticsearch[/\\]action[/\\]ActionRequestBuilder.java" checks="LineLength" />
<suppress files="core[/\\]src[/\\]main[/\\]java[/\\]org[/\\]elasticsearch[/\\]action[/\\]ReplicationResponse.java" checks="LineLength" />
<suppress files="core[/\\]src[/\\]main[/\\]java[/\\]org[/\\]elasticsearch[/\\]action[/\\]admin[/\\]cluster[/\\]health[/\\]ClusterHealthRequestBuilder.java" checks="LineLength" />
<suppress files="core[/\\]src[/\\]main[/\\]java[/\\]org[/\\]elasticsearch[/\\]action[/\\]admin[/\\]cluster[/\\]health[/\\]TransportClusterHealthAction.java" checks="LineLength" />
<suppress files="core[/\\]src[/\\]main[/\\]java[/\\]org[/\\]elasticsearch[/\\]action[/\\]admin[/\\]cluster[/\\]node[/\\]hotthreads[/\\]NodesHotThreadsRequestBuilder.java" checks="LineLength" />
@ -101,7 +100,6 @@
<suppress files="core[/\\]src[/\\]main[/\\]java[/\\]org[/\\]elasticsearch[/\\]action[/\\]admin[/\\]indices[/\\]open[/\\]TransportOpenIndexAction.java" checks="LineLength" />
<suppress files="core[/\\]src[/\\]main[/\\]java[/\\]org[/\\]elasticsearch[/\\]action[/\\]admin[/\\]indices[/\\]recovery[/\\]TransportRecoveryAction.java" checks="LineLength" />
<suppress files="core[/\\]src[/\\]main[/\\]java[/\\]org[/\\]elasticsearch[/\\]action[/\\]admin[/\\]indices[/\\]refresh[/\\]TransportRefreshAction.java" checks="LineLength" />
<suppress files="core[/\\]src[/\\]main[/\\]java[/\\]org[/\\]elasticsearch[/\\]action[/\\]admin[/\\]indices[/\\]refresh[/\\]TransportShardRefreshAction.java" checks="LineLength" />
<suppress files="core[/\\]src[/\\]main[/\\]java[/\\]org[/\\]elasticsearch[/\\]action[/\\]admin[/\\]indices[/\\]segments[/\\]IndexSegments.java" checks="LineLength" />
<suppress files="core[/\\]src[/\\]main[/\\]java[/\\]org[/\\]elasticsearch[/\\]action[/\\]admin[/\\]indices[/\\]segments[/\\]IndicesSegmentResponse.java" checks="LineLength" />
<suppress files="core[/\\]src[/\\]main[/\\]java[/\\]org[/\\]elasticsearch[/\\]action[/\\]admin[/\\]indices[/\\]segments[/\\]IndicesSegmentsRequestBuilder.java" checks="LineLength" />
@ -1273,14 +1271,6 @@
<suppress files="plugins[/\\]repository-s3[/\\]src[/\\]test[/\\]java[/\\]org[/\\]elasticsearch[/\\]cloud[/\\]aws[/\\]blobstore[/\\]MockDefaultS3OutputStream.java" checks="LineLength" />
<suppress files="plugins[/\\]repository-s3[/\\]src[/\\]test[/\\]java[/\\]org[/\\]elasticsearch[/\\]repositories[/\\]s3[/\\]AbstractS3SnapshotRestoreTest.java" checks="LineLength" />
<suppress files="plugins[/\\]store-smb[/\\]src[/\\]main[/\\]java[/\\]org[/\\]apache[/\\]lucene[/\\]store[/\\]SmbDirectoryWrapper.java" checks="LineLength" />
<suppress files="qa[/\\]evil-tests[/\\]src[/\\]test[/\\]java[/\\]org[/\\]elasticsearch[/\\]bootstrap[/\\]ESPolicyUnitTests.java" checks="LineLength" />
<suppress files="qa[/\\]evil-tests[/\\]src[/\\]test[/\\]java[/\\]org[/\\]elasticsearch[/\\]bootstrap[/\\]EvilSecurityTests.java" checks="LineLength" />
<suppress files="qa[/\\]evil-tests[/\\]src[/\\]test[/\\]java[/\\]org[/\\]elasticsearch[/\\]common[/\\]cli[/\\]CheckFileCommandTests.java" checks="LineLength" />
<suppress files="qa[/\\]evil-tests[/\\]src[/\\]test[/\\]java[/\\]org[/\\]elasticsearch[/\\]tribe[/\\]TribeUnitTests.java" checks="LineLength" />
<suppress files="qa[/\\]smoke-test-client[/\\]src[/\\]test[/\\]java[/\\]org[/\\]elasticsearch[/\\]smoketest[/\\]ESSmokeClientTestCase.java" checks="LineLength" />
<suppress files="qa[/\\]smoke-test-ingest-with-all-dependencies[/\\]src[/\\]test[/\\]java[/\\]org[/\\]elasticsearch[/\\]ingest[/\\]CombineProcessorsTests.java" checks="LineLength" />
<suppress files="qa[/\\]smoke-test-ingest-with-all-dependencies[/\\]src[/\\]test[/\\]java[/\\]org[/\\]elasticsearch[/\\]ingest[/\\]IngestDocumentMustacheIT.java" checks="LineLength" />
<suppress files="qa[/\\]smoke-test-ingest-with-all-dependencies[/\\]src[/\\]test[/\\]java[/\\]org[/\\]elasticsearch[/\\]ingest[/\\]IngestMustacheSetProcessorIT.java" checks="LineLength" />
<suppress files="test[/\\]framework[/\\]src[/\\]main[/\\]java[/\\]org[/\\]elasticsearch[/\\]bootstrap[/\\]BootstrapForTesting.java" checks="LineLength" />
<suppress files="test[/\\]framework[/\\]src[/\\]main[/\\]java[/\\]org[/\\]elasticsearch[/\\]cluster[/\\]MockInternalClusterInfoService.java" checks="LineLength" />
<suppress files="test[/\\]framework[/\\]src[/\\]main[/\\]java[/\\]org[/\\]elasticsearch[/\\]cluster[/\\]routing[/\\]TestShardRouting.java" checks="LineLength" />

View File

@ -18,10 +18,15 @@
*/
package org.elasticsearch.action;
import org.elasticsearch.action.support.WriteRequest;
import org.elasticsearch.action.support.WriteResponse;
import org.elasticsearch.action.support.WriteRequest.RefreshPolicy;
import org.elasticsearch.action.support.replication.ReplicationResponse;
import org.elasticsearch.common.io.stream.StreamInput;
import org.elasticsearch.common.io.stream.StreamOutput;
import org.elasticsearch.common.xcontent.StatusToXContent;
import org.elasticsearch.common.xcontent.XContentBuilder;
import org.elasticsearch.index.IndexSettings;
import org.elasticsearch.index.shard.ShardId;
import org.elasticsearch.rest.RestStatus;
@ -30,12 +35,13 @@ import java.io.IOException;
/**
* A base class for the response of a write operation that involves a single doc
*/
public abstract class DocWriteResponse extends ReplicationResponse implements StatusToXContent {
public abstract class DocWriteResponse extends ReplicationResponse implements WriteResponse, StatusToXContent {
private ShardId shardId;
private String id;
private String type;
private long version;
private boolean forcedRefresh;
public DocWriteResponse(ShardId shardId, String type, String id, long version) {
this.shardId = shardId;
@ -84,6 +90,20 @@ public abstract class DocWriteResponse extends ReplicationResponse implements St
return this.version;
}
/**
* Did this request force a refresh? Requests that set {@link WriteRequest#setRefreshPolicy(RefreshPolicy)} to
* {@link RefreshPolicy#IMMEDIATE} will always return true for this. Requests that set it to {@link RefreshPolicy#WAIT_UNTIL} will
* only return true here if they run out of refresh listener slots (see {@link IndexSettings#MAX_REFRESH_LISTENERS_PER_SHARD}).
*/
public boolean forcedRefresh() {
return forcedRefresh;
}
@Override
public void setForcedRefresh(boolean forcedRefresh) {
this.forcedRefresh = forcedRefresh;
}
/** returns the rest status for this response (based on {@link ShardInfo#status()} */
public RestStatus status() {
return getShardInfo().status();
@ -97,6 +117,7 @@ public abstract class DocWriteResponse extends ReplicationResponse implements St
type = in.readString();
id = in.readString();
version = in.readZLong();
forcedRefresh = in.readBoolean();
}
@Override
@ -106,6 +127,7 @@ public abstract class DocWriteResponse extends ReplicationResponse implements St
out.writeString(type);
out.writeString(id);
out.writeZLong(version);
out.writeBoolean(forcedRefresh);
}
static final class Fields {
@ -121,7 +143,8 @@ public abstract class DocWriteResponse extends ReplicationResponse implements St
builder.field(Fields._INDEX, shardId.getIndexName())
.field(Fields._TYPE, type)
.field(Fields._ID, id)
.field(Fields._VERSION, version);
.field(Fields._VERSION, version)
.field("forced_refresh", forcedRefresh);
shardInfo.toXContent(builder, params);
return builder;
}

View File

@ -19,9 +19,9 @@
package org.elasticsearch.action.admin.indices.flush;
import org.elasticsearch.action.ReplicationResponse;
import org.elasticsearch.action.ShardOperationFailedException;
import org.elasticsearch.action.support.ActionFilters;
import org.elasticsearch.action.support.replication.ReplicationResponse;
import org.elasticsearch.action.support.replication.TransportBroadcastReplicationAction;
import org.elasticsearch.cluster.metadata.IndexNameExpressionResolver;
import org.elasticsearch.cluster.service.ClusterService;

View File

@ -19,14 +19,13 @@
package org.elasticsearch.action.admin.indices.flush;
import org.elasticsearch.action.ReplicationResponse;
import org.elasticsearch.action.support.ActionFilters;
import org.elasticsearch.action.support.replication.ReplicationResponse;
import org.elasticsearch.action.support.replication.TransportReplicationAction;
import org.elasticsearch.cluster.action.shard.ShardStateAction;
import org.elasticsearch.cluster.block.ClusterBlockLevel;
import org.elasticsearch.cluster.metadata.IndexNameExpressionResolver;
import org.elasticsearch.cluster.service.ClusterService;
import org.elasticsearch.common.collect.Tuple;
import org.elasticsearch.common.inject.Inject;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.index.shard.IndexShard;
@ -55,18 +54,19 @@ public class TransportShardFlushAction extends TransportReplicationAction<ShardF
}
@Override
protected Tuple<ReplicationResponse, ShardFlushRequest> shardOperationOnPrimary(ShardFlushRequest shardRequest) {
protected PrimaryResult shardOperationOnPrimary(ShardFlushRequest shardRequest) {
IndexShard indexShard = indicesService.indexServiceSafe(shardRequest.shardId().getIndex()).getShard(shardRequest.shardId().id());
indexShard.flush(shardRequest.getRequest());
logger.trace("{} flush request executed on primary", indexShard.shardId());
return new Tuple<>(new ReplicationResponse(), shardRequest);
return new PrimaryResult(shardRequest, new ReplicationResponse());
}
@Override
protected void shardOperationOnReplica(ShardFlushRequest request) {
protected ReplicaResult shardOperationOnReplica(ShardFlushRequest request) {
IndexShard indexShard = indicesService.indexServiceSafe(request.shardId().getIndex()).getShard(request.shardId().id());
indexShard.flush(request.getRequest());
logger.trace("{} flush request executed on replica", indexShard.shardId());
return new ReplicaResult();
}
@Override

View File

@ -19,10 +19,10 @@
package org.elasticsearch.action.admin.indices.refresh;
import org.elasticsearch.action.ReplicationResponse;
import org.elasticsearch.action.ShardOperationFailedException;
import org.elasticsearch.action.support.ActionFilters;
import org.elasticsearch.action.support.replication.BasicReplicationRequest;
import org.elasticsearch.action.support.replication.ReplicationResponse;
import org.elasticsearch.action.support.replication.TransportBroadcastReplicationAction;
import org.elasticsearch.cluster.metadata.IndexNameExpressionResolver;
import org.elasticsearch.cluster.service.ClusterService;

View File

@ -19,15 +19,14 @@
package org.elasticsearch.action.admin.indices.refresh;
import org.elasticsearch.action.ReplicationResponse;
import org.elasticsearch.action.support.ActionFilters;
import org.elasticsearch.action.support.replication.BasicReplicationRequest;
import org.elasticsearch.action.support.replication.ReplicationResponse;
import org.elasticsearch.action.support.replication.TransportReplicationAction;
import org.elasticsearch.cluster.action.shard.ShardStateAction;
import org.elasticsearch.cluster.block.ClusterBlockLevel;
import org.elasticsearch.cluster.metadata.IndexNameExpressionResolver;
import org.elasticsearch.cluster.service.ClusterService;
import org.elasticsearch.common.collect.Tuple;
import org.elasticsearch.common.inject.Inject;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.index.shard.IndexShard;
@ -36,10 +35,8 @@ import org.elasticsearch.indices.IndicesService;
import org.elasticsearch.threadpool.ThreadPool;
import org.elasticsearch.transport.TransportService;
/**
*
*/
public class TransportShardRefreshAction extends TransportReplicationAction<BasicReplicationRequest, BasicReplicationRequest, ReplicationResponse> {
public class TransportShardRefreshAction
extends TransportReplicationAction<BasicReplicationRequest, BasicReplicationRequest, ReplicationResponse> {
public static final String NAME = RefreshAction.NAME + "[s]";
@ -47,8 +44,8 @@ public class TransportShardRefreshAction extends TransportReplicationAction<Basi
public TransportShardRefreshAction(Settings settings, TransportService transportService, ClusterService clusterService,
IndicesService indicesService, ThreadPool threadPool, ShardStateAction shardStateAction,
ActionFilters actionFilters, IndexNameExpressionResolver indexNameExpressionResolver) {
super(settings, NAME, transportService, clusterService, indicesService, threadPool, shardStateAction,
actionFilters, indexNameExpressionResolver, BasicReplicationRequest::new, BasicReplicationRequest::new, ThreadPool.Names.REFRESH);
super(settings, NAME, transportService, clusterService, indicesService, threadPool, shardStateAction, actionFilters,
indexNameExpressionResolver, BasicReplicationRequest::new, BasicReplicationRequest::new, ThreadPool.Names.REFRESH);
}
@Override
@ -57,19 +54,20 @@ public class TransportShardRefreshAction extends TransportReplicationAction<Basi
}
@Override
protected Tuple<ReplicationResponse, BasicReplicationRequest> shardOperationOnPrimary(BasicReplicationRequest shardRequest) {
protected PrimaryResult shardOperationOnPrimary(BasicReplicationRequest shardRequest) {
IndexShard indexShard = indicesService.indexServiceSafe(shardRequest.shardId().getIndex()).getShard(shardRequest.shardId().id());
indexShard.refresh("api");
logger.trace("{} refresh request executed on primary", indexShard.shardId());
return new Tuple<>(new ReplicationResponse(), shardRequest);
return new PrimaryResult(shardRequest, new ReplicationResponse());
}
@Override
protected void shardOperationOnReplica(BasicReplicationRequest request) {
protected ReplicaResult shardOperationOnReplica(BasicReplicationRequest request) {
final ShardId shardId = request.shardId();
IndexShard indexShard = indicesService.indexServiceSafe(shardId.getIndex()).getShard(shardId.id());
indexShard.refresh("api");
logger.trace("{} refresh request executed on replica", indexShard.shardId());
return new ReplicaResult();
}
@Override

View File

@ -26,6 +26,7 @@ import org.elasticsearch.action.IndicesRequest;
import org.elasticsearch.action.WriteConsistencyLevel;
import org.elasticsearch.action.delete.DeleteRequest;
import org.elasticsearch.action.index.IndexRequest;
import org.elasticsearch.action.support.WriteRequest;
import org.elasticsearch.action.update.UpdateRequest;
import org.elasticsearch.common.Nullable;
import org.elasticsearch.common.Strings;
@ -54,16 +55,21 @@ import static org.elasticsearch.action.ValidateActions.addValidationError;
* Note that we only support refresh on the bulk request not per item.
* @see org.elasticsearch.client.Client#bulk(BulkRequest)
*/
public class BulkRequest extends ActionRequest<BulkRequest> implements CompositeIndicesRequest {
public class BulkRequest extends ActionRequest<BulkRequest> implements CompositeIndicesRequest, WriteRequest<BulkRequest> {
private static final int REQUEST_OVERHEAD = 50;
/**
* Requests that are part of this request. It is only possible to add things that are both {@link ActionRequest}s and
* {@link WriteRequest}s to this but java doesn't support syntax to declare that everything in the array has both types so we declare
* the one with the least casts.
*/
final List<ActionRequest<?>> requests = new ArrayList<>();
List<Object> payloads = null;
protected TimeValue timeout = BulkShardRequest.DEFAULT_TIMEOUT;
private WriteConsistencyLevel consistencyLevel = WriteConsistencyLevel.DEFAULT;
private boolean refresh = false;
private RefreshPolicy refreshPolicy = RefreshPolicy.NONE;
private long sizeInBytes = 0;
@ -437,18 +443,15 @@ public class BulkRequest extends ActionRequest<BulkRequest> implements Composite
return this.consistencyLevel;
}
/**
* Should a refresh be executed post this bulk operation causing the operations to
* be searchable. Note, heavy indexing should not set this to <tt>true</tt>. Defaults
* to <tt>false</tt>.
*/
public BulkRequest refresh(boolean refresh) {
this.refresh = refresh;
@Override
public BulkRequest setRefreshPolicy(RefreshPolicy refreshPolicy) {
this.refreshPolicy = refreshPolicy;
return this;
}
public boolean refresh() {
return this.refresh;
@Override
public RefreshPolicy getRefreshPolicy() {
return refreshPolicy;
}
/**
@ -483,7 +486,7 @@ public class BulkRequest extends ActionRequest<BulkRequest> implements Composite
* @return Whether this bulk request contains index request with an ingest pipeline enabled.
*/
public boolean hasIndexRequestsWithPipelines() {
for (ActionRequest actionRequest : requests) {
for (ActionRequest<?> actionRequest : requests) {
if (actionRequest instanceof IndexRequest) {
IndexRequest indexRequest = (IndexRequest) actionRequest;
if (Strings.hasText(indexRequest.getPipeline())) {
@ -503,10 +506,9 @@ public class BulkRequest extends ActionRequest<BulkRequest> implements Composite
}
for (ActionRequest<?> request : requests) {
// We first check if refresh has been set
if ((request instanceof DeleteRequest && ((DeleteRequest)request).refresh()) ||
(request instanceof UpdateRequest && ((UpdateRequest)request).refresh()) ||
(request instanceof IndexRequest && ((IndexRequest)request).refresh())) {
validationException = addValidationError("Refresh is not supported on an item request, set the refresh flag on the BulkRequest instead.", validationException);
if (((WriteRequest<?>) request).getRefreshPolicy() != RefreshPolicy.NONE) {
validationException = addValidationError(
"RefreshPolicy is not supported on an item request. Set it on the BulkRequest instead.", validationException);
}
ActionRequestValidationException ex = request.validate();
if (ex != null) {
@ -541,7 +543,7 @@ public class BulkRequest extends ActionRequest<BulkRequest> implements Composite
requests.add(request);
}
}
refresh = in.readBoolean();
refreshPolicy = RefreshPolicy.readFrom(in);
timeout = TimeValue.readTimeValue(in);
}
@ -560,7 +562,7 @@ public class BulkRequest extends ActionRequest<BulkRequest> implements Composite
}
request.writeTo(out);
}
out.writeBoolean(refresh);
refreshPolicy.writeTo(out);
timeout.writeTo(out);
}
}

View File

@ -25,6 +25,7 @@ import org.elasticsearch.action.delete.DeleteRequest;
import org.elasticsearch.action.delete.DeleteRequestBuilder;
import org.elasticsearch.action.index.IndexRequest;
import org.elasticsearch.action.index.IndexRequestBuilder;
import org.elasticsearch.action.support.WriteRequestBuilder;
import org.elasticsearch.action.update.UpdateRequest;
import org.elasticsearch.action.update.UpdateRequestBuilder;
import org.elasticsearch.client.ElasticsearchClient;
@ -35,7 +36,8 @@ import org.elasticsearch.common.unit.TimeValue;
* A bulk request holds an ordered {@link IndexRequest}s and {@link DeleteRequest}s and allows to executes
* it in a single batch.
*/
public class BulkRequestBuilder extends ActionRequestBuilder<BulkRequest, BulkResponse, BulkRequestBuilder> {
public class BulkRequestBuilder extends ActionRequestBuilder<BulkRequest, BulkResponse, BulkRequestBuilder>
implements WriteRequestBuilder<BulkRequestBuilder> {
public BulkRequestBuilder(ElasticsearchClient client, BulkAction action) {
super(client, action, new BulkRequest());
@ -116,16 +118,6 @@ public class BulkRequestBuilder extends ActionRequestBuilder<BulkRequest, BulkRe
return this;
}
/**
* Should a refresh be executed post this bulk operation causing the operations to
* be searchable. Note, heavy indexing should not set this to <tt>true</tt>. Defaults
* to <tt>false</tt>.
*/
public BulkRequestBuilder setRefresh(boolean refresh) {
request.refresh(refresh);
return this;
}
/**
* A timeout to wait if the index operation can't be performed immediately. Defaults to <tt>1m</tt>.
*/

View File

@ -19,7 +19,7 @@
package org.elasticsearch.action.bulk;
import org.elasticsearch.action.support.replication.ReplicationRequest;
import org.elasticsearch.action.support.replication.ReplicatedWriteRequest;
import org.elasticsearch.common.io.stream.StreamInput;
import org.elasticsearch.common.io.stream.StreamOutput;
import org.elasticsearch.index.shard.ShardId;
@ -31,23 +31,17 @@ import java.util.List;
/**
*
*/
public class BulkShardRequest extends ReplicationRequest<BulkShardRequest> {
public class BulkShardRequest extends ReplicatedWriteRequest<BulkShardRequest> {
private BulkItemRequest[] items;
private boolean refresh;
public BulkShardRequest() {
}
BulkShardRequest(BulkRequest bulkRequest, ShardId shardId, boolean refresh, BulkItemRequest[] items) {
BulkShardRequest(BulkRequest bulkRequest, ShardId shardId, RefreshPolicy refreshPolicy, BulkItemRequest[] items) {
super(shardId);
this.items = items;
this.refresh = refresh;
}
boolean refresh() {
return this.refresh;
setRefreshPolicy(refreshPolicy);
}
BulkItemRequest[] items() {
@ -77,7 +71,6 @@ public class BulkShardRequest extends ReplicationRequest<BulkShardRequest> {
out.writeBoolean(false);
}
}
out.writeBoolean(refresh);
}
@Override
@ -89,7 +82,6 @@ public class BulkShardRequest extends ReplicationRequest<BulkShardRequest> {
items[i] = BulkItemRequest.readBulkItem(in);
}
}
refresh = in.readBoolean();
}
@Override
@ -97,8 +89,15 @@ public class BulkShardRequest extends ReplicationRequest<BulkShardRequest> {
// This is included in error messages so we'll try to make it somewhat user friendly.
StringBuilder b = new StringBuilder("BulkShardRequest to [");
b.append(index).append("] containing [").append(items.length).append("] requests");
if (refresh) {
switch (getRefreshPolicy()) {
case IMMEDIATE:
b.append(" and a refresh");
break;
case WAIT_UNTIL:
b.append(" blocking until refresh");
break;
case NONE:
break;
}
return b.toString();
}

View File

@ -19,7 +19,9 @@
package org.elasticsearch.action.bulk;
import org.elasticsearch.action.ReplicationResponse;
import org.elasticsearch.action.DocWriteResponse;
import org.elasticsearch.action.support.WriteResponse;
import org.elasticsearch.action.support.replication.ReplicationResponse;
import org.elasticsearch.common.io.stream.StreamInput;
import org.elasticsearch.common.io.stream.StreamOutput;
import org.elasticsearch.index.shard.ShardId;
@ -29,7 +31,7 @@ import java.io.IOException;
/**
*
*/
public class BulkShardResponse extends ReplicationResponse {
public class BulkShardResponse extends ReplicationResponse implements WriteResponse {
private ShardId shardId;
private BulkItemResponse[] responses;
@ -50,6 +52,20 @@ public class BulkShardResponse extends ReplicationResponse {
return responses;
}
@Override
public void setForcedRefresh(boolean forcedRefresh) {
/*
* Each DocWriteResponse already has a location for whether or not it forced a refresh so we just set that information on the
* response.
*/
for (BulkItemResponse response : responses) {
DocWriteResponse r = response.getResponse();
if (r != null) {
r.setForcedRefresh(forcedRefresh);
}
}
}
@Override
public void readFrom(StreamInput in) throws IOException {
super.readFrom(in);

View File

@ -344,7 +344,8 @@ public class TransportBulkAction extends HandledTransportAction<BulkRequest, Bul
for (Map.Entry<ShardId, List<BulkItemRequest>> entry : requestsByShard.entrySet()) {
final ShardId shardId = entry.getKey();
final List<BulkItemRequest> requests = entry.getValue();
BulkShardRequest bulkShardRequest = new BulkShardRequest(bulkRequest, shardId, bulkRequest.refresh(), requests.toArray(new BulkItemRequest[requests.size()]));
BulkShardRequest bulkShardRequest = new BulkShardRequest(bulkRequest, shardId, bulkRequest.getRefreshPolicy(),
requests.toArray(new BulkItemRequest[requests.size()]));
bulkShardRequest.consistencyLevel(bulkRequest.consistencyLevel());
bulkShardRequest.timeout(bulkRequest.timeout());
if (task != null) {

View File

@ -30,7 +30,8 @@ import org.elasticsearch.action.index.IndexResponse;
import org.elasticsearch.action.index.TransportIndexAction;
import org.elasticsearch.action.support.ActionFilters;
import org.elasticsearch.action.support.replication.ReplicationRequest;
import org.elasticsearch.action.support.replication.TransportReplicationAction;
import org.elasticsearch.action.support.replication.TransportWriteAction;
import org.elasticsearch.action.support.replication.ReplicationResponse.ShardInfo;
import org.elasticsearch.action.update.UpdateHelper;
import org.elasticsearch.action.update.UpdateRequest;
import org.elasticsearch.action.update.UpdateResponse;
@ -53,6 +54,7 @@ import org.elasticsearch.index.engine.VersionConflictEngineException;
import org.elasticsearch.index.shard.IndexShard;
import org.elasticsearch.index.shard.ShardId;
import org.elasticsearch.index.translog.Translog;
import org.elasticsearch.index.translog.Translog.Location;
import org.elasticsearch.indices.IndicesService;
import org.elasticsearch.rest.RestStatus;
import org.elasticsearch.threadpool.ThreadPool;
@ -67,7 +69,7 @@ import static org.elasticsearch.action.support.replication.ReplicationOperation.
/**
* Performs the index operation.
*/
public class TransportShardBulkAction extends TransportReplicationAction<BulkShardRequest, BulkShardRequest, BulkShardResponse> {
public class TransportShardBulkAction extends TransportWriteAction<BulkShardRequest, BulkShardResponse> {
private final static String OP_TYPE_UPDATE = "update";
private final static String OP_TYPE_DELETE = "delete";
@ -83,9 +85,8 @@ public class TransportShardBulkAction extends TransportReplicationAction<BulkSha
IndicesService indicesService, ThreadPool threadPool, ShardStateAction shardStateAction,
MappingUpdatedAction mappingUpdatedAction, UpdateHelper updateHelper, ActionFilters actionFilters,
IndexNameExpressionResolver indexNameExpressionResolver) {
super(settings, ACTION_NAME, transportService, clusterService, indicesService, threadPool, shardStateAction,
actionFilters, indexNameExpressionResolver,
BulkShardRequest::new, BulkShardRequest::new, ThreadPool.Names.BULK);
super(settings, ACTION_NAME, transportService, clusterService, indicesService, threadPool, shardStateAction, actionFilters,
indexNameExpressionResolver, BulkShardRequest::new, ThreadPool.Names.BULK);
this.updateHelper = updateHelper;
this.allowIdGeneration = settings.getAsBoolean("action.allow_id_generation", true);
this.mappingUpdatedAction = mappingUpdatedAction;
@ -107,10 +108,9 @@ public class TransportShardBulkAction extends TransportReplicationAction<BulkSha
}
@Override
protected Tuple<BulkShardResponse, BulkShardRequest> shardOperationOnPrimary(BulkShardRequest request) {
protected WriteResult<BulkShardResponse> onPrimaryShard(BulkShardRequest request, IndexShard indexShard) throws Exception {
ShardId shardId = request.shardId();
final IndexService indexService = indicesService.indexServiceSafe(shardId.getIndex());
final IndexShard indexShard = indexService.getShard(shardId.getId());
final IndexMetaData metaData = indexService.getIndexSettings().getIndexMetaData();
long[] preVersions = new long[request.items().length];
@ -121,13 +121,13 @@ public class TransportShardBulkAction extends TransportReplicationAction<BulkSha
location = handleItem(metaData, request, indexShard, preVersions, preVersionTypes, location, requestIndex, item);
}
processAfterWrite(request.refresh(), indexShard, location);
BulkItemResponse[] responses = new BulkItemResponse[request.items().length];
BulkItemRequest[] items = request.items();
for (int i = 0; i < items.length; i++) {
responses[i] = items[i].getPrimaryResponse();
}
return new Tuple<>(new BulkShardResponse(request.shardId(), responses), request);
BulkShardResponse response = new BulkShardResponse(request.shardId(), responses);
return new WriteResult<>(response, location);
}
private Translog.Location handleItem(IndexMetaData metaData, BulkShardRequest request, IndexShard indexShard, long[] preVersions, VersionType[] preVersionTypes, Translog.Location location, int requestIndex, BulkItemRequest item) {
@ -154,9 +154,9 @@ public class TransportShardBulkAction extends TransportReplicationAction<BulkSha
preVersionTypes[requestIndex] = indexRequest.versionType();
try {
WriteResult<IndexResponse> result = shardIndexOperation(request, indexRequest, metaData, indexShard, true);
location = locationToSync(location, result.location);
location = locationToSync(location, result.getLocation());
// add the response
IndexResponse indexResponse = result.response();
IndexResponse indexResponse = result.getResponse();
setResponse(item, new BulkItemResponse(item.id(), indexRequest.opType().lowercase(), indexResponse));
} catch (Throwable e) {
// rethrow the failure if we are going to retry on primary and let parent failure to handle it
@ -197,8 +197,8 @@ public class TransportShardBulkAction extends TransportReplicationAction<BulkSha
try {
// add the response
final WriteResult<DeleteResponse> writeResult = TransportDeleteAction.executeDeleteRequestOnPrimary(deleteRequest, indexShard);
DeleteResponse deleteResponse = writeResult.response();
location = locationToSync(location, writeResult.location);
DeleteResponse deleteResponse = writeResult.getResponse();
location = locationToSync(location, writeResult.getLocation());
setResponse(item, new BulkItemResponse(item.id(), OP_TYPE_DELETE, deleteResponse));
} catch (Throwable e) {
// rethrow the failure if we are going to retry on primary and let parent failure to handle it
@ -237,16 +237,17 @@ public class TransportShardBulkAction extends TransportReplicationAction<BulkSha
}
if (updateResult.success()) {
if (updateResult.writeResult != null) {
location = locationToSync(location, updateResult.writeResult.location);
location = locationToSync(location, updateResult.writeResult.getLocation());
}
switch (updateResult.result.operation()) {
case UPSERT:
case INDEX:
@SuppressWarnings("unchecked")
WriteResult<IndexResponse> result = updateResult.writeResult;
IndexRequest indexRequest = updateResult.request();
BytesReference indexSourceAsBytes = indexRequest.source();
// add the response
IndexResponse indexResponse = result.response();
IndexResponse indexResponse = result.getResponse();
UpdateResponse updateResponse = new UpdateResponse(indexResponse.getShardInfo(), indexResponse.getShardId(), indexResponse.getType(), indexResponse.getId(), indexResponse.getVersion(), indexResponse.isCreated());
if (updateRequest.fields() != null && updateRequest.fields().length > 0) {
Tuple<XContentType, Map<String, Object>> sourceAndContent = XContentHelper.convertToMap(indexSourceAsBytes, true);
@ -256,8 +257,9 @@ public class TransportShardBulkAction extends TransportReplicationAction<BulkSha
setResponse(item, new BulkItemResponse(item.id(), OP_TYPE_UPDATE, updateResponse));
break;
case DELETE:
@SuppressWarnings("unchecked")
WriteResult<DeleteResponse> writeResult = updateResult.writeResult;
DeleteResponse response = writeResult.response();
DeleteResponse response = writeResult.getResponse();
DeleteRequest deleteRequest = updateResult.request();
updateResponse = new UpdateResponse(response.getShardInfo(), response.getShardId(), response.getType(), response.getId(), response.getVersion(), false);
updateResponse.setGetResult(updateHelper.extractGetResult(updateRequest, request.index(), response.getVersion(), updateResult.result.updatedSourceAsMap(), updateResult.result.updateSourceContentType(), null));
@ -326,11 +328,14 @@ public class TransportShardBulkAction extends TransportReplicationAction<BulkSha
request.setPrimaryResponse(response);
if (response.isFailed()) {
request.setIgnoreOnReplica();
} else {
// Set the ShardInfo to 0 so we can safely send it to the replicas. We won't use it in the real response though.
response.getResponse().setShardInfo(new ShardInfo());
}
}
private WriteResult shardIndexOperation(BulkShardRequest request, IndexRequest indexRequest, IndexMetaData metaData,
IndexShard indexShard, boolean processed) throws Throwable {
private WriteResult<IndexResponse> shardIndexOperation(BulkShardRequest request, IndexRequest indexRequest, IndexMetaData metaData,
IndexShard indexShard, boolean processed) throws Throwable {
MappingMetaData mappingMd = metaData.mappingOrDefault(indexRequest.type());
if (!processed) {
@ -431,12 +436,8 @@ public class TransportShardBulkAction extends TransportReplicationAction<BulkSha
}
}
@Override
protected void shardOperationOnReplica(BulkShardRequest request) {
final ShardId shardId = request.shardId();
IndexService indexService = indicesService.indexServiceSafe(shardId.getIndex());
IndexShard indexShard = indexService.getShard(shardId.id());
protected Location onReplicaShard(BulkShardRequest request, IndexShard indexShard) {
Translog.Location location = null;
for (int i = 0; i < request.items().length; i++) {
BulkItemRequest item = request.items()[i];
@ -472,8 +473,7 @@ public class TransportShardBulkAction extends TransportReplicationAction<BulkSha
throw new IllegalStateException("Unexpected index operation: " + item.request());
}
}
processAfterWrite(request.refresh(), indexShard, location);
return location;
}
private void applyVersion(BulkItemRequest item, long version, VersionType versionType) {

View File

@ -21,7 +21,7 @@ package org.elasticsearch.action.delete;
import org.elasticsearch.action.ActionRequestValidationException;
import org.elasticsearch.action.DocumentRequest;
import org.elasticsearch.action.support.replication.ReplicationRequest;
import org.elasticsearch.action.support.replication.ReplicatedWriteRequest;
import org.elasticsearch.common.Nullable;
import org.elasticsearch.common.io.stream.StreamInput;
import org.elasticsearch.common.io.stream.StreamOutput;
@ -43,7 +43,7 @@ import static org.elasticsearch.action.ValidateActions.addValidationError;
* @see org.elasticsearch.client.Client#delete(DeleteRequest)
* @see org.elasticsearch.client.Requests#deleteRequest(String)
*/
public class DeleteRequest extends ReplicationRequest<DeleteRequest> implements DocumentRequest<DeleteRequest> {
public class DeleteRequest extends ReplicatedWriteRequest<DeleteRequest> implements DocumentRequest<DeleteRequest> {
private String type;
private String id;
@ -51,7 +51,6 @@ public class DeleteRequest extends ReplicationRequest<DeleteRequest> implements
private String routing;
@Nullable
private String parent;
private boolean refresh;
private long version = Versions.MATCH_ANY;
private VersionType versionType = VersionType.INTERNAL;
@ -165,20 +164,6 @@ public class DeleteRequest extends ReplicationRequest<DeleteRequest> implements
return this.routing;
}
/**
* Should a refresh be executed post this index operation causing the operation to
* be searchable. Note, heavy indexing should not set this to <tt>true</tt>. Defaults
* to <tt>false</tt>.
*/
public DeleteRequest refresh(boolean refresh) {
this.refresh = refresh;
return this;
}
public boolean refresh() {
return this.refresh;
}
/**
* Sets the version, which will cause the delete operation to only be performed if a matching
* version exists and no changes happened on the doc since then.
@ -208,7 +193,6 @@ public class DeleteRequest extends ReplicationRequest<DeleteRequest> implements
id = in.readString();
routing = in.readOptionalString();
parent = in.readOptionalString();
refresh = in.readBoolean();
version = in.readLong();
versionType = VersionType.fromValue(in.readByte());
}
@ -220,7 +204,6 @@ public class DeleteRequest extends ReplicationRequest<DeleteRequest> implements
out.writeString(id);
out.writeOptionalString(routing());
out.writeOptionalString(parent());
out.writeBoolean(refresh);
out.writeLong(version);
out.writeByte(versionType.getValue());
}

View File

@ -19,6 +19,7 @@
package org.elasticsearch.action.delete;
import org.elasticsearch.action.support.WriteRequestBuilder;
import org.elasticsearch.action.support.replication.ReplicationRequestBuilder;
import org.elasticsearch.client.ElasticsearchClient;
import org.elasticsearch.common.Nullable;
@ -27,7 +28,8 @@ import org.elasticsearch.index.VersionType;
/**
* A delete document action request builder.
*/
public class DeleteRequestBuilder extends ReplicationRequestBuilder<DeleteRequest, DeleteResponse, DeleteRequestBuilder> {
public class DeleteRequestBuilder extends ReplicationRequestBuilder<DeleteRequest, DeleteResponse, DeleteRequestBuilder>
implements WriteRequestBuilder<DeleteRequestBuilder> {
public DeleteRequestBuilder(ElasticsearchClient client, DeleteAction action) {
super(client, action, new DeleteRequest());
@ -71,16 +73,6 @@ public class DeleteRequestBuilder extends ReplicationRequestBuilder<DeleteReques
return this;
}
/**
* Should a refresh be executed post this index operation causing the operation to
* be searchable. Note, heavy indexing should not set this to <tt>true</tt>. Defaults
* to <tt>false</tt>.
*/
public DeleteRequestBuilder setRefresh(boolean refresh) {
request.refresh(refresh);
return this;
}
/**
* Sets the version, which will cause the delete operation to only be performed if a matching
* version exists and no changes happened on the doc since then.

View File

@ -27,19 +27,19 @@ import org.elasticsearch.action.admin.indices.create.CreateIndexResponse;
import org.elasticsearch.action.admin.indices.create.TransportCreateIndexAction;
import org.elasticsearch.action.support.ActionFilters;
import org.elasticsearch.action.support.AutoCreateIndex;
import org.elasticsearch.action.support.replication.TransportReplicationAction;
import org.elasticsearch.action.support.replication.TransportWriteAction;
import org.elasticsearch.cluster.ClusterState;
import org.elasticsearch.cluster.action.shard.ShardStateAction;
import org.elasticsearch.cluster.metadata.IndexMetaData;
import org.elasticsearch.cluster.metadata.IndexNameExpressionResolver;
import org.elasticsearch.cluster.metadata.MetaData;
import org.elasticsearch.cluster.service.ClusterService;
import org.elasticsearch.common.collect.Tuple;
import org.elasticsearch.common.inject.Inject;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.index.engine.Engine;
import org.elasticsearch.index.shard.IndexShard;
import org.elasticsearch.index.shard.ShardId;
import org.elasticsearch.index.translog.Translog.Location;
import org.elasticsearch.indices.IndexAlreadyExistsException;
import org.elasticsearch.indices.IndicesService;
import org.elasticsearch.tasks.Task;
@ -49,7 +49,7 @@ import org.elasticsearch.transport.TransportService;
/**
* Performs the delete operation.
*/
public class TransportDeleteAction extends TransportReplicationAction<DeleteRequest, DeleteRequest, DeleteResponse> {
public class TransportDeleteAction extends TransportWriteAction<DeleteRequest, DeleteResponse> {
private final AutoCreateIndex autoCreateIndex;
private final TransportCreateIndexAction createIndexAction;
@ -60,9 +60,8 @@ public class TransportDeleteAction extends TransportReplicationAction<DeleteRequ
TransportCreateIndexAction createIndexAction, ActionFilters actionFilters,
IndexNameExpressionResolver indexNameExpressionResolver,
AutoCreateIndex autoCreateIndex) {
super(settings, DeleteAction.NAME, transportService, clusterService, indicesService, threadPool, shardStateAction,
actionFilters, indexNameExpressionResolver,
DeleteRequest::new, DeleteRequest::new, ThreadPool.Names.INDEX);
super(settings, DeleteAction.NAME, transportService, clusterService, indicesService, threadPool, shardStateAction, actionFilters,
indexNameExpressionResolver, DeleteRequest::new, ThreadPool.Names.INDEX);
this.createIndexAction = createIndexAction;
this.autoCreateIndex = autoCreateIndex;
}
@ -119,11 +118,13 @@ public class TransportDeleteAction extends TransportReplicationAction<DeleteRequ
}
@Override
protected Tuple<DeleteResponse, DeleteRequest> shardOperationOnPrimary(DeleteRequest request) {
IndexShard indexShard = indicesService.indexServiceSafe(request.shardId().getIndex()).getShard(request.shardId().id());
final WriteResult<DeleteResponse> result = executeDeleteRequestOnPrimary(request, indexShard);
processAfterWrite(request.refresh(), indexShard, result.location);
return new Tuple<>(result.response, request);
protected WriteResult<DeleteResponse> onPrimaryShard(DeleteRequest request, IndexShard indexShard) {
return executeDeleteRequestOnPrimary(request, indexShard);
}
@Override
protected Location onReplicaShard(DeleteRequest request, IndexShard indexShard) {
return executeDeleteRequestOnReplica(request, indexShard).getTranslogLocation();
}
public static WriteResult<DeleteResponse> executeDeleteRequestOnPrimary(DeleteRequest request, IndexShard indexShard) {
@ -134,9 +135,8 @@ public class TransportDeleteAction extends TransportReplicationAction<DeleteRequ
request.version(delete.version());
assert request.versionType().validateVersionForWrites(request.version());
return new WriteResult<>(
new DeleteResponse(indexShard.shardId(), request.type(), request.id(), delete.version(), delete.found()),
delete.getTranslogLocation());
DeleteResponse response = new DeleteResponse(indexShard.shardId(), request.type(), request.id(), delete.version(), delete.found());
return new WriteResult<>(response, delete.getTranslogLocation());
}
public static Engine.Delete executeDeleteRequestOnReplica(DeleteRequest request, IndexShard indexShard) {
@ -144,13 +144,4 @@ public class TransportDeleteAction extends TransportReplicationAction<DeleteRequ
indexShard.delete(delete);
return delete;
}
@Override
protected void shardOperationOnReplica(DeleteRequest request) {
final ShardId shardId = request.shardId();
IndexShard indexShard = indicesService.indexServiceSafe(shardId.getIndex()).getShard(shardId.id());
Engine.Delete delete = executeDeleteRequestOnReplica(request, indexShard);
processAfterWrite(request.refresh(), indexShard, delete.getTranslogLocation());
}
}

View File

@ -24,7 +24,7 @@ import org.elasticsearch.action.ActionRequestValidationException;
import org.elasticsearch.action.DocumentRequest;
import org.elasticsearch.action.RoutingMissingException;
import org.elasticsearch.action.TimestampParsingException;
import org.elasticsearch.action.support.replication.ReplicationRequest;
import org.elasticsearch.action.support.replication.ReplicatedWriteRequest;
import org.elasticsearch.client.Requests;
import org.elasticsearch.cluster.metadata.MappingMetaData;
import org.elasticsearch.cluster.metadata.MetaData;
@ -67,7 +67,7 @@ import static org.elasticsearch.action.ValidateActions.addValidationError;
* @see org.elasticsearch.client.Requests#indexRequest(String)
* @see org.elasticsearch.client.Client#index(IndexRequest)
*/
public class IndexRequest extends ReplicationRequest<IndexRequest> implements DocumentRequest<IndexRequest> {
public class IndexRequest extends ReplicatedWriteRequest<IndexRequest> implements DocumentRequest<IndexRequest> {
/**
* Operation type controls if the type of the index operation.
@ -145,7 +145,6 @@ public class IndexRequest extends ReplicationRequest<IndexRequest> implements Do
private OpType opType = OpType.INDEX;
private boolean refresh = false;
private long version = Versions.MATCH_ANY;
private VersionType versionType = VersionType.INTERNAL;
@ -542,20 +541,6 @@ public class IndexRequest extends ReplicationRequest<IndexRequest> implements Do
return this.opType;
}
/**
* Should a refresh be executed post this index operation causing the operation to
* be searchable. Note, heavy indexing should not set this to <tt>true</tt>. Defaults
* to <tt>false</tt>.
*/
public IndexRequest refresh(boolean refresh) {
this.refresh = refresh;
return this;
}
public boolean refresh() {
return this.refresh;
}
/**
* Sets the version, which will cause the index operation to only be performed if a matching
* version exists and no changes happened on the doc since then.
@ -652,7 +637,6 @@ public class IndexRequest extends ReplicationRequest<IndexRequest> implements Do
source = in.readBytesReference();
opType = OpType.fromId(in.readByte());
refresh = in.readBoolean();
version = in.readLong();
versionType = VersionType.fromValue(in.readByte());
pipeline = in.readOptionalString();
@ -674,7 +658,6 @@ public class IndexRequest extends ReplicationRequest<IndexRequest> implements Do
}
out.writeBytesReference(source);
out.writeByte(opType.id());
out.writeBoolean(refresh);
out.writeLong(version);
out.writeByte(versionType.getValue());
out.writeOptionalString(pipeline);

View File

@ -19,6 +19,7 @@
package org.elasticsearch.action.index;
import org.elasticsearch.action.support.WriteRequestBuilder;
import org.elasticsearch.action.support.replication.ReplicationRequestBuilder;
import org.elasticsearch.client.ElasticsearchClient;
import org.elasticsearch.common.Nullable;
@ -33,7 +34,8 @@ import java.util.Map;
/**
* An index document action request builder.
*/
public class IndexRequestBuilder extends ReplicationRequestBuilder<IndexRequest, IndexResponse, IndexRequestBuilder> {
public class IndexRequestBuilder extends ReplicationRequestBuilder<IndexRequest, IndexResponse, IndexRequestBuilder>
implements WriteRequestBuilder<IndexRequestBuilder> {
public IndexRequestBuilder(ElasticsearchClient client, IndexAction action) {
super(client, action, new IndexRequest());
@ -220,16 +222,6 @@ public class IndexRequestBuilder extends ReplicationRequestBuilder<IndexRequest,
return this;
}
/**
* Should a refresh be executed post this index operation causing the operation to
* be searchable. Note, heavy indexing should not set this to <tt>true</tt>. Defaults
* to <tt>false</tt>.
*/
public IndexRequestBuilder setRefresh(boolean refresh) {
request.refresh(refresh);
return this;
}
/**
* Sets the version, which will cause the index operation to only be performed if a matching
* version exists and no changes happened on the doc since then.

View File

@ -27,7 +27,7 @@ import org.elasticsearch.action.admin.indices.create.TransportCreateIndexAction;
import org.elasticsearch.action.support.ActionFilters;
import org.elasticsearch.action.support.AutoCreateIndex;
import org.elasticsearch.action.support.replication.ReplicationOperation;
import org.elasticsearch.action.support.replication.TransportReplicationAction;
import org.elasticsearch.action.support.replication.TransportWriteAction;
import org.elasticsearch.cluster.ClusterState;
import org.elasticsearch.cluster.action.index.MappingUpdatedAction;
import org.elasticsearch.cluster.action.shard.ShardStateAction;
@ -36,16 +36,14 @@ import org.elasticsearch.cluster.metadata.IndexNameExpressionResolver;
import org.elasticsearch.cluster.metadata.MappingMetaData;
import org.elasticsearch.cluster.metadata.MetaData;
import org.elasticsearch.cluster.service.ClusterService;
import org.elasticsearch.common.collect.Tuple;
import org.elasticsearch.common.inject.Inject;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.index.IndexService;
import org.elasticsearch.index.engine.Engine;
import org.elasticsearch.index.mapper.Mapping;
import org.elasticsearch.index.mapper.SourceToParse;
import org.elasticsearch.index.shard.IndexShard;
import org.elasticsearch.index.shard.ShardId;
import org.elasticsearch.index.translog.Translog;
import org.elasticsearch.index.translog.Translog.Location;
import org.elasticsearch.indices.IndexAlreadyExistsException;
import org.elasticsearch.indices.IndicesService;
import org.elasticsearch.tasks.Task;
@ -62,7 +60,7 @@ import org.elasticsearch.transport.TransportService;
* <li><b>allowIdGeneration</b>: If the id is set not, should it be generated. Defaults to <tt>true</tt>.
* </ul>
*/
public class TransportIndexAction extends TransportReplicationAction<IndexRequest, IndexRequest, IndexResponse> {
public class TransportIndexAction extends TransportWriteAction<IndexRequest, IndexResponse> {
private final AutoCreateIndex autoCreateIndex;
private final boolean allowIdGeneration;
@ -78,7 +76,7 @@ public class TransportIndexAction extends TransportReplicationAction<IndexReques
ActionFilters actionFilters, IndexNameExpressionResolver indexNameExpressionResolver,
AutoCreateIndex autoCreateIndex) {
super(settings, IndexAction.NAME, transportService, clusterService, indicesService, threadPool, shardStateAction,
actionFilters, indexNameExpressionResolver, IndexRequest::new, IndexRequest::new, ThreadPool.Names.INDEX);
actionFilters, indexNameExpressionResolver, IndexRequest::new, ThreadPool.Names.INDEX);
this.mappingUpdatedAction = mappingUpdatedAction;
this.createIndexAction = createIndexAction;
this.autoCreateIndex = autoCreateIndex;
@ -141,26 +139,13 @@ public class TransportIndexAction extends TransportReplicationAction<IndexReques
}
@Override
protected Tuple<IndexResponse, IndexRequest> shardOperationOnPrimary(IndexRequest request) throws Exception {
IndexService indexService = indicesService.indexServiceSafe(request.shardId().getIndex());
IndexShard indexShard = indexService.getShard(request.shardId().id());
final WriteResult<IndexResponse> result = executeIndexRequestOnPrimary(request, indexShard, mappingUpdatedAction);
final IndexResponse response = result.response;
final Translog.Location location = result.location;
processAfterWrite(request.refresh(), indexShard, location);
return new Tuple<>(response, request);
protected WriteResult<IndexResponse> onPrimaryShard(IndexRequest request, IndexShard indexShard) throws Exception {
return executeIndexRequestOnPrimary(request, indexShard, mappingUpdatedAction);
}
@Override
protected void shardOperationOnReplica(IndexRequest request) {
final ShardId shardId = request.shardId();
IndexService indexService = indicesService.indexServiceSafe(shardId.getIndex());
IndexShard indexShard = indexService.getShard(shardId.id());
final Engine.Index operation = executeIndexRequestOnReplica(request, indexShard);
processAfterWrite(request.refresh(), indexShard, operation.getTranslogLocation());
protected Location onReplicaShard(IndexRequest request, IndexShard indexShard) {
return executeIndexRequestOnReplica(request, indexShard).getTranslogLocation();
}
/**
@ -188,11 +173,8 @@ public class TransportIndexAction extends TransportReplicationAction<IndexReques
return indexShard.prepareIndexOnPrimary(sourceToParse, request.version(), request.versionType());
}
/**
* Execute the given {@link IndexRequest} on a primary shard, throwing a
* {@link ReplicationOperation.RetryOnPrimaryException} if the operation needs to be re-tried.
*/
public static WriteResult<IndexResponse> executeIndexRequestOnPrimary(IndexRequest request, IndexShard indexShard, MappingUpdatedAction mappingUpdatedAction) throws Exception {
public static WriteResult<IndexResponse> executeIndexRequestOnPrimary(IndexRequest request, IndexShard indexShard,
MappingUpdatedAction mappingUpdatedAction) throws Exception {
Engine.Index operation = prepareIndexOperationOnPrimary(request, indexShard);
Mapping update = operation.parsedDoc().dynamicMappingsUpdate();
final ShardId shardId = indexShard.shardId();
@ -214,8 +196,8 @@ public class TransportIndexAction extends TransportReplicationAction<IndexReques
assert request.versionType().validateVersionForWrites(request.version());
return new WriteResult<>(new IndexResponse(shardId, request.type(), request.id(), request.version(), created), operation.getTranslogLocation());
IndexResponse response = new IndexResponse(shardId, request.type(), request.id(), request.version(), created);
return new WriteResult<>(response, operation.getTranslogLocation());
}
}

View File

@ -162,7 +162,7 @@ public final class IngestActionFilter extends AbstractComponent implements Actio
return bulkRequest;
} else {
BulkRequest modifiedBulkRequest = new BulkRequest();
modifiedBulkRequest.refresh(bulkRequest.refresh());
modifiedBulkRequest.setRefreshPolicy(bulkRequest.getRefreshPolicy());
modifiedBulkRequest.consistencyLevel(bulkRequest.consistencyLevel());
modifiedBulkRequest.timeout(bulkRequest.timeout());

View File

@ -0,0 +1,109 @@
/*
* 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;
import org.elasticsearch.action.ActionRequestValidationException;
import org.elasticsearch.action.bulk.BulkRequest;
import org.elasticsearch.action.index.IndexRequest;
import org.elasticsearch.action.support.replication.ReplicatedWriteRequest;
import org.elasticsearch.action.update.UpdateRequest;
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.io.stream.Writeable;
import java.io.IOException;
/**
* Interface implemented by requests that modify the documents in an index like {@link IndexRequest}, {@link UpdateRequest}, and
* {@link BulkRequest}. Rather than implement this directly most implementers should extend {@link ReplicatedWriteRequest}.
*/
public interface WriteRequest<R extends WriteRequest<R>> extends Streamable {
/**
* Should this request trigger a refresh ({@linkplain RefreshPolicy#IMMEDIATE}), wait for a refresh (
* {@linkplain RefreshPolicy#WAIT_UNTIL}), or proceed ignore refreshes entirely ({@linkplain RefreshPolicy#NONE}, the default).
*/
R setRefreshPolicy(RefreshPolicy refreshPolicy);
/**
* Parse the refresh policy from a string, only modifying it if the string is non null. Convenient to use with request parsing.
*/
@SuppressWarnings("unchecked")
default R setRefreshPolicy(String refreshPolicy) {
if (refreshPolicy != null) {
setRefreshPolicy(RefreshPolicy.parse(refreshPolicy));
}
return (R) this;
}
/**
* Should this request trigger a refresh ({@linkplain RefreshPolicy#IMMEDIATE}), wait for a refresh (
* {@linkplain RefreshPolicy#WAIT_UNTIL}), or proceed ignore refreshes entirely ({@linkplain RefreshPolicy#NONE}, the default).
*/
RefreshPolicy getRefreshPolicy();
ActionRequestValidationException validate();
enum RefreshPolicy implements Writeable {
/**
* Don't refresh after this request. The default.
*/
NONE,
/**
* Force a refresh as part of this request. This refresh policy does not scale for high indexing or search throughput but is useful
* to present a consistent view to for indices with very low traffic. And it is wonderful for tests!
*/
IMMEDIATE,
/**
* Leave this request open until a refresh has made the contents of this request visible to search. This refresh policy is
* compatible with high indexing and search throughput but it causes the request to wait to reply until a refresh occurs.
*/
WAIT_UNTIL;
/**
* Parse the string representation of a refresh policy, usually from a request parameter.
*/
public static RefreshPolicy parse(String string) {
switch (string) {
case "false":
return NONE;
/*
* Empty string is IMMEDIATE because that makes "POST /test/test/1?refresh" perform a refresh which reads well and is what folks
* are used to.
*/
case "":
case "true":
return IMMEDIATE;
case "wait_for":
return WAIT_UNTIL;
}
throw new IllegalArgumentException("Unknown value for refresh: [" + string + "].");
}
public static RefreshPolicy readFrom(StreamInput in) throws IOException {
return RefreshPolicy.values()[in.readByte()];
}
@Override
public void writeTo(StreamOutput out) throws IOException {
out.writeByte((byte) ordinal());
}
}
}

View File

@ -0,0 +1,50 @@
/*
* 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;
import org.elasticsearch.Version;
import org.elasticsearch.action.support.WriteRequest.RefreshPolicy;
public interface WriteRequestBuilder<B extends WriteRequestBuilder<B>> {
WriteRequest<?> request();
/**
* Should this request trigger a refresh ({@linkplain RefreshPolicy#IMMEDIATE}), wait for a refresh (
* {@linkplain RefreshPolicy#WAIT_UNTIL}), or proceed ignore refreshes entirely ({@linkplain RefreshPolicy#NONE}, the default).
*/
@SuppressWarnings("unchecked")
default B setRefreshPolicy(RefreshPolicy refreshPolicy) {
request().setRefreshPolicy(refreshPolicy);
return (B) this;
}
/**
* If set to true then this request will force an immediate refresh. Backwards compatibility layer for Elasticsearch's old
* {@code setRefresh} calls.
*
* @deprecated use {@link #setRefreshPolicy(RefreshPolicy)} with {@link RefreshPolicy#IMMEDIATE} or {@link RefreshPolicy#NONE} instead.
* Will be removed in 6.0.
*/
@Deprecated
default B setRefresh(boolean refresh) {
assert Version.CURRENT.major < 6 : "Remove setRefresh(boolean) in 6.0";
return setRefreshPolicy(refresh ? RefreshPolicy.IMMEDIATE : RefreshPolicy.NONE);
}
}

View File

@ -0,0 +1,40 @@
/*
* 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;
import org.elasticsearch.action.DocWriteResponse;
import org.elasticsearch.action.bulk.BulkResponse;
import org.elasticsearch.action.index.IndexResponse;
import org.elasticsearch.action.support.WriteRequest.RefreshPolicy;
import org.elasticsearch.action.update.UpdateResponse;
import org.elasticsearch.index.IndexSettings;
/**
* Interface implemented by responses for actions that modify the documents in an index like {@link IndexResponse}, {@link UpdateResponse},
* and {@link BulkResponse}. Rather than implement this directly most implementers should extend {@link DocWriteResponse}.
*/
public interface WriteResponse {
/**
* Mark the response as having forced a refresh? Requests that set {@link WriteRequest#setRefreshPolicy(RefreshPolicy)} to
* {@link RefreshPolicy#IMMEDIATE} should always mark this as true. Requests that set it to {@link RefreshPolicy#WAIT_UNTIL} will only
* set this to true if they run out of refresh listener slots (see {@link IndexSettings#MAX_REFRESH_LISTENERS_PER_SHARD}).
*/
public abstract void setForcedRefresh(boolean forcedRefresh);
}

View File

@ -0,0 +1,72 @@
/*
* 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.replication;
import org.elasticsearch.action.bulk.BulkShardRequest;
import org.elasticsearch.action.delete.DeleteRequest;
import org.elasticsearch.action.index.IndexRequest;
import org.elasticsearch.action.support.WriteRequest;
import org.elasticsearch.common.io.stream.StreamInput;
import org.elasticsearch.common.io.stream.StreamOutput;
import org.elasticsearch.index.shard.ShardId;
import java.io.IOException;
/**
* Requests that are both {@linkplain ReplicationRequest}s (run on a shard's primary first, then the replica) and {@linkplain WriteRequest}
* (modify documents on a shard), for example {@link BulkShardRequest}, {@link IndexRequest}, and {@link DeleteRequest}.
*/
public abstract class ReplicatedWriteRequest<R extends ReplicatedWriteRequest<R>> extends ReplicationRequest<R> implements WriteRequest<R> {
private RefreshPolicy refreshPolicy = RefreshPolicy.NONE;
/**
* Constructor for deserialization.
*/
public ReplicatedWriteRequest() {
}
public ReplicatedWriteRequest(ShardId shardId) {
super(shardId);
}
@Override
@SuppressWarnings("unchecked")
public R setRefreshPolicy(RefreshPolicy refreshPolicy) {
this.refreshPolicy = refreshPolicy;
return (R) this;
}
@Override
public RefreshPolicy getRefreshPolicy() {
return refreshPolicy;
}
@Override
public void readFrom(StreamInput in) throws IOException {
super.readFrom(in);
refreshPolicy = RefreshPolicy.readFrom(in);
}
@Override
public void writeTo(StreamOutput out) throws IOException {
super.writeTo(out);
refreshPolicy.writeTo(out);
}
}

View File

@ -21,7 +21,6 @@ package org.elasticsearch.action.support.replication;
import org.elasticsearch.ElasticsearchException;
import org.elasticsearch.ExceptionsHelper;
import org.elasticsearch.action.ActionListener;
import org.elasticsearch.action.ReplicationResponse;
import org.elasticsearch.action.UnavailableShardsException;
import org.elasticsearch.action.WriteConsistencyLevel;
import org.elasticsearch.action.support.TransportActions;
@ -29,7 +28,6 @@ import org.elasticsearch.cluster.ClusterState;
import org.elasticsearch.cluster.routing.IndexRoutingTable;
import org.elasticsearch.cluster.routing.IndexShardRoutingTable;
import org.elasticsearch.cluster.routing.ShardRouting;
import org.elasticsearch.common.collect.Tuple;
import org.elasticsearch.common.io.stream.StreamInput;
import org.elasticsearch.common.logging.ESLogger;
import org.elasticsearch.index.engine.VersionConflictEngineException;
@ -47,28 +45,41 @@ import java.util.concurrent.atomic.AtomicInteger;
import java.util.function.Consumer;
import java.util.function.Supplier;
public class ReplicationOperation<Request extends ReplicationRequest<Request>, ReplicaRequest extends ReplicationRequest<ReplicaRequest>,
Response extends ReplicationResponse> {
public class ReplicationOperation<
Request extends ReplicationRequest<Request>,
ReplicaRequest extends ReplicationRequest<ReplicaRequest>,
PrimaryResultT extends ReplicationOperation.PrimaryResult<ReplicaRequest>
> {
final private ESLogger logger;
final private Request request;
final private Supplier<ClusterState> clusterStateSupplier;
final private String opType;
final private AtomicInteger totalShards = new AtomicInteger();
/**
* The number of pending sub-operations in this operation. This is incremented when the following operations start and decremented when
* they complete:
* <ul>
* <li>The operation on the primary</li>
* <li>The operation on each replica</li>
* <li>Coordination of the operation as a whole. This prevents the operation from terminating early if we haven't started any replica
* operations and the primary finishes.</li>
* </ul>
*/
final private AtomicInteger pendingShards = new AtomicInteger();
final private AtomicInteger successfulShards = new AtomicInteger();
final private boolean executeOnReplicas;
final private boolean checkWriteConsistency;
final private Primary<Request, ReplicaRequest, Response> primary;
final private Primary<Request, ReplicaRequest, PrimaryResultT> primary;
final private Replicas<ReplicaRequest> replicasProxy;
final private AtomicBoolean finished = new AtomicBoolean();
final protected ActionListener<Response> finalResponseListener;
final protected ActionListener<PrimaryResultT> resultListener;
private volatile Response finalResponse = null;
private volatile PrimaryResultT primaryResult = null;
private final List<ReplicationResponse.ShardInfo.Failure> shardReplicaFailures = Collections.synchronizedList(new ArrayList<>());
ReplicationOperation(Request request, Primary<Request, ReplicaRequest, Response> primary,
ActionListener<Response> listener,
ReplicationOperation(Request request, Primary<Request, ReplicaRequest, PrimaryResultT> primary,
ActionListener<PrimaryResultT> listener,
boolean executeOnReplicas, boolean checkWriteConsistency,
Replicas<ReplicaRequest> replicas,
Supplier<ClusterState> clusterStateSupplier, ESLogger logger, String opType) {
@ -76,7 +87,7 @@ public class ReplicationOperation<Request extends ReplicationRequest<Request>, R
this.executeOnReplicas = executeOnReplicas;
this.replicasProxy = replicas;
this.primary = primary;
this.finalResponseListener = listener;
this.resultListener = listener;
this.logger = logger;
this.request = request;
this.clusterStateSupplier = clusterStateSupplier;
@ -85,28 +96,27 @@ public class ReplicationOperation<Request extends ReplicationRequest<Request>, R
void execute() throws Exception {
final String writeConsistencyFailure = checkWriteConsistency ? checkWriteConsistency() : null;
final ShardId shardId = primary.routingEntry().shardId();
final ShardRouting primaryRouting = primary.routingEntry();
final ShardId primaryId = primaryRouting.shardId();
if (writeConsistencyFailure != null) {
finishAsFailed(new UnavailableShardsException(shardId,
finishAsFailed(new UnavailableShardsException(primaryId,
"{} Timeout: [{}], request: [{}]", writeConsistencyFailure, request.timeout(), request));
return;
}
totalShards.incrementAndGet();
pendingShards.incrementAndGet(); // increase by 1 until we finish all primary coordination
Tuple<Response, ReplicaRequest> primaryResponse = primary.perform(request);
successfulShards.incrementAndGet(); // mark primary as successful
finalResponse = primaryResponse.v1();
ReplicaRequest replicaRequest = primaryResponse.v2();
pendingShards.incrementAndGet();
primaryResult = primary.perform(request);
final ReplicaRequest replicaRequest = primaryResult.replicaRequest();
assert replicaRequest.primaryTerm() > 0 : "replicaRequest doesn't have a primary term";
if (logger.isTraceEnabled()) {
logger.trace("[{}] op [{}] completed on primary for request [{}]", shardId, opType, request);
logger.trace("[{}] op [{}] completed on primary for request [{}]", primaryId, opType, request);
}
// we have to get a new state after successfully indexing into the primary in order to honour recovery semantics.
// we have to make sure that every operation indexed into the primary after recovery start will also be replicated
// to the recovery target. If we use an old cluster state, we may miss a relocation that has started since then.
// If the index gets deleted after primary operation, we skip replication
List<ShardRouting> shards = getShards(shardId, clusterStateSupplier.get());
final List<ShardRouting> shards = getShards(primaryId, clusterStateSupplier.get());
final String localNodeId = primary.routingEntry().currentNodeId();
for (final ShardRouting shard : shards) {
if (executeOnReplicas == false || shard.unassigned()) {
@ -125,8 +135,8 @@ public class ReplicationOperation<Request extends ReplicationRequest<Request>, R
}
}
// decrement pending and finish (if there are no replicas, or those are done)
decPendingAndFinishIfNeeded(); // incremented in the beginning of this method
successfulShards.incrementAndGet();
decPendingAndFinishIfNeeded();
}
private void performOnReplica(final ShardRouting shard, final ReplicaRequest replicaRequest) {
@ -241,19 +251,19 @@ public class ReplicationOperation<Request extends ReplicationRequest<Request>, R
failuresArray = new ReplicationResponse.ShardInfo.Failure[shardReplicaFailures.size()];
shardReplicaFailures.toArray(failuresArray);
}
finalResponse.setShardInfo(new ReplicationResponse.ShardInfo(
primaryResult.setShardInfo(new ReplicationResponse.ShardInfo(
totalShards.get(),
successfulShards.get(),
failuresArray
)
);
finalResponseListener.onResponse(finalResponse);
resultListener.onResponse(primaryResult);
}
}
private void finishAsFailed(Throwable throwable) {
if (finished.compareAndSet(false, true)) {
finalResponseListener.onFailure(throwable);
resultListener.onFailure(throwable);
}
}
@ -284,22 +294,31 @@ public class ReplicationOperation<Request extends ReplicationRequest<Request>, R
}
interface Primary<Request extends ReplicationRequest<Request>, ReplicaRequest extends ReplicationRequest<ReplicaRequest>,
Response extends ReplicationResponse> {
interface Primary<
Request extends ReplicationRequest<Request>,
ReplicaRequest extends ReplicationRequest<ReplicaRequest>,
PrimaryResultT extends PrimaryResult<ReplicaRequest>
> {
/** routing entry for this primary */
/**
* routing entry for this primary
*/
ShardRouting routingEntry();
/** fail the primary, typically due to the fact that the operation has learned the primary has been demoted by the master */
/**
* fail the primary, typically due to the fact that the operation has learned the primary has been demoted by the master
*/
void failShard(String message, Throwable throwable);
/**
* Performs the given request on this primary
* Performs the given request on this primary. Yes, this returns as soon as it can with the request for the replicas and calls a
* listener when the primary request is completed. Yes, the primary request might complete before the method returns. Yes, it might
* also complete after. Deal with it.
*
* @return A tuple containing not null values, as first value the result of the primary operation and as second value
* the request to be executed on the replica shards.
* @param request the request to perform
* @return the request to send to the repicas
*/
Tuple<Response, ReplicaRequest> perform(Request request) throws Exception;
PrimaryResultT perform(Request request) throws Exception;
}
@ -308,19 +327,20 @@ public class ReplicationOperation<Request extends ReplicationRequest<Request>, R
/**
* performs the the given request on the specified replica
*
* @param replica {@link ShardRouting} of the shard this request should be executed on
* @param replica {@link ShardRouting} of the shard this request should be executed on
* @param replicaRequest operation to peform
* @param listener a callback to call once the operation has been complicated, either successfully or with an error.
* @param listener a callback to call once the operation has been complicated, either successfully or with an error.
*/
void performOn(ShardRouting replica, ReplicaRequest replicaRequest, ActionListener<TransportResponse.Empty> listener);
/**
* Fail the specified shard, removing it from the current set of active shards
* @param replica shard to fail
* @param primary the primary shard that requested the failure
* @param message a (short) description of the reason
* @param throwable the original exception which caused the ReplicationOperation to request the shard to be failed
* @param onSuccess a callback to call when the shard has been successfully removed from the active set.
*
* @param replica shard to fail
* @param primary the primary shard that requested the failure
* @param message a (short) description of the reason
* @param throwable the original exception which caused the ReplicationOperation to request the shard to be failed
* @param onSuccess a callback to call when the shard has been successfully removed from the active set.
* @param onPrimaryDemoted a callback to call when the shard can not be failed because the current primary has been demoted
* by the master.
* @param onIgnoredFailure a callback to call when failing a shard has failed, but it that failure can be safely ignored and the
@ -345,4 +365,11 @@ public class ReplicationOperation<Request extends ReplicationRequest<Request>, R
super(in);
}
}
interface PrimaryResult<R extends ReplicationRequest<R>> {
R replicaRequest();
void setShardInfo(ReplicationResponse.ShardInfo shardInfo);
}
}

View File

@ -23,6 +23,8 @@ import org.elasticsearch.action.ActionRequest;
import org.elasticsearch.action.ActionRequestValidationException;
import org.elasticsearch.action.IndicesRequest;
import org.elasticsearch.action.WriteConsistencyLevel;
import org.elasticsearch.action.admin.indices.refresh.TransportShardRefreshAction;
import org.elasticsearch.action.index.IndexRequest;
import org.elasticsearch.action.support.IndicesOptions;
import org.elasticsearch.common.Nullable;
import org.elasticsearch.common.io.stream.StreamInput;
@ -38,7 +40,8 @@ import java.util.concurrent.TimeUnit;
import static org.elasticsearch.action.ValidateActions.addValidationError;
/**
*
* Requests that are run on a particular replica, first on the primary and then on the replicas like {@link IndexRequest} or
* {@link TransportShardRefreshAction}.
*/
public abstract class ReplicationRequest<Request extends ReplicationRequest<Request>> extends ActionRequest<Request>
implements IndicesRequest {
@ -65,7 +68,6 @@ public abstract class ReplicationRequest<Request extends ReplicationRequest<Requ
}
/**
* Creates a new request with resolved shard id
*/

View File

@ -17,10 +17,12 @@
* under the License.
*/
package org.elasticsearch.action;
package org.elasticsearch.action.support.replication;
import org.elasticsearch.ElasticsearchException;
import org.elasticsearch.ExceptionsHelper;
import org.elasticsearch.action.ActionResponse;
import org.elasticsearch.action.ShardOperationFailedException;
import org.elasticsearch.common.Nullable;
import org.elasticsearch.common.Strings;
import org.elasticsearch.common.io.stream.StreamInput;
@ -79,14 +81,16 @@ public class ReplicationResponse extends ActionResponse {
}
/**
* @return the total number of shards the write should go to (replicas and primaries). This includes relocating shards, so this number can be higher than the number of shards.
* @return the total number of shards the write should go to (replicas and primaries). This includes relocating shards, so this
* number can be higher than the number of shards.
*/
public int getTotal() {
return total;
}
/**
* @return the total number of shards the write succeeded on (replicas and primaries). This includes relocating shards, so this number can be higher than the number of shards.
* @return the total number of shards the write succeeded on (replicas and primaries). This includes relocating shards, so this
* number can be higher than the number of shards.
*/
public int getSuccessful() {
return successful;

View File

@ -22,7 +22,6 @@ package org.elasticsearch.action.support.replication;
import com.carrotsearch.hppc.cursors.IntObjectCursor;
import org.elasticsearch.ExceptionsHelper;
import org.elasticsearch.action.ActionListener;
import org.elasticsearch.action.ReplicationResponse;
import org.elasticsearch.action.ShardOperationFailedException;
import org.elasticsearch.action.support.ActionFilters;
import org.elasticsearch.action.support.DefaultShardOperationFailedException;

View File

@ -22,7 +22,6 @@ package org.elasticsearch.action.support.replication;
import org.elasticsearch.ElasticsearchException;
import org.elasticsearch.action.ActionListener;
import org.elasticsearch.action.ActionListenerResponseHandler;
import org.elasticsearch.action.ReplicationResponse;
import org.elasticsearch.action.UnavailableShardsException;
import org.elasticsearch.action.WriteConsistencyLevel;
import org.elasticsearch.action.support.ActionFilters;
@ -41,7 +40,6 @@ import org.elasticsearch.cluster.node.DiscoveryNode;
import org.elasticsearch.cluster.routing.IndexShardRoutingTable;
import org.elasticsearch.cluster.routing.ShardRouting;
import org.elasticsearch.cluster.service.ClusterService;
import org.elasticsearch.common.collect.Tuple;
import org.elasticsearch.common.io.stream.StreamInput;
import org.elasticsearch.common.lease.Releasable;
import org.elasticsearch.common.settings.Settings;
@ -53,7 +51,6 @@ import org.elasticsearch.index.IndexService;
import org.elasticsearch.index.shard.IndexShard;
import org.elasticsearch.index.shard.IndexShardState;
import org.elasticsearch.index.shard.ShardId;
import org.elasticsearch.index.translog.Translog;
import org.elasticsearch.indices.IndicesService;
import org.elasticsearch.node.NodeClosedException;
import org.elasticsearch.tasks.Task;
@ -66,6 +63,7 @@ import org.elasticsearch.transport.TransportException;
import org.elasticsearch.transport.TransportRequestHandler;
import org.elasticsearch.transport.TransportRequestOptions;
import org.elasticsearch.transport.TransportResponse;
import org.elasticsearch.transport.TransportResponse.Empty;
import org.elasticsearch.transport.TransportService;
import java.io.IOException;
@ -81,9 +79,11 @@ import java.util.function.Supplier;
* primary node to validate request before primary operation followed by sampling state again for resolving
* nodes with replica copies to perform replication.
*/
public abstract class TransportReplicationAction<Request extends ReplicationRequest<Request>,
ReplicaRequest extends ReplicationRequest<ReplicaRequest>,
Response extends ReplicationResponse> extends TransportAction<Request, Response> {
public abstract class TransportReplicationAction<
Request extends ReplicationRequest<Request>,
ReplicaRequest extends ReplicationRequest<ReplicaRequest>,
Response extends ReplicationResponse
> extends TransportAction<Request, Response> {
final protected TransportService transportService;
final protected ClusterService clusterService;
@ -149,17 +149,17 @@ public abstract class TransportReplicationAction<Request extends ReplicationRequ
}
/**
* Primary operation on node with primary copy
* Primary operation on node with primary copy.
*
* @return A tuple containing not null values, as first value the result of the primary operation and as second value
* the request to be executed on the replica shards.
* @param shardRequest the request to the primary shard
*/
protected abstract Tuple<Response, ReplicaRequest> shardOperationOnPrimary(Request shardRequest) throws Exception;
protected abstract PrimaryResult shardOperationOnPrimary(Request shardRequest) throws Exception;
/**
* Replica operation on nodes with replica copies
* Synchronous replica operation on nodes with replica copies. This is done under the lock form
* {@link #acquireReplicaOperationLock(ShardId, long)}.
*/
protected abstract void shardOperationOnReplica(ReplicaRequest shardRequest);
protected abstract ReplicaResult shardOperationOnReplica(ReplicaRequest shardRequest);
/**
* True if write consistency should be checked for an implementation
@ -198,26 +198,6 @@ public abstract class TransportReplicationAction<Request extends ReplicationRequ
|| TransportActions.isShardNotAvailableException(e);
}
protected static class WriteResult<T extends ReplicationResponse> {
public final T response;
public final Translog.Location location;
public WriteResult(T response, Translog.Location location) {
this.response = response;
this.location = location;
}
@SuppressWarnings("unchecked")
public <T extends ReplicationResponse> T response() {
// this sets total, pending and failed to 0 and this is ok, because we will embed this into the replica
// request and not use it
response.setShardInfo(new ReplicationResponse.ShardInfo());
return (T) response;
}
}
class OperationTransportHandler implements TransportRequestHandler<Request> {
@Override
public void messageReceived(final Request request, final TransportChannel channel, Task task) throws Exception {
@ -289,7 +269,17 @@ public abstract class TransportReplicationAction<Request extends ReplicationRequ
final IndexMetaData indexMetaData = clusterService.state().getMetaData().index(request.shardId().getIndex());
final boolean executeOnReplicas = (indexMetaData == null) || shouldExecuteReplication(indexMetaData.getSettings());
final ActionListener<Response> listener = createResponseListener(channel, replicationTask, primaryShardReference);
createReplicatedOperation(request, listener, primaryShardReference, executeOnReplicas).execute();
createReplicatedOperation(request, new ActionListener<PrimaryResult>() {
@Override
public void onResponse(PrimaryResult result) {
result.respond(listener);
}
@Override
public void onFailure(Throwable e) {
listener.onFailure(e);
}
}, primaryShardReference, executeOnReplicas).execute();
success = true;
}
} finally {
@ -299,9 +289,9 @@ public abstract class TransportReplicationAction<Request extends ReplicationRequ
}
}
protected ReplicationOperation<Request, ReplicaRequest, Response>
createReplicatedOperation(Request request, ActionListener<Response> listener,
PrimaryShardReference primaryShardReference, boolean executeOnReplicas) {
protected ReplicationOperation<Request, ReplicaRequest, PrimaryResult> createReplicatedOperation(
Request request, ActionListener<PrimaryResult> listener,
PrimaryShardReference primaryShardReference, boolean executeOnReplicas) {
return new ReplicationOperation<>(request, primaryShardReference, listener,
executeOnReplicas, checkWriteConsistency(), replicasProxy, clusterService::state, logger, actionName
);
@ -339,6 +329,41 @@ public abstract class TransportReplicationAction<Request extends ReplicationRequ
}
}
protected class PrimaryResult implements ReplicationOperation.PrimaryResult<ReplicaRequest> {
final ReplicaRequest replicaRequest;
final Response finalResponse;
public PrimaryResult(ReplicaRequest replicaRequest, Response finalResponse) {
this.replicaRequest = replicaRequest;
this.finalResponse = finalResponse;
}
@Override
public ReplicaRequest replicaRequest() {
return replicaRequest;
}
@Override
public void setShardInfo(ReplicationResponse.ShardInfo shardInfo) {
finalResponse.setShardInfo(shardInfo);
}
public void respond(ActionListener<Response> listener) {
listener.onResponse(finalResponse);
}
}
protected class ReplicaResult {
/**
* Public constructor so subclasses can call it.
*/
public ReplicaResult() {}
public void respond(ActionListener<TransportResponse.Empty> listener) {
listener.onResponse(TransportResponse.Empty.INSTANCE);
}
}
class ReplicaOperationTransportHandler implements TransportRequestHandler<ReplicaRequest> {
@Override
public void messageReceived(final ReplicaRequest request, final TransportChannel channel) throws Exception {
@ -426,15 +451,35 @@ public abstract class TransportReplicationAction<Request extends ReplicationRequ
protected void doRun() throws Exception {
setPhase(task, "replica");
assert request.shardId() != null : "request shardId must be set";
ReplicaResult result;
try (Releasable ignored = acquireReplicaOperationLock(request.shardId(), request.primaryTerm())) {
shardOperationOnReplica(request);
result = shardOperationOnReplica(request);
}
result.respond(new ResponseListener());
}
/**
* Listens for the response on the replica and sends the response back to the primary.
*/
private class ResponseListener implements ActionListener<TransportResponse.Empty> {
@Override
public void onResponse(Empty response) {
if (logger.isTraceEnabled()) {
logger.trace("action [{}] completed on shard [{}] for request [{}]", transportReplicaAction, request.shardId(),
request);
request);
}
setPhase(task, "finished");
try {
channel.sendResponse(response);
} catch (Exception e) {
onFailure(e);
}
}
setPhase(task, "finished");
channel.sendResponse(TransportResponse.Empty.INSTANCE);
@Override
public void onFailure(Throwable e) {
responseWithFailure(e);
}
}
}
@ -722,7 +767,7 @@ public abstract class TransportReplicationAction<Request extends ReplicationRequ
return IndexMetaData.isIndexUsingShadowReplicas(settings) == false;
}
class PrimaryShardReference implements ReplicationOperation.Primary<Request, ReplicaRequest, Response>, Releasable {
class PrimaryShardReference implements ReplicationOperation.Primary<Request, ReplicaRequest, PrimaryResult>, Releasable {
private final IndexShard indexShard;
private final Releasable operationLock;
@ -751,9 +796,9 @@ public abstract class TransportReplicationAction<Request extends ReplicationRequ
}
@Override
public Tuple<Response, ReplicaRequest> perform(Request request) throws Exception {
Tuple<Response, ReplicaRequest> result = shardOperationOnPrimary(request);
result.v2().primaryTerm(indexShard.getPrimaryTerm());
public PrimaryResult perform(Request request) throws Exception {
PrimaryResult result = shardOperationOnPrimary(request);
result.replicaRequest().primaryTerm(indexShard.getPrimaryTerm());
return result;
}
@ -805,20 +850,6 @@ public abstract class TransportReplicationAction<Request extends ReplicationRequ
}
}
protected final void processAfterWrite(boolean refresh, IndexShard indexShard, Translog.Location location) {
if (refresh) {
try {
indexShard.refresh("refresh_flag_index");
} catch (Throwable e) {
// ignore
}
}
if (indexShard.getTranslogDurability() == Translog.Durability.REQUEST && location != null) {
indexShard.sync(location);
}
indexShard.maybeFlush();
}
/**
* Sets the current phase on the task if it isn't null. Pulled into its own
* method because its more convenient that way.

View File

@ -0,0 +1,227 @@
/*
* 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.replication;
import org.elasticsearch.action.ActionListener;
import org.elasticsearch.action.support.ActionFilters;
import org.elasticsearch.action.support.WriteRequest;
import org.elasticsearch.action.support.WriteResponse;
import org.elasticsearch.cluster.action.shard.ShardStateAction;
import org.elasticsearch.cluster.metadata.IndexNameExpressionResolver;
import org.elasticsearch.cluster.service.ClusterService;
import org.elasticsearch.common.Nullable;
import org.elasticsearch.common.logging.ESLogger;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.index.IndexService;
import org.elasticsearch.index.shard.IndexShard;
import org.elasticsearch.index.shard.ShardId;
import org.elasticsearch.index.translog.Translog;
import org.elasticsearch.index.translog.Translog.Location;
import org.elasticsearch.indices.IndicesService;
import org.elasticsearch.threadpool.ThreadPool;
import org.elasticsearch.transport.TransportResponse;
import org.elasticsearch.transport.TransportService;
import java.util.function.Supplier;
/**
* Base class for transport actions that modify data in some shard like index, delete, and shardBulk.
*/
public abstract class TransportWriteAction<
Request extends ReplicatedWriteRequest<Request>,
Response extends ReplicationResponse & WriteResponse
> extends TransportReplicationAction<Request, Request, Response> {
protected TransportWriteAction(Settings settings, String actionName, TransportService transportService,
ClusterService clusterService, IndicesService indicesService, ThreadPool threadPool, ShardStateAction shardStateAction,
ActionFilters actionFilters, IndexNameExpressionResolver indexNameExpressionResolver, Supplier<Request> request,
String executor) {
super(settings, actionName, transportService, clusterService, indicesService, threadPool, shardStateAction, actionFilters,
indexNameExpressionResolver, request, request, executor);
}
/**
* Called on the primary with a reference to the {@linkplain IndexShard} to modify.
*/
protected abstract WriteResult<Response> onPrimaryShard(Request request, IndexShard indexShard) throws Exception;
/**
* Called once per replica with a reference to the {@linkplain IndexShard} to modify.
*
* @return the translog location of the {@linkplain IndexShard} after the write was completed or null if no write occurred
*/
protected abstract Translog.Location onReplicaShard(Request request, IndexShard indexShard);
@Override
protected final WritePrimaryResult shardOperationOnPrimary(Request request) throws Exception {
IndexShard indexShard = indexShard(request);
WriteResult<Response> result = onPrimaryShard(request, indexShard);
return new WritePrimaryResult(request, result.getResponse(), result.getLocation(), indexShard);
}
@Override
protected final WriteReplicaResult shardOperationOnReplica(Request request) {
IndexShard indexShard = indexShard(request);
Translog.Location location = onReplicaShard(request, indexShard);
return new WriteReplicaResult(indexShard, request, location);
}
/**
* Fetch the IndexShard for the request. Protected so it can be mocked in tests.
*/
protected IndexShard indexShard(Request request) {
final ShardId shardId = request.shardId();
IndexService indexService = indicesService.indexServiceSafe(shardId.getIndex());
return indexService.getShard(shardId.id());
}
/**
* Simple result from a write action. Write actions have static method to return these so they can integrate with bulk.
*/
public static class WriteResult<Response extends ReplicationResponse> {
private final Response response;
private final Translog.Location location;
public WriteResult(Response response, @Nullable Location location) {
this.response = response;
this.location = location;
}
public Response getResponse() {
return response;
}
public Translog.Location getLocation() {
return location;
}
}
/**
* Result of taking the action on the primary.
*/
class WritePrimaryResult extends PrimaryResult implements RespondingWriteResult {
boolean finishedAsyncActions;
ActionListener<Response> listener = null;
public WritePrimaryResult(Request request, Response finalResponse,
@Nullable Translog.Location location,
IndexShard indexShard) {
super(request, finalResponse);
/*
* We call this before replication because this might wait for a refresh and that can take a while. This way we wait for the
* refresh in parallel on the primary and on the replica.
*/
postWriteActions(indexShard, request, location, this, logger);
}
@Override
public synchronized void respond(ActionListener<Response> listener) {
this.listener = listener;
respondIfPossible();
}
/**
* Respond if the refresh has occurred and the listener is ready. Always called while synchronized on {@code this}.
*/
protected void respondIfPossible() {
if (finishedAsyncActions && listener != null) {
super.respond(listener);
}
}
@Override
public synchronized void respondAfterAsyncAction(boolean forcedRefresh) {
finalResponse.setForcedRefresh(forcedRefresh);
finishedAsyncActions = true;
respondIfPossible();
}
}
/**
* Result of taking the action on the replica.
*/
class WriteReplicaResult extends ReplicaResult implements RespondingWriteResult {
boolean finishedAsyncActions;
private ActionListener<TransportResponse.Empty> listener;
public WriteReplicaResult(IndexShard indexShard, ReplicatedWriteRequest<?> request, Translog.Location location) {
postWriteActions(indexShard, request, location, this, logger);
}
@Override
public void respond(ActionListener<TransportResponse.Empty> listener) {
this.listener = listener;
respondIfPossible();
}
/**
* Respond if the refresh has occurred and the listener is ready. Always called while synchronized on {@code this}.
*/
protected void respondIfPossible() {
if (finishedAsyncActions && listener != null) {
super.respond(listener);
}
}
@Override
public synchronized void respondAfterAsyncAction(boolean forcedRefresh) {
finishedAsyncActions = true;
respondIfPossible();
}
}
private interface RespondingWriteResult {
void respondAfterAsyncAction(boolean forcedRefresh);
}
static void postWriteActions(final IndexShard indexShard,
final WriteRequest<?> request,
@Nullable final Translog.Location location,
final RespondingWriteResult respond,
final ESLogger logger) {
boolean pendingOps = false;
boolean immediateRefresh = false;
switch (request.getRefreshPolicy()) {
case IMMEDIATE:
indexShard.refresh("refresh_flag_index");
immediateRefresh = true;
break;
case WAIT_UNTIL:
if (location != null) {
pendingOps = true;
indexShard.addRefreshListener(location, forcedRefresh -> {
logger.warn("block_until_refresh request ran out of slots and forced a refresh: [{}]", request);
respond.respondAfterAsyncAction(forcedRefresh);
});
}
break;
case NONE:
break;
}
boolean fsyncTranslog = indexShard.getTranslogDurability() == Translog.Durability.REQUEST && location != null;
if (fsyncTranslog) {
indexShard.sync(location);
}
indexShard.maybeFlush();
if (pendingOps == false) {
respond.respondAfterAsyncAction(immediateRefresh);
}
}
}

View File

@ -26,6 +26,7 @@ import org.elasticsearch.action.RoutingMissingException;
import org.elasticsearch.action.admin.indices.create.CreateIndexRequest;
import org.elasticsearch.action.admin.indices.create.CreateIndexResponse;
import org.elasticsearch.action.admin.indices.create.TransportCreateIndexAction;
import org.elasticsearch.action.delete.DeleteRequest;
import org.elasticsearch.action.delete.DeleteResponse;
import org.elasticsearch.action.delete.TransportDeleteAction;
import org.elasticsearch.action.index.IndexRequest;
@ -187,6 +188,7 @@ public class TransportUpdateAction extends TransportInstanceSingleOperationActio
} else {
update.setGetResult(null);
}
update.setForcedRefresh(response.forcedRefresh());
listener.onResponse(update);
}
@ -219,6 +221,7 @@ public class TransportUpdateAction extends TransportInstanceSingleOperationActio
public void onResponse(IndexResponse response) {
UpdateResponse update = new UpdateResponse(response.getShardInfo(), response.getShardId(), response.getType(), response.getId(), response.getVersion(), response.isCreated());
update.setGetResult(updateHelper.extractGetResult(request, request.concreteIndex(), response.getVersion(), result.updatedSourceAsMap(), result.updateSourceContentType(), indexSourceBytes));
update.setForcedRefresh(response.forcedRefresh());
listener.onResponse(update);
}
@ -241,11 +244,13 @@ public class TransportUpdateAction extends TransportInstanceSingleOperationActio
});
break;
case DELETE:
deleteAction.execute(result.action(), new ActionListener<DeleteResponse>() {
DeleteRequest deleteRequest = result.action();
deleteAction.execute(deleteRequest, new ActionListener<DeleteResponse>() {
@Override
public void onResponse(DeleteResponse response) {
UpdateResponse update = new UpdateResponse(response.getShardInfo(), response.getShardId(), response.getType(), response.getId(), response.getVersion(), false);
update.setGetResult(updateHelper.extractGetResult(request, request.concreteIndex(), response.getVersion(), result.updatedSourceAsMap(), result.updateSourceContentType(), null));
update.setForcedRefresh(response.forcedRefresh());
listener.onResponse(update);
}

View File

@ -131,7 +131,7 @@ public class UpdateHelper extends AbstractComponent {
// it has to be a "create!"
.create(true)
.ttl(ttl)
.refresh(request.refresh())
.setRefreshPolicy(request.getRefreshPolicy())
.routing(request.routing())
.parent(request.parent())
.consistencyLevel(request.consistencyLevel());
@ -229,12 +229,13 @@ public class UpdateHelper extends AbstractComponent {
.version(updateVersion).versionType(request.versionType())
.consistencyLevel(request.consistencyLevel())
.timestamp(timestamp).ttl(ttl)
.refresh(request.refresh());
.setRefreshPolicy(request.getRefreshPolicy());
return new Result(indexRequest, Operation.INDEX, updatedSourceAsMap, updateSourceContentType);
} else if ("delete".equals(operation)) {
DeleteRequest deleteRequest = Requests.deleteRequest(request.index()).type(request.type()).id(request.id()).routing(routing).parent(parent)
.version(updateVersion).versionType(request.versionType())
.consistencyLevel(request.consistencyLevel());
.consistencyLevel(request.consistencyLevel())
.setRefreshPolicy(request.getRefreshPolicy());
return new Result(deleteRequest, Operation.DELETE, updatedSourceAsMap, updateSourceContentType);
} else if ("none".equals(operation)) {
UpdateResponse update = new UpdateResponse(shardId, getResult.getType(), getResult.getId(), getResult.getVersion(), false);

View File

@ -23,6 +23,7 @@ import org.elasticsearch.action.ActionRequestValidationException;
import org.elasticsearch.action.DocumentRequest;
import org.elasticsearch.action.WriteConsistencyLevel;
import org.elasticsearch.action.index.IndexRequest;
import org.elasticsearch.action.support.WriteRequest;
import org.elasticsearch.action.support.single.instance.InstanceShardOperationRequest;
import org.elasticsearch.common.Nullable;
import org.elasticsearch.common.ParseFieldMatcher;
@ -53,7 +54,8 @@ import static org.elasticsearch.action.ValidateActions.addValidationError;
/**
*/
public class UpdateRequest extends InstanceShardOperationRequest<UpdateRequest> implements DocumentRequest<UpdateRequest> {
public class UpdateRequest extends InstanceShardOperationRequest<UpdateRequest>
implements DocumentRequest<UpdateRequest>, WriteRequest<UpdateRequest> {
private String type;
private String id;
@ -72,7 +74,7 @@ public class UpdateRequest extends InstanceShardOperationRequest<UpdateRequest>
private VersionType versionType = VersionType.INTERNAL;
private int retryOnConflict = 0;
private boolean refresh = false;
private RefreshPolicy refreshPolicy = RefreshPolicy.NONE;
private WriteConsistencyLevel consistencyLevel = WriteConsistencyLevel.DEFAULT;
@ -422,18 +424,15 @@ public class UpdateRequest extends InstanceShardOperationRequest<UpdateRequest>
return this.versionType;
}
/**
* Should a refresh be executed post this update operation causing the operation to
* be searchable. Note, heavy indexing should not set this to <tt>true</tt>. Defaults
* to <tt>false</tt>.
*/
public UpdateRequest refresh(boolean refresh) {
this.refresh = refresh;
@Override
public UpdateRequest setRefreshPolicy(RefreshPolicy refreshPolicy) {
this.refreshPolicy = refreshPolicy;
return this;
}
public boolean refresh() {
return this.refresh;
@Override
public RefreshPolicy getRefreshPolicy() {
return refreshPolicy;
}
public WriteConsistencyLevel consistencyLevel() {
@ -730,7 +729,7 @@ public class UpdateRequest extends InstanceShardOperationRequest<UpdateRequest>
script = new Script(in);
}
retryOnConflict = in.readVInt();
refresh = in.readBoolean();
refreshPolicy = RefreshPolicy.readFrom(in);
if (in.readBoolean()) {
doc = new IndexRequest();
doc.readFrom(in);
@ -767,7 +766,7 @@ public class UpdateRequest extends InstanceShardOperationRequest<UpdateRequest>
script.writeTo(out);
}
out.writeVInt(retryOnConflict);
out.writeBoolean(refresh);
refreshPolicy.writeTo(out);
if (doc == null) {
out.writeBoolean(false);
} else {

View File

@ -21,6 +21,7 @@ package org.elasticsearch.action.update;
import org.elasticsearch.action.WriteConsistencyLevel;
import org.elasticsearch.action.index.IndexRequest;
import org.elasticsearch.action.support.WriteRequestBuilder;
import org.elasticsearch.action.support.single.instance.InstanceShardOperationRequestBuilder;
import org.elasticsearch.client.ElasticsearchClient;
import org.elasticsearch.common.bytes.BytesReference;
@ -32,9 +33,8 @@ import org.elasticsearch.script.Script;
import java.util.Map;
/**
*/
public class UpdateRequestBuilder extends InstanceShardOperationRequestBuilder<UpdateRequest, UpdateResponse, UpdateRequestBuilder> {
public class UpdateRequestBuilder extends InstanceShardOperationRequestBuilder<UpdateRequest, UpdateResponse, UpdateRequestBuilder>
implements WriteRequestBuilder<UpdateRequestBuilder> {
public UpdateRequestBuilder(ElasticsearchClient client, UpdateAction action) {
super(client, action, new UpdateRequest());
@ -121,17 +121,6 @@ public class UpdateRequestBuilder extends InstanceShardOperationRequestBuilder<U
return this;
}
/**
* Should a refresh be executed post this update operation causing the operation to
* be searchable. Note, heavy indexing should not set this to <tt>true</tt>. Defaults
* to <tt>false</tt>.
*/
public UpdateRequestBuilder setRefresh(boolean refresh) {
request.refresh(refresh);
return this;
}
/**
* Sets the consistency level of write. Defaults to {@link org.elasticsearch.action.WriteConsistencyLevel#DEFAULT}
*/

View File

@ -211,7 +211,18 @@ public class TransportClientNodesService extends AbstractComponent {
}
public <Response> void execute(NodeListenerCallback<Response> callback, ActionListener<Response> listener) {
List<DiscoveryNode> nodes = this.nodes;
// we first read nodes before checking the closed state; this
// is because otherwise we could be subject to a race where we
// read the state as not being closed, and then the client is
// closed and the nodes list is cleared, and then a
// NoNodeAvailableException is thrown
// it is important that the order of first setting the state of
// closed and then clearing the list of nodes is maintained in
// the close method
final List<DiscoveryNode> nodes = this.nodes;
if (closed) {
throw new IllegalStateException("transport client is closed");
}
ensureNodesAreAvailable(nodes);
int index = getNodeNumber();
RetryListener<Response> retryListener = new RetryListener<>(callback, listener, nodes, index);

View File

@ -256,7 +256,7 @@ public class MetaDataIndexTemplateService extends AbstractComponent {
validationErrors.add("template must not start with '_'");
}
if (!Strings.validFileNameExcludingAstrix(request.template)) {
validationErrors.add("template must not container the following characters " + Strings.INVALID_FILENAME_CHARS);
validationErrors.add("template must not contain the following characters " + Strings.INVALID_FILENAME_CHARS);
}
List<String> indexSettingsValidation = metaDataCreateIndexService.getIndexSettingsValidationErrors(request.settings);

View File

@ -108,18 +108,18 @@ public class RoutingNodes implements Iterable<RoutingNode> {
// add the counterpart shard with relocatingNodeId reflecting the source from which
// it's relocating from.
ShardRouting targetShardRouting = shard.buildTargetRelocatingShard();
addInitialRecovery(targetShardRouting, routingTable);
addInitialRecovery(targetShardRouting, indexShard.primary);
previousValue = entries.put(targetShardRouting.shardId(), targetShardRouting);
if (previousValue != null) {
throw new IllegalArgumentException("Cannot have two different shards with same shard id on same node");
}
assignedShardsAdd(targetShardRouting);
} else if (shard.active() == false) { // shards that are initializing without being relocated
} else if (shard.initializing()) {
if (shard.primary()) {
inactivePrimaryCount++;
}
inactiveShardCount++;
addInitialRecovery(shard, routingTable);
addInitialRecovery(shard, indexShard.primary);
}
} else {
unassignedShards.add(shard);
@ -134,48 +134,44 @@ public class RoutingNodes implements Iterable<RoutingNode> {
}
private void addRecovery(ShardRouting routing) {
addRecovery(routing, true, null);
updateRecoveryCounts(routing, true, findAssignedPrimaryIfPeerRecovery(routing));
}
private void removeRecovery(ShardRouting routing) {
addRecovery(routing, false, null);
updateRecoveryCounts(routing, false, findAssignedPrimaryIfPeerRecovery(routing));
}
private void addInitialRecovery(ShardRouting routing, RoutingTable routingTable) {
addRecovery(routing, true, routingTable);
private void addInitialRecovery(ShardRouting routing, ShardRouting initialPrimaryShard) {
updateRecoveryCounts(routing, true, initialPrimaryShard);
}
private void addRecovery(final ShardRouting routing, final boolean increment, final RoutingTable routingTable) {
private void updateRecoveryCounts(final ShardRouting routing, final boolean increment, @Nullable final ShardRouting primary) {
final int howMany = increment ? 1 : -1;
assert routing.initializing() : "routing must be initializing: " + routing;
// TODO: check primary == null || primary.active() after all tests properly add ReplicaAfterPrimaryActiveAllocationDecider
assert primary == null || primary.assignedToNode() :
"shard is initializing but its primary is not assigned to a node";
Recoveries.getOrAdd(recoveriesPerNode, routing.currentNodeId()).addIncoming(howMany);
final String sourceNodeId;
if (routing.relocatingNodeId() != null) { // this is a relocation-target
sourceNodeId = routing.relocatingNodeId();
if (routing.primary() && increment == false) { // primary is done relocating
if (routing.isPeerRecovery()) {
// add/remove corresponding outgoing recovery on node with primary shard
if (primary == null) {
throw new IllegalStateException("shard is peer recovering but primary is unassigned");
}
Recoveries.getOrAdd(recoveriesPerNode, primary.currentNodeId()).addOutgoing(howMany);
if (increment == false && routing.primary() && routing.relocatingNodeId() != null) {
// primary is done relocating, move non-primary recoveries from old primary to new primary
int numRecoveringReplicas = 0;
for (ShardRouting assigned : assignedShards(routing.shardId())) {
if (assigned.primary() == false && assigned.initializing() && assigned.relocatingNodeId() == null) {
if (assigned.primary() == false && assigned.isPeerRecovery()) {
numRecoveringReplicas++;
}
}
// we transfer the recoveries to the relocated primary
recoveriesPerNode.get(sourceNodeId).addOutgoing(-numRecoveringReplicas);
recoveriesPerNode.get(routing.relocatingNodeId()).addOutgoing(-numRecoveringReplicas);
recoveriesPerNode.get(routing.currentNodeId()).addOutgoing(numRecoveringReplicas);
}
} else if (routing.primary() == false) { // primary without relocationID is initial recovery
ShardRouting primary = findPrimary(routing);
if (primary == null && routingTable != null) {
primary = routingTable.index(routing.index().getName()).shard(routing.shardId().id()).primary;
} else if (primary == null) {
throw new IllegalStateException("replica is initializing but primary is unassigned");
}
sourceNodeId = primary.currentNodeId();
} else {
sourceNodeId = null;
}
if (sourceNodeId != null) {
Recoveries.getOrAdd(recoveriesPerNode, sourceNodeId).addOutgoing(howMany);
}
}
@ -187,18 +183,21 @@ public class RoutingNodes implements Iterable<RoutingNode> {
return recoveriesPerNode.getOrDefault(nodeId, Recoveries.EMPTY).getOutgoing();
}
private ShardRouting findPrimary(ShardRouting routing) {
List<ShardRouting> shardRoutings = assignedShards.get(routing.shardId());
@Nullable
private ShardRouting findAssignedPrimaryIfPeerRecovery(ShardRouting routing) {
ShardRouting primary = null;
if (shardRoutings != null) {
for (ShardRouting shardRouting : shardRoutings) {
if (shardRouting.primary()) {
if (shardRouting.active()) {
return shardRouting;
} else if (primary == null) {
primary = shardRouting;
} else if (primary.relocatingNodeId() != null) {
primary = shardRouting;
if (routing.isPeerRecovery()) {
List<ShardRouting> shardRoutings = assignedShards.get(routing.shardId());
if (shardRoutings != null) {
for (ShardRouting shardRouting : shardRoutings) {
if (shardRouting.primary()) {
if (shardRouting.active()) {
return shardRouting;
} else if (primary == null) {
primary = shardRouting;
} else if (primary.relocatingNodeId() != null) {
primary = shardRouting;
}
}
}
}
@ -500,7 +499,6 @@ public class RoutingNodes implements Iterable<RoutingNode> {
ShardRouting relocationMarkerRemoved = shard.removeRelocationSource();
updateAssigned(shard, relocationMarkerRemoved);
inactiveShardCount++; // relocation targets are not counted as inactive shards whereas initializing shards are
Recoveries.getOrAdd(recoveriesPerNode, shard.relocatingNodeId()).addOutgoing(-1);
return relocationMarkerRemoved;
}
@ -856,20 +854,17 @@ public class RoutingNodes implements Iterable<RoutingNode> {
for (ShardRouting routing : routingNode) {
if (routing.initializing()) {
incoming++;
} else if (routing.relocating()) {
outgoing++;
}
if (routing.primary() && (routing.initializing() && routing.relocatingNodeId() != null) == false) { // we don't count the initialization end of the primary relocation
List<ShardRouting> shardRoutings = routingNodes.assignedShards.get(routing.shardId());
for (ShardRouting assigned : shardRoutings) {
if (assigned.primary() == false && assigned.initializing() && assigned.relocatingNodeId() == null) {
if (routing.primary() && routing.isPeerRecovery() == false) {
for (ShardRouting assigned : routingNodes.assignedShards.get(routing.shardId())) {
if (assigned.isPeerRecovery()) {
outgoing++;
}
}
}
}
}
assert incoming == value.incoming : incoming + " != " + value.incoming;
assert incoming == value.incoming : incoming + " != " + value.incoming + " node: " + routingNode;
assert outgoing == value.outgoing : outgoing + " != " + value.outgoing + " node: " + routingNode;
}

View File

@ -28,6 +28,9 @@ import org.elasticsearch.common.settings.Setting;
import org.elasticsearch.common.settings.Setting.Property;
import org.elasticsearch.common.settings.Settings;
import static org.elasticsearch.cluster.routing.allocation.decider.Decision.THROTTLE;
import static org.elasticsearch.cluster.routing.allocation.decider.Decision.YES;
/**
* {@link ThrottlingAllocationDecider} controls the recovery process per node in
* the cluster. It exposes two settings via the cluster update API that allow
@ -109,50 +112,83 @@ public class ThrottlingAllocationDecider extends AllocationDecider {
@Override
public Decision canAllocate(ShardRouting shardRouting, RoutingNode node, RoutingAllocation allocation) {
if (shardRouting.primary()) {
assert shardRouting.unassigned() || shardRouting.active();
if (shardRouting.unassigned()) {
// primary is unassigned, means we are going to do recovery from gateway
// count *just the primary* currently doing recovery on the node and check against concurrent_recoveries
int primariesInRecovery = 0;
for (ShardRouting shard : node) {
// when a primary shard is INITIALIZING, it can be because of *initial recovery* or *relocation from another node*
// we only count initial recoveries here, so we need to make sure that relocating node is null
if (shard.initializing() && shard.primary() && shard.relocatingNodeId() == null) {
primariesInRecovery++;
}
if (shardRouting.primary() && shardRouting.unassigned()) {
assert initializingShard(shardRouting, node.nodeId()).isPeerRecovery() == false;
// primary is unassigned, means we are going to do recovery from store, snapshot or local shards
// count *just the primaries* currently doing recovery on the node and check against primariesInitialRecoveries
int primariesInRecovery = 0;
for (ShardRouting shard : node) {
// when a primary shard is INITIALIZING, it can be because of *initial recovery* or *relocation from another node*
// we only count initial recoveries here, so we need to make sure that relocating node is null
if (shard.initializing() && shard.primary() && shard.relocatingNodeId() == null) {
primariesInRecovery++;
}
if (primariesInRecovery >= primariesInitialRecoveries) {
return allocation.decision(Decision.THROTTLE, NAME, "too many primaries are currently recovering [%d], limit: [%d]",
primariesInRecovery, primariesInitialRecoveries);
}
if (primariesInRecovery >= primariesInitialRecoveries) {
// TODO: Should index creation not be throttled for primary shards?
return allocation.decision(THROTTLE, NAME, "too many primaries are currently recovering [%d], limit: [%d]",
primariesInRecovery, primariesInitialRecoveries);
} else {
return allocation.decision(YES, NAME, "below primary recovery limit of [%d]", primariesInitialRecoveries);
}
} else {
// Peer recovery
assert initializingShard(shardRouting, node.nodeId()).isPeerRecovery();
// Allocating a shard to this node will increase the incoming recoveries
int currentInRecoveries = allocation.routingNodes().getIncomingRecoveries(node.nodeId());
if (currentInRecoveries >= concurrentIncomingRecoveries) {
return allocation.decision(THROTTLE, NAME, "too many incoming shards are currently recovering [%d], limit: [%d]",
currentInRecoveries, concurrentIncomingRecoveries);
} else {
// search for corresponding recovery source (= primary shard) and check number of outgoing recoveries on that node
ShardRouting primaryShard = allocation.routingNodes().activePrimary(shardRouting.shardId());
if (primaryShard == null) {
return allocation.decision(Decision.NO, NAME, "primary shard for this replica is not yet active");
}
int primaryNodeOutRecoveries = allocation.routingNodes().getOutgoingRecoveries(primaryShard.currentNodeId());
if (primaryNodeOutRecoveries >= concurrentOutgoingRecoveries) {
return allocation.decision(THROTTLE, NAME, "too many outgoing shards are currently recovering [%d], limit: [%d]",
primaryNodeOutRecoveries, concurrentOutgoingRecoveries);
} else {
return allocation.decision(Decision.YES, NAME, "below primary recovery limit of [%d]", primariesInitialRecoveries);
return allocation.decision(YES, NAME, "below shard recovery limit of outgoing: [%d < %d] incoming: [%d < %d]",
primaryNodeOutRecoveries,
concurrentOutgoingRecoveries,
currentInRecoveries,
concurrentIncomingRecoveries);
}
}
}
// TODO should we allow shards not allocated post API to always allocate?
// either primary or replica doing recovery (from peer shard)
// count the number of recoveries on the node, its for both target (INITIALIZING) and source (RELOCATING)
return canAllocate(node, allocation);
}
@Override
public Decision canAllocate(RoutingNode node, RoutingAllocation allocation) {
int currentOutRecoveries = allocation.routingNodes().getOutgoingRecoveries(node.nodeId());
int currentInRecoveries = allocation.routingNodes().getIncomingRecoveries(node.nodeId());
if (currentOutRecoveries >= concurrentOutgoingRecoveries) {
return allocation.decision(Decision.THROTTLE, NAME, "too many outgoing shards are currently recovering [%d], limit: [%d]",
currentOutRecoveries, concurrentOutgoingRecoveries);
} else if (currentInRecoveries >= concurrentIncomingRecoveries) {
return allocation.decision(Decision.THROTTLE, NAME, "too many incoming shards are currently recovering [%d], limit: [%d]",
currentInRecoveries, concurrentIncomingRecoveries);
} else {
return allocation.decision(Decision.YES, NAME, "below shard recovery limit of outgoing: [%d < %d] incoming: [%d < %d]",
currentOutRecoveries,
concurrentOutgoingRecoveries,
currentInRecoveries,
concurrentIncomingRecoveries);
/**
* The shard routing passed to {@link #canAllocate(ShardRouting, RoutingNode, RoutingAllocation)} is not the initializing shard to this
* node but:
* - the unassigned shard routing in case if we want to assign an unassigned shard to this node.
* - the initializing shard routing if we want to assign the initializing shard to this node instead
* - the started shard routing in case if we want to check if we can relocate to this node.
* - the relocating shard routing if we want to relocate to this node now instead.
*
* This method returns the corresponding initializing shard that would be allocated to this node.
*/
private ShardRouting initializingShard(ShardRouting shardRouting, String currentNodeId) {
final ShardRouting initializingShard;
if (shardRouting.unassigned()) {
initializingShard = shardRouting.initialize(currentNodeId, null, ShardRouting.UNAVAILABLE_EXPECTED_SHARD_SIZE);
} else if (shardRouting.initializing()) {
initializingShard = shardRouting.moveToUnassigned(shardRouting.unassignedInfo())
.initialize(currentNodeId, null, ShardRouting.UNAVAILABLE_EXPECTED_SHARD_SIZE);
} else if (shardRouting.relocating()) {
initializingShard = shardRouting.cancelRelocation()
.relocate(currentNodeId, ShardRouting.UNAVAILABLE_EXPECTED_SHARD_SIZE)
.buildTargetRelocatingShard();
} else {
assert shardRouting.started();
initializingShard = shardRouting.relocate(currentNodeId, ShardRouting.UNAVAILABLE_EXPECTED_SHARD_SIZE)
.buildTargetRelocatingShard();
}
assert initializingShard.initializing();
return initializingShard;
}
}

View File

@ -170,7 +170,7 @@ public class LogConfigurator {
static void loadConfig(Path file, Settings.Builder settingsBuilder) {
try {
settingsBuilder.loadFromPath(file);
} catch (SettingsException | NoClassDefFoundError e) {
} catch (IOException | SettingsException | NoClassDefFoundError e) {
// ignore
}
}

View File

@ -190,7 +190,6 @@ public final class ClusterSettings extends AbstractScopedSettings {
RecoverySettings.INDICES_RECOVERY_ACTIVITY_TIMEOUT_SETTING,
RecoverySettings.INDICES_RECOVERY_INTERNAL_ACTION_TIMEOUT_SETTING,
RecoverySettings.INDICES_RECOVERY_INTERNAL_LONG_ACTION_TIMEOUT_SETTING,
ThreadPool.THREADPOOL_GROUP_SETTING,
ThrottlingAllocationDecider.CLUSTER_ROUTING_ALLOCATION_NODE_INITIAL_PRIMARIES_RECOVERIES_SETTING,
ThrottlingAllocationDecider.CLUSTER_ROUTING_ALLOCATION_NODE_CONCURRENT_INCOMING_RECOVERIES_SETTING,
ThrottlingAllocationDecider.CLUSTER_ROUTING_ALLOCATION_NODE_CONCURRENT_OUTGOING_RECOVERIES_SETTING,
@ -419,6 +418,7 @@ public final class ClusterSettings extends AbstractScopedSettings {
ResourceWatcherService.RELOAD_INTERVAL_HIGH,
ResourceWatcherService.RELOAD_INTERVAL_MEDIUM,
ResourceWatcherService.RELOAD_INTERVAL_LOW,
SearchModule.INDICES_MAX_CLAUSE_COUNT_SETTING
SearchModule.INDICES_MAX_CLAUSE_COUNT_SETTING,
ThreadPool.ESTIMATED_TIME_INTERVAL_SETTING
)));
}

View File

@ -115,6 +115,7 @@ public final class IndexScopedSettings extends AbstractScopedSettings {
IndexSettings.QUERY_STRING_LENIENT_SETTING,
IndexSettings.ALLOW_UNMAPPED,
IndexSettings.INDEX_CHECK_ON_STARTUP,
IndexSettings.MAX_REFRESH_LISTENERS_PER_SHARD,
ShardsLimitAllocationDecider.INDEX_TOTAL_SHARDS_PER_NODE_SETTING,
IndexSettings.INDEX_GC_DELETES_SETTING,
IndicesRequestCache.INDEX_CACHE_REQUEST_ENABLED_SETTING,

View File

@ -17,7 +17,7 @@
* under the License.
*/
package org.elasticsearch.common.property;
package org.elasticsearch.common.settings;
import org.elasticsearch.common.Strings;
@ -34,23 +34,12 @@ import java.util.Set;
* Values for substitution can be supplied using a {@link Properties} instance or using a
* {@link PlaceholderResolver}.
*/
public class PropertyPlaceholder {
class PropertyPlaceholder {
private final String placeholderPrefix;
private final String placeholderSuffix;
private final boolean ignoreUnresolvablePlaceholders;
/**
* Creates a new <code>PropertyPlaceholderHelper</code> that uses the supplied prefix and suffix. Unresolvable
* placeholders are ignored.
*
* @param placeholderPrefix the prefix that denotes the start of a placeholder.
* @param placeholderSuffix the suffix that denotes the end of a placeholder.
*/
public PropertyPlaceholder(String placeholderPrefix, String placeholderSuffix) {
this(placeholderPrefix, placeholderSuffix, true);
}
/**
* Creates a new <code>PropertyPlaceholderHelper</code> that uses the supplied prefix and suffix.
*
@ -59,12 +48,10 @@ public class PropertyPlaceholder {
* @param ignoreUnresolvablePlaceholders indicates whether unresolvable placeholders should be ignored
* (<code>true</code>) or cause an exception (<code>false</code>).
*/
public PropertyPlaceholder(String placeholderPrefix, String placeholderSuffix,
PropertyPlaceholder(String placeholderPrefix, String placeholderSuffix,
boolean ignoreUnresolvablePlaceholders) {
Objects.requireNonNull(placeholderPrefix, "Argument 'placeholderPrefix' must not be null.");
Objects.requireNonNull(placeholderSuffix, "Argument 'placeholderSuffix' must not be null.");
this.placeholderPrefix = placeholderPrefix;
this.placeholderSuffix = placeholderSuffix;
this.placeholderPrefix = Objects.requireNonNull(placeholderPrefix);
this.placeholderSuffix = Objects.requireNonNull(placeholderSuffix);
this.ignoreUnresolvablePlaceholders = ignoreUnresolvablePlaceholders;
}
@ -75,15 +62,15 @@ public class PropertyPlaceholder {
* @param value the value containing the placeholders to be replaced.
* @param placeholderResolver the <code>PlaceholderResolver</code> to use for replacement.
* @return the supplied value with placeholders replaced inline.
* @throws NullPointerException if value is null
*/
public String replacePlaceholders(String key, String value, PlaceholderResolver placeholderResolver) {
Objects.requireNonNull(key);
Objects.requireNonNull(value, "value can not be null for [" + key + "]");
return parseStringValue(value, placeholderResolver, new HashSet<String>());
String replacePlaceholders(String value, PlaceholderResolver placeholderResolver) {
Objects.requireNonNull(value);
return parseStringValue(value, placeholderResolver, new HashSet<>());
}
protected String parseStringValue(String strVal, PlaceholderResolver placeholderResolver,
Set<String> visitedPlaceholders) {
private String parseStringValue(String strVal, PlaceholderResolver placeholderResolver,
Set<String> visitedPlaceholders) {
StringBuilder buf = new StringBuilder(strVal);
int startIndex = strVal.indexOf(this.placeholderPrefix);
@ -164,7 +151,7 @@ public class PropertyPlaceholder {
*
* @see PropertyPlaceholder
*/
public interface PlaceholderResolver {
interface PlaceholderResolver {
/**
* Resolves the supplied placeholder name into the replacement value.

View File

@ -25,7 +25,6 @@ import org.elasticsearch.common.Strings;
import org.elasticsearch.common.io.Streams;
import org.elasticsearch.common.io.stream.StreamInput;
import org.elasticsearch.common.io.stream.StreamOutput;
import org.elasticsearch.common.property.PropertyPlaceholder;
import org.elasticsearch.common.settings.loader.SettingsLoader;
import org.elasticsearch.common.settings.loader.SettingsLoaderFactory;
import org.elasticsearch.common.unit.ByteSizeUnit;
@ -920,27 +919,21 @@ public final class Settings implements ToXContent {
* Loads settings from a url that represents them using the
* {@link SettingsLoaderFactory#loaderFromSource(String)}.
*/
public Builder loadFromPath(Path path) throws SettingsException {
try {
return loadFromStream(path.getFileName().toString(), Files.newInputStream(path));
} catch (IOException e) {
throw new SettingsException("Failed to open stream for url [" + path + "]", e);
}
public Builder loadFromPath(Path path) throws IOException {
// NOTE: loadFromStream will close the input stream
return loadFromStream(path.getFileName().toString(), Files.newInputStream(path));
}
/**
* Loads settings from a stream that represents them using the
* {@link SettingsLoaderFactory#loaderFromSource(String)}.
*/
public Builder loadFromStream(String resourceName, InputStream is) throws SettingsException {
public Builder loadFromStream(String resourceName, InputStream is) throws IOException {
SettingsLoader settingsLoader = SettingsLoaderFactory.loaderFromResource(resourceName);
try {
Map<String, String> loadedSettings = settingsLoader
.load(Streams.copyToString(new InputStreamReader(is, StandardCharsets.UTF_8)));
put(loadedSettings);
} catch (Exception e) {
throw new SettingsException("Failed to load settings from [" + resourceName + "]", e);
}
// NOTE: copyToString will close the input stream
Map<String, String> loadedSettings =
settingsLoader.load(Streams.copyToString(new InputStreamReader(is, StandardCharsets.UTF_8)));
put(loadedSettings);
return this;
}
@ -992,14 +985,21 @@ public final class Settings implements ToXContent {
return true;
}
};
for (Map.Entry<String, String> entry : new HashMap<>(map).entrySet()) {
String value = propertyPlaceholder.replacePlaceholders(entry.getKey(), entry.getValue(), placeholderResolver);
Iterator<Map.Entry<String, String>> entryItr = map.entrySet().iterator();
while (entryItr.hasNext()) {
Map.Entry<String, String> entry = entryItr.next();
if (entry.getValue() == null) {
// a null value obviously can't be replaced
continue;
}
String value = propertyPlaceholder.replacePlaceholders(entry.getValue(), placeholderResolver);
// if the values exists and has length, we should maintain it in the map
// otherwise, the replace process resolved into removing it
if (Strings.hasLength(value)) {
map.put(entry.getKey(), value);
entry.setValue(value);
} else {
map.remove(entry.getKey());
entryItr.remove();
}
}
return this;

View File

@ -83,13 +83,16 @@ public class EsExecutors {
}
public static String threadName(Settings settings, String namePrefix) {
String name = settings.get("node.name");
if (name == null) {
name = "elasticsearch";
String nodeName = settings.get("node.name");
if (nodeName == null) {
return threadName("", namePrefix);
} else {
name = "elasticsearch[" + name + "]";
return threadName(nodeName, namePrefix);
}
return name + "[" + namePrefix + "]";
}
public static String threadName(final String nodeName, final String namePrefix) {
return "elasticsearch" + (nodeName.isEmpty() ? "" : "[") + nodeName + (nodeName.isEmpty() ? "" : "]") + "[" + namePrefix + "]";
}
public static ThreadFactory daemonThreadFactory(Settings settings, String namePrefix) {

View File

@ -115,6 +115,11 @@ public final class IndexSettings {
public static final Setting<TimeValue> INDEX_GC_DELETES_SETTING =
Setting.timeSetting("index.gc_deletes", DEFAULT_GC_DELETES, new TimeValue(-1, TimeUnit.MILLISECONDS), Property.Dynamic,
Property.IndexScope);
/**
* The maximum number of refresh listeners allows on this shard.
*/
public static final Setting<Integer> MAX_REFRESH_LISTENERS_PER_SHARD = Setting.intSetting("index.max_refresh_listeners", 1000, 0,
Property.Dynamic, Property.IndexScope);
private final Index index;
private final Version version;
@ -145,6 +150,10 @@ public final class IndexSettings {
private volatile int maxResultWindow;
private volatile int maxRescoreWindow;
private volatile boolean TTLPurgeDisabled;
/**
* The maximum number of refresh listeners allows on this shard.
*/
private volatile int maxRefreshListeners;
/**
* Returns the default search field for this index.
@ -229,6 +238,7 @@ public final class IndexSettings {
maxResultWindow = scopedSettings.get(MAX_RESULT_WINDOW_SETTING);
maxRescoreWindow = scopedSettings.get(MAX_RESCORE_WINDOW_SETTING);
TTLPurgeDisabled = scopedSettings.get(INDEX_TTL_DISABLE_PURGE_SETTING);
maxRefreshListeners = scopedSettings.get(MAX_REFRESH_LISTENERS_PER_SHARD);
this.mergePolicyConfig = new MergePolicyConfig(logger, this);
assert indexNameMatcher.test(indexMetaData.getIndex().getName());
@ -251,6 +261,7 @@ public final class IndexSettings {
scopedSettings.addSettingsUpdateConsumer(INDEX_GC_DELETES_SETTING, this::setGCDeletes);
scopedSettings.addSettingsUpdateConsumer(INDEX_TRANSLOG_FLUSH_THRESHOLD_SIZE_SETTING, this::setTranslogFlushThresholdSize);
scopedSettings.addSettingsUpdateConsumer(INDEX_REFRESH_INTERVAL_SETTING, this::setRefreshInterval);
scopedSettings.addSettingsUpdateConsumer(MAX_REFRESH_LISTENERS_PER_SHARD, this::setMaxRefreshListeners);
}
private void setTranslogFlushThresholdSize(ByteSizeValue byteSizeValue) {
@ -499,6 +510,16 @@ public final class IndexSettings {
return scopedSettings.get(setting);
}
/**
* The maximum number of refresh listeners allows on this shard.
*/
public int getMaxRefreshListeners() {
return maxRefreshListeners;
}
private void setMaxRefreshListeners(int maxRefreshListeners) {
this.maxRefreshListeners = maxRefreshListeners;
}
IndexScopedSettings getScopedSettings() { return scopedSettings;}
}

View File

@ -607,6 +607,7 @@ public abstract class Engine implements Closeable {
* Synchronously refreshes the engine for new search operations to reflect the latest
* changes.
*/
@Nullable
public abstract void refresh(String source) throws EngineException;
/**
@ -999,6 +1000,9 @@ public abstract class Engine implements Closeable {
public static final GetResult NOT_EXISTS = new GetResult(false, Versions.NOT_FOUND, null);
/**
* Build a realtime get result from the translog.
*/
public GetResult(boolean exists, long version, @Nullable Translog.Source source) {
this.source = source;
this.exists = exists;
@ -1007,6 +1011,9 @@ public abstract class Engine implements Closeable {
this.searcher = null;
}
/**
* Build a non-realtime get result from the searcher.
*/
public GetResult(Searcher searcher, Versions.DocIdAndVersion docIdAndVersion) {
this.exists = true;
this.source = null;

View File

@ -25,14 +25,15 @@ import org.apache.lucene.index.SnapshotDeletionPolicy;
import org.apache.lucene.search.QueryCache;
import org.apache.lucene.search.QueryCachingPolicy;
import org.apache.lucene.search.similarities.Similarity;
import org.elasticsearch.common.Nullable;
import org.elasticsearch.common.settings.Setting;
import org.elasticsearch.common.settings.Setting.Property;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.unit.ByteSizeUnit;
import org.elasticsearch.common.unit.ByteSizeValue;
import org.elasticsearch.common.unit.TimeValue;
import org.elasticsearch.index.IndexSettings;
import org.elasticsearch.index.codec.CodecService;
import org.elasticsearch.index.shard.RefreshListeners;
import org.elasticsearch.index.shard.ShardId;
import org.elasticsearch.index.shard.TranslogRecoveryPerformer;
import org.elasticsearch.index.store.Store;
@ -40,8 +41,6 @@ import org.elasticsearch.index.translog.TranslogConfig;
import org.elasticsearch.indices.IndexingMemoryController;
import org.elasticsearch.threadpool.ThreadPool;
import java.util.function.Function;
/*
* Holds all the configuration that is used to create an {@link Engine}.
* Once {@link Engine} has been created with this object, changes to this
@ -66,6 +65,8 @@ public final class EngineConfig {
private final Engine.EventListener eventListener;
private final QueryCache queryCache;
private final QueryCachingPolicy queryCachingPolicy;
@Nullable
private final RefreshListeners refreshListeners;
/**
* Index setting to change the low level lucene codec used for writing new segments.
@ -99,7 +100,7 @@ public final class EngineConfig {
MergePolicy mergePolicy,Analyzer analyzer,
Similarity similarity, CodecService codecService, Engine.EventListener eventListener,
TranslogRecoveryPerformer translogRecoveryPerformer, QueryCache queryCache, QueryCachingPolicy queryCachingPolicy,
TranslogConfig translogConfig, TimeValue flushMergesAfter) {
TranslogConfig translogConfig, TimeValue flushMergesAfter, RefreshListeners refreshListeners) {
if (openMode == null) {
throw new IllegalArgumentException("openMode must not be null");
}
@ -125,6 +126,7 @@ public final class EngineConfig {
this.translogConfig = translogConfig;
this.flushMergesAfter = flushMergesAfter;
this.openMode = openMode;
this.refreshListeners = refreshListeners;
}
/**
@ -303,4 +305,10 @@ public final class EngineConfig {
OPEN_INDEX_AND_TRANSLOG;
}
/**
* {@linkplain RefreshListeners} instance to configure.
*/
public RefreshListeners getRefreshListeners() {
return refreshListeners;
}
}

View File

@ -154,6 +154,10 @@ public class InternalEngine extends Engine {
this.versionMap.setManager(searcherManager);
// don't allow commits until we are done with recovering
allowCommits.compareAndSet(true, openMode != EngineConfig.OpenMode.OPEN_INDEX_AND_TRANSLOG);
if (engineConfig.getRefreshListeners() != null) {
searcherManager.addListener(engineConfig.getRefreshListeners());
engineConfig.getRefreshListeners().setTranslog(translog);
}
success = true;
} finally {
if (success == false) {

View File

@ -30,7 +30,6 @@ import org.elasticsearch.common.lucene.Lucene;
import org.elasticsearch.common.lucene.index.ElasticsearchDirectoryReader;
import org.elasticsearch.common.unit.TimeValue;
import org.elasticsearch.common.util.concurrent.ReleasableLock;
import org.elasticsearch.index.shard.TranslogRecoveryPerformer;
import org.elasticsearch.index.translog.Translog;
import java.io.IOException;
@ -68,6 +67,9 @@ public class ShadowEngine extends Engine {
public ShadowEngine(EngineConfig engineConfig) {
super(engineConfig);
if (engineConfig.getRefreshListeners() != null) {
throw new IllegalArgumentException("ShadowEngine doesn't support RefreshListeners");
}
SearcherFactory searcherFactory = new EngineSearcherFactory(engineConfig);
final long nonexistentRetryTime = engineConfig.getIndexSettings().getSettings()
.getAsTime(NONEXISTENT_INDEX_RETRY_WAIT, DEFAULT_NONEXISTENT_INDEX_RETRY_WAIT)

View File

@ -126,7 +126,7 @@ public class WrapperQueryBuilder extends AbstractQueryBuilder<WrapperQueryBuilde
}
String fieldName = parser.currentName();
if (! parseContext.getParseFieldMatcher().match(fieldName, QUERY_FIELD)) {
throw new ParsingException(parser.getTokenLocation(), "[wrapper] query malformed, expected `query` but was" + fieldName);
throw new ParsingException(parser.getTokenLocation(), "[wrapper] query malformed, expected `query` but was " + fieldName);
}
parser.nextToken();

View File

@ -131,6 +131,7 @@ import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicBoolean;
import java.util.concurrent.atomic.AtomicLong;
import java.util.concurrent.atomic.AtomicReference;
import java.util.function.Consumer;
import java.util.function.BiConsumer;
public class IndexShard extends AbstractIndexShardComponent {
@ -203,6 +204,12 @@ public class IndexShard extends AbstractIndexShardComponent {
* IndexingMemoryController}).
*/
private final AtomicBoolean active = new AtomicBoolean();
/**
* Allows for the registration of listeners that are called when a change becomes visible for search. This is nullable because
* {@linkplain ShadowIndexShard} doesn't support this.
*/
@Nullable
private final RefreshListeners refreshListeners;
public IndexShard(ShardRouting shardRouting, IndexSettings indexSettings, ShardPath path, Store store, IndexCache indexCache,
MapperService mapperService, SimilarityService similarityService, IndexFieldDataService indexFieldDataService,
@ -255,6 +262,7 @@ public class IndexShard extends AbstractIndexShardComponent {
suspendableRefContainer = new SuspendableRefContainer();
searcherWrapper = indexSearcherWrapper;
primaryTerm = indexSettings.getIndexMetaData().primaryTerm(shardId.id());
refreshListeners = buildRefreshListeners();
persistMetadata(shardRouting, null);
}
@ -579,6 +587,7 @@ public class IndexShard extends AbstractIndexShardComponent {
*/
public void refresh(String source) {
verifyNotClosed();
if (canIndex()) {
long bytes = getEngine().getIndexBufferRAMBytesUsed();
writingBytes.addAndGet(bytes);
@ -1530,7 +1539,7 @@ public class IndexShard extends AbstractIndexShardComponent {
return new EngineConfig(openMode, shardId,
threadPool, indexSettings, warmer, store, deletionPolicy, indexSettings.getMergePolicy(),
mapperService.indexAnalyzer(), similarityService.similarity(mapperService), codecService, shardEventListener, translogRecoveryPerformer, indexCache.query(), cachingPolicy, translogConfig,
IndexingMemoryController.SHARD_INACTIVE_TIME_SETTING.get(indexSettings.getSettings()));
IndexingMemoryController.SHARD_INACTIVE_TIME_SETTING.get(indexSettings.getSettings()), refreshListeners);
}
public Releasable acquirePrimaryOperationLock() {
@ -1626,6 +1635,17 @@ public class IndexShard extends AbstractIndexShardComponent {
return false;
}
/**
* Build {@linkplain RefreshListeners} for this shard. Protected so {@linkplain ShadowIndexShard} can override it to return null.
*/
protected RefreshListeners buildRefreshListeners() {
return new RefreshListeners(
indexSettings::getMaxRefreshListeners,
() -> refresh("too_many_listeners"),
threadPool.executor(ThreadPool.Names.LISTENER)::execute,
logger);
}
/**
* Simple struct encapsulating a shard failure
*
@ -1651,14 +1671,26 @@ public class IndexShard extends AbstractIndexShardComponent {
}
/**
* Returns <code>true</code> iff one or more changes to the engine are not visible to via the current searcher.
* Returns <code>true</code> iff one or more changes to the engine are not visible to via the current searcher *or* there are pending
* refresh listeners.
* Otherwise <code>false</code>.
*
* @throws EngineClosedException if the engine is already closed
* @throws AlreadyClosedException if the internal indexwriter in the engine is already closed
*/
public boolean isRefreshNeeded() {
return getEngine().refreshNeeded();
return getEngine().refreshNeeded() || (refreshListeners != null && refreshListeners.refreshNeeded());
}
/**
* Add a listener for refreshes.
*
* @param location the location to listen for
* @param listener for the refresh. Called with true if registering the listener ran it out of slots and forced a refresh. Called with
* false otherwise.
*/
public void addRefreshListener(Translog.Location location, Consumer<Boolean> listener) {
refreshListeners.addOrNotify(location, listener);
}
private class IndexShardRecoveryPerformer extends TranslogRecoveryPerformer {

View File

@ -0,0 +1,208 @@
/*
* 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 org.apache.lucene.search.ReferenceManager;
import org.elasticsearch.common.collect.Tuple;
import org.elasticsearch.common.logging.ESLogger;
import org.elasticsearch.index.translog.Translog;
import java.io.IOException;
import java.util.ArrayList;
import java.util.List;
import java.util.concurrent.Executor;
import java.util.function.Consumer;
import java.util.function.IntSupplier;
import static java.util.Objects.requireNonNull;
/**
* Allows for the registration of listeners that are called when a change becomes visible for search. This functionality is exposed from
* {@link IndexShard} but kept here so it can be tested without standing up the entire thing.
*/
public final class RefreshListeners implements ReferenceManager.RefreshListener {
private final IntSupplier getMaxRefreshListeners;
private final Runnable forceRefresh;
private final Executor listenerExecutor;
private final ESLogger logger;
/**
* List of refresh listeners. Defaults to null and built on demand because most refresh cycles won't need it. Entries are never removed
* from it, rather, it is nulled and rebuilt when needed again. The (hopefully) rare entries that didn't make the current refresh cycle
* are just added back to the new list. Both the reference and the contents are always modified while synchronized on {@code this}.
*/
private volatile List<Tuple<Translog.Location, Consumer<Boolean>>> refreshListeners = null;
/**
* The translog location that was last made visible by a refresh.
*/
private volatile Translog.Location lastRefreshedLocation;
public RefreshListeners(IntSupplier getMaxRefreshListeners, Runnable forceRefresh, Executor listenerExecutor, ESLogger logger) {
this.getMaxRefreshListeners = getMaxRefreshListeners;
this.forceRefresh = forceRefresh;
this.listenerExecutor = listenerExecutor;
this.logger = logger;
}
/**
* Add a listener for refreshes, calling it immediately if the location is already visible. If this runs out of listener slots then it
* forces a refresh and calls the listener immediately as well.
*
* @param location the location to listen for
* @param listener for the refresh. Called with true if registering the listener ran it out of slots and forced a refresh. Called with
* false otherwise.
*/
public void addOrNotify(Translog.Location location, Consumer<Boolean> listener) {
requireNonNull(listener, "listener cannot be null");
requireNonNull(location, "location cannot be null");
if (lastRefreshedLocation != null && lastRefreshedLocation.compareTo(location) >= 0) {
// Location already visible, just call the listener
listener.accept(false);
return;
}
synchronized (this) {
if (refreshListeners == null) {
refreshListeners = new ArrayList<>();
}
if (refreshListeners.size() < getMaxRefreshListeners.getAsInt()) {
// We have a free slot so register the listener
refreshListeners.add(new Tuple<>(location, listener));
return;
}
}
// No free slot so force a refresh and call the listener in this thread
forceRefresh.run();
listener.accept(true);
}
/**
* Returns true if there are pending listeners.
*/
public boolean refreshNeeded() {
// No need to synchronize here because we're doing a single volatile read
return refreshListeners != null;
}
/**
* Setup the translog used to find the last refreshed location.
*/
public void setTranslog(Translog translog) {
this.translog = translog;
}
// Implementation of ReferenceManager.RefreshListener that adapts Lucene's RefreshListener into Elasticsearch's refresh listeners.
private Translog translog;
/**
* Snapshot of the translog location before the current refresh if there is a refresh going on or null. Doesn't have to be volatile
* because when it is used by the refreshing thread.
*/
private Translog.Location currentRefreshLocation;
@Override
public void beforeRefresh() throws IOException {
currentRefreshLocation = translog.getLastWriteLocation();
}
@Override
public void afterRefresh(boolean didRefresh) throws IOException {
/*
* We intentionally ignore didRefresh here because our timing is a little off. It'd be a useful flag if we knew everything that made
* it into the refresh, but the way we snapshot the translog position before the refresh, things can sneak into the refresh that we
* don't know about.
*/
if (null == currentRefreshLocation) {
/*
* The translog had an empty last write location at the start of the refresh so we can't alert anyone to anything. This
* usually happens during recovery. The next refresh cycle out to pick up this refresh.
*/
return;
}
// First check if we've actually moved forward. If not then just bail immediately.
assert lastRefreshedLocation == null || currentRefreshLocation.compareTo(lastRefreshedLocation) >= 0;
if (lastRefreshedLocation != null && currentRefreshLocation.compareTo(lastRefreshedLocation) == 0) {
return;
}
/*
* Set the lastRefreshedLocation so listeners that come in for locations before that will just execute inline without messing
* around with refreshListeners or synchronizing at all.
*/
lastRefreshedLocation = currentRefreshLocation;
/*
* Grab the current refresh listeners and replace them with null while synchronized. Any listeners that come in after this won't be
* in the list we iterate over and very likely won't be candidates for refresh anyway because we've already moved the
* lastRefreshedLocation.
*/
List<Tuple<Translog.Location, Consumer<Boolean>>> candidates;
synchronized (this) {
candidates = refreshListeners;
// No listeners to check so just bail early
if (candidates == null) {
return;
}
refreshListeners = null;
}
// Iterate the list of listeners, copying the listeners to fire to one list and those to preserve to another list.
List<Consumer<Boolean>> listenersToFire = null;
List<Tuple<Translog.Location, Consumer<Boolean>>> preservedListeners = null;
for (Tuple<Translog.Location, Consumer<Boolean>> tuple : candidates) {
Translog.Location location = tuple.v1();
Consumer<Boolean> listener = tuple.v2();
if (location.compareTo(currentRefreshLocation) <= 0) {
if (listenersToFire == null) {
listenersToFire = new ArrayList<>();
}
listenersToFire.add(listener);
} else {
if (preservedListeners == null) {
preservedListeners = new ArrayList<>();
}
preservedListeners.add(tuple);
}
}
/*
* Now add any preserved listeners back to the running list of refresh listeners while under lock. We'll try them next time. While
* we were iterating the list of listeners new listeners could have come in. That means that adding all of our preserved listeners
* might push our list of listeners above the maximum number of slots allowed. This seems unlikely because we expect few listeners
* to be preserved. And the next listener while we're full will trigger a refresh anyway.
*/
if (preservedListeners != null) {
synchronized (this) {
if (refreshListeners == null) {
refreshListeners = new ArrayList<>();
}
refreshListeners.addAll(preservedListeners);
}
}
// Lastly, fire the listeners that are ready on the listener thread pool
if (listenersToFire != null) {
final List<Consumer<Boolean>> finalListenersToFire = listenersToFire;
listenerExecutor.execute(() -> {
for (Consumer<Boolean> listener : finalListenersToFire) {
try {
listener.accept(false);
} catch (Throwable t) {
logger.warn("Error firing refresh listener", t);
}
}
});
}
}
}

View File

@ -31,12 +31,14 @@ import org.elasticsearch.index.mapper.MapperService;
import org.elasticsearch.index.merge.MergeStats;
import org.elasticsearch.index.similarity.SimilarityService;
import org.elasticsearch.index.store.Store;
import org.elasticsearch.index.translog.Translog;
import org.elasticsearch.index.translog.TranslogStats;
import org.elasticsearch.threadpool.ThreadPool;
import java.io.IOException;
import java.util.Collections;
import java.util.List;
import java.util.function.Consumer;
/**
* ShadowIndexShard extends {@link IndexShard} to add file synchronization
@ -86,6 +88,12 @@ public final class ShadowIndexShard extends IndexShard {
return engineFactory.newReadOnlyEngine(config);
}
@Override
protected RefreshListeners buildRefreshListeners() {
// ShadowEngine doesn't have a translog so it shouldn't try to support RefreshListeners.
return null;
}
@Override
public boolean shouldFlush() {
// we don't need to flush since we don't write - all dominated by the primary
@ -96,4 +104,9 @@ public final class ShadowIndexShard extends IndexShard {
public TranslogStats translogStats() {
return null; // shadow engine has no translog
}
@Override
public void addRefreshListener(Translog.Location location, Consumer<Boolean> listener) {
throw new UnsupportedOperationException("Can't listen for a refresh on a shadow engine because it doesn't have a translog");
}
}

View File

@ -447,6 +447,21 @@ public class Translog extends AbstractIndexShardComponent implements IndexShardC
}
}
/**
* The a {@linkplain Location} that will sort after the {@linkplain Location} returned by the last write but before any locations which
* can be returned by the next write.
*/
public Location getLastWriteLocation() {
try (ReleasableLock lock = readLock.acquire()) {
/*
* We use position = current - 1 and size = Integer.MAX_VALUE here instead of position current and size = 0 for two reasons:
* 1. Translog.Location's compareTo doesn't actually pay attention to size even though it's equals method does.
* 2. It feels more right to return a *position* that is before the next write's position rather than rely on the size.
*/
return new Location(current.generation, current.sizeInBytes() - 1, Integer.MAX_VALUE);
}
}
boolean assertBytesAtLocation(Translog.Location location, BytesReference expectedBytes) throws IOException {
// tests can override this
ByteBuffer buffer = ByteBuffer.allocate(location.size);

View File

@ -26,6 +26,7 @@ import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.IOUtils;
import org.elasticsearch.common.bytes.BytesReference;
import org.elasticsearch.common.io.Channels;
import org.elasticsearch.common.logging.ESLoggerFactory;
import org.elasticsearch.common.unit.ByteSizeValue;
import org.elasticsearch.index.shard.ShardId;

View File

@ -112,6 +112,7 @@ import java.util.concurrent.ExecutorService;
import java.util.concurrent.Executors;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicBoolean;
import java.util.concurrent.atomic.AtomicInteger;
import java.util.function.Predicate;
import java.util.stream.Collectors;
@ -141,6 +142,7 @@ public class IndicesService extends AbstractLifecycleComponent<IndicesService> i
private final CircuitBreakerService circuitBreakerService;
private volatile Map<String, IndexService> indices = emptyMap();
private final Map<Index, List<PendingDelete>> pendingDeletes = new HashMap<>();
private final AtomicInteger numUncompletedDeletes = new AtomicInteger();
private final OldShardsStats oldShardsStats = new OldShardsStats();
private final IndexStoreConfig indexStoreConfig;
private final MapperRegistry mapperRegistry;
@ -782,6 +784,7 @@ public class IndicesService extends AbstractLifecycleComponent<IndicesService> i
pendingDeletes.put(index, list);
}
list.add(pendingDelete);
numUncompletedDeletes.incrementAndGet();
}
}
@ -840,6 +843,7 @@ public class IndicesService extends AbstractLifecycleComponent<IndicesService> i
logger.debug("{} processing pending deletes", index);
final long startTimeNS = System.nanoTime();
final List<ShardLock> shardLocks = nodeEnv.lockAllForIndex(index, indexSettings, timeout.millis());
int numRemoved = 0;
try {
Map<ShardId, ShardLock> locks = new HashMap<>();
for (ShardLock lock : shardLocks) {
@ -850,6 +854,7 @@ public class IndicesService extends AbstractLifecycleComponent<IndicesService> i
remove = pendingDeletes.remove(index);
}
if (remove != null && remove.isEmpty() == false) {
numRemoved = remove.size();
CollectionUtil.timSort(remove); // make sure we delete indices first
final long maxSleepTimeMs = 10 * 1000; // ensure we retry after 10 sec
long sleepTime = 10;
@ -896,6 +901,10 @@ public class IndicesService extends AbstractLifecycleComponent<IndicesService> i
}
} finally {
IOUtils.close(shardLocks);
if (numRemoved > 0) {
int remainingUncompletedDeletes = numUncompletedDeletes.addAndGet(-numRemoved);
assert remainingUncompletedDeletes >= 0;
}
}
}
@ -909,6 +918,14 @@ public class IndicesService extends AbstractLifecycleComponent<IndicesService> i
}
}
/**
* Checks if all pending deletes have completed. Used by tests to ensure we don't check directory contents while deletion still ongoing.
* The reason is that, on Windows, browsing the directory contents can interfere with the deletion process and delay it unnecessarily.
*/
public boolean hasUncompletedPendingDeletes() {
return numUncompletedDeletes.get() > 0;
}
/**
* Returns this nodes {@link IndicesQueriesRegistry}
*/

View File

@ -210,7 +210,7 @@ public class HunspellService extends AbstractComponent {
* @param defaults The default settings for this dictionary
* @return The resolved settings.
*/
private static Settings loadDictionarySettings(Path dir, Settings defaults) {
private static Settings loadDictionarySettings(Path dir, Settings defaults) throws IOException {
Path file = dir.resolve("settings.yml");
if (Files.exists(file)) {
return Settings.builder().loadFromPath(file).put(defaults).build();

View File

@ -20,6 +20,7 @@
package org.elasticsearch.indices.cluster;
import com.carrotsearch.hppc.cursors.ObjectCursor;
import org.apache.lucene.store.LockObtainFailedException;
import org.elasticsearch.ElasticsearchException;
import org.elasticsearch.cluster.ClusterChangedEvent;
import org.elasticsearch.cluster.ClusterState;
@ -41,11 +42,14 @@ import org.elasticsearch.common.inject.Inject;
import org.elasticsearch.common.logging.ESLogger;
import org.elasticsearch.common.lucene.Lucene;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.unit.TimeValue;
import org.elasticsearch.common.util.Callback;
import org.elasticsearch.common.util.concurrent.AbstractRunnable;
import org.elasticsearch.common.util.concurrent.ConcurrentCollections;
import org.elasticsearch.gateway.GatewayService;
import org.elasticsearch.index.Index;
import org.elasticsearch.index.IndexService;
import org.elasticsearch.index.IndexSettings;
import org.elasticsearch.index.IndexShardAlreadyExistsException;
import org.elasticsearch.index.NodeServicesProvider;
import org.elasticsearch.index.mapper.DocumentMapper;
@ -67,7 +71,6 @@ import org.elasticsearch.snapshots.RestoreService;
import org.elasticsearch.threadpool.ThreadPool;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.HashSet;
import java.util.Iterator;
@ -75,6 +78,8 @@ import java.util.List;
import java.util.Map;
import java.util.Set;
import java.util.concurrent.ConcurrentMap;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicInteger;
/**
*
@ -213,11 +218,14 @@ public class IndicesClusterStateService extends AbstractLifecycleComponent<Indic
logger.debug("[{}] cleaning index, no longer part of the metadata", index);
}
final IndexService idxService = indicesService.indexService(index);
final IndexSettings indexSettings;
if (idxService != null) {
indexSettings = idxService.getIndexSettings();
deleteIndex(index, "index no longer part of the metadata");
} else if (previousState.metaData().hasIndex(index.getName())) {
// The deleted index was part of the previous cluster state, but not loaded on the local node
final IndexMetaData metaData = previousState.metaData().index(index);
indexSettings = new IndexSettings(metaData, settings);
indicesService.deleteUnassignedIndex("deleted index was not assigned to local node", metaData, event.state());
} else {
// The previous cluster state's metadata also does not contain the index,
@ -227,7 +235,35 @@ public class IndicesClusterStateService extends AbstractLifecycleComponent<Indic
// First, though, verify the precondition for applying this case by
// asserting that the previous cluster state is not initialized/recovered.
assert previousState.blocks().hasGlobalBlock(GatewayService.STATE_NOT_RECOVERED_BLOCK);
indicesService.verifyIndexIsDeleted(index, event.state());
final IndexMetaData metaData = indicesService.verifyIndexIsDeleted(index, event.state());
if (metaData != null) {
indexSettings = new IndexSettings(metaData, settings);
} else {
indexSettings = null;
}
}
if (indexSettings != null) {
threadPool.generic().execute(new AbstractRunnable() {
@Override
public void onFailure(Throwable t) {
logger.warn("[{}] failed to complete pending deletion for index", t, index);
}
@Override
protected void doRun() throws Exception {
try {
// we are waiting until we can lock the index / all shards on the node and then we ack the delete of the store to the
// master. If we can't acquire the locks here immediately there might be a shard of this index still holding on to the lock
// due to a "currently canceled recovery" or so. The shard will delete itself BEFORE the lock is released so it's guaranteed to be
// deleted by the time we get the lock
indicesService.processPendingDeletes(index, indexSettings, new TimeValue(30, TimeUnit.MINUTES));
} catch (LockObtainFailedException exc) {
logger.warn("[{}] failed to lock all shards for index - timed out after 30 seconds", index);
} catch (InterruptedException e) {
logger.warn("[{}] failed to lock all shards for index - interrupted", index);
}
}
});
}
}

View File

@ -98,6 +98,7 @@ import org.elasticsearch.search.SearchService;
import org.elasticsearch.snapshots.SnapshotShardsService;
import org.elasticsearch.snapshots.SnapshotsService;
import org.elasticsearch.tasks.TaskResultsService;
import org.elasticsearch.threadpool.ExecutorBuilder;
import org.elasticsearch.threadpool.ThreadPool;
import org.elasticsearch.threadpool.ThreadPoolModule;
import org.elasticsearch.transport.TransportService;
@ -210,11 +211,12 @@ public class Node implements Closeable {
throw new IllegalStateException("Failed to created node environment", ex);
}
final NetworkService networkService = new NetworkService(settings);
final ThreadPool threadPool = new ThreadPool(settings);
final List<ExecutorBuilder<?>> executorBuilders = pluginsService.getExecutorBuilders(settings);
final ThreadPool threadPool = new ThreadPool(settings, executorBuilders.toArray(new ExecutorBuilder[0]));
NamedWriteableRegistry namedWriteableRegistry = new NamedWriteableRegistry();
boolean success = false;
try {
final MonitorService monitorService = new MonitorService(settings, nodeEnvironment, threadPool);
ModulesBuilder modules = new ModulesBuilder();
modules.add(new Version.Module(version));
modules.add(new CircuitBreakerModule(settings));
@ -222,6 +224,7 @@ public class Node implements Closeable {
for (Module pluginModule : pluginsService.nodeModules()) {
modules.add(pluginModule);
}
final MonitorService monitorService = new MonitorService(settings, nodeEnvironment, threadPool);
modules.add(new PluginsModule(pluginsService));
SettingsModule settingsModule = new SettingsModule(this.settings);
modules.add(settingsModule);
@ -232,7 +235,8 @@ public class Node implements Closeable {
modules.add(scriptModule);
modules.add(new NodeEnvironmentModule(nodeEnvironment));
modules.add(new ClusterNameModule(this.settings));
modules.add(new ThreadPoolModule(threadPool));
final ThreadPoolModule threadPoolModule = new ThreadPoolModule(threadPool);
modules.add(threadPoolModule);
modules.add(new DiscoveryModule(this.settings));
modules.add(new ClusterModule(this.settings));
modules.add(new IndicesModule());
@ -246,11 +250,14 @@ public class Node implements Closeable {
modules.add(new AnalysisModule(environment));
pluginsService.processModules(modules);
scriptModule.prepareSettings(settingsModule);
threadPoolModule.prepareSettings(settingsModule);
injector = modules.createInjector();
client = injector.getInstance(Client.class);
threadPool.setClusterSettings(injector.getInstance(ClusterSettings.class));
success = true;
} catch (IOException ex) {
throw new ElasticsearchException("failed to bind service", ex);

View File

@ -103,7 +103,11 @@ public class InternalSettingsPreparer {
Path path = environment.configFile().resolve("elasticsearch" + allowedSuffix);
if (Files.exists(path)) {
if (!settingsFileFound) {
output.loadFromPath(path);
try {
output.loadFromPath(path);
} catch (IOException e) {
throw new SettingsException("Failed to settings from " + path.toString(), e);
}
}
settingsFileFound = true;
foundSuffixes.add(allowedSuffix);

View File

@ -23,9 +23,12 @@ import org.elasticsearch.common.component.LifecycleComponent;
import org.elasticsearch.common.inject.Module;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.index.IndexModule;
import org.elasticsearch.threadpool.ExecutorBuilder;
import org.elasticsearch.threadpool.ThreadPool;
import java.util.Collection;
import java.util.Collections;
import java.util.List;
/**
* An extension point allowing to plug in custom functionality.
@ -80,4 +83,15 @@ public abstract class Plugin {
*/
@Deprecated
public final void onModule(IndexModule indexModule) {}
/**
* Provides the list of this plugin's custom thread pools, empty if
* none.
*
* @param settings the current settings
* @return executors builders for this plugin's custom thread pools
*/
public List<ExecutorBuilder<?>> getExecutorBuilders(Settings settings) {
return Collections.emptyList();
}
}

View File

@ -40,6 +40,7 @@ import org.elasticsearch.common.settings.Setting;
import org.elasticsearch.common.settings.Setting.Property;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.index.IndexModule;
import org.elasticsearch.threadpool.ExecutorBuilder;
import java.io.IOException;
import java.lang.reflect.InvocationTargetException;
@ -261,6 +262,14 @@ public class PluginsService extends AbstractComponent {
return modules;
}
public List<ExecutorBuilder<?>> getExecutorBuilders(Settings settings) {
final ArrayList<ExecutorBuilder<?>> builders = new ArrayList<>();
for (final Tuple<PluginInfo, Plugin> plugin : plugins) {
builders.addAll(plugin.v2().getExecutorBuilders(settings));
}
return builders;
}
public Collection<Class<? extends LifecycleComponent>> nodeServices() {
List<Class<? extends LifecycleComponent>> services = new ArrayList<>();
for (Tuple<PluginInfo, Plugin> plugin : plugins) {

View File

@ -84,7 +84,7 @@ public class RestBulkAction extends BaseRestHandler {
bulkRequest.consistencyLevel(WriteConsistencyLevel.fromString(consistencyLevel));
}
bulkRequest.timeout(request.paramAsTime("timeout", BulkShardRequest.DEFAULT_TIMEOUT));
bulkRequest.refresh(request.paramAsBoolean("refresh", bulkRequest.refresh()));
bulkRequest.setRefreshPolicy(request.param("refresh"));
bulkRequest.add(request.content(), defaultIndex, defaultType, defaultRouting, defaultFields, defaultPipeline, null, allowExplicitIndex);
client.bulk(bulkRequest, new RestBuilderListener<BulkResponse>(channel) {

View File

@ -412,13 +412,13 @@ public class RestIndicesAction extends AbstractCatAction {
table.addCell(indexStats == null ? null : indexStats.getPrimaries().getIndexing().getTotal().getIndexFailedCount());
table.addCell(indexStats == null ? null : indexStats.getTotal().getMerge().getCurrent());
table.addCell(indexStats == null ? null : indexStats.getPrimaries().getMerge().getCurrentSize());
table.addCell(indexStats == null ? null : indexStats.getPrimaries().getMerge().getCurrent());
table.addCell(indexStats == null ? null : indexStats.getTotal().getMerge().getCurrentNumDocs());
table.addCell(indexStats == null ? null : indexStats.getPrimaries().getMerge().getCurrentNumDocs());
table.addCell(indexStats == null ? null : indexStats.getTotal().getMerge().getCurrentSize());
table.addCell(indexStats == null ? null : indexStats.getPrimaries().getMerge().getCurrent());
table.addCell(indexStats == null ? null : indexStats.getPrimaries().getMerge().getCurrentSize());
table.addCell(indexStats == null ? null : indexStats.getTotal().getMerge().getTotal());
table.addCell(indexStats == null ? null : indexStats.getPrimaries().getMerge().getTotal());

View File

@ -51,7 +51,7 @@ public class RestDeleteAction extends BaseRestHandler {
deleteRequest.routing(request.param("routing"));
deleteRequest.parent(request.param("parent")); // order is important, set it after routing, so it will set the routing
deleteRequest.timeout(request.paramAsTime("timeout", DeleteRequest.DEFAULT_TIMEOUT));
deleteRequest.refresh(request.paramAsBoolean("refresh", deleteRequest.refresh()));
deleteRequest.setRefreshPolicy(request.param("refresh"));
deleteRequest.version(RestActions.parseVersion(request));
deleteRequest.versionType(VersionType.fromString(request.param("version_type"), deleteRequest.versionType()));

View File

@ -80,7 +80,7 @@ public class RestIndexAction extends BaseRestHandler {
indexRequest.setPipeline(request.param("pipeline"));
indexRequest.source(request.content());
indexRequest.timeout(request.paramAsTime("timeout", IndexRequest.DEFAULT_TIMEOUT));
indexRequest.refresh(request.paramAsBoolean("refresh", indexRequest.refresh()));
indexRequest.setRefreshPolicy(request.param("refresh"));
indexRequest.version(RestActions.parseVersion(request));
indexRequest.versionType(VersionType.fromString(request.param("version_type"), indexRequest.versionType()));
String sOpType = request.param("op_type");

View File

@ -157,7 +157,7 @@ public class RestMultiSearchAction extends BaseRestHandler {
Object value = entry.getValue();
if ("index".equals(entry.getKey()) || "indices".equals(entry.getKey())) {
if (!allowExplicitIndex) {
throw new IllegalArgumentException("explicit index in multi percolate is not allowed");
throw new IllegalArgumentException("explicit index in multi search is not allowed");
}
searchRequest.indices(nodeStringArrayValue(value));
} else if ("type".equals(entry.getKey()) || "types".equals(entry.getKey())) {

View File

@ -58,7 +58,7 @@ public class RestUpdateAction extends BaseRestHandler {
updateRequest.routing(request.param("routing"));
updateRequest.parent(request.param("parent"));
updateRequest.timeout(request.paramAsTime("timeout", updateRequest.timeout()));
updateRequest.refresh(request.paramAsBoolean("refresh", updateRequest.refresh()));
updateRequest.setRefreshPolicy(request.param("refresh"));
String consistencyLevel = request.param("consistency");
if (consistencyLevel != null) {
updateRequest.consistencyLevel(WriteConsistencyLevel.fromString(consistencyLevel));

View File

@ -546,14 +546,14 @@ public class SearchService extends AbstractLifecycleComponent<SearchService> imp
indexShard, scriptService, bigArrays, threadPool.estimatedTimeInMillisCounter(), parseFieldMatcher,
defaultSearchTimeout, fetchPhase);
SearchContext.setCurrent(context);
request.rewrite(context.getQueryShardContext());
// reset that we have used nowInMillis from the context since it may
// have been rewritten so its no longer in the query and the request can
// be cached. If it is still present in the request (e.g. in a range
// aggregation) it will still be caught when the aggregation is
// evaluated.
context.resetNowInMillisUsed();
try {
request.rewrite(context.getQueryShardContext());
// reset that we have used nowInMillis from the context since it may
// have been rewritten so its no longer in the query and the request can
// be cached. If it is still present in the request (e.g. in a range
// aggregation) it will still be caught when the aggregation is
// evaluated.
context.resetNowInMillisUsed();
if (request.scroll() != null) {
context.scrollContext(new ScrollContext());
context.scrollContext().scroll = request.scroll();

View File

@ -0,0 +1,91 @@
/*
* 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.threadpool;
import org.elasticsearch.common.settings.Setting;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.util.concurrent.ThreadContext;
import java.util.List;
/**
* Base class for executor builders.
*
* @param <U> the underlying type of the executor settings
*/
public abstract class ExecutorBuilder<U extends ExecutorBuilder.ExecutorSettings> {
private final String name;
public ExecutorBuilder(String name) {
this.name = name;
}
protected String name() {
return name;
}
protected static String settingsKey(final String prefix, final String key) {
return String.join(".", prefix, key);
}
/**
* The list of settings this builder will register.
*
* @return the list of registered settings
*/
abstract List<Setting<?>> getRegisteredSettings();
/**
* Return an executor settings object from the node-level settings.
*
* @param settings the node-level settings
* @return the executor settings object
*/
abstract U getSettings(Settings settings);
/**
* Builds the executor with the specified executor settings.
*
* @param settings the executor settings
* @param threadContext the current thread context
* @return a new executor built from the specified executor settings
*/
abstract ThreadPool.ExecutorHolder build(U settings, ThreadContext threadContext);
/**
* Format the thread pool info object for this executor.
*
* @param info the thread pool info object to format
* @return a formatted thread pool info (useful for logging)
*/
abstract String formatInfo(ThreadPool.Info info);
static abstract class ExecutorSettings {
protected final String nodeName;
public ExecutorSettings(String nodeName) {
this.nodeName = nodeName;
}
}
}

View File

@ -0,0 +1,135 @@
/*
* 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.threadpool;
import org.elasticsearch.common.settings.Setting;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.unit.SizeValue;
import org.elasticsearch.common.util.concurrent.EsExecutors;
import org.elasticsearch.common.util.concurrent.EsThreadPoolExecutor;
import org.elasticsearch.common.util.concurrent.ThreadContext;
import org.elasticsearch.node.Node;
import java.util.Arrays;
import java.util.List;
import java.util.Locale;
import java.util.concurrent.Executor;
import java.util.concurrent.ThreadFactory;
/**
* A builder for fixed executors.
*/
public final class FixedExecutorBuilder extends ExecutorBuilder<FixedExecutorBuilder.FixedExecutorSettings> {
private final Setting<Integer> sizeSetting;
private final Setting<Integer> queueSizeSetting;
/**
* Construct a fixed executor builder; the settings will have the
* key prefix "thread_pool." followed by the executor name.
*
* @param settings the node-level settings
* @param name the name of the executor
* @param size the fixed number of threads
* @param queueSize the size of the backing queue, -1 for unbounded
*/
FixedExecutorBuilder(final Settings settings, final String name, final int size, final int queueSize) {
this(settings, name, size, queueSize, "thread_pool." + name);
}
/**
* Construct a fixed executor builder.
*
* @param settings the node-level settings
* @param name the name of the executor
* @param size the fixed number of threads
* @param queueSize the size of the backing queue, -1 for unbounded
* @param prefix the prefix for the settings keys
*/
public FixedExecutorBuilder(final Settings settings, final String name, final int size, final int queueSize, final String prefix) {
super(name);
final String sizeKey = settingsKey(prefix, "size");
this.sizeSetting =
new Setting<>(
sizeKey,
s -> Integer.toString(size),
s -> Setting.parseInt(s, 1, applyHardSizeLimit(settings, name), sizeKey),
Setting.Property.NodeScope);
final String queueSizeKey = settingsKey(prefix, "queue_size");
this.queueSizeSetting =
Setting.intSetting(queueSizeKey, queueSize, Setting.Property.NodeScope);
}
private int applyHardSizeLimit(final Settings settings, final String name) {
if (name.equals(ThreadPool.Names.BULK) || name.equals(ThreadPool.Names.INDEX)) {
return 1 + EsExecutors.boundedNumberOfProcessors(settings);
} else {
return Integer.MAX_VALUE;
}
}
@Override
List<Setting<?>> getRegisteredSettings() {
return Arrays.asList(sizeSetting, queueSizeSetting);
}
@Override
FixedExecutorSettings getSettings(Settings settings) {
final String nodeName = Node.NODE_NAME_SETTING.get(settings);
final int size = sizeSetting.get(settings);
final int queueSize = queueSizeSetting.get(settings);
return new FixedExecutorSettings(nodeName, size, queueSize);
}
@Override
ThreadPool.ExecutorHolder build(final FixedExecutorSettings settings, final ThreadContext threadContext) {
int size = settings.size;
int queueSize = settings.queueSize;
final ThreadFactory threadFactory = EsExecutors.daemonThreadFactory(EsExecutors.threadName(settings.nodeName, name()));
Executor executor = EsExecutors.newFixed(name(), size, queueSize, threadFactory, threadContext);
final ThreadPool.Info info =
new ThreadPool.Info(name(), ThreadPool.ThreadPoolType.FIXED, size, size, null, queueSize < 0 ? null : new SizeValue(queueSize));
return new ThreadPool.ExecutorHolder(executor, info);
}
@Override
String formatInfo(ThreadPool.Info info) {
return String.format(
Locale.ROOT,
"name [%s], size [%d], queue size [%s]",
info.getName(),
info.getMax(),
info.getQueueSize() == null ? "unbounded" : info.getQueueSize());
}
static class FixedExecutorSettings extends ExecutorBuilder.ExecutorSettings {
private final int size;
private final int queueSize;
public FixedExecutorSettings(final String nodeName, final int size, final int queueSize) {
super(nodeName);
this.size = size;
this.queueSize = queueSize;
}
}
}

View File

@ -0,0 +1,129 @@
/*
* 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.threadpool;
import org.elasticsearch.common.settings.Setting;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.unit.TimeValue;
import org.elasticsearch.common.util.concurrent.EsExecutors;
import org.elasticsearch.common.util.concurrent.ThreadContext;
import org.elasticsearch.node.Node;
import java.util.Arrays;
import java.util.List;
import java.util.Locale;
import java.util.concurrent.Executor;
import java.util.concurrent.ThreadFactory;
import java.util.concurrent.TimeUnit;
/**
* A builder for scaling executors.
*/
public final class ScalingExecutorBuilder extends ExecutorBuilder<ScalingExecutorBuilder.ScalingExecutorSettings> {
private final Setting<Integer> coreSetting;
private final Setting<Integer> maxSetting;
private final Setting<TimeValue> keepAliveSetting;
/**
* Construct a scaling executor builder; the settings will have the
* key prefix "thread_pool." followed by the executor name.
*
* @param name the name of the executor
* @param core the minimum number of threads in the pool
* @param max the maximum number of threads in the pool
* @param keepAlive the time that spare threads above {@code core}
* threads will be kept alive
*/
public ScalingExecutorBuilder(final String name, final int core, final int max, final TimeValue keepAlive) {
this(name, core, max, keepAlive, "thread_pool." + name);
}
/**
* Construct a scaling executor builder; the settings will have the
* specified key prefix.
*
* @param name the name of the executor
* @param core the minimum number of threads in the pool
* @param max the maximum number of threads in the pool
* @param keepAlive the time that spare threads above {@code core}
* threads will be kept alive
* @param prefix the prefix for the settings keys
*/
public ScalingExecutorBuilder(final String name, final int core, final int max, final TimeValue keepAlive, final String prefix) {
super(name);
this.coreSetting =
Setting.intSetting(settingsKey(prefix, "core"), core, Setting.Property.NodeScope);
this.maxSetting = Setting.intSetting(settingsKey(prefix, "max"), max, Setting.Property.NodeScope);
this.keepAliveSetting =
Setting.timeSetting(settingsKey(prefix, "keep_alive"), keepAlive, Setting.Property.NodeScope);
}
@Override
List<Setting<?>> getRegisteredSettings() {
return Arrays.asList(coreSetting, maxSetting, keepAliveSetting);
}
@Override
ScalingExecutorSettings getSettings(Settings settings) {
final String nodeName = Node.NODE_NAME_SETTING.get(settings);
final int coreThreads = coreSetting.get(settings);
final int maxThreads = maxSetting.get(settings);
final TimeValue keepAlive = keepAliveSetting.get(settings);
return new ScalingExecutorSettings(nodeName, coreThreads, maxThreads, keepAlive);
}
ThreadPool.ExecutorHolder build(final ScalingExecutorSettings settings, final ThreadContext threadContext) {
TimeValue keepAlive = settings.keepAlive;
int core = settings.core;
int max = settings.max;
final ThreadPool.Info info = new ThreadPool.Info(name(), ThreadPool.ThreadPoolType.SCALING, core, max, keepAlive, null);
final ThreadFactory threadFactory = EsExecutors.daemonThreadFactory(EsExecutors.threadName(settings.nodeName, name()));
final Executor executor =
EsExecutors.newScaling(name(), core, max, keepAlive.millis(), TimeUnit.MILLISECONDS, threadFactory, threadContext);
return new ThreadPool.ExecutorHolder(executor, info);
}
@Override
String formatInfo(ThreadPool.Info info) {
return String.format(
Locale.ROOT,
"name [%s], core [%d], max [%d], keep alive [%s]",
info.getName(),
info.getMin(),
info.getMax(),
info.getKeepAlive());
}
static class ScalingExecutorSettings extends ExecutorBuilder.ExecutorSettings {
private final int core;
private final int max;
private final TimeValue keepAlive;
public ScalingExecutorSettings(final String nodeName, final int core, final int max, final TimeValue keepAlive) {
super(nodeName);
this.core = core;
this.max = max;
this.keepAlive = keepAlive;
}
}
}

View File

@ -26,11 +26,8 @@ import org.elasticsearch.common.component.AbstractComponent;
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.ClusterSettings;
import org.elasticsearch.common.settings.Setting;
import org.elasticsearch.common.settings.Setting.Property;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.settings.SettingsException;
import org.elasticsearch.common.unit.SizeValue;
import org.elasticsearch.common.unit.TimeValue;
import org.elasticsearch.common.util.concurrent.EsAbortPolicy;
@ -45,31 +42,22 @@ import org.elasticsearch.node.Node;
import java.io.Closeable;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Collection;
import java.util.Collections;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.Objects;
import java.util.Queue;
import java.util.concurrent.ConcurrentLinkedQueue;
import java.util.concurrent.Executor;
import java.util.concurrent.ExecutorService;
import java.util.concurrent.RejectedExecutionHandler;
import java.util.concurrent.ScheduledExecutorService;
import java.util.concurrent.ScheduledFuture;
import java.util.concurrent.ScheduledThreadPoolExecutor;
import java.util.concurrent.ThreadFactory;
import java.util.concurrent.ThreadPoolExecutor;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicBoolean;
import static java.util.Collections.unmodifiableMap;
import static org.elasticsearch.common.unit.SizeValue.parseSizeValue;
import static org.elasticsearch.common.unit.TimeValue.timeValueMinutes;
/**
*
*/
public class ThreadPool extends AbstractComponent implements Closeable {
public static class Names {
@ -146,164 +134,85 @@ public class ThreadPool extends AbstractComponent implements Closeable {
THREAD_POOL_TYPES = Collections.unmodifiableMap(map);
}
private static void add(Map<String, Settings> executorSettings, ExecutorSettingsBuilder builder) {
Settings settings = builder.build();
String name = settings.get("name");
executorSettings.put(name, settings);
}
private static abstract class ExecutorSettingsBuilder<T extends ExecutorSettingsBuilder<T>> {
private final Settings.Builder builder;
protected ExecutorSettingsBuilder(String name, ThreadPoolType threadPoolType) {
if (THREAD_POOL_TYPES.get(name) != threadPoolType) {
throw new IllegalArgumentException("thread pool [" + name + "] must be of type [" + threadPoolType + "]");
}
builder = Settings.builder();
builder.put("name", name);
builder.put("type", threadPoolType.getType());
}
public T keepAlive(String keepAlive) {
return add("keep_alive", keepAlive);
}
public T queueSize(int queueSize) {
return add("queue_size", queueSize);
}
protected T add(String setting, int value) {
return add(setting, Integer.toString(value));
}
protected T add(String setting, String value) {
builder.put(setting, value);
@SuppressWarnings("unchecked") final T executor = (T)this;
return executor;
}
public final Settings build() { return builder.build(); }
}
private static class FixedExecutorSettingsBuilder extends ExecutorSettingsBuilder<FixedExecutorSettingsBuilder> {
public FixedExecutorSettingsBuilder(String name) {
super(name, ThreadPoolType.FIXED);
}
public FixedExecutorSettingsBuilder size(int size) {
return add("size", Integer.toString(size));
}
}
private static class ScalingExecutorSettingsBuilder extends ExecutorSettingsBuilder<ScalingExecutorSettingsBuilder> {
public ScalingExecutorSettingsBuilder(String name) {
super(name, ThreadPoolType.SCALING);
}
public ScalingExecutorSettingsBuilder min(int min) {
return add("min", min);
}
public ScalingExecutorSettingsBuilder size(int size) {
return add("size", size);
}
}
public static final Setting<Settings> THREADPOOL_GROUP_SETTING =
Setting.groupSetting("threadpool.", Property.Dynamic, Property.NodeScope);
private volatile Map<String, ExecutorHolder> executors;
private final Map<String, Settings> defaultExecutorTypeSettings;
private final Queue<ExecutorHolder> retiredExecutors = new ConcurrentLinkedQueue<>();
private Map<String, ExecutorHolder> executors = new HashMap<>();
private final ScheduledThreadPoolExecutor scheduler;
private final EstimatedTimeThread estimatedTimeThread;
private final AtomicBoolean settingsListenerIsSet = new AtomicBoolean(false);
static final Executor DIRECT_EXECUTOR = command -> command.run();
private final ThreadContext threadContext;
public ThreadPool(String name) {
this(Settings.builder().put(Node.NODE_NAME_SETTING.getKey(), name).build());
private final Map<String, ExecutorBuilder> builders;
public Collection<ExecutorBuilder> builders() {
return Collections.unmodifiableCollection(builders.values());
}
public ThreadPool(Settings settings) {
public static Setting<TimeValue> ESTIMATED_TIME_INTERVAL_SETTING =
Setting.timeSetting("thread_pool.estimated_time_interval", TimeValue.timeValueMillis(200), Setting.Property.NodeScope);
public ThreadPool(final Settings settings, final ExecutorBuilder<?>... customBuilders) {
super(settings);
assert Node.NODE_NAME_SETTING.exists(settings) : "ThreadPool's settings should contain a name";
threadContext = new ThreadContext(settings);
Map<String, Settings> groupSettings = THREADPOOL_GROUP_SETTING.get(settings).getAsGroups();
validate(groupSettings);
assert Node.NODE_NAME_SETTING.exists(settings);
int availableProcessors = EsExecutors.boundedNumberOfProcessors(settings);
int halfProcMaxAt5 = halfNumberOfProcessorsMaxFive(availableProcessors);
int halfProcMaxAt10 = halfNumberOfProcessorsMaxTen(availableProcessors);
Map<String, Settings> defaultExecutorTypeSettings = new HashMap<>();
int genericThreadPoolMax = boundedBy(4 * availableProcessors, 128, 512);
add(defaultExecutorTypeSettings, new ScalingExecutorSettingsBuilder(Names.GENERIC).min(4).size(genericThreadPoolMax).keepAlive("30s"));
add(defaultExecutorTypeSettings, new FixedExecutorSettingsBuilder(Names.INDEX).size(availableProcessors).queueSize(200));
add(defaultExecutorTypeSettings, new FixedExecutorSettingsBuilder(Names.BULK).size(availableProcessors).queueSize(50));
add(defaultExecutorTypeSettings, new FixedExecutorSettingsBuilder(Names.GET).size(availableProcessors).queueSize(1000));
add(defaultExecutorTypeSettings, new FixedExecutorSettingsBuilder(Names.SEARCH).size(((availableProcessors * 3) / 2) + 1).queueSize(1000));
add(defaultExecutorTypeSettings, new ScalingExecutorSettingsBuilder(Names.MANAGEMENT).min(1).size(5).keepAlive("5m"));
final Map<String, ExecutorBuilder> builders = new HashMap<>();
final int availableProcessors = EsExecutors.boundedNumberOfProcessors(settings);
final int halfProcMaxAt5 = halfNumberOfProcessorsMaxFive(availableProcessors);
final int halfProcMaxAt10 = halfNumberOfProcessorsMaxTen(availableProcessors);
final int genericThreadPoolMax = boundedBy(4 * availableProcessors, 128, 512);
builders.put(Names.GENERIC, new ScalingExecutorBuilder(Names.GENERIC, 4, genericThreadPoolMax, TimeValue.timeValueSeconds(30)));
builders.put(Names.INDEX, new FixedExecutorBuilder(settings, Names.INDEX, availableProcessors, 200));
builders.put(Names.BULK, new FixedExecutorBuilder(settings, Names.BULK, availableProcessors, 50));
builders.put(Names.GET, new FixedExecutorBuilder(settings, Names.GET, availableProcessors, 1000));
builders.put(Names.SEARCH, new FixedExecutorBuilder(settings, Names.SEARCH, ((availableProcessors * 3) / 2) + 1, 1000));
builders.put(Names.MANAGEMENT, new ScalingExecutorBuilder(Names.MANAGEMENT, 1, 5, TimeValue.timeValueMinutes(5)));
// no queue as this means clients will need to handle rejections on listener queue even if the operation succeeded
// the assumption here is that the listeners should be very lightweight on the listeners side
add(defaultExecutorTypeSettings, new FixedExecutorSettingsBuilder(Names.LISTENER).size(halfProcMaxAt10));
add(defaultExecutorTypeSettings, new ScalingExecutorSettingsBuilder(Names.FLUSH).min(1).size(halfProcMaxAt5).keepAlive("5m"));
add(defaultExecutorTypeSettings, new ScalingExecutorSettingsBuilder(Names.REFRESH).min(1).size(halfProcMaxAt10).keepAlive("5m"));
add(defaultExecutorTypeSettings, new ScalingExecutorSettingsBuilder(Names.WARMER).min(1).size(halfProcMaxAt5).keepAlive("5m"));
add(defaultExecutorTypeSettings, new ScalingExecutorSettingsBuilder(Names.SNAPSHOT).min(1).size(halfProcMaxAt5).keepAlive("5m"));
add(defaultExecutorTypeSettings, new FixedExecutorSettingsBuilder(Names.FORCE_MERGE).size(1));
add(defaultExecutorTypeSettings, new ScalingExecutorSettingsBuilder(Names.FETCH_SHARD_STARTED).min(1).size(availableProcessors * 2).keepAlive("5m"));
add(defaultExecutorTypeSettings, new ScalingExecutorSettingsBuilder(Names.FETCH_SHARD_STORE).min(1).size(availableProcessors * 2).keepAlive("5m"));
this.defaultExecutorTypeSettings = unmodifiableMap(defaultExecutorTypeSettings);
Map<String, ExecutorHolder> executors = new HashMap<>();
for (Map.Entry<String, Settings> executor : defaultExecutorTypeSettings.entrySet()) {
executors.put(executor.getKey(), build(executor.getKey(), groupSettings.get(executor.getKey()), executor.getValue()));
}
// Building custom thread pools
for (Map.Entry<String, Settings> entry : groupSettings.entrySet()) {
if (executors.containsKey(entry.getKey())) {
continue;
builders.put(Names.LISTENER, new FixedExecutorBuilder(settings, Names.LISTENER, halfProcMaxAt10, -1));
builders.put(Names.FLUSH, new ScalingExecutorBuilder(Names.FLUSH, 1, halfProcMaxAt5, TimeValue.timeValueMinutes(5)));
builders.put(Names.REFRESH, new ScalingExecutorBuilder(Names.REFRESH, 1, halfProcMaxAt10, TimeValue.timeValueMinutes(5)));
builders.put(Names.WARMER, new ScalingExecutorBuilder(Names.WARMER, 1, halfProcMaxAt5, TimeValue.timeValueMinutes(5)));
builders.put(Names.SNAPSHOT, new ScalingExecutorBuilder(Names.SNAPSHOT, 1, halfProcMaxAt5, TimeValue.timeValueMinutes(5)));
builders.put(Names.FETCH_SHARD_STARTED, new ScalingExecutorBuilder(Names.FETCH_SHARD_STARTED, 1, 2 * availableProcessors, TimeValue.timeValueMinutes(5)));
builders.put(Names.FORCE_MERGE, new FixedExecutorBuilder(settings, Names.FORCE_MERGE, 1, -1));
builders.put(Names.FETCH_SHARD_STORE, new ScalingExecutorBuilder(Names.FETCH_SHARD_STORE, 1, 2 * availableProcessors, TimeValue.timeValueMinutes(5)));
for (final ExecutorBuilder<?> builder : customBuilders) {
if (builders.containsKey(builder.name())) {
throw new IllegalArgumentException("builder with name [" + builder.name() + "] already exists");
}
executors.put(entry.getKey(), build(entry.getKey(), entry.getValue(), Settings.EMPTY));
builders.put(builder.name(), builder);
}
this.builders = Collections.unmodifiableMap(builders);
threadContext = new ThreadContext(settings);
final Map<String, ExecutorHolder> executors = new HashMap<>();
for (@SuppressWarnings("unchecked") final Map.Entry<String, ExecutorBuilder> entry : builders.entrySet()) {
final ExecutorBuilder.ExecutorSettings executorSettings = entry.getValue().getSettings(settings);
final ExecutorHolder executorHolder = entry.getValue().build(executorSettings, threadContext);
if (executors.containsKey(executorHolder.info.getName())) {
throw new IllegalStateException("duplicate executors with name [" + executorHolder.info.getName() + "] registered");
}
logger.debug("created thread pool: " + entry.getValue().formatInfo(executorHolder.info));
executors.put(entry.getKey(), executorHolder);
}
executors.put(Names.SAME, new ExecutorHolder(DIRECT_EXECUTOR, new Info(Names.SAME, ThreadPoolType.DIRECT)));
this.executors = unmodifiableMap(executors);
this.scheduler = new ScheduledThreadPoolExecutor(1, EsExecutors.daemonThreadFactory(settings, "scheduler"), new EsAbortPolicy());
this.scheduler.setExecuteExistingDelayedTasksAfterShutdownPolicy(false);
this.scheduler.setContinueExistingPeriodicTasksAfterShutdownPolicy(false);
this.scheduler.setRemoveOnCancelPolicy(true);
TimeValue estimatedTimeInterval = settings.getAsTime("threadpool.estimated_time_interval", TimeValue.timeValueMillis(200));
TimeValue estimatedTimeInterval = ESTIMATED_TIME_INTERVAL_SETTING.get(settings);
this.estimatedTimeThread = new EstimatedTimeThread(EsExecutors.threadName(settings, "[timer]"), estimatedTimeInterval.millis());
this.estimatedTimeThread.start();
}
public void setClusterSettings(ClusterSettings clusterSettings) {
if(settingsListenerIsSet.compareAndSet(false, true)) {
clusterSettings.addSettingsUpdateConsumer(THREADPOOL_GROUP_SETTING, this::updateSettings, (s) -> validate(s.getAsGroups()));
} else {
throw new IllegalStateException("the node settings listener was set more then once");
}
}
public long estimatedTimeInMillis() {
return estimatedTimeThread.estimatedTimeInMillis();
}
@ -440,12 +349,6 @@ public class ThreadPool extends AbstractComponent implements Closeable {
((ThreadPoolExecutor) executor.executor()).shutdownNow();
}
}
ExecutorHolder holder;
while ((holder = retiredExecutors.poll()) != null) {
ThreadPoolExecutor executor = (ThreadPoolExecutor) holder.executor();
executor.shutdownNow();
}
}
public boolean awaitTermination(long timeout, TimeUnit unit) throws InterruptedException {
@ -456,142 +359,10 @@ public class ThreadPool extends AbstractComponent implements Closeable {
}
}
ExecutorHolder holder;
while ((holder = retiredExecutors.poll()) != null) {
ThreadPoolExecutor executor = (ThreadPoolExecutor) holder.executor();
result &= executor.awaitTermination(timeout, unit);
}
estimatedTimeThread.join(unit.toMillis(timeout));
return result;
}
private ExecutorHolder build(String name, @Nullable Settings settings, Settings defaultSettings) {
return rebuild(name, null, settings, defaultSettings);
}
private ExecutorHolder rebuild(String name, ExecutorHolder previousExecutorHolder, @Nullable Settings settings, Settings defaultSettings) {
if (Names.SAME.equals(name)) {
// Don't allow to change the "same" thread executor
return previousExecutorHolder;
}
if (settings == null) {
settings = Settings.Builder.EMPTY_SETTINGS;
}
Info previousInfo = previousExecutorHolder != null ? previousExecutorHolder.info : null;
String type = settings.get("type", previousInfo != null ? previousInfo.getThreadPoolType().getType() : defaultSettings.get("type"));
ThreadPoolType threadPoolType = ThreadPoolType.fromType(type);
ThreadFactory threadFactory = EsExecutors.daemonThreadFactory(this.settings, name);
if (ThreadPoolType.DIRECT == threadPoolType) {
if (previousExecutorHolder != null) {
logger.debug("updating thread pool [{}], type [{}]", name, type);
} else {
logger.debug("creating thread pool [{}], type [{}]", name, type);
}
return new ExecutorHolder(DIRECT_EXECUTOR, new Info(name, threadPoolType));
} else if (ThreadPoolType.FIXED == threadPoolType) {
int defaultSize = defaultSettings.getAsInt("size", EsExecutors.boundedNumberOfProcessors(settings));
SizeValue defaultQueueSize = getAsSizeOrUnbounded(defaultSettings, "queue", getAsSizeOrUnbounded(defaultSettings, "queue_size", null));
if (previousExecutorHolder != null) {
assert previousInfo != null;
if (ThreadPoolType.FIXED == previousInfo.getThreadPoolType()) {
SizeValue updatedQueueSize = getAsSizeOrUnbounded(settings, "capacity", getAsSizeOrUnbounded(settings, "queue", getAsSizeOrUnbounded(settings, "queue_size", previousInfo.getQueueSize())));
if (Objects.equals(previousInfo.getQueueSize(), updatedQueueSize)) {
int updatedSize = applyHardSizeLimit(name, settings.getAsInt("size", previousInfo.getMax()));
if (previousInfo.getMax() != updatedSize) {
logger.debug("updating thread pool [{}], type [{}], size [{}], queue_size [{}]", name, type, updatedSize, updatedQueueSize);
// if you think this code is crazy: that's because it is!
if (updatedSize > previousInfo.getMax()) {
((EsThreadPoolExecutor) previousExecutorHolder.executor()).setMaximumPoolSize(updatedSize);
((EsThreadPoolExecutor) previousExecutorHolder.executor()).setCorePoolSize(updatedSize);
} else {
((EsThreadPoolExecutor) previousExecutorHolder.executor()).setCorePoolSize(updatedSize);
((EsThreadPoolExecutor) previousExecutorHolder.executor()).setMaximumPoolSize(updatedSize);
}
return new ExecutorHolder(previousExecutorHolder.executor(), new Info(name, threadPoolType, updatedSize, updatedSize, null, updatedQueueSize));
}
return previousExecutorHolder;
}
}
if (previousInfo.getMax() >= 0) {
defaultSize = previousInfo.getMax();
}
defaultQueueSize = previousInfo.getQueueSize();
}
int size = applyHardSizeLimit(name, settings.getAsInt("size", defaultSize));
SizeValue queueSize = getAsSizeOrUnbounded(settings, "capacity", getAsSizeOrUnbounded(settings, "queue", getAsSizeOrUnbounded(settings, "queue_size", defaultQueueSize)));
logger.debug("creating thread pool [{}], type [{}], size [{}], queue_size [{}]", name, type, size, queueSize);
Executor executor = EsExecutors.newFixed(name, size, queueSize == null ? -1 : (int) queueSize.singles(), threadFactory, threadContext);
return new ExecutorHolder(executor, new Info(name, threadPoolType, size, size, null, queueSize));
} else if (ThreadPoolType.SCALING == threadPoolType) {
TimeValue defaultKeepAlive = defaultSettings.getAsTime("keep_alive", timeValueMinutes(5));
int defaultMin = defaultSettings.getAsInt("min", 1);
int defaultSize = defaultSettings.getAsInt("size", EsExecutors.boundedNumberOfProcessors(settings));
final Integer queueSize = settings.getAsInt("queue_size", defaultSettings.getAsInt("queue_size", null));
if (queueSize != null) {
throw new IllegalArgumentException("thread pool [" + name + "] of type scaling can not have its queue re-sized but was [" + queueSize + "]");
}
if (previousExecutorHolder != null) {
if (ThreadPoolType.SCALING == previousInfo.getThreadPoolType()) {
TimeValue updatedKeepAlive = settings.getAsTime("keep_alive", previousInfo.getKeepAlive());
int updatedMin = settings.getAsInt("min", previousInfo.getMin());
int updatedSize = settings.getAsInt("max", settings.getAsInt("size", previousInfo.getMax()));
if (!previousInfo.getKeepAlive().equals(updatedKeepAlive) || previousInfo.getMin() != updatedMin || previousInfo.getMax() != updatedSize) {
logger.debug("updating thread pool [{}], type [{}], keep_alive [{}]", name, type, updatedKeepAlive);
if (!previousInfo.getKeepAlive().equals(updatedKeepAlive)) {
((EsThreadPoolExecutor) previousExecutorHolder.executor()).setKeepAliveTime(updatedKeepAlive.millis(), TimeUnit.MILLISECONDS);
}
if (previousInfo.getMin() != updatedMin) {
((EsThreadPoolExecutor) previousExecutorHolder.executor()).setCorePoolSize(updatedMin);
}
if (previousInfo.getMax() != updatedSize) {
((EsThreadPoolExecutor) previousExecutorHolder.executor()).setMaximumPoolSize(updatedSize);
}
return new ExecutorHolder(previousExecutorHolder.executor(), new Info(name, threadPoolType, updatedMin, updatedSize, updatedKeepAlive, null));
}
return previousExecutorHolder;
}
if (previousInfo.getKeepAlive() != null) {
defaultKeepAlive = previousInfo.getKeepAlive();
}
if (previousInfo.getMin() >= 0) {
defaultMin = previousInfo.getMin();
}
if (previousInfo.getMax() >= 0) {
defaultSize = previousInfo.getMax();
}
}
TimeValue keepAlive = settings.getAsTime("keep_alive", defaultKeepAlive);
int min = settings.getAsInt("min", defaultMin);
int size = settings.getAsInt("max", settings.getAsInt("size", defaultSize));
if (previousExecutorHolder != null) {
logger.debug("updating thread pool [{}], type [{}], min [{}], size [{}], keep_alive [{}]", name, type, min, size, keepAlive);
} else {
logger.debug("creating thread pool [{}], type [{}], min [{}], size [{}], keep_alive [{}]", name, type, min, size, keepAlive);
}
Executor executor = EsExecutors.newScaling(name, min, size, keepAlive.millis(), TimeUnit.MILLISECONDS, threadFactory, threadContext);
return new ExecutorHolder(executor, new Info(name, threadPoolType, min, size, keepAlive, null));
}
throw new IllegalArgumentException("No type found [" + type + "], for [" + name + "]");
}
private int applyHardSizeLimit(String name, int size) {
int availableProcessors = EsExecutors.boundedNumberOfProcessors(settings);
if ((name.equals(Names.BULK) || name.equals(Names.INDEX)) && size > availableProcessors) {
// We use a hard max size for the indexing pools, because if too many threads enter Lucene's IndexWriter, it means
// too many segments written, too frequently, too much merging, etc:
// TODO: I would love to be loud here (throw an exception if you ask for a too-big size), but I think this is dangerous
// because on upgrade this setting could be in cluster state and hard for the user to correct?
logger.warn("requested thread pool size [{}] for [{}] is too large; setting to maximum [{}] instead",
size, name, availableProcessors);
size = availableProcessors;
}
return size;
}
/**
* Constrains a value between minimum and maximum values
* (inclusive).
@ -618,92 +389,6 @@ public class ThreadPool extends AbstractComponent implements Closeable {
return boundedBy(2 * numberOfProcessors, 2, Integer.MAX_VALUE);
}
private void updateSettings(Settings settings) {
Map<String, Settings> groupSettings = settings.getAsGroups();
if (groupSettings.isEmpty()) {
return;
}
for (Map.Entry<String, Settings> executor : defaultExecutorTypeSettings.entrySet()) {
Settings updatedSettings = groupSettings.get(executor.getKey());
if (updatedSettings == null) {
continue;
}
ExecutorHolder oldExecutorHolder = executors.get(executor.getKey());
ExecutorHolder newExecutorHolder = rebuild(executor.getKey(), oldExecutorHolder, updatedSettings, executor.getValue());
if (!oldExecutorHolder.equals(newExecutorHolder)) {
Map<String, ExecutorHolder> newExecutors = new HashMap<>(executors);
newExecutors.put(executor.getKey(), newExecutorHolder);
executors = unmodifiableMap(newExecutors);
if (!oldExecutorHolder.executor().equals(newExecutorHolder.executor()) && oldExecutorHolder.executor() instanceof EsThreadPoolExecutor) {
retiredExecutors.add(oldExecutorHolder);
((EsThreadPoolExecutor) oldExecutorHolder.executor()).shutdown(new ExecutorShutdownListener(oldExecutorHolder));
}
}
}
// Building custom thread pools
for (Map.Entry<String, Settings> entry : groupSettings.entrySet()) {
if (defaultExecutorTypeSettings.containsKey(entry.getKey())) {
continue;
}
ExecutorHolder oldExecutorHolder = executors.get(entry.getKey());
ExecutorHolder newExecutorHolder = rebuild(entry.getKey(), oldExecutorHolder, entry.getValue(), Settings.EMPTY);
// Can't introduce new thread pools at runtime, because The oldExecutorHolder variable will be null in the
// case the settings contains a thread pool not defined in the initial settings in the constructor. The if
// statement will then fail and so this prevents the addition of new thread groups at runtime, which is desired.
if (!newExecutorHolder.equals(oldExecutorHolder)) {
Map<String, ExecutorHolder> newExecutors = new HashMap<>(executors);
newExecutors.put(entry.getKey(), newExecutorHolder);
executors = unmodifiableMap(newExecutors);
if (!oldExecutorHolder.executor().equals(newExecutorHolder.executor()) && oldExecutorHolder.executor() instanceof EsThreadPoolExecutor) {
retiredExecutors.add(oldExecutorHolder);
((EsThreadPoolExecutor) oldExecutorHolder.executor()).shutdown(new ExecutorShutdownListener(oldExecutorHolder));
}
}
}
}
private void validate(Map<String, Settings> groupSettings) {
for (String key : groupSettings.keySet()) {
if (!THREAD_POOL_TYPES.containsKey(key)) {
continue;
}
String type = groupSettings.get(key).get("type");
ThreadPoolType correctThreadPoolType = THREAD_POOL_TYPES.get(key);
// TODO: the type equality check can be removed after #3760/#6732 are addressed
if (type != null && !correctThreadPoolType.getType().equals(type)) {
throw new IllegalArgumentException("setting " + THREADPOOL_GROUP_SETTING.getKey() + key + ".type to " + type + " is not permitted; must be " + correctThreadPoolType.getType());
}
}
}
/**
* A thread pool size can also be unbounded and is represented by -1, which is not supported by SizeValue (which only supports positive numbers)
*/
private SizeValue getAsSizeOrUnbounded(Settings settings, String setting, SizeValue defaultValue) throws SettingsException {
if ("-1".equals(settings.get(setting))) {
return null;
}
return parseSizeValue(settings.get(setting), defaultValue);
}
class ExecutorShutdownListener implements EsThreadPoolExecutor.ShutdownListener {
private ExecutorHolder holder;
public ExecutorShutdownListener(ExecutorHolder holder) {
this.holder = holder;
}
@Override
public void onTerminated() {
retiredExecutors.remove(holder);
}
}
class LoggingRunnable implements Runnable {
private final Runnable runnable;

View File

@ -20,20 +20,25 @@
package org.elasticsearch.threadpool;
import org.elasticsearch.common.inject.AbstractModule;
import org.elasticsearch.common.settings.SettingsModule;
/**
*
*/
public class ThreadPoolModule extends AbstractModule {
private final ThreadPool threadPool;
public ThreadPoolModule(ThreadPool threadPool) {
public ThreadPoolModule(final ThreadPool threadPool) {
this.threadPool = threadPool;
}
public void prepareSettings(SettingsModule settingsModule) {
for (final ExecutorBuilder<?> builder : threadPool.builders()) {
builder.getRegisteredSettings().forEach(settingsModule::registerSetting);
}
}
@Override
protected void configure() {
bind(ThreadPool.class).toInstance(threadPool);
}
}

View File

@ -45,12 +45,12 @@ public class RejectionActionIT extends ESIntegTestCase {
protected Settings nodeSettings(int nodeOrdinal) {
return Settings.builder()
.put(super.nodeSettings(nodeOrdinal))
.put("threadpool.search.size", 1)
.put("threadpool.search.queue_size", 1)
.put("threadpool.index.size", 1)
.put("threadpool.index.queue_size", 1)
.put("threadpool.get.size", 1)
.put("threadpool.get.queue_size", 1)
.put("thread_pool.search.size", 1)
.put("thread_pool.search.queue_size", 1)
.put("thread_pool.index.size", 1)
.put("thread_pool.index.queue_size", 1)
.put("thread_pool.get.size", 1)
.put("thread_pool.get.queue_size", 1)
.build();
}

View File

@ -42,6 +42,7 @@ import org.elasticsearch.indices.breaker.NoneCircuitBreakerService;
import org.elasticsearch.tasks.TaskManager;
import org.elasticsearch.test.ESTestCase;
import org.elasticsearch.test.tasks.MockTaskManager;
import org.elasticsearch.threadpool.TestThreadPool;
import org.elasticsearch.threadpool.ThreadPool;
import org.elasticsearch.transport.TransportService;
import org.elasticsearch.transport.local.LocalTransport;
@ -72,7 +73,7 @@ public abstract class TaskManagerTestCase extends ESTestCase {
@BeforeClass
public static void beforeClass() {
threadPool = new ThreadPool(TransportTasksActionTests.class.getSimpleName());
threadPool = new TestThreadPool(TransportTasksActionTests.class.getSimpleName());
}
@AfterClass

View File

@ -300,6 +300,10 @@ public class CreateIndexIT extends ESIntegTestCase {
assertTrue("at least 2 nodes but was: " + dataNodes.size(), dataNodes.size() >= 2);
DiscoveryNode[] discoveryNodes = dataNodes.values().toArray(DiscoveryNode.class);
String mergeNode = discoveryNodes[0].getName();
// ensure all shards are allocated otherwise the ensure green below might not succeed since we require the merge node
// if we change the setting too quickly we will end up with one replica unassigned which can't be assigned anymore due
// to the require._name below.
ensureGreen();
// relocate all shards to one node such that we can merge it.
client().admin().indices().prepareUpdateSettings("source")
.setSettings(Settings.builder()
@ -343,6 +347,10 @@ public class CreateIndexIT extends ESIntegTestCase {
DiscoveryNode[] discoveryNodes = dataNodes.values().toArray(DiscoveryNode.class);
String spareNode = discoveryNodes[0].getName();
String mergeNode = discoveryNodes[1].getName();
// ensure all shards are allocated otherwise the ensure green below might not succeed since we require the merge node
// if we change the setting too quickly we will end up with one replica unassigned which can't be assigned anymore due
// to the require._name below.
ensureGreen();
// relocate all shards to one node such that we can merge it.
client().admin().indices().prepareUpdateSettings("source")
.setSettings(Settings.builder().put("index.routing.allocation.require._name", mergeNode)

View File

@ -52,10 +52,10 @@ public class BulkProcessorRetryIT extends ESIntegTestCase {
.put(super.nodeSettings(nodeOrdinal))
// don't mess with this one! It's quite sensitive to a low queue size
// (see also ThreadedActionListener which is happily spawning threads even when we already got rejected)
//.put("threadpool.listener.queue_size", 1)
.put("threadpool.get.queue_size", 1)
//.put("thread_pool.listener.queue_size", 1)
.put("thread_pool.get.queue_size", 1)
// default is 50
.put("threadpool.bulk.queue_size", 30)
.put("thread_pool.bulk.queue_size", 30)
.build();
}

View File

@ -24,6 +24,7 @@ import org.elasticsearch.action.ActionRequest;
import org.elasticsearch.action.ActionRequestValidationException;
import org.elasticsearch.action.delete.DeleteRequest;
import org.elasticsearch.action.index.IndexRequest;
import org.elasticsearch.action.support.WriteRequest.RefreshPolicy;
import org.elasticsearch.action.update.UpdateRequest;
import org.elasticsearch.client.Requests;
import org.elasticsearch.common.Strings;
@ -180,22 +181,22 @@ public class BulkRequestTests extends ESTestCase {
public void testBulkRequestWithRefresh() throws Exception {
BulkRequest bulkRequest = new BulkRequest();
// We force here a "id is missing" validation error
bulkRequest.add(new DeleteRequest("index", "type", null).refresh(true));
bulkRequest.add(new DeleteRequest("index", "type", null).setRefreshPolicy(RefreshPolicy.IMMEDIATE));
// We force here a "type is missing" validation error
bulkRequest.add(new DeleteRequest("index", null, "id"));
bulkRequest.add(new DeleteRequest("index", "type", "id").refresh(true));
bulkRequest.add(new UpdateRequest("index", "type", "id").doc("{}").refresh(true));
bulkRequest.add(new IndexRequest("index", "type", "id").source("{}").refresh(true));
bulkRequest.add(new DeleteRequest("index", "type", "id").setRefreshPolicy(RefreshPolicy.IMMEDIATE));
bulkRequest.add(new UpdateRequest("index", "type", "id").doc("{}").setRefreshPolicy(RefreshPolicy.IMMEDIATE));
bulkRequest.add(new IndexRequest("index", "type", "id").source("{}").setRefreshPolicy(RefreshPolicy.IMMEDIATE));
ActionRequestValidationException validate = bulkRequest.validate();
assertThat(validate, notNullValue());
assertThat(validate.validationErrors(), not(empty()));
assertThat(validate.validationErrors(), contains(
"Refresh is not supported on an item request, set the refresh flag on the BulkRequest instead.",
"RefreshPolicy is not supported on an item request. Set it on the BulkRequest instead.",
"id is missing",
"type is missing",
"Refresh is not supported on an item request, set the refresh flag on the BulkRequest instead.",
"Refresh is not supported on an item request, set the refresh flag on the BulkRequest instead.",
"Refresh is not supported on an item request, set the refresh flag on the BulkRequest instead."));
"RefreshPolicy is not supported on an item request. Set it on the BulkRequest instead.",
"RefreshPolicy is not supported on an item request. Set it on the BulkRequest instead.",
"RefreshPolicy is not supported on an item request. Set it on the BulkRequest instead."));
}
// issue 15120

View File

@ -19,6 +19,7 @@
package org.elasticsearch.action.bulk;
import org.elasticsearch.action.support.WriteRequest.RefreshPolicy;
import org.elasticsearch.index.shard.ShardId;
import org.elasticsearch.test.ESTestCase;
@ -28,9 +29,11 @@ public class BulkShardRequestTests extends ESTestCase {
public void testToString() {
String index = randomSimpleString(random(), 10);
int count = between(1, 100);
BulkShardRequest r = new BulkShardRequest(null, new ShardId(index, "ignored", 0), false, new BulkItemRequest[count]);
BulkShardRequest r = new BulkShardRequest(null, new ShardId(index, "ignored", 0), RefreshPolicy.NONE, new BulkItemRequest[count]);
assertEquals("BulkShardRequest to [" + index + "] containing [" + count + "] requests", r.toString());
r = new BulkShardRequest(null, new ShardId(index, "ignored", 0), true, new BulkItemRequest[count]);
r = new BulkShardRequest(null, new ShardId(index, "ignored", 0), RefreshPolicy.IMMEDIATE, new BulkItemRequest[count]);
assertEquals("BulkShardRequest to [" + index + "] containing [" + count + "] requests and a refresh", r.toString());
r = new BulkShardRequest(null, new ShardId(index, "ignored", 0), RefreshPolicy.WAIT_UNTIL, new BulkItemRequest[count]);
assertEquals("BulkShardRequest to [" + index + "] containing [" + count + "] requests blocking until refresh", r.toString());
}
}

View File

@ -38,6 +38,7 @@ import org.elasticsearch.common.util.concurrent.AtomicArray;
import org.elasticsearch.tasks.Task;
import org.elasticsearch.test.ESTestCase;
import org.elasticsearch.test.transport.CapturingTransport;
import org.elasticsearch.threadpool.TestThreadPool;
import org.elasticsearch.threadpool.ThreadPool;
import org.elasticsearch.transport.TransportService;
import org.junit.After;
@ -63,7 +64,7 @@ public class TransportBulkActionTookTests extends ESTestCase {
@BeforeClass
public static void beforeClass() {
threadPool = new ThreadPool("TransportBulkActionTookTests");
threadPool = new TestThreadPool("TransportBulkActionTookTests");
}
@AfterClass

View File

@ -21,6 +21,7 @@ package org.elasticsearch.action.support;
import org.elasticsearch.action.ActionListener;
import org.elasticsearch.common.util.concurrent.AbstractRunnable;
import org.elasticsearch.test.ESTestCase;
import org.elasticsearch.threadpool.TestThreadPool;
import org.elasticsearch.threadpool.ThreadPool;
import org.elasticsearch.transport.Transports;
@ -31,7 +32,7 @@ import java.util.concurrent.atomic.AtomicReference;
public class ListenableActionFutureTests extends ESTestCase {
public void testListenerIsCallableFromNetworkThreads() throws Throwable {
ThreadPool threadPool = new ThreadPool("testListenerIsCallableFromNetworkThreads");
ThreadPool threadPool = new TestThreadPool("testListenerIsCallableFromNetworkThreads");
try {
final PlainListenableActionFuture<Object> future = new PlainListenableActionFuture<>(threadPool);
final CountDownLatch listenerCalled = new CountDownLatch(1);

View File

@ -55,6 +55,7 @@ import org.elasticsearch.index.shard.ShardId;
import org.elasticsearch.rest.RestStatus;
import org.elasticsearch.test.ESTestCase;
import org.elasticsearch.test.transport.CapturingTransport;
import org.elasticsearch.threadpool.TestThreadPool;
import org.elasticsearch.threadpool.ThreadPool;
import org.elasticsearch.transport.TransportChannel;
import org.elasticsearch.transport.TransportResponse;
@ -182,7 +183,7 @@ public class TransportBroadcastByNodeActionTests extends ESTestCase {
@BeforeClass
public static void startThreadPool() {
THREAD_POOL = new ThreadPool(TransportBroadcastByNodeActionTests.class.getSimpleName());
THREAD_POOL = new TestThreadPool(TransportBroadcastByNodeActionTests.class.getSimpleName());
}
@Before

View File

@ -45,6 +45,7 @@ import org.elasticsearch.rest.RestStatus;
import org.elasticsearch.tasks.Task;
import org.elasticsearch.test.ESTestCase;
import org.elasticsearch.test.transport.CapturingTransport;
import org.elasticsearch.threadpool.TestThreadPool;
import org.elasticsearch.threadpool.ThreadPool;
import org.elasticsearch.transport.ConnectTransportException;
import org.elasticsearch.transport.TransportService;
@ -76,7 +77,7 @@ public class TransportMasterNodeActionTests extends ESTestCase {
@BeforeClass
public static void beforeClass() {
threadPool = new ThreadPool("TransportMasterNodeActionTests");
threadPool = new TestThreadPool("TransportMasterNodeActionTests");
}
@Override

View File

@ -35,6 +35,7 @@ import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.transport.DummyTransportAddress;
import org.elasticsearch.test.ESTestCase;
import org.elasticsearch.test.transport.CapturingTransport;
import org.elasticsearch.threadpool.TestThreadPool;
import org.elasticsearch.threadpool.ThreadPool;
import org.elasticsearch.transport.TransportService;
import org.junit.After;
@ -162,7 +163,7 @@ public class TransportNodesActionTests extends ESTestCase {
@BeforeClass
public static void startThreadPool() {
THREAD_POOL = new ThreadPool(TransportBroadcastByNodeActionTests.class.getSimpleName());
THREAD_POOL = new TestThreadPool(TransportBroadcastByNodeActionTests.class.getSimpleName());
}
@AfterClass

View File

@ -21,7 +21,6 @@ package org.elasticsearch.action.support.replication;
import org.elasticsearch.Version;
import org.elasticsearch.action.ActionListener;
import org.elasticsearch.action.NoShardAvailableActionException;
import org.elasticsearch.action.ReplicationResponse;
import org.elasticsearch.action.ShardOperationFailedException;
import org.elasticsearch.action.UnavailableShardsException;
import org.elasticsearch.action.admin.indices.flush.FlushRequest;
@ -45,6 +44,7 @@ import org.elasticsearch.indices.breaker.NoneCircuitBreakerService;
import org.elasticsearch.rest.RestStatus;
import org.elasticsearch.tasks.Task;
import org.elasticsearch.test.ESTestCase;
import org.elasticsearch.threadpool.TestThreadPool;
import org.elasticsearch.threadpool.ThreadPool;
import org.elasticsearch.transport.TransportService;
import org.elasticsearch.transport.local.LocalTransport;
@ -81,7 +81,7 @@ public class BroadcastReplicationTests extends ESTestCase {
@BeforeClass
public static void beforeClass() {
threadPool = new ThreadPool("BroadcastReplicationTests");
threadPool = new TestThreadPool("BroadcastReplicationTests");
circuitBreakerService = new NoneCircuitBreakerService();
}

View File

@ -21,16 +21,15 @@ package org.elasticsearch.action.support.replication;
import org.apache.lucene.index.CorruptIndexException;
import org.elasticsearch.ElasticsearchException;
import org.elasticsearch.action.ActionListener;
import org.elasticsearch.action.ReplicationResponse;
import org.elasticsearch.action.UnavailableShardsException;
import org.elasticsearch.action.WriteConsistencyLevel;
import org.elasticsearch.action.support.PlainActionFuture;
import org.elasticsearch.action.support.replication.ReplicationResponse.ShardInfo;
import org.elasticsearch.cluster.ClusterState;
import org.elasticsearch.cluster.node.DiscoveryNodes;
import org.elasticsearch.cluster.routing.IndexShardRoutingTable;
import org.elasticsearch.cluster.routing.ShardRouting;
import org.elasticsearch.cluster.routing.ShardRoutingState;
import org.elasticsearch.common.collect.Tuple;
import org.elasticsearch.common.io.stream.StreamInput;
import org.elasticsearch.common.io.stream.StreamOutput;
import org.elasticsearch.common.logging.ESLogger;
@ -102,7 +101,7 @@ public class ReplicationOperationTests extends ESTestCase {
}
Request request = new Request(shardId);
PlainActionFuture<Response> listener = new PlainActionFuture<>();
PlainActionFuture<TestPrimary.Result> listener = new PlainActionFuture<>();
final ClusterState finalState = state;
final TestReplicaProxy replicasProxy = new TestReplicaProxy(expectedFailures);
final TestReplicationOperation op = new TestReplicationOperation(request,
@ -114,7 +113,7 @@ public class ReplicationOperationTests extends ESTestCase {
assertThat(request.processedOnReplicas, equalTo(expectedReplicas));
assertThat(replicasProxy.failedReplicas, equalTo(expectedFailedShards));
assertTrue("listener is not marked as done", listener.isDone());
Response.ShardInfo shardInfo = listener.actionGet().getShardInfo();
ShardInfo shardInfo = listener.actionGet().getShardInfo();
assertThat(shardInfo.getFailed(), equalTo(expectedFailedShards.size()));
assertThat(shardInfo.getFailures(), arrayWithSize(expectedFailedShards.size()));
assertThat(shardInfo.getSuccessful(), equalTo(1 + expectedReplicas.size() - expectedFailures.size()));
@ -135,7 +134,7 @@ public class ReplicationOperationTests extends ESTestCase {
final ShardRouting primaryShard = indexShardRoutingTable.primaryShard();
Request request = new Request(shardId);
PlainActionFuture<Response> listener = new PlainActionFuture<>();
PlainActionFuture<TestPrimary.Result> listener = new PlainActionFuture<>();
final TestReplicationOperation op = new TestReplicationOperation(request,
new TestPrimary(primaryShard, primaryTerm), listener, false, false,
new TestReplicaProxy(), () -> state, logger, "test");
@ -143,7 +142,7 @@ public class ReplicationOperationTests extends ESTestCase {
assertThat("request was not processed on primary", request.processedOnPrimary.get(), equalTo(true));
assertThat(request.processedOnReplicas, equalTo(Collections.emptySet()));
assertTrue("listener is not marked as done", listener.isDone());
Response.ShardInfo shardInfo = listener.actionGet().getShardInfo();
ShardInfo shardInfo = listener.actionGet().getShardInfo();
assertThat(shardInfo.getFailed(), equalTo(0));
assertThat(shardInfo.getFailures(), arrayWithSize(0));
assertThat(shardInfo.getSuccessful(), equalTo(1));
@ -172,7 +171,7 @@ public class ReplicationOperationTests extends ESTestCase {
expectedFailures.put(failedReplica, new CorruptIndexException("simulated", (String) null));
Request request = new Request(shardId);
PlainActionFuture<Response> listener = new PlainActionFuture<>();
PlainActionFuture<TestPrimary.Result> listener = new PlainActionFuture<>();
final ClusterState finalState = state;
final TestReplicaProxy replicasProxy = new TestReplicaProxy(expectedFailures) {
@Override
@ -233,16 +232,16 @@ public class ReplicationOperationTests extends ESTestCase {
final ShardRouting primaryShard = state.get().routingTable().shardRoutingTable(shardId).primaryShard();
final TestPrimary primary = new TestPrimary(primaryShard, primaryTerm) {
@Override
public Tuple<Response, Request> perform(Request request) throws Exception {
final Tuple<Response, Request> tuple = super.perform(request);
public Result perform(Request request) throws Exception {
Result result = super.perform(request);
state.set(changedState);
logger.debug("--> state after primary operation:\n{}", state.get().prettyPrint());
return tuple;
return result;
}
};
Request request = new Request(shardId);
PlainActionFuture<Response> listener = new PlainActionFuture<>();
PlainActionFuture<TestPrimary.Result> listener = new PlainActionFuture<>();
final TestReplicationOperation op = new TestReplicationOperation(request, primary, listener,
new TestReplicaProxy(), state::get);
op.execute();
@ -296,7 +295,7 @@ public class ReplicationOperationTests extends ESTestCase {
state.prettyPrint());
final long primaryTerm = state.metaData().index(index).primaryTerm(shardId.id());
final IndexShardRoutingTable shardRoutingTable = state.routingTable().index(index).shard(shardId.id());
PlainActionFuture<Response> listener = new PlainActionFuture<>();
PlainActionFuture<TestPrimary.Result> listener = new PlainActionFuture<>();
final ShardRouting primaryShard = shardRoutingTable.primaryShard();
final TestReplicationOperation op = new TestReplicationOperation(request,
new TestPrimary(primaryShard, primaryTerm),
@ -362,10 +361,7 @@ public class ReplicationOperationTests extends ESTestCase {
}
}
static class Response extends ReplicationResponse {
}
static class TestPrimary implements ReplicationOperation.Primary<Request, Request, Response> {
static class TestPrimary implements ReplicationOperation.Primary<Request, Request, TestPrimary.Result> {
final ShardRouting routing;
final long term;
@ -385,12 +381,35 @@ public class ReplicationOperationTests extends ESTestCase {
}
@Override
public Tuple<Response, Request> perform(Request request) throws Exception {
public Result perform(Request request) throws Exception {
if (request.processedOnPrimary.compareAndSet(false, true) == false) {
fail("processed [" + request + "] twice");
}
request.primaryTerm(term);
return new Tuple<>(new Response(), request);
return new Result(request);
}
static class Result implements ReplicationOperation.PrimaryResult<Request> {
private final Request request;
private ShardInfo shardInfo;
public Result(Request request) {
this.request = request;
}
@Override
public Request replicaRequest() {
return request;
}
@Override
public void setShardInfo(ShardInfo shardInfo) {
this.shardInfo = shardInfo;
}
public ShardInfo getShardInfo() {
return shardInfo;
}
}
}
@ -436,15 +455,15 @@ public class ReplicationOperationTests extends ESTestCase {
}
}
class TestReplicationOperation extends ReplicationOperation<Request, Request, Response> {
public TestReplicationOperation(Request request, Primary<Request, Request, Response> primary, ActionListener<Response> listener,
Replicas<Request> replicas, Supplier<ClusterState> clusterStateSupplier) {
class TestReplicationOperation extends ReplicationOperation<Request, Request, TestPrimary.Result> {
public TestReplicationOperation(Request request, Primary<Request, Request, TestPrimary.Result> primary,
ActionListener<TestPrimary.Result> listener, Replicas<Request> replicas, Supplier<ClusterState> clusterStateSupplier) {
this(request, primary, listener, true, false, replicas, clusterStateSupplier, ReplicationOperationTests.this.logger, "test");
}
public TestReplicationOperation(Request request, Primary<Request, Request, Response> primary, ActionListener<Response> listener,
boolean executeOnReplicas, boolean checkWriteConsistency, Replicas<Request> replicas,
Supplier<ClusterState> clusterStateSupplier, ESLogger logger, String opType) {
public TestReplicationOperation(Request request, Primary<Request, Request, TestPrimary.Result> primary,
ActionListener<TestPrimary.Result> listener, boolean executeOnReplicas, boolean checkWriteConsistency,
Replicas<Request> replicas, Supplier<ClusterState> clusterStateSupplier, ESLogger logger, String opType) {
super(request, primary, listener, executeOnReplicas, checkWriteConsistency, replicas, clusterStateSupplier, logger, opType);
}
}

View File

@ -20,7 +20,6 @@ package org.elasticsearch.action.support.replication;
import org.elasticsearch.ElasticsearchException;
import org.elasticsearch.action.ActionListener;
import org.elasticsearch.action.ReplicationResponse;
import org.elasticsearch.action.UnavailableShardsException;
import org.elasticsearch.action.support.ActionFilters;
import org.elasticsearch.action.support.PlainActionFuture;
@ -43,7 +42,6 @@ import org.elasticsearch.cluster.routing.allocation.AllocationService;
import org.elasticsearch.cluster.routing.allocation.RoutingAllocation;
import org.elasticsearch.cluster.service.ClusterService;
import org.elasticsearch.common.Nullable;
import org.elasticsearch.common.collect.Tuple;
import org.elasticsearch.common.io.stream.StreamInput;
import org.elasticsearch.common.io.stream.StreamOutput;
import org.elasticsearch.common.lease.Releasable;
@ -59,6 +57,7 @@ import org.elasticsearch.rest.RestStatus;
import org.elasticsearch.test.ESAllocationTestCase;
import org.elasticsearch.test.ESTestCase;
import org.elasticsearch.test.transport.CapturingTransport;
import org.elasticsearch.threadpool.TestThreadPool;
import org.elasticsearch.threadpool.ThreadPool;
import org.elasticsearch.transport.TransportChannel;
import org.elasticsearch.transport.TransportException;
@ -111,7 +110,7 @@ public class TransportReplicationActionTests extends ESTestCase {
@BeforeClass
public static void beforeClass() {
threadPool = new ThreadPool("ShardReplicationTests");
threadPool = new TestThreadPool("ShardReplicationTests");
}
@Override
@ -155,7 +154,7 @@ public class TransportReplicationActionTests extends ESTestCase {
ClusterBlocks.Builder block = ClusterBlocks.builder()
.addGlobalBlock(new ClusterBlock(1, "non retryable", false, true, RestStatus.SERVICE_UNAVAILABLE, ClusterBlockLevel.ALL));
setState(clusterService, ClusterState.builder(clusterService.state()).blocks(block));
TransportReplicationAction.ReroutePhase reroutePhase = action.new ReroutePhase(task, request, listener);
Action.ReroutePhase reroutePhase = action.new ReroutePhase(task, request, listener);
reroutePhase.run();
assertListenerThrows("primary phase should fail operation", listener, ClusterBlockException.class);
assertPhase(task, "failed");
@ -199,7 +198,7 @@ public class TransportReplicationActionTests extends ESTestCase {
Request request = new Request(shardId).timeout("1ms");
PlainActionFuture<Response> listener = new PlainActionFuture<>();
TransportReplicationAction.ReroutePhase reroutePhase = action.new ReroutePhase(task, request, listener);
Action.ReroutePhase reroutePhase = action.new ReroutePhase(task, request, listener);
reroutePhase.run();
assertListenerThrows("unassigned primary didn't cause a timeout", listener, UnavailableShardsException.class);
assertPhase(task, "failed");
@ -245,7 +244,7 @@ public class TransportReplicationActionTests extends ESTestCase {
Request request = new Request(shardId).timeout("1ms").routedBasedOnClusterVersion(clusterService.state().version() + 1);
PlainActionFuture<Response> listener = new PlainActionFuture<>();
TransportReplicationAction.ReroutePhase reroutePhase = action.new ReroutePhase(null, request, listener);
Action.ReroutePhase reroutePhase = action.new ReroutePhase(null, request, listener);
reroutePhase.run();
assertListenerThrows("cluster state too old didn't cause a timeout", listener, UnavailableShardsException.class);
@ -285,7 +284,7 @@ public class TransportReplicationActionTests extends ESTestCase {
PlainActionFuture<Response> listener = new PlainActionFuture<>();
ReplicationTask task = maybeTask();
TransportReplicationAction.ReroutePhase reroutePhase = action.new ReroutePhase(task, request, listener);
Action.ReroutePhase reroutePhase = action.new ReroutePhase(task, request, listener);
reroutePhase.run();
assertListenerThrows("must throw index not found exception", listener, IndexNotFoundException.class);
assertPhase(task, "failed");
@ -312,7 +311,7 @@ public class TransportReplicationActionTests extends ESTestCase {
PlainActionFuture<Response> listener = new PlainActionFuture<>();
ReplicationTask task = maybeTask();
TransportReplicationAction.ReroutePhase reroutePhase = action.new ReroutePhase(task, request, listener);
Action.ReroutePhase reroutePhase = action.new ReroutePhase(task, request, listener);
reroutePhase.run();
CapturingTransport.CapturedRequest[] capturedRequests = transport.getCapturedRequestsAndClear();
assertThat(capturedRequests, arrayWithSize(1));
@ -364,7 +363,7 @@ public class TransportReplicationActionTests extends ESTestCase {
Request request = new Request(shardId);
PlainActionFuture<Response> listener = new PlainActionFuture<>();
TransportReplicationAction.ReroutePhase reroutePhase = action.new ReroutePhase(task, request, listener);
Action.ReroutePhase reroutePhase = action.new ReroutePhase(task, request, listener);
reroutePhase.run();
assertThat(request.shardId(), equalTo(shardId));
logger.info("--> primary is assigned to [{}], checking request forwarded", primaryNodeId);
@ -393,9 +392,9 @@ public class TransportReplicationActionTests extends ESTestCase {
AtomicBoolean executed = new AtomicBoolean();
Action.PrimaryOperationTransportHandler primaryPhase = action.new PrimaryOperationTransportHandler() {
@Override
protected ReplicationOperation createReplicatedOperation(Request request, ActionListener<Response> actionListener,
Action.PrimaryShardReference primaryShardReference,
boolean executeOnReplicas) {
protected ReplicationOperation<Request, Request, Action.PrimaryResult> createReplicatedOperation(Request request,
ActionListener<Action.PrimaryResult> actionListener, Action.PrimaryShardReference primaryShardReference,
boolean executeOnReplicas) {
return new NoopReplicationOperation(request, actionListener) {
public void execute() throws Exception {
assertPhase(task, "primary");
@ -448,9 +447,9 @@ public class TransportReplicationActionTests extends ESTestCase {
AtomicBoolean executed = new AtomicBoolean();
Action.PrimaryOperationTransportHandler primaryPhase = action.new PrimaryOperationTransportHandler() {
@Override
protected ReplicationOperation createReplicatedOperation(Request request, ActionListener<Response> actionListener,
Action.PrimaryShardReference primaryShardReference,
boolean executeOnReplicas) {
protected ReplicationOperation<Request, Request, Action.PrimaryResult> createReplicatedOperation(Request request,
ActionListener<Action.PrimaryResult> actionListener, Action.PrimaryShardReference primaryShardReference,
boolean executeOnReplicas) {
return new NoopReplicationOperation(request, actionListener) {
public void execute() throws Exception {
assertPhase(task, "primary");
@ -478,9 +477,9 @@ public class TransportReplicationActionTests extends ESTestCase {
};
Action.PrimaryShardReference primary = action.new PrimaryShardReference(shard, releasable);
final Request request = new Request();
Tuple<Response, Request> result = primary.perform(request);
Request replicaRequest = primary.perform(request).replicaRequest;
assertThat(result.v2().primaryTerm(), equalTo(primaryTerm));
assertThat(replicaRequest.primaryTerm(), equalTo(primaryTerm));
final ElasticsearchException exception = new ElasticsearchException("testing");
primary.failShard("test", exception);
@ -582,9 +581,9 @@ public class TransportReplicationActionTests extends ESTestCase {
setState(clusterService, state);
Action.PrimaryOperationTransportHandler primaryPhase = action.new PrimaryOperationTransportHandler() {
@Override
protected ReplicationOperation createReplicatedOperation(Request request, ActionListener<Response> actionListener,
Action.PrimaryShardReference primaryShardReference,
boolean executeOnReplicas) {
protected ReplicationOperation<Request, Request, Action.PrimaryResult> createReplicatedOperation(Request request,
ActionListener<Action.PrimaryResult> actionListener, Action.PrimaryShardReference primaryShardReference,
boolean executeOnReplicas) {
assertFalse(executeOnReplicas);
return new NoopReplicationOperation(request, actionListener);
}
@ -608,9 +607,9 @@ public class TransportReplicationActionTests extends ESTestCase {
Action.PrimaryOperationTransportHandler primaryPhase = action.new PrimaryOperationTransportHandler() {
@Override
protected ReplicationOperation createReplicatedOperation(Request request, ActionListener<Response> listener,
Action.PrimaryShardReference primaryShardReference,
boolean executeOnReplicas) {
protected ReplicationOperation<Request, Request, Action.PrimaryResult> createReplicatedOperation(Request request,
ActionListener<Action.PrimaryResult> listener, Action.PrimaryShardReference primaryShardReference,
boolean executeOnReplicas) {
assertIndexShardCounter(1);
if (throwExceptionOnCreation) {
throw new ElasticsearchException("simulated exception, during createReplicatedOperation");
@ -623,7 +622,7 @@ public class TransportReplicationActionTests extends ESTestCase {
if (throwExceptionOnRun) {
throw new ElasticsearchException("simulated exception, during performOnPrimary");
} else if (respondWithError) {
this.finalResponseListener.onFailure(new ElasticsearchException("simulated exception, as a response"));
this.resultListener.onFailure(new ElasticsearchException("simulated exception, as a response"));
} else {
super.execute();
}
@ -667,13 +666,13 @@ public class TransportReplicationActionTests extends ESTestCase {
final ReplicationTask task = maybeTask();
Action action = new Action(Settings.EMPTY, "testActionWithExceptions", transportService, clusterService, threadPool) {
@Override
protected void shardOperationOnReplica(Request request) {
protected ReplicaResult shardOperationOnReplica(Request request) {
assertIndexShardCounter(1);
assertPhase(task, "replica");
if (throwException) {
throw new ElasticsearchException("simulated");
}
super.shardOperationOnReplica(request);
return new ReplicaResult();
}
};
final Action.ReplicaOperationTransportHandler replicaOperationTransportHandler = action.new ReplicaOperationTransportHandler();
@ -765,15 +764,16 @@ public class TransportReplicationActionTests extends ESTestCase {
}
@Override
protected Tuple<Response, Request> shardOperationOnPrimary(Request shardRequest) throws Exception {
protected PrimaryResult shardOperationOnPrimary(Request shardRequest) throws Exception {
boolean executedBefore = shardRequest.processedOnPrimary.getAndSet(true);
assert executedBefore == false : "request has already been executed on the primary";
return new Tuple<>(new Response(), shardRequest);
return new PrimaryResult(shardRequest, new Response());
}
@Override
protected void shardOperationOnReplica(Request request) {
protected ReplicaResult shardOperationOnReplica(Request request) {
request.processedOnReplicas.incrementAndGet();
return new ReplicaResult();
}
@Override
@ -822,15 +822,14 @@ public class TransportReplicationActionTests extends ESTestCase {
}
}
class NoopReplicationOperation extends ReplicationOperation<Request, Request, Response> {
public NoopReplicationOperation(Request request, ActionListener<Response> listener) {
class NoopReplicationOperation extends ReplicationOperation<Request, Request, Action.PrimaryResult> {
public NoopReplicationOperation(Request request, ActionListener<Action.PrimaryResult> listener) {
super(request, null, listener, true, true, null, null, TransportReplicationActionTests.this.logger, "noop");
}
@Override
public void execute() throws Exception {
this.finalResponseListener.onResponse(new Response());
this.resultListener.onResponse(action.new PrimaryResult(null, new Response()));
}
}

View File

@ -0,0 +1,190 @@
/*
* 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.replication;
import org.elasticsearch.action.ActionListener;
import org.elasticsearch.action.support.ActionFilters;
import org.elasticsearch.action.support.WriteRequest.RefreshPolicy;
import org.elasticsearch.action.support.WriteResponse;
import org.elasticsearch.cluster.metadata.IndexNameExpressionResolver;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.index.shard.IndexShard;
import org.elasticsearch.index.shard.ShardId;
import org.elasticsearch.index.translog.Translog;
import org.elasticsearch.index.translog.Translog.Location;
import org.elasticsearch.test.ESTestCase;
import org.elasticsearch.threadpool.ThreadPool;
import org.elasticsearch.transport.TransportService;
import org.junit.Before;
import org.mockito.ArgumentCaptor;
import java.util.HashSet;
import java.util.function.BiConsumer;
import java.util.function.Consumer;
import static org.mockito.Matchers.any;
import static org.mockito.Mockito.mock;
import static org.mockito.Mockito.never;
import static org.mockito.Mockito.verify;
public class TransportWriteActionTests extends ESTestCase {
private IndexShard indexShard;
private Translog.Location location;
@Before
public void initCommonMocks() {
indexShard = mock(IndexShard.class);
location = mock(Translog.Location.class);
}
public void testPrimaryNoRefreshCall() throws Exception {
noRefreshCall(TestAction::shardOperationOnPrimary, TestAction.WritePrimaryResult::respond);
}
public void testReplicaNoRefreshCall() throws Exception {
noRefreshCall(TestAction::shardOperationOnReplica, TestAction.WriteReplicaResult::respond);
}
private <Result, Response> void noRefreshCall(ThrowingBiFunction<TestAction, TestRequest, Result> action,
BiConsumer<Result, CapturingActionListener<Response>> responder)
throws Exception {
TestRequest request = new TestRequest();
request.setRefreshPolicy(RefreshPolicy.NONE); // The default, but we'll set it anyway just to be explicit
Result result = action.apply(new TestAction(), request);
CapturingActionListener<Response> listener = new CapturingActionListener<>();
responder.accept(result, listener);
assertNotNull(listener.response);
verify(indexShard, never()).refresh(any());
verify(indexShard, never()).addRefreshListener(any(), any());
}
public void testPrimaryImmediateRefresh() throws Exception {
immediateRefresh(TestAction::shardOperationOnPrimary, TestAction.WritePrimaryResult::respond, r -> assertTrue(r.forcedRefresh));
}
public void testReplicaImmediateRefresh() throws Exception {
immediateRefresh(TestAction::shardOperationOnReplica, TestAction.WriteReplicaResult::respond, r -> {});
}
private <Result, Response> void immediateRefresh(ThrowingBiFunction<TestAction, TestRequest, Result> action,
BiConsumer<Result, CapturingActionListener<Response>> responder,
Consumer<Response> responseChecker) throws Exception {
TestRequest request = new TestRequest();
request.setRefreshPolicy(RefreshPolicy.IMMEDIATE);
Result result = action.apply(new TestAction(), request);
CapturingActionListener<Response> listener = new CapturingActionListener<>();
responder.accept(result, listener);
assertNotNull(listener.response);
responseChecker.accept(listener.response);
verify(indexShard).refresh("refresh_flag_index");
verify(indexShard, never()).addRefreshListener(any(), any());
}
public void testPrimaryWaitForRefresh() throws Exception {
waitForRefresh(TestAction::shardOperationOnPrimary, TestAction.WritePrimaryResult::respond,
(r, forcedRefresh) -> assertEquals(forcedRefresh, r.forcedRefresh));
}
public void testReplicaWaitForRefresh() throws Exception {
waitForRefresh(TestAction::shardOperationOnReplica, TestAction.WriteReplicaResult::respond, (r, forcedRefresh) -> {});
}
private <Result, Response> void waitForRefresh(ThrowingBiFunction<TestAction, TestRequest, Result> action,
BiConsumer<Result, CapturingActionListener<Response>> responder,
BiConsumer<Response, Boolean> resultChecker) throws Exception {
TestRequest request = new TestRequest();
request.setRefreshPolicy(RefreshPolicy.WAIT_UNTIL);
Result result = action.apply(new TestAction(), request);
CapturingActionListener<Response> listener = new CapturingActionListener<>();
responder.accept(result, listener);
assertNull(listener.response); // Haven't reallresponded yet
@SuppressWarnings({ "unchecked", "rawtypes" })
ArgumentCaptor<Consumer<Boolean>> refreshListener = ArgumentCaptor.forClass((Class) Consumer.class);
verify(indexShard, never()).refresh(any());
verify(indexShard).addRefreshListener(any(), refreshListener.capture());
// Now we can fire the listener manually and we'll get a response
boolean forcedRefresh = randomBoolean();
refreshListener.getValue().accept(forcedRefresh);
assertNotNull(listener.response);
resultChecker.accept(listener.response, forcedRefresh);
}
private class TestAction extends TransportWriteAction<TestRequest, TestResponse> {
protected TestAction() {
super(Settings.EMPTY, "test", mock(TransportService.class), null, null, null, null, new ActionFilters(new HashSet<>()),
new IndexNameExpressionResolver(Settings.EMPTY), TestRequest::new, ThreadPool.Names.SAME);
}
@Override
protected IndexShard indexShard(TestRequest request) {
return indexShard;
}
@Override
protected WriteResult<TestResponse> onPrimaryShard(TestRequest request, IndexShard indexShard) throws Exception {
return new WriteResult<>(new TestResponse(), location);
}
@Override
protected Location onReplicaShard(TestRequest request, IndexShard indexShard) {
return location;
}
@Override
protected TestResponse newResponseInstance() {
return new TestResponse();
}
}
private static class TestRequest extends ReplicatedWriteRequest<TestRequest> {
public TestRequest() {
setShardId(new ShardId("test", "test", 1));
}
}
private static class TestResponse extends ReplicationResponse implements WriteResponse {
boolean forcedRefresh;
@Override
public void setForcedRefresh(boolean forcedRefresh) {
this.forcedRefresh = forcedRefresh;
}
}
private static class CapturingActionListener<R> implements ActionListener<R> {
private R response;
@Override
public void onResponse(R response) {
this.response = response;
}
@Override
public void onFailure(Throwable e) {
throw new RuntimeException(e);
}
}
private interface ThrowingBiFunction<A, B, R> {
R apply(A a, B b) throws Exception;
}
}

View File

@ -43,6 +43,7 @@ import org.elasticsearch.index.shard.ShardId;
import org.elasticsearch.rest.RestStatus;
import org.elasticsearch.test.ESTestCase;
import org.elasticsearch.test.transport.CapturingTransport;
import org.elasticsearch.threadpool.TestThreadPool;
import org.elasticsearch.threadpool.ThreadPool;
import org.elasticsearch.transport.ConnectTransportException;
import org.elasticsearch.transport.TransportException;
@ -133,7 +134,7 @@ public class TransportInstanceSingleOperationActionTests extends ESTestCase {
@BeforeClass
public static void startThreadPool() {
THREAD_POOL = new ThreadPool(TransportInstanceSingleOperationActionTests.class.getSimpleName());
THREAD_POOL = new TestThreadPool(TransportInstanceSingleOperationActionTests.class.getSimpleName());
}
@Before

View File

@ -29,6 +29,7 @@ import org.elasticsearch.action.admin.indices.alias.get.GetAliasesResponse;
import org.elasticsearch.action.admin.indices.create.CreateIndexRequestBuilder;
import org.elasticsearch.action.index.IndexResponse;
import org.elasticsearch.action.search.SearchResponse;
import org.elasticsearch.action.support.WriteRequest.RefreshPolicy;
import org.elasticsearch.cluster.ClusterState;
import org.elasticsearch.cluster.metadata.AliasAction;
import org.elasticsearch.cluster.metadata.AliasMetaData;
@ -175,10 +176,15 @@ public class IndexAliasesIT extends ESIntegTestCase {
assertAcked(admin().indices().prepareAliases().addAlias("test", "tests", termQuery("name", "test")));
logger.info("--> indexing against [test]");
client().index(indexRequest("test").type("type1").id("1").source(source("1", "foo test")).refresh(true)).actionGet();
client().index(indexRequest("test").type("type1").id("2").source(source("2", "bar test")).refresh(true)).actionGet();
client().index(indexRequest("test").type("type1").id("3").source(source("3", "baz test")).refresh(true)).actionGet();
client().index(indexRequest("test").type("type1").id("4").source(source("4", "something else")).refresh(true)).actionGet();
client().index(indexRequest("test").type("type1").id("1").source(source("1", "foo test")).setRefreshPolicy(RefreshPolicy.IMMEDIATE))
.actionGet();
client().index(indexRequest("test").type("type1").id("2").source(source("2", "bar test")).setRefreshPolicy(RefreshPolicy.IMMEDIATE))
.actionGet();
client().index(indexRequest("test").type("type1").id("3").source(source("3", "baz test")).setRefreshPolicy(RefreshPolicy.IMMEDIATE))
.actionGet();
client().index(
indexRequest("test").type("type1").id("4").source(source("4", "something else")).setRefreshPolicy(RefreshPolicy.IMMEDIATE))
.actionGet();
logger.info("--> checking single filtering alias search");
SearchResponse searchResponse = client().prepareSearch("foos").setQuery(QueryBuilders.matchAllQuery()).get();

View File

@ -28,6 +28,7 @@ import org.elasticsearch.cluster.node.DiscoveryNode;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.transport.LocalTransportAddress;
import org.elasticsearch.test.ESTestCase;
import org.elasticsearch.threadpool.TestThreadPool;
import org.elasticsearch.threadpool.ThreadPool;
import org.elasticsearch.transport.BaseTransportResponseHandler;
import org.elasticsearch.transport.TransportException;
@ -63,7 +64,7 @@ public class TransportClientNodesServiceTests extends ESTestCase {
TestIteration() {
ClusterName clusterName = new ClusterName("test");
threadPool = new ThreadPool("transport-client-nodes-service-tests");
threadPool = new TestThreadPool("transport-client-nodes-service-tests");
transport = new FailAndRetryMockTransport<TestResponse>(random(), clusterName) {
@Override
public List<String> getLocalAddresses() {

View File

@ -0,0 +1,41 @@
/*
* 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.client.transport;
import org.elasticsearch.action.admin.cluster.health.ClusterHealthRequest;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.test.ESTestCase;
import java.util.concurrent.ExecutionException;
import static org.hamcrest.CoreMatchers.containsString;
import static org.hamcrest.object.HasToString.hasToString;
public class TransportClientTests extends ESTestCase {
public void testThatUsingAClosedClientThrowsAnException() throws ExecutionException, InterruptedException {
final TransportClient client = TransportClient.builder().settings(Settings.EMPTY).build();
client.close();
final IllegalStateException e =
expectThrows(IllegalStateException.class, () -> client.admin().cluster().health(new ClusterHealthRequest()).get());
assertThat(e, hasToString(containsString("transport client is closed")));
}
}

View File

@ -38,6 +38,7 @@ import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.discovery.Discovery;
import org.elasticsearch.test.ESTestCase;
import org.elasticsearch.test.transport.CapturingTransport;
import org.elasticsearch.threadpool.TestThreadPool;
import org.elasticsearch.threadpool.ThreadPool;
import org.elasticsearch.transport.NodeDisconnectedException;
import org.elasticsearch.transport.NodeNotConnectedException;
@ -97,7 +98,7 @@ public class ShardStateActionTests extends ESTestCase {
@BeforeClass
public static void startThreadPool() {
THREAD_POOL = new ThreadPool("ShardStateActionTest");
THREAD_POOL = new TestThreadPool("ShardStateActionTest");
}
@Override

View File

@ -22,6 +22,7 @@ package org.elasticsearch.cluster.allocation;
import org.apache.lucene.util.IOUtils;
import org.elasticsearch.action.admin.cluster.health.ClusterHealthResponse;
import org.elasticsearch.action.admin.cluster.reroute.ClusterRerouteResponse;
import org.elasticsearch.action.support.WriteRequest.RefreshPolicy;
import org.elasticsearch.cluster.ClusterState;
import org.elasticsearch.cluster.health.ClusterHealthStatus;
import org.elasticsearch.cluster.node.DiscoveryNode;
@ -225,7 +226,7 @@ public class ClusterRerouteIT extends ESIntegTestCase {
assertThat(state.getRoutingNodes().unassigned().size(), equalTo(1));
assertThat(state.getRoutingNodes().node(state.nodes().resolveNode(node_1).getId()).iterator().next().state(), equalTo(ShardRoutingState.STARTED));
client().prepareIndex("test", "type", "1").setSource("field", "value").setRefresh(true).execute().actionGet();
client().prepareIndex("test", "type", "1").setSource("field", "value").setRefreshPolicy(RefreshPolicy.IMMEDIATE).get();
final Index index = resolveIndex("test");
logger.info("--> closing all nodes");

View File

@ -41,6 +41,7 @@ import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.test.ESTestCase;
import org.elasticsearch.test.gateway.NoopGatewayAllocator;
import org.elasticsearch.test.transport.CapturingTransport;
import org.elasticsearch.threadpool.TestThreadPool;
import org.elasticsearch.threadpool.ThreadPool;
import org.elasticsearch.transport.TransportService;
import org.junit.After;
@ -72,7 +73,7 @@ public class ClusterStateHealthTests extends ESTestCase {
@BeforeClass
public static void beforeClass() {
threadPool = new ThreadPool("ClusterStateHealthTests");
threadPool = new TestThreadPool("ClusterStateHealthTests");
}
@Override

View File

@ -37,6 +37,7 @@ import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.common.unit.TimeValue;
import org.elasticsearch.gateway.GatewayAllocator;
import org.elasticsearch.test.ESAllocationTestCase;
import org.elasticsearch.threadpool.TestThreadPool;
import org.elasticsearch.threadpool.ThreadPool;
import org.junit.After;
import org.junit.Before;
@ -73,7 +74,7 @@ public class DelayedAllocationServiceTests extends ESAllocationTestCase {
@Before
public void createDelayedAllocationService() {
threadPool = new ThreadPool(getTestName());
threadPool = new TestThreadPool(getTestName());
clusterService = mock(ClusterService.class);
allocationService = createAllocationService(Settings.EMPTY, new DelayedShardsMockGatewayAllocator());
delayedAllocationService = new TestDelayAllocationService(Settings.EMPTY, threadPool, clusterService, allocationService);

View File

@ -34,7 +34,7 @@ public final class RandomShardRoutingMutator {
public static ShardRouting randomChange(ShardRouting shardRouting, String[] nodes) {
switch (randomInt(2)) {
case 0:
if (shardRouting.unassigned() == false) {
if (shardRouting.unassigned() == false && shardRouting.primary() == false) {
shardRouting = shardRouting.moveToUnassigned(new UnassignedInfo(randomReason(), randomAsciiOfLength(10)));
} else if (shardRouting.unassignedInfo() != null) {
shardRouting = shardRouting.updateUnassignedInfo(new UnassignedInfo(randomReason(), randomAsciiOfLength(10)));

View File

@ -19,11 +19,13 @@
package org.elasticsearch.cluster.routing.allocation;
import com.carrotsearch.hppc.IntHashSet;
import org.elasticsearch.Version;
import org.elasticsearch.cluster.ClusterState;
import org.elasticsearch.cluster.metadata.IndexMetaData;
import org.elasticsearch.cluster.metadata.MetaData;
import org.elasticsearch.cluster.node.DiscoveryNodes;
import org.elasticsearch.cluster.routing.RestoreSource;
import org.elasticsearch.cluster.routing.RoutingTable;
import org.elasticsearch.cluster.routing.allocation.command.AllocationCommands;
import org.elasticsearch.cluster.routing.allocation.command.MoveAllocationCommand;
@ -31,6 +33,8 @@ import org.elasticsearch.cluster.routing.allocation.decider.Decision;
import org.elasticsearch.common.logging.ESLogger;
import org.elasticsearch.common.logging.Loggers;
import org.elasticsearch.common.settings.Settings;
import org.elasticsearch.snapshots.Snapshot;
import org.elasticsearch.snapshots.SnapshotId;
import org.elasticsearch.test.ESAllocationTestCase;
import static org.elasticsearch.cluster.routing.ShardRoutingState.INITIALIZING;
@ -57,9 +61,7 @@ public class ThrottlingAllocationTests extends ESAllocationTestCase {
.put(IndexMetaData.builder("test").settings(settings(Version.CURRENT)).numberOfShards(10).numberOfReplicas(1))
.build();
RoutingTable routingTable = RoutingTable.builder()
.addAsNew(metaData.index("test"))
.build();
RoutingTable routingTable = createRecoveryRoutingTable(metaData.index("test"));
ClusterState clusterState = ClusterState.builder(org.elasticsearch.cluster.ClusterName.DEFAULT).metaData(metaData).routingTable(routingTable).build();
@ -118,9 +120,7 @@ public class ThrottlingAllocationTests extends ESAllocationTestCase {
.put(IndexMetaData.builder("test").settings(settings(Version.CURRENT)).numberOfShards(5).numberOfReplicas(1))
.build();
RoutingTable routingTable = RoutingTable.builder()
.addAsNew(metaData.index("test"))
.build();
RoutingTable routingTable = createRecoveryRoutingTable(metaData.index("test"));
ClusterState clusterState = ClusterState.builder(org.elasticsearch.cluster.ClusterName.DEFAULT).metaData(metaData).routingTable(routingTable).build();
@ -188,9 +188,7 @@ public class ThrottlingAllocationTests extends ESAllocationTestCase {
.put(IndexMetaData.builder("test").settings(settings(Version.CURRENT)).numberOfShards(9).numberOfReplicas(0))
.build();
RoutingTable routingTable = RoutingTable.builder()
.addAsNew(metaData.index("test"))
.build();
RoutingTable routingTable = createRecoveryRoutingTable(metaData.index("test"));
ClusterState clusterState = ClusterState.builder(org.elasticsearch.cluster.ClusterName.DEFAULT).metaData(metaData).routingTable(routingTable).build();
@ -242,89 +240,107 @@ public class ThrottlingAllocationTests extends ESAllocationTestCase {
assertEquals(clusterState.getRoutingNodes().getOutgoingRecoveries("node1"), 1);
}
public void testOutgoingThrottlesAllocaiton() {
Settings settings = Settings.builder()
.put("cluster.routing.allocation.node_concurrent_recoveries", 1)
.put("cluster.routing.allocation.node_initial_primaries_recoveries", 1)
.put("cluster.routing.allocation.cluster_concurrent_rebalance", 1)
.build();
AllocationService strategy = createAllocationService(settings);
public void testOutgoingThrottlesAllocation() {
AllocationService strategy = createAllocationService(Settings.builder()
.put("cluster.routing.allocation.node_concurrent_outgoing_recoveries", 1)
.build());
logger.info("Building initial routing table");
MetaData metaData = MetaData.builder()
.put(IndexMetaData.builder("test").settings(settings(Version.CURRENT)).numberOfShards(3).numberOfReplicas(0))
.put(IndexMetaData.builder("test").settings(settings(Version.CURRENT)).numberOfShards(1).numberOfReplicas(2))
.build();
RoutingTable routingTable = RoutingTable.builder()
.addAsNew(metaData.index("test"))
.build();
RoutingTable routingTable = createRecoveryRoutingTable(metaData.index("test"));
ClusterState clusterState = ClusterState.builder(org.elasticsearch.cluster.ClusterName.DEFAULT).metaData(metaData).routingTable(routingTable).build();
clusterState = ClusterState.builder(clusterState).nodes(DiscoveryNodes.builder().put(newNode("node1")).put(newNode("node2")).put(newNode("node3"))).build();
logger.info("start one node, do reroute, only 1 should initialize");
clusterState = ClusterState.builder(clusterState).nodes(DiscoveryNodes.builder().put(newNode("node1"))).build();
routingTable = strategy.reroute(clusterState, "reroute").routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
assertThat(routingTable.shardsWithState(STARTED).size(), equalTo(0));
assertThat(routingTable.shardsWithState(INITIALIZING).size(), equalTo(3));
assertThat(routingTable.shardsWithState(UNASSIGNED).size(), equalTo(0));
assertEquals(clusterState.getRoutingNodes().getIncomingRecoveries("node1"), 1);
assertEquals(clusterState.getRoutingNodes().getIncomingRecoveries("node2"), 1);
assertEquals(clusterState.getRoutingNodes().getIncomingRecoveries("node3"), 1);
assertEquals(clusterState.getRoutingNodes().getOutgoingRecoveries("node1"), 0);
assertEquals(clusterState.getRoutingNodes().getOutgoingRecoveries("node2"), 0);
assertEquals(clusterState.getRoutingNodes().getOutgoingRecoveries("node3"), 0);
assertThat(routingTable.shardsWithState(STARTED).size(), equalTo(0));
assertThat(routingTable.shardsWithState(INITIALIZING).size(), equalTo(1));
assertThat(routingTable.shardsWithState(UNASSIGNED).size(), equalTo(2));
logger.info("start initializing");
routingTable = strategy.applyStartedShards(clusterState, routingTable.shardsWithState(INITIALIZING)).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
assertEquals(clusterState.getRoutingNodes().getIncomingRecoveries("node1"), 0);
assertEquals(clusterState.getRoutingNodes().getIncomingRecoveries("node2"), 0);
assertEquals(clusterState.getRoutingNodes().getIncomingRecoveries("node3"), 0);
assertEquals(clusterState.getRoutingNodes().getOutgoingRecoveries("node1"), 0);
assertEquals(clusterState.getRoutingNodes().getOutgoingRecoveries("node2"), 0);
assertEquals(clusterState.getRoutingNodes().getOutgoingRecoveries("node3"), 0);
assertThat(routingTable.shardsWithState(STARTED).size(), equalTo(1));
assertThat(routingTable.shardsWithState(INITIALIZING).size(), equalTo(0));
assertThat(routingTable.shardsWithState(UNASSIGNED).size(), equalTo(2));
RoutingAllocation.Result reroute = strategy.reroute(clusterState, new AllocationCommands(new MoveAllocationCommand("test", clusterState.getRoutingNodes().node("node1").iterator().next().shardId().id(), "node1", "node2")), false, false);
assertEquals(reroute.explanations().explanations().size(), 1);
assertEquals(reroute.explanations().explanations().get(0).decisions().type(), Decision.Type.YES);
routingTable = reroute.routingTable();
logger.info("start one more node, first non-primary should start being allocated");
clusterState = ClusterState.builder(clusterState).nodes(DiscoveryNodes.builder(clusterState.nodes()).put(newNode("node2"))).build();
routingTable = strategy.reroute(clusterState, "reroute").routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
assertEquals(clusterState.getRoutingNodes().getIncomingRecoveries("node1"), 0);
assertEquals(clusterState.getRoutingNodes().getIncomingRecoveries("node2"), 1);
assertEquals(clusterState.getRoutingNodes().getIncomingRecoveries("node3"), 0);
assertEquals(clusterState.getRoutingNodes().getOutgoingRecoveries("node1"), 1);
assertEquals(clusterState.getRoutingNodes().getOutgoingRecoveries("node2"), 0);
assertEquals(clusterState.getRoutingNodes().getOutgoingRecoveries("node3"), 0);
// outgoing throttles
reroute = strategy.reroute(clusterState, new AllocationCommands(new MoveAllocationCommand("test", clusterState.getRoutingNodes().node("node3").iterator().next().shardId().id(), "node3", "node1")), true, false);
assertEquals(reroute.explanations().explanations().size(), 1);
assertEquals(reroute.explanations().explanations().get(0).decisions().type(), Decision.Type.THROTTLE);
assertEquals(clusterState.getRoutingNodes().getIncomingRecoveries("node1"), 0);
assertEquals(clusterState.getRoutingNodes().getIncomingRecoveries("node2"), 1);
assertEquals(clusterState.getRoutingNodes().getIncomingRecoveries("node3"), 0);
assertThat(routingTable.shardsWithState(STARTED).size(), equalTo(1));
assertThat(routingTable.shardsWithState(INITIALIZING).size(), equalTo(1));
assertThat(routingTable.shardsWithState(UNASSIGNED).size(), equalTo(1));
assertEquals(clusterState.getRoutingNodes().getOutgoingRecoveries("node1"), 1);
assertEquals(clusterState.getRoutingNodes().getOutgoingRecoveries("node2"), 0);
assertEquals(clusterState.getRoutingNodes().getOutgoingRecoveries("node3"), 0);
logger.info("start initializing non-primary");
routingTable = strategy.applyStartedShards(clusterState, routingTable.shardsWithState(INITIALIZING)).routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
assertThat(routingTable.shardsWithState(STARTED).size(), equalTo(2));
assertThat(routingTable.shardsWithState(INITIALIZING).size(), equalTo(0));
assertThat(routingTable.shardsWithState(UNASSIGNED).size(), equalTo(1));
assertEquals(clusterState.getRoutingNodes().getOutgoingRecoveries("node1"), 0);
logger.info("start one more node, initializing second non-primary");
clusterState = ClusterState.builder(clusterState).nodes(DiscoveryNodes.builder(clusterState.nodes()).put(newNode("node3"))).build();
routingTable = strategy.reroute(clusterState, "reroute").routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
assertThat(routingTable.shardsWithState(STARTED).size(), equalTo(2));
assertThat(routingTable.shardsWithState(INITIALIZING).size(), equalTo(1));
assertThat(routingTable.shardsWithState(RELOCATING).size(), equalTo(1));
assertThat(routingTable.shardsWithState(UNASSIGNED).size(), equalTo(0));
assertEquals(clusterState.getRoutingNodes().getOutgoingRecoveries("node1"), 1);
// incoming throttles
reroute = strategy.reroute(clusterState, new AllocationCommands(new MoveAllocationCommand("test", clusterState.getRoutingNodes().node("node3").iterator().next().shardId().id(), "node3", "node2")), true, false);
logger.info("start one more node");
clusterState = ClusterState.builder(clusterState).nodes(DiscoveryNodes.builder(clusterState.nodes()).put(newNode("node4"))).build();
routingTable = strategy.reroute(clusterState, "reroute").routingTable();
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
assertEquals(clusterState.getRoutingNodes().getOutgoingRecoveries("node1"), 1);
logger.info("move started non-primary to new node");
RoutingAllocation.Result reroute = strategy.reroute(clusterState, new AllocationCommands(
new MoveAllocationCommand("test", 0, "node2", "node4")), true, false);
assertEquals(reroute.explanations().explanations().size(), 1);
assertEquals(reroute.explanations().explanations().get(0).decisions().type(), Decision.Type.THROTTLE);
// even though it is throttled, move command still forces allocation
assertEquals(clusterState.getRoutingNodes().getIncomingRecoveries("node1"), 0);
assertEquals(clusterState.getRoutingNodes().getIncomingRecoveries("node2"), 1);
assertEquals(clusterState.getRoutingNodes().getIncomingRecoveries("node3"), 0);
assertEquals(clusterState.getRoutingNodes().getOutgoingRecoveries("node1"), 1);
assertEquals(clusterState.getRoutingNodes().getOutgoingRecoveries("node2"), 0);
assertEquals(clusterState.getRoutingNodes().getOutgoingRecoveries("node3"), 0);
assertThat(routingTable.shardsWithState(STARTED).size(), equalTo(2));
assertThat(routingTable.shardsWithState(INITIALIZING).size(), equalTo(1));
clusterState = ClusterState.builder(clusterState).routingResult(reroute).build();
routingTable = clusterState.routingTable();
assertThat(routingTable.shardsWithState(STARTED).size(), equalTo(1));
assertThat(routingTable.shardsWithState(RELOCATING).size(), equalTo(1));
assertThat(routingTable.shardsWithState(INITIALIZING).size(), equalTo(2));
assertThat(routingTable.shardsWithState(UNASSIGNED).size(), equalTo(0));
assertEquals(clusterState.getRoutingNodes().getOutgoingRecoveries("node1"), 2);
assertEquals(clusterState.getRoutingNodes().getOutgoingRecoveries("node2"), 0);
}
private RoutingTable createRecoveryRoutingTable(IndexMetaData indexMetaData) {
RoutingTable.Builder routingTableBuilder = RoutingTable.builder();
switch (randomInt(5)) {
case 0: routingTableBuilder.addAsRecovery(indexMetaData); break;
case 1: routingTableBuilder.addAsFromCloseToOpen(indexMetaData); break;
case 2: routingTableBuilder.addAsFromDangling(indexMetaData); break;
case 3: routingTableBuilder.addAsNewRestore(indexMetaData,
new RestoreSource(new Snapshot("repo", new SnapshotId("snap", "randomId")), Version.CURRENT,
indexMetaData.getIndex().getName()), new IntHashSet()); break;
case 4: routingTableBuilder.addAsRestore(indexMetaData,
new RestoreSource(new Snapshot("repo", new SnapshotId("snap", "randomId")), Version.CURRENT,
indexMetaData.getIndex().getName())); break;
case 5: routingTableBuilder.addAsNew(indexMetaData); break;
default: throw new IndexOutOfBoundsException();
}
return routingTableBuilder.build();
}
}

View File

@ -42,6 +42,7 @@ import org.elasticsearch.common.unit.TimeValue;
import org.elasticsearch.test.ESTestCase;
import org.elasticsearch.test.MockLogAppender;
import org.elasticsearch.test.junit.annotations.TestLogging;
import org.elasticsearch.threadpool.TestThreadPool;
import org.elasticsearch.threadpool.ThreadPool;
import org.junit.After;
import org.junit.AfterClass;
@ -76,7 +77,7 @@ public class ClusterServiceTests extends ESTestCase {
@BeforeClass
public static void createThreadPool() {
threadPool = new ThreadPool(ClusterServiceTests.class.getName());
threadPool = new TestThreadPool(ClusterServiceTests.class.getName());
}
@AfterClass

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