Merge branch 'master' into feature/seq_no
* master: (51 commits) Switch QueryBuilders to new MatchPhraseQueryBuilder Added method to allow creation of new methods on-the-fly. more cleanups Remove cluster name from data path Remove explicit parallel new GC flag rehash the docvalues in DocValuesSliceQuery using BitMixer.mix instead of the naive Long.hashCode. switch FunctionRef over to methodhandles ingest: Move processors from core to ingest-common module. Fix some typos (#18746) Fix ut convert FunctionRef/Def usage to methodhandles. Add the ability to partition a scroll in multiple slices. API: use painless types in FunctionRef Update ingest-node.asciidoc compute functional interface stuff in Definition Use method name in bootstrap check might fork test Make checkstyle happy (add Lookup import, line length) Don't hide LambdaConversionException and behave like real javac compiled code when a conversion fails. This works anyways, because fallback is allowed to throw any Throwable Pass through the lookup given by invokedynamic to the LambdaMetaFactory. Without it real lambdas won't work, as their implementations are private to script class checkstyle have your upper L ...
This commit is contained in:
commit
d896886973
|
@ -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" />
|
||||
|
@ -503,7 +501,6 @@
|
|||
<suppress files="core[/\\]src[/\\]main[/\\]java[/\\]org[/\\]elasticsearch[/\\]ingest[/\\]core[/\\]CompoundProcessor.java" checks="LineLength" />
|
||||
<suppress files="core[/\\]src[/\\]main[/\\]java[/\\]org[/\\]elasticsearch[/\\]ingest[/\\]core[/\\]IngestDocument.java" checks="LineLength" />
|
||||
<suppress files="core[/\\]src[/\\]main[/\\]java[/\\]org[/\\]elasticsearch[/\\]ingest[/\\]core[/\\]Pipeline.java" checks="LineLength" />
|
||||
<suppress files="core[/\\]src[/\\]main[/\\]java[/\\]org[/\\]elasticsearch[/\\]ingest[/\\]processor[/\\]ConvertProcessor.java" checks="LineLength" />
|
||||
<suppress files="core[/\\]src[/\\]main[/\\]java[/\\]org[/\\]elasticsearch[/\\]monitor[/\\]jvm[/\\]GcNames.java" checks="LineLength" />
|
||||
<suppress files="core[/\\]src[/\\]main[/\\]java[/\\]org[/\\]elasticsearch[/\\]monitor[/\\]jvm[/\\]HotThreads.java" checks="LineLength" />
|
||||
<suppress files="core[/\\]src[/\\]main[/\\]java[/\\]org[/\\]elasticsearch[/\\]monitor[/\\]jvm[/\\]JvmStats.java" checks="LineLength" />
|
||||
|
@ -1037,14 +1034,11 @@
|
|||
<suppress files="core[/\\]src[/\\]test[/\\]java[/\\]org[/\\]elasticsearch[/\\]indices[/\\]store[/\\]IndicesStoreIntegrationIT.java" checks="LineLength" />
|
||||
<suppress files="core[/\\]src[/\\]test[/\\]java[/\\]org[/\\]elasticsearch[/\\]indices[/\\]store[/\\]IndicesStoreTests.java" checks="LineLength" />
|
||||
<suppress files="core[/\\]src[/\\]test[/\\]java[/\\]org[/\\]elasticsearch[/\\]indices[/\\]template[/\\]SimpleIndexTemplateIT.java" checks="LineLength" />
|
||||
<suppress files="core[/\\]src[/\\]test[/\\]java[/\\]org[/\\]elasticsearch[/\\]ingest[/\\]processor[/\\]AbstractStringProcessorTestCase.java" checks="LineLength" />
|
||||
<suppress files="core[/\\]src[/\\]test[/\\]java[/\\]org[/\\]elasticsearch[/\\]ingest[/\\]processor[/\\]AppendProcessorTests.java" checks="LineLength" />
|
||||
<suppress files="core[/\\]src[/\\]test[/\\]java[/\\]org[/\\]elasticsearch[/\\]ingest[/\\]processor[/\\]DateFormatTests.java" checks="LineLength" />
|
||||
<suppress files="core[/\\]src[/\\]test[/\\]java[/\\]org[/\\]elasticsearch[/\\]ingest[/\\]processor[/\\]DateProcessorTests.java" checks="LineLength" />
|
||||
<suppress files="core[/\\]src[/\\]test[/\\]java[/\\]org[/\\]elasticsearch[/\\]ingest[/\\]processor[/\\]GsubProcessorTests.java" checks="LineLength" />
|
||||
<suppress files="core[/\\]src[/\\]test[/\\]java[/\\]org[/\\]elasticsearch[/\\]ingest[/\\]processor[/\\]RenameProcessorTests.java" checks="LineLength" />
|
||||
<suppress files="core[/\\]src[/\\]test[/\\]java[/\\]org[/\\]elasticsearch[/\\]ingest[/\\]processor[/\\]SetProcessorTests.java" checks="LineLength" />
|
||||
<suppress files="core[/\\]src[/\\]test[/\\]java[/\\]org[/\\]elasticsearch[/\\]ingest[/\\]processor[/\\]SplitProcessorTests.java" checks="LineLength" />
|
||||
<suppress files="core[/\\]src[/\\]test[/\\]java[/\\]org[/\\]elasticsearch[/\\]ingest[/\\]PipelineExecutionServiceTests.java" checks="LineLength" />
|
||||
<suppress files="core[/\\]src[/\\]test[/\\]java[/\\]org[/\\]elasticsearch[/\\]ingest[/\\]PipelineStoreTests.java" checks="LineLength" />
|
||||
<suppress files="core[/\\]src[/\\]test[/\\]java[/\\]org[/\\]elasticsearch[/\\]ingest[/\\]core[/\\]CompoundProcessorTests.java" checks="LineLength" />
|
||||
<suppress files="core[/\\]src[/\\]test[/\\]java[/\\]org[/\\]elasticsearch[/\\]ingest[/\\]core[/\\]PipelineFactoryTests.java" checks="LineLength" />
|
||||
<suppress files="core[/\\]src[/\\]test[/\\]java[/\\]org[/\\]elasticsearch[/\\]ingest[/\\]core[/\\]ValueSourceTests.java" checks="LineLength" />
|
||||
<suppress files="core[/\\]src[/\\]test[/\\]java[/\\]org[/\\]elasticsearch[/\\]mget[/\\]SimpleMgetIT.java" checks="LineLength" />
|
||||
<suppress files="core[/\\]src[/\\]test[/\\]java[/\\]org[/\\]elasticsearch[/\\]monitor[/\\]jvm[/\\]JvmGcMonitorServiceSettingsTests.java" checks="LineLength" />
|
||||
<suppress files="core[/\\]src[/\\]test[/\\]java[/\\]org[/\\]elasticsearch[/\\]monitor[/\\]os[/\\]OsProbeTests.java" checks="LineLength" />
|
||||
|
@ -1156,9 +1150,6 @@
|
|||
<suppress files="core[/\\]src[/\\]test[/\\]java[/\\]org[/\\]elasticsearch[/\\]update[/\\]UpdateIT.java" checks="LineLength" />
|
||||
<suppress files="core[/\\]src[/\\]test[/\\]java[/\\]org[/\\]elasticsearch[/\\]validate[/\\]SimpleValidateQueryIT.java" checks="LineLength" />
|
||||
<suppress files="core[/\\]src[/\\]test[/\\]java[/\\]org[/\\]elasticsearch[/\\]versioning[/\\]SimpleVersioningIT.java" checks="LineLength" />
|
||||
<suppress files="modules[/\\]ingest-grok[/\\]src[/\\]main[/\\]java[/\\]org[/\\]elasticsearch[/\\]ingest[/\\]grok[/\\]Grok.java" checks="LineLength" />
|
||||
<suppress files="modules[/\\]ingest-grok[/\\]src[/\\]test[/\\]java[/\\]org[/\\]elasticsearch[/\\]ingest[/\\]grok[/\\]GrokProcessorTests.java" checks="LineLength" />
|
||||
<suppress files="modules[/\\]ingest-grok[/\\]src[/\\]test[/\\]java[/\\]org[/\\]elasticsearch[/\\]ingest[/\\]grok[/\\]GrokTests.java" checks="LineLength" />
|
||||
<suppress files="modules[/\\]lang-expression[/\\]src[/\\]main[/\\]java[/\\]org[/\\]elasticsearch[/\\]script[/\\]expression[/\\]ExpressionPlugin.java" checks="LineLength" />
|
||||
<suppress files="modules[/\\]lang-expression[/\\]src[/\\]main[/\\]java[/\\]org[/\\]elasticsearch[/\\]script[/\\]expression[/\\]ExpressionScriptEngineService.java" checks="LineLength" />
|
||||
<suppress files="modules[/\\]lang-expression[/\\]src[/\\]main[/\\]java[/\\]org[/\\]elasticsearch[/\\]script[/\\]expression[/\\]ExpressionSearchScript.java" checks="LineLength" />
|
||||
|
|
|
@ -92,3 +92,8 @@ org.joda.time.DateTime#<init>(int, int, int, int, int, int)
|
|||
org.joda.time.DateTime#<init>(int, int, int, int, int, int, int)
|
||||
org.joda.time.DateTime#now()
|
||||
org.joda.time.DateTimeZone#getDefault()
|
||||
|
||||
@defaultMessage Don't use MethodHandles in slow ways, except in tests.
|
||||
java.lang.invoke.MethodHandle#invoke(java.lang.Object[])
|
||||
java.lang.invoke.MethodHandle#invokeWithArguments(java.lang.Object[])
|
||||
java.lang.invoke.MethodHandle#invokeWithArguments(java.util.List)
|
||||
|
|
|
@ -18,10 +18,15 @@
|
|||
*/
|
||||
package org.elasticsearch.action;
|
||||
|
||||
import org.elasticsearch.action.support.WriteRequest;
|
||||
import org.elasticsearch.action.support.WriteRequest.RefreshPolicy;
|
||||
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.common.xcontent.StatusToXContent;
|
||||
import org.elasticsearch.common.xcontent.XContentBuilder;
|
||||
import org.elasticsearch.index.IndexSettings;
|
||||
import org.elasticsearch.index.seqno.SequenceNumbersService;
|
||||
import org.elasticsearch.index.shard.ShardId;
|
||||
import org.elasticsearch.rest.RestStatus;
|
||||
|
@ -31,13 +36,14 @@ 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 long seqNo;
|
||||
private boolean forcedRefresh;
|
||||
|
||||
public DocWriteResponse(ShardId shardId, String type, String id, long seqNo, long version) {
|
||||
this.shardId = shardId;
|
||||
|
@ -94,6 +100,20 @@ public abstract class DocWriteResponse extends ReplicationResponse implements St
|
|||
return seqNo;
|
||||
}
|
||||
|
||||
/**
|
||||
* 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();
|
||||
|
@ -107,6 +127,7 @@ public abstract class DocWriteResponse extends ReplicationResponse implements St
|
|||
id = in.readString();
|
||||
version = in.readZLong();
|
||||
seqNo = in.readZLong();
|
||||
forcedRefresh = in.readBoolean();
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -117,6 +138,7 @@ public abstract class DocWriteResponse extends ReplicationResponse implements St
|
|||
out.writeString(id);
|
||||
out.writeZLong(version);
|
||||
out.writeZLong(seqNo);
|
||||
out.writeBoolean(forcedRefresh);
|
||||
}
|
||||
|
||||
static final class Fields {
|
||||
|
@ -132,9 +154,10 @@ public abstract class DocWriteResponse extends ReplicationResponse implements St
|
|||
public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException {
|
||||
ReplicationResponse.ShardInfo shardInfo = getShardInfo();
|
||||
builder.field(Fields._INDEX, shardId.getIndexName())
|
||||
.field(Fields._TYPE, type)
|
||||
.field(Fields._ID, id)
|
||||
.field(Fields._VERSION, version);
|
||||
.field(Fields._TYPE, type)
|
||||
.field(Fields._ID, id)
|
||||
.field(Fields._VERSION, version)
|
||||
.field("forced_refresh", forcedRefresh);
|
||||
shardInfo.toXContent(builder, params);
|
||||
//nocommit: i'm not sure we want to expose it in the api but it will be handy for debugging while we work...
|
||||
builder.field(Fields._SHARD_ID, shardId.id());
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -94,7 +94,7 @@ public class ShrinkRequest extends AcknowledgedRequest<ShrinkRequest> implements
|
|||
/**
|
||||
* Returns the {@link CreateIndexRequest} for the shrink index
|
||||
*/
|
||||
public CreateIndexRequest getShrinkIndexReqeust() {
|
||||
public CreateIndexRequest getShrinkIndexRequest() {
|
||||
return shrinkIndexRequest;
|
||||
}
|
||||
|
||||
|
|
|
@ -41,7 +41,7 @@ public class ShrinkRequestBuilder extends AcknowledgedRequestBuilder<ShrinkReque
|
|||
}
|
||||
|
||||
public ShrinkRequestBuilder setSettings(Settings settings) {
|
||||
this.request.getShrinkIndexReqeust().settings(settings);
|
||||
this.request.getShrinkIndexRequest().settings(settings);
|
||||
return this;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -23,6 +23,7 @@ import org.apache.lucene.index.IndexWriter;
|
|||
import org.elasticsearch.action.ActionListener;
|
||||
import org.elasticsearch.action.admin.indices.create.CreateIndexClusterStateUpdateRequest;
|
||||
import org.elasticsearch.action.admin.indices.create.CreateIndexRequest;
|
||||
import org.elasticsearch.action.admin.indices.stats.IndexShardStats;
|
||||
import org.elasticsearch.action.admin.indices.stats.IndicesStatsResponse;
|
||||
import org.elasticsearch.action.support.ActionFilters;
|
||||
import org.elasticsearch.action.support.master.TransportMasterNodeAction;
|
||||
|
@ -34,27 +35,17 @@ import org.elasticsearch.cluster.block.ClusterBlockLevel;
|
|||
import org.elasticsearch.cluster.metadata.IndexMetaData;
|
||||
import org.elasticsearch.cluster.metadata.IndexNameExpressionResolver;
|
||||
import org.elasticsearch.cluster.metadata.MetaDataCreateIndexService;
|
||||
import org.elasticsearch.cluster.routing.IndexRoutingTable;
|
||||
import org.elasticsearch.cluster.routing.ShardRouting;
|
||||
import org.elasticsearch.cluster.routing.ShardRoutingState;
|
||||
import org.elasticsearch.cluster.routing.allocation.AllocationService;
|
||||
import org.elasticsearch.cluster.service.ClusterService;
|
||||
import org.elasticsearch.common.Strings;
|
||||
import org.elasticsearch.common.inject.Inject;
|
||||
import org.elasticsearch.common.settings.Settings;
|
||||
import org.elasticsearch.index.IndexNotFoundException;
|
||||
import org.elasticsearch.index.shard.DocsStats;
|
||||
import org.elasticsearch.index.shard.ShardId;
|
||||
import org.elasticsearch.indices.IndexAlreadyExistsException;
|
||||
import org.elasticsearch.threadpool.ThreadPool;
|
||||
import org.elasticsearch.transport.TransportService;
|
||||
|
||||
import java.util.ArrayList;
|
||||
import java.util.HashMap;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Set;
|
||||
import java.util.concurrent.atomic.AtomicInteger;
|
||||
import java.util.function.Predicate;
|
||||
import java.util.function.IntFunction;
|
||||
|
||||
/**
|
||||
* Main class to initiate shrinking an index into a new index with a single shard
|
||||
|
@ -87,7 +78,7 @@ public class TransportShrinkAction extends TransportMasterNodeAction<ShrinkReque
|
|||
|
||||
@Override
|
||||
protected ClusterBlockException checkBlock(ShrinkRequest request, ClusterState state) {
|
||||
return state.blocks().indexBlockedException(ClusterBlockLevel.METADATA_WRITE, request.getShrinkIndexReqeust().index());
|
||||
return state.blocks().indexBlockedException(ClusterBlockLevel.METADATA_WRITE, request.getShrinkIndexRequest().index());
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -98,7 +89,10 @@ public class TransportShrinkAction extends TransportMasterNodeAction<ShrinkReque
|
|||
@Override
|
||||
public void onResponse(IndicesStatsResponse indicesStatsResponse) {
|
||||
CreateIndexClusterStateUpdateRequest updateRequest = prepareCreateIndexRequest(shrinkRequest, state,
|
||||
indicesStatsResponse.getTotal().getDocs(), indexNameExpressionResolver);
|
||||
(i) -> {
|
||||
IndexShardStats shard = indicesStatsResponse.getIndex(sourceIndex).getIndexShards().get(i);
|
||||
return shard == null ? null : shard.getPrimary().getDocs();
|
||||
}, indexNameExpressionResolver);
|
||||
createIndexService.createIndex(updateRequest, new ActionListener<ClusterStateUpdateResponse>() {
|
||||
@Override
|
||||
public void onResponse(ClusterStateUpdateResponse response) {
|
||||
|
@ -127,24 +121,36 @@ public class TransportShrinkAction extends TransportMasterNodeAction<ShrinkReque
|
|||
|
||||
// static for unittesting this method
|
||||
static CreateIndexClusterStateUpdateRequest prepareCreateIndexRequest(final ShrinkRequest shrinkReqeust, final ClusterState state
|
||||
, final DocsStats docsStats, IndexNameExpressionResolver indexNameExpressionResolver) {
|
||||
, final IntFunction<DocsStats> perShardDocStats, IndexNameExpressionResolver indexNameExpressionResolver) {
|
||||
final String sourceIndex = indexNameExpressionResolver.resolveDateMathExpression(shrinkReqeust.getSourceIndex());
|
||||
final CreateIndexRequest targetIndex = shrinkReqeust.getShrinkIndexReqeust();
|
||||
final CreateIndexRequest targetIndex = shrinkReqeust.getShrinkIndexRequest();
|
||||
final String targetIndexName = indexNameExpressionResolver.resolveDateMathExpression(targetIndex.index());
|
||||
final IndexMetaData metaData = state.metaData().index(sourceIndex);
|
||||
final Settings targetIndexSettings = Settings.builder().put(targetIndex.settings())
|
||||
.normalizePrefix(IndexMetaData.INDEX_SETTING_PREFIX).build();
|
||||
long count = docsStats.getCount();
|
||||
if (count >= IndexWriter.MAX_DOCS) {
|
||||
throw new IllegalStateException("Can't merge index with more than [" + IndexWriter.MAX_DOCS
|
||||
+ "] docs - too many documents");
|
||||
int numShards = 1;
|
||||
if (IndexMetaData.INDEX_NUMBER_OF_SHARDS_SETTING.exists(targetIndexSettings)) {
|
||||
numShards = IndexMetaData.INDEX_NUMBER_OF_SHARDS_SETTING.get(targetIndexSettings);
|
||||
}
|
||||
for (int i = 0; i < numShards; i++) {
|
||||
Set<ShardId> shardIds = IndexMetaData.selectShrinkShards(i, metaData, numShards);
|
||||
long count = 0;
|
||||
for (ShardId id : shardIds) {
|
||||
DocsStats docsStats = perShardDocStats.apply(id.id());
|
||||
if (docsStats != null) {
|
||||
count += docsStats.getCount();
|
||||
}
|
||||
if (count > IndexWriter.MAX_DOCS) {
|
||||
throw new IllegalStateException("Can't merge index with more than [" + IndexWriter.MAX_DOCS
|
||||
+ "] docs - too many documents in shards " + shardIds);
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
targetIndex.cause("shrink_index");
|
||||
targetIndex.settings(Settings.builder()
|
||||
.put(targetIndexSettings)
|
||||
// we can only shrink to 1 index so far!
|
||||
.put("index.number_of_shards", 1)
|
||||
);
|
||||
Settings.Builder settingsBuilder = Settings.builder().put(targetIndexSettings);
|
||||
settingsBuilder.put("index.number_of_shards", numShards);
|
||||
targetIndex.settings(settingsBuilder);
|
||||
|
||||
return new CreateIndexClusterStateUpdateRequest(targetIndex,
|
||||
"shrink_index", targetIndexName, true)
|
||||
|
|
|
@ -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.ReplicationResponse.ShardInfo;
|
||||
import org.elasticsearch.action.support.replication.TransportWriteAction;
|
||||
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) {
|
||||
// nocommit: since we now have RetryOnPrimaryException, retrying doesn't always mean the shard is closed.
|
||||
|
@ -199,8 +199,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) {
|
||||
// nocommit: since we now have RetryOnPrimaryException, retrying doesn't always mean the shard is closed.
|
||||
|
@ -241,16 +241,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.getSeqNo(), indexResponse.getVersion(), indexResponse.isCreated());
|
||||
if (updateRequest.fields() != null && updateRequest.fields().length > 0) {
|
||||
Tuple<XContentType, Map<String, Object>> sourceAndContent = XContentHelper.convertToMap(indexSourceAsBytes, true);
|
||||
|
@ -260,8 +261,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.getSeqNo(), response.getVersion(), false);
|
||||
updateResponse.setGetResult(updateHelper.extractGetResult(updateRequest, request.index(), response.getVersion(), updateResult.result.updatedSourceAsMap(), updateResult.result.updateSourceContentType(), null));
|
||||
|
@ -332,11 +334,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) {
|
||||
|
@ -437,12 +442,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];
|
||||
|
@ -478,8 +479,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) {
|
||||
|
@ -504,4 +504,4 @@ public class TransportShardBulkAction extends TransportReplicationAction<BulkSha
|
|||
assert current == null || current.compareTo(next) < 0 : "translog locations are not increasing";
|
||||
return next;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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) {
|
||||
|
@ -135,9 +136,8 @@ public class TransportDeleteAction extends TransportReplicationAction<DeleteRequ
|
|||
request.seqNo(delete.seqNo());
|
||||
|
||||
assert request.versionType().validateVersionForWrites(request.version());
|
||||
return new WriteResult<>(
|
||||
new DeleteResponse(indexShard.shardId(), request.type(), request.id(), delete.seqNo(), delete.version(), delete.found()),
|
||||
delete.getTranslogLocation());
|
||||
DeleteResponse response = new DeleteResponse(indexShard.shardId(), request.type(), request.id(), delete.seqNo(), delete.version(), delete.found());
|
||||
return new WriteResult<>(response, delete.getTranslogLocation());
|
||||
}
|
||||
|
||||
public static Engine.Delete executeDeleteRequestOnReplica(DeleteRequest request, IndexShard indexShard) {
|
||||
|
@ -145,13 +145,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();
|
||||
|
@ -215,7 +197,8 @@ public class TransportIndexAction extends TransportReplicationAction<IndexReques
|
|||
|
||||
assert request.versionType().validateVersionForWrites(request.version());
|
||||
|
||||
return new WriteResult<>(new IndexResponse(shardId, request.type(), request.id(), request.seqNo(), request.version(), created), operation.getTranslogLocation());
|
||||
IndexResponse response = new IndexResponse(shardId, request.type(), request.id(), request.seqNo(), 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());
|
||||
|
||||
|
|
|
@ -29,7 +29,7 @@ import org.elasticsearch.threadpool.ThreadPool;
|
|||
import java.util.ArrayList;
|
||||
import java.util.List;
|
||||
|
||||
import static org.elasticsearch.ingest.processor.TrackingResultProcessor.decorate;
|
||||
import static org.elasticsearch.action.ingest.TrackingResultProcessor.decorate;
|
||||
|
||||
class SimulateExecutionService {
|
||||
|
||||
|
|
|
@ -17,9 +17,8 @@
|
|||
* under the License.
|
||||
*/
|
||||
|
||||
package org.elasticsearch.ingest.processor;
|
||||
package org.elasticsearch.action.ingest;
|
||||
|
||||
import org.elasticsearch.action.ingest.SimulateProcessorResult;
|
||||
import org.elasticsearch.ingest.core.CompoundProcessor;
|
||||
import org.elasticsearch.ingest.core.IngestDocument;
|
||||
import org.elasticsearch.ingest.core.Processor;
|
|
@ -30,6 +30,7 @@ import org.elasticsearch.script.Template;
|
|||
import org.elasticsearch.search.Scroll;
|
||||
import org.elasticsearch.search.aggregations.AggregationBuilder;
|
||||
import org.elasticsearch.search.aggregations.PipelineAggregatorBuilder;
|
||||
import org.elasticsearch.search.slice.SliceBuilder;
|
||||
import org.elasticsearch.search.builder.SearchSourceBuilder;
|
||||
import org.elasticsearch.search.highlight.HighlightBuilder;
|
||||
import org.elasticsearch.search.rescore.RescoreBuilder;
|
||||
|
@ -352,6 +353,11 @@ public class SearchRequestBuilder extends ActionRequestBuilder<SearchRequest, Se
|
|||
return this;
|
||||
}
|
||||
|
||||
public SearchRequestBuilder slice(SliceBuilder builder) {
|
||||
sourceBuilder().slice(builder);
|
||||
return this;
|
||||
}
|
||||
|
||||
/**
|
||||
* Applies when sorting, and controls if scores will be tracked as well. Defaults to
|
||||
* <tt>false</tt>.
|
||||
|
|
|
@ -167,7 +167,7 @@ public class SearchResponse extends ActionResponse implements StatusToXContent {
|
|||
* If profiling was enabled, this returns an object containing the profile results from
|
||||
* each shard. If profiling was not enabled, this will return null
|
||||
*
|
||||
* @return The profile results or null
|
||||
* @return The profile results or an empty map
|
||||
*/
|
||||
public @Nullable Map<String, List<ProfileShardResult>> getProfileResults() {
|
||||
return internalResponse.profile();
|
||||
|
|
|
@ -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;
|
||||
|
@ -46,28 +44,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) {
|
||||
|
@ -75,7 +86,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,30 +96,28 @@ public class ReplicationOperation<Request extends ReplicationRequest<Request>, R
|
|||
void execute() throws Exception {
|
||||
final String writeConsistencyFailure = checkWriteConsistency ? checkWriteConsistency() : null;
|
||||
final ShardRouting primaryRouting = primary.routingEntry();
|
||||
final ShardId shardId = primaryRouting.shardId();
|
||||
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
|
||||
primaryResult = primary.perform(request);
|
||||
primary.updateLocalCheckpointForShard(primaryRouting.allocationId().getId(), primary.localCheckpoint());
|
||||
finalResponse = primaryResponse.v1();
|
||||
ReplicaRequest replicaRequest = primaryResponse.v2();
|
||||
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 String localNodeId = primaryRouting.currentNodeId();
|
||||
final List<ShardRouting> shards = getShards(primaryId, clusterStateSupplier.get());
|
||||
final String localNodeId = primary.routingEntry().currentNodeId();
|
||||
for (final ShardRouting shard : shards) {
|
||||
if (executeOnReplicas == false || shard.unassigned()) {
|
||||
if (shard.primary() == false) {
|
||||
|
@ -126,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(); // mark primary as successful
|
||||
decPendingAndFinishIfNeeded();
|
||||
}
|
||||
|
||||
private void performOnReplica(final ShardRouting shard, final ReplicaRequest replicaRequest) {
|
||||
|
@ -243,19 +252,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);
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -286,22 +295,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;
|
||||
|
||||
|
||||
/**
|
||||
|
@ -322,19 +340,21 @@ 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<ReplicaResponse> 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
|
||||
|
@ -371,4 +391,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 {
|
||||
|
@ -66,7 +69,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;
|
||||
|
|
|
@ -23,7 +23,6 @@ import org.elasticsearch.ElasticsearchException;
|
|||
import org.elasticsearch.action.ActionListener;
|
||||
import org.elasticsearch.action.ActionListenerResponseHandler;
|
||||
import org.elasticsearch.action.ActionResponse;
|
||||
import org.elasticsearch.action.ReplicationResponse;
|
||||
import org.elasticsearch.action.UnavailableShardsException;
|
||||
import org.elasticsearch.action.WriteConsistencyLevel;
|
||||
import org.elasticsearch.action.support.ActionFilters;
|
||||
|
@ -42,7 +41,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.io.stream.StreamOutput;
|
||||
import org.elasticsearch.common.lease.Releasable;
|
||||
|
@ -55,7 +53,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;
|
||||
|
@ -68,6 +65,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;
|
||||
|
@ -83,9 +81,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;
|
||||
|
@ -151,17 +151,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 #getReplicaShardReference(ShardId, long)} (ShardId, long)}.
|
||||
*/
|
||||
protected abstract void shardOperationOnReplica(ReplicaRequest shardRequest);
|
||||
protected abstract ReplicaResult shardOperationOnReplica(ReplicaRequest shardRequest);
|
||||
|
||||
/**
|
||||
* True if write consistency should be checked for an implementation
|
||||
|
@ -200,26 +200,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 {
|
||||
|
@ -291,7 +271,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 {
|
||||
|
@ -301,9 +291,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
|
||||
);
|
||||
|
@ -341,6 +331,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(Empty.INSTANCE);
|
||||
}
|
||||
}
|
||||
|
||||
class ReplicaOperationTransportHandler implements TransportRequestHandler<ReplicaRequest> {
|
||||
@Override
|
||||
public void messageReceived(final ReplicaRequest request, final TransportChannel channel) throws Exception {
|
||||
|
@ -429,16 +454,42 @@ public abstract class TransportReplicationAction<Request extends ReplicationRequ
|
|||
setPhase(task, "replica");
|
||||
final ReplicaResponse response;
|
||||
assert request.shardId() != null : "request shardId must be set";
|
||||
ReplicaResult result;
|
||||
try (ShardReference replica = getReplicaShardReference(request.shardId(), request.primaryTerm())) {
|
||||
shardOperationOnReplica(request);
|
||||
result = shardOperationOnReplica(request);
|
||||
response = new ReplicaResponse(replica.routingEntry().allocationId().getId(), replica.getLocalCheckpoint());
|
||||
}
|
||||
result.respond(new ResponseListener(response));
|
||||
}
|
||||
|
||||
/**
|
||||
* Listens for the response on the replica and sends the response back to the primary.
|
||||
*/
|
||||
private class ResponseListener implements ActionListener<TransportResponse.Empty> {
|
||||
private final ReplicaResponse replicaResponse;
|
||||
|
||||
public ResponseListener(ReplicaResponse replicaResponse) {
|
||||
this.replicaResponse = replicaResponse;
|
||||
}
|
||||
|
||||
@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(replicaResponse);
|
||||
} catch (Exception e) {
|
||||
onFailure(e);
|
||||
}
|
||||
}
|
||||
setPhase(task, "finished");
|
||||
channel.sendResponse(response);
|
||||
|
||||
@Override
|
||||
public void onFailure(Throwable e) {
|
||||
responseWithFailure(e);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -751,7 +802,7 @@ public abstract class TransportReplicationAction<Request extends ReplicationRequ
|
|||
|
||||
}
|
||||
|
||||
class PrimaryShardReference extends ShardReference implements ReplicationOperation.Primary<Request, ReplicaRequest, Response> {
|
||||
class PrimaryShardReference extends ShardReference implements ReplicationOperation.Primary<Request, ReplicaRequest, PrimaryResult> {
|
||||
|
||||
PrimaryShardReference(IndexShard indexShard, Releasable operationLock) {
|
||||
super(indexShard, operationLock);
|
||||
|
@ -771,9 +822,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;
|
||||
}
|
||||
|
||||
|
@ -786,6 +837,7 @@ public abstract class TransportReplicationAction<Request extends ReplicationRequ
|
|||
public long localCheckpoint() {
|
||||
return indexShard.getLocalCheckpoint();
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
|
||||
|
@ -869,20 +921,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;
|
||||
|
@ -188,6 +189,7 @@ public class TransportUpdateAction extends TransportInstanceSingleOperationActio
|
|||
} else {
|
||||
update.setGetResult(null);
|
||||
}
|
||||
update.setForcedRefresh(response.forcedRefresh());
|
||||
listener.onResponse(update);
|
||||
}
|
||||
|
||||
|
@ -221,6 +223,7 @@ public class TransportUpdateAction extends TransportInstanceSingleOperationActio
|
|||
UpdateResponse update = new UpdateResponse(response.getShardInfo(), response.getShardId(), response.getType(), response.getId(),
|
||||
response.getSeqNo(), 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);
|
||||
}
|
||||
|
||||
|
@ -243,12 +246,14 @@ 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.getSeqNo(), 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}
|
||||
*/
|
||||
|
|
|
@ -106,7 +106,7 @@ final class BootstrapCheck {
|
|||
|
||||
for (final Check check : checks) {
|
||||
if (check.check()) {
|
||||
if (!enforceLimits || (check.isSystemCheck() && ignoreSystemChecks)) {
|
||||
if ((!enforceLimits || (check.isSystemCheck() && ignoreSystemChecks)) && !check.alwaysEnforce()) {
|
||||
ignoredErrors.add(check.errorMessage());
|
||||
} else {
|
||||
errors.add(check.errorMessage());
|
||||
|
@ -164,6 +164,8 @@ final class BootstrapCheck {
|
|||
checks.add(new MaxMapCountCheck());
|
||||
}
|
||||
checks.add(new ClientJvmCheck());
|
||||
checks.add(new OnErrorCheck());
|
||||
checks.add(new OnOutOfMemoryErrorCheck());
|
||||
return Collections.unmodifiableList(checks);
|
||||
}
|
||||
|
||||
|
@ -194,6 +196,10 @@ final class BootstrapCheck {
|
|||
*/
|
||||
boolean isSystemCheck();
|
||||
|
||||
default boolean alwaysEnforce() {
|
||||
return false;
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
static class HeapSizeCheck implements BootstrapCheck.Check {
|
||||
|
@ -245,7 +251,6 @@ final class BootstrapCheck {
|
|||
|
||||
}
|
||||
|
||||
// visible for testing
|
||||
static class FileDescriptorCheck implements Check {
|
||||
|
||||
private final int limit;
|
||||
|
@ -288,7 +293,6 @@ final class BootstrapCheck {
|
|||
|
||||
}
|
||||
|
||||
// visible for testing
|
||||
static class MlockallCheck implements Check {
|
||||
|
||||
private final boolean mlockallSet;
|
||||
|
@ -504,4 +508,81 @@ final class BootstrapCheck {
|
|||
|
||||
}
|
||||
|
||||
static abstract class MightForkCheck implements BootstrapCheck.Check {
|
||||
|
||||
@Override
|
||||
public boolean check() {
|
||||
return isSeccompInstalled() && mightFork();
|
||||
}
|
||||
|
||||
// visible for testing
|
||||
boolean isSeccompInstalled() {
|
||||
return Natives.isSeccompInstalled();
|
||||
}
|
||||
|
||||
// visible for testing
|
||||
abstract boolean mightFork();
|
||||
|
||||
@Override
|
||||
public final boolean isSystemCheck() {
|
||||
return false;
|
||||
}
|
||||
|
||||
@Override
|
||||
public final boolean alwaysEnforce() {
|
||||
return true;
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
static class OnErrorCheck extends MightForkCheck {
|
||||
|
||||
@Override
|
||||
boolean mightFork() {
|
||||
final String onError = onError();
|
||||
return onError != null && !onError.equals("");
|
||||
}
|
||||
|
||||
// visible for testing
|
||||
String onError() {
|
||||
return JvmInfo.jvmInfo().onError();
|
||||
}
|
||||
|
||||
@Override
|
||||
public String errorMessage() {
|
||||
return String.format(
|
||||
Locale.ROOT,
|
||||
"OnError [%s] requires forking but is prevented by system call filters ([%s=true]);" +
|
||||
" upgrade to at least Java 8u92 and use ExitOnOutOfMemoryError",
|
||||
onError(),
|
||||
BootstrapSettings.SECCOMP_SETTING.getKey());
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
static class OnOutOfMemoryErrorCheck extends MightForkCheck {
|
||||
|
||||
@Override
|
||||
boolean mightFork() {
|
||||
final String onOutOfMemoryError = onOutOfMemoryError();
|
||||
return onOutOfMemoryError != null && !onOutOfMemoryError.equals("");
|
||||
}
|
||||
|
||||
// visible for testing
|
||||
String onOutOfMemoryError() {
|
||||
return JvmInfo.jvmInfo().onOutOfMemoryError();
|
||||
}
|
||||
|
||||
@Override
|
||||
public String errorMessage() {
|
||||
return String.format(
|
||||
Locale.ROOT,
|
||||
"OnOutOfMemoryError [%s] requires forking but is prevented by system call filters ([%s=true]);" +
|
||||
" upgrade to at least Java 8u92 and use ExitOnOutOfMemoryError",
|
||||
onOutOfMemoryError(),
|
||||
BootstrapSettings.SECCOMP_SETTING.getKey());
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
@ -20,6 +20,7 @@
|
|||
package org.elasticsearch.bootstrap;
|
||||
|
||||
import org.elasticsearch.SecureSM;
|
||||
import org.elasticsearch.Version;
|
||||
import org.elasticsearch.common.Strings;
|
||||
import org.elasticsearch.common.SuppressForbidden;
|
||||
import org.elasticsearch.common.io.PathUtils;
|
||||
|
@ -256,8 +257,10 @@ final class Security {
|
|||
for (Path path : environment.dataFiles()) {
|
||||
addPath(policy, Environment.PATH_DATA_SETTING.getKey(), path, "read,readlink,write,delete");
|
||||
}
|
||||
// TODO: this should be removed in ES 6.0! We will no longer support data paths with the cluster as a folder
|
||||
assert Version.CURRENT.major < 6 : "cluster name is no longer used in data path";
|
||||
for (Path path : environment.dataWithClusterFiles()) {
|
||||
addPath(policy, Environment.PATH_DATA_SETTING.getKey(), path, "read,readlink,write,delete");
|
||||
addPathIfExists(policy, Environment.PATH_DATA_SETTING.getKey(), path, "read,readlink,write,delete");
|
||||
}
|
||||
for (Path path : environment.repoFiles()) {
|
||||
addPath(policy, Environment.PATH_REPO_SETTING.getKey(), path, "read,readlink,write,delete");
|
||||
|
@ -318,6 +321,27 @@ final class Security {
|
|||
policy.add(new FilePermission(path.toString() + path.getFileSystem().getSeparator() + "-", permissions));
|
||||
}
|
||||
|
||||
/**
|
||||
* Add access to a directory iff it exists already
|
||||
* @param policy current policy to add permissions to
|
||||
* @param configurationName the configuration name associated with the path (for error messages only)
|
||||
* @param path the path itself
|
||||
* @param permissions set of filepermissions to grant to the path
|
||||
*/
|
||||
static void addPathIfExists(Permissions policy, String configurationName, Path path, String permissions) {
|
||||
if (Files.isDirectory(path)) {
|
||||
// add each path twice: once for itself, again for files underneath it
|
||||
policy.add(new FilePermission(path.toString(), permissions));
|
||||
policy.add(new FilePermission(path.toString() + path.getFileSystem().getSeparator() + "-", permissions));
|
||||
try {
|
||||
path.getFileSystem().provider().checkAccess(path.toRealPath(), AccessMode.READ);
|
||||
} catch (IOException e) {
|
||||
throw new IllegalStateException("Unable to access '" + configurationName + "' (" + path + ")", e);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
/**
|
||||
* Ensures configured directory {@code path} exists.
|
||||
* @throws IOException if {@code path} exists, but is not a directory, not accessible, or broken symbolic link.
|
||||
|
|
|
@ -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);
|
||||
|
|
|
@ -30,10 +30,8 @@ import org.elasticsearch.cluster.Diffable;
|
|||
import org.elasticsearch.cluster.DiffableUtils;
|
||||
import org.elasticsearch.cluster.block.ClusterBlock;
|
||||
import org.elasticsearch.cluster.block.ClusterBlockLevel;
|
||||
import org.elasticsearch.cluster.node.DiscoveryNode;
|
||||
import org.elasticsearch.cluster.node.DiscoveryNodeFilters;
|
||||
import org.elasticsearch.cluster.routing.RoutingTable;
|
||||
import org.elasticsearch.cluster.routing.ShardRouting;
|
||||
import org.elasticsearch.cluster.routing.allocation.AllocationService;
|
||||
import org.elasticsearch.common.Nullable;
|
||||
import org.elasticsearch.common.ParseFieldMatcher;
|
||||
|
@ -56,6 +54,7 @@ import org.elasticsearch.common.xcontent.XContentType;
|
|||
import org.elasticsearch.gateway.MetaDataStateFormat;
|
||||
import org.elasticsearch.index.Index;
|
||||
import org.elasticsearch.index.mapper.MapperService;
|
||||
import org.elasticsearch.index.shard.ShardId;
|
||||
import org.elasticsearch.rest.RestStatus;
|
||||
import org.joda.time.DateTime;
|
||||
import org.joda.time.DateTimeZone;
|
||||
|
@ -226,6 +225,7 @@ public class IndexMetaData implements Diffable<IndexMetaData>, FromXContentBuild
|
|||
|
||||
public static final String KEY_ACTIVE_ALLOCATIONS = "active_allocations";
|
||||
static final String KEY_VERSION = "version";
|
||||
static final String KEY_ROUTING_NUM_SHARDS = "routing_num_shards";
|
||||
static final String KEY_SETTINGS = "settings";
|
||||
static final String KEY_STATE = "state";
|
||||
static final String KEY_MAPPINGS = "mappings";
|
||||
|
@ -233,6 +233,8 @@ public class IndexMetaData implements Diffable<IndexMetaData>, FromXContentBuild
|
|||
public static final String KEY_PRIMARY_TERMS = "primary_terms";
|
||||
|
||||
public static final String INDEX_STATE_FILE_PREFIX = "state-";
|
||||
private final int routingNumShards;
|
||||
private final int routingFactor;
|
||||
|
||||
private final int numberOfShards;
|
||||
private final int numberOfReplicas;
|
||||
|
@ -268,7 +270,8 @@ public class IndexMetaData implements Diffable<IndexMetaData>, FromXContentBuild
|
|||
ImmutableOpenMap<String, MappingMetaData> mappings, ImmutableOpenMap<String, AliasMetaData> aliases,
|
||||
ImmutableOpenMap<String, Custom> customs, ImmutableOpenIntMap<Set<String>> activeAllocationIds,
|
||||
DiscoveryNodeFilters requireFilters, DiscoveryNodeFilters initialRecoveryFilters, DiscoveryNodeFilters includeFilters, DiscoveryNodeFilters excludeFilters,
|
||||
Version indexCreatedVersion, Version indexUpgradedVersion, org.apache.lucene.util.Version minimumCompatibleLuceneVersion) {
|
||||
Version indexCreatedVersion, Version indexUpgradedVersion, org.apache.lucene.util.Version minimumCompatibleLuceneVersion,
|
||||
int routingNumShards) {
|
||||
|
||||
this.index = index;
|
||||
this.version = version;
|
||||
|
@ -290,6 +293,9 @@ public class IndexMetaData implements Diffable<IndexMetaData>, FromXContentBuild
|
|||
this.indexCreatedVersion = indexCreatedVersion;
|
||||
this.indexUpgradedVersion = indexUpgradedVersion;
|
||||
this.minimumCompatibleLuceneVersion = minimumCompatibleLuceneVersion;
|
||||
this.routingNumShards = routingNumShards;
|
||||
this.routingFactor = routingNumShards / numberOfShards;
|
||||
assert numberOfShards * routingFactor == routingNumShards : routingNumShards + " must be a multiple of " + numberOfShards;
|
||||
}
|
||||
|
||||
public Index getIndex() {
|
||||
|
@ -484,7 +490,12 @@ public class IndexMetaData implements Diffable<IndexMetaData>, FromXContentBuild
|
|||
if (!customs.equals(that.customs)) {
|
||||
return false;
|
||||
}
|
||||
|
||||
if (routingNumShards != that.routingNumShards) {
|
||||
return false;
|
||||
}
|
||||
if (routingFactor != that.routingFactor) {
|
||||
return false;
|
||||
}
|
||||
if (Arrays.equals(primaryTerms, that.primaryTerms) == false) {
|
||||
return false;
|
||||
}
|
||||
|
@ -503,6 +514,8 @@ public class IndexMetaData implements Diffable<IndexMetaData>, FromXContentBuild
|
|||
result = 31 * result + settings.hashCode();
|
||||
result = 31 * result + mappings.hashCode();
|
||||
result = 31 * result + customs.hashCode();
|
||||
result = 31 * result + Long.hashCode(routingFactor);
|
||||
result = 31 * result + Long.hashCode(routingNumShards);
|
||||
result = 31 * result + Arrays.hashCode(primaryTerms);
|
||||
result = 31 * result + activeAllocationIds.hashCode();
|
||||
return result;
|
||||
|
@ -533,6 +546,7 @@ public class IndexMetaData implements Diffable<IndexMetaData>, FromXContentBuild
|
|||
private static class IndexMetaDataDiff implements Diff<IndexMetaData> {
|
||||
|
||||
private final String index;
|
||||
private final int routingNumShards;
|
||||
private final long version;
|
||||
private final long[] primaryTerms;
|
||||
private final State state;
|
||||
|
@ -545,6 +559,7 @@ public class IndexMetaData implements Diffable<IndexMetaData>, FromXContentBuild
|
|||
public IndexMetaDataDiff(IndexMetaData before, IndexMetaData after) {
|
||||
index = after.index.getName();
|
||||
version = after.version;
|
||||
routingNumShards = after.routingNumShards;
|
||||
state = after.state;
|
||||
settings = after.settings;
|
||||
primaryTerms = after.primaryTerms;
|
||||
|
@ -557,6 +572,7 @@ public class IndexMetaData implements Diffable<IndexMetaData>, FromXContentBuild
|
|||
|
||||
public IndexMetaDataDiff(StreamInput in) throws IOException {
|
||||
index = in.readString();
|
||||
routingNumShards = in.readInt();
|
||||
version = in.readLong();
|
||||
state = State.fromId(in.readByte());
|
||||
settings = Settings.readSettingsFromStream(in);
|
||||
|
@ -582,6 +598,7 @@ public class IndexMetaData implements Diffable<IndexMetaData>, FromXContentBuild
|
|||
@Override
|
||||
public void writeTo(StreamOutput out) throws IOException {
|
||||
out.writeString(index);
|
||||
out.writeInt(routingNumShards);
|
||||
out.writeLong(version);
|
||||
out.writeByte(state.id);
|
||||
Settings.writeSettingsToStream(settings, out);
|
||||
|
@ -596,6 +613,7 @@ public class IndexMetaData implements Diffable<IndexMetaData>, FromXContentBuild
|
|||
public IndexMetaData apply(IndexMetaData part) {
|
||||
Builder builder = builder(index);
|
||||
builder.version(version);
|
||||
builder.setRoutingNumShards(routingNumShards);
|
||||
builder.state(state);
|
||||
builder.settings(settings);
|
||||
builder.primaryTerms(primaryTerms);
|
||||
|
@ -611,6 +629,7 @@ public class IndexMetaData implements Diffable<IndexMetaData>, FromXContentBuild
|
|||
public IndexMetaData readFrom(StreamInput in) throws IOException {
|
||||
Builder builder = new Builder(in.readString());
|
||||
builder.version(in.readLong());
|
||||
builder.setRoutingNumShards(in.readInt());
|
||||
builder.state(State.fromId(in.readByte()));
|
||||
builder.settings(readSettingsFromStream(in));
|
||||
builder.primaryTerms(in.readVLongArray());
|
||||
|
@ -643,6 +662,7 @@ public class IndexMetaData implements Diffable<IndexMetaData>, FromXContentBuild
|
|||
public void writeTo(StreamOutput out) throws IOException {
|
||||
out.writeString(index.getName()); // uuid will come as part of settings
|
||||
out.writeLong(version);
|
||||
out.writeInt(routingNumShards);
|
||||
out.writeByte(state.id());
|
||||
writeSettingsToStream(settings, out);
|
||||
out.writeVLongArray(primaryTerms);
|
||||
|
@ -685,6 +705,7 @@ public class IndexMetaData implements Diffable<IndexMetaData>, FromXContentBuild
|
|||
private final ImmutableOpenMap.Builder<String, AliasMetaData> aliases;
|
||||
private final ImmutableOpenMap.Builder<String, Custom> customs;
|
||||
private final ImmutableOpenIntMap.Builder<Set<String>> activeAllocationIds;
|
||||
private Integer routingNumShards;
|
||||
|
||||
public Builder(String index) {
|
||||
this.index = index;
|
||||
|
@ -703,6 +724,7 @@ public class IndexMetaData implements Diffable<IndexMetaData>, FromXContentBuild
|
|||
this.mappings = ImmutableOpenMap.builder(indexMetaData.mappings);
|
||||
this.aliases = ImmutableOpenMap.builder(indexMetaData.aliases);
|
||||
this.customs = ImmutableOpenMap.builder(indexMetaData.customs);
|
||||
this.routingNumShards = indexMetaData.routingNumShards;
|
||||
this.activeAllocationIds = ImmutableOpenIntMap.builder(indexMetaData.activeAllocationIds);
|
||||
}
|
||||
|
||||
|
@ -720,6 +742,26 @@ public class IndexMetaData implements Diffable<IndexMetaData>, FromXContentBuild
|
|||
return this;
|
||||
}
|
||||
|
||||
/**
|
||||
* Sets the number of shards that should be used for routing. This should only be used if the number of shards in
|
||||
* an index has changed ie if the index is shrunk.
|
||||
*/
|
||||
public Builder setRoutingNumShards(int routingNumShards) {
|
||||
this.routingNumShards = routingNumShards;
|
||||
return this;
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns number of shards that should be used for routing. By default this method will return the number of shards
|
||||
* for this index.
|
||||
*
|
||||
* @see #setRoutingNumShards(int)
|
||||
* @see #numberOfShards()
|
||||
*/
|
||||
public int getRoutingNumShards() {
|
||||
return routingNumShards == null ? numberOfShards() : routingNumShards;
|
||||
}
|
||||
|
||||
public int numberOfShards() {
|
||||
return settings.getAsInt(SETTING_NUMBER_OF_SHARDS, -1);
|
||||
}
|
||||
|
@ -934,13 +976,14 @@ public class IndexMetaData implements Diffable<IndexMetaData>, FromXContentBuild
|
|||
final String uuid = settings.get(SETTING_INDEX_UUID, INDEX_UUID_NA_VALUE);
|
||||
return new IndexMetaData(new Index(index, uuid), version, primaryTerms, state, numberOfShards, numberOfReplicas, tmpSettings, mappings.build(),
|
||||
tmpAliases.build(), customs.build(), filledActiveAllocationIds.build(), requireFilters, initialRecoveryFilters, includeFilters, excludeFilters,
|
||||
indexCreatedVersion, indexUpgradedVersion, minimumCompatibleLuceneVersion);
|
||||
indexCreatedVersion, indexUpgradedVersion, minimumCompatibleLuceneVersion, getRoutingNumShards());
|
||||
}
|
||||
|
||||
public static void toXContent(IndexMetaData indexMetaData, XContentBuilder builder, ToXContent.Params params) throws IOException {
|
||||
builder.startObject(indexMetaData.getIndex().getName());
|
||||
|
||||
builder.field(KEY_VERSION, indexMetaData.getVersion());
|
||||
builder.field(KEY_ROUTING_NUM_SHARDS, indexMetaData.getRoutingNumShards());
|
||||
builder.field(KEY_STATE, indexMetaData.getState().toString().toLowerCase(Locale.ENGLISH));
|
||||
|
||||
boolean binary = params.paramAsBoolean("binary", false);
|
||||
|
@ -1101,6 +1144,8 @@ public class IndexMetaData implements Diffable<IndexMetaData>, FromXContentBuild
|
|||
builder.state(State.fromString(parser.text()));
|
||||
} else if (KEY_VERSION.equals(currentFieldName)) {
|
||||
builder.version(parser.longValue());
|
||||
} else if (KEY_ROUTING_NUM_SHARDS.equals(currentFieldName)) {
|
||||
builder.setRoutingNumShards(parser.intValue());
|
||||
} else {
|
||||
throw new IllegalArgumentException("Unexpected field [" + currentFieldName + "]");
|
||||
}
|
||||
|
@ -1175,4 +1220,68 @@ public class IndexMetaData implements Diffable<IndexMetaData>, FromXContentBuild
|
|||
return Builder.fromXContent(parser);
|
||||
}
|
||||
};
|
||||
|
||||
/**
|
||||
* Returns the number of shards that should be used for routing. This basically defines the hash space we use in
|
||||
* {@link org.elasticsearch.cluster.routing.OperationRouting#generateShardId(IndexMetaData, String, String)} to route documents
|
||||
* to shards based on their ID or their specific routing value. The default value is {@link #getNumberOfShards()}. This value only
|
||||
* changes if and index is shrunk.
|
||||
*/
|
||||
public int getRoutingNumShards() {
|
||||
return routingNumShards;
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns the routing factor for this index. The default is <tt>1</tt>.
|
||||
*
|
||||
* @see #getRoutingFactor(IndexMetaData, int) for details
|
||||
*/
|
||||
public int getRoutingFactor() {
|
||||
return routingFactor;
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns the source shard ids to shrink into the given shard id.
|
||||
* @param shardId the id of the target shard to shrink to
|
||||
* @param sourceIndexMetadata the source index metadata
|
||||
* @param numTargetShards the total number of shards in the target index
|
||||
* @return a set of shard IDs to shrink into the given shard ID.
|
||||
*/
|
||||
public static Set<ShardId> selectShrinkShards(int shardId, IndexMetaData sourceIndexMetadata, int numTargetShards) {
|
||||
if (shardId >= numTargetShards) {
|
||||
throw new IllegalArgumentException("the number of target shards (" + numTargetShards + ") must be greater than the shard id: "
|
||||
+ shardId);
|
||||
}
|
||||
int routingFactor = getRoutingFactor(sourceIndexMetadata, numTargetShards);
|
||||
Set<ShardId> shards = new HashSet<>(routingFactor);
|
||||
for (int i = shardId * routingFactor; i < routingFactor*shardId + routingFactor; i++) {
|
||||
shards.add(new ShardId(sourceIndexMetadata.getIndex(), i));
|
||||
}
|
||||
return shards;
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns the routing factor for and shrunk index with the given number of target shards.
|
||||
* This factor is used in the hash function in
|
||||
* {@link org.elasticsearch.cluster.routing.OperationRouting#generateShardId(IndexMetaData, String, String)} to guarantee consistent
|
||||
* hashing / routing of documents even if the number of shards changed (ie. a shrunk index).
|
||||
*
|
||||
* @param sourceIndexMetadata the metadata of the source index
|
||||
* @param targetNumberOfShards the total number of shards in the target index
|
||||
* @return the routing factor for and shrunk index with the given number of target shards.
|
||||
* @throws IllegalArgumentException if the number of source shards is greater than the number of target shards or if the source shards
|
||||
* are not divisible by the number of target shards.
|
||||
*/
|
||||
public static int getRoutingFactor(IndexMetaData sourceIndexMetadata, int targetNumberOfShards) {
|
||||
int sourceNumberOfShards = sourceIndexMetadata.getNumberOfShards();
|
||||
if (sourceNumberOfShards < targetNumberOfShards) {
|
||||
throw new IllegalArgumentException("the number of target shards must be less that the number of source shards");
|
||||
}
|
||||
int factor = sourceNumberOfShards / targetNumberOfShards;
|
||||
if (factor * targetNumberOfShards != sourceNumberOfShards || factor <= 1) {
|
||||
throw new IllegalArgumentException("the number of source shards [" + sourceNumberOfShards + "] must be a must be a multiple of ["
|
||||
+ targetNumberOfShards + "]");
|
||||
}
|
||||
return factor;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -21,7 +21,6 @@ package org.elasticsearch.cluster.metadata;
|
|||
|
||||
import com.carrotsearch.hppc.cursors.ObjectCursor;
|
||||
import com.carrotsearch.hppc.cursors.ObjectObjectCursor;
|
||||
import org.apache.lucene.index.IndexWriter;
|
||||
import org.apache.lucene.util.CollectionUtil;
|
||||
import org.elasticsearch.ElasticsearchException;
|
||||
import org.elasticsearch.Version;
|
||||
|
@ -65,7 +64,6 @@ import org.elasticsearch.index.mapper.DocumentMapper;
|
|||
import org.elasticsearch.index.mapper.MapperParsingException;
|
||||
import org.elasticsearch.index.mapper.MapperService;
|
||||
import org.elasticsearch.index.query.QueryShardContext;
|
||||
import org.elasticsearch.index.shard.DocsStats;
|
||||
import org.elasticsearch.indices.IndexAlreadyExistsException;
|
||||
import org.elasticsearch.indices.IndexCreationException;
|
||||
import org.elasticsearch.indices.IndicesService;
|
||||
|
@ -299,15 +297,19 @@ public class MetaDataCreateIndexService extends AbstractComponent {
|
|||
|
||||
indexSettingsBuilder.put(SETTING_INDEX_UUID, UUIDs.randomBase64UUID());
|
||||
final Index shrinkFromIndex = request.shrinkFrom();
|
||||
int routingNumShards = IndexMetaData.INDEX_NUMBER_OF_SHARDS_SETTING.get(indexSettingsBuilder.build());;
|
||||
if (shrinkFromIndex != null) {
|
||||
prepareShrinkIndexSettings(currentState, mappings.keySet(), indexSettingsBuilder, shrinkFromIndex,
|
||||
request.index());
|
||||
IndexMetaData sourceMetaData = currentState.metaData().getIndexSafe(shrinkFromIndex);
|
||||
routingNumShards = sourceMetaData.getRoutingNumShards();
|
||||
}
|
||||
|
||||
Settings actualIndexSettings = indexSettingsBuilder.build();
|
||||
|
||||
IndexMetaData.Builder tmpImdBuilder = IndexMetaData.builder(request.index())
|
||||
.setRoutingNumShards(routingNumShards);
|
||||
// Set up everything, now locally create the index to see that things are ok, and apply
|
||||
final IndexMetaData tmpImd = IndexMetaData.builder(request.index()).settings(actualIndexSettings).build();
|
||||
final IndexMetaData tmpImd = tmpImdBuilder.settings(actualIndexSettings).build();
|
||||
// create the index here (on the master) to validate it can be created, as well as adding the mapping
|
||||
final IndexService indexService = indicesService.createIndex(nodeServicesProvider, tmpImd,
|
||||
Collections.emptyList(), shardId -> {});
|
||||
|
@ -340,7 +342,9 @@ public class MetaDataCreateIndexService extends AbstractComponent {
|
|||
mappingsMetaData.put(mapper.type(), mappingMd);
|
||||
}
|
||||
|
||||
final IndexMetaData.Builder indexMetaDataBuilder = IndexMetaData.builder(request.index()).settings(actualIndexSettings);
|
||||
final IndexMetaData.Builder indexMetaDataBuilder = IndexMetaData.builder(request.index())
|
||||
.settings(actualIndexSettings)
|
||||
.setRoutingNumShards(routingNumShards);
|
||||
for (MappingMetaData mappingMd : mappingsMetaData.values()) {
|
||||
indexMetaDataBuilder.putMapping(mappingMd);
|
||||
}
|
||||
|
@ -495,14 +499,16 @@ public class MetaDataCreateIndexService extends AbstractComponent {
|
|||
throw new IllegalArgumentException("can't shrink an index with only one shard");
|
||||
}
|
||||
|
||||
|
||||
if ((targetIndexMappingsTypes.size() > 1 ||
|
||||
(targetIndexMappingsTypes.isEmpty() || targetIndexMappingsTypes.contains(MapperService.DEFAULT_MAPPING)) == false)) {
|
||||
throw new IllegalArgumentException("mappings are not allowed when shrinking indices" +
|
||||
", all mappings are copied from the source index");
|
||||
}
|
||||
if (IndexMetaData.INDEX_NUMBER_OF_SHARDS_SETTING.exists(targetIndexSettings)
|
||||
&& IndexMetaData.INDEX_NUMBER_OF_SHARDS_SETTING.get(targetIndexSettings) > 1) {
|
||||
throw new IllegalArgumentException("can not shrink index into more than one shard");
|
||||
if (IndexMetaData.INDEX_NUMBER_OF_SHARDS_SETTING.exists(targetIndexSettings)) {
|
||||
// this method applies all necessary checks ie. if the target shards are less than the source shards
|
||||
// of if the source shards are divisible by the number of target shards
|
||||
IndexMetaData.getRoutingFactor(sourceMetaData, IndexMetaData.INDEX_NUMBER_OF_SHARDS_SETTING.get(targetIndexSettings));
|
||||
}
|
||||
|
||||
// now check that index is all on one node
|
||||
|
@ -534,8 +540,6 @@ public class MetaDataCreateIndexService extends AbstractComponent {
|
|||
final Predicate<String> analysisSimilarityPredicate = (s) -> s.startsWith("index.similarity.")
|
||||
|| s.startsWith("index.analysis.");
|
||||
indexSettingsBuilder
|
||||
// we can only shrink to 1 shard so far!
|
||||
.put("index.number_of_shards", 1)
|
||||
// we use "i.r.a.initial_recovery" rather than "i.r.a.require|include" since we want the replica to allocate right away
|
||||
// once we are allocated.
|
||||
.put("index.routing.allocation.initial_recovery._id",
|
||||
|
|
|
@ -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);
|
||||
|
|
|
@ -218,14 +218,16 @@ public class OperationRouting extends AbstractComponent {
|
|||
return new ShardId(indexMetaData.getIndex(), generateShardId(indexMetaData, id, routing));
|
||||
}
|
||||
|
||||
private int generateShardId(IndexMetaData indexMetaData, String id, @Nullable String routing) {
|
||||
static int generateShardId(IndexMetaData indexMetaData, String id, @Nullable String routing) {
|
||||
final int hash;
|
||||
if (routing == null) {
|
||||
hash = Murmur3HashFunction.hash(id);
|
||||
} else {
|
||||
hash = Murmur3HashFunction.hash(routing);
|
||||
}
|
||||
return Math.floorMod(hash, indexMetaData.getNumberOfShards());
|
||||
// we don't use IMD#getNumberOfShards since the index might have been shrunk such that we need to use the size
|
||||
// of original index to hash documents
|
||||
return Math.floorMod(hash, indexMetaData.getRoutingNumShards()) / indexMetaData.getRoutingFactor();
|
||||
}
|
||||
|
||||
private void ensureNodeIdExists(DiscoveryNodes nodes, String nodeId) {
|
||||
|
|
|
@ -46,6 +46,7 @@ import org.elasticsearch.common.unit.RatioValue;
|
|||
import org.elasticsearch.common.unit.TimeValue;
|
||||
import org.elasticsearch.common.util.set.Sets;
|
||||
import org.elasticsearch.index.Index;
|
||||
import org.elasticsearch.index.shard.ShardId;
|
||||
|
||||
import java.util.Set;
|
||||
|
||||
|
@ -653,10 +654,14 @@ public class DiskThresholdDecider extends AllocationDecider {
|
|||
if (metaData.getMergeSourceIndex() != null && shard.allocatedPostIndexCreate(metaData) == false) {
|
||||
// in the shrink index case we sum up the source index shards since we basically make a copy of the shard in
|
||||
// the worst case
|
||||
Index mergeSourceIndex = metaData.getMergeSourceIndex();
|
||||
long targetShardSize = 0;
|
||||
final Index mergeSourceIndex = metaData.getMergeSourceIndex();
|
||||
final IndexMetaData sourceIndexMeta = allocation.metaData().getIndexSafe(metaData.getMergeSourceIndex());
|
||||
final Set<ShardId> shardIds = IndexMetaData.selectShrinkShards(shard.id(), sourceIndexMeta, metaData.getNumberOfShards());
|
||||
for (IndexShardRoutingTable shardRoutingTable : allocation.routingTable().index(mergeSourceIndex.getName())) {
|
||||
targetShardSize += info.getShardSize(shardRoutingTable.primaryShard(), 0);
|
||||
if (shardIds.contains(shardRoutingTable.shardId())) {
|
||||
targetShardSize += info.getShardSize(shardRoutingTable.primaryShard(), 0);
|
||||
}
|
||||
}
|
||||
return targetShardSize == 0 ? defaultValue : targetShardSize;
|
||||
} else {
|
||||
|
|
|
@ -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
|
||||
)));
|
||||
}
|
||||
|
|
|
@ -118,6 +118,7 @@ public final class IndexScopedSettings extends AbstractScopedSettings {
|
|||
IndexSettings.INDEX_CHECK_ON_STARTUP,
|
||||
IndexSettings.INDEX_SEQ_NO_CHECKPOINT_SYNC_INTERVAL,
|
||||
LocalCheckpointService.SETTINGS_BIT_ARRAYS_SIZE,
|
||||
IndexSettings.MAX_REFRESH_LISTENERS_PER_SHARD,
|
||||
ShardsLimitAllocationDecider.INDEX_TOTAL_SHARDS_PER_NODE_SETTING,
|
||||
IndexSettings.INDEX_GC_DELETES_SETTING,
|
||||
IndicesRequestCache.INDEX_CACHE_REQUEST_ENABLED_SETTING,
|
||||
|
|
|
@ -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) {
|
||||
|
|
|
@ -195,7 +195,11 @@ public class Environment {
|
|||
|
||||
/**
|
||||
* The data location with the cluster name as a sub directory.
|
||||
*
|
||||
* @deprecated Used to upgrade old data paths to new ones that do not include the cluster name, should not be used to write files to and
|
||||
* will be removed in ES 6.0
|
||||
*/
|
||||
@Deprecated
|
||||
public Path[] dataWithClusterFiles() {
|
||||
return dataWithClusterFiles;
|
||||
}
|
||||
|
|
|
@ -32,9 +32,12 @@ import org.elasticsearch.ElasticsearchException;
|
|||
import org.elasticsearch.cluster.metadata.IndexMetaData;
|
||||
import org.elasticsearch.cluster.node.DiscoveryNode;
|
||||
import org.elasticsearch.common.SuppressForbidden;
|
||||
import org.elasticsearch.common.UUIDs;
|
||||
import org.elasticsearch.common.component.AbstractComponent;
|
||||
import org.elasticsearch.common.inject.Inject;
|
||||
import org.elasticsearch.common.io.FileSystemUtils;
|
||||
import org.elasticsearch.common.logging.DeprecationLogger;
|
||||
import org.elasticsearch.common.logging.ESLogger;
|
||||
import org.elasticsearch.common.settings.Setting;
|
||||
import org.elasticsearch.common.settings.Setting.Property;
|
||||
import org.elasticsearch.common.settings.Settings;
|
||||
|
@ -63,6 +66,7 @@ import java.util.Arrays;
|
|||
import java.util.Collection;
|
||||
import java.util.HashMap;
|
||||
import java.util.HashSet;
|
||||
import java.util.Iterator;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Set;
|
||||
|
@ -161,6 +165,7 @@ public final class NodeEnvironment extends AbstractComponent implements Closeabl
|
|||
public static final String NODES_FOLDER = "nodes";
|
||||
public static final String INDICES_FOLDER = "indices";
|
||||
public static final String NODE_LOCK_FILENAME = "node.lock";
|
||||
public static final String UPGRADE_LOCK_FILENAME = "upgrade.lock";
|
||||
|
||||
@Inject
|
||||
public NodeEnvironment(Settings settings, Environment environment) throws IOException {
|
||||
|
@ -175,7 +180,7 @@ public final class NodeEnvironment extends AbstractComponent implements Closeabl
|
|||
localNodeId = -1;
|
||||
return;
|
||||
}
|
||||
final NodePath[] nodePaths = new NodePath[environment.dataWithClusterFiles().length];
|
||||
final NodePath[] nodePaths = new NodePath[environment.dataFiles().length];
|
||||
final Lock[] locks = new Lock[nodePaths.length];
|
||||
boolean success = false;
|
||||
|
||||
|
@ -185,8 +190,17 @@ public final class NodeEnvironment extends AbstractComponent implements Closeabl
|
|||
IOException lastException = null;
|
||||
int maxLocalStorageNodes = MAX_LOCAL_STORAGE_NODES_SETTING.get(settings);
|
||||
for (int possibleLockId = 0; possibleLockId < maxLocalStorageNodes; possibleLockId++) {
|
||||
for (int dirIndex = 0; dirIndex < environment.dataWithClusterFiles().length; dirIndex++) {
|
||||
Path dir = environment.dataWithClusterFiles()[dirIndex].resolve(NODES_FOLDER).resolve(Integer.toString(possibleLockId));
|
||||
for (int dirIndex = 0; dirIndex < environment.dataFiles().length; dirIndex++) {
|
||||
Path dataDirWithClusterName = environment.dataWithClusterFiles()[dirIndex];
|
||||
Path dataDir = environment.dataFiles()[dirIndex];
|
||||
// TODO: Remove this in 6.0, we are no longer going to read from the cluster name directory
|
||||
if (readFromDataPathWithClusterName(dataDirWithClusterName)) {
|
||||
DeprecationLogger deprecationLogger = new DeprecationLogger(logger);
|
||||
deprecationLogger.deprecated("ES has detected the [path.data] folder using the cluster name as a folder [{}], " +
|
||||
"Elasticsearch 6.0 will not allow the cluster name as a folder within the data path", dataDir);
|
||||
dataDir = dataDirWithClusterName;
|
||||
}
|
||||
Path dir = dataDir.resolve(NODES_FOLDER).resolve(Integer.toString(possibleLockId));
|
||||
Files.createDirectories(dir);
|
||||
|
||||
try (Directory luceneDir = FSDirectory.open(dir, NativeFSLockFactory.INSTANCE)) {
|
||||
|
@ -218,7 +232,7 @@ public final class NodeEnvironment extends AbstractComponent implements Closeabl
|
|||
|
||||
if (locks[0] == null) {
|
||||
throw new IllegalStateException("Failed to obtain node lock, is the following location writable?: "
|
||||
+ Arrays.toString(environment.dataWithClusterFiles()), lastException);
|
||||
+ Arrays.toString(environment.dataFiles()), lastException);
|
||||
}
|
||||
|
||||
this.localNodeId = localNodeId;
|
||||
|
@ -242,6 +256,32 @@ public final class NodeEnvironment extends AbstractComponent implements Closeabl
|
|||
}
|
||||
}
|
||||
|
||||
/** Returns true if the directory is empty */
|
||||
private static boolean dirEmpty(final Path path) throws IOException {
|
||||
try (DirectoryStream<Path> stream = Files.newDirectoryStream(path)) {
|
||||
return stream.iterator().hasNext() == false;
|
||||
}
|
||||
}
|
||||
|
||||
// Visible for testing
|
||||
/** Returns true if data should be read from the data path that includes the cluster name (ie, it has data in it) */
|
||||
static boolean readFromDataPathWithClusterName(Path dataPathWithClusterName) throws IOException {
|
||||
if (Files.exists(dataPathWithClusterName) == false || // If it doesn't exist
|
||||
Files.isDirectory(dataPathWithClusterName) == false || // Or isn't a directory
|
||||
dirEmpty(dataPathWithClusterName)) { // Or if it's empty
|
||||
// No need to read from cluster-name folder!
|
||||
return false;
|
||||
}
|
||||
// The "nodes" directory inside of the cluster name
|
||||
Path nodesPath = dataPathWithClusterName.resolve(NODES_FOLDER);
|
||||
if (Files.isDirectory(nodesPath)) {
|
||||
// The cluster has data in the "nodes" so we should read from the cluster-named folder for now
|
||||
return true;
|
||||
}
|
||||
// Hey the nodes directory didn't exist, so we can safely use whatever directory we feel appropriate
|
||||
return false;
|
||||
}
|
||||
|
||||
private static void releaseAndNullLocks(Lock[] locks) {
|
||||
for (int i = 0; i < locks.length; i++) {
|
||||
if (locks[i] != null) {
|
||||
|
|
|
@ -118,6 +118,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;
|
||||
|
@ -149,6 +154,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.
|
||||
|
@ -234,6 +243,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());
|
||||
|
||||
|
@ -256,6 +266,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) {
|
||||
|
@ -511,6 +522,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;}
|
||||
}
|
||||
|
|
|
@ -611,6 +611,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;
|
||||
|
||||
/**
|
||||
|
@ -1018,6 +1019,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;
|
||||
|
@ -1026,6 +1030,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;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -158,6 +158,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) {
|
||||
|
|
|
@ -68,6 +68,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)
|
||||
|
|
|
@ -27,7 +27,6 @@ import org.elasticsearch.common.geo.builders.ShapeBuilder;
|
|||
import org.elasticsearch.index.query.MoreLikeThisQueryBuilder.Item;
|
||||
import org.elasticsearch.index.query.functionscore.FunctionScoreQueryBuilder;
|
||||
import org.elasticsearch.index.query.functionscore.ScoreFunctionBuilder;
|
||||
import org.elasticsearch.index.search.MatchQuery;
|
||||
import org.elasticsearch.indices.TermsLookup;
|
||||
import org.elasticsearch.script.Script;
|
||||
import org.elasticsearch.script.ScriptService;
|
||||
|
@ -57,7 +56,7 @@ public abstract class QueryBuilders {
|
|||
* @param text The query text (to be analyzed).
|
||||
*/
|
||||
public static MatchQueryBuilder matchQuery(String name, Object text) {
|
||||
return new MatchQueryBuilder(name, text).type(MatchQuery.Type.BOOLEAN);
|
||||
return new MatchQueryBuilder(name, text);
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -86,8 +85,8 @@ public abstract class QueryBuilders {
|
|||
* @param name The field name.
|
||||
* @param text The query text (to be analyzed).
|
||||
*/
|
||||
public static MatchQueryBuilder matchPhraseQuery(String name, Object text) {
|
||||
return new MatchQueryBuilder(name, text).type(MatchQuery.Type.PHRASE);
|
||||
public static MatchPhraseQueryBuilder matchPhraseQuery(String name, Object text) {
|
||||
return new MatchPhraseQueryBuilder(name, text);
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -96,8 +95,8 @@ public abstract class QueryBuilders {
|
|||
* @param name The field name.
|
||||
* @param text The query text (to be analyzed).
|
||||
*/
|
||||
public static MatchQueryBuilder matchPhrasePrefixQuery(String name, Object text) {
|
||||
return new MatchQueryBuilder(name, text).type(MatchQuery.Type.PHRASE_PREFIX);
|
||||
public static MatchPhrasePrefixQueryBuilder matchPhrasePrefixQuery(String name, Object text) {
|
||||
return new MatchPhrasePrefixQueryBuilder(name, text);
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
|
@ -19,9 +19,9 @@
|
|||
package org.elasticsearch.index.seqno;
|
||||
|
||||
import org.elasticsearch.action.ActionListener;
|
||||
import org.elasticsearch.action.ReplicationResponse;
|
||||
import org.elasticsearch.action.support.ActionFilters;
|
||||
import org.elasticsearch.action.support.replication.ReplicationRequest;
|
||||
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.metadata.IndexNameExpressionResolver;
|
||||
|
@ -62,18 +62,19 @@ public class GlobalCheckpointSyncAction extends TransportReplicationAction<Globa
|
|||
}
|
||||
|
||||
@Override
|
||||
protected Tuple<ReplicationResponse, ReplicaRequest> shardOperationOnPrimary(PrimaryRequest request) {
|
||||
protected PrimaryResult shardOperationOnPrimary(PrimaryRequest request) {
|
||||
IndexService indexService = indicesService.indexServiceSafe(request.shardId().getIndex());
|
||||
IndexShard indexShard = indexService.getShard(request.shardId().id());
|
||||
long checkpoint = indexShard.getGlobalCheckpoint();
|
||||
return new Tuple<>(new ReplicationResponse(), new ReplicaRequest(request, checkpoint));
|
||||
return new PrimaryResult(new ReplicaRequest(request, checkpoint), new ReplicationResponse());
|
||||
}
|
||||
|
||||
@Override
|
||||
protected void shardOperationOnReplica(ReplicaRequest request) {
|
||||
protected ReplicaResult shardOperationOnReplica(ReplicaRequest request) {
|
||||
IndexService indexService = indicesService.indexServiceSafe(request.shardId().getIndex());
|
||||
IndexShard indexShard = indexService.getShard(request.shardId().id());
|
||||
indexShard.updateGlobalCheckpointOnReplica(request.checkpoint);
|
||||
return new ReplicaResult();
|
||||
}
|
||||
|
||||
public void updateCheckpointForShard(ShardId shardId) {
|
||||
|
|
|
@ -135,7 +135,9 @@ 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;
|
||||
import java.util.stream.Collectors;
|
||||
|
||||
public class IndexShard extends AbstractIndexShardComponent {
|
||||
|
||||
|
@ -210,6 +212,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,
|
||||
|
@ -264,6 +272,7 @@ public class IndexShard extends AbstractIndexShardComponent {
|
|||
suspendableRefContainer = new SuspendableRefContainer();
|
||||
searcherWrapper = indexSearcherWrapper;
|
||||
primaryTerm = indexSettings.getIndexMetaData().primaryTerm(shardId.id());
|
||||
refreshListeners = buildRefreshListeners();
|
||||
persistMetadata(shardRouting, null);
|
||||
}
|
||||
|
||||
|
@ -595,6 +604,7 @@ public class IndexShard extends AbstractIndexShardComponent {
|
|||
*/
|
||||
public void refresh(String source) {
|
||||
verifyNotClosed();
|
||||
|
||||
if (canIndex()) {
|
||||
long bytes = getEngine().getIndexBufferRAMBytesUsed();
|
||||
writingBytes.addAndGet(bytes);
|
||||
|
@ -1490,7 +1500,10 @@ public class IndexShard extends AbstractIndexShardComponent {
|
|||
markAsRecovering("from local shards", recoveryState); // mark the shard as recovering on the cluster state thread
|
||||
threadPool.generic().execute(() -> {
|
||||
try {
|
||||
if (recoverFromLocalShards(mappingUpdateConsumer, startedShards)) {
|
||||
final Set<ShardId> shards = IndexMetaData.selectShrinkShards(shardId().id(), sourceIndexService.getMetaData(),
|
||||
indexMetaData.getNumberOfShards());
|
||||
if (recoverFromLocalShards(mappingUpdateConsumer, startedShards.stream()
|
||||
.filter((s) -> shards.contains(s.shardId())).collect(Collectors.toList()))) {
|
||||
recoveryListener.onRecoveryDone(recoveryState);
|
||||
}
|
||||
} catch (Throwable t) {
|
||||
|
@ -1618,7 +1631,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() {
|
||||
|
@ -1714,6 +1727,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
|
||||
*
|
||||
|
@ -1739,14 +1763,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);
|
||||
}
|
||||
}
|
||||
});
|
||||
}
|
||||
}
|
||||
}
|
|
@ -32,12 +32,14 @@ import org.elasticsearch.index.merge.MergeStats;
|
|||
import org.elasticsearch.index.seqno.SeqNoStats;
|
||||
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
|
||||
|
@ -93,6 +95,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
|
||||
|
@ -104,7 +112,6 @@ public final class ShadowIndexShard extends IndexShard {
|
|||
return null; // shadow engine has no translog
|
||||
}
|
||||
|
||||
|
||||
@Override
|
||||
public void updateGlobalCheckpointOnReplica(long checkpoint) {
|
||||
// nocommit: think shadow replicas through
|
||||
|
@ -121,4 +128,10 @@ public final class ShadowIndexShard extends IndexShard {
|
|||
// nocommit: think shadow replicas through
|
||||
return -1;
|
||||
}
|
||||
|
||||
@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");
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
@ -448,6 +448,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;
|
||||
|
||||
|
|
|
@ -115,6 +115,18 @@ public class JvmInfo implements Streamable, ToXContent {
|
|||
Method vmOptionMethod = clazz.getMethod("getVMOption", String.class);
|
||||
Method valueMethod = vmOptionClazz.getMethod("getValue");
|
||||
|
||||
try {
|
||||
Object onError = vmOptionMethod.invoke(hotSpotDiagnosticMXBean, "OnError");
|
||||
info.onError = (String) valueMethod.invoke(onError);
|
||||
} catch (Exception ignored) {
|
||||
}
|
||||
|
||||
try {
|
||||
Object onOutOfMemoryError = vmOptionMethod.invoke(hotSpotDiagnosticMXBean, "OnOutOfMemoryError");
|
||||
info.onOutOfMemoryError = (String) valueMethod.invoke(onOutOfMemoryError);
|
||||
} catch (Exception ignored) {
|
||||
}
|
||||
|
||||
try {
|
||||
Object useCompressedOopsVmOption = vmOptionMethod.invoke(hotSpotDiagnosticMXBean, "UseCompressedOops");
|
||||
info.useCompressedOops = (String) valueMethod.invoke(useCompressedOopsVmOption);
|
||||
|
@ -179,6 +191,10 @@ public class JvmInfo implements Streamable, ToXContent {
|
|||
String[] gcCollectors = Strings.EMPTY_ARRAY;
|
||||
String[] memoryPools = Strings.EMPTY_ARRAY;
|
||||
|
||||
private String onError;
|
||||
|
||||
private String onOutOfMemoryError;
|
||||
|
||||
private String useCompressedOops = "unknown";
|
||||
|
||||
private String useG1GC = "unknown";
|
||||
|
@ -314,6 +330,14 @@ public class JvmInfo implements Streamable, ToXContent {
|
|||
return configuredMaxHeapSize;
|
||||
}
|
||||
|
||||
public String onError() {
|
||||
return onError;
|
||||
}
|
||||
|
||||
public String onOutOfMemoryError() {
|
||||
return onOutOfMemoryError;
|
||||
}
|
||||
|
||||
/**
|
||||
* The value of the JVM flag UseCompressedOops, if available otherwise
|
||||
* "unknown". The value "unknown" indicates that an attempt was
|
||||
|
|
|
@ -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);
|
||||
|
|
|
@ -24,22 +24,6 @@ import org.elasticsearch.common.util.BigArrays;
|
|||
import org.elasticsearch.ingest.ProcessorsRegistry;
|
||||
import org.elasticsearch.ingest.core.Processor;
|
||||
import org.elasticsearch.ingest.core.TemplateService;
|
||||
import org.elasticsearch.ingest.processor.AppendProcessor;
|
||||
import org.elasticsearch.ingest.processor.ConvertProcessor;
|
||||
import org.elasticsearch.ingest.processor.DateProcessor;
|
||||
import org.elasticsearch.ingest.processor.DateIndexNameProcessor;
|
||||
import org.elasticsearch.ingest.processor.FailProcessor;
|
||||
import org.elasticsearch.ingest.processor.ForEachProcessor;
|
||||
import org.elasticsearch.ingest.processor.GsubProcessor;
|
||||
import org.elasticsearch.ingest.processor.JoinProcessor;
|
||||
import org.elasticsearch.ingest.processor.LowercaseProcessor;
|
||||
import org.elasticsearch.ingest.processor.RemoveProcessor;
|
||||
import org.elasticsearch.ingest.processor.RenameProcessor;
|
||||
import org.elasticsearch.ingest.processor.SetProcessor;
|
||||
import org.elasticsearch.ingest.processor.SortProcessor;
|
||||
import org.elasticsearch.ingest.processor.SplitProcessor;
|
||||
import org.elasticsearch.ingest.processor.TrimProcessor;
|
||||
import org.elasticsearch.ingest.processor.UppercaseProcessor;
|
||||
import org.elasticsearch.monitor.MonitorService;
|
||||
import org.elasticsearch.node.service.NodeService;
|
||||
|
||||
|
@ -61,23 +45,6 @@ public class NodeModule extends AbstractModule {
|
|||
this.node = node;
|
||||
this.monitorService = monitorService;
|
||||
this.processorsRegistryBuilder = new ProcessorsRegistry.Builder();
|
||||
|
||||
registerProcessor(DateProcessor.TYPE, (templateService, registry) -> new DateProcessor.Factory());
|
||||
registerProcessor(SetProcessor.TYPE, (templateService, registry) -> new SetProcessor.Factory(templateService));
|
||||
registerProcessor(AppendProcessor.TYPE, (templateService, registry) -> new AppendProcessor.Factory(templateService));
|
||||
registerProcessor(RenameProcessor.TYPE, (templateService, registry) -> new RenameProcessor.Factory());
|
||||
registerProcessor(RemoveProcessor.TYPE, (templateService, registry) -> new RemoveProcessor.Factory(templateService));
|
||||
registerProcessor(SplitProcessor.TYPE, (templateService, registry) -> new SplitProcessor.Factory());
|
||||
registerProcessor(JoinProcessor.TYPE, (templateService, registry) -> new JoinProcessor.Factory());
|
||||
registerProcessor(UppercaseProcessor.TYPE, (templateService, registry) -> new UppercaseProcessor.Factory());
|
||||
registerProcessor(LowercaseProcessor.TYPE, (templateService, registry) -> new LowercaseProcessor.Factory());
|
||||
registerProcessor(TrimProcessor.TYPE, (templateService, registry) -> new TrimProcessor.Factory());
|
||||
registerProcessor(ConvertProcessor.TYPE, (templateService, registry) -> new ConvertProcessor.Factory());
|
||||
registerProcessor(GsubProcessor.TYPE, (templateService, registry) -> new GsubProcessor.Factory());
|
||||
registerProcessor(FailProcessor.TYPE, (templateService, registry) -> new FailProcessor.Factory(templateService));
|
||||
registerProcessor(ForEachProcessor.TYPE, (templateService, registry) -> new ForEachProcessor.Factory(registry));
|
||||
registerProcessor(DateIndexNameProcessor.TYPE, (templateService, registry) -> new DateIndexNameProcessor.Factory());
|
||||
registerProcessor(SortProcessor.TYPE, (templateService, registry) -> new SortProcessor.Factory());
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -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) {
|
||||
|
|
|
@ -52,7 +52,7 @@ public class RestShrinkIndexAction extends BaseRestHandler {
|
|||
}
|
||||
ShrinkRequest shrinkIndexRequest = new ShrinkRequest(request.param("target"), request.param("index"));
|
||||
if (request.hasContent()) {
|
||||
shrinkIndexRequest.getShrinkIndexReqeust().source(request.content());
|
||||
shrinkIndexRequest.getShrinkIndexRequest().source(request.content());
|
||||
}
|
||||
shrinkIndexRequest.timeout(request.paramAsTime("timeout", shrinkIndexRequest.timeout()));
|
||||
shrinkIndexRequest.masterNodeTimeout(request.paramAsTime("master_timeout", shrinkIndexRequest.masterNodeTimeout()));
|
||||
|
|
|
@ -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) {
|
||||
|
|
|
@ -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");
|
||||
|
|
|
@ -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));
|
||||
|
|
|
@ -821,6 +821,15 @@ public class SearchService extends AbstractLifecycleComponent<SearchService> imp
|
|||
FieldDoc fieldDoc = SearchAfterBuilder.buildFieldDoc(context.sort(), source.searchAfter());
|
||||
context.searchAfter(fieldDoc);
|
||||
}
|
||||
|
||||
if (source.slice() != null) {
|
||||
if (context.scrollContext() == null) {
|
||||
throw new SearchContextException(context, "`slice` cannot be used outside of a scroll context");
|
||||
}
|
||||
context.sliceFilter(source.slice().toFilter(queryShardContext,
|
||||
context.shardTarget().getShardId().getId(),
|
||||
queryShardContext.getIndexSettings().getNumberOfShards()));
|
||||
}
|
||||
}
|
||||
|
||||
private static final int[] EMPTY_DOC_IDS = new int[0];
|
||||
|
|
|
@ -42,6 +42,7 @@ import org.elasticsearch.index.query.QueryParseContext;
|
|||
import org.elasticsearch.index.query.QueryShardContext;
|
||||
import org.elasticsearch.script.Script;
|
||||
import org.elasticsearch.search.aggregations.AggregationBuilder;
|
||||
import org.elasticsearch.search.slice.SliceBuilder;
|
||||
import org.elasticsearch.search.aggregations.AggregatorFactories;
|
||||
import org.elasticsearch.search.aggregations.AggregatorParsers;
|
||||
import org.elasticsearch.search.aggregations.PipelineAggregatorBuilder;
|
||||
|
@ -98,6 +99,7 @@ public final class SearchSourceBuilder extends ToXContentToBytes implements Writ
|
|||
public static final ParseField EXT_FIELD = new ParseField("ext");
|
||||
public static final ParseField PROFILE_FIELD = new ParseField("profile");
|
||||
public static final ParseField SEARCH_AFTER = new ParseField("search_after");
|
||||
public static final ParseField SLICE = new ParseField("slice");
|
||||
|
||||
public static SearchSourceBuilder fromXContent(QueryParseContext context, AggregatorParsers aggParsers,
|
||||
Suggesters suggesters) throws IOException {
|
||||
|
@ -138,6 +140,8 @@ public final class SearchSourceBuilder extends ToXContentToBytes implements Writ
|
|||
|
||||
private SearchAfterBuilder searchAfterBuilder;
|
||||
|
||||
private SliceBuilder sliceBuilder;
|
||||
|
||||
private Float minScore;
|
||||
|
||||
private long timeoutInMillis = -1;
|
||||
|
@ -175,9 +179,7 @@ public final class SearchSourceBuilder extends ToXContentToBytes implements Writ
|
|||
* Read from a stream.
|
||||
*/
|
||||
public SearchSourceBuilder(StreamInput in) throws IOException {
|
||||
if (in.readBoolean()) {
|
||||
aggregations = new AggregatorFactories.Builder(in);
|
||||
}
|
||||
aggregations = in.readOptionalWriteable(AggregatorFactories.Builder::new);
|
||||
explain = in.readOptionalBoolean();
|
||||
fetchSourceContext = in.readOptionalStreamable(FetchSourceContext::new);
|
||||
boolean hasFieldDataFields = in.readBoolean();
|
||||
|
@ -206,15 +208,9 @@ public final class SearchSourceBuilder extends ToXContentToBytes implements Writ
|
|||
indexBoost.put(in.readString(), in.readFloat());
|
||||
}
|
||||
}
|
||||
if (in.readBoolean()) {
|
||||
minScore = in.readFloat();
|
||||
}
|
||||
if (in.readBoolean()) {
|
||||
postQueryBuilder = in.readNamedWriteable(QueryBuilder.class);
|
||||
}
|
||||
if (in.readBoolean()) {
|
||||
queryBuilder = in.readNamedWriteable(QueryBuilder.class);
|
||||
}
|
||||
minScore = in.readOptionalFloat();
|
||||
postQueryBuilder = in.readOptionalNamedWriteable(QueryBuilder.class);
|
||||
queryBuilder = in.readOptionalNamedWriteable(QueryBuilder.class);
|
||||
if (in.readBoolean()) {
|
||||
int size = in.readVInt();
|
||||
rescoreBuilders = new ArrayList<>();
|
||||
|
@ -244,29 +240,20 @@ public final class SearchSourceBuilder extends ToXContentToBytes implements Writ
|
|||
stats.add(in.readString());
|
||||
}
|
||||
}
|
||||
if (in.readBoolean()) {
|
||||
suggestBuilder = new SuggestBuilder(in);
|
||||
}
|
||||
suggestBuilder = in.readOptionalWriteable(SuggestBuilder::new);
|
||||
terminateAfter = in.readVInt();
|
||||
timeoutInMillis = in.readLong();
|
||||
trackScores = in.readBoolean();
|
||||
version = in.readOptionalBoolean();
|
||||
if (in.readBoolean()) {
|
||||
ext = in.readBytesReference();
|
||||
}
|
||||
ext = in.readOptionalBytesReference();
|
||||
profile = in.readBoolean();
|
||||
if (in.readBoolean()) {
|
||||
searchAfterBuilder = new SearchAfterBuilder(in);
|
||||
}
|
||||
searchAfterBuilder = in.readOptionalWriteable(SearchAfterBuilder::new);
|
||||
sliceBuilder = in.readOptionalWriteable(SliceBuilder::new);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void writeTo(StreamOutput out) throws IOException {
|
||||
boolean hasAggregations = aggregations != null;
|
||||
out.writeBoolean(hasAggregations);
|
||||
if (hasAggregations) {
|
||||
aggregations.writeTo(out);
|
||||
}
|
||||
out.writeOptionalWriteable(aggregations);
|
||||
out.writeOptionalBoolean(explain);
|
||||
out.writeOptionalStreamable(fetchSourceContext);
|
||||
boolean hasFieldDataFields = fieldDataFields != null;
|
||||
|
@ -296,21 +283,9 @@ public final class SearchSourceBuilder extends ToXContentToBytes implements Writ
|
|||
out.writeFloat(indexBoost.get(key.value));
|
||||
}
|
||||
}
|
||||
boolean hasMinScore = minScore != null;
|
||||
out.writeBoolean(hasMinScore);
|
||||
if (hasMinScore) {
|
||||
out.writeFloat(minScore);
|
||||
}
|
||||
boolean hasPostQuery = postQueryBuilder != null;
|
||||
out.writeBoolean(hasPostQuery);
|
||||
if (hasPostQuery) {
|
||||
out.writeNamedWriteable(postQueryBuilder);
|
||||
}
|
||||
boolean hasQuery = queryBuilder != null;
|
||||
out.writeBoolean(hasQuery);
|
||||
if (hasQuery) {
|
||||
out.writeNamedWriteable(queryBuilder);
|
||||
}
|
||||
out.writeOptionalFloat(minScore);
|
||||
out.writeOptionalNamedWriteable(postQueryBuilder);
|
||||
out.writeOptionalNamedWriteable(queryBuilder);
|
||||
boolean hasRescoreBuilders = rescoreBuilders != null;
|
||||
out.writeBoolean(hasRescoreBuilders);
|
||||
if (hasRescoreBuilders) {
|
||||
|
@ -344,26 +319,15 @@ public final class SearchSourceBuilder extends ToXContentToBytes implements Writ
|
|||
out.writeString(stat);
|
||||
}
|
||||
}
|
||||
boolean hasSuggestBuilder = suggestBuilder != null;
|
||||
out.writeBoolean(hasSuggestBuilder);
|
||||
if (hasSuggestBuilder) {
|
||||
suggestBuilder.writeTo(out);
|
||||
}
|
||||
out.writeOptionalWriteable(suggestBuilder);
|
||||
out.writeVInt(terminateAfter);
|
||||
out.writeLong(timeoutInMillis);
|
||||
out.writeBoolean(trackScores);
|
||||
out.writeOptionalBoolean(version);
|
||||
boolean hasExt = ext != null;
|
||||
out.writeBoolean(hasExt);
|
||||
if (hasExt) {
|
||||
out.writeBytesReference(ext);
|
||||
}
|
||||
out.writeOptionalBytesReference(ext);
|
||||
out.writeBoolean(profile);
|
||||
boolean hasSearchAfter = searchAfterBuilder != null;
|
||||
out.writeBoolean(hasSearchAfter);
|
||||
if (hasSearchAfter) {
|
||||
searchAfterBuilder.writeTo(out);
|
||||
}
|
||||
out.writeOptionalWriteable(searchAfterBuilder);
|
||||
out.writeOptionalWriteable(sliceBuilder);
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -597,6 +561,22 @@ public final class SearchSourceBuilder extends ToXContentToBytes implements Writ
|
|||
return this;
|
||||
}
|
||||
|
||||
/**
|
||||
* Sets a filter that will restrict the search hits, the top hits and the aggregations to a slice of the results
|
||||
* of the main query.
|
||||
*/
|
||||
public SearchSourceBuilder slice(SliceBuilder builder) {
|
||||
this.sliceBuilder = builder;
|
||||
return this;
|
||||
}
|
||||
|
||||
/**
|
||||
* Gets the slice used to filter the search hits, the top hits and the aggregations.
|
||||
*/
|
||||
public SliceBuilder slice() {
|
||||
return sliceBuilder;
|
||||
}
|
||||
|
||||
/**
|
||||
* Add an aggregation to perform as part of the search.
|
||||
*/
|
||||
|
@ -943,6 +923,7 @@ public final class SearchSourceBuilder extends ToXContentToBytes implements Writ
|
|||
rewrittenBuilder.rescoreBuilders = rescoreBuilders;
|
||||
rewrittenBuilder.scriptFields = scriptFields;
|
||||
rewrittenBuilder.searchAfterBuilder = searchAfterBuilder;
|
||||
rewrittenBuilder.sliceBuilder = sliceBuilder;
|
||||
rewrittenBuilder.size = size;
|
||||
rewrittenBuilder.sorts = sorts;
|
||||
rewrittenBuilder.stats = stats;
|
||||
|
@ -1039,6 +1020,8 @@ public final class SearchSourceBuilder extends ToXContentToBytes implements Writ
|
|||
} else if (context.getParseFieldMatcher().match(currentFieldName, EXT_FIELD)) {
|
||||
XContentBuilder xContentBuilder = XContentFactory.jsonBuilder().copyCurrentStructure(parser);
|
||||
ext = xContentBuilder.bytes();
|
||||
} else if (context.getParseFieldMatcher().match(currentFieldName, SLICE)) {
|
||||
sliceBuilder = SliceBuilder.fromXContent(context);
|
||||
} else {
|
||||
throw new ParsingException(parser.getTokenLocation(), "Unknown key for a " + token + " in [" + currentFieldName + "].",
|
||||
parser.getTokenLocation());
|
||||
|
@ -1193,6 +1176,10 @@ public final class SearchSourceBuilder extends ToXContentToBytes implements Writ
|
|||
builder.field(SEARCH_AFTER.getPreferredName(), searchAfterBuilder.getSortValues());
|
||||
}
|
||||
|
||||
if (sliceBuilder != null) {
|
||||
builder.field(SLICE.getPreferredName(), sliceBuilder);
|
||||
}
|
||||
|
||||
if (indexBoost != null) {
|
||||
builder.startObject(INDICES_BOOST_FIELD.getPreferredName());
|
||||
assert !indexBoost.containsKey(null);
|
||||
|
@ -1355,7 +1342,7 @@ public final class SearchSourceBuilder extends ToXContentToBytes implements Writ
|
|||
public int hashCode() {
|
||||
return Objects.hash(aggregations, explain, fetchSourceContext, fieldDataFields, fieldNames, from,
|
||||
highlightBuilder, indexBoost, minScore, postQueryBuilder, queryBuilder, rescoreBuilders, scriptFields,
|
||||
size, sorts, searchAfterBuilder, stats, suggestBuilder, terminateAfter, timeoutInMillis, trackScores, version, profile);
|
||||
size, sorts, searchAfterBuilder, sliceBuilder, stats, suggestBuilder, terminateAfter, timeoutInMillis, trackScores, version, profile);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -1383,6 +1370,7 @@ public final class SearchSourceBuilder extends ToXContentToBytes implements Writ
|
|||
&& Objects.equals(size, other.size)
|
||||
&& Objects.equals(sorts, other.sorts)
|
||||
&& Objects.equals(searchAfterBuilder, other.searchAfterBuilder)
|
||||
&& Objects.equals(sliceBuilder, other.sliceBuilder)
|
||||
&& Objects.equals(stats, other.stats)
|
||||
&& Objects.equals(suggestBuilder, other.suggestBuilder)
|
||||
&& Objects.equals(terminateAfter, other.terminateAfter)
|
||||
|
|
|
@ -115,6 +115,9 @@ public class DefaultSearchContext extends SearchContext {
|
|||
private Float minimumScore;
|
||||
private boolean trackScores = false; // when sorting, track scores as well...
|
||||
private FieldDoc searchAfter;
|
||||
// filter for sliced scroll
|
||||
private Query sliceFilter;
|
||||
|
||||
/**
|
||||
* The original query as sent by the user without the types and aliases
|
||||
* applied. Putting things in here leaks them into highlighting so don't add
|
||||
|
@ -122,8 +125,7 @@ public class DefaultSearchContext extends SearchContext {
|
|||
*/
|
||||
private ParsedQuery originalQuery;
|
||||
/**
|
||||
* Just like originalQuery but with the filters from types and aliases
|
||||
* applied.
|
||||
* Just like originalQuery but with the filters from types, aliases and slice applied.
|
||||
*/
|
||||
private ParsedQuery filteredQuery;
|
||||
/**
|
||||
|
@ -210,7 +212,7 @@ public class DefaultSearchContext extends SearchContext {
|
|||
if (rescoreContext.window() > maxWindow) {
|
||||
throw new QueryPhaseExecutionException(this, "Rescore window [" + rescoreContext.window() + "] is too large. It must "
|
||||
+ "be less than [" + maxWindow + "]. This prevents allocating massive heaps for storing the results to be "
|
||||
+ "rescored. This limit can be set by chaning the [" + IndexSettings.MAX_RESCORE_WINDOW_SETTING.getKey()
|
||||
+ "rescored. This limit can be set by chaining the [" + IndexSettings.MAX_RESCORE_WINDOW_SETTING.getKey()
|
||||
+ "] index level setting.");
|
||||
|
||||
}
|
||||
|
@ -254,7 +256,17 @@ public class DefaultSearchContext extends SearchContext {
|
|||
@Override
|
||||
@Nullable
|
||||
public Query searchFilter(String[] types) {
|
||||
return createSearchFilter(types, aliasFilter, mapperService().hasNested());
|
||||
Query typesFilter = createSearchFilter(types, aliasFilter, mapperService().hasNested());
|
||||
if (sliceFilter == null) {
|
||||
return typesFilter;
|
||||
}
|
||||
if (typesFilter == null) {
|
||||
return sliceFilter;
|
||||
}
|
||||
return new BooleanQuery.Builder()
|
||||
.add(typesFilter, Occur.FILTER)
|
||||
.add(sliceFilter, Occur.FILTER)
|
||||
.build();
|
||||
}
|
||||
|
||||
// extracted to static helper method to make writing unit tests easier:
|
||||
|
@ -550,6 +562,11 @@ public class DefaultSearchContext extends SearchContext {
|
|||
return searchAfter;
|
||||
}
|
||||
|
||||
public SearchContext sliceFilter(Query filter) {
|
||||
this.sliceFilter = filter;
|
||||
return this;
|
||||
}
|
||||
|
||||
@Override
|
||||
public SearchContext parsedPostFilter(ParsedQuery postFilter) {
|
||||
this.postFilter = postFilter;
|
||||
|
|
|
@ -29,5 +29,4 @@ public class ScrollContext {
|
|||
public float maxScore;
|
||||
public ScoreDoc lastEmittedDoc;
|
||||
public Scroll scroll;
|
||||
|
||||
}
|
||||
|
|
|
@ -20,7 +20,6 @@ package org.elasticsearch.search.internal;
|
|||
|
||||
import org.apache.lucene.search.Query;
|
||||
import org.apache.lucene.util.Counter;
|
||||
import org.elasticsearch.action.search.SearchType;
|
||||
import org.elasticsearch.index.query.ParsedQuery;
|
||||
import org.elasticsearch.search.aggregations.SearchContextAggregations;
|
||||
import org.elasticsearch.search.fetch.FetchSearchResult;
|
||||
|
|
|
@ -0,0 +1,68 @@
|
|||
/*
|
||||
* Licensed to Elasticsearch under one or more contributor
|
||||
* license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright
|
||||
* ownership. Elasticsearch licenses this file to you under
|
||||
* the Apache License, Version 2.0 (the "License"); you may
|
||||
* not use this file except in compliance with the License.
|
||||
* You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing,
|
||||
* software distributed under the License is distributed on an
|
||||
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
|
||||
* KIND, either express or implied. See the License for the
|
||||
* specific language governing permissions and limitations
|
||||
* under the License.
|
||||
*/
|
||||
|
||||
package org.elasticsearch.search.slice;
|
||||
|
||||
import com.carrotsearch.hppc.BitMixer;
|
||||
import org.apache.lucene.index.LeafReaderContext;
|
||||
import org.apache.lucene.index.DocValues;
|
||||
import org.apache.lucene.index.SortedNumericDocValues;
|
||||
import org.apache.lucene.search.IndexSearcher;
|
||||
import org.apache.lucene.search.Weight;
|
||||
import org.apache.lucene.search.RandomAccessWeight;
|
||||
import org.apache.lucene.util.Bits;
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
/**
|
||||
* A {@link SliceQuery} that uses the numeric doc values of a field to do the slicing.
|
||||
*
|
||||
* <b>NOTE</b>: With deterministic field values this query can be used across different readers safely.
|
||||
* If updates are accepted on the field you must ensure that the same reader is used for all `slice` queries.
|
||||
*/
|
||||
public final class DocValuesSliceQuery extends SliceQuery {
|
||||
public DocValuesSliceQuery(String field, int id, int max) {
|
||||
super(field, id, max);
|
||||
}
|
||||
|
||||
@Override
|
||||
public Weight createWeight(IndexSearcher searcher, boolean needsScores) throws IOException {
|
||||
return new RandomAccessWeight(this) {
|
||||
@Override
|
||||
protected Bits getMatchingDocs(final LeafReaderContext context) throws IOException {
|
||||
final SortedNumericDocValues values = DocValues.getSortedNumeric(context.reader(), getField());
|
||||
return new Bits() {
|
||||
@Override
|
||||
public boolean get(int doc) {
|
||||
values.setDocument(doc);
|
||||
for (int i = 0; i < values.count(); i++) {
|
||||
return contains(BitMixer.mix(values.valueAt(i)));
|
||||
}
|
||||
return contains(0);
|
||||
}
|
||||
|
||||
@Override
|
||||
public int length() {
|
||||
return context.reader().maxDoc();
|
||||
}
|
||||
};
|
||||
}
|
||||
};
|
||||
}
|
||||
}
|
|
@ -0,0 +1,251 @@
|
|||
/*
|
||||
* Licensed to Elasticsearch under one or more contributor
|
||||
* license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright
|
||||
* ownership. Elasticsearch licenses this file to you under
|
||||
* the Apache License, Version 2.0 (the "License"); you may
|
||||
* not use this file except in compliance with the License.
|
||||
* You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing,
|
||||
* software distributed under the License is distributed on an
|
||||
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
|
||||
* KIND, either express or implied. See the License for the
|
||||
* specific language governing permissions and limitations
|
||||
* under the License.
|
||||
*/
|
||||
|
||||
package org.elasticsearch.search.slice;
|
||||
|
||||
import org.apache.lucene.search.MatchAllDocsQuery;
|
||||
import org.apache.lucene.search.Query;
|
||||
import org.elasticsearch.action.support.ToXContentToBytes;
|
||||
import org.elasticsearch.common.ParseField;
|
||||
import org.elasticsearch.common.Strings;
|
||||
import org.elasticsearch.common.io.stream.StreamInput;
|
||||
import org.elasticsearch.common.io.stream.StreamOutput;
|
||||
import org.elasticsearch.common.io.stream.Writeable;
|
||||
import org.elasticsearch.common.lucene.search.MatchNoDocsQuery;
|
||||
import org.elasticsearch.common.xcontent.ObjectParser;
|
||||
import org.elasticsearch.common.xcontent.XContentBuilder;
|
||||
import org.elasticsearch.index.fielddata.IndexFieldData;
|
||||
import org.elasticsearch.index.fielddata.IndexNumericFieldData;
|
||||
import org.elasticsearch.index.mapper.MappedFieldType;
|
||||
import org.elasticsearch.index.mapper.internal.UidFieldMapper;
|
||||
import org.elasticsearch.index.query.QueryParseContext;
|
||||
import org.elasticsearch.index.query.QueryShardContext;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.Objects;
|
||||
|
||||
/**
|
||||
* A slice builder allowing to split a scroll in multiple partitions.
|
||||
* If the provided field is the "_uid" it uses a {@link org.elasticsearch.search.slice.TermsSliceQuery}
|
||||
* to do the slicing. The slicing is done at the shard level first and then each shard is splitted in multiple slices.
|
||||
* For instance if the number of shards is equal to 2 and the user requested 4 slices
|
||||
* then the slices 0 and 2 are assigned to the first shard and the slices 1 and 3 are assigned to the second shard.
|
||||
* This way the total number of bitsets that we need to build on each shard is bounded by the number of slices
|
||||
* (instead of {@code numShards*numSlices}).
|
||||
* Otherwise the provided field must be a numeric and doc_values must be enabled. In that case a
|
||||
* {@link org.elasticsearch.search.slice.DocValuesSliceQuery} is used to filter the results.
|
||||
*/
|
||||
public class SliceBuilder extends ToXContentToBytes implements Writeable {
|
||||
public static final ParseField FIELD_FIELD = new ParseField("field");
|
||||
public static final ParseField ID_FIELD = new ParseField("id");
|
||||
public static final ParseField MAX_FIELD = new ParseField("max");
|
||||
private final static ObjectParser<SliceBuilder, QueryParseContext> PARSER =
|
||||
new ObjectParser<>("slice", SliceBuilder::new);
|
||||
|
||||
static {
|
||||
PARSER.declareString(SliceBuilder::setField, FIELD_FIELD);
|
||||
PARSER.declareInt(SliceBuilder::setId, ID_FIELD);
|
||||
PARSER.declareInt(SliceBuilder::setMax, MAX_FIELD);
|
||||
}
|
||||
|
||||
/** Name of field to slice against (_uid by default) */
|
||||
private String field = UidFieldMapper.NAME;
|
||||
/** The id of the slice */
|
||||
private int id = -1;
|
||||
/** Max number of slices */
|
||||
private int max = -1;
|
||||
|
||||
private SliceBuilder() {}
|
||||
|
||||
public SliceBuilder(int id, int max) {
|
||||
this(UidFieldMapper.NAME, id, max);
|
||||
}
|
||||
|
||||
/**
|
||||
*
|
||||
* @param field The name of the field
|
||||
* @param id The id of the slice
|
||||
* @param max The maximum number of slices
|
||||
*/
|
||||
public SliceBuilder(String field, int id, int max) {
|
||||
setField(field);
|
||||
setId(id);
|
||||
setMax(max);
|
||||
}
|
||||
|
||||
public SliceBuilder(StreamInput in) throws IOException {
|
||||
this.field = in.readString();
|
||||
this.id = in.readVInt();
|
||||
this.max = in.readVInt();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void writeTo(StreamOutput out) throws IOException {
|
||||
out.writeString(field);
|
||||
out.writeVInt(id);
|
||||
out.writeVInt(max);
|
||||
}
|
||||
|
||||
private SliceBuilder setField(String field) {
|
||||
if (Strings.isEmpty(field)) {
|
||||
throw new IllegalArgumentException("field name is null or empty");
|
||||
}
|
||||
this.field = field;
|
||||
return this;
|
||||
}
|
||||
|
||||
/**
|
||||
* The name of the field to slice against
|
||||
*/
|
||||
public String getField() {
|
||||
return this.field;
|
||||
}
|
||||
|
||||
private SliceBuilder setId(int id) {
|
||||
if (id < 0) {
|
||||
throw new IllegalArgumentException("id must be greater than or equal to 0");
|
||||
}
|
||||
if (max != -1 && id >= max) {
|
||||
throw new IllegalArgumentException("max must be greater than id");
|
||||
}
|
||||
this.id = id;
|
||||
return this;
|
||||
}
|
||||
|
||||
/**
|
||||
* The id of the slice.
|
||||
*/
|
||||
public int getId() {
|
||||
return id;
|
||||
}
|
||||
|
||||
private SliceBuilder setMax(int max) {
|
||||
if (max <= 1) {
|
||||
throw new IllegalArgumentException("max must be greater than 1");
|
||||
}
|
||||
if (id != -1 && id >= max) {
|
||||
throw new IllegalArgumentException("max must be greater than id");
|
||||
}
|
||||
this.max = max;
|
||||
return this;
|
||||
}
|
||||
|
||||
/**
|
||||
* The maximum number of slices.
|
||||
*/
|
||||
public int getMax() {
|
||||
return max;
|
||||
}
|
||||
|
||||
@Override
|
||||
public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException {
|
||||
builder.startObject();
|
||||
innerToXContent(builder);
|
||||
builder.endObject();
|
||||
return builder;
|
||||
}
|
||||
|
||||
void innerToXContent(XContentBuilder builder) throws IOException {
|
||||
builder.field(FIELD_FIELD.getPreferredName(), field);
|
||||
builder.field(ID_FIELD.getPreferredName(), id);
|
||||
builder.field(MAX_FIELD.getPreferredName(), max);
|
||||
}
|
||||
|
||||
public static SliceBuilder fromXContent(QueryParseContext context) throws IOException {
|
||||
SliceBuilder builder = PARSER.parse(context.parser(), new SliceBuilder(), context);
|
||||
return builder;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean equals(Object other) {
|
||||
if (!(other instanceof SliceBuilder)) {
|
||||
return false;
|
||||
}
|
||||
|
||||
SliceBuilder o = (SliceBuilder) other;
|
||||
return ((field == null && o.field == null) || field.equals(o.field))
|
||||
&& id == o.id && o.max == max;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int hashCode() {
|
||||
return Objects.hash(this.field, this.id, this.max);
|
||||
}
|
||||
|
||||
public Query toFilter(QueryShardContext context, int shardId, int numShards) {
|
||||
final MappedFieldType type = context.fieldMapper(field);
|
||||
if (type == null) {
|
||||
throw new IllegalArgumentException("field " + field + " not found");
|
||||
}
|
||||
|
||||
boolean useTermQuery = false;
|
||||
if (UidFieldMapper.NAME.equals(field)) {
|
||||
useTermQuery = true;
|
||||
} else if (type.hasDocValues() == false) {
|
||||
throw new IllegalArgumentException("cannot load numeric doc values on " + field);
|
||||
} else {
|
||||
IndexFieldData ifm = context.getForField(type);
|
||||
if (ifm instanceof IndexNumericFieldData == false) {
|
||||
throw new IllegalArgumentException("cannot load numeric doc values on " + field);
|
||||
}
|
||||
}
|
||||
|
||||
if (numShards == 1) {
|
||||
return useTermQuery ? new TermsSliceQuery(field, id, max) :
|
||||
new DocValuesSliceQuery(field, id, max);
|
||||
}
|
||||
if (max >= numShards) {
|
||||
// the number of slices is greater than the number of shards
|
||||
// in such case we can reduce the number of requested shards by slice
|
||||
|
||||
// first we check if the slice is responsible of this shard
|
||||
int targetShard = id % numShards;
|
||||
if (targetShard != shardId) {
|
||||
// the shard is not part of this slice, we can skip it.
|
||||
return new MatchNoDocsQuery("this shard is not part of the slice");
|
||||
}
|
||||
// compute the number of slices where this shard appears
|
||||
int numSlicesInShard = max / numShards;
|
||||
int rest = max % numShards;
|
||||
if (rest > targetShard) {
|
||||
numSlicesInShard++;
|
||||
}
|
||||
|
||||
if (numSlicesInShard == 1) {
|
||||
// this shard has only one slice so we must check all the documents
|
||||
return new MatchAllDocsQuery();
|
||||
}
|
||||
// get the new slice id for this shard
|
||||
int shardSlice = id / numShards;
|
||||
|
||||
return useTermQuery ?
|
||||
new TermsSliceQuery(field, shardSlice, numSlicesInShard) :
|
||||
new DocValuesSliceQuery(field, shardSlice, numSlicesInShard);
|
||||
}
|
||||
// the number of shards is greater than the number of slices
|
||||
|
||||
// check if the shard is assigned to the slice
|
||||
int targetSlice = shardId % max;
|
||||
if (id != targetSlice) {
|
||||
// the shard is not part of this slice, we can skip it.
|
||||
return new MatchNoDocsQuery("this shard is not part of the slice");
|
||||
}
|
||||
return new MatchAllDocsQuery();
|
||||
}
|
||||
}
|
|
@ -0,0 +1,81 @@
|
|||
/*
|
||||
* Licensed to Elasticsearch under one or more contributor
|
||||
* license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright
|
||||
* ownership. Elasticsearch licenses this file to you under
|
||||
* the Apache License, Version 2.0 (the "License"); you may
|
||||
* not use this file except in compliance with the License.
|
||||
* You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing,
|
||||
* software distributed under the License is distributed on an
|
||||
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
|
||||
* KIND, either express or implied. See the License for the
|
||||
* specific language governing permissions and limitations
|
||||
* under the License.
|
||||
*/
|
||||
|
||||
package org.elasticsearch.search.slice;
|
||||
|
||||
import org.apache.lucene.search.Query;
|
||||
|
||||
import java.util.Objects;
|
||||
|
||||
/**
|
||||
* An abstract {@link Query} that defines an hash function to partition the documents in multiple slices.
|
||||
*/
|
||||
public abstract class SliceQuery extends Query {
|
||||
private final String field;
|
||||
private final int id;
|
||||
private final int max;
|
||||
|
||||
/**
|
||||
* @param field The name of the field
|
||||
* @param id The id of the slice
|
||||
* @param max The maximum number of slices
|
||||
*/
|
||||
public SliceQuery(String field, int id, int max) {
|
||||
this.field = field;
|
||||
this.id = id;
|
||||
this.max = max;
|
||||
}
|
||||
|
||||
// Returns true if the value matches the predicate
|
||||
protected final boolean contains(long value) {
|
||||
return Math.floorMod(value, max) == id;
|
||||
}
|
||||
|
||||
public String getField() {
|
||||
return field;
|
||||
}
|
||||
|
||||
public int getId() {
|
||||
return id;
|
||||
}
|
||||
|
||||
public int getMax() {
|
||||
return max;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean equals(Object o) {
|
||||
if (super.equals(o) == false) {
|
||||
return false;
|
||||
}
|
||||
SliceQuery that = (SliceQuery) o;
|
||||
return field.equals(that.field) && id == that.id && max == that.max;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int hashCode() {
|
||||
return Objects.hash(super.hashCode(), field, id, max);
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString(String f) {
|
||||
return getClass().getSimpleName() + "[field=" + field + ", id=" + id + ", max=" + max + "]";
|
||||
}
|
||||
|
||||
}
|
|
@ -0,0 +1,86 @@
|
|||
/*
|
||||
* Licensed to Elasticsearch under one or more contributor
|
||||
* license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright
|
||||
* ownership. Elasticsearch licenses this file to you under
|
||||
* the Apache License, Version 2.0 (the "License"); you may
|
||||
* not use this file except in compliance with the License.
|
||||
* You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing,
|
||||
* software distributed under the License is distributed on an
|
||||
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
|
||||
* KIND, either express or implied. See the License for the
|
||||
* specific language governing permissions and limitations
|
||||
* under the License.
|
||||
*/
|
||||
|
||||
package org.elasticsearch.search.slice;
|
||||
|
||||
import org.apache.lucene.index.LeafReaderContext;
|
||||
import org.apache.lucene.index.LeafReader;
|
||||
import org.apache.lucene.index.Terms;
|
||||
import org.apache.lucene.index.TermsEnum;
|
||||
import org.apache.lucene.index.PostingsEnum;
|
||||
import org.apache.lucene.search.IndexSearcher;
|
||||
import org.apache.lucene.search.DocIdSet;
|
||||
import org.apache.lucene.search.DocIdSetIterator;
|
||||
import org.apache.lucene.search.Weight;
|
||||
import org.apache.lucene.search.ConstantScoreWeight;
|
||||
import org.apache.lucene.search.Scorer;
|
||||
import org.apache.lucene.search.ConstantScoreScorer;
|
||||
import org.apache.lucene.util.BytesRef;
|
||||
import org.apache.lucene.util.DocIdSetBuilder;
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
/**
|
||||
* A {@link SliceQuery} that uses the terms dictionary of a field to do the slicing.
|
||||
*
|
||||
* <b>NOTE</b>: The cost of this filter is O(N*M) where N is the number of unique terms in the dictionary
|
||||
* and M is the average number of documents per term.
|
||||
* For each segment this filter enumerates the terms dictionary, computes the hash code for each term and fills
|
||||
* a bit set with the documents of all terms whose hash code matches the predicate.
|
||||
* <b>NOTE</b>: Documents with no value for that field are ignored.
|
||||
*/
|
||||
public final class TermsSliceQuery extends SliceQuery {
|
||||
public TermsSliceQuery(String field, int id, int max) {
|
||||
super(field, id, max);
|
||||
}
|
||||
|
||||
@Override
|
||||
public Weight createWeight(IndexSearcher searcher, boolean needsScores) throws IOException {
|
||||
return new ConstantScoreWeight(this) {
|
||||
@Override
|
||||
public Scorer scorer(LeafReaderContext context) throws IOException {
|
||||
final DocIdSet disi = build(context.reader());
|
||||
final DocIdSetIterator leafIt = disi.iterator();
|
||||
return new ConstantScoreScorer(this, score(), leafIt);
|
||||
}
|
||||
};
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns a DocIdSet per segments containing the matching docs for the specified slice.
|
||||
*/
|
||||
private DocIdSet build(LeafReader reader) throws IOException {
|
||||
final DocIdSetBuilder builder = new DocIdSetBuilder(reader.maxDoc());
|
||||
final Terms terms = reader.terms(getField());
|
||||
final TermsEnum te = terms.iterator();
|
||||
PostingsEnum docsEnum = null;
|
||||
for (BytesRef term = te.next(); term != null; term = te.next()) {
|
||||
int hashCode = term.hashCode();
|
||||
if (contains(hashCode)) {
|
||||
docsEnum = te.postings(docsEnum, PostingsEnum.NONE);
|
||||
int docId = docsEnum.nextDoc();
|
||||
while (docId != DocIdSetIterator.NO_MORE_DOCS) {
|
||||
builder.add(docId);
|
||||
docId = docsEnum.nextDoc();
|
||||
}
|
||||
}
|
||||
}
|
||||
return builder.build();
|
||||
}
|
||||
}
|
|
@ -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
|
||||
|
|
|
@ -289,6 +289,76 @@ public class CreateIndexIT extends ESIntegTestCase {
|
|||
ensureGreen("test");
|
||||
}
|
||||
|
||||
public void testCreateShrinkIndexToN() {
|
||||
int[][] possibleShardSplits = new int[][] {{8,4,2}, {9, 3, 1}, {4, 2, 1}, {15,5,1}};
|
||||
int[] shardSplits = randomFrom(possibleShardSplits);
|
||||
assertEquals(shardSplits[0], (shardSplits[0] / shardSplits[1]) * shardSplits[1]);
|
||||
assertEquals(shardSplits[1], (shardSplits[1] / shardSplits[2]) * shardSplits[2]);
|
||||
internalCluster().ensureAtLeastNumDataNodes(2);
|
||||
prepareCreate("source").setSettings(Settings.builder().put(indexSettings()).put("number_of_shards", shardSplits[0])).get();
|
||||
for (int i = 0; i < 20; i++) {
|
||||
client().prepareIndex("source", "t1", Integer.toString(i)).setSource("{\"foo\" : \"bar\", \"i\" : " + i + "}").get();
|
||||
}
|
||||
ImmutableOpenMap<String, DiscoveryNode> dataNodes = client().admin().cluster().prepareState().get().getState().nodes()
|
||||
.getDataNodes();
|
||||
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()
|
||||
.put("index.routing.allocation.require._name", mergeNode)
|
||||
.put("index.blocks.write", true)).get();
|
||||
ensureGreen();
|
||||
// now merge source into a 4 shard index
|
||||
assertAcked(client().admin().indices().prepareShrinkIndex("source", "first_shrink")
|
||||
.setSettings(Settings.builder()
|
||||
.put("index.number_of_replicas", 0)
|
||||
.put("index.number_of_shards", shardSplits[1]).build()).get());
|
||||
ensureGreen();
|
||||
assertHitCount(client().prepareSearch("first_shrink").setSize(100).setQuery(new TermsQueryBuilder("foo", "bar")).get(), 20);
|
||||
|
||||
for (int i = 0; i < 20; i++) { // now update
|
||||
client().prepareIndex("first_shrink", "t1", Integer.toString(i)).setSource("{\"foo\" : \"bar\", \"i\" : " + i + "}").get();
|
||||
}
|
||||
flushAndRefresh();
|
||||
assertHitCount(client().prepareSearch("first_shrink").setSize(100).setQuery(new TermsQueryBuilder("foo", "bar")).get(), 20);
|
||||
assertHitCount(client().prepareSearch("source").setSize(100).setQuery(new TermsQueryBuilder("foo", "bar")).get(), 20);
|
||||
|
||||
// relocate all shards to one node such that we can merge it.
|
||||
client().admin().indices().prepareUpdateSettings("first_shrink")
|
||||
.setSettings(Settings.builder()
|
||||
.put("index.routing.allocation.require._name", mergeNode)
|
||||
.put("index.blocks.write", true)).get();
|
||||
ensureGreen();
|
||||
// now merge source into a 2 shard index
|
||||
assertAcked(client().admin().indices().prepareShrinkIndex("first_shrink", "second_shrink")
|
||||
.setSettings(Settings.builder()
|
||||
.put("index.number_of_replicas", 0)
|
||||
.put("index.number_of_shards", shardSplits[2]).build()).get());
|
||||
ensureGreen();
|
||||
assertHitCount(client().prepareSearch("second_shrink").setSize(100).setQuery(new TermsQueryBuilder("foo", "bar")).get(), 20);
|
||||
// let it be allocated anywhere and bump replicas
|
||||
client().admin().indices().prepareUpdateSettings("second_shrink")
|
||||
.setSettings(Settings.builder()
|
||||
.putNull("index.routing.allocation.include._id")
|
||||
.put("index.number_of_replicas", 1)).get();
|
||||
ensureGreen();
|
||||
assertHitCount(client().prepareSearch("second_shrink").setSize(100).setQuery(new TermsQueryBuilder("foo", "bar")).get(), 20);
|
||||
|
||||
for (int i = 0; i < 20; i++) { // now update
|
||||
client().prepareIndex("second_shrink", "t1", Integer.toString(i)).setSource("{\"foo\" : \"bar\", \"i\" : " + i + "}").get();
|
||||
}
|
||||
flushAndRefresh();
|
||||
assertHitCount(client().prepareSearch("second_shrink").setSize(100).setQuery(new TermsQueryBuilder("foo", "bar")).get(), 20);
|
||||
assertHitCount(client().prepareSearch("first_shrink").setSize(100).setQuery(new TermsQueryBuilder("foo", "bar")).get(), 20);
|
||||
assertHitCount(client().prepareSearch("source").setSize(100).setQuery(new TermsQueryBuilder("foo", "bar")).get(), 20);
|
||||
}
|
||||
|
||||
public void testCreateShrinkIndex() {
|
||||
internalCluster().ensureAtLeastNumDataNodes(2);
|
||||
prepareCreate("source").setSettings(Settings.builder().put(indexSettings()).put("number_of_shards", randomIntBetween(2, 7))).get();
|
||||
|
@ -300,6 +370,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 +417,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)
|
||||
|
|
|
@ -28,7 +28,6 @@ import org.elasticsearch.cluster.block.ClusterBlocks;
|
|||
import org.elasticsearch.cluster.metadata.IndexMetaData;
|
||||
import org.elasticsearch.cluster.metadata.IndexNameExpressionResolver;
|
||||
import org.elasticsearch.cluster.metadata.MetaData;
|
||||
import org.elasticsearch.cluster.metadata.MetaDataCreateIndexService;
|
||||
import org.elasticsearch.cluster.node.DiscoveryNode;
|
||||
import org.elasticsearch.cluster.node.DiscoveryNodes;
|
||||
import org.elasticsearch.cluster.routing.RoutingTable;
|
||||
|
@ -39,9 +38,7 @@ import org.elasticsearch.cluster.routing.allocation.decider.AllocationDeciders;
|
|||
import org.elasticsearch.cluster.routing.allocation.decider.MaxRetryAllocationDecider;
|
||||
import org.elasticsearch.common.settings.Settings;
|
||||
import org.elasticsearch.common.transport.DummyTransportAddress;
|
||||
import org.elasticsearch.index.IndexNotFoundException;
|
||||
import org.elasticsearch.index.shard.DocsStats;
|
||||
import org.elasticsearch.indices.IndexAlreadyExistsException;
|
||||
import org.elasticsearch.test.ESTestCase;
|
||||
import org.elasticsearch.test.gateway.NoopGatewayAllocator;
|
||||
|
||||
|
@ -70,15 +67,26 @@ public class TransportShrinkActionTests extends ESTestCase {
|
|||
}
|
||||
|
||||
public void testErrorCondition() {
|
||||
ClusterState state = createClusterState("source", randomIntBetween(2, 100), randomIntBetween(0, 10),
|
||||
ClusterState state = createClusterState("source", randomIntBetween(2, 42), randomIntBetween(0, 10),
|
||||
Settings.builder().put("index.blocks.write", true).build());
|
||||
DocsStats stats = new DocsStats(randomIntBetween(0, IndexWriter.MAX_DOCS-1), randomIntBetween(1, 1000));
|
||||
|
||||
assertEquals("Can't merge index with more than [2147483519] docs - too many documents",
|
||||
assertTrue(
|
||||
expectThrows(IllegalStateException.class, () ->
|
||||
TransportShrinkAction.prepareCreateIndexRequest(new ShrinkRequest("target", "source"), state,
|
||||
new DocsStats(Integer.MAX_VALUE, randomIntBetween(1, 1000)), new IndexNameExpressionResolver(Settings.EMPTY))
|
||||
).getMessage());
|
||||
(i) -> new DocsStats(Integer.MAX_VALUE, randomIntBetween(1, 1000)), new IndexNameExpressionResolver(Settings.EMPTY))
|
||||
).getMessage().startsWith("Can't merge index with more than [2147483519] docs - too many documents in shards "));
|
||||
|
||||
|
||||
assertTrue(
|
||||
expectThrows(IllegalStateException.class, () -> {
|
||||
ShrinkRequest req = new ShrinkRequest("target", "source");
|
||||
req.getShrinkIndexRequest().settings(Settings.builder().put("index.number_of_shards", 4));
|
||||
ClusterState clusterState = createClusterState("source", 8, 1,
|
||||
Settings.builder().put("index.blocks.write", true).build());
|
||||
TransportShrinkAction.prepareCreateIndexRequest(req, clusterState,
|
||||
(i) -> i == 2 || i == 3 ? new DocsStats(Integer.MAX_VALUE/2, randomIntBetween(1, 1000)) : null,
|
||||
new IndexNameExpressionResolver(Settings.EMPTY));
|
||||
}
|
||||
).getMessage().startsWith("Can't merge index with more than [2147483519] docs - too many documents in shards "));
|
||||
|
||||
|
||||
// create one that won't fail
|
||||
|
@ -96,8 +104,8 @@ public class TransportShrinkActionTests extends ESTestCase {
|
|||
routingTable.index("source").shardsWithState(ShardRoutingState.INITIALIZING)).routingTable();
|
||||
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
|
||||
|
||||
TransportShrinkAction.prepareCreateIndexRequest(new ShrinkRequest("target", "source"), clusterState, stats,
|
||||
new IndexNameExpressionResolver(Settings.EMPTY));
|
||||
TransportShrinkAction.prepareCreateIndexRequest(new ShrinkRequest("target", "source"), clusterState,
|
||||
(i) -> new DocsStats(randomIntBetween(1, 1000), randomIntBetween(1, 1000)), new IndexNameExpressionResolver(Settings.EMPTY));
|
||||
}
|
||||
|
||||
public void testShrinkIndexSettings() {
|
||||
|
@ -118,11 +126,12 @@ public class TransportShrinkActionTests extends ESTestCase {
|
|||
routingTable = service.applyStartedShards(clusterState,
|
||||
routingTable.index(indexName).shardsWithState(ShardRoutingState.INITIALIZING)).routingTable();
|
||||
clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build();
|
||||
|
||||
DocsStats stats = new DocsStats(randomIntBetween(0, IndexWriter.MAX_DOCS-1), randomIntBetween(1, 1000));
|
||||
int numSourceShards = clusterState.metaData().index(indexName).getNumberOfShards();
|
||||
DocsStats stats = new DocsStats(randomIntBetween(0, (IndexWriter.MAX_DOCS) / numSourceShards), randomIntBetween(1, 1000));
|
||||
ShrinkRequest target = new ShrinkRequest("target", indexName);
|
||||
CreateIndexClusterStateUpdateRequest request = TransportShrinkAction.prepareCreateIndexRequest(
|
||||
target, clusterState, stats, new IndexNameExpressionResolver(Settings.EMPTY));
|
||||
target, clusterState, (i) -> stats,
|
||||
new IndexNameExpressionResolver(Settings.EMPTY));
|
||||
assertNotNull(request.shrinkFrom());
|
||||
assertEquals(indexName, request.shrinkFrom().getName());
|
||||
assertEquals("1", request.settings().get("index.number_of_shards"));
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -17,10 +17,11 @@
|
|||
* under the License.
|
||||
*/
|
||||
|
||||
package org.elasticsearch.ingest.processor;
|
||||
package org.elasticsearch.action.ingest;
|
||||
|
||||
import org.elasticsearch.ElasticsearchException;
|
||||
import org.elasticsearch.action.ingest.SimulateProcessorResult;
|
||||
import org.elasticsearch.action.ingest.TrackingResultProcessor;
|
||||
import org.elasticsearch.ingest.TestProcessor;
|
||||
import org.elasticsearch.ingest.core.CompoundProcessor;
|
||||
import org.elasticsearch.ingest.core.IngestDocument;
|
||||
|
@ -36,7 +37,7 @@ import java.util.Map;
|
|||
import static org.elasticsearch.ingest.core.CompoundProcessor.ON_FAILURE_MESSAGE_FIELD;
|
||||
import static org.elasticsearch.ingest.core.CompoundProcessor.ON_FAILURE_PROCESSOR_TAG_FIELD;
|
||||
import static org.elasticsearch.ingest.core.CompoundProcessor.ON_FAILURE_PROCESSOR_TYPE_FIELD;
|
||||
import static org.elasticsearch.ingest.processor.TrackingResultProcessor.decorate;
|
||||
import static org.elasticsearch.action.ingest.TrackingResultProcessor.decorate;
|
||||
import static org.hamcrest.CoreMatchers.equalTo;
|
||||
import static org.hamcrest.Matchers.nullValue;
|
||||
|
|
@ -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);
|
||||
|
|
Some files were not shown because too many files have changed in this diff Show More
Loading…
Reference in New Issue