Merge branch 'master' into painless_method_references
This commit is contained in:
commit
950b093fc7
|
@ -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) {
|
||||
|
|
|
@ -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" />
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
|
|
|
@ -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;
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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;
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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>.
|
||||
*/
|
||||
|
|
|
@ -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();
|
||||
}
|
||||
|
|
|
@ -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);
|
||||
|
|
|
@ -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) {
|
||||
|
|
|
@ -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) {
|
||||
|
|
|
@ -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());
|
||||
}
|
||||
|
|
|
@ -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.
|
||||
|
|
|
@ -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());
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
@ -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);
|
||||
|
|
|
@ -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.
|
||||
|
|
|
@ -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());
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
|
|
|
@ -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());
|
||||
|
||||
|
|
|
@ -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());
|
||||
}
|
||||
}
|
||||
}
|
|
@ -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);
|
||||
}
|
||||
}
|
|
@ -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);
|
||||
}
|
|
@ -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);
|
||||
}
|
||||
}
|
|
@ -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);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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
|
||||
*/
|
||||
|
|
|
@ -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;
|
|
@ -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;
|
||||
|
|
|
@ -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.
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
}
|
||||
}
|
|
@ -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);
|
||||
}
|
||||
|
||||
|
|
|
@ -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);
|
||||
|
|
|
@ -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 {
|
||||
|
|
|
@ -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}
|
||||
*/
|
||||
|
|
|
@ -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);
|
||||
|
|
|
@ -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);
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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
|
||||
)));
|
||||
}
|
||||
|
|
|
@ -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,
|
||||
|
|
|
@ -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.
|
|
@ -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;
|
||||
|
|
|
@ -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) {
|
||||
|
|
|
@ -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;}
|
||||
}
|
||||
|
|
|
@ -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;
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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) {
|
||||
|
|
|
@ -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)
|
||||
|
|
|
@ -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();
|
||||
|
||||
|
|
|
@ -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 {
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
}
|
||||
});
|
||||
}
|
||||
}
|
||||
}
|
|
@ -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");
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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);
|
||||
|
|
|
@ -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;
|
||||
|
||||
|
|
|
@ -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}
|
||||
*/
|
||||
|
|
|
@ -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();
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
}
|
||||
});
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -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);
|
||||
|
|
|
@ -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);
|
||||
|
|
|
@ -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();
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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) {
|
||||
|
|
|
@ -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) {
|
||||
|
|
|
@ -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());
|
||||
|
|
|
@ -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()));
|
||||
|
||||
|
|
|
@ -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");
|
||||
|
|
|
@ -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())) {
|
||||
|
|
|
@ -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));
|
||||
|
|
|
@ -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();
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
}
|
|
@ -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;
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
}
|
|
@ -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;
|
||||
}
|
||||
}
|
||||
|
||||
}
|
|
@ -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;
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
@ -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();
|
||||
}
|
||||
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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)
|
||||
|
|
|
@ -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();
|
||||
}
|
||||
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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());
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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);
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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();
|
||||
}
|
||||
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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()));
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
}
|
|
@ -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
|
||||
|
|
|
@ -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();
|
||||
|
|
|
@ -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() {
|
||||
|
|
|
@ -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")));
|
||||
}
|
||||
|
||||
}
|
|
@ -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
|
||||
|
|
|
@ -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");
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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);
|
||||
|
|
|
@ -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)));
|
||||
|
|
|
@ -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();
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
@ -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
Loading…
Reference in New Issue