diff --git a/buildSrc/src/main/groovy/org/elasticsearch/gradle/doc/RestTestsFromSnippetsTask.groovy b/buildSrc/src/main/groovy/org/elasticsearch/gradle/doc/RestTestsFromSnippetsTask.groovy index 9f840df36e1..c9f5668a326 100644 --- a/buildSrc/src/main/groovy/org/elasticsearch/gradle/doc/RestTestsFromSnippetsTask.groovy +++ b/buildSrc/src/main/groovy/org/elasticsearch/gradle/doc/RestTestsFromSnippetsTask.groovy @@ -170,6 +170,13 @@ public class RestTestsFromSnippetsTask extends SnippetsTask { current.println('---') current.println("setup:") body(setup) + // always wait for yellow before anything is executed + current.println( + " - do:\n" + + " raw:\n" + + " method: GET\n" + + " path: \"_cluster/health\"\n" + + " wait_for_status: \"yellow\"") } private void body(Snippet snippet) { diff --git a/buildSrc/src/main/resources/checkstyle_suppressions.xml b/buildSrc/src/main/resources/checkstyle_suppressions.xml index 985ffefa411..03627b30e15 100644 --- a/buildSrc/src/main/resources/checkstyle_suppressions.xml +++ b/buildSrc/src/main/resources/checkstyle_suppressions.xml @@ -19,7 +19,6 @@ - @@ -101,7 +100,6 @@ - @@ -503,7 +501,6 @@ - @@ -961,7 +958,6 @@ - @@ -1037,14 +1033,11 @@ - - - - - - - - + + + + + @@ -1155,9 +1148,6 @@ - - - @@ -1272,14 +1262,6 @@ - - - - - - - - diff --git a/buildSrc/src/main/resources/forbidden/es-core-signatures.txt b/buildSrc/src/main/resources/forbidden/es-core-signatures.txt index 059be403a67..31084c7556c 100644 --- a/buildSrc/src/main/resources/forbidden/es-core-signatures.txt +++ b/buildSrc/src/main/resources/forbidden/es-core-signatures.txt @@ -92,3 +92,8 @@ org.joda.time.DateTime#(int, int, int, int, int, int) org.joda.time.DateTime#(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) diff --git a/core/src/main/java/org/elasticsearch/action/DocWriteResponse.java b/core/src/main/java/org/elasticsearch/action/DocWriteResponse.java index 4df43b75401..0925c744144 100644 --- a/core/src/main/java/org/elasticsearch/action/DocWriteResponse.java +++ b/core/src/main/java/org/elasticsearch/action/DocWriteResponse.java @@ -18,10 +18,15 @@ */ package org.elasticsearch.action; +import org.elasticsearch.action.support.WriteRequest; +import org.elasticsearch.action.support.WriteResponse; +import org.elasticsearch.action.support.WriteRequest.RefreshPolicy; +import org.elasticsearch.action.support.replication.ReplicationResponse; import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.io.stream.StreamOutput; import org.elasticsearch.common.xcontent.StatusToXContent; import org.elasticsearch.common.xcontent.XContentBuilder; +import org.elasticsearch.index.IndexSettings; import org.elasticsearch.index.shard.ShardId; import org.elasticsearch.rest.RestStatus; @@ -30,12 +35,13 @@ import java.io.IOException; /** * A base class for the response of a write operation that involves a single doc */ -public abstract class DocWriteResponse extends ReplicationResponse implements StatusToXContent { +public abstract class DocWriteResponse extends ReplicationResponse implements WriteResponse, StatusToXContent { private ShardId shardId; private String id; private String type; private long version; + private boolean forcedRefresh; public DocWriteResponse(ShardId shardId, String type, String id, long version) { this.shardId = shardId; @@ -84,6 +90,20 @@ public abstract class DocWriteResponse extends ReplicationResponse implements St return this.version; } + /** + * Did this request force a refresh? Requests that set {@link WriteRequest#setRefreshPolicy(RefreshPolicy)} to + * {@link RefreshPolicy#IMMEDIATE} will always return true for this. Requests that set it to {@link RefreshPolicy#WAIT_UNTIL} will + * only return true here if they run out of refresh listener slots (see {@link IndexSettings#MAX_REFRESH_LISTENERS_PER_SHARD}). + */ + public boolean forcedRefresh() { + return forcedRefresh; + } + + @Override + public void setForcedRefresh(boolean forcedRefresh) { + this.forcedRefresh = forcedRefresh; + } + /** returns the rest status for this response (based on {@link ShardInfo#status()} */ public RestStatus status() { return getShardInfo().status(); @@ -97,6 +117,7 @@ public abstract class DocWriteResponse extends ReplicationResponse implements St type = in.readString(); id = in.readString(); version = in.readZLong(); + forcedRefresh = in.readBoolean(); } @Override @@ -106,6 +127,7 @@ public abstract class DocWriteResponse extends ReplicationResponse implements St out.writeString(type); out.writeString(id); out.writeZLong(version); + out.writeBoolean(forcedRefresh); } static final class Fields { @@ -121,7 +143,8 @@ public abstract class DocWriteResponse extends ReplicationResponse implements St builder.field(Fields._INDEX, shardId.getIndexName()) .field(Fields._TYPE, type) .field(Fields._ID, id) - .field(Fields._VERSION, version); + .field(Fields._VERSION, version) + .field("forced_refresh", forcedRefresh); shardInfo.toXContent(builder, params); return builder; } diff --git a/core/src/main/java/org/elasticsearch/action/admin/cluster/snapshots/restore/RestoreSnapshotRequest.java b/core/src/main/java/org/elasticsearch/action/admin/cluster/snapshots/restore/RestoreSnapshotRequest.java index bf4e4f148fc..1a41a776c73 100644 --- a/core/src/main/java/org/elasticsearch/action/admin/cluster/snapshots/restore/RestoreSnapshotRequest.java +++ b/core/src/main/java/org/elasticsearch/action/admin/cluster/snapshots/restore/RestoreSnapshotRequest.java @@ -57,7 +57,7 @@ public class RestoreSnapshotRequest extends MasterNodeRequest 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 diff --git a/core/src/main/java/org/elasticsearch/action/admin/indices/refresh/TransportRefreshAction.java b/core/src/main/java/org/elasticsearch/action/admin/indices/refresh/TransportRefreshAction.java index 34bf39daabd..ac64e276778 100644 --- a/core/src/main/java/org/elasticsearch/action/admin/indices/refresh/TransportRefreshAction.java +++ b/core/src/main/java/org/elasticsearch/action/admin/indices/refresh/TransportRefreshAction.java @@ -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; diff --git a/core/src/main/java/org/elasticsearch/action/admin/indices/refresh/TransportShardRefreshAction.java b/core/src/main/java/org/elasticsearch/action/admin/indices/refresh/TransportShardRefreshAction.java index 0670c1f3cc6..d7d0c289953 100644 --- a/core/src/main/java/org/elasticsearch/action/admin/indices/refresh/TransportShardRefreshAction.java +++ b/core/src/main/java/org/elasticsearch/action/admin/indices/refresh/TransportShardRefreshAction.java @@ -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 { +public class TransportShardRefreshAction + extends TransportReplicationAction { public static final String NAME = RefreshAction.NAME + "[s]"; @@ -47,8 +44,8 @@ public class TransportShardRefreshAction extends TransportReplicationAction 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 diff --git a/core/src/main/java/org/elasticsearch/action/admin/indices/shrink/ShrinkRequest.java b/core/src/main/java/org/elasticsearch/action/admin/indices/shrink/ShrinkRequest.java index 069c27ce475..a14a3f37a86 100644 --- a/core/src/main/java/org/elasticsearch/action/admin/indices/shrink/ShrinkRequest.java +++ b/core/src/main/java/org/elasticsearch/action/admin/indices/shrink/ShrinkRequest.java @@ -94,7 +94,7 @@ public class ShrinkRequest extends AcknowledgedRequest implements /** * Returns the {@link CreateIndexRequest} for the shrink index */ - public CreateIndexRequest getShrinkIndexReqeust() { + public CreateIndexRequest getShrinkIndexRequest() { return shrinkIndexRequest; } diff --git a/core/src/main/java/org/elasticsearch/action/admin/indices/shrink/ShrinkRequestBuilder.java b/core/src/main/java/org/elasticsearch/action/admin/indices/shrink/ShrinkRequestBuilder.java index a098215b750..ab392a7f824 100644 --- a/core/src/main/java/org/elasticsearch/action/admin/indices/shrink/ShrinkRequestBuilder.java +++ b/core/src/main/java/org/elasticsearch/action/admin/indices/shrink/ShrinkRequestBuilder.java @@ -41,7 +41,7 @@ public class ShrinkRequestBuilder extends AcknowledgedRequestBuilder { + IndexShardStats shard = indicesStatsResponse.getIndex(sourceIndex).getIndexShards().get(i); + return shard == null ? null : shard.getPrimary().getDocs(); + }, indexNameExpressionResolver); createIndexService.createIndex(updateRequest, new ActionListener() { @Override public void onResponse(ClusterStateUpdateResponse response) { @@ -127,24 +121,36 @@ public class TransportShrinkAction extends TransportMasterNodeAction 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 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) diff --git a/core/src/main/java/org/elasticsearch/action/bulk/BulkRequest.java b/core/src/main/java/org/elasticsearch/action/bulk/BulkRequest.java index a5775656475..85d7147ada0 100644 --- a/core/src/main/java/org/elasticsearch/action/bulk/BulkRequest.java +++ b/core/src/main/java/org/elasticsearch/action/bulk/BulkRequest.java @@ -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 implements CompositeIndicesRequest { +public class BulkRequest extends ActionRequest implements CompositeIndicesRequest, WriteRequest { 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> requests = new ArrayList<>(); List 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 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 true. Defaults - * to false. - */ - 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 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 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 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 implements Composite } request.writeTo(out); } - out.writeBoolean(refresh); + refreshPolicy.writeTo(out); timeout.writeTo(out); } } diff --git a/core/src/main/java/org/elasticsearch/action/bulk/BulkRequestBuilder.java b/core/src/main/java/org/elasticsearch/action/bulk/BulkRequestBuilder.java index 3744055d26c..4f2b7aa702e 100644 --- a/core/src/main/java/org/elasticsearch/action/bulk/BulkRequestBuilder.java +++ b/core/src/main/java/org/elasticsearch/action/bulk/BulkRequestBuilder.java @@ -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 { +public class BulkRequestBuilder extends ActionRequestBuilder + implements WriteRequestBuilder { public BulkRequestBuilder(ElasticsearchClient client, BulkAction action) { super(client, action, new BulkRequest()); @@ -116,16 +118,6 @@ public class BulkRequestBuilder extends ActionRequestBuildertrue. Defaults - * to false. - */ - 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 1m. */ diff --git a/core/src/main/java/org/elasticsearch/action/bulk/BulkShardRequest.java b/core/src/main/java/org/elasticsearch/action/bulk/BulkShardRequest.java index 874789e8d61..321b7e2a8e5 100644 --- a/core/src/main/java/org/elasticsearch/action/bulk/BulkShardRequest.java +++ b/core/src/main/java/org/elasticsearch/action/bulk/BulkShardRequest.java @@ -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 { +public class BulkShardRequest extends ReplicatedWriteRequest { 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 { out.writeBoolean(false); } } - out.writeBoolean(refresh); } @Override @@ -89,7 +82,6 @@ public class BulkShardRequest extends ReplicationRequest { items[i] = BulkItemRequest.readBulkItem(in); } } - refresh = in.readBoolean(); } @Override @@ -97,8 +89,15 @@ public class BulkShardRequest extends ReplicationRequest { // 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(); } diff --git a/core/src/main/java/org/elasticsearch/action/bulk/BulkShardResponse.java b/core/src/main/java/org/elasticsearch/action/bulk/BulkShardResponse.java index 76c80a9b064..22260181bb1 100644 --- a/core/src/main/java/org/elasticsearch/action/bulk/BulkShardResponse.java +++ b/core/src/main/java/org/elasticsearch/action/bulk/BulkShardResponse.java @@ -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); diff --git a/core/src/main/java/org/elasticsearch/action/bulk/TransportBulkAction.java b/core/src/main/java/org/elasticsearch/action/bulk/TransportBulkAction.java index 667e691f6c8..4cbebd0739a 100644 --- a/core/src/main/java/org/elasticsearch/action/bulk/TransportBulkAction.java +++ b/core/src/main/java/org/elasticsearch/action/bulk/TransportBulkAction.java @@ -344,7 +344,8 @@ public class TransportBulkAction extends HandledTransportAction> entry : requestsByShard.entrySet()) { final ShardId shardId = entry.getKey(); final List 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) { diff --git a/core/src/main/java/org/elasticsearch/action/bulk/TransportShardBulkAction.java b/core/src/main/java/org/elasticsearch/action/bulk/TransportShardBulkAction.java index a2f642374b7..4ad1136e668 100644 --- a/core/src/main/java/org/elasticsearch/action/bulk/TransportShardBulkAction.java +++ b/core/src/main/java/org/elasticsearch/action/bulk/TransportShardBulkAction.java @@ -30,7 +30,8 @@ import org.elasticsearch.action.index.IndexResponse; import org.elasticsearch.action.index.TransportIndexAction; import org.elasticsearch.action.support.ActionFilters; import org.elasticsearch.action.support.replication.ReplicationRequest; -import org.elasticsearch.action.support.replication.TransportReplicationAction; +import org.elasticsearch.action.support.replication.TransportWriteAction; +import org.elasticsearch.action.support.replication.ReplicationResponse.ShardInfo; import org.elasticsearch.action.update.UpdateHelper; import org.elasticsearch.action.update.UpdateRequest; import org.elasticsearch.action.update.UpdateResponse; @@ -53,6 +54,7 @@ import org.elasticsearch.index.engine.VersionConflictEngineException; import org.elasticsearch.index.shard.IndexShard; import org.elasticsearch.index.shard.ShardId; import org.elasticsearch.index.translog.Translog; +import org.elasticsearch.index.translog.Translog.Location; import org.elasticsearch.indices.IndicesService; import org.elasticsearch.rest.RestStatus; import org.elasticsearch.threadpool.ThreadPool; @@ -67,7 +69,7 @@ import static org.elasticsearch.action.support.replication.ReplicationOperation. /** * Performs the index operation. */ -public class TransportShardBulkAction extends TransportReplicationAction { +public class TransportShardBulkAction extends TransportWriteAction { 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 shardOperationOnPrimary(BulkShardRequest request) { + protected WriteResult 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(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 result = shardIndexOperation(request, indexRequest, metaData, indexShard, true); - location = locationToSync(location, result.location); + location = locationToSync(location, result.getLocation()); // add the response - IndexResponse indexResponse = result.response(); + IndexResponse indexResponse = result.getResponse(); setResponse(item, new BulkItemResponse(item.id(), indexRequest.opType().lowercase(), indexResponse)); } catch (Throwable e) { // rethrow the failure if we are going to retry on primary and let parent failure to handle it @@ -197,8 +197,8 @@ public class TransportShardBulkAction extends TransportReplicationAction writeResult = TransportDeleteAction.executeDeleteRequestOnPrimary(deleteRequest, indexShard); - DeleteResponse deleteResponse = writeResult.response(); - location = locationToSync(location, writeResult.location); + DeleteResponse deleteResponse = writeResult.getResponse(); + location = locationToSync(location, writeResult.getLocation()); setResponse(item, new BulkItemResponse(item.id(), OP_TYPE_DELETE, deleteResponse)); } catch (Throwable e) { // rethrow the failure if we are going to retry on primary and let parent failure to handle it @@ -237,16 +237,17 @@ public class TransportShardBulkAction extends TransportReplicationAction result = updateResult.writeResult; IndexRequest indexRequest = updateResult.request(); BytesReference indexSourceAsBytes = indexRequest.source(); // add the response - IndexResponse indexResponse = result.response(); + IndexResponse indexResponse = result.getResponse(); UpdateResponse updateResponse = new UpdateResponse(indexResponse.getShardInfo(), indexResponse.getShardId(), indexResponse.getType(), indexResponse.getId(), indexResponse.getVersion(), indexResponse.isCreated()); if (updateRequest.fields() != null && updateRequest.fields().length > 0) { Tuple> sourceAndContent = XContentHelper.convertToMap(indexSourceAsBytes, true); @@ -256,8 +257,9 @@ public class TransportShardBulkAction extends TransportReplicationAction writeResult = updateResult.writeResult; - DeleteResponse response = writeResult.response(); + DeleteResponse response = writeResult.getResponse(); DeleteRequest deleteRequest = updateResult.request(); updateResponse = new UpdateResponse(response.getShardInfo(), response.getShardId(), response.getType(), response.getId(), response.getVersion(), false); updateResponse.setGetResult(updateHelper.extractGetResult(updateRequest, request.index(), response.getVersion(), updateResult.result.updatedSourceAsMap(), updateResult.result.updateSourceContentType(), null)); @@ -326,11 +328,14 @@ public class TransportShardBulkAction extends TransportReplicationAction shardIndexOperation(BulkShardRequest request, IndexRequest indexRequest, IndexMetaData metaData, + IndexShard indexShard, boolean processed) throws Throwable { MappingMetaData mappingMd = metaData.mappingOrDefault(indexRequest.type()); if (!processed) { @@ -431,12 +436,8 @@ public class TransportShardBulkAction extends TransportReplicationAction implements DocumentRequest { +public class DeleteRequest extends ReplicatedWriteRequest implements DocumentRequest { private String type; private String id; @@ -51,7 +51,6 @@ public class DeleteRequest extends ReplicationRequest 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 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 true. Defaults - * to false. - */ - 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 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 implements out.writeString(id); out.writeOptionalString(routing()); out.writeOptionalString(parent()); - out.writeBoolean(refresh); out.writeLong(version); out.writeByte(versionType.getValue()); } diff --git a/core/src/main/java/org/elasticsearch/action/delete/DeleteRequestBuilder.java b/core/src/main/java/org/elasticsearch/action/delete/DeleteRequestBuilder.java index 0ce907bac1d..b9b0f95f8de 100644 --- a/core/src/main/java/org/elasticsearch/action/delete/DeleteRequestBuilder.java +++ b/core/src/main/java/org/elasticsearch/action/delete/DeleteRequestBuilder.java @@ -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 { +public class DeleteRequestBuilder extends ReplicationRequestBuilder + implements WriteRequestBuilder { public DeleteRequestBuilder(ElasticsearchClient client, DeleteAction action) { super(client, action, new DeleteRequest()); @@ -71,16 +73,6 @@ public class DeleteRequestBuilder extends ReplicationRequestBuildertrue. Defaults - * to false. - */ - 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. diff --git a/core/src/main/java/org/elasticsearch/action/delete/TransportDeleteAction.java b/core/src/main/java/org/elasticsearch/action/delete/TransportDeleteAction.java index 62d46766c47..beced23c338 100644 --- a/core/src/main/java/org/elasticsearch/action/delete/TransportDeleteAction.java +++ b/core/src/main/java/org/elasticsearch/action/delete/TransportDeleteAction.java @@ -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 { +public class TransportDeleteAction extends TransportWriteAction { private final AutoCreateIndex autoCreateIndex; private final TransportCreateIndexAction createIndexAction; @@ -60,9 +60,8 @@ public class TransportDeleteAction extends TransportReplicationAction shardOperationOnPrimary(DeleteRequest request) { - IndexShard indexShard = indicesService.indexServiceSafe(request.shardId().getIndex()).getShard(request.shardId().id()); - final WriteResult result = executeDeleteRequestOnPrimary(request, indexShard); - processAfterWrite(request.refresh(), indexShard, result.location); - return new Tuple<>(result.response, request); + protected WriteResult 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 executeDeleteRequestOnPrimary(DeleteRequest request, IndexShard indexShard) { @@ -134,9 +135,8 @@ public class TransportDeleteAction extends TransportReplicationAction( - new DeleteResponse(indexShard.shardId(), request.type(), request.id(), delete.version(), delete.found()), - delete.getTranslogLocation()); + DeleteResponse response = new DeleteResponse(indexShard.shardId(), request.type(), request.id(), delete.version(), delete.found()); + return new WriteResult<>(response, delete.getTranslogLocation()); } public static Engine.Delete executeDeleteRequestOnReplica(DeleteRequest request, IndexShard indexShard) { @@ -144,13 +144,4 @@ public class TransportDeleteAction extends TransportReplicationAction implements DocumentRequest { +public class IndexRequest extends ReplicatedWriteRequest implements DocumentRequest { /** * Operation type controls if the type of the index operation. @@ -145,7 +145,6 @@ public class IndexRequest extends ReplicationRequest 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 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 true. Defaults - * to false. - */ - 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 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 implements Do } out.writeBytesReference(source); out.writeByte(opType.id()); - out.writeBoolean(refresh); out.writeLong(version); out.writeByte(versionType.getValue()); out.writeOptionalString(pipeline); diff --git a/core/src/main/java/org/elasticsearch/action/index/IndexRequestBuilder.java b/core/src/main/java/org/elasticsearch/action/index/IndexRequestBuilder.java index 4116755e4eb..20587bf0ea9 100644 --- a/core/src/main/java/org/elasticsearch/action/index/IndexRequestBuilder.java +++ b/core/src/main/java/org/elasticsearch/action/index/IndexRequestBuilder.java @@ -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 { +public class IndexRequestBuilder extends ReplicationRequestBuilder + implements WriteRequestBuilder { public IndexRequestBuilder(ElasticsearchClient client, IndexAction action) { super(client, action, new IndexRequest()); @@ -220,16 +222,6 @@ public class IndexRequestBuilder extends ReplicationRequestBuildertrue. Defaults - * to false. - */ - 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. diff --git a/core/src/main/java/org/elasticsearch/action/index/TransportIndexAction.java b/core/src/main/java/org/elasticsearch/action/index/TransportIndexAction.java index 10e18c82b86..00be64757ae 100644 --- a/core/src/main/java/org/elasticsearch/action/index/TransportIndexAction.java +++ b/core/src/main/java/org/elasticsearch/action/index/TransportIndexAction.java @@ -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; *
  • allowIdGeneration: If the id is set not, should it be generated. Defaults to true. * */ -public class TransportIndexAction extends TransportReplicationAction { +public class TransportIndexAction extends TransportWriteAction { private final AutoCreateIndex autoCreateIndex; private final boolean allowIdGeneration; @@ -78,7 +76,7 @@ public class TransportIndexAction extends TransportReplicationAction shardOperationOnPrimary(IndexRequest request) throws Exception { - - IndexService indexService = indicesService.indexServiceSafe(request.shardId().getIndex()); - IndexShard indexShard = indexService.getShard(request.shardId().id()); - - final WriteResult 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 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 executeIndexRequestOnPrimary(IndexRequest request, IndexShard indexShard, MappingUpdatedAction mappingUpdatedAction) throws Exception { + public static WriteResult executeIndexRequestOnPrimary(IndexRequest request, IndexShard indexShard, + MappingUpdatedAction mappingUpdatedAction) throws Exception { Engine.Index operation = prepareIndexOperationOnPrimary(request, indexShard); Mapping update = operation.parsedDoc().dynamicMappingsUpdate(); final ShardId shardId = indexShard.shardId(); @@ -214,8 +196,8 @@ public class TransportIndexAction extends TransportReplicationAction(new IndexResponse(shardId, request.type(), request.id(), request.version(), created), operation.getTranslogLocation()); + IndexResponse response = new IndexResponse(shardId, request.type(), request.id(), request.version(), created); + return new WriteResult<>(response, operation.getTranslogLocation()); } - } diff --git a/core/src/main/java/org/elasticsearch/action/ingest/IngestActionFilter.java b/core/src/main/java/org/elasticsearch/action/ingest/IngestActionFilter.java index 1eb9337c814..850cac040dd 100644 --- a/core/src/main/java/org/elasticsearch/action/ingest/IngestActionFilter.java +++ b/core/src/main/java/org/elasticsearch/action/ingest/IngestActionFilter.java @@ -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()); diff --git a/core/src/main/java/org/elasticsearch/action/ingest/SimulateExecutionService.java b/core/src/main/java/org/elasticsearch/action/ingest/SimulateExecutionService.java index bc40a8368f0..2636dcb3ba8 100644 --- a/core/src/main/java/org/elasticsearch/action/ingest/SimulateExecutionService.java +++ b/core/src/main/java/org/elasticsearch/action/ingest/SimulateExecutionService.java @@ -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 { diff --git a/core/src/main/java/org/elasticsearch/ingest/processor/TrackingResultProcessor.java b/core/src/main/java/org/elasticsearch/action/ingest/TrackingResultProcessor.java similarity index 97% rename from core/src/main/java/org/elasticsearch/ingest/processor/TrackingResultProcessor.java rename to core/src/main/java/org/elasticsearch/action/ingest/TrackingResultProcessor.java index 861b0d68d2d..3693b5dcac7 100644 --- a/core/src/main/java/org/elasticsearch/ingest/processor/TrackingResultProcessor.java +++ b/core/src/main/java/org/elasticsearch/action/ingest/TrackingResultProcessor.java @@ -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; diff --git a/core/src/main/java/org/elasticsearch/action/search/SearchRequestBuilder.java b/core/src/main/java/org/elasticsearch/action/search/SearchRequestBuilder.java index c5666fa16a5..173c15ec1a5 100644 --- a/core/src/main/java/org/elasticsearch/action/search/SearchRequestBuilder.java +++ b/core/src/main/java/org/elasticsearch/action/search/SearchRequestBuilder.java @@ -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 ActionRequestBuilderfalse. diff --git a/core/src/main/java/org/elasticsearch/action/search/SearchResponse.java b/core/src/main/java/org/elasticsearch/action/search/SearchResponse.java index ab8395c72a1..b76e640fa95 100644 --- a/core/src/main/java/org/elasticsearch/action/search/SearchResponse.java +++ b/core/src/main/java/org/elasticsearch/action/search/SearchResponse.java @@ -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> getProfileResults() { return internalResponse.profile(); diff --git a/core/src/main/java/org/elasticsearch/action/support/WriteRequest.java b/core/src/main/java/org/elasticsearch/action/support/WriteRequest.java new file mode 100644 index 00000000000..6379a4fb259 --- /dev/null +++ b/core/src/main/java/org/elasticsearch/action/support/WriteRequest.java @@ -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> 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()); + } + } +} diff --git a/core/src/main/java/org/elasticsearch/action/support/WriteRequestBuilder.java b/core/src/main/java/org/elasticsearch/action/support/WriteRequestBuilder.java new file mode 100644 index 00000000000..eaaf90fd53b --- /dev/null +++ b/core/src/main/java/org/elasticsearch/action/support/WriteRequestBuilder.java @@ -0,0 +1,45 @@ +/* + * 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.support.WriteRequest.RefreshPolicy; + +public interface WriteRequestBuilder> { + 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; + } + + /** + * 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 B setRefreshPolicy(String refreshPolicy) { + request().setRefreshPolicy(refreshPolicy); + return (B) this; + } +} diff --git a/core/src/main/java/org/elasticsearch/action/support/WriteResponse.java b/core/src/main/java/org/elasticsearch/action/support/WriteResponse.java new file mode 100644 index 00000000000..07f5ea695d9 --- /dev/null +++ b/core/src/main/java/org/elasticsearch/action/support/WriteResponse.java @@ -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); +} diff --git a/core/src/main/java/org/elasticsearch/action/support/replication/ReplicatedWriteRequest.java b/core/src/main/java/org/elasticsearch/action/support/replication/ReplicatedWriteRequest.java new file mode 100644 index 00000000000..fa02dac9e1e --- /dev/null +++ b/core/src/main/java/org/elasticsearch/action/support/replication/ReplicatedWriteRequest.java @@ -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> extends ReplicationRequest implements WriteRequest { + 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); + } +} diff --git a/core/src/main/java/org/elasticsearch/action/support/replication/ReplicationOperation.java b/core/src/main/java/org/elasticsearch/action/support/replication/ReplicationOperation.java index 1f7313c1943..8442e705257 100644 --- a/core/src/main/java/org/elasticsearch/action/support/replication/ReplicationOperation.java +++ b/core/src/main/java/org/elasticsearch/action/support/replication/ReplicationOperation.java @@ -21,7 +21,6 @@ package org.elasticsearch.action.support.replication; import org.elasticsearch.ElasticsearchException; import org.elasticsearch.ExceptionsHelper; import org.elasticsearch.action.ActionListener; -import org.elasticsearch.action.ReplicationResponse; import org.elasticsearch.action.UnavailableShardsException; import org.elasticsearch.action.WriteConsistencyLevel; import org.elasticsearch.action.support.TransportActions; @@ -29,7 +28,6 @@ import org.elasticsearch.cluster.ClusterState; import org.elasticsearch.cluster.routing.IndexRoutingTable; import org.elasticsearch.cluster.routing.IndexShardRoutingTable; import org.elasticsearch.cluster.routing.ShardRouting; -import org.elasticsearch.common.collect.Tuple; import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.logging.ESLogger; import org.elasticsearch.index.engine.VersionConflictEngineException; @@ -47,28 +45,41 @@ import java.util.concurrent.atomic.AtomicInteger; import java.util.function.Consumer; import java.util.function.Supplier; -public class ReplicationOperation, ReplicaRequest extends ReplicationRequest, - Response extends ReplicationResponse> { +public class ReplicationOperation< + Request extends ReplicationRequest, + ReplicaRequest extends ReplicationRequest, + PrimaryResultT extends ReplicationOperation.PrimaryResult + > { final private ESLogger logger; final private Request request; final private Supplier 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: + *
      + *
    • The operation on the primary
    • + *
    • The operation on each replica
    • + *
    • 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.
    • + *
    + */ final private AtomicInteger pendingShards = new AtomicInteger(); final private AtomicInteger successfulShards = new AtomicInteger(); final private boolean executeOnReplicas; final private boolean checkWriteConsistency; - final private Primary primary; + final private Primary primary; final private Replicas replicasProxy; final private AtomicBoolean finished = new AtomicBoolean(); - final protected ActionListener finalResponseListener; + final protected ActionListener resultListener; - private volatile Response finalResponse = null; + private volatile PrimaryResultT primaryResult = null; private final List shardReplicaFailures = Collections.synchronizedList(new ArrayList<>()); - ReplicationOperation(Request request, Primary primary, - ActionListener listener, + ReplicationOperation(Request request, Primary primary, + ActionListener listener, boolean executeOnReplicas, boolean checkWriteConsistency, Replicas replicas, Supplier clusterStateSupplier, ESLogger logger, String opType) { @@ -76,7 +87,7 @@ public class ReplicationOperation, R this.executeOnReplicas = executeOnReplicas; this.replicasProxy = replicas; this.primary = primary; - this.finalResponseListener = listener; + this.resultListener = listener; this.logger = logger; this.request = request; this.clusterStateSupplier = clusterStateSupplier; @@ -85,28 +96,27 @@ public class ReplicationOperation, R void execute() throws Exception { final String writeConsistencyFailure = checkWriteConsistency ? checkWriteConsistency() : null; - final ShardId shardId = primary.routingEntry().shardId(); + final ShardRouting primaryRouting = primary.routingEntry(); + final ShardId primaryId = primaryRouting.shardId(); if (writeConsistencyFailure != null) { - finishAsFailed(new UnavailableShardsException(shardId, + finishAsFailed(new UnavailableShardsException(primaryId, "{} Timeout: [{}], request: [{}]", writeConsistencyFailure, request.timeout(), request)); return; } totalShards.incrementAndGet(); - pendingShards.incrementAndGet(); // increase by 1 until we finish all primary coordination - Tuple primaryResponse = primary.perform(request); - successfulShards.incrementAndGet(); // mark primary as successful - finalResponse = primaryResponse.v1(); - ReplicaRequest replicaRequest = primaryResponse.v2(); + pendingShards.incrementAndGet(); + primaryResult = primary.perform(request); + final ReplicaRequest replicaRequest = primaryResult.replicaRequest(); assert replicaRequest.primaryTerm() > 0 : "replicaRequest doesn't have a primary term"; if (logger.isTraceEnabled()) { - logger.trace("[{}] op [{}] completed on primary for request [{}]", shardId, opType, request); + logger.trace("[{}] op [{}] completed on primary for request [{}]", primaryId, opType, request); } // we have to get a new state after successfully indexing into the primary in order to honour recovery semantics. // we have to make sure that every operation indexed into the primary after recovery start will also be replicated // to the recovery target. If we use an old cluster state, we may miss a relocation that has started since then. // If the index gets deleted after primary operation, we skip replication - List shards = getShards(shardId, clusterStateSupplier.get()); + final List shards = getShards(primaryId, clusterStateSupplier.get()); final String localNodeId = primary.routingEntry().currentNodeId(); for (final ShardRouting shard : shards) { if (executeOnReplicas == false || shard.unassigned()) { @@ -125,8 +135,8 @@ public class ReplicationOperation, R } } - // decrement pending and finish (if there are no replicas, or those are done) - decPendingAndFinishIfNeeded(); // incremented in the beginning of this method + successfulShards.incrementAndGet(); + decPendingAndFinishIfNeeded(); } private void performOnReplica(final ShardRouting shard, final ReplicaRequest replicaRequest) { @@ -241,19 +251,19 @@ public class ReplicationOperation, R failuresArray = new ReplicationResponse.ShardInfo.Failure[shardReplicaFailures.size()]; shardReplicaFailures.toArray(failuresArray); } - finalResponse.setShardInfo(new ReplicationResponse.ShardInfo( + primaryResult.setShardInfo(new ReplicationResponse.ShardInfo( totalShards.get(), successfulShards.get(), failuresArray ) ); - finalResponseListener.onResponse(finalResponse); + resultListener.onResponse(primaryResult); } } private void finishAsFailed(Throwable throwable) { if (finished.compareAndSet(false, true)) { - finalResponseListener.onFailure(throwable); + resultListener.onFailure(throwable); } } @@ -284,22 +294,31 @@ public class ReplicationOperation, R } - interface Primary, ReplicaRequest extends ReplicationRequest, - Response extends ReplicationResponse> { + interface Primary< + Request extends ReplicationRequest, + ReplicaRequest extends ReplicationRequest, + PrimaryResultT extends PrimaryResult + > { - /** 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 perform(Request request) throws Exception; + PrimaryResultT perform(Request request) throws Exception; } @@ -308,19 +327,20 @@ public class ReplicationOperation, 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 listener); /** * Fail the specified shard, removing it from the current set of active shards - * @param replica shard to fail - * @param primary the primary shard that requested the failure - * @param message a (short) description of the reason - * @param throwable the original exception which caused the ReplicationOperation to request the shard to be failed - * @param onSuccess a callback to call when the shard has been successfully removed from the active set. + * + * @param replica shard to fail + * @param primary the primary shard that requested the failure + * @param message a (short) description of the reason + * @param throwable the original exception which caused the ReplicationOperation to request the shard to be failed + * @param onSuccess a callback to call when the shard has been successfully removed from the active set. * @param onPrimaryDemoted a callback to call when the shard can not be failed because the current primary has been demoted * by the master. * @param onIgnoredFailure a callback to call when failing a shard has failed, but it that failure can be safely ignored and the @@ -345,4 +365,11 @@ public class ReplicationOperation, R super(in); } } + + interface PrimaryResult> { + + R replicaRequest(); + + void setShardInfo(ReplicationResponse.ShardInfo shardInfo); + } } diff --git a/core/src/main/java/org/elasticsearch/action/support/replication/ReplicationRequest.java b/core/src/main/java/org/elasticsearch/action/support/replication/ReplicationRequest.java index 3e88575b717..44c420598b5 100644 --- a/core/src/main/java/org/elasticsearch/action/support/replication/ReplicationRequest.java +++ b/core/src/main/java/org/elasticsearch/action/support/replication/ReplicationRequest.java @@ -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> extends ActionRequest implements IndicesRequest { @@ -65,7 +68,6 @@ public abstract class ReplicationRequest, - ReplicaRequest extends ReplicationRequest, - Response extends ReplicationResponse> extends TransportAction { +public abstract class TransportReplicationAction< + Request extends ReplicationRequest, + ReplicaRequest extends ReplicationRequest, + Response extends ReplicationResponse + > extends TransportAction { final protected TransportService transportService; final protected ClusterService clusterService; @@ -149,17 +149,17 @@ public abstract class TransportReplicationAction shardOperationOnPrimary(Request shardRequest) throws Exception; + protected abstract PrimaryResult shardOperationOnPrimary(Request shardRequest) throws Exception; /** - * Replica operation on nodes with replica copies + * Synchronous replica operation on nodes with replica copies. This is done under the lock form + * {@link #acquireReplicaOperationLock(ShardId, long)}. */ - protected abstract void shardOperationOnReplica(ReplicaRequest shardRequest); + protected abstract ReplicaResult shardOperationOnReplica(ReplicaRequest shardRequest); /** * True if write consistency should be checked for an implementation @@ -198,26 +198,6 @@ public abstract class TransportReplicationAction { - - 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 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 { @Override public void messageReceived(final Request request, final TransportChannel channel, Task task) throws Exception { @@ -289,7 +269,17 @@ public abstract class TransportReplicationAction listener = createResponseListener(channel, replicationTask, primaryShardReference); - createReplicatedOperation(request, listener, primaryShardReference, executeOnReplicas).execute(); + createReplicatedOperation(request, new ActionListener() { + @Override + public void onResponse(PrimaryResult result) { + result.respond(listener); + } + + @Override + public void onFailure(Throwable e) { + listener.onFailure(e); + } + }, primaryShardReference, executeOnReplicas).execute(); success = true; } } finally { @@ -299,9 +289,9 @@ public abstract class TransportReplicationAction - createReplicatedOperation(Request request, ActionListener listener, - PrimaryShardReference primaryShardReference, boolean executeOnReplicas) { + protected ReplicationOperation createReplicatedOperation( + Request request, ActionListener listener, + PrimaryShardReference primaryShardReference, boolean executeOnReplicas) { return new ReplicationOperation<>(request, primaryShardReference, listener, executeOnReplicas, checkWriteConsistency(), replicasProxy, clusterService::state, logger, actionName ); @@ -339,6 +329,41 @@ public abstract class TransportReplicationAction { + 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 listener) { + listener.onResponse(finalResponse); + } + } + + protected class ReplicaResult { + /** + * Public constructor so subclasses can call it. + */ + public ReplicaResult() {} + + public void respond(ActionListener listener) { + listener.onResponse(TransportResponse.Empty.INSTANCE); + } + } + class ReplicaOperationTransportHandler implements TransportRequestHandler { @Override public void messageReceived(final ReplicaRequest request, final TransportChannel channel) throws Exception { @@ -426,15 +451,35 @@ public abstract class TransportReplicationAction { + @Override + public void onResponse(Empty response) { if (logger.isTraceEnabled()) { logger.trace("action [{}] completed on shard [{}] for request [{}]", transportReplicaAction, request.shardId(), - request); + request); + } + setPhase(task, "finished"); + try { + channel.sendResponse(response); + } catch (Exception e) { + onFailure(e); } } - setPhase(task, "finished"); - channel.sendResponse(TransportResponse.Empty.INSTANCE); + + @Override + public void onFailure(Throwable e) { + responseWithFailure(e); + } } } @@ -722,7 +767,7 @@ public abstract class TransportReplicationAction, Releasable { + class PrimaryShardReference implements ReplicationOperation.Primary, Releasable { private final IndexShard indexShard; private final Releasable operationLock; @@ -751,9 +796,9 @@ public abstract class TransportReplicationAction perform(Request request) throws Exception { - Tuple result = shardOperationOnPrimary(request); - result.v2().primaryTerm(indexShard.getPrimaryTerm()); + public PrimaryResult perform(Request request) throws Exception { + PrimaryResult result = shardOperationOnPrimary(request); + result.replicaRequest().primaryTerm(indexShard.getPrimaryTerm()); return result; } @@ -805,20 +850,6 @@ public abstract class TransportReplicationAction, + Response extends ReplicationResponse & WriteResponse + > extends TransportReplicationAction { + + protected TransportWriteAction(Settings settings, String actionName, TransportService transportService, + ClusterService clusterService, IndicesService indicesService, ThreadPool threadPool, ShardStateAction shardStateAction, + ActionFilters actionFilters, IndexNameExpressionResolver indexNameExpressionResolver, Supplier 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 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 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 { + 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 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 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 listener; + + public WriteReplicaResult(IndexShard indexShard, ReplicatedWriteRequest request, Translog.Location location) { + postWriteActions(indexShard, request, location, this, logger); + } + + @Override + public void respond(ActionListener 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); + } + } +} diff --git a/core/src/main/java/org/elasticsearch/action/update/TransportUpdateAction.java b/core/src/main/java/org/elasticsearch/action/update/TransportUpdateAction.java index 0363ef8fe43..ca55a63c1d6 100644 --- a/core/src/main/java/org/elasticsearch/action/update/TransportUpdateAction.java +++ b/core/src/main/java/org/elasticsearch/action/update/TransportUpdateAction.java @@ -26,6 +26,7 @@ import org.elasticsearch.action.RoutingMissingException; import org.elasticsearch.action.admin.indices.create.CreateIndexRequest; import org.elasticsearch.action.admin.indices.create.CreateIndexResponse; import org.elasticsearch.action.admin.indices.create.TransportCreateIndexAction; +import org.elasticsearch.action.delete.DeleteRequest; import org.elasticsearch.action.delete.DeleteResponse; import org.elasticsearch.action.delete.TransportDeleteAction; import org.elasticsearch.action.index.IndexRequest; @@ -187,6 +188,7 @@ public class TransportUpdateAction extends TransportInstanceSingleOperationActio } else { update.setGetResult(null); } + update.setForcedRefresh(response.forcedRefresh()); listener.onResponse(update); } @@ -219,6 +221,7 @@ public class TransportUpdateAction extends TransportInstanceSingleOperationActio public void onResponse(IndexResponse response) { UpdateResponse update = new UpdateResponse(response.getShardInfo(), response.getShardId(), response.getType(), response.getId(), response.getVersion(), response.isCreated()); update.setGetResult(updateHelper.extractGetResult(request, request.concreteIndex(), response.getVersion(), result.updatedSourceAsMap(), result.updateSourceContentType(), indexSourceBytes)); + update.setForcedRefresh(response.forcedRefresh()); listener.onResponse(update); } @@ -241,11 +244,13 @@ public class TransportUpdateAction extends TransportInstanceSingleOperationActio }); break; case DELETE: - deleteAction.execute(result.action(), new ActionListener() { + DeleteRequest deleteRequest = result.action(); + deleteAction.execute(deleteRequest, new ActionListener() { @Override public void onResponse(DeleteResponse response) { UpdateResponse update = new UpdateResponse(response.getShardInfo(), response.getShardId(), response.getType(), response.getId(), response.getVersion(), false); update.setGetResult(updateHelper.extractGetResult(request, request.concreteIndex(), response.getVersion(), result.updatedSourceAsMap(), result.updateSourceContentType(), null)); + update.setForcedRefresh(response.forcedRefresh()); listener.onResponse(update); } diff --git a/core/src/main/java/org/elasticsearch/action/update/UpdateHelper.java b/core/src/main/java/org/elasticsearch/action/update/UpdateHelper.java index 9ac77050202..0c9c1c67978 100644 --- a/core/src/main/java/org/elasticsearch/action/update/UpdateHelper.java +++ b/core/src/main/java/org/elasticsearch/action/update/UpdateHelper.java @@ -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); diff --git a/core/src/main/java/org/elasticsearch/action/update/UpdateRequest.java b/core/src/main/java/org/elasticsearch/action/update/UpdateRequest.java index 31f219fd4c7..e0846c1ce5d 100644 --- a/core/src/main/java/org/elasticsearch/action/update/UpdateRequest.java +++ b/core/src/main/java/org/elasticsearch/action/update/UpdateRequest.java @@ -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 implements DocumentRequest { +public class UpdateRequest extends InstanceShardOperationRequest + implements DocumentRequest, WriteRequest { private String type; private String id; @@ -72,7 +74,7 @@ public class UpdateRequest extends InstanceShardOperationRequest 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 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 true. Defaults - * to false. - */ - 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 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 script.writeTo(out); } out.writeVInt(retryOnConflict); - out.writeBoolean(refresh); + refreshPolicy.writeTo(out); if (doc == null) { out.writeBoolean(false); } else { diff --git a/core/src/main/java/org/elasticsearch/action/update/UpdateRequestBuilder.java b/core/src/main/java/org/elasticsearch/action/update/UpdateRequestBuilder.java index 30b636f4efc..403f4265fcd 100644 --- a/core/src/main/java/org/elasticsearch/action/update/UpdateRequestBuilder.java +++ b/core/src/main/java/org/elasticsearch/action/update/UpdateRequestBuilder.java @@ -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 { +public class UpdateRequestBuilder extends InstanceShardOperationRequestBuilder + implements WriteRequestBuilder { public UpdateRequestBuilder(ElasticsearchClient client, UpdateAction action) { super(client, action, new UpdateRequest()); @@ -121,17 +121,6 @@ public class UpdateRequestBuilder extends InstanceShardOperationRequestBuildertrue. Defaults - * to false. - */ - public UpdateRequestBuilder setRefresh(boolean refresh) { - request.refresh(refresh); - return this; - } - /** * Sets the consistency level of write. Defaults to {@link org.elasticsearch.action.WriteConsistencyLevel#DEFAULT} */ diff --git a/core/src/main/java/org/elasticsearch/bootstrap/BootstrapCheck.java b/core/src/main/java/org/elasticsearch/bootstrap/BootstrapCheck.java index 51ddb752b44..fc36e4fb671 100644 --- a/core/src/main/java/org/elasticsearch/bootstrap/BootstrapCheck.java +++ b/core/src/main/java/org/elasticsearch/bootstrap/BootstrapCheck.java @@ -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()); + } + + } + } diff --git a/core/src/main/java/org/elasticsearch/bootstrap/Security.java b/core/src/main/java/org/elasticsearch/bootstrap/Security.java index e1c6d320df6..c44b46c6b0c 100644 --- a/core/src/main/java/org/elasticsearch/bootstrap/Security.java +++ b/core/src/main/java/org/elasticsearch/bootstrap/Security.java @@ -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. diff --git a/core/src/main/java/org/elasticsearch/client/transport/TransportClientNodesService.java b/core/src/main/java/org/elasticsearch/client/transport/TransportClientNodesService.java index c3379c9ceaf..71c5895669c 100644 --- a/core/src/main/java/org/elasticsearch/client/transport/TransportClientNodesService.java +++ b/core/src/main/java/org/elasticsearch/client/transport/TransportClientNodesService.java @@ -211,7 +211,18 @@ public class TransportClientNodesService extends AbstractComponent { } public void execute(NodeListenerCallback callback, ActionListener listener) { - List 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 nodes = this.nodes; + if (closed) { + throw new IllegalStateException("transport client is closed"); + } ensureNodesAreAvailable(nodes); int index = getNodeNumber(); RetryListener retryListener = new RetryListener<>(callback, listener, nodes, index); diff --git a/core/src/main/java/org/elasticsearch/cluster/metadata/IndexMetaData.java b/core/src/main/java/org/elasticsearch/cluster/metadata/IndexMetaData.java index a87c887a4a1..14d00d3fc64 100644 --- a/core/src/main/java/org/elasticsearch/cluster/metadata/IndexMetaData.java +++ b/core/src/main/java/org/elasticsearch/cluster/metadata/IndexMetaData.java @@ -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, 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, 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, FromXContentBuild ImmutableOpenMap mappings, ImmutableOpenMap aliases, ImmutableOpenMap customs, ImmutableOpenIntMap> 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, 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, 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, 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, FromXContentBuild private static class IndexMetaDataDiff implements Diff { 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, 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, 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, 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, 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, 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, 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, FromXContentBuild private final ImmutableOpenMap.Builder aliases; private final ImmutableOpenMap.Builder customs; private final ImmutableOpenIntMap.Builder> activeAllocationIds; + private Integer routingNumShards; public Builder(String index) { this.index = index; @@ -703,6 +724,7 @@ public class IndexMetaData implements Diffable, 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, 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, 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, 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, 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 1. + * + * @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 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 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; + } } diff --git a/core/src/main/java/org/elasticsearch/cluster/metadata/MetaDataCreateIndexService.java b/core/src/main/java/org/elasticsearch/cluster/metadata/MetaDataCreateIndexService.java index 47528428928..2b482635879 100644 --- a/core/src/main/java/org/elasticsearch/cluster/metadata/MetaDataCreateIndexService.java +++ b/core/src/main/java/org/elasticsearch/cluster/metadata/MetaDataCreateIndexService.java @@ -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()); createdIndex = indexService.index(); @@ -339,7 +341,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); } @@ -494,14 +498,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 @@ -533,8 +539,6 @@ public class MetaDataCreateIndexService extends AbstractComponent { final Predicate 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", diff --git a/core/src/main/java/org/elasticsearch/cluster/metadata/MetaDataIndexTemplateService.java b/core/src/main/java/org/elasticsearch/cluster/metadata/MetaDataIndexTemplateService.java index 167dea5cf42..8f59ea3a527 100644 --- a/core/src/main/java/org/elasticsearch/cluster/metadata/MetaDataIndexTemplateService.java +++ b/core/src/main/java/org/elasticsearch/cluster/metadata/MetaDataIndexTemplateService.java @@ -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 indexSettingsValidation = metaDataCreateIndexService.getIndexSettingsValidationErrors(request.settings); diff --git a/core/src/main/java/org/elasticsearch/cluster/routing/OperationRouting.java b/core/src/main/java/org/elasticsearch/cluster/routing/OperationRouting.java index 3fb6f55a919..9b1c82e7c06 100644 --- a/core/src/main/java/org/elasticsearch/cluster/routing/OperationRouting.java +++ b/core/src/main/java/org/elasticsearch/cluster/routing/OperationRouting.java @@ -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) { diff --git a/core/src/main/java/org/elasticsearch/cluster/routing/allocation/decider/DiskThresholdDecider.java b/core/src/main/java/org/elasticsearch/cluster/routing/allocation/decider/DiskThresholdDecider.java index f64791d834a..ca341451fed 100644 --- a/core/src/main/java/org/elasticsearch/cluster/routing/allocation/decider/DiskThresholdDecider.java +++ b/core/src/main/java/org/elasticsearch/cluster/routing/allocation/decider/DiskThresholdDecider.java @@ -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 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 { diff --git a/core/src/main/java/org/elasticsearch/common/logging/LogConfigurator.java b/core/src/main/java/org/elasticsearch/common/logging/LogConfigurator.java index c49b0364e28..e203999d332 100644 --- a/core/src/main/java/org/elasticsearch/common/logging/LogConfigurator.java +++ b/core/src/main/java/org/elasticsearch/common/logging/LogConfigurator.java @@ -170,7 +170,7 @@ public class LogConfigurator { static void loadConfig(Path file, Settings.Builder settingsBuilder) { try { settingsBuilder.loadFromPath(file); - } catch (SettingsException | NoClassDefFoundError e) { + } catch (IOException | SettingsException | NoClassDefFoundError e) { // ignore } } diff --git a/core/src/main/java/org/elasticsearch/common/rounding/TimeZoneRounding.java b/core/src/main/java/org/elasticsearch/common/rounding/TimeZoneRounding.java index 5bb868ed253..c78d6c730b6 100644 --- a/core/src/main/java/org/elasticsearch/common/rounding/TimeZoneRounding.java +++ b/core/src/main/java/org/elasticsearch/common/rounding/TimeZoneRounding.java @@ -216,10 +216,9 @@ public abstract class TimeZoneRounding extends Rounding { @Override public long roundKey(long utcMillis) { - long timeLocal = utcMillis; - timeLocal = timeZone.convertUTCToLocal(utcMillis); + long timeLocal = timeZone.convertUTCToLocal(utcMillis); long rounded = Rounding.Interval.roundValue(Rounding.Interval.roundKey(timeLocal, interval), interval); - return timeZone.convertLocalToUTC(rounded, false); + return timeZone.convertLocalToUTC(rounded, false, utcMillis); } @Override diff --git a/core/src/main/java/org/elasticsearch/common/settings/ClusterSettings.java b/core/src/main/java/org/elasticsearch/common/settings/ClusterSettings.java index c9abafbbe54..6eb8df68242 100644 --- a/core/src/main/java/org/elasticsearch/common/settings/ClusterSettings.java +++ b/core/src/main/java/org/elasticsearch/common/settings/ClusterSettings.java @@ -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 ))); } diff --git a/core/src/main/java/org/elasticsearch/common/settings/IndexScopedSettings.java b/core/src/main/java/org/elasticsearch/common/settings/IndexScopedSettings.java index 027100b3469..b9d0c6b4c70 100644 --- a/core/src/main/java/org/elasticsearch/common/settings/IndexScopedSettings.java +++ b/core/src/main/java/org/elasticsearch/common/settings/IndexScopedSettings.java @@ -115,6 +115,7 @@ public final class IndexScopedSettings extends AbstractScopedSettings { IndexSettings.QUERY_STRING_LENIENT_SETTING, IndexSettings.ALLOW_UNMAPPED, IndexSettings.INDEX_CHECK_ON_STARTUP, + IndexSettings.MAX_REFRESH_LISTENERS_PER_SHARD, ShardsLimitAllocationDecider.INDEX_TOTAL_SHARDS_PER_NODE_SETTING, IndexSettings.INDEX_GC_DELETES_SETTING, IndicesRequestCache.INDEX_CACHE_REQUEST_ENABLED_SETTING, diff --git a/core/src/main/java/org/elasticsearch/common/property/PropertyPlaceholder.java b/core/src/main/java/org/elasticsearch/common/settings/PropertyPlaceholder.java similarity index 83% rename from core/src/main/java/org/elasticsearch/common/property/PropertyPlaceholder.java rename to core/src/main/java/org/elasticsearch/common/settings/PropertyPlaceholder.java index 70e6807cb92..2eb7e2b8e70 100644 --- a/core/src/main/java/org/elasticsearch/common/property/PropertyPlaceholder.java +++ b/core/src/main/java/org/elasticsearch/common/settings/PropertyPlaceholder.java @@ -17,7 +17,7 @@ * under the License. */ -package org.elasticsearch.common.property; +package org.elasticsearch.common.settings; import org.elasticsearch.common.Strings; @@ -34,23 +34,12 @@ import java.util.Set; * Values for substitution can be supplied using a {@link Properties} instance or using a * {@link PlaceholderResolver}. */ -public class PropertyPlaceholder { +class PropertyPlaceholder { private final String placeholderPrefix; private final String placeholderSuffix; private final boolean ignoreUnresolvablePlaceholders; - /** - * Creates a new PropertyPlaceholderHelper that uses the supplied prefix and suffix. Unresolvable - * placeholders are ignored. - * - * @param placeholderPrefix the prefix that denotes the start of a placeholder. - * @param placeholderSuffix the suffix that denotes the end of a placeholder. - */ - public PropertyPlaceholder(String placeholderPrefix, String placeholderSuffix) { - this(placeholderPrefix, placeholderSuffix, true); - } - /** * Creates a new PropertyPlaceholderHelper that uses the supplied prefix and suffix. * @@ -59,12 +48,10 @@ public class PropertyPlaceholder { * @param ignoreUnresolvablePlaceholders indicates whether unresolvable placeholders should be ignored * (true) or cause an exception (false). */ - public PropertyPlaceholder(String placeholderPrefix, String placeholderSuffix, + PropertyPlaceholder(String placeholderPrefix, String placeholderSuffix, boolean ignoreUnresolvablePlaceholders) { - Objects.requireNonNull(placeholderPrefix, "Argument 'placeholderPrefix' must not be null."); - Objects.requireNonNull(placeholderSuffix, "Argument 'placeholderSuffix' must not be null."); - this.placeholderPrefix = placeholderPrefix; - this.placeholderSuffix = placeholderSuffix; + this.placeholderPrefix = Objects.requireNonNull(placeholderPrefix); + this.placeholderSuffix = Objects.requireNonNull(placeholderSuffix); this.ignoreUnresolvablePlaceholders = ignoreUnresolvablePlaceholders; } @@ -75,15 +62,15 @@ public class PropertyPlaceholder { * @param value the value containing the placeholders to be replaced. * @param placeholderResolver the PlaceholderResolver to use for replacement. * @return the supplied value with placeholders replaced inline. + * @throws NullPointerException if value is null */ - public String replacePlaceholders(String key, String value, PlaceholderResolver placeholderResolver) { - Objects.requireNonNull(key); - Objects.requireNonNull(value, "value can not be null for [" + key + "]"); - return parseStringValue(value, placeholderResolver, new HashSet()); + String replacePlaceholders(String value, PlaceholderResolver placeholderResolver) { + Objects.requireNonNull(value); + return parseStringValue(value, placeholderResolver, new HashSet<>()); } - protected String parseStringValue(String strVal, PlaceholderResolver placeholderResolver, - Set visitedPlaceholders) { + private String parseStringValue(String strVal, PlaceholderResolver placeholderResolver, + Set visitedPlaceholders) { StringBuilder buf = new StringBuilder(strVal); int startIndex = strVal.indexOf(this.placeholderPrefix); @@ -164,7 +151,7 @@ public class PropertyPlaceholder { * * @see PropertyPlaceholder */ - public interface PlaceholderResolver { + interface PlaceholderResolver { /** * Resolves the supplied placeholder name into the replacement value. diff --git a/core/src/main/java/org/elasticsearch/common/settings/Settings.java b/core/src/main/java/org/elasticsearch/common/settings/Settings.java index 15554e5ccaa..9e5dd0efbe2 100644 --- a/core/src/main/java/org/elasticsearch/common/settings/Settings.java +++ b/core/src/main/java/org/elasticsearch/common/settings/Settings.java @@ -25,7 +25,6 @@ import org.elasticsearch.common.Strings; import org.elasticsearch.common.io.Streams; import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.io.stream.StreamOutput; -import org.elasticsearch.common.property.PropertyPlaceholder; import org.elasticsearch.common.settings.loader.SettingsLoader; import org.elasticsearch.common.settings.loader.SettingsLoaderFactory; import org.elasticsearch.common.unit.ByteSizeUnit; @@ -920,27 +919,21 @@ public final class Settings implements ToXContent { * Loads settings from a url that represents them using the * {@link SettingsLoaderFactory#loaderFromSource(String)}. */ - public Builder loadFromPath(Path path) throws SettingsException { - try { - return loadFromStream(path.getFileName().toString(), Files.newInputStream(path)); - } catch (IOException e) { - throw new SettingsException("Failed to open stream for url [" + path + "]", e); - } + public Builder loadFromPath(Path path) throws IOException { + // NOTE: loadFromStream will close the input stream + return loadFromStream(path.getFileName().toString(), Files.newInputStream(path)); } /** * Loads settings from a stream that represents them using the * {@link SettingsLoaderFactory#loaderFromSource(String)}. */ - public Builder loadFromStream(String resourceName, InputStream is) throws SettingsException { + public Builder loadFromStream(String resourceName, InputStream is) throws IOException { SettingsLoader settingsLoader = SettingsLoaderFactory.loaderFromResource(resourceName); - try { - Map loadedSettings = settingsLoader - .load(Streams.copyToString(new InputStreamReader(is, StandardCharsets.UTF_8))); - put(loadedSettings); - } catch (Exception e) { - throw new SettingsException("Failed to load settings from [" + resourceName + "]", e); - } + // NOTE: copyToString will close the input stream + Map loadedSettings = + settingsLoader.load(Streams.copyToString(new InputStreamReader(is, StandardCharsets.UTF_8))); + put(loadedSettings); return this; } @@ -992,14 +985,21 @@ public final class Settings implements ToXContent { return true; } }; - for (Map.Entry entry : new HashMap<>(map).entrySet()) { - String value = propertyPlaceholder.replacePlaceholders(entry.getKey(), entry.getValue(), placeholderResolver); + + Iterator> entryItr = map.entrySet().iterator(); + while (entryItr.hasNext()) { + Map.Entry entry = entryItr.next(); + if (entry.getValue() == null) { + // a null value obviously can't be replaced + continue; + } + String value = propertyPlaceholder.replacePlaceholders(entry.getValue(), placeholderResolver); // if the values exists and has length, we should maintain it in the map // otherwise, the replace process resolved into removing it if (Strings.hasLength(value)) { - map.put(entry.getKey(), value); + entry.setValue(value); } else { - map.remove(entry.getKey()); + entryItr.remove(); } } return this; diff --git a/core/src/main/java/org/elasticsearch/common/util/concurrent/EsExecutors.java b/core/src/main/java/org/elasticsearch/common/util/concurrent/EsExecutors.java index 2d45a6fecff..5ac94f8b386 100644 --- a/core/src/main/java/org/elasticsearch/common/util/concurrent/EsExecutors.java +++ b/core/src/main/java/org/elasticsearch/common/util/concurrent/EsExecutors.java @@ -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) { diff --git a/core/src/main/java/org/elasticsearch/common/xcontent/ConstructingObjectParser.java b/core/src/main/java/org/elasticsearch/common/xcontent/ConstructingObjectParser.java index 3c4d8875d2a..e1400463a72 100644 --- a/core/src/main/java/org/elasticsearch/common/xcontent/ConstructingObjectParser.java +++ b/core/src/main/java/org/elasticsearch/common/xcontent/ConstructingObjectParser.java @@ -26,7 +26,6 @@ import org.elasticsearch.common.xcontent.ObjectParser.ValueType; import java.io.IOException; import java.util.ArrayList; -import java.util.Arrays; import java.util.List; import java.util.function.BiConsumer; import java.util.function.BiFunction; @@ -53,12 +52,13 @@ import java.util.function.Function; *

    *
    {@code
      *   private static final ConstructingObjectParser PARSER = new ConstructingObjectParser<>("thing",
    - *           a -> new Thing((String) a[0], (String) a[1]));
    + *           a -> new Thing((String) a[0], (String) a[1], (Integer) a[2]));
      *   static {
      *       PARSER.declareString(constructorArg(), new ParseField("animal"));
      *       PARSER.declareString(constructorArg(), new ParseField("vegetable"));
    - *       PARSER.declareInt(Thing::setMineral, new ParseField("mineral"));
    + *       PARSER.declareInt(optionalConstructorArg(), new ParseField("mineral"));
      *       PARSER.declareInt(Thing::setFruit, new ParseField("fruit"));
    + *       PARSER.declareInt(Thing::setBug, new ParseField("bug"));
      *   }
      * }
    *

    @@ -70,19 +70,29 @@ import java.util.function.Function; * it allocates 3 + 2 * param_count objects per parse. If this overhead is too much for you then feel free to have ObjectParser * parse a secondary object and have that one call the target object's constructor. That ought to be rare though. *

    + *

    + * Note: if optional constructor arguments aren't specified then the number of allocations is always the worst case. + *

    */ public final class ConstructingObjectParser extends AbstractObjectParser { /** - * Consumer that marks a field as a constructor argument instead of a real object field. + * Consumer that marks a field as a required constructor argument instead of a real object field. */ - private static final BiConsumer CONSTRUCTOR_ARG_MARKER = (a, b) -> { + private static final BiConsumer REQUIRED_CONSTRUCTOR_ARG_MARKER = (a, b) -> { + throw new UnsupportedOperationException("I am just a marker I should never be called."); + }; + + /** + * Consumer that marks a field as an optional constructor argument instead of a real object field. + */ + private static final BiConsumer OPTIONAL_CONSTRUCTOR_ARG_MARKER = (a, b) -> { throw new UnsupportedOperationException("I am just a marker I should never be called."); }; /** * List of constructor names used for generating the error message if not all arrive. */ - private final List constructorArgNames = new ArrayList<>(); + private final List constructorArgInfos = new ArrayList<>(); private final ObjectParser objectParser; private final Function builder; /** @@ -120,27 +130,39 @@ public final class ConstructingObjectParser BiConsumer constructorArg() { - return (BiConsumer) CONSTRUCTOR_ARG_MARKER; + return (BiConsumer) REQUIRED_CONSTRUCTOR_ARG_MARKER; + } + + /** + * Pass the {@linkplain BiConsumer} this returns the declare methods to declare an optional constructor argument. See this class's + * javadoc for an example. The order in which these are declared matters: it is the order that they come in the array passed to + * {@link #builder} and the order that missing arguments are reported to the user if any are missing. When all of these parameters are + * parsed from the {@linkplain XContentParser} the target object is immediately built. + */ + @SuppressWarnings("unchecked") // Safe because we never call the method. This is just trickery to make the interface pretty. + public static BiConsumer optionalConstructorArg() { + return (BiConsumer) OPTIONAL_CONSTRUCTOR_ARG_MARKER; } @Override public void declareField(BiConsumer consumer, ContextParser parser, ParseField parseField, ValueType type) { - if (consumer == CONSTRUCTOR_ARG_MARKER) { + if (consumer == REQUIRED_CONSTRUCTOR_ARG_MARKER || consumer == OPTIONAL_CONSTRUCTOR_ARG_MARKER) { /* * Constructor arguments are detected by this "marker" consumer. It keeps the API looking clean even if it is a bit sleezy. We * then build a new consumer directly against the object parser that triggers the "constructor arg just arrived behavior" of the * parser. Conveniently, we can close over the position of the constructor in the argument list so we don't need to do any fancy * or expensive lookups whenever the constructor args come in. */ - int position = constructorArgNames.size(); - constructorArgNames.add(parseField); + int position = constructorArgInfos.size(); + boolean required = consumer == REQUIRED_CONSTRUCTOR_ARG_MARKER; + constructorArgInfos.add(new ConstructorArgInfo(parseField, required)); objectParser.declareField((target, v) -> target.constructorArg(position, parseField, v), parser, parseField, type); } else { numberOfFields += 1; @@ -186,7 +208,7 @@ public final class ConstructingObjectParser 0) { - queuedFieldsCount -= 1; - queuedFields[queuedFieldsCount].accept(targetObject); - } - } catch (ParsingException e) { - throw new ParsingException(e.getLineNumber(), e.getColumnNumber(), - "failed to build [" + objectParser.getName() + "] after last required field arrived", e); - } catch (Exception e) { - throw new ParsingException(null, "Failed to build [" + objectParser.getName() + "] after last required field arrived", e); + if (constructorArgsCollected == constructorArgInfos.size()) { + buildTarget(); } } @@ -263,36 +273,62 @@ public final class ConstructingObjectParser 0) { + queuedFieldsCount -= 1; + queuedFields[queuedFieldsCount].accept(targetObject); + } + } catch (ParsingException e) { + throw new ParsingException(e.getLineNumber(), e.getColumnNumber(), + "failed to build [" + objectParser.getName() + "] after last required field arrived", e); + } catch (Exception e) { + throw new ParsingException(null, "Failed to build [" + objectParser.getName() + "] after last required field arrived", e); } - if (message == null) { - throw new IllegalStateException("The targetObject wasn't built but we aren't missing any constructor args. This is a bug " - + " in ConstructingObjectParser. Here are the constructor arguments " + Arrays.toString(constructorArgs) - + " and here are is the count [" + constructorArgsCollected + "]. Good luck figuring out what happened." - + " I'm truly sorry you got here."); - } - throw new IllegalArgumentException(message.append(']').toString()); + } + } + + private static class ConstructorArgInfo { + final ParseField field; + final boolean required; + + public ConstructorArgInfo(ParseField field, boolean required) { + this.field = field; + this.required = required; } } } diff --git a/core/src/main/java/org/elasticsearch/env/Environment.java b/core/src/main/java/org/elasticsearch/env/Environment.java index c44db111a5a..6bec05fc82b 100644 --- a/core/src/main/java/org/elasticsearch/env/Environment.java +++ b/core/src/main/java/org/elasticsearch/env/Environment.java @@ -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; } diff --git a/core/src/main/java/org/elasticsearch/env/NodeEnvironment.java b/core/src/main/java/org/elasticsearch/env/NodeEnvironment.java index 98b47d92246..2b2aab2ccd2 100644 --- a/core/src/main/java/org/elasticsearch/env/NodeEnvironment.java +++ b/core/src/main/java/org/elasticsearch/env/NodeEnvironment.java @@ -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 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) { diff --git a/core/src/main/java/org/elasticsearch/index/IndexSettings.java b/core/src/main/java/org/elasticsearch/index/IndexSettings.java index 7c8cb4ff8c8..592c1ff1125 100644 --- a/core/src/main/java/org/elasticsearch/index/IndexSettings.java +++ b/core/src/main/java/org/elasticsearch/index/IndexSettings.java @@ -115,6 +115,11 @@ public final class IndexSettings { public static final Setting 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 MAX_REFRESH_LISTENERS_PER_SHARD = Setting.intSetting("index.max_refresh_listeners", 1000, 0, + Property.Dynamic, Property.IndexScope); private final Index index; private final Version version; @@ -145,6 +150,10 @@ public final class IndexSettings { private volatile int maxResultWindow; private volatile int maxRescoreWindow; private volatile boolean TTLPurgeDisabled; + /** + * The maximum number of refresh listeners allows on this shard. + */ + private volatile int maxRefreshListeners; /** * Returns the default search field for this index. @@ -229,6 +238,7 @@ public final class IndexSettings { maxResultWindow = scopedSettings.get(MAX_RESULT_WINDOW_SETTING); maxRescoreWindow = scopedSettings.get(MAX_RESCORE_WINDOW_SETTING); TTLPurgeDisabled = scopedSettings.get(INDEX_TTL_DISABLE_PURGE_SETTING); + maxRefreshListeners = scopedSettings.get(MAX_REFRESH_LISTENERS_PER_SHARD); this.mergePolicyConfig = new MergePolicyConfig(logger, this); assert indexNameMatcher.test(indexMetaData.getIndex().getName()); @@ -251,6 +261,7 @@ public final class IndexSettings { scopedSettings.addSettingsUpdateConsumer(INDEX_GC_DELETES_SETTING, this::setGCDeletes); scopedSettings.addSettingsUpdateConsumer(INDEX_TRANSLOG_FLUSH_THRESHOLD_SIZE_SETTING, this::setTranslogFlushThresholdSize); scopedSettings.addSettingsUpdateConsumer(INDEX_REFRESH_INTERVAL_SETTING, this::setRefreshInterval); + scopedSettings.addSettingsUpdateConsumer(MAX_REFRESH_LISTENERS_PER_SHARD, this::setMaxRefreshListeners); } private void setTranslogFlushThresholdSize(ByteSizeValue byteSizeValue) { @@ -499,6 +510,16 @@ public final class IndexSettings { return scopedSettings.get(setting); } + /** + * The maximum number of refresh listeners allows on this shard. + */ + public int getMaxRefreshListeners() { + return maxRefreshListeners; + } + + private void setMaxRefreshListeners(int maxRefreshListeners) { + this.maxRefreshListeners = maxRefreshListeners; + } IndexScopedSettings getScopedSettings() { return scopedSettings;} } diff --git a/core/src/main/java/org/elasticsearch/index/engine/Engine.java b/core/src/main/java/org/elasticsearch/index/engine/Engine.java index ab142f9dd51..87ffb9331a6 100644 --- a/core/src/main/java/org/elasticsearch/index/engine/Engine.java +++ b/core/src/main/java/org/elasticsearch/index/engine/Engine.java @@ -607,6 +607,7 @@ public abstract class Engine implements Closeable { * Synchronously refreshes the engine for new search operations to reflect the latest * changes. */ + @Nullable public abstract void refresh(String source) throws EngineException; /** @@ -999,6 +1000,9 @@ public abstract class Engine implements Closeable { public static final GetResult NOT_EXISTS = new GetResult(false, Versions.NOT_FOUND, null); + /** + * Build a realtime get result from the translog. + */ public GetResult(boolean exists, long version, @Nullable Translog.Source source) { this.source = source; this.exists = exists; @@ -1007,6 +1011,9 @@ public abstract class Engine implements Closeable { this.searcher = null; } + /** + * Build a non-realtime get result from the searcher. + */ public GetResult(Searcher searcher, Versions.DocIdAndVersion docIdAndVersion) { this.exists = true; this.source = null; diff --git a/core/src/main/java/org/elasticsearch/index/engine/EngineConfig.java b/core/src/main/java/org/elasticsearch/index/engine/EngineConfig.java index 8a56feff70f..13408408e7e 100644 --- a/core/src/main/java/org/elasticsearch/index/engine/EngineConfig.java +++ b/core/src/main/java/org/elasticsearch/index/engine/EngineConfig.java @@ -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; + } } diff --git a/core/src/main/java/org/elasticsearch/index/engine/InternalEngine.java b/core/src/main/java/org/elasticsearch/index/engine/InternalEngine.java index c120b07fce8..15667e79421 100644 --- a/core/src/main/java/org/elasticsearch/index/engine/InternalEngine.java +++ b/core/src/main/java/org/elasticsearch/index/engine/InternalEngine.java @@ -154,6 +154,10 @@ public class InternalEngine extends Engine { this.versionMap.setManager(searcherManager); // don't allow commits until we are done with recovering allowCommits.compareAndSet(true, openMode != EngineConfig.OpenMode.OPEN_INDEX_AND_TRANSLOG); + if (engineConfig.getRefreshListeners() != null) { + searcherManager.addListener(engineConfig.getRefreshListeners()); + engineConfig.getRefreshListeners().setTranslog(translog); + } success = true; } finally { if (success == false) { diff --git a/core/src/main/java/org/elasticsearch/index/engine/ShadowEngine.java b/core/src/main/java/org/elasticsearch/index/engine/ShadowEngine.java index c30b2e9bf50..0a55803a5ec 100644 --- a/core/src/main/java/org/elasticsearch/index/engine/ShadowEngine.java +++ b/core/src/main/java/org/elasticsearch/index/engine/ShadowEngine.java @@ -30,7 +30,6 @@ import org.elasticsearch.common.lucene.Lucene; import org.elasticsearch.common.lucene.index.ElasticsearchDirectoryReader; import org.elasticsearch.common.unit.TimeValue; import org.elasticsearch.common.util.concurrent.ReleasableLock; -import org.elasticsearch.index.shard.TranslogRecoveryPerformer; import org.elasticsearch.index.translog.Translog; import java.io.IOException; @@ -68,6 +67,9 @@ public class ShadowEngine extends Engine { public ShadowEngine(EngineConfig engineConfig) { super(engineConfig); + if (engineConfig.getRefreshListeners() != null) { + throw new IllegalArgumentException("ShadowEngine doesn't support RefreshListeners"); + } SearcherFactory searcherFactory = new EngineSearcherFactory(engineConfig); final long nonexistentRetryTime = engineConfig.getIndexSettings().getSettings() .getAsTime(NONEXISTENT_INDEX_RETRY_WAIT, DEFAULT_NONEXISTENT_INDEX_RETRY_WAIT) diff --git a/core/src/main/java/org/elasticsearch/index/query/QueryBuilders.java b/core/src/main/java/org/elasticsearch/index/query/QueryBuilders.java index d048f73381e..96585ce810f 100644 --- a/core/src/main/java/org/elasticsearch/index/query/QueryBuilders.java +++ b/core/src/main/java/org/elasticsearch/index/query/QueryBuilders.java @@ -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); } /** diff --git a/core/src/main/java/org/elasticsearch/index/query/WrapperQueryBuilder.java b/core/src/main/java/org/elasticsearch/index/query/WrapperQueryBuilder.java index 4e1eb83272a..7e469d6bbe4 100644 --- a/core/src/main/java/org/elasticsearch/index/query/WrapperQueryBuilder.java +++ b/core/src/main/java/org/elasticsearch/index/query/WrapperQueryBuilder.java @@ -126,7 +126,7 @@ public class WrapperQueryBuilder extends AbstractQueryBuilder { try { - if (recoverFromLocalShards(mappingUpdateConsumer, startedShards)) { + final Set 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) { @@ -1530,7 +1544,7 @@ public class IndexShard extends AbstractIndexShardComponent { return new EngineConfig(openMode, shardId, threadPool, indexSettings, warmer, store, deletionPolicy, indexSettings.getMergePolicy(), mapperService.indexAnalyzer(), similarityService.similarity(mapperService), codecService, shardEventListener, translogRecoveryPerformer, indexCache.query(), cachingPolicy, translogConfig, - IndexingMemoryController.SHARD_INACTIVE_TIME_SETTING.get(indexSettings.getSettings())); + IndexingMemoryController.SHARD_INACTIVE_TIME_SETTING.get(indexSettings.getSettings()), refreshListeners); } public Releasable acquirePrimaryOperationLock() { @@ -1626,6 +1640,17 @@ public class IndexShard extends AbstractIndexShardComponent { return false; } + /** + * Build {@linkplain RefreshListeners} for this shard. Protected so {@linkplain ShadowIndexShard} can override it to return null. + */ + protected RefreshListeners buildRefreshListeners() { + return new RefreshListeners( + indexSettings::getMaxRefreshListeners, + () -> refresh("too_many_listeners"), + threadPool.executor(ThreadPool.Names.LISTENER)::execute, + logger); + } + /** * Simple struct encapsulating a shard failure * @@ -1651,14 +1676,26 @@ public class IndexShard extends AbstractIndexShardComponent { } /** - * Returns true iff one or more changes to the engine are not visible to via the current searcher. + * Returns true iff one or more changes to the engine are not visible to via the current searcher *or* there are pending + * refresh listeners. * Otherwise false. * * @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 listener) { + refreshListeners.addOrNotify(location, listener); } private class IndexShardRecoveryPerformer extends TranslogRecoveryPerformer { diff --git a/core/src/main/java/org/elasticsearch/index/shard/RefreshListeners.java b/core/src/main/java/org/elasticsearch/index/shard/RefreshListeners.java new file mode 100644 index 00000000000..ab3e334714a --- /dev/null +++ b/core/src/main/java/org/elasticsearch/index/shard/RefreshListeners.java @@ -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>> 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 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>> 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> listenersToFire = null; + List>> preservedListeners = null; + for (Tuple> tuple : candidates) { + Translog.Location location = tuple.v1(); + Consumer 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> finalListenersToFire = listenersToFire; + listenerExecutor.execute(() -> { + for (Consumer listener : finalListenersToFire) { + try { + listener.accept(false); + } catch (Throwable t) { + logger.warn("Error firing refresh listener", t); + } + } + }); + } + } +} diff --git a/core/src/main/java/org/elasticsearch/index/shard/ShadowIndexShard.java b/core/src/main/java/org/elasticsearch/index/shard/ShadowIndexShard.java index bf35d02fea2..e22f684637e 100644 --- a/core/src/main/java/org/elasticsearch/index/shard/ShadowIndexShard.java +++ b/core/src/main/java/org/elasticsearch/index/shard/ShadowIndexShard.java @@ -31,12 +31,14 @@ import org.elasticsearch.index.mapper.MapperService; import org.elasticsearch.index.merge.MergeStats; import org.elasticsearch.index.similarity.SimilarityService; import org.elasticsearch.index.store.Store; +import org.elasticsearch.index.translog.Translog; import org.elasticsearch.index.translog.TranslogStats; import org.elasticsearch.threadpool.ThreadPool; import java.io.IOException; import java.util.Collections; import java.util.List; +import java.util.function.Consumer; /** * ShadowIndexShard extends {@link IndexShard} to add file synchronization @@ -86,6 +88,12 @@ public final class ShadowIndexShard extends IndexShard { return engineFactory.newReadOnlyEngine(config); } + @Override + protected RefreshListeners buildRefreshListeners() { + // ShadowEngine doesn't have a translog so it shouldn't try to support RefreshListeners. + return null; + } + @Override public boolean shouldFlush() { // we don't need to flush since we don't write - all dominated by the primary @@ -96,4 +104,9 @@ public final class ShadowIndexShard extends IndexShard { public TranslogStats translogStats() { return null; // shadow engine has no translog } + + @Override + public void addRefreshListener(Translog.Location location, Consumer listener) { + throw new UnsupportedOperationException("Can't listen for a refresh on a shadow engine because it doesn't have a translog"); + } } diff --git a/core/src/main/java/org/elasticsearch/index/translog/Translog.java b/core/src/main/java/org/elasticsearch/index/translog/Translog.java index b66c82d4083..57847972e42 100644 --- a/core/src/main/java/org/elasticsearch/index/translog/Translog.java +++ b/core/src/main/java/org/elasticsearch/index/translog/Translog.java @@ -447,6 +447,21 @@ public class Translog extends AbstractIndexShardComponent implements IndexShardC } } + /** + * The a {@linkplain Location} that will sort after the {@linkplain Location} returned by the last write but before any locations which + * can be returned by the next write. + */ + public Location getLastWriteLocation() { + try (ReleasableLock lock = readLock.acquire()) { + /* + * We use position = current - 1 and size = Integer.MAX_VALUE here instead of position current and size = 0 for two reasons: + * 1. Translog.Location's compareTo doesn't actually pay attention to size even though it's equals method does. + * 2. It feels more right to return a *position* that is before the next write's position rather than rely on the size. + */ + return new Location(current.generation, current.sizeInBytes() - 1, Integer.MAX_VALUE); + } + } + boolean assertBytesAtLocation(Translog.Location location, BytesReference expectedBytes) throws IOException { // tests can override this ByteBuffer buffer = ByteBuffer.allocate(location.size); diff --git a/core/src/main/java/org/elasticsearch/index/translog/TranslogWriter.java b/core/src/main/java/org/elasticsearch/index/translog/TranslogWriter.java index 0109995f80f..b2c0cc88cf9 100644 --- a/core/src/main/java/org/elasticsearch/index/translog/TranslogWriter.java +++ b/core/src/main/java/org/elasticsearch/index/translog/TranslogWriter.java @@ -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; diff --git a/core/src/main/java/org/elasticsearch/indices/IndicesService.java b/core/src/main/java/org/elasticsearch/indices/IndicesService.java index 188e3608cb3..ba512379868 100644 --- a/core/src/main/java/org/elasticsearch/indices/IndicesService.java +++ b/core/src/main/java/org/elasticsearch/indices/IndicesService.java @@ -112,6 +112,7 @@ import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.atomic.AtomicInteger; import java.util.function.Predicate; import java.util.stream.Collectors; @@ -141,6 +142,7 @@ public class IndicesService extends AbstractLifecycleComponent i private final CircuitBreakerService circuitBreakerService; private volatile Map indices = emptyMap(); private final Map> pendingDeletes = new HashMap<>(); + private final AtomicInteger numUncompletedDeletes = new AtomicInteger(); private final OldShardsStats oldShardsStats = new OldShardsStats(); private final IndexStoreConfig indexStoreConfig; private final MapperRegistry mapperRegistry; @@ -782,6 +784,7 @@ public class IndicesService extends AbstractLifecycleComponent i pendingDeletes.put(index, list); } list.add(pendingDelete); + numUncompletedDeletes.incrementAndGet(); } } @@ -840,6 +843,7 @@ public class IndicesService extends AbstractLifecycleComponent i logger.debug("{} processing pending deletes", index); final long startTimeNS = System.nanoTime(); final List shardLocks = nodeEnv.lockAllForIndex(index, indexSettings, timeout.millis()); + int numRemoved = 0; try { Map locks = new HashMap<>(); for (ShardLock lock : shardLocks) { @@ -850,6 +854,7 @@ public class IndicesService extends AbstractLifecycleComponent i remove = pendingDeletes.remove(index); } if (remove != null && remove.isEmpty() == false) { + numRemoved = remove.size(); CollectionUtil.timSort(remove); // make sure we delete indices first final long maxSleepTimeMs = 10 * 1000; // ensure we retry after 10 sec long sleepTime = 10; @@ -896,6 +901,10 @@ public class IndicesService extends AbstractLifecycleComponent i } } finally { IOUtils.close(shardLocks); + if (numRemoved > 0) { + int remainingUncompletedDeletes = numUncompletedDeletes.addAndGet(-numRemoved); + assert remainingUncompletedDeletes >= 0; + } } } @@ -909,6 +918,14 @@ public class IndicesService extends AbstractLifecycleComponent i } } + /** + * Checks if all pending deletes have completed. Used by tests to ensure we don't check directory contents while deletion still ongoing. + * The reason is that, on Windows, browsing the directory contents can interfere with the deletion process and delay it unnecessarily. + */ + public boolean hasUncompletedPendingDeletes() { + return numUncompletedDeletes.get() > 0; + } + /** * Returns this nodes {@link IndicesQueriesRegistry} */ diff --git a/core/src/main/java/org/elasticsearch/indices/analysis/HunspellService.java b/core/src/main/java/org/elasticsearch/indices/analysis/HunspellService.java index d2da2a5d672..27b7ab23c9f 100644 --- a/core/src/main/java/org/elasticsearch/indices/analysis/HunspellService.java +++ b/core/src/main/java/org/elasticsearch/indices/analysis/HunspellService.java @@ -210,7 +210,7 @@ public class HunspellService extends AbstractComponent { * @param defaults The default settings for this dictionary * @return The resolved settings. */ - private static Settings loadDictionarySettings(Path dir, Settings defaults) { + private static Settings loadDictionarySettings(Path dir, Settings defaults) throws IOException { Path file = dir.resolve("settings.yml"); if (Files.exists(file)) { return Settings.builder().loadFromPath(file).put(defaults).build(); diff --git a/core/src/main/java/org/elasticsearch/indices/cluster/IndicesClusterStateService.java b/core/src/main/java/org/elasticsearch/indices/cluster/IndicesClusterStateService.java index 719fb812c74..2c77f863c47 100644 --- a/core/src/main/java/org/elasticsearch/indices/cluster/IndicesClusterStateService.java +++ b/core/src/main/java/org/elasticsearch/indices/cluster/IndicesClusterStateService.java @@ -20,6 +20,7 @@ package org.elasticsearch.indices.cluster; import com.carrotsearch.hppc.cursors.ObjectCursor; +import org.apache.lucene.store.LockObtainFailedException; import org.elasticsearch.ElasticsearchException; import org.elasticsearch.cluster.ClusterChangedEvent; import org.elasticsearch.cluster.ClusterState; @@ -41,11 +42,14 @@ import org.elasticsearch.common.inject.Inject; import org.elasticsearch.common.logging.ESLogger; import org.elasticsearch.common.lucene.Lucene; import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.common.unit.TimeValue; import org.elasticsearch.common.util.Callback; +import org.elasticsearch.common.util.concurrent.AbstractRunnable; import org.elasticsearch.common.util.concurrent.ConcurrentCollections; import org.elasticsearch.gateway.GatewayService; import org.elasticsearch.index.Index; import org.elasticsearch.index.IndexService; +import org.elasticsearch.index.IndexSettings; import org.elasticsearch.index.IndexShardAlreadyExistsException; import org.elasticsearch.index.NodeServicesProvider; import org.elasticsearch.index.mapper.DocumentMapper; @@ -69,12 +73,16 @@ import org.elasticsearch.threadpool.ThreadPool; import java.io.IOException; import java.util.ArrayList; import java.util.Arrays; +import java.util.Collections; +import java.util.HashMap; import java.util.HashSet; import java.util.Iterator; import java.util.List; import java.util.Map; import java.util.Set; import java.util.concurrent.ConcurrentMap; +import java.util.concurrent.TimeUnit; +import java.util.stream.Collectors; /** * @@ -213,11 +221,14 @@ public class IndicesClusterStateService extends AbstractLifecycleComponent newShardAllocationIds = new HashSet<>(); + + final Map> shardsByIndex = new HashMap<>(); + for (ShardRouting shard : routingNode) { + shardsByIndex.computeIfAbsent(shard.index(), k -> new HashSet<>()).add(shard.allocationId().getId()); + } + for (IndexService indexService : indicesService) { Index index = indexService.index(); IndexMetaData indexMetaData = event.state().metaData().index(index); assert indexMetaData != null : "local index doesn't have metadata, should have been cleaned up by applyDeletedIndices: " + index; // now, go over and delete shards that needs to get deleted - newShardAllocationIds.clear(); - for (ShardRouting shard : routingNode) { - if (shard.index().equals(index)) { - // use the allocation id and not object so we won't be influence by relocation targets - newShardAllocationIds.add(shard.allocationId().getId()); - } - } + Set newShardAllocationIds = shardsByIndex.getOrDefault(index, Collections.emptySet()); for (IndexShard existingShard : indexService) { if (newShardAllocationIds.contains(existingShard.routingEntry().allocationId().getId()) == false) { if (indexMetaData.getState() == IndexMetaData.State.CLOSE) { diff --git a/core/src/main/java/org/elasticsearch/monitor/jvm/JvmInfo.java b/core/src/main/java/org/elasticsearch/monitor/jvm/JvmInfo.java index 3e39edaa47c..69abd0752f6 100644 --- a/core/src/main/java/org/elasticsearch/monitor/jvm/JvmInfo.java +++ b/core/src/main/java/org/elasticsearch/monitor/jvm/JvmInfo.java @@ -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 diff --git a/core/src/main/java/org/elasticsearch/node/Node.java b/core/src/main/java/org/elasticsearch/node/Node.java index cf33770fd16..04063ce5864 100644 --- a/core/src/main/java/org/elasticsearch/node/Node.java +++ b/core/src/main/java/org/elasticsearch/node/Node.java @@ -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> 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); diff --git a/core/src/main/java/org/elasticsearch/node/NodeModule.java b/core/src/main/java/org/elasticsearch/node/NodeModule.java index 08b60970229..a9696593ec4 100644 --- a/core/src/main/java/org/elasticsearch/node/NodeModule.java +++ b/core/src/main/java/org/elasticsearch/node/NodeModule.java @@ -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 diff --git a/core/src/main/java/org/elasticsearch/node/internal/InternalSettingsPreparer.java b/core/src/main/java/org/elasticsearch/node/internal/InternalSettingsPreparer.java index 24a9cf589bd..7a1c6321c43 100644 --- a/core/src/main/java/org/elasticsearch/node/internal/InternalSettingsPreparer.java +++ b/core/src/main/java/org/elasticsearch/node/internal/InternalSettingsPreparer.java @@ -103,7 +103,11 @@ public class InternalSettingsPreparer { Path path = environment.configFile().resolve("elasticsearch" + allowedSuffix); if (Files.exists(path)) { if (!settingsFileFound) { - output.loadFromPath(path); + try { + output.loadFromPath(path); + } catch (IOException e) { + throw new SettingsException("Failed to settings from " + path.toString(), e); + } } settingsFileFound = true; foundSuffixes.add(allowedSuffix); diff --git a/core/src/main/java/org/elasticsearch/plugins/Plugin.java b/core/src/main/java/org/elasticsearch/plugins/Plugin.java index 1efc151836d..0283567bf80 100644 --- a/core/src/main/java/org/elasticsearch/plugins/Plugin.java +++ b/core/src/main/java/org/elasticsearch/plugins/Plugin.java @@ -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> getExecutorBuilders(Settings settings) { + return Collections.emptyList(); + } } diff --git a/core/src/main/java/org/elasticsearch/plugins/PluginsService.java b/core/src/main/java/org/elasticsearch/plugins/PluginsService.java index f373da6987d..4f123625387 100644 --- a/core/src/main/java/org/elasticsearch/plugins/PluginsService.java +++ b/core/src/main/java/org/elasticsearch/plugins/PluginsService.java @@ -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> getExecutorBuilders(Settings settings) { + final ArrayList> builders = new ArrayList<>(); + for (final Tuple plugin : plugins) { + builders.addAll(plugin.v2().getExecutorBuilders(settings)); + } + return builders; + } + public Collection> nodeServices() { List> services = new ArrayList<>(); for (Tuple plugin : plugins) { diff --git a/core/src/main/java/org/elasticsearch/rest/action/admin/indices/RestShrinkIndexAction.java b/core/src/main/java/org/elasticsearch/rest/action/admin/indices/RestShrinkIndexAction.java index b5cc861b65e..42117cbf328 100644 --- a/core/src/main/java/org/elasticsearch/rest/action/admin/indices/RestShrinkIndexAction.java +++ b/core/src/main/java/org/elasticsearch/rest/action/admin/indices/RestShrinkIndexAction.java @@ -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())); diff --git a/core/src/main/java/org/elasticsearch/rest/action/bulk/RestBulkAction.java b/core/src/main/java/org/elasticsearch/rest/action/bulk/RestBulkAction.java index 620418eb087..d9dbb21e804 100644 --- a/core/src/main/java/org/elasticsearch/rest/action/bulk/RestBulkAction.java +++ b/core/src/main/java/org/elasticsearch/rest/action/bulk/RestBulkAction.java @@ -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(channel) { diff --git a/core/src/main/java/org/elasticsearch/rest/action/delete/RestDeleteAction.java b/core/src/main/java/org/elasticsearch/rest/action/delete/RestDeleteAction.java index 8e3449344c4..29316893504 100644 --- a/core/src/main/java/org/elasticsearch/rest/action/delete/RestDeleteAction.java +++ b/core/src/main/java/org/elasticsearch/rest/action/delete/RestDeleteAction.java @@ -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())); diff --git a/core/src/main/java/org/elasticsearch/rest/action/index/RestIndexAction.java b/core/src/main/java/org/elasticsearch/rest/action/index/RestIndexAction.java index 26dd1eca78d..f807e68088a 100644 --- a/core/src/main/java/org/elasticsearch/rest/action/index/RestIndexAction.java +++ b/core/src/main/java/org/elasticsearch/rest/action/index/RestIndexAction.java @@ -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"); diff --git a/core/src/main/java/org/elasticsearch/rest/action/search/RestMultiSearchAction.java b/core/src/main/java/org/elasticsearch/rest/action/search/RestMultiSearchAction.java index 879a1e750b6..8c76525c857 100644 --- a/core/src/main/java/org/elasticsearch/rest/action/search/RestMultiSearchAction.java +++ b/core/src/main/java/org/elasticsearch/rest/action/search/RestMultiSearchAction.java @@ -157,7 +157,7 @@ public class RestMultiSearchAction extends BaseRestHandler { Object value = entry.getValue(); if ("index".equals(entry.getKey()) || "indices".equals(entry.getKey())) { if (!allowExplicitIndex) { - throw new IllegalArgumentException("explicit index in multi percolate is not allowed"); + throw new IllegalArgumentException("explicit index in multi search is not allowed"); } searchRequest.indices(nodeStringArrayValue(value)); } else if ("type".equals(entry.getKey()) || "types".equals(entry.getKey())) { diff --git a/core/src/main/java/org/elasticsearch/rest/action/update/RestUpdateAction.java b/core/src/main/java/org/elasticsearch/rest/action/update/RestUpdateAction.java index 88f90374523..bdea4e33e6d 100644 --- a/core/src/main/java/org/elasticsearch/rest/action/update/RestUpdateAction.java +++ b/core/src/main/java/org/elasticsearch/rest/action/update/RestUpdateAction.java @@ -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)); diff --git a/core/src/main/java/org/elasticsearch/search/SearchService.java b/core/src/main/java/org/elasticsearch/search/SearchService.java index 2f0ea96c6fc..57841466a62 100644 --- a/core/src/main/java/org/elasticsearch/search/SearchService.java +++ b/core/src/main/java/org/elasticsearch/search/SearchService.java @@ -546,14 +546,14 @@ public class SearchService extends AbstractLifecycleComponent imp indexShard, scriptService, bigArrays, threadPool.estimatedTimeInMillisCounter(), parseFieldMatcher, defaultSearchTimeout, fetchPhase); SearchContext.setCurrent(context); - request.rewrite(context.getQueryShardContext()); - // reset that we have used nowInMillis from the context since it may - // have been rewritten so its no longer in the query and the request can - // be cached. If it is still present in the request (e.g. in a range - // aggregation) it will still be caught when the aggregation is - // evaluated. - context.resetNowInMillisUsed(); try { + request.rewrite(context.getQueryShardContext()); + // reset that we have used nowInMillis from the context since it may + // have been rewritten so its no longer in the query and the request can + // be cached. If it is still present in the request (e.g. in a range + // aggregation) it will still be caught when the aggregation is + // evaluated. + context.resetNowInMillisUsed(); if (request.scroll() != null) { context.scrollContext(new ScrollContext()); context.scrollContext().scroll = request.scroll(); @@ -821,6 +821,15 @@ public class SearchService extends AbstractLifecycleComponent 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]; diff --git a/core/src/main/java/org/elasticsearch/search/builder/SearchSourceBuilder.java b/core/src/main/java/org/elasticsearch/search/builder/SearchSourceBuilder.java index 8c4be225127..b1a5d2d03e6 100644 --- a/core/src/main/java/org/elasticsearch/search/builder/SearchSourceBuilder.java +++ b/core/src/main/java/org/elasticsearch/search/builder/SearchSourceBuilder.java @@ -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) diff --git a/core/src/main/java/org/elasticsearch/search/internal/DefaultSearchContext.java b/core/src/main/java/org/elasticsearch/search/internal/DefaultSearchContext.java index a001ab22ac4..30e994b7656 100644 --- a/core/src/main/java/org/elasticsearch/search/internal/DefaultSearchContext.java +++ b/core/src/main/java/org/elasticsearch/search/internal/DefaultSearchContext.java @@ -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; diff --git a/core/src/main/java/org/elasticsearch/search/internal/ScrollContext.java b/core/src/main/java/org/elasticsearch/search/internal/ScrollContext.java index 1744b6fd745..1b7bcfb93c7 100644 --- a/core/src/main/java/org/elasticsearch/search/internal/ScrollContext.java +++ b/core/src/main/java/org/elasticsearch/search/internal/ScrollContext.java @@ -29,5 +29,4 @@ public class ScrollContext { public float maxScore; public ScoreDoc lastEmittedDoc; public Scroll scroll; - } diff --git a/core/src/main/java/org/elasticsearch/search/internal/SubSearchContext.java b/core/src/main/java/org/elasticsearch/search/internal/SubSearchContext.java index 6cacf86d65f..37fb608fd0c 100644 --- a/core/src/main/java/org/elasticsearch/search/internal/SubSearchContext.java +++ b/core/src/main/java/org/elasticsearch/search/internal/SubSearchContext.java @@ -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; diff --git a/core/src/main/java/org/elasticsearch/search/slice/DocValuesSliceQuery.java b/core/src/main/java/org/elasticsearch/search/slice/DocValuesSliceQuery.java new file mode 100644 index 00000000000..95cafacedde --- /dev/null +++ b/core/src/main/java/org/elasticsearch/search/slice/DocValuesSliceQuery.java @@ -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. + * + * NOTE: 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(); + } + }; + } + }; + } +} diff --git a/core/src/main/java/org/elasticsearch/search/slice/SliceBuilder.java b/core/src/main/java/org/elasticsearch/search/slice/SliceBuilder.java new file mode 100644 index 00000000000..97c79aefa71 --- /dev/null +++ b/core/src/main/java/org/elasticsearch/search/slice/SliceBuilder.java @@ -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 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(); + } +} diff --git a/core/src/main/java/org/elasticsearch/search/slice/SliceQuery.java b/core/src/main/java/org/elasticsearch/search/slice/SliceQuery.java new file mode 100644 index 00000000000..0d87b275403 --- /dev/null +++ b/core/src/main/java/org/elasticsearch/search/slice/SliceQuery.java @@ -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 + "]"; + } + +} diff --git a/core/src/main/java/org/elasticsearch/search/slice/TermsSliceQuery.java b/core/src/main/java/org/elasticsearch/search/slice/TermsSliceQuery.java new file mode 100644 index 00000000000..b967a6b6e71 --- /dev/null +++ b/core/src/main/java/org/elasticsearch/search/slice/TermsSliceQuery.java @@ -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. + * + * NOTE: 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. + * NOTE: 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(); + } +} diff --git a/core/src/main/java/org/elasticsearch/threadpool/ExecutorBuilder.java b/core/src/main/java/org/elasticsearch/threadpool/ExecutorBuilder.java new file mode 100644 index 00000000000..434e6fc509c --- /dev/null +++ b/core/src/main/java/org/elasticsearch/threadpool/ExecutorBuilder.java @@ -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 the underlying type of the executor settings + */ +public abstract class ExecutorBuilder { + + 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> 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; + } + + } + +} diff --git a/core/src/main/java/org/elasticsearch/threadpool/FixedExecutorBuilder.java b/core/src/main/java/org/elasticsearch/threadpool/FixedExecutorBuilder.java new file mode 100644 index 00000000000..0735774d972 --- /dev/null +++ b/core/src/main/java/org/elasticsearch/threadpool/FixedExecutorBuilder.java @@ -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 { + + private final Setting sizeSetting; + private final Setting 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> 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; + } + + } + +} diff --git a/core/src/main/java/org/elasticsearch/threadpool/ScalingExecutorBuilder.java b/core/src/main/java/org/elasticsearch/threadpool/ScalingExecutorBuilder.java new file mode 100644 index 00000000000..68c70c83c19 --- /dev/null +++ b/core/src/main/java/org/elasticsearch/threadpool/ScalingExecutorBuilder.java @@ -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 { + + private final Setting coreSetting; + private final Setting maxSetting; + private final Setting 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> 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; + } + } + +} diff --git a/core/src/main/java/org/elasticsearch/threadpool/ThreadPool.java b/core/src/main/java/org/elasticsearch/threadpool/ThreadPool.java index 0d24bb74e17..5c31323b3d8 100644 --- a/core/src/main/java/org/elasticsearch/threadpool/ThreadPool.java +++ b/core/src/main/java/org/elasticsearch/threadpool/ThreadPool.java @@ -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 executorSettings, ExecutorSettingsBuilder builder) { - Settings settings = builder.build(); - String name = settings.get("name"); - executorSettings.put(name, settings); - } - - private static abstract class ExecutorSettingsBuilder> { - - 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 { - - 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 { - - 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 THREADPOOL_GROUP_SETTING = - Setting.groupSetting("threadpool.", Property.Dynamic, Property.NodeScope); - - private volatile Map executors; - - private final Map defaultExecutorTypeSettings; - - private final Queue retiredExecutors = new ConcurrentLinkedQueue<>(); + private Map 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 builders; + + public Collection builders() { + return Collections.unmodifiableCollection(builders.values()); } - public ThreadPool(Settings settings) { + public static Setting 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 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 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 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 executors = new HashMap<>(); - for (Map.Entry executor : defaultExecutorTypeSettings.entrySet()) { - executors.put(executor.getKey(), build(executor.getKey(), groupSettings.get(executor.getKey()), executor.getValue())); - } - - // Building custom thread pools - for (Map.Entry 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 executors = new HashMap<>(); + for (@SuppressWarnings("unchecked") final Map.Entry 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 groupSettings = settings.getAsGroups(); - if (groupSettings.isEmpty()) { - return; - } - - for (Map.Entry 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 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 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 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 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; diff --git a/core/src/main/java/org/elasticsearch/threadpool/ThreadPoolModule.java b/core/src/main/java/org/elasticsearch/threadpool/ThreadPoolModule.java index 9a507f883ac..843febfef8c 100644 --- a/core/src/main/java/org/elasticsearch/threadpool/ThreadPoolModule.java +++ b/core/src/main/java/org/elasticsearch/threadpool/ThreadPoolModule.java @@ -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); } + } diff --git a/core/src/test/java/org/elasticsearch/action/RejectionActionIT.java b/core/src/test/java/org/elasticsearch/action/RejectionActionIT.java index fb0283db48f..6f100170250 100644 --- a/core/src/test/java/org/elasticsearch/action/RejectionActionIT.java +++ b/core/src/test/java/org/elasticsearch/action/RejectionActionIT.java @@ -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(); } diff --git a/core/src/test/java/org/elasticsearch/action/admin/cluster/node/tasks/TaskManagerTestCase.java b/core/src/test/java/org/elasticsearch/action/admin/cluster/node/tasks/TaskManagerTestCase.java index f04c758ef9a..a83a05b8ad8 100644 --- a/core/src/test/java/org/elasticsearch/action/admin/cluster/node/tasks/TaskManagerTestCase.java +++ b/core/src/test/java/org/elasticsearch/action/admin/cluster/node/tasks/TaskManagerTestCase.java @@ -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 diff --git a/core/src/test/java/org/elasticsearch/action/admin/indices/create/CreateIndexIT.java b/core/src/test/java/org/elasticsearch/action/admin/indices/create/CreateIndexIT.java index 31c5749697b..8fbb489d9c2 100644 --- a/core/src/test/java/org/elasticsearch/action/admin/indices/create/CreateIndexIT.java +++ b/core/src/test/java/org/elasticsearch/action/admin/indices/create/CreateIndexIT.java @@ -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 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) diff --git a/core/src/test/java/org/elasticsearch/action/admin/indices/shrink/TransportShrinkActionTests.java b/core/src/test/java/org/elasticsearch/action/admin/indices/shrink/TransportShrinkActionTests.java index 50bf8715f19..01be8da8944 100644 --- a/core/src/test/java/org/elasticsearch/action/admin/indices/shrink/TransportShrinkActionTests.java +++ b/core/src/test/java/org/elasticsearch/action/admin/indices/shrink/TransportShrinkActionTests.java @@ -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")); diff --git a/core/src/test/java/org/elasticsearch/action/bulk/BulkProcessorRetryIT.java b/core/src/test/java/org/elasticsearch/action/bulk/BulkProcessorRetryIT.java index e950da05d37..87249bc8b5a 100644 --- a/core/src/test/java/org/elasticsearch/action/bulk/BulkProcessorRetryIT.java +++ b/core/src/test/java/org/elasticsearch/action/bulk/BulkProcessorRetryIT.java @@ -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(); } diff --git a/core/src/test/java/org/elasticsearch/action/bulk/BulkRequestTests.java b/core/src/test/java/org/elasticsearch/action/bulk/BulkRequestTests.java index f2afc1e7f6e..337f881d41b 100644 --- a/core/src/test/java/org/elasticsearch/action/bulk/BulkRequestTests.java +++ b/core/src/test/java/org/elasticsearch/action/bulk/BulkRequestTests.java @@ -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 diff --git a/core/src/test/java/org/elasticsearch/action/bulk/BulkShardRequestTests.java b/core/src/test/java/org/elasticsearch/action/bulk/BulkShardRequestTests.java index ff1a24d6900..b26d2531ff0 100644 --- a/core/src/test/java/org/elasticsearch/action/bulk/BulkShardRequestTests.java +++ b/core/src/test/java/org/elasticsearch/action/bulk/BulkShardRequestTests.java @@ -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()); } } diff --git a/core/src/test/java/org/elasticsearch/action/bulk/TransportBulkActionTookTests.java b/core/src/test/java/org/elasticsearch/action/bulk/TransportBulkActionTookTests.java index 5131ddebca7..908fb2ddd4d 100644 --- a/core/src/test/java/org/elasticsearch/action/bulk/TransportBulkActionTookTests.java +++ b/core/src/test/java/org/elasticsearch/action/bulk/TransportBulkActionTookTests.java @@ -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 diff --git a/core/src/test/java/org/elasticsearch/ingest/processor/TrackingResultProcessorTests.java b/core/src/test/java/org/elasticsearch/action/ingest/TrackingResultProcessorTests.java similarity index 97% rename from core/src/test/java/org/elasticsearch/ingest/processor/TrackingResultProcessorTests.java rename to core/src/test/java/org/elasticsearch/action/ingest/TrackingResultProcessorTests.java index e19a4df05c9..ac7282051d5 100644 --- a/core/src/test/java/org/elasticsearch/ingest/processor/TrackingResultProcessorTests.java +++ b/core/src/test/java/org/elasticsearch/action/ingest/TrackingResultProcessorTests.java @@ -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; diff --git a/core/src/test/java/org/elasticsearch/action/support/ListenableActionFutureTests.java b/core/src/test/java/org/elasticsearch/action/support/ListenableActionFutureTests.java index a6c21ca3d66..80492f0be61 100644 --- a/core/src/test/java/org/elasticsearch/action/support/ListenableActionFutureTests.java +++ b/core/src/test/java/org/elasticsearch/action/support/ListenableActionFutureTests.java @@ -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 future = new PlainListenableActionFuture<>(threadPool); final CountDownLatch listenerCalled = new CountDownLatch(1); diff --git a/core/src/test/java/org/elasticsearch/action/support/broadcast/node/TransportBroadcastByNodeActionTests.java b/core/src/test/java/org/elasticsearch/action/support/broadcast/node/TransportBroadcastByNodeActionTests.java index 0fac744625f..07ddfa8e49f 100644 --- a/core/src/test/java/org/elasticsearch/action/support/broadcast/node/TransportBroadcastByNodeActionTests.java +++ b/core/src/test/java/org/elasticsearch/action/support/broadcast/node/TransportBroadcastByNodeActionTests.java @@ -55,6 +55,7 @@ import org.elasticsearch.index.shard.ShardId; import org.elasticsearch.rest.RestStatus; import org.elasticsearch.test.ESTestCase; import org.elasticsearch.test.transport.CapturingTransport; +import org.elasticsearch.threadpool.TestThreadPool; import org.elasticsearch.threadpool.ThreadPool; import org.elasticsearch.transport.TransportChannel; import org.elasticsearch.transport.TransportResponse; @@ -182,7 +183,7 @@ public class TransportBroadcastByNodeActionTests extends ESTestCase { @BeforeClass public static void startThreadPool() { - THREAD_POOL = new ThreadPool(TransportBroadcastByNodeActionTests.class.getSimpleName()); + THREAD_POOL = new TestThreadPool(TransportBroadcastByNodeActionTests.class.getSimpleName()); } @Before diff --git a/core/src/test/java/org/elasticsearch/action/support/master/TransportMasterNodeActionTests.java b/core/src/test/java/org/elasticsearch/action/support/master/TransportMasterNodeActionTests.java index b92ba64c2fb..d6edb972ef4 100644 --- a/core/src/test/java/org/elasticsearch/action/support/master/TransportMasterNodeActionTests.java +++ b/core/src/test/java/org/elasticsearch/action/support/master/TransportMasterNodeActionTests.java @@ -45,6 +45,7 @@ import org.elasticsearch.rest.RestStatus; import org.elasticsearch.tasks.Task; import org.elasticsearch.test.ESTestCase; import org.elasticsearch.test.transport.CapturingTransport; +import org.elasticsearch.threadpool.TestThreadPool; import org.elasticsearch.threadpool.ThreadPool; import org.elasticsearch.transport.ConnectTransportException; import org.elasticsearch.transport.TransportService; @@ -76,7 +77,7 @@ public class TransportMasterNodeActionTests extends ESTestCase { @BeforeClass public static void beforeClass() { - threadPool = new ThreadPool("TransportMasterNodeActionTests"); + threadPool = new TestThreadPool("TransportMasterNodeActionTests"); } @Override diff --git a/core/src/test/java/org/elasticsearch/action/support/nodes/TransportNodesActionTests.java b/core/src/test/java/org/elasticsearch/action/support/nodes/TransportNodesActionTests.java index c53adb08ce2..1b0a6f8d622 100644 --- a/core/src/test/java/org/elasticsearch/action/support/nodes/TransportNodesActionTests.java +++ b/core/src/test/java/org/elasticsearch/action/support/nodes/TransportNodesActionTests.java @@ -35,6 +35,7 @@ import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.transport.DummyTransportAddress; import org.elasticsearch.test.ESTestCase; import org.elasticsearch.test.transport.CapturingTransport; +import org.elasticsearch.threadpool.TestThreadPool; import org.elasticsearch.threadpool.ThreadPool; import org.elasticsearch.transport.TransportService; import org.junit.After; @@ -162,7 +163,7 @@ public class TransportNodesActionTests extends ESTestCase { @BeforeClass public static void startThreadPool() { - THREAD_POOL = new ThreadPool(TransportBroadcastByNodeActionTests.class.getSimpleName()); + THREAD_POOL = new TestThreadPool(TransportBroadcastByNodeActionTests.class.getSimpleName()); } @AfterClass diff --git a/core/src/test/java/org/elasticsearch/action/support/replication/BroadcastReplicationTests.java b/core/src/test/java/org/elasticsearch/action/support/replication/BroadcastReplicationTests.java index 6175f822b6a..ea4e55c02f4 100644 --- a/core/src/test/java/org/elasticsearch/action/support/replication/BroadcastReplicationTests.java +++ b/core/src/test/java/org/elasticsearch/action/support/replication/BroadcastReplicationTests.java @@ -21,7 +21,6 @@ package org.elasticsearch.action.support.replication; import org.elasticsearch.Version; import org.elasticsearch.action.ActionListener; import org.elasticsearch.action.NoShardAvailableActionException; -import org.elasticsearch.action.ReplicationResponse; import org.elasticsearch.action.ShardOperationFailedException; import org.elasticsearch.action.UnavailableShardsException; import org.elasticsearch.action.admin.indices.flush.FlushRequest; @@ -45,6 +44,7 @@ import org.elasticsearch.indices.breaker.NoneCircuitBreakerService; import org.elasticsearch.rest.RestStatus; import org.elasticsearch.tasks.Task; import org.elasticsearch.test.ESTestCase; +import org.elasticsearch.threadpool.TestThreadPool; import org.elasticsearch.threadpool.ThreadPool; import org.elasticsearch.transport.TransportService; import org.elasticsearch.transport.local.LocalTransport; @@ -81,7 +81,7 @@ public class BroadcastReplicationTests extends ESTestCase { @BeforeClass public static void beforeClass() { - threadPool = new ThreadPool("BroadcastReplicationTests"); + threadPool = new TestThreadPool("BroadcastReplicationTests"); circuitBreakerService = new NoneCircuitBreakerService(); } diff --git a/core/src/test/java/org/elasticsearch/action/support/replication/ReplicationOperationTests.java b/core/src/test/java/org/elasticsearch/action/support/replication/ReplicationOperationTests.java index cc7558d1de8..55e2a9d3cf2 100644 --- a/core/src/test/java/org/elasticsearch/action/support/replication/ReplicationOperationTests.java +++ b/core/src/test/java/org/elasticsearch/action/support/replication/ReplicationOperationTests.java @@ -21,16 +21,15 @@ package org.elasticsearch.action.support.replication; import org.apache.lucene.index.CorruptIndexException; import org.elasticsearch.ElasticsearchException; import org.elasticsearch.action.ActionListener; -import org.elasticsearch.action.ReplicationResponse; import org.elasticsearch.action.UnavailableShardsException; import org.elasticsearch.action.WriteConsistencyLevel; import org.elasticsearch.action.support.PlainActionFuture; +import org.elasticsearch.action.support.replication.ReplicationResponse.ShardInfo; import org.elasticsearch.cluster.ClusterState; import org.elasticsearch.cluster.node.DiscoveryNodes; import org.elasticsearch.cluster.routing.IndexShardRoutingTable; import org.elasticsearch.cluster.routing.ShardRouting; import org.elasticsearch.cluster.routing.ShardRoutingState; -import org.elasticsearch.common.collect.Tuple; import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.io.stream.StreamOutput; import org.elasticsearch.common.logging.ESLogger; @@ -102,7 +101,7 @@ public class ReplicationOperationTests extends ESTestCase { } Request request = new Request(shardId); - PlainActionFuture listener = new PlainActionFuture<>(); + PlainActionFuture listener = new PlainActionFuture<>(); final ClusterState finalState = state; final TestReplicaProxy replicasProxy = new TestReplicaProxy(expectedFailures); final TestReplicationOperation op = new TestReplicationOperation(request, @@ -114,7 +113,7 @@ public class ReplicationOperationTests extends ESTestCase { assertThat(request.processedOnReplicas, equalTo(expectedReplicas)); assertThat(replicasProxy.failedReplicas, equalTo(expectedFailedShards)); assertTrue("listener is not marked as done", listener.isDone()); - Response.ShardInfo shardInfo = listener.actionGet().getShardInfo(); + ShardInfo shardInfo = listener.actionGet().getShardInfo(); assertThat(shardInfo.getFailed(), equalTo(expectedFailedShards.size())); assertThat(shardInfo.getFailures(), arrayWithSize(expectedFailedShards.size())); assertThat(shardInfo.getSuccessful(), equalTo(1 + expectedReplicas.size() - expectedFailures.size())); @@ -135,7 +134,7 @@ public class ReplicationOperationTests extends ESTestCase { final ShardRouting primaryShard = indexShardRoutingTable.primaryShard(); Request request = new Request(shardId); - PlainActionFuture listener = new PlainActionFuture<>(); + PlainActionFuture listener = new PlainActionFuture<>(); final TestReplicationOperation op = new TestReplicationOperation(request, new TestPrimary(primaryShard, primaryTerm), listener, false, false, new TestReplicaProxy(), () -> state, logger, "test"); @@ -143,7 +142,7 @@ public class ReplicationOperationTests extends ESTestCase { assertThat("request was not processed on primary", request.processedOnPrimary.get(), equalTo(true)); assertThat(request.processedOnReplicas, equalTo(Collections.emptySet())); assertTrue("listener is not marked as done", listener.isDone()); - Response.ShardInfo shardInfo = listener.actionGet().getShardInfo(); + ShardInfo shardInfo = listener.actionGet().getShardInfo(); assertThat(shardInfo.getFailed(), equalTo(0)); assertThat(shardInfo.getFailures(), arrayWithSize(0)); assertThat(shardInfo.getSuccessful(), equalTo(1)); @@ -172,7 +171,7 @@ public class ReplicationOperationTests extends ESTestCase { expectedFailures.put(failedReplica, new CorruptIndexException("simulated", (String) null)); Request request = new Request(shardId); - PlainActionFuture listener = new PlainActionFuture<>(); + PlainActionFuture listener = new PlainActionFuture<>(); final ClusterState finalState = state; final TestReplicaProxy replicasProxy = new TestReplicaProxy(expectedFailures) { @Override @@ -233,16 +232,16 @@ public class ReplicationOperationTests extends ESTestCase { final ShardRouting primaryShard = state.get().routingTable().shardRoutingTable(shardId).primaryShard(); final TestPrimary primary = new TestPrimary(primaryShard, primaryTerm) { @Override - public Tuple perform(Request request) throws Exception { - final Tuple tuple = super.perform(request); + public Result perform(Request request) throws Exception { + Result result = super.perform(request); state.set(changedState); logger.debug("--> state after primary operation:\n{}", state.get().prettyPrint()); - return tuple; + return result; } }; Request request = new Request(shardId); - PlainActionFuture listener = new PlainActionFuture<>(); + PlainActionFuture listener = new PlainActionFuture<>(); final TestReplicationOperation op = new TestReplicationOperation(request, primary, listener, new TestReplicaProxy(), state::get); op.execute(); @@ -296,7 +295,7 @@ public class ReplicationOperationTests extends ESTestCase { state.prettyPrint()); final long primaryTerm = state.metaData().index(index).primaryTerm(shardId.id()); final IndexShardRoutingTable shardRoutingTable = state.routingTable().index(index).shard(shardId.id()); - PlainActionFuture listener = new PlainActionFuture<>(); + PlainActionFuture listener = new PlainActionFuture<>(); final ShardRouting primaryShard = shardRoutingTable.primaryShard(); final TestReplicationOperation op = new TestReplicationOperation(request, new TestPrimary(primaryShard, primaryTerm), @@ -362,10 +361,7 @@ public class ReplicationOperationTests extends ESTestCase { } } - static class Response extends ReplicationResponse { - } - - static class TestPrimary implements ReplicationOperation.Primary { + static class TestPrimary implements ReplicationOperation.Primary { final ShardRouting routing; final long term; @@ -385,12 +381,35 @@ public class ReplicationOperationTests extends ESTestCase { } @Override - public Tuple perform(Request request) throws Exception { + public Result perform(Request request) throws Exception { if (request.processedOnPrimary.compareAndSet(false, true) == false) { fail("processed [" + request + "] twice"); } request.primaryTerm(term); - return new Tuple<>(new Response(), request); + return new Result(request); + } + + static class Result implements ReplicationOperation.PrimaryResult { + private final Request request; + private ShardInfo shardInfo; + + public Result(Request request) { + this.request = request; + } + + @Override + public Request replicaRequest() { + return request; + } + + @Override + public void setShardInfo(ShardInfo shardInfo) { + this.shardInfo = shardInfo; + } + + public ShardInfo getShardInfo() { + return shardInfo; + } } } @@ -436,15 +455,15 @@ public class ReplicationOperationTests extends ESTestCase { } } - class TestReplicationOperation extends ReplicationOperation { - public TestReplicationOperation(Request request, Primary primary, ActionListener listener, - Replicas replicas, Supplier clusterStateSupplier) { + class TestReplicationOperation extends ReplicationOperation { + public TestReplicationOperation(Request request, Primary primary, + ActionListener listener, Replicas replicas, Supplier clusterStateSupplier) { this(request, primary, listener, true, false, replicas, clusterStateSupplier, ReplicationOperationTests.this.logger, "test"); } - public TestReplicationOperation(Request request, Primary primary, ActionListener listener, - boolean executeOnReplicas, boolean checkWriteConsistency, Replicas replicas, - Supplier clusterStateSupplier, ESLogger logger, String opType) { + public TestReplicationOperation(Request request, Primary primary, + ActionListener listener, boolean executeOnReplicas, boolean checkWriteConsistency, + Replicas replicas, Supplier clusterStateSupplier, ESLogger logger, String opType) { super(request, primary, listener, executeOnReplicas, checkWriteConsistency, replicas, clusterStateSupplier, logger, opType); } } diff --git a/core/src/test/java/org/elasticsearch/action/support/replication/TransportReplicationActionTests.java b/core/src/test/java/org/elasticsearch/action/support/replication/TransportReplicationActionTests.java index 2e81ec712eb..8ee2499d040 100644 --- a/core/src/test/java/org/elasticsearch/action/support/replication/TransportReplicationActionTests.java +++ b/core/src/test/java/org/elasticsearch/action/support/replication/TransportReplicationActionTests.java @@ -20,7 +20,6 @@ package org.elasticsearch.action.support.replication; import org.elasticsearch.ElasticsearchException; import org.elasticsearch.action.ActionListener; -import org.elasticsearch.action.ReplicationResponse; import org.elasticsearch.action.UnavailableShardsException; import org.elasticsearch.action.support.ActionFilters; import org.elasticsearch.action.support.PlainActionFuture; @@ -43,7 +42,6 @@ import org.elasticsearch.cluster.routing.allocation.AllocationService; import org.elasticsearch.cluster.routing.allocation.RoutingAllocation; import org.elasticsearch.cluster.service.ClusterService; import org.elasticsearch.common.Nullable; -import org.elasticsearch.common.collect.Tuple; import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.io.stream.StreamOutput; import org.elasticsearch.common.lease.Releasable; @@ -59,6 +57,7 @@ import org.elasticsearch.rest.RestStatus; import org.elasticsearch.test.ESAllocationTestCase; import org.elasticsearch.test.ESTestCase; import org.elasticsearch.test.transport.CapturingTransport; +import org.elasticsearch.threadpool.TestThreadPool; import org.elasticsearch.threadpool.ThreadPool; import org.elasticsearch.transport.TransportChannel; import org.elasticsearch.transport.TransportException; @@ -111,7 +110,7 @@ public class TransportReplicationActionTests extends ESTestCase { @BeforeClass public static void beforeClass() { - threadPool = new ThreadPool("ShardReplicationTests"); + threadPool = new TestThreadPool("ShardReplicationTests"); } @Override @@ -155,7 +154,7 @@ public class TransportReplicationActionTests extends ESTestCase { ClusterBlocks.Builder block = ClusterBlocks.builder() .addGlobalBlock(new ClusterBlock(1, "non retryable", false, true, RestStatus.SERVICE_UNAVAILABLE, ClusterBlockLevel.ALL)); setState(clusterService, ClusterState.builder(clusterService.state()).blocks(block)); - TransportReplicationAction.ReroutePhase reroutePhase = action.new ReroutePhase(task, request, listener); + Action.ReroutePhase reroutePhase = action.new ReroutePhase(task, request, listener); reroutePhase.run(); assertListenerThrows("primary phase should fail operation", listener, ClusterBlockException.class); assertPhase(task, "failed"); @@ -199,7 +198,7 @@ public class TransportReplicationActionTests extends ESTestCase { Request request = new Request(shardId).timeout("1ms"); PlainActionFuture listener = new PlainActionFuture<>(); - TransportReplicationAction.ReroutePhase reroutePhase = action.new ReroutePhase(task, request, listener); + Action.ReroutePhase reroutePhase = action.new ReroutePhase(task, request, listener); reroutePhase.run(); assertListenerThrows("unassigned primary didn't cause a timeout", listener, UnavailableShardsException.class); assertPhase(task, "failed"); @@ -245,7 +244,7 @@ public class TransportReplicationActionTests extends ESTestCase { Request request = new Request(shardId).timeout("1ms").routedBasedOnClusterVersion(clusterService.state().version() + 1); PlainActionFuture listener = new PlainActionFuture<>(); - TransportReplicationAction.ReroutePhase reroutePhase = action.new ReroutePhase(null, request, listener); + Action.ReroutePhase reroutePhase = action.new ReroutePhase(null, request, listener); reroutePhase.run(); assertListenerThrows("cluster state too old didn't cause a timeout", listener, UnavailableShardsException.class); @@ -285,7 +284,7 @@ public class TransportReplicationActionTests extends ESTestCase { PlainActionFuture listener = new PlainActionFuture<>(); ReplicationTask task = maybeTask(); - TransportReplicationAction.ReroutePhase reroutePhase = action.new ReroutePhase(task, request, listener); + Action.ReroutePhase reroutePhase = action.new ReroutePhase(task, request, listener); reroutePhase.run(); assertListenerThrows("must throw index not found exception", listener, IndexNotFoundException.class); assertPhase(task, "failed"); @@ -312,7 +311,7 @@ public class TransportReplicationActionTests extends ESTestCase { PlainActionFuture listener = new PlainActionFuture<>(); ReplicationTask task = maybeTask(); - TransportReplicationAction.ReroutePhase reroutePhase = action.new ReroutePhase(task, request, listener); + Action.ReroutePhase reroutePhase = action.new ReroutePhase(task, request, listener); reroutePhase.run(); CapturingTransport.CapturedRequest[] capturedRequests = transport.getCapturedRequestsAndClear(); assertThat(capturedRequests, arrayWithSize(1)); @@ -364,7 +363,7 @@ public class TransportReplicationActionTests extends ESTestCase { Request request = new Request(shardId); PlainActionFuture listener = new PlainActionFuture<>(); - TransportReplicationAction.ReroutePhase reroutePhase = action.new ReroutePhase(task, request, listener); + Action.ReroutePhase reroutePhase = action.new ReroutePhase(task, request, listener); reroutePhase.run(); assertThat(request.shardId(), equalTo(shardId)); logger.info("--> primary is assigned to [{}], checking request forwarded", primaryNodeId); @@ -393,9 +392,9 @@ public class TransportReplicationActionTests extends ESTestCase { AtomicBoolean executed = new AtomicBoolean(); Action.PrimaryOperationTransportHandler primaryPhase = action.new PrimaryOperationTransportHandler() { @Override - protected ReplicationOperation createReplicatedOperation(Request request, ActionListener actionListener, - Action.PrimaryShardReference primaryShardReference, - boolean executeOnReplicas) { + protected ReplicationOperation createReplicatedOperation(Request request, + ActionListener actionListener, Action.PrimaryShardReference primaryShardReference, + boolean executeOnReplicas) { return new NoopReplicationOperation(request, actionListener) { public void execute() throws Exception { assertPhase(task, "primary"); @@ -448,9 +447,9 @@ public class TransportReplicationActionTests extends ESTestCase { AtomicBoolean executed = new AtomicBoolean(); Action.PrimaryOperationTransportHandler primaryPhase = action.new PrimaryOperationTransportHandler() { @Override - protected ReplicationOperation createReplicatedOperation(Request request, ActionListener actionListener, - Action.PrimaryShardReference primaryShardReference, - boolean executeOnReplicas) { + protected ReplicationOperation createReplicatedOperation(Request request, + ActionListener actionListener, Action.PrimaryShardReference primaryShardReference, + boolean executeOnReplicas) { return new NoopReplicationOperation(request, actionListener) { public void execute() throws Exception { assertPhase(task, "primary"); @@ -478,9 +477,9 @@ public class TransportReplicationActionTests extends ESTestCase { }; Action.PrimaryShardReference primary = action.new PrimaryShardReference(shard, releasable); final Request request = new Request(); - Tuple result = primary.perform(request); + Request replicaRequest = primary.perform(request).replicaRequest; - assertThat(result.v2().primaryTerm(), equalTo(primaryTerm)); + assertThat(replicaRequest.primaryTerm(), equalTo(primaryTerm)); final ElasticsearchException exception = new ElasticsearchException("testing"); primary.failShard("test", exception); @@ -582,9 +581,9 @@ public class TransportReplicationActionTests extends ESTestCase { setState(clusterService, state); Action.PrimaryOperationTransportHandler primaryPhase = action.new PrimaryOperationTransportHandler() { @Override - protected ReplicationOperation createReplicatedOperation(Request request, ActionListener actionListener, - Action.PrimaryShardReference primaryShardReference, - boolean executeOnReplicas) { + protected ReplicationOperation createReplicatedOperation(Request request, + ActionListener actionListener, Action.PrimaryShardReference primaryShardReference, + boolean executeOnReplicas) { assertFalse(executeOnReplicas); return new NoopReplicationOperation(request, actionListener); } @@ -608,9 +607,9 @@ public class TransportReplicationActionTests extends ESTestCase { Action.PrimaryOperationTransportHandler primaryPhase = action.new PrimaryOperationTransportHandler() { @Override - protected ReplicationOperation createReplicatedOperation(Request request, ActionListener listener, - Action.PrimaryShardReference primaryShardReference, - boolean executeOnReplicas) { + protected ReplicationOperation createReplicatedOperation(Request request, + ActionListener listener, Action.PrimaryShardReference primaryShardReference, + boolean executeOnReplicas) { assertIndexShardCounter(1); if (throwExceptionOnCreation) { throw new ElasticsearchException("simulated exception, during createReplicatedOperation"); @@ -623,7 +622,7 @@ public class TransportReplicationActionTests extends ESTestCase { if (throwExceptionOnRun) { throw new ElasticsearchException("simulated exception, during performOnPrimary"); } else if (respondWithError) { - this.finalResponseListener.onFailure(new ElasticsearchException("simulated exception, as a response")); + this.resultListener.onFailure(new ElasticsearchException("simulated exception, as a response")); } else { super.execute(); } @@ -667,13 +666,13 @@ public class TransportReplicationActionTests extends ESTestCase { final ReplicationTask task = maybeTask(); Action action = new Action(Settings.EMPTY, "testActionWithExceptions", transportService, clusterService, threadPool) { @Override - protected void shardOperationOnReplica(Request request) { + protected ReplicaResult shardOperationOnReplica(Request request) { assertIndexShardCounter(1); assertPhase(task, "replica"); if (throwException) { throw new ElasticsearchException("simulated"); } - super.shardOperationOnReplica(request); + return new ReplicaResult(); } }; final Action.ReplicaOperationTransportHandler replicaOperationTransportHandler = action.new ReplicaOperationTransportHandler(); @@ -765,15 +764,16 @@ public class TransportReplicationActionTests extends ESTestCase { } @Override - protected Tuple shardOperationOnPrimary(Request shardRequest) throws Exception { + protected PrimaryResult shardOperationOnPrimary(Request shardRequest) throws Exception { boolean executedBefore = shardRequest.processedOnPrimary.getAndSet(true); assert executedBefore == false : "request has already been executed on the primary"; - return new Tuple<>(new Response(), shardRequest); + return new PrimaryResult(shardRequest, new Response()); } @Override - protected void shardOperationOnReplica(Request request) { + protected ReplicaResult shardOperationOnReplica(Request request) { request.processedOnReplicas.incrementAndGet(); + return new ReplicaResult(); } @Override @@ -822,15 +822,14 @@ public class TransportReplicationActionTests extends ESTestCase { } } - class NoopReplicationOperation extends ReplicationOperation { - - public NoopReplicationOperation(Request request, ActionListener listener) { + class NoopReplicationOperation extends ReplicationOperation { + public NoopReplicationOperation(Request request, ActionListener listener) { super(request, null, listener, true, true, null, null, TransportReplicationActionTests.this.logger, "noop"); } @Override public void execute() throws Exception { - this.finalResponseListener.onResponse(new Response()); + this.resultListener.onResponse(action.new PrimaryResult(null, new Response())); } } diff --git a/core/src/test/java/org/elasticsearch/action/support/replication/TransportWriteActionTests.java b/core/src/test/java/org/elasticsearch/action/support/replication/TransportWriteActionTests.java new file mode 100644 index 00000000000..7b312959631 --- /dev/null +++ b/core/src/test/java/org/elasticsearch/action/support/replication/TransportWriteActionTests.java @@ -0,0 +1,190 @@ +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch licenses this file to you under + * the Apache License, Version 2.0 (the "License"); you may + * not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.elasticsearch.action.support.replication; + +import org.elasticsearch.action.ActionListener; +import org.elasticsearch.action.support.ActionFilters; +import org.elasticsearch.action.support.WriteRequest.RefreshPolicy; +import org.elasticsearch.action.support.WriteResponse; +import org.elasticsearch.cluster.metadata.IndexNameExpressionResolver; +import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.index.shard.IndexShard; +import org.elasticsearch.index.shard.ShardId; +import org.elasticsearch.index.translog.Translog; +import org.elasticsearch.index.translog.Translog.Location; +import org.elasticsearch.test.ESTestCase; +import org.elasticsearch.threadpool.ThreadPool; +import org.elasticsearch.transport.TransportService; +import org.junit.Before; +import org.mockito.ArgumentCaptor; + +import java.util.HashSet; +import java.util.function.BiConsumer; +import java.util.function.Consumer; + +import static org.mockito.Matchers.any; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.never; +import static org.mockito.Mockito.verify; + +public class TransportWriteActionTests extends ESTestCase { + private IndexShard indexShard; + private Translog.Location location; + + @Before + public void initCommonMocks() { + indexShard = mock(IndexShard.class); + location = mock(Translog.Location.class); + } + + public void testPrimaryNoRefreshCall() throws Exception { + noRefreshCall(TestAction::shardOperationOnPrimary, TestAction.WritePrimaryResult::respond); + } + + public void testReplicaNoRefreshCall() throws Exception { + noRefreshCall(TestAction::shardOperationOnReplica, TestAction.WriteReplicaResult::respond); + } + + private void noRefreshCall(ThrowingBiFunction action, + BiConsumer> responder) + throws Exception { + TestRequest request = new TestRequest(); + request.setRefreshPolicy(RefreshPolicy.NONE); // The default, but we'll set it anyway just to be explicit + Result result = action.apply(new TestAction(), request); + CapturingActionListener listener = new CapturingActionListener<>(); + responder.accept(result, listener); + assertNotNull(listener.response); + verify(indexShard, never()).refresh(any()); + verify(indexShard, never()).addRefreshListener(any(), any()); + } + + public void testPrimaryImmediateRefresh() throws Exception { + immediateRefresh(TestAction::shardOperationOnPrimary, TestAction.WritePrimaryResult::respond, r -> assertTrue(r.forcedRefresh)); + } + + public void testReplicaImmediateRefresh() throws Exception { + immediateRefresh(TestAction::shardOperationOnReplica, TestAction.WriteReplicaResult::respond, r -> {}); + } + + private void immediateRefresh(ThrowingBiFunction action, + BiConsumer> responder, + Consumer responseChecker) throws Exception { + TestRequest request = new TestRequest(); + request.setRefreshPolicy(RefreshPolicy.IMMEDIATE); + Result result = action.apply(new TestAction(), request); + CapturingActionListener listener = new CapturingActionListener<>(); + responder.accept(result, listener); + assertNotNull(listener.response); + responseChecker.accept(listener.response); + verify(indexShard).refresh("refresh_flag_index"); + verify(indexShard, never()).addRefreshListener(any(), any()); + } + + public void testPrimaryWaitForRefresh() throws Exception { + waitForRefresh(TestAction::shardOperationOnPrimary, TestAction.WritePrimaryResult::respond, + (r, forcedRefresh) -> assertEquals(forcedRefresh, r.forcedRefresh)); + } + + public void testReplicaWaitForRefresh() throws Exception { + waitForRefresh(TestAction::shardOperationOnReplica, TestAction.WriteReplicaResult::respond, (r, forcedRefresh) -> {}); + } + + private void waitForRefresh(ThrowingBiFunction action, + BiConsumer> responder, + BiConsumer resultChecker) throws Exception { + TestRequest request = new TestRequest(); + request.setRefreshPolicy(RefreshPolicy.WAIT_UNTIL); + Result result = action.apply(new TestAction(), request); + CapturingActionListener listener = new CapturingActionListener<>(); + responder.accept(result, listener); + assertNull(listener.response); // Haven't reallresponded yet + + @SuppressWarnings({ "unchecked", "rawtypes" }) + ArgumentCaptor> refreshListener = ArgumentCaptor.forClass((Class) Consumer.class); + verify(indexShard, never()).refresh(any()); + verify(indexShard).addRefreshListener(any(), refreshListener.capture()); + + // Now we can fire the listener manually and we'll get a response + boolean forcedRefresh = randomBoolean(); + refreshListener.getValue().accept(forcedRefresh); + assertNotNull(listener.response); + resultChecker.accept(listener.response, forcedRefresh); + } + + private class TestAction extends TransportWriteAction { + protected TestAction() { + super(Settings.EMPTY, "test", mock(TransportService.class), null, null, null, null, new ActionFilters(new HashSet<>()), + new IndexNameExpressionResolver(Settings.EMPTY), TestRequest::new, ThreadPool.Names.SAME); + } + + @Override + protected IndexShard indexShard(TestRequest request) { + return indexShard; + } + + @Override + protected WriteResult onPrimaryShard(TestRequest request, IndexShard indexShard) throws Exception { + return new WriteResult<>(new TestResponse(), location); + } + + @Override + protected Location onReplicaShard(TestRequest request, IndexShard indexShard) { + return location; + } + + @Override + protected TestResponse newResponseInstance() { + return new TestResponse(); + } + } + + private static class TestRequest extends ReplicatedWriteRequest { + public TestRequest() { + setShardId(new ShardId("test", "test", 1)); + } + } + + private static class TestResponse extends ReplicationResponse implements WriteResponse { + boolean forcedRefresh; + + @Override + public void setForcedRefresh(boolean forcedRefresh) { + this.forcedRefresh = forcedRefresh; + } + } + + private static class CapturingActionListener implements ActionListener { + private R response; + + @Override + public void onResponse(R response) { + this.response = response; + } + + @Override + public void onFailure(Throwable e) { + throw new RuntimeException(e); + } + } + + private interface ThrowingBiFunction { + R apply(A a, B b) throws Exception; + } +} diff --git a/core/src/test/java/org/elasticsearch/action/support/single/instance/TransportInstanceSingleOperationActionTests.java b/core/src/test/java/org/elasticsearch/action/support/single/instance/TransportInstanceSingleOperationActionTests.java index 10132af1b1d..6d7ac6128c9 100644 --- a/core/src/test/java/org/elasticsearch/action/support/single/instance/TransportInstanceSingleOperationActionTests.java +++ b/core/src/test/java/org/elasticsearch/action/support/single/instance/TransportInstanceSingleOperationActionTests.java @@ -43,6 +43,7 @@ import org.elasticsearch.index.shard.ShardId; import org.elasticsearch.rest.RestStatus; import org.elasticsearch.test.ESTestCase; import org.elasticsearch.test.transport.CapturingTransport; +import org.elasticsearch.threadpool.TestThreadPool; import org.elasticsearch.threadpool.ThreadPool; import org.elasticsearch.transport.ConnectTransportException; import org.elasticsearch.transport.TransportException; @@ -133,7 +134,7 @@ public class TransportInstanceSingleOperationActionTests extends ESTestCase { @BeforeClass public static void startThreadPool() { - THREAD_POOL = new ThreadPool(TransportInstanceSingleOperationActionTests.class.getSimpleName()); + THREAD_POOL = new TestThreadPool(TransportInstanceSingleOperationActionTests.class.getSimpleName()); } @Before diff --git a/core/src/test/java/org/elasticsearch/aliases/IndexAliasesIT.java b/core/src/test/java/org/elasticsearch/aliases/IndexAliasesIT.java index 08f8970f1e4..2a16625d037 100644 --- a/core/src/test/java/org/elasticsearch/aliases/IndexAliasesIT.java +++ b/core/src/test/java/org/elasticsearch/aliases/IndexAliasesIT.java @@ -29,6 +29,7 @@ import org.elasticsearch.action.admin.indices.alias.get.GetAliasesResponse; import org.elasticsearch.action.admin.indices.create.CreateIndexRequestBuilder; import org.elasticsearch.action.index.IndexResponse; import org.elasticsearch.action.search.SearchResponse; +import org.elasticsearch.action.support.WriteRequest.RefreshPolicy; import org.elasticsearch.cluster.ClusterState; import org.elasticsearch.cluster.metadata.AliasAction; import org.elasticsearch.cluster.metadata.AliasMetaData; @@ -175,10 +176,15 @@ public class IndexAliasesIT extends ESIntegTestCase { assertAcked(admin().indices().prepareAliases().addAlias("test", "tests", termQuery("name", "test"))); logger.info("--> indexing against [test]"); - client().index(indexRequest("test").type("type1").id("1").source(source("1", "foo test")).refresh(true)).actionGet(); - client().index(indexRequest("test").type("type1").id("2").source(source("2", "bar test")).refresh(true)).actionGet(); - client().index(indexRequest("test").type("type1").id("3").source(source("3", "baz test")).refresh(true)).actionGet(); - client().index(indexRequest("test").type("type1").id("4").source(source("4", "something else")).refresh(true)).actionGet(); + client().index(indexRequest("test").type("type1").id("1").source(source("1", "foo test")).setRefreshPolicy(RefreshPolicy.IMMEDIATE)) + .actionGet(); + client().index(indexRequest("test").type("type1").id("2").source(source("2", "bar test")).setRefreshPolicy(RefreshPolicy.IMMEDIATE)) + .actionGet(); + client().index(indexRequest("test").type("type1").id("3").source(source("3", "baz test")).setRefreshPolicy(RefreshPolicy.IMMEDIATE)) + .actionGet(); + client().index( + indexRequest("test").type("type1").id("4").source(source("4", "something else")).setRefreshPolicy(RefreshPolicy.IMMEDIATE)) + .actionGet(); logger.info("--> checking single filtering alias search"); SearchResponse searchResponse = client().prepareSearch("foos").setQuery(QueryBuilders.matchAllQuery()).get(); diff --git a/core/src/test/java/org/elasticsearch/bootstrap/BootstrapCheckTests.java b/core/src/test/java/org/elasticsearch/bootstrap/BootstrapCheckTests.java index d354adc7544..5336066b1e2 100644 --- a/core/src/test/java/org/elasticsearch/bootstrap/BootstrapCheckTests.java +++ b/core/src/test/java/org/elasticsearch/bootstrap/BootstrapCheckTests.java @@ -30,8 +30,10 @@ import java.util.ArrayList; import java.util.Arrays; import java.util.Collections; import java.util.List; +import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicLong; import java.util.concurrent.atomic.AtomicReference; +import java.util.function.Consumer; import static org.hamcrest.CoreMatchers.allOf; import static org.hamcrest.CoreMatchers.containsString; @@ -383,7 +385,7 @@ public class BootstrapCheckTests extends ESTestCase { } }; - RuntimeException e = expectThrows( + final RuntimeException e = expectThrows( RuntimeException.class, () -> BootstrapCheck.check(true, false, Collections.singletonList(check), "testClientJvmCheck")); assertThat( @@ -395,8 +397,129 @@ public class BootstrapCheckTests extends ESTestCase { BootstrapCheck.check(true, false, Collections.singletonList(check), "testClientJvmCheck"); } + public void testMightForkCheck() { + final AtomicBoolean isSeccompInstalled = new AtomicBoolean(); + final AtomicBoolean mightFork = new AtomicBoolean(); + final BootstrapCheck.MightForkCheck check = new BootstrapCheck.MightForkCheck() { + @Override + boolean isSeccompInstalled() { + return isSeccompInstalled.get(); + } + + @Override + boolean mightFork() { + return mightFork.get(); + } + + @Override + public String errorMessage() { + return "error"; + } + }; + + runMightForkTest( + check, + isSeccompInstalled, + () -> mightFork.set(false), + () -> mightFork.set(true), + e -> assertThat(e.getMessage(), containsString("error"))); + } + + public void testOnErrorCheck() { + final AtomicBoolean isSeccompInstalled = new AtomicBoolean(); + final AtomicReference onError = new AtomicReference<>(); + final BootstrapCheck.MightForkCheck check = new BootstrapCheck.OnErrorCheck() { + @Override + boolean isSeccompInstalled() { + return isSeccompInstalled.get(); + } + + @Override + String onError() { + return onError.get(); + } + }; + + final String command = randomAsciiOfLength(16); + runMightForkTest( + check, + isSeccompInstalled, + () -> onError.set(randomBoolean() ? "" : null), + () -> onError.set(command), + e -> assertThat( + e.getMessage(), + containsString( + "OnError [" + command + "] requires forking but is prevented by system call filters ([bootstrap.seccomp=true]);" + + " upgrade to at least Java 8u92 and use ExitOnOutOfMemoryError"))); + } + + public void testOnOutOfMemoryErrorCheck() { + final AtomicBoolean isSeccompInstalled = new AtomicBoolean(); + final AtomicReference onOutOfMemoryError = new AtomicReference<>(); + final BootstrapCheck.MightForkCheck check = new BootstrapCheck.OnOutOfMemoryErrorCheck() { + @Override + boolean isSeccompInstalled() { + return isSeccompInstalled.get(); + } + + @Override + String onOutOfMemoryError() { + return onOutOfMemoryError.get(); + } + }; + + final String command = randomAsciiOfLength(16); + runMightForkTest( + check, + isSeccompInstalled, + () -> onOutOfMemoryError.set(randomBoolean() ? "" : null), + () -> onOutOfMemoryError.set(command), + e -> assertThat( + e.getMessage(), + containsString( + "OnOutOfMemoryError [" + command + "]" + + " requires forking but is prevented by system call filters ([bootstrap.seccomp=true]);" + + " upgrade to at least Java 8u92 and use ExitOnOutOfMemoryError"))); + } + + private void runMightForkTest( + final BootstrapCheck.MightForkCheck check, + final AtomicBoolean isSeccompInstalled, + final Runnable disableMightFork, + final Runnable enableMightFork, + final Consumer consumer) { + + final String methodName = Thread.currentThread().getStackTrace()[2].getMethodName(); + + // if seccomp is disabled, nothing should happen + isSeccompInstalled.set(false); + if (randomBoolean()) { + disableMightFork.run(); + } else { + enableMightFork.run(); + } + BootstrapCheck.check(true, randomBoolean(), Collections.singletonList(check), methodName); + + // if seccomp is enabled, but we will not fork, nothing should + // happen + isSeccompInstalled.set(true); + disableMightFork.run(); + BootstrapCheck.check(true, randomBoolean(), Collections.singletonList(check), methodName); + + // if seccomp is enabled, and we might fork, the check should + // be enforced, regardless of bootstrap checks being enabled or + // not + isSeccompInstalled.set(true); + enableMightFork.run(); + + final RuntimeException e = expectThrows( + RuntimeException.class, + () -> BootstrapCheck.check(randomBoolean(), randomBoolean(), Collections.singletonList(check), methodName)); + consumer.accept(e); + } + public void testIgnoringSystemChecks() { - BootstrapCheck.Check check = new BootstrapCheck.Check() { + final BootstrapCheck.Check check = new BootstrapCheck.Check() { @Override public boolean check() { return true; @@ -430,4 +553,33 @@ public class BootstrapCheckTests extends ESTestCase { verify(logger).warn("error"); } + public void testAlwaysEnforcedChecks() { + final BootstrapCheck.Check check = new BootstrapCheck.Check() { + @Override + public boolean check() { + return true; + } + + @Override + public String errorMessage() { + return "error"; + } + + @Override + public boolean isSystemCheck() { + return randomBoolean(); + } + + @Override + public boolean alwaysEnforce() { + return true; + } + }; + + final RuntimeException alwaysEnforced = expectThrows( + RuntimeException.class, + () -> BootstrapCheck.check(randomBoolean(), randomBoolean(), Collections.singletonList(check), "testAlwaysEnforcedChecks")); + assertThat(alwaysEnforced, hasToString(containsString("error"))); + } + } diff --git a/core/src/test/java/org/elasticsearch/client/transport/TransportClientNodesServiceTests.java b/core/src/test/java/org/elasticsearch/client/transport/TransportClientNodesServiceTests.java index b566d764231..0dddb301444 100644 --- a/core/src/test/java/org/elasticsearch/client/transport/TransportClientNodesServiceTests.java +++ b/core/src/test/java/org/elasticsearch/client/transport/TransportClientNodesServiceTests.java @@ -28,6 +28,7 @@ import org.elasticsearch.cluster.node.DiscoveryNode; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.transport.LocalTransportAddress; import org.elasticsearch.test.ESTestCase; +import org.elasticsearch.threadpool.TestThreadPool; import org.elasticsearch.threadpool.ThreadPool; import org.elasticsearch.transport.BaseTransportResponseHandler; import org.elasticsearch.transport.TransportException; @@ -63,7 +64,7 @@ public class TransportClientNodesServiceTests extends ESTestCase { TestIteration() { ClusterName clusterName = new ClusterName("test"); - threadPool = new ThreadPool("transport-client-nodes-service-tests"); + threadPool = new TestThreadPool("transport-client-nodes-service-tests"); transport = new FailAndRetryMockTransport(random(), clusterName) { @Override public List getLocalAddresses() { diff --git a/core/src/test/java/org/elasticsearch/client/transport/TransportClientTests.java b/core/src/test/java/org/elasticsearch/client/transport/TransportClientTests.java new file mode 100644 index 00000000000..ec2065b67e2 --- /dev/null +++ b/core/src/test/java/org/elasticsearch/client/transport/TransportClientTests.java @@ -0,0 +1,41 @@ +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch licenses this file to you under + * the Apache License, Version 2.0 (the "License"); you may + * not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.elasticsearch.client.transport; + +import org.elasticsearch.action.admin.cluster.health.ClusterHealthRequest; +import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.test.ESTestCase; + +import java.util.concurrent.ExecutionException; + +import static org.hamcrest.CoreMatchers.containsString; +import static org.hamcrest.object.HasToString.hasToString; + +public class TransportClientTests extends ESTestCase { + + public void testThatUsingAClosedClientThrowsAnException() throws ExecutionException, InterruptedException { + final TransportClient client = TransportClient.builder().settings(Settings.EMPTY).build(); + client.close(); + final IllegalStateException e = + expectThrows(IllegalStateException.class, () -> client.admin().cluster().health(new ClusterHealthRequest()).get()); + assertThat(e, hasToString(containsString("transport client is closed"))); + } + +} diff --git a/core/src/test/java/org/elasticsearch/cluster/action/shard/ShardStateActionTests.java b/core/src/test/java/org/elasticsearch/cluster/action/shard/ShardStateActionTests.java index 261da3e2bb8..bfc3929dd14 100644 --- a/core/src/test/java/org/elasticsearch/cluster/action/shard/ShardStateActionTests.java +++ b/core/src/test/java/org/elasticsearch/cluster/action/shard/ShardStateActionTests.java @@ -38,6 +38,7 @@ import org.elasticsearch.common.settings.Settings; import org.elasticsearch.discovery.Discovery; import org.elasticsearch.test.ESTestCase; import org.elasticsearch.test.transport.CapturingTransport; +import org.elasticsearch.threadpool.TestThreadPool; import org.elasticsearch.threadpool.ThreadPool; import org.elasticsearch.transport.NodeDisconnectedException; import org.elasticsearch.transport.NodeNotConnectedException; @@ -97,7 +98,7 @@ public class ShardStateActionTests extends ESTestCase { @BeforeClass public static void startThreadPool() { - THREAD_POOL = new ThreadPool("ShardStateActionTest"); + THREAD_POOL = new TestThreadPool("ShardStateActionTest"); } @Override diff --git a/core/src/test/java/org/elasticsearch/cluster/allocation/ClusterRerouteIT.java b/core/src/test/java/org/elasticsearch/cluster/allocation/ClusterRerouteIT.java index 0d43580c9bc..c86535e40c5 100644 --- a/core/src/test/java/org/elasticsearch/cluster/allocation/ClusterRerouteIT.java +++ b/core/src/test/java/org/elasticsearch/cluster/allocation/ClusterRerouteIT.java @@ -22,6 +22,7 @@ package org.elasticsearch.cluster.allocation; import org.apache.lucene.util.IOUtils; import org.elasticsearch.action.admin.cluster.health.ClusterHealthResponse; import org.elasticsearch.action.admin.cluster.reroute.ClusterRerouteResponse; +import org.elasticsearch.action.support.WriteRequest.RefreshPolicy; import org.elasticsearch.cluster.ClusterState; import org.elasticsearch.cluster.health.ClusterHealthStatus; import org.elasticsearch.cluster.node.DiscoveryNode; @@ -225,7 +226,7 @@ public class ClusterRerouteIT extends ESIntegTestCase { assertThat(state.getRoutingNodes().unassigned().size(), equalTo(1)); assertThat(state.getRoutingNodes().node(state.nodes().resolveNode(node_1).getId()).iterator().next().state(), equalTo(ShardRoutingState.STARTED)); - client().prepareIndex("test", "type", "1").setSource("field", "value").setRefresh(true).execute().actionGet(); + client().prepareIndex("test", "type", "1").setSource("field", "value").setRefreshPolicy(RefreshPolicy.IMMEDIATE).get(); final Index index = resolveIndex("test"); logger.info("--> closing all nodes"); diff --git a/core/src/test/java/org/elasticsearch/cluster/health/ClusterStateHealthTests.java b/core/src/test/java/org/elasticsearch/cluster/health/ClusterStateHealthTests.java index dde35e33993..e99c862bfec 100644 --- a/core/src/test/java/org/elasticsearch/cluster/health/ClusterStateHealthTests.java +++ b/core/src/test/java/org/elasticsearch/cluster/health/ClusterStateHealthTests.java @@ -41,6 +41,7 @@ import org.elasticsearch.common.settings.Settings; import org.elasticsearch.test.ESTestCase; import org.elasticsearch.test.gateway.NoopGatewayAllocator; import org.elasticsearch.test.transport.CapturingTransport; +import org.elasticsearch.threadpool.TestThreadPool; import org.elasticsearch.threadpool.ThreadPool; import org.elasticsearch.transport.TransportService; import org.junit.After; @@ -72,7 +73,7 @@ public class ClusterStateHealthTests extends ESTestCase { @BeforeClass public static void beforeClass() { - threadPool = new ThreadPool("ClusterStateHealthTests"); + threadPool = new TestThreadPool("ClusterStateHealthTests"); } @Override diff --git a/core/src/test/java/org/elasticsearch/cluster/metadata/IndexMetaDataTests.java b/core/src/test/java/org/elasticsearch/cluster/metadata/IndexMetaDataTests.java new file mode 100644 index 00000000000..0c9827587ea --- /dev/null +++ b/core/src/test/java/org/elasticsearch/cluster/metadata/IndexMetaDataTests.java @@ -0,0 +1,126 @@ +/* + * 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.cluster.metadata; + +import org.elasticsearch.common.io.stream.BytesStreamOutput; +import org.elasticsearch.common.io.stream.StreamInput; +import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.common.util.set.Sets; +import org.elasticsearch.common.xcontent.ToXContent; +import org.elasticsearch.common.xcontent.XContentBuilder; +import org.elasticsearch.common.xcontent.XContentParser; +import org.elasticsearch.common.xcontent.XContentType; +import org.elasticsearch.common.xcontent.json.JsonXContent; +import org.elasticsearch.index.shard.ShardId; +import org.elasticsearch.test.ESTestCase; + +import java.io.IOException; +import java.util.Set; + +public class IndexMetaDataTests extends ESTestCase { + + public void testIndexMetaDataSerialization() throws IOException { + Integer numShard = randomFrom(1, 2, 4, 8, 16); + int numberOfReplicas = randomIntBetween(0, 10); + IndexMetaData metaData = IndexMetaData.builder("foo") + .settings(Settings.builder() + .put("index.version.created", 1) + .put("index.number_of_shards", numShard) + .put("index.number_of_replicas", numberOfReplicas) + .build()) + .creationDate(randomLong()) + .primaryTerm(0, 2) + .setRoutingNumShards(32) + .build(); + + final XContentBuilder builder = JsonXContent.contentBuilder(); + builder.startObject(); + metaData.toXContent(builder, ToXContent.EMPTY_PARAMS); + builder.endObject(); + XContentParser parser = XContentType.JSON.xContent().createParser(builder.bytes()); + final IndexMetaData fromXContentMeta = IndexMetaData.PROTO.fromXContent(parser, null); + assertEquals(metaData, fromXContentMeta); + assertEquals(metaData.hashCode(), fromXContentMeta.hashCode()); + + assertEquals(metaData.getNumberOfReplicas(), fromXContentMeta.getNumberOfReplicas()); + assertEquals(metaData.getNumberOfShards(), fromXContentMeta.getNumberOfShards()); + assertEquals(metaData.getCreationVersion(), fromXContentMeta.getCreationVersion()); + assertEquals(metaData.getRoutingNumShards(), fromXContentMeta.getRoutingNumShards()); + assertEquals(metaData.getCreationDate(), fromXContentMeta.getCreationDate()); + assertEquals(metaData.getRoutingFactor(), fromXContentMeta.getRoutingFactor()); + assertEquals(metaData.primaryTerm(0), fromXContentMeta.primaryTerm(0)); + + final BytesStreamOutput out = new BytesStreamOutput(); + metaData.writeTo(out); + IndexMetaData deserialized = IndexMetaData.PROTO.readFrom(StreamInput.wrap(out.bytes())); + assertEquals(metaData, deserialized); + assertEquals(metaData.hashCode(), deserialized.hashCode()); + + assertEquals(metaData.getNumberOfReplicas(), deserialized.getNumberOfReplicas()); + assertEquals(metaData.getNumberOfShards(), deserialized.getNumberOfShards()); + assertEquals(metaData.getCreationVersion(), deserialized.getCreationVersion()); + assertEquals(metaData.getRoutingNumShards(), deserialized.getRoutingNumShards()); + assertEquals(metaData.getCreationDate(), deserialized.getCreationDate()); + assertEquals(metaData.getRoutingFactor(), deserialized.getRoutingFactor()); + assertEquals(metaData.primaryTerm(0), deserialized.primaryTerm(0)); + } + + public void testGetRoutingFactor() { + int numberOfReplicas = randomIntBetween(0, 10); + IndexMetaData metaData = IndexMetaData.builder("foo") + .settings(Settings.builder() + .put("index.version.created", 1) + .put("index.number_of_shards", 32) + .put("index.number_of_replicas", numberOfReplicas) + .build()) + .creationDate(randomLong()) + .build(); + Integer numShard = randomFrom(1, 2, 4, 8, 16); + int routingFactor = IndexMetaData.getRoutingFactor(metaData, numShard); + assertEquals(routingFactor * numShard, metaData.getNumberOfShards()); + + Integer brokenNumShards = randomFrom(3, 5, 9, 12, 29, 42, 64); + expectThrows(IllegalArgumentException.class, () -> IndexMetaData.getRoutingFactor(metaData, brokenNumShards)); + } + + public void testSelectShrinkShards() { + int numberOfReplicas = randomIntBetween(0, 10); + IndexMetaData metaData = IndexMetaData.builder("foo") + .settings(Settings.builder() + .put("index.version.created", 1) + .put("index.number_of_shards", 32) + .put("index.number_of_replicas", numberOfReplicas) + .build()) + .creationDate(randomLong()) + .build(); + Set shardIds = IndexMetaData.selectShrinkShards(0, metaData, 8); + assertEquals(shardIds, Sets.newHashSet(new ShardId(metaData.getIndex(), 0), new ShardId(metaData.getIndex(), 1), + new ShardId(metaData.getIndex(), 2), new ShardId(metaData.getIndex(), 3))); + shardIds = IndexMetaData.selectShrinkShards(1, metaData, 8); + assertEquals(shardIds, Sets.newHashSet(new ShardId(metaData.getIndex(), 4), new ShardId(metaData.getIndex(), 5), + new ShardId(metaData.getIndex(), 6), new ShardId(metaData.getIndex(), 7))); + shardIds = IndexMetaData.selectShrinkShards(7, metaData, 8); + assertEquals(shardIds, Sets.newHashSet(new ShardId(metaData.getIndex(), 28), new ShardId(metaData.getIndex(), 29), + new ShardId(metaData.getIndex(), 30), new ShardId(metaData.getIndex(), 31))); + + assertEquals("the number of target shards (8) must be greater than the shard id: 8", + expectThrows(IllegalArgumentException.class, () -> IndexMetaData.selectShrinkShards(8, metaData, 8)).getMessage()); + } +} diff --git a/core/src/test/java/org/elasticsearch/cluster/metadata/MetaDataCreateIndexServiceTests.java b/core/src/test/java/org/elasticsearch/cluster/metadata/MetaDataCreateIndexServiceTests.java index aa226e311dd..53dffc799b1 100644 --- a/core/src/test/java/org/elasticsearch/cluster/metadata/MetaDataCreateIndexServiceTests.java +++ b/core/src/test/java/org/elasticsearch/cluster/metadata/MetaDataCreateIndexServiceTests.java @@ -66,8 +66,15 @@ public class MetaDataCreateIndexServiceTests extends ESTestCase { return clusterState; } + public static boolean isShrinkable(int source, int target) { + int x = source / target; + assert source > target : source + " <= " + target; + return target * x == source; + } + public void testValidateShrinkIndex() { - ClusterState state = createClusterState("source", randomIntBetween(2, 100), randomIntBetween(0, 10), + int numShards = randomIntBetween(2, 42); + ClusterState state = createClusterState("source", numShards, randomIntBetween(0, 10), Settings.builder().put("index.blocks.write", true).build()); assertEquals("index [source] already exists", @@ -81,12 +88,18 @@ public class MetaDataCreateIndexServiceTests extends ESTestCase { ).getMessage()); assertEquals("can't shrink an index with only one shard", - expectThrows(IllegalArgumentException.class, () -> - MetaDataCreateIndexService.validateShrinkIndex(createClusterState("source", 1, 0, - Settings.builder().put("index.blocks.write", true).build()), "source", Collections.emptySet(), + expectThrows(IllegalArgumentException.class, () -> MetaDataCreateIndexService.validateShrinkIndex(createClusterState("source", + 1, 0, Settings.builder().put("index.blocks.write", true).build()), "source", Collections.emptySet(), "target", Settings.EMPTY) ).getMessage()); + assertEquals("the number of target shards must be less that the number of source shards", + expectThrows(IllegalArgumentException.class, () -> MetaDataCreateIndexService.validateShrinkIndex(createClusterState("source", + 5, 0, Settings.builder().put("index.blocks.write", true).build()), "source", Collections.emptySet(), + "target", Settings.builder().put("index.number_of_shards", 10).build()) + ).getMessage()); + + assertEquals("index source must be read-only to shrink index. use \"index.blocks.write=true\"", expectThrows(IllegalStateException.class, () -> MetaDataCreateIndexService.validateShrinkIndex( @@ -99,11 +112,11 @@ public class MetaDataCreateIndexServiceTests extends ESTestCase { MetaDataCreateIndexService.validateShrinkIndex(state, "source", Collections.emptySet(), "target", Settings.EMPTY) ).getMessage()); - - assertEquals("can not shrink index into more than one shard", + assertEquals("the number of source shards [8] must be a must be a multiple of [3]", expectThrows(IllegalArgumentException.class, () -> - MetaDataCreateIndexService.validateShrinkIndex(state, "source", Collections.emptySet(), "target", - Settings.builder().put("index.number_of_shards", 2).build()) + MetaDataCreateIndexService.validateShrinkIndex(createClusterState("source", 8, randomIntBetween(0, 10), + Settings.builder().put("index.blocks.write", true).build()), "source", Collections.emptySet(), "target", + Settings.builder().put("index.number_of_shards", 3).build()) ).getMessage()); assertEquals("mappings are not allowed when shrinking indices, all mappings are copied from the source index", @@ -114,7 +127,7 @@ public class MetaDataCreateIndexServiceTests extends ESTestCase { ).getMessage()); // create one that won't fail - ClusterState clusterState = ClusterState.builder(createClusterState("source", randomIntBetween(2, 10), 0, + ClusterState clusterState = ClusterState.builder(createClusterState("source", numShards, 0, Settings.builder().put("index.blocks.write", true).build())).nodes(DiscoveryNodes.builder().put(newNode("node1"))) .build(); AllocationService service = new AllocationService(Settings.builder().build(), new AllocationDeciders(Settings.EMPTY, @@ -127,8 +140,12 @@ public class MetaDataCreateIndexServiceTests extends ESTestCase { routingTable = service.applyStartedShards(clusterState, routingTable.index("source").shardsWithState(ShardRoutingState.INITIALIZING)).routingTable(); clusterState = ClusterState.builder(clusterState).routingTable(routingTable).build(); - - MetaDataCreateIndexService.validateShrinkIndex(clusterState, "source", Collections.emptySet(), "target", Settings.EMPTY); + int targetShards; + do { + targetShards = randomIntBetween(1, numShards/2); + } while (isShrinkable(numShards, targetShards) == false); + MetaDataCreateIndexService.validateShrinkIndex(clusterState, "source", Collections.emptySet(), "target", + Settings.builder().put("index.number_of_shards", targetShards).build()); } public void testShrinkIndexSettings() { @@ -155,7 +172,6 @@ public class MetaDataCreateIndexServiceTests extends ESTestCase { Settings.Builder builder = Settings.builder(); MetaDataCreateIndexService.prepareShrinkIndexSettings( clusterState, Collections.emptySet(), builder, clusterState.metaData().index(indexName).getIndex(), "target"); - assertEquals("1", builder.build().get("index.number_of_shards")); assertEquals("similarity settings must be copied", "BM25", builder.build().get("index.similarity.default.type")); assertEquals("analysis settings must be copied", "keyword", builder.build().get("index.analysis.analyzer.my_analyzer.tokenizer")); diff --git a/core/src/test/java/org/elasticsearch/cluster/routing/DelayedAllocationServiceTests.java b/core/src/test/java/org/elasticsearch/cluster/routing/DelayedAllocationServiceTests.java index 630e5b034af..2e1cdea56a4 100644 --- a/core/src/test/java/org/elasticsearch/cluster/routing/DelayedAllocationServiceTests.java +++ b/core/src/test/java/org/elasticsearch/cluster/routing/DelayedAllocationServiceTests.java @@ -37,6 +37,7 @@ import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.unit.TimeValue; import org.elasticsearch.gateway.GatewayAllocator; import org.elasticsearch.test.ESAllocationTestCase; +import org.elasticsearch.threadpool.TestThreadPool; import org.elasticsearch.threadpool.ThreadPool; import org.junit.After; import org.junit.Before; @@ -73,7 +74,7 @@ public class DelayedAllocationServiceTests extends ESAllocationTestCase { @Before public void createDelayedAllocationService() { - threadPool = new ThreadPool(getTestName()); + threadPool = new TestThreadPool(getTestName()); clusterService = mock(ClusterService.class); allocationService = createAllocationService(Settings.EMPTY, new DelayedShardsMockGatewayAllocator()); delayedAllocationService = new TestDelayAllocationService(Settings.EMPTY, threadPool, clusterService, allocationService); diff --git a/core/src/test/java/org/elasticsearch/cluster/routing/OperationRoutingTests.java b/core/src/test/java/org/elasticsearch/cluster/routing/OperationRoutingTests.java new file mode 100644 index 00000000000..e515e0f9bf4 --- /dev/null +++ b/core/src/test/java/org/elasticsearch/cluster/routing/OperationRoutingTests.java @@ -0,0 +1,173 @@ +/* + * 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.cluster.routing; + +import org.elasticsearch.Version; +import org.elasticsearch.cluster.metadata.IndexMetaData; +import org.elasticsearch.index.shard.ShardId; +import org.elasticsearch.test.ESTestCase; + +import java.util.Arrays; +import java.util.Map; +import java.util.Set; +import java.util.TreeMap; + +public class OperationRoutingTests extends ESTestCase{ + + public void testGenerateShardId() { + int[][] possibleValues = new int[][] { + {8,4,2}, {20, 10, 2}, {36, 12, 3}, {15,5,1} + }; + for (int i = 0; i < 10; i++) { + int[] shardSplits = randomFrom(possibleValues); + assertEquals(shardSplits[0], (shardSplits[0] / shardSplits[1]) * shardSplits[1]); + assertEquals(shardSplits[1], (shardSplits[1] / shardSplits[2]) * shardSplits[2]); + IndexMetaData metaData = IndexMetaData.builder("test").settings(settings(Version.CURRENT)).numberOfShards(shardSplits[0]) + .numberOfReplicas(1).build(); + String term = randomAsciiOfLength(10); + final int shard = OperationRouting.generateShardId(metaData, term, null); + IndexMetaData shrunk = IndexMetaData.builder("test").settings(settings(Version.CURRENT)).numberOfShards(shardSplits[1]) + .numberOfReplicas(1) + .setRoutingNumShards(shardSplits[0]).build(); + int shrunkShard = OperationRouting.generateShardId(shrunk, term, null); + Set shardIds = IndexMetaData.selectShrinkShards(shrunkShard, metaData, shrunk.getNumberOfShards()); + assertEquals(1, shardIds.stream().filter((sid) -> sid.id() == shard).count()); + + shrunk = IndexMetaData.builder("test").settings(settings(Version.CURRENT)).numberOfShards(shardSplits[2]).numberOfReplicas(1) + .setRoutingNumShards(shardSplits[0]).build(); + shrunkShard = OperationRouting.generateShardId(shrunk, term, null); + shardIds = IndexMetaData.selectShrinkShards(shrunkShard, metaData, shrunk.getNumberOfShards()); + assertEquals(Arrays.toString(shardSplits), 1, shardIds.stream().filter((sid) -> sid.id() == shard).count()); + } + } + + /** + * Ensures that all changes to the hash-function / shard selection are BWC + */ + public void testBWC() { + Map termToShard = new TreeMap<>(); + termToShard.put("sEERfFzPSI", 1); + termToShard.put("cNRiIrjzYd", 7); + termToShard.put("BgfLBXUyWT", 5); + termToShard.put("cnepjZhQnb", 3); + termToShard.put("OKCmuYkeCK", 6); + termToShard.put("OutXGRQUja", 5); + termToShard.put("yCdyocKWou", 1); + termToShard.put("KXuNWWNgVj", 2); + termToShard.put("DGJOYrpESx", 4); + termToShard.put("upLDybdTGs", 5); + termToShard.put("yhZhzCPQby", 1); + termToShard.put("EyCVeiCouA", 1); + termToShard.put("tFyVdQauWR", 6); + termToShard.put("nyeRYDnDQr", 6); + termToShard.put("hswhrppvDH", 0); + termToShard.put("BSiWvDOsNE", 5); + termToShard.put("YHicpFBSaY", 1); + termToShard.put("EquPtdKaBZ", 4); + termToShard.put("rSjLZHCDfT", 5); + termToShard.put("qoZALVcite", 7); + termToShard.put("yDCCPVBiCm", 7); + termToShard.put("ngizYtQgGK", 5); + termToShard.put("FYQRIBcNqz", 0); + termToShard.put("EBzEDAPODe", 2); + termToShard.put("YePigbXgKb", 1); + termToShard.put("PeGJjomyik", 3); + termToShard.put("cyQIvDmyYD", 7); + termToShard.put("yIEfZrYfRk", 5); + termToShard.put("kblouyFUbu", 7); + termToShard.put("xvIGbRiGJF", 3); + termToShard.put("KWimwsREPf", 4); + termToShard.put("wsNavvIcdk", 7); + termToShard.put("xkWaPcCmpT", 0); + termToShard.put("FKKTOnJMDy", 7); + termToShard.put("RuLzobYixn", 2); + termToShard.put("mFohLeFRvF", 4); + termToShard.put("aAMXnamRJg", 7); + termToShard.put("zKBMYJDmBI", 0); + termToShard.put("ElSVuJQQuw", 7); + termToShard.put("pezPtTQAAm", 7); + termToShard.put("zBjjNEjAex", 2); + termToShard.put("PGgHcLNPYX", 7); + termToShard.put("hOkpeQqTDF", 3); + termToShard.put("chZXraUPBH", 7); + termToShard.put("FAIcSmmNXq", 5); + termToShard.put("EZmDicyayC", 0); + termToShard.put("GRIueBeIyL", 7); + termToShard.put("qCChjGZYLp", 3); + termToShard.put("IsSZQwwnUT", 3); + termToShard.put("MGlxLFyyCK", 3); + termToShard.put("YmscwrKSpB", 0); + termToShard.put("czSljcjMop", 5); + termToShard.put("XhfGWwNlng", 1); + termToShard.put("cWpKJjlzgj", 7); + termToShard.put("eDzIfMKbvk", 1); + termToShard.put("WFFWYBfnTb", 0); + termToShard.put("oDdHJxGxja", 7); + termToShard.put("PDOQQqgIKE", 1); + termToShard.put("bGEIEBLATe", 6); + termToShard.put("xpRkJPWVpu", 2); + termToShard.put("kTwZnPEeIi", 2); + termToShard.put("DifcuqSsKk", 1); + termToShard.put("CEmLmljpXe", 5); + termToShard.put("cuNKtLtyJQ", 7); + termToShard.put("yNjiAnxAmt", 5); + termToShard.put("bVDJDCeaFm", 2); + termToShard.put("vdnUhGLFtl", 0); + termToShard.put("LnqSYezXbr", 5); + termToShard.put("EzHgydDCSR", 3); + termToShard.put("ZSKjhJlcpn", 1); + termToShard.put("WRjUoZwtUz", 3); + termToShard.put("RiBbcCdIgk", 4); + termToShard.put("yizTqyjuDn", 4); + termToShard.put("QnFjcpcZUT", 4); + termToShard.put("agYhXYUUpl", 7); + termToShard.put("UOjiTugjNC", 7); + termToShard.put("nICGuWTdfV", 0); + termToShard.put("NrnSmcnUVF", 2); + termToShard.put("ZSzFcbpDqP", 3); + termToShard.put("YOhahLSzzE", 5); + termToShard.put("iWswCilUaT", 1); + termToShard.put("zXAamKsRwj", 2); + termToShard.put("aqGsrUPHFq", 5); + termToShard.put("eDItImYWTS", 1); + termToShard.put("JAYDZMRcpW", 4); + termToShard.put("lmvAaEPflK", 7); + termToShard.put("IKuOwPjKCx", 5); + termToShard.put("schsINzlYB", 1); + termToShard.put("OqbFNxrKrF", 2); + termToShard.put("QrklDfvEJU", 6); + termToShard.put("VLxKRKdLbx", 4); + termToShard.put("imoydNTZhV", 1); + termToShard.put("uFZyTyOMRO", 4); + termToShard.put("nVAZVMPNNx", 3); + termToShard.put("rPIdESYaAO", 5); + termToShard.put("nbZWPWJsIM", 0); + termToShard.put("wRZXPSoEgd", 3); + termToShard.put("nGzpgwsSBc", 4); + termToShard.put("AITyyoyLLs", 4); + IndexMetaData metaData = IndexMetaData.builder("test").settings(settings(Version.CURRENT)).numberOfShards(8) + .numberOfReplicas(1).build(); + for (Map.Entry entry : termToShard.entrySet()) { + String key = entry.getKey(); + int shard = randomBoolean() ? + OperationRouting.generateShardId(metaData, key, null) : OperationRouting.generateShardId(metaData, "foobar", key); + assertEquals(shard, entry.getValue().intValue()); + } + } +} diff --git a/core/src/test/java/org/elasticsearch/cluster/routing/allocation/decider/DiskThresholdDeciderUnitTests.java b/core/src/test/java/org/elasticsearch/cluster/routing/allocation/decider/DiskThresholdDeciderUnitTests.java index 52ba45ab7ab..8966659e63c 100644 --- a/core/src/test/java/org/elasticsearch/cluster/routing/allocation/decider/DiskThresholdDeciderUnitTests.java +++ b/core/src/test/java/org/elasticsearch/cluster/routing/allocation/decider/DiskThresholdDeciderUnitTests.java @@ -298,17 +298,22 @@ public class DiskThresholdDeciderUnitTests extends ESAllocationTestCase { ImmutableOpenMap.Builder shardSizes = ImmutableOpenMap.builder(); shardSizes.put("[test][0][p]", 10L); shardSizes.put("[test][1][p]", 100L); - shardSizes.put("[test][2][p]", 1000L); + shardSizes.put("[test][2][p]", 500L); + shardSizes.put("[test][3][p]", 500L); + ClusterInfo info = new DevNullClusterInfo(ImmutableOpenMap.of(), ImmutableOpenMap.of(), shardSizes.build()); MetaData.Builder metaBuilder = MetaData.builder(); metaBuilder.put(IndexMetaData.builder("test").settings(settings(Version.CURRENT).put("index.uuid", "1234")) - .numberOfShards(3).numberOfReplicas(0)); + .numberOfShards(4).numberOfReplicas(0)); metaBuilder.put(IndexMetaData.builder("target").settings(settings(Version.CURRENT).put("index.uuid", "5678") .put("index.shrink.source.name", "test").put("index.shrink.source.uuid", "1234")).numberOfShards(1).numberOfReplicas(0)); + metaBuilder.put(IndexMetaData.builder("target2").settings(settings(Version.CURRENT).put("index.uuid", "9101112") + .put("index.shrink.source.name", "test").put("index.shrink.source.uuid", "1234")).numberOfShards(2).numberOfReplicas(0)); MetaData metaData = metaBuilder.build(); RoutingTable.Builder routingTableBuilder = RoutingTable.builder(); routingTableBuilder.addAsNew(metaData.index("test")); routingTableBuilder.addAsNew(metaData.index("target")); + routingTableBuilder.addAsNew(metaData.index("target2")); ClusterState clusterState = ClusterState.builder(org.elasticsearch.cluster.ClusterName.DEFAULT) .metaData(metaData).routingTable(routingTableBuilder.build()).build(); @@ -339,18 +344,26 @@ public class DiskThresholdDeciderUnitTests extends ESAllocationTestCase { new UnassignedInfo(UnassignedInfo.Reason.INDEX_CREATED, "foo")); test_2 = ShardRoutingHelper.initialize(test_2, "node1"); - assertEquals(1000L, DiskThresholdDecider.getExpectedShardSize(test_2, allocation, 0)); + ShardRouting test_3 = ShardRouting.newUnassigned(new ShardId(index, 3), null, true, + new UnassignedInfo(UnassignedInfo.Reason.INDEX_CREATED, "foo")); + test_3 = ShardRoutingHelper.initialize(test_3, "node1"); + assertEquals(500L, DiskThresholdDecider.getExpectedShardSize(test_3, allocation, 0)); + assertEquals(500L, DiskThresholdDecider.getExpectedShardSize(test_2, allocation, 0)); assertEquals(100L, DiskThresholdDecider.getExpectedShardSize(test_1, allocation, 0)); assertEquals(10L, DiskThresholdDecider.getExpectedShardSize(test_0, allocation, 0)); ShardRouting target = ShardRouting.newUnassigned(new ShardId(new Index("target", "5678"), 0), null, true, new UnassignedInfo(UnassignedInfo.Reason.INDEX_CREATED, "foo")); - - assertEquals(1110L, DiskThresholdDecider.getExpectedShardSize(target, allocation, 0)); + ShardRouting target2 = ShardRouting.newUnassigned(new ShardId(new Index("target2", "9101112"), 0), + null, true, new UnassignedInfo(UnassignedInfo.Reason.INDEX_CREATED, "foo")); + assertEquals(110L, DiskThresholdDecider.getExpectedShardSize(target2, allocation, 0)); + target2 = ShardRouting.newUnassigned(new ShardId(new Index("target2", "9101112"), 1), + null, true, new UnassignedInfo(UnassignedInfo.Reason.INDEX_CREATED, "foo")); + assertEquals(1000L, DiskThresholdDecider.getExpectedShardSize(target2, allocation, 0)); } } diff --git a/core/src/test/java/org/elasticsearch/cluster/service/ClusterServiceTests.java b/core/src/test/java/org/elasticsearch/cluster/service/ClusterServiceTests.java index 81fc9d3752d..bf85f67e48e 100644 --- a/core/src/test/java/org/elasticsearch/cluster/service/ClusterServiceTests.java +++ b/core/src/test/java/org/elasticsearch/cluster/service/ClusterServiceTests.java @@ -42,6 +42,7 @@ import org.elasticsearch.common.unit.TimeValue; import org.elasticsearch.test.ESTestCase; import org.elasticsearch.test.MockLogAppender; import org.elasticsearch.test.junit.annotations.TestLogging; +import org.elasticsearch.threadpool.TestThreadPool; import org.elasticsearch.threadpool.ThreadPool; import org.junit.After; import org.junit.AfterClass; @@ -76,7 +77,7 @@ public class ClusterServiceTests extends ESTestCase { @BeforeClass public static void createThreadPool() { - threadPool = new ThreadPool(ClusterServiceTests.class.getName()); + threadPool = new TestThreadPool(ClusterServiceTests.class.getName()); } @AfterClass diff --git a/core/src/test/java/org/elasticsearch/common/rounding/TimeZoneRoundingTests.java b/core/src/test/java/org/elasticsearch/common/rounding/TimeZoneRoundingTests.java index af7b53712c1..49ba26231da 100644 --- a/core/src/test/java/org/elasticsearch/common/rounding/TimeZoneRoundingTests.java +++ b/core/src/test/java/org/elasticsearch/common/rounding/TimeZoneRoundingTests.java @@ -258,6 +258,57 @@ public class TimeZoneRoundingTests extends ESTestCase { } } + /** + * test DST end with interval rounding + * CET: 25 October 2015, 03:00:00 clocks were turned backward 1 hour to 25 October 2015, 02:00:00 local standard time + */ + public void testTimeIntervalCET_DST_End() { + long interval = TimeUnit.MINUTES.toMillis(20); + TimeZoneRounding rounding = new TimeIntervalRounding(interval, DateTimeZone.forID("CET")); + + assertThat(rounding.round(time("2015-10-25T01:55:00+02:00")), equalTo(time("2015-10-25T01:40:00+02:00"))); + assertThat(rounding.round(time("2015-10-25T02:15:00+02:00")), equalTo(time("2015-10-25T02:00:00+02:00"))); + assertThat(rounding.round(time("2015-10-25T02:35:00+02:00")), equalTo(time("2015-10-25T02:20:00+02:00"))); + assertThat(rounding.round(time("2015-10-25T02:55:00+02:00")), equalTo(time("2015-10-25T02:40:00+02:00"))); + // after DST shift + assertThat(rounding.round(time("2015-10-25T02:15:00+01:00")), equalTo(time("2015-10-25T02:00:00+01:00"))); + assertThat(rounding.round(time("2015-10-25T02:35:00+01:00")), equalTo(time("2015-10-25T02:20:00+01:00"))); + assertThat(rounding.round(time("2015-10-25T02:55:00+01:00")), equalTo(time("2015-10-25T02:40:00+01:00"))); + assertThat(rounding.round(time("2015-10-25T03:15:00+01:00")), equalTo(time("2015-10-25T03:00:00+01:00"))); + } + + /** + * test DST start with interval rounding + * CET: 27 March 2016, 02:00:00 clocks were turned forward 1 hour to 27 March 2016, 03:00:00 local daylight time + */ + public void testTimeIntervalCET_DST_Start() { + long interval = TimeUnit.MINUTES.toMillis(20); + TimeZoneRounding rounding = new TimeIntervalRounding(interval, DateTimeZone.forID("CET")); + // test DST start + assertThat(rounding.round(time("2016-03-27T01:55:00+01:00")), equalTo(time("2016-03-27T01:40:00+01:00"))); + assertThat(rounding.round(time("2016-03-27T02:00:00+01:00")), equalTo(time("2016-03-27T03:00:00+02:00"))); + assertThat(rounding.round(time("2016-03-27T03:15:00+02:00")), equalTo(time("2016-03-27T03:00:00+02:00"))); + assertThat(rounding.round(time("2016-03-27T03:35:00+02:00")), equalTo(time("2016-03-27T03:20:00+02:00"))); + } + + /** + * test DST start with offset not fitting interval, e.g. Asia/Kathmandu + * adding 15min on 1986-01-01T00:00:00 the interval from + * 1986-01-01T00:15:00+05:45 to 1986-01-01T00:20:00+05:45 to only be 5min + * long + */ + public void testTimeInterval_Kathmandu_DST_Start() { + long interval = TimeUnit.MINUTES.toMillis(20); + TimeZoneRounding rounding = new TimeIntervalRounding(interval, DateTimeZone.forID("Asia/Kathmandu")); + assertThat(rounding.round(time("1985-12-31T23:55:00+05:30")), equalTo(time("1985-12-31T23:40:00+05:30"))); + assertThat(rounding.round(time("1986-01-01T00:16:00+05:45")), equalTo(time("1986-01-01T00:15:00+05:45"))); + assertThat(time("1986-01-01T00:15:00+05:45") - time("1985-12-31T23:40:00+05:30"), equalTo(TimeUnit.MINUTES.toMillis(20))); + assertThat(rounding.round(time("1986-01-01T00:26:00+05:45")), equalTo(time("1986-01-01T00:20:00+05:45"))); + assertThat(time("1986-01-01T00:20:00+05:45") - time("1986-01-01T00:15:00+05:45"), equalTo(TimeUnit.MINUTES.toMillis(5))); + assertThat(rounding.round(time("1986-01-01T00:46:00+05:45")), equalTo(time("1986-01-01T00:40:00+05:45"))); + assertThat(time("1986-01-01T00:40:00+05:45") - time("1986-01-01T00:20:00+05:45"), equalTo(TimeUnit.MINUTES.toMillis(20))); + } + /** * randomized test on {@link TimeIntervalRounding} with random interval and time zone offsets */ diff --git a/core/src/test/java/org/elasticsearch/common/property/PropertyPlaceholderTests.java b/core/src/test/java/org/elasticsearch/common/settings/PropertyPlaceholderTests.java similarity index 78% rename from core/src/test/java/org/elasticsearch/common/property/PropertyPlaceholderTests.java rename to core/src/test/java/org/elasticsearch/common/settings/PropertyPlaceholderTests.java index 405ac566686..78176bc1d80 100644 --- a/core/src/test/java/org/elasticsearch/common/property/PropertyPlaceholderTests.java +++ b/core/src/test/java/org/elasticsearch/common/settings/PropertyPlaceholderTests.java @@ -17,14 +17,13 @@ * under the License. */ -package org.elasticsearch.common.property; - -import org.elasticsearch.test.ESTestCase; +package org.elasticsearch.common.settings; import java.util.LinkedHashMap; import java.util.Map; -import static org.hamcrest.Matchers.hasToString; +import org.elasticsearch.test.ESTestCase; + import static org.hamcrest.Matchers.is; public class PropertyPlaceholderTests extends ESTestCase { @@ -34,10 +33,10 @@ public class PropertyPlaceholderTests extends ESTestCase { map.put("foo1", "bar1"); map.put("foo2", "bar2"); PropertyPlaceholder.PlaceholderResolver placeholderResolver = new SimplePlaceholderResolver(map, false, true); - assertEquals("bar1", propertyPlaceholder.replacePlaceholders("key", "{foo1}", placeholderResolver)); - assertEquals("a bar1b", propertyPlaceholder.replacePlaceholders("key", "a {foo1}b", placeholderResolver)); - assertEquals("bar1bar2", propertyPlaceholder.replacePlaceholders("key", "{foo1}{foo2}", placeholderResolver)); - assertEquals("a bar1 b bar2 c", propertyPlaceholder.replacePlaceholders("key", "a {foo1} b {foo2} c", placeholderResolver)); + assertEquals("bar1", propertyPlaceholder.replacePlaceholders("{foo1}", placeholderResolver)); + assertEquals("a bar1b", propertyPlaceholder.replacePlaceholders("a {foo1}b", placeholderResolver)); + assertEquals("bar1bar2", propertyPlaceholder.replacePlaceholders("{foo1}{foo2}", placeholderResolver)); + assertEquals("a bar1 b bar2 c", propertyPlaceholder.replacePlaceholders("a {foo1} b {foo2} c", placeholderResolver)); } public void testVariousPrefixSuffix() { @@ -48,24 +47,24 @@ public class PropertyPlaceholderTests extends ESTestCase { Map map = new LinkedHashMap<>(); map.put("foo", "bar"); PropertyPlaceholder.PlaceholderResolver placeholderResolver = new SimplePlaceholderResolver(map, false, true); - assertEquals("bar", ppEqualsPrefix.replacePlaceholders("key", "{foo}", placeholderResolver)); - assertEquals("bar", ppLongerPrefix.replacePlaceholders("key", "${foo}", placeholderResolver)); - assertEquals("bar", ppShorterPrefix.replacePlaceholders("key", "{foo}}", placeholderResolver)); + assertEquals("bar", ppEqualsPrefix.replacePlaceholders("{foo}", placeholderResolver)); + assertEquals("bar", ppLongerPrefix.replacePlaceholders("${foo}", placeholderResolver)); + assertEquals("bar", ppShorterPrefix.replacePlaceholders("{foo}}", placeholderResolver)); } public void testDefaultValue() { PropertyPlaceholder propertyPlaceholder = new PropertyPlaceholder("${", "}", false); Map map = new LinkedHashMap<>(); PropertyPlaceholder.PlaceholderResolver placeholderResolver = new SimplePlaceholderResolver(map, false, true); - assertEquals("bar", propertyPlaceholder.replacePlaceholders("key", "${foo:bar}", placeholderResolver)); - assertEquals("", propertyPlaceholder.replacePlaceholders("key", "${foo:}", placeholderResolver)); + assertEquals("bar", propertyPlaceholder.replacePlaceholders("${foo:bar}", placeholderResolver)); + assertEquals("", propertyPlaceholder.replacePlaceholders("${foo:}", placeholderResolver)); } public void testIgnoredUnresolvedPlaceholder() { PropertyPlaceholder propertyPlaceholder = new PropertyPlaceholder("${", "}", true); Map map = new LinkedHashMap<>(); PropertyPlaceholder.PlaceholderResolver placeholderResolver = new SimplePlaceholderResolver(map, false, true); - assertEquals("${foo}", propertyPlaceholder.replacePlaceholders("key", "${foo}", placeholderResolver)); + assertEquals("${foo}", propertyPlaceholder.replacePlaceholders("${foo}", placeholderResolver)); } public void testNotIgnoredUnresolvedPlaceholder() { @@ -73,7 +72,7 @@ public class PropertyPlaceholderTests extends ESTestCase { Map map = new LinkedHashMap<>(); PropertyPlaceholder.PlaceholderResolver placeholderResolver = new SimplePlaceholderResolver(map, false, true); try { - propertyPlaceholder.replacePlaceholders("key", "${foo}", placeholderResolver); + propertyPlaceholder.replacePlaceholders("${foo}", placeholderResolver); fail("Expected IllegalArgumentException"); } catch (IllegalArgumentException e) { assertThat(e.getMessage(), is("Could not resolve placeholder 'foo'")); @@ -84,7 +83,7 @@ public class PropertyPlaceholderTests extends ESTestCase { PropertyPlaceholder propertyPlaceholder = new PropertyPlaceholder("${", "}", false); Map map = new LinkedHashMap<>(); PropertyPlaceholder.PlaceholderResolver placeholderResolver = new SimplePlaceholderResolver(map, true, true); - assertEquals("bar", propertyPlaceholder.replacePlaceholders("key", "bar${foo}", placeholderResolver)); + assertEquals("bar", propertyPlaceholder.replacePlaceholders("bar${foo}", placeholderResolver)); } public void testRecursive() { @@ -94,8 +93,8 @@ public class PropertyPlaceholderTests extends ESTestCase { map.put("foo1", "${foo2}"); map.put("foo2", "bar"); PropertyPlaceholder.PlaceholderResolver placeholderResolver = new SimplePlaceholderResolver(map, false, true); - assertEquals("bar", propertyPlaceholder.replacePlaceholders("key", "${foo}", placeholderResolver)); - assertEquals("abarb", propertyPlaceholder.replacePlaceholders("key", "a${foo}b", placeholderResolver)); + assertEquals("bar", propertyPlaceholder.replacePlaceholders("${foo}", placeholderResolver)); + assertEquals("abarb", propertyPlaceholder.replacePlaceholders("a${foo}b", placeholderResolver)); } public void testNestedLongerPrefix() { @@ -106,7 +105,7 @@ public class PropertyPlaceholderTests extends ESTestCase { map.put("foo2", "bar"); map.put("barbar", "baz"); PropertyPlaceholder.PlaceholderResolver placeholderResolver = new SimplePlaceholderResolver(map, false, true); - assertEquals("baz", propertyPlaceholder.replacePlaceholders("key", "${bar${foo}}", placeholderResolver)); + assertEquals("baz", propertyPlaceholder.replacePlaceholders("${bar${foo}}", placeholderResolver)); } public void testNestedSameLengthPrefixSuffix() { @@ -117,7 +116,7 @@ public class PropertyPlaceholderTests extends ESTestCase { map.put("foo2", "bar"); map.put("barbar", "baz"); PropertyPlaceholder.PlaceholderResolver placeholderResolver = new SimplePlaceholderResolver(map, false, true); - assertEquals("baz", propertyPlaceholder.replacePlaceholders("key", "{bar{foo}}", placeholderResolver)); + assertEquals("baz", propertyPlaceholder.replacePlaceholders("{bar{foo}}", placeholderResolver)); } public void testNestedShorterPrefix() { @@ -128,7 +127,7 @@ public class PropertyPlaceholderTests extends ESTestCase { map.put("foo2", "bar"); map.put("barbar", "baz"); PropertyPlaceholder.PlaceholderResolver placeholderResolver = new SimplePlaceholderResolver(map, false, true); - assertEquals("baz", propertyPlaceholder.replacePlaceholders("key", "{bar{foo}}}}", placeholderResolver)); + assertEquals("baz", propertyPlaceholder.replacePlaceholders("{bar{foo}}}}", placeholderResolver)); } public void testCircularReference() { @@ -138,7 +137,7 @@ public class PropertyPlaceholderTests extends ESTestCase { map.put("bar", "${foo}"); PropertyPlaceholder.PlaceholderResolver placeholderResolver = new SimplePlaceholderResolver(map, false, true); try { - propertyPlaceholder.replacePlaceholders("key", "${foo}", placeholderResolver); + propertyPlaceholder.replacePlaceholders("${foo}", placeholderResolver); fail("Expected IllegalArgumentException"); } catch (IllegalArgumentException e) { assertThat(e.getMessage(), is("Circular placeholder reference 'foo' in property definitions")); @@ -149,24 +148,7 @@ public class PropertyPlaceholderTests extends ESTestCase { PropertyPlaceholder propertyPlaceholder = new PropertyPlaceholder("${", "}", false); Map map = new LinkedHashMap<>(); PropertyPlaceholder.PlaceholderResolver placeholderResolver = new SimplePlaceholderResolver(map, true, false); - assertEquals("bar${foo}", propertyPlaceholder.replacePlaceholders("key", "bar${foo}", placeholderResolver)); - } - - public void testNullKey() { - final PropertyPlaceholder propertyPlaceholder = new PropertyPlaceholder("${", "}", false); - final Map map = new LinkedHashMap<>(); - final PropertyPlaceholder.PlaceholderResolver placeholderResolver = new SimplePlaceholderResolver(map, true, false); - expectThrows(NullPointerException.class, () -> propertyPlaceholder.replacePlaceholders(null, "value", placeholderResolver)); - } - - public void testNullValue() { - final PropertyPlaceholder propertyPlaceholder = new PropertyPlaceholder("${", "}", false); - final Map map = new LinkedHashMap<>(); - final PropertyPlaceholder.PlaceholderResolver placeholderResolver = new SimplePlaceholderResolver(map, true, false); - final String key = randomAsciiOfLength(10); - NullPointerException e = - expectThrows(NullPointerException.class, () -> propertyPlaceholder.replacePlaceholders(key, null, placeholderResolver)); - assertThat(e, hasToString("java.lang.NullPointerException: value can not be null for [" + key + "]")); + assertEquals("bar${foo}", propertyPlaceholder.replacePlaceholders("bar${foo}", placeholderResolver)); } private class SimplePlaceholderResolver implements PropertyPlaceholder.PlaceholderResolver { diff --git a/core/src/test/java/org/elasticsearch/common/settings/loader/YamlSettingsLoaderTests.java b/core/src/test/java/org/elasticsearch/common/settings/loader/YamlSettingsLoaderTests.java index e06b7589423..7c956de8f9a 100644 --- a/core/src/test/java/org/elasticsearch/common/settings/loader/YamlSettingsLoaderTests.java +++ b/core/src/test/java/org/elasticsearch/common/settings/loader/YamlSettingsLoaderTests.java @@ -24,7 +24,11 @@ import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.settings.SettingsException; import org.elasticsearch.test.ESTestCase; -import static org.hamcrest.Matchers.containsString; +import java.nio.charset.StandardCharsets; +import java.nio.file.Files; +import java.nio.file.Path; +import java.util.Collections; + import static org.hamcrest.Matchers.equalTo; public class YamlSettingsLoaderTests extends ESTestCase { @@ -47,42 +51,43 @@ public class YamlSettingsLoaderTests extends ESTestCase { assertThat(settings.getAsArray("test1.test3")[1], equalTo("test3-2")); } - public void testIndentation() { - final String yaml = "/org/elasticsearch/common/settings/loader/indentation-settings.yml"; - final SettingsException e = - expectThrows( - SettingsException.class, - () -> Settings.builder().loadFromStream(yaml, getClass().getResourceAsStream(yaml)).build()); - assertThat(e.getMessage(), containsString("Failed to load settings")); + public void testIndentation() throws Exception { + String yaml = "/org/elasticsearch/common/settings/loader/indentation-settings.yml"; + ElasticsearchParseException e = expectThrows(ElasticsearchParseException.class, () -> { + Settings.builder().loadFromStream(yaml, getClass().getResourceAsStream(yaml)); + }); + assertTrue(e.getMessage(), e.getMessage().contains("malformed")); } - public void testIndentationWithExplicitDocumentStart() { - final String yaml = "/org/elasticsearch/common/settings/loader/indentation-with-explicit-document-start-settings.yml"; - final SettingsException e = - expectThrows( - SettingsException.class, - () -> Settings.builder().loadFromStream(yaml, getClass().getResourceAsStream(yaml)).build()); - assertThat(e.getMessage(), containsString("Failed to load settings")); + public void testIndentationWithExplicitDocumentStart() throws Exception { + String yaml = "/org/elasticsearch/common/settings/loader/indentation-with-explicit-document-start-settings.yml"; + ElasticsearchParseException e = expectThrows(ElasticsearchParseException.class, () -> { + Settings.builder().loadFromStream(yaml, getClass().getResourceAsStream(yaml)); + }); + assertTrue(e.getMessage(), e.getMessage().contains("malformed")); } public void testDuplicateKeysThrowsException() { - final String yaml = "foo: bar\nfoo: baz"; - final SettingsException e = expectThrows(SettingsException.class, () -> Settings.builder().loadFromSource(yaml).build()); + String yaml = "foo: bar\nfoo: baz"; + SettingsException e = expectThrows(SettingsException.class, () -> { + Settings.builder().loadFromSource(yaml); + }); assertEquals(e.getCause().getClass(), ElasticsearchParseException.class); - assertThat( - e.toString(), - containsString("duplicate settings key [foo] " + - "found at line number [2], " + - "column number [6], " + - "previous value [bar], " + - "current value [baz]")); + String msg = e.getCause().getMessage(); + assertTrue( + msg, + msg.contains("duplicate settings key [foo] found at line number [2], column number [6], " + + "previous value [bar], current value [baz]")); } - public void testNullValuedSettingThrowsException() { - final String yaml = "foo:"; - final ElasticsearchParseException e = - expectThrows(ElasticsearchParseException.class, () -> new YamlSettingsLoader(false).load(yaml)); - assertThat(e.toString(), containsString("null-valued setting found for key [foo] found at line number [1], column number [5]")); + public void testMissingValue() throws Exception { + Path tmp = createTempFile("test", ".yaml"); + Files.write(tmp, Collections.singletonList("foo: # missing value\n"), StandardCharsets.UTF_8); + ElasticsearchParseException e = expectThrows(ElasticsearchParseException.class, () -> { + Settings.builder().loadFromPath(tmp); + }); + assertTrue( + e.getMessage(), + e.getMessage().contains("null-valued setting found for key [foo] found at line number [1], column number [5]")); } - } diff --git a/core/src/test/java/org/elasticsearch/common/util/concurrent/PrioritizedExecutorsTests.java b/core/src/test/java/org/elasticsearch/common/util/concurrent/PrioritizedExecutorsTests.java index 50b7d5f775c..df51e6e2e0d 100644 --- a/core/src/test/java/org/elasticsearch/common/util/concurrent/PrioritizedExecutorsTests.java +++ b/core/src/test/java/org/elasticsearch/common/util/concurrent/PrioritizedExecutorsTests.java @@ -22,6 +22,7 @@ import org.elasticsearch.common.Priority; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.unit.TimeValue; import org.elasticsearch.test.ESTestCase; +import org.elasticsearch.threadpool.TestThreadPool; import org.elasticsearch.threadpool.ThreadPool; import java.util.ArrayList; @@ -244,7 +245,7 @@ public class PrioritizedExecutorsTests extends ESTestCase { } public void testTimeoutCleanup() throws Exception { - ThreadPool threadPool = new ThreadPool("test"); + ThreadPool threadPool = new TestThreadPool("test"); final ScheduledThreadPoolExecutor timer = (ScheduledThreadPoolExecutor) threadPool.scheduler(); final AtomicBoolean timeoutCalled = new AtomicBoolean(); PrioritizedEsThreadPoolExecutor executor = EsExecutors.newSinglePrioritizing(getTestName(), EsExecutors.daemonThreadFactory(getTestName()), holder); diff --git a/core/src/test/java/org/elasticsearch/common/xcontent/ConstructingObjectParserTests.java b/core/src/test/java/org/elasticsearch/common/xcontent/ConstructingObjectParserTests.java index e04fdab12aa..f3592936765 100644 --- a/core/src/test/java/org/elasticsearch/common/xcontent/ConstructingObjectParserTests.java +++ b/core/src/test/java/org/elasticsearch/common/xcontent/ConstructingObjectParserTests.java @@ -19,17 +19,26 @@ package org.elasticsearch.common.xcontent; +import org.elasticsearch.common.Nullable; import org.elasticsearch.common.ParseField; import org.elasticsearch.common.ParseFieldMatcher; import org.elasticsearch.common.ParseFieldMatcherSupplier; import org.elasticsearch.common.ParsingException; import org.elasticsearch.common.bytes.BytesReference; import org.elasticsearch.test.ESTestCase; +import org.hamcrest.Matcher; import java.io.IOException; +import java.util.Arrays; +import java.util.List; +import static java.util.Collections.unmodifiableList; import static org.elasticsearch.common.xcontent.ConstructingObjectParser.constructorArg; +import static org.elasticsearch.common.xcontent.ConstructingObjectParser.optionalConstructorArg; +import static org.hamcrest.Matchers.anyOf; +import static org.hamcrest.Matchers.equalTo; import static org.hamcrest.Matchers.instanceOf; +import static org.hamcrest.Matchers.nullValue; public class ConstructingObjectParserTests extends ESTestCase { private static final ParseFieldMatcherSupplier MATCHER = () -> ParseFieldMatcher.STRICT; @@ -38,7 +47,7 @@ public class ConstructingObjectParserTests extends ESTestCase { * Builds the object in random order and parses it. */ public void testRandomOrder() throws Exception { - HasRequiredArguments expected = new HasRequiredArguments(randomAsciiOfLength(5), randomInt()); + HasCtorArguments expected = new HasCtorArguments(randomAsciiOfLength(5), randomInt()); expected.setMineral(randomInt()); expected.setFruit(randomInt()); expected.setA(randomBoolean() ? null : randomAsciiOfLength(5)); @@ -49,9 +58,8 @@ public class ConstructingObjectParserTests extends ESTestCase { expected.toXContent(builder, ToXContent.EMPTY_PARAMS); builder = shuffleXContent(builder); BytesReference bytes = builder.bytes(); - XContentParser parser = XContentFactory.xContent(bytes).createParser(bytes); - try { - HasRequiredArguments parsed = HasRequiredArguments.PARSER.apply(parser, MATCHER); + try (XContentParser parser = XContentFactory.xContent(bytes).createParser(bytes)) { + HasCtorArguments parsed = randomFrom(HasCtorArguments.ALL_PARSERS).apply(parser, MATCHER); assertEquals(expected.animal, parsed.animal); assertEquals(expected.vegetable, parsed.vegetable); assertEquals(expected.mineral, parsed.mineral); @@ -66,44 +74,84 @@ public class ConstructingObjectParserTests extends ESTestCase { } } - public void testMissingAllConstructorParams() throws IOException { + public void testMissingAllConstructorArgs() throws IOException { XContentParser parser = XContentType.JSON.xContent().createParser( "{\n" + " \"mineral\": 1\n" + "}"); - IllegalArgumentException e = expectThrows(IllegalArgumentException.class, - () -> HasRequiredArguments.PARSER.apply(parser, MATCHER)); - assertEquals("Required [animal, vegetable]", e.getMessage()); + ConstructingObjectParser objectParser = randomBoolean() ? HasCtorArguments.PARSER + : HasCtorArguments.PARSER_VEGETABLE_OPTIONAL; + IllegalArgumentException e = expectThrows(IllegalArgumentException.class, () -> objectParser.apply(parser, MATCHER)); + if (objectParser == HasCtorArguments.PARSER) { + assertEquals("Required [animal, vegetable]", e.getMessage()); + } else { + assertEquals("Required [animal]", e.getMessage()); + } } - public void testMissingSecondConstructorParam() throws IOException { + public void testMissingAllConstructorArgsButNotRequired() throws IOException { + XContentParser parser = XContentType.JSON.xContent().createParser( + "{\n" + + " \"mineral\": 1\n" + + "}"); + HasCtorArguments parsed = HasCtorArguments.PARSER_ALL_OPTIONAL.apply(parser, MATCHER); + assertEquals(1, parsed.mineral); + } + + public void testMissingSecondConstructorArg() throws IOException { XContentParser parser = XContentType.JSON.xContent().createParser( "{\n" + " \"mineral\": 1,\n" + " \"animal\": \"cat\"\n" + "}"); IllegalArgumentException e = expectThrows(IllegalArgumentException.class, - () -> HasRequiredArguments.PARSER.apply(parser, MATCHER)); + () -> HasCtorArguments.PARSER.apply(parser, MATCHER)); assertEquals("Required [vegetable]", e.getMessage()); } - public void testMissingFirstConstructorParam() throws IOException { + public void testMissingSecondConstructorArgButNotRequired() throws IOException { + XContentParser parser = XContentType.JSON.xContent().createParser( + "{\n" + + " \"mineral\": 1,\n" + + " \"animal\": \"cat\"\n" + + "}"); + @SuppressWarnings("unchecked") + HasCtorArguments parsed = randomFrom(HasCtorArguments.PARSER_VEGETABLE_OPTIONAL, HasCtorArguments.PARSER_ALL_OPTIONAL).apply(parser, + MATCHER); + assertEquals(1, parsed.mineral); + assertEquals("cat", parsed.animal); + } + + public void testMissingFirstConstructorArg() throws IOException { XContentParser parser = XContentType.JSON.xContent().createParser( "{\n" + " \"mineral\": 1,\n" + " \"vegetable\": 2\n" + "}"); - IllegalArgumentException e = expectThrows(IllegalArgumentException.class, () -> HasRequiredArguments.PARSER.apply(parser, MATCHER)); + @SuppressWarnings("unchecked") + IllegalArgumentException e = expectThrows(IllegalArgumentException.class, + () -> randomFrom(HasCtorArguments.PARSER, HasCtorArguments.PARSER_VEGETABLE_OPTIONAL).apply(parser, MATCHER)); assertEquals("Required [animal]", e.getMessage()); } + public void testMissingFirstConstructorArgButNotRequired() throws IOException { + XContentParser parser = XContentType.JSON.xContent().createParser( + "{\n" + + " \"mineral\": 1,\n" + + " \"vegetable\": 2\n" + + "}"); + HasCtorArguments parsed = HasCtorArguments.PARSER_ALL_OPTIONAL.apply(parser, MATCHER); + assertEquals(1, parsed.mineral); + assertEquals((Integer) 2, parsed.vegetable); + } + public void testRepeatedConstructorParam() throws IOException { XContentParser parser = XContentType.JSON.xContent().createParser( "{\n" + " \"vegetable\": 1,\n" + " \"vegetable\": 2\n" + "}"); - Throwable e = expectThrows(ParsingException.class, () -> HasRequiredArguments.PARSER.apply(parser, MATCHER)); + Throwable e = expectThrows(ParsingException.class, () -> randomFrom(HasCtorArguments.ALL_PARSERS).apply(parser, MATCHER)); assertEquals("[has_required_arguments] failed to parse field [vegetable]", e.getMessage()); e = e.getCause(); assertThat(e, instanceOf(IllegalArgumentException.class)); @@ -117,7 +165,7 @@ public class ConstructingObjectParserTests extends ESTestCase { + " \"vegetable\": 2,\n" + " \"a\": \"supercalifragilisticexpialidocious\"\n" + "}"); - ParsingException e = expectThrows(ParsingException.class, () -> HasRequiredArguments.PARSER.apply(parser, MATCHER)); + ParsingException e = expectThrows(ParsingException.class, () -> randomFrom(HasCtorArguments.ALL_PARSERS).apply(parser, MATCHER)); assertEquals("[has_required_arguments] failed to parse field [a]", e.getMessage()); assertEquals(4, e.getLineNumber()); assertEquals("[a] must be less than 10 characters in length but was [supercalifragilisticexpialidocious]", @@ -131,7 +179,7 @@ public class ConstructingObjectParserTests extends ESTestCase { + " \"animal\": \"cat\"\n," + " \"vegetable\": 2\n" + "}"); - ParsingException e = expectThrows(ParsingException.class, () -> HasRequiredArguments.PARSER.apply(parser, MATCHER)); + ParsingException e = expectThrows(ParsingException.class, () -> randomFrom(HasCtorArguments.ALL_PARSERS).apply(parser, MATCHER)); assertEquals("[has_required_arguments] failed to parse field [vegetable]", e.getMessage()); assertEquals(4, e.getLineNumber()); e = (ParsingException) e.getCause(); @@ -149,18 +197,28 @@ public class ConstructingObjectParserTests extends ESTestCase { } ConstructingObjectParser parser = new ConstructingObjectParser<>( "constructor_args_required", (a) -> new NoConstructorArgs()); - Exception e = expectThrows(IllegalStateException.class, () -> parser.apply(XContentType.JSON.xContent().createParser("{}"), null)); - assertEquals("[constructor_args_required] must configure at least on constructor argument. If it doens't have any it " - + "should use ObjectParser instead of ConstructingObjectParser. This is a bug in the parser declaration.", e.getMessage()); + try { + parser.apply(XContentType.JSON.xContent().createParser("{}"), null); + fail("Expected AssertionError"); + } catch (AssertionError e) { + assertEquals("[constructor_args_required] must configure at least on constructor argument. If it doesn't have any it should " + + "use ObjectParser instead of ConstructingObjectParser. This is a bug in the parser declaration.", e.getMessage()); + } } /** * Tests the non-constructor fields are only set on time. */ public void testCalledOneTime() throws IOException { + boolean ctorArgOptional = randomBoolean(); class CalledOneTime { public CalledOneTime(String yeah) { - assertEquals("!", yeah); + Matcher yeahMatcher = equalTo("!"); + if (ctorArgOptional) { + // either(yeahMatcher).or(nullValue) is broken by https://github.com/hamcrest/JavaHamcrest/issues/49 + yeahMatcher = anyOf(yeahMatcher, nullValue()); + } + assertThat(yeah, yeahMatcher); } boolean fooSet = false; @@ -172,7 +230,7 @@ public class ConstructingObjectParserTests extends ESTestCase { ConstructingObjectParser parser = new ConstructingObjectParser<>("one_time_test", (a) -> new CalledOneTime((String) a[0])); parser.declareString(CalledOneTime::setFoo, new ParseField("foo")); - parser.declareString(constructorArg(), new ParseField("yeah")); + parser.declareString(ctorArgOptional ? optionalConstructorArg() : constructorArg(), new ParseField("yeah")); // ctor arg first so we can test for the bug we found one time XContentParser xcontent = XContentType.JSON.xContent().createParser( @@ -191,12 +249,23 @@ public class ConstructingObjectParserTests extends ESTestCase { + "}"); result = parser.apply(xcontent, MATCHER); assertTrue(result.fooSet); + + if (ctorArgOptional) { + // and without the constructor arg if we've made it optional + xcontent = XContentType.JSON.xContent().createParser( + "{\n" + + " \"foo\": \"foo\"\n" + + "}"); + result = parser.apply(xcontent, MATCHER); + } + assertTrue(result.fooSet); } - - private static class HasRequiredArguments implements ToXContent { + private static class HasCtorArguments implements ToXContent { + @Nullable final String animal; - final int vegetable; + @Nullable + final Integer vegetable; int mineral; int fruit; String a; @@ -204,7 +273,7 @@ public class ConstructingObjectParserTests extends ESTestCase { String c; boolean d; - public HasRequiredArguments(String animal, int vegetable) { + public HasCtorArguments(@Nullable String animal, @Nullable Integer vegetable) { this.animal = animal; this.vegetable = vegetable; } @@ -263,17 +332,31 @@ public class ConstructingObjectParserTests extends ESTestCase { return builder; } - public static final ConstructingObjectParser PARSER = - new ConstructingObjectParser<>("has_required_arguments", a -> new HasRequiredArguments((String) a[0], (Integer) a[1])); - static { - PARSER.declareString(constructorArg(), new ParseField("animal")); - PARSER.declareInt(constructorArg(), new ParseField("vegetable")); - PARSER.declareInt(HasRequiredArguments::setMineral, new ParseField("mineral")); - PARSER.declareInt(HasRequiredArguments::setFruit, new ParseField("fruit")); - PARSER.declareString(HasRequiredArguments::setA, new ParseField("a")); - PARSER.declareString(HasRequiredArguments::setB, new ParseField("b")); - PARSER.declareString(HasRequiredArguments::setC, new ParseField("c")); - PARSER.declareBoolean(HasRequiredArguments::setD, new ParseField("d")); + /* + * It is normal just to declare a single PARSER but we use a couple of different parsers for testing so we have all of these. Don't + * this this style is normal just because it is in the test. + */ + public static final ConstructingObjectParser PARSER = buildParser(true, true); + public static final ConstructingObjectParser PARSER_VEGETABLE_OPTIONAL = buildParser( + true, false); + public static final ConstructingObjectParser PARSER_ALL_OPTIONAL = buildParser(false, + false); + public static final List> ALL_PARSERS = unmodifiableList( + Arrays.asList(PARSER, PARSER_VEGETABLE_OPTIONAL, PARSER_ALL_OPTIONAL)); + + private static ConstructingObjectParser buildParser(boolean animalRequired, + boolean vegetableRequired) { + ConstructingObjectParser parser = new ConstructingObjectParser<>( + "has_required_arguments", a -> new HasCtorArguments((String) a[0], (Integer) a[1])); + parser.declareString(animalRequired ? constructorArg() : optionalConstructorArg(), new ParseField("animal")); + parser.declareInt(vegetableRequired ? constructorArg() : optionalConstructorArg(), new ParseField("vegetable")); + parser.declareInt(HasCtorArguments::setMineral, new ParseField("mineral")); + parser.declareInt(HasCtorArguments::setFruit, new ParseField("fruit")); + parser.declareString(HasCtorArguments::setA, new ParseField("a")); + parser.declareString(HasCtorArguments::setB, new ParseField("b")); + parser.declareString(HasCtorArguments::setC, new ParseField("c")); + parser.declareBoolean(HasCtorArguments::setD, new ParseField("d")); + return parser; } } } diff --git a/core/src/test/java/org/elasticsearch/discovery/ZenFaultDetectionTests.java b/core/src/test/java/org/elasticsearch/discovery/ZenFaultDetectionTests.java index d3da4513d95..ea81e958342 100644 --- a/core/src/test/java/org/elasticsearch/discovery/ZenFaultDetectionTests.java +++ b/core/src/test/java/org/elasticsearch/discovery/ZenFaultDetectionTests.java @@ -39,6 +39,7 @@ import org.elasticsearch.indices.breaker.CircuitBreakerService; import org.elasticsearch.indices.breaker.HierarchyCircuitBreakerService; import org.elasticsearch.test.ESTestCase; import org.elasticsearch.test.transport.MockTransportService; +import org.elasticsearch.threadpool.TestThreadPool; import org.elasticsearch.threadpool.ThreadPool; import org.elasticsearch.transport.TransportConnectionListener; import org.elasticsearch.transport.TransportRequestOptions; @@ -85,7 +86,7 @@ public class ZenFaultDetectionTests extends ESTestCase { .put(HierarchyCircuitBreakerService.IN_FLIGHT_REQUESTS_CIRCUIT_BREAKER_LIMIT_SETTING.getKey(), new ByteSizeValue(0)) .build(); ClusterSettings clusterSettings = new ClusterSettings(settings, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS); - threadPool = new ThreadPool(getClass().getName()); + threadPool = new TestThreadPool(getClass().getName()); clusterServiceA = createClusterService(threadPool); clusterServiceB = createClusterService(threadPool); circuitBreakerService = new HierarchyCircuitBreakerService(settings, clusterSettings); diff --git a/core/src/test/java/org/elasticsearch/discovery/zen/NodeJoinControllerTests.java b/core/src/test/java/org/elasticsearch/discovery/zen/NodeJoinControllerTests.java index 651eab421ad..b7db9f9e609 100644 --- a/core/src/test/java/org/elasticsearch/discovery/zen/NodeJoinControllerTests.java +++ b/core/src/test/java/org/elasticsearch/discovery/zen/NodeJoinControllerTests.java @@ -43,6 +43,7 @@ import org.elasticsearch.discovery.zen.elect.ElectMasterService; import org.elasticsearch.discovery.zen.membership.MembershipAction; import org.elasticsearch.test.ESTestCase; import org.elasticsearch.test.junit.annotations.TestLogging; +import org.elasticsearch.threadpool.TestThreadPool; import org.elasticsearch.threadpool.ThreadPool; import org.junit.After; import org.junit.AfterClass; @@ -81,7 +82,7 @@ public class NodeJoinControllerTests extends ESTestCase { @BeforeClass public static void beforeClass() { - threadPool = new ThreadPool("ShardReplicationTests"); + threadPool = new TestThreadPool("ShardReplicationTests"); } @AfterClass diff --git a/core/src/test/java/org/elasticsearch/discovery/zen/ping/unicast/UnicastZenPingIT.java b/core/src/test/java/org/elasticsearch/discovery/zen/ping/unicast/UnicastZenPingIT.java index 7847d7027d0..2845e6a5bf0 100644 --- a/core/src/test/java/org/elasticsearch/discovery/zen/ping/unicast/UnicastZenPingIT.java +++ b/core/src/test/java/org/elasticsearch/discovery/zen/ping/unicast/UnicastZenPingIT.java @@ -37,6 +37,7 @@ import org.elasticsearch.discovery.zen.ping.ZenPing; import org.elasticsearch.indices.breaker.NoneCircuitBreakerService; import org.elasticsearch.test.ESTestCase; import org.elasticsearch.test.VersionUtils; +import org.elasticsearch.threadpool.TestThreadPool; import org.elasticsearch.threadpool.ThreadPool; import org.elasticsearch.transport.TransportConnectionListener; import org.elasticsearch.transport.TransportService; @@ -60,7 +61,7 @@ public class UnicastZenPingIT extends ESTestCase { int endPort = startPort + 10; settings = Settings.builder().put(settings).put(TransportSettings.PORT.getKey(), startPort + "-" + endPort).build(); - ThreadPool threadPool = new ThreadPool(getClass().getName()); + ThreadPool threadPool = new TestThreadPool(getClass().getName()); ClusterName test = new ClusterName("test"); ClusterName mismatch = new ClusterName("mismatch"); NetworkService networkService = new NetworkService(settings); diff --git a/core/src/test/java/org/elasticsearch/discovery/zen/publish/PublishClusterStateActionTests.java b/core/src/test/java/org/elasticsearch/discovery/zen/publish/PublishClusterStateActionTests.java index e6b160eabf8..7cc319c8579 100644 --- a/core/src/test/java/org/elasticsearch/discovery/zen/publish/PublishClusterStateActionTests.java +++ b/core/src/test/java/org/elasticsearch/discovery/zen/publish/PublishClusterStateActionTests.java @@ -47,6 +47,7 @@ import org.elasticsearch.node.Node; import org.elasticsearch.test.ESTestCase; import org.elasticsearch.test.junit.annotations.TestLogging; import org.elasticsearch.test.transport.MockTransportService; +import org.elasticsearch.threadpool.TestThreadPool; import org.elasticsearch.threadpool.ThreadPool; import org.elasticsearch.transport.BytesTransportRequest; import org.elasticsearch.transport.TransportChannel; @@ -217,7 +218,7 @@ public class PublishClusterStateActionTests extends ESTestCase { @Before public void setUp() throws Exception { super.setUp(); - threadPool = new ThreadPool(getClass().getName()); + threadPool = new TestThreadPool(getClass().getName()); } @Override diff --git a/core/src/test/java/org/elasticsearch/document/DocumentActionsIT.java b/core/src/test/java/org/elasticsearch/document/DocumentActionsIT.java index 6f002e8404d..065128af918 100644 --- a/core/src/test/java/org/elasticsearch/document/DocumentActionsIT.java +++ b/core/src/test/java/org/elasticsearch/document/DocumentActionsIT.java @@ -28,6 +28,7 @@ import org.elasticsearch.action.delete.DeleteResponse; import org.elasticsearch.action.get.GetResponse; import org.elasticsearch.action.index.IndexResponse; import org.elasticsearch.action.search.SearchResponse; +import org.elasticsearch.action.support.WriteRequest.RefreshPolicy; import org.elasticsearch.cluster.health.ClusterHealthStatus; import org.elasticsearch.common.xcontent.XContentBuilder; import org.elasticsearch.common.xcontent.XContentFactory; @@ -45,7 +46,7 @@ import static org.hamcrest.Matchers.equalTo; import static org.hamcrest.Matchers.nullValue; /** - * + * Integration test for document action like index, bulk, and get. It has a very long history: it was in the second commit of Elasticsearch. */ public class DocumentActionsIT extends ESIntegTestCase { protected void createIndex() { @@ -62,7 +63,8 @@ public class DocumentActionsIT extends ESIntegTestCase { logger.info("Running Cluster Health"); ensureGreen(); logger.info("Indexing [type1/1]"); - IndexResponse indexResponse = client().prepareIndex().setIndex("test").setType("type1").setId("1").setSource(source("1", "test")).setRefresh(true).execute().actionGet(); + IndexResponse indexResponse = client().prepareIndex().setIndex("test").setType("type1").setId("1").setSource(source("1", "test")) + .setRefreshPolicy(RefreshPolicy.IMMEDIATE).get(); assertThat(indexResponse.getIndex(), equalTo(getConcreteIndexName())); assertThat(indexResponse.getId(), equalTo("1")); assertThat(indexResponse.getType(), equalTo("type1")); diff --git a/core/src/test/java/org/elasticsearch/document/ShardInfoIT.java b/core/src/test/java/org/elasticsearch/document/ShardInfoIT.java index 4f28cf19d7b..765cee3b6e8 100644 --- a/core/src/test/java/org/elasticsearch/document/ShardInfoIT.java +++ b/core/src/test/java/org/elasticsearch/document/ShardInfoIT.java @@ -19,7 +19,6 @@ package org.elasticsearch.document; -import org.elasticsearch.action.ReplicationResponse; import org.elasticsearch.action.admin.cluster.health.ClusterHealthResponse; import org.elasticsearch.action.admin.indices.recovery.RecoveryResponse; import org.elasticsearch.action.bulk.BulkItemResponse; @@ -27,6 +26,7 @@ import org.elasticsearch.action.bulk.BulkRequestBuilder; import org.elasticsearch.action.bulk.BulkResponse; import org.elasticsearch.action.delete.DeleteResponse; import org.elasticsearch.action.index.IndexResponse; +import org.elasticsearch.action.support.replication.ReplicationResponse; import org.elasticsearch.action.update.UpdateResponse; import org.elasticsearch.cluster.ClusterState; import org.elasticsearch.cluster.metadata.IndexMetaData; diff --git a/core/src/test/java/org/elasticsearch/env/NodeEnvironmentTests.java b/core/src/test/java/org/elasticsearch/env/NodeEnvironmentTests.java index 3c13351a125..ad425d8afc9 100644 --- a/core/src/test/java/org/elasticsearch/env/NodeEnvironmentTests.java +++ b/core/src/test/java/org/elasticsearch/env/NodeEnvironmentTests.java @@ -22,6 +22,7 @@ import org.apache.lucene.index.SegmentInfos; import org.apache.lucene.store.LockObtainFailedException; import org.apache.lucene.util.IOUtils; import org.apache.lucene.util.LuceneTestCase; +import org.elasticsearch.ElasticsearchException; import org.elasticsearch.cluster.metadata.IndexMetaData; import org.elasticsearch.common.SuppressForbidden; import org.elasticsearch.common.io.PathUtils; @@ -47,6 +48,8 @@ import java.util.concurrent.CountDownLatch; import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicReference; +import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertFileExists; +import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertFileNotExists; import static org.hamcrest.CoreMatchers.equalTo; import static org.hamcrest.Matchers.arrayWithSize; import static org.hamcrest.Matchers.containsString; @@ -382,10 +385,10 @@ public class NodeEnvironmentTests extends ESTestCase { assertThat("shard paths with a custom data_path should contain only regular paths", env.availableShardPaths(sid), - equalTo(stringsToPaths(dataPaths, "elasticsearch/nodes/0/indices/" + index.getUUID() + "/0"))); + equalTo(stringsToPaths(dataPaths, "nodes/0/indices/" + index.getUUID() + "/0"))); assertThat("index paths uses the regular template", - env.indexPaths(index), equalTo(stringsToPaths(dataPaths, "elasticsearch/nodes/0/indices/" + index.getUUID()))); + env.indexPaths(index), equalTo(stringsToPaths(dataPaths, "nodes/0/indices/" + index.getUUID()))); env.close(); NodeEnvironment env2 = newNodeEnvironment(dataPaths, "/tmp", @@ -396,14 +399,57 @@ public class NodeEnvironmentTests extends ESTestCase { assertThat("shard paths with a custom data_path should contain only regular paths", env2.availableShardPaths(sid), - equalTo(stringsToPaths(dataPaths, "elasticsearch/nodes/0/indices/" + index.getUUID() + "/0"))); + equalTo(stringsToPaths(dataPaths, "nodes/0/indices/" + index.getUUID() + "/0"))); assertThat("index paths uses the regular template", - env2.indexPaths(index), equalTo(stringsToPaths(dataPaths, "elasticsearch/nodes/0/indices/" + index.getUUID()))); + env2.indexPaths(index), equalTo(stringsToPaths(dataPaths, "nodes/0/indices/" + index.getUUID()))); env2.close(); } + public void testWhetherClusterFolderShouldBeUsed() throws Exception { + Path tempNoCluster = createTempDir(); + Path tempDataPath = tempNoCluster.toAbsolutePath(); + + Path tempPath = tempNoCluster.resolve("foo"); // "foo" is the cluster name + Path tempClusterPath = tempPath.toAbsolutePath(); + + assertFalse("non-existent directory should not be used", NodeEnvironment.readFromDataPathWithClusterName(tempPath)); + Settings settings = Settings.builder() + .put("cluster.name", "foo") + .put(Environment.PATH_HOME_SETTING.getKey(), createTempDir().toAbsolutePath().toString()) + .put(Environment.PATH_DATA_SETTING.getKey(), tempDataPath.toString()).build(); + try (NodeEnvironment env = new NodeEnvironment(settings, new Environment(settings))) { + Path nodeDataPath = env.nodeDataPaths()[0]; + assertEquals(nodeDataPath, tempDataPath.resolve("nodes").resolve("0")); + } + IOUtils.rm(tempNoCluster); + + Files.createDirectories(tempPath); + assertFalse("empty directory should not be read from", NodeEnvironment.readFromDataPathWithClusterName(tempPath)); + settings = Settings.builder() + .put("cluster.name", "foo") + .put(Environment.PATH_HOME_SETTING.getKey(), createTempDir().toAbsolutePath().toString()) + .put(Environment.PATH_DATA_SETTING.getKey(), tempDataPath.toString()).build(); + try (NodeEnvironment env = new NodeEnvironment(settings, new Environment(settings))) { + Path nodeDataPath = env.nodeDataPaths()[0]; + assertEquals(nodeDataPath, tempDataPath.resolve("nodes").resolve("0")); + } + IOUtils.rm(tempNoCluster); + + // Create a directory for the cluster name + Files.createDirectories(tempPath.resolve(NodeEnvironment.NODES_FOLDER)); + assertTrue("there is data in the directory", NodeEnvironment.readFromDataPathWithClusterName(tempPath)); + settings = Settings.builder() + .put("cluster.name", "foo") + .put(Environment.PATH_HOME_SETTING.getKey(), createTempDir().toAbsolutePath().toString()) + .put(Environment.PATH_DATA_SETTING.getKey(), tempClusterPath.toString()).build(); + try (NodeEnvironment env = new NodeEnvironment(settings, new Environment(settings))) { + Path nodeDataPath = env.nodeDataPaths()[0]; + assertEquals(nodeDataPath, tempClusterPath.resolve("nodes").resolve("0")); + } + } + /** Converts an array of Strings to an array of Paths, adding an additional child if specified */ private Path[] stringsToPaths(String[] strings, String additional) { Path[] locations = new Path[strings.length]; diff --git a/core/src/test/java/org/elasticsearch/gateway/AsyncShardFetchTests.java b/core/src/test/java/org/elasticsearch/gateway/AsyncShardFetchTests.java index b710aa50ee0..9ce2aa44ab6 100644 --- a/core/src/test/java/org/elasticsearch/gateway/AsyncShardFetchTests.java +++ b/core/src/test/java/org/elasticsearch/gateway/AsyncShardFetchTests.java @@ -27,6 +27,7 @@ import org.elasticsearch.common.logging.Loggers; import org.elasticsearch.common.transport.DummyTransportAddress; import org.elasticsearch.index.shard.ShardId; import org.elasticsearch.test.ESTestCase; +import org.elasticsearch.threadpool.TestThreadPool; import org.elasticsearch.threadpool.ThreadPool; import org.junit.After; import org.junit.Before; @@ -60,7 +61,7 @@ public class AsyncShardFetchTests extends ESTestCase { @Before public void setUp() throws Exception { super.setUp(); - this.threadPool = new ThreadPool(getTestName()); + this.threadPool = new TestThreadPool(getTestName()); this.test = new TestFetch(threadPool); } diff --git a/core/src/test/java/org/elasticsearch/gateway/GatewayIndexStateIT.java b/core/src/test/java/org/elasticsearch/gateway/GatewayIndexStateIT.java index ebf1085cfbe..d5dee3c1bdc 100644 --- a/core/src/test/java/org/elasticsearch/gateway/GatewayIndexStateIT.java +++ b/core/src/test/java/org/elasticsearch/gateway/GatewayIndexStateIT.java @@ -53,6 +53,7 @@ import java.nio.file.Files; import java.nio.file.Path; import java.util.List; +import static org.elasticsearch.action.support.WriteRequest.RefreshPolicy.IMMEDIATE; import static org.elasticsearch.index.query.QueryBuilders.matchAllQuery; import static org.elasticsearch.index.query.QueryBuilders.matchQuery; import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertAcked; @@ -245,7 +246,7 @@ public class GatewayIndexStateIT extends ESIntegTestCase { internalCluster().startNodesAsync(2).get(); logger.info("--> indexing a simple document"); - client().prepareIndex("test", "type1", "1").setSource("field1", "value1").setRefresh(true).execute().actionGet(); + client().prepareIndex("test", "type1", "1").setSource("field1", "value1").setRefreshPolicy(IMMEDIATE).get(); logger.info("--> waiting for green status"); ClusterHealthResponse health = client().admin().cluster().prepareHealth().setWaitForEvents(Priority.LANGUID).setWaitForGreenStatus() @@ -285,7 +286,7 @@ public class GatewayIndexStateIT extends ESIntegTestCase { final String node_1 = internalCluster().startNodesAsync(2).get().get(0); logger.info("--> indexing a simple document"); - client().prepareIndex("test", "type1", "1").setSource("field1", "value1").setRefresh(true).execute().actionGet(); + client().prepareIndex("test", "type1", "1").setSource("field1", "value1").setRefreshPolicy(IMMEDIATE).get(); logger.info("--> waiting for green status"); ensureGreen(); @@ -408,7 +409,7 @@ public class GatewayIndexStateIT extends ESIntegTestCase { logger.info("--> starting one node"); internalCluster().startNode(); logger.info("--> indexing a simple document"); - client().prepareIndex("test", "type1", "1").setSource("field1", "value1").setRefresh(true).execute().actionGet(); + client().prepareIndex("test", "type1", "1").setSource("field1", "value1").setRefreshPolicy(IMMEDIATE).get(); logger.info("--> waiting for green status"); if (usually()) { ensureYellow(); @@ -477,7 +478,7 @@ public class GatewayIndexStateIT extends ESIntegTestCase { " }\n" + " }}").get(); logger.info("--> indexing a simple document"); - client().prepareIndex("test", "type1", "1").setSource("field1", "value one").setRefresh(true).execute().actionGet(); + client().prepareIndex("test", "type1", "1").setSource("field1", "value one").setRefreshPolicy(IMMEDIATE).get(); logger.info("--> waiting for green status"); if (usually()) { ensureYellow(); @@ -521,7 +522,7 @@ public class GatewayIndexStateIT extends ESIntegTestCase { public void testArchiveBrokenClusterSettings() throws Exception { logger.info("--> starting one node"); internalCluster().startNode(); - client().prepareIndex("test", "type1", "1").setSource("field1", "value1").setRefresh(true).execute().actionGet(); + client().prepareIndex("test", "type1", "1").setSource("field1", "value1").setRefreshPolicy(IMMEDIATE).get(); logger.info("--> waiting for green status"); if (usually()) { ensureYellow(); diff --git a/core/src/test/java/org/elasticsearch/http/netty/NettyHttpChannelTests.java b/core/src/test/java/org/elasticsearch/http/netty/NettyHttpChannelTests.java index e7e3c41820b..a56e9993434 100644 --- a/core/src/test/java/org/elasticsearch/http/netty/NettyHttpChannelTests.java +++ b/core/src/test/java/org/elasticsearch/http/netty/NettyHttpChannelTests.java @@ -30,6 +30,7 @@ import org.elasticsearch.indices.breaker.NoneCircuitBreakerService; import org.elasticsearch.rest.RestResponse; import org.elasticsearch.rest.RestStatus; import org.elasticsearch.test.ESTestCase; +import org.elasticsearch.threadpool.TestThreadPool; import org.elasticsearch.threadpool.ThreadPool; import org.jboss.netty.buffer.ChannelBuffer; import org.jboss.netty.buffer.ChannelBuffers; @@ -70,7 +71,7 @@ public class NettyHttpChannelTests extends ESTestCase { @Before public void setup() throws Exception { networkService = new NetworkService(Settings.EMPTY); - threadPool = new ThreadPool("test"); + threadPool = new TestThreadPool("test"); bigArrays = new MockBigArrays(Settings.EMPTY, new NoneCircuitBreakerService()); } diff --git a/core/src/test/java/org/elasticsearch/http/netty/NettyHttpServerPipeliningTests.java b/core/src/test/java/org/elasticsearch/http/netty/NettyHttpServerPipeliningTests.java index db48a56a122..6fc9a4e674a 100644 --- a/core/src/test/java/org/elasticsearch/http/netty/NettyHttpServerPipeliningTests.java +++ b/core/src/test/java/org/elasticsearch/http/netty/NettyHttpServerPipeliningTests.java @@ -29,6 +29,7 @@ import org.elasticsearch.http.netty.pipelining.OrderedDownstreamChannelEvent; import org.elasticsearch.http.netty.pipelining.OrderedUpstreamMessageEvent; import org.elasticsearch.indices.breaker.NoneCircuitBreakerService; import org.elasticsearch.test.ESTestCase; +import org.elasticsearch.threadpool.TestThreadPool; import org.elasticsearch.threadpool.ThreadPool; import org.jboss.netty.buffer.ChannelBuffer; import org.jboss.netty.buffer.ChannelBuffers; @@ -73,7 +74,7 @@ public class NettyHttpServerPipeliningTests extends ESTestCase { @Before public void setup() throws Exception { networkService = new NetworkService(Settings.EMPTY); - threadPool = new ThreadPool("test"); + threadPool = new TestThreadPool("test"); bigArrays = new MockBigArrays(Settings.EMPTY, new NoneCircuitBreakerService()); } diff --git a/core/src/test/java/org/elasticsearch/http/netty/NettyHttpServerTransportTests.java b/core/src/test/java/org/elasticsearch/http/netty/NettyHttpServerTransportTests.java index da548c93985..3cf9c1aa029 100644 --- a/core/src/test/java/org/elasticsearch/http/netty/NettyHttpServerTransportTests.java +++ b/core/src/test/java/org/elasticsearch/http/netty/NettyHttpServerTransportTests.java @@ -26,6 +26,7 @@ import org.elasticsearch.common.util.MockBigArrays; import org.elasticsearch.http.netty.cors.CorsConfig; import org.elasticsearch.indices.breaker.NoneCircuitBreakerService; import org.elasticsearch.test.ESTestCase; +import org.elasticsearch.threadpool.TestThreadPool; import org.elasticsearch.threadpool.ThreadPool; import org.jboss.netty.handler.codec.http.HttpMethod; import org.junit.After; @@ -54,7 +55,7 @@ public class NettyHttpServerTransportTests extends ESTestCase { @Before public void setup() throws Exception { networkService = new NetworkService(Settings.EMPTY); - threadPool = new ThreadPool("test"); + threadPool = new TestThreadPool("test"); bigArrays = new MockBigArrays(Settings.EMPTY, new NoneCircuitBreakerService()); } diff --git a/core/src/test/java/org/elasticsearch/index/IndexModuleTests.java b/core/src/test/java/org/elasticsearch/index/IndexModuleTests.java index 38b7341cd24..f552ab9cf17 100644 --- a/core/src/test/java/org/elasticsearch/index/IndexModuleTests.java +++ b/core/src/test/java/org/elasticsearch/index/IndexModuleTests.java @@ -76,6 +76,7 @@ import org.elasticsearch.test.ESTestCase; import org.elasticsearch.test.IndexSettingsModule; import org.elasticsearch.test.TestSearchContext; import org.elasticsearch.test.engine.MockEngineFactory; +import org.elasticsearch.threadpool.TestThreadPool; import org.elasticsearch.threadpool.ThreadPool; import org.elasticsearch.watcher.ResourceWatcherService; @@ -110,7 +111,7 @@ public class IndexModuleTests extends ESTestCase { static NodeServicesProvider newNodeServiceProvider(Settings settings, Environment environment, Client client, ScriptEngineService... scriptEngineServices) throws IOException { // TODO this can be used in other place too - lets first refactor the IndicesQueriesRegistry - ThreadPool threadPool = new ThreadPool("test"); + ThreadPool threadPool = new TestThreadPool("test"); CircuitBreakerService circuitBreakerService = new NoneCircuitBreakerService(); BigArrays bigArrays = new BigArrays(settings, circuitBreakerService); Set scriptEngines = Collections.emptySet(); diff --git a/core/src/test/java/org/elasticsearch/index/IndexWithShadowReplicasIT.java b/core/src/test/java/org/elasticsearch/index/IndexWithShadowReplicasIT.java index 1b7c71c644f..f0e12abeac8 100644 --- a/core/src/test/java/org/elasticsearch/index/IndexWithShadowReplicasIT.java +++ b/core/src/test/java/org/elasticsearch/index/IndexWithShadowReplicasIT.java @@ -70,7 +70,6 @@ import java.util.List; import java.util.concurrent.CopyOnWriteArrayList; import java.util.concurrent.CountDownLatch; import java.util.concurrent.ExecutionException; -import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicInteger; @@ -586,8 +585,8 @@ public class IndexWithShadowReplicasIT extends ESIntegTestCase { logger.info("--> deleting index " + IDX); assertAcked(client().admin().indices().prepareDelete(IDX)); - - // assertBusy(() -> assertPathHasBeenCleared(dataPath), 1, TimeUnit.MINUTES); + assertAllIndicesRemovedAndDeletionCompleted(internalCluster().getInstances(IndicesService.class)); + assertPathHasBeenCleared(dataPath); //norelease //TODO: uncomment the test below when https://github.com/elastic/elasticsearch/issues/17695 is resolved. //assertIndicesDirsDeleted(nodes); @@ -647,8 +646,8 @@ public class IndexWithShadowReplicasIT extends ESIntegTestCase { assertHitCount(resp, docCount); assertAcked(client().admin().indices().prepareDelete(IDX)); - - // assertBusy(() -> assertPathHasBeenCleared(dataPath), 1, TimeUnit.MINUTES); + assertAllIndicesRemovedAndDeletionCompleted(internalCluster().getInstances(IndicesService.class)); + assertPathHasBeenCleared(dataPath); //norelease //TODO: uncomment the test below when https://github.com/elastic/elasticsearch/issues/17695 is resolved. //assertIndicesDirsDeleted(nodes); @@ -839,8 +838,8 @@ public class IndexWithShadowReplicasIT extends ESIntegTestCase { logger.info("--> deleting closed index"); client().admin().indices().prepareDelete(IDX).get(); - - assertBusy(() -> assertPathHasBeenCleared(dataPath), 1, TimeUnit.MINUTES); + assertAllIndicesRemovedAndDeletionCompleted(internalCluster().getInstances(IndicesService.class)); + assertPathHasBeenCleared(dataPath); assertIndicesDirsDeleted(nodes); } diff --git a/core/src/test/java/org/elasticsearch/index/WaitUntilRefreshIT.java b/core/src/test/java/org/elasticsearch/index/WaitUntilRefreshIT.java new file mode 100644 index 00000000000..b2cb2d96818 --- /dev/null +++ b/core/src/test/java/org/elasticsearch/index/WaitUntilRefreshIT.java @@ -0,0 +1,217 @@ +/* + * 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; + +import org.elasticsearch.action.ListenableActionFuture; +import org.elasticsearch.action.bulk.BulkItemResponse; +import org.elasticsearch.action.bulk.BulkRequestBuilder; +import org.elasticsearch.action.bulk.BulkResponse; +import org.elasticsearch.action.delete.DeleteResponse; +import org.elasticsearch.action.index.IndexResponse; +import org.elasticsearch.action.support.WriteRequest.RefreshPolicy; +import org.elasticsearch.action.update.UpdateResponse; +import org.elasticsearch.common.network.NetworkModule; +import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.plugins.Plugin; +import org.elasticsearch.rest.RestStatus; +import org.elasticsearch.script.ExecutableScript; +import org.elasticsearch.script.NativeScriptFactory; +import org.elasticsearch.script.Script; +import org.elasticsearch.script.ScriptModule; +import org.elasticsearch.script.ScriptService.ScriptType; +import org.elasticsearch.test.ESIntegTestCase; +import org.junit.Before; + +import java.util.Collection; +import java.util.Map; +import java.util.concurrent.ExecutionException; + +import static java.util.Collections.emptyMap; +import static java.util.Collections.singleton; +import static java.util.Collections.singletonMap; +import static org.elasticsearch.index.query.QueryBuilders.matchQuery; +import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertNoFailures; +import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertNoSearchHits; +import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertSearchHits; + +/** + * Tests that requests with RefreshPolicy.WAIT_UNTIL will be visible when they return. + */ +public class WaitUntilRefreshIT extends ESIntegTestCase { + @Override + protected Settings nodeSettings(int nodeOrdinal) { + return Settings.builder().put(super.nodeSettings(nodeOrdinal)).put(NetworkModule.HTTP_ENABLED.getKey(), true).build(); + } + + @Override + public Settings indexSettings() { + // Use a shorter refresh interval to speed up the tests. We'll be waiting on this interval several times. + return Settings.builder().put(super.indexSettings()).put("index.refresh_interval", "40ms").build(); + } + + @Before + public void createTestIndex() { + createIndex("test"); + } + + public void testIndex() { + IndexResponse index = client().prepareIndex("test", "index", "1").setSource("foo", "bar").setRefreshPolicy(RefreshPolicy.WAIT_UNTIL) + .get(); + assertEquals(RestStatus.CREATED, index.status()); + assertFalse("request shouldn't have forced a refresh", index.forcedRefresh()); + assertSearchHits(client().prepareSearch("test").setQuery(matchQuery("foo", "bar")).get(), "1"); + } + + public void testDelete() throws InterruptedException, ExecutionException { + // Index normally + indexRandom(true, client().prepareIndex("test", "test", "1").setSource("foo", "bar")); + assertSearchHits(client().prepareSearch("test").setQuery(matchQuery("foo", "bar")).get(), "1"); + + // Now delete with blockUntilRefresh + DeleteResponse delete = client().prepareDelete("test", "test", "1").setRefreshPolicy(RefreshPolicy.WAIT_UNTIL).get(); + assertTrue("document was deleted", delete.isFound()); + assertFalse("request shouldn't have forced a refresh", delete.forcedRefresh()); + assertNoSearchHits(client().prepareSearch("test").setQuery(matchQuery("foo", "bar")).get()); + } + + public void testUpdate() throws InterruptedException, ExecutionException { + // Index normally + indexRandom(true, client().prepareIndex("test", "test", "1").setSource("foo", "bar")); + assertSearchHits(client().prepareSearch("test").setQuery(matchQuery("foo", "bar")).get(), "1"); + + // Update with RefreshPolicy.WAIT_UNTIL + UpdateResponse update = client().prepareUpdate("test", "test", "1").setDoc("foo", "baz").setRefreshPolicy(RefreshPolicy.WAIT_UNTIL) + .get(); + assertEquals(2, update.getVersion()); + assertFalse("request shouldn't have forced a refresh", update.forcedRefresh()); + assertSearchHits(client().prepareSearch("test").setQuery(matchQuery("foo", "baz")).get(), "1"); + + // Upsert with RefreshPolicy.WAIT_UNTIL + update = client().prepareUpdate("test", "test", "2").setDocAsUpsert(true).setDoc("foo", "cat") + .setRefreshPolicy(RefreshPolicy.WAIT_UNTIL).get(); + assertEquals(1, update.getVersion()); + assertFalse("request shouldn't have forced a refresh", update.forcedRefresh()); + assertSearchHits(client().prepareSearch("test").setQuery(matchQuery("foo", "cat")).get(), "2"); + + // Update-becomes-delete with RefreshPolicy.WAIT_UNTIL + update = client().prepareUpdate("test", "test", "2").setScript(new Script("delete_plz", ScriptType.INLINE, "native", emptyMap())) + .setRefreshPolicy(RefreshPolicy.WAIT_UNTIL).get(); + assertEquals(2, update.getVersion()); + assertFalse("request shouldn't have forced a refresh", update.forcedRefresh()); + assertNoSearchHits(client().prepareSearch("test").setQuery(matchQuery("foo", "cat")).get()); + } + + public void testBulk() { + // Index by bulk with RefreshPolicy.WAIT_UNTIL + BulkRequestBuilder bulk = client().prepareBulk().setRefreshPolicy(RefreshPolicy.WAIT_UNTIL); + bulk.add(client().prepareIndex("test", "test", "1").setSource("foo", "bar")); + assertBulkSuccess(bulk.get()); + assertSearchHits(client().prepareSearch("test").setQuery(matchQuery("foo", "bar")).get(), "1"); + + // Update by bulk with RefreshPolicy.WAIT_UNTIL + bulk = client().prepareBulk().setRefreshPolicy(RefreshPolicy.WAIT_UNTIL); + bulk.add(client().prepareUpdate("test", "test", "1").setDoc("foo", "baz")); + assertBulkSuccess(bulk.get()); + assertSearchHits(client().prepareSearch("test").setQuery(matchQuery("foo", "baz")).get(), "1"); + + // Delete by bulk with RefreshPolicy.WAIT_UNTIL + bulk = client().prepareBulk().setRefreshPolicy(RefreshPolicy.WAIT_UNTIL); + bulk.add(client().prepareDelete("test", "test", "1")); + assertBulkSuccess(bulk.get()); + assertNoSearchHits(client().prepareSearch("test").setQuery(matchQuery("foo", "bar")).get()); + + // Update makes a noop + bulk = client().prepareBulk().setRefreshPolicy(RefreshPolicy.WAIT_UNTIL); + bulk.add(client().prepareDelete("test", "test", "1")); + assertBulkSuccess(bulk.get()); + } + + /** + * Tests that an explicit request makes block_until_refresh return. It doesn't check that block_until_refresh doesn't return until the + * explicit refresh if the interval is -1 because we don't have that kind of control over refresh. It can happen all on its own. + */ + public void testNoRefreshInterval() throws InterruptedException, ExecutionException { + client().admin().indices().prepareUpdateSettings("test").setSettings(singletonMap("index.refresh_interval", -1)).get(); + ListenableActionFuture index = client().prepareIndex("test", "index", "1").setSource("foo", "bar") + .setRefreshPolicy(RefreshPolicy.WAIT_UNTIL).execute(); + while (false == index.isDone()) { + client().admin().indices().prepareRefresh("test").get(); + } + assertEquals(RestStatus.CREATED, index.get().status()); + assertFalse("request shouldn't have forced a refresh", index.get().forcedRefresh()); + assertSearchHits(client().prepareSearch("test").setQuery(matchQuery("foo", "bar")).get(), "1"); + } + + private void assertBulkSuccess(BulkResponse response) { + assertNoFailures(response); + for (BulkItemResponse item : response) { + assertFalse("request shouldn't have forced a refresh", item.getResponse().forcedRefresh()); + } + } + + @Override + protected Collection> nodePlugins() { + return singleton(DeletePlzPlugin.class); + } + + public static class DeletePlzPlugin extends Plugin { + @Override + public String name() { + return "delete_please"; + } + + @Override + public String description() { + return "adds a script that converts any update into a delete for testing"; + } + + public void onModule(ScriptModule scriptModule) { + scriptModule.registerScript("delete_plz", DeletePlzFactory.class); + } + } + + public static class DeletePlzFactory implements NativeScriptFactory { + @Override + public ExecutableScript newScript(Map params) { + return new ExecutableScript() { + private Map ctx; + + @Override + @SuppressWarnings("unchecked") // Elasicsearch convention + public void setNextVar(String name, Object value) { + if (name.equals("ctx")) { + ctx = (Map) value; + } + } + + @Override + public Object run() { + ctx.put("op", "delete"); + return null; + } + }; + } + + @Override + public boolean needsScores() { + return false; + } + } +} diff --git a/core/src/test/java/org/elasticsearch/index/analysis/AnalysisModuleTests.java b/core/src/test/java/org/elasticsearch/index/analysis/AnalysisModuleTests.java index 329396cb6c5..e20e7d1a7c9 100644 --- a/core/src/test/java/org/elasticsearch/index/analysis/AnalysisModuleTests.java +++ b/core/src/test/java/org/elasticsearch/index/analysis/AnalysisModuleTests.java @@ -75,8 +75,8 @@ public class AnalysisModuleTests extends ModuleTestCase { return new AnalysisRegistry(null, new Environment(settings), Collections.emptyMap(), Collections.singletonMap("myfilter", MyFilterTokenFilterFactory::new), Collections.emptyMap(), Collections.emptyMap()); } - - private Settings loadFromClasspath(String path) { + + private Settings loadFromClasspath(String path) throws IOException { return Settings.builder().loadFromStream(path, getClass().getResourceAsStream(path)) .put(IndexMetaData.SETTING_VERSION_CREATED, Version.CURRENT) .put(Environment.PATH_HOME_SETTING.getKey(), createTempDir().toString()) diff --git a/core/src/test/java/org/elasticsearch/index/analysis/CompoundAnalysisTests.java b/core/src/test/java/org/elasticsearch/index/analysis/CompoundAnalysisTests.java index 61fb078c9ab..0e3af58dc90 100644 --- a/core/src/test/java/org/elasticsearch/index/analysis/CompoundAnalysisTests.java +++ b/core/src/test/java/org/elasticsearch/index/analysis/CompoundAnalysisTests.java @@ -88,7 +88,7 @@ public class CompoundAnalysisTests extends ESTestCase { return terms; } - private Settings getJsonSettings() { + private Settings getJsonSettings() throws IOException { String json = "/org/elasticsearch/index/analysis/test1.json"; return Settings.builder() .loadFromStream(json, getClass().getResourceAsStream(json)) @@ -97,7 +97,7 @@ public class CompoundAnalysisTests extends ESTestCase { .build(); } - private Settings getYamlSettings() { + private Settings getYamlSettings() throws IOException { String yaml = "/org/elasticsearch/index/analysis/test1.yml"; return Settings.builder() .loadFromStream(yaml, getClass().getResourceAsStream(yaml)) diff --git a/core/src/test/java/org/elasticsearch/index/engine/InternalEngineTests.java b/core/src/test/java/org/elasticsearch/index/engine/InternalEngineTests.java index 51df3ee0386..05428ee3cd5 100644 --- a/core/src/test/java/org/elasticsearch/index/engine/InternalEngineTests.java +++ b/core/src/test/java/org/elasticsearch/index/engine/InternalEngineTests.java @@ -98,6 +98,7 @@ import org.elasticsearch.indices.mapper.MapperRegistry; import org.elasticsearch.test.DummyShardLock; import org.elasticsearch.test.ESTestCase; import org.elasticsearch.test.IndexSettingsModule; +import org.elasticsearch.threadpool.TestThreadPool; import org.elasticsearch.threadpool.ThreadPool; import org.hamcrest.MatcherAssert; import org.junit.After; @@ -170,8 +171,10 @@ public class InternalEngineTests extends ESTestCase { .put(IndexSettings.INDEX_GC_DELETES_SETTING, "1h") // make sure this doesn't kick in on us .put(EngineConfig.INDEX_CODEC_SETTING.getKey(), codecName) .put(IndexMetaData.SETTING_VERSION_CREATED, Version.CURRENT) + .put(IndexSettings.MAX_REFRESH_LISTENERS_PER_SHARD, + between(10, 10 * IndexSettings.MAX_REFRESH_LISTENERS_PER_SHARD.get(Settings.EMPTY))) .build()); // TODO randomize more settings - threadPool = new ThreadPool(getClass().getName()); + threadPool = new TestThreadPool(getClass().getName()); store = createStore(); storeReplica = createStore(); Lucene.cleanLuceneIndex(store.directory()); @@ -200,7 +203,7 @@ public class InternalEngineTests extends ESTestCase { return new EngineConfig(openMode, config.getShardId(), config.getThreadPool(), config.getIndexSettings(), config.getWarmer(), config.getStore(), config.getDeletionPolicy(), config.getMergePolicy(), config.getAnalyzer(), config.getSimilarity(), new CodecService(null, logger), config.getEventListener(), config.getTranslogRecoveryPerformer(), config.getQueryCache(), - config.getQueryCachingPolicy(), config.getTranslogConfig(), config.getFlushMergesAfter()); + config.getQueryCachingPolicy(), config.getTranslogConfig(), config.getFlushMergesAfter(), config.getRefreshListeners()); } @Override @@ -291,14 +294,16 @@ public class InternalEngineTests extends ESTestCase { } catch (IOException e) { throw new ElasticsearchException("can't find index?", e); } - EngineConfig config = new EngineConfig(openMode, shardId, threadPool, indexSettings - , null, store, createSnapshotDeletionPolicy(), mergePolicy, - iwc.getAnalyzer(), iwc.getSimilarity(), new CodecService(null, logger), new Engine.EventListener() { + Engine.EventListener listener = new Engine.EventListener() { @Override public void onFailedEngine(String reason, @Nullable Throwable t) { // we don't need to notify anybody in this test } - }, new TranslogHandler(shardId.getIndexName(), logger), IndexSearcher.getDefaultQueryCache(), IndexSearcher.getDefaultQueryCachingPolicy(), translogConfig, TimeValue.timeValueMinutes(5)); + }; + EngineConfig config = new EngineConfig(openMode, shardId, threadPool, indexSettings, null, store, createSnapshotDeletionPolicy(), + mergePolicy, iwc.getAnalyzer(), iwc.getSimilarity(), new CodecService(null, logger), listener, + new TranslogHandler(shardId.getIndexName(), logger), IndexSearcher.getDefaultQueryCache(), + IndexSearcher.getDefaultQueryCachingPolicy(), translogConfig, TimeValue.timeValueMinutes(5), null); return config; } @@ -309,18 +314,20 @@ public class InternalEngineTests extends ESTestCase { public void testSegments() throws Exception { try (Store store = createStore(); - Engine engine = createEngine(defaultSettings, store, createTempDir(), NoMergePolicy.INSTANCE)) { + Engine engine = createEngine(defaultSettings, store, createTempDir(), NoMergePolicy.INSTANCE)) { List segments = engine.segments(false); assertThat(segments.isEmpty(), equalTo(true)); assertThat(engine.segmentsStats(false).getCount(), equalTo(0L)); assertThat(engine.segmentsStats(false).getMemoryInBytes(), equalTo(0L)); - // create a doc and refresh + // create two docs and refresh ParsedDocument doc = testParsedDocument("1", "1", "test", null, -1, -1, testDocumentWithTextField(), B_1, null); - engine.index(new Engine.Index(newUid("1"), doc)); - + Engine.Index first = new Engine.Index(newUid("1"), doc); + engine.index(first); ParsedDocument doc2 = testParsedDocument("2", "2", "test", null, -1, -1, testDocumentWithTextField(), B_2, null); - engine.index(new Engine.Index(newUid("2"), doc2)); + Engine.Index second = new Engine.Index(newUid("2"), doc2); + engine.index(second); + assertThat(second.getTranslogLocation(), greaterThan(first.getTranslogLocation())); engine.refresh("test"); segments = engine.segments(false); @@ -2064,10 +2071,11 @@ public class InternalEngineTests extends ESTestCase { /* create a TranslogConfig that has been created with a different UUID */ TranslogConfig translogConfig = new TranslogConfig(shardId, translog.location(), config.getIndexSettings(), BigArrays.NON_RECYCLING_INSTANCE); - EngineConfig brokenConfig = new EngineConfig(EngineConfig.OpenMode.OPEN_INDEX_AND_TRANSLOG, shardId, threadPool, config.getIndexSettings() - , null, store, createSnapshotDeletionPolicy(), newMergePolicy(), - config.getAnalyzer(), config.getSimilarity(), new CodecService(null, logger), config.getEventListener() - , config.getTranslogRecoveryPerformer(), IndexSearcher.getDefaultQueryCache(), IndexSearcher.getDefaultQueryCachingPolicy(), translogConfig, TimeValue.timeValueMinutes(5)); + EngineConfig brokenConfig = new EngineConfig(EngineConfig.OpenMode.OPEN_INDEX_AND_TRANSLOG, shardId, threadPool, + config.getIndexSettings(), null, store, createSnapshotDeletionPolicy(), newMergePolicy(), config.getAnalyzer(), + config.getSimilarity(), new CodecService(null, logger), config.getEventListener(), config.getTranslogRecoveryPerformer(), + IndexSearcher.getDefaultQueryCache(), IndexSearcher.getDefaultQueryCachingPolicy(), translogConfig, + TimeValue.timeValueMinutes(5), config.getRefreshListeners()); try { InternalEngine internalEngine = new InternalEngine(brokenConfig); diff --git a/core/src/test/java/org/elasticsearch/index/engine/ShadowEngineTests.java b/core/src/test/java/org/elasticsearch/index/engine/ShadowEngineTests.java index f71f2d72019..ef443d1e102 100644 --- a/core/src/test/java/org/elasticsearch/index/engine/ShadowEngineTests.java +++ b/core/src/test/java/org/elasticsearch/index/engine/ShadowEngineTests.java @@ -55,6 +55,7 @@ import org.elasticsearch.index.mapper.ParseContext; import org.elasticsearch.index.mapper.ParsedDocument; import org.elasticsearch.index.mapper.internal.SourceFieldMapper; import org.elasticsearch.index.mapper.internal.UidFieldMapper; +import org.elasticsearch.index.shard.RefreshListeners; import org.elasticsearch.index.shard.DocsStats; import org.elasticsearch.index.shard.ShardId; import org.elasticsearch.index.shard.ShardUtils; @@ -66,6 +67,7 @@ import org.elasticsearch.index.translog.TranslogConfig; import org.elasticsearch.test.DummyShardLock; import org.elasticsearch.test.ESTestCase; import org.elasticsearch.test.IndexSettingsModule; +import org.elasticsearch.threadpool.TestThreadPool; import org.elasticsearch.threadpool.ThreadPool; import org.hamcrest.MatcherAssert; import org.junit.After; @@ -125,7 +127,7 @@ public class ShadowEngineTests extends ESTestCase { .put(IndexMetaData.SETTING_VERSION_CREATED, Version.CURRENT) .build()); // TODO randomize more settings - threadPool = new ThreadPool(getClass().getName()); + threadPool = new TestThreadPool(getClass().getName()); dirPath = createTempDir(); store = createStore(dirPath); storeReplica = createStore(dirPath); @@ -213,7 +215,7 @@ public class ShadowEngineTests extends ESTestCase { } protected ShadowEngine createShadowEngine(IndexSettings indexSettings, Store store) { - return new ShadowEngine(config(indexSettings, store, null, null)); + return new ShadowEngine(config(indexSettings, store, null, null, null)); } protected InternalEngine createInternalEngine(IndexSettings indexSettings, Store store, Path translogPath) { @@ -221,11 +223,12 @@ public class ShadowEngineTests extends ESTestCase { } protected InternalEngine createInternalEngine(IndexSettings indexSettings, Store store, Path translogPath, MergePolicy mergePolicy) { - EngineConfig config = config(indexSettings, store, translogPath, mergePolicy); + EngineConfig config = config(indexSettings, store, translogPath, mergePolicy, null); return new InternalEngine(config); } - public EngineConfig config(IndexSettings indexSettings, Store store, Path translogPath, MergePolicy mergePolicy) { + public EngineConfig config(IndexSettings indexSettings, Store store, Path translogPath, MergePolicy mergePolicy, + RefreshListeners refreshListeners) { IndexWriterConfig iwc = newIndexWriterConfig(); final EngineConfig.OpenMode openMode; try { @@ -237,14 +240,17 @@ public class ShadowEngineTests extends ESTestCase { } catch (IOException e) { throw new ElasticsearchException("can't find index?", e); } - TranslogConfig translogConfig = new TranslogConfig(shardId, translogPath, indexSettings, BigArrays.NON_RECYCLING_INSTANCE); - EngineConfig config = new EngineConfig(openMode, shardId, threadPool, indexSettings - , null, store, createSnapshotDeletionPolicy(), mergePolicy, - iwc.getAnalyzer(), iwc.getSimilarity() , new CodecService(null, logger), new Engine.EventListener() { + Engine.EventListener eventListener = new Engine.EventListener() { @Override public void onFailedEngine(String reason, @Nullable Throwable t) { // we don't need to notify anybody in this test - }}, null, IndexSearcher.getDefaultQueryCache(), IndexSearcher.getDefaultQueryCachingPolicy(), translogConfig, TimeValue.timeValueMinutes(5)); + } + }; + TranslogConfig translogConfig = new TranslogConfig(shardId, translogPath, indexSettings, BigArrays.NON_RECYCLING_INSTANCE); + EngineConfig config = new EngineConfig(openMode, shardId, threadPool, indexSettings, null, store, createSnapshotDeletionPolicy(), + mergePolicy, iwc.getAnalyzer(), iwc.getSimilarity(), new CodecService(null, logger), eventListener, null, + IndexSearcher.getDefaultQueryCache(), IndexSearcher.getDefaultQueryCachingPolicy(), translogConfig, + TimeValue.timeValueMinutes(5), refreshListeners); return config; } @@ -1011,4 +1017,11 @@ public class ShadowEngineTests extends ESTestCase { assertEquals(0, docStats.getDeleted()); primaryEngine.forceMerge(randomBoolean(), 1, false, false, false); } + + public void testRefreshListenersFails() throws IOException { + EngineConfig config = config(defaultSettings, store, createTempDir(), newMergePolicy(), + new RefreshListeners(null, null, null, logger)); + Exception e = expectThrows(IllegalArgumentException.class, () -> new ShadowEngine(config)); + assertEquals("ShadowEngine doesn't support RefreshListeners", e.getMessage()); + } } diff --git a/core/src/test/java/org/elasticsearch/index/fielddata/IndexFieldDataServiceTests.java b/core/src/test/java/org/elasticsearch/index/fielddata/IndexFieldDataServiceTests.java index 2887d9b2561..bd6decfb60e 100644 --- a/core/src/test/java/org/elasticsearch/index/fielddata/IndexFieldDataServiceTests.java +++ b/core/src/test/java/org/elasticsearch/index/fielddata/IndexFieldDataServiceTests.java @@ -55,6 +55,7 @@ import org.elasticsearch.plugins.Plugin; import org.elasticsearch.test.ESSingleNodeTestCase; import org.elasticsearch.test.IndexSettingsModule; import org.elasticsearch.test.InternalSettingsPlugin; +import org.elasticsearch.threadpool.TestThreadPool; import org.elasticsearch.threadpool.ThreadPool; import java.util.Arrays; @@ -177,7 +178,7 @@ public class IndexFieldDataServiceTests extends ESSingleNodeTestCase { } private void doTestRequireDocValues(MappedFieldType ft) { - ThreadPool threadPool = new ThreadPool("random_threadpool_name"); + ThreadPool threadPool = new TestThreadPool("random_threadpool_name"); try { IndicesFieldDataCache cache = new IndicesFieldDataCache(Settings.EMPTY, null); IndexFieldDataService ifds = new IndexFieldDataService(IndexSettingsModule.newIndexSettings("test", Settings.EMPTY), cache, null, null); diff --git a/core/src/test/java/org/elasticsearch/index/mapper/DynamicMappingDisabledTests.java b/core/src/test/java/org/elasticsearch/index/mapper/DynamicMappingDisabledTests.java index 26399218829..8960f9dfe6e 100644 --- a/core/src/test/java/org/elasticsearch/index/mapper/DynamicMappingDisabledTests.java +++ b/core/src/test/java/org/elasticsearch/index/mapper/DynamicMappingDisabledTests.java @@ -35,6 +35,7 @@ import org.elasticsearch.index.IndexNotFoundException; import org.elasticsearch.indices.IndicesService; import org.elasticsearch.indices.breaker.NoneCircuitBreakerService; import org.elasticsearch.test.ESSingleNodeTestCase; +import org.elasticsearch.threadpool.TestThreadPool; import org.elasticsearch.threadpool.ThreadPool; import org.elasticsearch.transport.TransportService; import org.elasticsearch.transport.local.LocalTransport; @@ -64,7 +65,7 @@ public class DynamicMappingDisabledTests extends ESSingleNodeTestCase { @BeforeClass public static void createThreadPool() { - THREAD_POOL = new ThreadPool("DynamicMappingDisabledTests"); + THREAD_POOL = new TestThreadPool("DynamicMappingDisabledTests"); } @Override diff --git a/core/src/test/java/org/elasticsearch/index/mapper/all/AllFieldMapperPositionIncrementGapTests.java b/core/src/test/java/org/elasticsearch/index/mapper/all/AllFieldMapperPositionIncrementGapTests.java index 702c9b85da4..7b106863341 100644 --- a/core/src/test/java/org/elasticsearch/index/mapper/all/AllFieldMapperPositionIncrementGapTests.java +++ b/core/src/test/java/org/elasticsearch/index/mapper/all/AllFieldMapperPositionIncrementGapTests.java @@ -19,6 +19,7 @@ package org.elasticsearch.index.mapper.all; +import org.elasticsearch.action.support.WriteRequest.RefreshPolicy; import org.elasticsearch.client.Client; import org.elasticsearch.index.query.MatchPhraseQueryBuilder; import org.elasticsearch.plugins.Plugin; @@ -87,7 +88,7 @@ public class AllFieldMapperPositionIncrementGapTests extends ESSingleNodeTestCas private static void testGap(Client client, String indexName, String type, int positionIncrementGap) throws IOException { client.prepareIndex(indexName, type, "position_gap_test") - .setSource("string1", "one", "string2", "two three").setRefresh(true).get(); + .setSource("string1", "one", "string2", "two three").setRefreshPolicy(RefreshPolicy.IMMEDIATE).get(); // Baseline - phrase query finds matches in the same field value assertHitCount(client.prepareSearch(indexName) diff --git a/core/src/test/java/org/elasticsearch/index/mapper/geo/GeoPointFieldMapperTests.java b/core/src/test/java/org/elasticsearch/index/mapper/geo/GeoPointFieldMapperTests.java index c7142be3c6c..a1fdb7ec60f 100644 --- a/core/src/test/java/org/elasticsearch/index/mapper/geo/GeoPointFieldMapperTests.java +++ b/core/src/test/java/org/elasticsearch/index/mapper/geo/GeoPointFieldMapperTests.java @@ -47,6 +47,7 @@ import java.util.Map; import java.lang.NumberFormatException; import static org.apache.lucene.spatial.util.GeoEncodingUtils.mortonHash; +import static org.elasticsearch.action.support.WriteRequest.RefreshPolicy.IMMEDIATE; import static org.elasticsearch.common.geo.GeoHashUtils.stringEncode; import static org.elasticsearch.common.xcontent.XContentFactory.jsonBuilder; import static org.elasticsearch.index.query.QueryBuilders.matchAllQuery; @@ -812,7 +813,7 @@ public class GeoPointFieldMapperTests extends ESSingleNodeTestCase { mappingRequest.execute().actionGet(); client().admin().cluster().prepareHealth().setWaitForEvents(Priority.LANGUID).setWaitForGreenStatus().execute().actionGet(); client().prepareIndex("test", "pin", "1").setSource(jsonBuilder().startObject().startObject("location").field("lat", 40.7143528) - .field("lon", -74.0059731).endObject().endObject()).setRefresh(true).execute().actionGet(); + .field("lon", -74.0059731).endObject().endObject()).setRefreshPolicy(IMMEDIATE).get(); // match all search with geohash field SearchResponse searchResponse = client().prepareSearch().addField("location.geohash").setQuery(matchAllQuery()).execute().actionGet(); @@ -837,7 +838,7 @@ public class GeoPointFieldMapperTests extends ESSingleNodeTestCase { mappingRequest.execute().actionGet(); client().admin().cluster().prepareHealth().setWaitForEvents(Priority.LANGUID).setWaitForGreenStatus().execute().actionGet(); client().prepareIndex("test", "pin", "1").setSource(jsonBuilder().startObject().startObject("location").field("lat", 40.7143528) - .field("lon", -74.0059731).endObject().endObject()).setRefresh(true).execute().actionGet(); + .field("lon", -74.0059731).endObject().endObject()).setRefreshPolicy(IMMEDIATE).get(); // match all search with geohash field (includes prefixes) SearchResponse searchResponse = client().prepareSearch().addField("location.geohash").setQuery(matchAllQuery()).execute().actionGet(); @@ -867,7 +868,7 @@ public class GeoPointFieldMapperTests extends ESSingleNodeTestCase { for (int i=0; i 0) { // No match across gaps when slop < position gap - assertHitCount(client.prepareSearch(indexName).setQuery(matchPhraseQuery("string", "one two").slop(positionIncrementGap - 1)).get(), + assertHitCount( + client.prepareSearch(indexName).setQuery(matchPhraseQuery("string", "one two").slop(positionIncrementGap - 1)).get(), 0); } // Match across gaps when slop >= position gap assertHitCount(client.prepareSearch(indexName).setQuery(matchPhraseQuery("string", "one two").slop(positionIncrementGap)).get(), 1); - assertHitCount(client.prepareSearch(indexName).setQuery(matchPhraseQuery("string", "one two").slop(positionIncrementGap + 1)).get(), 1); + assertHitCount(client.prepareSearch(indexName).setQuery(matchPhraseQuery("string", "one two").slop(positionIncrementGap + 1)).get(), + 1); } } diff --git a/core/src/test/java/org/elasticsearch/index/shard/IndexShardTests.java b/core/src/test/java/org/elasticsearch/index/shard/IndexShardTests.java index 7774537c734..8889ebfb910 100644 --- a/core/src/test/java/org/elasticsearch/index/shard/IndexShardTests.java +++ b/core/src/test/java/org/elasticsearch/index/shard/IndexShardTests.java @@ -49,8 +49,6 @@ import org.elasticsearch.cluster.ClusterState; import org.elasticsearch.cluster.InternalClusterInfoService; import org.elasticsearch.cluster.metadata.IndexMetaData; import org.elasticsearch.cluster.metadata.MappingMetaData; -import org.elasticsearch.common.UUIDs; -import org.elasticsearch.snapshots.SnapshotId; import org.elasticsearch.cluster.node.DiscoveryNode; import org.elasticsearch.cluster.routing.AllocationId; import org.elasticsearch.cluster.routing.RestoreSource; @@ -60,6 +58,7 @@ import org.elasticsearch.cluster.routing.ShardRoutingState; import org.elasticsearch.cluster.routing.TestShardRouting; import org.elasticsearch.cluster.routing.UnassignedInfo; import org.elasticsearch.cluster.service.ClusterService; +import org.elasticsearch.common.UUIDs; import org.elasticsearch.common.bytes.BytesArray; import org.elasticsearch.common.bytes.BytesReference; import org.elasticsearch.common.collect.ImmutableOpenMap; @@ -99,6 +98,7 @@ import org.elasticsearch.indices.IndicesService; import org.elasticsearch.indices.recovery.RecoveryState; import org.elasticsearch.plugins.Plugin; import org.elasticsearch.snapshots.Snapshot; +import org.elasticsearch.snapshots.SnapshotId; import org.elasticsearch.test.DummyShardLock; import org.elasticsearch.test.ESSingleNodeTestCase; import org.elasticsearch.test.FieldMaskingReader; @@ -128,6 +128,8 @@ import java.util.function.BiConsumer; import static java.util.Collections.emptyMap; import static java.util.Collections.emptySet; +import static org.elasticsearch.action.support.WriteRequest.RefreshPolicy.IMMEDIATE; +import static org.elasticsearch.action.support.WriteRequest.RefreshPolicy.NONE; import static org.elasticsearch.cluster.metadata.IndexMetaData.SETTING_NUMBER_OF_REPLICAS; import static org.elasticsearch.cluster.metadata.IndexMetaData.SETTING_NUMBER_OF_SHARDS; import static org.elasticsearch.cluster.metadata.IndexMetaData.SETTING_VERSION_CREATED; @@ -524,7 +526,7 @@ public class IndexShardTests extends ESSingleNodeTestCase { public void testRecoverIntoLeftover() throws IOException { createIndex("test"); ensureGreen("test"); - client().prepareIndex("test", "bar", "1").setSource("{}").setRefresh(true).get(); + client().prepareIndex("test", "bar", "1").setSource("{}").setRefreshPolicy(IMMEDIATE).get(); client().admin().indices().prepareFlush("test").get(); SearchResponse response = client().prepareSearch("test").get(); assertHitCount(response, 1L); @@ -554,11 +556,12 @@ public class IndexShardTests extends ESSingleNodeTestCase { .build(); createIndex("test", idxSettings); ensureGreen("test"); - client().prepareIndex("test", "bar", "1").setSource("{}").setRefresh(true).get(); + client().prepareIndex("test", "bar", "1").setSource("{}").setRefreshPolicy(IMMEDIATE).get(); SearchResponse response = client().prepareSearch("test").get(); assertHitCount(response, 1L); client().admin().indices().prepareDelete("test").get(); - assertBusyPathHasBeenCleared(idxPath); + assertAllIndicesRemovedAndDeletionCompleted(Collections.singleton(getInstanceFromNode(IndicesService.class))); + assertPathHasBeenCleared(idxPath); } public void testExpectedShardSizeIsPresent() throws InterruptedException { @@ -600,7 +603,7 @@ public class IndexShardTests extends ESSingleNodeTestCase { logger.info("--> creating an index with data_path [{}]", startDir.toAbsolutePath().toString()); createIndex(INDEX, sb); ensureGreen(INDEX); - client().prepareIndex(INDEX, "bar", "1").setSource("{}").setRefresh(true).get(); + client().prepareIndex(INDEX, "bar", "1").setSource("{}").setRefreshPolicy(IMMEDIATE).get(); SearchResponse resp = client().prepareSearch(INDEX).setQuery(matchAllQuery()).get(); assertThat("found the hit", resp.getHits().getTotalHits(), equalTo(1L)); @@ -641,8 +644,9 @@ public class IndexShardTests extends ESSingleNodeTestCase { assertThat("found the hit", resp.getHits().getTotalHits(), equalTo(1L)); assertAcked(client().admin().indices().prepareDelete(INDEX)); - assertBusyPathHasBeenCleared(startDir.toAbsolutePath()); - assertBusyPathHasBeenCleared(endDir.toAbsolutePath()); + assertAllIndicesRemovedAndDeletionCompleted(Collections.singleton(getInstanceFromNode(IndicesService.class))); + assertPathHasBeenCleared(startDir.toAbsolutePath()); + assertPathHasBeenCleared(endDir.toAbsolutePath()); } public void testShardStats() throws IOException { @@ -687,7 +691,7 @@ public class IndexShardTests extends ESSingleNodeTestCase { public void testIndexingOperationsListeners() throws IOException { createIndex("test_iol"); ensureGreen(); - client().prepareIndex("test_iol", "test", "0").setSource("{\"foo\" : \"bar\"}").setRefresh(true).get(); + client().prepareIndex("test_iol", "test", "0").setSource("{\"foo\" : \"bar\"}").setRefreshPolicy(IMMEDIATE).get(); IndicesService indicesService = getInstanceFromNode(IndicesService.class); IndexService test = indicesService.indexService(resolveIndex("test_iol")); IndexShard shard = test.getShardOrNull(0); @@ -810,14 +814,14 @@ public class IndexShardTests extends ESSingleNodeTestCase { IndexShard shard = test.getShardOrNull(0); assertFalse(shard.shouldFlush()); client().admin().indices().prepareUpdateSettings("test").setSettings(Settings.builder().put(IndexSettings.INDEX_TRANSLOG_FLUSH_THRESHOLD_SIZE_SETTING.getKey(), new ByteSizeValue(133 /* size of the operation + header&footer*/, ByteSizeUnit.BYTES)).build()).get(); - client().prepareIndex("test", "test", "0").setSource("{}").setRefresh(randomBoolean()).get(); + client().prepareIndex("test", "test", "0").setSource("{}").setRefreshPolicy(randomBoolean() ? IMMEDIATE : NONE).get(); assertFalse(shard.shouldFlush()); ParsedDocument doc = testParsedDocument("1", "1", "test", null, -1, -1, new ParseContext.Document(), new BytesArray(new byte[]{1}), null); Engine.Index index = new Engine.Index(new Term("_uid", "1"), doc); shard.index(index); assertTrue(shard.shouldFlush()); assertEquals(2, shard.getEngine().getTranslog().totalOperations()); - client().prepareIndex("test", "test", "2").setSource("{}").setRefresh(randomBoolean()).get(); + client().prepareIndex("test", "test", "2").setSource("{}").setRefreshPolicy(randomBoolean() ? IMMEDIATE : NONE).get(); assertBusy(() -> { // this is async assertFalse(shard.shouldFlush()); }); @@ -844,7 +848,7 @@ public class IndexShardTests extends ESSingleNodeTestCase { final IndexShard shard = test.getShardOrNull(0); assertFalse(shard.shouldFlush()); client().admin().indices().prepareUpdateSettings("test").setSettings(Settings.builder().put(IndexSettings.INDEX_TRANSLOG_FLUSH_THRESHOLD_SIZE_SETTING.getKey(), new ByteSizeValue(133/* size of the operation + header&footer*/, ByteSizeUnit.BYTES)).build()).get(); - client().prepareIndex("test", "test", "0").setSource("{}").setRefresh(randomBoolean()).get(); + client().prepareIndex("test", "test", "0").setSource("{}").setRefreshPolicy(randomBoolean() ? IMMEDIATE : NONE).get(); assertFalse(shard.shouldFlush()); final AtomicBoolean running = new AtomicBoolean(true); final int numThreads = randomIntBetween(2, 4); @@ -970,7 +974,7 @@ public class IndexShardTests extends ESSingleNodeTestCase { IndexService test = indicesService.indexService(resolveIndex("test")); final IndexShard shard = test.getShardOrNull(0); int translogOps = 1; - client().prepareIndex("test", "test", "0").setSource("{}").setRefresh(randomBoolean()).get(); + client().prepareIndex("test", "test", "0").setSource("{}").setRefreshPolicy(randomBoolean() ? IMMEDIATE : NONE).get(); if (randomBoolean()) { client().admin().indices().prepareFlush().get(); translogOps = 0; @@ -998,7 +1002,7 @@ public class IndexShardTests extends ESSingleNodeTestCase { IndicesService indicesService = getInstanceFromNode(IndicesService.class); IndexService test = indicesService.indexService(resolveIndex("test")); final IndexShard shard = test.getShardOrNull(0); - client().prepareIndex("test", "test", "0").setSource("{}").setRefresh(randomBoolean()).get(); + client().prepareIndex("test", "test", "0").setSource("{}").setRefreshPolicy(randomBoolean() ? IMMEDIATE : NONE).get(); if (randomBoolean()) { client().admin().indices().prepareFlush().get(); } @@ -1028,7 +1032,7 @@ public class IndexShardTests extends ESSingleNodeTestCase { IndexService test = indicesService.indexService(resolveIndex("test")); final IndexShard shard = test.getShardOrNull(0); - client().prepareIndex("test", "test", "0").setSource("{}").setRefresh(randomBoolean()).get(); + client().prepareIndex("test", "test", "0").setSource("{}").setRefreshPolicy(randomBoolean() ? IMMEDIATE : NONE).get(); if (randomBoolean()) { client().admin().indices().prepareFlush().get(); } @@ -1105,8 +1109,8 @@ public class IndexShardTests extends ESSingleNodeTestCase { IndexService test_target = indicesService.indexService(resolveIndex("test_target")); final IndexShard test_shard = test.getShardOrNull(0); - client().prepareIndex("test", "test", "0").setSource("{}").setRefresh(randomBoolean()).get(); - client().prepareIndex("test_target", "test", "1").setSource("{}").setRefresh(true).get(); + client().prepareIndex("test", "test", "0").setSource("{}").setRefreshPolicy(randomBoolean() ? IMMEDIATE : NONE).get(); + client().prepareIndex("test_target", "test", "1").setSource("{}").setRefreshPolicy(IMMEDIATE).get(); assertHitCount(client().prepareSearch("test_target").get(), 1); assertSearchHits(client().prepareSearch("test_target").get(), "1"); client().admin().indices().prepareFlush("test").get(); // only flush test @@ -1163,8 +1167,8 @@ public class IndexShardTests extends ESSingleNodeTestCase { IndicesService indicesService = getInstanceFromNode(IndicesService.class); IndexService indexService = indicesService.indexService(resolveIndex("test")); IndexShard shard = indexService.getShardOrNull(0); - client().prepareIndex("test", "test", "0").setSource("{\"foo\" : \"bar\"}").setRefresh(true).get(); - client().prepareIndex("test", "test", "1").setSource("{\"foobar\" : \"bar\"}").setRefresh(true).get(); + client().prepareIndex("test", "test", "0").setSource("{\"foo\" : \"bar\"}").setRefreshPolicy(IMMEDIATE).get(); + client().prepareIndex("test", "test", "1").setSource("{\"foobar\" : \"bar\"}").setRefreshPolicy(IMMEDIATE).get(); Engine.GetResult getResult = shard.get(new Engine.Get(false, new Term(UidFieldMapper.NAME, Uid.createUid("test", "1")))); assertTrue(getResult.exists()); @@ -1213,8 +1217,8 @@ public class IndexShardTests extends ESSingleNodeTestCase { IndexService indexService = indicesService.indexService(resolveIndex("test")); IndexShard shard = indexService.getShardOrNull(0); client().admin().indices().preparePutMapping("test").setType("test").setSource("foo", "type=text,fielddata=true").get(); - client().prepareIndex("test", "test", "0").setSource("{\"foo\" : \"bar\"}").setRefresh(true).get(); - client().prepareIndex("test", "test", "1").setSource("{\"foobar\" : \"bar\"}").setRefresh(true).get(); + client().prepareIndex("test", "test", "0").setSource("{\"foo\" : \"bar\"}").setRefreshPolicy(IMMEDIATE).get(); + client().prepareIndex("test", "test", "1").setSource("{\"foobar\" : \"bar\"}").setRefreshPolicy(IMMEDIATE).get(); IndexSearcherWrapper wrapper = new IndexSearcherWrapper() { @Override @@ -1264,7 +1268,7 @@ public class IndexShardTests extends ESSingleNodeTestCase { IndexShard shard = indexService.getShardOrNull(0); client().prepareIndex("test", "test", "0").setSource("{\"foo\" : \"bar\"}").get(); client().prepareDelete("test", "test", "0").get(); - client().prepareIndex("test", "test", "1").setSource("{\"foo\" : \"bar\"}").setRefresh(true).get(); + client().prepareIndex("test", "test", "1").setSource("{\"foo\" : \"bar\"}").setRefreshPolicy(IMMEDIATE).get(); IndexSearcherWrapper wrapper = new IndexSearcherWrapper() {}; shard.close("simon says", false); @@ -1322,7 +1326,7 @@ public class IndexShardTests extends ESSingleNodeTestCase { IndexShard shard = indexService.getShardOrNull(0); client().prepareIndex("test", "test", "0").setSource("{\"foo\" : \"bar\"}").get(); client().prepareDelete("test", "test", "0").get(); - client().prepareIndex("test", "test", "1").setSource("{\"foo\" : \"bar\"}").setRefresh(true).get(); + client().prepareIndex("test", "test", "1").setSource("{\"foo\" : \"bar\"}").setRefreshPolicy(IMMEDIATE).get(); IndexSearcherWrapper wrapper = new IndexSearcherWrapper() {}; shard.close("simon says", false); @@ -1374,7 +1378,7 @@ public class IndexShardTests extends ESSingleNodeTestCase { IndicesService indicesService = getInstanceFromNode(IndicesService.class); IndexService indexService = indicesService.indexService(resolveIndex("test")); IndexShard shard = indexService.getShardOrNull(0); - client().prepareIndex("test", "test", "0").setSource("{\"foo\" : \"bar\"}").setRefresh(true).get(); + client().prepareIndex("test", "test", "0").setSource("{\"foo\" : \"bar\"}").setRefreshPolicy(IMMEDIATE).get(); IndexSearcherWrapper wrapper = new IndexSearcherWrapper() { @Override public DirectoryReader wrap(DirectoryReader reader) throws IOException { @@ -1527,8 +1531,8 @@ public class IndexShardTests extends ESSingleNodeTestCase { .field("type", "text") .endObject() .endObject().endObject().endObject()).get(); - client().prepareIndex("index", "test", "0").setSource("{\"foo\" : \"bar\"}").setRefresh(true).get(); - client().prepareIndex("index", "test", "1").setSource("{\"foo\" : \"bar\"}").setRefresh(true).get(); + client().prepareIndex("index", "test", "0").setSource("{\"foo\" : \"bar\"}").setRefreshPolicy(IMMEDIATE).get(); + client().prepareIndex("index", "test", "1").setSource("{\"foo\" : \"bar\"}").setRefreshPolicy(IMMEDIATE).get(); IndicesService indicesService = getInstanceFromNode(IndicesService.class); diff --git a/core/src/test/java/org/elasticsearch/index/shard/RefreshListenersTests.java b/core/src/test/java/org/elasticsearch/index/shard/RefreshListenersTests.java new file mode 100644 index 00000000000..eb0fca7ce9e --- /dev/null +++ b/core/src/test/java/org/elasticsearch/index/shard/RefreshListenersTests.java @@ -0,0 +1,309 @@ +/* + * 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.document.Field; +import org.apache.lucene.document.NumericDocValuesField; +import org.apache.lucene.document.TextField; +import org.apache.lucene.index.IndexWriterConfig; +import org.apache.lucene.index.KeepOnlyLastCommitDeletionPolicy; +import org.apache.lucene.index.SnapshotDeletionPolicy; +import org.apache.lucene.index.Term; +import org.apache.lucene.search.IndexSearcher; +import org.apache.lucene.store.Directory; +import org.apache.lucene.util.IOUtils; +import org.elasticsearch.common.Nullable; +import org.elasticsearch.common.bytes.BytesArray; +import org.elasticsearch.common.bytes.BytesReference; +import org.elasticsearch.common.lucene.uid.Versions; +import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.common.unit.TimeValue; +import org.elasticsearch.common.util.BigArrays; +import org.elasticsearch.common.util.concurrent.FutureUtils; +import org.elasticsearch.index.Index; +import org.elasticsearch.index.IndexSettings; +import org.elasticsearch.index.codec.CodecService; +import org.elasticsearch.index.engine.Engine; +import org.elasticsearch.index.engine.EngineConfig; +import org.elasticsearch.index.engine.InternalEngine; +import org.elasticsearch.index.engine.InternalEngineTests.TranslogHandler; +import org.elasticsearch.index.fieldvisitor.SingleFieldsVisitor; +import org.elasticsearch.index.mapper.ParseContext.Document; +import org.elasticsearch.index.mapper.ParsedDocument; +import org.elasticsearch.index.mapper.internal.UidFieldMapper; +import org.elasticsearch.index.store.DirectoryService; +import org.elasticsearch.index.store.Store; +import org.elasticsearch.index.translog.TranslogConfig; +import org.elasticsearch.test.DummyShardLock; +import org.elasticsearch.test.ESTestCase; +import org.elasticsearch.test.IndexSettingsModule; +import org.elasticsearch.threadpool.TestThreadPool; +import org.elasticsearch.threadpool.ThreadPool; +import org.junit.After; +import org.junit.Before; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.List; +import java.util.Locale; +import java.util.concurrent.ScheduledFuture; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.atomic.AtomicReference; +import java.util.function.Consumer; + +import static org.elasticsearch.common.unit.TimeValue.timeValueMillis; + +/** + * Tests how {@linkplain RefreshListeners} interacts with {@linkplain InternalEngine}. + */ +public class RefreshListenersTests extends ESTestCase { + private RefreshListeners listeners; + private Engine engine; + private volatile int maxListeners; + private ThreadPool threadPool; + private Store store; + + @Before + public void setupListeners() throws Exception { + // Setup dependencies of the listeners + maxListeners = randomIntBetween(1, 1000); + listeners = new RefreshListeners( + () -> maxListeners, + () -> engine.refresh("too-many-listeners"), + // Immediately run listeners rather than adding them to the listener thread pool like IndexShard does to simplify the test. + Runnable::run, + logger + ); + + // Now setup the InternalEngine which is much more complicated because we aren't mocking anything + threadPool = new TestThreadPool(getTestName()); + IndexSettings indexSettings = IndexSettingsModule.newIndexSettings("index", Settings.EMPTY); + ShardId shardId = new ShardId(new Index("index", "_na_"), 1); + Directory directory = newDirectory(); + DirectoryService directoryService = new DirectoryService(shardId, indexSettings) { + @Override + public Directory newDirectory() throws IOException { + return directory; + } + + @Override + public long throttleTimeInNanos() { + return 0; + } + }; + store = new Store(shardId, indexSettings, directoryService, new DummyShardLock(shardId)); + IndexWriterConfig iwc = newIndexWriterConfig(); + TranslogConfig translogConfig = new TranslogConfig(shardId, createTempDir("translog"), indexSettings, + BigArrays.NON_RECYCLING_INSTANCE); + Engine.EventListener eventListener = new Engine.EventListener() { + @Override + public void onFailedEngine(String reason, @Nullable Throwable t) { + // we don't need to notify anybody in this test + } + }; + EngineConfig config = new EngineConfig(EngineConfig.OpenMode.CREATE_INDEX_AND_TRANSLOG, shardId, threadPool, indexSettings, null, + store, new SnapshotDeletionPolicy(new KeepOnlyLastCommitDeletionPolicy()), newMergePolicy(), iwc.getAnalyzer(), + iwc.getSimilarity(), new CodecService(null, logger), eventListener, new TranslogHandler(shardId.getIndexName(), logger), + IndexSearcher.getDefaultQueryCache(), IndexSearcher.getDefaultQueryCachingPolicy(), translogConfig, + TimeValue.timeValueMinutes(5), listeners); + engine = new InternalEngine(config); + } + + @After + public void tearDownListeners() throws Exception { + IOUtils.close(engine, store); + terminate(threadPool); + } + + public void testTooMany() throws Exception { + assertFalse(listeners.refreshNeeded()); + Engine.Index index = index("1"); + + // Fill the listener slots + List nonForcedListeners = new ArrayList<>(maxListeners); + for (int i = 0; i < maxListeners; i++) { + DummyRefreshListener listener = new DummyRefreshListener(); + nonForcedListeners.add(listener); + listeners.addOrNotify(index.getTranslogLocation(), listener); + assertTrue(listeners.refreshNeeded()); + } + + // We shouldn't have called any of them + for (DummyRefreshListener listener : nonForcedListeners) { + assertNull("Called listener too early!", listener.forcedRefresh.get()); + } + + // Add one more listener which should cause a refresh. + DummyRefreshListener forcingListener = new DummyRefreshListener(); + listeners.addOrNotify(index.getTranslogLocation(), forcingListener); + assertTrue("Forced listener wasn't forced?", forcingListener.forcedRefresh.get()); + forcingListener.assertNoError(); + + // That forces all the listeners through. It would be on the listener ThreadPool but we've made all of those execute immediately. + for (DummyRefreshListener listener : nonForcedListeners) { + assertEquals("Expected listener called with unforced refresh!", Boolean.FALSE, listener.forcedRefresh.get()); + listener.assertNoError(); + } + assertFalse(listeners.refreshNeeded()); + } + + public void testAfterRefresh() throws Exception { + Engine.Index index = index("1"); + engine.refresh("I said so"); + if (randomBoolean()) { + index(randomFrom("1" /* same document */, "2" /* different document */)); + if (randomBoolean()) { + engine.refresh("I said so"); + } + } + + DummyRefreshListener listener = new DummyRefreshListener(); + listeners.addOrNotify(index.getTranslogLocation(), listener); + assertFalse(listener.forcedRefresh.get()); + listener.assertNoError(); + } + + /** + * Attempts to add a listener at the same time as a refresh occurs by having a background thread force a refresh as fast as it can while + * adding listeners. This can catch the situation where a refresh happens right as the listener is being added such that the listener + * misses the refresh and has to catch the next one. If the listener wasn't able to properly catch the next one then this would fail. + */ + public void testConcurrentRefresh() throws Exception { + AtomicBoolean run = new AtomicBoolean(true); + Thread refresher = new Thread(() -> { + while (run.get()) { + engine.refresh("test"); + } + }); + refresher.start(); + try { + for (int i = 0; i < 100; i++) { + Engine.Index index = index("1"); + + DummyRefreshListener listener = new DummyRefreshListener(); + listeners.addOrNotify(index.getTranslogLocation(), listener); + assertBusy(() -> assertNotNull(listener.forcedRefresh.get())); + assertFalse(listener.forcedRefresh.get()); + listener.assertNoError(); + } + } finally { + run.set(false); + refresher.join(); + } + } + + /** + * Uses a bunch of threads to index, wait for refresh, and non-realtime get documents to validate that they are visible after waiting + * regardless of what crazy sequence of events causes the refresh listener to fire. + */ + public void testLotsOfThreads() throws Exception { + int threadCount = between(3, 10); + maxListeners = between(1, threadCount * 2); + + // This thread just refreshes every once in a while to cause trouble. + ScheduledFuture refresher = threadPool.scheduleWithFixedDelay(() -> engine.refresh("because test"), timeValueMillis(100)); + + // These threads add and block until the refresh makes the change visible and then do a non-realtime get. + Thread[] indexers = new Thread[threadCount]; + for (int thread = 0; thread < threadCount; thread++) { + final String threadId = String.format(Locale.ROOT, "%04d", thread); + indexers[thread] = new Thread(() -> { + for (int iteration = 1; iteration <= 50; iteration++) { + try { + String testFieldValue = String.format(Locale.ROOT, "%s%04d", threadId, iteration); + Engine.Index index = index(threadId, testFieldValue); + assertEquals(iteration, index.version()); + + DummyRefreshListener listener = new DummyRefreshListener(); + listeners.addOrNotify(index.getTranslogLocation(), listener); + assertBusy(() -> assertNotNull("listener never called", listener.forcedRefresh.get())); + if (threadCount < maxListeners) { + assertFalse(listener.forcedRefresh.get()); + } + listener.assertNoError(); + + Engine.Get get = new Engine.Get(false, index.uid()); + try (Engine.GetResult getResult = engine.get(get)) { + assertTrue("document not found", getResult.exists()); + assertEquals(iteration, getResult.version()); + SingleFieldsVisitor visitor = new SingleFieldsVisitor("test"); + getResult.docIdAndVersion().context.reader().document(getResult.docIdAndVersion().docId, visitor); + assertEquals(Arrays.asList(testFieldValue), visitor.fields().get("test")); + } + } catch (Throwable t) { + throw new RuntimeException("failure on the [" + iteration + "] iteration of thread [" + threadId + "]", t); + } + } + }); + indexers[thread].start(); + } + + for (Thread indexer: indexers) { + indexer.join(); + } + FutureUtils.cancel(refresher); + } + + private Engine.Index index(String id) { + return index(id, "test"); + } + + private Engine.Index index(String id, String testFieldValue) { + String type = "test"; + String uid = type + ":" + id; + Document document = new Document(); + document.add(new TextField("test", testFieldValue, Field.Store.YES)); + Field uidField = new Field("_uid", type + ":" + id, UidFieldMapper.Defaults.FIELD_TYPE); + Field versionField = new NumericDocValuesField("_version", Versions.MATCH_ANY); + document.add(uidField); + document.add(versionField); + BytesReference source = new BytesArray(new byte[] { 1 }); + ParsedDocument doc = new ParsedDocument(versionField, id, type, null, -1, -1, Arrays.asList(document), source, null); + Engine.Index index = new Engine.Index(new Term("_uid", uid), doc); + engine.index(index); + return index; + } + + private static class DummyRefreshListener implements Consumer { + /** + * When the listener is called this captures it's only argument. + */ + AtomicReference forcedRefresh = new AtomicReference<>(); + private volatile Throwable error; + + @Override + public void accept(Boolean forcedRefresh) { + try { + assertNotNull(forcedRefresh); + Boolean oldValue = this.forcedRefresh.getAndSet(forcedRefresh); + assertNull("Listener called twice", oldValue); + } catch (Throwable e) { + error = e; + } + } + + public void assertNoError() { + if (error != null) { + throw new RuntimeException(error); + } + } + } +} diff --git a/core/src/test/java/org/elasticsearch/index/store/StoreTests.java b/core/src/test/java/org/elasticsearch/index/store/StoreTests.java index be45b1ad330..4f3e68a9115 100644 --- a/core/src/test/java/org/elasticsearch/index/store/StoreTests.java +++ b/core/src/test/java/org/elasticsearch/index/store/StoreTests.java @@ -199,6 +199,19 @@ public class StoreTests extends ESTestCase { IOUtils.close(indexInput, verifyingOutput, dir); } + public void testVerifyingIndexOutputOnEmptyFile() throws IOException { + Directory dir = newDirectory(); + IndexOutput verifyingOutput = new Store.LuceneVerifyingIndexOutput(new StoreFileMetaData("foo.bar", 0, Store.digestToString(0)), + dir.createOutput("foo1.bar", IOContext.DEFAULT)); + try { + Store.verify(verifyingOutput); + fail("should be a corrupted index"); + } catch (CorruptIndexException | IndexFormatTooOldException | IndexFormatTooNewException ex) { + // ok + } + IOUtils.close(verifyingOutput, dir); + } + public void testChecksumCorrupted() throws IOException { Directory dir = newDirectory(); IndexOutput output = dir.createOutput("foo.bar", IOContext.DEFAULT); diff --git a/core/src/test/java/org/elasticsearch/index/termvectors/TermVectorsServiceTests.java b/core/src/test/java/org/elasticsearch/index/termvectors/TermVectorsServiceTests.java index a4b80577923..c79a61a22b9 100644 --- a/core/src/test/java/org/elasticsearch/index/termvectors/TermVectorsServiceTests.java +++ b/core/src/test/java/org/elasticsearch/index/termvectors/TermVectorsServiceTests.java @@ -34,6 +34,7 @@ import java.util.stream.Stream; import static java.lang.Math.abs; import static java.util.stream.Collectors.toList; +import static org.elasticsearch.action.support.WriteRequest.RefreshPolicy.IMMEDIATE; import static org.elasticsearch.common.xcontent.XContentFactory.jsonBuilder; import static org.hamcrest.Matchers.equalTo; import static org.hamcrest.Matchers.notNullValue; @@ -55,7 +56,7 @@ public class TermVectorsServiceTests extends ESSingleNodeTestCase { createIndex("test", Settings.EMPTY, "type1", mapping); ensureGreen(); - client().prepareIndex("test", "type1", "0").setSource("field", "foo bar").setRefresh(true).execute().get(); + client().prepareIndex("test", "type1", "0").setSource("field", "foo bar").setRefreshPolicy(IMMEDIATE).get(); IndicesService indicesService = getInstanceFromNode(IndicesService.class); IndexService test = indicesService.indexService(resolveIndex("test")); diff --git a/core/src/test/java/org/elasticsearch/index/translog/TranslogTests.java b/core/src/test/java/org/elasticsearch/index/translog/TranslogTests.java index b4d2423921c..889897e21f6 100644 --- a/core/src/test/java/org/elasticsearch/index/translog/TranslogTests.java +++ b/core/src/test/java/org/elasticsearch/index/translog/TranslogTests.java @@ -20,6 +20,7 @@ package org.elasticsearch.index.translog; import com.carrotsearch.randomizedtesting.generators.RandomPicks; + import org.apache.lucene.codecs.CodecUtil; import org.apache.lucene.index.Term; import org.apache.lucene.mockfile.FilterFileChannel; @@ -44,6 +45,7 @@ import org.elasticsearch.common.util.concurrent.AbstractRunnable; import org.elasticsearch.common.util.concurrent.ConcurrentCollections; import org.elasticsearch.index.VersionType; import org.elasticsearch.index.shard.ShardId; +import org.elasticsearch.index.translog.Translog.Location; import org.elasticsearch.test.ESTestCase; import org.elasticsearch.test.IndexSettingsModule; import org.hamcrest.Matchers; @@ -204,18 +206,40 @@ public class TranslogTests extends ESTestCase { } public void testRead() throws IOException { + Location loc0 = translog.getLastWriteLocation(); + assertNotNull(loc0); + Translog.Location loc1 = translog.add(new Translog.Index("test", "1", new byte[]{1})); + assertThat(loc1, greaterThan(loc0)); + assertThat(translog.getLastWriteLocation(), greaterThan(loc1)); Translog.Location loc2 = translog.add(new Translog.Index("test", "2", new byte[]{2})); + assertThat(loc2, greaterThan(loc1)); + assertThat(translog.getLastWriteLocation(), greaterThan(loc2)); assertThat(translog.read(loc1).getSource().source.toBytesArray(), equalTo(new BytesArray(new byte[]{1}))); assertThat(translog.read(loc2).getSource().source.toBytesArray(), equalTo(new BytesArray(new byte[]{2}))); + + Translog.Location lastLocBeforeSync = translog.getLastWriteLocation(); translog.sync(); + assertEquals(lastLocBeforeSync, translog.getLastWriteLocation()); assertThat(translog.read(loc1).getSource().source.toBytesArray(), equalTo(new BytesArray(new byte[]{1}))); assertThat(translog.read(loc2).getSource().source.toBytesArray(), equalTo(new BytesArray(new byte[]{2}))); + Translog.Location loc3 = translog.add(new Translog.Index("test", "2", new byte[]{3})); + assertThat(loc3, greaterThan(loc2)); + assertThat(translog.getLastWriteLocation(), greaterThan(loc3)); assertThat(translog.read(loc3).getSource().source.toBytesArray(), equalTo(new BytesArray(new byte[]{3}))); + + lastLocBeforeSync = translog.getLastWriteLocation(); translog.sync(); + assertEquals(lastLocBeforeSync, translog.getLastWriteLocation()); assertThat(translog.read(loc3).getSource().source.toBytesArray(), equalTo(new BytesArray(new byte[]{3}))); translog.prepareCommit(); + /* + * The commit adds to the lastWriteLocation even though is isn't really a write. This is just an implementation artifact but it can + * safely be ignored because the lastWriteLocation continues to be greater than the Location returned from the last write operation + * and less than the location of the next write operation. + */ + assertThat(translog.getLastWriteLocation(), greaterThan(lastLocBeforeSync)); assertThat(translog.read(loc3).getSource().source.toBytesArray(), equalTo(new BytesArray(new byte[]{3}))); translog.commit(); assertNull(translog.read(loc1)); diff --git a/core/src/test/java/org/elasticsearch/indices/IndicesOptionsIntegrationIT.java b/core/src/test/java/org/elasticsearch/indices/IndicesOptionsIntegrationIT.java index e0446fe329b..0256fab1b7f 100644 --- a/core/src/test/java/org/elasticsearch/indices/IndicesOptionsIntegrationIT.java +++ b/core/src/test/java/org/elasticsearch/indices/IndicesOptionsIntegrationIT.java @@ -53,6 +53,7 @@ import org.elasticsearch.test.ESIntegTestCase; import java.util.Collection; import java.util.function.Function; +import static org.elasticsearch.action.support.WriteRequest.RefreshPolicy.IMMEDIATE; import static org.elasticsearch.index.query.QueryBuilders.matchAllQuery; import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertAcked; import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertHitCount; @@ -332,7 +333,7 @@ public class IndicesOptionsIntegrationIT extends ESIntegTestCase { verify(getSettings(indices).setIndicesOptions(options), false); assertAcked(prepareCreate("foobar")); - client().prepareIndex("foobar", "type", "1").setSource("k", "v").setRefresh(true).execute().actionGet(); + client().prepareIndex("foobar", "type", "1").setSource("k", "v").setRefreshPolicy(IMMEDIATE).get(); // Verify defaults for wildcards, with one wildcard expression and one existing index indices = new String[]{"foo*"}; @@ -422,7 +423,7 @@ public class IndicesOptionsIntegrationIT extends ESIntegTestCase { public void testAllMissingLenient() throws Exception { createIndex("test1"); - client().prepareIndex("test1", "type", "1").setSource("k", "v").setRefresh(true).execute().actionGet(); + client().prepareIndex("test1", "type", "1").setSource("k", "v").setRefreshPolicy(IMMEDIATE).get(); SearchResponse response = client().prepareSearch("test2") .setIndicesOptions(IndicesOptions.lenientExpandOpen()) .setQuery(matchAllQuery()) diff --git a/core/src/test/java/org/elasticsearch/indices/IndicesServiceTests.java b/core/src/test/java/org/elasticsearch/indices/IndicesServiceTests.java index 76f7a30e078..75c968e1d67 100644 --- a/core/src/test/java/org/elasticsearch/indices/IndicesServiceTests.java +++ b/core/src/test/java/org/elasticsearch/indices/IndicesServiceTests.java @@ -49,6 +49,7 @@ import java.util.Arrays; import java.util.concurrent.CountDownLatch; import java.util.concurrent.TimeUnit; +import static org.elasticsearch.action.support.WriteRequest.RefreshPolicy.IMMEDIATE; import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertAcked; import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertHitCount; import static org.hamcrest.Matchers.containsString; @@ -131,7 +132,7 @@ public class IndicesServiceTests extends ESSingleNodeTestCase { test = createIndex("test"); - client().prepareIndex("test", "type", "1").setSource("field", "value").setRefresh(true).get(); + client().prepareIndex("test", "type", "1").setSource("field", "value").setRefreshPolicy(IMMEDIATE).get(); client().admin().indices().prepareFlush("test").get(); assertHitCount(client().prepareSearch("test").get(), 1); IndexMetaData secondMetaData = clusterService.state().metaData().index("test"); @@ -190,10 +191,12 @@ public class IndicesServiceTests extends ESSingleNodeTestCase { assertTrue(path.exists()); assertEquals(indicesService.numPendingDeletes(test.index()), numPending); + assertTrue(indicesService.hasUncompletedPendingDeletes()); // shard lock released... we can now delete indicesService.processPendingDeletes(test.index(), test.getIndexSettings(), new TimeValue(0, TimeUnit.MILLISECONDS)); assertEquals(indicesService.numPendingDeletes(test.index()), 0); + assertFalse(indicesService.hasUncompletedPendingDeletes()); assertFalse(path.exists()); if (randomBoolean()) { @@ -201,9 +204,11 @@ public class IndicesServiceTests extends ESSingleNodeTestCase { indicesService.addPendingDelete(new ShardId(test.index(), 1), test.getIndexSettings()); indicesService.addPendingDelete(new ShardId("bogus", "_na_", 1), test.getIndexSettings()); assertEquals(indicesService.numPendingDeletes(test.index()), 2); + assertTrue(indicesService.hasUncompletedPendingDeletes()); // shard lock released... we can now delete indicesService.processPendingDeletes(test.index(), test.getIndexSettings(), new TimeValue(0, TimeUnit.MILLISECONDS)); assertEquals(indicesService.numPendingDeletes(test.index()), 0); + assertTrue(indicesService.hasUncompletedPendingDeletes()); // "bogus" index has not been removed } assertAcked(client().admin().indices().prepareOpen("test")); diff --git a/core/src/test/java/org/elasticsearch/indices/store/IndicesStoreTests.java b/core/src/test/java/org/elasticsearch/indices/store/IndicesStoreTests.java index f877ae6629b..f197073033d 100644 --- a/core/src/test/java/org/elasticsearch/indices/store/IndicesStoreTests.java +++ b/core/src/test/java/org/elasticsearch/indices/store/IndicesStoreTests.java @@ -35,6 +35,7 @@ import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.transport.LocalTransportAddress; import org.elasticsearch.index.shard.ShardId; import org.elasticsearch.test.ESTestCase; +import org.elasticsearch.threadpool.TestThreadPool; import org.elasticsearch.threadpool.ThreadPool; import org.elasticsearch.transport.TransportService; import org.junit.After; @@ -74,7 +75,7 @@ public class IndicesStoreTests extends ESTestCase { @BeforeClass public static void beforeClass() { - threadPool = new ThreadPool("ShardReplicationTests"); + threadPool = new TestThreadPool("ShardReplicationTests"); } @AfterClass diff --git a/core/src/test/java/org/elasticsearch/indices/template/SimpleIndexTemplateIT.java b/core/src/test/java/org/elasticsearch/indices/template/SimpleIndexTemplateIT.java index 416dbd19761..298cb9cd9e3 100644 --- a/core/src/test/java/org/elasticsearch/indices/template/SimpleIndexTemplateIT.java +++ b/core/src/test/java/org/elasticsearch/indices/template/SimpleIndexTemplateIT.java @@ -45,6 +45,7 @@ import java.util.HashSet; import java.util.List; import java.util.Set; +import static org.elasticsearch.action.support.WriteRequest.RefreshPolicy.IMMEDIATE; import static org.elasticsearch.index.query.QueryBuilders.termQuery; import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertAcked; import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertHitCount; @@ -110,7 +111,7 @@ public class SimpleIndexTemplateIT extends ESIntegTestCase { // index something into test_index, will match on both templates - client().prepareIndex("test_index", "type1", "1").setSource("field1", "value1", "field2", "value 2").setRefresh(true).execute().actionGet(); + client().prepareIndex("test_index", "type1", "1").setSource("field1", "value1", "field2", "value 2").setRefreshPolicy(IMMEDIATE).get(); ensureGreen(); SearchResponse searchResponse = client().prepareSearch("test_index") @@ -123,7 +124,7 @@ public class SimpleIndexTemplateIT extends ESIntegTestCase { // field2 is not stored. assertThat(searchResponse.getHits().getAt(0).field("field2"), nullValue()); - client().prepareIndex("text_index", "type1", "1").setSource("field1", "value1", "field2", "value 2").setRefresh(true).execute().actionGet(); + client().prepareIndex("text_index", "type1", "1").setSource("field1", "value1", "field2", "value 2").setRefreshPolicy(IMMEDIATE).get(); ensureGreen(); // now only match on one template (template_1) diff --git a/core/src/test/java/org/elasticsearch/ingest/IngestClientIT.java b/core/src/test/java/org/elasticsearch/ingest/IngestClientIT.java index 111133a0521..72856c143e0 100644 --- a/core/src/test/java/org/elasticsearch/ingest/IngestClientIT.java +++ b/core/src/test/java/org/elasticsearch/ingest/IngestClientIT.java @@ -38,7 +38,6 @@ import org.elasticsearch.action.ingest.WritePipelineResponse; import org.elasticsearch.common.bytes.BytesReference; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.ingest.core.IngestDocument; -import org.elasticsearch.node.NodeModule; import org.elasticsearch.plugins.Plugin; import org.elasticsearch.test.ESIntegTestCase; @@ -69,7 +68,7 @@ public class IngestClientIT extends ESIntegTestCase { @Override protected Collection> nodePlugins() { - return pluginList(IngestPlugin.class); + return pluginList(IngestTestPlugin.class); } public void testSimulate() throws Exception { @@ -234,28 +233,4 @@ public class IngestClientIT extends ESIntegTestCase { protected Collection> getMockPlugins() { return Collections.singletonList(TestSeedPlugin.class); } - - public static class IngestPlugin extends Plugin { - - @Override - public String name() { - return "ingest"; - } - - @Override - public String description() { - return "ingest mock"; - } - - public void onModule(NodeModule nodeModule) { - nodeModule.registerProcessor("test", (templateService, registry) -> config -> - new TestProcessor("id", "test", ingestDocument -> { - ingestDocument.setFieldValue("processed", true); - if (ingestDocument.getFieldValue("fail", Boolean.class)) { - throw new IllegalArgumentException("test processor failed"); - } - }) - ); - } - } } diff --git a/core/src/test/java/org/elasticsearch/ingest/IngestProcessorNotInstalledOnAllNodesIT.java b/core/src/test/java/org/elasticsearch/ingest/IngestProcessorNotInstalledOnAllNodesIT.java index a415b0992a7..c6a096f238e 100644 --- a/core/src/test/java/org/elasticsearch/ingest/IngestProcessorNotInstalledOnAllNodesIT.java +++ b/core/src/test/java/org/elasticsearch/ingest/IngestProcessorNotInstalledOnAllNodesIT.java @@ -57,7 +57,7 @@ public class IngestProcessorNotInstalledOnAllNodesIT extends ESIntegTestCase { @Override protected Collection> nodePlugins() { - return installPlugin ? pluginList(IngestClientIT.IngestPlugin.class) : Collections.emptyList(); + return installPlugin ? pluginList(IngestTestPlugin.class) : Collections.emptyList(); } @Override diff --git a/core/src/test/java/org/elasticsearch/ingest/PipelineStoreTests.java b/core/src/test/java/org/elasticsearch/ingest/PipelineStoreTests.java index c60bb3eecde..26dc1957757 100644 --- a/core/src/test/java/org/elasticsearch/ingest/PipelineStoreTests.java +++ b/core/src/test/java/org/elasticsearch/ingest/PipelineStoreTests.java @@ -31,11 +31,11 @@ import org.elasticsearch.cluster.node.DiscoveryNode; import org.elasticsearch.common.bytes.BytesArray; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.transport.LocalTransportAddress; +import org.elasticsearch.ingest.core.IngestDocument; import org.elasticsearch.ingest.core.IngestInfo; import org.elasticsearch.ingest.core.Pipeline; +import org.elasticsearch.ingest.core.Processor; import org.elasticsearch.ingest.core.ProcessorInfo; -import org.elasticsearch.ingest.processor.RemoveProcessor; -import org.elasticsearch.ingest.processor.SetProcessor; import org.elasticsearch.test.ESTestCase; import org.junit.Before; @@ -60,9 +60,45 @@ public class PipelineStoreTests extends ESTestCase { public void init() throws Exception { store = new PipelineStore(Settings.EMPTY); ProcessorsRegistry.Builder registryBuilder = new ProcessorsRegistry.Builder(); - registryBuilder.registerProcessor("set", (templateService, registry) -> new SetProcessor.Factory(TestTemplateService.instance())); - registryBuilder.registerProcessor("remove", (templateService, registry) -> - new RemoveProcessor.Factory(TestTemplateService.instance())); + registryBuilder.registerProcessor("set", (templateService, registry) -> config -> { + String field = (String) config.remove("field"); + String value = (String) config.remove("value"); + return new Processor() { + @Override + public void execute(IngestDocument ingestDocument) throws Exception { + ingestDocument.setFieldValue(field, value); + } + + @Override + public String getType() { + return "set"; + } + + @Override + public String getTag() { + return null; + } + }; + }); + registryBuilder.registerProcessor("remove", (templateService, registry) -> config -> { + String field = (String) config.remove("field"); + return new Processor() { + @Override + public void execute(IngestDocument ingestDocument) throws Exception { + ingestDocument.removeField(field); + } + + @Override + public String getType() { + return "remove"; + } + + @Override + public String getTag() { + return null; + } + }; + }); store.buildProcessorFactoryRegistry(registryBuilder, null); } diff --git a/core/src/test/java/org/elasticsearch/ingest/core/PipelineFactoryTests.java b/core/src/test/java/org/elasticsearch/ingest/core/PipelineFactoryTests.java index 449f1836f2b..b6840b7a554 100644 --- a/core/src/test/java/org/elasticsearch/ingest/core/PipelineFactoryTests.java +++ b/core/src/test/java/org/elasticsearch/ingest/core/PipelineFactoryTests.java @@ -23,8 +23,6 @@ import org.elasticsearch.ElasticsearchParseException; import org.elasticsearch.ingest.ProcessorsRegistry; import org.elasticsearch.ingest.TestProcessor; import org.elasticsearch.ingest.TestTemplateService; -import org.elasticsearch.ingest.processor.FailProcessor; -import org.elasticsearch.ingest.processor.SetProcessor; import org.elasticsearch.test.ESTestCase; import java.util.Arrays; @@ -32,7 +30,6 @@ import java.util.Collections; import java.util.HashMap; import java.util.List; import java.util.Map; -import java.util.prefs.PreferencesFactory; import static org.hamcrest.CoreMatchers.equalTo; import static org.hamcrest.Matchers.is; diff --git a/core/src/test/java/org/elasticsearch/mget/SimpleMgetIT.java b/core/src/test/java/org/elasticsearch/mget/SimpleMgetIT.java index 21be585862d..e7101869692 100644 --- a/core/src/test/java/org/elasticsearch/mget/SimpleMgetIT.java +++ b/core/src/test/java/org/elasticsearch/mget/SimpleMgetIT.java @@ -33,6 +33,7 @@ import org.elasticsearch.test.ESIntegTestCase; import java.io.IOException; import java.util.Map; +import static org.elasticsearch.action.support.WriteRequest.RefreshPolicy.IMMEDIATE; import static org.elasticsearch.common.xcontent.XContentFactory.jsonBuilder; import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertAcked; import static org.hamcrest.Matchers.equalTo; @@ -45,7 +46,8 @@ public class SimpleMgetIT extends ESIntegTestCase { createIndex("test"); ensureYellow(); - client().prepareIndex("test", "test", "1").setSource(jsonBuilder().startObject().field("foo", "bar").endObject()).setRefresh(true).execute().actionGet(); + client().prepareIndex("test", "test", "1").setSource(jsonBuilder().startObject().field("foo", "bar").endObject()) + .setRefreshPolicy(IMMEDIATE).get(); MultiGetResponse mgetResponse = client().prepareMultiGet() .add(new MultiGetRequest.Item("test", "test", "1")) @@ -86,9 +88,9 @@ public class SimpleMgetIT extends ESIntegTestCase { .endObject())); ensureYellow(); - client().prepareIndex("test", "test", "1").setParent("4").setRefresh(true) + client().prepareIndex("test", "test", "1").setParent("4").setRefreshPolicy(IMMEDIATE) .setSource(jsonBuilder().startObject().field("foo", "bar").endObject()) - .execute().actionGet(); + .get(); MultiGetResponse mgetResponse = client().prepareMultiGet() .add(new MultiGetRequest.Item(indexOrAlias(), "test", "1").parent("4")) @@ -154,9 +156,9 @@ public class SimpleMgetIT extends ESIntegTestCase { final String id = routingKeyForShard("test", "test", 0); final String routingOtherShard = routingKeyForShard("test", "test", 1); - client().prepareIndex("test", "test", id).setRefresh(true).setRouting(routingOtherShard) + client().prepareIndex("test", "test", id).setRefreshPolicy(IMMEDIATE).setRouting(routingOtherShard) .setSource(jsonBuilder().startObject().field("foo", "bar").endObject()) - .execute().actionGet(); + .get(); MultiGetResponse mgetResponse = client().prepareMultiGet() .add(new MultiGetRequest.Item(indexOrAlias(), "test", id).routing(routingOtherShard)) diff --git a/core/src/test/java/org/elasticsearch/monitor/jvm/JvmGcMonitorServiceSettingsTests.java b/core/src/test/java/org/elasticsearch/monitor/jvm/JvmGcMonitorServiceSettingsTests.java index 97a2b065627..a1f4d381911 100644 --- a/core/src/test/java/org/elasticsearch/monitor/jvm/JvmGcMonitorServiceSettingsTests.java +++ b/core/src/test/java/org/elasticsearch/monitor/jvm/JvmGcMonitorServiceSettingsTests.java @@ -22,6 +22,7 @@ package org.elasticsearch.monitor.jvm; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.unit.TimeValue; import org.elasticsearch.test.ESTestCase; +import org.elasticsearch.threadpool.TestThreadPool; import org.elasticsearch.threadpool.ThreadPool; import java.util.AbstractMap; @@ -128,7 +129,7 @@ public class JvmGcMonitorServiceSettingsTests extends ESTestCase { assert constructionShouldFail == (asserts == null); ThreadPool threadPool = null; try { - threadPool = new ThreadPool(JvmGcMonitorServiceSettingsTests.class.getCanonicalName()) { + threadPool = new TestThreadPool(JvmGcMonitorServiceSettingsTests.class.getCanonicalName()) { @Override public ScheduledFuture scheduleWithFixedDelay(Runnable command, TimeValue interval) { return scheduler.apply(command, interval); diff --git a/core/src/test/java/org/elasticsearch/routing/AliasRoutingIT.java b/core/src/test/java/org/elasticsearch/routing/AliasRoutingIT.java index 6a4de4b9ff2..6a4d965706a 100644 --- a/core/src/test/java/org/elasticsearch/routing/AliasRoutingIT.java +++ b/core/src/test/java/org/elasticsearch/routing/AliasRoutingIT.java @@ -21,6 +21,7 @@ package org.elasticsearch.routing; import org.elasticsearch.action.search.SearchResponse; import org.elasticsearch.action.search.SearchType; +import org.elasticsearch.action.support.WriteRequest.RefreshPolicy; import org.elasticsearch.common.xcontent.XContentFactory; import org.elasticsearch.index.query.QueryBuilders; import org.elasticsearch.test.ESIntegTestCase; @@ -45,7 +46,7 @@ public class AliasRoutingIT extends ESIntegTestCase { assertAcked(admin().indices().prepareAliases().addAliasAction(newAddAliasAction("test", "alias0").routing("0"))); logger.info("--> indexing with id [1], and routing [0] using alias"); - client().prepareIndex("alias0", "type1", "1").setSource("field", "value1").setRefresh(true).execute().actionGet(); + client().prepareIndex("alias0", "type1", "1").setSource("field", "value1").setRefreshPolicy(RefreshPolicy.IMMEDIATE).get(); logger.info("--> verifying get with no routing, should not find anything"); for (int i = 0; i < 5; i++) { assertThat(client().prepareGet("test", "type1", "1").execute().actionGet().isExists(), equalTo(false)); @@ -72,7 +73,7 @@ public class AliasRoutingIT extends ESIntegTestCase { logger.info("--> deleting with no routing, should not delete anything"); - client().prepareDelete("test", "type1", "1").setRefresh(true).execute().actionGet(); + client().prepareDelete("test", "type1", "1").setRefreshPolicy(RefreshPolicy.IMMEDIATE).get(); for (int i = 0; i < 5; i++) { assertThat(client().prepareGet("test", "type1", "1").execute().actionGet().isExists(), equalTo(false)); assertThat(client().prepareGet("test", "type1", "1").setRouting("0").execute().actionGet().isExists(), equalTo(true)); @@ -80,7 +81,7 @@ public class AliasRoutingIT extends ESIntegTestCase { } logger.info("--> deleting with routing alias, should delete"); - client().prepareDelete("alias0", "type1", "1").setRefresh(true).execute().actionGet(); + client().prepareDelete("alias0", "type1", "1").setRefreshPolicy(RefreshPolicy.IMMEDIATE).get(); for (int i = 0; i < 5; i++) { assertThat(client().prepareGet("test", "type1", "1").execute().actionGet().isExists(), equalTo(false)); assertThat(client().prepareGet("test", "type1", "1").setRouting("0").execute().actionGet().isExists(), equalTo(false)); @@ -88,7 +89,7 @@ public class AliasRoutingIT extends ESIntegTestCase { } logger.info("--> indexing with id [1], and routing [0] using alias"); - client().prepareIndex("alias0", "type1", "1").setSource("field", "value1").setRefresh(true).execute().actionGet(); + client().prepareIndex("alias0", "type1", "1").setSource("field", "value1").setRefreshPolicy(RefreshPolicy.IMMEDIATE).get(); logger.info("--> verifying get with no routing, should not find anything"); for (int i = 0; i < 5; i++) { assertThat(client().prepareGet("test", "type1", "1").execute().actionGet().isExists(), equalTo(false)); @@ -110,7 +111,7 @@ public class AliasRoutingIT extends ESIntegTestCase { .addAliasAction(newAddAliasAction("test", "alias01").searchRouting("0,1"))); logger.info("--> indexing with id [1], and routing [0] using alias"); - client().prepareIndex("alias0", "type1", "1").setSource("field", "value1").setRefresh(true).execute().actionGet(); + client().prepareIndex("alias0", "type1", "1").setSource("field", "value1").setRefreshPolicy(RefreshPolicy.IMMEDIATE).get(); logger.info("--> verifying get with no routing, should not find anything"); for (int i = 0; i < 5; i++) { assertThat(client().prepareGet("test", "type1", "1").execute().actionGet().isExists(), equalTo(false)); @@ -142,7 +143,7 @@ public class AliasRoutingIT extends ESIntegTestCase { } logger.info("--> indexing with id [2], and routing [1] using alias"); - client().prepareIndex("alias1", "type1", "2").setSource("field", "value1").setRefresh(true).execute().actionGet(); + client().prepareIndex("alias1", "type1", "2").setSource("field", "value1").setRefreshPolicy(RefreshPolicy.IMMEDIATE).get(); logger.info("--> search with no routing, should fine two"); for (int i = 0; i < 5; i++) { @@ -207,7 +208,7 @@ public class AliasRoutingIT extends ESIntegTestCase { .addAliasAction(newAddAliasAction("test-b", "alias-ab").searchRouting("1"))); ensureGreen(); // wait for events again to make sure we got the aliases on all nodes logger.info("--> indexing with id [1], and routing [0] using alias to test-a"); - client().prepareIndex("alias-a0", "type1", "1").setSource("field", "value1").setRefresh(true).execute().actionGet(); + client().prepareIndex("alias-a0", "type1", "1").setSource("field", "value1").setRefreshPolicy(RefreshPolicy.IMMEDIATE).get(); logger.info("--> verifying get with no routing, should not find anything"); for (int i = 0; i < 5; i++) { assertThat(client().prepareGet("test-a", "type1", "1").execute().actionGet().isExists(), equalTo(false)); @@ -218,7 +219,7 @@ public class AliasRoutingIT extends ESIntegTestCase { } logger.info("--> indexing with id [0], and routing [1] using alias to test-b"); - client().prepareIndex("alias-b1", "type1", "1").setSource("field", "value1").setRefresh(true).execute().actionGet(); + client().prepareIndex("alias-b1", "type1", "1").setSource("field", "value1").setRefreshPolicy(RefreshPolicy.IMMEDIATE).get(); logger.info("--> verifying get with no routing, should not find anything"); for (int i = 0; i < 5; i++) { assertThat(client().prepareGet("test-a", "type1", "1").execute().actionGet().isExists(), equalTo(false)); @@ -261,9 +262,9 @@ public class AliasRoutingIT extends ESIntegTestCase { .addAliasAction(newAddAliasAction("index", "index_1").routing("1"))); logger.info("--> indexing on index_1 which is an alias for index with routing [1]"); - client().prepareIndex("index_1", "type1", "1").setSource("field", "value1").setRefresh(true).execute().actionGet(); + client().prepareIndex("index_1", "type1", "1").setSource("field", "value1").setRefreshPolicy(RefreshPolicy.IMMEDIATE).get(); logger.info("--> indexing on index_2 which is a concrete index"); - client().prepareIndex("index_2", "type2", "2").setSource("field", "value2").setRefresh(true).execute().actionGet(); + client().prepareIndex("index_2", "type2", "2").setSource("field", "value2").setRefreshPolicy(RefreshPolicy.IMMEDIATE).get(); logger.info("--> search all on index_* should find two"); @@ -286,9 +287,9 @@ public class AliasRoutingIT extends ESIntegTestCase { .addAliasAction(newAddAliasAction("index", "index_1").routing("1"))); logger.info("--> indexing on index_1 which is an alias for index with routing [1]"); - client().prepareIndex("index_1", "type1", "1").setSource("field", "value1").setRefresh(true).execute().actionGet(); + client().prepareIndex("index_1", "type1", "1").setSource("field", "value1").setRefreshPolicy(RefreshPolicy.IMMEDIATE).get(); logger.info("--> indexing on index_2 which is a concrete index"); - client().prepareIndex("index_2", "type2", "2").setSource("field", "value2").setRefresh(true).execute().actionGet(); + client().prepareIndex("index_2", "type2", "2").setSource("field", "value2").setRefreshPolicy(RefreshPolicy.IMMEDIATE).get(); SearchResponse searchResponse = client().prepareSearch("index_*").setSearchType(SearchType.QUERY_THEN_FETCH).setSize(1).setQuery(QueryBuilders.matchAllQuery()).execute().actionGet(); @@ -307,7 +308,7 @@ public class AliasRoutingIT extends ESIntegTestCase { .addAliasAction(newAddAliasAction("test", "alias").routing("3"))); logger.info("--> indexing with id [0], and routing [3]"); - client().prepareIndex("alias", "type1", "0").setSource("field", "value1").setRefresh(true).execute().actionGet(); + client().prepareIndex("alias", "type1", "0").setSource("field", "value1").setRefreshPolicy(RefreshPolicy.IMMEDIATE).get(); logger.info("--> verifying get with no routing, should not find anything"); logger.info("--> verifying get and search with routing, should find"); @@ -332,7 +333,7 @@ public class AliasRoutingIT extends ESIntegTestCase { .addAliasAction(newAddAliasAction("test", "alias").searchRouting("3,4").indexRouting("4"))); logger.info("--> indexing with id [1], and routing [4]"); - client().prepareIndex("alias", "type1", "1").setSource("field", "value2").setRefresh(true).execute().actionGet(); + client().prepareIndex("alias", "type1", "1").setSource("field", "value2").setRefreshPolicy(RefreshPolicy.IMMEDIATE).get(); logger.info("--> verifying get with no routing, should not find anything"); logger.info("--> verifying get and search with routing, should find"); diff --git a/core/src/test/java/org/elasticsearch/routing/SimpleRoutingIT.java b/core/src/test/java/org/elasticsearch/routing/SimpleRoutingIT.java index 03e6cbf9ef1..d8cf1e7b5ec 100644 --- a/core/src/test/java/org/elasticsearch/routing/SimpleRoutingIT.java +++ b/core/src/test/java/org/elasticsearch/routing/SimpleRoutingIT.java @@ -28,6 +28,7 @@ import org.elasticsearch.action.explain.ExplainResponse; import org.elasticsearch.action.get.GetResponse; import org.elasticsearch.action.get.MultiGetRequest; import org.elasticsearch.action.get.MultiGetResponse; +import org.elasticsearch.action.support.WriteRequest.RefreshPolicy; import org.elasticsearch.action.termvectors.MultiTermVectorsResponse; import org.elasticsearch.action.termvectors.TermVectorsRequest; import org.elasticsearch.action.termvectors.TermVectorsResponse; @@ -56,7 +57,8 @@ public class SimpleRoutingIT extends ESIntegTestCase { ensureGreen(); logger.info("--> indexing with id [1], and routing [0]"); - client().prepareIndex("test", "type1", "1").setRouting("0").setSource("field", "value1").setRefresh(true).execute().actionGet(); + client().prepareIndex("test", "type1", "1").setRouting("0").setSource("field", "value1").setRefreshPolicy(RefreshPolicy.IMMEDIATE) + .get(); logger.info("--> verifying get with no routing, should not find anything"); for (int i = 0; i < 5; i++) { assertThat(client().prepareGet("test", "type1", "1").execute().actionGet().isExists(), equalTo(false)); @@ -67,21 +69,22 @@ public class SimpleRoutingIT extends ESIntegTestCase { } logger.info("--> deleting with no routing, should not delete anything"); - client().prepareDelete("test", "type1", "1").setRefresh(true).execute().actionGet(); + client().prepareDelete("test", "type1", "1").setRefreshPolicy(RefreshPolicy.IMMEDIATE).get(); for (int i = 0; i < 5; i++) { assertThat(client().prepareGet("test", "type1", "1").execute().actionGet().isExists(), equalTo(false)); assertThat(client().prepareGet("test", "type1", "1").setRouting("0").execute().actionGet().isExists(), equalTo(true)); } logger.info("--> deleting with routing, should delete"); - client().prepareDelete("test", "type1", "1").setRouting("0").setRefresh(true).execute().actionGet(); + client().prepareDelete("test", "type1", "1").setRouting("0").setRefreshPolicy(RefreshPolicy.IMMEDIATE).get(); for (int i = 0; i < 5; i++) { assertThat(client().prepareGet("test", "type1", "1").execute().actionGet().isExists(), equalTo(false)); assertThat(client().prepareGet("test", "type1", "1").setRouting("0").execute().actionGet().isExists(), equalTo(false)); } logger.info("--> indexing with id [1], and routing [0]"); - client().prepareIndex("test", "type1", "1").setRouting("0").setSource("field", "value1").setRefresh(true).execute().actionGet(); + client().prepareIndex("test", "type1", "1").setRouting("0").setSource("field", "value1").setRefreshPolicy(RefreshPolicy.IMMEDIATE) + .get(); logger.info("--> verifying get with no routing, should not find anything"); for (int i = 0; i < 5; i++) { assertThat(client().prepareGet("test", "type1", "1").execute().actionGet().isExists(), equalTo(false)); @@ -97,7 +100,8 @@ public class SimpleRoutingIT extends ESIntegTestCase { ensureGreen(); logger.info("--> indexing with id [1], and routing [0]"); - client().prepareIndex("test", "type1", "1").setRouting("0").setSource("field", "value1").setRefresh(true).execute().actionGet(); + client().prepareIndex("test", "type1", "1").setRouting("0").setSource("field", "value1").setRefreshPolicy(RefreshPolicy.IMMEDIATE) + .get(); logger.info("--> verifying get with no routing, should not find anything"); for (int i = 0; i < 5; i++) { assertThat(client().prepareGet("test", "type1", "1").execute().actionGet().isExists(), equalTo(false)); @@ -125,7 +129,7 @@ public class SimpleRoutingIT extends ESIntegTestCase { } logger.info("--> indexing with id [2], and routing [1]"); - client().prepareIndex("test", "type1", "2").setRouting("1").setSource("field", "value1").setRefresh(true).execute().actionGet(); + client().prepareIndex("test", "type1", "2").setRouting("1").setSource("field", "value1").setRefreshPolicy(RefreshPolicy.IMMEDIATE).get(); logger.info("--> search with no routing, should fine two"); for (int i = 0; i < 5; i++) { @@ -165,12 +169,13 @@ public class SimpleRoutingIT extends ESIntegTestCase { ensureGreen(); logger.info("--> indexing with id [1], and routing [0]"); - client().prepareIndex(indexOrAlias(), "type1", "1").setRouting("0").setSource("field", "value1").setRefresh(true).execute().actionGet(); + client().prepareIndex(indexOrAlias(), "type1", "1").setRouting("0").setSource("field", "value1") + .setRefreshPolicy(RefreshPolicy.IMMEDIATE).get(); logger.info("--> verifying get with no routing, should fail"); logger.info("--> indexing with id [1], with no routing, should fail"); try { - client().prepareIndex(indexOrAlias(), "type1", "1").setSource("field", "value1").setRefresh(true).execute().actionGet(); + client().prepareIndex(indexOrAlias(), "type1", "1").setSource("field", "value1").get(); fail("index with missing routing when routing is required should fail"); } catch (ElasticsearchException e) { assertThat(e.unwrapCause(), instanceOf(RoutingMissingException.class)); @@ -183,7 +188,7 @@ public class SimpleRoutingIT extends ESIntegTestCase { logger.info("--> deleting with no routing, should fail"); try { - client().prepareDelete(indexOrAlias(), "type1", "1").setRefresh(true).execute().actionGet(); + client().prepareDelete(indexOrAlias(), "type1", "1").get(); fail("delete with missing routing when routing is required should fail"); } catch (ElasticsearchException e) { assertThat(e.unwrapCause(), instanceOf(RoutingMissingException.class)); @@ -223,7 +228,7 @@ public class SimpleRoutingIT extends ESIntegTestCase { assertThat(getResponse.getSourceAsMap().get("field"), equalTo("value2")); } - client().prepareDelete(indexOrAlias(), "type1", "1").setRouting("0").setRefresh(true).execute().actionGet(); + client().prepareDelete(indexOrAlias(), "type1", "1").setRouting("0").setRefreshPolicy(RefreshPolicy.IMMEDIATE).get(); for (int i = 0; i < 5; i++) { try { @@ -320,7 +325,8 @@ public class SimpleRoutingIT extends ESIntegTestCase { logger.info("--> indexing with id [1], and routing [0]"); client().prepareIndex(indexOrAlias(), "type1", "1").setRouting("0").setSource("field", "value1").get(); logger.info("--> indexing with id [2], and routing [0]"); - client().prepareIndex(indexOrAlias(), "type1", "2").setRouting("0").setSource("field", "value2").setRefresh(true).get(); + client().prepareIndex(indexOrAlias(), "type1", "2").setRouting("0").setSource("field", "value2") + .setRefreshPolicy(RefreshPolicy.IMMEDIATE).get(); logger.info("--> verifying get with id [1] with routing [0], should succeed"); assertThat(client().prepareGet(indexOrAlias(), "type1", "1").setRouting("0").execute().actionGet().isExists(), equalTo(true)); diff --git a/core/src/test/java/org/elasticsearch/script/NativeScriptTests.java b/core/src/test/java/org/elasticsearch/script/NativeScriptTests.java index 5e1dc740f9e..e0eed8387a6 100644 --- a/core/src/test/java/org/elasticsearch/script/NativeScriptTests.java +++ b/core/src/test/java/org/elasticsearch/script/NativeScriptTests.java @@ -55,9 +55,10 @@ public class NativeScriptTests extends ESTestCase { ScriptModule scriptModule = new ScriptModule(); scriptModule.prepareSettings(settingsModule); scriptModule.registerScript("my", MyNativeScriptFactory.class); + final ThreadPool threadPool = new ThreadPool(settings); Injector injector = new ModulesBuilder().add( new EnvironmentModule(new Environment(settings)), - new ThreadPoolModule(new ThreadPool(settings)), + new ThreadPoolModule(threadPool), new SettingsModule(settings), scriptModule).createInjector(); diff --git a/core/src/test/java/org/elasticsearch/search/SearchServiceTests.java b/core/src/test/java/org/elasticsearch/search/SearchServiceTests.java index a222ff52722..b465d25043f 100644 --- a/core/src/test/java/org/elasticsearch/search/SearchServiceTests.java +++ b/core/src/test/java/org/elasticsearch/search/SearchServiceTests.java @@ -19,11 +19,28 @@ package org.elasticsearch.search; +import org.apache.lucene.search.Query; +import org.elasticsearch.action.search.SearchPhaseExecutionException; import org.elasticsearch.action.search.SearchResponse; +import org.elasticsearch.common.ParseField; +import org.elasticsearch.common.io.stream.StreamInput; +import org.elasticsearch.common.io.stream.StreamOutput; +import org.elasticsearch.common.xcontent.XContentBuilder; +import org.elasticsearch.index.IndexService; +import org.elasticsearch.index.query.AbstractQueryBuilder; +import org.elasticsearch.index.query.QueryBuilder; +import org.elasticsearch.index.query.QueryRewriteContext; +import org.elasticsearch.index.query.QueryShardContext; +import org.elasticsearch.index.shard.IndexShard; +import org.elasticsearch.indices.IndicesService; +import org.elasticsearch.plugins.Plugin; import org.elasticsearch.test.ESSingleNodeTestCase; +import java.io.IOException; +import java.util.Collection; import java.util.concurrent.ExecutionException; +import static org.elasticsearch.action.support.WriteRequest.RefreshPolicy.IMMEDIATE; import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertAcked; import static org.hamcrest.Matchers.is; import static org.hamcrest.Matchers.notNullValue; @@ -35,9 +52,14 @@ public class SearchServiceTests extends ESSingleNodeTestCase { return true; } + @Override + protected Collection> getPlugins() { + return pluginList(FailOnRewriteQueryPlugin.class); + } + public void testClearOnClose() throws ExecutionException, InterruptedException { createIndex("index"); - client().prepareIndex("index", "type", "1").setSource("field", "value").setRefresh(true).get(); + client().prepareIndex("index", "type", "1").setSource("field", "value").setRefreshPolicy(IMMEDIATE).get(); SearchResponse searchResponse = client().prepareSearch("index").setSize(1).setScroll("1m").get(); assertThat(searchResponse.getScrollId(), is(notNullValue())); SearchService service = getInstanceFromNode(SearchService.class); @@ -49,7 +71,7 @@ public class SearchServiceTests extends ESSingleNodeTestCase { public void testClearOnStop() throws ExecutionException, InterruptedException { createIndex("index"); - client().prepareIndex("index", "type", "1").setSource("field", "value").setRefresh(true).get(); + client().prepareIndex("index", "type", "1").setSource("field", "value").setRefreshPolicy(IMMEDIATE).get(); SearchResponse searchResponse = client().prepareSearch("index").setSize(1).setScroll("1m").get(); assertThat(searchResponse.getScrollId(), is(notNullValue())); SearchService service = getInstanceFromNode(SearchService.class); @@ -61,7 +83,7 @@ public class SearchServiceTests extends ESSingleNodeTestCase { public void testClearIndexDelete() throws ExecutionException, InterruptedException { createIndex("index"); - client().prepareIndex("index", "type", "1").setSource("field", "value").setRefresh(true).get(); + client().prepareIndex("index", "type", "1").setSource("field", "value").setRefreshPolicy(IMMEDIATE).get(); SearchResponse searchResponse = client().prepareSearch("index").setSize(1).setScroll("1m").get(); assertThat(searchResponse.getScrollId(), is(notNullValue())); SearchService service = getInstanceFromNode(SearchService.class); @@ -70,4 +92,83 @@ public class SearchServiceTests extends ESSingleNodeTestCase { assertAcked(client().admin().indices().prepareDelete("index")); assertEquals(0, service.getActiveContexts()); } + + public void testCloseSearchContextOnRewriteException() { + createIndex("index"); + client().prepareIndex("index", "type", "1").setSource("field", "value").setRefreshPolicy(IMMEDIATE).get(); + + SearchService service = getInstanceFromNode(SearchService.class); + IndicesService indicesService = getInstanceFromNode(IndicesService.class); + IndexService indexService = indicesService.indexServiceSafe(resolveIndex("index")); + IndexShard indexShard = indexService.getShard(0); + + final int activeContexts = service.getActiveContexts(); + final int activeRefs = indexShard.store().refCount(); + expectThrows(SearchPhaseExecutionException.class, () -> + client().prepareSearch("index").setQuery(new FailOnRewriteQueryBuilder()).get()); + assertEquals(activeContexts, service.getActiveContexts()); + assertEquals(activeRefs, indexShard.store().refCount()); + } + + public static class FailOnRewriteQueryPlugin extends Plugin { + + @Override + public String name() { + return FailOnRewriteQueryPlugin.class.getSimpleName(); + } + + @Override + public String description() { + return "This plugin registers a query that always fails at rewrite phase"; + } + + public void onModule(SearchModule module) { + module.registerQuery(FailOnRewriteQueryBuilder::new, parseContext -> { + throw new UnsupportedOperationException("No query parser for this plugin"); + }, new ParseField("fail_on_rewrite_query")); + } + } + + public static class FailOnRewriteQueryBuilder extends AbstractQueryBuilder { + + public FailOnRewriteQueryBuilder(StreamInput in) throws IOException { + super(in); + } + + public FailOnRewriteQueryBuilder() { + } + + @Override + protected QueryBuilder doRewrite(QueryRewriteContext queryShardContext) throws IOException { + throw new IllegalStateException("Fail on rewrite phase"); + } + + @Override + protected void doWriteTo(StreamOutput out) throws IOException { + } + + @Override + protected void doXContent(XContentBuilder builder, Params params) throws IOException { + } + + @Override + protected Query doToQuery(QueryShardContext context) throws IOException { + return null; + } + + @Override + protected boolean doEquals(FailOnRewriteQueryBuilder other) { + return false; + } + + @Override + protected int doHashCode() { + return 0; + } + + @Override + public String getWriteableName() { + return null; + } + } } diff --git a/core/src/test/java/org/elasticsearch/search/SearchTimeoutIT.java b/core/src/test/java/org/elasticsearch/search/SearchTimeoutIT.java index ef922e84048..b2eb12159f8 100644 --- a/core/src/test/java/org/elasticsearch/search/SearchTimeoutIT.java +++ b/core/src/test/java/org/elasticsearch/search/SearchTimeoutIT.java @@ -36,6 +36,7 @@ import java.util.Collections; import java.util.Map; import java.util.concurrent.TimeUnit; +import static org.elasticsearch.action.support.WriteRequest.RefreshPolicy.IMMEDIATE; import static org.elasticsearch.index.query.QueryBuilders.scriptQuery; import static org.hamcrest.Matchers.equalTo; @@ -55,7 +56,7 @@ public class SearchTimeoutIT extends ESIntegTestCase { } public void testSimpleTimeout() throws Exception { - client().prepareIndex("test", "type", "1").setSource("field", "value").setRefresh(true).execute().actionGet(); + client().prepareIndex("test", "type", "1").setSource("field", "value").setRefreshPolicy(IMMEDIATE).get(); SearchResponse searchResponse = client().prepareSearch("test").setTimeout(new TimeValue(10, TimeUnit.MILLISECONDS)) .setQuery(scriptQuery(new Script(NativeTestScriptedTimeout.TEST_NATIVE_SCRIPT_TIMEOUT, ScriptType.INLINE, "native", null))) diff --git a/core/src/test/java/org/elasticsearch/search/SearchWithRejectionsIT.java b/core/src/test/java/org/elasticsearch/search/SearchWithRejectionsIT.java index 68b4d980928..2bb39ad10ea 100644 --- a/core/src/test/java/org/elasticsearch/search/SearchWithRejectionsIT.java +++ b/core/src/test/java/org/elasticsearch/search/SearchWithRejectionsIT.java @@ -36,8 +36,8 @@ public class SearchWithRejectionsIT extends ESIntegTestCase { @Override public Settings nodeSettings(int nodeOrdinal) { return Settings.builder().put(super.nodeSettings(nodeOrdinal)) - .put("threadpool.search.size", 1) - .put("threadpool.search.queue_size", 1) + .put("thread_pool.search.size", 1) + .put("thread_pool.search.queue_size", 1) .build(); } diff --git a/core/src/test/java/org/elasticsearch/search/aggregations/bucket/ReverseNestedIT.java b/core/src/test/java/org/elasticsearch/search/aggregations/bucket/ReverseNestedIT.java index 93e6536e47c..21b4a74fe41 100644 --- a/core/src/test/java/org/elasticsearch/search/aggregations/bucket/ReverseNestedIT.java +++ b/core/src/test/java/org/elasticsearch/search/aggregations/bucket/ReverseNestedIT.java @@ -35,6 +35,7 @@ import java.util.ArrayList; import java.util.Arrays; import java.util.List; +import static org.elasticsearch.action.support.WriteRequest.RefreshPolicy.IMMEDIATE; import static org.elasticsearch.cluster.metadata.IndexMetaData.SETTING_NUMBER_OF_REPLICAS; import static org.elasticsearch.cluster.metadata.IndexMetaData.SETTING_NUMBER_OF_SHARDS; import static org.elasticsearch.common.xcontent.XContentFactory.jsonBuilder; @@ -509,7 +510,7 @@ public class ReverseNestedIT extends ESIntegTestCase { .addMapping("product", mapping) ); - client().prepareIndex("idx3", "product", "1").setRefresh(true).setSource( + client().prepareIndex("idx3", "product", "1").setRefreshPolicy(IMMEDIATE).setSource( jsonBuilder().startObject() .startArray("sku") .startObject() diff --git a/core/src/test/java/org/elasticsearch/search/builder/SearchSourceBuilderTests.java b/core/src/test/java/org/elasticsearch/search/builder/SearchSourceBuilderTests.java index fc3c90b5165..7ef3fa30e96 100644 --- a/core/src/test/java/org/elasticsearch/search/builder/SearchSourceBuilderTests.java +++ b/core/src/test/java/org/elasticsearch/search/builder/SearchSourceBuilderTests.java @@ -71,6 +71,7 @@ import org.elasticsearch.search.highlight.HighlightBuilderTests; import org.elasticsearch.search.rescore.QueryRescoreBuilderTests; import org.elasticsearch.search.rescore.QueryRescorerBuilder; import org.elasticsearch.search.searchafter.SearchAfterBuilder; +import org.elasticsearch.search.slice.SliceBuilder; import org.elasticsearch.search.sort.FieldSortBuilder; import org.elasticsearch.search.sort.ScoreSortBuilder; import org.elasticsearch.search.sort.ScriptSortBuilder.ScriptSortType; @@ -426,6 +427,16 @@ public class SearchSourceBuilderTests extends ESTestCase { xContentBuilder.endObject(); builder.ext(xContentBuilder); } + if (randomBoolean()) { + String field = randomBoolean() ? null : randomAsciiOfLengthBetween(5, 20); + int max = randomInt(1000); + int id = randomInt(max-1); + if (field == null) { + builder.slice(new SliceBuilder(id, max)); + } else { + builder.slice(new SliceBuilder(field, id, max)); + } + } return builder; } diff --git a/core/src/test/java/org/elasticsearch/search/child/ChildQuerySearchIT.java b/core/src/test/java/org/elasticsearch/search/child/ChildQuerySearchIT.java index 90f5c65c066..f8ca1e1aaf7 100644 --- a/core/src/test/java/org/elasticsearch/search/child/ChildQuerySearchIT.java +++ b/core/src/test/java/org/elasticsearch/search/child/ChildQuerySearchIT.java @@ -26,6 +26,7 @@ import org.elasticsearch.action.index.IndexRequestBuilder; import org.elasticsearch.action.search.SearchPhaseExecutionException; import org.elasticsearch.action.search.SearchResponse; import org.elasticsearch.action.search.SearchType; +import org.elasticsearch.action.support.WriteRequest.RefreshPolicy; import org.elasticsearch.common.lucene.search.function.CombineFunction; import org.elasticsearch.common.lucene.search.function.FiltersFunctionScoreQuery; import org.elasticsearch.common.settings.Settings; @@ -754,10 +755,11 @@ public class ChildQuerySearchIT extends ESIntegTestCase { assertNoFailures(response); assertThat(response.getHits().totalHits(), equalTo(0L)); - client().prepareIndex("test", "child1").setSource(jsonBuilder().startObject().field("text", "value").endObject()).setRefresh(true) - .get(); + client().prepareIndex("test", "child1").setSource(jsonBuilder().startObject().field("text", "value").endObject()) + .setRefreshPolicy(RefreshPolicy.IMMEDIATE).get(); - response = client().prepareSearch("test").setQuery(QueryBuilders.hasChildQuery("child", matchQuery("text", "value"), ScoreMode.None)).get(); + response = client().prepareSearch("test") + .setQuery(QueryBuilders.hasChildQuery("child", matchQuery("text", "value"), ScoreMode.None)).get(); assertNoFailures(response); assertThat(response.getHits().totalHits(), equalTo(0L)); diff --git a/core/src/test/java/org/elasticsearch/search/functionscore/DecayFunctionScoreIT.java b/core/src/test/java/org/elasticsearch/search/functionscore/DecayFunctionScoreIT.java index 83e62072683..e17f7bb3efd 100644 --- a/core/src/test/java/org/elasticsearch/search/functionscore/DecayFunctionScoreIT.java +++ b/core/src/test/java/org/elasticsearch/search/functionscore/DecayFunctionScoreIT.java @@ -51,6 +51,7 @@ import java.util.Collection; import java.util.List; import java.util.Locale; +import static org.elasticsearch.action.support.WriteRequest.RefreshPolicy.IMMEDIATE; import static org.elasticsearch.client.Requests.indexRequest; import static org.elasticsearch.client.Requests.searchRequest; import static org.elasticsearch.common.xcontent.XContentFactory.jsonBuilder; @@ -318,8 +319,13 @@ public class DecayFunctionScoreIT extends ESIntegTestCase { .setId("1") .setIndex("test") .setSource( - jsonBuilder().startObject().field("test", "value").startObject("loc").field("lat", 20).field("lon", 11).endObject() - .endObject()).setRefresh(true).get(); + jsonBuilder().startObject() + .field("test", "value") + .startObject("loc") + .field("lat", 20) + .field("lon", 11) + .endObject() + .endObject()).setRefreshPolicy(IMMEDIATE).get(); FunctionScoreQueryBuilder baseQuery = functionScoreQuery(constantScoreQuery(termQuery("test", "value")), ScoreFunctionBuilders.weightFactorFunction(randomIntBetween(1, 10))); GeoPoint point = new GeoPoint(20, 11); @@ -354,8 +360,8 @@ public class DecayFunctionScoreIT extends ESIntegTestCase { .endObject().startObject("num").field("type", "double").endObject().endObject().endObject().endObject())); ensureYellow(); - client().prepareIndex().setType("type1").setId("1").setIndex("test") - .setSource(jsonBuilder().startObject().field("test", "value value").field("num", 1.0).endObject()).setRefresh(true).get(); + client().prepareIndex().setType("type1").setId("1").setIndex("test").setRefreshPolicy(IMMEDIATE) + .setSource(jsonBuilder().startObject().field("test", "value value").field("num", 1.0).endObject()).get(); FunctionScoreQueryBuilder baseQuery = functionScoreQuery(constantScoreQuery(termQuery("test", "value")), ScoreFunctionBuilders.weightFactorFunction(2)); // decay score should return 0.5 for this function and baseQuery should return 2.0f as it's score diff --git a/core/src/test/java/org/elasticsearch/search/geo/GeoBoundingBoxIT.java b/core/src/test/java/org/elasticsearch/search/geo/GeoBoundingBoxIT.java index 46704410bd5..7e01f575822 100644 --- a/core/src/test/java/org/elasticsearch/search/geo/GeoBoundingBoxIT.java +++ b/core/src/test/java/org/elasticsearch/search/geo/GeoBoundingBoxIT.java @@ -34,6 +34,7 @@ import org.elasticsearch.test.VersionUtils; import java.util.Collection; +import static org.elasticsearch.action.support.WriteRequest.RefreshPolicy.IMMEDIATE; import static org.elasticsearch.common.xcontent.XContentFactory.jsonBuilder; import static org.elasticsearch.index.query.QueryBuilders.boolQuery; import static org.elasticsearch.index.query.QueryBuilders.geoBoundingBoxQuery; @@ -249,16 +250,16 @@ public class GeoBoundingBoxIT extends ESIntegTestCase { .field("title", "Place in Stockholm") .startObject("location").field("lat", 59.328355000000002).field("lon", 18.036842).endObject() .endObject()) - .setRefresh(true) - .execute().actionGet(); + .setRefreshPolicy(IMMEDIATE) + .get(); client().prepareIndex("test", "type1", "2").setSource(jsonBuilder().startObject() .field("userid", 534) .field("title", "Place in Montreal") .startObject("location").field("lat", 45.509526999999999).field("lon", -73.570986000000005).endObject() .endObject()) - .setRefresh(true) - .execute().actionGet(); + .setRefreshPolicy(IMMEDIATE) + .get(); SearchResponse searchResponse = client().prepareSearch() .setQuery( @@ -304,16 +305,16 @@ public class GeoBoundingBoxIT extends ESIntegTestCase { .field("title", "Place in Stockholm") .startObject("location").field("lat", 59.328355000000002).field("lon", 18.036842).endObject() .endObject()) - .setRefresh(true) - .execute().actionGet(); + .setRefreshPolicy(IMMEDIATE) + .get(); client().prepareIndex("test", "type1", "2").setSource(jsonBuilder().startObject() .field("userid", 534) .field("title", "Place in Montreal") .startObject("location").field("lat", 45.509526999999999).field("lon", -73.570986000000005).endObject() .endObject()) - .setRefresh(true) - .execute().actionGet(); + .setRefreshPolicy(IMMEDIATE) + .get(); SearchResponse searchResponse = client().prepareSearch() .setQuery( diff --git a/core/src/test/java/org/elasticsearch/search/geo/GeoShapeQueryTests.java b/core/src/test/java/org/elasticsearch/search/geo/GeoShapeQueryTests.java index 0fe90f133b5..7da18342edf 100644 --- a/core/src/test/java/org/elasticsearch/search/geo/GeoShapeQueryTests.java +++ b/core/src/test/java/org/elasticsearch/search/geo/GeoShapeQueryTests.java @@ -42,6 +42,7 @@ import org.elasticsearch.test.geo.RandomShapeGenerator; import java.io.IOException; import java.util.Locale; +import static org.elasticsearch.action.support.WriteRequest.RefreshPolicy.IMMEDIATE; import static org.elasticsearch.common.xcontent.XContentFactory.jsonBuilder; import static org.elasticsearch.index.query.QueryBuilders.geoIntersectionQuery; import static org.elasticsearch.index.query.QueryBuilders.geoShapeQuery; @@ -65,8 +66,7 @@ public class GeoShapeQueryTests extends ESSingleNodeTestCase { client().admin().indices().prepareCreate("test").addMapping("type1", mapping).execute().actionGet(); ensureGreen(); - client().prepareIndex("test", "type1", "aNullshape").setSource("{\"location\": null}").setRefresh(true) - .execute().actionGet(); + client().prepareIndex("test", "type1", "aNullshape").setSource("{\"location\": null}").setRefreshPolicy(IMMEDIATE).get(); GetResponse result = client().prepareGet("test", "type1", "aNullshape").execute().actionGet(); assertThat(result.getField("location"), nullValue()); } @@ -87,7 +87,7 @@ public class GeoShapeQueryTests extends ESSingleNodeTestCase { .field("type", "point") .startArray("coordinates").value(-30).value(-30).endArray() .endObject() - .endObject()).setRefresh(true).execute().actionGet(); + .endObject()).setRefreshPolicy(IMMEDIATE).get(); client().prepareIndex("test", "type1", "2").setSource(jsonBuilder().startObject() .field("name", "Document 2") @@ -95,7 +95,7 @@ public class GeoShapeQueryTests extends ESSingleNodeTestCase { .field("type", "point") .startArray("coordinates").value(-45).value(-50).endArray() .endObject() - .endObject()).setRefresh(true).execute().actionGet(); + .endObject()).setRefreshPolicy(IMMEDIATE).get(); ShapeBuilder shape = ShapeBuilders.newEnvelope(new Coordinate(-45, 45), new Coordinate(45, -45)); @@ -139,7 +139,7 @@ public class GeoShapeQueryTests extends ESSingleNodeTestCase { .startArray().value(-122.83).value(48.57).endArray() // close the polygon .endArray().endArray() .endObject() - .endObject()).setRefresh(true).execute().actionGet(); + .endObject()).setRefreshPolicy(IMMEDIATE).get(); ShapeBuilder query = ShapeBuilders.newEnvelope(new Coordinate(-122.88, 48.62), new Coordinate(-122.82, 48.54)); @@ -169,14 +169,14 @@ public class GeoShapeQueryTests extends ESSingleNodeTestCase { ShapeBuilder shape = ShapeBuilders.newEnvelope(new Coordinate(-45, 45), new Coordinate(45, -45)); client().prepareIndex("shapes", "shape_type", "Big_Rectangle").setSource(jsonBuilder().startObject() - .field("shape", shape).endObject()).setRefresh(true).execute().actionGet(); + .field("shape", shape).endObject()).setRefreshPolicy(IMMEDIATE).get(); client().prepareIndex("test", "type1", "1").setSource(jsonBuilder().startObject() .field("name", "Document 1") .startObject("location") .field("type", "point") .startArray("coordinates").value(-30).value(-30).endArray() .endObject() - .endObject()).setRefresh(true).execute().actionGet(); + .endObject()).setRefreshPolicy(IMMEDIATE).get(); SearchResponse searchResponse = client().prepareSearch("test").setTypes("type1") .setQuery(geoIntersectionQuery("location", "Big_Rectangle", "shape_type")) @@ -226,7 +226,7 @@ public class GeoShapeQueryTests extends ESSingleNodeTestCase { String.format( Locale.ROOT, "{ %s, \"1\" : { %s, \"2\" : { %s, \"3\" : { %s } }} }", location, location, location, location ) - ).setRefresh(true).execute().actionGet(); + ).setRefreshPolicy(IMMEDIATE).get(); client().prepareIndex("test", "type", "1") .setSource(jsonBuilder().startObject().startObject("location") .field("type", "polygon") @@ -237,7 +237,7 @@ public class GeoShapeQueryTests extends ESSingleNodeTestCase { .startArray().value(-20).value(20).endArray() .startArray().value(-20).value(-20).endArray() .endArray().endArray() - .endObject().endObject()).setRefresh(true).execute().actionGet(); + .endObject().endObject()).setRefreshPolicy(IMMEDIATE).get(); GeoShapeQueryBuilder filter = QueryBuilders.geoShapeQuery("location", "1", "type").relation(ShapeRelation.INTERSECTS) .indexedShapeIndex("shapes") @@ -305,7 +305,7 @@ public class GeoShapeQueryTests extends ESSingleNodeTestCase { .execute().actionGet(); XContentBuilder docSource = gcb.toXContent(jsonBuilder().startObject().field("location"), null).endObject(); - client().prepareIndex("test", "type", "1").setSource(docSource).setRefresh(true).execute().actionGet(); + client().prepareIndex("test", "type", "1").setSource(docSource).setRefreshPolicy(IMMEDIATE).get(); ShapeBuilder filterShape = (gcb.getShapeAt(randomIntBetween(0, gcb.numShapes() - 1))); @@ -326,12 +326,12 @@ public class GeoShapeQueryTests extends ESSingleNodeTestCase { .execute().actionGet(); XContentBuilder docSource = gcb.toXContent(jsonBuilder().startObject().field("location"), null).endObject(); - client().prepareIndex("test", "type", "1").setSource(docSource).setRefresh(true).execute().actionGet(); + client().prepareIndex("test", "type", "1").setSource(docSource).setRefreshPolicy(IMMEDIATE).get(); // index the mbr of the collection EnvelopeBuilder env = new EnvelopeBuilder(new Coordinate(mbr.getMinX(), mbr.getMaxY()), new Coordinate(mbr.getMaxX(), mbr.getMinY())); docSource = env.toXContent(jsonBuilder().startObject().field("location"), null).endObject(); - client().prepareIndex("test", "type", "2").setSource(docSource).setRefresh(true).execute().actionGet(); + client().prepareIndex("test", "type", "2").setSource(docSource).setRefreshPolicy(IMMEDIATE).get(); ShapeBuilder filterShape = (gcb.getShapeAt(randomIntBetween(0, gcb.numShapes() - 1))); GeoShapeQueryBuilder filter = QueryBuilders.geoShapeQuery("location", filterShape) @@ -371,7 +371,7 @@ public class GeoShapeQueryTests extends ESSingleNodeTestCase { .endArray() .endObject().endObject(); client().prepareIndex("test", "type", "1") - .setSource(docSource).setRefresh(true).execute().actionGet(); + .setSource(docSource).setRefreshPolicy(IMMEDIATE).get(); GeoShapeQueryBuilder filter = QueryBuilders.geoShapeQuery( "location", @@ -427,7 +427,7 @@ public class GeoShapeQueryTests extends ESSingleNodeTestCase { try { client().prepareIndex("geo_points_only", "type1", "1") .setSource(jsonBuilder().startObject().field("location", shape).endObject()) - .setRefresh(true).execute().actionGet(); + .setRefreshPolicy(IMMEDIATE).get(); } catch (MapperParsingException e) { // RandomShapeGenerator created something other than a POINT type, verify the correct exception is thrown assertThat(e.getCause().getMessage(), containsString("is configured for points only")); diff --git a/core/src/test/java/org/elasticsearch/search/innerhits/InnerHitsIT.java b/core/src/test/java/org/elasticsearch/search/innerhits/InnerHitsIT.java index 0d1ba43659c..01f98564814 100644 --- a/core/src/test/java/org/elasticsearch/search/innerhits/InnerHitsIT.java +++ b/core/src/test/java/org/elasticsearch/search/innerhits/InnerHitsIT.java @@ -48,6 +48,7 @@ import java.util.Collections; import java.util.List; import java.util.Locale; +import static org.elasticsearch.action.support.WriteRequest.RefreshPolicy.IMMEDIATE; import static org.elasticsearch.common.xcontent.XContentFactory.jsonBuilder; import static org.elasticsearch.common.xcontent.support.XContentMapValues.extractValue; import static org.elasticsearch.index.query.QueryBuilders.boolQuery; @@ -920,7 +921,7 @@ public class InnerHitsIT extends ESIntegTestCase { .endObject() .endArray() .endObject()) - .setRefresh(true) + .setRefreshPolicy(IMMEDIATE) .get(); response = client().prepareSearch("index2") diff --git a/core/src/test/java/org/elasticsearch/search/nested/SimpleNestedIT.java b/core/src/test/java/org/elasticsearch/search/nested/SimpleNestedIT.java index 4d2f92c3c6c..ba378a3c404 100644 --- a/core/src/test/java/org/elasticsearch/search/nested/SimpleNestedIT.java +++ b/core/src/test/java/org/elasticsearch/search/nested/SimpleNestedIT.java @@ -38,6 +38,7 @@ import org.elasticsearch.search.sort.SortMode; import org.elasticsearch.search.sort.SortOrder; import org.elasticsearch.test.ESIntegTestCase; +import static org.elasticsearch.action.support.WriteRequest.RefreshPolicy.IMMEDIATE; import static org.elasticsearch.common.xcontent.XContentFactory.jsonBuilder; import static org.elasticsearch.index.query.QueryBuilders.boolQuery; import static org.elasticsearch.index.query.QueryBuilders.matchAllQuery; @@ -306,7 +307,7 @@ public class SimpleNestedIT extends ESIntegTestCase { .endObject() .endArray() .endObject()) - .setRefresh(true) + .setRefreshPolicy(IMMEDIATE) .execute().actionGet(); SearchResponse searchResponse = client().prepareSearch("test") diff --git a/core/src/test/java/org/elasticsearch/search/query/MultiMatchQueryIT.java b/core/src/test/java/org/elasticsearch/search/query/MultiMatchQueryIT.java index d90e465e0a0..33ef2b97216 100644 --- a/core/src/test/java/org/elasticsearch/search/query/MultiMatchQueryIT.java +++ b/core/src/test/java/org/elasticsearch/search/query/MultiMatchQueryIT.java @@ -386,7 +386,6 @@ public class MultiMatchQueryIT extends ESIntegTestCase { { String minShouldMatch = randomBoolean() ? null : "" + between(0, 1); - Operator op = randomBoolean() ? Operator.AND : Operator.OR; SearchResponse left = client().prepareSearch("test").setSize(numDocs) .addSort(SortBuilders.scoreSort()).addSort(SortBuilders.fieldSort("_uid")) .setQuery(randomizeType(multiMatchQuery("capta", "full_name", "first_name", "last_name", "category") @@ -396,15 +395,14 @@ public class MultiMatchQueryIT extends ESIntegTestCase { .addSort(SortBuilders.scoreSort()).addSort(SortBuilders.fieldSort("_uid")) .setQuery(boolQuery().minimumShouldMatch(minShouldMatch) .should(matchPhrasePrefixQuery("full_name", "capta")) - .should(matchPhrasePrefixQuery("first_name", "capta").operator(op)) - .should(matchPhrasePrefixQuery("last_name", "capta").operator(op)) - .should(matchPhrasePrefixQuery("category", "capta").operator(op)) + .should(matchPhrasePrefixQuery("first_name", "capta")) + .should(matchPhrasePrefixQuery("last_name", "capta")) + .should(matchPhrasePrefixQuery("category", "capta")) ).get(); assertEquivalent("capta", left, right); } { String minShouldMatch = randomBoolean() ? null : "" + between(0, 1); - Operator op = randomBoolean() ? Operator.AND : Operator.OR; SearchResponse left; if (randomBoolean()) { left = client().prepareSearch("test").setSize(numDocs) @@ -421,9 +419,9 @@ public class MultiMatchQueryIT extends ESIntegTestCase { .addSort(SortBuilders.scoreSort()).addSort(SortBuilders.fieldSort("_uid")) .setQuery(boolQuery().minimumShouldMatch(minShouldMatch) .should(matchPhraseQuery("full_name", "captain america")) - .should(matchPhraseQuery("first_name", "captain america").operator(op)) - .should(matchPhraseQuery("last_name", "captain america").operator(op)) - .should(matchPhraseQuery("category", "captain america").operator(op)) + .should(matchPhraseQuery("first_name", "captain america")) + .should(matchPhraseQuery("last_name", "captain america")) + .should(matchPhraseQuery("category", "captain america")) ).get(); assertEquivalent("captain america", left, right); } diff --git a/core/src/test/java/org/elasticsearch/search/query/SearchQueryIT.java b/core/src/test/java/org/elasticsearch/search/query/SearchQueryIT.java index d7d4a3fd188..5e4a7d6f77f 100644 --- a/core/src/test/java/org/elasticsearch/search/query/SearchQueryIT.java +++ b/core/src/test/java/org/elasticsearch/search/query/SearchQueryIT.java @@ -55,6 +55,7 @@ import java.io.IOException; import java.util.Random; import java.util.concurrent.ExecutionException; +import static org.elasticsearch.action.support.WriteRequest.RefreshPolicy.IMMEDIATE; import static org.elasticsearch.cluster.metadata.IndexMetaData.SETTING_NUMBER_OF_SHARDS; import static org.elasticsearch.common.xcontent.XContentFactory.jsonBuilder; import static org.elasticsearch.index.query.QueryBuilders.boolQuery; @@ -731,7 +732,7 @@ public class SearchQueryIT extends ESIntegTestCase { public void testPassQueryOrFilterAsJSONString() throws Exception { createIndex("test"); - client().prepareIndex("test", "type1", "1").setSource("field1", "value1_1", "field2", "value2_1").setRefresh(true).get(); + client().prepareIndex("test", "type1", "1").setSource("field1", "value1_1", "field2", "value2_1").setRefreshPolicy(IMMEDIATE).get(); WrapperQueryBuilder wrapper = new WrapperQueryBuilder("{ \"term\" : { \"field1\" : \"value1_1\" } }"); assertHitCount(client().prepareSearch().setQuery(wrapper).get(), 1L); @@ -1548,7 +1549,7 @@ public class SearchQueryIT extends ESIntegTestCase { } public void testMultiFieldQueryString() { - client().prepareIndex("test", "s", "1").setSource("field1", "value1", "field2", "value2").setRefresh(true).get(); + client().prepareIndex("test", "s", "1").setSource("field1", "value1", "field2", "value2").setRefreshPolicy(IMMEDIATE).get(); logger.info("regular"); assertHitCount(client().prepareSearch("test").setQuery(queryStringQuery("value1").field("field1").field("field2")).get(), 1); @@ -1702,7 +1703,7 @@ public class SearchQueryIT extends ESIntegTestCase { } public void testAllFieldEmptyMapping() throws Exception { - client().prepareIndex("myindex", "mytype").setId("1").setSource("{}").setRefresh(true).get(); + client().prepareIndex("myindex", "mytype").setId("1").setSource("{}").setRefreshPolicy(IMMEDIATE).get(); SearchResponse response = client().prepareSearch("myindex").setQuery(matchQuery("_all", "foo")).get(); assertNoFailures(response); } @@ -1712,7 +1713,7 @@ public class SearchQueryIT extends ESIntegTestCase { assertAcked(client().admin().indices().preparePutMapping("myindex").setType("mytype").setSource( jsonBuilder().startObject().startObject("mytype").startObject("_all").field("enabled", false) .endObject().endObject().endObject())); - client().prepareIndex("myindex", "mytype").setId("1").setSource("bar", "foo").setRefresh(true).get(); + client().prepareIndex("myindex", "mytype").setId("1").setSource("bar", "foo").setRefreshPolicy(IMMEDIATE).get(); SearchResponse response = client().prepareSearch("myindex").setQuery(matchQuery("_all", "foo")).get(); assertNoFailures(response); assertHitCount(response, 0); @@ -2025,7 +2026,7 @@ public class SearchQueryIT extends ESIntegTestCase { client().prepareIndex("test", "test", "1").setSource("origin", "C.A1234.5678") - .setRefresh(true) + .setRefreshPolicy(IMMEDIATE) .get(); SearchResponse searchResponse = client().prepareSearch("test") diff --git a/core/src/test/java/org/elasticsearch/search/scroll/SearchScrollIT.java b/core/src/test/java/org/elasticsearch/search/scroll/SearchScrollIT.java index 9ee9e0841cf..3bce6e10f83 100644 --- a/core/src/test/java/org/elasticsearch/search/scroll/SearchScrollIT.java +++ b/core/src/test/java/org/elasticsearch/search/scroll/SearchScrollIT.java @@ -48,6 +48,7 @@ import org.elasticsearch.test.hamcrest.ElasticsearchAssertions; import java.io.IOException; import java.util.Map; +import static org.elasticsearch.action.support.WriteRequest.RefreshPolicy.IMMEDIATE; import static org.elasticsearch.common.xcontent.XContentFactory.jsonBuilder; import static org.elasticsearch.index.query.QueryBuilders.matchAllQuery; import static org.elasticsearch.index.query.QueryBuilders.queryStringQuery; @@ -421,7 +422,7 @@ public class SearchScrollIT extends ESIntegTestCase { public void testDeepScrollingDoesNotBlowUp() throws Exception { client().prepareIndex("index", "type", "1") .setSource("field", "value") - .setRefresh(true) + .setRefreshPolicy(IMMEDIATE) .execute().get(); for (SearchType searchType : SearchType.values()) { diff --git a/core/src/test/java/org/elasticsearch/search/simple/SimpleSearchIT.java b/core/src/test/java/org/elasticsearch/search/simple/SimpleSearchIT.java index 6dc10232b21..9751df26329 100644 --- a/core/src/test/java/org/elasticsearch/search/simple/SimpleSearchIT.java +++ b/core/src/test/java/org/elasticsearch/search/simple/SimpleSearchIT.java @@ -36,6 +36,7 @@ import org.elasticsearch.rest.RestStatus; import org.elasticsearch.search.rescore.QueryRescorerBuilder; import org.elasticsearch.test.ESIntegTestCase; +import static org.elasticsearch.action.support.WriteRequest.RefreshPolicy.IMMEDIATE; import static org.elasticsearch.cluster.metadata.IndexMetaData.SETTING_NUMBER_OF_REPLICAS; import static org.elasticsearch.cluster.metadata.IndexMetaData.SETTING_NUMBER_OF_SHARDS; import static org.elasticsearch.common.xcontent.XContentFactory.jsonBuilder; @@ -101,7 +102,8 @@ public class SimpleSearchIT extends ESIntegTestCase { .endObject().endObject().endObject()) .execute().actionGet(); - client().prepareIndex("test", "type1", "1").setSource("from", "192.168.0.5", "to", "192.168.0.10").setRefresh(true).execute().actionGet(); + client().prepareIndex("test", "type1", "1").setSource("from", "192.168.0.5", "to", "192.168.0.10").setRefreshPolicy(IMMEDIATE) + .get(); SearchResponse search = client().prepareSearch() .setQuery(boolQuery().must(rangeQuery("from").lte("192.168.0.7")).must(rangeQuery("to").gte("192.168.0.7"))) @@ -185,7 +187,7 @@ public class SimpleSearchIT extends ESIntegTestCase { public void testSimpleId() { createIndex("test"); - client().prepareIndex("test", "type", "XXX1").setSource("field", "value").setRefresh(true).execute().actionGet(); + client().prepareIndex("test", "type", "XXX1").setSource("field", "value").setRefreshPolicy(IMMEDIATE).get(); // id is not indexed, but lets see that we automatically convert to SearchResponse searchResponse = client().prepareSearch().setQuery(QueryBuilders.termQuery("_id", "XXX1")).execute().actionGet(); assertHitCount(searchResponse, 1L); @@ -441,6 +443,6 @@ public class SimpleSearchIT extends ESIntegTestCase { assertThat(e.toString(), containsString("Rescore window [" + windowSize + "] is too large. It must " + "be less than [" + IndexSettings.MAX_RESCORE_WINDOW_SETTING.get(Settings.EMPTY))); assertThat(e.toString(), containsString( - "This limit can be set by chaning the [" + IndexSettings.MAX_RESCORE_WINDOW_SETTING.getKey() + "] index level setting.")); + "This limit can be set by chaining the [" + IndexSettings.MAX_RESCORE_WINDOW_SETTING.getKey() + "] index level setting.")); } } diff --git a/core/src/test/java/org/elasticsearch/search/slice/DocValuesSliceQueryTests.java b/core/src/test/java/org/elasticsearch/search/slice/DocValuesSliceQueryTests.java new file mode 100644 index 00000000000..846c411881f --- /dev/null +++ b/core/src/test/java/org/elasticsearch/search/slice/DocValuesSliceQueryTests.java @@ -0,0 +1,125 @@ +/* + * 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.document.Document; +import org.apache.lucene.document.Field; +import org.apache.lucene.document.SortedNumericDocValuesField; +import org.apache.lucene.document.StringField; +import org.apache.lucene.index.IndexReader; +import org.apache.lucene.index.LeafReaderContext; +import org.apache.lucene.index.RandomIndexWriter; +import org.apache.lucene.search.IndexSearcher; +import org.apache.lucene.search.Collector; +import org.apache.lucene.search.LeafCollector; +import org.apache.lucene.search.Scorer; +import org.apache.lucene.search.QueryUtils; +import org.apache.lucene.store.Directory; +import org.apache.lucene.util.NumericUtils; +import org.elasticsearch.common.UUIDs; +import org.elasticsearch.test.ESTestCase; + +import java.io.IOException; +import java.util.Collections; +import java.util.HashSet; +import java.util.Set; + +import static org.hamcrest.Matchers.equalTo; + +public class DocValuesSliceQueryTests extends ESTestCase { + + public void testBasics() { + DocValuesSliceQuery query1 = + new DocValuesSliceQuery("field1", 1, 10); + DocValuesSliceQuery query2 = + new DocValuesSliceQuery("field1", 1, 10); + DocValuesSliceQuery query3 = + new DocValuesSliceQuery("field2", 1, 10); + DocValuesSliceQuery query4 = + new DocValuesSliceQuery("field1", 2, 10); + QueryUtils.check(query1); + QueryUtils.checkEqual(query1, query2); + QueryUtils.checkUnequal(query1, query3); + QueryUtils.checkUnequal(query1, query4); + } + + public void testSearch() throws Exception { + final int numDocs = randomIntBetween(100, 200); + final Directory dir = newDirectory(); + final RandomIndexWriter w = new RandomIndexWriter(random(), dir); + int max = randomIntBetween(2, 10); + int[] sliceCounters1 = new int[max]; + int[] sliceCounters2 = new int[max]; + Set keys = new HashSet<>(); + for (int i = 0; i < numDocs; ++i) { + Document doc = new Document(); + String uuid = UUIDs.base64UUID(); + int intValue = randomInt(); + long doubleValue = NumericUtils.doubleToSortableLong(randomDouble()); + doc.add(new StringField("uuid", uuid, Field.Store.YES)); + doc.add(new SortedNumericDocValuesField("intField", intValue)); + doc.add(new SortedNumericDocValuesField("doubleField", doubleValue)); + w.addDocument(doc); + sliceCounters1[Math.floorMod(BitMixer.mix((long) intValue), max)] ++; + sliceCounters2[Math.floorMod(BitMixer.mix(doubleValue), max)] ++; + keys.add(uuid); + } + final IndexReader reader = w.getReader(); + final IndexSearcher searcher = newSearcher(reader); + + for (int id = 0; id < max; id++) { + DocValuesSliceQuery query1 = + new DocValuesSliceQuery("intField", id, max); + assertThat(searcher.count(query1), equalTo(sliceCounters1[id])); + + DocValuesSliceQuery query2 = + new DocValuesSliceQuery("doubleField", id, max); + assertThat(searcher.count(query2), equalTo(sliceCounters2[id])); + searcher.search(query1, new Collector() { + @Override + public LeafCollector getLeafCollector(LeafReaderContext context) throws IOException { + return new LeafCollector() { + @Override + public void setScorer(Scorer scorer) throws IOException { + } + + @Override + public void collect(int doc) throws IOException { + Document d = context.reader().document(doc, Collections.singleton("uuid")); + String uuid = d.get("uuid"); + assertThat(keys.contains(uuid), equalTo(true)); + keys.remove(uuid); + } + }; + } + + @Override + public boolean needsScores() { + return false; + } + }); + } + assertThat(keys.size(), equalTo(0)); + w.close(); + reader.close(); + dir.close(); + } +} diff --git a/core/src/test/java/org/elasticsearch/search/slice/SearchSliceIT.java b/core/src/test/java/org/elasticsearch/search/slice/SearchSliceIT.java new file mode 100644 index 00000000000..ad93d14f21f --- /dev/null +++ b/core/src/test/java/org/elasticsearch/search/slice/SearchSliceIT.java @@ -0,0 +1,215 @@ +/* + * 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.elasticsearch.action.index.IndexRequestBuilder; +import org.elasticsearch.action.search.SearchPhaseExecutionException; +import org.elasticsearch.action.search.SearchRequestBuilder; +import org.elasticsearch.action.search.SearchResponse; +import org.elasticsearch.common.unit.TimeValue; +import org.elasticsearch.common.xcontent.XContentBuilder; +import org.elasticsearch.common.xcontent.XContentFactory; +import org.elasticsearch.search.Scroll; +import org.elasticsearch.search.SearchContextException; +import org.elasticsearch.search.SearchHit; +import org.elasticsearch.search.sort.SortBuilders; +import org.elasticsearch.test.ESIntegTestCase; + +import java.io.IOException; +import java.util.List; +import java.util.ArrayList; +import java.util.Set; +import java.util.HashSet; +import java.util.concurrent.ExecutionException; + +import static org.elasticsearch.common.xcontent.XContentFactory.jsonBuilder; +import static org.elasticsearch.index.query.QueryBuilders.matchAllQuery; +import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertAcked; +import static org.hamcrest.Matchers.equalTo; +import static org.hamcrest.Matchers.greaterThan; +import static org.hamcrest.Matchers.startsWith; + +public class SearchSliceIT extends ESIntegTestCase { + private static final int NUM_DOCS = 1000; + + private int setupIndex(boolean withDocs) throws IOException, ExecutionException, InterruptedException { + String mapping = XContentFactory.jsonBuilder(). + startObject() + .startObject("type") + .startObject("properties") + .startObject("invalid_random_kw") + .field("type", "keyword") + .field("doc_values", "false") + .endObject() + .startObject("random_int") + .field("type", "integer") + .field("doc_values", "true") + .endObject() + .startObject("invalid_random_int") + .field("type", "integer") + .field("doc_values", "false") + .endObject() + .endObject() + .endObject() + .endObject().string(); + int numberOfShards = randomIntBetween(1, 7); + assertAcked(client().admin().indices().prepareCreate("test") + .setSettings("number_of_shards", numberOfShards) + .addMapping("type", mapping)); + ensureGreen(); + + if (withDocs == false) { + return numberOfShards; + } + + List requests = new ArrayList<>(); + for (int i = 0; i < NUM_DOCS; i++) { + XContentBuilder builder = jsonBuilder(); + builder.startObject(); + builder.field("invalid_random_kw", randomAsciiOfLengthBetween(5, 20)); + builder.field("random_int", randomInt()); + builder.field("static_int", 0); + builder.field("invalid_random_int", randomInt()); + builder.endObject(); + requests.add(client().prepareIndex("test", "test").setSource(builder)); + } + indexRandom(true, requests); + return numberOfShards; + } + + public void testDocIdSort() throws Exception { + int numShards = setupIndex(true); + SearchResponse sr = client().prepareSearch("test") + .setQuery(matchAllQuery()) + .setSize(0) + .get(); + int numDocs = (int) sr.getHits().getTotalHits(); + assertThat(numDocs, equalTo(NUM_DOCS)); + int max = randomIntBetween(2, numShards*3); + for (String field : new String[]{"_uid", "random_int", "static_int"}) { + int fetchSize = randomIntBetween(10, 100); + SearchRequestBuilder request = client().prepareSearch("test") + .setQuery(matchAllQuery()) + .setScroll(new Scroll(TimeValue.timeValueSeconds(10))) + .setSize(fetchSize) + .addSort(SortBuilders.fieldSort("_doc")); + assertSearchSlicesWithScroll(request, field, max); + } + } + + public void testNumericSort() throws Exception { + int numShards = setupIndex(true); + SearchResponse sr = client().prepareSearch("test") + .setQuery(matchAllQuery()) + .setSize(0) + .get(); + int numDocs = (int) sr.getHits().getTotalHits(); + assertThat(numDocs, equalTo(NUM_DOCS)); + + int max = randomIntBetween(2, numShards*3); + for (String field : new String[]{"_uid", "random_int", "static_int"}) { + int fetchSize = randomIntBetween(10, 100); + SearchRequestBuilder request = client().prepareSearch("test") + .setQuery(matchAllQuery()) + .setScroll(new Scroll(TimeValue.timeValueSeconds(10))) + .addSort(SortBuilders.fieldSort("random_int")) + .setSize(fetchSize); + assertSearchSlicesWithScroll(request, field, max); + } + } + + public void testInvalidFields() throws Exception { + setupIndex(false); + SearchPhaseExecutionException exc = expectThrows(SearchPhaseExecutionException.class, + () -> client().prepareSearch("test") + .setQuery(matchAllQuery()) + .setScroll(new Scroll(TimeValue.timeValueSeconds(10))) + .slice(new SliceBuilder("invalid_random_int", 0, 10)) + .get()); + Throwable rootCause = findRootCause(exc); + assertThat(rootCause.getClass(), equalTo(IllegalArgumentException.class)); + assertThat(rootCause.getMessage(), + startsWith("cannot load numeric doc values")); + + exc = expectThrows(SearchPhaseExecutionException.class, () -> client().prepareSearch("test") + .setQuery(matchAllQuery()) + .setScroll(new Scroll(TimeValue.timeValueSeconds(10))) + .slice(new SliceBuilder("invalid_random_kw", 0, 10)) + .get()); + rootCause = findRootCause(exc); + assertThat(rootCause.getClass(), equalTo(IllegalArgumentException.class)); + assertThat(rootCause.getMessage(), + startsWith("cannot load numeric doc values")); + } + + public void testInvalidQuery() throws Exception { + setupIndex(false); + SearchPhaseExecutionException exc = expectThrows(SearchPhaseExecutionException.class, + () -> client().prepareSearch() + .setQuery(matchAllQuery()) + .slice(new SliceBuilder("invalid_random_int", 0, 10)) + .get()); + Throwable rootCause = findRootCause(exc); + assertThat(rootCause.getClass(), equalTo(SearchContextException.class)); + assertThat(rootCause.getMessage(), + equalTo("`slice` cannot be used outside of a scroll context")); + } + + private void assertSearchSlicesWithScroll(SearchRequestBuilder request, String field, int numSlice) { + int totalResults = 0; + List keys = new ArrayList<>(); + for (int id = 0; id < numSlice; id++) { + SliceBuilder sliceBuilder = new SliceBuilder(field, id, numSlice); + SearchResponse searchResponse = request.slice(sliceBuilder).get(); + totalResults += searchResponse.getHits().getHits().length; + int expectedSliceResults = (int) searchResponse.getHits().getTotalHits(); + int numSliceResults = searchResponse.getHits().getHits().length; + String scrollId = searchResponse.getScrollId(); + for (SearchHit hit : searchResponse.getHits().getHits()) { + keys.add(hit.getId()); + } + while (searchResponse.getHits().getHits().length > 0) { + searchResponse = client().prepareSearchScroll("test") + .setScrollId(scrollId) + .setScroll(new Scroll(TimeValue.timeValueSeconds(10))) + .get(); + scrollId = searchResponse.getScrollId(); + totalResults += searchResponse.getHits().getHits().length; + numSliceResults += searchResponse.getHits().getHits().length; + for (SearchHit hit : searchResponse.getHits().getHits()) { + keys.add(hit.getId()); + } + } + assertThat(numSliceResults, equalTo(expectedSliceResults)); + clearScroll(scrollId); + } + assertThat(totalResults, equalTo(NUM_DOCS)); + assertThat(keys.size(), equalTo(NUM_DOCS)); + assertThat(new HashSet(keys).size(), equalTo(NUM_DOCS)); + } + + private Throwable findRootCause(Exception e) { + Throwable ret = e; + while (ret.getCause() != null) { + ret = ret.getCause(); + } + return ret; + } +} diff --git a/core/src/test/java/org/elasticsearch/search/slice/SliceBuilderTests.java b/core/src/test/java/org/elasticsearch/search/slice/SliceBuilderTests.java new file mode 100644 index 00000000000..217f97ace0a --- /dev/null +++ b/core/src/test/java/org/elasticsearch/search/slice/SliceBuilderTests.java @@ -0,0 +1,339 @@ +/* + * 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.analysis.MockAnalyzer; +import org.apache.lucene.index.DirectoryReader; +import org.apache.lucene.index.DocValuesType; +import org.apache.lucene.index.IndexReader; +import org.apache.lucene.index.IndexWriter; +import org.apache.lucene.search.MatchAllDocsQuery; +import org.apache.lucene.search.Query; +import org.apache.lucene.store.Directory; +import org.apache.lucene.store.RAMDirectory; +import org.elasticsearch.common.Nullable; +import org.elasticsearch.common.ParseFieldMatcher; +import org.elasticsearch.common.io.stream.BytesStreamOutput; +import org.elasticsearch.common.io.stream.NamedWriteableAwareStreamInput; +import org.elasticsearch.common.io.stream.NamedWriteableRegistry; +import org.elasticsearch.common.io.stream.StreamInput; +import org.elasticsearch.common.lucene.search.MatchNoDocsQuery; +import org.elasticsearch.common.xcontent.XContentParser; +import org.elasticsearch.common.xcontent.XContentType; +import org.elasticsearch.common.xcontent.XContentFactory; +import org.elasticsearch.common.xcontent.XContentHelper; +import org.elasticsearch.common.xcontent.XContentBuilder; +import org.elasticsearch.index.fielddata.IndexNumericFieldData; +import org.elasticsearch.index.mapper.MappedFieldType; +import org.elasticsearch.index.mapper.internal.UidFieldMapper; +import org.elasticsearch.index.query.MatchAllQueryBuilder; +import org.elasticsearch.index.query.QueryParseContext; +import org.elasticsearch.index.query.QueryParser; +import org.elasticsearch.index.query.QueryShardContext; +import org.elasticsearch.indices.query.IndicesQueriesRegistry; +import org.elasticsearch.test.ESTestCase; +import org.junit.AfterClass; +import org.junit.BeforeClass; + +import java.io.IOException; +import java.util.List; +import java.util.ArrayList; +import java.util.Map; +import java.util.HashMap; +import java.util.HashSet; +import java.util.concurrent.atomic.AtomicInteger; + +import static org.hamcrest.Matchers.equalTo; +import static org.hamcrest.Matchers.instanceOf; +import static org.hamcrest.Matchers.containsString; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; + +public class SliceBuilderTests extends ESTestCase { + private static final int MAX_SLICE = 20; + private static NamedWriteableRegistry namedWriteableRegistry; + private static IndicesQueriesRegistry indicesQueriesRegistry; + + /** + * setup for the whole base test class + */ + @BeforeClass + public static void init() { + namedWriteableRegistry = new NamedWriteableRegistry(); + indicesQueriesRegistry = new IndicesQueriesRegistry(); + QueryParser parser = MatchAllQueryBuilder::fromXContent; + indicesQueriesRegistry.register(parser, MatchAllQueryBuilder.QUERY_NAME_FIELD); + } + + @AfterClass + public static void afterClass() throws Exception { + namedWriteableRegistry = null; + indicesQueriesRegistry = null; + } + + private final SliceBuilder randomSliceBuilder() throws IOException { + int max = randomIntBetween(2, MAX_SLICE); + int id = randomInt(max - 1); + String field = randomAsciiOfLengthBetween(5, 20); + return new SliceBuilder(field, id, max); + } + + private static SliceBuilder serializedCopy(SliceBuilder original) throws IOException { + try (BytesStreamOutput output = new BytesStreamOutput()) { + original.writeTo(output); + try (StreamInput in = + new NamedWriteableAwareStreamInput(StreamInput.wrap(output.bytes()), namedWriteableRegistry)) { + return new SliceBuilder(in); + } + } + } + + public void testSerialization() throws Exception { + SliceBuilder original = randomSliceBuilder(); + SliceBuilder deserialized = serializedCopy(original); + assertEquals(deserialized, original); + assertEquals(deserialized.hashCode(), original.hashCode()); + assertNotSame(deserialized, original); + } + + public void testEqualsAndHashcode() throws Exception { + SliceBuilder firstBuilder = randomSliceBuilder(); + assertFalse("sliceBuilder is equal to null", firstBuilder.equals(null)); + assertFalse("sliceBuilder is equal to incompatible type", firstBuilder.equals("")); + assertTrue("sliceBuilder is not equal to self", firstBuilder.equals(firstBuilder)); + assertThat("same searchFrom's hashcode returns different values if called multiple times", + firstBuilder.hashCode(), equalTo(firstBuilder.hashCode())); + + SliceBuilder secondBuilder = serializedCopy(firstBuilder); + assertTrue("sliceBuilder is not equal to self", secondBuilder.equals(secondBuilder)); + assertTrue("sliceBuilder is not equal to its copy", firstBuilder.equals(secondBuilder)); + assertTrue("equals is not symmetric", secondBuilder.equals(firstBuilder)); + assertThat("sliceBuilder copy's hashcode is different from original hashcode", secondBuilder.hashCode(), + equalTo(firstBuilder.hashCode())); + SliceBuilder thirdBuilder = serializedCopy(secondBuilder); + assertTrue("sliceBuilder is not equal to self", thirdBuilder.equals(thirdBuilder)); + assertTrue("sliceBuilder is not equal to its copy", secondBuilder.equals(thirdBuilder)); + assertThat("sliceBuilder copy's hashcode is different from original hashcode", secondBuilder.hashCode(), + equalTo(thirdBuilder.hashCode())); + assertTrue("equals is not transitive", firstBuilder.equals(thirdBuilder)); + assertThat("sliceBuilder copy's hashcode is different from original hashcode", firstBuilder.hashCode(), + equalTo(thirdBuilder.hashCode())); + assertTrue("sliceBuilder is not symmetric", thirdBuilder.equals(secondBuilder)); + assertTrue("sliceBuilder is not symmetric", thirdBuilder.equals(firstBuilder)); + } + + public void testFromXContent() throws Exception { + SliceBuilder sliceBuilder = randomSliceBuilder(); + XContentBuilder builder = XContentFactory.contentBuilder(randomFrom(XContentType.values())); + if (randomBoolean()) { + builder.prettyPrint(); + } + builder.startObject(); + sliceBuilder.innerToXContent(builder); + builder.endObject(); + XContentParser parser = XContentHelper.createParser(shuffleXContent(builder).bytes()); + QueryParseContext context = new QueryParseContext(indicesQueriesRegistry, parser, + ParseFieldMatcher.STRICT); + SliceBuilder secondSliceBuilder = SliceBuilder.fromXContent(context); + assertNotSame(sliceBuilder, secondSliceBuilder); + assertEquals(sliceBuilder, secondSliceBuilder); + assertEquals(sliceBuilder.hashCode(), secondSliceBuilder.hashCode()); + } + + public void testInvalidArguments() throws Exception { + Exception e = expectThrows(IllegalArgumentException.class, () -> new SliceBuilder("field", -1, 10)); + assertEquals(e.getMessage(), "id must be greater than or equal to 0"); + + e = expectThrows(IllegalArgumentException.class, () -> new SliceBuilder("field", 10, -1)); + assertEquals(e.getMessage(), "max must be greater than 1"); + + e = expectThrows(IllegalArgumentException.class, () -> new SliceBuilder("field", 10, 0)); + assertEquals(e.getMessage(), "max must be greater than 1"); + + e = expectThrows(IllegalArgumentException.class, () -> new SliceBuilder("field", 10, 5)); + assertEquals(e.getMessage(), "max must be greater than id"); + + e = expectThrows(IllegalArgumentException.class, () -> new SliceBuilder("field", 1000, 1000)); + assertEquals(e.getMessage(), "max must be greater than id"); + e = expectThrows(IllegalArgumentException.class, () -> new SliceBuilder("field", 1001, 1000)); + assertEquals(e.getMessage(), "max must be greater than id"); + } + + public void testToFilter() throws IOException { + Directory dir = new RAMDirectory(); + try (IndexWriter writer = new IndexWriter(dir, newIndexWriterConfig(new MockAnalyzer(random())))) { + writer.commit(); + } + QueryShardContext context = mock(QueryShardContext.class); + try (IndexReader reader = DirectoryReader.open(dir)) { + MappedFieldType fieldType = new MappedFieldType() { + @Override + public MappedFieldType clone() { + return null; + } + + @Override + public String typeName() { + return null; + } + + @Override + public Query termQuery(Object value, @Nullable QueryShardContext context) { + return null; + } + }; + fieldType.setName(UidFieldMapper.NAME); + fieldType.setHasDocValues(false); + when(context.fieldMapper(UidFieldMapper.NAME)).thenReturn(fieldType); + when(context.getIndexReader()).thenReturn(reader); + SliceBuilder builder = new SliceBuilder(5, 10); + Query query = builder.toFilter(context, 0, 1); + assertThat(query, instanceOf(TermsSliceQuery.class)); + + assertThat(builder.toFilter(context, 0, 1), equalTo(query)); + try (IndexReader newReader = DirectoryReader.open(dir)) { + when(context.getIndexReader()).thenReturn(newReader); + assertThat(builder.toFilter(context, 0, 1), equalTo(query)); + } + } + + try (IndexReader reader = DirectoryReader.open(dir)) { + MappedFieldType fieldType = new MappedFieldType() { + @Override + public MappedFieldType clone() { + return null; + } + + @Override + public String typeName() { + return null; + } + + @Override + public Query termQuery(Object value, @Nullable QueryShardContext context) { + return null; + } + }; + fieldType.setName("field_doc_values"); + fieldType.setHasDocValues(true); + fieldType.setDocValuesType(DocValuesType.SORTED_NUMERIC); + when(context.fieldMapper("field_doc_values")).thenReturn(fieldType); + when(context.getIndexReader()).thenReturn(reader); + IndexNumericFieldData fd = mock(IndexNumericFieldData.class); + when(context.getForField(fieldType)).thenReturn(fd); + SliceBuilder builder = new SliceBuilder("field_doc_values", 5, 10); + Query query = builder.toFilter(context, 0, 1); + assertThat(query, instanceOf(DocValuesSliceQuery.class)); + + assertThat(builder.toFilter(context, 0, 1), equalTo(query)); + try (IndexReader newReader = DirectoryReader.open(dir)) { + when(context.getIndexReader()).thenReturn(newReader); + assertThat(builder.toFilter(context, 0, 1), equalTo(query)); + } + + // numSlices > numShards + int numSlices = randomIntBetween(10, 100); + int numShards = randomIntBetween(1, 9); + Map numSliceMap = new HashMap<>(); + for (int i = 0; i < numSlices; i++) { + for (int j = 0; j < numShards; j++) { + SliceBuilder slice = new SliceBuilder("_uid", i, numSlices); + Query q = slice.toFilter(context, j, numShards); + if (q instanceof TermsSliceQuery || q instanceof MatchAllDocsQuery) { + AtomicInteger count = numSliceMap.get(j); + if (count == null) { + count = new AtomicInteger(0); + numSliceMap.put(j, count); + } + count.incrementAndGet(); + if (q instanceof MatchAllDocsQuery) { + assertThat(count.get(), equalTo(1)); + } + } else { + assertThat(q, instanceOf(MatchNoDocsQuery.class)); + } + } + } + int total = 0; + for (Map.Entry e : numSliceMap.entrySet()) { + total += e.getValue().get(); + } + assertThat(total, equalTo(numSlices)); + + // numShards > numSlices + numShards = randomIntBetween(4, 100); + numSlices = randomIntBetween(2, numShards-1); + List targetShards = new ArrayList<>(); + for (int i = 0; i < numSlices; i++) { + for (int j = 0; j < numShards; j++) { + SliceBuilder slice = new SliceBuilder("_uid", i, numSlices); + Query q = slice.toFilter(context, j, numShards); + if (q instanceof MatchNoDocsQuery == false) { + assertThat(q, instanceOf(MatchAllDocsQuery.class)); + targetShards.add(j); + } + } + } + assertThat(targetShards.size(), equalTo(numShards)); + assertThat(new HashSet<>(targetShards).size(), equalTo(numShards)); + + // numShards == numSlices + numShards = randomIntBetween(2, 10); + numSlices = numShards; + for (int i = 0; i < numSlices; i++) { + for (int j = 0; j < numShards; j++) { + SliceBuilder slice = new SliceBuilder("_uid", i, numSlices); + Query q = slice.toFilter(context, j, numShards); + if (i == j) { + assertThat(q, instanceOf(MatchAllDocsQuery.class)); + } else { + assertThat(q, instanceOf(MatchNoDocsQuery.class)); + } + } + } + } + + try (IndexReader reader = DirectoryReader.open(dir)) { + MappedFieldType fieldType = new MappedFieldType() { + @Override + public MappedFieldType clone() { + return null; + } + + @Override + public String typeName() { + return null; + } + + @Override + public Query termQuery(Object value, @Nullable QueryShardContext context) { + return null; + } + }; + fieldType.setName("field_without_doc_values"); + when(context.fieldMapper("field_without_doc_values")).thenReturn(fieldType); + when(context.getIndexReader()).thenReturn(reader); + SliceBuilder builder = new SliceBuilder("field_without_doc_values", 5, 10); + IllegalArgumentException exc = + expectThrows(IllegalArgumentException.class, () -> builder.toFilter(context, 0, 1)); + assertThat(exc.getMessage(), containsString("cannot load numeric doc values")); + } + } +} diff --git a/core/src/test/java/org/elasticsearch/search/slice/TermsSliceQueryTests.java b/core/src/test/java/org/elasticsearch/search/slice/TermsSliceQueryTests.java new file mode 100644 index 00000000000..e00dabc6363 --- /dev/null +++ b/core/src/test/java/org/elasticsearch/search/slice/TermsSliceQueryTests.java @@ -0,0 +1,117 @@ +/* + * 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.analysis.core.KeywordAnalyzer; +import org.apache.lucene.document.Document; +import org.apache.lucene.document.Field; +import org.apache.lucene.document.StringField; +import org.apache.lucene.index.IndexReader; +import org.apache.lucene.index.LeafReaderContext; +import org.apache.lucene.index.RandomIndexWriter; +import org.apache.lucene.search.IndexSearcher; +import org.apache.lucene.search.Collector; +import org.apache.lucene.search.LeafCollector; +import org.apache.lucene.search.Scorer; +import org.apache.lucene.search.QueryUtils; + +import org.apache.lucene.store.Directory; +import org.apache.lucene.util.BytesRef; +import org.elasticsearch.common.UUIDs; +import org.elasticsearch.test.ESTestCase; + +import java.io.IOException; +import java.util.Collections; +import java.util.HashSet; +import java.util.Set; + +import static org.hamcrest.Matchers.equalTo; + +public class TermsSliceQueryTests extends ESTestCase { + + public void testBasics() { + TermsSliceQuery query1 = + new TermsSliceQuery("field1", 1, 10); + TermsSliceQuery query2 = + new TermsSliceQuery("field1", 1, 10); + TermsSliceQuery query3 = + new TermsSliceQuery("field2", 1, 10); + TermsSliceQuery query4 = + new TermsSliceQuery("field1", 2, 10); + QueryUtils.check(query1); + QueryUtils.checkEqual(query1, query2); + QueryUtils.checkUnequal(query1, query3); + QueryUtils.checkUnequal(query1, query4); + } + + public void testSearch() throws Exception { + final int numDocs = randomIntBetween(100, 200); + final Directory dir = newDirectory(); + final RandomIndexWriter w = new RandomIndexWriter(random(), dir, new KeywordAnalyzer()); + int max = randomIntBetween(2, 10); + int[] sliceCounters = new int[max]; + Set keys = new HashSet<>(); + for (int i = 0; i < numDocs; ++i) { + Document doc = new Document(); + String uuid = UUIDs.base64UUID(); + BytesRef br = new BytesRef(uuid); + int id = Math.floorMod(br.hashCode(), max); + sliceCounters[id] ++; + doc.add(new StringField("uuid", uuid, Field.Store.YES)); + w.addDocument(doc); + keys.add(uuid); + } + final IndexReader reader = w.getReader(); + final IndexSearcher searcher = newSearcher(reader); + + for (int id = 0; id < max; id++) { + TermsSliceQuery query1 = + new TermsSliceQuery("uuid", id, max); + assertThat(searcher.count(query1), equalTo(sliceCounters[id])); + searcher.search(query1, new Collector() { + @Override + public LeafCollector getLeafCollector(LeafReaderContext context) throws IOException { + return new LeafCollector() { + @Override + public void setScorer(Scorer scorer) throws IOException { + } + + @Override + public void collect(int doc) throws IOException { + Document d = context.reader().document(doc, Collections.singleton("uuid")); + String uuid = d.get("uuid"); + assertThat(keys.contains(uuid), equalTo(true)); + keys.remove(uuid); + } + }; + } + + @Override + public boolean needsScores() { + return false; + } + }); + } + assertThat(keys.size(), equalTo(0)); + w.close(); + reader.close(); + dir.close(); + } +} diff --git a/core/src/test/java/org/elasticsearch/search/suggest/CompletionSuggestSearch2xIT.java b/core/src/test/java/org/elasticsearch/search/suggest/CompletionSuggestSearch2xIT.java index 812e43918d9..484edf61f7b 100644 --- a/core/src/test/java/org/elasticsearch/search/suggest/CompletionSuggestSearch2xIT.java +++ b/core/src/test/java/org/elasticsearch/search/suggest/CompletionSuggestSearch2xIT.java @@ -31,6 +31,7 @@ import org.elasticsearch.action.admin.indices.stats.IndicesStatsResponse; import org.elasticsearch.action.index.IndexRequestBuilder; import org.elasticsearch.action.search.SearchPhaseExecutionException; import org.elasticsearch.action.search.SearchResponse; +import org.elasticsearch.action.support.WriteRequest.RefreshPolicy; import org.elasticsearch.cluster.metadata.IndexMetaData; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.unit.Fuzziness; @@ -473,7 +474,7 @@ public class CompletionSuggestSearch2xIT extends ESIntegTestCase { .setSettings(Settings.builder().put(IndexMetaData.SETTING_VERSION_CREATED, PRE2X_VERSION.id)) .addMapping(TYPE, mapping)); client().prepareIndex(INDEX, TYPE, "1") - .setRefresh(true) + .setRefreshPolicy(RefreshPolicy.IMMEDIATE) .setSource(jsonBuilder().startObject().field(FIELD, "Foo Fighters").endObject()).get(); ensureGreen(INDEX); @@ -496,7 +497,7 @@ public class CompletionSuggestSearch2xIT extends ESIntegTestCase { ).execute().actionGet(); assertSuggestions(suggestResponse, "suggs"); - client().prepareIndex(INDEX, TYPE, "1").setRefresh(true) + client().prepareIndex(INDEX, TYPE, "1").setRefreshPolicy(RefreshPolicy.IMMEDIATE) .setSource(jsonBuilder().startObject().field(FIELD, "Foo Fighters").endObject()).get(); ensureGreen(INDEX); @@ -522,7 +523,7 @@ public class CompletionSuggestSearch2xIT extends ESIntegTestCase { .addMapping(TYPE, mapping) .setSettings(Settings.builder().put(IndexMetaData.SETTING_VERSION_CREATED, PRE2X_VERSION.id))); client().prepareIndex(INDEX, TYPE, "1") - .setRefresh(true) + .setRefreshPolicy(RefreshPolicy.IMMEDIATE) .setSource(jsonBuilder().startObject().field(FIELD, "Foo Fighters").endObject()).get(); ensureGreen(INDEX); @@ -545,7 +546,7 @@ public class CompletionSuggestSearch2xIT extends ESIntegTestCase { ).execute().actionGet(); assertSuggestions(suggestResponse, "suggs"); - client().prepareIndex(INDEX, TYPE, "1").setRefresh(true) + client().prepareIndex(INDEX, TYPE, "1").setRefreshPolicy(RefreshPolicy.IMMEDIATE) .setSource(jsonBuilder().startObject().field(FIELD, "Foo Fighters").endObject()).get(); ensureGreen(INDEX); @@ -731,10 +732,10 @@ public class CompletionSuggestSearch2xIT extends ESIntegTestCase { assertThat(putMappingResponse.isAcknowledged(), is(true)); // Index two entities - client().prepareIndex(INDEX, TYPE, "1").setRefresh(true) + client().prepareIndex(INDEX, TYPE, "1").setRefreshPolicy(RefreshPolicy.IMMEDIATE) .setSource(jsonBuilder().startObject().field(FIELD, "Foo Fighters").field(otherField, "WHATEVER").endObject()) .get(); - client().prepareIndex(INDEX, TYPE, "2").setRefresh(true) + client().prepareIndex(INDEX, TYPE, "2").setRefreshPolicy(RefreshPolicy.IMMEDIATE) .setSource(jsonBuilder().startObject().field(FIELD, "Bar Fighters").field(otherField, "WHATEVER2").endObject()) .get(); @@ -1040,7 +1041,7 @@ public class CompletionSuggestSearch2xIT extends ESIntegTestCase { .startArray("input").value(str).endArray() .field("output", "foobar") .endObject().endObject() - ).setRefresh(true).get(); + ).setRefreshPolicy(RefreshPolicy.IMMEDIATE).get(); // need to flush and refresh, because we keep changing the same document // we have to make sure that segments without any live documents are deleted flushAndRefresh(); @@ -1074,7 +1075,7 @@ public class CompletionSuggestSearch2xIT extends ESIntegTestCase { .startArray("input").value(longString).endArray() .field("output", "foobar") .endObject().endObject() - ).setRefresh(true).get(); + ).setRefreshPolicy(RefreshPolicy.IMMEDIATE).get(); } @@ -1096,7 +1097,7 @@ public class CompletionSuggestSearch2xIT extends ESIntegTestCase { .startArray("input").value(string).endArray() .field("output", "foobar") .endObject().endObject() - ).setRefresh(true).get(); + ).setRefreshPolicy(RefreshPolicy.IMMEDIATE).get(); fail("expected MapperParsingException"); } catch (MapperParsingException expected) {} } @@ -1116,7 +1117,7 @@ public class CompletionSuggestSearch2xIT extends ESIntegTestCase { .startObject() .field(FIELD, string) .endObject() - ).setRefresh(true).get(); + ).setRefreshPolicy(RefreshPolicy.IMMEDIATE).get(); try { client().prepareSearch(INDEX).addAggregation(AggregationBuilders.terms("suggest_agg").field(FIELD) @@ -1148,11 +1149,11 @@ public class CompletionSuggestSearch2xIT extends ESIntegTestCase { ensureGreen(); client().prepareIndex(INDEX, TYPE, "1").setSource(FIELD, "strings make me happy", FIELD + "_1", "nulls make me sad") - .setRefresh(true).get(); + .setRefreshPolicy(RefreshPolicy.IMMEDIATE).get(); try { client().prepareIndex(INDEX, TYPE, "2").setSource(FIELD, null, FIELD + "_1", "nulls make me sad") - .setRefresh(true).get(); + .setRefreshPolicy(RefreshPolicy.IMMEDIATE).get(); fail("Expected MapperParsingException for null value"); } catch (MapperParsingException e) { // make sure that the exception has the name of the field causing the error diff --git a/core/src/test/java/org/elasticsearch/search/suggest/CompletionSuggestSearchIT.java b/core/src/test/java/org/elasticsearch/search/suggest/CompletionSuggestSearchIT.java index 9ef3c898cab..0c6179e0693 100644 --- a/core/src/test/java/org/elasticsearch/search/suggest/CompletionSuggestSearchIT.java +++ b/core/src/test/java/org/elasticsearch/search/suggest/CompletionSuggestSearchIT.java @@ -64,6 +64,7 @@ import java.util.List; import java.util.Locale; import java.util.Map; +import static org.elasticsearch.action.support.WriteRequest.RefreshPolicy.IMMEDIATE; import static org.elasticsearch.cluster.metadata.IndexMetaData.SETTING_NUMBER_OF_REPLICAS; import static org.elasticsearch.cluster.metadata.IndexMetaData.SETTING_NUMBER_OF_SHARDS; import static org.elasticsearch.common.util.CollectionUtils.iterableAsArrayList; @@ -622,7 +623,8 @@ public class CompletionSuggestSearchIT extends ESIntegTestCase { .endObject() .endObject(); assertAcked(prepareCreate(INDEX).addMapping(TYPE, mapping)); - client().prepareIndex(INDEX, TYPE, "1").setRefresh(true).setSource(jsonBuilder().startObject().field(FIELD, "Foo Fighters").endObject()).get(); + client().prepareIndex(INDEX, TYPE, "1").setRefreshPolicy(IMMEDIATE) + .setSource(jsonBuilder().startObject().field(FIELD, "Foo Fighters").endObject()).get(); ensureGreen(INDEX); PutMappingResponse putMappingResponse = client().admin().indices().preparePutMapping(INDEX).setType(TYPE).setSource(jsonBuilder().startObject() @@ -643,7 +645,8 @@ public class CompletionSuggestSearchIT extends ESIntegTestCase { ).execute().actionGet(); assertSuggestions(searchResponse, "suggs"); - client().prepareIndex(INDEX, TYPE, "1").setRefresh(true).setSource(jsonBuilder().startObject().field(FIELD, "Foo Fighters").endObject()).get(); + client().prepareIndex(INDEX, TYPE, "1").setRefreshPolicy(IMMEDIATE) + .setSource(jsonBuilder().startObject().field(FIELD, "Foo Fighters").endObject()).get(); ensureGreen(INDEX); SearchResponse afterReindexingResponse = client().prepareSearch(INDEX).suggest( @@ -1089,7 +1092,7 @@ public class CompletionSuggestSearchIT extends ESIntegTestCase { .startObject().startObject(FIELD) .startArray("input").value(longString).endArray() .endObject().endObject() - ).setRefresh(true).get(); + ).setRefreshPolicy(IMMEDIATE).get(); } @@ -1111,7 +1114,7 @@ public class CompletionSuggestSearchIT extends ESIntegTestCase { .startArray("input").value(string).endArray() .field("output", "foobar") .endObject().endObject() - ).setRefresh(true).get(); + ).get(); fail("Expected MapperParsingException"); } catch (MapperParsingException e) { assertThat(e.getMessage(), containsString("failed to parse")); @@ -1133,7 +1136,7 @@ public class CompletionSuggestSearchIT extends ESIntegTestCase { .startObject() .field(FIELD, string) .endObject() - ).setRefresh(true).get(); + ).setRefreshPolicy(IMMEDIATE).get(); try { client().prepareSearch(INDEX).addAggregation(AggregationBuilders.terms("suggest_agg").field(FIELD) @@ -1163,11 +1166,10 @@ public class CompletionSuggestSearchIT extends ESIntegTestCase { ensureGreen(); client().prepareIndex(INDEX, TYPE, "1").setSource(FIELD, "strings make me happy", FIELD + "_1", "nulls make me sad") - .setRefresh(true).get(); + .setRefreshPolicy(IMMEDIATE).get(); try { - client().prepareIndex(INDEX, TYPE, "2").setSource(FIELD, null, FIELD + "_1", "nulls make me sad") - .setRefresh(true).get(); + client().prepareIndex(INDEX, TYPE, "2").setSource(FIELD, null, FIELD + "_1", "nulls make me sad").get(); fail("Expected MapperParsingException for null value"); } catch (MapperParsingException e) { // make sure that the exception has the name of the field causing the error diff --git a/core/src/test/java/org/elasticsearch/search/suggest/ContextSuggestSearch2xIT.java b/core/src/test/java/org/elasticsearch/search/suggest/ContextSuggestSearch2xIT.java index 9152fe61847..f5726622f62 100644 --- a/core/src/test/java/org/elasticsearch/search/suggest/ContextSuggestSearch2xIT.java +++ b/core/src/test/java/org/elasticsearch/search/suggest/ContextSuggestSearch2xIT.java @@ -53,6 +53,7 @@ import java.util.Map; import java.util.Set; import static com.carrotsearch.randomizedtesting.RandomizedTest.getRandom; +import static org.elasticsearch.action.support.WriteRequest.RefreshPolicy.IMMEDIATE; import static org.elasticsearch.common.xcontent.XContentFactory.jsonBuilder; import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertAcked; import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertSuggestion; @@ -504,7 +505,7 @@ public class ContextSuggestSearch2xIT extends ESIntegTestCase { ensureGreen(); client().prepareIndex(INDEX, TYPE, "1").setSource(FIELD, "") - .setRefresh(true).get(); + .setRefreshPolicy(IMMEDIATE).get(); } diff --git a/core/src/test/java/org/elasticsearch/search/suggest/CustomSuggesterSearchIT.java b/core/src/test/java/org/elasticsearch/search/suggest/CustomSuggesterSearchIT.java index 4f97a5179ed..28f8ec49a57 100644 --- a/core/src/test/java/org/elasticsearch/search/suggest/CustomSuggesterSearchIT.java +++ b/core/src/test/java/org/elasticsearch/search/suggest/CustomSuggesterSearchIT.java @@ -45,6 +45,7 @@ import java.util.Locale; import java.util.Map; import java.util.Objects; +import static org.elasticsearch.action.support.WriteRequest.RefreshPolicy.IMMEDIATE; import static org.elasticsearch.common.xcontent.XContentFactory.jsonBuilder; import static org.hamcrest.Matchers.hasSize; import static org.hamcrest.Matchers.is; @@ -65,7 +66,7 @@ public class CustomSuggesterSearchIT extends ESIntegTestCase { .startObject() .field("name", "arbitrary content") .endObject()) - .setRefresh(true).execute().actionGet(); + .setRefreshPolicy(IMMEDIATE).get(); ensureYellow(); String randomText = randomAsciiOfLength(10); diff --git a/core/src/test/java/org/elasticsearch/similarity/SimilarityIT.java b/core/src/test/java/org/elasticsearch/similarity/SimilarityIT.java index a123f7133b3..66c3509c76d 100644 --- a/core/src/test/java/org/elasticsearch/similarity/SimilarityIT.java +++ b/core/src/test/java/org/elasticsearch/similarity/SimilarityIT.java @@ -23,6 +23,7 @@ import org.elasticsearch.action.search.SearchResponse; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.test.ESIntegTestCase; +import static org.elasticsearch.action.support.WriteRequest.RefreshPolicy.IMMEDIATE; import static org.elasticsearch.common.xcontent.XContentFactory.jsonBuilder; import static org.elasticsearch.index.query.QueryBuilders.matchQuery; import static org.hamcrest.Matchers.equalTo; @@ -61,7 +62,7 @@ public class SimilarityIT extends ESIntegTestCase { client().prepareIndex("test", "type1", "1").setSource("field1", "the quick brown fox jumped over the lazy dog", "field2", "the quick brown fox jumped over the lazy dog") - .setRefresh(true).execute().actionGet(); + .setRefreshPolicy(IMMEDIATE).execute().actionGet(); SearchResponse bm25SearchResponse = client().prepareSearch().setQuery(matchQuery("field1", "quick brown fox")).execute().actionGet(); assertThat(bm25SearchResponse.getHits().totalHits(), equalTo(1L)); diff --git a/core/src/test/java/org/elasticsearch/threadpool/FixedThreadPoolTests.java b/core/src/test/java/org/elasticsearch/threadpool/FixedThreadPoolTests.java index 85daebb86d5..48ea8b6c8c9 100644 --- a/core/src/test/java/org/elasticsearch/threadpool/FixedThreadPoolTests.java +++ b/core/src/test/java/org/elasticsearch/threadpool/FixedThreadPoolTests.java @@ -19,7 +19,6 @@ package org.elasticsearch.threadpool; -import org.elasticsearch.common.settings.ClusterSettings; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.util.concurrent.EsExecutors; import org.elasticsearch.common.util.concurrent.EsRejectedExecutionException; @@ -42,8 +41,8 @@ public class FixedThreadPoolTests extends ESThreadPoolTestCase { final Settings nodeSettings = Settings.builder() .put("node.name", "testRejectedExecutionCounter") - .put("threadpool." + threadPoolName + ".size", size) - .put("threadpool." + threadPoolName + ".queue_size", queueSize) + .put("thread_pool." + threadPoolName + ".size", size) + .put("thread_pool." + threadPoolName + ".queue_size", queueSize) .build(); try { threadPool = new ThreadPool(nodeSettings); @@ -86,18 +85,6 @@ public class FixedThreadPoolTests extends ESThreadPoolTestCase { assertThat(counter, equalTo(rejections)); assertThat(stats(threadPool, threadPoolName).getRejected(), equalTo(rejections)); - - // the rejected execution count resets to zero when the - // queue is resized - final ClusterSettings clusterSettings = - new ClusterSettings(nodeSettings, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS); - threadPool.setClusterSettings(clusterSettings); - clusterSettings.applySettings( - Settings.builder() - .put("threadpool." + threadPoolName + ".queue_size", queueSize + 1) - .build()); - assertThat(stats(threadPool, threadPoolName).getRejected(), equalTo(0L)); - } finally { terminateThreadPoolIfNeeded(threadPool); } diff --git a/core/src/test/java/org/elasticsearch/threadpool/ScalingThreadPoolTests.java b/core/src/test/java/org/elasticsearch/threadpool/ScalingThreadPoolTests.java index 2212f162eb6..d065abb884c 100644 --- a/core/src/test/java/org/elasticsearch/threadpool/ScalingThreadPoolTests.java +++ b/core/src/test/java/org/elasticsearch/threadpool/ScalingThreadPoolTests.java @@ -40,35 +40,35 @@ public class ScalingThreadPoolTests extends ESThreadPoolTestCase { final String threadPoolName = randomThreadPool(ThreadPool.ThreadPoolType.SCALING); final Settings.Builder builder = Settings.builder(); - final int min; + final int core; if (randomBoolean()) { - min = randomIntBetween(0, 8); - builder.put("threadpool." + threadPoolName + ".min", min); + core = randomIntBetween(0, 8); + builder.put("thread_pool." + threadPoolName + ".core", core); } else { - min = "generic".equals(threadPoolName) ? 4 : 1; // the defaults + core = "generic".equals(threadPoolName) ? 4 : 1; // the defaults } - final int sizeBasedOnNumberOfProcessors; + final int maxBasedOnNumberOfProcessors; if (randomBoolean()) { final int processors = randomIntBetween(1, 64); - sizeBasedOnNumberOfProcessors = expectedSize(threadPoolName, processors); + maxBasedOnNumberOfProcessors = expectedSize(threadPoolName, processors); builder.put("processors", processors); } else { - sizeBasedOnNumberOfProcessors = expectedSize(threadPoolName, Math.min(32, Runtime.getRuntime().availableProcessors())); + maxBasedOnNumberOfProcessors = expectedSize(threadPoolName, Math.min(32, Runtime.getRuntime().availableProcessors())); } - final int expectedSize; - if (sizeBasedOnNumberOfProcessors < min || randomBoolean()) { - expectedSize = randomIntBetween(Math.max(1, min), 16); - builder.put("threadpool." + threadPoolName + ".size", expectedSize); + final int expectedMax; + if (maxBasedOnNumberOfProcessors < core || randomBoolean()) { + expectedMax = randomIntBetween(Math.max(1, core), 16); + builder.put("thread_pool." + threadPoolName + ".max", expectedMax); } else { - expectedSize = sizeBasedOnNumberOfProcessors; + expectedMax = maxBasedOnNumberOfProcessors; } final long keepAlive; if (randomBoolean()) { keepAlive = randomIntBetween(1, 300); - builder.put("threadpool." + threadPoolName + ".keep_alive", keepAlive + "s"); + builder.put("thread_pool." + threadPoolName + ".keep_alive", keepAlive + "s"); } else { keepAlive = "generic".equals(threadPoolName) ? 30 : 300; // the defaults } @@ -88,10 +88,10 @@ public class ScalingThreadPoolTests extends ESThreadPoolTestCase { assertNull(info.getQueueSize()); assertThat(esThreadPoolExecutor.getQueue().remainingCapacity(), equalTo(Integer.MAX_VALUE)); - assertThat(info.getMin(), equalTo(min)); - assertThat(esThreadPoolExecutor.getCorePoolSize(), equalTo(min)); - assertThat(info.getMax(), equalTo(expectedSize)); - assertThat(esThreadPoolExecutor.getMaximumPoolSize(), equalTo(expectedSize)); + assertThat(info.getMin(), equalTo(core)); + assertThat(esThreadPoolExecutor.getCorePoolSize(), equalTo(core)); + assertThat(info.getMax(), equalTo(expectedMax)); + assertThat(esThreadPoolExecutor.getMaximumPoolSize(), equalTo(expectedMax)); }); } @@ -113,23 +113,10 @@ public class ScalingThreadPoolTests extends ESThreadPoolTestCase { return sizes.get(threadPoolName).size(numberOfProcessors); } - public void testValidDynamicKeepAlive() throws InterruptedException { - final String threadPoolName = randomThreadPool(ThreadPool.ThreadPoolType.SCALING); - runScalingThreadPoolTest(Settings.EMPTY, (clusterSettings, threadPool) -> { - final Executor beforeExecutor = threadPool.executor(threadPoolName); - final long seconds = randomIntBetween(1, 300); - clusterSettings.applySettings(settings("threadpool." + threadPoolName + ".keep_alive", seconds + "s")); - final Executor afterExecutor = threadPool.executor(threadPoolName); - assertSame(beforeExecutor, afterExecutor); - final ThreadPool.Info info = info(threadPool, threadPoolName); - assertThat(info.getKeepAlive().seconds(), equalTo(seconds)); - }); - } - public void testScalingThreadPoolIsBounded() throws InterruptedException { final String threadPoolName = randomThreadPool(ThreadPool.ThreadPoolType.SCALING); final int size = randomIntBetween(32, 512); - final Settings settings = Settings.builder().put("threadpool." + threadPoolName + ".size", size).build(); + final Settings settings = Settings.builder().put("thread_pool." + threadPoolName + ".max", size).build(); runScalingThreadPoolTest(settings, (clusterSettings, threadPool) -> { final CountDownLatch latch = new CountDownLatch(1); final int numberOfTasks = 2 * size; @@ -161,8 +148,8 @@ public class ScalingThreadPoolTests extends ESThreadPoolTestCase { final int min = "generic".equals(threadPoolName) ? 4 : 1; final Settings settings = Settings.builder() - .put("threadpool." + threadPoolName + ".size", 128) - .put("threadpool." + threadPoolName + ".keep_alive", "1ms") + .put("thread_pool." + threadPoolName + ".max", 128) + .put("thread_pool." + threadPoolName + ".keep_alive", "1ms") .build(); runScalingThreadPoolTest(settings, ((clusterSettings, threadPool) -> { final CountDownLatch latch = new CountDownLatch(1); @@ -197,40 +184,6 @@ public class ScalingThreadPoolTests extends ESThreadPoolTestCase { })); } - public void testDynamicThreadPoolSize() throws InterruptedException { - final String threadPoolName = randomThreadPool(ThreadPool.ThreadPoolType.SCALING); - runScalingThreadPoolTest(Settings.EMPTY, (clusterSettings, threadPool) -> { - final Executor beforeExecutor = threadPool.executor(threadPoolName); - int expectedMin = "generic".equals(threadPoolName) ? 4 : 1; - final int size = randomIntBetween(expectedMin, Integer.MAX_VALUE); - clusterSettings.applySettings(settings("threadpool." + threadPoolName + ".size", size)); - final Executor afterExecutor = threadPool.executor(threadPoolName); - assertSame(beforeExecutor, afterExecutor); - final ThreadPool.Info info = info(threadPool, threadPoolName); - assertThat(info.getMin(), equalTo(expectedMin)); - assertThat(info.getMax(), equalTo(size)); - - assertThat(afterExecutor, instanceOf(EsThreadPoolExecutor.class)); - final EsThreadPoolExecutor executor = (EsThreadPoolExecutor)afterExecutor; - assertThat(executor.getCorePoolSize(), equalTo(expectedMin)); - assertThat(executor.getMaximumPoolSize(), equalTo(size)); - }); - } - - public void testResizingScalingThreadPoolQueue() throws InterruptedException { - final String threadPoolName = randomThreadPool(ThreadPool.ThreadPoolType.SCALING); - runScalingThreadPoolTest(Settings.EMPTY, (clusterSettings, threadPool) -> { - final int size = randomIntBetween(1, Integer.MAX_VALUE); - final IllegalArgumentException e = expectThrows( - IllegalArgumentException.class, - () -> clusterSettings.applySettings(settings("threadpool." + threadPoolName + ".queue_size", size))); - assertThat(e, hasToString( - "java.lang.IllegalArgumentException: thread pool [" + threadPoolName + - "] of type scaling can not have its queue re-sized but was [" + - size + "]")); - }); - } - public void runScalingThreadPoolTest( final Settings settings, final BiConsumer consumer) throws InterruptedException { @@ -240,7 +193,6 @@ public class ScalingThreadPoolTests extends ESThreadPoolTestCase { final Settings nodeSettings = Settings.builder().put(settings).put("node.name", test).build(); threadPool = new ThreadPool(nodeSettings); final ClusterSettings clusterSettings = new ClusterSettings(nodeSettings, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS); - threadPool.setClusterSettings(clusterSettings); consumer.accept(clusterSettings, threadPool); } finally { terminateThreadPoolIfNeeded(threadPool); diff --git a/core/src/test/java/org/elasticsearch/threadpool/SimpleThreadPoolIT.java b/core/src/test/java/org/elasticsearch/threadpool/SimpleThreadPoolIT.java index baf017bc7b9..28267e9beb7 100644 --- a/core/src/test/java/org/elasticsearch/threadpool/SimpleThreadPoolIT.java +++ b/core/src/test/java/org/elasticsearch/threadpool/SimpleThreadPoolIT.java @@ -125,63 +125,6 @@ public class SimpleThreadPoolIT extends ESIntegTestCase { } } - public void testUpdatingThreadPoolSettings() throws Exception { - internalCluster().startNodesAsync(2).get(); - ThreadPool threadPool = internalCluster().getDataNodeInstance(ThreadPool.class); - // Check that settings are changed - assertThat(((ThreadPoolExecutor) threadPool.executor(Names.SEARCH)).getQueue().remainingCapacity(), equalTo(1000)); - client().admin().cluster().prepareUpdateSettings().setTransientSettings(Settings.builder().put("threadpool.search.queue_size", 2000).build()).execute().actionGet(); - assertThat(((ThreadPoolExecutor) threadPool.executor(Names.SEARCH)).getQueue().remainingCapacity(), equalTo(2000)); - - // Make sure that threads continue executing when executor is replaced - final CyclicBarrier barrier = new CyclicBarrier(2); - Executor oldExecutor = threadPool.executor(Names.SEARCH); - threadPool.executor(Names.SEARCH).execute(() -> { - try { - barrier.await(); - } catch (InterruptedException ex) { - Thread.currentThread().interrupt(); - } catch (BrokenBarrierException ex) { - // - } - }); - client().admin().cluster().prepareUpdateSettings().setTransientSettings(Settings.builder().put("threadpool.search.queue_size", 1000).build()).execute().actionGet(); - assertThat(threadPool.executor(Names.SEARCH), not(sameInstance(oldExecutor))); - assertThat(((ThreadPoolExecutor) oldExecutor).isShutdown(), equalTo(true)); - assertThat(((ThreadPoolExecutor) oldExecutor).isTerminating(), equalTo(true)); - assertThat(((ThreadPoolExecutor) oldExecutor).isTerminated(), equalTo(false)); - barrier.await(10, TimeUnit.SECONDS); - - // Make sure that new thread executor is functional - threadPool.executor(Names.SEARCH).execute(() -> { - try { - barrier.await(); - } catch (InterruptedException ex) { - Thread.currentThread().interrupt(); - } catch (BrokenBarrierException ex) { - // - } - } - ); - client().admin().cluster().prepareUpdateSettings().setTransientSettings(Settings.builder().put("threadpool.search.queue_size", 500)).execute().actionGet(); - barrier.await(10, TimeUnit.SECONDS); - - // Check that node info is correct - NodesInfoResponse nodesInfoResponse = client().admin().cluster().prepareNodesInfo().all().execute().actionGet(); - assertEquals(2, nodesInfoResponse.getNodes().size()); - for (NodeInfo nodeInfo : nodesInfoResponse.getNodes()) { - boolean found = false; - for (ThreadPool.Info info : nodeInfo.getThreadPool()) { - if (info.getName().equals(Names.SEARCH)) { - assertEquals(info.getThreadPoolType(), ThreadPool.ThreadPoolType.FIXED); - found = true; - break; - } - } - assertThat(found, equalTo(true)); - } - } - public void testThreadPoolLeakingThreadsWithTribeNode() { Settings settings = Settings.builder() .put("node.name", "thread_pool_leaking_threads_tribe_node") diff --git a/core/src/test/java/org/elasticsearch/threadpool/ThreadPoolSerializationTests.java b/core/src/test/java/org/elasticsearch/threadpool/ThreadPoolSerializationTests.java index 19dd25a7e25..486b0635c64 100644 --- a/core/src/test/java/org/elasticsearch/threadpool/ThreadPoolSerializationTests.java +++ b/core/src/test/java/org/elasticsearch/threadpool/ThreadPoolSerializationTests.java @@ -96,7 +96,7 @@ public class ThreadPoolSerializationTests extends ESTestCase { } public void testThatNegativeSettingAllowsToStart() throws InterruptedException { - Settings settings = Settings.builder().put("node.name", "index").put("threadpool.index.queue_size", "-1").build(); + Settings settings = Settings.builder().put("node.name", "index").put("thread_pool.index.queue_size", "-1").build(); ThreadPool threadPool = new ThreadPool(settings); assertThat(threadPool.info("index").getQueueSize(), is(nullValue())); terminate(threadPool); diff --git a/core/src/test/java/org/elasticsearch/threadpool/UpdateThreadPoolSettingsTests.java b/core/src/test/java/org/elasticsearch/threadpool/UpdateThreadPoolSettingsTests.java index 43e8e7e7af5..87accf057ad 100644 --- a/core/src/test/java/org/elasticsearch/threadpool/UpdateThreadPoolSettingsTests.java +++ b/core/src/test/java/org/elasticsearch/threadpool/UpdateThreadPoolSettingsTests.java @@ -19,32 +19,24 @@ package org.elasticsearch.threadpool; -import org.elasticsearch.common.settings.ClusterSettings; 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.EsThreadPoolExecutor; import org.elasticsearch.threadpool.ThreadPool.Names; import java.lang.reflect.Field; -import java.util.Arrays; -import java.util.HashSet; -import java.util.Map; import java.util.Set; import java.util.concurrent.CountDownLatch; -import java.util.concurrent.Executor; import java.util.concurrent.ThreadPoolExecutor; import java.util.concurrent.TimeUnit; -import java.util.stream.Collectors; +import static org.hamcrest.CoreMatchers.containsString; import static org.hamcrest.Matchers.equalTo; +import static org.hamcrest.Matchers.hasToString; import static org.hamcrest.Matchers.instanceOf; import static org.hamcrest.Matchers.is; -import static org.hamcrest.Matchers.not; -import static org.hamcrest.Matchers.nullValue; -import static org.hamcrest.Matchers.sameInstance; -/** - */ public class UpdateThreadPoolSettingsTests extends ESThreadPoolTestCase { public void testCorrectThreadPoolTypePermittedInSettings() throws InterruptedException { @@ -53,12 +45,12 @@ public class UpdateThreadPoolSettingsTests extends ESThreadPoolTestCase { ThreadPool threadPool = null; try { threadPool = new ThreadPool(Settings.builder() - .put("node.name", "testCorrectThreadPoolTypePermittedInSettings") - .put("threadpool." + threadPoolName + ".type", correctThreadPoolType.getType()) - .build()); + .put("node.name", "testCorrectThreadPoolTypePermittedInSettings") + .put("thread_pool." + threadPoolName + ".type", correctThreadPoolType.getType()) + .build()); ThreadPool.Info info = info(threadPool, threadPoolName); if (ThreadPool.Names.SAME.equals(threadPoolName)) { - assertNull(info); // we don't report on the "same" threadpool + assertNull(info); // we don't report on the "same" thread pool } else { // otherwise check we have the expected type assertEquals(info.getThreadPoolType(), correctThreadPoolType); @@ -68,97 +60,31 @@ public class UpdateThreadPoolSettingsTests extends ESThreadPoolTestCase { } } - public void testThreadPoolCanNotOverrideThreadPoolType() throws InterruptedException { - String threadPoolName = randomThreadPoolName(); - ThreadPool.ThreadPoolType incorrectThreadPoolType = randomIncorrectThreadPoolType(threadPoolName); - ThreadPool.ThreadPoolType correctThreadPoolType = ThreadPool.THREAD_POOL_TYPES.get(threadPoolName); - ThreadPool threadPool = null; - try { - threadPool = new ThreadPool( - Settings.builder() - .put("node.name", "testThreadPoolCanNotOverrideThreadPoolType") - .put("threadpool." + threadPoolName + ".type", incorrectThreadPoolType.getType()) - .build()); - terminate(threadPool); - fail("expected IllegalArgumentException"); - } catch (IllegalArgumentException e) { - assertThat( - e.getMessage(), - is("setting threadpool." + threadPoolName + ".type to " + incorrectThreadPoolType.getType() + " is not permitted; must be " + correctThreadPoolType.getType())); - } finally { - terminateThreadPoolIfNeeded(threadPool); - } - } - public void testIndexingThreadPoolsMaxSize() throws InterruptedException { - String threadPoolName = randomThreadPoolName(); - for (String name : new String[] {ThreadPool.Names.BULK, ThreadPool.Names.INDEX}) { - ThreadPool threadPool = null; - try { + final String name = randomFrom(Names.BULK, Names.INDEX); + final int maxSize = 1 + EsExecutors.boundedNumberOfProcessors(Settings.EMPTY); + final int tooBig = randomIntBetween(1 + maxSize, Integer.MAX_VALUE); - int maxSize = EsExecutors.boundedNumberOfProcessors(Settings.EMPTY); + // try to create a too big thread pool + final IllegalArgumentException initial = + expectThrows( + IllegalArgumentException.class, + () -> { + ThreadPool tp = null; + try { + tp = new ThreadPool(Settings.builder() + .put("node.name", "testIndexingThreadPoolsMaxSize") + .put("thread_pool." + name + ".size", tooBig) + .build()); + } finally { + terminateThreadPoolIfNeeded(tp); + } + }); - // try to create a too-big (maxSize+1) thread pool - threadPool = new ThreadPool(Settings.builder() - .put("node.name", "testIndexingThreadPoolsMaxSize") - .put("threadpool." + name + ".size", maxSize+1) - .build()); - - // confirm it clipped us at the maxSize: - assertEquals(maxSize, ((ThreadPoolExecutor) threadPool.executor(name)).getMaximumPoolSize()); - - ClusterSettings clusterSettings = new ClusterSettings(Settings.EMPTY, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS); - threadPool.setClusterSettings(clusterSettings); - - // update it to a tiny size: - clusterSettings.applySettings( - Settings.builder() - .put("threadpool." + name + ".size", 1) - .build() - ); - - // confirm it worked: - assertEquals(1, ((ThreadPoolExecutor) threadPool.executor(name)).getMaximumPoolSize()); - - // try to update to too-big size: - clusterSettings.applySettings( - Settings.builder() - .put("threadpool." + name + ".size", maxSize+1) - .build() - ); - - // confirm it clipped us at the maxSize: - assertEquals(maxSize, ((ThreadPoolExecutor) threadPool.executor(name)).getMaximumPoolSize()); - } finally { - terminateThreadPoolIfNeeded(threadPool); - } - } - } - - public void testUpdateSettingsCanNotChangeThreadPoolType() throws InterruptedException { - String threadPoolName = randomThreadPoolName(); - ThreadPool.ThreadPoolType invalidThreadPoolType = randomIncorrectThreadPoolType(threadPoolName); - ThreadPool.ThreadPoolType validThreadPoolType = ThreadPool.THREAD_POOL_TYPES.get(threadPoolName); - ThreadPool threadPool = null; - try { - threadPool = new ThreadPool(Settings.builder().put("node.name", "testUpdateSettingsCanNotChangeThreadPoolType").build()); - ClusterSettings clusterSettings = new ClusterSettings(Settings.EMPTY, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS); - threadPool.setClusterSettings(clusterSettings); - - clusterSettings.applySettings( - Settings.builder() - .put("threadpool." + threadPoolName + ".type", invalidThreadPoolType.getType()) - .build() - ); - fail("expected IllegalArgumentException"); - } catch (IllegalArgumentException e) { - assertEquals("illegal value can't update [threadpool.] from [{}] to [{" + threadPoolName + ".type=" + invalidThreadPoolType.getType() + "}]", e.getMessage()); - assertThat( - e.getCause().getMessage(), - is("setting threadpool." + threadPoolName + ".type to " + invalidThreadPoolType.getType() + " is not permitted; must be " + validThreadPoolType.getType())); - } finally { - terminateThreadPoolIfNeeded(threadPool); - } + assertThat( + initial, + hasToString(containsString( + "Failed to parse value [" + tooBig + "] for setting [thread_pool." + name + ".size] must be "))); } private static int getExpectedThreadPoolSize(Settings settings, String name, int size) { @@ -174,17 +100,14 @@ public class UpdateThreadPoolSettingsTests extends ESThreadPoolTestCase { ThreadPool threadPool = null; try { + int expectedSize = getExpectedThreadPoolSize(Settings.EMPTY, threadPoolName, 15); Settings nodeSettings = Settings.builder() - .put("node.name", "testFixedExecutorType").build(); + .put("node.name", "testFixedExecutorType") + .put("thread_pool." + threadPoolName + ".size", expectedSize) + .build(); threadPool = new ThreadPool(nodeSettings); - ClusterSettings clusterSettings = new ClusterSettings(nodeSettings, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS); - threadPool.setClusterSettings(clusterSettings); assertThat(threadPool.executor(threadPoolName), instanceOf(EsThreadPoolExecutor.class)); - Settings settings = clusterSettings.applySettings(Settings.builder() - .put("threadpool." + threadPoolName + ".size", "15") - .build()); - int expectedSize = getExpectedThreadPoolSize(nodeSettings, threadPoolName, 15); assertEquals(info(threadPool, threadPoolName).getThreadPoolType(), ThreadPool.ThreadPoolType.FIXED); assertThat(threadPool.executor(threadPoolName), instanceOf(EsThreadPoolExecutor.class)); assertThat(((EsThreadPoolExecutor) threadPool.executor(threadPoolName)).getCorePoolSize(), equalTo(expectedSize)); @@ -193,37 +116,6 @@ public class UpdateThreadPoolSettingsTests extends ESThreadPoolTestCase { assertThat(info(threadPool, threadPoolName).getMax(), equalTo(expectedSize)); // keep alive does not apply to fixed thread pools assertThat(((EsThreadPoolExecutor) threadPool.executor(threadPoolName)).getKeepAliveTime(TimeUnit.MINUTES), equalTo(0L)); - - // Put old type back - settings = clusterSettings.applySettings(Settings.EMPTY); - assertEquals(info(threadPool, threadPoolName).getThreadPoolType(), ThreadPool.ThreadPoolType.FIXED); - // Make sure keep alive value is not used - assertThat(info(threadPool, threadPoolName).getKeepAlive(), nullValue()); - // Make sure keep pool size value were reused - assertThat(info(threadPool, threadPoolName).getMin(), equalTo(expectedSize)); - assertThat(info(threadPool, threadPoolName).getMax(), equalTo(expectedSize)); - assertThat(threadPool.executor(threadPoolName), instanceOf(EsThreadPoolExecutor.class)); - assertThat(((EsThreadPoolExecutor) threadPool.executor(threadPoolName)).getCorePoolSize(), equalTo(expectedSize)); - assertThat(((EsThreadPoolExecutor) threadPool.executor(threadPoolName)).getMaximumPoolSize(), equalTo(expectedSize)); - - // Change size - Executor oldExecutor = threadPool.executor(threadPoolName); - settings = clusterSettings.applySettings(Settings.builder().put(settings).put("threadpool." + threadPoolName + ".size", "10").build()); - - expectedSize = getExpectedThreadPoolSize(nodeSettings, threadPoolName, 10); - - // Make sure size values changed - assertThat(info(threadPool, threadPoolName).getMax(), equalTo(expectedSize)); - assertThat(info(threadPool, threadPoolName).getMin(), equalTo(expectedSize)); - assertThat(((EsThreadPoolExecutor) threadPool.executor(threadPoolName)).getMaximumPoolSize(), equalTo(expectedSize)); - assertThat(((EsThreadPoolExecutor) threadPool.executor(threadPoolName)).getCorePoolSize(), equalTo(expectedSize)); - // Make sure executor didn't change - assertEquals(info(threadPool, threadPoolName).getThreadPoolType(), ThreadPool.ThreadPoolType.FIXED); - assertThat(threadPool.executor(threadPoolName), sameInstance(oldExecutor)); - - // Change queue capacity - clusterSettings.applySettings(Settings.builder().put(settings).put("threadpool." + threadPoolName + ".queue", "500") - .build()); } finally { terminateThreadPoolIfNeeded(threadPool); } @@ -234,11 +126,10 @@ public class UpdateThreadPoolSettingsTests extends ESThreadPoolTestCase { ThreadPool threadPool = null; try { Settings nodeSettings = Settings.builder() - .put("threadpool." + threadPoolName + ".size", 10) - .put("node.name", "testScalingExecutorType").build(); + .put("thread_pool." + threadPoolName + ".max", 10) + .put("node.name", "testScalingExecutorType") + .build(); threadPool = new ThreadPool(nodeSettings); - ClusterSettings clusterSettings = new ClusterSettings(nodeSettings, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS); - threadPool.setClusterSettings(clusterSettings); final int expectedMinimum = "generic".equals(threadPoolName) ? 4 : 1; assertThat(info(threadPool, threadPoolName).getMin(), equalTo(expectedMinimum)); assertThat(info(threadPool, threadPoolName).getMax(), equalTo(10)); @@ -246,24 +137,6 @@ public class UpdateThreadPoolSettingsTests extends ESThreadPoolTestCase { assertThat(info(threadPool, threadPoolName).getKeepAlive().seconds(), equalTo(expectedKeepAlive)); assertEquals(info(threadPool, threadPoolName).getThreadPoolType(), ThreadPool.ThreadPoolType.SCALING); assertThat(threadPool.executor(threadPoolName), instanceOf(EsThreadPoolExecutor.class)); - - // Change settings that doesn't require pool replacement - Executor oldExecutor = threadPool.executor(threadPoolName); - clusterSettings.applySettings(Settings.builder() - .put("threadpool." + threadPoolName + ".keep_alive", "10m") - .put("threadpool." + threadPoolName + ".min", "2") - .put("threadpool." + threadPoolName + ".size", "15") - .build()); - assertEquals(info(threadPool, threadPoolName).getThreadPoolType(), ThreadPool.ThreadPoolType.SCALING); - assertThat(threadPool.executor(threadPoolName), instanceOf(EsThreadPoolExecutor.class)); - assertThat(((EsThreadPoolExecutor) threadPool.executor(threadPoolName)).getCorePoolSize(), equalTo(2)); - assertThat(((EsThreadPoolExecutor) threadPool.executor(threadPoolName)).getMaximumPoolSize(), equalTo(15)); - assertThat(info(threadPool, threadPoolName).getMin(), equalTo(2)); - assertThat(info(threadPool, threadPoolName).getMax(), equalTo(15)); - // Make sure keep alive value changed - assertThat(info(threadPool, threadPoolName).getKeepAlive().minutes(), equalTo(10L)); - assertThat(((EsThreadPoolExecutor) threadPool.executor(threadPoolName)).getKeepAliveTime(TimeUnit.MINUTES), equalTo(10L)); - assertThat(threadPool.executor(threadPoolName), sameInstance(oldExecutor)); } finally { terminateThreadPoolIfNeeded(threadPool); } @@ -274,17 +147,18 @@ public class UpdateThreadPoolSettingsTests extends ESThreadPoolTestCase { ThreadPool threadPool = null; try { Settings nodeSettings = Settings.builder() - .put("threadpool." + threadPoolName + ".queue_size", 1000) - .put("node.name", "testShutdownNowInterrupts").build(); + .put("thread_pool." + threadPoolName + ".queue_size", 1000) + .put("node.name", "testShutdownNowInterrupts") + .build(); threadPool = new ThreadPool(nodeSettings); - ClusterSettings clusterSettings = new ClusterSettings(nodeSettings, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS); - threadPool.setClusterSettings(clusterSettings); assertEquals(info(threadPool, threadPoolName).getQueueSize().getSingles(), 1000L); + final CountDownLatch shutDownLatch = new CountDownLatch(1); final CountDownLatch latch = new CountDownLatch(1); ThreadPoolExecutor oldExecutor = (ThreadPoolExecutor) threadPool.executor(threadPoolName); threadPool.executor(threadPoolName).execute(() -> { try { + shutDownLatch.countDown(); new CountDownLatch(1).await(); } catch (InterruptedException ex) { latch.countDown(); @@ -292,13 +166,11 @@ public class UpdateThreadPoolSettingsTests extends ESThreadPoolTestCase { } } ); - clusterSettings.applySettings(Settings.builder().put("threadpool." + threadPoolName + ".queue_size", 2000).build()); - assertThat(threadPool.executor(threadPoolName), not(sameInstance(oldExecutor))); + shutDownLatch.await(); + threadPool.shutdownNow(); + latch.await(3, TimeUnit.SECONDS); // if this throws then ThreadPool#shutdownNow did not interrupt assertThat(oldExecutor.isShutdown(), equalTo(true)); - assertThat(oldExecutor.isTerminating(), equalTo(true)); - assertThat(oldExecutor.isTerminated(), equalTo(false)); - threadPool.shutdownNow(); // should interrupt the thread - latch.await(3, TimeUnit.SECONDS); // If this throws then ThreadPool#shutdownNow didn't interrupt + assertThat(oldExecutor.isTerminating() || oldExecutor.isTerminated(), equalTo(true)); } finally { terminateThreadPoolIfNeeded(threadPool); } @@ -307,18 +179,19 @@ public class UpdateThreadPoolSettingsTests extends ESThreadPoolTestCase { public void testCustomThreadPool() throws Exception { ThreadPool threadPool = null; try { - Settings nodeSettings = Settings.builder() - .put("threadpool.my_pool1.type", "scaling") - .put("threadpool.my_pool1.min", 1) - .put("threadpool.my_pool1.size", EsExecutors.boundedNumberOfProcessors(Settings.EMPTY)) - .put("threadpool.my_pool1.keep_alive", "1m") - .put("threadpool.my_pool2.type", "fixed") - .put("threadpool.my_pool2.size", "1") - .put("threadpool.my_pool2.queue_size", "1") - .put("node.name", "testCustomThreadPool").build(); - threadPool = new ThreadPool(nodeSettings); - ClusterSettings clusterSettings = new ClusterSettings(nodeSettings, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS); - threadPool.setClusterSettings(clusterSettings); + + + final ScalingExecutorBuilder scaling = + new ScalingExecutorBuilder( + "my_pool1", + 1, + EsExecutors.boundedNumberOfProcessors(Settings.EMPTY), + TimeValue.timeValueMinutes(1)); + + final FixedExecutorBuilder fixed = new FixedExecutorBuilder(Settings.EMPTY, "my_pool2", 1, 1); + + threadPool = new ThreadPool(Settings.builder().put("node.name", "testCustomThreadPool").build(), scaling, fixed); + ThreadPoolInfo groups = threadPool.info(); boolean foundPool1 = false; boolean foundPool2 = false; @@ -345,39 +218,6 @@ public class UpdateThreadPoolSettingsTests extends ESThreadPoolTestCase { } assertThat(foundPool1, is(true)); assertThat(foundPool2, is(true)); - - // Updating my_pool2 - Settings settings = Settings.builder() - .put("threadpool.my_pool2.size", "10") - .build(); - clusterSettings.applySettings(settings); - - groups = threadPool.info(); - foundPool1 = false; - foundPool2 = false; - outer: - for (ThreadPool.Info info : groups) { - if ("my_pool1".equals(info.getName())) { - foundPool1 = true; - assertEquals(info.getThreadPoolType(), ThreadPool.ThreadPoolType.SCALING); - } else if ("my_pool2".equals(info.getName())) { - foundPool2 = true; - assertThat(info.getMax(), equalTo(10)); - assertThat(info.getMin(), equalTo(10)); - assertThat(info.getQueueSize().singles(), equalTo(1L)); - assertEquals(info.getThreadPoolType(), ThreadPool.ThreadPoolType.FIXED); - } else { - for (Field field : Names.class.getFields()) { - if (info.getName().equalsIgnoreCase(field.getName())) { - // This is ok it is a default thread pool - continue outer; - } - } - fail("Unexpected pool name: " + info.getName()); - } - } - assertThat(foundPool1, is(true)); - assertThat(foundPool2, is(true)); } finally { terminateThreadPoolIfNeeded(threadPool); } @@ -388,11 +228,4 @@ public class UpdateThreadPoolSettingsTests extends ESThreadPoolTestCase { return randomFrom(threadPoolNames.toArray(new String[threadPoolNames.size()])); } - private ThreadPool.ThreadPoolType randomIncorrectThreadPoolType(String threadPoolName) { - Set set = new HashSet<>(); - set.addAll(Arrays.asList(ThreadPool.ThreadPoolType.values())); - set.remove(ThreadPool.THREAD_POOL_TYPES.get(threadPoolName)); - return randomFrom(set.toArray(new ThreadPool.ThreadPoolType[set.size()])); - } - } diff --git a/core/src/test/java/org/elasticsearch/timestamp/SimpleTimestampIT.java b/core/src/test/java/org/elasticsearch/timestamp/SimpleTimestampIT.java index 21408e7b1df..2fa94bc2357 100644 --- a/core/src/test/java/org/elasticsearch/timestamp/SimpleTimestampIT.java +++ b/core/src/test/java/org/elasticsearch/timestamp/SimpleTimestampIT.java @@ -29,6 +29,7 @@ import org.elasticsearch.test.ESIntegTestCase; import java.util.Locale; +import static org.elasticsearch.action.support.WriteRequest.RefreshPolicy.IMMEDIATE; import static org.elasticsearch.common.xcontent.XContentFactory.jsonBuilder; import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertAcked; import static org.hamcrest.Matchers.equalTo; @@ -48,7 +49,7 @@ public class SimpleTimestampIT extends ESIntegTestCase { logger.info("--> check with automatic timestamp"); long now1 = System.currentTimeMillis(); - client().prepareIndex("test", "type1", "1").setSource("field1", "value1").setRefresh(true).execute().actionGet(); + client().prepareIndex("test", "type1", "1").setSource("field1", "value1").setRefreshPolicy(IMMEDIATE).get(); long now2 = System.currentTimeMillis(); // we check both realtime get and non realtime get @@ -70,7 +71,7 @@ public class SimpleTimestampIT extends ESIntegTestCase { assertThat(((Number) getResponse.getField("_timestamp").getValue()).longValue(), equalTo(timestamp)); logger.info("--> check with custom timestamp (numeric)"); - client().prepareIndex("test", "type1", "1").setSource("field1", "value1").setTimestamp("10").setRefresh(true).execute().actionGet(); + client().prepareIndex("test", "type1", "1").setSource("field1", "value1").setTimestamp("10").setRefreshPolicy(IMMEDIATE).get(); getResponse = client().prepareGet("test", "type1", "1").setFields("_timestamp").setRealtime(false).execute().actionGet(); timestamp = ((Number) getResponse.getField("_timestamp").getValue()).longValue(); @@ -80,7 +81,8 @@ public class SimpleTimestampIT extends ESIntegTestCase { assertThat(((Number) getResponse.getField("_timestamp").getValue()).longValue(), equalTo(timestamp)); logger.info("--> check with custom timestamp (string)"); - client().prepareIndex("test", "type1", "1").setSource("field1", "value1").setTimestamp("1970-01-01T00:00:00.020").setRefresh(true).execute().actionGet(); + client().prepareIndex("test", "type1", "1").setSource("field1", "value1").setTimestamp("1970-01-01T00:00:00.020") + .setRefreshPolicy(IMMEDIATE).get(); getResponse = client().prepareGet("test", "type1", "1").setFields("_timestamp").setRealtime(false).execute().actionGet(); timestamp = ((Number) getResponse.getField("_timestamp").getValue()).longValue(); diff --git a/core/src/test/java/org/elasticsearch/transport/AbstractSimpleTransportTestCase.java b/core/src/test/java/org/elasticsearch/transport/AbstractSimpleTransportTestCase.java index f285a1db52e..68c4c76d7e8 100644 --- a/core/src/test/java/org/elasticsearch/transport/AbstractSimpleTransportTestCase.java +++ b/core/src/test/java/org/elasticsearch/transport/AbstractSimpleTransportTestCase.java @@ -31,6 +31,7 @@ import org.elasticsearch.common.unit.TimeValue; import org.elasticsearch.test.ESTestCase; import org.elasticsearch.test.junit.annotations.TestLogging; import org.elasticsearch.test.transport.MockTransportService; +import org.elasticsearch.threadpool.TestThreadPool; import org.elasticsearch.threadpool.ThreadPool; import org.junit.After; import org.junit.Before; @@ -71,7 +72,7 @@ public abstract class AbstractSimpleTransportTestCase extends ESTestCase { @Before public void setUp() throws Exception { super.setUp(); - threadPool = new ThreadPool(getClass().getName()); + threadPool = new TestThreadPool(getClass().getName()); serviceA = build( Settings.builder() .put("name", "TS_A") diff --git a/core/src/test/java/org/elasticsearch/transport/NettySizeHeaderFrameDecoderTests.java b/core/src/test/java/org/elasticsearch/transport/NettySizeHeaderFrameDecoderTests.java index ca5faf68b51..3c382c39509 100644 --- a/core/src/test/java/org/elasticsearch/transport/NettySizeHeaderFrameDecoderTests.java +++ b/core/src/test/java/org/elasticsearch/transport/NettySizeHeaderFrameDecoderTests.java @@ -64,7 +64,6 @@ public class NettySizeHeaderFrameDecoderTests extends ESTestCase { @Before public void startThreadPool() { threadPool = new ThreadPool(settings); - threadPool.setClusterSettings(new ClusterSettings(settings, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS)); NetworkService networkService = new NetworkService(settings); BigArrays bigArrays = new MockBigArrays(Settings.EMPTY, new NoneCircuitBreakerService()); nettyTransport = new NettyTransport(settings, threadPool, networkService, bigArrays, Version.CURRENT, new NamedWriteableRegistry(), diff --git a/core/src/test/java/org/elasticsearch/transport/NettyTransportServiceHandshakeTests.java b/core/src/test/java/org/elasticsearch/transport/NettyTransportServiceHandshakeTests.java index 6a7e0a0de36..a111911e994 100644 --- a/core/src/test/java/org/elasticsearch/transport/NettyTransportServiceHandshakeTests.java +++ b/core/src/test/java/org/elasticsearch/transport/NettyTransportServiceHandshakeTests.java @@ -30,6 +30,7 @@ import org.elasticsearch.indices.breaker.NoneCircuitBreakerService; import org.elasticsearch.test.ESTestCase; import org.elasticsearch.test.VersionUtils; import org.elasticsearch.test.transport.MockTransportService; +import org.elasticsearch.threadpool.TestThreadPool; import org.elasticsearch.threadpool.ThreadPool; import org.elasticsearch.transport.netty.NettyTransport; import org.junit.After; @@ -51,7 +52,7 @@ public class NettyTransportServiceHandshakeTests extends ESTestCase { @BeforeClass public static void startThreadPool() { - threadPool = new ThreadPool(NettyTransportServiceHandshakeTests.class.getSimpleName()); + threadPool = new TestThreadPool(NettyTransportServiceHandshakeTests.class.getSimpleName()); } private List transportServices = new ArrayList<>(); diff --git a/core/src/test/java/org/elasticsearch/transport/netty/NettyScheduledPingTests.java b/core/src/test/java/org/elasticsearch/transport/netty/NettyScheduledPingTests.java index 8c00ae01b74..de156659082 100644 --- a/core/src/test/java/org/elasticsearch/transport/netty/NettyScheduledPingTests.java +++ b/core/src/test/java/org/elasticsearch/transport/netty/NettyScheduledPingTests.java @@ -30,6 +30,7 @@ import org.elasticsearch.indices.breaker.CircuitBreakerService; import org.elasticsearch.indices.breaker.NoneCircuitBreakerService; import org.elasticsearch.test.ESTestCase; import org.elasticsearch.test.transport.MockTransportService; +import org.elasticsearch.threadpool.TestThreadPool; import org.elasticsearch.threadpool.ThreadPool; import org.elasticsearch.transport.BaseTransportResponseHandler; import org.elasticsearch.transport.TransportChannel; @@ -52,7 +53,7 @@ import static org.hamcrest.Matchers.greaterThan; */ public class NettyScheduledPingTests extends ESTestCase { public void testScheduledPing() throws Exception { - ThreadPool threadPool = new ThreadPool(getClass().getName()); + ThreadPool threadPool = new TestThreadPool(getClass().getName()); Settings settings = Settings.builder() .put(NettyTransport.PING_SCHEDULE.getKey(), "5ms") diff --git a/core/src/test/java/org/elasticsearch/transport/netty/NettyTransportMultiPortTests.java b/core/src/test/java/org/elasticsearch/transport/netty/NettyTransportMultiPortTests.java index 352e7bb5a59..0ed0cf3ec97 100644 --- a/core/src/test/java/org/elasticsearch/transport/netty/NettyTransportMultiPortTests.java +++ b/core/src/test/java/org/elasticsearch/transport/netty/NettyTransportMultiPortTests.java @@ -28,6 +28,7 @@ import org.elasticsearch.common.util.BigArrays; import org.elasticsearch.common.util.MockBigArrays; import org.elasticsearch.indices.breaker.NoneCircuitBreakerService; import org.elasticsearch.test.ESTestCase; +import org.elasticsearch.threadpool.TestThreadPool; import org.elasticsearch.threadpool.ThreadPool; import org.elasticsearch.transport.TransportService; import org.elasticsearch.transport.TransportSettings; @@ -56,7 +57,7 @@ public class NettyTransportMultiPortTests extends ESTestCase { .put("transport.profiles.client1.port", 0) .build(); - ThreadPool threadPool = new ThreadPool("tst"); + ThreadPool threadPool = new TestThreadPool("tst"); try (NettyTransport transport = startNettyTransport(settings, threadPool)) { assertEquals(1, transport.profileBoundAddresses().size()); assertEquals(1, transport.boundAddress().boundAddresses().length); @@ -72,7 +73,7 @@ public class NettyTransportMultiPortTests extends ESTestCase { .put("transport.profiles.client1.port", 0) .build(); - ThreadPool threadPool = new ThreadPool("tst"); + ThreadPool threadPool = new TestThreadPool("tst"); try (NettyTransport transport = startNettyTransport(settings, threadPool)) { assertEquals(1, transport.profileBoundAddresses().size()); assertEquals(1, transport.boundAddress().boundAddresses().length); @@ -89,7 +90,7 @@ public class NettyTransportMultiPortTests extends ESTestCase { .put("transport.profiles.client1.whatever", "foo") .build(); - ThreadPool threadPool = new ThreadPool("tst"); + ThreadPool threadPool = new TestThreadPool("tst"); try (NettyTransport transport = startNettyTransport(settings, threadPool)) { assertEquals(0, transport.profileBoundAddresses().size()); assertEquals(1, transport.boundAddress().boundAddresses().length); @@ -105,7 +106,7 @@ public class NettyTransportMultiPortTests extends ESTestCase { .put("transport.profiles.default.port", 0) .build(); - ThreadPool threadPool = new ThreadPool("tst"); + ThreadPool threadPool = new TestThreadPool("tst"); try (NettyTransport transport = startNettyTransport(settings, threadPool)) { assertEquals(0, transport.profileBoundAddresses().size()); assertEquals(1, transport.boundAddress().boundAddresses().length); @@ -123,7 +124,7 @@ public class NettyTransportMultiPortTests extends ESTestCase { .put("transport.profiles..port", 23) // will not actually bind to this .build(); - ThreadPool threadPool = new ThreadPool("tst"); + ThreadPool threadPool = new TestThreadPool("tst"); try (NettyTransport transport = startNettyTransport(settings, threadPool)) { assertEquals(0, transport.profileBoundAddresses().size()); assertEquals(1, transport.boundAddress().boundAddresses().length); diff --git a/core/src/test/java/org/elasticsearch/ttl/SimpleTTLIT.java b/core/src/test/java/org/elasticsearch/ttl/SimpleTTLIT.java index f6a74faac39..003c08134bc 100644 --- a/core/src/test/java/org/elasticsearch/ttl/SimpleTTLIT.java +++ b/core/src/test/java/org/elasticsearch/ttl/SimpleTTLIT.java @@ -38,6 +38,7 @@ import java.util.Locale; import java.util.Map; import java.util.concurrent.TimeUnit; +import static org.elasticsearch.action.support.WriteRequest.RefreshPolicy.IMMEDIATE; import static org.elasticsearch.common.xcontent.XContentFactory.jsonBuilder; import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertAcked; import static org.hamcrest.Matchers.both; @@ -92,10 +93,10 @@ public class SimpleTTLIT extends ESIntegTestCase { // Index one doc without routing, one doc with routing, one doc with not TTL and no default and one doc with default TTL long now = System.currentTimeMillis(); IndexResponse indexResponse = client().prepareIndex("test", "type1", "1").setSource("field1", "value1") - .setTimestamp(String.valueOf(now)).setTTL(providedTTLValue).setRefresh(true).get(); + .setTimestamp(String.valueOf(now)).setTTL(providedTTLValue).setRefreshPolicy(IMMEDIATE).get(); assertThat(indexResponse.isCreated(), is(true)); indexResponse = client().prepareIndex("test", "type1", "with_routing").setSource("field1", "value1") - .setTimestamp(String.valueOf(now)).setTTL(providedTTLValue).setRouting("routing").setRefresh(true).get(); + .setTimestamp(String.valueOf(now)).setTTL(providedTTLValue).setRouting("routing").setRefreshPolicy(IMMEDIATE).get(); assertThat(indexResponse.isCreated(), is(true)); indexResponse = client().prepareIndex("test", "type1", "no_ttl").setSource("field1", "value1").get(); assertThat(indexResponse.isCreated(), is(true)); @@ -245,7 +246,7 @@ public class SimpleTTLIT extends ESIntegTestCase { long secondTtl = aLongTime * 2; long thirdTtl = aLongTime * 1; IndexResponse indexResponse = client().prepareIndex("test", "type1", "1").setSource("field1", "value1") - .setTTL(firstTtl).setRefresh(true).get(); + .setTTL(firstTtl).setRefreshPolicy(IMMEDIATE).get(); assertTrue(indexResponse.isCreated()); assertThat(getTtl("type1", 1), both(lessThanOrEqualTo(firstTtl)).and(greaterThan(secondTtl))); diff --git a/core/src/test/java/org/elasticsearch/update/UpdateIT.java b/core/src/test/java/org/elasticsearch/update/UpdateIT.java index 0445da61096..078bcd8cc3f 100644 --- a/core/src/test/java/org/elasticsearch/update/UpdateIT.java +++ b/core/src/test/java/org/elasticsearch/update/UpdateIT.java @@ -63,6 +63,7 @@ import java.util.concurrent.CountDownLatch; import java.util.concurrent.Semaphore; import java.util.concurrent.TimeUnit; +import static org.elasticsearch.action.support.WriteRequest.RefreshPolicy.IMMEDIATE; import static org.elasticsearch.common.xcontent.XContentFactory.jsonBuilder; import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertAcked; import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertThrows; @@ -694,7 +695,7 @@ public class UpdateIT extends ESIntegTestCase { } // check TTL is kept after an update without TTL - client().prepareIndex("test", "type1", "2").setSource("field", 1).setTTL(86400000L).setRefresh(true).execute().actionGet(); + client().prepareIndex("test", "type1", "2").setSource("field", 1).setTTL(86400000L).setRefreshPolicy(IMMEDIATE).get(); GetResponse getResponse = client().prepareGet("test", "type1", "2").setFields("_ttl").execute().actionGet(); long ttl = ((Number) getResponse.getField("_ttl").getValue()).longValue(); assertThat(ttl, greaterThan(0L)); @@ -713,7 +714,7 @@ public class UpdateIT extends ESIntegTestCase { assertThat(ttl, lessThanOrEqualTo(3600000L)); // check timestamp update - client().prepareIndex("test", "type1", "3").setSource("field", 1).setRefresh(true).execute().actionGet(); + client().prepareIndex("test", "type1", "3").setSource("field", 1).setRefreshPolicy(IMMEDIATE).get(); client().prepareUpdate(indexOrAlias(), "type1", "3") .setScript(new Script("", ScriptService.ScriptType.INLINE, "put_values", Collections.singletonMap("_ctx", Collections.singletonMap("_timestamp", "2009-11-15T14:12:12")))).execute() .actionGet(); diff --git a/core/src/test/java/org/elasticsearch/watcher/ResourceWatcherServiceTests.java b/core/src/test/java/org/elasticsearch/watcher/ResourceWatcherServiceTests.java index 6c6c45e9cfd..82a3a55868a 100644 --- a/core/src/test/java/org/elasticsearch/watcher/ResourceWatcherServiceTests.java +++ b/core/src/test/java/org/elasticsearch/watcher/ResourceWatcherServiceTests.java @@ -21,6 +21,7 @@ package org.elasticsearch.watcher; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.test.ESTestCase; +import org.elasticsearch.threadpool.TestThreadPool; import org.elasticsearch.threadpool.ThreadPool; import static org.elasticsearch.common.unit.TimeValue.timeValueSeconds; @@ -33,7 +34,7 @@ import static org.hamcrest.Matchers.notNullValue; */ public class ResourceWatcherServiceTests extends ESTestCase { public void testSettings() throws Exception { - ThreadPool threadPool = new ThreadPool("test"); + ThreadPool threadPool = new TestThreadPool("test"); // checking the defaults Settings settings = Settings.builder().build(); @@ -65,7 +66,7 @@ public class ResourceWatcherServiceTests extends ESTestCase { } public void testHandle() throws Exception { - ThreadPool threadPool = new ThreadPool("test"); + ThreadPool threadPool = new TestThreadPool("test"); Settings settings = Settings.builder().build(); ResourceWatcherService service = new ResourceWatcherService(settings, threadPool); ResourceWatcher watcher = new ResourceWatcher() { diff --git a/distribution/src/main/resources/bin/elasticsearch-plugin.bat b/distribution/src/main/resources/bin/elasticsearch-plugin.bat index 5604b57dbef..ba35ad1c214 100644 --- a/distribution/src/main/resources/bin/elasticsearch-plugin.bat +++ b/distribution/src/main/resources/bin/elasticsearch-plugin.bat @@ -2,7 +2,15 @@ SETLOCAL enabledelayedexpansion -if NOT DEFINED JAVA_HOME goto err +IF DEFINED JAVA_HOME ( + set JAVA=%JAVA_HOME%\bin\java.exe +) ELSE ( + FOR %%I IN (java.exe) DO set JAVA=%%~$PATH:I +) +IF NOT EXIST "%JAVA%" ( + ECHO Could not find any executable java binary. Please install java in your PATH or set JAVA_HOME 1>&2 + EXIT /B 1 +) set SCRIPT_DIR=%~dp0 for %%I in ("%SCRIPT_DIR%..") do set ES_HOME=%%~dpfI @@ -48,15 +56,6 @@ GOTO loop SET HOSTNAME=%COMPUTERNAME% -"%JAVA_HOME%\bin\java" %ES_JAVA_OPTS% -Des.path.home="%ES_HOME%" !properties! -cp "%ES_HOME%/lib/*;" "org.elasticsearch.plugins.PluginCli" !args! -goto finally - - -:err -echo JAVA_HOME environment variable must be set! -pause - - -:finally +"%JAVA%" %ES_JAVA_OPTS% -Des.path.home="%ES_HOME%" !properties! -cp "%ES_HOME%/lib/*;" "org.elasticsearch.plugins.PluginCli" !args! ENDLOCAL diff --git a/distribution/src/main/resources/bin/elasticsearch.bat b/distribution/src/main/resources/bin/elasticsearch.bat index 9f1d871d0a2..37d7fc026a8 100644 --- a/distribution/src/main/resources/bin/elasticsearch.bat +++ b/distribution/src/main/resources/bin/elasticsearch.bat @@ -35,14 +35,14 @@ FOR /F "usebackq tokens=1* delims= " %%A IN (!params!) DO ( SET current=%%A SET params='%%B' SET silent=N - + IF "!current!" == "-s" ( SET silent=Y ) IF "!current!" == "--silent" ( SET silent=Y - ) - + ) + IF "!silent!" == "Y" ( SET nopauseonerror=Y ) ELSE ( @@ -52,7 +52,7 @@ FOR /F "usebackq tokens=1* delims= " %%A IN (!params!) DO ( SET newparams=!current! ) ) - + IF "x!params!" NEQ "x" ( GOTO loop ) @@ -79,6 +79,6 @@ IF ERRORLEVEL 1 ( EXIT /B %ERRORLEVEL% ) -"%JAVA_HOME%\bin\java" %ES_JAVA_OPTS% %ES_PARAMS% -cp "%ES_CLASSPATH%" "org.elasticsearch.bootstrap.Elasticsearch" !newparams! +"%JAVA%" %ES_JAVA_OPTS% %ES_PARAMS% -cp "%ES_CLASSPATH%" "org.elasticsearch.bootstrap.Elasticsearch" !newparams! ENDLOCAL diff --git a/distribution/src/main/resources/bin/elasticsearch.in.bat b/distribution/src/main/resources/bin/elasticsearch.in.bat index d6f1ef308bd..2d73ed2b57b 100644 --- a/distribution/src/main/resources/bin/elasticsearch.in.bat +++ b/distribution/src/main/resources/bin/elasticsearch.in.bat @@ -1,12 +1,15 @@ @echo off -if DEFINED JAVA_HOME goto cont +IF DEFINED JAVA_HOME ( + set JAVA=%JAVA_HOME%\bin\java.exe +) ELSE ( + FOR %%I IN (java.exe) DO set JAVA=%%~$PATH:I +) +IF NOT EXIST "%JAVA%" ( + ECHO Could not find any executable java binary. Please install java in your PATH or set JAVA_HOME 1>&2 + EXIT /B 1 +) -:err -ECHO JAVA_HOME environment variable must be set! 1>&2 -EXIT /B 1 - -:cont set SCRIPT_DIR=%~dp0 for %%I in ("%SCRIPT_DIR%..") do set ES_HOME=%%~dpfI diff --git a/distribution/src/main/resources/bin/service.bat b/distribution/src/main/resources/bin/service.bat index 81b6c8a5df5..344782030ee 100644 --- a/distribution/src/main/resources/bin/service.bat +++ b/distribution/src/main/resources/bin/service.bat @@ -28,27 +28,38 @@ if %bad_env_var% == 1 ( ) rem end TODO: remove for Elasticsearch 6.x -if NOT DEFINED JAVA_HOME goto err +IF DEFINED JAVA_HOME ( + SET JAVA=%JAVA_HOME%\bin\java.exe +) ELSE ( + FOR %%I IN (java.exe) DO set JAVA=%%~$PATH:I +) +IF NOT EXIST "%JAVA%" ( + ECHO Could not find any executable java binary. Please install java in your PATH or set JAVA_HOME 1>&2 + EXIT /B 1 +) +IF DEFINED JAVA_HOME GOTO :cont +IF NOT "%JAVA:~-13%" == "\bin\java.exe" ( + FOR /f "tokens=2 delims=[]" %%I IN ('dir %JAVA%') DO @set JAVA=%%I +) +IF "%JAVA:~-13%" == "\bin\java.exe" ( + SET JAVA_HOME=%JAVA:~0,-13% +) + +:cont if not "%CONF_FILE%" == "" goto conffileset set SCRIPT_DIR=%~dp0 for %%I in ("%SCRIPT_DIR%..") do set ES_HOME=%%~dpfI -rem Detect JVM version to figure out appropriate executable to use -if not exist "%JAVA_HOME%\bin\java.exe" ( -echo JAVA_HOME points to an invalid Java installation (no java.exe found in "%JAVA_HOME%"^). Exiting... -goto:eof -) - -"%JAVA_HOME%\bin\java" -Xmx50M -version > nul 2>&1 +"%JAVA%" -Xmx50M -version > nul 2>&1 if errorlevel 1 ( echo Warning: Could not start JVM to detect version, defaulting to x86: goto x86 ) -"%JAVA_HOME%\bin\java" -Xmx50M -version 2>&1 | "%windir%\System32\find" "64-Bit" >nul: +"%JAVA%" -Xmx50M -version 2>&1 | "%windir%\System32\find" "64-Bit" >nul: if errorlevel 1 goto x86 set EXECUTABLE=%ES_HOME%\bin\elasticsearch-service-x64.exe diff --git a/distribution/src/main/resources/config/jvm.options b/distribution/src/main/resources/config/jvm.options index b27b0a0a0f0..fecb7e00493 100644 --- a/distribution/src/main/resources/config/jvm.options +++ b/distribution/src/main/resources/config/jvm.options @@ -33,7 +33,6 @@ ################################################################ ## GC configuration --XX:+UseParNewGC -XX:+UseConcMarkSweepGC -XX:CMSInitiatingOccupancyFraction=75 -XX:+UseCMSInitiatingOccupancyOnly diff --git a/docs/plugins/ingest-attachment.asciidoc b/docs/plugins/ingest-attachment.asciidoc index 290bfabd2ff..a5544844441 100644 --- a/docs/plugins/ingest-attachment.asciidoc +++ b/docs/plugins/ingest-attachment.asciidoc @@ -11,6 +11,36 @@ the overhead of converting back and forth between base64, you can use the CBOR format instead of JSON and specify the field as a bytes array instead of a string representation. The processor will skip the base64 decoding then. +[[ingest-attachment-install]] +[float] +==== Installation + +This plugin can be installed using the plugin manager: + +[source,sh] +---------------------------------------------------------------- +sudo bin/elasticsearch-plugin install ingest-attachment +---------------------------------------------------------------- + +The plugin must be installed on every node in the cluster, and each node must +be restarted after installation. + +[[ingest-attachment-remove]] +[float] +==== Removal + +The plugin can be removed with the following command: + +[source,sh] +---------------------------------------------------------------- +sudo bin/elasticsearch-plugin remove ingest-attachment +---------------------------------------------------------------- + +The node must be stopped before removing the plugin. + +[[using-ingest-attachment]] +==== Using the Attachment Processor in a Pipeline + [[ingest-attachment-options]] .Attachment options [options="header"] diff --git a/docs/plugins/ingest-geoip.asciidoc b/docs/plugins/ingest-geoip.asciidoc index 1a555090fdc..434d713f945 100644 --- a/docs/plugins/ingest-geoip.asciidoc +++ b/docs/plugins/ingest-geoip.asciidoc @@ -8,17 +8,47 @@ The ingest-geoip plugin ships by default with the GeoLite2 City and GeoLite2 Cou under the CCA-ShareAlike 3.0 license. For more details see, http://dev.maxmind.com/geoip/geoip2/geolite2/ The GeoIP processor can run with other geoip2 databases from Maxmind. The files must be copied into the geoip config directory, -and the `database_file` option should be used to specify the filename of the custom database. The geoip config directory -is located at `$ES_HOME/config/ingest/geoip` and holds the shipped databases too. +and the `database_file` option should be used to specify the filename of the custom database. Custom database files must be compressed +with gzip. The geoip config directory is located at `$ES_HOME/config/ingest/geoip` and holds the shipped databases too. -[[geoip-options]] +[[ingest-geoip-install]] +[float] +==== Installation + +This plugin can be installed using the plugin manager: + +[source,sh] +---------------------------------------------------------------- +sudo bin/elasticsearch-plugin install ingest-geoip +---------------------------------------------------------------- + +The plugin must be installed on every node in the cluster, and each node must +be restarted after installation. + +[[ingest-geoip-remove]] +[float] +==== Removal + +The plugin can be removed with the following command: + +[source,sh] +---------------------------------------------------------------- +sudo bin/elasticsearch-plugin remove ingest-geoip +---------------------------------------------------------------- + +The node must be stopped before removing the plugin. + +[[using-ingest-geoip]] +==== Using the Geoip Processor in a Pipeline + +[[ingest-geoip-options]] .Geoip options [options="header"] |====== | Name | Required | Default | Description | `field` | yes | - | The field to get the ip address from for the geographical lookup. | `target_field` | no | geoip | The field that will hold the geographical information looked up from the Maxmind database. -| `database_file` | no | GeoLite2-City.mmdb | The database filename in the geoip config directory. The ingest-geoip plugin ships with the GeoLite2-City.mmdb and GeoLite2-Country.mmdb files. +| `database_file` | no | GeoLite2-City.mmdb | The database filename in the geoip config directory. The ingest-geoip plugin ships with the GeoLite2-City.mmdb.gz and GeoLite2-Country.mmdb.gz files. | `properties` | no | [`continent_name`, `country_iso_code`, `region_name`, `city_name`, `location`] * | Controls what properties are added to the `target_field` based on the geoip lookup. |====== diff --git a/docs/reference/aggregations.asciidoc b/docs/reference/aggregations.asciidoc index f3f24e8704c..ada134238bb 100644 --- a/docs/reference/aggregations.asciidoc +++ b/docs/reference/aggregations.asciidoc @@ -23,6 +23,11 @@ it is often easier to break them into three main families: <>:: Aggregations that keep track and compute metrics over a set of documents. +<>:: + A family of aggregations that operate on multiple fields and produce a matrix result based on the + values extracted from the requested document fields. Unlike metric and bucket aggregations, this + aggregation family does not yet support scripting. + <>:: Aggregations that aggregate the output of other aggregations and their associated metrics @@ -100,4 +105,6 @@ include::aggregations/bucket.asciidoc[] include::aggregations/pipeline.asciidoc[] +include::aggregations/matrix.asciidoc[] + include::aggregations/misc.asciidoc[] diff --git a/docs/reference/modules/aggregations-matrix.asciidoc b/docs/reference/aggregations/matrix.asciidoc similarity index 78% rename from docs/reference/modules/aggregations-matrix.asciidoc rename to docs/reference/aggregations/matrix.asciidoc index e8f741e7df6..e0d3d489ce1 100644 --- a/docs/reference/modules/aggregations-matrix.asciidoc +++ b/docs/reference/aggregations/matrix.asciidoc @@ -1,4 +1,4 @@ -[[modules-aggregations-matrix]] +[[search-aggregations-matrix]] == Matrix Aggregations experimental[] @@ -6,4 +6,4 @@ experimental[] The aggregations in this family operate on multiple fields and produce a matrix result based on the values extracted from the requested document fields. Unlike metric and bucket aggregations, this aggregation family does not yet support scripting. -include::aggregations/matrix/stats.asciidoc[] +include::matrix/stats-aggregation.asciidoc[] \ No newline at end of file diff --git a/docs/reference/modules/aggregations/matrix/stats.asciidoc b/docs/reference/aggregations/matrix/stats-aggregation.asciidoc similarity index 98% rename from docs/reference/modules/aggregations/matrix/stats.asciidoc rename to docs/reference/aggregations/matrix/stats-aggregation.asciidoc index 2649ecd5fe7..8dafb252f08 100644 --- a/docs/reference/modules/aggregations/matrix/stats.asciidoc +++ b/docs/reference/aggregations/matrix/stats-aggregation.asciidoc @@ -1,4 +1,4 @@ -[[modules-matrix-aggregations-stats]] +[[search-aggregations-matrix-stats-aggregation]] === Matrix Stats The `matrix_stats` aggregation is a numeric aggregation that computes the following statistics over a set of document fields: diff --git a/docs/reference/docs.asciidoc b/docs/reference/docs.asciidoc index f3b30e7f0c3..04049663e84 100644 --- a/docs/reference/docs.asciidoc +++ b/docs/reference/docs.asciidoc @@ -15,6 +15,8 @@ This section describes the following CRUD APIs: .Multi-document APIs * <> * <> +* <> +* <> NOTE: All CRUD APIs are single-index APIs. The `index` parameter accepts a single index name, or an `alias` which points to a single index. @@ -42,3 +44,5 @@ include::docs/reindex.asciidoc[] include::docs/termvectors.asciidoc[] include::docs/multi-termvectors.asciidoc[] + +include::docs/refresh.asciidoc[] diff --git a/docs/reference/docs/bulk.asciidoc b/docs/reference/docs/bulk.asciidoc index ee8f75c6cdd..6b0a5b2e40d 100644 --- a/docs/reference/docs/bulk.asciidoc +++ b/docs/reference/docs/bulk.asciidoc @@ -167,12 +167,8 @@ are the same). [[bulk-refresh]] === Refresh -The `refresh` parameter can be set to `true` in order to refresh the relevant -primary and replica shards immediately after the bulk operation has occurred -and make it searchable, instead of waiting for the normal refresh interval to -expire. Setting it to `true` can trigger additional load, and may slow down -indexing. Due to its costly nature, the `refresh` parameter is set on the bulk request level -and is not supported on each individual bulk item. +Control when the changes made by this request are visible to search. See +<>. [float] [[bulk-update]] diff --git a/docs/reference/docs/delete.asciidoc b/docs/reference/docs/delete.asciidoc index 175c07d005e..18a370fc416 100644 --- a/docs/reference/docs/delete.asciidoc +++ b/docs/reference/docs/delete.asciidoc @@ -113,11 +113,9 @@ is the same). [[delete-refresh]] === Refresh -The `refresh` parameter can be set to `true` in order to refresh the relevant -primary and replica shards after the delete operation has occurred and make it -searchable. Setting it to `true` should be done after careful thought and -verification that this does not cause a heavy load on the system (and slows -down indexing). +Control when the changes made by this request are visible to search. See +<>. + [float] [[delete-timeout]] diff --git a/docs/reference/docs/index_.asciidoc b/docs/reference/docs/index_.asciidoc index eb1e45d6160..aa62b65292e 100644 --- a/docs/reference/docs/index_.asciidoc +++ b/docs/reference/docs/index_.asciidoc @@ -30,7 +30,8 @@ The result of the above index operation is: "_type" : "tweet", "_id" : "1", "_version" : 1, - "created" : true + "created" : true, + "forced_refresh": false } -------------------------------------------------- // TESTRESPONSE[s/"successful" : 2/"successful" : 1/] @@ -221,7 +222,8 @@ The result of the above index operation is: "_type" : "tweet", "_id" : "6a8ca01c-7896-48e9-81cc-9f70661fcb32", "_version" : 1, - "created" : true + "created" : true, + "forced_refresh": false } -------------------------------------------------- // TESTRESPONSE[s/6a8ca01c-7896-48e9-81cc-9f70661fcb32/$body._id/ s/"successful" : 2/"successful" : 1/] @@ -385,13 +387,8 @@ replication group have indexed the document (sync replication). [[index-refresh]] === Refresh -To refresh the shard (not the whole index) immediately after the operation -occurs, so that the document appears in search results immediately, the -`refresh` parameter can be set to `true`. Setting this option to `true` should -*ONLY* be done after careful thought and verification that it does not lead to -poor performance, both from an indexing and a search standpoint. Note, getting -a document using the get API is completely realtime and doesn't require a -refresh. +Control when the changes made by this request are visible to search. See +<>. [float] [[index-noop]] diff --git a/docs/reference/docs/refresh.asciidoc b/docs/reference/docs/refresh.asciidoc new file mode 100644 index 00000000000..3e5153341c8 --- /dev/null +++ b/docs/reference/docs/refresh.asciidoc @@ -0,0 +1,109 @@ +[[docs-refresh]] +== `?refresh` + +The <>, <>, <>, and +<> APIs support setting `refresh` to control when changes made +by this request are made visible to search. These are the allowed values: + +Empty string or `true`:: + +Refresh the relevant primary and replica shards (not the whole index) +immediately after the operation occurs, so that the updated document appears +in search results immediately. This should *ONLY* be done after careful thought +and verification that it does not lead to poor performance, both from an +indexing and a search standpoint. + +`wait_for`:: + +Wait for the changes made by the request to be made visible by a refresh before +replying. This doesn't force an immediate refresh, rather, it waits for a +refresh happen. Elasticsearch automatically refreshes shards that have changed +every `index.refresh_interval` which defaults to one second. That setting is +<>. The <> API will also +cause the request to return, as will setting `refresh` to `true` on any of the +APIs that support it. + +`false` (the default):: + +Take no refresh related actions. The changes made by this request will be made +visible at some point after the request returns. + +=== Choosing which setting to use + +Unless you have a good reason to wait for the change to become visible always +use `refresh=false`, or, because that is the default, just leave the `refresh` +parameter out of the URL. That is the simplest and fastest choice. + +If you absolutely must have the changes made by a request visible synchronously +with the request then you must get to pick between putting more load on +Elasticsearch (`true`) and waiting longer for the response (`wait_for`). Here +are a few points that should inform that decision: + +* The more changes being made to the index the more work `wait_for` saves +compared to `true`. In the case that the index is only changed once every +`index.refresh_interval` then it saves no work. +* `true` creates less efficient indexes constructs (tiny segments) that must +later be merged into more efficient index constructs (larger segments). Meaning +that the cost of `true` is payed at index time to create the tiny segment, at +search time to search the tiny segment, and at merge time to make the larger +segments. +* Never start multiple `refresh=wait_for` requests in a row. Instead batch them +into a single bulk request with `refresh=wait_for` and Elasticsearch will start +them all in parallel and return only when they have all finished. +* If the refresh interval is set to `-1`, disabling the automatic refreshes, +then requests with `refresh=wait_for` will wait indefinitely until some action +causes a refresh. Conversely, setting `index.refresh_interval` to something +shorter than the default like `200ms` will make `refresh=wait_for` come back +faster, but it'll still generate inefficient segments. +* `refresh=wait_for` only affects the request that it is on, but, by forcing a +refresh immediately, `refresh=true` will affect other ongoing request. In +general, if you have a running system you don't wish to disturb then +`refresh=wait_for` is a smaller modification. + +=== `refresh=wait_for` Can Force a Refresh + +If a `refresh=wait_for` request comes in when there are already +`index.max_refresh_listeners` (defaults to 1000) requests waiting for a refresh +on that shard then that request will behave just as though it had `refresh` set +to `true` instead: it will force a refresh. This keeps the promise that when a +`refresh=wait_for` request returns that its changes are visible for search +while preventing unchecked resource usage for blocked requests. If a request +forced a refresh because it ran out of listener slots then its response will +contain `"forced_refresh": true`. + +Bulk requests only take up one slot on each shard that they touch no matter how +many times they modify the shard. + +=== Examples + +These will create a document and immediately refresh the index so it is visible: + +[source,js] +-------------------------------------------------- +PUT /test/test/1?refresh +{"test": "test"} +PUT /test/test/2?refresh=true +{"test": "test"} +-------------------------------------------------- +// CONSOLE + +These will create a document without doing anything to make it visible for +search: + +[source,js] +-------------------------------------------------- +PUT /test/test/3 +{"test": "test"} +PUT /test/test/4?refresh=true +{"test": "test"} +-------------------------------------------------- +// CONSOLE + +This will create a document and wait for it to become visible for search: + +[source,js] +-------------------------------------------------- +PUT /test/test/4?refresh=wait_for +{"test": "test"} +-------------------------------------------------- +// CONSOLE diff --git a/docs/reference/docs/reindex.asciidoc b/docs/reference/docs/reindex.asciidoc index 0bfb44b2aaf..3311049c699 100644 --- a/docs/reference/docs/reindex.asciidoc +++ b/docs/reference/docs/reindex.asciidoc @@ -30,6 +30,7 @@ That will return something like this: "timed_out": false, "created": 120, "updated": 0, + "deleted": 0, "batches": 1, "version_conflicts": 0, "noops": 0, @@ -244,6 +245,24 @@ POST _reindex // CONSOLE // TEST[setup:twitter] +Just as in `_update_by_query`, you can set `ctx.op` to change the +operation that is executed on the destination index: + +`noop`:: + +Set `ctx.op = "noop"` if your script decides that the document doesn't have +to be indexed in the destination index. This no operation will be reported +in the `noop` counter in the <>. + +`delete`:: + +Set `ctx.op = "delete"` if your script decides that the document must be + deleted from the destination index. The deletion will be reported in the + `deleted` counter in the <>. + +Setting `ctx.op` to anything else is an error. Setting any +other field in `ctx` is an error. + Think of the possibilities! Just be careful! With great power.... You can change: @@ -377,6 +396,7 @@ starting the next set. This is "bursty" instead of "smooth". The default is `unlimited` which is also the only non-number value that it accepts. [float] +[[docs-reindex-response-body]] === Response body The JSON response looks like this: diff --git a/docs/reference/docs/update-by-query.asciidoc b/docs/reference/docs/update-by-query.asciidoc index ac4da4251be..f3a147e36c1 100644 --- a/docs/reference/docs/update-by-query.asciidoc +++ b/docs/reference/docs/update-by-query.asciidoc @@ -23,6 +23,7 @@ That will return something like this: "took" : 147, "timed_out": false, "updated": 120, + "deleted": 0, "batches": 1, "version_conflicts": 0, "noops": 0, @@ -115,11 +116,24 @@ POST twitter/_update_by_query // CONSOLE // TEST[setup:twitter] -Just as in <> you can set `ctx.op = "noop"` if -your script decides that it doesn't have to make any changes. That will cause -`_update_by_query` to omit that document from its updates. Setting `ctx.op` to -anything else is an error. If you want to delete by a query you can use the -<> instead. Setting any +Just as in <> you can set `ctx.op` to change the +operation that is executed: + + +`noop`:: + +Set `ctx.op = "noop"` if your script decides that it doesn't have to make any +changes. That will cause `_update_by_query` to omit that document from its updates. + This no operation will be reported in the `noop` counter in the +<>. + +`delete`:: + +Set `ctx.op = "delete"` if your script decides that the document must be + deleted. The deletion will be reported in the `deleted` counter in the +<>. + +Setting `ctx.op` to anything else is an error. Setting any other field in `ctx` is an error. Note that we stopped specifying `conflicts=proceed`. In this case we want a @@ -212,6 +226,7 @@ starting the next set. This is "bursty" instead of "smooth". The default is `unlimited` which is also the only non-number value that it accepts. [float] +[[docs-update-by-query-response-body]] === Response body The JSON response looks like this: diff --git a/docs/reference/docs/update.asciidoc b/docs/reference/docs/update.asciidoc index 405f9b0494b..39261c5d21f 100644 --- a/docs/reference/docs/update.asciidoc +++ b/docs/reference/docs/update.asciidoc @@ -235,9 +235,8 @@ The write consistency of the index/delete operation. `refresh`:: -Refresh the relevant primary and replica shards (not the whole index) -immediately after the operation occurs, so that the updated document appears -in search results immediately. +Control when the changes made by this request are visible to search. See +<>. `fields`:: diff --git a/docs/reference/index-modules.asciidoc b/docs/reference/index-modules.asciidoc index c7119af4168..3cc95e5af9a 100644 --- a/docs/reference/index-modules.asciidoc +++ b/docs/reference/index-modules.asciidoc @@ -136,6 +136,11 @@ specific index module: experimental[] Disables the purge of <> on the current index. +`index.max_refresh_listeners`:: + + Maximum number of refresh listeners available on each shard of the index. + These listeners are used to implement <>. + [float] === Settings in other index modules diff --git a/docs/reference/index-modules/allocation/filtering.asciidoc b/docs/reference/index-modules/allocation/filtering.asciidoc index 05007b46188..0eba1282759 100644 --- a/docs/reference/index-modules/allocation/filtering.asciidoc +++ b/docs/reference/index-modules/allocation/filtering.asciidoc @@ -9,7 +9,7 @@ conjunction with the cluster-wide allocation filters explained in <>. It is possible to assign arbitrary metadata attributes to each node at -startup. For instance, nodes could be assigned a `rack` and a `group` +startup. For instance, nodes could be assigned a `rack` and a `size` attribute as follows: [source,sh] diff --git a/docs/reference/indices/shrink-index.asciidoc b/docs/reference/indices/shrink-index.asciidoc index d531477e77c..39c5134e23c 100644 --- a/docs/reference/indices/shrink-index.asciidoc +++ b/docs/reference/indices/shrink-index.asciidoc @@ -2,13 +2,18 @@ == Shrink Index The shrink index API allows you to shrink an existing index into a new index -with a single primary shard. Before shrinking, a (primary or replica) copy of -every shard in the index must be present on the same node. +with fewer primary shards. The number of primary shards in the target index +must be a factor of the shards in the source index. For example an index with +`8` primary shards can be shrunk into `4`, `2` or `1` primary shards or an index +with `15` primary shards can be shrunk into `5`, `3` or `1`. If the number +of shards in the index is a prime number it can only be shrunk into a single +primary shard. Before shrinking, a (primary or replica) copy of every shard +in the index must be present on the same node. Shrinking works as follows: * First, it creates a new target index with the same definition as the source - index, but with a single primary shard. + index, but with a smaller number of primary shards. * Then it hard-links segments from the source index into the target index. (If the file system doesn't support hard-linking, then all segments are copied @@ -64,15 +69,19 @@ the cluster state -- it doesn't wait for the shrink operation to start. [IMPORTANT] ===================================== -Indices can only be shrunk into a single shard if they satisfy the following requirements: +Indices can only be shrunk if they satisfy the following requirements: - * the target index must not exist +* the target index must not exist -* The index must have more than one primary shard. +* The index must have more primary shards than the target index. + +* The number of primary shards in the target index must be a factor of the + number of primary shards in the source index. must have more primary shards + than the target index. * The index must not contain more than `2,147,483,519` documents in total - across all shards as this is the maximum number of docs that can fit into a - single shard. + across all shards that will be shrunk into a single shard on the target index + as this is the maximum number of docs that can fit into a single shard. * The node handling the shrink process must have sufficient free disk space to accommodate a second copy of the existing index. @@ -88,7 +97,8 @@ POST my_source_index/_shrink/my_target_index { "settings": { "index.number_of_replicas": 1, - "index.codec": "best_compression" <1> + "index.number_of_shards": 1, <1> + "index.codec": "best_compression" <2> }, "aliases": { "my_search_indices": {} @@ -96,10 +106,13 @@ POST my_source_index/_shrink/my_target_index } -------------------------------------------------- -<1> Best compression will only take affect when new writes are made to the +<1> The number of shards in the target index. This must be a factor of the + number of shards in the source index. +<2> Best compression will only take affect when new writes are made to the index, such as when <> the shard to a single segment. + NOTE: Mappings may not be specified in the `_shrink` request, and all `index.analysis.*` and `index.similarity.*` settings will be overwritten with the settings from the source index. diff --git a/docs/reference/ingest/ingest-node.asciidoc b/docs/reference/ingest/ingest-node.asciidoc index 9cf51374ab4..ec641383beb 100644 --- a/docs/reference/ingest/ingest-node.asciidoc +++ b/docs/reference/ingest/ingest-node.asciidoc @@ -744,7 +744,7 @@ in the same order they were defined as part of the processor definition. | Name | Required | Default | Description | `field` | yes | - | The field to get the date from. | `target_field` | no | @timestamp | The field that will hold the parsed date. -| `match_formats` | yes | - | An array of the expected date formats. Can be a Joda pattern or one of the following formats: ISO8601, UNIX, UNIX_MS, or TAI64N. +| `formats` | yes | - | An array of the expected date formats. Can be a Joda pattern or one of the following formats: ISO8601, UNIX, UNIX_MS, or TAI64N. | `timezone` | no | UTC | The timezone to use when parsing the date. | `locale` | no | ENGLISH | The locale to use when parsing the date, relevant when parsing month names or week days. |====== @@ -760,7 +760,7 @@ Here is an example that adds the parsed date to the `timestamp` field based on t "date" : { "field" : "initial_date", "target_field" : "timestamp", - "match_formats" : ["dd/MM/yyyy hh:mm:ss"], + "formats" : ["dd/MM/yyyy hh:mm:ss"], "timezone" : "Europe/Amsterdam" } } diff --git a/docs/reference/migration/migrate_5_0/docs.asciidoc b/docs/reference/migration/migrate_5_0/docs.asciidoc index 85e4e901e5c..9149eed6142 100644 --- a/docs/reference/migration/migrate_5_0/docs.asciidoc +++ b/docs/reference/migration/migrate_5_0/docs.asciidoc @@ -1,6 +1,16 @@ [[breaking_50_document_api_changes]] === Document API changes +==== `?refresh` no longer supports truthy and falsy values +The `?refresh` request parameter used to accept any value other than `false`, +`0`, `off`, and `no` to mean "make the changes from this request visible for +search immediately." Now it only accepts `?refresh` and `?refresh=true` to +mean that. You can set it to `?refresh=false` and the request will take no +refresh-related action. The same is true if you leave `refresh` off of the +url entirely. If you add `?refresh=wait_for` Elasticsearch will wait for the +changes to become visible before replying to the request but won't take any +immediate refresh related action. See <>. + ==== Reindex and Update By Query Before 5.0.0 `_reindex` and `_update_by_query` only retried bulk failures so they used the following response format: diff --git a/docs/reference/migration/migrate_5_0/fs.asciidoc b/docs/reference/migration/migrate_5_0/fs.asciidoc index 2d582702690..859f3092823 100644 --- a/docs/reference/migration/migrate_5_0/fs.asciidoc +++ b/docs/reference/migration/migrate_5_0/fs.asciidoc @@ -8,3 +8,18 @@ there is nothing to worry about since this is only address space consumption and the actual memory usage of Elasticsearch will stay similar to what it was in 2.x. See http://blog.thetaphi.de/2012/07/use-lucenes-mmapdirectory-on-64bit.html for more information. + +=== Path to data on disk + +In prior versions of Elasticsearch, the `path.data` directory included a folder +for the cluster name, so that data was in a folder such as +`$DATA_DIR/$CLUSTER_NAME/nodes/$nodeOrdinal`. In 5.0 the cluster name as a +directory is deprecated. Data will now be stored in +`$DATA_DIR/nodes/$nodeOrdinal` if there is no existing data. Upon startup, +Elasticsearch will check to see if the cluster folder exists and has data, and +will read from it if necessary. In Elasticsearch 6.0 this backwards-compatible +behavior will be removed. + +If you are using a multi-cluster setup with both instances of Elasticsearch +pointing to the same data path, you will need to add the cluster name to the +data path so that different clusters do not overwrite data. diff --git a/docs/reference/migration/migrate_5_0/java.asciidoc b/docs/reference/migration/migrate_5_0/java.asciidoc index da97d360b43..1ad34d11a7c 100644 --- a/docs/reference/migration/migrate_5_0/java.asciidoc +++ b/docs/reference/migration/migrate_5_0/java.asciidoc @@ -84,14 +84,14 @@ static factory methods in QueryBuilders accordingly. Making sure that query contains at least one clause by making initial clause mandatory in constructor. -Renaming method to add clauses from `clause(SpanQueryBuilder)` to `addClause(SpanQueryBuilder)`. +Renaming method to add clauses from `clause(SpanQueryBuilder)` to `addClause(SpanQueryBuilder)`. ===== SpanNearQueryBuilder Removed setter for mandatory slop parameter, needs to be set in constructor now. Also making sure that query contains at least one clause by making initial clause mandatory in constructor. Updated the static factory methods in QueryBuilders accordingly. -Renaming method to add clauses from `clause(SpanQueryBuilder)` to `addClause(SpanQueryBuilder)`. +Renaming method to add clauses from `clause(SpanQueryBuilder)` to `addClause(SpanQueryBuilder)`. ===== SpanNotQueryBuilder @@ -304,3 +304,9 @@ The `setQuery(BytesReference)` method have been removed in favor of using `setQu Removed the `getMemoryAvailable` method from `OsStats`, which could be previously accessed calling `clusterStatsResponse.getNodesStats().getOs().getMemoryAvailable()`. + +=== setRefresh(boolean) has been removed + +`setRefresh(boolean)` has been removed in favor of `setRefreshPolicy(RefreshPolicy)` because there +are now three options (NONE, IMMEDIATE, and WAIT_FOR). `setRefresh(IMMEDIATE)` has the same behavior +as `setRefresh(true)` used to have. See `setRefreshPolicy`'s javadoc for more. diff --git a/docs/reference/migration/migrate_5_0/settings.asciidoc b/docs/reference/migration/migrate_5_0/settings.asciidoc index 5edd4e449e8..ed405428ab6 100644 --- a/docs/reference/migration/migrate_5_0/settings.asciidoc +++ b/docs/reference/migration/migrate_5_0/settings.asciidoc @@ -79,6 +79,19 @@ on the thread pool type, `keep_alive`, `queue_size`, etc.). The `suggest` threadpool has been removed, now suggest requests use the `search` threadpool. +The prefix on all thread pool settings has been changed from +`threadpool` to `thread_pool`. + +The minimum size setting for a scaling thread pool has been changed +from `min` to `core`. + +The maximum size setting for a scaling thread pool has been changed +from `size` to `max`. + +The queue size setting for a fixed thread pool must be `queue_size` +(all other variants that were previously supported are no longer +supported). + ==== Analysis settings The `index.analysis.analyzer.default_index` analyzer is not supported anymore. @@ -264,3 +277,9 @@ should be used instead. The setting `bootstrap.mlockall` has been renamed to `bootstrap.memory_lock`. + +==== Snapshot settings + +The default setting `include_global_state` for restoring snapshots has been +changed from `true` to `false`. It has not been changed for taking snapshots and +still defaults to `true` in that case. diff --git a/docs/reference/modules.asciidoc b/docs/reference/modules.asciidoc index 01b0edd0676..5a39cdfd790 100644 --- a/docs/reference/modules.asciidoc +++ b/docs/reference/modules.asciidoc @@ -18,10 +18,6 @@ These settings can be dynamically updated on a live cluster with the The modules in this section are: -<>:: - - A family of aggregations that operate on multiple document fields and produce a matrix as output. - <>:: Settings to control where, when, and how shards are allocated to nodes. @@ -84,8 +80,6 @@ The modules in this section are: -- -include::modules/aggregations-matrix.asciidoc[] - include::modules/cluster.asciidoc[] include::modules/discovery.asciidoc[] diff --git a/docs/reference/modules/snapshots.asciidoc b/docs/reference/modules/snapshots.asciidoc index 0f87744d317..2d8106e981c 100644 --- a/docs/reference/modules/snapshots.asciidoc +++ b/docs/reference/modules/snapshots.asciidoc @@ -323,13 +323,17 @@ POST /_snapshot/my_backup/snapshot_1/_restore ----------------------------------- // CONSOLE -By default, all indices in the snapshot as well as cluster state are restored. It's possible to select indices that -should be restored as well as prevent global cluster state from being restored by using `indices` and -`include_global_state` options in the restore request body. The list of indices supports -<>. The `rename_pattern` and `rename_replacement` options can be also used to -rename index on restore using regular expression that supports referencing the original text as explained +By default, all indices in the snapshot are restored, and the cluster state is +*not* restored. It's possible to select indices that should be restored as well +as to allow the global cluster state from being restored by using `indices` and +`include_global_state` options in the restore request body. The list of indices +supports <>. The `rename_pattern` +and `rename_replacement` options can be also used to rename index on restore +using regular expression that supports referencing the original text as +explained http://docs.oracle.com/javase/6/docs/api/java/util/regex/Matcher.html#appendReplacement(java.lang.StringBuffer,%20java.lang.String)[here]. -Set `include_aliases` to `false` to prevent aliases from being restored together with associated indices +Set `include_aliases` to `false` to prevent aliases from being restored together +with associated indices [source,js] ----------------------------------- @@ -337,19 +341,22 @@ POST /_snapshot/my_backup/snapshot_1/_restore { "indices": "index_1,index_2", "ignore_unavailable": "true", - "include_global_state": false, + "include_global_state": true, "rename_pattern": "index_(.+)", "rename_replacement": "restored_index_$1" } ----------------------------------- // CONSOLE -The restore operation can be performed on a functioning cluster. However, an existing index can be only restored if it's -<> and has the same number of shards as the index in the snapshot. -The restore operation automatically opens restored indices if they were closed and creates new indices if they -didn't exist in the cluster. If cluster state is restored, the restored templates that don't currently exist in the -cluster are added and existing templates with the same name are replaced by the restored templates. The restored -persistent settings are added to the existing persistent settings. +The restore operation can be performed on a functioning cluster. However, an +existing index can be only restored if it's <> and +has the same number of shards as the index in the snapshot. The restore +operation automatically opens restored indices if they were closed and creates +new indices if they didn't exist in the cluster. If cluster state is restored +with `include_cluster_state` (defaults to `false`), the restored templates that +don't currently exist in the cluster are added and existing templates with the +same name are replaced by the restored templates. The restored persistent +settings are added to the existing persistent settings. [float] ==== Partial restore diff --git a/docs/reference/modules/threadpool.asciidoc b/docs/reference/modules/threadpool.asciidoc index 65069e25940..d8f8545c213 100644 --- a/docs/reference/modules/threadpool.asciidoc +++ b/docs/reference/modules/threadpool.asciidoc @@ -17,7 +17,7 @@ There are several thread pools, but the important ones include: For index/delete operations. Thread pool type is `fixed` with a size of `# of available processors`, queue_size of `200`. The maximum size for this pool - is `# of available processors`. + is `1 + # of available processors`. `search`:: For count/search/suggest operations. Thread pool type is `fixed` @@ -33,7 +33,7 @@ There are several thread pools, but the important ones include: For bulk operations. Thread pool type is `fixed` with a size of `# of available processors`, queue_size of `50`. The maximum size for this pool - is `# of available processors`. + is `1 + # of available processors`. `percolate`:: For percolate operations. Thread pool type is `fixed` @@ -42,26 +42,26 @@ There are several thread pools, but the important ones include: `snapshot`:: For snapshot/restore operations. Thread pool type is `scaling` with a - keep-alive of `5m` and a size of `min(5, (# of available processors)/2)`. + keep-alive of `5m` and a max of `min(5, (# of available processors)/2)`. `warmer`:: For segment warm-up operations. Thread pool type is `scaling` with a - keep-alive of `5m` and a size of `min(5, (# of available processors)/2)`. + keep-alive of `5m` and a max of `min(5, (# of available processors)/2)`. `refresh`:: For refresh operations. Thread pool type is `scaling` with a - keep-alive of `5m` and a size of `min(10, (# of available processors)/2)`. + keep-alive of `5m` and a max of `min(10, (# of available processors)/2)`. `listener`:: Mainly for java client executing of action when listener threaded is set to true. - Thread pool type is `scaling` with a default size of `min(10, (# of available processors)/2)`. + Thread pool type is `scaling` with a default max of `min(10, (# of available processors)/2)`. Changing a specific thread pool can be done by setting its type-specific parameters; for example, changing the `index` thread pool to have more threads: [source,js] -------------------------------------------------- -threadpool: +thread_pool: index: size: 30 -------------------------------------------------- @@ -91,7 +91,7 @@ full, it will abort the request. [source,js] -------------------------------------------------- -threadpool: +thread_pool: index: size: 30 queue_size: 1000 @@ -102,17 +102,17 @@ threadpool: The `scaling` thread pool holds a dynamic number of threads. This number is proportional to the workload and varies between the value of -the `min` and `size` parameters. +the `core` and `max` parameters. The `keep_alive` parameter determines how long a thread should be kept around in the thread pool without it doing any work. [source,js] -------------------------------------------------- -threadpool: +thread_pool: warmer: - min: 1 - size: 8 + core: 1 + max: 8 keep_alive: 2m -------------------------------------------------- diff --git a/docs/reference/query-dsl/nested-query.asciidoc b/docs/reference/query-dsl/nested-query.asciidoc index 6e990e07f91..176473174dd 100644 --- a/docs/reference/query-dsl/nested-query.asciidoc +++ b/docs/reference/query-dsl/nested-query.asciidoc @@ -23,7 +23,6 @@ PUT /my_index } } -GET _cluster/health?wait_for_status=yellow -------------------------------------------------- // CONSOLE // TESTSETUP diff --git a/docs/reference/query-dsl/parent-id-query.asciidoc b/docs/reference/query-dsl/parent-id-query.asciidoc index 8ea07a6d0b7..a7a28cf88e8 100644 --- a/docs/reference/query-dsl/parent-id-query.asciidoc +++ b/docs/reference/query-dsl/parent-id-query.asciidoc @@ -29,7 +29,6 @@ PUT /my_index } } -GET /_cluster/health?wait_for_status=yellow ------------------------------------------ // CONSOLE // TESTSETUP diff --git a/docs/reference/search/request/scroll.asciidoc b/docs/reference/search/request/scroll.asciidoc index e18593d21cc..9f9558aa979 100644 --- a/docs/reference/search/request/scroll.asciidoc +++ b/docs/reference/search/request/scroll.asciidoc @@ -175,3 +175,92 @@ curl -XDELETE localhost:9200/_search/scroll \ -d 'c2Nhbjs2OzM0NDg1ODpzRlBLc0FXNlNyNm5JWUc1,aGVuRmV0Y2g7NTsxOnkxaDZ' --------------------------------------- + +==== Sliced Scroll + +For scroll queries that return a lot of documents it is possible to split the scroll in multiple slices which +can be consumed independently: + +[source,js] +-------------------------------------------------- +curl -XGET 'localhost:9200/twitter/tweet/_search?scroll=1m' -d ' +{ + "slice": { + "id": 0, <1> + "max": 2 <2> + }, + "query": { + "match" : { + "title" : "elasticsearch" + } + } +} +' + +curl -XGET 'localhost:9200/twitter/tweet/_search?scroll=1m' -d ' +{ + "slice": { + "id": 1, + "max": 2 + }, + "query": { + "match" : { + "title" : "elasticsearch" + } + } +} +' +-------------------------------------------------- + +<1> The id of the slice +<2> The maximum number of slices + +The result from the first request returned documents that belong to the first slice (id: 0) and the result from the +second request returned documents that belong to the second slice. Since the maximum number of slices is set to 2 + the union of the results of the two requests is equivalent to the results of a scroll query without slicing. +By default the splitting is done on the shards first and then locally on each shard using the _uid field +with the following formula: +`slice(doc) = floorMod(hashCode(doc._uid), max)` +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. + +Each scroll is independent and can be processed in parallel like any scroll request. + +NOTE: If the number of slices is bigger than the number of shards the slice filter is very slow on the first calls, it has a complexity of O(N) and a memory cost equals +to N bits per slice where N is the total number of documents in the shard. +After few calls the filter should be cached and subsequent calls should be faster but you should limit the number of + sliced query you perform in parallel to avoid the memory explosion. + +To avoid this cost entirely it is possible to use the `doc_values` of another field to do the slicing +but the user must ensure that the field has the following properties: + + * The field is numeric. + + * `doc_values` are enabled on that field + + * Every document should contain a single value. If a document has multiple values for the specified field, the first value is used. + + * The value for each document should be set once when the document is created and never updated. This ensures that each +slice gets deterministic results. + + * The cardinality of the field should be high. This ensures that each slice gets approximately the same amount of documents. + +[source,js] +-------------------------------------------------- +curl -XGET 'localhost:9200/twitter/tweet/_search?scroll=1m' -d ' +{ + "slice": { + "field": "my_random_integer_field", + "id": 0, + "max": 10 + }, + "query": { + "match" : { + "title" : "elasticsearch" + } + } +} +' +-------------------------------------------------- + +For append only time-based indices, the `timestamp` field can be used safely. \ No newline at end of file diff --git a/docs/reference/search/request/sort.asciidoc b/docs/reference/search/request/sort.asciidoc index 2f2e8882032..fd3dbffc1d6 100644 --- a/docs/reference/search/request/sort.asciidoc +++ b/docs/reference/search/request/sort.asciidoc @@ -26,6 +26,8 @@ PUT /my_index } } } + +GET /_cluster/health?wait_for_status=yellow -------------------------------------------------- // CONSOLE diff --git a/docs/reference/setup/bootstrap-checks.asciidoc b/docs/reference/setup/bootstrap-checks.asciidoc index 2a5c55bf8b3..1da61360033 100644 --- a/docs/reference/setup/bootstrap-checks.asciidoc +++ b/docs/reference/setup/bootstrap-checks.asciidoc @@ -16,6 +16,10 @@ checks that fail appear as warnings in the Elasticsearch log. If Elasticsearch is in production mode, any bootstrap checks that fail will cause Elasticsearch to refuse to start. +There are some bootstrap checks that are always enforced to prevent +Elasticsearch from running with incompatible settings. These checks are +documented individually. + [float] === Development vs. production mode @@ -152,3 +156,19 @@ JVM check, you must start Elasticsearch with the server VM. On modern systems and operating systems, the server VM is the default. Additionally, Elasticsearch is configured by default to force the server VM. + +=== OnError and OnOutOfMemoryError checks + +The JVM options `OnError` and `OnOutOfMemoryError` enable executing +arbitrary commands if the JVM encounters a fatal error (`OnError`) or an +`OutOfMemoryError` (`OnOutOfMemoryError`). However, by default, +Elasticsearch system call filters (seccomp) are enabled and these +filters prevent forking. Thus, using `OnError` or `OnOutOfMemoryError` +and system call filters are incompatible. The `OnError` and +`OnOutOfMemoryError` checks prevent Elasticsearch from starting if +either of these JVM options are used and system call filters are +enabled. This check is always enforced. To pass this check do not enable +`OnError` nor `OnOutOfMemoryError`; instead, upgrade to Java 8u92 and +use the JVM flag `ExitOnOutOfMemoryError`. While this does not have the +full capabilities of `OnError` nor `OnOutOfMemoryError`, arbitrary +forking will not be supported with seccomp enabled. diff --git a/modules/ingest-grok/build.gradle b/modules/ingest-common/build.gradle similarity index 77% rename from modules/ingest-grok/build.gradle rename to modules/ingest-common/build.gradle index 26722345544..e7935cd2738 100644 --- a/modules/ingest-grok/build.gradle +++ b/modules/ingest-common/build.gradle @@ -18,8 +18,8 @@ */ esplugin { - description 'Ingest processor that uses grok patterns to split text' - classname 'org.elasticsearch.ingest.grok.IngestGrokPlugin' + description 'Module for ingest processors that do not require additional security permissions or have large dependencies and resources' + classname 'org.elasticsearch.ingest.IngestCommonPlugin' } dependencies { @@ -28,8 +28,8 @@ dependencies { compile 'org.jruby.jcodings:jcodings:1.0.12' } -compileJava.options.compilerArgs << "-Xlint:-rawtypes,-unchecked,-serial" -compileTestJava.options.compilerArgs << "-Xlint:-rawtypes,-unchecked" +compileJava.options.compilerArgs << "-Xlint:-unchecked,-rawtypes" +compileTestJava.options.compilerArgs << "-Xlint:-unchecked,-rawtypes" thirdPartyAudit.excludes = [ // joni has AsmCompilerSupport, but that isn't being used: diff --git a/modules/ingest-grok/licenses/jcodings-1.0.12.jar.sha1 b/modules/ingest-common/licenses/jcodings-1.0.12.jar.sha1 similarity index 100% rename from modules/ingest-grok/licenses/jcodings-1.0.12.jar.sha1 rename to modules/ingest-common/licenses/jcodings-1.0.12.jar.sha1 diff --git a/modules/ingest-grok/licenses/jcodings-LICENSE.txt b/modules/ingest-common/licenses/jcodings-LICENSE.txt similarity index 100% rename from modules/ingest-grok/licenses/jcodings-LICENSE.txt rename to modules/ingest-common/licenses/jcodings-LICENSE.txt diff --git a/modules/ingest-grok/licenses/jcodings-NOTICE.txt b/modules/ingest-common/licenses/jcodings-NOTICE.txt similarity index 100% rename from modules/ingest-grok/licenses/jcodings-NOTICE.txt rename to modules/ingest-common/licenses/jcodings-NOTICE.txt diff --git a/modules/ingest-grok/licenses/joni-2.1.6.jar.sha1 b/modules/ingest-common/licenses/joni-2.1.6.jar.sha1 similarity index 100% rename from modules/ingest-grok/licenses/joni-2.1.6.jar.sha1 rename to modules/ingest-common/licenses/joni-2.1.6.jar.sha1 diff --git a/modules/ingest-grok/licenses/joni-LICENSE.txt b/modules/ingest-common/licenses/joni-LICENSE.txt similarity index 100% rename from modules/ingest-grok/licenses/joni-LICENSE.txt rename to modules/ingest-common/licenses/joni-LICENSE.txt diff --git a/modules/ingest-grok/licenses/joni-NOTICE.txt b/modules/ingest-common/licenses/joni-NOTICE.txt similarity index 100% rename from modules/ingest-grok/licenses/joni-NOTICE.txt rename to modules/ingest-common/licenses/joni-NOTICE.txt diff --git a/core/src/main/java/org/elasticsearch/ingest/processor/AbstractStringProcessor.java b/modules/ingest-common/src/main/java/org/elasticsearch/ingest/AbstractStringProcessor.java similarity index 98% rename from core/src/main/java/org/elasticsearch/ingest/processor/AbstractStringProcessor.java rename to modules/ingest-common/src/main/java/org/elasticsearch/ingest/AbstractStringProcessor.java index e882c79ed85..2dc11d43af1 100644 --- a/core/src/main/java/org/elasticsearch/ingest/processor/AbstractStringProcessor.java +++ b/modules/ingest-common/src/main/java/org/elasticsearch/ingest/AbstractStringProcessor.java @@ -17,7 +17,7 @@ * under the License. */ -package org.elasticsearch.ingest.processor; +package org.elasticsearch.ingest; import org.elasticsearch.ingest.core.AbstractProcessor; import org.elasticsearch.ingest.core.AbstractProcessorFactory; diff --git a/core/src/main/java/org/elasticsearch/ingest/processor/AppendProcessor.java b/modules/ingest-common/src/main/java/org/elasticsearch/ingest/AppendProcessor.java similarity index 98% rename from core/src/main/java/org/elasticsearch/ingest/processor/AppendProcessor.java rename to modules/ingest-common/src/main/java/org/elasticsearch/ingest/AppendProcessor.java index cca31453f37..4019f8312ef 100644 --- a/core/src/main/java/org/elasticsearch/ingest/processor/AppendProcessor.java +++ b/modules/ingest-common/src/main/java/org/elasticsearch/ingest/AppendProcessor.java @@ -17,7 +17,7 @@ * under the License. */ -package org.elasticsearch.ingest.processor; +package org.elasticsearch.ingest; import org.elasticsearch.ingest.core.AbstractProcessor; import org.elasticsearch.ingest.core.AbstractProcessorFactory; diff --git a/core/src/main/java/org/elasticsearch/ingest/processor/ConvertProcessor.java b/modules/ingest-common/src/main/java/org/elasticsearch/ingest/ConvertProcessor.java similarity index 97% rename from core/src/main/java/org/elasticsearch/ingest/processor/ConvertProcessor.java rename to modules/ingest-common/src/main/java/org/elasticsearch/ingest/ConvertProcessor.java index 1a6ce94e3db..d019ab01ef7 100644 --- a/core/src/main/java/org/elasticsearch/ingest/processor/ConvertProcessor.java +++ b/modules/ingest-common/src/main/java/org/elasticsearch/ingest/ConvertProcessor.java @@ -17,7 +17,7 @@ * under the License. */ -package org.elasticsearch.ingest.processor; +package org.elasticsearch.ingest; import org.elasticsearch.ingest.core.AbstractProcessor; import org.elasticsearch.ingest.core.AbstractProcessorFactory; @@ -103,7 +103,8 @@ public final class ConvertProcessor extends AbstractProcessor { try { return Type.valueOf(type.toUpperCase(Locale.ROOT)); } catch(IllegalArgumentException e) { - throw newConfigurationException(TYPE, processorTag, propertyName, "type [" + type + "] not supported, cannot convert field."); + throw newConfigurationException(TYPE, processorTag, propertyName, "type [" + type + + "] not supported, cannot convert field."); } } } diff --git a/core/src/main/java/org/elasticsearch/ingest/processor/DateFormat.java b/modules/ingest-common/src/main/java/org/elasticsearch/ingest/DateFormat.java similarity index 98% rename from core/src/main/java/org/elasticsearch/ingest/processor/DateFormat.java rename to modules/ingest-common/src/main/java/org/elasticsearch/ingest/DateFormat.java index 282b29176bf..2e27bd72cd2 100644 --- a/core/src/main/java/org/elasticsearch/ingest/processor/DateFormat.java +++ b/modules/ingest-common/src/main/java/org/elasticsearch/ingest/DateFormat.java @@ -17,7 +17,7 @@ * under the License. */ -package org.elasticsearch.ingest.processor; +package org.elasticsearch.ingest; import org.joda.time.DateTime; import org.joda.time.DateTimeZone; diff --git a/core/src/main/java/org/elasticsearch/ingest/processor/DateIndexNameProcessor.java b/modules/ingest-common/src/main/java/org/elasticsearch/ingest/DateIndexNameProcessor.java similarity index 99% rename from core/src/main/java/org/elasticsearch/ingest/processor/DateIndexNameProcessor.java rename to modules/ingest-common/src/main/java/org/elasticsearch/ingest/DateIndexNameProcessor.java index 2de44addaf5..cad66a3187b 100644 --- a/core/src/main/java/org/elasticsearch/ingest/processor/DateIndexNameProcessor.java +++ b/modules/ingest-common/src/main/java/org/elasticsearch/ingest/DateIndexNameProcessor.java @@ -17,7 +17,7 @@ * under the License. */ -package org.elasticsearch.ingest.processor; +package org.elasticsearch.ingest; import org.elasticsearch.ExceptionsHelper; import org.elasticsearch.ingest.core.AbstractProcessor; diff --git a/core/src/main/java/org/elasticsearch/ingest/processor/DateProcessor.java b/modules/ingest-common/src/main/java/org/elasticsearch/ingest/DateProcessor.java similarity index 99% rename from core/src/main/java/org/elasticsearch/ingest/processor/DateProcessor.java rename to modules/ingest-common/src/main/java/org/elasticsearch/ingest/DateProcessor.java index 8a1e58dc622..273b42d2b8c 100644 --- a/core/src/main/java/org/elasticsearch/ingest/processor/DateProcessor.java +++ b/modules/ingest-common/src/main/java/org/elasticsearch/ingest/DateProcessor.java @@ -17,7 +17,7 @@ * under the License. */ -package org.elasticsearch.ingest.processor; +package org.elasticsearch.ingest; import org.elasticsearch.ExceptionsHelper; import org.elasticsearch.ingest.core.AbstractProcessor; diff --git a/core/src/main/java/org/elasticsearch/ingest/processor/FailProcessor.java b/modules/ingest-common/src/main/java/org/elasticsearch/ingest/FailProcessor.java similarity index 98% rename from core/src/main/java/org/elasticsearch/ingest/processor/FailProcessor.java rename to modules/ingest-common/src/main/java/org/elasticsearch/ingest/FailProcessor.java index 7fdbca06c2b..2e0b79501e0 100644 --- a/core/src/main/java/org/elasticsearch/ingest/processor/FailProcessor.java +++ b/modules/ingest-common/src/main/java/org/elasticsearch/ingest/FailProcessor.java @@ -17,7 +17,7 @@ * under the License. */ -package org.elasticsearch.ingest.processor; +package org.elasticsearch.ingest; import org.elasticsearch.ingest.core.AbstractProcessor; import org.elasticsearch.ingest.core.AbstractProcessorFactory; diff --git a/core/src/main/java/org/elasticsearch/ingest/processor/FailProcessorException.java b/modules/ingest-common/src/main/java/org/elasticsearch/ingest/FailProcessorException.java similarity index 96% rename from core/src/main/java/org/elasticsearch/ingest/processor/FailProcessorException.java rename to modules/ingest-common/src/main/java/org/elasticsearch/ingest/FailProcessorException.java index bfdfe11178e..bd010b1423c 100644 --- a/core/src/main/java/org/elasticsearch/ingest/processor/FailProcessorException.java +++ b/modules/ingest-common/src/main/java/org/elasticsearch/ingest/FailProcessorException.java @@ -17,7 +17,7 @@ * under the License. */ -package org.elasticsearch.ingest.processor; +package org.elasticsearch.ingest; /** * Exception class thrown by {@link FailProcessor}. diff --git a/core/src/main/java/org/elasticsearch/ingest/processor/ForEachProcessor.java b/modules/ingest-common/src/main/java/org/elasticsearch/ingest/ForEachProcessor.java similarity index 97% rename from core/src/main/java/org/elasticsearch/ingest/processor/ForEachProcessor.java rename to modules/ingest-common/src/main/java/org/elasticsearch/ingest/ForEachProcessor.java index b6f6a85d219..2ca895779af 100644 --- a/core/src/main/java/org/elasticsearch/ingest/processor/ForEachProcessor.java +++ b/modules/ingest-common/src/main/java/org/elasticsearch/ingest/ForEachProcessor.java @@ -17,9 +17,8 @@ * under the License. */ -package org.elasticsearch.ingest.processor; +package org.elasticsearch.ingest; -import org.elasticsearch.ingest.ProcessorsRegistry; import org.elasticsearch.ingest.core.AbstractProcessor; import org.elasticsearch.ingest.core.AbstractProcessorFactory; import org.elasticsearch.ingest.core.ConfigurationUtils; diff --git a/modules/ingest-grok/src/main/java/org/elasticsearch/ingest/grok/Grok.java b/modules/ingest-common/src/main/java/org/elasticsearch/ingest/Grok.java similarity index 93% rename from modules/ingest-grok/src/main/java/org/elasticsearch/ingest/grok/Grok.java rename to modules/ingest-common/src/main/java/org/elasticsearch/ingest/Grok.java index abed8413287..7e5d1e6b516 100644 --- a/modules/ingest-grok/src/main/java/org/elasticsearch/ingest/grok/Grok.java +++ b/modules/ingest-common/src/main/java/org/elasticsearch/ingest/Grok.java @@ -17,7 +17,7 @@ * under the License. */ -package org.elasticsearch.ingest.grok; +package org.elasticsearch.ingest; import org.jcodings.specific.UTF8Encoding; import org.joni.Matcher; @@ -52,7 +52,8 @@ final class Grok { ")+" + ")" + ")?" + "\\}"; - private static final Regex GROK_PATTERN_REGEX = new Regex(GROK_PATTERN.getBytes(StandardCharsets.UTF_8), 0, GROK_PATTERN.getBytes(StandardCharsets.UTF_8).length, Option.NONE, UTF8Encoding.INSTANCE, Syntax.DEFAULT); + private static final Regex GROK_PATTERN_REGEX = new Regex(GROK_PATTERN.getBytes(StandardCharsets.UTF_8), 0, + GROK_PATTERN.getBytes(StandardCharsets.UTF_8).length, Option.NONE, UTF8Encoding.INSTANCE, Syntax.DEFAULT); private final Map patternBank; private final boolean namedCaptures; private final Regex compiledExpression; @@ -76,7 +77,8 @@ final class Grok { public String groupMatch(String name, Region region, String pattern) { try { - int number = GROK_PATTERN_REGEX.nameToBackrefNumber(name.getBytes(StandardCharsets.UTF_8), 0, name.getBytes(StandardCharsets.UTF_8).length, region); + int number = GROK_PATTERN_REGEX.nameToBackrefNumber(name.getBytes(StandardCharsets.UTF_8), 0, + name.getBytes(StandardCharsets.UTF_8).length, region); int begin = region.beg[number]; int end = region.end[number]; return new String(pattern.getBytes(StandardCharsets.UTF_8), begin, end - begin, StandardCharsets.UTF_8); @@ -143,7 +145,8 @@ final class Grok { String groupName = new String(e.name, e.nameP, e.nameEnd - e.nameP, StandardCharsets.UTF_8); String matchValue = null; if (region.beg[number] >= 0) { - matchValue = new String(textAsBytes, region.beg[number], region.end[number] - region.beg[number], StandardCharsets.UTF_8); + matchValue = new String(textAsBytes, region.beg[number], region.end[number] - region.beg[number], + StandardCharsets.UTF_8); } GrokMatchGroup match = new GrokMatchGroup(groupName, matchValue); fields.put(match.getName(), match.getValue()); diff --git a/modules/ingest-grok/src/main/java/org/elasticsearch/ingest/grok/GrokMatchGroup.java b/modules/ingest-common/src/main/java/org/elasticsearch/ingest/GrokMatchGroup.java similarity index 97% rename from modules/ingest-grok/src/main/java/org/elasticsearch/ingest/grok/GrokMatchGroup.java rename to modules/ingest-common/src/main/java/org/elasticsearch/ingest/GrokMatchGroup.java index 2cebf620c96..6fd979f179a 100644 --- a/modules/ingest-grok/src/main/java/org/elasticsearch/ingest/grok/GrokMatchGroup.java +++ b/modules/ingest-common/src/main/java/org/elasticsearch/ingest/GrokMatchGroup.java @@ -17,7 +17,7 @@ * under the License. */ -package org.elasticsearch.ingest.grok; +package org.elasticsearch.ingest; final class GrokMatchGroup { private static final String DEFAULT_TYPE = "string"; diff --git a/modules/ingest-grok/src/main/java/org/elasticsearch/ingest/grok/GrokProcessor.java b/modules/ingest-common/src/main/java/org/elasticsearch/ingest/GrokProcessor.java similarity index 99% rename from modules/ingest-grok/src/main/java/org/elasticsearch/ingest/grok/GrokProcessor.java rename to modules/ingest-common/src/main/java/org/elasticsearch/ingest/GrokProcessor.java index 6a10c8db058..b709df6e7d4 100644 --- a/modules/ingest-grok/src/main/java/org/elasticsearch/ingest/grok/GrokProcessor.java +++ b/modules/ingest-common/src/main/java/org/elasticsearch/ingest/GrokProcessor.java @@ -17,7 +17,7 @@ * under the License. */ -package org.elasticsearch.ingest.grok; +package org.elasticsearch.ingest; import org.elasticsearch.ingest.core.AbstractProcessor; import org.elasticsearch.ingest.core.AbstractProcessorFactory; diff --git a/core/src/main/java/org/elasticsearch/ingest/processor/GsubProcessor.java b/modules/ingest-common/src/main/java/org/elasticsearch/ingest/GsubProcessor.java similarity index 95% rename from core/src/main/java/org/elasticsearch/ingest/processor/GsubProcessor.java rename to modules/ingest-common/src/main/java/org/elasticsearch/ingest/GsubProcessor.java index d986bf522e5..8b17766f845 100644 --- a/core/src/main/java/org/elasticsearch/ingest/processor/GsubProcessor.java +++ b/modules/ingest-common/src/main/java/org/elasticsearch/ingest/GsubProcessor.java @@ -17,13 +17,11 @@ * under the License. */ -package org.elasticsearch.ingest.processor; +package org.elasticsearch.ingest; -import org.elasticsearch.ElasticsearchParseException; import org.elasticsearch.ingest.core.AbstractProcessor; import org.elasticsearch.ingest.core.AbstractProcessorFactory; import org.elasticsearch.ingest.core.IngestDocument; -import org.elasticsearch.ingest.core.ConfigurationUtils; import java.util.Map; import java.util.regex.Matcher; diff --git a/modules/ingest-common/src/main/java/org/elasticsearch/ingest/IngestCommonPlugin.java b/modules/ingest-common/src/main/java/org/elasticsearch/ingest/IngestCommonPlugin.java new file mode 100644 index 00000000000..8e5a821b8a5 --- /dev/null +++ b/modules/ingest-common/src/main/java/org/elasticsearch/ingest/IngestCommonPlugin.java @@ -0,0 +1,107 @@ +/* + * 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.ingest; + +import org.elasticsearch.node.NodeModule; +import org.elasticsearch.plugins.Plugin; + +import java.io.BufferedReader; +import java.io.IOException; +import java.io.InputStream; +import java.io.InputStreamReader; +import java.nio.charset.StandardCharsets; +import java.util.Collections; +import java.util.HashMap; +import java.util.Map; + +public class IngestCommonPlugin extends Plugin { + + public static final String NAME = "ingest-common"; + + private final Map builtinPatterns; + + public IngestCommonPlugin() throws IOException { + this.builtinPatterns = loadBuiltinPatterns(); + } + + @Override + public String name() { + return NAME; + } + + @Override + public String description() { + return "Module for ingest processors that do not require additional security permissions or have large dependencies and resources"; + } + + public void onModule(NodeModule nodeModule) { + nodeModule.registerProcessor(DateProcessor.TYPE, (templateService, registry) -> new DateProcessor.Factory()); + nodeModule.registerProcessor(SetProcessor.TYPE, (templateService, registry) -> new SetProcessor.Factory(templateService)); + nodeModule.registerProcessor(AppendProcessor.TYPE, (templateService, registry) -> new AppendProcessor.Factory(templateService)); + nodeModule.registerProcessor(RenameProcessor.TYPE, (templateService, registry) -> new RenameProcessor.Factory()); + nodeModule.registerProcessor(RemoveProcessor.TYPE, (templateService, registry) -> new RemoveProcessor.Factory(templateService)); + nodeModule.registerProcessor(SplitProcessor.TYPE, (templateService, registry) -> new SplitProcessor.Factory()); + nodeModule.registerProcessor(JoinProcessor.TYPE, (templateService, registry) -> new JoinProcessor.Factory()); + nodeModule.registerProcessor(UppercaseProcessor.TYPE, (templateService, registry) -> new UppercaseProcessor.Factory()); + nodeModule.registerProcessor(LowercaseProcessor.TYPE, (templateService, registry) -> new LowercaseProcessor.Factory()); + nodeModule.registerProcessor(TrimProcessor.TYPE, (templateService, registry) -> new TrimProcessor.Factory()); + nodeModule.registerProcessor(ConvertProcessor.TYPE, (templateService, registry) -> new ConvertProcessor.Factory()); + nodeModule.registerProcessor(GsubProcessor.TYPE, (templateService, registry) -> new GsubProcessor.Factory()); + nodeModule.registerProcessor(FailProcessor.TYPE, (templateService, registry) -> new FailProcessor.Factory(templateService)); + nodeModule.registerProcessor(ForEachProcessor.TYPE, (templateService, registry) -> new ForEachProcessor.Factory(registry)); + nodeModule.registerProcessor(DateIndexNameProcessor.TYPE, (templateService, registry) -> new DateIndexNameProcessor.Factory()); + nodeModule.registerProcessor(SortProcessor.TYPE, (templateService, registry) -> new SortProcessor.Factory()); + nodeModule.registerProcessor(GrokProcessor.TYPE, (templateService, registry) -> new GrokProcessor.Factory(builtinPatterns)); + } + + // Code for loading built-in grok patterns packaged with the jar file: + + private static final String[] PATTERN_NAMES = new String[] { + "aws", "bacula", "bro", "exim", "firewalls", "grok-patterns", "haproxy", + "java", "junos", "linux-syslog", "mcollective-patterns", "mongodb", "nagios", + "postgresql", "rails", "redis", "ruby" + }; + + public static Map loadBuiltinPatterns() throws IOException { + Map builtinPatterns = new HashMap<>(); + for (String pattern : PATTERN_NAMES) { + try(InputStream is = IngestCommonPlugin.class.getResourceAsStream("/patterns/" + pattern)) { + loadPatterns(builtinPatterns, is); + } + } + return Collections.unmodifiableMap(builtinPatterns); + } + + private static void loadPatterns(Map patternBank, InputStream inputStream) throws IOException { + String line; + BufferedReader br = new BufferedReader(new InputStreamReader(inputStream, StandardCharsets.UTF_8)); + while ((line = br.readLine()) != null) { + String trimmedLine = line.replaceAll("^\\s+", ""); + if (trimmedLine.startsWith("#") || trimmedLine.length() == 0) { + continue; + } + + String[] parts = trimmedLine.split("\\s+", 2); + if (parts.length == 2) { + patternBank.put(parts[0], parts[1]); + } + } + } +} diff --git a/core/src/main/java/org/elasticsearch/ingest/processor/JoinProcessor.java b/modules/ingest-common/src/main/java/org/elasticsearch/ingest/JoinProcessor.java similarity index 98% rename from core/src/main/java/org/elasticsearch/ingest/processor/JoinProcessor.java rename to modules/ingest-common/src/main/java/org/elasticsearch/ingest/JoinProcessor.java index cd7655dd953..7db5e00b145 100644 --- a/core/src/main/java/org/elasticsearch/ingest/processor/JoinProcessor.java +++ b/modules/ingest-common/src/main/java/org/elasticsearch/ingest/JoinProcessor.java @@ -17,7 +17,7 @@ * under the License. */ -package org.elasticsearch.ingest.processor; +package org.elasticsearch.ingest; import org.elasticsearch.ingest.core.AbstractProcessor; import org.elasticsearch.ingest.core.AbstractProcessorFactory; diff --git a/core/src/main/java/org/elasticsearch/ingest/processor/LowercaseProcessor.java b/modules/ingest-common/src/main/java/org/elasticsearch/ingest/LowercaseProcessor.java similarity index 97% rename from core/src/main/java/org/elasticsearch/ingest/processor/LowercaseProcessor.java rename to modules/ingest-common/src/main/java/org/elasticsearch/ingest/LowercaseProcessor.java index deb2809e711..8d5aebcf115 100644 --- a/core/src/main/java/org/elasticsearch/ingest/processor/LowercaseProcessor.java +++ b/modules/ingest-common/src/main/java/org/elasticsearch/ingest/LowercaseProcessor.java @@ -17,7 +17,7 @@ * under the License. */ -package org.elasticsearch.ingest.processor; +package org.elasticsearch.ingest; import java.util.Locale; diff --git a/core/src/main/java/org/elasticsearch/ingest/processor/RemoveProcessor.java b/modules/ingest-common/src/main/java/org/elasticsearch/ingest/RemoveProcessor.java similarity index 98% rename from core/src/main/java/org/elasticsearch/ingest/processor/RemoveProcessor.java rename to modules/ingest-common/src/main/java/org/elasticsearch/ingest/RemoveProcessor.java index 3d1aff2741a..6670c462d6f 100644 --- a/core/src/main/java/org/elasticsearch/ingest/processor/RemoveProcessor.java +++ b/modules/ingest-common/src/main/java/org/elasticsearch/ingest/RemoveProcessor.java @@ -17,7 +17,7 @@ * under the License. */ -package org.elasticsearch.ingest.processor; +package org.elasticsearch.ingest; import org.elasticsearch.ingest.core.AbstractProcessor; import org.elasticsearch.ingest.core.AbstractProcessorFactory; diff --git a/core/src/main/java/org/elasticsearch/ingest/processor/RenameProcessor.java b/modules/ingest-common/src/main/java/org/elasticsearch/ingest/RenameProcessor.java similarity index 98% rename from core/src/main/java/org/elasticsearch/ingest/processor/RenameProcessor.java rename to modules/ingest-common/src/main/java/org/elasticsearch/ingest/RenameProcessor.java index ea2f02d066a..4a6e8bf3ae5 100644 --- a/core/src/main/java/org/elasticsearch/ingest/processor/RenameProcessor.java +++ b/modules/ingest-common/src/main/java/org/elasticsearch/ingest/RenameProcessor.java @@ -17,7 +17,7 @@ * under the License. */ -package org.elasticsearch.ingest.processor; +package org.elasticsearch.ingest; import org.elasticsearch.ingest.core.AbstractProcessor; import org.elasticsearch.ingest.core.AbstractProcessorFactory; diff --git a/core/src/main/java/org/elasticsearch/ingest/processor/SetProcessor.java b/modules/ingest-common/src/main/java/org/elasticsearch/ingest/SetProcessor.java similarity index 98% rename from core/src/main/java/org/elasticsearch/ingest/processor/SetProcessor.java rename to modules/ingest-common/src/main/java/org/elasticsearch/ingest/SetProcessor.java index 6a67aca6bfb..6f576dcafc0 100644 --- a/core/src/main/java/org/elasticsearch/ingest/processor/SetProcessor.java +++ b/modules/ingest-common/src/main/java/org/elasticsearch/ingest/SetProcessor.java @@ -17,7 +17,7 @@ * under the License. */ -package org.elasticsearch.ingest.processor; +package org.elasticsearch.ingest; import org.elasticsearch.ingest.core.AbstractProcessor; import org.elasticsearch.ingest.core.AbstractProcessorFactory; diff --git a/core/src/main/java/org/elasticsearch/ingest/processor/SortProcessor.java b/modules/ingest-common/src/main/java/org/elasticsearch/ingest/SortProcessor.java similarity index 96% rename from core/src/main/java/org/elasticsearch/ingest/processor/SortProcessor.java rename to modules/ingest-common/src/main/java/org/elasticsearch/ingest/SortProcessor.java index 313a8bed7fc..ee708d97d47 100644 --- a/core/src/main/java/org/elasticsearch/ingest/processor/SortProcessor.java +++ b/modules/ingest-common/src/main/java/org/elasticsearch/ingest/SortProcessor.java @@ -17,19 +17,16 @@ * under the License. */ -package org.elasticsearch.ingest.processor; +package org.elasticsearch.ingest; -import org.elasticsearch.common.ParseField; import org.elasticsearch.ingest.core.AbstractProcessor; import org.elasticsearch.ingest.core.AbstractProcessorFactory; import org.elasticsearch.ingest.core.IngestDocument; import org.elasticsearch.ingest.core.ConfigurationUtils; import java.util.Collections; -import java.util.Comparator; import java.util.List; import java.util.Map; -import java.util.stream.Collectors; /** * Processor that sorts an array of items. diff --git a/core/src/main/java/org/elasticsearch/ingest/processor/SplitProcessor.java b/modules/ingest-common/src/main/java/org/elasticsearch/ingest/SplitProcessor.java similarity index 98% rename from core/src/main/java/org/elasticsearch/ingest/processor/SplitProcessor.java rename to modules/ingest-common/src/main/java/org/elasticsearch/ingest/SplitProcessor.java index aca4fe7026f..2d43cae3f09 100644 --- a/core/src/main/java/org/elasticsearch/ingest/processor/SplitProcessor.java +++ b/modules/ingest-common/src/main/java/org/elasticsearch/ingest/SplitProcessor.java @@ -17,7 +17,7 @@ * under the License. */ -package org.elasticsearch.ingest.processor; +package org.elasticsearch.ingest; import org.elasticsearch.ingest.core.AbstractProcessor; import org.elasticsearch.ingest.core.AbstractProcessorFactory; diff --git a/core/src/main/java/org/elasticsearch/ingest/processor/TrimProcessor.java b/modules/ingest-common/src/main/java/org/elasticsearch/ingest/TrimProcessor.java similarity index 97% rename from core/src/main/java/org/elasticsearch/ingest/processor/TrimProcessor.java rename to modules/ingest-common/src/main/java/org/elasticsearch/ingest/TrimProcessor.java index 919264fb866..294cd59265f 100644 --- a/core/src/main/java/org/elasticsearch/ingest/processor/TrimProcessor.java +++ b/modules/ingest-common/src/main/java/org/elasticsearch/ingest/TrimProcessor.java @@ -17,7 +17,7 @@ * under the License. */ -package org.elasticsearch.ingest.processor; +package org.elasticsearch.ingest; /** * Processor that trims the content of string fields. diff --git a/core/src/main/java/org/elasticsearch/ingest/processor/UppercaseProcessor.java b/modules/ingest-common/src/main/java/org/elasticsearch/ingest/UppercaseProcessor.java similarity index 97% rename from core/src/main/java/org/elasticsearch/ingest/processor/UppercaseProcessor.java rename to modules/ingest-common/src/main/java/org/elasticsearch/ingest/UppercaseProcessor.java index 7b3678b89ee..cd51f0dbc71 100644 --- a/core/src/main/java/org/elasticsearch/ingest/processor/UppercaseProcessor.java +++ b/modules/ingest-common/src/main/java/org/elasticsearch/ingest/UppercaseProcessor.java @@ -17,7 +17,7 @@ * under the License. */ -package org.elasticsearch.ingest.processor; +package org.elasticsearch.ingest; import java.util.Locale; diff --git a/modules/ingest-grok/src/main/resources/patterns/aws b/modules/ingest-common/src/main/resources/patterns/aws similarity index 100% rename from modules/ingest-grok/src/main/resources/patterns/aws rename to modules/ingest-common/src/main/resources/patterns/aws diff --git a/modules/ingest-grok/src/main/resources/patterns/bacula b/modules/ingest-common/src/main/resources/patterns/bacula similarity index 100% rename from modules/ingest-grok/src/main/resources/patterns/bacula rename to modules/ingest-common/src/main/resources/patterns/bacula diff --git a/modules/ingest-grok/src/main/resources/patterns/bro b/modules/ingest-common/src/main/resources/patterns/bro similarity index 100% rename from modules/ingest-grok/src/main/resources/patterns/bro rename to modules/ingest-common/src/main/resources/patterns/bro diff --git a/modules/ingest-grok/src/main/resources/patterns/exim b/modules/ingest-common/src/main/resources/patterns/exim similarity index 100% rename from modules/ingest-grok/src/main/resources/patterns/exim rename to modules/ingest-common/src/main/resources/patterns/exim diff --git a/modules/ingest-grok/src/main/resources/patterns/firewalls b/modules/ingest-common/src/main/resources/patterns/firewalls similarity index 100% rename from modules/ingest-grok/src/main/resources/patterns/firewalls rename to modules/ingest-common/src/main/resources/patterns/firewalls diff --git a/modules/ingest-grok/src/main/resources/patterns/grok-patterns b/modules/ingest-common/src/main/resources/patterns/grok-patterns similarity index 100% rename from modules/ingest-grok/src/main/resources/patterns/grok-patterns rename to modules/ingest-common/src/main/resources/patterns/grok-patterns diff --git a/modules/ingest-grok/src/main/resources/patterns/haproxy b/modules/ingest-common/src/main/resources/patterns/haproxy similarity index 100% rename from modules/ingest-grok/src/main/resources/patterns/haproxy rename to modules/ingest-common/src/main/resources/patterns/haproxy diff --git a/modules/ingest-grok/src/main/resources/patterns/java b/modules/ingest-common/src/main/resources/patterns/java similarity index 100% rename from modules/ingest-grok/src/main/resources/patterns/java rename to modules/ingest-common/src/main/resources/patterns/java diff --git a/modules/ingest-grok/src/main/resources/patterns/junos b/modules/ingest-common/src/main/resources/patterns/junos similarity index 100% rename from modules/ingest-grok/src/main/resources/patterns/junos rename to modules/ingest-common/src/main/resources/patterns/junos diff --git a/modules/ingest-grok/src/main/resources/patterns/linux-syslog b/modules/ingest-common/src/main/resources/patterns/linux-syslog similarity index 100% rename from modules/ingest-grok/src/main/resources/patterns/linux-syslog rename to modules/ingest-common/src/main/resources/patterns/linux-syslog diff --git a/modules/ingest-grok/src/main/resources/patterns/mcollective-patterns b/modules/ingest-common/src/main/resources/patterns/mcollective-patterns similarity index 100% rename from modules/ingest-grok/src/main/resources/patterns/mcollective-patterns rename to modules/ingest-common/src/main/resources/patterns/mcollective-patterns diff --git a/modules/ingest-grok/src/main/resources/patterns/mongodb b/modules/ingest-common/src/main/resources/patterns/mongodb similarity index 100% rename from modules/ingest-grok/src/main/resources/patterns/mongodb rename to modules/ingest-common/src/main/resources/patterns/mongodb diff --git a/modules/ingest-grok/src/main/resources/patterns/nagios b/modules/ingest-common/src/main/resources/patterns/nagios similarity index 100% rename from modules/ingest-grok/src/main/resources/patterns/nagios rename to modules/ingest-common/src/main/resources/patterns/nagios diff --git a/modules/ingest-grok/src/main/resources/patterns/postgresql b/modules/ingest-common/src/main/resources/patterns/postgresql similarity index 100% rename from modules/ingest-grok/src/main/resources/patterns/postgresql rename to modules/ingest-common/src/main/resources/patterns/postgresql diff --git a/modules/ingest-grok/src/main/resources/patterns/rails b/modules/ingest-common/src/main/resources/patterns/rails similarity index 100% rename from modules/ingest-grok/src/main/resources/patterns/rails rename to modules/ingest-common/src/main/resources/patterns/rails diff --git a/modules/ingest-grok/src/main/resources/patterns/redis b/modules/ingest-common/src/main/resources/patterns/redis similarity index 100% rename from modules/ingest-grok/src/main/resources/patterns/redis rename to modules/ingest-common/src/main/resources/patterns/redis diff --git a/modules/ingest-grok/src/main/resources/patterns/ruby b/modules/ingest-common/src/main/resources/patterns/ruby similarity index 100% rename from modules/ingest-grok/src/main/resources/patterns/ruby rename to modules/ingest-common/src/main/resources/patterns/ruby diff --git a/core/src/test/java/org/elasticsearch/ingest/processor/AbstractStringProcessorTestCase.java b/modules/ingest-common/src/test/java/org/elasticsearch/ingest/AbstractStringProcessorTestCase.java similarity index 95% rename from core/src/test/java/org/elasticsearch/ingest/processor/AbstractStringProcessorTestCase.java rename to modules/ingest-common/src/test/java/org/elasticsearch/ingest/AbstractStringProcessorTestCase.java index 1113a4b402f..b1f517b3d88 100644 --- a/core/src/test/java/org/elasticsearch/ingest/processor/AbstractStringProcessorTestCase.java +++ b/modules/ingest-common/src/test/java/org/elasticsearch/ingest/AbstractStringProcessorTestCase.java @@ -17,11 +17,10 @@ * under the License. */ -package org.elasticsearch.ingest.processor; +package org.elasticsearch.ingest; import org.elasticsearch.ingest.core.IngestDocument; import org.elasticsearch.ingest.core.Processor; -import org.elasticsearch.ingest.RandomDocumentPicks; import org.elasticsearch.test.ESTestCase; import java.util.Collections; @@ -81,7 +80,8 @@ public abstract class AbstractStringProcessorTestCase extends ESTestCase { processor.execute(ingestDocument); fail("processor should have failed"); } catch (IllegalArgumentException e) { - assertThat(e.getMessage(), equalTo("field [" + fieldName + "] of type [java.lang.Integer] cannot be cast to [java.lang.String]")); + assertThat(e.getMessage(), equalTo("field [" + fieldName + + "] of type [java.lang.Integer] cannot be cast to [java.lang.String]")); } } } diff --git a/core/src/test/java/org/elasticsearch/ingest/processor/AppendProcessorFactoryTests.java b/modules/ingest-common/src/test/java/org/elasticsearch/ingest/AppendProcessorFactoryTests.java similarity index 97% rename from core/src/test/java/org/elasticsearch/ingest/processor/AppendProcessorFactoryTests.java rename to modules/ingest-common/src/test/java/org/elasticsearch/ingest/AppendProcessorFactoryTests.java index 7350e3d9c43..884bbb7903a 100644 --- a/core/src/test/java/org/elasticsearch/ingest/processor/AppendProcessorFactoryTests.java +++ b/modules/ingest-common/src/test/java/org/elasticsearch/ingest/AppendProcessorFactoryTests.java @@ -17,10 +17,9 @@ * under the License. */ -package org.elasticsearch.ingest.processor; +package org.elasticsearch.ingest; import org.elasticsearch.ElasticsearchParseException; -import org.elasticsearch.ingest.TestTemplateService; import org.elasticsearch.ingest.core.AbstractProcessorFactory; import org.elasticsearch.test.ESTestCase; import org.junit.Before; diff --git a/core/src/test/java/org/elasticsearch/ingest/processor/AppendProcessorTests.java b/modules/ingest-common/src/test/java/org/elasticsearch/ingest/AppendProcessorTests.java similarity index 97% rename from core/src/test/java/org/elasticsearch/ingest/processor/AppendProcessorTests.java rename to modules/ingest-common/src/test/java/org/elasticsearch/ingest/AppendProcessorTests.java index 4a78ba621ce..63a1bb9cce2 100644 --- a/core/src/test/java/org/elasticsearch/ingest/processor/AppendProcessorTests.java +++ b/modules/ingest-common/src/test/java/org/elasticsearch/ingest/AppendProcessorTests.java @@ -17,12 +17,10 @@ * under the License. */ -package org.elasticsearch.ingest.processor; +package org.elasticsearch.ingest; import org.elasticsearch.ingest.core.IngestDocument; -import org.elasticsearch.ingest.RandomDocumentPicks; import org.elasticsearch.ingest.core.TemplateService; -import org.elasticsearch.ingest.TestTemplateService; import org.elasticsearch.ingest.core.ValueSource; import org.elasticsearch.ingest.core.Processor; import org.elasticsearch.test.ESTestCase; @@ -158,7 +156,8 @@ public class AppendProcessorTests extends ESTestCase { private static Processor createAppendProcessor(String fieldName, Object fieldValue) { TemplateService templateService = TestTemplateService.instance(); - return new AppendProcessor(randomAsciiOfLength(10), templateService.compile(fieldName), ValueSource.wrap(fieldValue, templateService)); + return new AppendProcessor(randomAsciiOfLength(10), templateService.compile(fieldName), ValueSource.wrap(fieldValue, + templateService)); } private enum Scalar { diff --git a/core/src/test/java/org/elasticsearch/ingest/processor/ConvertProcessorFactoryTests.java b/modules/ingest-common/src/test/java/org/elasticsearch/ingest/ConvertProcessorFactoryTests.java similarity index 99% rename from core/src/test/java/org/elasticsearch/ingest/processor/ConvertProcessorFactoryTests.java rename to modules/ingest-common/src/test/java/org/elasticsearch/ingest/ConvertProcessorFactoryTests.java index f54f04c0cf8..ce043f6aed4 100644 --- a/core/src/test/java/org/elasticsearch/ingest/processor/ConvertProcessorFactoryTests.java +++ b/modules/ingest-common/src/test/java/org/elasticsearch/ingest/ConvertProcessorFactoryTests.java @@ -17,7 +17,7 @@ * under the License. */ -package org.elasticsearch.ingest.processor; +package org.elasticsearch.ingest; import org.elasticsearch.ElasticsearchParseException; import org.elasticsearch.ingest.core.AbstractProcessorFactory; diff --git a/core/src/test/java/org/elasticsearch/ingest/processor/ConvertProcessorTests.java b/modules/ingest-common/src/test/java/org/elasticsearch/ingest/ConvertProcessorTests.java similarity index 99% rename from core/src/test/java/org/elasticsearch/ingest/processor/ConvertProcessorTests.java rename to modules/ingest-common/src/test/java/org/elasticsearch/ingest/ConvertProcessorTests.java index 0efacb145bd..f9ec7883190 100644 --- a/core/src/test/java/org/elasticsearch/ingest/processor/ConvertProcessorTests.java +++ b/modules/ingest-common/src/test/java/org/elasticsearch/ingest/ConvertProcessorTests.java @@ -17,10 +17,9 @@ * under the License. */ -package org.elasticsearch.ingest.processor; +package org.elasticsearch.ingest; import org.elasticsearch.ingest.core.IngestDocument; -import org.elasticsearch.ingest.RandomDocumentPicks; import org.elasticsearch.ingest.core.Processor; import org.elasticsearch.test.ESTestCase; @@ -31,7 +30,7 @@ import java.util.List; import java.util.Locale; import java.util.Map; -import static org.elasticsearch.ingest.processor.ConvertProcessor.Type; +import static org.elasticsearch.ingest.ConvertProcessor.Type; import static org.hamcrest.Matchers.containsString; import static org.hamcrest.Matchers.equalTo; import static org.hamcrest.Matchers.sameInstance; diff --git a/core/src/test/java/org/elasticsearch/ingest/processor/DateFormatTests.java b/modules/ingest-common/src/test/java/org/elasticsearch/ingest/DateFormatTests.java similarity index 90% rename from core/src/test/java/org/elasticsearch/ingest/processor/DateFormatTests.java rename to modules/ingest-common/src/test/java/org/elasticsearch/ingest/DateFormatTests.java index 401dd44d44a..8fcff4789c7 100644 --- a/core/src/test/java/org/elasticsearch/ingest/processor/DateFormatTests.java +++ b/modules/ingest-common/src/test/java/org/elasticsearch/ingest/DateFormatTests.java @@ -17,7 +17,7 @@ * under the License. */ -package org.elasticsearch.ingest.processor; +package org.elasticsearch.ingest; import org.elasticsearch.test.ESTestCase; import org.joda.time.DateTime; @@ -34,7 +34,8 @@ import static org.hamcrest.core.IsEqual.equalTo; public class DateFormatTests extends ESTestCase { public void testParseJoda() { - Function jodaFunction = DateFormat.Joda.getFunction("MMM dd HH:mm:ss Z", DateTimeZone.forOffsetHours(-8), Locale.ENGLISH); + Function jodaFunction = DateFormat.Joda.getFunction("MMM dd HH:mm:ss Z", + DateTimeZone.forOffsetHours(-8), Locale.ENGLISH); assertThat(Instant.ofEpochMilli(jodaFunction.apply("Nov 24 01:29:01 -0800").getMillis()) .atZone(ZoneId.of("GMT-8")) .format(DateTimeFormatter.ofPattern("MM dd HH:mm:ss", Locale.ENGLISH)), @@ -50,7 +51,8 @@ public class DateFormatTests extends ESTestCase { } public void testParseISO8601() { - assertThat(DateFormat.Iso8601.getFunction(null, DateTimeZone.UTC, null).apply("2001-01-01T00:00:00-0800").getMillis(), equalTo(978336000000L)); + assertThat(DateFormat.Iso8601.getFunction(null, DateTimeZone.UTC, null).apply("2001-01-01T00:00:00-0800").getMillis(), + equalTo(978336000000L)); } public void testParseISO8601Failure() { @@ -66,7 +68,8 @@ public class DateFormatTests extends ESTestCase { public void testTAI64NParse() { String input = "4000000050d506482dbdf024"; String expected = "2012-12-22T03:00:46.767+02:00"; - assertThat(DateFormat.Tai64n.getFunction(null, DateTimeZone.forOffsetHours(2), null).apply((randomBoolean() ? "@" : "") + input).toString(), equalTo(expected)); + assertThat(DateFormat.Tai64n.getFunction(null, DateTimeZone.forOffsetHours(2), null) + .apply((randomBoolean() ? "@" : "") + input).toString(), equalTo(expected)); } public void testFromString() { diff --git a/core/src/test/java/org/elasticsearch/ingest/processor/DateIndexNameFactoryTests.java b/modules/ingest-common/src/test/java/org/elasticsearch/ingest/DateIndexNameFactoryTests.java similarity index 98% rename from core/src/test/java/org/elasticsearch/ingest/processor/DateIndexNameFactoryTests.java rename to modules/ingest-common/src/test/java/org/elasticsearch/ingest/DateIndexNameFactoryTests.java index 5bb674e950a..d6b5f7a0ff7 100644 --- a/core/src/test/java/org/elasticsearch/ingest/processor/DateIndexNameFactoryTests.java +++ b/modules/ingest-common/src/test/java/org/elasticsearch/ingest/DateIndexNameFactoryTests.java @@ -17,7 +17,7 @@ * under the License. */ -package org.elasticsearch.ingest.processor; +package org.elasticsearch.ingest; import org.elasticsearch.ElasticsearchParseException; import org.elasticsearch.test.ESTestCase; diff --git a/core/src/test/java/org/elasticsearch/ingest/processor/DateIndexNameProcessorTests.java b/modules/ingest-common/src/test/java/org/elasticsearch/ingest/DateIndexNameProcessorTests.java similarity index 98% rename from core/src/test/java/org/elasticsearch/ingest/processor/DateIndexNameProcessorTests.java rename to modules/ingest-common/src/test/java/org/elasticsearch/ingest/DateIndexNameProcessorTests.java index 2dd5934387c..3baa3d3ac7c 100644 --- a/core/src/test/java/org/elasticsearch/ingest/processor/DateIndexNameProcessorTests.java +++ b/modules/ingest-common/src/test/java/org/elasticsearch/ingest/DateIndexNameProcessorTests.java @@ -16,7 +16,7 @@ * specific language governing permissions and limitations * under the License. */ -package org.elasticsearch.ingest.processor; +package org.elasticsearch.ingest; import org.elasticsearch.ingest.core.IngestDocument; import org.elasticsearch.test.ESTestCase; diff --git a/core/src/test/java/org/elasticsearch/ingest/processor/DateProcessorFactoryTests.java b/modules/ingest-common/src/test/java/org/elasticsearch/ingest/DateProcessorFactoryTests.java similarity index 98% rename from core/src/test/java/org/elasticsearch/ingest/processor/DateProcessorFactoryTests.java rename to modules/ingest-common/src/test/java/org/elasticsearch/ingest/DateProcessorFactoryTests.java index 68b5d175509..9933f3fbb65 100644 --- a/core/src/test/java/org/elasticsearch/ingest/processor/DateProcessorFactoryTests.java +++ b/modules/ingest-common/src/test/java/org/elasticsearch/ingest/DateProcessorFactoryTests.java @@ -17,18 +17,16 @@ * under the License. */ -package org.elasticsearch.ingest.processor; +package org.elasticsearch.ingest; import org.elasticsearch.ElasticsearchParseException; import org.elasticsearch.ingest.core.AbstractProcessorFactory; import org.elasticsearch.test.ESTestCase; import org.joda.time.DateTimeZone; -import java.util.ArrayList; import java.util.Arrays; import java.util.Collections; import java.util.HashMap; -import java.util.List; import java.util.Locale; import java.util.Map; diff --git a/core/src/test/java/org/elasticsearch/ingest/processor/DateProcessorTests.java b/modules/ingest-common/src/test/java/org/elasticsearch/ingest/DateProcessorTests.java similarity index 97% rename from core/src/test/java/org/elasticsearch/ingest/processor/DateProcessorTests.java rename to modules/ingest-common/src/test/java/org/elasticsearch/ingest/DateProcessorTests.java index 5daab95a5d0..2d32282d332 100644 --- a/core/src/test/java/org/elasticsearch/ingest/processor/DateProcessorTests.java +++ b/modules/ingest-common/src/test/java/org/elasticsearch/ingest/DateProcessorTests.java @@ -17,10 +17,9 @@ * under the License. */ -package org.elasticsearch.ingest.processor; +package org.elasticsearch.ingest; import org.elasticsearch.ingest.core.IngestDocument; -import org.elasticsearch.ingest.RandomDocumentPicks; import org.elasticsearch.test.ESTestCase; import org.joda.time.DateTime; import org.joda.time.DateTimeZone; @@ -111,7 +110,8 @@ public class DateProcessorTests extends ESTestCase { document.put("date_as_string", "12/06"); IngestDocument ingestDocument = RandomDocumentPicks.randomIngestDocument(random(), document); dateProcessor.execute(ingestDocument); - assertThat(ingestDocument.getFieldValue("date_as_date", String.class), equalTo(DateTime.now().getYear() + "-06-12T00:00:00.000+02:00")); + assertThat(ingestDocument.getFieldValue("date_as_date", String.class), equalTo(DateTime.now().getYear() + + "-06-12T00:00:00.000+02:00")); } public void testTAI64N() { diff --git a/core/src/test/java/org/elasticsearch/ingest/processor/FailProcessorFactoryTests.java b/modules/ingest-common/src/test/java/org/elasticsearch/ingest/FailProcessorFactoryTests.java similarity index 93% rename from core/src/test/java/org/elasticsearch/ingest/processor/FailProcessorFactoryTests.java rename to modules/ingest-common/src/test/java/org/elasticsearch/ingest/FailProcessorFactoryTests.java index 0d88710c80d..1f1e9c72b12 100644 --- a/core/src/test/java/org/elasticsearch/ingest/processor/FailProcessorFactoryTests.java +++ b/modules/ingest-common/src/test/java/org/elasticsearch/ingest/FailProcessorFactoryTests.java @@ -17,12 +17,10 @@ * under the License. */ -package org.elasticsearch.ingest.processor; +package org.elasticsearch.ingest; import org.elasticsearch.ElasticsearchParseException; -import org.elasticsearch.ingest.TestTemplateService; import org.elasticsearch.ingest.core.AbstractProcessorFactory; -import org.elasticsearch.ingest.core.Processor; import org.elasticsearch.test.ESTestCase; import org.junit.Before; diff --git a/core/src/test/java/org/elasticsearch/ingest/processor/FailProcessorTests.java b/modules/ingest-common/src/test/java/org/elasticsearch/ingest/FailProcessorTests.java similarity index 91% rename from core/src/test/java/org/elasticsearch/ingest/processor/FailProcessorTests.java rename to modules/ingest-common/src/test/java/org/elasticsearch/ingest/FailProcessorTests.java index 3fdc2073c09..457aaf45c66 100644 --- a/core/src/test/java/org/elasticsearch/ingest/processor/FailProcessorTests.java +++ b/modules/ingest-common/src/test/java/org/elasticsearch/ingest/FailProcessorTests.java @@ -17,12 +17,10 @@ * under the License. */ -package org.elasticsearch.ingest.processor; +package org.elasticsearch.ingest; import org.elasticsearch.ingest.core.IngestDocument; import org.elasticsearch.ingest.core.Processor; -import org.elasticsearch.ingest.RandomDocumentPicks; -import org.elasticsearch.ingest.TestTemplateService; import org.elasticsearch.test.ESTestCase; import static org.hamcrest.Matchers.equalTo; diff --git a/core/src/test/java/org/elasticsearch/ingest/processor/ForEachProcessorFactoryTests.java b/modules/ingest-common/src/test/java/org/elasticsearch/ingest/ForEachProcessorFactoryTests.java similarity index 93% rename from core/src/test/java/org/elasticsearch/ingest/processor/ForEachProcessorFactoryTests.java rename to modules/ingest-common/src/test/java/org/elasticsearch/ingest/ForEachProcessorFactoryTests.java index d03c2c2c546..74771b5768e 100644 --- a/core/src/test/java/org/elasticsearch/ingest/processor/ForEachProcessorFactoryTests.java +++ b/modules/ingest-common/src/test/java/org/elasticsearch/ingest/ForEachProcessorFactoryTests.java @@ -17,11 +17,8 @@ * under the License. */ -package org.elasticsearch.ingest.processor; +package org.elasticsearch.ingest; -import org.elasticsearch.ingest.ProcessorsRegistry; -import org.elasticsearch.ingest.TestProcessor; -import org.elasticsearch.ingest.TestTemplateService; import org.elasticsearch.ingest.core.Processor; import org.elasticsearch.test.ESTestCase; import org.hamcrest.Matchers; diff --git a/core/src/test/java/org/elasticsearch/ingest/processor/ForEachProcessorTests.java b/modules/ingest-common/src/test/java/org/elasticsearch/ingest/ForEachProcessorTests.java similarity index 98% rename from core/src/test/java/org/elasticsearch/ingest/processor/ForEachProcessorTests.java rename to modules/ingest-common/src/test/java/org/elasticsearch/ingest/ForEachProcessorTests.java index d1c2c6f1777..19f1e15f09e 100644 --- a/core/src/test/java/org/elasticsearch/ingest/processor/ForEachProcessorTests.java +++ b/modules/ingest-common/src/test/java/org/elasticsearch/ingest/ForEachProcessorTests.java @@ -17,10 +17,8 @@ * under the License. */ -package org.elasticsearch.ingest.processor; +package org.elasticsearch.ingest; -import org.elasticsearch.ingest.TestProcessor; -import org.elasticsearch.ingest.TestTemplateService; import org.elasticsearch.ingest.core.CompoundProcessor; import org.elasticsearch.ingest.core.IngestDocument; import org.elasticsearch.ingest.core.Processor; diff --git a/modules/ingest-grok/src/test/java/org/elasticsearch/ingest/grok/GrokProcessorFactoryTests.java b/modules/ingest-common/src/test/java/org/elasticsearch/ingest/GrokProcessorFactoryTests.java similarity index 99% rename from modules/ingest-grok/src/test/java/org/elasticsearch/ingest/grok/GrokProcessorFactoryTests.java rename to modules/ingest-common/src/test/java/org/elasticsearch/ingest/GrokProcessorFactoryTests.java index ec3b908aac0..374a15f4f25 100644 --- a/modules/ingest-grok/src/test/java/org/elasticsearch/ingest/grok/GrokProcessorFactoryTests.java +++ b/modules/ingest-common/src/test/java/org/elasticsearch/ingest/GrokProcessorFactoryTests.java @@ -17,7 +17,7 @@ * under the License. */ -package org.elasticsearch.ingest.grok; +package org.elasticsearch.ingest; import org.elasticsearch.ElasticsearchParseException; import org.elasticsearch.ingest.core.AbstractProcessorFactory; diff --git a/modules/ingest-grok/src/test/java/org/elasticsearch/ingest/grok/GrokProcessorTests.java b/modules/ingest-common/src/test/java/org/elasticsearch/ingest/GrokProcessorTests.java similarity index 99% rename from modules/ingest-grok/src/test/java/org/elasticsearch/ingest/grok/GrokProcessorTests.java rename to modules/ingest-common/src/test/java/org/elasticsearch/ingest/GrokProcessorTests.java index 82354a943ae..919cd5ca548 100644 --- a/modules/ingest-grok/src/test/java/org/elasticsearch/ingest/grok/GrokProcessorTests.java +++ b/modules/ingest-common/src/test/java/org/elasticsearch/ingest/GrokProcessorTests.java @@ -17,7 +17,7 @@ * under the License. */ -package org.elasticsearch.ingest.grok; +package org.elasticsearch.ingest; import org.elasticsearch.ingest.RandomDocumentPicks; import org.elasticsearch.ingest.core.IngestDocument; diff --git a/modules/ingest-grok/src/test/java/org/elasticsearch/ingest/grok/GrokTests.java b/modules/ingest-common/src/test/java/org/elasticsearch/ingest/GrokTests.java similarity index 79% rename from modules/ingest-grok/src/test/java/org/elasticsearch/ingest/grok/GrokTests.java rename to modules/ingest-common/src/test/java/org/elasticsearch/ingest/GrokTests.java index 21ca17ad713..fd757c4962c 100644 --- a/modules/ingest-grok/src/test/java/org/elasticsearch/ingest/grok/GrokTests.java +++ b/modules/ingest-common/src/test/java/org/elasticsearch/ingest/GrokTests.java @@ -17,7 +17,7 @@ * under the License. */ -package org.elasticsearch.ingest.grok; +package org.elasticsearch.ingest; import org.elasticsearch.test.ESTestCase; import org.junit.Before; @@ -38,7 +38,7 @@ public class GrokTests extends ESTestCase { @Before public void setup() throws IOException { - basePatterns = IngestGrokPlugin.loadBuiltinPatterns(); + basePatterns = IngestCommonPlugin.loadBuiltinPatterns(); } public void testMatchWithoutCaptures() { @@ -60,7 +60,8 @@ public class GrokTests extends ESTestCase { } public void testSyslog5424Line() { - String line = "<191>1 2009-06-30T18:30:00+02:00 paxton.local grokdebug 4123 - [id1 foo=\\\"bar\\\"][id2 baz=\\\"something\\\"] Hello, syslog."; + String line = "<191>1 2009-06-30T18:30:00+02:00 paxton.local grokdebug 4123 - [id1 foo=\\\"bar\\\"][id2 baz=\\\"something\\\"] " + + "Hello, syslog."; Grok grok = new Grok(basePatterns, "%{SYSLOG5424LINE}"); Map matches = grok.captures(line); assertEquals("191", matches.get("syslog5424_pri")); @@ -98,8 +99,12 @@ public class GrokTests extends ESTestCase { } public void testUnicodeSyslog() { - Grok grok = new Grok(basePatterns, "<%{POSINT:syslog_pri}>%{SPACE}%{SYSLOGTIMESTAMP:syslog_timestamp} %{SYSLOGHOST:syslog_hostname} %{PROG:syslog_program}(:?)(?:\\[%{GREEDYDATA:syslog_pid}\\])?(:?) %{GREEDYDATA:syslog_message}"); - Map matches = grok.captures("<22>Jan 4 07:50:46 mailmaster postfix/policy-spf[9454]: : SPF permerror (Junk encountered in record 'v=spf1 mx a:mail.domain.no ip4:192.168.0.4 �all'): Envelope-from: email@domain.no"); + Grok grok = new Grok(basePatterns, "<%{POSINT:syslog_pri}>%{SPACE}%{SYSLOGTIMESTAMP:syslog_timestamp} " + + "%{SYSLOGHOST:syslog_hostname} %{PROG:syslog_program}(:?)(?:\\[%{GREEDYDATA:syslog_pid}\\])?(:?) " + + "%{GREEDYDATA:syslog_message}"); + Map matches = grok.captures("<22>Jan 4 07:50:46 mailmaster postfix/policy-spf[9454]: : " + + "SPF permerror (Junk encountered in record 'v=spf1 mx a:mail.domain.no ip4:192.168.0.4 �all'): Envelope-from: " + + "email@domain.no"); assertThat(matches.get("syslog_pri"), equalTo("22")); assertThat(matches.get("syslog_program"), equalTo("postfix/policy-spf")); assertThat(matches.get("tags"), nullValue()); @@ -212,7 +217,9 @@ public class GrokTests extends ESTestCase { } public void testApacheLog() { - String logLine = "31.184.238.164 - - [24/Jul/2014:05:35:37 +0530] \"GET /logs/access.log HTTP/1.0\" 200 69849 \"http://8rursodiol.enjin.com\" \"Mozilla/5.0 (Windows NT 6.1; WOW64) AppleWebKit/537.36 (KHTML, like Gecko) Chrome/30.0.1599.12785 YaBrowser/13.12.1599.12785 Safari/537.36\" \"www.dlwindianrailways.com\""; + String logLine = "31.184.238.164 - - [24/Jul/2014:05:35:37 +0530] \"GET /logs/access.log HTTP/1.0\" 200 69849 " + + "\"http://8rursodiol.enjin.com\" \"Mozilla/5.0 (Windows NT 6.1; WOW64) AppleWebKit/537.36 (KHTML, like Gecko) " + + "Chrome/30.0.1599.12785 YaBrowser/13.12.1599.12785 Safari/537.36\" \"www.dlwindianrailways.com\""; Grok grok = new Grok(basePatterns, "%{COMBINEDAPACHELOG}"); Map matches = grok.captures(logLine); @@ -227,13 +234,15 @@ public class GrokTests extends ESTestCase { assertEquals("69849", matches.get("bytes")); assertEquals("\"http://8rursodiol.enjin.com\"", matches.get("referrer")); assertEquals(null, matches.get("port")); - assertEquals("\"Mozilla/5.0 (Windows NT 6.1; WOW64) AppleWebKit/537.36 (KHTML, like Gecko) Chrome/30.0.1599.12785 YaBrowser/13.12.1599.12785 Safari/537.36\"", matches.get("agent")); + assertEquals("\"Mozilla/5.0 (Windows NT 6.1; WOW64) AppleWebKit/537.36 (KHTML, like Gecko) Chrome/30.0.1599.12785 " + + "YaBrowser/13.12.1599.12785 Safari/537.36\"", matches.get("agent")); } public void testComplete() { Map bank = new HashMap<>(); bank.put("MONTHDAY", "(?:(?:0[1-9])|(?:[12][0-9])|(?:3[01])|[1-9])"); - bank.put("MONTH", "\\b(?:Jan(?:uary|uar)?|Feb(?:ruary|ruar)?|M(?:a|ä)?r(?:ch|z)?|Apr(?:il)?|Ma(?:y|i)?|Jun(?:e|i)?|Jul(?:y)?|Aug(?:ust)?|Sep(?:tember)?|O(?:c|k)?t(?:ober)?|Nov(?:ember)?|De(?:c|z)(?:ember)?)\\b"); + bank.put("MONTH", "\\b(?:Jan(?:uary|uar)?|Feb(?:ruary|ruar)?|M(?:a|ä)?r(?:ch|z)?|Apr(?:il)?|Ma(?:y|i)?|Jun(?:e|i)" + + "?|Jul(?:y)?|Aug(?:ust)?|Sep(?:tember)?|O(?:c|k)?t(?:ober)?|Nov(?:ember)?|De(?:c|z)(?:ember)?)\\b"); bank.put("MINUTE", "(?:[0-5][0-9])"); bank.put("YEAR", "(?>\\d\\d){1,2}"); bank.put("HOUR", "(?:2[0123]|[01]?[0-9])"); @@ -244,8 +253,19 @@ public class GrokTests extends ESTestCase { bank.put("WORD", "\\b\\w+\\b"); bank.put("BASE10NUM", "(?[+-]?(?:(?:[0-9]+(?:\\.[0-9]+)?)|(?:\\.[0-9]+)))"); bank.put("NUMBER", "(?:%{BASE10NUM})"); - bank.put("IPV6", "((([0-9A-Fa-f]{1,4}:){7}([0-9A-Fa-f]{1,4}|:))|(([0-9A-Fa-f]{1,4}:){6}(:[0-9A-Fa-f]{1,4}|((25[0-5]|2[0-4]\\d|1\\d\\d|[1-9]?\\d)(\\.(25[0-5]|2[0-4]\\d|1\\d\\d|[1-9]?\\d)){3})|:))|(([0-9A-Fa-f]{1,4}:){5}(((:[0-9A-Fa-f]{1,4}){1,2})|:((25[0-5]|2[0-4]\\d|1\\d\\d|[1-9]?\\d)(\\.(25[0-5]|2[0-4]\\d|1\\d\\d|[1-9]?\\d)){3})|:))|(([0-9A-Fa-f]{1,4}:){4}(((:[0-9A-Fa-f]{1,4}){1,3})|((:[0-9A-Fa-f]{1,4})?:((25[0-5]|2[0-4]\\d|1\\d\\d|[1-9]?\\d)(\\.(25[0-5]|2[0-4]\\d|1\\d\\d|[1-9]?\\d)){3}))|:))|(([0-9A-Fa-f]{1,4}:){3}(((:[0-9A-Fa-f]{1,4}){1,4})|((:[0-9A-Fa-f]{1,4}){0,2}:((25[0-5]|2[0-4]\\d|1\\d\\d|[1-9]?\\d)(\\.(25[0-5]|2[0-4]\\d|1\\d\\d|[1-9]?\\d)){3}))|:))|(([0-9A-Fa-f]{1,4}:){2}(((:[0-9A-Fa-f]{1,4}){1,5})|((:[0-9A-Fa-f]{1,4}){0,3}:((25[0-5]|2[0-4]\\d|1\\d\\d|[1-9]?\\d)(\\.(25[0-5]|2[0-4]\\d|1\\d\\d|[1-9]?\\d)){3}))|:))|(([0-9A-Fa-f]{1,4}:){1}(((:[0-9A-Fa-f]{1,4}){1,6})|((:[0-9A-Fa-f]{1,4}){0,4}:((25[0-5]|2[0-4]\\d|1\\d\\d|[1-9]?\\d)(\\.(25[0-5]|2[0-4]\\d|1\\d\\d|[1-9]?\\d)){3}))|:))|(:(((:[0-9A-Fa-f]{1,4}){1,7})|((:[0-9A-Fa-f]{1,4}){0,5}:((25[0-5]|2[0-4]\\d|1\\d\\d|[1-9]?\\d)(\\.(25[0-5]|2[0-4]\\d|1\\d\\d|[1-9]?\\d)){3}))|:)))(%.+)?"); - bank.put("IPV4", "(?(?\"(?>\\\\.|[^\\\\\"]+)+\"|\"\"|(?>'(?>\\\\.|[^\\\\']+)+')|''|(?>`(?>\\\\.|[^\\\\`]+)+`)|``))"); - String text = "83.149.9.216 - - [19/Jul/2015:08:13:42 +0000] \"GET /presentations/logstash-monitorama-2013/images/kibana-dashboard3.png HTTP/1.1\" 200 171717 \"http://semicomplete.com/presentations/logstash-monitorama-2013/\" \"Mozilla/5.0 (Macintosh; Intel Mac OS X 10_9_1) AppleWebKit/537.36 (KHTML, like Gecko) Chrome/32.0.1700.77 Safari/537.36\""; - String pattern = "%{IPORHOST:clientip} %{USER:ident} %{USER:auth} \\[%{HTTPDATE:timestamp}\\] \"%{WORD:verb} %{DATA:request} HTTP/%{NUMBER:httpversion}\" %{NUMBER:response:int} (?:-|%{NUMBER:bytes:int}) %{QS:referrer} %{QS:agent}"; + String text = "83.149.9.216 - - [19/Jul/2015:08:13:42 +0000] \"GET /presentations/logstash-monitorama-2013/images/" + + "kibana-dashboard3.png HTTP/1.1\" 200 171717 \"http://semicomplete.com/presentations/logstash-monitorama-2013/\" " + + "\"Mozilla" + + "/5.0 (Macintosh; Intel Mac OS X 10_9_1) AppleWebKit/537.36 (KHTML, like Gecko) Chrome/32.0.1700.77 Safari/537.36\""; + String pattern = "%{IPORHOST:clientip} %{USER:ident} %{USER:auth} \\[%{HTTPDATE:timestamp}\\] \"%{WORD:verb} %{DATA:request} " + + "HTTP/%{NUMBER:httpversion}\" %{NUMBER:response:int} (?:-|%{NUMBER:bytes:int}) %{QS:referrer} %{QS:agent}"; Grok grok = new Grok(bank, pattern); @@ -269,7 +293,8 @@ public class GrokTests extends ESTestCase { expected.put("response", 200); expected.put("bytes", 171717); expected.put("referrer", "\"http://semicomplete.com/presentations/logstash-monitorama-2013/\""); - expected.put("agent", "\"Mozilla/5.0 (Macintosh; Intel Mac OS X 10_9_1) AppleWebKit/537.36 (KHTML, like Gecko) Chrome/32.0.1700.77 Safari/537.36\""); + expected.put("agent", "\"Mozilla/5.0 (Macintosh; Intel Mac OS X 10_9_1) AppleWebKit/537.36 (KHTML, like Gecko) " + + "Chrome/32.0.1700.77 Safari/537.36\""); Map actual = grok.captures(text); diff --git a/core/src/test/java/org/elasticsearch/ingest/processor/GsubProcessorFactoryTests.java b/modules/ingest-common/src/test/java/org/elasticsearch/ingest/GsubProcessorFactoryTests.java similarity index 98% rename from core/src/test/java/org/elasticsearch/ingest/processor/GsubProcessorFactoryTests.java rename to modules/ingest-common/src/test/java/org/elasticsearch/ingest/GsubProcessorFactoryTests.java index 7f56429df9e..f840576e522 100644 --- a/core/src/test/java/org/elasticsearch/ingest/processor/GsubProcessorFactoryTests.java +++ b/modules/ingest-common/src/test/java/org/elasticsearch/ingest/GsubProcessorFactoryTests.java @@ -17,7 +17,7 @@ * under the License. */ -package org.elasticsearch.ingest.processor; +package org.elasticsearch.ingest; import org.elasticsearch.ElasticsearchParseException; import org.elasticsearch.ingest.core.AbstractProcessorFactory; diff --git a/core/src/test/java/org/elasticsearch/ingest/processor/GsubProcessorTests.java b/modules/ingest-common/src/test/java/org/elasticsearch/ingest/GsubProcessorTests.java similarity index 95% rename from core/src/test/java/org/elasticsearch/ingest/processor/GsubProcessorTests.java rename to modules/ingest-common/src/test/java/org/elasticsearch/ingest/GsubProcessorTests.java index fe44f33b614..066609f5e3a 100644 --- a/core/src/test/java/org/elasticsearch/ingest/processor/GsubProcessorTests.java +++ b/modules/ingest-common/src/test/java/org/elasticsearch/ingest/GsubProcessorTests.java @@ -17,10 +17,9 @@ * under the License. */ -package org.elasticsearch.ingest.processor; +package org.elasticsearch.ingest; import org.elasticsearch.ingest.core.IngestDocument; -import org.elasticsearch.ingest.RandomDocumentPicks; import org.elasticsearch.ingest.core.Processor; import org.elasticsearch.test.ESTestCase; @@ -50,7 +49,8 @@ public class GsubProcessorTests extends ESTestCase { processor.execute(ingestDocument); fail("processor execution should have failed"); } catch (IllegalArgumentException e) { - assertThat(e.getMessage(), equalTo("field [" + fieldName + "] of type [java.lang.Integer] cannot be cast to [java.lang.String]")); + assertThat(e.getMessage(), equalTo("field [" + fieldName + + "] of type [java.lang.Integer] cannot be cast to [java.lang.String]")); } } diff --git a/modules/ingest-grok/src/test/java/org/elasticsearch/ingest/grok/IngestGrokRestIT.java b/modules/ingest-common/src/test/java/org/elasticsearch/ingest/IngestCommonRestIT.java similarity index 81% rename from modules/ingest-grok/src/test/java/org/elasticsearch/ingest/grok/IngestGrokRestIT.java rename to modules/ingest-common/src/test/java/org/elasticsearch/ingest/IngestCommonRestIT.java index 3f4bdf1f8b9..544e47b8e94 100644 --- a/modules/ingest-grok/src/test/java/org/elasticsearch/ingest/grok/IngestGrokRestIT.java +++ b/modules/ingest-common/src/test/java/org/elasticsearch/ingest/IngestCommonRestIT.java @@ -17,22 +17,19 @@ * under the License. */ -package org.elasticsearch.ingest.grok; +package org.elasticsearch.ingest; import com.carrotsearch.randomizedtesting.annotations.Name; import com.carrotsearch.randomizedtesting.annotations.ParametersFactory; -import org.elasticsearch.ingest.grok.IngestGrokPlugin; -import org.elasticsearch.plugins.Plugin; import org.elasticsearch.test.rest.ESRestTestCase; import org.elasticsearch.test.rest.RestTestCandidate; import org.elasticsearch.test.rest.parser.RestTestParseException; import java.io.IOException; -import java.util.Collection; -public class IngestGrokRestIT extends ESRestTestCase { +public class IngestCommonRestIT extends ESRestTestCase { - public IngestGrokRestIT(@Name("yaml") RestTestCandidate testCandidate) { + public IngestCommonRestIT(@Name("yaml") RestTestCandidate testCandidate) { super(testCandidate); } diff --git a/core/src/test/java/org/elasticsearch/ingest/processor/JoinProcessorFactoryTests.java b/modules/ingest-common/src/test/java/org/elasticsearch/ingest/JoinProcessorFactoryTests.java similarity index 96% rename from core/src/test/java/org/elasticsearch/ingest/processor/JoinProcessorFactoryTests.java rename to modules/ingest-common/src/test/java/org/elasticsearch/ingest/JoinProcessorFactoryTests.java index c374b8a3318..45e643579a4 100644 --- a/core/src/test/java/org/elasticsearch/ingest/processor/JoinProcessorFactoryTests.java +++ b/modules/ingest-common/src/test/java/org/elasticsearch/ingest/JoinProcessorFactoryTests.java @@ -17,11 +17,10 @@ * under the License. */ -package org.elasticsearch.ingest.processor; +package org.elasticsearch.ingest; import org.elasticsearch.ElasticsearchParseException; import org.elasticsearch.ingest.core.AbstractProcessorFactory; -import org.elasticsearch.ingest.core.Processor; import org.elasticsearch.test.ESTestCase; import java.util.HashMap; diff --git a/core/src/test/java/org/elasticsearch/ingest/processor/JoinProcessorTests.java b/modules/ingest-common/src/test/java/org/elasticsearch/ingest/JoinProcessorTests.java similarity index 98% rename from core/src/test/java/org/elasticsearch/ingest/processor/JoinProcessorTests.java rename to modules/ingest-common/src/test/java/org/elasticsearch/ingest/JoinProcessorTests.java index 2aa3ac2e035..39906fb4850 100644 --- a/core/src/test/java/org/elasticsearch/ingest/processor/JoinProcessorTests.java +++ b/modules/ingest-common/src/test/java/org/elasticsearch/ingest/JoinProcessorTests.java @@ -17,10 +17,9 @@ * under the License. */ -package org.elasticsearch.ingest.processor; +package org.elasticsearch.ingest; import org.elasticsearch.ingest.core.IngestDocument; -import org.elasticsearch.ingest.RandomDocumentPicks; import org.elasticsearch.ingest.core.Processor; import org.elasticsearch.test.ESTestCase; diff --git a/core/src/test/java/org/elasticsearch/ingest/processor/LowercaseProcessorFactoryTests.java b/modules/ingest-common/src/test/java/org/elasticsearch/ingest/LowercaseProcessorFactoryTests.java similarity index 95% rename from core/src/test/java/org/elasticsearch/ingest/processor/LowercaseProcessorFactoryTests.java rename to modules/ingest-common/src/test/java/org/elasticsearch/ingest/LowercaseProcessorFactoryTests.java index 09d676b3b30..b7a315b176b 100644 --- a/core/src/test/java/org/elasticsearch/ingest/processor/LowercaseProcessorFactoryTests.java +++ b/modules/ingest-common/src/test/java/org/elasticsearch/ingest/LowercaseProcessorFactoryTests.java @@ -17,11 +17,10 @@ * under the License. */ -package org.elasticsearch.ingest.processor; +package org.elasticsearch.ingest; import org.elasticsearch.ElasticsearchParseException; import org.elasticsearch.ingest.core.AbstractProcessorFactory; -import org.elasticsearch.ingest.core.Processor; import org.elasticsearch.test.ESTestCase; import java.util.HashMap; diff --git a/core/src/test/java/org/elasticsearch/ingest/processor/LowercaseProcessorTests.java b/modules/ingest-common/src/test/java/org/elasticsearch/ingest/LowercaseProcessorTests.java similarity index 96% rename from core/src/test/java/org/elasticsearch/ingest/processor/LowercaseProcessorTests.java rename to modules/ingest-common/src/test/java/org/elasticsearch/ingest/LowercaseProcessorTests.java index 77e22b0ca1e..81e23ef4689 100644 --- a/core/src/test/java/org/elasticsearch/ingest/processor/LowercaseProcessorTests.java +++ b/modules/ingest-common/src/test/java/org/elasticsearch/ingest/LowercaseProcessorTests.java @@ -17,7 +17,7 @@ * under the License. */ -package org.elasticsearch.ingest.processor; +package org.elasticsearch.ingest; import java.util.Locale; diff --git a/core/src/test/java/org/elasticsearch/ingest/processor/RemoveProcessorFactoryTests.java b/modules/ingest-common/src/test/java/org/elasticsearch/ingest/RemoveProcessorFactoryTests.java similarity index 93% rename from core/src/test/java/org/elasticsearch/ingest/processor/RemoveProcessorFactoryTests.java rename to modules/ingest-common/src/test/java/org/elasticsearch/ingest/RemoveProcessorFactoryTests.java index 1b9d88160bd..672058b61e4 100644 --- a/core/src/test/java/org/elasticsearch/ingest/processor/RemoveProcessorFactoryTests.java +++ b/modules/ingest-common/src/test/java/org/elasticsearch/ingest/RemoveProcessorFactoryTests.java @@ -17,12 +17,10 @@ * under the License. */ -package org.elasticsearch.ingest.processor; +package org.elasticsearch.ingest; import org.elasticsearch.ElasticsearchParseException; -import org.elasticsearch.ingest.TestTemplateService; import org.elasticsearch.ingest.core.AbstractProcessorFactory; -import org.elasticsearch.ingest.core.Processor; import org.elasticsearch.test.ESTestCase; import org.junit.Before; diff --git a/core/src/test/java/org/elasticsearch/ingest/processor/RemoveProcessorTests.java b/modules/ingest-common/src/test/java/org/elasticsearch/ingest/RemoveProcessorTests.java similarity index 93% rename from core/src/test/java/org/elasticsearch/ingest/processor/RemoveProcessorTests.java rename to modules/ingest-common/src/test/java/org/elasticsearch/ingest/RemoveProcessorTests.java index d134b0213eb..abc68aa60b6 100644 --- a/core/src/test/java/org/elasticsearch/ingest/processor/RemoveProcessorTests.java +++ b/modules/ingest-common/src/test/java/org/elasticsearch/ingest/RemoveProcessorTests.java @@ -17,11 +17,9 @@ * under the License. */ -package org.elasticsearch.ingest.processor; +package org.elasticsearch.ingest; import org.elasticsearch.ingest.core.IngestDocument; -import org.elasticsearch.ingest.RandomDocumentPicks; -import org.elasticsearch.ingest.TestTemplateService; import org.elasticsearch.ingest.core.Processor; import org.elasticsearch.test.ESTestCase; diff --git a/core/src/test/java/org/elasticsearch/ingest/processor/RenameProcessorFactoryTests.java b/modules/ingest-common/src/test/java/org/elasticsearch/ingest/RenameProcessorFactoryTests.java similarity index 98% rename from core/src/test/java/org/elasticsearch/ingest/processor/RenameProcessorFactoryTests.java rename to modules/ingest-common/src/test/java/org/elasticsearch/ingest/RenameProcessorFactoryTests.java index 768575991d9..005abb8c0d0 100644 --- a/core/src/test/java/org/elasticsearch/ingest/processor/RenameProcessorFactoryTests.java +++ b/modules/ingest-common/src/test/java/org/elasticsearch/ingest/RenameProcessorFactoryTests.java @@ -17,7 +17,7 @@ * under the License. */ -package org.elasticsearch.ingest.processor; +package org.elasticsearch.ingest; import org.elasticsearch.ElasticsearchParseException; import org.elasticsearch.ingest.core.AbstractProcessorFactory; diff --git a/core/src/test/java/org/elasticsearch/ingest/processor/RenameProcessorTests.java b/modules/ingest-common/src/test/java/org/elasticsearch/ingest/RenameProcessorTests.java similarity index 97% rename from core/src/test/java/org/elasticsearch/ingest/processor/RenameProcessorTests.java rename to modules/ingest-common/src/test/java/org/elasticsearch/ingest/RenameProcessorTests.java index 1f9bddaac6b..85faca03e9c 100644 --- a/core/src/test/java/org/elasticsearch/ingest/processor/RenameProcessorTests.java +++ b/modules/ingest-common/src/test/java/org/elasticsearch/ingest/RenameProcessorTests.java @@ -17,10 +17,9 @@ * under the License. */ -package org.elasticsearch.ingest.processor; +package org.elasticsearch.ingest; import org.elasticsearch.ingest.core.IngestDocument; -import org.elasticsearch.ingest.RandomDocumentPicks; import org.elasticsearch.ingest.core.Processor; import org.elasticsearch.test.ESTestCase; @@ -102,7 +101,8 @@ public class RenameProcessorTests extends ESTestCase { public void testRenameNewFieldAlreadyExists() throws Exception { IngestDocument ingestDocument = RandomDocumentPicks.randomIngestDocument(random()); String fieldName = RandomDocumentPicks.randomExistingFieldName(random(), ingestDocument); - Processor processor = new RenameProcessor(randomAsciiOfLength(10), RandomDocumentPicks.randomExistingFieldName(random(), ingestDocument), fieldName); + Processor processor = new RenameProcessor(randomAsciiOfLength(10), RandomDocumentPicks.randomExistingFieldName( + random(), ingestDocument), fieldName); try { processor.execute(ingestDocument); fail("processor execute should have failed"); diff --git a/core/src/test/java/org/elasticsearch/ingest/processor/SetProcessorFactoryTests.java b/modules/ingest-common/src/test/java/org/elasticsearch/ingest/SetProcessorFactoryTests.java similarity index 97% rename from core/src/test/java/org/elasticsearch/ingest/processor/SetProcessorFactoryTests.java rename to modules/ingest-common/src/test/java/org/elasticsearch/ingest/SetProcessorFactoryTests.java index 8810517de8f..acb09b0c96f 100644 --- a/core/src/test/java/org/elasticsearch/ingest/processor/SetProcessorFactoryTests.java +++ b/modules/ingest-common/src/test/java/org/elasticsearch/ingest/SetProcessorFactoryTests.java @@ -17,10 +17,9 @@ * under the License. */ -package org.elasticsearch.ingest.processor; +package org.elasticsearch.ingest; import org.elasticsearch.ElasticsearchParseException; -import org.elasticsearch.ingest.TestTemplateService; import org.elasticsearch.ingest.core.AbstractProcessorFactory; import org.elasticsearch.test.ESTestCase; import org.junit.Before; diff --git a/core/src/test/java/org/elasticsearch/ingest/processor/SetProcessorTests.java b/modules/ingest-common/src/test/java/org/elasticsearch/ingest/SetProcessorTests.java similarity index 94% rename from core/src/test/java/org/elasticsearch/ingest/processor/SetProcessorTests.java rename to modules/ingest-common/src/test/java/org/elasticsearch/ingest/SetProcessorTests.java index 37c20e186b9..1d28124d55a 100644 --- a/core/src/test/java/org/elasticsearch/ingest/processor/SetProcessorTests.java +++ b/modules/ingest-common/src/test/java/org/elasticsearch/ingest/SetProcessorTests.java @@ -17,12 +17,10 @@ * under the License. */ -package org.elasticsearch.ingest.processor; +package org.elasticsearch.ingest; import org.elasticsearch.ingest.core.IngestDocument; -import org.elasticsearch.ingest.RandomDocumentPicks; import org.elasticsearch.ingest.core.TemplateService; -import org.elasticsearch.ingest.TestTemplateService; import org.elasticsearch.ingest.core.ValueSource; import org.elasticsearch.ingest.core.Processor; import org.elasticsearch.test.ESTestCase; @@ -64,7 +62,8 @@ public class SetProcessorTests extends ESTestCase { processor.execute(ingestDocument); fail("processor execute should have failed"); } catch(IllegalArgumentException e) { - assertThat(e.getMessage(), equalTo("cannot set [inner] with parent object of type [java.lang.String] as part of path [field.inner]")); + assertThat(e.getMessage(), equalTo("cannot set [inner] with parent object of type [java.lang.String] as " + + "part of path [field.inner]")); } } @@ -109,6 +108,7 @@ public class SetProcessorTests extends ESTestCase { private static Processor createSetProcessor(String fieldName, Object fieldValue, boolean overrideEnabled) { TemplateService templateService = TestTemplateService.instance(); - return new SetProcessor(randomAsciiOfLength(10), templateService.compile(fieldName), ValueSource.wrap(fieldValue, templateService), overrideEnabled); + return new SetProcessor(randomAsciiOfLength(10), templateService.compile(fieldName), + ValueSource.wrap(fieldValue, templateService), overrideEnabled); } } diff --git a/core/src/test/java/org/elasticsearch/ingest/processor/SortProcessorTests.java b/modules/ingest-common/src/test/java/org/elasticsearch/ingest/SortProcessorTests.java similarity index 98% rename from core/src/test/java/org/elasticsearch/ingest/processor/SortProcessorTests.java rename to modules/ingest-common/src/test/java/org/elasticsearch/ingest/SortProcessorTests.java index 18e35d00a79..40da784bf34 100644 --- a/core/src/test/java/org/elasticsearch/ingest/processor/SortProcessorTests.java +++ b/modules/ingest-common/src/test/java/org/elasticsearch/ingest/SortProcessorTests.java @@ -17,12 +17,11 @@ * under the License. */ -package org.elasticsearch.ingest.processor; +package org.elasticsearch.ingest; import org.elasticsearch.ingest.core.IngestDocument; -import org.elasticsearch.ingest.RandomDocumentPicks; import org.elasticsearch.ingest.core.Processor; -import org.elasticsearch.ingest.processor.SortProcessor.SortOrder; +import org.elasticsearch.ingest.SortProcessor.SortOrder; import org.elasticsearch.test.ESTestCase; import java.util.ArrayList; @@ -30,8 +29,6 @@ import java.util.Arrays; import java.util.Collections; import java.util.HashMap; import java.util.List; -import java.util.Random; -import java.util.stream.Collectors; import static org.hamcrest.Matchers.containsString; import static org.hamcrest.Matchers.equalTo; diff --git a/core/src/test/java/org/elasticsearch/ingest/processor/SplitProcessorFactoryTests.java b/modules/ingest-common/src/test/java/org/elasticsearch/ingest/SplitProcessorFactoryTests.java similarity index 96% rename from core/src/test/java/org/elasticsearch/ingest/processor/SplitProcessorFactoryTests.java rename to modules/ingest-common/src/test/java/org/elasticsearch/ingest/SplitProcessorFactoryTests.java index 70fca6f501b..7d0c3c4d970 100644 --- a/core/src/test/java/org/elasticsearch/ingest/processor/SplitProcessorFactoryTests.java +++ b/modules/ingest-common/src/test/java/org/elasticsearch/ingest/SplitProcessorFactoryTests.java @@ -17,11 +17,10 @@ * under the License. */ -package org.elasticsearch.ingest.processor; +package org.elasticsearch.ingest; import org.elasticsearch.ElasticsearchParseException; import org.elasticsearch.ingest.core.AbstractProcessorFactory; -import org.elasticsearch.ingest.core.Processor; import org.elasticsearch.test.ESTestCase; import java.util.HashMap; diff --git a/core/src/test/java/org/elasticsearch/ingest/processor/SplitProcessorTests.java b/modules/ingest-common/src/test/java/org/elasticsearch/ingest/SplitProcessorTests.java similarity index 94% rename from core/src/test/java/org/elasticsearch/ingest/processor/SplitProcessorTests.java rename to modules/ingest-common/src/test/java/org/elasticsearch/ingest/SplitProcessorTests.java index e1c8a626a4b..95bb4fe4109 100644 --- a/core/src/test/java/org/elasticsearch/ingest/processor/SplitProcessorTests.java +++ b/modules/ingest-common/src/test/java/org/elasticsearch/ingest/SplitProcessorTests.java @@ -17,9 +17,8 @@ * under the License. */ -package org.elasticsearch.ingest.processor; +package org.elasticsearch.ingest; -import org.elasticsearch.ingest.RandomDocumentPicks; import org.elasticsearch.ingest.core.IngestDocument; import org.elasticsearch.ingest.core.Processor; import org.elasticsearch.test.ESTestCase; @@ -75,7 +74,8 @@ public class SplitProcessorTests extends ESTestCase { processor.execute(ingestDocument); fail("split processor should have failed"); } catch (IllegalArgumentException e) { - assertThat(e.getMessage(), equalTo("field [" + fieldName + "] of type [java.lang.Integer] cannot be cast to [java.lang.String]")); + assertThat(e.getMessage(), equalTo("field [" + fieldName + "] of type [java.lang.Integer] cannot be cast " + + "to [java.lang.String]")); } } @@ -92,6 +92,7 @@ public class SplitProcessorTests extends ESTestCase { List flags = (List)ingestDocument.getFieldValue("flags", List.class); assertThat(flags, equalTo(Arrays.asList("new", "hot", "super", "fun", "interesting"))); ingestDocument.appendFieldValue("flags", "additional_flag"); - assertThat(ingestDocument.getFieldValue("flags", List.class), equalTo(Arrays.asList("new", "hot", "super", "fun", "interesting", "additional_flag"))); + assertThat(ingestDocument.getFieldValue("flags", List.class), equalTo(Arrays.asList("new", "hot", "super", + "fun", "interesting", "additional_flag"))); } } diff --git a/core/src/test/java/org/elasticsearch/ingest/processor/TrimProcessorFactoryTests.java b/modules/ingest-common/src/test/java/org/elasticsearch/ingest/TrimProcessorFactoryTests.java similarity index 95% rename from core/src/test/java/org/elasticsearch/ingest/processor/TrimProcessorFactoryTests.java rename to modules/ingest-common/src/test/java/org/elasticsearch/ingest/TrimProcessorFactoryTests.java index 1e74b78f973..0692d53c9e3 100644 --- a/core/src/test/java/org/elasticsearch/ingest/processor/TrimProcessorFactoryTests.java +++ b/modules/ingest-common/src/test/java/org/elasticsearch/ingest/TrimProcessorFactoryTests.java @@ -17,11 +17,10 @@ * under the License. */ -package org.elasticsearch.ingest.processor; +package org.elasticsearch.ingest; import org.elasticsearch.ElasticsearchParseException; import org.elasticsearch.ingest.core.AbstractProcessorFactory; -import org.elasticsearch.ingest.core.Processor; import org.elasticsearch.test.ESTestCase; import java.util.HashMap; diff --git a/core/src/test/java/org/elasticsearch/ingest/processor/TrimProcessorTests.java b/modules/ingest-common/src/test/java/org/elasticsearch/ingest/TrimProcessorTests.java similarity index 97% rename from core/src/test/java/org/elasticsearch/ingest/processor/TrimProcessorTests.java rename to modules/ingest-common/src/test/java/org/elasticsearch/ingest/TrimProcessorTests.java index a0e5fdeab2b..ce51dea3acb 100644 --- a/core/src/test/java/org/elasticsearch/ingest/processor/TrimProcessorTests.java +++ b/modules/ingest-common/src/test/java/org/elasticsearch/ingest/TrimProcessorTests.java @@ -17,7 +17,7 @@ * under the License. */ -package org.elasticsearch.ingest.processor; +package org.elasticsearch.ingest; public class TrimProcessorTests extends AbstractStringProcessorTestCase { diff --git a/core/src/test/java/org/elasticsearch/ingest/processor/UppercaseProcessorFactoryTests.java b/modules/ingest-common/src/test/java/org/elasticsearch/ingest/UppercaseProcessorFactoryTests.java similarity index 95% rename from core/src/test/java/org/elasticsearch/ingest/processor/UppercaseProcessorFactoryTests.java rename to modules/ingest-common/src/test/java/org/elasticsearch/ingest/UppercaseProcessorFactoryTests.java index 40e14b5f14d..0b5eafd2173 100644 --- a/core/src/test/java/org/elasticsearch/ingest/processor/UppercaseProcessorFactoryTests.java +++ b/modules/ingest-common/src/test/java/org/elasticsearch/ingest/UppercaseProcessorFactoryTests.java @@ -17,11 +17,10 @@ * under the License. */ -package org.elasticsearch.ingest.processor; +package org.elasticsearch.ingest; import org.elasticsearch.ElasticsearchParseException; import org.elasticsearch.ingest.core.AbstractProcessorFactory; -import org.elasticsearch.ingest.core.Processor; import org.elasticsearch.test.ESTestCase; import java.util.HashMap; diff --git a/core/src/test/java/org/elasticsearch/ingest/processor/UppercaseProcessorTests.java b/modules/ingest-common/src/test/java/org/elasticsearch/ingest/UppercaseProcessorTests.java similarity index 96% rename from core/src/test/java/org/elasticsearch/ingest/processor/UppercaseProcessorTests.java rename to modules/ingest-common/src/test/java/org/elasticsearch/ingest/UppercaseProcessorTests.java index 4ab61f7b5e3..6881d117946 100644 --- a/core/src/test/java/org/elasticsearch/ingest/processor/UppercaseProcessorTests.java +++ b/modules/ingest-common/src/test/java/org/elasticsearch/ingest/UppercaseProcessorTests.java @@ -17,7 +17,7 @@ * under the License. */ -package org.elasticsearch.ingest.processor; +package org.elasticsearch.ingest; import java.util.Locale; diff --git a/rest-api-spec/src/main/resources/rest-api-spec/test/ingest/90_date_index_name_processor.yaml b/modules/ingest-common/src/test/resources/rest-api-spec/test/ingest/100_date_index_name_processor.yaml similarity index 100% rename from rest-api-spec/src/main/resources/rest-api-spec/test/ingest/90_date_index_name_processor.yaml rename to modules/ingest-common/src/test/resources/rest-api-spec/test/ingest/100_date_index_name_processor.yaml diff --git a/modules/ingest-grok/src/test/resources/rest-api-spec/test/ingest_grok/10_basic.yaml b/modules/ingest-common/src/test/resources/rest-api-spec/test/ingest/10_basic.yaml similarity index 93% rename from modules/ingest-grok/src/test/resources/rest-api-spec/test/ingest_grok/10_basic.yaml rename to modules/ingest-common/src/test/resources/rest-api-spec/test/ingest/10_basic.yaml index c41793fcdb8..1f6962c6e2f 100644 --- a/modules/ingest-grok/src/test/resources/rest-api-spec/test/ingest_grok/10_basic.yaml +++ b/modules/ingest-common/src/test/resources/rest-api-spec/test/ingest/10_basic.yaml @@ -1,4 +1,4 @@ -"Ingest grok installed": +"Ingest common installed": - do: cluster.state: {} @@ -8,7 +8,7 @@ - do: nodes.info: {} - - match: { nodes.$master.modules.0.name: ingest-grok } + - match: { nodes.$master.modules.0.name: ingest-common } - match: { nodes.$master.ingest.processors.0.type: append } - match: { nodes.$master.ingest.processors.1.type: convert } - match: { nodes.$master.ingest.processors.2.type: date } diff --git a/rest-api-spec/src/main/resources/rest-api-spec/test/ingest/100_sort.yaml b/modules/ingest-common/src/test/resources/rest-api-spec/test/ingest/110_sort.yaml similarity index 100% rename from rest-api-spec/src/main/resources/rest-api-spec/test/ingest/100_sort.yaml rename to modules/ingest-common/src/test/resources/rest-api-spec/test/ingest/110_sort.yaml diff --git a/modules/ingest-grok/src/test/resources/rest-api-spec/test/ingest_grok/20_grok.yaml b/modules/ingest-common/src/test/resources/rest-api-spec/test/ingest/120_grok.yaml similarity index 100% rename from modules/ingest-grok/src/test/resources/rest-api-spec/test/ingest_grok/20_grok.yaml rename to modules/ingest-common/src/test/resources/rest-api-spec/test/ingest/120_grok.yaml diff --git a/rest-api-spec/src/main/resources/rest-api-spec/test/ingest/10_crud.yaml b/modules/ingest-common/src/test/resources/rest-api-spec/test/ingest/20_crud.yaml similarity index 100% rename from rest-api-spec/src/main/resources/rest-api-spec/test/ingest/10_crud.yaml rename to modules/ingest-common/src/test/resources/rest-api-spec/test/ingest/20_crud.yaml diff --git a/rest-api-spec/src/main/resources/rest-api-spec/test/ingest/20_date_processor.yaml b/modules/ingest-common/src/test/resources/rest-api-spec/test/ingest/30_date_processor.yaml similarity index 100% rename from rest-api-spec/src/main/resources/rest-api-spec/test/ingest/20_date_processor.yaml rename to modules/ingest-common/src/test/resources/rest-api-spec/test/ingest/30_date_processor.yaml diff --git a/rest-api-spec/src/main/resources/rest-api-spec/test/ingest/30_mutate.yaml b/modules/ingest-common/src/test/resources/rest-api-spec/test/ingest/40_mutate.yaml similarity index 100% rename from rest-api-spec/src/main/resources/rest-api-spec/test/ingest/30_mutate.yaml rename to modules/ingest-common/src/test/resources/rest-api-spec/test/ingest/40_mutate.yaml diff --git a/rest-api-spec/src/main/resources/rest-api-spec/test/ingest/50_on_failure.yaml b/modules/ingest-common/src/test/resources/rest-api-spec/test/ingest/50_on_failure.yaml similarity index 100% rename from rest-api-spec/src/main/resources/rest-api-spec/test/ingest/50_on_failure.yaml rename to modules/ingest-common/src/test/resources/rest-api-spec/test/ingest/50_on_failure.yaml diff --git a/rest-api-spec/src/main/resources/rest-api-spec/test/ingest/60_fail.yaml b/modules/ingest-common/src/test/resources/rest-api-spec/test/ingest/60_fail.yaml similarity index 100% rename from rest-api-spec/src/main/resources/rest-api-spec/test/ingest/60_fail.yaml rename to modules/ingest-common/src/test/resources/rest-api-spec/test/ingest/60_fail.yaml diff --git a/rest-api-spec/src/main/resources/rest-api-spec/test/ingest/70_bulk.yaml b/modules/ingest-common/src/test/resources/rest-api-spec/test/ingest/70_bulk.yaml similarity index 100% rename from rest-api-spec/src/main/resources/rest-api-spec/test/ingest/70_bulk.yaml rename to modules/ingest-common/src/test/resources/rest-api-spec/test/ingest/70_bulk.yaml diff --git a/rest-api-spec/src/main/resources/rest-api-spec/test/ingest/80_foreach.yaml b/modules/ingest-common/src/test/resources/rest-api-spec/test/ingest/80_foreach.yaml similarity index 100% rename from rest-api-spec/src/main/resources/rest-api-spec/test/ingest/80_foreach.yaml rename to modules/ingest-common/src/test/resources/rest-api-spec/test/ingest/80_foreach.yaml diff --git a/rest-api-spec/src/main/resources/rest-api-spec/test/ingest/40_simulate.yaml b/modules/ingest-common/src/test/resources/rest-api-spec/test/ingest/90_simulate.yaml similarity index 100% rename from rest-api-spec/src/main/resources/rest-api-spec/test/ingest/40_simulate.yaml rename to modules/ingest-common/src/test/resources/rest-api-spec/test/ingest/90_simulate.yaml diff --git a/modules/ingest-grok/src/main/java/org/elasticsearch/ingest/grok/IngestGrokPlugin.java b/modules/ingest-grok/src/main/java/org/elasticsearch/ingest/grok/IngestGrokPlugin.java deleted file mode 100644 index 998c536b474..00000000000 --- a/modules/ingest-grok/src/main/java/org/elasticsearch/ingest/grok/IngestGrokPlugin.java +++ /dev/null @@ -1,87 +0,0 @@ -/* - * Licensed to Elasticsearch under one or more contributor - * license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright - * ownership. Elasticsearch licenses this file to you under - * the Apache License, Version 2.0 (the "License"); you may - * not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ - -package org.elasticsearch.ingest.grok; - -import org.elasticsearch.node.NodeModule; -import org.elasticsearch.plugins.Plugin; - -import java.io.BufferedReader; -import java.io.IOException; -import java.io.InputStream; -import java.io.InputStreamReader; -import java.nio.charset.StandardCharsets; -import java.util.Collections; -import java.util.HashMap; -import java.util.Map; - -public class IngestGrokPlugin extends Plugin { - - private static final String[] PATTERN_NAMES = new String[] { - "aws", "bacula", "bro", "exim", "firewalls", "grok-patterns", "haproxy", - "java", "junos", "linux-syslog", "mcollective-patterns", "mongodb", "nagios", - "postgresql", "rails", "redis", "ruby" - }; - - private final Map builtinPatterns; - - public IngestGrokPlugin() throws IOException { - this.builtinPatterns = loadBuiltinPatterns(); - } - - @Override - public String name() { - return "ingest-grok"; - } - - @Override - public String description() { - return "Ingest processor that uses grok patterns to split text"; - } - - public void onModule(NodeModule nodeModule) { - nodeModule.registerProcessor(GrokProcessor.TYPE, (templateService, registry) -> new GrokProcessor.Factory(builtinPatterns)); - } - - public static Map loadBuiltinPatterns() throws IOException { - Map builtinPatterns = new HashMap<>(); - for (String pattern : PATTERN_NAMES) { - try(InputStream is = IngestGrokPlugin.class.getResourceAsStream("/patterns/" + pattern)) { - loadPatterns(builtinPatterns, is); - } - } - return Collections.unmodifiableMap(builtinPatterns); - } - - private static void loadPatterns(Map patternBank, InputStream inputStream) throws IOException { - String line; - BufferedReader br = new BufferedReader(new InputStreamReader(inputStream, StandardCharsets.UTF_8)); - while ((line = br.readLine()) != null) { - String trimmedLine = line.replaceAll("^\\s+", ""); - if (trimmedLine.startsWith("#") || trimmedLine.length() == 0) { - continue; - } - - String[] parts = trimmedLine.split("\\s+", 2); - if (parts.length == 2) { - patternBank.put(parts[0], parts[1]); - } - } - } -} diff --git a/modules/lang-expression/src/test/java/org/elasticsearch/script/expression/MoreExpressionTests.java b/modules/lang-expression/src/test/java/org/elasticsearch/script/expression/MoreExpressionTests.java index da2f117c78f..6796c1e048b 100644 --- a/modules/lang-expression/src/test/java/org/elasticsearch/script/expression/MoreExpressionTests.java +++ b/modules/lang-expression/src/test/java/org/elasticsearch/script/expression/MoreExpressionTests.java @@ -59,6 +59,7 @@ import org.elasticsearch.test.ESIntegTestCase; import org.elasticsearch.test.VersionUtils; import org.elasticsearch.test.hamcrest.ElasticsearchAssertions; +import static org.elasticsearch.action.support.WriteRequest.RefreshPolicy.IMMEDIATE; import static org.elasticsearch.common.xcontent.XContentFactory.jsonBuilder; import static org.elasticsearch.search.aggregations.AggregationBuilders.histogram; import static org.elasticsearch.search.aggregations.AggregationBuilders.sum; @@ -97,7 +98,7 @@ public class MoreExpressionTests extends ESIntegTestCase { public void testBasic() throws Exception { createIndex("test"); ensureGreen("test"); - client().prepareIndex("test", "doc", "1").setSource("foo", 4).setRefresh(true).get(); + client().prepareIndex("test", "doc", "1").setSource("foo", 4).setRefreshPolicy(IMMEDIATE).get(); SearchResponse rsp = buildRequest("doc['foo'] + 1").get(); assertEquals(1, rsp.getHits().getTotalHits()); assertEquals(5.0, rsp.getHits().getAt(0).field("foo").getValue(), 0.0D); @@ -106,7 +107,7 @@ public class MoreExpressionTests extends ESIntegTestCase { public void testFunction() throws Exception { createIndex("test"); ensureGreen("test"); - client().prepareIndex("test", "doc", "1").setSource("foo", 4).setRefresh(true).get(); + client().prepareIndex("test", "doc", "1").setSource("foo", 4).setRefreshPolicy(IMMEDIATE).get(); SearchResponse rsp = buildRequest("doc['foo'] + abs(1)").get(); assertSearchResponse(rsp); assertEquals(1, rsp.getHits().getTotalHits()); @@ -116,7 +117,7 @@ public class MoreExpressionTests extends ESIntegTestCase { public void testBasicUsingDotValue() throws Exception { createIndex("test"); ensureGreen("test"); - client().prepareIndex("test", "doc", "1").setSource("foo", 4).setRefresh(true).get(); + client().prepareIndex("test", "doc", "1").setSource("foo", 4).setRefreshPolicy(IMMEDIATE).get(); SearchResponse rsp = buildRequest("doc['foo'].value + 1").get(); assertEquals(1, rsp.getHits().getTotalHits()); assertEquals(5.0, rsp.getHits().getAt(0).field("foo").getValue(), 0.0D); @@ -314,7 +315,7 @@ public class MoreExpressionTests extends ESIntegTestCase { public void testMissingField() throws Exception { createIndex("test"); ensureGreen("test"); - client().prepareIndex("test", "doc", "1").setSource("x", 4).setRefresh(true).get(); + client().prepareIndex("test", "doc", "1").setSource("x", 4).setRefreshPolicy(IMMEDIATE).get(); try { buildRequest("doc['bogus']").get(); fail("Expected missing field to cause failure"); @@ -344,7 +345,7 @@ public class MoreExpressionTests extends ESIntegTestCase { } public void testCompileFailure() { - client().prepareIndex("test", "doc", "1").setSource("x", 1).setRefresh(true).get(); + client().prepareIndex("test", "doc", "1").setSource("x", 1).setRefreshPolicy(IMMEDIATE).get(); try { buildRequest("garbage%@#%@").get(); fail("Expected expression compilation failure"); @@ -357,7 +358,7 @@ public class MoreExpressionTests extends ESIntegTestCase { } public void testNonNumericParam() { - client().prepareIndex("test", "doc", "1").setSource("x", 1).setRefresh(true).get(); + client().prepareIndex("test", "doc", "1").setSource("x", 1).setRefreshPolicy(IMMEDIATE).get(); try { buildRequest("a", "a", "astring").get(); fail("Expected string parameter to cause failure"); @@ -370,7 +371,7 @@ public class MoreExpressionTests extends ESIntegTestCase { } public void testNonNumericField() { - client().prepareIndex("test", "doc", "1").setSource("text", "this is not a number").setRefresh(true).get(); + client().prepareIndex("test", "doc", "1").setSource("text", "this is not a number").setRefreshPolicy(IMMEDIATE).get(); try { buildRequest("doc['text.keyword']").get(); fail("Expected text field to cause execution failure"); @@ -383,7 +384,7 @@ public class MoreExpressionTests extends ESIntegTestCase { } public void testInvalidGlobalVariable() { - client().prepareIndex("test", "doc", "1").setSource("foo", 5).setRefresh(true).get(); + client().prepareIndex("test", "doc", "1").setSource("foo", 5).setRefreshPolicy(IMMEDIATE).get(); try { buildRequest("bogus").get(); fail("Expected bogus variable to cause execution failure"); @@ -396,7 +397,7 @@ public class MoreExpressionTests extends ESIntegTestCase { } public void testDocWithoutField() { - client().prepareIndex("test", "doc", "1").setSource("foo", 5).setRefresh(true).get(); + client().prepareIndex("test", "doc", "1").setSource("foo", 5).setRefreshPolicy(IMMEDIATE).get(); try { buildRequest("doc").get(); fail("Expected doc variable without field to cause execution failure"); @@ -409,7 +410,7 @@ public class MoreExpressionTests extends ESIntegTestCase { } public void testInvalidFieldMember() { - client().prepareIndex("test", "doc", "1").setSource("foo", 5).setRefresh(true).get(); + client().prepareIndex("test", "doc", "1").setSource("foo", 5).setRefreshPolicy(IMMEDIATE).get(); try { buildRequest("doc['foo'].bogus").get(); fail("Expected bogus field member to cause execution failure"); diff --git a/modules/lang-groovy/src/test/java/org/elasticsearch/messy/tests/BulkTests.java b/modules/lang-groovy/src/test/java/org/elasticsearch/messy/tests/BulkTests.java index dd9daa75e76..13868566eac 100644 --- a/modules/lang-groovy/src/test/java/org/elasticsearch/messy/tests/BulkTests.java +++ b/modules/lang-groovy/src/test/java/org/elasticsearch/messy/tests/BulkTests.java @@ -29,6 +29,7 @@ import org.elasticsearch.action.get.GetResponse; import org.elasticsearch.action.index.IndexRequest; import org.elasticsearch.action.index.IndexResponse; import org.elasticsearch.action.search.SearchResponse; +import org.elasticsearch.action.support.WriteRequest.RefreshPolicy; import org.elasticsearch.action.update.UpdateRequest; import org.elasticsearch.action.update.UpdateRequestBuilder; import org.elasticsearch.action.update.UpdateResponse; @@ -584,7 +585,7 @@ public class BulkTests extends ESIntegTestCase { .add(new IndexRequest("test", "type", "4").source("{ \"title\" : \"Great Title of doc 4\" }")) .add(new IndexRequest("test", "type", "5").source("{ \"title\" : \"Great Title of doc 5\" }")) .add(new IndexRequest("test", "type", "6").source("{ \"title\" : \"Great Title of doc 6\" }")) - .setRefresh(true) + .setRefreshPolicy(RefreshPolicy.IMMEDIATE) .get(); assertNoFailures(indexBulkItemResponse); @@ -622,7 +623,7 @@ public class BulkTests extends ESIntegTestCase { .add(new IndexRequest("bulkindex2", "index2_type").source("text", "hallo2")) .add(new UpdateRequest("bulkindex2", "index2_type", "2").doc("foo", "bar")) .add(new DeleteRequest("bulkindex2", "index2_type", "3")) - .refresh(true); + .setRefreshPolicy(RefreshPolicy.IMMEDIATE); client().bulk(bulkRequest).get(); SearchResponse searchResponse = client().prepareSearch("bulkindex*").get(); @@ -643,10 +644,10 @@ public class BulkTests extends ESIntegTestCase { client().prepareIndex("bulkindex1", "index1_type", "1").setSource("text", "test").get(); assertAcked(client().admin().indices().prepareClose("bulkindex1")); - BulkRequest bulkRequest = new BulkRequest(); + BulkRequest bulkRequest = new BulkRequest().setRefreshPolicy(RefreshPolicy.IMMEDIATE); bulkRequest.add(new IndexRequest("bulkindex1", "index1_type", "1").source("text", "hallo1")) .add(new UpdateRequest("bulkindex1", "index1_type", "1").doc("foo", "bar")) - .add(new DeleteRequest("bulkindex1", "index1_type", "1")).refresh(true); + .add(new DeleteRequest("bulkindex1", "index1_type", "1")); BulkResponse bulkResponse = client().bulk(bulkRequest).get(); assertThat(bulkResponse.hasFailures(), is(true)); diff --git a/modules/lang-groovy/src/test/java/org/elasticsearch/messy/tests/SearchFieldsTests.java b/modules/lang-groovy/src/test/java/org/elasticsearch/messy/tests/SearchFieldsTests.java index 4e3accfdc06..bc604987a70 100644 --- a/modules/lang-groovy/src/test/java/org/elasticsearch/messy/tests/SearchFieldsTests.java +++ b/modules/lang-groovy/src/test/java/org/elasticsearch/messy/tests/SearchFieldsTests.java @@ -57,6 +57,7 @@ import java.util.Set; import java.util.concurrent.ExecutionException; import static java.util.Collections.singleton; +import static org.elasticsearch.action.support.WriteRequest.RefreshPolicy.IMMEDIATE; import static org.elasticsearch.client.Requests.refreshRequest; import static org.elasticsearch.common.util.set.Sets.newHashSet; import static org.elasticsearch.common.xcontent.XContentFactory.jsonBuilder; @@ -456,7 +457,7 @@ public class SearchFieldsTests extends ESIntegTestCase { client().prepareIndex("my-index", "my-type1", "1") .setRouting("1") .setSource(jsonBuilder().startObject().field("field1", "value").endObject()) - .setRefresh(true) + .setRefreshPolicy(IMMEDIATE) .get(); SearchResponse searchResponse = client().prepareSearch("my-index") @@ -473,7 +474,7 @@ public class SearchFieldsTests extends ESIntegTestCase { public void testSearchFieldsNonLeafField() throws Exception { client().prepareIndex("my-index", "my-type1", "1") .setSource(jsonBuilder().startObject().startObject("field1").field("field2", "value1").endObject().endObject()) - .setRefresh(true) + .setRefreshPolicy(IMMEDIATE) .get(); assertFailures(client().prepareSearch("my-index").setTypes("my-type1").addField("field1"), @@ -536,7 +537,7 @@ public class SearchFieldsTests extends ESIntegTestCase { .endObject().bytes(); client().prepareIndex("my-index", "my-type1", "1").setSource(source).get(); - client().prepareIndex("my-index", "my-type2", "1").setRefresh(true).setSource(source).get(); + client().prepareIndex("my-index", "my-type2", "1").setRefreshPolicy(IMMEDIATE).setSource(source).get(); String field = "field1.field2.field3.field4"; diff --git a/modules/lang-groovy/src/test/java/org/elasticsearch/script/groovy/GroovyScriptTests.java b/modules/lang-groovy/src/test/java/org/elasticsearch/script/groovy/GroovyScriptTests.java index 4e13219bae5..4a7b4350d23 100644 --- a/modules/lang-groovy/src/test/java/org/elasticsearch/script/groovy/GroovyScriptTests.java +++ b/modules/lang-groovy/src/test/java/org/elasticsearch/script/groovy/GroovyScriptTests.java @@ -37,6 +37,7 @@ import java.util.Collection; import java.util.Collections; import java.util.List; +import static org.elasticsearch.action.support.WriteRequest.RefreshPolicy.IMMEDIATE; import static org.elasticsearch.index.query.QueryBuilders.constantScoreQuery; import static org.elasticsearch.index.query.QueryBuilders.functionScoreQuery; import static org.elasticsearch.index.query.QueryBuilders.matchQuery; @@ -58,7 +59,7 @@ public class GroovyScriptTests extends ESIntegTestCase { } public void testGroovyBigDecimalTransformation() { - client().prepareIndex("test", "doc", "1").setSource("foo", 5).setRefresh(true).get(); + client().prepareIndex("test", "doc", "1").setSource("foo", 5).setRefreshPolicy(IMMEDIATE).get(); // Test that something that would usually be a BigDecimal is transformed into a Double assertScript("def n = 1.23; assert n instanceof Double; return n;"); diff --git a/modules/lang-mustache/src/test/java/org/elasticsearch/messy/tests/TemplateQueryParserTests.java b/modules/lang-mustache/src/test/java/org/elasticsearch/messy/tests/TemplateQueryParserTests.java index 9aa5e1892ef..ddb330b01a7 100644 --- a/modules/lang-mustache/src/test/java/org/elasticsearch/messy/tests/TemplateQueryParserTests.java +++ b/modules/lang-mustache/src/test/java/org/elasticsearch/messy/tests/TemplateQueryParserTests.java @@ -108,10 +108,11 @@ public class TemplateQueryParserTests extends ESTestCase { // TODO: make this use a mock engine instead of mustache and it will no longer be messy! scriptModule.addScriptEngine(new ScriptEngineRegistry.ScriptEngineRegistration(MustacheScriptEngineService.class, MustacheScriptEngineService.NAME, true)); settingsModule.registerSetting(InternalSettingsPlugin.VERSION_CREATED); + final ThreadPool threadPool = new ThreadPool(settings); injector = new ModulesBuilder().add( new EnvironmentModule(new Environment(settings)), settingsModule, - new ThreadPoolModule(new ThreadPool(settings)), + new ThreadPoolModule(threadPool), new SearchModule(settings, new NamedWriteableRegistry()) { @Override protected void configureSearch() { diff --git a/modules/lang-painless/src/main/antlr/PainlessLexer.g4 b/modules/lang-painless/src/main/antlr/PainlessLexer.g4 index 6e34d35111d..4ca5778ad3c 100644 --- a/modules/lang-painless/src/main/antlr/PainlessLexer.g4 +++ b/modules/lang-painless/src/main/antlr/PainlessLexer.g4 @@ -32,6 +32,10 @@ LBRACE: '['; RBRACE: ']'; LP: '('; RP: ')'; +// We switch modes after a dot to ensure there are not conflicts +// between shortcuts and decimal values. Without the mode switch +// shortcuts such as id.0.0 will fail because 0.0 will be interpreted +// as a decimal value instead of two individual list-style shortcuts. DOT: '.' -> mode(AFTER_DOT); COMMA: ','; SEMICOLON: ';'; diff --git a/modules/lang-painless/src/main/antlr/PainlessParser.g4 b/modules/lang-painless/src/main/antlr/PainlessParser.g4 index 5125e7b29a6..141ff25e318 100644 --- a/modules/lang-painless/src/main/antlr/PainlessParser.g4 +++ b/modules/lang-painless/src/main/antlr/PainlessParser.g4 @@ -33,6 +33,7 @@ statement | WHILE LP expression RP ( trailer | empty ) # while | DO block WHILE LP expression RP delimiter # do | FOR LP initializer? SEMICOLON expression? SEMICOLON afterthought? RP ( trailer | empty ) # for + | FOR LP decltype ID COLON expression RP trailer # each | declaration delimiter # decl | CONTINUE delimiter # continue | BREAK delimiter # break @@ -73,7 +74,7 @@ decltype ; funcref - : TYPE REF ID + : TYPE REF ( ID | NEW ) ; declvar diff --git a/modules/lang-painless/src/main/java/org/elasticsearch/painless/AnalyzerCaster.java b/modules/lang-painless/src/main/java/org/elasticsearch/painless/AnalyzerCaster.java index 6e542d6aca9..e724fa9e810 100644 --- a/modules/lang-painless/src/main/java/org/elasticsearch/painless/AnalyzerCaster.java +++ b/modules/lang-painless/src/main/java/org/elasticsearch/painless/AnalyzerCaster.java @@ -22,7 +22,6 @@ package org.elasticsearch.painless; import org.elasticsearch.painless.Definition.Cast; import org.elasticsearch.painless.Definition.Sort; import org.elasticsearch.painless.Definition.Type; -import org.elasticsearch.painless.node.ANode; /** * Used during the analysis phase to collect legal type casts and promotions diff --git a/modules/lang-painless/src/main/java/org/elasticsearch/painless/Compiler.java b/modules/lang-painless/src/main/java/org/elasticsearch/painless/Compiler.java index a780088a222..d0a17f64a48 100644 --- a/modules/lang-painless/src/main/java/org/elasticsearch/painless/Compiler.java +++ b/modules/lang-painless/src/main/java/org/elasticsearch/painless/Compiler.java @@ -104,17 +104,16 @@ final class Compiler { SSource root = Walker.buildPainlessTree(name, source, reserved, settings); Variables variables = Analyzer.analyze(reserved, root); BitSet expressions = new BitSet(source.length()); - byte[] bytes = Writer.write(settings, name, source, variables, root, expressions); + try { Class clazz = loader.define(CLASS_NAME, bytes); - java.lang.reflect.Constructor constructor = + java.lang.reflect.Constructor constructor = clazz.getConstructor(String.class, String.class, BitSet.class); return constructor.newInstance(name, source, expressions); } catch (Exception exception) { // Catch everything to let the user know this is something caused internally. - throw new IllegalStateException( - "An internal error occurred attempting to define the script [" + name + "].", exception); + throw new IllegalStateException("An internal error occurred attempting to define the script [" + name + "].", exception); } } diff --git a/modules/lang-painless/src/main/java/org/elasticsearch/painless/Def.java b/modules/lang-painless/src/main/java/org/elasticsearch/painless/Def.java index bd8e09f504b..1edaddea2c1 100644 --- a/modules/lang-painless/src/main/java/org/elasticsearch/painless/Def.java +++ b/modules/lang-painless/src/main/java/org/elasticsearch/painless/Def.java @@ -22,11 +22,15 @@ package org.elasticsearch.painless; import org.elasticsearch.painless.Definition.Method; import org.elasticsearch.painless.Definition.RuntimeClass; +import java.lang.invoke.CallSite; +import java.lang.invoke.LambdaConversionException; +import java.lang.invoke.LambdaMetafactory; import java.lang.invoke.MethodHandle; import java.lang.invoke.MethodHandles; import java.lang.invoke.MethodHandles.Lookup; import java.lang.invoke.MethodType; import java.util.Collections; +import java.util.Iterator; import java.util.List; import java.util.Map; import java.util.function.Function; @@ -53,7 +57,7 @@ public final class Def { /** Helper class for isolating MethodHandles and methods to get the length of arrays * (to emulate a "arraystore" bytecode using MethodHandles). - * This should really be a method in {@link MethodHandles} class! + * See: https://bugs.openjdk.java.net/browse/JDK-8156915 */ @SuppressWarnings("unused") // getArrayLength() methods are are actually used, javac just does not know :) private static final class ArrayLengthHelper { @@ -103,6 +107,10 @@ public final class Def { private static final MethodHandle LIST_GET; /** pointer to List.set(int,Object) */ private static final MethodHandle LIST_SET; + /** pointer to Iterable.iterator() */ + private static final MethodHandle ITERATOR; + /** factory for arraylength MethodHandle (intrinsic) from Java 9 */ + private static final MethodHandle JAVA9_ARRAY_LENGTH_MH_FACTORY; static { final Lookup lookup = MethodHandles.publicLookup(); @@ -112,18 +120,90 @@ public final class Def { MAP_PUT = lookup.findVirtual(Map.class , "put", MethodType.methodType(Object.class, Object.class, Object.class)); LIST_GET = lookup.findVirtual(List.class, "get", MethodType.methodType(Object.class, int.class)); LIST_SET = lookup.findVirtual(List.class, "set", MethodType.methodType(Object.class, int.class, Object.class)); + ITERATOR = lookup.findVirtual(Iterable.class, "iterator", MethodType.methodType(Iterator.class)); } catch (final ReflectiveOperationException roe) { throw new AssertionError(roe); } + + // lookup up the factory for arraylength MethodHandle (intrinsic) from Java 9: + // https://bugs.openjdk.java.net/browse/JDK-8156915 + MethodHandle arrayLengthMHFactory; + try { + arrayLengthMHFactory = lookup.findStatic(MethodHandles.class, "arrayLength", + MethodType.methodType(MethodHandle.class, Class.class)); + } catch (final ReflectiveOperationException roe) { + arrayLengthMHFactory = null; + } + JAVA9_ARRAY_LENGTH_MH_FACTORY = arrayLengthMHFactory; } + /** Hack to rethrow unknown Exceptions from {@link MethodHandle#invokeExact}: */ + @SuppressWarnings("unchecked") + private static void rethrow(Throwable t) throws T { + throw (T) t; + } + /** Returns an array length getter MethodHandle for the given array type */ static MethodHandle arrayLengthGetter(Class arrayType) { - return ArrayLengthHelper.arrayLengthGetter(arrayType); + if (JAVA9_ARRAY_LENGTH_MH_FACTORY != null) { + try { + return (MethodHandle) JAVA9_ARRAY_LENGTH_MH_FACTORY.invokeExact(arrayType); + } catch (Throwable t) { + rethrow(t); + throw new AssertionError(t); + } + } else { + return ArrayLengthHelper.arrayLengthGetter(arrayType); + } } /** - * Looks up handle for a dynamic method call. + * Looks up method entry for a dynamic method call. + *

    + * A dynamic method call for variable {@code x} of type {@code def} looks like: + * {@code x.method(args...)} + *

    + * This method traverses {@code recieverClass}'s class hierarchy (including interfaces) + * until it finds a matching whitelisted method. If one is not found, it throws an exception. + * Otherwise it returns the matching method. + *

    + * @param receiverClass Class of the object to invoke the method on. + * @param name Name of the method. + * @param arity arity of method + * @return matching method to invoke. never returns null. + * @throws IllegalArgumentException if no matching whitelisted method was found. + */ + static Method lookupMethodInternal(Class receiverClass, String name, int arity) { + Definition.MethodKey key = new Definition.MethodKey(name, arity); + // check whitelist for matching method + for (Class clazz = receiverClass; clazz != null; clazz = clazz.getSuperclass()) { + RuntimeClass struct = Definition.getRuntimeClass(clazz); + + if (struct != null) { + Method method = struct.methods.get(key); + if (method != null) { + return method; + } + } + + for (Class iface : clazz.getInterfaces()) { + struct = Definition.getRuntimeClass(iface); + + if (struct != null) { + Method method = struct.methods.get(key); + if (method != null) { + return method; + } + } + } + } + + throw new IllegalArgumentException("Unable to find dynamic method [" + name + "] with [" + arity + "] arguments " + + "for class [" + receiverClass.getCanonicalName() + "]."); + } + + /** + * Looks up handle for a dynamic method call, with lambda replacement *

    * A dynamic method call for variable {@code x} of type {@code def} looks like: * {@code x.method(args...)} @@ -134,41 +214,62 @@ public final class Def { *

    * @param receiverClass Class of the object to invoke the method on. * @param name Name of the method. - * @param type Callsite signature. Need not match exactly, except the number of parameters. + * @param args args passed to callsite + * @param recipe bitset marking functional parameters * @return pointer to matching method to invoke. never returns null. + * @throws LambdaConversionException if a method reference cannot be converted to an functional interface * @throws IllegalArgumentException if no matching whitelisted method was found. */ - static MethodHandle lookupMethod(Class receiverClass, String name, MethodType type) { - // we don't consider receiver an argument/counting towards arity - type = type.dropParameterTypes(0, 1); - Definition.MethodKey key = new Definition.MethodKey(name, type.parameterCount()); - // check whitelist for matching method - for (Class clazz = receiverClass; clazz != null; clazz = clazz.getSuperclass()) { - RuntimeClass struct = Definition.getRuntimeClass(clazz); + static MethodHandle lookupMethod(Lookup lookup, Class receiverClass, String name, + Object args[], long recipe) throws LambdaConversionException { + Method method = lookupMethodInternal(receiverClass, name, args.length - 1); + MethodHandle handle = method.handle; - if (struct != null) { - Method method = struct.methods.get(key); - if (method != null) { - return method.handle; - } - } - - for (final Class iface : clazz.getInterfaces()) { - struct = Definition.getRuntimeClass(iface); - - if (struct != null) { - Method method = struct.methods.get(key); - if (method != null) { - return method.handle; - } + if (recipe != 0) { + MethodHandle filters[] = new MethodHandle[args.length]; + for (int i = 0; i < args.length; i++) { + // its a functional reference, replace the argument with an impl + if ((recipe & (1L << (i - 1))) != 0) { + filters[i] = lookupReference(lookup, method.arguments.get(i - 1), (String) args[i]); } } + handle = MethodHandles.filterArguments(handle, 0, filters); } - - // no matching methods in whitelist found - throw new IllegalArgumentException("Unable to find dynamic method [" + name + "] with signature [" + type + "] " + - "for class [" + receiverClass.getCanonicalName() + "]."); - } + + return handle; + } + + /** Returns a method handle to an implementation of clazz, given method reference signature + * @throws LambdaConversionException if a method reference cannot be converted to an functional interface + */ + private static MethodHandle lookupReference(Lookup lookup, Definition.Type clazz, String signature) throws LambdaConversionException { + int separator = signature.indexOf('.'); + FunctionRef ref = new FunctionRef(clazz, signature.substring(0, separator), signature.substring(separator+1)); + final CallSite callSite; + if (ref.needsBridges()) { + callSite = LambdaMetafactory.altMetafactory(lookup, + ref.invokedName, + ref.invokedType, + ref.samMethodType, + ref.implMethod, + ref.samMethodType, + LambdaMetafactory.FLAG_BRIDGES, + 1, + ref.interfaceMethodType); + } else { + callSite = LambdaMetafactory.altMetafactory(lookup, + ref.invokedName, + ref.invokedType, + ref.samMethodType, + ref.implMethod, + ref.samMethodType, + 0); + } + // we could actually invoke and cache here (in non-capturing cases), but this is not a speedup. + MethodHandle factory = callSite.dynamicInvoker().asType(MethodType.methodType(clazz.clazz)); + return MethodHandles.dropArguments(factory, 0, Object.class); + } + /** * Looks up handle for a dynamic field getter (field load) @@ -346,6 +447,118 @@ public final class Def { throw new IllegalArgumentException("Attempting to address a non-array type " + "[" + receiverClass.getCanonicalName() + "] as an array."); } + + /** Helper class for isolating MethodHandles and methods to get iterators over arrays + * (to emulate "enhanced for loop" using MethodHandles). These cause boxing, and are not as efficient + * as they could be, but works. + */ + @SuppressWarnings("unused") // iterator() methods are are actually used, javac just does not know :) + private static final class ArrayIteratorHelper { + private static final Lookup PRIV_LOOKUP = MethodHandles.lookup(); + + private static final Map,MethodHandle> ARRAY_TYPE_MH_MAPPING = Collections.unmodifiableMap( + Stream.of(boolean[].class, byte[].class, short[].class, int[].class, long[].class, + char[].class, float[].class, double[].class, Object[].class) + .collect(Collectors.toMap(Function.identity(), type -> { + try { + return PRIV_LOOKUP.findStatic(PRIV_LOOKUP.lookupClass(), "iterator", MethodType.methodType(Iterator.class, type)); + } catch (ReflectiveOperationException e) { + throw new AssertionError(e); + } + })) + ); + + private static final MethodHandle OBJECT_ARRAY_MH = ARRAY_TYPE_MH_MAPPING.get(Object[].class); + + static Iterator iterator(final boolean[] array) { + return new Iterator() { + int index = 0; + @Override public boolean hasNext() { return index < array.length; } + @Override public Boolean next() { return array[index++]; } + }; + } + static Iterator iterator(final byte[] array) { + return new Iterator() { + int index = 0; + @Override public boolean hasNext() { return index < array.length; } + @Override public Byte next() { return array[index++]; } + }; + } + static Iterator iterator(final short[] array) { + return new Iterator() { + int index = 0; + @Override public boolean hasNext() { return index < array.length; } + @Override public Short next() { return array[index++]; } + }; + } + static Iterator iterator(final int[] array) { + return new Iterator() { + int index = 0; + @Override public boolean hasNext() { return index < array.length; } + @Override public Integer next() { return array[index++]; } + }; + } + static Iterator iterator(final long[] array) { + return new Iterator() { + int index = 0; + @Override public boolean hasNext() { return index < array.length; } + @Override public Long next() { return array[index++]; } + }; + } + static Iterator iterator(final char[] array) { + return new Iterator() { + int index = 0; + @Override public boolean hasNext() { return index < array.length; } + @Override public Character next() { return array[index++]; } + }; + } + static Iterator iterator(final float[] array) { + return new Iterator() { + int index = 0; + @Override public boolean hasNext() { return index < array.length; } + @Override public Float next() { return array[index++]; } + }; + } + static Iterator iterator(final double[] array) { + return new Iterator() { + int index = 0; + @Override public boolean hasNext() { return index < array.length; } + @Override public Double next() { return array[index++]; } + }; + } + static Iterator iterator(final Object[] array) { + return new Iterator() { + int index = 0; + @Override public boolean hasNext() { return index < array.length; } + @Override public Object next() { return array[index++]; } + }; + } + + static MethodHandle newIterator(Class arrayType) { + if (!arrayType.isArray()) { + throw new IllegalArgumentException("type must be an array"); + } + return (ARRAY_TYPE_MH_MAPPING.containsKey(arrayType)) ? + ARRAY_TYPE_MH_MAPPING.get(arrayType) : + OBJECT_ARRAY_MH.asType(OBJECT_ARRAY_MH.type().changeParameterType(0, arrayType)); + } + + private ArrayIteratorHelper() {} + } + /** + * Returns a method handle to do iteration (for enhanced for loop) + * @param receiverClass Class of the array to load the value from + * @return a MethodHandle that accepts the receiver as first argument, returns iterator + */ + static MethodHandle lookupIterator(Class receiverClass) { + if (Iterable.class.isAssignableFrom(receiverClass)) { + return ITERATOR; + } else if (receiverClass.isArray()) { + return ArrayIteratorHelper.newIterator(receiverClass); + } else { + throw new IllegalArgumentException("Cannot iterate over [" + receiverClass.getCanonicalName() + "]"); + } + } // NOTE: Below methods are not cached, instead invoked directly because they are performant. // We also check for Long values first when possible since the type is more diff --git a/modules/lang-painless/src/main/java/org/elasticsearch/painless/DefBootstrap.java b/modules/lang-painless/src/main/java/org/elasticsearch/painless/DefBootstrap.java index 40b9cc6cbe8..5dc773672a7 100644 --- a/modules/lang-painless/src/main/java/org/elasticsearch/painless/DefBootstrap.java +++ b/modules/lang-painless/src/main/java/org/elasticsearch/painless/DefBootstrap.java @@ -1,5 +1,7 @@ package org.elasticsearch.painless; +import org.elasticsearch.common.SuppressForbidden; + /* * Licensed to Elasticsearch under one or more contributor * license agreements. See the NOTICE file distributed with @@ -56,6 +58,8 @@ public final class DefBootstrap { public static final int ARRAY_LOAD = 3; /** static bootstrap parameter indicating a dynamic array store, e.g. foo[bar] = baz */ public static final int ARRAY_STORE = 4; + /** static bootstrap parameter indicating a dynamic iteration, e.g. for (x : y) */ + public static final int ITERATOR = 5; /** * CallSite that implements the polymorphic inlining cache (PIC). @@ -64,14 +68,20 @@ public final class DefBootstrap { /** maximum number of types before we go megamorphic */ static final int MAX_DEPTH = 5; + private final Lookup lookup; private final String name; private final int flavor; + private final long recipe; int depth; // pkg-protected for testing - PIC(String name, MethodType type, int flavor) { + PIC(Lookup lookup, String name, MethodType type, int flavor, long recipe) { super(type); + this.lookup = lookup; this.name = name; this.flavor = flavor; + this.recipe = recipe; + assert recipe == 0 || flavor == METHOD_CALL; + assert Long.bitCount(recipe) <= type.parameterCount(); final MethodHandle fallback = FALLBACK.bindTo(this) .asCollector(Object[].class, type.parameterCount()) @@ -91,10 +101,10 @@ public final class DefBootstrap { /** * Does a slow lookup against the whitelist. */ - private static MethodHandle lookup(int flavor, Class clazz, String name, MethodType type) { + private MethodHandle lookup(int flavor, Class clazz, String name, Object[] args, long recipe) throws Throwable { switch(flavor) { case METHOD_CALL: - return Def.lookupMethod(clazz, name, type); + return Def.lookupMethod(lookup, clazz, name, args, recipe); case LOAD: return Def.lookupGetter(clazz, name); case STORE: @@ -103,6 +113,8 @@ public final class DefBootstrap { return Def.lookupArrayLoad(clazz); case ARRAY_STORE: return Def.lookupArrayStore(clazz); + case ITERATOR: + return Def.lookupIterator(clazz); default: throw new AssertionError(); } } @@ -111,11 +123,12 @@ public final class DefBootstrap { * Called when a new type is encountered (or, when we have encountered more than {@code MAX_DEPTH} * types at this call site and given up on caching). */ + @SuppressForbidden(reason = "slow path") Object fallback(Object[] args) throws Throwable { final MethodType type = type(); final Object receiver = args[0]; final Class receiverClass = receiver.getClass(); - final MethodHandle target = lookup(flavor, receiverClass, name, type).asType(type); + final MethodHandle target = lookup(flavor, receiverClass, name, args, recipe).asType(type); if (depth >= MAX_DEPTH) { // revert to a vtable call @@ -157,8 +170,8 @@ public final class DefBootstrap { *

    * see https://docs.oracle.com/javase/specs/jvms/se7/html/jvms-6.html#jvms-6.5.invokedynamic */ - public static CallSite bootstrap(Lookup lookup, String name, MethodType type, int flavor) { - return new PIC(name, type, flavor); + public static CallSite bootstrap(Lookup lookup, String name, MethodType type, int flavor, long recipe) { + return new PIC(lookup, name, type, flavor, recipe); } } diff --git a/modules/lang-painless/src/main/java/org/elasticsearch/painless/Definition.java b/modules/lang-painless/src/main/java/org/elasticsearch/painless/Definition.java index 50ee4649074..dbfc9993c31 100644 --- a/modules/lang-painless/src/main/java/org/elasticsearch/painless/Definition.java +++ b/modules/lang-painless/src/main/java/org/elasticsearch/painless/Definition.java @@ -19,18 +19,26 @@ package org.elasticsearch.painless; +import org.apache.lucene.util.Constants; +import org.apache.lucene.util.SetOnce; + import java.io.InputStream; import java.io.InputStreamReader; import java.io.LineNumberReader; import java.lang.invoke.MethodHandle; import java.lang.invoke.MethodHandles; +import java.lang.reflect.Modifier; import java.nio.charset.StandardCharsets; +import java.time.LocalDate; +import java.util.ArrayList; import java.util.Arrays; import java.util.Collections; import java.util.HashMap; import java.util.List; import java.util.Map; import java.util.Objects; +import java.util.PrimitiveIterator; +import java.util.Spliterator; /** * The entire API for Painless. Also used as a whitelist for checking for legal @@ -171,21 +179,6 @@ public final class Definition { } } - public static final class Constructor { - public final String name; - public final Struct owner; - public final List arguments; - public final org.objectweb.asm.commons.Method method; - - private Constructor(String name, Struct owner, List arguments, - org.objectweb.asm.commons.Method method) { - this.name = name; - this.owner = owner; - this.arguments = Collections.unmodifiableList(arguments); - this.method = method; - } - } - public static class Method { public final String name; public final Struct owner; @@ -287,12 +280,14 @@ public final class Definition { public final Class clazz; public final org.objectweb.asm.Type type; - public final Map constructors; + public final Map constructors; public final Map staticMethods; public final Map methods; public final Map staticMembers; public final Map members; + + private final SetOnce functionalMethod; private Struct(final String name, final Class clazz, final org.objectweb.asm.Type type) { this.name = name; @@ -305,6 +300,8 @@ public final class Definition { staticMembers = new HashMap<>(); members = new HashMap<>(); + + functionalMethod = new SetOnce(); } private Struct(final Struct struct) { @@ -318,6 +315,8 @@ public final class Definition { staticMembers = Collections.unmodifiableMap(struct.staticMembers); members = Collections.unmodifiableMap(struct.members); + + functionalMethod = struct.functionalMethod; } private Struct freeze() { @@ -343,6 +342,14 @@ public final class Definition { public int hashCode() { return name.hashCode(); } + + /** + * If this class is a functional interface according to JLS, returns its method. + * Otherwise returns null. + */ + public Method getFunctionalMethod() { + return functionalMethod.get(); + } } public static class Cast { @@ -439,6 +446,23 @@ public final class Definition { for (Map.Entry> clazz : hierarchy.entrySet()) { copyStruct(clazz.getKey(), clazz.getValue()); } + // if someone declares an interface type, its still an Object + for (Map.Entry clazz : structsMap.entrySet()) { + String name = clazz.getKey(); + Class javaPeer = clazz.getValue().clazz; + if (javaPeer.isInterface()) { + copyStruct(name, Collections.singletonList("Object")); + } else if (name.equals("def") == false && name.equals("Object") == false && javaPeer.isPrimitive() == false) { + // but otherwise, unless its a primitive type, it really should + assert hierarchy.get(name) != null : "class '" + name + "' does not extend Object!"; + assert hierarchy.get(name).contains("Object") : "class '" + name + "' does not extend Object!"; + } + } + // mark functional interfaces (or set null, to mark class is not) + for (Struct clazz : structsMap.values()) { + clazz.functionalMethod.set(computeFunctionalInterfaceMethod(clazz)); + } + // precompute runtime classes for (Struct struct : structsMap.values()) { addRuntimeClass(struct); @@ -573,7 +597,7 @@ public final class Definition { "Owner struct [" + struct + "] not defined for constructor [" + name + "]."); } - if (!name.matches("^[_a-zA-Z][_a-zA-Z0-9]*$")) { + if (!name.matches("")) { throw new IllegalArgumentException( "Invalid constructor name [" + name + "] with the struct [" + owner.name + "]."); } @@ -611,7 +635,18 @@ public final class Definition { } final org.objectweb.asm.commons.Method asm = org.objectweb.asm.commons.Method.getMethod(reflect); - final Constructor constructor = new Constructor(name, owner, Arrays.asList(args), asm); + final Type returnType = getTypeInternal("void"); + final MethodHandle handle; + + try { + handle = MethodHandles.publicLookup().in(owner.clazz).unreflectConstructor(reflect); + } catch (final IllegalAccessException exception) { + throw new IllegalArgumentException("Constructor " + + " not found for class [" + owner.clazz.getName() + "]" + + " with arguments " + Arrays.toString(classes) + "."); + } + + final Method constructor = new Method(name, owner, returnType, Arrays.asList(args), asm, reflect.getModifiers(), handle); owner.constructors.put(methodKey, constructor); } @@ -653,7 +688,7 @@ public final class Definition { if (!elements[0].equals(className)) { throw new IllegalArgumentException("Constructors must return their own type"); } - addConstructorInternal(className, "new", args); + addConstructorInternal(className, "", args); } else { if (methodName.indexOf('/') >= 0) { String nameAndAlias[] = methodName.split("/"); @@ -720,7 +755,7 @@ public final class Definition { " method [" + name + "]" + " within the struct [" + owner.name + "]."); } - + final org.objectweb.asm.commons.Method asm = org.objectweb.asm.commons.Method.getMethod(reflect); MethodHandle handle; @@ -821,11 +856,40 @@ public final class Definition { throw new ClassCastException("Child struct [" + child.name + "]" + " is not a super type of owner struct [" + owner.name + "] in copy."); } - + for (Map.Entry kvPair : child.methods.entrySet()) { MethodKey methodKey = kvPair.getKey(); Method method = kvPair.getValue(); if (owner.methods.get(methodKey) == null) { + // sanity check, look for missing covariant/generic override + if (owner.clazz.isInterface() && child.clazz == Object.class) { + // ok + } else if (child.clazz == Spliterator.OfPrimitive.class || child.clazz == PrimitiveIterator.class) { + // ok, we rely on generics erasure for these (its guaranteed in the javadocs though!!!!) + } else if (Constants.JRE_IS_MINIMUM_JAVA9 && owner.clazz == LocalDate.class) { + // ok, java 9 added covariant override for LocalDate.getEra() to return IsoEra: + // https://bugs.openjdk.java.net/browse/JDK-8072746 + } else { + try { + Class arguments[] = new Class[method.arguments.size()]; + for (int i = 0; i < method.arguments.size(); i++) { + arguments[i] = method.arguments.get(i).clazz; + } + java.lang.reflect.Method m = owner.clazz.getMethod(method.method.getName(), arguments); + if (m.getReturnType() != method.rtn.clazz) { + throw new IllegalStateException("missing covariant override for: " + m + " in " + owner.name); + } + if (m.isBridge() && !Modifier.isVolatile(method.modifiers)) { + // its a bridge in the destination, but not in the source, but it might still be ok, check generics: + java.lang.reflect.Method source = child.clazz.getMethod(method.method.getName(), arguments); + if (!Arrays.equals(source.getGenericParameterTypes(), source.getParameterTypes())) { + throw new IllegalStateException("missing generic override for: " + m + " in " + owner.name); + } + } + } catch (ReflectiveOperationException e) { + throw new AssertionError(e); + } + } owner.methods.put(methodKey, new Method(method.name, owner, method.rtn, method.arguments, method.method, method.modifiers, method.handle)); } @@ -890,6 +954,50 @@ public final class Definition { runtimeMap.put(struct.clazz, new RuntimeClass(methods, getters, setters)); } + + /** computes the functional interface method for a class, or returns null */ + private Method computeFunctionalInterfaceMethod(Struct clazz) { + if (!clazz.clazz.isInterface()) { + return null; + } + // if its marked with this annotation, we fail if the conditions don't hold (means whitelist bug) + // otherwise, this annotation is pretty useless. + boolean hasAnnotation = clazz.clazz.isAnnotationPresent(FunctionalInterface.class); + List methods = new ArrayList<>(); + for (java.lang.reflect.Method m : clazz.clazz.getMethods()) { + // default interface methods don't count + if (m.isDefault()) { + continue; + } + // static methods don't count + if (Modifier.isStatic(m.getModifiers())) { + continue; + } + // if its from Object, it doesn't count + try { + Object.class.getMethod(m.getName(), m.getParameterTypes()); + continue; + } catch (ReflectiveOperationException e) { + // it counts + } + methods.add(m); + } + if (methods.size() != 1) { + if (hasAnnotation) { + throw new IllegalArgumentException("Class: " + clazz.name + + " is marked with FunctionalInterface but doesn't fit the bill: " + methods); + } + return null; + } + // inspect the one method found from the reflection API, it should match the whitelist! + java.lang.reflect.Method oneMethod = methods.get(0); + Method painless = clazz.methods.get(new Definition.MethodKey(oneMethod.getName(), oneMethod.getParameterCount())); + if (painless == null || painless.method.equals(org.objectweb.asm.commons.Method.getMethod(oneMethod)) == false) { + throw new IllegalArgumentException("Class: " + clazz.name + " is functional but the functional " + + "method is not whitelisted!"); + } + return painless; + } private Type getTypeInternal(String name) { // simple types (e.g. 0 array dimensions) are a simple hash lookup for speed diff --git a/modules/lang-painless/src/main/java/org/elasticsearch/painless/FunctionRef.java b/modules/lang-painless/src/main/java/org/elasticsearch/painless/FunctionRef.java new file mode 100644 index 00000000000..03b6cc604c9 --- /dev/null +++ b/modules/lang-painless/src/main/java/org/elasticsearch/painless/FunctionRef.java @@ -0,0 +1,119 @@ +/* + * 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.painless; + +import org.elasticsearch.painless.Definition.Method; +import org.objectweb.asm.Handle; +import org.objectweb.asm.Opcodes; + +import java.lang.invoke.MethodHandle; +import java.lang.invoke.MethodType; +import java.lang.reflect.Modifier; + +/** + * Reference to a function or lambda. + *

    + * Once you have created one of these, you have "everything you need" to call LambdaMetaFactory + * either statically from bytecode with invokedynamic, or at runtime from Java. + */ +public class FunctionRef { + /** Function Object's method name */ + public final String invokedName; + /** CallSite signature */ + public final MethodType invokedType; + /** Implementation method */ + public final MethodHandle implMethod; + /** Function Object's method signature */ + public final MethodType samMethodType; + /** When bridging is required, request this bridge interface */ + public final MethodType interfaceMethodType; + + /** ASM "Handle" to the method, for the constant pool */ + public final Handle implMethodASM; + + /** + * Creates a new FunctionRef. + * @param expected interface type to implement. + * @param type the left hand side of a method reference expression + * @param call the right hand side of a method reference expression + */ + public FunctionRef(Definition.Type expected, String type, String call) { + boolean isCtorReference = "new".equals(call); + // check its really a functional interface + // for e.g. Comparable + Method method = expected.struct.getFunctionalMethod(); + if (method == null) { + throw new IllegalArgumentException("Cannot convert function reference [" + type + "::" + call + "] " + + "to [" + expected.name + "], not a functional interface"); + } + // e.g. compareTo + invokedName = method.name; + // e.g. (Object)Comparator + invokedType = MethodType.methodType(expected.clazz); + // e.g. (Object,Object)int + interfaceMethodType = method.handle.type().dropParameterTypes(0, 1); + // lookup requested method + Definition.Struct struct = Definition.getType(type).struct; + final Definition.Method impl; + // ctor ref + if (isCtorReference) { + impl = struct.constructors.get(new Definition.MethodKey("", method.arguments.size())); + } else { + // look for a static impl first + Definition.Method staticImpl = struct.staticMethods.get(new Definition.MethodKey(call, method.arguments.size())); + if (staticImpl == null) { + // otherwise a virtual impl + impl = struct.methods.get(new Definition.MethodKey(call, method.arguments.size()-1)); + } else { + impl = staticImpl; + } + } + if (impl == null) { + throw new IllegalArgumentException("Unknown reference [" + type + "::" + call + "] matching " + + "[" + expected + "]"); + } + + final int tag; + if (isCtorReference) { + tag = Opcodes.H_NEWINVOKESPECIAL; + } else if (Modifier.isStatic(impl.modifiers)) { + tag = Opcodes.H_INVOKESTATIC; + } else { + tag = Opcodes.H_INVOKEVIRTUAL; + } + implMethodASM = new Handle(tag, struct.type.getInternalName(), impl.name, impl.method.getDescriptor()); + implMethod = impl.handle; + if (isCtorReference) { + samMethodType = MethodType.methodType(interfaceMethodType.returnType(), impl.handle.type().parameterArray()); + } else if (Modifier.isStatic(impl.modifiers)) { + samMethodType = impl.handle.type(); + } else { + // ensure the receiver type is exact and not a superclass type + samMethodType = impl.handle.type().changeParameterType(0, struct.clazz); + } + } + + /** Returns true if you should ask LambdaMetaFactory to construct a bridge for the interface signature */ + public boolean needsBridges() { + // currently if the interface differs, we ask for a bridge, but maybe we should do smarter checking? + // either way, stuff will fail if its wrong :) + return interfaceMethodType.equals(samMethodType) == false; + } +} diff --git a/modules/lang-painless/src/main/java/org/elasticsearch/painless/MethodWriter.java b/modules/lang-painless/src/main/java/org/elasticsearch/painless/MethodWriter.java index 2663fe9c717..7b7caec320e 100644 --- a/modules/lang-painless/src/main/java/org/elasticsearch/painless/MethodWriter.java +++ b/modules/lang-painless/src/main/java/org/elasticsearch/painless/MethodWriter.java @@ -23,6 +23,7 @@ import org.elasticsearch.painless.Definition.Cast; import org.elasticsearch.painless.Definition.Sort; import org.elasticsearch.painless.Definition.Type; import org.objectweb.asm.ClassVisitor; +import org.objectweb.asm.ClassWriter; import org.objectweb.asm.Label; import org.objectweb.asm.Opcodes; import org.objectweb.asm.commons.GeneratorAdapter; @@ -87,30 +88,29 @@ import static org.elasticsearch.painless.WriterConstants.UTILITY_TYPE; * shared by the nodes of the Painless tree. */ public final class MethodWriter extends GeneratorAdapter { + private final ClassWriter parent; private final BitSet statements; private final Deque> stringConcatArgs = (INDY_STRING_CONCAT_BOOTSTRAP_HANDLE == null) ? null : new ArrayDeque<>(); - MethodWriter(int access, Method method, org.objectweb.asm.Type[] exceptions, ClassVisitor cv, BitSet statements) { - super(Opcodes.ASM5, cv.visitMethod(access, method.getName(), method.getDescriptor(), null, getInternalNames(exceptions)), + MethodWriter(int access, Method method, ClassWriter cw, BitSet statements) { + super(Opcodes.ASM5, cw.visitMethod(access, method.getName(), method.getDescriptor(), null, null), access, method.getName(), method.getDescriptor()); + + this.parent = cw; this.statements = statements; } - private static String[] getInternalNames(final org.objectweb.asm.Type[] types) { - if (types == null) { - return null; - } - String[] names = new String[types.length]; - for (int i = 0; i < names.length; ++i) { - names[i] = types[i].getInternalName(); - } - return names; + /** + * @return A new {@link MethodWriter} with the specified access and signature. + */ + MethodWriter newMethodWriter(int access, Method method) { + return new MethodWriter(access, method, parent, statements); } - /** - * Marks a new statement boundary. + /** + * Marks a new statement boundary. *

    * This is invoked for each statement boundary (leaf {@code S*} nodes). */ @@ -122,7 +122,7 @@ public final class MethodWriter extends GeneratorAdapter { statements.set(offset); } - /** + /** * Encodes the offset into the line number table as {@code offset + 1}. *

    * This is invoked before instructions that can hit exceptions. diff --git a/modules/lang-painless/src/main/java/org/elasticsearch/painless/PainlessScriptEngineService.java b/modules/lang-painless/src/main/java/org/elasticsearch/painless/PainlessScriptEngineService.java index 4971c08b086..0d8d9fa4a8b 100644 --- a/modules/lang-painless/src/main/java/org/elasticsearch/painless/PainlessScriptEngineService.java +++ b/modules/lang-painless/src/main/java/org/elasticsearch/painless/PainlessScriptEngineService.java @@ -40,7 +40,6 @@ import java.security.Permissions; import java.security.PrivilegedAction; import java.security.ProtectionDomain; import java.util.ArrayList; -import java.util.Collections; import java.util.HashMap; import java.util.List; import java.util.Map; diff --git a/modules/lang-painless/src/main/java/org/elasticsearch/painless/ScriptImpl.java b/modules/lang-painless/src/main/java/org/elasticsearch/painless/ScriptImpl.java index 380b6c2bf21..8fdc48e19ba 100644 --- a/modules/lang-painless/src/main/java/org/elasticsearch/painless/ScriptImpl.java +++ b/modules/lang-painless/src/main/java/org/elasticsearch/painless/ScriptImpl.java @@ -119,11 +119,17 @@ final class ScriptImpl implements ExecutableScript, LeafSearchScript { public Object run() { try { return executable.execute(variables, scorer, doc, aggregationValue); - } catch (PainlessError | Exception t) { + } catch (PainlessError | BootstrapMethodError | Exception t) { throw convertToScriptException(t); } } - + + /** + * Adds stack trace and other useful information to exceptiosn thrown + * from a Painless script. + * @param t The throwable to build an exception around. + * @return The generated ScriptException. + */ private ScriptException convertToScriptException(Throwable t) { // create a script stack: this is just the script portion List scriptStack = new ArrayList<>(); @@ -169,7 +175,7 @@ final class ScriptImpl implements ExecutableScript, LeafSearchScript { } throw new ScriptException("runtime error", t, scriptStack, name, PainlessScriptEngineService.NAME); } - + /** returns true for methods that are part of the runtime */ private static boolean shouldFilter(StackTraceElement element) { return element.getClassName().startsWith("org.elasticsearch.painless.") || diff --git a/modules/lang-painless/src/main/java/org/elasticsearch/painless/Variables.java b/modules/lang-painless/src/main/java/org/elasticsearch/painless/Variables.java index 16130476c38..4905011520a 100644 --- a/modules/lang-painless/src/main/java/org/elasticsearch/painless/Variables.java +++ b/modules/lang-painless/src/main/java/org/elasticsearch/painless/Variables.java @@ -138,15 +138,15 @@ public final class Variables { public void decrementScope() { int remove = scopes.pop(); - + while (remove > 0) { Variable variable = variables.pop(); - + // TODO: is this working? the code reads backwards... if (variable.read) { throw variable.location.createError(new IllegalArgumentException("Variable [" + variable.name + "] never used.")); } - + --remove; } } diff --git a/modules/lang-painless/src/main/java/org/elasticsearch/painless/Writer.java b/modules/lang-painless/src/main/java/org/elasticsearch/painless/Writer.java index f09bf5b87eb..4d4a778d558 100644 --- a/modules/lang-painless/src/main/java/org/elasticsearch/painless/Writer.java +++ b/modules/lang-painless/src/main/java/org/elasticsearch/painless/Writer.java @@ -41,8 +41,7 @@ import java.util.BitSet; final class Writer { static byte[] write(CompilerSettings settings, String name, String source, Variables variables, SSource root, BitSet expressions) { - Writer writer = new Writer(settings, name, source, variables, root, expressions); - return writer.getBytes(); + return new Writer(settings, name, source, variables, root, expressions).getBytes(); } private final CompilerSettings settings; @@ -66,7 +65,7 @@ final class Writer { writeBegin(); writeConstructor(); - adapter = new MethodWriter(Opcodes.ACC_PUBLIC, EXECUTE, null, writer, expressions); + adapter = new MethodWriter(Opcodes.ACC_PUBLIC, EXECUTE, writer, expressions); writeExecute(); writeEnd(); diff --git a/modules/lang-painless/src/main/java/org/elasticsearch/painless/WriterConstants.java b/modules/lang-painless/src/main/java/org/elasticsearch/painless/WriterConstants.java index a4804cde434..80082c4487b 100644 --- a/modules/lang-painless/src/main/java/org/elasticsearch/painless/WriterConstants.java +++ b/modules/lang-painless/src/main/java/org/elasticsearch/painless/WriterConstants.java @@ -27,9 +27,11 @@ import org.objectweb.asm.Type; import org.objectweb.asm.commons.Method; import java.lang.invoke.CallSite; +import java.lang.invoke.LambdaMetafactory; import java.lang.invoke.MethodHandles; import java.lang.invoke.MethodType; import java.util.BitSet; +import java.util.Iterator; import java.util.Map; /** @@ -56,13 +58,17 @@ public final class WriterConstants { public final static Type MAP_TYPE = Type.getType(Map.class); public final static Method MAP_GET = getAsmMethod(Object.class, "get", Object.class); + public final static Type ITERATOR_TYPE = Type.getType(Iterator.class); + public final static Method ITERATOR_HASNEXT = getAsmMethod(boolean.class, "hasNext"); + public final static Method ITERATOR_NEXT = getAsmMethod(Object.class, "next"); + public final static Type UTILITY_TYPE = Type.getType(Utility.class); public final static Method STRING_TO_CHAR = getAsmMethod(char.class, "StringTochar", String.class); public final static Method CHAR_TO_STRING = getAsmMethod(String.class, "charToString", char.class); /** dynamic callsite bootstrap signature */ public final static MethodType DEF_BOOTSTRAP_TYPE = - MethodType.methodType(CallSite.class, MethodHandles.Lookup.class, String.class, MethodType.class, int.class); + MethodType.methodType(CallSite.class, MethodHandles.Lookup.class, String.class, MethodType.class, int.class, long.class); public final static Handle DEF_BOOTSTRAP_HANDLE = new Handle(Opcodes.H_INVOKESTATIC, Type.getInternalName(DefBootstrap.class), "bootstrap", DEF_BOOTSTRAP_TYPE.toMethodDescriptorString()); @@ -101,7 +107,15 @@ public final class WriterConstants { public final static Method DEF_LTE_CALL = getAsmMethod(boolean.class, "lte", Object.class, Object.class); public final static Method DEF_GT_CALL = getAsmMethod(boolean.class, "gt" , Object.class, Object.class); public final static Method DEF_GTE_CALL = getAsmMethod(boolean.class, "gte", Object.class, Object.class); - + + /** invokedynamic bootstrap for lambda expression/method references */ + public final static MethodType LAMBDA_BOOTSTRAP_TYPE = + MethodType.methodType(CallSite.class, MethodHandles.Lookup.class, String.class, + MethodType.class, Object[].class); + public final static Handle LAMBDA_BOOTSTRAP_HANDLE = + new Handle(Opcodes.H_INVOKESTATIC, Type.getInternalName(LambdaMetafactory.class), + "altMetafactory", LAMBDA_BOOTSTRAP_TYPE.toMethodDescriptorString()); + /** dynamic invokedynamic bootstrap for indy string concats (Java 9+) */ public final static Handle INDY_STRING_CONCAT_BOOTSTRAP_HANDLE; static { @@ -138,7 +152,7 @@ public final class WriterConstants { public final static Method CHECKEQUALS = getAsmMethod(boolean.class, "checkEquals", Object.class, Object.class); - private static Method getAsmMethod(final Class rtype, final String name, final Class... ptypes) { + public static Method getAsmMethod(final Class rtype, final String name, final Class... ptypes) { return new Method(name, MethodType.methodType(rtype, ptypes).toMethodDescriptorString()); } diff --git a/modules/lang-painless/src/main/java/org/elasticsearch/painless/antlr/PainlessParser.java b/modules/lang-painless/src/main/java/org/elasticsearch/painless/antlr/PainlessParser.java index 98eb16732fd..540773fbf70 100644 --- a/modules/lang-painless/src/main/java/org/elasticsearch/painless/antlr/PainlessParser.java +++ b/modules/lang-painless/src/main/java/org/elasticsearch/painless/antlr/PainlessParser.java @@ -356,6 +356,28 @@ class PainlessParser extends Parser { else return visitor.visitChildren(this); } } + public static class EachContext extends StatementContext { + public TerminalNode FOR() { return getToken(PainlessParser.FOR, 0); } + public TerminalNode LP() { return getToken(PainlessParser.LP, 0); } + public DecltypeContext decltype() { + return getRuleContext(DecltypeContext.class,0); + } + public TerminalNode ID() { return getToken(PainlessParser.ID, 0); } + public TerminalNode COLON() { return getToken(PainlessParser.COLON, 0); } + public ExpressionContext expression() { + return getRuleContext(ExpressionContext.class,0); + } + public TerminalNode RP() { return getToken(PainlessParser.RP, 0); } + public TrailerContext trailer() { + return getRuleContext(TrailerContext.class,0); + } + public EachContext(StatementContext ctx) { copyFrom(ctx); } + @Override + public T accept(ParseTreeVisitor visitor) { + if ( visitor instanceof PainlessParserVisitor ) return ((PainlessParserVisitor)visitor).visitEach(this); + else return visitor.visitChildren(this); + } + } public static class ReturnContext extends StatementContext { public TerminalNode RETURN() { return getToken(PainlessParser.RETURN, 0); } public ExpressionContext expression() { @@ -377,7 +399,7 @@ class PainlessParser extends Parser { enterRule(_localctx, 2, RULE_statement); try { int _alt; - setState(121); + setState(130); switch ( getInterpreter().adaptivePredict(_input,8,_ctx) ) { case 1: _localctx = new IfContext(_localctx); @@ -520,56 +542,78 @@ class PainlessParser extends Parser { } break; case 5: - _localctx = new DeclContext(_localctx); + _localctx = new EachContext(_localctx); enterOuterAlt(_localctx, 5); { setState(96); - declaration(); + match(FOR); setState(97); - delimiter(); + match(LP); + setState(98); + decltype(); + setState(99); + match(ID); + setState(100); + match(COLON); + setState(101); + expression(0); + setState(102); + match(RP); + setState(103); + trailer(); } break; case 6: - _localctx = new ContinueContext(_localctx); + _localctx = new DeclContext(_localctx); enterOuterAlt(_localctx, 6); { - setState(99); - match(CONTINUE); - setState(100); + setState(105); + declaration(); + setState(106); delimiter(); } break; case 7: - _localctx = new BreakContext(_localctx); + _localctx = new ContinueContext(_localctx); enterOuterAlt(_localctx, 7); { - setState(101); - match(BREAK); - setState(102); + setState(108); + match(CONTINUE); + setState(109); delimiter(); } break; case 8: - _localctx = new ReturnContext(_localctx); + _localctx = new BreakContext(_localctx); enterOuterAlt(_localctx, 8); { - setState(103); - match(RETURN); - setState(104); - expression(0); - setState(105); + setState(110); + match(BREAK); + setState(111); delimiter(); } break; case 9: - _localctx = new TryContext(_localctx); + _localctx = new ReturnContext(_localctx); enterOuterAlt(_localctx, 9); { - setState(107); + setState(112); + match(RETURN); + setState(113); + expression(0); + setState(114); + delimiter(); + } + break; + case 10: + _localctx = new TryContext(_localctx); + enterOuterAlt(_localctx, 10); + { + setState(116); match(TRY); - setState(108); + setState(117); block(); - setState(110); + setState(119); _errHandler.sync(this); _alt = 1; do { @@ -577,7 +621,7 @@ class PainlessParser extends Parser { case 1: { { - setState(109); + setState(118); trap(); } } @@ -585,31 +629,31 @@ class PainlessParser extends Parser { default: throw new NoViableAltException(this); } - setState(112); + setState(121); _errHandler.sync(this); _alt = getInterpreter().adaptivePredict(_input,7,_ctx); } while ( _alt!=2 && _alt!=org.antlr.v4.runtime.atn.ATN.INVALID_ALT_NUMBER ); } break; - case 10: + case 11: _localctx = new ThrowContext(_localctx); - enterOuterAlt(_localctx, 10); + enterOuterAlt(_localctx, 11); { - setState(114); + setState(123); match(THROW); - setState(115); + setState(124); expression(0); - setState(116); + setState(125); delimiter(); } break; - case 11: + case 12: _localctx = new ExprContext(_localctx); - enterOuterAlt(_localctx, 11); + enterOuterAlt(_localctx, 12); { - setState(118); + setState(127); expression(0); - setState(119); + setState(128); delimiter(); } break; @@ -648,19 +692,19 @@ class PainlessParser extends Parser { TrailerContext _localctx = new TrailerContext(_ctx, getState()); enterRule(_localctx, 4, RULE_trailer); try { - setState(125); + setState(134); switch ( getInterpreter().adaptivePredict(_input,9,_ctx) ) { case 1: enterOuterAlt(_localctx, 1); { - setState(123); + setState(132); block(); } break; case 2: enterOuterAlt(_localctx, 2); { - setState(124); + setState(133); statement(); } break; @@ -704,25 +748,25 @@ class PainlessParser extends Parser { int _alt; enterOuterAlt(_localctx, 1); { - setState(127); + setState(136); match(LBRACK); - setState(131); + setState(140); _errHandler.sync(this); _alt = getInterpreter().adaptivePredict(_input,10,_ctx); while ( _alt!=2 && _alt!=org.antlr.v4.runtime.atn.ATN.INVALID_ALT_NUMBER ) { if ( _alt==1 ) { { { - setState(128); + setState(137); statement(); } } } - setState(133); + setState(142); _errHandler.sync(this); _alt = getInterpreter().adaptivePredict(_input,10,_ctx); } - setState(134); + setState(143); match(RBRACK); } } @@ -756,7 +800,7 @@ class PainlessParser extends Parser { try { enterOuterAlt(_localctx, 1); { - setState(136); + setState(145); match(SEMICOLON); } } @@ -793,19 +837,19 @@ class PainlessParser extends Parser { InitializerContext _localctx = new InitializerContext(_ctx, getState()); enterRule(_localctx, 10, RULE_initializer); try { - setState(140); + setState(149); switch ( getInterpreter().adaptivePredict(_input,11,_ctx) ) { case 1: enterOuterAlt(_localctx, 1); { - setState(138); + setState(147); declaration(); } break; case 2: enterOuterAlt(_localctx, 2); { - setState(139); + setState(148); expression(0); } break; @@ -843,7 +887,7 @@ class PainlessParser extends Parser { try { enterOuterAlt(_localctx, 1); { - setState(142); + setState(151); expression(0); } } @@ -890,23 +934,23 @@ class PainlessParser extends Parser { try { enterOuterAlt(_localctx, 1); { - setState(144); + setState(153); decltype(); - setState(145); + setState(154); declvar(); - setState(150); + setState(159); _errHandler.sync(this); _la = _input.LA(1); while (_la==COMMA) { { { - setState(146); + setState(155); match(COMMA); - setState(147); + setState(156); declvar(); } } - setState(152); + setState(161); _errHandler.sync(this); _la = _input.LA(1); } @@ -951,21 +995,21 @@ class PainlessParser extends Parser { try { enterOuterAlt(_localctx, 1); { - setState(153); + setState(162); match(TYPE); - setState(158); + setState(167); _errHandler.sync(this); _la = _input.LA(1); while (_la==LBRACE) { { { - setState(154); + setState(163); match(LBRACE); - setState(155); + setState(164); match(RBRACE); } } - setState(160); + setState(169); _errHandler.sync(this); _la = _input.LA(1); } @@ -986,6 +1030,7 @@ class PainlessParser extends Parser { public TerminalNode TYPE() { return getToken(PainlessParser.TYPE, 0); } public TerminalNode REF() { return getToken(PainlessParser.REF, 0); } public TerminalNode ID() { return getToken(PainlessParser.ID, 0); } + public TerminalNode NEW() { return getToken(PainlessParser.NEW, 0); } public FuncrefContext(ParserRuleContext parent, int invokingState) { super(parent, invokingState); } @@ -1000,15 +1045,21 @@ class PainlessParser extends Parser { public final FuncrefContext funcref() throws RecognitionException { FuncrefContext _localctx = new FuncrefContext(_ctx, getState()); enterRule(_localctx, 18, RULE_funcref); + int _la; try { enterOuterAlt(_localctx, 1); { - setState(161); + setState(170); match(TYPE); - setState(162); + setState(171); match(REF); - setState(163); - match(ID); + setState(172); + _la = _input.LA(1); + if ( !(_la==NEW || _la==ID) ) { + _errHandler.recoverInline(this); + } else { + consume(); + } } } catch (RecognitionException re) { @@ -1046,15 +1097,15 @@ class PainlessParser extends Parser { try { enterOuterAlt(_localctx, 1); { - setState(165); + setState(174); match(ID); - setState(168); + setState(177); _la = _input.LA(1); if (_la==ASSIGN) { { - setState(166); + setState(175); match(ASSIGN); - setState(167); + setState(176); expression(0); } } @@ -1098,17 +1149,17 @@ class PainlessParser extends Parser { try { enterOuterAlt(_localctx, 1); { - setState(170); + setState(179); match(CATCH); - setState(171); + setState(180); match(LP); - setState(172); + setState(181); match(TYPE); - setState(173); + setState(182); match(ID); - setState(174); + setState(183); match(RP); - setState(175); + setState(184); block(); } } @@ -1144,7 +1195,7 @@ class PainlessParser extends Parser { try { enterOuterAlt(_localctx, 1); { - setState(177); + setState(186); _la = _input.LA(1); if ( !(_la==EOF || _la==SEMICOLON) ) { _errHandler.recoverInline(this); @@ -1313,7 +1364,7 @@ class PainlessParser extends Parser { int _alt; enterOuterAlt(_localctx, 1); { - setState(188); + setState(197); switch ( getInterpreter().adaptivePredict(_input,15,_ctx) ) { case 1: { @@ -1321,16 +1372,16 @@ class PainlessParser extends Parser { _ctx = _localctx; _prevctx = _localctx; - setState(180); + setState(189); chain(true); - setState(181); + setState(190); _la = _input.LA(1); if ( !((((_la) & ~0x3f) == 0 && ((1L << _la) & ((1L << ASSIGN) | (1L << AADD) | (1L << ASUB) | (1L << AMUL) | (1L << ADIV) | (1L << AREM) | (1L << AAND) | (1L << AXOR) | (1L << AOR) | (1L << ALSH) | (1L << ARSH) | (1L << AUSH))) != 0)) ) { _errHandler.recoverInline(this); } else { consume(); } - setState(182); + setState(191); expression(1); ((AssignmentContext)_localctx).s = false; } @@ -1340,14 +1391,14 @@ class PainlessParser extends Parser { _localctx = new SingleContext(_localctx); _ctx = _localctx; _prevctx = _localctx; - setState(185); + setState(194); ((SingleContext)_localctx).u = unary(false); ((SingleContext)_localctx).s = ((SingleContext)_localctx).u.s; } break; } _ctx.stop = _input.LT(-1); - setState(249); + setState(258); _errHandler.sync(this); _alt = getInterpreter().adaptivePredict(_input,17,_ctx); while ( _alt!=2 && _alt!=org.antlr.v4.runtime.atn.ATN.INVALID_ALT_NUMBER ) { @@ -1355,22 +1406,22 @@ class PainlessParser extends Parser { if ( _parseListeners!=null ) triggerExitRuleEvent(); _prevctx = _localctx; { - setState(247); + setState(256); switch ( getInterpreter().adaptivePredict(_input,16,_ctx) ) { case 1: { _localctx = new BinaryContext(new ExpressionContext(_parentctx, _parentState)); pushNewRecursionContext(_localctx, _startState, RULE_expression); - setState(190); + setState(199); if (!(precpred(_ctx, 12))) throw new FailedPredicateException(this, "precpred(_ctx, 12)"); - setState(191); + setState(200); _la = _input.LA(1); if ( !((((_la) & ~0x3f) == 0 && ((1L << _la) & ((1L << MUL) | (1L << DIV) | (1L << REM))) != 0)) ) { _errHandler.recoverInline(this); } else { consume(); } - setState(192); + setState(201); expression(13); ((BinaryContext)_localctx).s = false; } @@ -1379,16 +1430,16 @@ class PainlessParser extends Parser { { _localctx = new BinaryContext(new ExpressionContext(_parentctx, _parentState)); pushNewRecursionContext(_localctx, _startState, RULE_expression); - setState(195); + setState(204); if (!(precpred(_ctx, 11))) throw new FailedPredicateException(this, "precpred(_ctx, 11)"); - setState(196); + setState(205); _la = _input.LA(1); if ( !(_la==ADD || _la==SUB) ) { _errHandler.recoverInline(this); } else { consume(); } - setState(197); + setState(206); expression(12); ((BinaryContext)_localctx).s = false; } @@ -1397,16 +1448,16 @@ class PainlessParser extends Parser { { _localctx = new BinaryContext(new ExpressionContext(_parentctx, _parentState)); pushNewRecursionContext(_localctx, _startState, RULE_expression); - setState(200); + setState(209); if (!(precpred(_ctx, 10))) throw new FailedPredicateException(this, "precpred(_ctx, 10)"); - setState(201); + setState(210); _la = _input.LA(1); if ( !((((_la) & ~0x3f) == 0 && ((1L << _la) & ((1L << LSH) | (1L << RSH) | (1L << USH))) != 0)) ) { _errHandler.recoverInline(this); } else { consume(); } - setState(202); + setState(211); expression(11); ((BinaryContext)_localctx).s = false; } @@ -1415,16 +1466,16 @@ class PainlessParser extends Parser { { _localctx = new CompContext(new ExpressionContext(_parentctx, _parentState)); pushNewRecursionContext(_localctx, _startState, RULE_expression); - setState(205); + setState(214); if (!(precpred(_ctx, 9))) throw new FailedPredicateException(this, "precpred(_ctx, 9)"); - setState(206); + setState(215); _la = _input.LA(1); if ( !((((_la) & ~0x3f) == 0 && ((1L << _la) & ((1L << LT) | (1L << LTE) | (1L << GT) | (1L << GTE))) != 0)) ) { _errHandler.recoverInline(this); } else { consume(); } - setState(207); + setState(216); expression(10); ((CompContext)_localctx).s = false; } @@ -1433,16 +1484,16 @@ class PainlessParser extends Parser { { _localctx = new CompContext(new ExpressionContext(_parentctx, _parentState)); pushNewRecursionContext(_localctx, _startState, RULE_expression); - setState(210); + setState(219); if (!(precpred(_ctx, 8))) throw new FailedPredicateException(this, "precpred(_ctx, 8)"); - setState(211); + setState(220); _la = _input.LA(1); if ( !((((_la) & ~0x3f) == 0 && ((1L << _la) & ((1L << EQ) | (1L << EQR) | (1L << NE) | (1L << NER))) != 0)) ) { _errHandler.recoverInline(this); } else { consume(); } - setState(212); + setState(221); expression(9); ((CompContext)_localctx).s = false; } @@ -1451,11 +1502,11 @@ class PainlessParser extends Parser { { _localctx = new BinaryContext(new ExpressionContext(_parentctx, _parentState)); pushNewRecursionContext(_localctx, _startState, RULE_expression); - setState(215); + setState(224); if (!(precpred(_ctx, 7))) throw new FailedPredicateException(this, "precpred(_ctx, 7)"); - setState(216); + setState(225); match(BWAND); - setState(217); + setState(226); expression(8); ((BinaryContext)_localctx).s = false; } @@ -1464,11 +1515,11 @@ class PainlessParser extends Parser { { _localctx = new BinaryContext(new ExpressionContext(_parentctx, _parentState)); pushNewRecursionContext(_localctx, _startState, RULE_expression); - setState(220); + setState(229); if (!(precpred(_ctx, 6))) throw new FailedPredicateException(this, "precpred(_ctx, 6)"); - setState(221); + setState(230); match(XOR); - setState(222); + setState(231); expression(7); ((BinaryContext)_localctx).s = false; } @@ -1477,11 +1528,11 @@ class PainlessParser extends Parser { { _localctx = new BinaryContext(new ExpressionContext(_parentctx, _parentState)); pushNewRecursionContext(_localctx, _startState, RULE_expression); - setState(225); + setState(234); if (!(precpred(_ctx, 5))) throw new FailedPredicateException(this, "precpred(_ctx, 5)"); - setState(226); + setState(235); match(BWOR); - setState(227); + setState(236); expression(6); ((BinaryContext)_localctx).s = false; } @@ -1490,11 +1541,11 @@ class PainlessParser extends Parser { { _localctx = new BoolContext(new ExpressionContext(_parentctx, _parentState)); pushNewRecursionContext(_localctx, _startState, RULE_expression); - setState(230); + setState(239); if (!(precpred(_ctx, 4))) throw new FailedPredicateException(this, "precpred(_ctx, 4)"); - setState(231); + setState(240); match(BOOLAND); - setState(232); + setState(241); expression(5); ((BoolContext)_localctx).s = false; } @@ -1503,11 +1554,11 @@ class PainlessParser extends Parser { { _localctx = new BoolContext(new ExpressionContext(_parentctx, _parentState)); pushNewRecursionContext(_localctx, _startState, RULE_expression); - setState(235); + setState(244); if (!(precpred(_ctx, 3))) throw new FailedPredicateException(this, "precpred(_ctx, 3)"); - setState(236); + setState(245); match(BOOLOR); - setState(237); + setState(246); expression(4); ((BoolContext)_localctx).s = false; } @@ -1516,15 +1567,15 @@ class PainlessParser extends Parser { { _localctx = new ConditionalContext(new ExpressionContext(_parentctx, _parentState)); pushNewRecursionContext(_localctx, _startState, RULE_expression); - setState(240); + setState(249); if (!(precpred(_ctx, 2))) throw new FailedPredicateException(this, "precpred(_ctx, 2)"); - setState(241); + setState(250); match(COND); - setState(242); + setState(251); ((ConditionalContext)_localctx).e0 = expression(0); - setState(243); + setState(252); match(COLON); - setState(244); + setState(253); ((ConditionalContext)_localctx).e1 = expression(2); ((ConditionalContext)_localctx).s = ((ConditionalContext)_localctx).e0.s && ((ConditionalContext)_localctx).e1.s; } @@ -1532,7 +1583,7 @@ class PainlessParser extends Parser { } } } - setState(251); + setState(260); _errHandler.sync(this); _alt = getInterpreter().adaptivePredict(_input,17,_ctx); } @@ -1679,22 +1730,22 @@ class PainlessParser extends Parser { enterRule(_localctx, 28, RULE_unary); int _la; try { - setState(281); + setState(290); switch ( getInterpreter().adaptivePredict(_input,18,_ctx) ) { case 1: _localctx = new PreContext(_localctx); enterOuterAlt(_localctx, 1); { - setState(252); + setState(261); if (!( !_localctx.c )) throw new FailedPredicateException(this, " !$c "); - setState(253); + setState(262); _la = _input.LA(1); if ( !(_la==INCR || _la==DECR) ) { _errHandler.recoverInline(this); } else { consume(); } - setState(254); + setState(263); chain(true); } break; @@ -1702,11 +1753,11 @@ class PainlessParser extends Parser { _localctx = new PostContext(_localctx); enterOuterAlt(_localctx, 2); { - setState(255); + setState(264); if (!( !_localctx.c )) throw new FailedPredicateException(this, " !$c "); - setState(256); + setState(265); chain(true); - setState(257); + setState(266); _la = _input.LA(1); if ( !(_la==INCR || _la==DECR) ) { _errHandler.recoverInline(this); @@ -1719,9 +1770,9 @@ class PainlessParser extends Parser { _localctx = new ReadContext(_localctx); enterOuterAlt(_localctx, 3); { - setState(259); + setState(268); if (!( !_localctx.c )) throw new FailedPredicateException(this, " !$c "); - setState(260); + setState(269); chain(false); } break; @@ -1729,9 +1780,9 @@ class PainlessParser extends Parser { _localctx = new NumericContext(_localctx); enterOuterAlt(_localctx, 4); { - setState(261); + setState(270); if (!( !_localctx.c )) throw new FailedPredicateException(this, " !$c "); - setState(262); + setState(271); _la = _input.LA(1); if ( !(((((_la - 64)) & ~0x3f) == 0 && ((1L << (_la - 64)) & ((1L << (OCTAL - 64)) | (1L << (HEX - 64)) | (1L << (INTEGER - 64)) | (1L << (DECIMAL - 64)))) != 0)) ) { _errHandler.recoverInline(this); @@ -1745,9 +1796,9 @@ class PainlessParser extends Parser { _localctx = new TrueContext(_localctx); enterOuterAlt(_localctx, 5); { - setState(264); + setState(273); if (!( !_localctx.c )) throw new FailedPredicateException(this, " !$c "); - setState(265); + setState(274); match(TRUE); ((TrueContext)_localctx).s = false; } @@ -1756,9 +1807,9 @@ class PainlessParser extends Parser { _localctx = new FalseContext(_localctx); enterOuterAlt(_localctx, 6); { - setState(267); + setState(276); if (!( !_localctx.c )) throw new FailedPredicateException(this, " !$c "); - setState(268); + setState(277); match(FALSE); ((FalseContext)_localctx).s = false; } @@ -1767,9 +1818,9 @@ class PainlessParser extends Parser { _localctx = new NullContext(_localctx); enterOuterAlt(_localctx, 7); { - setState(270); + setState(279); if (!( !_localctx.c )) throw new FailedPredicateException(this, " !$c "); - setState(271); + setState(280); match(NULL); ((NullContext)_localctx).s = false; } @@ -1778,16 +1829,16 @@ class PainlessParser extends Parser { _localctx = new OperatorContext(_localctx); enterOuterAlt(_localctx, 8); { - setState(273); + setState(282); if (!( !_localctx.c )) throw new FailedPredicateException(this, " !$c "); - setState(274); + setState(283); _la = _input.LA(1); if ( !((((_la) & ~0x3f) == 0 && ((1L << _la) & ((1L << BOOLNOT) | (1L << BWNOT) | (1L << ADD) | (1L << SUB))) != 0)) ) { _errHandler.recoverInline(this); } else { consume(); } - setState(275); + setState(284); unary(false); } break; @@ -1795,13 +1846,13 @@ class PainlessParser extends Parser { _localctx = new CastContext(_localctx); enterOuterAlt(_localctx, 9); { - setState(276); + setState(285); match(LP); - setState(277); + setState(286); decltype(); - setState(278); + setState(287); match(RP); - setState(279); + setState(288); unary(_localctx.c); } break; @@ -1910,27 +1961,27 @@ class PainlessParser extends Parser { enterRule(_localctx, 30, RULE_chain); try { int _alt; - setState(317); + setState(326); switch ( getInterpreter().adaptivePredict(_input,24,_ctx) ) { case 1: _localctx = new DynamicContext(_localctx); enterOuterAlt(_localctx, 1); { - setState(283); + setState(292); ((DynamicContext)_localctx).p = primary(_localctx.c); - setState(287); + setState(296); _errHandler.sync(this); _alt = getInterpreter().adaptivePredict(_input,19,_ctx); while ( _alt!=2 && _alt!=org.antlr.v4.runtime.atn.ATN.INVALID_ALT_NUMBER ) { if ( _alt==1 ) { { { - setState(284); + setState(293); secondary(((DynamicContext)_localctx).p.s); } } } - setState(289); + setState(298); _errHandler.sync(this); _alt = getInterpreter().adaptivePredict(_input,19,_ctx); } @@ -1940,23 +1991,23 @@ class PainlessParser extends Parser { _localctx = new StaticContext(_localctx); enterOuterAlt(_localctx, 2); { - setState(290); + setState(299); decltype(); - setState(291); + setState(300); dot(); - setState(295); + setState(304); _errHandler.sync(this); _alt = getInterpreter().adaptivePredict(_input,20,_ctx); while ( _alt!=2 && _alt!=org.antlr.v4.runtime.atn.ATN.INVALID_ALT_NUMBER ) { if ( _alt==1 ) { { { - setState(292); + setState(301); secondary(true); } } } - setState(297); + setState(306); _errHandler.sync(this); _alt = getInterpreter().adaptivePredict(_input,20,_ctx); } @@ -1966,11 +2017,11 @@ class PainlessParser extends Parser { _localctx = new NewarrayContext(_localctx); enterOuterAlt(_localctx, 3); { - setState(298); + setState(307); match(NEW); - setState(299); + setState(308); match(TYPE); - setState(304); + setState(313); _errHandler.sync(this); _alt = 1; do { @@ -1978,11 +2029,11 @@ class PainlessParser extends Parser { case 1: { { - setState(300); + setState(309); match(LBRACE); - setState(301); + setState(310); expression(0); - setState(302); + setState(311); match(RBRACE); } } @@ -1990,29 +2041,29 @@ class PainlessParser extends Parser { default: throw new NoViableAltException(this); } - setState(306); + setState(315); _errHandler.sync(this); _alt = getInterpreter().adaptivePredict(_input,21,_ctx); } while ( _alt!=2 && _alt!=org.antlr.v4.runtime.atn.ATN.INVALID_ALT_NUMBER ); - setState(315); + setState(324); switch ( getInterpreter().adaptivePredict(_input,23,_ctx) ) { case 1: { - setState(308); + setState(317); dot(); - setState(312); + setState(321); _errHandler.sync(this); _alt = getInterpreter().adaptivePredict(_input,22,_ctx); while ( _alt!=2 && _alt!=org.antlr.v4.runtime.atn.ATN.INVALID_ALT_NUMBER ) { if ( _alt==1 ) { { { - setState(309); + setState(318); secondary(true); } } } - setState(314); + setState(323); _errHandler.sync(this); _alt = getInterpreter().adaptivePredict(_input,22,_ctx); } @@ -2114,19 +2165,19 @@ class PainlessParser extends Parser { PrimaryContext _localctx = new PrimaryContext(_ctx, getState(), c); enterRule(_localctx, 32, RULE_primary); try { - setState(335); + setState(344); switch ( getInterpreter().adaptivePredict(_input,25,_ctx) ) { case 1: _localctx = new ExprprecContext(_localctx); enterOuterAlt(_localctx, 1); { - setState(319); + setState(328); if (!( !_localctx.c )) throw new FailedPredicateException(this, " !$c "); - setState(320); + setState(329); match(LP); - setState(321); + setState(330); ((ExprprecContext)_localctx).e = expression(0); - setState(322); + setState(331); match(RP); ((ExprprecContext)_localctx).s = ((ExprprecContext)_localctx).e.s; } @@ -2135,13 +2186,13 @@ class PainlessParser extends Parser { _localctx = new ChainprecContext(_localctx); enterOuterAlt(_localctx, 2); { - setState(325); + setState(334); if (!( _localctx.c )) throw new FailedPredicateException(this, " $c "); - setState(326); + setState(335); match(LP); - setState(327); + setState(336); unary(true); - setState(328); + setState(337); match(RP); } break; @@ -2149,7 +2200,7 @@ class PainlessParser extends Parser { _localctx = new StringContext(_localctx); enterOuterAlt(_localctx, 3); { - setState(330); + setState(339); match(STRING); } break; @@ -2157,7 +2208,7 @@ class PainlessParser extends Parser { _localctx = new VariableContext(_localctx); enterOuterAlt(_localctx, 4); { - setState(331); + setState(340); match(ID); } break; @@ -2165,11 +2216,11 @@ class PainlessParser extends Parser { _localctx = new NewobjectContext(_localctx); enterOuterAlt(_localctx, 5); { - setState(332); + setState(341); match(NEW); - setState(333); + setState(342); match(TYPE); - setState(334); + setState(343); arguments(); } break; @@ -2211,23 +2262,23 @@ class PainlessParser extends Parser { SecondaryContext _localctx = new SecondaryContext(_ctx, getState(), s); enterRule(_localctx, 34, RULE_secondary); try { - setState(341); + setState(350); switch ( getInterpreter().adaptivePredict(_input,26,_ctx) ) { case 1: enterOuterAlt(_localctx, 1); { - setState(337); + setState(346); if (!( _localctx.s )) throw new FailedPredicateException(this, " $s "); - setState(338); + setState(347); dot(); } break; case 2: enterOuterAlt(_localctx, 2); { - setState(339); + setState(348); if (!( _localctx.s )) throw new FailedPredicateException(this, " $s "); - setState(340); + setState(349); brace(); } break; @@ -2285,17 +2336,17 @@ class PainlessParser extends Parser { enterRule(_localctx, 36, RULE_dot); int _la; try { - setState(348); + setState(357); switch ( getInterpreter().adaptivePredict(_input,27,_ctx) ) { case 1: _localctx = new CallinvokeContext(_localctx); enterOuterAlt(_localctx, 1); { - setState(343); + setState(352); match(DOT); - setState(344); + setState(353); match(DOTID); - setState(345); + setState(354); arguments(); } break; @@ -2303,9 +2354,9 @@ class PainlessParser extends Parser { _localctx = new FieldaccessContext(_localctx); enterOuterAlt(_localctx, 2); { - setState(346); + setState(355); match(DOT); - setState(347); + setState(356); _la = _input.LA(1); if ( !(_la==DOTINTEGER || _la==DOTID) ) { _errHandler.recoverInline(this); @@ -2359,11 +2410,11 @@ class PainlessParser extends Parser { _localctx = new BraceaccessContext(_localctx); enterOuterAlt(_localctx, 1); { - setState(350); + setState(359); match(LBRACE); - setState(351); + setState(360); expression(0); - setState(352); + setState(361); match(RBRACE); } } @@ -2410,34 +2461,34 @@ class PainlessParser extends Parser { enterOuterAlt(_localctx, 1); { { - setState(354); - match(LP); setState(363); + match(LP); + setState(372); switch ( getInterpreter().adaptivePredict(_input,29,_ctx) ) { case 1: { - setState(355); + setState(364); argument(); - setState(360); + setState(369); _errHandler.sync(this); _la = _input.LA(1); while (_la==COMMA) { { { - setState(356); + setState(365); match(COMMA); - setState(357); + setState(366); argument(); } } - setState(362); + setState(371); _errHandler.sync(this); _la = _input.LA(1); } } break; } - setState(365); + setState(374); match(RP); } } @@ -2475,19 +2526,19 @@ class PainlessParser extends Parser { ArgumentContext _localctx = new ArgumentContext(_ctx, getState()); enterRule(_localctx, 42, RULE_argument); try { - setState(369); + setState(378); switch ( getInterpreter().adaptivePredict(_input,30,_ctx) ) { case 1: enterOuterAlt(_localctx, 1); { - setState(367); + setState(376); expression(0); } break; case 2: enterOuterAlt(_localctx, 2); { - setState(368); + setState(377); funcref(); } break; @@ -2594,141 +2645,146 @@ class PainlessParser extends Parser { } public static final String _serializedATN = - "\3\u0430\ud6d1\u8206\uad2d\u4417\uaef1\u8d80\uaadd\3M\u0176\4\2\t\2\4"+ + "\3\u0430\ud6d1\u8206\uad2d\u4417\uaef1\u8d80\uaadd\3M\u017f\4\2\t\2\4"+ "\3\t\3\4\4\t\4\4\5\t\5\4\6\t\6\4\7\t\7\4\b\t\b\4\t\t\t\4\n\t\n\4\13\t"+ "\13\4\f\t\f\4\r\t\r\4\16\t\16\4\17\t\17\4\20\t\20\4\21\t\21\4\22\t\22"+ "\4\23\t\23\4\24\t\24\4\25\t\25\4\26\t\26\4\27\t\27\3\2\7\2\60\n\2\f\2"+ "\16\2\63\13\2\3\2\3\2\3\3\3\3\3\3\3\3\3\3\3\3\3\3\3\3\5\3?\n\3\3\3\3\3"+ "\3\3\3\3\3\3\3\3\5\3G\n\3\3\3\3\3\3\3\3\3\3\3\3\3\3\3\3\3\3\3\3\3\3\3"+ "\5\3T\n\3\3\3\3\3\5\3X\n\3\3\3\3\3\5\3\\\n\3\3\3\3\3\3\3\5\3a\n\3\3\3"+ - "\3\3\3\3\3\3\3\3\3\3\3\3\3\3\3\3\3\3\3\3\3\3\3\3\3\3\6\3q\n\3\r\3\16\3"+ - "r\3\3\3\3\3\3\3\3\3\3\3\3\3\3\5\3|\n\3\3\4\3\4\5\4\u0080\n\4\3\5\3\5\7"+ - "\5\u0084\n\5\f\5\16\5\u0087\13\5\3\5\3\5\3\6\3\6\3\7\3\7\5\7\u008f\n\7"+ - "\3\b\3\b\3\t\3\t\3\t\3\t\7\t\u0097\n\t\f\t\16\t\u009a\13\t\3\n\3\n\3\n"+ - "\7\n\u009f\n\n\f\n\16\n\u00a2\13\n\3\13\3\13\3\13\3\13\3\f\3\f\3\f\5\f"+ - "\u00ab\n\f\3\r\3\r\3\r\3\r\3\r\3\r\3\r\3\16\3\16\3\17\3\17\3\17\3\17\3"+ - "\17\3\17\3\17\3\17\3\17\5\17\u00bf\n\17\3\17\3\17\3\17\3\17\3\17\3\17"+ + "\3\3\3\3\3\3\3\3\3\3\3\3\3\3\3\3\3\3\3\3\3\3\3\3\3\3\3\3\3\3\3\3\3\3\3"+ + "\3\3\3\3\3\3\3\3\3\6\3z\n\3\r\3\16\3{\3\3\3\3\3\3\3\3\3\3\3\3\3\3\5\3"+ + "\u0085\n\3\3\4\3\4\5\4\u0089\n\4\3\5\3\5\7\5\u008d\n\5\f\5\16\5\u0090"+ + "\13\5\3\5\3\5\3\6\3\6\3\7\3\7\5\7\u0098\n\7\3\b\3\b\3\t\3\t\3\t\3\t\7"+ + "\t\u00a0\n\t\f\t\16\t\u00a3\13\t\3\n\3\n\3\n\7\n\u00a8\n\n\f\n\16\n\u00ab"+ + "\13\n\3\13\3\13\3\13\3\13\3\f\3\f\3\f\5\f\u00b4\n\f\3\r\3\r\3\r\3\r\3"+ + "\r\3\r\3\r\3\16\3\16\3\17\3\17\3\17\3\17\3\17\3\17\3\17\3\17\3\17\5\17"+ + "\u00c8\n\17\3\17\3\17\3\17\3\17\3\17\3\17\3\17\3\17\3\17\3\17\3\17\3\17"+ "\3\17\3\17\3\17\3\17\3\17\3\17\3\17\3\17\3\17\3\17\3\17\3\17\3\17\3\17"+ "\3\17\3\17\3\17\3\17\3\17\3\17\3\17\3\17\3\17\3\17\3\17\3\17\3\17\3\17"+ "\3\17\3\17\3\17\3\17\3\17\3\17\3\17\3\17\3\17\3\17\3\17\3\17\3\17\3\17"+ - "\3\17\3\17\3\17\3\17\3\17\3\17\3\17\3\17\3\17\7\17\u00fa\n\17\f\17\16"+ - "\17\u00fd\13\17\3\20\3\20\3\20\3\20\3\20\3\20\3\20\3\20\3\20\3\20\3\20"+ + "\3\17\3\17\3\17\7\17\u0103\n\17\f\17\16\17\u0106\13\17\3\20\3\20\3\20"+ "\3\20\3\20\3\20\3\20\3\20\3\20\3\20\3\20\3\20\3\20\3\20\3\20\3\20\3\20"+ - "\3\20\3\20\3\20\3\20\5\20\u011c\n\20\3\21\3\21\7\21\u0120\n\21\f\21\16"+ - "\21\u0123\13\21\3\21\3\21\3\21\7\21\u0128\n\21\f\21\16\21\u012b\13\21"+ - "\3\21\3\21\3\21\3\21\3\21\3\21\6\21\u0133\n\21\r\21\16\21\u0134\3\21\3"+ - "\21\7\21\u0139\n\21\f\21\16\21\u013c\13\21\5\21\u013e\n\21\5\21\u0140"+ - "\n\21\3\22\3\22\3\22\3\22\3\22\3\22\3\22\3\22\3\22\3\22\3\22\3\22\3\22"+ - "\3\22\3\22\3\22\5\22\u0152\n\22\3\23\3\23\3\23\3\23\5\23\u0158\n\23\3"+ - "\24\3\24\3\24\3\24\3\24\5\24\u015f\n\24\3\25\3\25\3\25\3\25\3\26\3\26"+ - "\3\26\3\26\7\26\u0169\n\26\f\26\16\26\u016c\13\26\5\26\u016e\n\26\3\26"+ - "\3\26\3\27\3\27\5\27\u0174\n\27\3\27\2\3\34\30\2\4\6\b\n\f\16\20\22\24"+ - "\26\30\32\34\36 \"$&(*,\2\r\3\3\r\r\3\2\66A\3\2\34\36\3\2\37 \3\2!#\3"+ - "\2$\'\3\2(+\3\2\64\65\3\2BE\4\2\32\33\37 \3\2LM\u019b\2\61\3\2\2\2\4{"+ - "\3\2\2\2\6\177\3\2\2\2\b\u0081\3\2\2\2\n\u008a\3\2\2\2\f\u008e\3\2\2\2"+ - "\16\u0090\3\2\2\2\20\u0092\3\2\2\2\22\u009b\3\2\2\2\24\u00a3\3\2\2\2\26"+ - "\u00a7\3\2\2\2\30\u00ac\3\2\2\2\32\u00b3\3\2\2\2\34\u00be\3\2\2\2\36\u011b"+ - "\3\2\2\2 \u013f\3\2\2\2\"\u0151\3\2\2\2$\u0157\3\2\2\2&\u015e\3\2\2\2"+ - "(\u0160\3\2\2\2*\u0164\3\2\2\2,\u0173\3\2\2\2.\60\5\4\3\2/.\3\2\2\2\60"+ - "\63\3\2\2\2\61/\3\2\2\2\61\62\3\2\2\2\62\64\3\2\2\2\63\61\3\2\2\2\64\65"+ - "\7\2\2\3\65\3\3\2\2\2\66\67\7\16\2\2\678\7\t\2\289\5\34\17\29:\7\n\2\2"+ - ":>\5\6\4\2;<\7\17\2\2;\3\2\2\2>=\3\2\2\2?|\3\2\2"+ - "\2@A\7\20\2\2AB\7\t\2\2BC\5\34\17\2CF\7\n\2\2DG\5\6\4\2EG\5\n\6\2FD\3"+ - "\2\2\2FE\3\2\2\2G|\3\2\2\2HI\7\21\2\2IJ\5\b\5\2JK\7\20\2\2KL\7\t\2\2L"+ - "M\5\34\17\2MN\7\n\2\2NO\5\32\16\2O|\3\2\2\2PQ\7\22\2\2QS\7\t\2\2RT\5\f"+ - "\7\2SR\3\2\2\2ST\3\2\2\2TU\3\2\2\2UW\7\r\2\2VX\5\34\17\2WV\3\2\2\2WX\3"+ - "\2\2\2XY\3\2\2\2Y[\7\r\2\2Z\\\5\16\b\2[Z\3\2\2\2[\\\3\2\2\2\\]\3\2\2\2"+ - "]`\7\n\2\2^a\5\6\4\2_a\5\n\6\2`^\3\2\2\2`_\3\2\2\2a|\3\2\2\2bc\5\20\t"+ - "\2cd\5\32\16\2d|\3\2\2\2ef\7\23\2\2f|\5\32\16\2gh\7\24\2\2h|\5\32\16\2"+ - "ij\7\25\2\2jk\5\34\17\2kl\5\32\16\2l|\3\2\2\2mn\7\27\2\2np\5\b\5\2oq\5"+ - "\30\r\2po\3\2\2\2qr\3\2\2\2rp\3\2\2\2rs\3\2\2\2s|\3\2\2\2tu\7\31\2\2u"+ - "v\5\34\17\2vw\5\32\16\2w|\3\2\2\2xy\5\34\17\2yz\5\32\16\2z|\3\2\2\2{\66"+ - "\3\2\2\2{@\3\2\2\2{H\3\2\2\2{P\3\2\2\2{b\3\2\2\2{e\3\2\2\2{g\3\2\2\2{"+ - "i\3\2\2\2{m\3\2\2\2{t\3\2\2\2{x\3\2\2\2|\5\3\2\2\2}\u0080\5\b\5\2~\u0080"+ - "\5\4\3\2\177}\3\2\2\2\177~\3\2\2\2\u0080\7\3\2\2\2\u0081\u0085\7\5\2\2"+ - "\u0082\u0084\5\4\3\2\u0083\u0082\3\2\2\2\u0084\u0087\3\2\2\2\u0085\u0083"+ - "\3\2\2\2\u0085\u0086\3\2\2\2\u0086\u0088\3\2\2\2\u0087\u0085\3\2\2\2\u0088"+ - "\u0089\7\6\2\2\u0089\t\3\2\2\2\u008a\u008b\7\r\2\2\u008b\13\3\2\2\2\u008c"+ - "\u008f\5\20\t\2\u008d\u008f\5\34\17\2\u008e\u008c\3\2\2\2\u008e\u008d"+ - "\3\2\2\2\u008f\r\3\2\2\2\u0090\u0091\5\34\17\2\u0091\17\3\2\2\2\u0092"+ - "\u0093\5\22\n\2\u0093\u0098\5\26\f\2\u0094\u0095\7\f\2\2\u0095\u0097\5"+ - "\26\f\2\u0096\u0094\3\2\2\2\u0097\u009a\3\2\2\2\u0098\u0096\3\2\2\2\u0098"+ - "\u0099\3\2\2\2\u0099\21\3\2\2\2\u009a\u0098\3\2\2\2\u009b\u00a0\7J\2\2"+ - "\u009c\u009d\7\7\2\2\u009d\u009f\7\b\2\2\u009e\u009c\3\2\2\2\u009f\u00a2"+ - "\3\2\2\2\u00a0\u009e\3\2\2\2\u00a0\u00a1\3\2\2\2\u00a1\23\3\2\2\2\u00a2"+ - "\u00a0\3\2\2\2\u00a3\u00a4\7J\2\2\u00a4\u00a5\7\63\2\2\u00a5\u00a6\7K"+ - "\2\2\u00a6\25\3\2\2\2\u00a7\u00aa\7K\2\2\u00a8\u00a9\7\66\2\2\u00a9\u00ab"+ - "\5\34\17\2\u00aa\u00a8\3\2\2\2\u00aa\u00ab\3\2\2\2\u00ab\27\3\2\2\2\u00ac"+ - "\u00ad\7\30\2\2\u00ad\u00ae\7\t\2\2\u00ae\u00af\7J\2\2\u00af\u00b0\7K"+ - "\2\2\u00b0\u00b1\7\n\2\2\u00b1\u00b2\5\b\5\2\u00b2\31\3\2\2\2\u00b3\u00b4"+ - "\t\2\2\2\u00b4\33\3\2\2\2\u00b5\u00b6\b\17\1\2\u00b6\u00b7\5 \21\2\u00b7"+ - "\u00b8\t\3\2\2\u00b8\u00b9\5\34\17\3\u00b9\u00ba\b\17\1\2\u00ba\u00bf"+ - "\3\2\2\2\u00bb\u00bc\5\36\20\2\u00bc\u00bd\b\17\1\2\u00bd\u00bf\3\2\2"+ - "\2\u00be\u00b5\3\2\2\2\u00be\u00bb\3\2\2\2\u00bf\u00fb\3\2\2\2\u00c0\u00c1"+ - "\f\16\2\2\u00c1\u00c2\t\4\2\2\u00c2\u00c3\5\34\17\17\u00c3\u00c4\b\17"+ - "\1\2\u00c4\u00fa\3\2\2\2\u00c5\u00c6\f\r\2\2\u00c6\u00c7\t\5\2\2\u00c7"+ - "\u00c8\5\34\17\16\u00c8\u00c9\b\17\1\2\u00c9\u00fa\3\2\2\2\u00ca\u00cb"+ - "\f\f\2\2\u00cb\u00cc\t\6\2\2\u00cc\u00cd\5\34\17\r\u00cd\u00ce\b\17\1"+ - "\2\u00ce\u00fa\3\2\2\2\u00cf\u00d0\f\13\2\2\u00d0\u00d1\t\7\2\2\u00d1"+ - "\u00d2\5\34\17\f\u00d2\u00d3\b\17\1\2\u00d3\u00fa\3\2\2\2\u00d4\u00d5"+ - "\f\n\2\2\u00d5\u00d6\t\b\2\2\u00d6\u00d7\5\34\17\13\u00d7\u00d8\b\17\1"+ - "\2\u00d8\u00fa\3\2\2\2\u00d9\u00da\f\t\2\2\u00da\u00db\7,\2\2\u00db\u00dc"+ - "\5\34\17\n\u00dc\u00dd\b\17\1\2\u00dd\u00fa\3\2\2\2\u00de\u00df\f\b\2"+ - "\2\u00df\u00e0\7-\2\2\u00e0\u00e1\5\34\17\t\u00e1\u00e2\b\17\1\2\u00e2"+ - "\u00fa\3\2\2\2\u00e3\u00e4\f\7\2\2\u00e4\u00e5\7.\2\2\u00e5\u00e6\5\34"+ - "\17\b\u00e6\u00e7\b\17\1\2\u00e7\u00fa\3\2\2\2\u00e8\u00e9\f\6\2\2\u00e9"+ - "\u00ea\7/\2\2\u00ea\u00eb\5\34\17\7\u00eb\u00ec\b\17\1\2\u00ec\u00fa\3"+ - "\2\2\2\u00ed\u00ee\f\5\2\2\u00ee\u00ef\7\60\2\2\u00ef\u00f0\5\34\17\6"+ - "\u00f0\u00f1\b\17\1\2\u00f1\u00fa\3\2\2\2\u00f2\u00f3\f\4\2\2\u00f3\u00f4"+ - "\7\61\2\2\u00f4\u00f5\5\34\17\2\u00f5\u00f6\7\62\2\2\u00f6\u00f7\5\34"+ - "\17\4\u00f7\u00f8\b\17\1\2\u00f8\u00fa\3\2\2\2\u00f9\u00c0\3\2\2\2\u00f9"+ - "\u00c5\3\2\2\2\u00f9\u00ca\3\2\2\2\u00f9\u00cf\3\2\2\2\u00f9\u00d4\3\2"+ - "\2\2\u00f9\u00d9\3\2\2\2\u00f9\u00de\3\2\2\2\u00f9\u00e3\3\2\2\2\u00f9"+ - "\u00e8\3\2\2\2\u00f9\u00ed\3\2\2\2\u00f9\u00f2\3\2\2\2\u00fa\u00fd\3\2"+ - "\2\2\u00fb\u00f9\3\2\2\2\u00fb\u00fc\3\2\2\2\u00fc\35\3\2\2\2\u00fd\u00fb"+ - "\3\2\2\2\u00fe\u00ff\6\20\16\3\u00ff\u0100\t\t\2\2\u0100\u011c\5 \21\2"+ - "\u0101\u0102\6\20\17\3\u0102\u0103\5 \21\2\u0103\u0104\t\t\2\2\u0104\u011c"+ - "\3\2\2\2\u0105\u0106\6\20\20\3\u0106\u011c\5 \21\2\u0107\u0108\6\20\21"+ - "\3\u0108\u0109\t\n\2\2\u0109\u011c\b\20\1\2\u010a\u010b\6\20\22\3\u010b"+ - "\u010c\7G\2\2\u010c\u011c\b\20\1\2\u010d\u010e\6\20\23\3\u010e\u010f\7"+ - "H\2\2\u010f\u011c\b\20\1\2\u0110\u0111\6\20\24\3\u0111\u0112\7I\2\2\u0112"+ - "\u011c\b\20\1\2\u0113\u0114\6\20\25\3\u0114\u0115\t\13\2\2\u0115\u011c"+ - "\5\36\20\2\u0116\u0117\7\t\2\2\u0117\u0118\5\22\n\2\u0118\u0119\7\n\2"+ - "\2\u0119\u011a\5\36\20\2\u011a\u011c\3\2\2\2\u011b\u00fe\3\2\2\2\u011b"+ - "\u0101\3\2\2\2\u011b\u0105\3\2\2\2\u011b\u0107\3\2\2\2\u011b\u010a\3\2"+ - "\2\2\u011b\u010d\3\2\2\2\u011b\u0110\3\2\2\2\u011b\u0113\3\2\2\2\u011b"+ - "\u0116\3\2\2\2\u011c\37\3\2\2\2\u011d\u0121\5\"\22\2\u011e\u0120\5$\23"+ - "\2\u011f\u011e\3\2\2\2\u0120\u0123\3\2\2\2\u0121\u011f\3\2\2\2\u0121\u0122"+ - "\3\2\2\2\u0122\u0140\3\2\2\2\u0123\u0121\3\2\2\2\u0124\u0125\5\22\n\2"+ - "\u0125\u0129\5&\24\2\u0126\u0128\5$\23\2\u0127\u0126\3\2\2\2\u0128\u012b"+ - "\3\2\2\2\u0129\u0127\3\2\2\2\u0129\u012a\3\2\2\2\u012a\u0140\3\2\2\2\u012b"+ - "\u0129\3\2\2\2\u012c\u012d\7\26\2\2\u012d\u0132\7J\2\2\u012e\u012f\7\7"+ - "\2\2\u012f\u0130\5\34\17\2\u0130\u0131\7\b\2\2\u0131\u0133\3\2\2\2\u0132"+ - "\u012e\3\2\2\2\u0133\u0134\3\2\2\2\u0134\u0132\3\2\2\2\u0134\u0135\3\2"+ - "\2\2\u0135\u013d\3\2\2\2\u0136\u013a\5&\24\2\u0137\u0139\5$\23\2\u0138"+ - "\u0137\3\2\2\2\u0139\u013c\3\2\2\2\u013a\u0138\3\2\2\2\u013a\u013b\3\2"+ - "\2\2\u013b\u013e\3\2\2\2\u013c\u013a\3\2\2\2\u013d\u0136\3\2\2\2\u013d"+ - "\u013e\3\2\2\2\u013e\u0140\3\2\2\2\u013f\u011d\3\2\2\2\u013f\u0124\3\2"+ - "\2\2\u013f\u012c\3\2\2\2\u0140!\3\2\2\2\u0141\u0142\6\22\26\3\u0142\u0143"+ - "\7\t\2\2\u0143\u0144\5\34\17\2\u0144\u0145\7\n\2\2\u0145\u0146\b\22\1"+ - "\2\u0146\u0152\3\2\2\2\u0147\u0148\6\22\27\3\u0148\u0149\7\t\2\2\u0149"+ - "\u014a\5\36\20\2\u014a\u014b\7\n\2\2\u014b\u0152\3\2\2\2\u014c\u0152\7"+ - "F\2\2\u014d\u0152\7K\2\2\u014e\u014f\7\26\2\2\u014f\u0150\7J\2\2\u0150"+ - "\u0152\5*\26\2\u0151\u0141\3\2\2\2\u0151\u0147\3\2\2\2\u0151\u014c\3\2"+ - "\2\2\u0151\u014d\3\2\2\2\u0151\u014e\3\2\2\2\u0152#\3\2\2\2\u0153\u0154"+ - "\6\23\30\3\u0154\u0158\5&\24\2\u0155\u0156\6\23\31\3\u0156\u0158\5(\25"+ - "\2\u0157\u0153\3\2\2\2\u0157\u0155\3\2\2\2\u0158%\3\2\2\2\u0159\u015a"+ - "\7\13\2\2\u015a\u015b\7M\2\2\u015b\u015f\5*\26\2\u015c\u015d\7\13\2\2"+ - "\u015d\u015f\t\f\2\2\u015e\u0159\3\2\2\2\u015e\u015c\3\2\2\2\u015f\'\3"+ - "\2\2\2\u0160\u0161\7\7\2\2\u0161\u0162\5\34\17\2\u0162\u0163\7\b\2\2\u0163"+ - ")\3\2\2\2\u0164\u016d\7\t\2\2\u0165\u016a\5,\27\2\u0166\u0167\7\f\2\2"+ - "\u0167\u0169\5,\27\2\u0168\u0166\3\2\2\2\u0169\u016c\3\2\2\2\u016a\u0168"+ - "\3\2\2\2\u016a\u016b\3\2\2\2\u016b\u016e\3\2\2\2\u016c\u016a\3\2\2\2\u016d"+ - "\u0165\3\2\2\2\u016d\u016e\3\2\2\2\u016e\u016f\3\2\2\2\u016f\u0170\7\n"+ - "\2\2\u0170+\3\2\2\2\u0171\u0174\5\34\17\2\u0172\u0174\5\24\13\2\u0173"+ - "\u0171\3\2\2\2\u0173\u0172\3\2\2\2\u0174-\3\2\2\2!\61>FSW[`r{\177\u0085"+ - "\u008e\u0098\u00a0\u00aa\u00be\u00f9\u00fb\u011b\u0121\u0129\u0134\u013a"+ - "\u013d\u013f\u0151\u0157\u015e\u016a\u016d\u0173"; + "\3\20\3\20\3\20\3\20\3\20\3\20\3\20\3\20\3\20\3\20\3\20\3\20\5\20\u0125"+ + "\n\20\3\21\3\21\7\21\u0129\n\21\f\21\16\21\u012c\13\21\3\21\3\21\3\21"+ + "\7\21\u0131\n\21\f\21\16\21\u0134\13\21\3\21\3\21\3\21\3\21\3\21\3\21"+ + "\6\21\u013c\n\21\r\21\16\21\u013d\3\21\3\21\7\21\u0142\n\21\f\21\16\21"+ + "\u0145\13\21\5\21\u0147\n\21\5\21\u0149\n\21\3\22\3\22\3\22\3\22\3\22"+ + "\3\22\3\22\3\22\3\22\3\22\3\22\3\22\3\22\3\22\3\22\3\22\5\22\u015b\n\22"+ + "\3\23\3\23\3\23\3\23\5\23\u0161\n\23\3\24\3\24\3\24\3\24\3\24\5\24\u0168"+ + "\n\24\3\25\3\25\3\25\3\25\3\26\3\26\3\26\3\26\7\26\u0172\n\26\f\26\16"+ + "\26\u0175\13\26\5\26\u0177\n\26\3\26\3\26\3\27\3\27\5\27\u017d\n\27\3"+ + "\27\2\3\34\30\2\4\6\b\n\f\16\20\22\24\26\30\32\34\36 \"$&(*,\2\16\4\2"+ + "\26\26KK\3\3\r\r\3\2\66A\3\2\34\36\3\2\37 \3\2!#\3\2$\'\3\2(+\3\2\64\65"+ + "\3\2BE\4\2\32\33\37 \3\2LM\u01a5\2\61\3\2\2\2\4\u0084\3\2\2\2\6\u0088"+ + "\3\2\2\2\b\u008a\3\2\2\2\n\u0093\3\2\2\2\f\u0097\3\2\2\2\16\u0099\3\2"+ + "\2\2\20\u009b\3\2\2\2\22\u00a4\3\2\2\2\24\u00ac\3\2\2\2\26\u00b0\3\2\2"+ + "\2\30\u00b5\3\2\2\2\32\u00bc\3\2\2\2\34\u00c7\3\2\2\2\36\u0124\3\2\2\2"+ + " \u0148\3\2\2\2\"\u015a\3\2\2\2$\u0160\3\2\2\2&\u0167\3\2\2\2(\u0169\3"+ + "\2\2\2*\u016d\3\2\2\2,\u017c\3\2\2\2.\60\5\4\3\2/.\3\2\2\2\60\63\3\2\2"+ + "\2\61/\3\2\2\2\61\62\3\2\2\2\62\64\3\2\2\2\63\61\3\2\2\2\64\65\7\2\2\3"+ + "\65\3\3\2\2\2\66\67\7\16\2\2\678\7\t\2\289\5\34\17\29:\7\n\2\2:>\5\6\4"+ + "\2;<\7\17\2\2;\3\2\2\2>=\3\2\2\2?\u0085\3\2\2\2@"+ + "A\7\20\2\2AB\7\t\2\2BC\5\34\17\2CF\7\n\2\2DG\5\6\4\2EG\5\n\6\2FD\3\2\2"+ + "\2FE\3\2\2\2G\u0085\3\2\2\2HI\7\21\2\2IJ\5\b\5\2JK\7\20\2\2KL\7\t\2\2"+ + "LM\5\34\17\2MN\7\n\2\2NO\5\32\16\2O\u0085\3\2\2\2PQ\7\22\2\2QS\7\t\2\2"+ + "RT\5\f\7\2SR\3\2\2\2ST\3\2\2\2TU\3\2\2\2UW\7\r\2\2VX\5\34\17\2WV\3\2\2"+ + "\2WX\3\2\2\2XY\3\2\2\2Y[\7\r\2\2Z\\\5\16\b\2[Z\3\2\2\2[\\\3\2\2\2\\]\3"+ + "\2\2\2]`\7\n\2\2^a\5\6\4\2_a\5\n\6\2`^\3\2\2\2`_\3\2\2\2a\u0085\3\2\2"+ + "\2bc\7\22\2\2cd\7\t\2\2de\5\22\n\2ef\7K\2\2fg\7\62\2\2gh\5\34\17\2hi\7"+ + "\n\2\2ij\5\6\4\2j\u0085\3\2\2\2kl\5\20\t\2lm\5\32\16\2m\u0085\3\2\2\2"+ + "no\7\23\2\2o\u0085\5\32\16\2pq\7\24\2\2q\u0085\5\32\16\2rs\7\25\2\2st"+ + "\5\34\17\2tu\5\32\16\2u\u0085\3\2\2\2vw\7\27\2\2wy\5\b\5\2xz\5\30\r\2"+ + "yx\3\2\2\2z{\3\2\2\2{y\3\2\2\2{|\3\2\2\2|\u0085\3\2\2\2}~\7\31\2\2~\177"+ + "\5\34\17\2\177\u0080\5\32\16\2\u0080\u0085\3\2\2\2\u0081\u0082\5\34\17"+ + "\2\u0082\u0083\5\32\16\2\u0083\u0085\3\2\2\2\u0084\66\3\2\2\2\u0084@\3"+ + "\2\2\2\u0084H\3\2\2\2\u0084P\3\2\2\2\u0084b\3\2\2\2\u0084k\3\2\2\2\u0084"+ + "n\3\2\2\2\u0084p\3\2\2\2\u0084r\3\2\2\2\u0084v\3\2\2\2\u0084}\3\2\2\2"+ + "\u0084\u0081\3\2\2\2\u0085\5\3\2\2\2\u0086\u0089\5\b\5\2\u0087\u0089\5"+ + "\4\3\2\u0088\u0086\3\2\2\2\u0088\u0087\3\2\2\2\u0089\7\3\2\2\2\u008a\u008e"+ + "\7\5\2\2\u008b\u008d\5\4\3\2\u008c\u008b\3\2\2\2\u008d\u0090\3\2\2\2\u008e"+ + "\u008c\3\2\2\2\u008e\u008f\3\2\2\2\u008f\u0091\3\2\2\2\u0090\u008e\3\2"+ + "\2\2\u0091\u0092\7\6\2\2\u0092\t\3\2\2\2\u0093\u0094\7\r\2\2\u0094\13"+ + "\3\2\2\2\u0095\u0098\5\20\t\2\u0096\u0098\5\34\17\2\u0097\u0095\3\2\2"+ + "\2\u0097\u0096\3\2\2\2\u0098\r\3\2\2\2\u0099\u009a\5\34\17\2\u009a\17"+ + "\3\2\2\2\u009b\u009c\5\22\n\2\u009c\u00a1\5\26\f\2\u009d\u009e\7\f\2\2"+ + "\u009e\u00a0\5\26\f\2\u009f\u009d\3\2\2\2\u00a0\u00a3\3\2\2\2\u00a1\u009f"+ + "\3\2\2\2\u00a1\u00a2\3\2\2\2\u00a2\21\3\2\2\2\u00a3\u00a1\3\2\2\2\u00a4"+ + "\u00a9\7J\2\2\u00a5\u00a6\7\7\2\2\u00a6\u00a8\7\b\2\2\u00a7\u00a5\3\2"+ + "\2\2\u00a8\u00ab\3\2\2\2\u00a9\u00a7\3\2\2\2\u00a9\u00aa\3\2\2\2\u00aa"+ + "\23\3\2\2\2\u00ab\u00a9\3\2\2\2\u00ac\u00ad\7J\2\2\u00ad\u00ae\7\63\2"+ + "\2\u00ae\u00af\t\2\2\2\u00af\25\3\2\2\2\u00b0\u00b3\7K\2\2\u00b1\u00b2"+ + "\7\66\2\2\u00b2\u00b4\5\34\17\2\u00b3\u00b1\3\2\2\2\u00b3\u00b4\3\2\2"+ + "\2\u00b4\27\3\2\2\2\u00b5\u00b6\7\30\2\2\u00b6\u00b7\7\t\2\2\u00b7\u00b8"+ + "\7J\2\2\u00b8\u00b9\7K\2\2\u00b9\u00ba\7\n\2\2\u00ba\u00bb\5\b\5\2\u00bb"+ + "\31\3\2\2\2\u00bc\u00bd\t\3\2\2\u00bd\33\3\2\2\2\u00be\u00bf\b\17\1\2"+ + "\u00bf\u00c0\5 \21\2\u00c0\u00c1\t\4\2\2\u00c1\u00c2\5\34\17\3\u00c2\u00c3"+ + "\b\17\1\2\u00c3\u00c8\3\2\2\2\u00c4\u00c5\5\36\20\2\u00c5\u00c6\b\17\1"+ + "\2\u00c6\u00c8\3\2\2\2\u00c7\u00be\3\2\2\2\u00c7\u00c4\3\2\2\2\u00c8\u0104"+ + "\3\2\2\2\u00c9\u00ca\f\16\2\2\u00ca\u00cb\t\5\2\2\u00cb\u00cc\5\34\17"+ + "\17\u00cc\u00cd\b\17\1\2\u00cd\u0103\3\2\2\2\u00ce\u00cf\f\r\2\2\u00cf"+ + "\u00d0\t\6\2\2\u00d0\u00d1\5\34\17\16\u00d1\u00d2\b\17\1\2\u00d2\u0103"+ + "\3\2\2\2\u00d3\u00d4\f\f\2\2\u00d4\u00d5\t\7\2\2\u00d5\u00d6\5\34\17\r"+ + "\u00d6\u00d7\b\17\1\2\u00d7\u0103\3\2\2\2\u00d8\u00d9\f\13\2\2\u00d9\u00da"+ + "\t\b\2\2\u00da\u00db\5\34\17\f\u00db\u00dc\b\17\1\2\u00dc\u0103\3\2\2"+ + "\2\u00dd\u00de\f\n\2\2\u00de\u00df\t\t\2\2\u00df\u00e0\5\34\17\13\u00e0"+ + "\u00e1\b\17\1\2\u00e1\u0103\3\2\2\2\u00e2\u00e3\f\t\2\2\u00e3\u00e4\7"+ + ",\2\2\u00e4\u00e5\5\34\17\n\u00e5\u00e6\b\17\1\2\u00e6\u0103\3\2\2\2\u00e7"+ + "\u00e8\f\b\2\2\u00e8\u00e9\7-\2\2\u00e9\u00ea\5\34\17\t\u00ea\u00eb\b"+ + "\17\1\2\u00eb\u0103\3\2\2\2\u00ec\u00ed\f\7\2\2\u00ed\u00ee\7.\2\2\u00ee"+ + "\u00ef\5\34\17\b\u00ef\u00f0\b\17\1\2\u00f0\u0103\3\2\2\2\u00f1\u00f2"+ + "\f\6\2\2\u00f2\u00f3\7/\2\2\u00f3\u00f4\5\34\17\7\u00f4\u00f5\b\17\1\2"+ + "\u00f5\u0103\3\2\2\2\u00f6\u00f7\f\5\2\2\u00f7\u00f8\7\60\2\2\u00f8\u00f9"+ + "\5\34\17\6\u00f9\u00fa\b\17\1\2\u00fa\u0103\3\2\2\2\u00fb\u00fc\f\4\2"+ + "\2\u00fc\u00fd\7\61\2\2\u00fd\u00fe\5\34\17\2\u00fe\u00ff\7\62\2\2\u00ff"+ + "\u0100\5\34\17\4\u0100\u0101\b\17\1\2\u0101\u0103\3\2\2\2\u0102\u00c9"+ + "\3\2\2\2\u0102\u00ce\3\2\2\2\u0102\u00d3\3\2\2\2\u0102\u00d8\3\2\2\2\u0102"+ + "\u00dd\3\2\2\2\u0102\u00e2\3\2\2\2\u0102\u00e7\3\2\2\2\u0102\u00ec\3\2"+ + "\2\2\u0102\u00f1\3\2\2\2\u0102\u00f6\3\2\2\2\u0102\u00fb\3\2\2\2\u0103"+ + "\u0106\3\2\2\2\u0104\u0102\3\2\2\2\u0104\u0105\3\2\2\2\u0105\35\3\2\2"+ + "\2\u0106\u0104\3\2\2\2\u0107\u0108\6\20\16\3\u0108\u0109\t\n\2\2\u0109"+ + "\u0125\5 \21\2\u010a\u010b\6\20\17\3\u010b\u010c\5 \21\2\u010c\u010d\t"+ + "\n\2\2\u010d\u0125\3\2\2\2\u010e\u010f\6\20\20\3\u010f\u0125\5 \21\2\u0110"+ + "\u0111\6\20\21\3\u0111\u0112\t\13\2\2\u0112\u0125\b\20\1\2\u0113\u0114"+ + "\6\20\22\3\u0114\u0115\7G\2\2\u0115\u0125\b\20\1\2\u0116\u0117\6\20\23"+ + "\3\u0117\u0118\7H\2\2\u0118\u0125\b\20\1\2\u0119\u011a\6\20\24\3\u011a"+ + "\u011b\7I\2\2\u011b\u0125\b\20\1\2\u011c\u011d\6\20\25\3\u011d\u011e\t"+ + "\f\2\2\u011e\u0125\5\36\20\2\u011f\u0120\7\t\2\2\u0120\u0121\5\22\n\2"+ + "\u0121\u0122\7\n\2\2\u0122\u0123\5\36\20\2\u0123\u0125\3\2\2\2\u0124\u0107"+ + "\3\2\2\2\u0124\u010a\3\2\2\2\u0124\u010e\3\2\2\2\u0124\u0110\3\2\2\2\u0124"+ + "\u0113\3\2\2\2\u0124\u0116\3\2\2\2\u0124\u0119\3\2\2\2\u0124\u011c\3\2"+ + "\2\2\u0124\u011f\3\2\2\2\u0125\37\3\2\2\2\u0126\u012a\5\"\22\2\u0127\u0129"+ + "\5$\23\2\u0128\u0127\3\2\2\2\u0129\u012c\3\2\2\2\u012a\u0128\3\2\2\2\u012a"+ + "\u012b\3\2\2\2\u012b\u0149\3\2\2\2\u012c\u012a\3\2\2\2\u012d\u012e\5\22"+ + "\n\2\u012e\u0132\5&\24\2\u012f\u0131\5$\23\2\u0130\u012f\3\2\2\2\u0131"+ + "\u0134\3\2\2\2\u0132\u0130\3\2\2\2\u0132\u0133\3\2\2\2\u0133\u0149\3\2"+ + "\2\2\u0134\u0132\3\2\2\2\u0135\u0136\7\26\2\2\u0136\u013b\7J\2\2\u0137"+ + "\u0138\7\7\2\2\u0138\u0139\5\34\17\2\u0139\u013a\7\b\2\2\u013a\u013c\3"+ + "\2\2\2\u013b\u0137\3\2\2\2\u013c\u013d\3\2\2\2\u013d\u013b\3\2\2\2\u013d"+ + "\u013e\3\2\2\2\u013e\u0146\3\2\2\2\u013f\u0143\5&\24\2\u0140\u0142\5$"+ + "\23\2\u0141\u0140\3\2\2\2\u0142\u0145\3\2\2\2\u0143\u0141\3\2\2\2\u0143"+ + "\u0144\3\2\2\2\u0144\u0147\3\2\2\2\u0145\u0143\3\2\2\2\u0146\u013f\3\2"+ + "\2\2\u0146\u0147\3\2\2\2\u0147\u0149\3\2\2\2\u0148\u0126\3\2\2\2\u0148"+ + "\u012d\3\2\2\2\u0148\u0135\3\2\2\2\u0149!\3\2\2\2\u014a\u014b\6\22\26"+ + "\3\u014b\u014c\7\t\2\2\u014c\u014d\5\34\17\2\u014d\u014e\7\n\2\2\u014e"+ + "\u014f\b\22\1\2\u014f\u015b\3\2\2\2\u0150\u0151\6\22\27\3\u0151\u0152"+ + "\7\t\2\2\u0152\u0153\5\36\20\2\u0153\u0154\7\n\2\2\u0154\u015b\3\2\2\2"+ + "\u0155\u015b\7F\2\2\u0156\u015b\7K\2\2\u0157\u0158\7\26\2\2\u0158\u0159"+ + "\7J\2\2\u0159\u015b\5*\26\2\u015a\u014a\3\2\2\2\u015a\u0150\3\2\2\2\u015a"+ + "\u0155\3\2\2\2\u015a\u0156\3\2\2\2\u015a\u0157\3\2\2\2\u015b#\3\2\2\2"+ + "\u015c\u015d\6\23\30\3\u015d\u0161\5&\24\2\u015e\u015f\6\23\31\3\u015f"+ + "\u0161\5(\25\2\u0160\u015c\3\2\2\2\u0160\u015e\3\2\2\2\u0161%\3\2\2\2"+ + "\u0162\u0163\7\13\2\2\u0163\u0164\7M\2\2\u0164\u0168\5*\26\2\u0165\u0166"+ + "\7\13\2\2\u0166\u0168\t\r\2\2\u0167\u0162\3\2\2\2\u0167\u0165\3\2\2\2"+ + "\u0168\'\3\2\2\2\u0169\u016a\7\7\2\2\u016a\u016b\5\34\17\2\u016b\u016c"+ + "\7\b\2\2\u016c)\3\2\2\2\u016d\u0176\7\t\2\2\u016e\u0173\5,\27\2\u016f"+ + "\u0170\7\f\2\2\u0170\u0172\5,\27\2\u0171\u016f\3\2\2\2\u0172\u0175\3\2"+ + "\2\2\u0173\u0171\3\2\2\2\u0173\u0174\3\2\2\2\u0174\u0177\3\2\2\2\u0175"+ + "\u0173\3\2\2\2\u0176\u016e\3\2\2\2\u0176\u0177\3\2\2\2\u0177\u0178\3\2"+ + "\2\2\u0178\u0179\7\n\2\2\u0179+\3\2\2\2\u017a\u017d\5\34\17\2\u017b\u017d"+ + "\5\24\13\2\u017c\u017a\3\2\2\2\u017c\u017b\3\2\2\2\u017d-\3\2\2\2!\61"+ + ">FSW[`{\u0084\u0088\u008e\u0097\u00a1\u00a9\u00b3\u00c7\u0102\u0104\u0124"+ + "\u012a\u0132\u013d\u0143\u0146\u0148\u015a\u0160\u0167\u0173\u0176\u017c"; public static final ATN _ATN = new ATNDeserializer().deserialize(_serializedATN.toCharArray()); static { diff --git a/modules/lang-painless/src/main/java/org/elasticsearch/painless/antlr/PainlessParserBaseVisitor.java b/modules/lang-painless/src/main/java/org/elasticsearch/painless/antlr/PainlessParserBaseVisitor.java index b4429fc0841..f116f087c5c 100644 --- a/modules/lang-painless/src/main/java/org/elasticsearch/painless/antlr/PainlessParserBaseVisitor.java +++ b/modules/lang-painless/src/main/java/org/elasticsearch/painless/antlr/PainlessParserBaseVisitor.java @@ -46,6 +46,13 @@ class PainlessParserBaseVisitor extends AbstractParseTreeVisitor implement * {@link #visitChildren} on {@code ctx}.

    */ @Override public T visitFor(PainlessParser.ForContext ctx) { return visitChildren(ctx); } + /** + * {@inheritDoc} + * + *

    The default implementation returns the result of calling + * {@link #visitChildren} on {@code ctx}.

    + */ + @Override public T visitEach(PainlessParser.EachContext ctx) { return visitChildren(ctx); } /** * {@inheritDoc} * diff --git a/modules/lang-painless/src/main/java/org/elasticsearch/painless/antlr/PainlessParserVisitor.java b/modules/lang-painless/src/main/java/org/elasticsearch/painless/antlr/PainlessParserVisitor.java index 6dc511cb4e9..f0943743ef8 100644 --- a/modules/lang-painless/src/main/java/org/elasticsearch/painless/antlr/PainlessParserVisitor.java +++ b/modules/lang-painless/src/main/java/org/elasticsearch/painless/antlr/PainlessParserVisitor.java @@ -44,6 +44,13 @@ interface PainlessParserVisitor extends ParseTreeVisitor { * @return the visitor result */ T visitFor(PainlessParser.ForContext ctx); + /** + * Visit a parse tree produced by the {@code each} + * labeled alternative in {@link PainlessParser#statement}. + * @param ctx the parse tree + * @return the visitor result + */ + T visitEach(PainlessParser.EachContext ctx); /** * Visit a parse tree produced by the {@code decl} * labeled alternative in {@link PainlessParser#statement}. diff --git a/modules/lang-painless/src/main/java/org/elasticsearch/painless/antlr/Walker.java b/modules/lang-painless/src/main/java/org/elasticsearch/painless/antlr/Walker.java index 0f155ceb4c1..471cb7d64ea 100644 --- a/modules/lang-painless/src/main/java/org/elasticsearch/painless/antlr/Walker.java +++ b/modules/lang-painless/src/main/java/org/elasticsearch/painless/antlr/Walker.java @@ -53,6 +53,7 @@ import org.elasticsearch.painless.antlr.PainlessParser.DeclvarContext; import org.elasticsearch.painless.antlr.PainlessParser.DelimiterContext; import org.elasticsearch.painless.antlr.PainlessParser.DoContext; import org.elasticsearch.painless.antlr.PainlessParser.DynamicContext; +import org.elasticsearch.painless.antlr.PainlessParser.EachContext; import org.elasticsearch.painless.antlr.PainlessParser.EmptyContext; import org.elasticsearch.painless.antlr.PainlessParser.ExprContext; import org.elasticsearch.painless.antlr.PainlessParser.ExpressionContext; @@ -118,6 +119,7 @@ import org.elasticsearch.painless.node.SContinue; import org.elasticsearch.painless.node.SDeclBlock; import org.elasticsearch.painless.node.SDeclaration; import org.elasticsearch.painless.node.SDo; +import org.elasticsearch.painless.node.SEach; import org.elasticsearch.painless.node.SExpression; import org.elasticsearch.painless.node.SFor; import org.elasticsearch.painless.node.SIf; @@ -261,16 +263,28 @@ public final class Walker extends PainlessParserBaseVisitor { if (ctx.trailer() != null) { SBlock block = (SBlock)visit(ctx.trailer()); - return new SFor(location(ctx), - settings.getMaxLoopCounter(), initializer, expression, afterthought, block); + return new SFor(location(ctx), settings.getMaxLoopCounter(), initializer, expression, afterthought, block); } else if (ctx.empty() != null) { - return new SFor(location(ctx), - settings.getMaxLoopCounter(), initializer, expression, afterthought, null); + return new SFor(location(ctx), settings.getMaxLoopCounter(), initializer, expression, afterthought, null); } else { throw location(ctx).createError(new IllegalStateException("Illegal tree structure.")); } } + @Override + public Object visitEach(EachContext ctx) { + if (settings.getMaxLoopCounter() > 0) { + reserved.usesLoop(); + } + + String type = ctx.decltype().getText(); + String name = ctx.ID().getText(); + AExpression expression = (AExpression)visitExpression(ctx.expression()); + SBlock block = (SBlock)visit(ctx.trailer()); + + return new SEach(location(ctx), settings.getMaxLoopCounter(), type, name, expression, block); + } + @Override public Object visitDecl(DeclContext ctx) { return visit(ctx.declaration()); @@ -391,7 +405,15 @@ public final class Walker extends PainlessParserBaseVisitor { @Override public Object visitFuncref(FuncrefContext ctx) { - return new EFunctionRef(location(ctx), ctx.TYPE().getText(), ctx.ID().getText()); + final String methodText; + if (ctx.ID() != null) { + methodText = ctx.ID().getText(); + } else if (ctx.NEW() != null ){ + methodText = ctx.NEW().getText(); + } else { + throw location(ctx).createError(new IllegalStateException("Illegal tree structure.")); + } + return new EFunctionRef(location(ctx), ctx.TYPE().getText(), methodText); } @Override diff --git a/modules/lang-painless/src/main/java/org/elasticsearch/painless/node/AExpression.java b/modules/lang-painless/src/main/java/org/elasticsearch/painless/node/AExpression.java index 0f3cae45a36..c5494b978d8 100644 --- a/modules/lang-painless/src/main/java/org/elasticsearch/painless/node/AExpression.java +++ b/modules/lang-painless/src/main/java/org/elasticsearch/painless/node/AExpression.java @@ -123,9 +123,19 @@ public abstract class AExpression extends ANode { if (cast == null) { if (constant == null || this instanceof EConstant) { + // For the case where a cast is not required and a constant is not set + // or the node is already an EConstant no changes are required to the tree. + return this; } else { - final EConstant econstant = new EConstant(location, constant); + // For the case where a cast is not required but a + // constant is set, an EConstant replaces this node + // with the constant copied from this node. Note that + // for constants output data does not need to be copied + // from this node because the output data for the EConstant + // will already be the same. + + EConstant econstant = new EConstant(location, constant); econstant.analyze(variables); if (!expected.equals(econstant.actual)) { @@ -136,7 +146,12 @@ public abstract class AExpression extends ANode { } } else { if (constant == null) { - final ECast ecast = new ECast(location, this, cast); + // For the case where a cast is required and a constant is not set. + // Modify the tree to add an ECast between this node and its parent. + // The output data from this node is copied to the ECast for + // further reads done by the parent. + + ECast ecast = new ECast(location, this, cast); ecast.statement = statement; ecast.actual = expected; ecast.isNull = isNull; @@ -144,9 +159,17 @@ public abstract class AExpression extends ANode { return ecast; } else { if (expected.sort.constant) { + // For the case where a cast is required, a constant is set, + // and the constant can be immediately cast to the expected type. + // An EConstant replaces this node with the constant cast appropriately + // from the constant value defined by this node. Note that + // for constants output data does not need to be copied + // from this node because the output data for the EConstant + // will already be the same. + constant = AnalyzerCaster.constCast(location, constant, cast); - final EConstant econstant = new EConstant(location, constant); + EConstant econstant = new EConstant(location, constant); econstant.analyze(variables); if (!expected.equals(econstant.actual)) { @@ -155,19 +178,36 @@ public abstract class AExpression extends ANode { return econstant; } else if (this instanceof EConstant) { - final ECast ecast = new ECast(location, this, cast); + // For the case where a cast is required, a constant is set, + // the constant cannot be immediately cast to the expected type, + // and this node is already an EConstant. Modify the tree to add + // an ECast between this node and its parent. Note that + // for constants output data does not need to be copied + // from this node because the output data for the EConstant + // will already be the same. + + ECast ecast = new ECast(location, this, cast); ecast.actual = expected; return ecast; } else { - final EConstant econstant = new EConstant(location, constant); + // For the case where a cast is required, a constant is set, + // the constant cannot be immediately cast to the expected type, + // and this node is not an EConstant. Replace this node with + // an Econstant node copying the constant from this node. + // Modify the tree to add an ECast between the EConstant node + // and its parent. Note that for constants output data does not + // need to be copied from this node because the output data for + // the EConstant will already be the same. + + EConstant econstant = new EConstant(location, constant); econstant.analyze(variables); if (!actual.equals(econstant.actual)) { throw createError(new IllegalStateException("Illegal tree structure.")); } - final ECast ecast = new ECast(location, econstant, cast); + ECast ecast = new ECast(location, econstant, cast); ecast.actual = expected; return ecast; diff --git a/modules/lang-painless/src/main/java/org/elasticsearch/painless/node/EBinary.java b/modules/lang-painless/src/main/java/org/elasticsearch/painless/node/EBinary.java index 1c7da092d55..32a6e2382f2 100644 --- a/modules/lang-painless/src/main/java/org/elasticsearch/painless/node/EBinary.java +++ b/modules/lang-painless/src/main/java/org/elasticsearch/painless/node/EBinary.java @@ -480,6 +480,7 @@ public final class EBinary extends AExpression { @Override void write(MethodWriter writer) { writer.writeDebugInfo(location); + if (actual.sort == Sort.STRING && operation == Operation.ADD) { if (!cat) { writer.writeNewStrings(); diff --git a/modules/lang-painless/src/main/java/org/elasticsearch/painless/node/EChain.java b/modules/lang-painless/src/main/java/org/elasticsearch/painless/node/EChain.java index dfb1106616c..9b2968db565 100644 --- a/modules/lang-painless/src/main/java/org/elasticsearch/painless/node/EChain.java +++ b/modules/lang-painless/src/main/java/org/elasticsearch/painless/node/EChain.java @@ -247,74 +247,115 @@ public final class EChain extends AExpression { actual = last.after; } + /** + * Handles writing byte code for variable/method chains for all given possibilities + * including String concatenation, compound assignment, regular assignment, and simple + * reads. Includes proper duplication for chained assignments and assignments that are + * also read from. + * + * Example given 'x[0] += 5;' where x is an array of shorts and x[0] is 1. + * Note this example has two links -- x (LVariable) and [0] (LBrace). + * The following steps occur: + * 1. call link{x}.write(...) -- no op [...] + * 2. call link{x}.load(...) -- loads the address of the x array onto the stack [..., address(x)] + * 3. call writer.dup(...) -- dup's the address of the x array onto the stack for later use with store [..., address(x), address(x)] + * 4. call link{[0]}.write(...) -- load the array index value of the constant int 0 onto the stack [..., address(x), address(x), int(0)] + * 5. call link{[0]}.load(...) -- load the short value from x[0] onto the stack [..., address(x), short(1)] + * 6. call writer.writeCast(there) -- casts the short on the stack to an int so it can be added with the rhs [..., address(x), int(1)] + * 7. call expression.write(...) -- puts the expression's value of the constant int 5 onto the stack [..., address(x), int(1), int(5)] + * 8. call writer.writeBinaryInstruction(operation) -- writes the int addition instruction [..., address(x), int(6)] + * 9. call writer.writeCast(back) -- convert the value on the stack back into a short [..., address(x), short(6)] + * 10. call link{[0]}.store(...) -- store the value on the stack into the 0th index of the array x [...] + */ @Override void write(MethodWriter writer) { - // can cause class cast exception among other things at runtime writer.writeDebugInfo(location); + // For the case where the chain represents a String concatenation + // we must, depending on the Java version, write a StringBuilder or + // track types going onto the stack. This must be done before the + // links in the chain are read because we need the StringBuilder to + // be placed on the stack ahead of any potential concatenation arguments. if (cat) { writer.writeNewStrings(); } ALink last = links.get(links.size() - 1); + // Go through all the links in the chain first calling write + // and then load, except for the final link which may be a store. + // See individual links for more information on what each of the + // write, load, and store methods do. for (ALink link : links) { - link.write(writer); + link.write(writer); // call the write method on the link to prepare for a load/store operation if (link == last && link.store) { if (cat) { - writer.writeDup(link.size, 1); - link.load(writer); - writer.writeAppendStrings(link.after); + // Handle the case where we are doing a compound assignment + // representing a String concatenation. - expression.write(writer); + writer.writeDup(link.size, 1); // dup the StringBuilder + link.load(writer); // read the current link's value + writer.writeAppendStrings(link.after); // append the link's value using the StringBuilder + + expression.write(writer); // write the bytecode for the rhs expression if (!(expression instanceof EBinary) || ((EBinary)expression).operation != Operation.ADD || expression.actual.sort != Sort.STRING) { - writer.writeAppendStrings(expression.actual); + writer.writeAppendStrings(expression.actual); // append the expression's value unless it's also a concatenation } - writer.writeToStrings(); - writer.writeCast(back); + writer.writeToStrings(); // put the value of the StringBuilder on the stack + writer.writeCast(back); // if necessary, cast the String to the lhs actual type if (link.load) { - writer.writeDup(link.after.sort.size, link.size); + writer.writeDup(link.after.sort.size, link.size); // if this link is also read from dup the value onto the stack } - link.store(writer); + link.store(writer); // store the link's value from the stack in its respective variable/field/array } else if (operation != null) { - writer.writeDup(link.size, 0); - link.load(writer); + // Handle the case where we are doing a compound assignment that + // does not represent a String concatenation. + + writer.writeDup(link.size, 0); // if necessary, dup the previous link's value to be both loaded from and stored to + link.load(writer); // load the current link's value if (link.load && post) { - writer.writeDup(link.after.sort.size, link.size); + writer.writeDup(link.after.sort.size, link.size); // dup the value if the link is also + // read from and is a post increment } - writer.writeCast(there); - expression.write(writer); - writer.writeBinaryInstruction(location, promote, operation); + writer.writeCast(there); // if necessary cast the current link's value + // to the promotion type between the lhs and rhs types + expression.write(writer); // write the bytecode for the rhs expression + writer.writeBinaryInstruction(location, promote, operation); // write the operation instruction for compound assignment - writer.writeCast(back); + writer.writeCast(back); // if necessary cast the promotion type value back to the link's type if (link.load && !post) { - writer.writeDup(link.after.sort.size, link.size); + writer.writeDup(link.after.sort.size, link.size); // dup the value if the link is also + // read from and is not a post increment } - link.store(writer); + link.store(writer); // store the link's value from the stack in its respective variable/field/array } else { - expression.write(writer); + // Handle the case for a simple write. + + expression.write(writer); // write the bytecode for the rhs expression if (link.load) { - writer.writeDup(link.after.sort.size, link.size); + writer.writeDup(link.after.sort.size, link.size); // dup the value if the link is also read from } - link.store(writer); + link.store(writer); // store the link's value from the stack in its respective variable/field/array } } else { - link.load(writer); + // Handle the case for a simple read. + + link.load(writer); // read the link's value onto the stack } } - writer.writeBranch(tru, fals); + writer.writeBranch(tru, fals); // if this is a branch node, write the bytecode to make an appropiate jump } } diff --git a/modules/lang-painless/src/main/java/org/elasticsearch/painless/node/EComp.java b/modules/lang-painless/src/main/java/org/elasticsearch/painless/node/EComp.java index c122dba877a..407b59f92cf 100644 --- a/modules/lang-painless/src/main/java/org/elasticsearch/painless/node/EComp.java +++ b/modules/lang-painless/src/main/java/org/elasticsearch/painless/node/EComp.java @@ -401,6 +401,7 @@ public final class EComp extends AExpression { @Override void write(MethodWriter writer) { writer.writeDebugInfo(location); + boolean branch = tru != null || fals != null; org.objectweb.asm.Type rtype = right.actual.type; Sort rsort = right.actual.sort; diff --git a/modules/lang-painless/src/main/java/org/elasticsearch/painless/node/EConditional.java b/modules/lang-painless/src/main/java/org/elasticsearch/painless/node/EConditional.java index 03e1f4c55c5..65231c45ed3 100644 --- a/modules/lang-painless/src/main/java/org/elasticsearch/painless/node/EConditional.java +++ b/modules/lang-painless/src/main/java/org/elasticsearch/painless/node/EConditional.java @@ -80,6 +80,7 @@ public final class EConditional extends AExpression { @Override void write(MethodWriter writer) { writer.writeDebugInfo(location); + Label localfals = new Label(); Label end = new Label(); diff --git a/modules/lang-painless/src/main/java/org/elasticsearch/painless/node/EFunctionRef.java b/modules/lang-painless/src/main/java/org/elasticsearch/painless/node/EFunctionRef.java index ec2bd6f7e47..38b7a1b8ddb 100644 --- a/modules/lang-painless/src/main/java/org/elasticsearch/painless/node/EFunctionRef.java +++ b/modules/lang-painless/src/main/java/org/elasticsearch/painless/node/EFunctionRef.java @@ -19,16 +19,25 @@ package org.elasticsearch.painless.node; +import org.elasticsearch.painless.Definition; +import org.elasticsearch.painless.FunctionRef; import org.elasticsearch.painless.Location; import org.elasticsearch.painless.MethodWriter; import org.elasticsearch.painless.Variables; +import org.objectweb.asm.Type; + +import static org.elasticsearch.painless.WriterConstants.LAMBDA_BOOTSTRAP_HANDLE; + +import java.lang.invoke.LambdaMetafactory; /** * Represents a function reference. */ public class EFunctionRef extends AExpression { - public String type; - public String call; + public final String type; + public final String call; + + private FunctionRef ref; public EFunctionRef(Location location, String type, String call) { super(location); @@ -39,12 +48,48 @@ public class EFunctionRef extends AExpression { @Override void analyze(Variables variables) { - throw createError(new UnsupportedOperationException("Function references [" + type + "::" + call + - "] are not currently supported.")); + if (expected == null) { + ref = null; + actual = Definition.getType("String"); + } else { + try { + ref = new FunctionRef(expected, type, call); + } catch (IllegalArgumentException e) { + throw createError(e); + } + actual = expected; + } } @Override void write(MethodWriter writer) { - throw createError(new IllegalStateException("Illegal tree structure.")); + if (ref == null) { + writer.push(type + "." + call); + } else { + writer.writeDebugInfo(location); + // convert MethodTypes to asm Type for the constant pool. + String invokedType = ref.invokedType.toMethodDescriptorString(); + Type samMethodType = Type.getMethodType(ref.samMethodType.toMethodDescriptorString()); + Type interfaceType = Type.getMethodType(ref.interfaceMethodType.toMethodDescriptorString()); + if (ref.needsBridges()) { + writer.invokeDynamic(ref.invokedName, + invokedType, + LAMBDA_BOOTSTRAP_HANDLE, + samMethodType, + ref.implMethodASM, + samMethodType, + LambdaMetafactory.FLAG_BRIDGES, + 1, + interfaceType); + } else { + writer.invokeDynamic(ref.invokedName, + invokedType, + LAMBDA_BOOTSTRAP_HANDLE, + samMethodType, + ref.implMethodASM, + samMethodType, + 0); + } + } } } diff --git a/modules/lang-painless/src/main/java/org/elasticsearch/painless/node/EUnary.java b/modules/lang-painless/src/main/java/org/elasticsearch/painless/node/EUnary.java index aa15999718c..89d654cdf1f 100644 --- a/modules/lang-painless/src/main/java/org/elasticsearch/painless/node/EUnary.java +++ b/modules/lang-painless/src/main/java/org/elasticsearch/painless/node/EUnary.java @@ -167,6 +167,7 @@ public final class EUnary extends AExpression { @Override void write(MethodWriter writer) { writer.writeDebugInfo(location); + if (operation == Operation.NOT) { if (tru == null && fals == null) { Label localfals = new Label(); diff --git a/modules/lang-painless/src/main/java/org/elasticsearch/painless/node/LBrace.java b/modules/lang-painless/src/main/java/org/elasticsearch/painless/node/LBrace.java index 0c883d36ecf..b0816540c5e 100644 --- a/modules/lang-painless/src/main/java/org/elasticsearch/painless/node/LBrace.java +++ b/modules/lang-painless/src/main/java/org/elasticsearch/painless/node/LBrace.java @@ -47,7 +47,7 @@ public final class LBrace extends ALink { throw createError(new IllegalArgumentException("Illegal array access made without target.")); } - final Sort sort = before.sort; + Sort sort = before.sort; if (sort == Sort.ARRAY) { index.expected = Definition.INT_TYPE; @@ -84,5 +84,4 @@ public final class LBrace extends ALink { writer.writeDebugInfo(location); writer.arrayStore(after.type); } - } diff --git a/modules/lang-painless/src/main/java/org/elasticsearch/painless/node/LCall.java b/modules/lang-painless/src/main/java/org/elasticsearch/painless/node/LCall.java index f7e91790108..53e66ee68b1 100644 --- a/modules/lang-painless/src/main/java/org/elasticsearch/painless/node/LCall.java +++ b/modules/lang-painless/src/main/java/org/elasticsearch/painless/node/LCall.java @@ -82,7 +82,7 @@ public final class LCall extends ALink { } throw createError(new IllegalArgumentException("Unknown call [" + name + "] with [" + arguments.size() + - "] arguments on type [" + struct.name + "].")); + "] arguments on type [" + struct.name + "].")); } @Override @@ -93,6 +93,7 @@ public final class LCall extends ALink { @Override void load(MethodWriter writer) { writer.writeDebugInfo(location); + for (AExpression argument : arguments) { argument.write(writer); } diff --git a/modules/lang-painless/src/main/java/org/elasticsearch/painless/node/LDefArray.java b/modules/lang-painless/src/main/java/org/elasticsearch/painless/node/LDefArray.java index 5fb2ff870ef..5e28cd31b98 100644 --- a/modules/lang-painless/src/main/java/org/elasticsearch/painless/node/LDefArray.java +++ b/modules/lang-painless/src/main/java/org/elasticsearch/painless/node/LDefArray.java @@ -60,14 +60,16 @@ final class LDefArray extends ALink implements IDefLink { @Override void load(MethodWriter writer) { writer.writeDebugInfo(location); + String desc = Type.getMethodDescriptor(after.type, Definition.DEF_TYPE.type, index.actual.type); - writer.invokeDynamic("arrayLoad", desc, DEF_BOOTSTRAP_HANDLE, (Object)DefBootstrap.ARRAY_LOAD); + writer.invokeDynamic("arrayLoad", desc, DEF_BOOTSTRAP_HANDLE, (Object)DefBootstrap.ARRAY_LOAD, 0); } @Override void store(MethodWriter writer) { writer.writeDebugInfo(location); + String desc = Type.getMethodDescriptor(Definition.VOID_TYPE.type, Definition.DEF_TYPE.type, index.actual.type, after.type); - writer.invokeDynamic("arrayStore", desc, DEF_BOOTSTRAP_HANDLE, (Object)DefBootstrap.ARRAY_STORE); + writer.invokeDynamic("arrayStore", desc, DEF_BOOTSTRAP_HANDLE, (Object)DefBootstrap.ARRAY_STORE, 0); } } diff --git a/modules/lang-painless/src/main/java/org/elasticsearch/painless/node/LDefCall.java b/modules/lang-painless/src/main/java/org/elasticsearch/painless/node/LDefCall.java index 3b07d367eba..28c57744860 100644 --- a/modules/lang-painless/src/main/java/org/elasticsearch/painless/node/LDefCall.java +++ b/modules/lang-painless/src/main/java/org/elasticsearch/painless/node/LDefCall.java @@ -36,6 +36,7 @@ final class LDefCall extends ALink implements IDefLink { final String name; final List arguments; + long recipe; LDefCall(Location location, String name, List arguments) { super(location, -1); @@ -46,9 +47,20 @@ final class LDefCall extends ALink implements IDefLink { @Override ALink analyze(Variables variables) { + if (arguments.size() > 63) { + // technically, the limitation is just methods with > 63 params, containing method references. + // this is because we are lazy and use a long as a bitset. we can always change to a "string" if need be. + // but NEED NOT BE. nothing with this many parameters is in the whitelist and we do not support varargs. + throw new UnsupportedOperationException("methods with > 63 arguments are currently not supported"); + } + + recipe = 0; for (int argument = 0; argument < arguments.size(); ++argument) { AExpression expression = arguments.get(argument); + if (expression instanceof EFunctionRef) { + recipe |= (1L << argument); // mark argument as deferred reference + } expression.internal = true; expression.analyze(variables); expression.expected = expression.actual; @@ -69,6 +81,7 @@ final class LDefCall extends ALink implements IDefLink { @Override void load(MethodWriter writer) { writer.writeDebugInfo(location); + StringBuilder signature = new StringBuilder(); signature.append('('); @@ -84,7 +97,7 @@ final class LDefCall extends ALink implements IDefLink { // return value signature.append(after.type.getDescriptor()); - writer.invokeDynamic(name, signature.toString(), DEF_BOOTSTRAP_HANDLE, (Object)DefBootstrap.METHOD_CALL); + writer.invokeDynamic(name, signature.toString(), DEF_BOOTSTRAP_HANDLE, (Object)DefBootstrap.METHOD_CALL, recipe); } @Override diff --git a/modules/lang-painless/src/main/java/org/elasticsearch/painless/node/LDefField.java b/modules/lang-painless/src/main/java/org/elasticsearch/painless/node/LDefField.java index 2b32d75d89c..ec9d5bbe9e7 100644 --- a/modules/lang-painless/src/main/java/org/elasticsearch/painless/node/LDefField.java +++ b/modules/lang-painless/src/main/java/org/elasticsearch/painless/node/LDefField.java @@ -57,14 +57,16 @@ final class LDefField extends ALink implements IDefLink { @Override void load(MethodWriter writer) { writer.writeDebugInfo(location); + String desc = Type.getMethodDescriptor(after.type, Definition.DEF_TYPE.type); - writer.invokeDynamic(value, desc, DEF_BOOTSTRAP_HANDLE, (Object)DefBootstrap.LOAD); + writer.invokeDynamic(value, desc, DEF_BOOTSTRAP_HANDLE, (Object)DefBootstrap.LOAD, 0); } @Override void store(MethodWriter writer) { writer.writeDebugInfo(location); + String desc = Type.getMethodDescriptor(Definition.VOID_TYPE.type, Definition.DEF_TYPE.type, after.type); - writer.invokeDynamic(value, desc, DEF_BOOTSTRAP_HANDLE, (Object)DefBootstrap.STORE); + writer.invokeDynamic(value, desc, DEF_BOOTSTRAP_HANDLE, (Object)DefBootstrap.STORE, 0); } } diff --git a/modules/lang-painless/src/main/java/org/elasticsearch/painless/node/LField.java b/modules/lang-painless/src/main/java/org/elasticsearch/painless/node/LField.java index b9b7f525a97..8f34692d666 100644 --- a/modules/lang-painless/src/main/java/org/elasticsearch/painless/node/LField.java +++ b/modules/lang-painless/src/main/java/org/elasticsearch/painless/node/LField.java @@ -107,6 +107,7 @@ public final class LField extends ALink { @Override void load(MethodWriter writer) { writer.writeDebugInfo(location); + if (java.lang.reflect.Modifier.isStatic(field.modifiers)) { writer.getStatic(field.owner.type, field.javaName, field.type.type); } else { @@ -117,6 +118,7 @@ public final class LField extends ALink { @Override void store(MethodWriter writer) { writer.writeDebugInfo(location); + if (java.lang.reflect.Modifier.isStatic(field.modifiers)) { writer.putStatic(field.owner.type, field.javaName, field.type.type); } else { diff --git a/modules/lang-painless/src/main/java/org/elasticsearch/painless/node/LListShortcut.java b/modules/lang-painless/src/main/java/org/elasticsearch/painless/node/LListShortcut.java index f1a176181c8..a8252b40770 100644 --- a/modules/lang-painless/src/main/java/org/elasticsearch/painless/node/LListShortcut.java +++ b/modules/lang-painless/src/main/java/org/elasticsearch/painless/node/LListShortcut.java @@ -81,6 +81,7 @@ final class LListShortcut extends ALink { @Override void load(MethodWriter writer) { writer.writeDebugInfo(location); + if (java.lang.reflect.Modifier.isInterface(getter.owner.clazz.getModifiers())) { writer.invokeInterface(getter.owner.type, getter.method); } else { @@ -95,6 +96,7 @@ final class LListShortcut extends ALink { @Override void store(MethodWriter writer) { writer.writeDebugInfo(location); + if (java.lang.reflect.Modifier.isInterface(setter.owner.clazz.getModifiers())) { writer.invokeInterface(setter.owner.type, setter.method); } else { diff --git a/modules/lang-painless/src/main/java/org/elasticsearch/painless/node/LMapShortcut.java b/modules/lang-painless/src/main/java/org/elasticsearch/painless/node/LMapShortcut.java index b7f2b96ebd0..f31179d135a 100644 --- a/modules/lang-painless/src/main/java/org/elasticsearch/painless/node/LMapShortcut.java +++ b/modules/lang-painless/src/main/java/org/elasticsearch/painless/node/LMapShortcut.java @@ -80,6 +80,7 @@ final class LMapShortcut extends ALink { @Override void load(MethodWriter writer) { writer.writeDebugInfo(location); + if (java.lang.reflect.Modifier.isInterface(getter.owner.clazz.getModifiers())) { writer.invokeInterface(getter.owner.type, getter.method); } else { @@ -94,6 +95,7 @@ final class LMapShortcut extends ALink { @Override void store(MethodWriter writer) { writer.writeDebugInfo(location); + if (java.lang.reflect.Modifier.isInterface(setter.owner.clazz.getModifiers())) { writer.invokeInterface(setter.owner.type, setter.method); } else { diff --git a/modules/lang-painless/src/main/java/org/elasticsearch/painless/node/LNewArray.java b/modules/lang-painless/src/main/java/org/elasticsearch/painless/node/LNewArray.java index cc8c0668eb4..75a4aecbe19 100644 --- a/modules/lang-painless/src/main/java/org/elasticsearch/painless/node/LNewArray.java +++ b/modules/lang-painless/src/main/java/org/elasticsearch/painless/node/LNewArray.java @@ -81,6 +81,7 @@ public final class LNewArray extends ALink { @Override void load(MethodWriter writer) { writer.writeDebugInfo(location); + for (AExpression argument : arguments) { argument.write(writer); } diff --git a/modules/lang-painless/src/main/java/org/elasticsearch/painless/node/LNewObj.java b/modules/lang-painless/src/main/java/org/elasticsearch/painless/node/LNewObj.java index a27aa25754d..2f80b254350 100644 --- a/modules/lang-painless/src/main/java/org/elasticsearch/painless/node/LNewObj.java +++ b/modules/lang-painless/src/main/java/org/elasticsearch/painless/node/LNewObj.java @@ -21,7 +21,7 @@ package org.elasticsearch.painless.node; import org.elasticsearch.painless.Definition; import org.elasticsearch.painless.Location; -import org.elasticsearch.painless.Definition.Constructor; +import org.elasticsearch.painless.Definition.Method; import org.elasticsearch.painless.Definition.Struct; import org.elasticsearch.painless.Definition.Type; import org.elasticsearch.painless.Variables; @@ -37,7 +37,7 @@ public final class LNewObj extends ALink { final String type; final List arguments; - Constructor constructor; + Method constructor; public LNewObj(Location location, String type, List arguments) { super(location, -1); @@ -63,7 +63,7 @@ public final class LNewObj extends ALink { } Struct struct = type.struct; - constructor = struct.constructors.get(new Definition.MethodKey("new", arguments.size())); + constructor = struct.constructors.get(new Definition.MethodKey("", arguments.size())); if (constructor != null) { Type[] types = new Type[constructor.arguments.size()]; diff --git a/modules/lang-painless/src/main/java/org/elasticsearch/painless/node/LShortcut.java b/modules/lang-painless/src/main/java/org/elasticsearch/painless/node/LShortcut.java index 5e4928c04d5..a91970fa577 100644 --- a/modules/lang-painless/src/main/java/org/elasticsearch/painless/node/LShortcut.java +++ b/modules/lang-painless/src/main/java/org/elasticsearch/painless/node/LShortcut.java @@ -86,6 +86,7 @@ final class LShortcut extends ALink { @Override void load(MethodWriter writer) { writer.writeDebugInfo(location); + if (java.lang.reflect.Modifier.isInterface(getter.owner.clazz.getModifiers())) { writer.invokeInterface(getter.owner.type, getter.method); } else { @@ -100,6 +101,7 @@ final class LShortcut extends ALink { @Override void store(MethodWriter writer) { writer.writeDebugInfo(location); + if (java.lang.reflect.Modifier.isInterface(setter.owner.clazz.getModifiers())) { writer.invokeInterface(setter.owner.type, setter.method); } else { diff --git a/modules/lang-painless/src/main/java/org/elasticsearch/painless/node/SBlock.java b/modules/lang-painless/src/main/java/org/elasticsearch/painless/node/SBlock.java index e1dd81ee54e..4dbbd80de54 100644 --- a/modules/lang-painless/src/main/java/org/elasticsearch/painless/node/SBlock.java +++ b/modules/lang-painless/src/main/java/org/elasticsearch/painless/node/SBlock.java @@ -45,9 +45,11 @@ public final class SBlock extends AStatement { throw createError(new IllegalArgumentException("A block must contain at least one statement.")); } - final AStatement last = statements.get(statements.size() - 1); + AStatement last = statements.get(statements.size() - 1); for (AStatement statement : statements) { + // Note that we do not need to check after the last statement because + // there is no statement that can be unreachable after the last. if (allEscape) { throw createError(new IllegalArgumentException("Unreachable statement.")); } diff --git a/modules/lang-painless/src/main/java/org/elasticsearch/painless/node/SCatch.java b/modules/lang-painless/src/main/java/org/elasticsearch/painless/node/SCatch.java index d78a4539580..8bcaf9d22cf 100644 --- a/modules/lang-painless/src/main/java/org/elasticsearch/painless/node/SCatch.java +++ b/modules/lang-painless/src/main/java/org/elasticsearch/painless/node/SCatch.java @@ -86,6 +86,7 @@ public final class SCatch extends AStatement { @Override void write(MethodWriter writer) { writer.writeStatementOffset(location); + Label jump = new Label(); writer.mark(jump); diff --git a/modules/lang-painless/src/main/java/org/elasticsearch/painless/node/SDeclBlock.java b/modules/lang-painless/src/main/java/org/elasticsearch/painless/node/SDeclBlock.java index e52819c7959..1ff187afe29 100644 --- a/modules/lang-painless/src/main/java/org/elasticsearch/painless/node/SDeclBlock.java +++ b/modules/lang-painless/src/main/java/org/elasticsearch/painless/node/SDeclBlock.java @@ -50,7 +50,7 @@ public final class SDeclBlock extends AStatement { @Override void write(MethodWriter writer) { - for (SDeclaration declaration : declarations) { + for (AStatement declaration : declarations) { declaration.write(writer); } } diff --git a/modules/lang-painless/src/main/java/org/elasticsearch/painless/node/SDeclaration.java b/modules/lang-painless/src/main/java/org/elasticsearch/painless/node/SDeclaration.java index 38c9bad315b..5f039184cb6 100644 --- a/modules/lang-painless/src/main/java/org/elasticsearch/painless/node/SDeclaration.java +++ b/modules/lang-painless/src/main/java/org/elasticsearch/painless/node/SDeclaration.java @@ -68,6 +68,7 @@ public final class SDeclaration extends AStatement { @Override void write(MethodWriter writer) { writer.writeStatementOffset(location); + if (expression == null) { switch (variable.type.sort) { case VOID: throw createError(new IllegalStateException("Illegal tree structure.")); diff --git a/modules/lang-painless/src/main/java/org/elasticsearch/painless/node/SDo.java b/modules/lang-painless/src/main/java/org/elasticsearch/painless/node/SDo.java index a8c76c98dcc..4989fb77a79 100644 --- a/modules/lang-painless/src/main/java/org/elasticsearch/painless/node/SDo.java +++ b/modules/lang-painless/src/main/java/org/elasticsearch/painless/node/SDo.java @@ -88,6 +88,7 @@ public final class SDo extends AStatement { @Override void write(MethodWriter writer) { writer.writeStatementOffset(location); + Label start = new Label(); Label begin = new Label(); Label end = new Label(); diff --git a/modules/lang-painless/src/main/java/org/elasticsearch/painless/node/SEach.java b/modules/lang-painless/src/main/java/org/elasticsearch/painless/node/SEach.java new file mode 100644 index 00000000000..9a6fe798957 --- /dev/null +++ b/modules/lang-painless/src/main/java/org/elasticsearch/painless/node/SEach.java @@ -0,0 +1,228 @@ +/* + * 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.painless.node; + +import org.elasticsearch.painless.AnalyzerCaster; +import org.elasticsearch.painless.DefBootstrap; +import org.elasticsearch.painless.Definition; +import org.elasticsearch.painless.Definition.Cast; +import org.elasticsearch.painless.Definition.Method; +import org.elasticsearch.painless.Definition.MethodKey; +import org.elasticsearch.painless.Definition.Sort; +import org.elasticsearch.painless.Definition.Type; +import org.elasticsearch.painless.Location; +import org.elasticsearch.painless.MethodWriter; +import org.elasticsearch.painless.Variables; +import org.elasticsearch.painless.Variables.Variable; +import org.objectweb.asm.Label; +import org.objectweb.asm.Opcodes; + +import static org.elasticsearch.painless.WriterConstants.DEF_BOOTSTRAP_HANDLE; +import static org.elasticsearch.painless.WriterConstants.ITERATOR_HASNEXT; +import static org.elasticsearch.painless.WriterConstants.ITERATOR_NEXT; +import static org.elasticsearch.painless.WriterConstants.ITERATOR_TYPE; + +/** + * Represents a for-each loop shortcut for iterables. Defers to other S-nodes for non-iterable types. + */ +public class SEach extends AStatement { + + final int maxLoopCounter; + final String type; + final String name; + AExpression expression; + final SBlock block; + + // Members for all cases. + Variable variable = null; + Cast cast = null; + + // Members for the array case. + Variable array = null; + Variable index = null; + Type indexed = null; + + // Members for the iterable case. + Variable iterator = null; + Method method = null; + + public SEach(Location location, int maxLoopCounter, String type, String name, AExpression expression, SBlock block) { + super(location); + + this.maxLoopCounter = maxLoopCounter; + this.type = type; + this.name = name; + this.expression = expression; + this.block = block; + } + + @Override + void analyze(Variables variables) { + expression.analyze(variables); + expression.expected = expression.actual; + expression = expression.cast(variables); + + final Type type; + + try { + type = Definition.getType(this.type); + } catch (IllegalArgumentException exception) { + throw createError(new IllegalArgumentException("Not a type [" + this.type + "].")); + } + + variables.incrementScope(); + + variable = variables.addVariable(location, type, name, true, false); + + if (expression.actual.sort == Sort.ARRAY) { + analyzeArray(variables, type); + } else if (expression.actual.sort == Sort.DEF || Iterable.class.isAssignableFrom(expression.actual.clazz)) { + analyzeIterable(variables, type); + } else { + throw location.createError(new IllegalArgumentException("Illegal for each type [" + expression.actual.name + "].")); + } + + if (block == null) { + throw location.createError(new IllegalArgumentException("Extraneous for each loop.")); + } + + block.beginLoop = true; + block.inLoop = true; + block.analyze(variables); + block.statementCount = Math.max(1, block.statementCount); + + if (block.loopEscape && !block.anyContinue) { + throw createError(new IllegalArgumentException("Extraneous for loop.")); + } + + statementCount = 1; + + if (maxLoopCounter > 0) { + loopCounterSlot = variables.getVariable(location, "#loop").slot; + } + + variables.decrementScope(); + } + + void analyzeArray(Variables variables, Type type) { + // We must store the array and index as variables for securing slots on the stack, and + // also add the location offset to make the names unique in case of nested for each loops. + array = variables.addVariable(location, expression.actual, "#array" + location.getOffset(), true, false); + index = variables.addVariable(location, Definition.INT_TYPE, "#index" + location.getOffset(), true, false); + indexed = Definition.getType(expression.actual.struct, expression.actual.dimensions - 1); + cast = AnalyzerCaster.getLegalCast(location, indexed, type, true, true); + } + + void analyzeIterable(Variables variables, Type type) { + // We must store the iterator as a variable for securing a slot on the stack, and + // also add the location offset to make the name unique in case of nested for each loops. + iterator = variables.addVariable(location, Definition.getType("Iterator"), "#itr" + location.getOffset(), true, false); + + if (expression.actual.sort == Sort.DEF) { + method = null; + } else { + method = expression.actual.struct.methods.get(new MethodKey("iterator", 0)); + + if (method == null) { + throw location.createError(new IllegalArgumentException( + "Unable to create iterator for the type [" + expression.actual.name + "].")); + } + } + + cast = AnalyzerCaster.getLegalCast(location, Definition.DEF_TYPE, type, true, true); + } + + @Override + void write(MethodWriter writer) { + writer.writeStatementOffset(location); + + if (array != null) { + writeArray(writer); + } else if (iterator != null) { + writeIterable(writer); + } else { + throw location.createError(new IllegalStateException("Illegal tree structure.")); + } + } + + void writeArray(MethodWriter writer) { + expression.write(writer); + writer.visitVarInsn(array.type.type.getOpcode(Opcodes.ISTORE), array.slot); + writer.push(-1); + writer.visitVarInsn(index.type.type.getOpcode(Opcodes.ISTORE), index.slot); + + Label begin = new Label(); + Label end = new Label(); + + writer.mark(begin); + + writer.visitIincInsn(index.slot, 1); + writer.visitVarInsn(index.type.type.getOpcode(Opcodes.ILOAD), index.slot); + writer.visitVarInsn(array.type.type.getOpcode(Opcodes.ILOAD), array.slot); + writer.arrayLength(); + writer.ifICmp(MethodWriter.GE, end); + + writer.visitVarInsn(array.type.type.getOpcode(Opcodes.ILOAD), array.slot); + writer.visitVarInsn(index.type.type.getOpcode(Opcodes.ILOAD), index.slot); + writer.arrayLoad(indexed.type); + writer.writeCast(cast); + writer.visitVarInsn(variable.type.type.getOpcode(Opcodes.ISTORE), variable.slot); + + block.write(writer); + + writer.goTo(begin); + writer.mark(end); + } + + void writeIterable(MethodWriter writer) { + expression.write(writer); + + if (method == null) { + Type itr = Definition.getType("Iterator"); + String desc = org.objectweb.asm.Type.getMethodDescriptor(itr.type, Definition.DEF_TYPE.type); + writer.invokeDynamic("iterator", desc, DEF_BOOTSTRAP_HANDLE, (Object)DefBootstrap.ITERATOR, 0); + } else if (java.lang.reflect.Modifier.isInterface(method.owner.clazz.getModifiers())) { + writer.invokeInterface(method.owner.type, method.method); + } else { + writer.invokeVirtual(method.owner.type, method.method); + } + + writer.visitVarInsn(iterator.type.type.getOpcode(Opcodes.ISTORE), iterator.slot); + + Label begin = new Label(); + Label end = new Label(); + + writer.mark(begin); + + writer.visitVarInsn(iterator.type.type.getOpcode(Opcodes.ILOAD), iterator.slot); + writer.invokeInterface(ITERATOR_TYPE, ITERATOR_HASNEXT); + writer.ifZCmp(MethodWriter.EQ, end); + + writer.visitVarInsn(iterator.type.type.getOpcode(Opcodes.ILOAD), iterator.slot); + writer.invokeInterface(ITERATOR_TYPE, ITERATOR_NEXT); + writer.writeCast(cast); + writer.visitVarInsn(variable.type.type.getOpcode(Opcodes.ISTORE), variable.slot); + + block.write(writer); + + writer.goTo(begin); + writer.mark(end); + } +} diff --git a/modules/lang-painless/src/main/java/org/elasticsearch/painless/node/SFor.java b/modules/lang-painless/src/main/java/org/elasticsearch/painless/node/SFor.java index 58fdb33fb3c..04475a91a1a 100644 --- a/modules/lang-painless/src/main/java/org/elasticsearch/painless/node/SFor.java +++ b/modules/lang-painless/src/main/java/org/elasticsearch/painless/node/SFor.java @@ -40,11 +40,11 @@ public final class SFor extends AStatement { ANode initializer, AExpression condition, AExpression afterthought, SBlock block) { super(location); + this.maxLoopCounter = maxLoopCounter; this.initializer = initializer; this.condition = condition; this.afterthought = afterthought; this.block = block; - this.maxLoopCounter = maxLoopCounter; } @Override @@ -54,8 +54,8 @@ public final class SFor extends AStatement { boolean continuous = false; if (initializer != null) { - if (initializer instanceof SDeclBlock) { - ((SDeclBlock)initializer).analyze(variables); + if (initializer instanceof AStatement) { + ((AStatement)initializer).analyze(variables); } else if (initializer instanceof AExpression) { AExpression initializer = (AExpression)this.initializer; @@ -129,6 +129,7 @@ public final class SFor extends AStatement { @Override void write(MethodWriter writer) { writer.writeStatementOffset(location); + Label start = new Label(); Label begin = afterthought == null ? start : new Label(); Label end = new Label(); diff --git a/modules/lang-painless/src/main/java/org/elasticsearch/painless/node/SIf.java b/modules/lang-painless/src/main/java/org/elasticsearch/painless/node/SIf.java index 5c871db5eb9..954ddac9c6a 100644 --- a/modules/lang-painless/src/main/java/org/elasticsearch/painless/node/SIf.java +++ b/modules/lang-painless/src/main/java/org/elasticsearch/painless/node/SIf.java @@ -70,6 +70,7 @@ public final class SIf extends AStatement { @Override void write(MethodWriter writer) { writer.writeStatementOffset(location); + Label fals = new Label(); condition.fals = fals; diff --git a/modules/lang-painless/src/main/java/org/elasticsearch/painless/node/SIfElse.java b/modules/lang-painless/src/main/java/org/elasticsearch/painless/node/SIfElse.java index 7c552ac40e2..1d801267054 100644 --- a/modules/lang-painless/src/main/java/org/elasticsearch/painless/node/SIfElse.java +++ b/modules/lang-painless/src/main/java/org/elasticsearch/painless/node/SIfElse.java @@ -91,6 +91,7 @@ public final class SIfElse extends AStatement { @Override void write(MethodWriter writer) { writer.writeStatementOffset(location); + Label end = new Label(); Label fals = elseblock != null ? new Label() : end; diff --git a/modules/lang-painless/src/main/java/org/elasticsearch/painless/node/SSource.java b/modules/lang-painless/src/main/java/org/elasticsearch/painless/node/SSource.java index 510c54da7a7..22fd2e3e28c 100644 --- a/modules/lang-painless/src/main/java/org/elasticsearch/painless/node/SSource.java +++ b/modules/lang-painless/src/main/java/org/elasticsearch/painless/node/SSource.java @@ -48,15 +48,17 @@ public final class SSource extends AStatement { variables.incrementScope(); - final AStatement last = statements.get(statements.size() - 1); + AStatement last = statements.get(statements.size() - 1); for (AStatement statement : statements) { - // TODO: why are we checking only statements 0..n-1 (this effectively checks only the previous statement) + // Note that we do not need to check after the last statement because + // there is no statement that can be unreachable after the last. if (allEscape) { throw createError(new IllegalArgumentException("Unreachable statement.")); } statement.lastSource = statement == last; + statement.analyze(variables); methodEscape = statement.methodEscape; diff --git a/modules/lang-painless/src/main/java/org/elasticsearch/painless/node/STry.java b/modules/lang-painless/src/main/java/org/elasticsearch/painless/node/STry.java index b522724280a..c24c8273dba 100644 --- a/modules/lang-painless/src/main/java/org/elasticsearch/painless/node/STry.java +++ b/modules/lang-painless/src/main/java/org/elasticsearch/painless/node/STry.java @@ -35,11 +35,11 @@ public final class STry extends AStatement { final SBlock block; final List catches; - public STry(Location location, SBlock block, List traps) { + public STry(Location location, SBlock block, List catches) { super(location); this.block = block; - this.catches = Collections.unmodifiableList(traps); + this.catches = Collections.unmodifiableList(catches); } @Override @@ -88,6 +88,7 @@ public final class STry extends AStatement { @Override void write(MethodWriter writer) { writer.writeStatementOffset(location); + Label begin = new Label(); Label end = new Label(); Label exception = new Label(); diff --git a/modules/lang-painless/src/main/java/org/elasticsearch/painless/node/SWhile.java b/modules/lang-painless/src/main/java/org/elasticsearch/painless/node/SWhile.java index 4b071be36b5..59c1bb75ee8 100644 --- a/modules/lang-painless/src/main/java/org/elasticsearch/painless/node/SWhile.java +++ b/modules/lang-painless/src/main/java/org/elasticsearch/painless/node/SWhile.java @@ -94,6 +94,7 @@ public final class SWhile extends AStatement { @Override void write(MethodWriter writer) { writer.writeStatementOffset(location); + Label begin = new Label(); Label end = new Label(); diff --git a/modules/lang-painless/src/main/java/org/elasticsearch/painless/node/package-info.java b/modules/lang-painless/src/main/java/org/elasticsearch/painless/node/package-info.java index 13ef13bdd77..71cde33e979 100644 --- a/modules/lang-painless/src/main/java/org/elasticsearch/painless/node/package-info.java +++ b/modules/lang-painless/src/main/java/org/elasticsearch/painless/node/package-info.java @@ -70,6 +70,7 @@ * {@link org.elasticsearch.painless.node.SDeclaration} - Represents a single variable declaration. * {@link org.elasticsearch.painless.node.SDeclBlock} - Represents a series of declarations. * {@link org.elasticsearch.painless.node.SDo} - Represents a do-while loop. + * {@link org.elasticsearch.painless.node.SEach} - Represents a for each loop shortcut for iterables. * {@link org.elasticsearch.painless.node.SExpression} - Represents the top-level node for an expression as a statement. * {@link org.elasticsearch.painless.node.SFor} - Represents a for loop. * {@link org.elasticsearch.painless.node.SIf} - Represents an if block. @@ -90,7 +91,7 @@ *

    * Generally, statement nodes have member data that evaluate legal control-flow during the analysis phase. * The typical order for statement nodes is for each node to call analyze on it's children during the analysis phase - * and write on it's children during the writing phase. No modifications are made to the structure of statement nodes. + * and write on it's children during the writing phase. *

    * Generally, expression nodes have member data that evaluate static types. The typical order for an expression node * during the analysis phase looks like the following: diff --git a/modules/lang-painless/src/main/resources/org/elasticsearch/painless/java.lang.txt b/modules/lang-painless/src/main/resources/org/elasticsearch/painless/java.lang.txt index 755c8d44c0f..ec1a46e633f 100644 --- a/modules/lang-painless/src/main/resources/org/elasticsearch/painless/java.lang.txt +++ b/modules/lang-painless/src/main/resources/org/elasticsearch/painless/java.lang.txt @@ -63,6 +63,7 @@ class Boolean -> java.lang.Boolean extends Comparable,Object { Boolean FALSE boolean booleanValue() int compare(boolean,boolean) + int compareTo(Boolean) int hashCode(boolean) boolean logicalAnd(boolean,boolean) boolean logicalOr(boolean,boolean) @@ -78,6 +79,7 @@ class Byte -> java.lang.Byte extends Number,Comparable,Object { byte MIN_VALUE int SIZE int compare(byte,byte) + int compareTo(Byte) Byte decode(String) int hashCode(byte) byte parseByte(String) @@ -163,6 +165,7 @@ class Character -> java.lang.Character extends Comparable,Object { int codePointBefore(CharSequence,int) int codePointCount(CharSequence,int,int) int compare(char,char) + int compareTo(Character) int digit(int,int) char forDigit(int,int) byte getDirectionality(int) @@ -454,6 +457,7 @@ class Double -> java.lang.Double extends Number,Comparable,Object { double POSITIVE_INFINITY int SIZE int compare(double,double) + int compareTo(Double) long doubleToLongBits(double) long doubleToRawLongBits(double) int hashCode(double) @@ -473,6 +477,7 @@ class Double -> java.lang.Double extends Number,Comparable,Object { } class Enum -> java.lang.Enum extends Comparable,Object { + int compareTo(Enum) String name(); int ordinal(); } @@ -489,6 +494,7 @@ class Float -> java.lang.Float extends Number,Comparable,Object { float POSITIVE_INFINITY int SIZE int compare(float,float) + int compareTo(Float) int floatToIntBits(float) int floatToRawIntBits(float) int hashCode(float) @@ -516,6 +522,7 @@ class Integer -> java.lang.Integer extends Number,Comparable,Object { int SIZE int bitCount(int) int compare(int,int) + int compareTo(Integer) int compareUnsigned(int,int) Integer decode(String) int divideUnsigned(int,int) @@ -555,6 +562,7 @@ class Long -> java.lang.Long extends Number,Comparable,Object { int SIZE int bitCount(long) int compare(long,long) + int compareTo(Long) int compareUnsigned(long,long) Long decode(String) long divideUnsigned(long,long) @@ -587,7 +595,7 @@ class Long -> java.lang.Long extends Number,Comparable,Object { Long valueOf(String,int) } -class Math -> java.lang.Math { +class Math -> java.lang.Math extends Object { double E double PI double abs(double) @@ -658,6 +666,7 @@ class Short -> java.lang.Short extends Number,Comparable,Object { short MIN_VALUE int SIZE int compare(short,short) + int compareTo(Short) Short decode(String) int hashCode(short) short parseShort(String) @@ -679,7 +688,7 @@ class StackTraceElement -> java.lang.StackTraceElement extends Object { boolean isNativeMethod() } -class StrictMath -> java.lang.StrictMath { +class StrictMath -> java.lang.StrictMath extends Object { double E double PI double abs(double) @@ -726,6 +735,7 @@ class String -> java.lang.String extends CharSequence,Comparable,Object { int codePointAt(int) int codePointBefore(int) int codePointCount(int,int) + int compareTo(String) int compareToIgnoreCase(String) String concat(String) boolean contains(CharSequence) diff --git a/modules/lang-painless/src/main/resources/org/elasticsearch/painless/java.math.txt b/modules/lang-painless/src/main/resources/org/elasticsearch/painless/java.math.txt index 42680f8a428..e3d13a0959c 100644 --- a/modules/lang-painless/src/main/resources/org/elasticsearch/painless/java.math.txt +++ b/modules/lang-painless/src/main/resources/org/elasticsearch/painless/java.math.txt @@ -35,6 +35,7 @@ class BigDecimal -> java.math.BigDecimal extends Number,Comparable,Object { BigDecimal add(BigDecimal) BigDecimal add(BigDecimal,MathContext) byte byteValueExact() + int compareTo(BigDecimal) BigDecimal divide(BigDecimal) BigDecimal divide(BigDecimal,MathContext) BigDecimal[] divideAndRemainder(BigDecimal) @@ -90,6 +91,7 @@ class BigInteger -> java.math.BigInteger extends Number,Comparable,Object { int bitLength() byte byteValueExact() BigInteger clearBit(int) + int compareTo(BigInteger) BigInteger divide(BigInteger) BigInteger[] divideAndRemainder(BigInteger) BigInteger flipBit(int) diff --git a/modules/lang-painless/src/main/resources/org/elasticsearch/painless/java.text.txt b/modules/lang-painless/src/main/resources/org/elasticsearch/painless/java.text.txt index eac725cfb94..2e93b14ab3d 100644 --- a/modules/lang-painless/src/main/resources/org/elasticsearch/painless/java.text.txt +++ b/modules/lang-painless/src/main/resources/org/elasticsearch/painless/java.text.txt @@ -149,6 +149,7 @@ class CollationElementIterator -> java.text.CollationElementIterator extends Obj } class CollationKey -> java.text.CollationKey extends Comparable,Object { + int compareTo(CollationKey) String getSourceString() byte[] toByteArray() } diff --git a/modules/lang-painless/src/main/resources/org/elasticsearch/painless/java.time.chrono.txt b/modules/lang-painless/src/main/resources/org/elasticsearch/painless/java.time.chrono.txt index 8dd0478605e..7374efea04b 100644 --- a/modules/lang-painless/src/main/resources/org/elasticsearch/painless/java.time.chrono.txt +++ b/modules/lang-painless/src/main/resources/org/elasticsearch/painless/java.time.chrono.txt @@ -24,8 +24,9 @@ #### Interfaces -class ChronoLocalDate -> java.time.chrono.ChronoLocalDate extends Comparable,Temporal,TemporalAccessor,TemporalAdjuster { +class ChronoLocalDate -> java.time.chrono.ChronoLocalDate extends Temporal,TemporalAccessor,TemporalAdjuster,Comparable { ChronoLocalDateTime atTime(LocalTime) + int compareTo(ChronoLocalDate) boolean equals(Object) String format(DateTimeFormatter) ChronoLocalDate from(TemporalAccessor) @@ -50,8 +51,9 @@ class ChronoLocalDate -> java.time.chrono.ChronoLocalDate extends Comparable,Tem ChronoLocalDate with(TemporalField,long) } -class ChronoLocalDateTime -> java.time.chrono.ChronoLocalDateTime extends Comparable,Temporal,TemporalAccessor,TemporalAdjuster { +class ChronoLocalDateTime -> java.time.chrono.ChronoLocalDateTime extends Temporal,TemporalAccessor,TemporalAdjuster,Comparable { ChronoZonedDateTime atZone(ZoneId) + int compareTo(ChronoLocalDateTime) boolean equals(Object) String format(DateTimeFormatter) ChronoLocalDateTime from(TemporalAccessor) @@ -75,6 +77,7 @@ class ChronoLocalDateTime -> java.time.chrono.ChronoLocalDateTime extends Compar } class Chronology -> java.time.chrono.Chronology extends Comparable { + int compareTo(Chronology) ChronoLocalDate date(TemporalAccessor) ChronoLocalDate date(Era,int,int,int) ChronoLocalDate date(int,int,int) @@ -121,7 +124,8 @@ class ChronoPeriod -> java.time.chrono.ChronoPeriod extends TemporalAmount { String toString() } -class ChronoZonedDateTime -> java.time.chrono.ChronoZonedDateTime extends Comparable,Temporal,TemporalAccessor { +class ChronoZonedDateTime -> java.time.chrono.ChronoZonedDateTime extends Temporal,TemporalAccessor,Comparable { + int compareTo(ChronoZonedDateTime) boolean equals(Object) String format(DateTimeFormatter) ChronoZonedDateTime from(TemporalAccessor) @@ -158,10 +162,10 @@ class Era -> java.time.chrono.Era extends TemporalAccessor,TemporalAdjuster { #### Classes -class AbstractChronology -> java.time.chrono.Chronology extends Comparable,Chronology,Object { +class AbstractChronology -> java.time.chrono.Chronology extends Chronology,Comparable,Object { } -class HijrahChronology -> java.time.chrono.HijrahChronology extends AbstractChronology,Comparable,Chronology,Object { +class HijrahChronology -> java.time.chrono.HijrahChronology extends AbstractChronology,Chronology,Comparable,Object { HijrahChronology INSTANCE HijrahDate date(TemporalAccessor) HijrahDate date(int,int,int) @@ -172,25 +176,26 @@ class HijrahChronology -> java.time.chrono.HijrahChronology extends AbstractChro HijrahDate dateYearDay(int,int) HijrahDate dateYearDay(Era,int,int) HijrahEra eraOf(int) + HijrahDate resolveDate(Map,ResolverStyle) } -class HijrahDate -> java.time.chrono.HijrahDate extends Comparable,ChronoLocalDate,Temporal,TemporalAccessor,TemporalAdjuster,Object { - HijrahDate now() - HijrahDate now(ZoneId) - HijrahDate of(int,int,int) +class HijrahDate -> java.time.chrono.HijrahDate extends ChronoLocalDate,Temporal,TemporalAccessor,TemporalAdjuster,Comparable,Object { HijrahDate from(TemporalAccessor) HijrahChronology getChronology() HijrahEra getEra() + HijrahDate minus(TemporalAmount) + HijrahDate minus(long,TemporalUnit) + HijrahDate now() + HijrahDate now(ZoneId) + HijrahDate of(int,int,int) + HijrahDate plus(TemporalAmount) + HijrahDate plus(long,TemporalUnit) HijrahDate with(TemporalField,long) HijrahDate with(TemporalAdjuster) HijrahDate withVariant(HijrahChronology) - HijrahDate plus(TemporalAmount) - HijrahDate minus(TemporalAmount) - HijrahDate plus(long,TemporalUnit) - HijrahDate minus(long,TemporalUnit) } -class IsoChronology -> java.time.chrono.IsoChronology extends AbstractChronology,Comparable,Chronology,Object { +class IsoChronology -> java.time.chrono.IsoChronology extends AbstractChronology,Chronology,Comparable,Object { IsoChronology INSTANCE LocalDate date(TemporalAccessor) LocalDate date(int,int,int) @@ -201,9 +206,14 @@ class IsoChronology -> java.time.chrono.IsoChronology extends AbstractChronology LocalDate dateYearDay(int,int) LocalDate dateYearDay(Era,int,int) IsoEra eraOf(int) + LocalDateTime localDateTime(TemporalAccessor) + Period period(int,int,int) + LocalDate resolveDate(Map,ResolverStyle) + ZonedDateTime zonedDateTime(TemporalAccessor) + ZonedDateTime zonedDateTime(Instant,ZoneId) } -class JapaneseChronology -> java.time.chrono.JapaneseChronology extends AbstractChronology,Comparable,Chronology,Object { +class JapaneseChronology -> java.time.chrono.JapaneseChronology extends AbstractChronology,Chronology,Comparable,Object { JapaneseChronology INSTANCE JapaneseDate date(TemporalAccessor) JapaneseDate date(int,int,int) @@ -214,9 +224,10 @@ class JapaneseChronology -> java.time.chrono.JapaneseChronology extends Abstract JapaneseDate dateYearDay(int,int) JapaneseDate dateYearDay(Era,int,int) JapaneseEra eraOf(int) + JapaneseDate resolveDate(Map,ResolverStyle) } -class JapaneseDate -> java.time.chrono.JapaneseDate extends Comparable,ChronoLocalDate,Temporal,TemporalAccessor,TemporalAdjuster,Object { +class JapaneseDate -> java.time.chrono.JapaneseDate extends ChronoLocalDate,Temporal,TemporalAccessor,TemporalAdjuster,Comparable,Object { JapaneseDate now() JapaneseDate now(ZoneId) JapaneseDate of(int,int,int) @@ -242,7 +253,7 @@ class JapaneseEra -> java.time.chrono.JapaneseEra extends Era,TemporalAccessor,T JapaneseEra[] values() } -class MinguoChronology -> java.time.chrono.MinguoChronology extends AbstractChronology,Comparable,Chronology,Object { +class MinguoChronology -> java.time.chrono.MinguoChronology extends AbstractChronology,Chronology,Comparable,Object { MinguoChronology INSTANCE MinguoDate date(TemporalAccessor) MinguoDate date(int,int,int) @@ -253,9 +264,10 @@ class MinguoChronology -> java.time.chrono.MinguoChronology extends AbstractChro MinguoDate dateYearDay(int,int) MinguoDate dateYearDay(Era,int,int) MinguoEra eraOf(int) + MinguoDate resolveDate(Map,ResolverStyle) } -class MinguoDate -> java.time.chrono.MinguoDate extends Comparable,ChronoLocalDate,Temporal,TemporalAccessor,TemporalAdjuster,Object { +class MinguoDate -> java.time.chrono.MinguoDate extends ChronoLocalDate,Temporal,TemporalAccessor,TemporalAdjuster,Comparable,Object { MinguoDate now() MinguoDate now(ZoneId) MinguoDate of(int,int,int) @@ -270,7 +282,7 @@ class MinguoDate -> java.time.chrono.MinguoDate extends Comparable,ChronoLocalDa MinguoDate minus(long,TemporalUnit) } -class ThaiBuddhistChronology -> java.time.chrono.ThaiBuddhistChronology extends AbstractChronology,Comparable,Chronology,Object { +class ThaiBuddhistChronology -> java.time.chrono.ThaiBuddhistChronology extends AbstractChronology,Chronology,Comparable,Object { ThaiBuddhistChronology INSTANCE ThaiBuddhistDate date(TemporalAccessor) ThaiBuddhistDate date(int,int,int) @@ -281,9 +293,10 @@ class ThaiBuddhistChronology -> java.time.chrono.ThaiBuddhistChronology extends ThaiBuddhistDate dateYearDay(int,int) ThaiBuddhistDate dateYearDay(Era,int,int) ThaiBuddhistEra eraOf(int) + ThaiBuddhistDate resolveDate(Map,ResolverStyle) } -class ThaiBuddhistDate -> java.time.chrono.ThaiBuddhistDate extends Comparable,ChronoLocalDate,Temporal,TemporalAccessor,TemporalAdjuster,Object { +class ThaiBuddhistDate -> java.time.chrono.ThaiBuddhistDate extends ChronoLocalDate,Temporal,TemporalAccessor,TemporalAdjuster,Comparable,Object { ThaiBuddhistDate now() ThaiBuddhistDate now(ZoneId) ThaiBuddhistDate of(int,int,int) diff --git a/modules/lang-painless/src/main/resources/org/elasticsearch/painless/java.time.txt b/modules/lang-painless/src/main/resources/org/elasticsearch/painless/java.time.txt index 1b7d19a6c9c..4481004fdf6 100644 --- a/modules/lang-painless/src/main/resources/org/elasticsearch/painless/java.time.txt +++ b/modules/lang-painless/src/main/resources/org/elasticsearch/painless/java.time.txt @@ -43,6 +43,7 @@ class Duration -> java.time.Duration extends Comparable,TemporalAmount,Object { Duration ZERO Duration abs() Duration between(Temporal,Temporal) + int compareTo(Duration) Duration dividedBy(long) Duration from(TemporalAmount) int getNano() @@ -91,6 +92,7 @@ class Instant -> java.time.Instant extends Comparable,Temporal,TemporalAccessor, Instant MIN OffsetDateTime atOffset(ZoneOffset) ZonedDateTime atZone(ZoneId) + int compareTo(Instant) Instant from(TemporalAccessor) long getEpochSecond() int getNano() @@ -118,7 +120,7 @@ class Instant -> java.time.Instant extends Comparable,Temporal,TemporalAccessor, Instant with(TemporalField,long) } -class LocalDate -> java.time.LocalDate extends Comparable,ChronoLocalDate,Temporal,TemporalAccessor,TemporalAdjuster,Object { +class LocalDate -> java.time.LocalDate extends ChronoLocalDate,Temporal,TemporalAccessor,TemporalAdjuster,Comparable,Object { LocalDate MAX LocalDate MIN LocalDateTime atStartOfDay() @@ -154,6 +156,7 @@ class LocalDate -> java.time.LocalDate extends Comparable,ChronoLocalDate,Tempor LocalDate plusMonths(long) LocalDate plusWeeks(long) LocalDate plusDays(long) + Period until(ChronoLocalDate) LocalDate with(TemporalAdjuster) LocalDate with(TemporalField,long) LocalDate withDayOfMonth(int) @@ -162,7 +165,7 @@ class LocalDate -> java.time.LocalDate extends Comparable,ChronoLocalDate,Tempor LocalDate withYear(int) } -class LocalDateTime -> java.time.LocalDateTime extends Comparable,ChronoLocalDateTime,Temporal,TemporalAccessor,TemporalAdjuster,Object { +class LocalDateTime -> java.time.LocalDateTime extends ChronoLocalDateTime,Temporal,TemporalAccessor,TemporalAdjuster,Comparable,Object { LocalDateTime MIN LocalDateTime MAX OffsetDateTime atOffset(ZoneOffset) @@ -208,6 +211,7 @@ class LocalDateTime -> java.time.LocalDateTime extends Comparable,ChronoLocalDat LocalDateTime plusSeconds(long) LocalDateTime plusWeeks(long) LocalDateTime plusYears(long) + LocalDate toLocalDate() LocalDateTime truncatedTo(TemporalUnit) LocalDateTime with(TemporalAdjuster) LocalDateTime with(TemporalField,long) @@ -220,13 +224,14 @@ class LocalDateTime -> java.time.LocalDateTime extends Comparable,ChronoLocalDat LocalDateTime withYear(int) } -class LocalTime -> java.time.LocalTime extends Comparable,Temporal,TemporalAccessor,TemporalAdjuster,Object { +class LocalTime -> java.time.LocalTime extends Temporal,TemporalAccessor,TemporalAdjuster,Comparable,Object { LocalTime MAX LocalTime MIDNIGHT LocalTime MIN LocalTime NOON LocalDateTime atDate(LocalDate) OffsetTime atOffset(ZoneOffset) + int compareTo(LocalTime) String format(DateTimeFormatter) LocalTime from(TemporalAccessor) int getHour() @@ -267,8 +272,9 @@ class LocalTime -> java.time.LocalTime extends Comparable,Temporal,TemporalAcces LocalTime withSecond(int) } -class MonthDay -> java.time.MonthDay extends Comparable,TemporalAccessor,TemporalAdjuster,Object { +class MonthDay -> java.time.MonthDay extends TemporalAccessor,TemporalAdjuster,Comparable,Object { LocalDate atYear(int) + int compareTo(MonthDay) String format(DateTimeFormatter) MonthDay from(TemporalAccessor) int getMonthValue() @@ -287,11 +293,12 @@ class MonthDay -> java.time.MonthDay extends Comparable,TemporalAccessor,Tempora MonthDay withMonth(int) } -class OffsetDateTime -> java.time.OffsetDateTime extends Comparable,Temporal,TemporalAccessor,TemporalAdjuster,Object { +class OffsetDateTime -> java.time.OffsetDateTime extends Temporal,TemporalAccessor,TemporalAdjuster,Comparable,Object { OffsetDateTime MAX OffsetDateTime MIN ZonedDateTime atZoneSameInstant(ZoneId) ZonedDateTime atZoneSimilarLocal(ZoneId) + int compareTo(OffsetDateTime) String format(DateTimeFormatter) OffsetDateTime from(TemporalAccessor) int getDayOfMonth() @@ -359,9 +366,10 @@ class OffsetDateTime -> java.time.OffsetDateTime extends Comparable,Temporal,Tem OffsetDateTime withOffsetSameInstant(ZoneOffset) } -class OffsetTime -> java.time.OffsetTime extends Comparable,Temporal,TemporalAccessor,TemporalAdjuster,Object { +class OffsetTime -> java.time.OffsetTime extends Temporal,TemporalAccessor,TemporalAdjuster,Comparable,Object { OffsetTime MAX OffsetTime MIN + int compareTo(OffsetTime) String format(DateTimeFormatter) OffsetTime from(TemporalAccessor) int getHour() @@ -434,12 +442,13 @@ class Period -> java.time.Period extends ChronoPeriod,TemporalAmount,Object { Period withYears(int) } -class Year -> java.time.Year extends Comparable,Temporal,TemporalAccessor,TemporalAdjuster,Object { +class Year -> java.time.Year extends Temporal,TemporalAccessor,TemporalAdjuster,Comparable,Object { int MAX_VALUE int MIN_VALUE LocalDate atDay(int) YearMonth atMonth(int) LocalDate atMonthDay(MonthDay) + int compareTo(Year) String format(DateTimeFormatter) Year from(TemporalAccessor) int getValue() @@ -463,9 +472,10 @@ class Year -> java.time.Year extends Comparable,Temporal,TemporalAccessor,Tempor Year with(TemporalField,long) } -class YearMonth -> java.time.YearMonth extends Comparable,Temporal,TemporalAccessor,TemporalAdjuster,Object { +class YearMonth -> java.time.YearMonth extends Temporal,TemporalAccessor,TemporalAdjuster,Comparable,Object { LocalDate atDay(int) LocalDate atEndOfMonth() + int compareTo(YearMonth) String format(DateTimeFormatter) YearMonth from(TemporalAccessor) Month getMonth() @@ -496,7 +506,7 @@ class YearMonth -> java.time.YearMonth extends Comparable,Temporal,TemporalAcces YearMonth withMonth(int) } -class ZonedDateTime -> java.time.ZonedDateTime extends Comparable,ChronoZonedDateTime,Temporal,TemporalAccessor,Object { +class ZonedDateTime -> java.time.ZonedDateTime extends ChronoZonedDateTime,Temporal,TemporalAccessor,Comparable,Object { int getDayOfMonth() DayOfWeek getDayOfWeek() int getDayOfYear() @@ -589,7 +599,7 @@ class ZoneOffset -> java.time.ZoneOffset extends ZoneId,Object { #### Enums -class DayOfWeek -> java.time.DayOfWeek extends Enum,Comparable,TemporalAccessor,TemporalAdjuster,Object { +class DayOfWeek -> java.time.DayOfWeek extends Enum,TemporalAccessor,TemporalAdjuster,Comparable,Object { DayOfWeek FRIDAY DayOfWeek MONDAY DayOfWeek SATURDAY @@ -607,7 +617,7 @@ class DayOfWeek -> java.time.DayOfWeek extends Enum,Comparable,TemporalAccessor, DayOfWeek[] values() } -class Month -> java.time.Month extends Enum,Comparable,TemporalAccessor,TemporalAdjuster,Object { +class Month -> java.time.Month extends Enum,TemporalAccessor,TemporalAdjuster,Comparable,Object { Month APRIL Month AUGUST Month DECEMBER diff --git a/modules/lang-painless/src/main/resources/org/elasticsearch/painless/java.time.zone.txt b/modules/lang-painless/src/main/resources/org/elasticsearch/painless/java.time.zone.txt index 036e2a734f9..d9d1cce5c10 100644 --- a/modules/lang-painless/src/main/resources/org/elasticsearch/painless/java.time.zone.txt +++ b/modules/lang-painless/src/main/resources/org/elasticsearch/painless/java.time.zone.txt @@ -25,6 +25,7 @@ #### Classes class ZoneOffsetTransition -> java.time.zone.ZoneOffsetTransition extends Comparable,Object { + int compareTo(ZoneOffsetTransition) LocalDateTime getDateTimeAfter() LocalDateTime getDateTimeBefore() Duration getDuration() diff --git a/modules/lang-painless/src/main/resources/org/elasticsearch/painless/java.util.txt b/modules/lang-painless/src/main/resources/org/elasticsearch/painless/java.util.txt index 5e240d0d45a..cccbf60f1a3 100644 --- a/modules/lang-painless/src/main/resources/org/elasticsearch/painless/java.util.txt +++ b/modules/lang-painless/src/main/resources/org/elasticsearch/painless/java.util.txt @@ -454,6 +454,7 @@ class Calendar -> java.util.Calendar extends Comparable,Object { void clear() void clear(int) def clone() + int compareTo(Calendar) int get(int) int getActualMaximum(int) int getActualMinimum(int) @@ -584,6 +585,7 @@ class Date -> java.util.Date extends Comparable,Object { boolean after(Date) boolean before(Date) def clone() + int compareTo(Date) Date from(Instant) long getTime() void setTime(long) @@ -999,6 +1001,7 @@ class TreeSet -> java.util.TreeSet extends AbstractSet,NavigableSet,SortedSet,Se class UUID -> java.util.UUID extends Comparable,Object { UUID (long,long) + int compareTo(UUID) int clockSequence() UUID fromString(String) long getLeastSignificantBits() @@ -1054,101 +1057,101 @@ class Locale.FilteringMode -> java.util.Locale$FilteringMode extends Enum,Compar #### Exceptions -class ConcurrentModificationException -> java.util.ConcurrentModificationException extends RuntimeException,Exception { +class ConcurrentModificationException -> java.util.ConcurrentModificationException extends RuntimeException,Exception,Object { ConcurrentModificationException () ConcurrentModificationException (String) } -class DuplicateFormatFlagsException -> java.util.DuplicateFormatFlagsException extends IllegalFormatException,IllegalArgumentException,RuntimeException,Exception { +class DuplicateFormatFlagsException -> java.util.DuplicateFormatFlagsException extends IllegalFormatException,IllegalArgumentException,RuntimeException,Exception,Object { DuplicateFormatFlagsException (String) String getFlags() } -class EmptyStackException -> java.util.EmptyStackException extends RuntimeException,Exception { +class EmptyStackException -> java.util.EmptyStackException extends RuntimeException,Exception,Object { EmptyStackException () } -class FormatFlagsConversionMismatchException -> java.util.FormatFlagsConversionMismatchException extends IllegalFormatException,IllegalArgumentException,RuntimeException,Exception { +class FormatFlagsConversionMismatchException -> java.util.FormatFlagsConversionMismatchException extends IllegalFormatException,IllegalArgumentException,RuntimeException,Exception,Object { FormatFlagsConversionMismatchException (String,char) char getConversion() String getFlags() } -class FormatterClosedException -> java.util.FormatterClosedException extends IllegalStateException,RuntimeException,Exception { +class FormatterClosedException -> java.util.FormatterClosedException extends IllegalStateException,RuntimeException,Exception,Object { FormatterClosedException () } -class IllegalFormatCodePointException -> java.util.IllegalFormatCodePointException extends IllegalFormatException,IllegalArgumentException,RuntimeException,Exception { +class IllegalFormatCodePointException -> java.util.IllegalFormatCodePointException extends IllegalFormatException,IllegalArgumentException,RuntimeException,Exception,Object { IllegalFormatCodePointException (int) int getCodePoint() } -class IllegalFormatConversionException -> java.util.IllegalFormatConversionException extends IllegalFormatException,IllegalArgumentException,RuntimeException,Exception { +class IllegalFormatConversionException -> java.util.IllegalFormatConversionException extends IllegalFormatException,IllegalArgumentException,RuntimeException,Exception,Object { char getConversion() } -class IllegalFormatException -> java.util.IllegalFormatException extends IllegalArgumentException,RuntimeException,Exception { +class IllegalFormatException -> java.util.IllegalFormatException extends IllegalArgumentException,RuntimeException,Exception,Object { } -class IllegalFormatFlagsException -> java.util.IllegalFormatFlagsException extends IllegalFormatException,IllegalArgumentException,RuntimeException,Exception { +class IllegalFormatFlagsException -> java.util.IllegalFormatFlagsException extends IllegalFormatException,IllegalArgumentException,RuntimeException,Exception,Object { IllegalFormatFlagsException (String) String getFlags() } -class IllegalFormatPrecisionException -> java.util.IllegalFormatPrecisionException extends IllegalFormatException,IllegalArgumentException,RuntimeException,Exception { +class IllegalFormatPrecisionException -> java.util.IllegalFormatPrecisionException extends IllegalFormatException,IllegalArgumentException,RuntimeException,Exception,Object { IllegalFormatPrecisionException (int) int getPrecision() } -class IllegalFormatWidthException -> java.util.IllegalFormatWidthException extends IllegalFormatException,IllegalArgumentException,RuntimeException,Exception { +class IllegalFormatWidthException -> java.util.IllegalFormatWidthException extends IllegalFormatException,IllegalArgumentException,RuntimeException,Exception,Object { IllegalFormatWidthException (int) int getWidth() } -class IllformedLocaleException -> java.util.IllformedLocaleException extends RuntimeException,Exception { +class IllformedLocaleException -> java.util.IllformedLocaleException extends RuntimeException,Exception,Object { IllformedLocaleException () IllformedLocaleException (String) IllformedLocaleException (String,int) int getErrorIndex() } -class InputMismatchException -> java.util.InputMismatchException extends NoSuchElementException,RuntimeException,Exception { +class InputMismatchException -> java.util.InputMismatchException extends NoSuchElementException,RuntimeException,Exception,Object { InputMismatchException () InputMismatchException (String) } -class MissingFormatArgumentException -> java.util.MissingFormatArgumentException extends IllegalFormatException,IllegalArgumentException,RuntimeException,Exception { +class MissingFormatArgumentException -> java.util.MissingFormatArgumentException extends IllegalFormatException,IllegalArgumentException,RuntimeException,Exception,Object { MissingFormatArgumentException (String) String getFormatSpecifier() } -class MissingFormatWidthException -> java.util.MissingFormatWidthException extends IllegalFormatException,IllegalArgumentException,RuntimeException,Exception { +class MissingFormatWidthException -> java.util.MissingFormatWidthException extends IllegalFormatException,IllegalArgumentException,RuntimeException,Exception,Object { MissingFormatWidthException (String) String getFormatSpecifier() } -class MissingResourceException -> java.util.MissingResourceException extends RuntimeException,Exception { +class MissingResourceException -> java.util.MissingResourceException extends RuntimeException,Exception,Object { MissingResourceException (String,String,String) String getClassName() String getKey() } -class NoSuchElementException -> java.util.NoSuchElementException extends RuntimeException,Exception { +class NoSuchElementException -> java.util.NoSuchElementException extends RuntimeException,Exception,Object { NoSuchElementException () NoSuchElementException (String) } -class TooManyListenersException -> java.util.TooManyListenersException extends Exception { +class TooManyListenersException -> java.util.TooManyListenersException extends Exception,Object { TooManyListenersException () TooManyListenersException (String) } -class UnknownFormatConversionException -> java.util.UnknownFormatConversionException extends IllegalFormatException,IllegalArgumentException,RuntimeException,Exception { +class UnknownFormatConversionException -> java.util.UnknownFormatConversionException extends IllegalFormatException,IllegalArgumentException,RuntimeException,Exception,Object { UnknownFormatConversionException (String) String getConversion() } -class UnknownFormatFlagsException -> java.util.UnknownFormatFlagsException extends IllegalFormatException,IllegalArgumentException,RuntimeException,Exception { +class UnknownFormatFlagsException -> java.util.UnknownFormatFlagsException extends IllegalFormatException,IllegalArgumentException,RuntimeException,Exception,Object { UnknownFormatFlagsException (String) String getFlags() } diff --git a/modules/lang-painless/src/main/resources/org/elasticsearch/painless/org.elasticsearch.txt b/modules/lang-painless/src/main/resources/org/elasticsearch/painless/org.elasticsearch.txt index 8627de26d26..7fff9ac0aa4 100644 --- a/modules/lang-painless/src/main/resources/org/elasticsearch/painless/org.elasticsearch.txt +++ b/modules/lang-painless/src/main/resources/org/elasticsearch/painless/org.elasticsearch.txt @@ -65,22 +65,26 @@ class org.elasticsearch.common.geo.GeoPoint -> org.elasticsearch.common.geo.GeoP } class org.elasticsearch.index.fielddata.ScriptDocValues.Strings -> org.elasticsearch.index.fielddata.ScriptDocValues$Strings extends List,Collection,Iterable,Object { + String get(int) String getValue() List getValues() } class org.elasticsearch.index.fielddata.ScriptDocValues.Longs -> org.elasticsearch.index.fielddata.ScriptDocValues$Longs extends List,Collection,Iterable,Object { + Long get(int) long getValue() List getValues() org.joda.time.ReadableDateTime getDate() } class org.elasticsearch.index.fielddata.ScriptDocValues.Doubles -> org.elasticsearch.index.fielddata.ScriptDocValues$Doubles extends List,Collection,Iterable,Object { + Double get(int) double getValue() List getValues() } class org.elasticsearch.index.fielddata.ScriptDocValues.GeoPoints -> org.elasticsearch.index.fielddata.ScriptDocValues$GeoPoints extends List,Collection,Iterable,Object { + org.elasticsearch.common.geo.GeoPoint get(int) org.elasticsearch.common.geo.GeoPoint getValue() List getValues() double getLat() diff --git a/modules/lang-painless/src/test/java/org/elasticsearch/painless/BasicAPITests.java b/modules/lang-painless/src/test/java/org/elasticsearch/painless/BasicAPITests.java index 46609a76173..885ce060aaa 100644 --- a/modules/lang-painless/src/test/java/org/elasticsearch/painless/BasicAPITests.java +++ b/modules/lang-painless/src/test/java/org/elasticsearch/painless/BasicAPITests.java @@ -19,7 +19,6 @@ package org.elasticsearch.painless; -import java.util.Arrays; import java.util.Collections; import java.util.HashMap; import java.util.Map; @@ -111,7 +110,13 @@ public class BasicAPITests extends ScriptTestCase { assertBytecodeExists("def x = 1D", "INVOKESTATIC java/lang/Double.valueOf (D)Ljava/lang/Double;"); } - void testStream() { - assertEquals(11, exec("params.list.stream().sum()", Collections.singletonMap("list", Arrays.asList(1,2,3,5)))); + public void testInterfaceDefaultMethods() { + assertEquals(1, exec("Map map = new HashMap(); return map.getOrDefault(5,1);")); + assertEquals(1, exec("def map = new HashMap(); return map.getOrDefault(5,1);")); + } + + public void testInterfacesHaveObject() { + assertEquals("{}", exec("Map map = new HashMap(); return map.toString();")); + assertEquals("{}", exec("def map = new HashMap(); return map.toString();")); } } diff --git a/modules/lang-painless/src/test/java/org/elasticsearch/painless/BasicStatementTests.java b/modules/lang-painless/src/test/java/org/elasticsearch/painless/BasicStatementTests.java index 0d6a54b515b..c392e86806e 100644 --- a/modules/lang-painless/src/test/java/org/elasticsearch/painless/BasicStatementTests.java +++ b/modules/lang-painless/src/test/java/org/elasticsearch/painless/BasicStatementTests.java @@ -125,6 +125,44 @@ public class BasicStatementTests extends ScriptTestCase { } } + public void testIterableForEachStatement() { + assertEquals(6, exec("List l = new ArrayList(); l.add(1); l.add(2); l.add(3); int total = 0;" + + " for (int x : l) total += x; return total")); + assertEquals("123", exec("List l = new ArrayList(); l.add('1'); l.add('2'); l.add('3'); String cat = '';" + + " for (String x : l) cat += x; return cat")); + assertEquals("1236", exec("Map m = new HashMap(); m.put('1', 1); m.put('2', 2); m.put('3', 3);" + + " String cat = ''; int total = 0;" + + " for (Map.Entry e : m.entrySet()) { cat += e.getKey(); total += e.getValue(); } return cat + total")); + } + + public void testIterableForEachStatementDef() { + assertEquals(6, exec("def l = new ArrayList(); l.add(1); l.add(2); l.add(3); int total = 0;" + + " for (int x : l) total += x; return total")); + assertEquals("123", exec("def l = new ArrayList(); l.add('1'); l.add('2'); l.add('3'); String cat = '';" + + " for (String x : l) cat += x; return cat")); + assertEquals("1236", exec("def m = new HashMap(); m.put('1', 1); m.put('2', 2); m.put('3', 3);" + + " String cat = ''; int total = 0;" + + " for (Map.Entry e : m.entrySet()) { cat += e.getKey(); total += e.getValue(); } return cat + total")); + } + + public void testArrayForEachStatement() { + assertEquals(6, exec("int[] a = new int[3]; a[0] = 1; a[1] = 2; a[2] = 3; int total = 0;" + + " for (int x : a) total += x; return total")); + assertEquals("123", exec("String[] a = new String[3]; a[0] = '1'; a[1] = '2'; a[2] = '3'; def total = '';" + + " for (String x : a) total += x; return total")); + assertEquals(6, exec("int[][] i = new int[3][1]; i[0][0] = 1; i[1][0] = 2; i[2][0] = 3; int total = 0;" + + " for (int[] j : i) total += j[0]; return total")); + } + + public void testArrayForEachStatementDef() { + assertEquals(6, exec("def a = new int[3]; a[0] = 1; a[1] = 2; a[2] = 3; int total = 0;" + + " for (int x : a) total += x; return total")); + assertEquals("123", exec("def a = new String[3]; a[0] = '1'; a[1] = '2'; a[2] = '3'; def total = '';" + + " for (String x : a) total += x; return total")); + assertEquals(6, exec("def i = new int[3][1]; i[0][0] = 1; i[1][0] = 2; i[2][0] = 3; int total = 0;" + + " for (int[] j : i) total += j[0]; return total")); + } + public void testDeclarationStatement() { assertEquals((byte)2, exec("byte a = 2; return a;")); assertEquals((short)2, exec("short a = 2; return a;")); diff --git a/modules/lang-painless/src/test/java/org/elasticsearch/painless/DefBootstrapTests.java b/modules/lang-painless/src/test/java/org/elasticsearch/painless/DefBootstrapTests.java index 4469279c8be..7a55844a224 100644 --- a/modules/lang-painless/src/test/java/org/elasticsearch/painless/DefBootstrapTests.java +++ b/modules/lang-painless/src/test/java/org/elasticsearch/painless/DefBootstrapTests.java @@ -33,7 +33,7 @@ public class DefBootstrapTests extends ESTestCase { CallSite site = DefBootstrap.bootstrap(MethodHandles.publicLookup(), "toString", MethodType.methodType(String.class, Object.class), - DefBootstrap.METHOD_CALL); + DefBootstrap.METHOD_CALL, 0); MethodHandle handle = site.dynamicInvoker(); assertDepthEquals(site, 0); @@ -50,7 +50,7 @@ public class DefBootstrapTests extends ESTestCase { CallSite site = DefBootstrap.bootstrap(MethodHandles.publicLookup(), "toString", MethodType.methodType(String.class, Object.class), - DefBootstrap.METHOD_CALL); + DefBootstrap.METHOD_CALL, 0); MethodHandle handle = site.dynamicInvoker(); assertDepthEquals(site, 0); @@ -72,7 +72,7 @@ public class DefBootstrapTests extends ESTestCase { CallSite site = DefBootstrap.bootstrap(MethodHandles.publicLookup(), "toString", MethodType.methodType(String.class, Object.class), - DefBootstrap.METHOD_CALL); + DefBootstrap.METHOD_CALL, 0); MethodHandle handle = site.dynamicInvoker(); assertDepthEquals(site, 0); diff --git a/modules/lang-painless/src/test/java/org/elasticsearch/painless/FunctionRefTests.java b/modules/lang-painless/src/test/java/org/elasticsearch/painless/FunctionRefTests.java index 20c48fe87d6..12265f161ab 100644 --- a/modules/lang-painless/src/test/java/org/elasticsearch/painless/FunctionRefTests.java +++ b/modules/lang-painless/src/test/java/org/elasticsearch/painless/FunctionRefTests.java @@ -20,11 +20,60 @@ package org.elasticsearch.painless; public class FunctionRefTests extends ScriptTestCase { - public void testUnsupported() { - expectScriptThrows(UnsupportedOperationException.class, () -> { - exec("DoubleStream.Builder builder = DoubleStream.builder();" + - "builder.add(2.0); builder.add(1.0); builder.add(3.0);" + - "builder.build().reduce(Double::unsupported);"); + + public void testStaticMethodReference() { + assertEquals(1, exec("List l = new ArrayList(); l.add(2); l.add(1); l.sort(Integer::compare); return l.get(0);")); + } + + public void testStaticMethodReferenceDef() { + assertEquals(1, exec("def l = new ArrayList(); l.add(2); l.add(1); l.sort(Integer::compare); return l.get(0);")); + } + + public void testVirtualMethodReference() { + assertEquals(2, exec("List l = new ArrayList(); l.add(1); l.add(1); return l.stream().mapToInt(Integer::intValue).sum();")); + } + + public void testVirtualMethodReferenceDef() { + assertEquals(2, exec("def l = new ArrayList(); l.add(1); l.add(1); return l.stream().mapToInt(Integer::intValue).sum();")); + } + + public void testCtorMethodReference() { + assertEquals(3.0D, + exec("List l = new ArrayList(); l.add(1.0); l.add(2.0); " + + "DoubleStream doubleStream = l.stream().mapToDouble(Double::doubleValue);" + + "DoubleSummaryStatistics stats = doubleStream.collect(DoubleSummaryStatistics::new, " + + "DoubleSummaryStatistics::accept, " + + "DoubleSummaryStatistics::combine); " + + "return stats.getSum()")); + } + + public void testCtorMethodReferenceDef() { + assertEquals(3.0D, + exec("def l = new ArrayList(); l.add(1.0); l.add(2.0); " + + "def doubleStream = l.stream().mapToDouble(Double::doubleValue);" + + "def stats = doubleStream.collect(DoubleSummaryStatistics::new, " + + "DoubleSummaryStatistics::accept, " + + "DoubleSummaryStatistics::combine); " + + "return stats.getSum()")); + } + + public void testMethodMissing() { + IllegalArgumentException expected = expectScriptThrows(IllegalArgumentException.class, () -> { + exec("List l = new ArrayList(); l.add(2); l.add(1); l.sort(Integer::bogus); return l.get(0);"); + }); + assertTrue(expected.getMessage().contains("Unknown reference")); + } + + public void testNotFunctionalInterface() { + IllegalArgumentException expected = expectScriptThrows(IllegalArgumentException.class, () -> { + exec("List l = new ArrayList(); l.add(2); l.add(1); l.add(Integer::bogus); return l.get(0);"); + }); + assertTrue(expected.getMessage().contains("Cannot convert function reference")); + } + + public void testIncompatible() { + expectScriptThrows(BootstrapMethodError.class, () -> { + exec("List l = new ArrayList(); l.add(2); l.add(1); l.sort(String::startsWith); return l.get(0);"); }); } } diff --git a/modules/lang-painless/src/test/java/org/elasticsearch/painless/OverloadTests.java b/modules/lang-painless/src/test/java/org/elasticsearch/painless/OverloadTests.java index ba095c05a9d..fce827e686c 100644 --- a/modules/lang-painless/src/test/java/org/elasticsearch/painless/OverloadTests.java +++ b/modules/lang-painless/src/test/java/org/elasticsearch/painless/OverloadTests.java @@ -37,7 +37,7 @@ public class OverloadTests extends ScriptTestCase { IllegalArgumentException expected = expectScriptThrows(IllegalArgumentException.class, () -> { exec("def x = 'abc123abc'; return x.indexOf('c', 3, 'bogus');"); }); - assertTrue(expected.getMessage().contains("dynamic method [indexOf] with signature [(String,int,String)")); + assertTrue(expected.getMessage().contains("dynamic method [indexOf]")); } public void testConstructor() { diff --git a/modules/percolator/src/test/java/org/elasticsearch/percolator/PercolatorIT.java b/modules/percolator/src/test/java/org/elasticsearch/percolator/PercolatorIT.java index ed9b65130a9..20c8c7f8b58 100644 --- a/modules/percolator/src/test/java/org/elasticsearch/percolator/PercolatorIT.java +++ b/modules/percolator/src/test/java/org/elasticsearch/percolator/PercolatorIT.java @@ -26,6 +26,7 @@ import org.elasticsearch.action.admin.indices.alias.Alias; import org.elasticsearch.action.admin.indices.alias.IndicesAliasesResponse; import org.elasticsearch.action.search.SearchResponse; import org.elasticsearch.action.support.IndicesOptions; +import org.elasticsearch.action.support.WriteRequest.RefreshPolicy; import org.elasticsearch.client.Requests; import org.elasticsearch.common.geo.builders.ShapeBuilders; import org.elasticsearch.common.lucene.search.function.CombineFunction; @@ -294,8 +295,8 @@ public class PercolatorIT extends ESIntegTestCase { .field("color", "blue") .field("query", termQuery("field1", "value1")) .endObject()) - .setRefresh(true) - .execute().actionGet(); + .setRefreshPolicy(RefreshPolicy.IMMEDIATE) + .get(); cluster().wipeIndices("test"); createIndex("test"); @@ -308,8 +309,8 @@ public class PercolatorIT extends ESIntegTestCase { .field("color", "blue") .field("query", termQuery("field1", "value1")) .endObject()) - .setRefresh(true) - .execute().actionGet(); + .setRefreshPolicy(RefreshPolicy.IMMEDIATE) + .get(); } // see #2814 @@ -338,8 +339,8 @@ public class PercolatorIT extends ESIntegTestCase { .field("source", "productizer") .field("query", QueryBuilders.constantScoreQuery(QueryBuilders.queryStringQuery("filingcategory:s"))) .endObject()) - .setRefresh(true) - .execute().actionGet(); + .setRefreshPolicy(RefreshPolicy.IMMEDIATE) + .get(); refresh(); PercolateResponse percolate = preparePercolate(client()) @@ -417,8 +418,8 @@ public class PercolatorIT extends ESIntegTestCase { .field("color", "blue") .field("query", termQuery("field1", "value1")) .endObject()) - .setRefresh(true) - .execute().actionGet(); + .setRefreshPolicy(RefreshPolicy.IMMEDIATE) + .get(); logger.info("--> register a query 2"); client().prepareIndex(INDEX_NAME, TYPE_NAME, "bubu") @@ -426,8 +427,8 @@ public class PercolatorIT extends ESIntegTestCase { .field("color", "green") .field("query", termQuery("field1", "value2")) .endObject()) - .setRefresh(true) - .execute().actionGet(); + .setRefreshPolicy(RefreshPolicy.IMMEDIATE) + .get(); PercolateResponse percolate = preparePercolate(client()) .setIndices(INDEX_NAME).setDocumentType("type1") @@ -461,8 +462,8 @@ public class PercolatorIT extends ESIntegTestCase { .field("color", "blue") .field("query", termQuery("field1", "value1")) .endObject()) - .setRefresh(true) - .execute().actionGet(); + .setRefreshPolicy(RefreshPolicy.IMMEDIATE) + .get(); PercolateResponse percolate = preparePercolate(client()) .setIndices(INDEX_NAME).setDocumentType("type1") @@ -478,8 +479,8 @@ public class PercolatorIT extends ESIntegTestCase { .field("color", "green") .field("query", termQuery("field1", "value2")) .endObject()) - .setRefresh(true) - .execute().actionGet(); + .setRefreshPolicy(RefreshPolicy.IMMEDIATE) + .get(); percolate = preparePercolate(client()) .setIndices(INDEX_NAME).setDocumentType("type1") @@ -495,8 +496,8 @@ public class PercolatorIT extends ESIntegTestCase { .field("color", "red") .field("query", termQuery("field1", "value2")) .endObject()) - .setRefresh(true) - .execute().actionGet(); + .setRefreshPolicy(RefreshPolicy.IMMEDIATE) + .get(); PercolateSourceBuilder sourceBuilder = new PercolateSourceBuilder() .setDoc(docBuilder().setDoc(jsonBuilder().startObject().field("field1", "value2").endObject())) @@ -510,7 +511,7 @@ public class PercolatorIT extends ESIntegTestCase { assertThat(convertFromTextArray(percolate.getMatches(), INDEX_NAME), arrayContaining("susu")); logger.info("--> deleting query 1"); - client().prepareDelete(INDEX_NAME, TYPE_NAME, "kuku").setRefresh(true).execute().actionGet(); + client().prepareDelete(INDEX_NAME, TYPE_NAME, "kuku").setRefreshPolicy(RefreshPolicy.IMMEDIATE).get(); percolate = preparePercolate(client()) .setIndices(INDEX_NAME).setDocumentType("type1") @@ -1461,8 +1462,8 @@ public class PercolatorIT extends ESIntegTestCase { .must(QueryBuilders.queryStringQuery("root")) .must(QueryBuilders.termQuery("message", "tree")))) .endObject()) - .setRefresh(true) - .execute().actionGet(); + .setRefreshPolicy(RefreshPolicy.IMMEDIATE) + .get(); refresh(); PercolateResponse percolate = preparePercolate(client()) diff --git a/modules/reindex/src/main/java/org/elasticsearch/index/reindex/AbstractAsyncBulkByScrollAction.java b/modules/reindex/src/main/java/org/elasticsearch/index/reindex/AbstractAsyncBulkByScrollAction.java index 0921abcb677..ceb00b2f81f 100644 --- a/modules/reindex/src/main/java/org/elasticsearch/index/reindex/AbstractAsyncBulkByScrollAction.java +++ b/modules/reindex/src/main/java/org/elasticsearch/index/reindex/AbstractAsyncBulkByScrollAction.java @@ -55,7 +55,6 @@ import java.util.Iterator; import java.util.List; import java.util.Set; import java.util.concurrent.ConcurrentHashMap; -import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicLong; import java.util.concurrent.atomic.AtomicReference; import java.util.function.Consumer; @@ -66,7 +65,6 @@ import static java.util.Collections.emptyList; import static java.util.Collections.unmodifiableList; import static org.elasticsearch.action.bulk.BackoffPolicy.exponentialBackoff; import static org.elasticsearch.common.unit.TimeValue.timeValueNanos; -import static org.elasticsearch.common.unit.TimeValue.timeValueSeconds; import static org.elasticsearch.index.reindex.AbstractBulkByScrollRequest.SIZE_ALL_MATCHES; import static org.elasticsearch.rest.RestStatus.CONFLICT; import static org.elasticsearch.search.sort.SortBuilders.fieldSort; @@ -85,7 +83,6 @@ public abstract class AbstractAsyncBulkByScrollAction scroll = new AtomicReference<>(); - private final AtomicLong lastBatchStartTime = new AtomicLong(-1); private final Set destinationIndices = Collections.newSetFromMap(new ConcurrentHashMap<>()); private final ESLogger logger; @@ -147,16 +144,17 @@ public abstract class AbstractAsyncBulkByScrollAction client.search(firstSearchRequest, listener), (SearchResponse response) -> { logger.debug("[{}] documents match query", response.getHits().getTotalHits()); - onScrollResponse(timeValueSeconds(0), response); + onScrollResponse(timeValueNanos(System.nanoTime()), 0, response); }); } /** * Process a scroll response. - * @param delay how long to delay processesing the response. This delay is how throttling is applied to the action. + * @param lastBatchStartTime the time when the last batch started. Used to calculate the throttling delay. + * @param lastBatchSize the size of the last batch. Used to calculate the throttling delay. * @param searchResponse the scroll response to process */ - void onScrollResponse(TimeValue delay, SearchResponse searchResponse) { + void onScrollResponse(TimeValue lastBatchStartTime, int lastBatchSize, SearchResponse searchResponse) { if (task.isCancelled()) { finishHim(null); return; @@ -179,7 +177,11 @@ public abstract class AbstractAsyncBulkByScrollAction() { @Override public void onResponse(BulkResponse response) { - onBulkResponse(response); + onBulkResponse(thisBatchStartTime, response); } @Override @@ -258,7 +259,7 @@ public abstract class AbstractAsyncBulkByScrollAction failures = new ArrayList(); Set destinationIndicesThisBatch = new HashSet<>(); @@ -306,7 +307,7 @@ public abstract class AbstractAsyncBulkByScrollAction client.searchScroll(request, listener), (SearchResponse response) -> { - onScrollResponse(timeValueNanos(max(0, earliestNextBatchStartTime - System.nanoTime())), response); + onScrollResponse(lastBatchStartTime, lastBatchSize, response); }); } - /** - * How many nanoseconds should a batch of lastBatchSize have taken if it were perfectly throttled? Package private for testing. - */ - float perfectlyThrottledBatchTime(int lastBatchSize) { - if (task.getRequestsPerSecond() == Float.POSITIVE_INFINITY) { - return 0; - } - // requests - // ------------------- == seconds - // request per seconds - float targetBatchTimeInSeconds = lastBatchSize / task.getRequestsPerSecond(); - // nanoseconds per seconds * seconds == nanoseconds - return TimeUnit.SECONDS.toNanos(1) * targetBatchTimeInSeconds; - } - private void recordFailure(Failure failure, List failures) { if (failure.getStatus() == CONFLICT) { task.countVersionConflict(); @@ -453,20 +438,6 @@ public abstract class AbstractAsyncBulkByScrollAction resultCtx = (Map) executable.unwrap(context); String newOp = (String) resultCtx.remove("op"); if (newOp == null) { - throw new IllegalArgumentException("Script cleared op!"); - } - if ("noop".equals(newOp)) { - task.countNoop(); - return null; - } - if (false == "update".equals(newOp)) { - throw new IllegalArgumentException("Invalid op [" + newOp + ']'); + throw new IllegalArgumentException("Script cleared operation type"); } /* @@ -468,12 +538,35 @@ public abstract class AbstractAsyncBulkIndexByScrollAction scriptChangedOpType(RequestWrapper request, OpType oldOpType, OpType newOpType) { + switch (newOpType) { + case NOOP: + task.countNoop(); + return null; + case DELETE: + RequestWrapper delete = wrap(new DeleteRequest(request.getIndex(), request.getType(), request.getId())); + delete.setVersion(request.getVersion()); + delete.setVersionType(VersionType.INTERNAL); + delete.setParent(request.getParent()); + delete.setRouting(request.getRouting()); + return delete; + default: + throw new IllegalArgumentException("Unsupported operation type change from [" + oldOpType + "] to [" + newOpType + "]"); + } + } + protected abstract void scriptChangedIndex(RequestWrapper request, Object to); protected abstract void scriptChangedType(RequestWrapper request, Object to); @@ -489,5 +582,39 @@ public abstract class AbstractAsyncBulkIndexByScrollAction request, Object to); protected abstract void scriptChangedTTL(RequestWrapper request, Object to); + + } + + public enum OpType { + + NOOP("noop"), + INDEX("index"), + DELETE("delete"); + + private final String id; + + OpType(String id) { + this.id = id; + } + + public static OpType fromString(String opType) { + String lowerOpType = opType.toLowerCase(Locale.ROOT); + switch (lowerOpType) { + case "noop": + return OpType.NOOP; + case "index": + return OpType.INDEX; + case "delete": + return OpType.DELETE; + default: + throw new IllegalArgumentException("Operation type [" + lowerOpType + "] not allowed, only " + + Arrays.toString(values()) + " are allowed"); + } + } + + @Override + public String toString() { + return id.toLowerCase(Locale.ROOT); + } } } diff --git a/modules/reindex/src/main/java/org/elasticsearch/index/reindex/AbstractBaseReindexRestHandler.java b/modules/reindex/src/main/java/org/elasticsearch/index/reindex/AbstractBaseReindexRestHandler.java index e78a6a9c350..3aea4dbce8a 100644 --- a/modules/reindex/src/main/java/org/elasticsearch/index/reindex/AbstractBaseReindexRestHandler.java +++ b/modules/reindex/src/main/java/org/elasticsearch/index/reindex/AbstractBaseReindexRestHandler.java @@ -64,7 +64,7 @@ public abstract class AbstractBaseReindexRestHandler< } protected void handleRequest(RestRequest request, RestChannel channel, - boolean includeCreated, boolean includeUpdated, boolean includeDeleted) throws IOException { + boolean includeCreated, boolean includeUpdated) throws IOException { // Build the internal request Request internal = setCommonOptions(request, buildRequest(request)); @@ -74,7 +74,6 @@ public abstract class AbstractBaseReindexRestHandler< Map params = new HashMap<>(); params.put(BulkByScrollTask.Status.INCLUDE_CREATED, Boolean.toString(includeCreated)); params.put(BulkByScrollTask.Status.INCLUDE_UPDATED, Boolean.toString(includeUpdated)); - params.put(BulkByScrollTask.Status.INCLUDE_DELETED, Boolean.toString(includeDeleted)); action.execute(internal, new BulkIndexByScrollResponseContentListener<>(channel, params)); return; diff --git a/modules/reindex/src/main/java/org/elasticsearch/index/reindex/BulkByScrollTask.java b/modules/reindex/src/main/java/org/elasticsearch/index/reindex/BulkByScrollTask.java index 40ee492b4a9..7af32740be2 100644 --- a/modules/reindex/src/main/java/org/elasticsearch/index/reindex/BulkByScrollTask.java +++ b/modules/reindex/src/main/java/org/elasticsearch/index/reindex/BulkByScrollTask.java @@ -49,7 +49,7 @@ import static org.elasticsearch.common.unit.TimeValue.timeValueNanos; * Task storing information about a currently running BulkByScroll request. */ public class BulkByScrollTask extends CancellableTask { - private static final ESLogger logger = ESLoggerFactory.getLogger(BulkByScrollTask.class.getName()); + private static final ESLogger logger = ESLoggerFactory.getLogger(BulkByScrollTask.class.getPackage().getName()); /** * The total number of documents this request will process. 0 means we don't yet know or, possibly, there are actually 0 documents @@ -126,12 +126,6 @@ public class BulkByScrollTask extends CancellableTask { */ public static final String INCLUDE_UPDATED = "include_updated"; - /** - * XContent param name to indicate if "deleted" count must be included - * in the response. - */ - public static final String INCLUDE_DELETED = "include_deleted"; - private final long total; private final long updated; private final long created; @@ -213,9 +207,7 @@ public class BulkByScrollTask extends CancellableTask { if (params.paramAsBoolean(INCLUDE_CREATED, true)) { builder.field("created", created); } - if (params.paramAsBoolean(INCLUDE_DELETED, true)) { - builder.field("deleted", deleted); - } + builder.field("deleted", deleted); builder.field("batches", batches); builder.field("version_conflicts", versionConflicts); builder.field("noops", noops); @@ -411,29 +403,36 @@ public class BulkByScrollTask extends CancellableTask { * Schedule prepareBulkRequestRunnable to run after some delay. This is where throttling plugs into reindexing so the request can be * rescheduled over and over again. */ - void delayPrepareBulkRequest(ThreadPool threadPool, TimeValue delay, AbstractRunnable prepareBulkRequestRunnable) { + void delayPrepareBulkRequest(ThreadPool threadPool, TimeValue lastBatchStartTime, int lastBatchSize, + AbstractRunnable prepareBulkRequestRunnable) { // Synchronize so we are less likely to schedule the same request twice. synchronized (delayedPrepareBulkRequestReference) { - AbstractRunnable oneTime = new AbstractRunnable() { - private final AtomicBoolean hasRun = new AtomicBoolean(false); - - @Override - protected void doRun() throws Exception { - // Paranoia to prevent furiously rethrottling from running the command multiple times. Without this we totally can. - if (hasRun.compareAndSet(false, true)) { - prepareBulkRequestRunnable.run(); - } - } - - @Override - public void onFailure(Throwable t) { - prepareBulkRequestRunnable.onFailure(t); - } - }; - delayedPrepareBulkRequestReference.set(new DelayedPrepareBulkRequest(threadPool, getRequestsPerSecond(), delay, oneTime)); + TimeValue delay = throttleWaitTime(lastBatchStartTime, lastBatchSize); + delayedPrepareBulkRequestReference.set(new DelayedPrepareBulkRequest(threadPool, getRequestsPerSecond(), + delay, new RunOnce(prepareBulkRequestRunnable))); } } + TimeValue throttleWaitTime(TimeValue lastBatchStartTime, int lastBatchSize) { + long earliestNextBatchStartTime = lastBatchStartTime.nanos() + (long) perfectlyThrottledBatchTime(lastBatchSize); + return timeValueNanos(max(0, earliestNextBatchStartTime - System.nanoTime())); + } + + /** + * How many nanoseconds should a batch of lastBatchSize have taken if it were perfectly throttled? Package private for testing. + */ + float perfectlyThrottledBatchTime(int lastBatchSize) { + if (requestsPerSecond == Float.POSITIVE_INFINITY) { + return 0; + } + // requests + // ------------------- == seconds + // request per seconds + float targetBatchTimeInSeconds = lastBatchSize / requestsPerSecond; + // nanoseconds per seconds * seconds == nanoseconds + return TimeUnit.SECONDS.toNanos(1) * targetBatchTimeInSeconds; + } + private void setRequestsPerSecond(float requestsPerSecond) { this.requestsPerSecond = requestsPerSecond; } @@ -526,4 +525,29 @@ public class BulkByScrollTask extends CancellableTask { return timeValueNanos(round(remainingDelay * requestsPerSecond / newRequestsPerSecond)); } } + + /** + * Runnable that can only be run one time. This is paranoia to prevent furiously rethrottling from running the command multiple times. + * Without it the command would be run multiple times. + */ + private static class RunOnce extends AbstractRunnable { + private final AtomicBoolean hasRun = new AtomicBoolean(false); + private final AbstractRunnable delegate; + + public RunOnce(AbstractRunnable delegate) { + this.delegate = delegate; + } + + @Override + protected void doRun() throws Exception { + if (hasRun.compareAndSet(false, true)) { + delegate.run(); + } + } + + @Override + public void onFailure(Throwable t) { + delegate.onFailure(t); + } + } } diff --git a/modules/reindex/src/main/java/org/elasticsearch/index/reindex/RestDeleteByQueryAction.java b/modules/reindex/src/main/java/org/elasticsearch/index/reindex/RestDeleteByQueryAction.java index 4750fe22313..bb894584c8b 100644 --- a/modules/reindex/src/main/java/org/elasticsearch/index/reindex/RestDeleteByQueryAction.java +++ b/modules/reindex/src/main/java/org/elasticsearch/index/reindex/RestDeleteByQueryAction.java @@ -55,7 +55,7 @@ public class RestDeleteByQueryAction extends AbstractBulkByQueryRestHandler> scriptBody) { + @SuppressWarnings("unchecked") + protected > T applyScript(Consumer> scriptBody) { IndexRequest index = new IndexRequest("index", "type", "1").source(singletonMap("foo", "bar")); Map fields = new HashMap<>(); InternalSearchHit doc = new InternalSearchHit(0, "id", new Text("type"), fields); @@ -66,8 +71,8 @@ public abstract class AbstractAsyncBulkIndexByScrollActionScriptTestCase< when(scriptService.executable(any(CompiledScript.class), Matchers.>any())) .thenReturn(executableScript); AbstractAsyncBulkIndexByScrollAction action = action(scriptService, request().setScript(EMPTY_SCRIPT)); - action.buildScriptApplier().apply(AbstractAsyncBulkIndexByScrollAction.wrap(index), doc); - return index; + RequestWrapper result = action.buildScriptApplier().apply(AbstractAsyncBulkIndexByScrollAction.wrap(index), doc); + return (result != null) ? (T) result.self() : null; } public void testScriptAddingJunkToCtxIsError() { @@ -88,5 +93,24 @@ public abstract class AbstractAsyncBulkIndexByScrollActionScriptTestCase< assertEquals("cat", index.sourceAsMap().get("bar")); } + public void testSetOpTypeNoop() throws Exception { + assertThat(task.getStatus().getNoops(), equalTo(0L)); + assertNull(applyScript((Map ctx) -> ctx.put("op", OpType.NOOP.toString()))); + assertThat(task.getStatus().getNoops(), equalTo(1L)); + } + + public void testSetOpTypeDelete() throws Exception { + DeleteRequest delete = applyScript((Map ctx) -> ctx.put("op", OpType.DELETE.toString())); + assertThat(delete.index(), equalTo("index")); + assertThat(delete.type(), equalTo("type")); + assertThat(delete.id(), equalTo("1")); + } + + public void testSetOpTypeUnknown() throws Exception { + IllegalArgumentException e = expectThrows(IllegalArgumentException.class, + () -> applyScript((Map ctx) -> ctx.put("op", "unknown"))); + assertThat(e.getMessage(), equalTo("Operation type [unknown] not allowed, only [noop, index, delete] are allowed")); + } + protected abstract AbstractAsyncBulkIndexByScrollAction action(ScriptService scriptService, Request request); } diff --git a/modules/reindex/src/test/java/org/elasticsearch/index/reindex/AbstractAsyncBulkIndexByScrollActionTestCase.java b/modules/reindex/src/test/java/org/elasticsearch/index/reindex/AbstractAsyncBulkIndexByScrollActionTestCase.java index b9489e9f5d9..66c636f4f1d 100644 --- a/modules/reindex/src/test/java/org/elasticsearch/index/reindex/AbstractAsyncBulkIndexByScrollActionTestCase.java +++ b/modules/reindex/src/test/java/org/elasticsearch/index/reindex/AbstractAsyncBulkIndexByScrollActionTestCase.java @@ -22,6 +22,7 @@ package org.elasticsearch.index.reindex; import org.elasticsearch.action.support.PlainActionFuture; import org.elasticsearch.tasks.TaskId; import org.elasticsearch.test.ESTestCase; +import org.elasticsearch.threadpool.TestThreadPool; import org.elasticsearch.threadpool.ThreadPool; import org.junit.After; import org.junit.Before; @@ -35,7 +36,7 @@ public abstract class AbstractAsyncBulkIndexByScrollActionTestCase< @Before public void setupForTest() { - threadPool = new ThreadPool(getTestName()); + threadPool = new TestThreadPool(getTestName()); task = new BulkByScrollTask(1, "test", "test", "test", TaskId.EMPTY_TASK_ID, 0); } diff --git a/modules/reindex/src/test/java/org/elasticsearch/index/reindex/AsyncBulkByScrollActionTests.java b/modules/reindex/src/test/java/org/elasticsearch/index/reindex/AsyncBulkByScrollActionTests.java index 9ce505bacd3..9b094a9e2d3 100644 --- a/modules/reindex/src/test/java/org/elasticsearch/index/reindex/AsyncBulkByScrollActionTests.java +++ b/modules/reindex/src/test/java/org/elasticsearch/index/reindex/AsyncBulkByScrollActionTests.java @@ -72,6 +72,7 @@ import org.elasticsearch.tasks.TaskId; import org.elasticsearch.tasks.TaskManager; import org.elasticsearch.test.ESTestCase; import org.elasticsearch.test.client.NoOpClient; +import org.elasticsearch.threadpool.TestThreadPool; import org.elasticsearch.threadpool.ThreadPool; import org.junit.After; import org.junit.Before; @@ -96,19 +97,18 @@ import static java.util.Collections.emptySet; import static java.util.Collections.singleton; import static org.apache.lucene.util.TestUtil.randomSimpleString; import static org.elasticsearch.action.bulk.BackoffPolicy.constantBackoff; -import static org.elasticsearch.common.unit.TimeValue.parseTimeValue; import static org.elasticsearch.common.unit.TimeValue.timeValueMillis; -import static org.elasticsearch.common.unit.TimeValue.timeValueMinutes; +import static org.elasticsearch.common.unit.TimeValue.timeValueNanos; import static org.elasticsearch.common.unit.TimeValue.timeValueSeconds; -import static org.hamcrest.Matchers.closeTo; import static org.hamcrest.Matchers.contains; import static org.hamcrest.Matchers.containsString; import static org.hamcrest.Matchers.either; import static org.hamcrest.Matchers.emptyCollectionOf; import static org.hamcrest.Matchers.equalTo; -import static org.hamcrest.Matchers.greaterThanOrEqualTo; +import static org.hamcrest.Matchers.greaterThan; import static org.hamcrest.Matchers.hasSize; import static org.hamcrest.Matchers.instanceOf; +import static org.hamcrest.Matchers.lessThanOrEqualTo; public class AsyncBulkByScrollActionTests extends ESTestCase { private MyMockClient client; @@ -126,7 +126,7 @@ public class AsyncBulkByScrollActionTests extends ESTestCase { @Before public void setupForTest() { client = new MyMockClient(new NoOpClient(getTestName())); - threadPool = new ThreadPool(getTestName()); + threadPool = new TestThreadPool(getTestName()); firstSearchRequest = new SearchRequest(); testRequest = new DummyAbstractBulkByScrollRequest(firstSearchRequest); listener = new PlainActionFuture<>(); @@ -188,7 +188,7 @@ public class AsyncBulkByScrollActionTests extends ESTestCase { client.scrollsToReject = randomIntBetween(0, testRequest.getMaxRetries() - 1); DummyAbstractAsyncBulkByScrollAction action = new DummyActionWithoutBackoff(); action.setScroll(scrollId()); - action.startNextScroll(0); + action.startNextScroll(timeValueNanos(System.nanoTime()), 0); assertBusy(() -> assertEquals(client.scrollsToReject + 1, client.scrollAttempts.get())); if (listener.isDone()) { Object result = listener.get(); @@ -202,7 +202,7 @@ public class AsyncBulkByScrollActionTests extends ESTestCase { client.scrollsToReject = testRequest.getMaxRetries() + randomIntBetween(1, 100); DummyAbstractAsyncBulkByScrollAction action = new DummyActionWithoutBackoff(); action.setScroll(scrollId()); - action.startNextScroll(0); + action.startNextScroll(timeValueNanos(System.nanoTime()), 0); assertBusy(() -> assertEquals(testRequest.getMaxRetries() + 1, client.scrollAttempts.get())); assertBusy(() -> assertTrue(listener.isDone())); ExecutionException e = expectThrows(ExecutionException.class, () -> listener.get()); @@ -218,7 +218,7 @@ public class AsyncBulkByScrollActionTests extends ESTestCase { long total = randomIntBetween(0, Integer.MAX_VALUE); InternalSearchHits hits = new InternalSearchHits(null, total, 0); InternalSearchResponse searchResponse = new InternalSearchResponse(hits, null, null, null, false, false); - new DummyAbstractAsyncBulkByScrollAction().onScrollResponse(timeValueSeconds(0), + new DummyAbstractAsyncBulkByScrollAction().onScrollResponse(timeValueSeconds(0), 0, new SearchResponse(searchResponse, scrollId(), 5, 4, randomLong(), null)); assertEquals(total, testTask.getStatus().getTotal()); } @@ -229,24 +229,17 @@ public class AsyncBulkByScrollActionTests extends ESTestCase { public void testScrollResponseBatchingBehavior() throws Exception { int maxBatches = randomIntBetween(0, 100); for (int batches = 1; batches < maxBatches; batches++) { - long now = System.nanoTime(); InternalSearchHit hit = new InternalSearchHit(0, "id", new Text("type"), emptyMap()); InternalSearchHits hits = new InternalSearchHits(new InternalSearchHit[] { hit }, 0, 0); InternalSearchResponse searchResponse = new InternalSearchResponse(hits, null, null, null, false, false); DummyAbstractAsyncBulkByScrollAction action = new DummyAbstractAsyncBulkByScrollAction(); - action.onScrollResponse(timeValueSeconds(0), + action.onScrollResponse(timeValueNanos(System.nanoTime()), 0, new SearchResponse(searchResponse, scrollId(), 5, 4, randomLong(), null)); // Use assert busy because the update happens on another thread final int expectedBatches = batches; assertBusy(() -> assertEquals(expectedBatches, testTask.getStatus().getBatches())); - /* - * While we're here we can check that getting a scroll response sets the last scroll start time which makes sure the wait time - * isn't counted as time that the last batch took. - */ - assertThat(action.getLastBatchStartTime(), greaterThanOrEqualTo(now)); - /* * Also while we're here check that we preserved the headers from the last request. assertBusy because no requests might have * come in yet. @@ -296,7 +289,7 @@ public class AsyncBulkByScrollActionTests extends ESTestCase { } responses[i] = new BulkItemResponse(i, opType, new IndexResponse(shardId, "type", "id" + i, randomInt(), createdResponse)); } - new DummyAbstractAsyncBulkByScrollAction().onBulkResponse(new BulkResponse(responses, 0)); + new DummyAbstractAsyncBulkByScrollAction().onBulkResponse(timeValueNanos(System.nanoTime()), new BulkResponse(responses, 0)); assertEquals(versionConflicts, testTask.getStatus().getVersionConflicts()); assertEquals(updated, testTask.getStatus().getUpdated()); assertEquals(created, testTask.getStatus().getCreated()); @@ -309,20 +302,22 @@ public class AsyncBulkByScrollActionTests extends ESTestCase { * Mimicks a ThreadPool rejecting execution of the task. */ public void testThreadPoolRejectionsAbortRequest() throws Exception { - TimeValue expectedDelay = parseTimeValue(randomPositiveTimeValue(), "test"); + testTask.rethrottle(1); threadPool.shutdown(); - threadPool = new ThreadPool(getTestName()) { + threadPool = new TestThreadPool(getTestName()) { @Override public ScheduledFuture schedule(TimeValue delay, String name, Runnable command) { - assertEquals(expectedDelay, delay); // While we're here we can check that the sleep made it through + // While we're here we can check that the sleep made it through + assertThat(delay.nanos(), greaterThan(0L)); + assertThat(delay.seconds(), lessThanOrEqualTo(10L)); ((AbstractRunnable) command).onRejection(new EsRejectedExecutionException("test")); return null; } }; InternalSearchHits hits = new InternalSearchHits(null, 0, 0); InternalSearchResponse searchResponse = new InternalSearchResponse(hits, null, null, null, false, false); - new DummyAbstractAsyncBulkByScrollAction() - .onScrollResponse(expectedDelay, new SearchResponse(searchResponse, scrollId(), 5, 4, randomLong(), null)); + new DummyAbstractAsyncBulkByScrollAction().onScrollResponse(timeValueNanos(System.nanoTime()), 10, + new SearchResponse(searchResponse, scrollId(), 5, 4, randomLong(), null)); try { listener.get(); fail("Expected a failure"); @@ -342,7 +337,7 @@ public class AsyncBulkByScrollActionTests extends ESTestCase { public void testShardFailuresAbortRequest() throws Exception { ShardSearchFailure shardFailure = new ShardSearchFailure(new RuntimeException("test")); InternalSearchResponse internalResponse = new InternalSearchResponse(null, null, null, null, false, null); - new DummyAbstractAsyncBulkByScrollAction().onScrollResponse(timeValueSeconds(0), + new DummyAbstractAsyncBulkByScrollAction().onScrollResponse(timeValueNanos(System.nanoTime()), 0, new SearchResponse(internalResponse, scrollId(), 5, 4, randomLong(), new ShardSearchFailure[] { shardFailure })); BulkIndexByScrollResponse response = listener.get(); assertThat(response.getIndexingFailures(), emptyCollectionOf(Failure.class)); @@ -357,7 +352,7 @@ public class AsyncBulkByScrollActionTests extends ESTestCase { */ public void testSearchTimeoutsAbortRequest() throws Exception { InternalSearchResponse internalResponse = new InternalSearchResponse(null, null, null, null, true, null); - new DummyAbstractAsyncBulkByScrollAction().onScrollResponse(timeValueSeconds(0), + new DummyAbstractAsyncBulkByScrollAction().onScrollResponse(timeValueNanos(System.nanoTime()), 0, new SearchResponse(internalResponse, scrollId(), 5, 4, randomLong(), new ShardSearchFailure[0])); BulkIndexByScrollResponse response = listener.get(); assertThat(response.getIndexingFailures(), emptyCollectionOf(Failure.class)); @@ -367,14 +362,14 @@ public class AsyncBulkByScrollActionTests extends ESTestCase { assertThat(client.scrollsCleared, contains(scrollId)); } - /** * Mimicks bulk indexing failures. */ public void testBulkFailuresAbortRequest() throws Exception { Failure failure = new Failure("index", "type", "id", new RuntimeException("test")); DummyAbstractAsyncBulkByScrollAction action = new DummyAbstractAsyncBulkByScrollAction(); - action.onBulkResponse(new BulkResponse(new BulkItemResponse[] {new BulkItemResponse(0, "index", failure)}, randomLong())); + BulkResponse bulkResponse = new BulkResponse(new BulkItemResponse[] {new BulkItemResponse(0, "index", failure)}, randomLong()); + action.onBulkResponse(timeValueNanos(System.nanoTime()), bulkResponse); BulkIndexByScrollResponse response = listener.get(); assertThat(response.getIndexingFailures(), contains(failure)); assertThat(response.getSearchFailures(), emptyCollectionOf(ShardSearchFailure.class)); @@ -393,15 +388,12 @@ public class AsyncBulkByScrollActionTests extends ESTestCase { }; InternalSearchHit hit = new InternalSearchHit(0, "id", new Text("type"), emptyMap()); InternalSearchHits hits = new InternalSearchHits(new InternalSearchHit[] {hit}, 0, 0); - InternalSearchResponse searchResponse = new InternalSearchResponse(hits, null, null, null, false, false); - action.onScrollResponse(timeValueSeconds(0), new SearchResponse(searchResponse, scrollId(), 5, 4, randomLong(), null)); - try { - listener.get(); - fail("Expected failure."); - } catch (ExecutionException e) { - assertThat(e.getCause(), instanceOf(RuntimeException.class)); - assertThat(e.getCause().getMessage(), equalTo("surprise")); - } + InternalSearchResponse internalResponse = new InternalSearchResponse(hits, null, null, null, false, false); + SearchResponse searchResponse = new SearchResponse(internalResponse, scrollId(), 5, 4, randomLong(), null); + action.onScrollResponse(timeValueNanos(System.nanoTime()), 0, searchResponse); + ExecutionException e = expectThrows(ExecutionException.class, () -> listener.get()); + assertThat(e.getCause(), instanceOf(RuntimeException.class)); + assertThat(e.getCause().getMessage(), equalTo("surprise")); } /** @@ -425,17 +417,6 @@ public class AsyncBulkByScrollActionTests extends ESTestCase { assertEquals(testRequest.getMaxRetries(), testTask.getStatus().getBulkRetries()); } - public void testPerfectlyThrottledBatchTime() { - DummyAbstractAsyncBulkByScrollAction action = new DummyAbstractAsyncBulkByScrollAction(); - testRequest.setRequestsPerSecond(Float.POSITIVE_INFINITY); - assertThat((double) action.perfectlyThrottledBatchTime(randomInt()), closeTo(0f, 0f)); - - int total = between(0, 1000000); - testTask.rethrottle(1); - assertThat((double) action.perfectlyThrottledBatchTime(total), - closeTo(TimeUnit.SECONDS.toNanos(total), TimeUnit.SECONDS.toNanos(1))); - } - public void testScrollDelay() throws Exception { /* * Replace the thread pool with one that will save the delay sent for the command. We'll use that to check that we used a proper @@ -444,7 +425,7 @@ public class AsyncBulkByScrollActionTests extends ESTestCase { AtomicReference capturedDelay = new AtomicReference<>(); AtomicReference capturedCommand = new AtomicReference<>(); threadPool.shutdown(); - threadPool = new ThreadPool(getTestName()) { + threadPool = new TestThreadPool(getTestName()) { @Override public ScheduledFuture schedule(TimeValue delay, String name, Runnable command) { capturedDelay.set(delay); @@ -459,12 +440,10 @@ public class AsyncBulkByScrollActionTests extends ESTestCase { // Set the base for the scroll to wait - this is added to the figure we calculate below firstSearchRequest.scroll(timeValueSeconds(10)); - // We'd like to get about 1 request a second + // Set throttle to 1 request per second to make the math simpler testTask.rethrottle(1f); - // Make the last scroll look nearly instant - action.setLastBatchStartTime(System.nanoTime()); - // The last batch had 100 documents - action.startNextScroll(100); + // Make the last batch look nearly instant but have 100 documents + action.startNextScroll(timeValueNanos(System.nanoTime()), 100); // So the next request is going to have to wait an extra 100 seconds or so (base was 10 seconds, so 110ish) assertThat(client.lastScroll.get().request.scroll().keepAlive().seconds(), either(equalTo(110L)).or(equalTo(109L))); @@ -472,11 +451,20 @@ public class AsyncBulkByScrollActionTests extends ESTestCase { // Now we can simulate a response and check the delay that we used for the task InternalSearchHit hit = new InternalSearchHit(0, "id", new Text("type"), emptyMap()); InternalSearchHits hits = new InternalSearchHits(new InternalSearchHit[] { hit }, 0, 0); - InternalSearchResponse searchResponse = new InternalSearchResponse(hits, null, null, null, false, false); - client.lastScroll.get().listener.onResponse(new SearchResponse(searchResponse, scrollId(), 5, 4, randomLong(), null)); + InternalSearchResponse internalResponse = new InternalSearchResponse(hits, null, null, null, false, false); + SearchResponse searchResponse = new SearchResponse(internalResponse, scrollId(), 5, 4, randomLong(), null); - // The delay is still 100ish seconds because there hasn't been much time between when we requested the bulk and when we got it. - assertThat(capturedDelay.get().seconds(), either(equalTo(100L)).or(equalTo(99L))); + if (randomBoolean()) { + client.lastScroll.get().listener.onResponse(searchResponse); + // The delay is still 100ish seconds because there hasn't been much time between when we requested the bulk and when we got it. + assertThat(capturedDelay.get().seconds(), either(equalTo(100L)).or(equalTo(99L))); + } else { + // Let's rethrottle between the starting the scroll and getting the response + testTask.rethrottle(10f); + client.lastScroll.get().listener.onResponse(searchResponse); + // The delay uses the new throttle + assertThat(capturedDelay.get().seconds(), either(equalTo(10L)).or(equalTo(9L))); + } // Running the command ought to increment the delay counter on the task. capturedCommand.get().run(); @@ -500,7 +488,7 @@ public class AsyncBulkByScrollActionTests extends ESTestCase { CountDownLatch successLatch = new CountDownLatch(1); DummyAbstractAsyncBulkByScrollAction action = new DummyActionWithoutBackoff() { @Override - void startNextScroll(int lastBatchSize) { + void startNextScroll(TimeValue lastBatchStartTime, int lastBatchSize) { successLatch.countDown(); } }; @@ -508,7 +496,7 @@ public class AsyncBulkByScrollActionTests extends ESTestCase { for (int i = 0; i < size + 1; i++) { request.add(new IndexRequest("index", "type", "id" + i)); } - action.sendBulkRequest(request); + action.sendBulkRequest(timeValueNanos(System.nanoTime()), request); if (failWithRejection) { BulkIndexByScrollResponse response = listener.get(); assertThat(response.getIndexingFailures(), hasSize(1)); @@ -575,22 +563,23 @@ public class AsyncBulkByScrollActionTests extends ESTestCase { public void testCancelBeforeScrollResponse() throws Exception { // We bail so early we don't need to pass in a half way valid response. - cancelTaskCase((DummyAbstractAsyncBulkByScrollAction action) -> action.onScrollResponse(timeValueSeconds(0), null)); + cancelTaskCase((DummyAbstractAsyncBulkByScrollAction action) -> action.onScrollResponse(timeValueNanos(System.nanoTime()), 1, + null)); } public void testCancelBeforeSendBulkRequest() throws Exception { // We bail so early we don't need to pass in a half way valid request. - cancelTaskCase((DummyAbstractAsyncBulkByScrollAction action) -> action.sendBulkRequest(null)); + cancelTaskCase((DummyAbstractAsyncBulkByScrollAction action) -> action.sendBulkRequest(timeValueNanos(System.nanoTime()), null)); } public void testCancelBeforeOnBulkResponse() throws Exception { // We bail so early we don't need to pass in a half way valid response. cancelTaskCase((DummyAbstractAsyncBulkByScrollAction action) -> - action.onBulkResponse(new BulkResponse(new BulkItemResponse[0], 0))); + action.onBulkResponse(timeValueNanos(System.nanoTime()), new BulkResponse(new BulkItemResponse[0], 0))); } public void testCancelBeforeStartNextScroll() throws Exception { - cancelTaskCase((DummyAbstractAsyncBulkByScrollAction action) -> action.startNextScroll(0)); + cancelTaskCase((DummyAbstractAsyncBulkByScrollAction action) -> action.startNextScroll(timeValueNanos(System.nanoTime()), 0)); } public void testCancelBeforeStartNormalTermination() throws Exception { @@ -612,7 +601,7 @@ public class AsyncBulkByScrollActionTests extends ESTestCase { * is a delay. */ threadPool.shutdown(); - threadPool = new ThreadPool(getTestName()) { + threadPool = new TestThreadPool(getTestName()) { @Override public ScheduledFuture schedule(TimeValue delay, String name, Runnable command) { /* @@ -639,7 +628,9 @@ public class AsyncBulkByScrollActionTests extends ESTestCase { InternalSearchHits hits = new InternalSearchHits(null, total, 0); InternalSearchResponse searchResponse = new InternalSearchResponse(hits, null, null, null, false, false); // Use a long delay here so the test will time out if the cancellation doesn't reschedule the throttled task - action.onScrollResponse(timeValueMinutes(10), new SearchResponse(searchResponse, scrollId(), 5, 4, randomLong(), null)); + SearchResponse scrollResponse = new SearchResponse(searchResponse, scrollId(), 5, 4, randomLong(), null); + testTask.rethrottle(1); + action.onScrollResponse(timeValueNanos(System.nanoTime()), 1000, scrollResponse); // Now that we've got our cancel we'll just verify that it all came through all right assertEquals(reason, listener.get(10, TimeUnit.SECONDS).getReasonCancelled()); diff --git a/modules/reindex/src/test/java/org/elasticsearch/index/reindex/BulkByScrollTaskTests.java b/modules/reindex/src/test/java/org/elasticsearch/index/reindex/BulkByScrollTaskTests.java index d64c69ba362..05699c6f7af 100644 --- a/modules/reindex/src/test/java/org/elasticsearch/index/reindex/BulkByScrollTaskTests.java +++ b/modules/reindex/src/test/java/org/elasticsearch/index/reindex/BulkByScrollTaskTests.java @@ -26,6 +26,7 @@ import org.elasticsearch.common.xcontent.XContentBuilder; import org.elasticsearch.common.xcontent.json.JsonXContent; import org.elasticsearch.tasks.TaskId; import org.elasticsearch.test.ESTestCase; +import org.elasticsearch.threadpool.TestThreadPool; import org.elasticsearch.threadpool.ThreadPool; import org.junit.Before; @@ -41,8 +42,10 @@ import java.util.concurrent.TimeoutException; import java.util.concurrent.atomic.AtomicBoolean; import static org.elasticsearch.common.unit.TimeValue.parseTimeValue; +import static org.elasticsearch.common.unit.TimeValue.timeValueNanos; import static org.elasticsearch.common.unit.TimeValue.timeValueSeconds; import static org.hamcrest.Matchers.both; +import static org.hamcrest.Matchers.closeTo; import static org.hamcrest.Matchers.containsString; import static org.hamcrest.Matchers.empty; import static org.hamcrest.Matchers.greaterThanOrEqualTo; @@ -164,11 +167,12 @@ public class BulkByScrollTaskTests extends ESTestCase { * We never end up waiting this long because the test rethrottles over and over again, ratcheting down the delay a random amount * each time. */ - float originalRequestsPerSecond = (float) randomDoubleBetween(0, 10000, true); + float originalRequestsPerSecond = (float) randomDoubleBetween(1, 10000, true); task.rethrottle(originalRequestsPerSecond); TimeValue maxDelay = timeValueSeconds(between(1, 5)); assertThat(maxDelay.nanos(), greaterThanOrEqualTo(0L)); - ThreadPool threadPool = new ThreadPool(getTestName()) { + int batchSizeForMaxDelay = (int) (maxDelay.seconds() * originalRequestsPerSecond); + ThreadPool threadPool = new TestThreadPool(getTestName()) { @Override public ScheduledFuture schedule(TimeValue delay, String name, Runnable command) { assertThat(delay.nanos(), both(greaterThanOrEqualTo(0L)).and(lessThanOrEqualTo(maxDelay.nanos()))); @@ -176,7 +180,7 @@ public class BulkByScrollTaskTests extends ESTestCase { } }; try { - task.delayPrepareBulkRequest(threadPool, maxDelay, new AbstractRunnable() { + task.delayPrepareBulkRequest(threadPool, timeValueNanos(System.nanoTime()), batchSizeForMaxDelay, new AbstractRunnable() { @Override protected void doRun() throws Exception { boolean oldValue = done.getAndSet(true); @@ -220,7 +224,7 @@ public class BulkByScrollTaskTests extends ESTestCase { public void testDelayNeverNegative() throws IOException { // Thread pool that returns a ScheduledFuture that claims to have a negative delay - ThreadPool threadPool = new ThreadPool("test") { + ThreadPool threadPool = new TestThreadPool("test") { public ScheduledFuture schedule(TimeValue delay, String name, Runnable command) { return new ScheduledFuture() { @Override @@ -262,7 +266,7 @@ public class BulkByScrollTaskTests extends ESTestCase { }; try { // Have the task use the thread pool to delay a task that does nothing - task.delayPrepareBulkRequest(threadPool, timeValueSeconds(0), new AbstractRunnable() { + task.delayPrepareBulkRequest(threadPool, timeValueSeconds(0), 1, new AbstractRunnable() { @Override protected void doRun() throws Exception { } @@ -283,4 +287,14 @@ public class BulkByScrollTaskTests extends ESTestCase { task.getStatus().toXContent(builder, ToXContent.EMPTY_PARAMS); assertThat(builder.string(), containsString("\"requests_per_second\":\"unlimited\"")); } + + public void testPerfectlyThrottledBatchTime() { + task.rethrottle(Float.POSITIVE_INFINITY); + assertThat((double) task.perfectlyThrottledBatchTime(randomInt()), closeTo(0f, 0f)); + + int total = between(0, 1000000); + task.rethrottle(1); + assertThat((double) task.perfectlyThrottledBatchTime(total), + closeTo(TimeUnit.SECONDS.toNanos(total), TimeUnit.SECONDS.toNanos(1))); + } } diff --git a/modules/reindex/src/test/java/org/elasticsearch/index/reindex/CancelTests.java b/modules/reindex/src/test/java/org/elasticsearch/index/reindex/CancelTests.java index 1f9cd6b1394..2bb7fd627e4 100644 --- a/modules/reindex/src/test/java/org/elasticsearch/index/reindex/CancelTests.java +++ b/modules/reindex/src/test/java/org/elasticsearch/index/reindex/CancelTests.java @@ -31,6 +31,7 @@ import org.elasticsearch.index.engine.Engine; import org.elasticsearch.index.engine.Engine.Operation.Origin; import org.elasticsearch.index.shard.IndexingOperationListener; import org.elasticsearch.plugins.Plugin; +import org.elasticsearch.ingest.IngestTestPlugin; import org.junit.BeforeClass; import java.util.ArrayList; @@ -68,6 +69,7 @@ public class CancelTests extends ReindexTestCase { @Override protected Collection> nodePlugins() { Collection> plugins = new ArrayList<>(super.nodePlugins()); + plugins.add(IngestTestPlugin.class); plugins.add(ReindexCancellationPlugin.class); return plugins; } @@ -157,19 +159,16 @@ public class CancelTests extends ReindexTestCase { public void testUpdateByQueryCancel() throws Exception { BytesReference pipeline = new BytesArray("{\n" + - " \"description\" : \"sets updated to true\",\n" + + " \"description\" : \"sets processed to true\",\n" + " \"processors\" : [ {\n" + - " \"set\" : {\n" + - " \"field\": \"updated\",\n" + - " \"value\": true" + - " }\n" + + " \"test\" : {}\n" + " } ]\n" + "}"); assertAcked(client().admin().cluster().preparePutPipeline("set-foo", pipeline).get()); testCancel(UpdateByQueryAction.NAME, updateByQuery().setPipeline("set-foo").source(INDEX), (response, total, modified) -> { assertThat(response, matcher().updated(modified).reasonCancelled(equalTo("by user request"))); - assertHitCount(client().prepareSearch(INDEX).setSize(0).setQuery(termQuery("updated", true)).get(), modified); + assertHitCount(client().prepareSearch(INDEX).setSize(0).setQuery(termQuery("processed", true)).get(), modified); }); assertAcked(client().admin().cluster().deletePipeline(new DeletePipelineRequest("set-foo")).get()); diff --git a/modules/reindex/src/test/java/org/elasticsearch/index/reindex/RethrottleTests.java b/modules/reindex/src/test/java/org/elasticsearch/index/reindex/RethrottleTests.java index acbc31bff99..38f9dbc6d97 100644 --- a/modules/reindex/src/test/java/org/elasticsearch/index/reindex/RethrottleTests.java +++ b/modules/reindex/src/test/java/org/elasticsearch/index/reindex/RethrottleTests.java @@ -21,7 +21,6 @@ package org.elasticsearch.index.reindex; import org.elasticsearch.action.ListenableActionFuture; import org.elasticsearch.action.admin.cluster.node.tasks.list.ListTasksResponse; -import org.elasticsearch.test.junit.annotations.TestLogging; import static org.hamcrest.Matchers.hasSize; @@ -30,9 +29,6 @@ import static org.hamcrest.Matchers.hasSize; * too but this is the only place that tests running against multiple nodes so it is the only integration tests that checks for * serialization. */ -// Extra logging in case of failure. We couldn't explain the last failure: -// https://elasticsearch-ci.elastic.co/job/elastic+elasticsearch+master+g1gc/359/consoleFull -@TestLogging("_root:DEBUG") public class RethrottleTests extends ReindexTestCase { public void testReindex() throws Exception { diff --git a/modules/reindex/src/test/java/org/elasticsearch/index/reindex/RetryTests.java b/modules/reindex/src/test/java/org/elasticsearch/index/reindex/RetryTests.java index 699ba483e30..09945c9372b 100644 --- a/modules/reindex/src/test/java/org/elasticsearch/index/reindex/RetryTests.java +++ b/modules/reindex/src/test/java/org/elasticsearch/index/reindex/RetryTests.java @@ -63,11 +63,11 @@ public class RetryTests extends ESSingleNodeTestCase { protected Settings nodeSettings() { Settings.Builder settings = Settings.builder().put(super.nodeSettings()); // Use pools of size 1 so we can block them - settings.put("threadpool.bulk.size", 1); - settings.put("threadpool.search.size", 1); + settings.put("thread_pool.bulk.size", 1); + settings.put("thread_pool.search.size", 1); // Use queues of size 1 because size 0 is broken and because search requests need the queue to function - settings.put("threadpool.bulk.queue_size", 1); - settings.put("threadpool.search.queue_size", 1); + settings.put("thread_pool.bulk.queue_size", 1); + settings.put("thread_pool.search.queue_size", 1); return settings.build(); } diff --git a/modules/reindex/src/test/java/org/elasticsearch/index/reindex/UpdateByQueryWhileModifyingTests.java b/modules/reindex/src/test/java/org/elasticsearch/index/reindex/UpdateByQueryWhileModifyingTests.java index 7be073e32af..faea69b870f 100644 --- a/modules/reindex/src/test/java/org/elasticsearch/index/reindex/UpdateByQueryWhileModifyingTests.java +++ b/modules/reindex/src/test/java/org/elasticsearch/index/reindex/UpdateByQueryWhileModifyingTests.java @@ -28,6 +28,7 @@ import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicReference; import static org.apache.lucene.util.TestUtil.randomSimpleString; +import static org.elasticsearch.action.support.WriteRequest.RefreshPolicy.IMMEDIATE; import static org.hamcrest.Matchers.either; import static org.hamcrest.Matchers.equalTo; @@ -64,7 +65,7 @@ public class UpdateByQueryWhileModifyingTests extends ReindexTestCase { assertEquals(value.get(), get.getSource().get("test")); value.set(randomSimpleString(random())); IndexRequestBuilder index = client().prepareIndex("test", "test", "test").setSource("test", value.get()) - .setRefresh(true); + .setRefreshPolicy(IMMEDIATE); /* * Update by query increments the version number so concurrent * indexes might get version conflict exceptions so we just diff --git a/modules/reindex/src/test/resources/rest-api-spec/test/reindex/80_throttle.yaml b/modules/reindex/src/test/resources/rest-api-spec/test/reindex/70_throttle.yaml similarity index 100% rename from modules/reindex/src/test/resources/rest-api-spec/test/reindex/80_throttle.yaml rename to modules/reindex/src/test/resources/rest-api-spec/test/reindex/70_throttle.yaml diff --git a/modules/reindex/src/test/resources/rest-api-spec/test/update_by_query/70_throttle.yaml b/modules/reindex/src/test/resources/rest-api-spec/test/update_by_query/60_throttle.yaml similarity index 100% rename from modules/reindex/src/test/resources/rest-api-spec/test/update_by_query/70_throttle.yaml rename to modules/reindex/src/test/resources/rest-api-spec/test/update_by_query/60_throttle.yaml diff --git a/plugins/discovery-azure/src/test/java/org/elasticsearch/cloud/azure/AbstractAzureTestCase.java b/plugins/discovery-azure/src/test/java/org/elasticsearch/cloud/azure/AbstractAzureTestCase.java deleted file mode 100644 index ad7140f5020..00000000000 --- a/plugins/discovery-azure/src/test/java/org/elasticsearch/cloud/azure/AbstractAzureTestCase.java +++ /dev/null @@ -1,72 +0,0 @@ -/* - * Licensed to Elasticsearch under one or more contributor - * license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright - * ownership. Elasticsearch licenses this file to you under - * the Apache License, Version 2.0 (the "License"); you may - * not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ - -package org.elasticsearch.cloud.azure; - -import org.elasticsearch.common.Strings; -import org.elasticsearch.common.io.PathUtils; -import org.elasticsearch.common.settings.Settings; -import org.elasticsearch.common.settings.SettingsException; -import org.elasticsearch.env.Environment; -import org.elasticsearch.plugin.discovery.azure.AzureDiscoveryPlugin; -import org.elasticsearch.plugins.Plugin; -import org.elasticsearch.test.ESIntegTestCase; -import org.elasticsearch.test.ESIntegTestCase.ThirdParty; - -import java.util.Collection; - -/** - * Base class for Azure tests that require credentials. - *

    - * You must specify {@code -Dtests.thirdparty=true -Dtests.config=/path/to/config} - * in order to run these tests. - */ -@ThirdParty -public abstract class AbstractAzureTestCase extends ESIntegTestCase { - - @Override - protected Settings nodeSettings(int nodeOrdinal) { - return Settings.builder() - .put(super.nodeSettings(nodeOrdinal)) - .put(readSettingsFromFile()) - .build(); - } - - @Override - protected Collection> nodePlugins() { - return pluginList(AzureDiscoveryPlugin.class); - } - - protected Settings readSettingsFromFile() { - Settings.Builder settings = Settings.builder(); - settings.put(Environment.PATH_HOME_SETTING.getKey(), createTempDir()); - - // if explicit, just load it and don't load from env - try { - if (Strings.hasText(System.getProperty("tests.config"))) { - settings.loadFromPath(PathUtils.get((System.getProperty("tests.config")))); - } else { - throw new IllegalStateException("to run integration tests, you need to set -Dtests.thirdparty=true and -Dtests.config=/path/to/elasticsearch.yml"); - } - } catch (SettingsException exception) { - throw new IllegalStateException("your test configuration file is incorrect: " + System.getProperty("tests.config"), exception); - } - return settings.build(); - } -} diff --git a/plugins/discovery-ec2/src/test/java/org/elasticsearch/cloud/aws/AbstractAwsTestCase.java b/plugins/discovery-ec2/src/test/java/org/elasticsearch/cloud/aws/AbstractAwsTestCase.java index 422f6087e7d..8fdc34a6595 100644 --- a/plugins/discovery-ec2/src/test/java/org/elasticsearch/cloud/aws/AbstractAwsTestCase.java +++ b/plugins/discovery-ec2/src/test/java/org/elasticsearch/cloud/aws/AbstractAwsTestCase.java @@ -29,6 +29,7 @@ import org.elasticsearch.plugins.Plugin; import org.elasticsearch.test.ESIntegTestCase; import org.elasticsearch.test.ESIntegTestCase.ThirdParty; +import java.io.IOException; import java.util.Collection; /** @@ -49,7 +50,11 @@ public abstract class AbstractAwsTestCase extends ESIntegTestCase { // if explicit, just load it and don't load from env try { if (Strings.hasText(System.getProperty("tests.config"))) { - settings.loadFromPath(PathUtils.get(System.getProperty("tests.config"))); + try { + settings.loadFromPath(PathUtils.get(System.getProperty("tests.config"))); + } catch (IOException e) { + throw new IllegalArgumentException("could not load aws tests config", e); + } } else { throw new IllegalStateException("to run integration tests, you need to set -Dtests.thirdparty=true and -Dtests.config=/path/to/elasticsearch.yml"); } diff --git a/plugins/discovery-ec2/src/test/java/org/elasticsearch/discovery/ec2/Ec2DiscoveryTests.java b/plugins/discovery-ec2/src/test/java/org/elasticsearch/discovery/ec2/Ec2DiscoveryTests.java index c4863680613..956732113c1 100644 --- a/plugins/discovery-ec2/src/test/java/org/elasticsearch/discovery/ec2/Ec2DiscoveryTests.java +++ b/plugins/discovery-ec2/src/test/java/org/elasticsearch/discovery/ec2/Ec2DiscoveryTests.java @@ -30,6 +30,7 @@ import org.elasticsearch.common.transport.LocalTransportAddress; import org.elasticsearch.common.transport.TransportAddress; import org.elasticsearch.test.ESTestCase; import org.elasticsearch.test.transport.MockTransportService; +import org.elasticsearch.threadpool.TestThreadPool; import org.elasticsearch.threadpool.ThreadPool; import org.elasticsearch.transport.TransportService; import org.junit.AfterClass; @@ -53,7 +54,7 @@ public class Ec2DiscoveryTests extends ESTestCase { @BeforeClass public static void createThreadPool() { - threadPool = new ThreadPool(Ec2DiscoveryTests.class.getName()); + threadPool = new TestThreadPool(Ec2DiscoveryTests.class.getName()); } @AfterClass diff --git a/plugins/discovery-gce/src/test/java/org/elasticsearch/discovery/gce/GceDiscoveryTests.java b/plugins/discovery-gce/src/test/java/org/elasticsearch/discovery/gce/GceDiscoveryTests.java index 4525b1ece1d..78d96fd0163 100644 --- a/plugins/discovery-gce/src/test/java/org/elasticsearch/discovery/gce/GceDiscoveryTests.java +++ b/plugins/discovery-gce/src/test/java/org/elasticsearch/discovery/gce/GceDiscoveryTests.java @@ -27,6 +27,7 @@ import org.elasticsearch.common.network.NetworkService; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.test.ESTestCase; import org.elasticsearch.test.transport.MockTransportService; +import org.elasticsearch.threadpool.TestThreadPool; import org.elasticsearch.threadpool.ThreadPool; import org.junit.After; import org.junit.AfterClass; @@ -72,7 +73,7 @@ public class GceDiscoveryTests extends ESTestCase { @BeforeClass public static void createThreadPool() { - threadPool = new ThreadPool(GceDiscoveryTests.class.getName()); + threadPool = new TestThreadPool(GceDiscoveryTests.class.getName()); } @AfterClass diff --git a/plugins/ingest-attachment/src/test/resources/rest-api-spec/test/ingest_attachment/10_basic.yaml b/plugins/ingest-attachment/src/test/resources/rest-api-spec/test/ingest_attachment/10_basic.yaml index 7546090d6ad..88accac7730 100644 --- a/plugins/ingest-attachment/src/test/resources/rest-api-spec/test/ingest_attachment/10_basic.yaml +++ b/plugins/ingest-attachment/src/test/resources/rest-api-spec/test/ingest_attachment/10_basic.yaml @@ -8,21 +8,5 @@ nodes.info: {} - match: { nodes.$master.plugins.0.name: ingest-attachment } - - match: { nodes.$master.ingest.processors.0.type: append } - - match: { nodes.$master.ingest.processors.1.type: attachment } - - match: { nodes.$master.ingest.processors.2.type: convert } - - match: { nodes.$master.ingest.processors.3.type: date } - - match: { nodes.$master.ingest.processors.4.type: date_index_name } - - match: { nodes.$master.ingest.processors.5.type: fail } - - match: { nodes.$master.ingest.processors.6.type: foreach } - - match: { nodes.$master.ingest.processors.7.type: gsub } - - match: { nodes.$master.ingest.processors.8.type: join } - - match: { nodes.$master.ingest.processors.9.type: lowercase } - - match: { nodes.$master.ingest.processors.10.type: remove } - - match: { nodes.$master.ingest.processors.11.type: rename } - - match: { nodes.$master.ingest.processors.12.type: set } - - match: { nodes.$master.ingest.processors.13.type: sort } - - match: { nodes.$master.ingest.processors.14.type: split } - - match: { nodes.$master.ingest.processors.15.type: trim } - - match: { nodes.$master.ingest.processors.16.type: uppercase } + - match: { nodes.$master.ingest.processors.0.type: attachment } diff --git a/plugins/ingest-geoip/build.gradle b/plugins/ingest-geoip/build.gradle index eee032d6d16..d9e90a61d40 100644 --- a/plugins/ingest-geoip/build.gradle +++ b/plugins/ingest-geoip/build.gradle @@ -23,19 +23,19 @@ esplugin { } dependencies { - compile ('com.maxmind.geoip2:geoip2:2.6.0') + compile ('com.maxmind.geoip2:geoip2:2.7.0') // geoip2 dependencies: compile('com.fasterxml.jackson.core:jackson-annotations:2.7.1') compile('com.fasterxml.jackson.core:jackson-databind:2.7.1') - compile('com.maxmind.db:maxmind-db:1.2.0') + compile('com.maxmind.db:maxmind-db:1.2.1') - testCompile 'org.elasticsearch:geolite2-databases:20151029' + testCompile 'org.elasticsearch:geolite2-databases:20160608' } task copyDefaultGeoIp2DatabaseFiles(type: Copy) { from { zipTree(configurations.testCompile.files.find { it.name.contains('geolite2-databases')}) } into "${project.buildDir}/ingest-geoip" - include "*.mmdb" + include "*.mmdb.gz" } project.bundlePlugin.dependsOn(copyDefaultGeoIp2DatabaseFiles) diff --git a/plugins/ingest-geoip/licenses/geoip2-2.6.0.jar.sha1 b/plugins/ingest-geoip/licenses/geoip2-2.6.0.jar.sha1 deleted file mode 100644 index ba5a0242ec8..00000000000 --- a/plugins/ingest-geoip/licenses/geoip2-2.6.0.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -2574c8b878f1cd39709559f1b96f1b5f0cdd69d3 \ No newline at end of file diff --git a/plugins/ingest-geoip/licenses/geoip2-2.7.0.jar.sha1 b/plugins/ingest-geoip/licenses/geoip2-2.7.0.jar.sha1 new file mode 100644 index 00000000000..2015e311d60 --- /dev/null +++ b/plugins/ingest-geoip/licenses/geoip2-2.7.0.jar.sha1 @@ -0,0 +1 @@ +2010d922191f5801939b462a5703ab79a7829626 \ No newline at end of file diff --git a/plugins/ingest-geoip/licenses/maxmind-db-1.2.0.jar.sha1 b/plugins/ingest-geoip/licenses/maxmind-db-1.2.0.jar.sha1 deleted file mode 100644 index ae48403db39..00000000000 --- a/plugins/ingest-geoip/licenses/maxmind-db-1.2.0.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -b842823f24555f5d26608fef8122898365b3cd63 \ No newline at end of file diff --git a/plugins/ingest-geoip/licenses/maxmind-db-1.2.1.jar.sha1 b/plugins/ingest-geoip/licenses/maxmind-db-1.2.1.jar.sha1 new file mode 100644 index 00000000000..2358241fdf6 --- /dev/null +++ b/plugins/ingest-geoip/licenses/maxmind-db-1.2.1.jar.sha1 @@ -0,0 +1 @@ +64b6b6a8c162fc9b0004fcdf9641cf1b408ffa33 \ No newline at end of file diff --git a/plugins/ingest-geoip/src/main/java/org/elasticsearch/ingest/geoip/GeoIpProcessor.java b/plugins/ingest-geoip/src/main/java/org/elasticsearch/ingest/geoip/GeoIpProcessor.java index c524bf15a3d..9d606419e57 100644 --- a/plugins/ingest-geoip/src/main/java/org/elasticsearch/ingest/geoip/GeoIpProcessor.java +++ b/plugins/ingest-geoip/src/main/java/org/elasticsearch/ingest/geoip/GeoIpProcessor.java @@ -233,7 +233,7 @@ public final class GeoIpProcessor extends AbstractProcessor { public GeoIpProcessor doCreate(String processorTag, Map config) throws Exception { String ipField = readStringProperty(TYPE, processorTag, config, "field"); String targetField = readStringProperty(TYPE, processorTag, config, "target_field", "geoip"); - String databaseFile = readStringProperty(TYPE, processorTag, config, "database_file", "GeoLite2-City.mmdb"); + String databaseFile = readStringProperty(TYPE, processorTag, config, "database_file", "GeoLite2-City.mmdb.gz"); List propertyNames = readOptionalList(TYPE, processorTag, config, "properties"); DatabaseReader databaseReader = databaseReaders.get(databaseFile); diff --git a/plugins/ingest-geoip/src/main/java/org/elasticsearch/ingest/geoip/IngestGeoIpPlugin.java b/plugins/ingest-geoip/src/main/java/org/elasticsearch/ingest/geoip/IngestGeoIpPlugin.java index 7c57349f4b6..2c7edd7ee96 100644 --- a/plugins/ingest-geoip/src/main/java/org/elasticsearch/ingest/geoip/IngestGeoIpPlugin.java +++ b/plugins/ingest-geoip/src/main/java/org/elasticsearch/ingest/geoip/IngestGeoIpPlugin.java @@ -34,6 +34,7 @@ import java.util.HashMap; import java.util.Iterator; import java.util.Map; import java.util.stream.Stream; +import java.util.zip.GZIPInputStream; public class IngestGeoIpPlugin extends Plugin { @@ -60,13 +61,13 @@ public class IngestGeoIpPlugin extends Plugin { Map databaseReaders = new HashMap<>(); try (Stream databaseFiles = Files.list(geoIpConfigDirectory)) { - PathMatcher pathMatcher = geoIpConfigDirectory.getFileSystem().getPathMatcher("glob:**.mmdb"); + PathMatcher pathMatcher = geoIpConfigDirectory.getFileSystem().getPathMatcher("glob:**.mmdb.gz"); // Use iterator instead of forEach otherwise IOException needs to be caught twice... Iterator iterator = databaseFiles.iterator(); while (iterator.hasNext()) { Path databasePath = iterator.next(); if (Files.isRegularFile(databasePath) && pathMatcher.matches(databasePath)) { - try (InputStream inputStream = Files.newInputStream(databasePath, StandardOpenOption.READ)) { + try (InputStream inputStream = new GZIPInputStream(Files.newInputStream(databasePath, StandardOpenOption.READ))) { databaseReaders.put(databasePath.getFileName().toString(), new DatabaseReader.Builder(inputStream).build()); } } diff --git a/plugins/ingest-geoip/src/test/java/org/elasticsearch/ingest/geoip/GeoIpProcessorFactoryTests.java b/plugins/ingest-geoip/src/test/java/org/elasticsearch/ingest/geoip/GeoIpProcessorFactoryTests.java index 0840b10b004..25d8f1f6f5f 100644 --- a/plugins/ingest-geoip/src/test/java/org/elasticsearch/ingest/geoip/GeoIpProcessorFactoryTests.java +++ b/plugins/ingest-geoip/src/test/java/org/elasticsearch/ingest/geoip/GeoIpProcessorFactoryTests.java @@ -54,8 +54,8 @@ public class GeoIpProcessorFactoryTests extends ESTestCase { Path configDir = createTempDir(); Path geoIpConfigDir = configDir.resolve("ingest-geoip"); Files.createDirectories(geoIpConfigDir); - Files.copy(new ByteArrayInputStream(StreamsUtils.copyToBytesFromClasspath("/GeoLite2-City.mmdb")), geoIpConfigDir.resolve("GeoLite2-City.mmdb")); - Files.copy(new ByteArrayInputStream(StreamsUtils.copyToBytesFromClasspath("/GeoLite2-Country.mmdb")), geoIpConfigDir.resolve("GeoLite2-Country.mmdb")); + Files.copy(new ByteArrayInputStream(StreamsUtils.copyToBytesFromClasspath("/GeoLite2-City.mmdb.gz")), geoIpConfigDir.resolve("GeoLite2-City.mmdb.gz")); + Files.copy(new ByteArrayInputStream(StreamsUtils.copyToBytesFromClasspath("/GeoLite2-Country.mmdb.gz")), geoIpConfigDir.resolve("GeoLite2-Country.mmdb.gz")); databaseReaders = IngestGeoIpPlugin.loadDatabaseReaders(geoIpConfigDir); } @@ -89,7 +89,7 @@ public class GeoIpProcessorFactoryTests extends ESTestCase { Map config = new HashMap<>(); config.put("field", "_field"); - config.put("database_file", "GeoLite2-Country.mmdb"); + config.put("database_file", "GeoLite2-Country.mmdb.gz"); String processorTag = randomAsciiOfLength(10); config.put(AbstractProcessorFactory.TAG_KEY, processorTag); @@ -116,7 +116,7 @@ public class GeoIpProcessorFactoryTests extends ESTestCase { GeoIpProcessor.Factory factory = new GeoIpProcessor.Factory(databaseReaders); Map config = new HashMap<>(); config.put("field", "_field"); - config.put("database_file", "GeoLite2-Country.mmdb"); + config.put("database_file", "GeoLite2-Country.mmdb.gz"); GeoIpProcessor processor = factory.create(config); assertThat(processor.getField(), equalTo("_field")); assertThat(processor.getTargetField(), equalTo("geoip")); @@ -128,7 +128,7 @@ public class GeoIpProcessorFactoryTests extends ESTestCase { GeoIpProcessor.Factory factory = new GeoIpProcessor.Factory(databaseReaders); Map config = new HashMap<>(); config.put("field", "_field"); - config.put("database_file", "GeoLite2-Country.mmdb"); + config.put("database_file", "GeoLite2-Country.mmdb.gz"); EnumSet cityOnlyProperties = EnumSet.complementOf(GeoIpProcessor.Property.ALL_COUNTRY_PROPERTIES); String cityProperty = RandomPicks.randomFrom(Randomness.get(), cityOnlyProperties).toString(); config.put("properties", Collections.singletonList(cityProperty)); @@ -145,12 +145,12 @@ public class GeoIpProcessorFactoryTests extends ESTestCase { Map config = new HashMap<>(); config.put("field", "_field"); - config.put("database_file", "does-not-exist.mmdb"); + config.put("database_file", "does-not-exist.mmdb.gz"); try { factory.create(config); fail("Exception expected"); } catch (ElasticsearchParseException e) { - assertThat(e.getMessage(), equalTo("[database_file] database file [does-not-exist.mmdb] doesn't exist")); + assertThat(e.getMessage(), equalTo("[database_file] database file [does-not-exist.mmdb.gz] doesn't exist")); } } diff --git a/plugins/ingest-geoip/src/test/java/org/elasticsearch/ingest/geoip/GeoIpProcessorTests.java b/plugins/ingest-geoip/src/test/java/org/elasticsearch/ingest/geoip/GeoIpProcessorTests.java index f646895d7cf..a64a42fa95e 100644 --- a/plugins/ingest-geoip/src/test/java/org/elasticsearch/ingest/geoip/GeoIpProcessorTests.java +++ b/plugins/ingest-geoip/src/test/java/org/elasticsearch/ingest/geoip/GeoIpProcessorTests.java @@ -24,10 +24,12 @@ import org.elasticsearch.ingest.RandomDocumentPicks; import org.elasticsearch.ingest.core.IngestDocument; import org.elasticsearch.test.ESTestCase; +import java.io.IOException; import java.io.InputStream; import java.util.EnumSet; import java.util.HashMap; import java.util.Map; +import java.util.zip.GZIPInputStream; import static org.hamcrest.Matchers.containsString; import static org.hamcrest.Matchers.equalTo; @@ -35,33 +37,33 @@ import static org.hamcrest.Matchers.equalTo; public class GeoIpProcessorTests extends ESTestCase { public void testCity() throws Exception { - InputStream database = GeoIpProcessor.class.getResourceAsStream("/GeoLite2-City.mmdb"); + InputStream database = getDatabaseFileInputStream("/GeoLite2-City.mmdb.gz"); GeoIpProcessor processor = new GeoIpProcessor(randomAsciiOfLength(10), "source_field", new DatabaseReader.Builder(database).build(), "target_field", EnumSet.allOf(GeoIpProcessor.Property.class)); Map document = new HashMap<>(); - document.put("source_field", "82.170.213.79"); + document.put("source_field", "8.8.8.8"); IngestDocument ingestDocument = RandomDocumentPicks.randomIngestDocument(random(), document); processor.execute(ingestDocument); - assertThat(ingestDocument.getSourceAndMetadata().get("source_field"), equalTo("82.170.213.79")); + assertThat(ingestDocument.getSourceAndMetadata().get("source_field"), equalTo("8.8.8.8")); @SuppressWarnings("unchecked") Map geoData = (Map) ingestDocument.getSourceAndMetadata().get("target_field"); assertThat(geoData.size(), equalTo(8)); - assertThat(geoData.get("ip"), equalTo("82.170.213.79")); - assertThat(geoData.get("country_iso_code"), equalTo("NL")); - assertThat(geoData.get("country_name"), equalTo("Netherlands")); - assertThat(geoData.get("continent_name"), equalTo("Europe")); - assertThat(geoData.get("region_name"), equalTo("North Holland")); - assertThat(geoData.get("city_name"), equalTo("Amsterdam")); - assertThat(geoData.get("timezone"), equalTo("Europe/Amsterdam")); + assertThat(geoData.get("ip"), equalTo("8.8.8.8")); + assertThat(geoData.get("country_iso_code"), equalTo("US")); + assertThat(geoData.get("country_name"), equalTo("United States")); + assertThat(geoData.get("continent_name"), equalTo("North America")); + assertThat(geoData.get("region_name"), equalTo("California")); + assertThat(geoData.get("city_name"), equalTo("Mountain View")); + assertThat(geoData.get("timezone"), equalTo("America/Los_Angeles")); Map location = new HashMap<>(); - location.put("lat", 52.374d); - location.put("lon", 4.8897d); + location.put("lat", 37.386d); + location.put("lon", -122.0838d); assertThat(geoData.get("location"), equalTo(location)); } public void testCountry() throws Exception { - InputStream database = GeoIpProcessor.class.getResourceAsStream("/GeoLite2-Country.mmdb"); + InputStream database = getDatabaseFileInputStream("/GeoLite2-Country.mmdb.gz"); GeoIpProcessor processor = new GeoIpProcessor(randomAsciiOfLength(10), "source_field", new DatabaseReader.Builder(database).build(), "target_field", EnumSet.allOf(GeoIpProcessor.Property.class)); Map document = new HashMap<>(); @@ -80,11 +82,11 @@ public class GeoIpProcessorTests extends ESTestCase { } public void testAddressIsNotInTheDatabase() throws Exception { - InputStream database = GeoIpProcessor.class.getResourceAsStream("/GeoLite2-City.mmdb"); + InputStream database = getDatabaseFileInputStream("/GeoLite2-City.mmdb.gz"); GeoIpProcessor processor = new GeoIpProcessor(randomAsciiOfLength(10), "source_field", new DatabaseReader.Builder(database).build(), "target_field", EnumSet.allOf(GeoIpProcessor.Property.class)); Map document = new HashMap<>(); - document.put("source_field", "202.45.11.11"); + document.put("source_field", "127.0.0.1"); IngestDocument ingestDocument = RandomDocumentPicks.randomIngestDocument(random(), document); processor.execute(ingestDocument); @SuppressWarnings("unchecked") @@ -94,7 +96,7 @@ public class GeoIpProcessorTests extends ESTestCase { /** Don't silently do DNS lookups or anything trappy on bogus data */ public void testInvalid() throws Exception { - InputStream database = GeoIpProcessor.class.getResourceAsStream("/GeoLite2-City.mmdb"); + InputStream database = getDatabaseFileInputStream("/GeoLite2-City.mmdb.gz"); GeoIpProcessor processor = new GeoIpProcessor(randomAsciiOfLength(10), "source_field", new DatabaseReader.Builder(database).build(), "target_field", EnumSet.allOf(GeoIpProcessor.Property.class)); Map document = new HashMap<>(); @@ -109,4 +111,8 @@ public class GeoIpProcessorTests extends ESTestCase { } } + static InputStream getDatabaseFileInputStream(String path) throws IOException { + return new GZIPInputStream(GeoIpProcessor.class.getResourceAsStream(path)); + } + } diff --git a/plugins/ingest-geoip/src/test/resources/rest-api-spec/test/ingest_geoip/10_basic.yaml b/plugins/ingest-geoip/src/test/resources/rest-api-spec/test/ingest_geoip/10_basic.yaml index 313a9b4544f..7a06326a864 100644 --- a/plugins/ingest-geoip/src/test/resources/rest-api-spec/test/ingest_geoip/10_basic.yaml +++ b/plugins/ingest-geoip/src/test/resources/rest-api-spec/test/ingest_geoip/10_basic.yaml @@ -8,20 +8,4 @@ nodes.info: {} - match: { nodes.$master.plugins.0.name: ingest-geoip } - - match: { nodes.$master.ingest.processors.0.type: append } - - match: { nodes.$master.ingest.processors.1.type: convert } - - match: { nodes.$master.ingest.processors.2.type: date } - - match: { nodes.$master.ingest.processors.3.type: date_index_name } - - match: { nodes.$master.ingest.processors.4.type: fail } - - match: { nodes.$master.ingest.processors.5.type: foreach } - - match: { nodes.$master.ingest.processors.6.type: geoip } - - match: { nodes.$master.ingest.processors.7.type: gsub } - - match: { nodes.$master.ingest.processors.8.type: join } - - match: { nodes.$master.ingest.processors.9.type: lowercase } - - match: { nodes.$master.ingest.processors.10.type: remove } - - match: { nodes.$master.ingest.processors.11.type: rename } - - match: { nodes.$master.ingest.processors.12.type: set } - - match: { nodes.$master.ingest.processors.13.type: sort } - - match: { nodes.$master.ingest.processors.14.type: split } - - match: { nodes.$master.ingest.processors.15.type: trim } - - match: { nodes.$master.ingest.processors.16.type: uppercase } + - match: { nodes.$master.ingest.processors.0.type: geoip } diff --git a/plugins/ingest-geoip/src/test/resources/rest-api-spec/test/ingest_geoip/20_geoip_processor.yaml b/plugins/ingest-geoip/src/test/resources/rest-api-spec/test/ingest_geoip/20_geoip_processor.yaml index 5e8a3e7c1ff..33e9ec1ca46 100644 --- a/plugins/ingest-geoip/src/test/resources/rest-api-spec/test/ingest_geoip/20_geoip_processor.yaml +++ b/plugins/ingest-geoip/src/test/resources/rest-api-spec/test/ingest_geoip/20_geoip_processor.yaml @@ -98,7 +98,7 @@ { "geoip" : { "field" : "field1", - "database_file" : "GeoLite2-Country.mmdb" + "database_file" : "GeoLite2-Country.mmdb.gz" } } ] diff --git a/plugins/repository-azure/src/test/java/org/elasticsearch/cloud/azure/AbstractAzureWithThirdPartyIntegTestCase.java b/plugins/repository-azure/src/test/java/org/elasticsearch/cloud/azure/AbstractAzureWithThirdPartyIntegTestCase.java index 8cae9424950..e358adc92ab 100644 --- a/plugins/repository-azure/src/test/java/org/elasticsearch/cloud/azure/AbstractAzureWithThirdPartyIntegTestCase.java +++ b/plugins/repository-azure/src/test/java/org/elasticsearch/cloud/azure/AbstractAzureWithThirdPartyIntegTestCase.java @@ -24,6 +24,7 @@ import org.elasticsearch.plugin.repository.azure.AzureRepositoryPlugin; import org.elasticsearch.plugins.Plugin; import org.elasticsearch.test.ESIntegTestCase.ThirdParty; +import java.io.IOException; import java.util.Collection; import static org.elasticsearch.cloud.azure.AzureTestUtils.readSettingsFromFile; @@ -49,4 +50,5 @@ public abstract class AbstractAzureWithThirdPartyIntegTestCase extends AbstractA protected Collection> nodePlugins() { return pluginList(AzureRepositoryPlugin.class); } + } diff --git a/plugins/repository-azure/src/test/java/org/elasticsearch/cloud/azure/AzureTestUtils.java b/plugins/repository-azure/src/test/java/org/elasticsearch/cloud/azure/AzureTestUtils.java index 80840db587f..097f519db03 100644 --- a/plugins/repository-azure/src/test/java/org/elasticsearch/cloud/azure/AzureTestUtils.java +++ b/plugins/repository-azure/src/test/java/org/elasticsearch/cloud/azure/AzureTestUtils.java @@ -24,6 +24,8 @@ import org.elasticsearch.common.io.PathUtils; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.settings.SettingsException; +import java.io.IOException; + public class AzureTestUtils { /** * Read settings from file when running integration tests with ThirdParty annotation. @@ -36,7 +38,11 @@ public class AzureTestUtils { // if explicit, just load it and don't load from env try { if (Strings.hasText(System.getProperty("tests.config"))) { - settings.loadFromPath(PathUtils.get((System.getProperty("tests.config")))); + try { + settings.loadFromPath(PathUtils.get((System.getProperty("tests.config")))); + } catch (IOException e) { + throw new IllegalArgumentException("could not load azure tests config", e); + } } else { throw new IllegalStateException("to run integration tests, you need to set -Dtests.thirdparty=true and " + "-Dtests.config=/path/to/elasticsearch.yml"); diff --git a/plugins/repository-s3/src/test/java/org/elasticsearch/cloud/aws/AbstractAwsTestCase.java b/plugins/repository-s3/src/test/java/org/elasticsearch/cloud/aws/AbstractAwsTestCase.java index ec8fb902d66..9d1768db58b 100644 --- a/plugins/repository-s3/src/test/java/org/elasticsearch/cloud/aws/AbstractAwsTestCase.java +++ b/plugins/repository-s3/src/test/java/org/elasticsearch/cloud/aws/AbstractAwsTestCase.java @@ -29,6 +29,7 @@ import org.elasticsearch.plugins.Plugin; import org.elasticsearch.test.ESIntegTestCase; import org.elasticsearch.test.ESIntegTestCase.ThirdParty; +import java.io.IOException; import java.util.Collection; /** @@ -52,7 +53,11 @@ public abstract class AbstractAwsTestCase extends ESIntegTestCase { // if explicit, just load it and don't load from env try { if (Strings.hasText(System.getProperty("tests.config"))) { - settings.loadFromPath(PathUtils.get(System.getProperty("tests.config"))); + try { + settings.loadFromPath(PathUtils.get(System.getProperty("tests.config"))); + } catch (IOException e) { + throw new IllegalArgumentException("could not load aws tests config", e); + } } else { throw new IllegalStateException("to run integration tests, you need to set -Dtests.thirdparty=true and -Dtests.config=/path/to/elasticsearch.yml"); } diff --git a/qa/evil-tests/src/test/java/org/elasticsearch/bootstrap/ESPolicyUnitTests.java b/qa/evil-tests/src/test/java/org/elasticsearch/bootstrap/ESPolicyUnitTests.java index 220d093301c..255dfb3ec5c 100644 --- a/qa/evil-tests/src/test/java/org/elasticsearch/bootstrap/ESPolicyUnitTests.java +++ b/qa/evil-tests/src/test/java/org/elasticsearch/bootstrap/ESPolicyUnitTests.java @@ -64,6 +64,7 @@ public class ESPolicyUnitTests extends ESTestCase { assumeTrue("test cannot run with security manager", System.getSecurityManager() == null); PermissionCollection noPermissions = new Permissions(); ESPolicy policy = new ESPolicy(noPermissions, Collections.emptyMap(), true); - assertFalse(policy.implies(new ProtectionDomain(new CodeSource(null, (Certificate[])null), noPermissions), new FilePermission("foo", "read"))); + assertFalse(policy.implies(new ProtectionDomain(new CodeSource(null, (Certificate[]) null), noPermissions), + new FilePermission("foo", "read"))); } } diff --git a/qa/evil-tests/src/test/java/org/elasticsearch/bootstrap/EvilSecurityTests.java b/qa/evil-tests/src/test/java/org/elasticsearch/bootstrap/EvilSecurityTests.java index df54cf87fa4..3bf238916a5 100644 --- a/qa/evil-tests/src/test/java/org/elasticsearch/bootstrap/EvilSecurityTests.java +++ b/qa/evil-tests/src/test/java/org/elasticsearch/bootstrap/EvilSecurityTests.java @@ -68,6 +68,7 @@ public class EvilSecurityTests extends ESTestCase { } /** test generated permissions for all configured paths */ + @SuppressWarnings("deprecation") // needs to check settings for deprecated path public void testEnvironmentPaths() throws Exception { Path path = createTempDir(); // make a fake ES home and ensure we only grant permissions to that. @@ -77,7 +78,8 @@ public class EvilSecurityTests extends ESTestCase { settingsBuilder.put(Environment.PATH_HOME_SETTING.getKey(), esHome.resolve("home").toString()); settingsBuilder.put(Environment.PATH_CONF_SETTING.getKey(), esHome.resolve("conf").toString()); settingsBuilder.put(Environment.PATH_SCRIPTS_SETTING.getKey(), esHome.resolve("scripts").toString()); - settingsBuilder.putArray(Environment.PATH_DATA_SETTING.getKey(), esHome.resolve("data1").toString(), esHome.resolve("data2").toString()); + settingsBuilder.putArray(Environment.PATH_DATA_SETTING.getKey(), esHome.resolve("data1").toString(), + esHome.resolve("data2").toString()); settingsBuilder.put(Environment.PATH_SHARED_DATA_SETTING.getKey(), esHome.resolve("custom").toString()); settingsBuilder.put(Environment.PATH_LOGS_SETTING.getKey(), esHome.resolve("logs").toString()); settingsBuilder.put(Environment.PIDFILE_SETTING.getKey(), esHome.resolve("test.pid").toString()); diff --git a/qa/evil-tests/src/test/java/org/elasticsearch/env/NodeEnvironmentEvilTests.java b/qa/evil-tests/src/test/java/org/elasticsearch/env/NodeEnvironmentEvilTests.java index 6c5fd797be4..ccf5e1b105e 100644 --- a/qa/evil-tests/src/test/java/org/elasticsearch/env/NodeEnvironmentEvilTests.java +++ b/qa/evil-tests/src/test/java/org/elasticsearch/env/NodeEnvironmentEvilTests.java @@ -63,7 +63,7 @@ public class NodeEnvironmentEvilTests extends ESTestCase { assumeTrue("posix filesystem", isPosix); final String[] tempPaths = tmpPaths(); Path path = PathUtils.get(randomFrom(tempPaths)); - Path fooIndex = path.resolve("elasticsearch").resolve("nodes").resolve("0").resolve(NodeEnvironment.INDICES_FOLDER) + Path fooIndex = path.resolve("nodes").resolve("0").resolve(NodeEnvironment.INDICES_FOLDER) .resolve("foo"); Files.createDirectories(fooIndex); try (PosixPermissionsResetter attr = new PosixPermissionsResetter(fooIndex)) { @@ -83,7 +83,7 @@ public class NodeEnvironmentEvilTests extends ESTestCase { assumeTrue("posix filesystem", isPosix); final String[] tempPaths = tmpPaths(); Path path = PathUtils.get(randomFrom(tempPaths)); - Path fooIndex = path.resolve("elasticsearch").resolve("nodes").resolve("0").resolve(NodeEnvironment.INDICES_FOLDER) + Path fooIndex = path.resolve("nodes").resolve("0").resolve(NodeEnvironment.INDICES_FOLDER) .resolve("foo"); Path fooShard = fooIndex.resolve("0"); Path fooShardIndex = fooShard.resolve("index"); diff --git a/qa/evil-tests/src/test/java/org/elasticsearch/tribe/TribeUnitTests.java b/qa/evil-tests/src/test/java/org/elasticsearch/tribe/TribeUnitTests.java index f9cdf5b4f66..4199a5d67cd 100644 --- a/qa/evil-tests/src/test/java/org/elasticsearch/tribe/TribeUnitTests.java +++ b/qa/evil-tests/src/test/java/org/elasticsearch/tribe/TribeUnitTests.java @@ -28,7 +28,6 @@ import org.elasticsearch.common.SuppressForbidden; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.env.Environment; import org.elasticsearch.node.Node; -import org.elasticsearch.node.internal.InternalSettingsPreparer; import org.elasticsearch.test.ESIntegTestCase; import org.elasticsearch.test.ESTestCase; import org.elasticsearch.test.InternalTestCluster; @@ -111,8 +110,8 @@ public class TribeUnitTests extends ESTestCase { assertThat(state.getClusterName().value(), equalTo("tribe_node_cluster")); assertThat(state.getNodes().getSize(), equalTo(5)); for (DiscoveryNode discoveryNode : state.getNodes()) { - assertThat(discoveryNode.getName(), either(equalTo("tribe1_node")).or(equalTo("tribe2_node")).or(equalTo("tribe_node")) - .or(equalTo("tribe_node/t1")).or(equalTo("tribe_node/t2"))); + assertThat(discoveryNode.getName(), either(equalTo("tribe1_node")).or(equalTo("tribe2_node")) + .or(equalTo("tribe_node")).or(equalTo("tribe_node/t1")).or(equalTo("tribe_node/t2"))); } } }); diff --git a/qa/smoke-test-ingest-disabled/build.gradle b/qa/smoke-test-ingest-disabled/build.gradle index 09b2d1409a1..08dfbf8ae7a 100644 --- a/qa/smoke-test-ingest-disabled/build.gradle +++ b/qa/smoke-test-ingest-disabled/build.gradle @@ -19,6 +19,10 @@ apply plugin: 'elasticsearch.rest-test' +dependencies { + testCompile project(path: ':modules:ingest-common', configuration: 'runtime') +} + integTest { cluster { setting 'node.ingest', 'false' diff --git a/qa/smoke-test-ingest-with-all-dependencies/build.gradle b/qa/smoke-test-ingest-with-all-dependencies/build.gradle index f81afc729b7..60cd9f80a84 100644 --- a/qa/smoke-test-ingest-with-all-dependencies/build.gradle +++ b/qa/smoke-test-ingest-with-all-dependencies/build.gradle @@ -20,9 +20,10 @@ apply plugin: 'elasticsearch.rest-test' dependencies { - testCompile project(path: ':modules:ingest-grok', configuration: 'runtime') + testCompile project(path: ':modules:ingest-common', configuration: 'runtime') testCompile project(path: ':plugins:ingest-geoip', configuration: 'runtime') testCompile project(path: ':modules:lang-mustache', configuration: 'runtime') + testCompile project(path: ':modules:reindex', configuration: 'runtime') } integTest { diff --git a/qa/smoke-test-ingest-with-all-dependencies/src/test/java/org/elasticsearch/ingest/IngestDocumentMustacheIT.java b/qa/smoke-test-ingest-with-all-dependencies/src/test/java/org/elasticsearch/ingest/IngestDocumentMustacheIT.java index c8c50603625..2314e273fc6 100644 --- a/qa/smoke-test-ingest-with-all-dependencies/src/test/java/org/elasticsearch/ingest/IngestDocumentMustacheIT.java +++ b/qa/smoke-test-ingest-with-all-dependencies/src/test/java/org/elasticsearch/ingest/IngestDocumentMustacheIT.java @@ -52,10 +52,12 @@ public class IngestDocumentMustacheIT extends AbstractMustacheTestCase { innerObject.put("qux", Collections.singletonMap("fubar", "hello qux and fubar")); document.put("foo", innerObject); IngestDocument ingestDocument = new IngestDocument("index", "type", "id", null, null, null, null, document); - ingestDocument.setFieldValue(templateService.compile("field1"), ValueSource.wrap("1 {{foo.bar}} {{foo.baz}} {{foo.qux.fubar}}", templateService)); + ingestDocument.setFieldValue(templateService.compile("field1"), + ValueSource.wrap("1 {{foo.bar}} {{foo.baz}} {{foo.qux.fubar}}", templateService)); assertThat(ingestDocument.getFieldValue("field1", String.class), equalTo("1 hello bar hello baz hello qux and fubar")); - ingestDocument.setFieldValue(templateService.compile("field1"), ValueSource.wrap("2 {{_source.foo.bar}} {{_source.foo.baz}} {{_source.foo.qux.fubar}}", templateService)); + ingestDocument.setFieldValue(templateService.compile("field1"), + ValueSource.wrap("2 {{_source.foo.bar}} {{_source.foo.baz}} {{_source.foo.qux.fubar}}", templateService)); assertThat(ingestDocument.getFieldValue("field1", String.class), equalTo("2 hello bar hello baz hello qux and fubar")); } @@ -79,7 +81,9 @@ public class IngestDocumentMustacheIT extends AbstractMustacheTestCase { ingestMap.put("timestamp", "bogus_timestamp"); document.put("_ingest", ingestMap); IngestDocument ingestDocument = new IngestDocument("index", "type", "id", null, null, null, null, document); - ingestDocument.setFieldValue(templateService.compile("ingest_timestamp"), ValueSource.wrap("{{_ingest.timestamp}} and {{_source._ingest.timestamp}}", templateService)); - assertThat(ingestDocument.getFieldValue("ingest_timestamp", String.class), equalTo(ingestDocument.getIngestMetadata().get("timestamp") + " and bogus_timestamp")); + ingestDocument.setFieldValue(templateService.compile("ingest_timestamp"), + ValueSource.wrap("{{_ingest.timestamp}} and {{_source._ingest.timestamp}}", templateService)); + assertThat(ingestDocument.getFieldValue("ingest_timestamp", String.class), + equalTo(ingestDocument.getIngestMetadata().get("timestamp") + " and bogus_timestamp")); } } diff --git a/qa/smoke-test-ingest-with-all-dependencies/src/test/java/org/elasticsearch/ingest/IngestMustacheSetProcessorIT.java b/qa/smoke-test-ingest-with-all-dependencies/src/test/java/org/elasticsearch/ingest/IngestMustacheSetProcessorIT.java deleted file mode 100644 index ed5ad65466b..00000000000 --- a/qa/smoke-test-ingest-with-all-dependencies/src/test/java/org/elasticsearch/ingest/IngestMustacheSetProcessorIT.java +++ /dev/null @@ -1,72 +0,0 @@ -/* - * Licensed to Elasticsearch under one or more contributor - * license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright - * ownership. Elasticsearch licenses this file to you under - * the Apache License, Version 2.0 (the "License"); you may - * not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ - -package org.elasticsearch.ingest; - - -import org.elasticsearch.ingest.core.IngestDocument; -import org.elasticsearch.ingest.core.ValueSource; -import org.elasticsearch.ingest.core.Processor; -import org.elasticsearch.ingest.processor.SetProcessor; -import org.hamcrest.Matchers; - -import java.util.Collections; -import java.util.HashMap; -import java.util.Map; - -import static org.hamcrest.CoreMatchers.equalTo; -import static org.hamcrest.CoreMatchers.instanceOf; - -public class IngestMustacheSetProcessorIT extends AbstractMustacheTestCase { - - public void testExpression() throws Exception { - SetProcessor processor = createSetProcessor("_index", "text {{var}}"); - assertThat(processor.getValue(), instanceOf(ValueSource.TemplatedValue.class)); - assertThat(processor.getValue().copyAndResolve(Collections.singletonMap("var", "_value")), equalTo("text _value")); - } - - public void testSetMetadataWithTemplates() throws Exception { - IngestDocument.MetaData randomMetaData = randomFrom(IngestDocument.MetaData.values()); - Processor processor = createSetProcessor(randomMetaData.getFieldName(), "_value {{field}}"); - IngestDocument ingestDocument = createIngestDocument(Collections.singletonMap("field", "value")); - processor.execute(ingestDocument); - assertThat(ingestDocument.getFieldValue(randomMetaData.getFieldName(), String.class), Matchers.equalTo("_value value")); - } - - public void testSetWithTemplates() throws Exception { - IngestDocument.MetaData randomMetaData = randomFrom(IngestDocument.MetaData.INDEX, IngestDocument.MetaData.TYPE, IngestDocument.MetaData.ID); - Processor processor = createSetProcessor("field{{_type}}", "_value {{" + randomMetaData.getFieldName() + "}}"); - IngestDocument ingestDocument = createIngestDocument(new HashMap<>()); - processor.execute(ingestDocument); - assertThat(ingestDocument.getFieldValue("field_type", String.class), Matchers.equalTo("_value " + ingestDocument.getFieldValue(randomMetaData.getFieldName(), String.class))); - } - - private SetProcessor createSetProcessor(String fieldName, Object fieldValue) throws Exception { - SetProcessor.Factory factory = new SetProcessor.Factory(templateService); - Map config = new HashMap<>(); - config.put("field", fieldName); - config.put("value", fieldValue); - return factory.create(config); - } - - private IngestDocument createIngestDocument(Map source) { - return new IngestDocument("_index", "_type", "_id", null, null, null, null, source); - } - -} diff --git a/qa/smoke-test-ingest-with-all-dependencies/src/test/resources/rest-api-spec/test/ingest/20_combine_processors.yaml b/qa/smoke-test-ingest-with-all-dependencies/src/test/resources/rest-api-spec/test/ingest/20_combine_processors.yaml index 45bda3146a6..7ccaa64b9f4 100644 --- a/qa/smoke-test-ingest-with-all-dependencies/src/test/resources/rest-api-spec/test/ingest/20_combine_processors.yaml +++ b/qa/smoke-test-ingest-with-all-dependencies/src/test/resources/rest-api-spec/test/ingest/20_combine_processors.yaml @@ -67,7 +67,7 @@ - match: { _source.httpversion: "1.1" } - match: { _source.timestamp: "2014-09-08T02:54:42.000Z" } - match: { _source.geoip.continent_name: "North America" } - - match: { _source.geoip.city_name: "Charlotte" } + - match: { _source.geoip.city_name: "Fayetteville" } - match: { _source.geoip.country_iso_code: "US" } - match: { _source.geoip.region_name: "North Carolina" } diff --git a/modules/reindex/src/test/resources/rest-api-spec/test/update_by_query/60_with_ingest.yaml b/qa/smoke-test-ingest-with-all-dependencies/src/test/resources/rest-api-spec/test/ingest/30_with_ingest.yaml similarity index 100% rename from modules/reindex/src/test/resources/rest-api-spec/test/update_by_query/60_with_ingest.yaml rename to qa/smoke-test-ingest-with-all-dependencies/src/test/resources/rest-api-spec/test/ingest/30_with_ingest.yaml diff --git a/modules/reindex/src/test/resources/rest-api-spec/test/reindex/70_with_ingest.yaml b/qa/smoke-test-ingest-with-all-dependencies/src/test/resources/rest-api-spec/test/ingest/40_with_ingest.yaml similarity index 100% rename from modules/reindex/src/test/resources/rest-api-spec/test/reindex/70_with_ingest.yaml rename to qa/smoke-test-ingest-with-all-dependencies/src/test/resources/rest-api-spec/test/ingest/40_with_ingest.yaml diff --git a/qa/smoke-test-reindex-with-groovy/src/test/resources/rest-api-spec/test/reindex/10_script.yaml b/qa/smoke-test-reindex-with-groovy/src/test/resources/rest-api-spec/test/reindex/10_script.yaml index aa553a5c9dc..783838d7f8e 100644 --- a/qa/smoke-test-reindex-with-groovy/src/test/resources/rest-api-spec/test/reindex/10_script.yaml +++ b/qa/smoke-test-reindex-with-groovy/src/test/resources/rest-api-spec/test/reindex/10_script.yaml @@ -284,6 +284,11 @@ user: notfoo - match: { hits.total: 0 } + - do: + count: + index: new_twitter + - match: {count: 1} + --- "Noop all docs": - do: @@ -313,6 +318,11 @@ - match: {updated: 0} - match: {noops: 2} + - do: + indices.exists: + index: new_twitter + - is_false: '' + --- "Set version to null to force an update": - do: @@ -443,3 +453,85 @@ match: user: another - match: { hits.total: 1 } + +--- +"Reindex all docs with one doc deletion": + # Source index + - do: + index: + index: index1 + type: type1 + id: 1 + body: { "lang": "en", "id": 123 } + - do: + index: + index: index1 + type: type1 + id: 2 + body: { "lang": "en", "id": 456 } + - do: + index: + index: index1 + type: type1 + id: 3 + body: { "lang": "fr", "id": 789 } + # Destination index + - do: + index: + index: index2 + type: type2 + id: fr_789 + body: { "lang": "fr", "id": 789 } + - do: + index: + index: index2 + type: type2 + id: en_123 + body: { "lang": "en", "id": 123 } + - do: + indices.refresh: {} + + # Reindex all documents from "index1" into "index2", changing their type + # to "type2" and their id to the concatened lang+id fields, + # trashing all non-english pre existing ones + - do: + reindex: + refresh: true + body: + source: + index: index1 + dest: + index: index2 + type: type2 + script: + inline: "ctx._id = ctx._source.lang + '_' + ctx._source.id; + if (ctx._source.lang != \"en\" ) {ctx.op = 'delete'}" + - match: {created: 1} + - match: {noops: 0} + - match: {updated: 1} + - match: {deleted: 1} + + - do: + mget: + body: + docs: + - { _index: index2, _type: type2, _id: en_123} + - { _index: index2, _type: type2, _id: en_456} + - { _index: index2, _type: type2, _id: fr_789} + + - is_true: docs.0.found + - match: { docs.0._index: index2 } + - match: { docs.0._type: type2 } + - match: { docs.0._id: en_123 } + - match: { docs.0._version: 2 } + + - is_true: docs.1.found + - match: { docs.1._index: index2 } + - match: { docs.1._type: type2 } + - match: { docs.1._id: en_456 } + - match: { docs.1._version: 1 } + + - is_false: docs.2.found + - match: { docs.2._index: index2 } + - match: { docs.2._type: type2 } + - match: { docs.2._id: fr_789 } diff --git a/qa/smoke-test-reindex-with-groovy/src/test/resources/rest-api-spec/test/update_by_query/10_script.yaml b/qa/smoke-test-reindex-with-groovy/src/test/resources/rest-api-spec/test/update_by_query/10_script.yaml index 54a79ac1e32..e4fef86b1d1 100644 --- a/qa/smoke-test-reindex-with-groovy/src/test/resources/rest-api-spec/test/update_by_query/10_script.yaml +++ b/qa/smoke-test-reindex-with-groovy/src/test/resources/rest-api-spec/test/update_by_query/10_script.yaml @@ -138,3 +138,171 @@ body: script: inline: ctx._id = "stuff" + +--- +"Update all docs with one doc deletion": + - do: + index: + index: twitter + type: tweet + id: 1 + body: { "level": 9, "last_updated": "2016-01-01T12:10:30Z" } + - do: + index: + index: twitter + type: tweet + id: 2 + body: { "level": 10, "last_updated": "2016-01-01T12:10:30Z" } + - do: + index: + index: twitter + type: tweet + id: 3 + body: { "level": 11, "last_updated": "2016-01-01T12:10:30Z" } + - do: + index: + index: twitter + type: tweet + id: 4 + body: { "level": 12, "last_updated": "2016-01-01T12:10:30Z" } + - do: + indices.refresh: {} + + - do: + update_by_query: + refresh: true + index: twitter + body: + script: + inline: if (ctx._source.level != 11) {ctx._source.last_updated = "2016-01-02T00:00:00Z"} else {ctx.op = "delete"} + - match: {updated: 3} + - match: {deleted: 1} + - match: {noops: 0} + + - do: + search: + index: twitter + body: + query: + match: + last_updated: "2016-01-02T00:00:00Z" + - match: { hits.total: 3 } + + - do: + search: + index: twitter + body: + query: + term: + level: 11 + - match: { hits.total: 0 } + + - do: + count: + index: twitter + - match: {count: 3} + +--- +"Update all docs with one deletion and one noop": + - do: + index: + index: twitter + type: tweet + id: 1 + body: { "level": 9, "last_updated": "2016-01-01T12:10:30Z" } + - do: + index: + index: twitter + type: tweet + id: 2 + body: { "level": 10, "last_updated": "2016-01-01T12:10:30Z" } + - do: + index: + index: twitter + type: tweet + id: 3 + body: { "level": 11, "last_updated": "2016-01-01T12:10:30Z" } + - do: + index: + index: twitter + type: tweet + id: 4 + body: { "level": 12, "last_updated": "2016-01-01T12:10:30Z" } + - do: + indices.refresh: {} + + - do: + update_by_query: + refresh: true + index: twitter + body: + script: + inline: "switch (ctx._source.level % 3) { + case 0: + ctx._source.last_updated = \"2016-01-02T00:00:00Z\"; + break; + case 1: + ctx.op = \"noop\"; + break; + case 2: + ctx.op = \"delete\"; + break; + }" + - match: {updated: 2} + - match: {deleted: 1} + - match: {noops: 1} + + - do: + search: + index: twitter + body: + query: + match: + last_updated: "2016-01-02T00:00:00Z" + - match: { hits.total: 2 } + + - do: + search: + index: twitter + body: + query: + match: + last_updated: "2016-01-01T12:10:30Z" + - match: { hits.total: 1 } + + - do: + search: + index: twitter + body: + query: + term: + level: 11 + - match: { hits.total: 0 } + +--- +"Set unsupported operation type": + - do: + index: + index: twitter + type: tweet + id: 1 + body: { "user": "kimchy" } + - do: + index: + index: twitter + type: tweet + id: 2 + body: { "user": "foo" } + - do: + indices.refresh: {} + + - do: + catch: request + update_by_query: + refresh: true + index: twitter + body: + script: + inline: if (ctx._source.user == "kimchy") {ctx.op = "update"} else {ctx.op = "junk"} + + - match: { error.reason: 'Operation type [junk] not allowed, only [noop, index, delete] are allowed' } diff --git a/qa/vagrant/src/test/resources/packaging/scripts/module_and_plugin_test_cases.bash b/qa/vagrant/src/test/resources/packaging/scripts/module_and_plugin_test_cases.bash index 32118fda389..ac54bf3b664 100644 --- a/qa/vagrant/src/test/resources/packaging/scripts/module_and_plugin_test_cases.bash +++ b/qa/vagrant/src/test/resources/packaging/scripts/module_and_plugin_test_cases.bash @@ -228,8 +228,8 @@ fi install_and_check_plugin ingest geoip geoip2-*.jar jackson-annotations-*.jar jackson-databind-*.jar maxmind-db-*.jar } -@test "[$GROUP] check ingest-grok module" { - check_module ingest-grok jcodings-*.jar joni-*.jar +@test "[$GROUP] check ingest-common module" { + check_module ingest-common jcodings-*.jar joni-*.jar } @test "[$GROUP] check lang-expression module" { diff --git a/rest-api-spec/src/main/resources/rest-api-spec/api/bulk.json b/rest-api-spec/src/main/resources/rest-api-spec/api/bulk.json index 590054b04a4..a75daf35204 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/api/bulk.json +++ b/rest-api-spec/src/main/resources/rest-api-spec/api/bulk.json @@ -22,8 +22,9 @@ "description" : "Explicit write consistency setting for the operation" }, "refresh": { - "type" : "boolean", - "description" : "Refresh the index after performing the operation" + "type" : "enum", + "options": ["true", "false", "wait_for"], + "description" : "If `true` then refresh the effected shards to make this operation visible to search, if `wait_for` then wait for a refresh to make this operation visible to search, if `false` (the default) then do nothing with refreshes." }, "routing": { "type" : "string", diff --git a/rest-api-spec/src/main/resources/rest-api-spec/api/delete.json b/rest-api-spec/src/main/resources/rest-api-spec/api/delete.json index be09c0179d4..5bb0e3fed4c 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/api/delete.json +++ b/rest-api-spec/src/main/resources/rest-api-spec/api/delete.json @@ -33,8 +33,9 @@ "description" : "ID of parent document" }, "refresh": { - "type" : "boolean", - "description" : "Refresh the index after performing the operation" + "type" : "enum", + "options": ["true", "false", "wait_for"], + "description" : "If `true` then refresh the effected shards to make this operation visible to search, if `wait_for` then wait for a refresh to make this operation visible to search, if `false` (the default) then do nothing with refreshes." }, "routing": { "type" : "string", diff --git a/rest-api-spec/src/main/resources/rest-api-spec/api/index.json b/rest-api-spec/src/main/resources/rest-api-spec/api/index.json index 5c13f67c212..b7f7eeb9ef5 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/api/index.json +++ b/rest-api-spec/src/main/resources/rest-api-spec/api/index.json @@ -38,8 +38,9 @@ "description" : "ID of the parent document" }, "refresh": { - "type" : "boolean", - "description" : "Refresh the index after performing the operation" + "type" : "enum", + "options": ["true", "false", "wait_for"], + "description" : "If `true` then refresh the effected shards to make this operation visible to search, if `wait_for` then wait for a refresh to make this operation visible to search, if `false` (the default) then do nothing with refreshes." }, "routing": { "type" : "string", diff --git a/rest-api-spec/src/main/resources/rest-api-spec/api/update.json b/rest-api-spec/src/main/resources/rest-api-spec/api/update.json index 20fc3524283..4a3f134301d 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/api/update.json +++ b/rest-api-spec/src/main/resources/rest-api-spec/api/update.json @@ -41,8 +41,9 @@ "description": "ID of the parent document. Is is only used for routing and when for the upsert request" }, "refresh": { - "type": "boolean", - "description": "Refresh the index after performing the operation" + "type" : "enum", + "options": ["true", "false", "wait_for"], + "description" : "If `true` then refresh the effected shards to make this operation visible to search, if `wait_for` then wait for a refresh to make this operation visible to search, if `false` (the default) then do nothing with refreshes." }, "retry_on_conflict": { "type": "number", diff --git a/rest-api-spec/src/main/resources/rest-api-spec/test/bulk/50_refresh.yaml b/rest-api-spec/src/main/resources/rest-api-spec/test/bulk/50_refresh.yaml new file mode 100644 index 00000000000..4906975bfab --- /dev/null +++ b/rest-api-spec/src/main/resources/rest-api-spec/test/bulk/50_refresh.yaml @@ -0,0 +1,48 @@ +--- +"refresh=true immediately makes changes are visible in search": + - do: + bulk: + refresh: true + body: | + {"index": {"_index": "test_index", "_type": "test_type", "_id": "test_id"}} + {"f1": "v1", "f2": 42} + {"index": {"_index": "test_index", "_type": "test_type", "_id": "test_id2"}} + {"f1": "v2", "f2": 47} + + - do: + count: + index: test_index + - match: {count: 2} + +--- +"refresh=empty string immediately makes changes are visible in search": + - do: + bulk: + refresh: "" + body: | + {"index": {"_index": "test_index", "_type": "test_type", "_id": "test_id"}} + {"f1": "v1", "f2": 42} + {"index": {"_index": "test_index", "_type": "test_type", "_id": "test_id2"}} + {"f1": "v2", "f2": 47} + + - do: + count: + index: test_index + - match: {count: 2} + + +--- +"refresh=wait_for waits until changes are visible in search": + - do: + bulk: + refresh: wait_for + body: | + {"index": {"_index": "test_index", "_type": "test_type", "_id": "test_id"}} + {"f1": "v1", "f2": 42} + {"index": {"_index": "test_index", "_type": "test_type", "_id": "test_id2"}} + {"f1": "v2", "f2": 47} + + - do: + count: + index: test_index + - match: {count: 2} diff --git a/rest-api-spec/src/main/resources/rest-api-spec/test/create/60_refresh.yaml b/rest-api-spec/src/main/resources/rest-api-spec/test/create/60_refresh.yaml index 99bfbc3cff6..90dc28bcfc0 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/test/create/60_refresh.yaml +++ b/rest-api-spec/src/main/resources/rest-api-spec/test/create/60_refresh.yaml @@ -33,8 +33,9 @@ index: test_1 type: test id: 2 - refresh: 1 + refresh: true body: { foo: bar } + - is_true: forced_refresh - do: search: @@ -44,3 +45,42 @@ query: { term: { _id: 2 }} - match: { hits.total: 1 } + +--- +"When refresh url parameter is an empty string that means \"refresh immediately\"": + - do: + create: + index: test_1 + type: test + id: 1 + refresh: "" + body: { foo: bar } + - is_true: forced_refresh + + - do: + search: + index: test_1 + type: test + body: + query: { term: { _id: 1 }} + + - match: { hits.total: 1 } + +--- +"refresh=wait_for waits until changes are visible in search": + - do: + index: + index: test_1 + type: test + id: 1 + body: { foo: bar } + refresh: wait_for + - is_false: forced_refresh + + - do: + search: + index: test_1 + type: test + body: + query: { term: { _id: 1 }} + - match: { hits.total: 1 } diff --git a/rest-api-spec/src/main/resources/rest-api-spec/test/delete/50_refresh.yaml b/rest-api-spec/src/main/resources/rest-api-spec/test/delete/50_refresh.yaml index 4d3f9fe039d..9ea6bc033de 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/test/delete/50_refresh.yaml +++ b/rest-api-spec/src/main/resources/rest-api-spec/test/delete/50_refresh.yaml @@ -19,7 +19,7 @@ type: test id: 1 body: { foo: bar } - refresh: 1 + refresh: true # If you wonder why this document get 3 as an id instead of 2, it is because the # current routing algorithm would route 1 and 2 to the same shard while we need @@ -30,7 +30,8 @@ type: test id: 3 body: { foo: bar } - refresh: 1 + refresh: true + - is_true: forced_refresh - do: search: @@ -61,7 +62,7 @@ index: test_1 type: test id: 3 - refresh: 1 + refresh: true # If a replica shard where doc 1 is located gets initialized at this point, doc 1 # won't be found by the following search as the shard gets automatically refreshed @@ -75,3 +76,72 @@ query: { terms: { _id: [1,3] }} - match: { hits.total: 1 } + +--- +"When refresh url parameter is an empty string that means \"refresh immediately\"": + - do: + index: + index: test_1 + type: test + id: 1 + body: { foo: bar } + refresh: true + - is_true: forced_refresh + + - do: + search: + index: test_1 + type: test + body: + query: { term: { _id: 1 }} + - match: { hits.total: 1 } + + - do: + delete: + index: test_1 + type: test + id: 1 + refresh: "" + + - do: + search: + index: test_1 + type: test + body: + query: { term: { _id: 1 }} + - match: { hits.total: 0 } + +--- +"refresh=wait_for waits until changes are visible in search": + - do: + index: + index: test_1 + type: test + id: 1 + body: { foo: bar } + refresh: true + - is_true: forced_refresh + + - do: + search: + index: test_1 + type: test + body: + query: { term: { _id: 1 }} + - match: { hits.total: 1 } + + - do: + delete: + index: test_1 + type: test + id: 1 + refresh: wait_for + - is_false: forced_refresh + + - do: + search: + index: test_1 + type: test + body: + query: { term: { _id: 1 }} + - match: { hits.total: 0 } diff --git a/rest-api-spec/src/main/resources/rest-api-spec/test/index/60_refresh.yaml b/rest-api-spec/src/main/resources/rest-api-spec/test/index/60_refresh.yaml index af6ea59766f..4ee26411432 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/test/index/60_refresh.yaml +++ b/rest-api-spec/src/main/resources/rest-api-spec/test/index/60_refresh.yaml @@ -33,8 +33,9 @@ index: test_1 type: test id: 2 - refresh: 1 + refresh: true body: { foo: bar } + - is_true: forced_refresh - do: search: @@ -44,3 +45,42 @@ query: { term: { _id: 2 }} - match: { hits.total: 1 } + +--- +"When refresh url parameter is an empty string that means \"refresh immediately\"": + - do: + index: + index: test_1 + type: test + id: 1 + refresh: "" + body: { foo: bar } + - is_true: forced_refresh + + - do: + search: + index: test_1 + type: test + body: + query: { term: { _id: 1 }} + + - match: { hits.total: 1 } + +--- +"refresh=wait_for waits until changes are visible in search": + - do: + index: + index: test_1 + type: test + id: 1 + body: { foo: bar } + refresh: wait_for + - is_false: forced_refresh + + - do: + search: + index: test_1 + type: test + body: + query: { term: { _id: 1 }} + - match: { hits.total: 1 } diff --git a/rest-api-spec/src/main/resources/rest-api-spec/test/ingest/10_basic.yaml b/rest-api-spec/src/main/resources/rest-api-spec/test/ingest/10_basic.yaml new file mode 100644 index 00000000000..74396249e81 --- /dev/null +++ b/rest-api-spec/src/main/resources/rest-api-spec/test/ingest/10_basic.yaml @@ -0,0 +1,45 @@ +--- +"Test basic pipeline crud": + - do: + ingest.put_pipeline: + id: "my_pipeline" + body: > + { + "description": "_description", + "processors": [ + ] + } + - match: { acknowledged: true } + + - do: + ingest.get_pipeline: + id: "my_pipeline" + - match: { pipelines.0.id: "my_pipeline" } + - match: { pipelines.0.config.description: "_description" } + + - do: + ingest.delete_pipeline: + id: "my_pipeline" + - match: { acknowledged: true } + + - do: + catch: missing + ingest.get_pipeline: + id: "my_pipeline" + +--- +"Test invalid config": + - do: + catch: /parse_exception/ + ingest.put_pipeline: + id: "my_pipeline" + body: > + { + "description": "_description", + "processors": [ + { + "set" : { + } + } + ] + } diff --git a/rest-api-spec/src/main/resources/rest-api-spec/test/update/60_refresh.yaml b/rest-api-spec/src/main/resources/rest-api-spec/test/update/60_refresh.yaml index 6048292ceab..8c0e7e66c97 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/test/update/60_refresh.yaml +++ b/rest-api-spec/src/main/resources/rest-api-spec/test/update/60_refresh.yaml @@ -35,10 +35,11 @@ index: test_1 type: test id: 2 - refresh: 1 + refresh: true body: doc: { foo: baz } upsert: { foo: bar } + - is_true: forced_refresh - do: search: @@ -48,3 +49,70 @@ query: { term: { _id: 2 }} - match: { hits.total: 1 } + +--- +"When refresh url parameter is an empty string that means \"refresh immediately\"": + - do: + index: + index: test_1 + type: test + id: 1 + refresh: true + body: { foo: bar } + - is_true: forced_refresh + + - do: + update: + index: test_1 + type: test + id: 1 + refresh: "" + body: + doc: {cat: dog} + - is_true: forced_refresh + + - do: + search: + index: test_1 + type: test + body: + query: { term: { cat: dog }} + + - match: { hits.total: 1 } + +--- +"refresh=wait_for waits until changes are visible in search": + - do: + index: + index: test_1 + type: test + id: 1 + body: { foo: bar } + refresh: true + - is_true: forced_refresh + + - do: + search: + index: test_1 + type: test + body: + query: { term: { _id: 1 }} + - match: { hits.total: 1 } + + - do: + update: + index: test_1 + type: test + id: 1 + refresh: wait_for + body: + doc: { test: asdf } + - is_false: forced_refresh + + - do: + search: + index: test_1 + type: test + body: + query: { match: { test: asdf } } + - match: { hits.total: 1 } diff --git a/settings.gradle b/settings.gradle index dbaf9fa8aa9..636a7774954 100644 --- a/settings.gradle +++ b/settings.gradle @@ -17,7 +17,7 @@ List projects = [ 'test:fixtures:hdfs-fixture', 'test:logger-usage', 'modules:aggs-matrix-stats', - 'modules:ingest-grok', + 'modules:ingest-common', 'modules:lang-expression', 'modules:lang-groovy', 'modules:lang-mustache', diff --git a/test/framework/src/main/java/org/elasticsearch/ingest/IngestTestPlugin.java b/test/framework/src/main/java/org/elasticsearch/ingest/IngestTestPlugin.java new file mode 100644 index 00000000000..696799d31fb --- /dev/null +++ b/test/framework/src/main/java/org/elasticsearch/ingest/IngestTestPlugin.java @@ -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.ingest; + +import org.elasticsearch.node.NodeModule; +import org.elasticsearch.plugins.Plugin; + +/** + * Adds an ingest processor to be used in tests. + */ +public class IngestTestPlugin extends Plugin { + + @Override + public String name() { + return "ingest-test"; + } + + @Override + public String description() { + return "Contains an ingest processor to be used in tests"; + } + + public void onModule(NodeModule nodeModule) { + nodeModule.registerProcessor("test", (templateService, registry) -> config -> + new TestProcessor("id", "test", doc -> { + doc.setFieldValue("processed", true); + if (doc.hasField("fail") && doc.getFieldValue("fail", Boolean.class)) { + throw new IllegalArgumentException("test processor failed"); + } + }) + ); + } +} diff --git a/test/framework/src/main/java/org/elasticsearch/test/ESTestCase.java b/test/framework/src/main/java/org/elasticsearch/test/ESTestCase.java index 972c1a50fbc..8f3fd5010e6 100644 --- a/test/framework/src/main/java/org/elasticsearch/test/ESTestCase.java +++ b/test/framework/src/main/java/org/elasticsearch/test/ESTestCase.java @@ -28,7 +28,6 @@ import com.carrotsearch.randomizedtesting.generators.RandomInts; import com.carrotsearch.randomizedtesting.generators.RandomPicks; import com.carrotsearch.randomizedtesting.generators.RandomStrings; import com.carrotsearch.randomizedtesting.rules.TestRuleAdapter; - import org.apache.lucene.uninverting.UninvertingReader; import org.apache.lucene.util.LuceneTestCase; import org.apache.lucene.util.LuceneTestCase.SuppressCodecs; @@ -56,6 +55,7 @@ import org.elasticsearch.env.Environment; import org.elasticsearch.env.NodeEnvironment; import org.elasticsearch.index.Index; import org.elasticsearch.index.analysis.AnalysisService; +import org.elasticsearch.indices.IndicesService; import org.elasticsearch.indices.analysis.AnalysisModule; import org.elasticsearch.search.MockSearchService; import org.elasticsearch.test.junit.listeners.LoggingListener; @@ -672,11 +672,11 @@ public abstract class ESTestCase extends LuceneTestCase { return enabled; } - /** - * Asserts busily that there are no files in the specified path - */ - public void assertBusyPathHasBeenCleared(Path path) throws Exception { - assertBusy(() -> assertPathHasBeenCleared(path)); + public void assertAllIndicesRemovedAndDeletionCompleted(Iterable indicesServices) throws Exception { + for (IndicesService indicesService : indicesServices) { + assertBusy(() -> assertFalse(indicesService.iterator().hasNext()), 1, TimeUnit.MINUTES); + assertBusy(() -> assertFalse(indicesService.hasUncompletedPendingDeletes()), 1, TimeUnit.MINUTES); + } } /** diff --git a/test/framework/src/main/java/org/elasticsearch/test/client/NoOpClient.java b/test/framework/src/main/java/org/elasticsearch/test/client/NoOpClient.java index 5f2237640e6..6ff45608700 100644 --- a/test/framework/src/main/java/org/elasticsearch/test/client/NoOpClient.java +++ b/test/framework/src/main/java/org/elasticsearch/test/client/NoOpClient.java @@ -27,6 +27,7 @@ import org.elasticsearch.action.ActionRequestBuilder; import org.elasticsearch.action.ActionResponse; import org.elasticsearch.client.support.AbstractClient; import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.threadpool.TestThreadPool; import org.elasticsearch.threadpool.ThreadPool; import java.util.concurrent.TimeUnit; @@ -34,7 +35,7 @@ import java.util.concurrent.TimeUnit; public class NoOpClient extends AbstractClient { public NoOpClient(String testName) { - super(Settings.EMPTY, new ThreadPool(testName)); + super(Settings.EMPTY, new TestThreadPool(testName)); } @Override diff --git a/qa/smoke-test-ingest-with-all-dependencies/src/test/java/org/elasticsearch/ingest/IngestMustacheRemoveProcessorIT.java b/test/framework/src/main/java/org/elasticsearch/threadpool/TestThreadPool.java similarity index 52% rename from qa/smoke-test-ingest-with-all-dependencies/src/test/java/org/elasticsearch/ingest/IngestMustacheRemoveProcessorIT.java rename to test/framework/src/main/java/org/elasticsearch/threadpool/TestThreadPool.java index 621cbbc2beb..0d525f7f59a 100644 --- a/qa/smoke-test-ingest-with-all-dependencies/src/test/java/org/elasticsearch/ingest/IngestMustacheRemoveProcessorIT.java +++ b/test/framework/src/main/java/org/elasticsearch/threadpool/TestThreadPool.java @@ -17,22 +17,15 @@ * under the License. */ -package org.elasticsearch.ingest; +package org.elasticsearch.threadpool; -import org.elasticsearch.ingest.processor.RemoveProcessor; -import org.hamcrest.CoreMatchers; +import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.node.Node; -import java.util.Collections; -import java.util.HashMap; -import java.util.Map; +public class TestThreadPool extends ThreadPool { -public class IngestMustacheRemoveProcessorIT extends AbstractMustacheTestCase { - - public void testRemoveProcessorMustacheExpression() throws Exception { - RemoveProcessor.Factory factory = new RemoveProcessor.Factory(templateService); - Map config = new HashMap<>(); - config.put("field", "field{{var}}"); - RemoveProcessor processor = factory.create(config); - assertThat(processor.getField().execute(Collections.singletonMap("var", "_value")), CoreMatchers.equalTo("field_value")); + public TestThreadPool(String name) { + super(Settings.builder().put(Node.NODE_NAME_SETTING.getKey(), name).build()); } + }