From 3ca885e9ac6b031b896e62bece07170abe0ac85c Mon Sep 17 00:00:00 2001 From: Tanguy Leroux Date: Fri, 7 Dec 2018 09:30:27 +0100 Subject: [PATCH 01/46] [Close Index API] Add TransportShardCloseAction for pre-closing verifications (#36249) This pull request adds the TransportShardCloseAction which is a transport replication action that acquires all index shard permits for its execution. This action will be used in the future by the MetaDataIndexStateService in a new index closing process, where we need to execute some sanity checks before closing an index. The action executes the following verifications on the primary and replicas: * there is no other on going operation active on the shard * the data node holding the shard knows that the index is blocked for writes * the shard's max sequence number is equal to the global checkpoint When the verifications are done and successful, the shard is flushed. Relates #33888 --- ...TransportVerifyShardBeforeCloseAction.java | 126 ++++++++++++++++++ ...portVerifyShardBeforeCloseActionTests.java | 122 +++++++++++++++++ 2 files changed, 248 insertions(+) create mode 100644 server/src/main/java/org/elasticsearch/action/admin/indices/close/TransportVerifyShardBeforeCloseAction.java create mode 100644 server/src/test/java/org/elasticsearch/action/admin/indices/close/TransportVerifyShardBeforeCloseActionTests.java diff --git a/server/src/main/java/org/elasticsearch/action/admin/indices/close/TransportVerifyShardBeforeCloseAction.java b/server/src/main/java/org/elasticsearch/action/admin/indices/close/TransportVerifyShardBeforeCloseAction.java new file mode 100644 index 0000000000000..ab895dd7af804 --- /dev/null +++ b/server/src/main/java/org/elasticsearch/action/admin/indices/close/TransportVerifyShardBeforeCloseAction.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.action.admin.indices.close; + +import org.elasticsearch.action.ActionListener; +import org.elasticsearch.action.admin.indices.flush.FlushRequest; +import org.elasticsearch.action.support.ActionFilters; +import org.elasticsearch.action.support.replication.ReplicationRequest; +import org.elasticsearch.action.support.replication.ReplicationResponse; +import org.elasticsearch.action.support.replication.TransportReplicationAction; +import org.elasticsearch.cluster.action.shard.ShardStateAction; +import org.elasticsearch.cluster.block.ClusterBlock; +import org.elasticsearch.cluster.block.ClusterBlocks; +import org.elasticsearch.cluster.metadata.IndexNameExpressionResolver; +import org.elasticsearch.cluster.metadata.MetaDataIndexStateService; +import org.elasticsearch.cluster.service.ClusterService; +import org.elasticsearch.common.inject.Inject; +import org.elasticsearch.common.lease.Releasable; +import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.index.shard.IndexShard; +import org.elasticsearch.index.shard.ShardId; +import org.elasticsearch.indices.IndicesService; +import org.elasticsearch.threadpool.ThreadPool; +import org.elasticsearch.transport.TransportService; + +public class TransportVerifyShardBeforeCloseAction extends TransportReplicationAction< + TransportVerifyShardBeforeCloseAction.ShardCloseRequest, TransportVerifyShardBeforeCloseAction.ShardCloseRequest, ReplicationResponse> { + + public static final String NAME = CloseIndexAction.NAME + "[s]"; + private static final ClusterBlock EXPECTED_BLOCK = MetaDataIndexStateService.INDEX_CLOSED_BLOCK; + + @Inject + public TransportVerifyShardBeforeCloseAction(final Settings settings, final TransportService transportService, + final ClusterService clusterService, final IndicesService indicesService, + final ThreadPool threadPool, final ShardStateAction stateAction, + final ActionFilters actionFilters, final IndexNameExpressionResolver resolver) { + super(settings, NAME, transportService, clusterService, indicesService, threadPool, stateAction, actionFilters, resolver, + ShardCloseRequest::new, ShardCloseRequest::new, ThreadPool.Names.MANAGEMENT); + } + + @Override + protected ReplicationResponse newResponseInstance() { + return new ReplicationResponse(); + } + + @Override + protected void acquirePrimaryOperationPermit(final IndexShard primary, + final ShardCloseRequest request, + final ActionListener onAcquired) { + primary.acquireAllPrimaryOperationsPermits(onAcquired, request.timeout()); + } + + @Override + protected void acquireReplicaOperationPermit(final IndexShard replica, + final ShardCloseRequest request, + final ActionListener onAcquired, + final long primaryTerm, + final long globalCheckpoint, + final long maxSeqNoOfUpdateOrDeletes) { + replica.acquireAllReplicaOperationsPermits(primaryTerm, globalCheckpoint, maxSeqNoOfUpdateOrDeletes, onAcquired, request.timeout()); + } + + @Override + protected PrimaryResult shardOperationOnPrimary(final ShardCloseRequest shardRequest, + final IndexShard primary) throws Exception { + executeShardOperation(primary); + return new PrimaryResult<>(shardRequest, new ReplicationResponse()); + } + + @Override + protected ReplicaResult shardOperationOnReplica(final ShardCloseRequest shardRequest, final IndexShard replica) throws Exception { + executeShardOperation(replica); + return new ReplicaResult(); + } + + private void executeShardOperation(final IndexShard indexShard) { + final ShardId shardId = indexShard.shardId(); + if (indexShard.getActiveOperationsCount() != 0) { + throw new IllegalStateException("On-going operations in progress while checking index shard " + shardId + " before closing"); + } + + final ClusterBlocks clusterBlocks = clusterService.state().blocks(); + if (clusterBlocks.hasIndexBlock(shardId.getIndexName(), EXPECTED_BLOCK) == false) { + throw new IllegalStateException("Index shard " + shardId + " must be blocked by " + EXPECTED_BLOCK + " before closing"); + } + + final long maxSeqNo = indexShard.seqNoStats().getMaxSeqNo(); + if (indexShard.getGlobalCheckpoint() != maxSeqNo) { + throw new IllegalStateException("Global checkpoint [" + indexShard.getGlobalCheckpoint() + + "] mismatches maximum sequence number [" + maxSeqNo + "] on index shard " + shardId); + } + indexShard.flush(new FlushRequest()); + logger.debug("{} shard is ready for closing", shardId); + } + + public static class ShardCloseRequest extends ReplicationRequest { + + ShardCloseRequest(){ + } + + public ShardCloseRequest(final ShardId shardId) { + super(shardId); + } + + @Override + public String toString() { + return "close shard {" + shardId + "}"; + } + } +} diff --git a/server/src/test/java/org/elasticsearch/action/admin/indices/close/TransportVerifyShardBeforeCloseActionTests.java b/server/src/test/java/org/elasticsearch/action/admin/indices/close/TransportVerifyShardBeforeCloseActionTests.java new file mode 100644 index 0000000000000..ae1a0b8768946 --- /dev/null +++ b/server/src/test/java/org/elasticsearch/action/admin/indices/close/TransportVerifyShardBeforeCloseActionTests.java @@ -0,0 +1,122 @@ +/* + * 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.admin.indices.close; + +import org.elasticsearch.action.admin.indices.flush.FlushRequest; +import org.elasticsearch.action.support.ActionFilters; +import org.elasticsearch.cluster.ClusterName; +import org.elasticsearch.cluster.ClusterState; +import org.elasticsearch.cluster.action.shard.ShardStateAction; +import org.elasticsearch.cluster.block.ClusterBlocks; +import org.elasticsearch.cluster.metadata.IndexNameExpressionResolver; +import org.elasticsearch.cluster.service.ClusterService; +import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.index.seqno.SeqNoStats; +import org.elasticsearch.index.seqno.SequenceNumbers; +import org.elasticsearch.index.shard.IndexShard; +import org.elasticsearch.index.shard.ShardId; +import org.elasticsearch.indices.IndicesService; +import org.elasticsearch.test.ESTestCase; +import org.elasticsearch.threadpool.ThreadPool; +import org.elasticsearch.transport.TransportService; +import org.junit.Before; + +import static org.elasticsearch.cluster.metadata.MetaDataIndexStateService.INDEX_CLOSED_BLOCK; +import static org.hamcrest.Matchers.equalTo; +import static org.mockito.Matchers.any; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.times; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.when; + +public class TransportVerifyShardBeforeCloseActionTests extends ESTestCase { + + private IndexShard indexShard; + private TransportVerifyShardBeforeCloseAction action; + private ClusterService clusterService; + + @Override + @Before + public void setUp() throws Exception { + super.setUp(); + + indexShard = mock(IndexShard.class); + when(indexShard.getActiveOperationsCount()).thenReturn(0); + when(indexShard.getGlobalCheckpoint()).thenReturn(0L); + when(indexShard.seqNoStats()).thenReturn(new SeqNoStats(0L, 0L, 0L)); + + final ShardId shardId = new ShardId("index", "_na_", randomIntBetween(0, 3)); + when(indexShard.shardId()).thenReturn(shardId); + + clusterService = mock(ClusterService.class); + when(clusterService.state()).thenReturn(new ClusterState.Builder(new ClusterName("test")) + .blocks(ClusterBlocks.builder().addIndexBlock("index", INDEX_CLOSED_BLOCK).build()).build()); + + action = new TransportVerifyShardBeforeCloseAction(Settings.EMPTY, mock(TransportService.class), clusterService, + mock(IndicesService.class), mock(ThreadPool.class), mock(ShardStateAction.class), mock(ActionFilters.class), + mock(IndexNameExpressionResolver.class)); + } + + private void executeOnPrimaryOrReplica() throws Exception { + final TransportVerifyShardBeforeCloseAction.ShardCloseRequest request = + new TransportVerifyShardBeforeCloseAction.ShardCloseRequest(indexShard.shardId()); + if (randomBoolean()) { + assertNotNull(action.shardOperationOnPrimary(request, indexShard)); + } else { + assertNotNull(action.shardOperationOnPrimary(request, indexShard)); + } + } + + public void testOperationSuccessful() throws Exception { + executeOnPrimaryOrReplica(); + verify(indexShard, times(1)).flush(any(FlushRequest.class)); + } + + public void testOperationFailsWithOnGoingOps() { + when(indexShard.getActiveOperationsCount()).thenReturn(randomIntBetween(1, 10)); + + IllegalStateException exception = expectThrows(IllegalStateException.class, this::executeOnPrimaryOrReplica); + assertThat(exception.getMessage(), + equalTo("On-going operations in progress while checking index shard " + indexShard.shardId() + " before closing")); + verify(indexShard, times(0)).flush(any(FlushRequest.class)); + } + + public void testOperationFailsWithNoBlock() { + when(clusterService.state()).thenReturn(new ClusterState.Builder(new ClusterName("test")).build()); + + IllegalStateException exception = expectThrows(IllegalStateException.class, this::executeOnPrimaryOrReplica); + assertThat(exception.getMessage(), + equalTo("Index shard " + indexShard.shardId() + " must be blocked by " + INDEX_CLOSED_BLOCK + " before closing")); + verify(indexShard, times(0)).flush(any(FlushRequest.class)); + } + + public void testOperationFailsWithGlobalCheckpointNotCaughtUp() { + final long maxSeqNo = randomLongBetween(SequenceNumbers.UNASSIGNED_SEQ_NO, Long.MAX_VALUE); + final long localCheckpoint = randomLongBetween(SequenceNumbers.UNASSIGNED_SEQ_NO, maxSeqNo); + final long globalCheckpoint = randomValueOtherThan(maxSeqNo, + () -> randomLongBetween(SequenceNumbers.UNASSIGNED_SEQ_NO, localCheckpoint)); + when(indexShard.seqNoStats()).thenReturn(new SeqNoStats(maxSeqNo, localCheckpoint, globalCheckpoint)); + when(indexShard.getGlobalCheckpoint()).thenReturn(globalCheckpoint); + + IllegalStateException exception = expectThrows(IllegalStateException.class, this::executeOnPrimaryOrReplica); + assertThat(exception.getMessage(), equalTo("Global checkpoint [" + globalCheckpoint + "] mismatches maximum sequence number [" + + maxSeqNo + "] on index shard " + indexShard.shardId())); + verify(indexShard, times(0)).flush(any(FlushRequest.class)); + } +} From 8e5dd20efb667d272fac4b2a151f8c09f0284024 Mon Sep 17 00:00:00 2001 From: Tanguy Leroux Date: Thu, 13 Dec 2018 17:36:23 +0100 Subject: [PATCH 02/46] [Close Index API] Refactor MetaDataIndexStateService (#36354) The commit changes how indices are closed in the MetaDataIndexStateService. It now uses a 3 steps process where writes are blocked on indices to be closed, then some verifications are done on shards using the TransportVerifyShardBeforeCloseAction added in #36249, and finally indices states are moved to CLOSE and their routing tables removed. The closing process also takes care of using the pre-7.0 way to close indices if the cluster contains mixed version of nodes and a node does not support the TransportVerifyShardBeforeCloseAction. It also closes unassigned indices. Related to #33888 --- .../test/rest/WaitForRefreshAndCloseIT.java | 4 +- .../CloseIndexClusterStateUpdateRequest.java | 2 +- .../close/TransportCloseIndexAction.java | 17 +- ...TransportVerifyShardBeforeCloseAction.java | 24 +- .../metadata/MetaDataIndexStateService.java | 310 +++++++++++++++--- ...portVerifyShardBeforeCloseActionTests.java | 4 +- .../MetaDataIndexStateServiceTests.java | 277 +++++++++++++++- .../MetaDataIndexStateServiceUtils.java | 46 +++ .../indices/cluster/ClusterStateChanges.java | 21 +- .../indices/state/CloseIndexIT.java | 273 +++++++++++++++ .../state/CloseWhileRelocatingShardsIT.java | 195 +++++++++++ .../indices/state/OpenCloseIndexIT.java | 59 ---- .../indices/state/SimpleIndexStateIT.java | 7 +- .../DedicatedClusterSnapshotRestoreIT.java | 2 +- .../elasticsearch/test/BackgroundIndexer.java | 17 +- .../action/TransportFreezeIndexAction.java | 43 ++- .../authz/privilege/IndexPrivilege.java | 2 +- 17 files changed, 1141 insertions(+), 162 deletions(-) create mode 100644 server/src/test/java/org/elasticsearch/cluster/metadata/MetaDataIndexStateServiceUtils.java create mode 100644 server/src/test/java/org/elasticsearch/indices/state/CloseIndexIT.java create mode 100644 server/src/test/java/org/elasticsearch/indices/state/CloseWhileRelocatingShardsIT.java diff --git a/distribution/archives/integ-test-zip/src/test/java/org/elasticsearch/test/rest/WaitForRefreshAndCloseIT.java b/distribution/archives/integ-test-zip/src/test/java/org/elasticsearch/test/rest/WaitForRefreshAndCloseIT.java index 52b918e97f190..d6f5f723b46ac 100644 --- a/distribution/archives/integ-test-zip/src/test/java/org/elasticsearch/test/rest/WaitForRefreshAndCloseIT.java +++ b/distribution/archives/integ-test-zip/src/test/java/org/elasticsearch/test/rest/WaitForRefreshAndCloseIT.java @@ -20,12 +20,13 @@ package org.elasticsearch.test.rest; import org.apache.http.util.EntityUtils; +import org.apache.lucene.util.LuceneTestCase; import org.elasticsearch.action.ActionFuture; import org.elasticsearch.action.support.PlainActionFuture; +import org.elasticsearch.client.Request; import org.elasticsearch.client.Response; import org.elasticsearch.client.ResponseException; import org.elasticsearch.client.ResponseListener; -import org.elasticsearch.client.Request; import org.junit.After; import org.junit.Before; @@ -40,6 +41,7 @@ /** * Tests that wait for refresh is fired if the index is closed. */ +@LuceneTestCase.AwaitsFix(bugUrl = "to be created") public class WaitForRefreshAndCloseIT extends ESRestTestCase { @Before public void setupIndex() throws IOException { diff --git a/server/src/main/java/org/elasticsearch/action/admin/indices/close/CloseIndexClusterStateUpdateRequest.java b/server/src/main/java/org/elasticsearch/action/admin/indices/close/CloseIndexClusterStateUpdateRequest.java index ba5cc2ab00eaa..8ad79f1676eb1 100644 --- a/server/src/main/java/org/elasticsearch/action/admin/indices/close/CloseIndexClusterStateUpdateRequest.java +++ b/server/src/main/java/org/elasticsearch/action/admin/indices/close/CloseIndexClusterStateUpdateRequest.java @@ -25,7 +25,7 @@ */ public class CloseIndexClusterStateUpdateRequest extends IndicesClusterStateUpdateRequest { - CloseIndexClusterStateUpdateRequest() { + public CloseIndexClusterStateUpdateRequest() { } } diff --git a/server/src/main/java/org/elasticsearch/action/admin/indices/close/TransportCloseIndexAction.java b/server/src/main/java/org/elasticsearch/action/admin/indices/close/TransportCloseIndexAction.java index fb5fdf536a248..605f0ed9217ac 100644 --- a/server/src/main/java/org/elasticsearch/action/admin/indices/close/TransportCloseIndexAction.java +++ b/server/src/main/java/org/elasticsearch/action/admin/indices/close/TransportCloseIndexAction.java @@ -26,7 +26,6 @@ import org.elasticsearch.action.support.master.AcknowledgedResponse; import org.elasticsearch.action.support.master.TransportMasterNodeAction; import org.elasticsearch.cluster.ClusterState; -import org.elasticsearch.cluster.ack.ClusterStateUpdateResponse; import org.elasticsearch.cluster.block.ClusterBlockException; import org.elasticsearch.cluster.block.ClusterBlockLevel; import org.elasticsearch.cluster.metadata.IndexNameExpressionResolver; @@ -105,19 +104,21 @@ protected void masterOperation(final CloseIndexRequest request, final ClusterSta listener.onResponse(new AcknowledgedResponse(true)); return; } - CloseIndexClusterStateUpdateRequest updateRequest = new CloseIndexClusterStateUpdateRequest() - .ackTimeout(request.timeout()).masterNodeTimeout(request.masterNodeTimeout()) - .indices(concreteIndices); - indexStateService.closeIndices(updateRequest, new ActionListener() { + final CloseIndexClusterStateUpdateRequest closeRequest = new CloseIndexClusterStateUpdateRequest() + .ackTimeout(request.timeout()) + .masterNodeTimeout(request.masterNodeTimeout()) + .indices(concreteIndices); + + indexStateService.closeIndices(closeRequest, new ActionListener() { @Override - public void onResponse(ClusterStateUpdateResponse response) { - listener.onResponse(new AcknowledgedResponse(response.isAcknowledged())); + public void onResponse(final AcknowledgedResponse response) { + listener.onResponse(response); } @Override - public void onFailure(Exception t) { + public void onFailure(final Exception t) { logger.debug(() -> new ParameterizedMessage("failed to close indices [{}]", (Object) concreteIndices), t); listener.onFailure(t); } diff --git a/server/src/main/java/org/elasticsearch/action/admin/indices/close/TransportVerifyShardBeforeCloseAction.java b/server/src/main/java/org/elasticsearch/action/admin/indices/close/TransportVerifyShardBeforeCloseAction.java index ab895dd7af804..de5e372dd1625 100644 --- a/server/src/main/java/org/elasticsearch/action/admin/indices/close/TransportVerifyShardBeforeCloseAction.java +++ b/server/src/main/java/org/elasticsearch/action/admin/indices/close/TransportVerifyShardBeforeCloseAction.java @@ -40,10 +40,10 @@ import org.elasticsearch.transport.TransportService; public class TransportVerifyShardBeforeCloseAction extends TransportReplicationAction< - TransportVerifyShardBeforeCloseAction.ShardCloseRequest, TransportVerifyShardBeforeCloseAction.ShardCloseRequest, ReplicationResponse> { + TransportVerifyShardBeforeCloseAction.ShardRequest, TransportVerifyShardBeforeCloseAction.ShardRequest, ReplicationResponse> { public static final String NAME = CloseIndexAction.NAME + "[s]"; - private static final ClusterBlock EXPECTED_BLOCK = MetaDataIndexStateService.INDEX_CLOSED_BLOCK; + public static final ClusterBlock EXPECTED_BLOCK = MetaDataIndexStateService.INDEX_CLOSED_BLOCK; @Inject public TransportVerifyShardBeforeCloseAction(final Settings settings, final TransportService transportService, @@ -51,7 +51,7 @@ public TransportVerifyShardBeforeCloseAction(final Settings settings, final Tran final ThreadPool threadPool, final ShardStateAction stateAction, final ActionFilters actionFilters, final IndexNameExpressionResolver resolver) { super(settings, NAME, transportService, clusterService, indicesService, threadPool, stateAction, actionFilters, resolver, - ShardCloseRequest::new, ShardCloseRequest::new, ThreadPool.Names.MANAGEMENT); + ShardRequest::new, ShardRequest::new, ThreadPool.Names.MANAGEMENT); } @Override @@ -61,14 +61,14 @@ protected ReplicationResponse newResponseInstance() { @Override protected void acquirePrimaryOperationPermit(final IndexShard primary, - final ShardCloseRequest request, + final ShardRequest request, final ActionListener onAcquired) { primary.acquireAllPrimaryOperationsPermits(onAcquired, request.timeout()); } @Override protected void acquireReplicaOperationPermit(final IndexShard replica, - final ShardCloseRequest request, + final ShardRequest request, final ActionListener onAcquired, final long primaryTerm, final long globalCheckpoint, @@ -77,14 +77,14 @@ protected void acquireReplicaOperationPermit(final IndexShard replica, } @Override - protected PrimaryResult shardOperationOnPrimary(final ShardCloseRequest shardRequest, - final IndexShard primary) throws Exception { + protected PrimaryResult shardOperationOnPrimary(final ShardRequest shardRequest, + final IndexShard primary) throws Exception { executeShardOperation(primary); return new PrimaryResult<>(shardRequest, new ReplicationResponse()); } @Override - protected ReplicaResult shardOperationOnReplica(final ShardCloseRequest shardRequest, final IndexShard replica) throws Exception { + protected ReplicaResult shardOperationOnReplica(final ShardRequest shardRequest, final IndexShard replica) throws Exception { executeShardOperation(replica); return new ReplicaResult(); } @@ -109,18 +109,18 @@ private void executeShardOperation(final IndexShard indexShard) { logger.debug("{} shard is ready for closing", shardId); } - public static class ShardCloseRequest extends ReplicationRequest { + public static class ShardRequest extends ReplicationRequest { - ShardCloseRequest(){ + ShardRequest(){ } - public ShardCloseRequest(final ShardId shardId) { + public ShardRequest(final ShardId shardId) { super(shardId); } @Override public String toString() { - return "close shard {" + shardId + "}"; + return "verify shard before close {" + shardId + "}"; } } } diff --git a/server/src/main/java/org/elasticsearch/cluster/metadata/MetaDataIndexStateService.java b/server/src/main/java/org/elasticsearch/cluster/metadata/MetaDataIndexStateService.java index a36a11f805785..cda8f9c6f0ac6 100644 --- a/server/src/main/java/org/elasticsearch/cluster/metadata/MetaDataIndexStateService.java +++ b/server/src/main/java/org/elasticsearch/cluster/metadata/MetaDataIndexStateService.java @@ -19,28 +19,46 @@ package org.elasticsearch.cluster.metadata; +import com.carrotsearch.hppc.cursors.IntObjectCursor; import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; +import org.apache.logging.log4j.message.ParameterizedMessage; import org.elasticsearch.ElasticsearchException; import org.elasticsearch.Version; import org.elasticsearch.action.ActionListener; +import org.elasticsearch.action.NotifyOnceListener; import org.elasticsearch.action.admin.indices.close.CloseIndexClusterStateUpdateRequest; +import org.elasticsearch.action.admin.indices.close.TransportVerifyShardBeforeCloseAction; import org.elasticsearch.action.admin.indices.open.OpenIndexClusterStateUpdateRequest; import org.elasticsearch.action.support.ActiveShardsObserver; +import org.elasticsearch.action.support.master.AcknowledgedResponse; +import org.elasticsearch.action.support.replication.ReplicationResponse; import org.elasticsearch.cluster.AckedClusterStateUpdateTask; import org.elasticsearch.cluster.ClusterState; +import org.elasticsearch.cluster.ClusterStateUpdateTask; import org.elasticsearch.cluster.ack.ClusterStateUpdateResponse; import org.elasticsearch.cluster.ack.OpenIndexClusterStateUpdateResponse; import org.elasticsearch.cluster.block.ClusterBlock; import org.elasticsearch.cluster.block.ClusterBlockLevel; import org.elasticsearch.cluster.block.ClusterBlocks; +import org.elasticsearch.cluster.routing.IndexRoutingTable; +import org.elasticsearch.cluster.routing.IndexShardRoutingTable; import org.elasticsearch.cluster.routing.RoutingTable; import org.elasticsearch.cluster.routing.allocation.AllocationService; import org.elasticsearch.cluster.service.ClusterService; +import org.elasticsearch.common.Nullable; import org.elasticsearch.common.Priority; import org.elasticsearch.common.ValidationException; +import org.elasticsearch.common.collect.ImmutableOpenIntMap; import org.elasticsearch.common.inject.Inject; +import org.elasticsearch.common.unit.TimeValue; +import org.elasticsearch.common.util.concurrent.AbstractRunnable; +import org.elasticsearch.common.util.concurrent.AtomicArray; +import org.elasticsearch.common.util.concurrent.ConcurrentCollections; +import org.elasticsearch.common.util.concurrent.CountDown; import org.elasticsearch.index.Index; +import org.elasticsearch.index.IndexNotFoundException; +import org.elasticsearch.index.shard.ShardId; import org.elasticsearch.indices.IndicesService; import org.elasticsearch.rest.RestStatus; import org.elasticsearch.snapshots.RestoreService; @@ -51,8 +69,13 @@ import java.util.Arrays; import java.util.HashSet; import java.util.List; +import java.util.Map; import java.util.Optional; import java.util.Set; +import java.util.function.Consumer; +import java.util.stream.Collectors; + +import static java.util.Collections.unmodifiableMap; /** * Service responsible for submitting open/close index requests @@ -64,50 +87,118 @@ public class MetaDataIndexStateService { false, false, RestStatus.FORBIDDEN, ClusterBlockLevel.READ_WRITE); private final ClusterService clusterService; - private final AllocationService allocationService; - private final MetaDataIndexUpgradeService metaDataIndexUpgradeService; private final IndicesService indicesService; + private final ThreadPool threadPool; + private final TransportVerifyShardBeforeCloseAction transportVerifyShardBeforeCloseAction; private final ActiveShardsObserver activeShardsObserver; @Inject public MetaDataIndexStateService(ClusterService clusterService, AllocationService allocationService, MetaDataIndexUpgradeService metaDataIndexUpgradeService, - IndicesService indicesService, ThreadPool threadPool) { + IndicesService indicesService, ThreadPool threadPool, + TransportVerifyShardBeforeCloseAction transportVerifyShardBeforeCloseAction) { this.indicesService = indicesService; this.clusterService = clusterService; this.allocationService = allocationService; + this.threadPool = threadPool; + this.transportVerifyShardBeforeCloseAction = transportVerifyShardBeforeCloseAction; this.metaDataIndexUpgradeService = metaDataIndexUpgradeService; this.activeShardsObserver = new ActiveShardsObserver(clusterService, threadPool); } - public void closeIndices(final CloseIndexClusterStateUpdateRequest request, final ActionListener listener) { - if (request.indices() == null || request.indices().length == 0) { + /** + * Closes one or more indices. + * + * Closing indices is a 3 steps process: it first adds a write block to every indices to close, then waits for the operations on shards + * to be terminated and finally closes the indices by moving their state to CLOSE. + */ + public void closeIndices(final CloseIndexClusterStateUpdateRequest request, final ActionListener listener) { + final Index[] concreteIndices = request.indices(); + if (concreteIndices == null || concreteIndices.length == 0) { throw new IllegalArgumentException("Index name is required"); } - final String indicesAsString = Arrays.toString(request.indices()); - clusterService.submitStateUpdateTask("close-indices " + indicesAsString, - new AckedClusterStateUpdateTask(Priority.URGENT, request, listener) { - @Override - protected ClusterStateUpdateResponse newResponse(boolean acknowledged) { - return new ClusterStateUpdateResponse(acknowledged); - } + final TimeValue timeout = request.ackTimeout(); + final TimeValue masterTimeout = request.masterNodeTimeout(); - @Override - public ClusterState execute(ClusterState currentState) { - return closeIndices(currentState, request.indices(), indicesAsString); + clusterService.submitStateUpdateTask("add-block-index-to-close " + Arrays.toString(concreteIndices), + new ClusterStateUpdateTask(Priority.URGENT) { + + private final Set blockedIndices = new HashSet<>(); + + @Override + public ClusterState execute(final ClusterState currentState) { + return addIndexClosedBlocks(concreteIndices, currentState, blockedIndices); + } + + @Override + public void clusterStateProcessed(final String source, final ClusterState oldState, final ClusterState newState) { + if (oldState == newState) { + assert blockedIndices.isEmpty() : "List of blocked indices is not empty but cluster state wasn't changed"; + listener.onResponse(new AcknowledgedResponse(true)); + } else { + assert blockedIndices.isEmpty() == false : "List of blocked indices is empty but cluster state was changed"; + threadPool.executor(ThreadPool.Names.MANAGEMENT) + .execute(new WaitForClosedBlocksApplied(blockedIndices, timeout, + ActionListener.wrap(closedBlocksResults -> + clusterService.submitStateUpdateTask("close-indices", new ClusterStateUpdateTask(Priority.URGENT) { + @Override + public ClusterState execute(final ClusterState currentState) throws Exception { + final ClusterState updatedState = closeRoutingTable(currentState, closedBlocksResults); + return allocationService.reroute(updatedState, "indices closed"); + } + + @Override + public void onFailure(final String source, final Exception e) { + listener.onFailure(e); + } + + @Override + public void clusterStateProcessed(final String source, + final ClusterState oldState, final ClusterState newState) { + boolean acknowledged = closedBlocksResults.values().stream() + .allMatch(AcknowledgedResponse::isAcknowledged); + listener.onResponse(new AcknowledgedResponse(acknowledged)); + } + }), + listener::onFailure) + ) + ); + } + } + + @Override + public void onFailure(final String source, final Exception e) { + listener.onFailure(e); + } + + @Override + public TimeValue timeout() { + return masterTimeout; + } } - }); + ); } - public ClusterState closeIndices(ClusterState currentState, final Index[] indices, String indicesAsString) { - Set indicesToClose = new HashSet<>(); + /** + * Step 1 - Start closing indices by adding a write block + * + * This step builds the list of indices to close (the ones explicitly requested that are not in CLOSE state) and adds the index block + * {@link #INDEX_CLOSED_BLOCK} to every index to close in the cluster state. After the cluster state is published, the shards should + * start to reject writing operations and we can proceed with step 2. + */ + static ClusterState addIndexClosedBlocks(final Index[] indices, final ClusterState currentState, final Set blockedIndices) { + final MetaData.Builder metadata = MetaData.builder(currentState.metaData()); + + final Set indicesToClose = new HashSet<>(); for (Index index : indices) { - final IndexMetaData indexMetaData = currentState.metaData().getIndexSafe(index); + final IndexMetaData indexMetaData = metadata.getSafe(index); if (indexMetaData.getState() != IndexMetaData.State.CLOSE) { indicesToClose.add(indexMetaData); + } else { + logger.debug("index {} is already closed, ignoring", index); } } @@ -119,28 +210,174 @@ public ClusterState closeIndices(ClusterState currentState, final Index[] indice RestoreService.checkIndexClosing(currentState, indicesToClose); // Check if index closing conflicts with any running snapshots SnapshotsService.checkIndexClosing(currentState, indicesToClose); - logger.info("closing indices [{}]", indicesAsString); - - MetaData.Builder mdBuilder = MetaData.builder(currentState.metaData()); - ClusterBlocks.Builder blocksBuilder = ClusterBlocks.builder() - .blocks(currentState.blocks()); - for (IndexMetaData openIndexMetadata : indicesToClose) { - final String indexName = openIndexMetadata.getIndex().getName(); - mdBuilder.put(IndexMetaData.builder(openIndexMetadata).state(IndexMetaData.State.CLOSE)); - blocksBuilder.addIndexBlock(indexName, INDEX_CLOSED_BLOCK); + + // If the cluster is in a mixed version that does not support the shard close action, + // we use the previous way to close indices and directly close them without sanity checks + final boolean useDirectClose = currentState.nodes().getMinNodeVersion().before(Version.V_7_0_0); + + final ClusterBlocks.Builder blocks = ClusterBlocks.builder().blocks(currentState.blocks()); + final RoutingTable.Builder routingTable = RoutingTable.builder(currentState.routingTable()); + + for (IndexMetaData indexToClose : indicesToClose) { + final Index index = indexToClose.getIndex(); + if (currentState.blocks().hasIndexBlock(index.getName(), INDEX_CLOSED_BLOCK) == false) { + blocks.addIndexBlock(index.getName(), INDEX_CLOSED_BLOCK); + } + if (useDirectClose) { + logger.debug("closing index {} directly", index); + metadata.put(IndexMetaData.builder(indexToClose).state(IndexMetaData.State.CLOSE)); + routingTable.remove(index.getName()); + } + blockedIndices.add(index); } - ClusterState updatedState = ClusterState.builder(currentState).metaData(mdBuilder).blocks(blocksBuilder).build(); + logger.info(() -> new ParameterizedMessage("closing indices {}", + blockedIndices.stream().map(Object::toString).collect(Collectors.joining(",")))); + return ClusterState.builder(currentState).blocks(blocks).metaData(metadata).routingTable(routingTable.build()).build(); + } + + /** + * Step 2 - Wait for indices to be ready for closing + *

+ * This step iterates over the indices previously blocked and sends a {@link TransportVerifyShardBeforeCloseAction} to each shard. If + * this action succeed then the shard is considered to be ready for closing. When all shards of a given index are ready for closing, + * the index is considered ready to be closed. + */ + class WaitForClosedBlocksApplied extends AbstractRunnable { + + private final Set blockedIndices; + private final @Nullable TimeValue timeout; + private final ActionListener> listener; + + private WaitForClosedBlocksApplied(final Set blockedIndices, + final @Nullable TimeValue timeout, + final ActionListener> listener) { + if (blockedIndices == null || blockedIndices.isEmpty()) { + throw new IllegalArgumentException("Cannot wait for closed block to be applied to null or empty list of blocked indices"); + } + this.blockedIndices = blockedIndices; + this.listener = listener; + this.timeout = timeout; + } + + @Override + public void onFailure(final Exception e) { + listener.onFailure(e); + } + + @Override + protected void doRun() throws Exception { + final Map results = ConcurrentCollections.newConcurrentMap(); + final CountDown countDown = new CountDown(blockedIndices.size()); + final ClusterState state = clusterService.state(); + for (Index blockedIndex : blockedIndices) { + waitForShardsReadyForClosing(blockedIndex, state, timeout, response -> { + results.put(blockedIndex, response); + if (countDown.countDown()) { + listener.onResponse(unmodifiableMap(results)); + } + }); + } + } + + private void waitForShardsReadyForClosing(final Index index, final ClusterState state, @Nullable final TimeValue timeout, + final Consumer onResponse) { + final IndexMetaData indexMetaData = state.metaData().index(index); + if (indexMetaData == null) { + logger.debug("index {} has been blocked before closing and is now deleted, ignoring", index); + onResponse.accept(new AcknowledgedResponse(true)); + return; + } + final IndexRoutingTable indexRoutingTable = state.routingTable().index(index); + if (indexRoutingTable == null || indexMetaData.getState() == IndexMetaData.State.CLOSE) { + logger.debug("index {} has been blocked before closing and is already closed, ignoring", index); + onResponse.accept(new AcknowledgedResponse(true)); + return; + } + + final ImmutableOpenIntMap shards = indexRoutingTable.getShards(); + final AtomicArray results = new AtomicArray<>(shards.size()); + final CountDown countDown = new CountDown(shards.size()); + + for (IntObjectCursor shard : shards) { + final IndexShardRoutingTable shardRoutingTable = shard.value; + final ShardId shardId = shardRoutingTable.shardId(); + sendVerifyShardBeforeCloseRequest(shardRoutingTable, timeout, new NotifyOnceListener() { + @Override + public void innerOnResponse(final ReplicationResponse replicationResponse) { + ReplicationResponse.ShardInfo shardInfo = replicationResponse.getShardInfo(); + results.setOnce(shardId.id(), new AcknowledgedResponse(shardInfo.getFailed() == 0)); + processIfFinished(); + } + + @Override + public void innerOnFailure(final Exception e) { + results.setOnce(shardId.id(), new AcknowledgedResponse(false)); + processIfFinished(); + } - RoutingTable.Builder rtBuilder = RoutingTable.builder(currentState.routingTable()); - for (IndexMetaData index : indicesToClose) { - rtBuilder.remove(index.getIndex().getName()); + private void processIfFinished() { + if (countDown.countDown()) { + final boolean acknowledged = results.asList().stream().allMatch(AcknowledgedResponse::isAcknowledged); + onResponse.accept(new AcknowledgedResponse(acknowledged)); + } + } + }); + } + } + + private void sendVerifyShardBeforeCloseRequest(final IndexShardRoutingTable shardRoutingTable, @Nullable final TimeValue timeout, + final ActionListener listener) { + final ShardId shardId = shardRoutingTable.shardId(); + if (shardRoutingTable.primaryShard().unassigned()) { + logger.debug("primary shard {} is unassigned, ignoring", shardId); + final ReplicationResponse response = new ReplicationResponse(); + response.setShardInfo(new ReplicationResponse.ShardInfo(shardRoutingTable.size(), shardRoutingTable.size())); + listener.onResponse(response); + return; + } + final TransportVerifyShardBeforeCloseAction.ShardRequest shardRequest = + new TransportVerifyShardBeforeCloseAction.ShardRequest(shardId); + if (timeout != null) { + shardRequest.timeout(timeout); + } + // TODO propagate a task id from the parent CloseIndexRequest to the ShardCloseRequests + transportVerifyShardBeforeCloseAction.execute(shardRequest, listener); } + } - //no explicit wait for other nodes needed as we use AckedClusterStateUpdateTask - return allocationService.reroute( - ClusterState.builder(updatedState).routingTable(rtBuilder.build()).build(), - "indices closed [" + indicesAsString + "]"); + /** + * Step 3 - Move index states from OPEN to CLOSE in cluster state for indices that are ready for closing. + */ + static ClusterState closeRoutingTable(final ClusterState currentState, final Map results) { + final MetaData.Builder metadata = MetaData.builder(currentState.metaData()); + final ClusterBlocks.Builder blocks = ClusterBlocks.builder().blocks(currentState.blocks()); + final RoutingTable.Builder routingTable = RoutingTable.builder(currentState.routingTable()); + + final Set closedIndices = new HashSet<>(); + for (Map.Entry result : results.entrySet()) { + final Index index = result.getKey(); + try { + final IndexMetaData indexMetaData = metadata.getSafe(index); + if (indexMetaData.getState() != IndexMetaData.State.CLOSE) { + if (result.getValue().isAcknowledged()) { + logger.debug("closing index {} succeed, removing index routing table", index); + metadata.put(IndexMetaData.builder(indexMetaData).state(IndexMetaData.State.CLOSE)); + routingTable.remove(index.getName()); + closedIndices.add(index.getName()); + } else { + logger.debug("closing index {} failed, removing index block because: {}", index, result.getValue()); + blocks.removeIndexBlock(index.getName(), INDEX_CLOSED_BLOCK); + } + } else { + logger.debug("index {} has been closed since it was blocked before closing, ignoring", index); + } + } catch (final IndexNotFoundException e) { + logger.debug("index {} has been deleted since it was blocked before closing, ignoring", index); + } + } + logger.info("completed closing of indices {}", closedIndices); + return ClusterState.builder(currentState).blocks(blocks).metaData(metadata).routingTable(routingTable.build()).build(); } public void openIndex(final OpenIndexClusterStateUpdateRequest request, @@ -250,7 +487,6 @@ static void validateShardLimit(ClusterState currentState, Index[] indices) { ex.addValidationError(error.get()); throw ex; } - } private static int getTotalShardCount(ClusterState state, Index index) { diff --git a/server/src/test/java/org/elasticsearch/action/admin/indices/close/TransportVerifyShardBeforeCloseActionTests.java b/server/src/test/java/org/elasticsearch/action/admin/indices/close/TransportVerifyShardBeforeCloseActionTests.java index ae1a0b8768946..de0cc5dfd5a37 100644 --- a/server/src/test/java/org/elasticsearch/action/admin/indices/close/TransportVerifyShardBeforeCloseActionTests.java +++ b/server/src/test/java/org/elasticsearch/action/admin/indices/close/TransportVerifyShardBeforeCloseActionTests.java @@ -74,8 +74,8 @@ public void setUp() throws Exception { } private void executeOnPrimaryOrReplica() throws Exception { - final TransportVerifyShardBeforeCloseAction.ShardCloseRequest request = - new TransportVerifyShardBeforeCloseAction.ShardCloseRequest(indexShard.shardId()); + final TransportVerifyShardBeforeCloseAction.ShardRequest request = + new TransportVerifyShardBeforeCloseAction.ShardRequest(indexShard.shardId()); if (randomBoolean()) { assertNotNull(action.shardOperationOnPrimary(request, indexShard)); } else { diff --git a/server/src/test/java/org/elasticsearch/cluster/metadata/MetaDataIndexStateServiceTests.java b/server/src/test/java/org/elasticsearch/cluster/metadata/MetaDataIndexStateServiceTests.java index 712f3ccdd4755..a5a9e5ee22796 100644 --- a/server/src/test/java/org/elasticsearch/cluster/metadata/MetaDataIndexStateServiceTests.java +++ b/server/src/test/java/org/elasticsearch/cluster/metadata/MetaDataIndexStateServiceTests.java @@ -20,27 +20,186 @@ package org.elasticsearch.cluster.metadata; import org.elasticsearch.Version; +import org.elasticsearch.action.support.master.AcknowledgedResponse; import org.elasticsearch.cluster.ClusterName; import org.elasticsearch.cluster.ClusterState; +import org.elasticsearch.cluster.RestoreInProgress; +import org.elasticsearch.cluster.SnapshotsInProgress; +import org.elasticsearch.cluster.block.ClusterBlock; +import org.elasticsearch.cluster.block.ClusterBlocks; import org.elasticsearch.cluster.node.DiscoveryNode; import org.elasticsearch.cluster.node.DiscoveryNodes; +import org.elasticsearch.cluster.routing.IndexRoutingTable; +import org.elasticsearch.cluster.routing.IndexShardRoutingTable; +import org.elasticsearch.cluster.routing.RoutingTable; +import org.elasticsearch.cluster.routing.ShardRouting; +import org.elasticsearch.cluster.routing.ShardRoutingState; import org.elasticsearch.cluster.shards.ClusterShardLimitIT; +import org.elasticsearch.common.Nullable; import org.elasticsearch.common.ValidationException; import org.elasticsearch.common.collect.ImmutableOpenMap; -import org.elasticsearch.common.logging.DeprecationLogger; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.index.Index; +import org.elasticsearch.index.IndexNotFoundException; +import org.elasticsearch.index.shard.ShardId; +import org.elasticsearch.repositories.IndexId; +import org.elasticsearch.snapshots.Snapshot; +import org.elasticsearch.snapshots.SnapshotId; import org.elasticsearch.test.ESTestCase; import java.util.Arrays; +import java.util.Collections; +import java.util.HashMap; +import java.util.HashSet; +import java.util.Map; +import java.util.Set; import java.util.stream.Collectors; +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; +import static org.elasticsearch.cluster.routing.TestShardRouting.newShardRouting; import static org.elasticsearch.cluster.shards.ClusterShardLimitIT.ShardCounts.forDataNodeCount; +import static org.hamcrest.Matchers.containsString; +import static org.hamcrest.Matchers.equalTo; +import static org.hamcrest.Matchers.is; +import static org.hamcrest.Matchers.notNullValue; +import static org.hamcrest.Matchers.nullValue; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.when; public class MetaDataIndexStateServiceTests extends ESTestCase { + public void testCloseRoutingTable() { + final Set nonBlockedIndices = new HashSet<>(); + final Map blockedIndices = new HashMap<>(); + + ClusterState state = ClusterState.builder(new ClusterName("testCloseRoutingTable")).build(); + for (int i = 0; i < randomIntBetween(1, 25); i++) { + final String indexName = randomAlphaOfLengthBetween(5, 15); + + if (randomBoolean()) { + state = addOpenedIndex(indexName, randomIntBetween(1, 5), randomIntBetween(0, 5), state); + nonBlockedIndices.add(state.metaData().index(indexName).getIndex()); + } else { + state = addBlockedIndex(indexName, randomIntBetween(1, 5), randomIntBetween(0, 5), state); + blockedIndices.put(state.metaData().index(indexName).getIndex(), new AcknowledgedResponse(randomBoolean())); + } + } + + final ClusterState updatedState = MetaDataIndexStateService.closeRoutingTable(state, blockedIndices); + assertThat(updatedState.metaData().indices().size(), equalTo(nonBlockedIndices.size() + blockedIndices.size())); + + for (Index nonBlockedIndex : nonBlockedIndices) { + assertIsOpened(nonBlockedIndex.getName(), updatedState); + } + for (Map.Entry blockedIndex : blockedIndices.entrySet()) { + if (blockedIndex.getValue().isAcknowledged()) { + assertIsClosed(blockedIndex.getKey().getName(), updatedState); + } else { + assertIsOpened(blockedIndex.getKey().getName(), updatedState); + } + } + } + + public void testAddIndexClosedBlocks() { + final ClusterState initialState = ClusterState.builder(new ClusterName("testAddIndexClosedBlocks")).build(); + { + final Set blockedIndices = new HashSet<>(); + expectThrows(IndexNotFoundException.class, () -> + MetaDataIndexStateService.addIndexClosedBlocks(new Index[]{new Index("_name", "_uid")}, initialState, blockedIndices)); + assertTrue(blockedIndices.isEmpty()); + } + { + final Set blockedIndices = new HashSet<>(); + Index[] indices = Index.EMPTY_ARRAY; + + ClusterState updatedState = MetaDataIndexStateService.addIndexClosedBlocks(indices, initialState, blockedIndices); + assertSame(initialState, updatedState); + assertTrue(blockedIndices.isEmpty()); + } + { + final Set blockedIndices = new HashSet<>(); + ClusterState state = addClosedIndex("closed", randomIntBetween(1, 3), randomIntBetween(0, 3), initialState); + Index[] indices = new Index[]{state.metaData().index("closed").getIndex()}; + + ClusterState updatedState = MetaDataIndexStateService.addIndexClosedBlocks(indices, state, blockedIndices); + assertSame(state, updatedState); + assertTrue(blockedIndices.isEmpty()); + } + { + final Set blockedIndices = new HashSet<>(); + ClusterState state = addClosedIndex("closed", randomIntBetween(1, 3), randomIntBetween(0, 3), initialState); + state = addOpenedIndex("opened", randomIntBetween(1, 3), randomIntBetween(0, 3), state); + Index[] indices = new Index[]{state.metaData().index("opened").getIndex(), state.metaData().index("closed").getIndex()}; + + ClusterState updatedState = MetaDataIndexStateService.addIndexClosedBlocks(indices, state, blockedIndices); + assertNotSame(state, updatedState); + assertTrue(blockedIndices.contains(updatedState.metaData().index("opened").getIndex())); + assertFalse(blockedIndices.contains(updatedState.metaData().index("closed").getIndex())); + assertIsBlocked("opened", updatedState, true); + } + { + IllegalArgumentException exception = expectThrows(IllegalArgumentException.class, () -> { + ClusterState state = addRestoredIndex("restored", randomIntBetween(1, 3), randomIntBetween(0, 3), initialState); + if (randomBoolean()) { + state = addOpenedIndex("opened", randomIntBetween(1, 3), randomIntBetween(0, 3), state); + } + if (randomBoolean()) { + state = addOpenedIndex("closed", randomIntBetween(1, 3), randomIntBetween(0, 3), state); + } + Index[] indices = new Index[]{state.metaData().index("restored").getIndex()}; + MetaDataIndexStateService.addIndexClosedBlocks(indices, state, new HashSet<>()); + }); + assertThat(exception.getMessage(), containsString("Cannot close indices that are being restored: [[restored]]")); + } + { + IllegalArgumentException exception = expectThrows(IllegalArgumentException.class, () -> { + ClusterState state = addSnapshotIndex("snapshotted", randomIntBetween(1, 3), randomIntBetween(0, 3), initialState); + if (randomBoolean()) { + state = addOpenedIndex("opened", randomIntBetween(1, 3), randomIntBetween(0, 3), state); + } + if (randomBoolean()) { + state = addOpenedIndex("closed", randomIntBetween(1, 3), randomIntBetween(0, 3), state); + } + Index[] indices = new Index[]{state.metaData().index("snapshotted").getIndex()}; + MetaDataIndexStateService.addIndexClosedBlocks(indices, state, new HashSet<>()); + }); + assertThat(exception.getMessage(), containsString("Cannot close indices that are being snapshotted: [[snapshotted]]")); + } + { + final Set blockedIndices = new HashSet<>(); + ClusterState state = addOpenedIndex("index-1", randomIntBetween(1, 3), randomIntBetween(0, 3), initialState); + state = addOpenedIndex("index-2", randomIntBetween(1, 3), randomIntBetween(0, 3), state); + state = addOpenedIndex("index-3", randomIntBetween(1, 3), randomIntBetween(0, 3), state); + final boolean mixedVersions = randomBoolean(); + if (mixedVersions) { + state = ClusterState.builder(state) + .nodes(DiscoveryNodes.builder(state.nodes()) + .add(new DiscoveryNode("old_node", buildNewFakeTransportAddress(), Collections.emptyMap(), + new HashSet<>(Arrays.asList(DiscoveryNode.Role.values())), Version.V_6_0_0))) + .build(); + } + Index[] indices = new Index[]{state.metaData().index("index-1").getIndex(), + state.metaData().index("index-2").getIndex(), state.metaData().index("index-3").getIndex()}; + + ClusterState updatedState = MetaDataIndexStateService.addIndexClosedBlocks(indices, state, blockedIndices); + assertNotSame(state, updatedState); + assertTrue(blockedIndices.contains(updatedState.metaData().index("index-1").getIndex())); + assertTrue(blockedIndices.contains(updatedState.metaData().index("index-2").getIndex())); + assertTrue(blockedIndices.contains(updatedState.metaData().index("index-3").getIndex())); + if (mixedVersions) { + assertIsClosed("index-1", updatedState); + assertIsClosed("index-2", updatedState); + assertIsClosed("index-2", updatedState); + } else { + assertIsBlocked("index-1", updatedState, true); + assertIsBlocked("index-2", updatedState, true); + assertIsBlocked("index-3", updatedState, true); + } + } + } + public void testValidateShardLimit() { int nodesInCluster = randomIntBetween(2,100); ClusterShardLimitIT.ShardCounts counts = forDataNodeCount(nodesInCluster); @@ -55,7 +214,6 @@ public void testValidateShardLimit() { .collect(Collectors.toList()) .toArray(new Index[2]); - DeprecationLogger deprecationLogger = new DeprecationLogger(logger); int totalShards = counts.getFailingIndexShards() * (1 + counts.getFailingIndexReplicas()); int currentShards = counts.getFirstIndexShards() * (1 + counts.getFirstIndexReplicas()); int maxShards = counts.getShardsPerNode() * nodesInCluster; @@ -69,32 +227,115 @@ public static ClusterState createClusterForShardLimitTest(int nodesInCluster, in int closedIndexShards, int closedIndexReplicas, Settings clusterSettings) { ImmutableOpenMap.Builder dataNodes = ImmutableOpenMap.builder(); for (int i = 0; i < nodesInCluster; i++) { - dataNodes.put(randomAlphaOfLengthBetween(5,15), mock(DiscoveryNode.class)); + dataNodes.put(randomAlphaOfLengthBetween(5, 15), mock(DiscoveryNode.class)); } DiscoveryNodes nodes = mock(DiscoveryNodes.class); when(nodes.getDataNodes()).thenReturn(dataNodes.build()); - IndexMetaData.Builder openIndexMetaData = IndexMetaData.builder(randomAlphaOfLengthBetween(5, 15)) - .settings(Settings.builder().put(IndexMetaData.SETTING_VERSION_CREATED, Version.CURRENT)) - .creationDate(randomLong()) - .numberOfShards(openIndexShards) - .numberOfReplicas(openIndexReplicas); - IndexMetaData.Builder closedIndexMetaData = IndexMetaData.builder(randomAlphaOfLengthBetween(5, 15)) - .settings(Settings.builder().put(IndexMetaData.SETTING_VERSION_CREATED, Version.CURRENT)) - .creationDate(randomLong()) - .state(IndexMetaData.State.CLOSE) - .numberOfShards(closedIndexShards) - .numberOfReplicas(closedIndexReplicas); - MetaData.Builder metaData = MetaData.builder().put(openIndexMetaData).put(closedIndexMetaData); + ClusterState state = ClusterState.builder(ClusterName.DEFAULT).build(); + state = addOpenedIndex(randomAlphaOfLengthBetween(5, 15), openIndexShards, openIndexReplicas, state); + state = addClosedIndex(randomAlphaOfLengthBetween(5, 15), closedIndexShards, closedIndexReplicas, state); + + final MetaData.Builder metaData = MetaData.builder(state.metaData()); if (randomBoolean()) { metaData.persistentSettings(clusterSettings); } else { metaData.transientSettings(clusterSettings); } + return ClusterState.builder(state).metaData(metaData).nodes(nodes).build(); + } + + private static ClusterState addOpenedIndex(final String index, final int numShards, final int numReplicas, final ClusterState state) { + return addIndex(state, index, numShards, numReplicas, IndexMetaData.State.OPEN, null); + } + + private static ClusterState addClosedIndex(final String index, final int numShards, final int numReplicas, final ClusterState state) { + return addIndex(state, index, numShards, numReplicas, IndexMetaData.State.CLOSE, MetaDataIndexStateService.INDEX_CLOSED_BLOCK); + } + + private static ClusterState addBlockedIndex(final String index, final int numShards, final int numReplicas, final ClusterState state) { + return addIndex(state, index, numShards, numReplicas, IndexMetaData.State.OPEN, MetaDataIndexStateService.INDEX_CLOSED_BLOCK); + } + + private static ClusterState addRestoredIndex(final String index, final int numShards, final int numReplicas, final ClusterState state) { + ClusterState newState = addOpenedIndex(index, numShards, numReplicas, state); + + final ImmutableOpenMap.Builder shardsBuilder = ImmutableOpenMap.builder(); + for (ShardRouting shardRouting : newState.routingTable().index(index).randomAllActiveShardsIt()) { + shardsBuilder.put(shardRouting.shardId(), new RestoreInProgress.ShardRestoreStatus(shardRouting.currentNodeId())); + } + + final Snapshot snapshot = new Snapshot(randomAlphaOfLength(10), new SnapshotId(randomAlphaOfLength(5), randomAlphaOfLength(5))); + final RestoreInProgress.Entry entry = + new RestoreInProgress.Entry(snapshot, RestoreInProgress.State.INIT, Collections.singletonList(index), shardsBuilder.build()); + return ClusterState.builder(newState).putCustom(RestoreInProgress.TYPE, new RestoreInProgress(entry)).build(); + } + + private static ClusterState addSnapshotIndex(final String index, final int numShards, final int numReplicas, final ClusterState state) { + ClusterState newState = addOpenedIndex(index, numShards, numReplicas, state); + + final ImmutableOpenMap.Builder shardsBuilder = ImmutableOpenMap.builder(); + for (ShardRouting shardRouting : newState.routingTable().index(index).randomAllActiveShardsIt()) { + shardsBuilder.put(shardRouting.shardId(), new SnapshotsInProgress.ShardSnapshotStatus(shardRouting.currentNodeId())); + } - return ClusterState.builder(ClusterName.DEFAULT) - .metaData(metaData) - .nodes(nodes) + final Snapshot snapshot = new Snapshot(randomAlphaOfLength(10), new SnapshotId(randomAlphaOfLength(5), randomAlphaOfLength(5))); + final SnapshotsInProgress.Entry entry = + new SnapshotsInProgress.Entry(snapshot, randomBoolean(), false, SnapshotsInProgress.State.INIT, + Collections.singletonList(new IndexId(index, index)), randomNonNegativeLong(), randomLong(), shardsBuilder.build()); + return ClusterState.builder(newState).putCustom(SnapshotsInProgress.TYPE, new SnapshotsInProgress(entry)).build(); + } + + private static ClusterState addIndex(final ClusterState currentState, + final String index, + final int numShards, + final int numReplicas, + final IndexMetaData.State state, + @Nullable final ClusterBlock block) { + final IndexMetaData indexMetaData = IndexMetaData.builder(index) + .state(state) + .creationDate(randomNonNegativeLong()) + .settings(Settings.builder() + .put(SETTING_VERSION_CREATED, Version.CURRENT) + .put(SETTING_NUMBER_OF_SHARDS, numShards) + .put(SETTING_NUMBER_OF_REPLICAS, numReplicas)) .build(); + + final ClusterState.Builder clusterStateBuilder = ClusterState.builder(currentState); + clusterStateBuilder.metaData(MetaData.builder(currentState.metaData()).put(indexMetaData, true)); + + if (state == IndexMetaData.State.OPEN) { + final IndexRoutingTable.Builder indexRoutingTable = IndexRoutingTable.builder(indexMetaData.getIndex()); + for (int j = 0; j < indexMetaData.getNumberOfShards(); j++) { + ShardId shardId = new ShardId(indexMetaData.getIndex(), j); + IndexShardRoutingTable.Builder indexShardRoutingBuilder = new IndexShardRoutingTable.Builder(shardId); + indexShardRoutingBuilder.addShard(newShardRouting(shardId, randomAlphaOfLength(10), true, ShardRoutingState.STARTED)); + for (int k = 0; k < indexMetaData.getNumberOfReplicas(); k++) { + indexShardRoutingBuilder.addShard(newShardRouting(shardId, randomAlphaOfLength(10), false, ShardRoutingState.STARTED)); + } + indexRoutingTable.addIndexShard(indexShardRoutingBuilder.build()); + } + clusterStateBuilder.routingTable(RoutingTable.builder(currentState.routingTable()).add(indexRoutingTable).build()); + } + if (block != null) { + clusterStateBuilder.blocks(ClusterBlocks.builder().blocks(currentState.blocks()).addIndexBlock(index, block)); + } + return clusterStateBuilder.build(); + } + + private static void assertIsOpened(final String indexName, final ClusterState clusterState) { + assertThat(clusterState.metaData().index(indexName).getState(), is(IndexMetaData.State.OPEN)); + assertThat(clusterState.routingTable().index(indexName), notNullValue()); + assertIsBlocked(indexName, clusterState, false); + } + + private static void assertIsClosed(final String indexName, final ClusterState clusterState) { + assertThat(clusterState.metaData().index(indexName).getState(), is(IndexMetaData.State.CLOSE)); + assertThat(clusterState.routingTable().index(indexName), nullValue()); + assertIsBlocked(indexName, clusterState, true); + } + + private static void assertIsBlocked(final String indexName, final ClusterState clusterState, final boolean blocked) { + assertThat(clusterState.blocks().hasIndexBlock(indexName, MetaDataIndexStateService.INDEX_CLOSED_BLOCK), is(blocked)); } } diff --git a/server/src/test/java/org/elasticsearch/cluster/metadata/MetaDataIndexStateServiceUtils.java b/server/src/test/java/org/elasticsearch/cluster/metadata/MetaDataIndexStateServiceUtils.java new file mode 100644 index 0000000000000..a9ffd4c47e161 --- /dev/null +++ b/server/src/test/java/org/elasticsearch/cluster/metadata/MetaDataIndexStateServiceUtils.java @@ -0,0 +1,46 @@ +/* + * 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.action.support.master.AcknowledgedResponse; +import org.elasticsearch.cluster.ClusterState; +import org.elasticsearch.index.Index; + +import java.util.Map; +import java.util.Set; + +public class MetaDataIndexStateServiceUtils { + + private MetaDataIndexStateServiceUtils(){ + } + + /** + * Allows to call {@link MetaDataIndexStateService#addIndexClosedBlocks(Index[], ClusterState, Set)} which is a protected method. + */ + public static ClusterState addIndexClosedBlocks(final Index[] indices, final ClusterState state, final Set blockedIndices) { + return MetaDataIndexStateService.addIndexClosedBlocks(indices, state, blockedIndices); + } + + /** + * Allows to call {@link MetaDataIndexStateService#closeRoutingTable(ClusterState, Map)} which is a protected method. + */ + public static ClusterState closeRoutingTable(final ClusterState state, final Map results) { + return MetaDataIndexStateService.closeRoutingTable(state, results); + } +} diff --git a/server/src/test/java/org/elasticsearch/indices/cluster/ClusterStateChanges.java b/server/src/test/java/org/elasticsearch/indices/cluster/ClusterStateChanges.java index af04fd5e27d9b..1403543078b28 100644 --- a/server/src/test/java/org/elasticsearch/indices/cluster/ClusterStateChanges.java +++ b/server/src/test/java/org/elasticsearch/indices/cluster/ClusterStateChanges.java @@ -28,6 +28,7 @@ import org.elasticsearch.action.admin.cluster.reroute.TransportClusterRerouteAction; import org.elasticsearch.action.admin.indices.close.CloseIndexRequest; import org.elasticsearch.action.admin.indices.close.TransportCloseIndexAction; +import org.elasticsearch.action.admin.indices.close.TransportVerifyShardBeforeCloseAction; import org.elasticsearch.action.admin.indices.create.CreateIndexRequest; import org.elasticsearch.action.admin.indices.create.TransportCreateIndexAction; import org.elasticsearch.action.admin.indices.delete.DeleteIndexRequest; @@ -39,6 +40,7 @@ import org.elasticsearch.action.support.ActionFilters; import org.elasticsearch.action.support.DestructiveOperations; import org.elasticsearch.action.support.PlainActionFuture; +import org.elasticsearch.action.support.master.AcknowledgedResponse; import org.elasticsearch.action.support.master.MasterNodeRequest; import org.elasticsearch.action.support.master.TransportMasterNodeAction; import org.elasticsearch.action.support.master.TransportMasterNodeActionUtils; @@ -58,6 +60,7 @@ import org.elasticsearch.cluster.metadata.MetaDataCreateIndexService; import org.elasticsearch.cluster.metadata.MetaDataDeleteIndexService; import org.elasticsearch.cluster.metadata.MetaDataIndexStateService; +import org.elasticsearch.cluster.metadata.MetaDataIndexStateServiceUtils; import org.elasticsearch.cluster.metadata.MetaDataIndexUpgradeService; import org.elasticsearch.cluster.metadata.MetaDataUpdateSettingsService; import org.elasticsearch.cluster.node.DiscoveryNode; @@ -77,6 +80,7 @@ import org.elasticsearch.common.xcontent.NamedXContentRegistry; import org.elasticsearch.env.Environment; import org.elasticsearch.env.TestEnvironment; +import org.elasticsearch.index.Index; import org.elasticsearch.index.IndexService; import org.elasticsearch.index.mapper.MapperService; import org.elasticsearch.index.shard.IndexEventListener; @@ -92,6 +96,8 @@ import java.util.Collections; import java.util.HashSet; import java.util.List; +import java.util.Set; +import java.util.function.Function; import java.util.stream.Collectors; import static com.carrotsearch.randomizedtesting.RandomizedTest.getRandom; @@ -179,8 +185,11 @@ public IndexMetaData upgradeIndexMetaData(IndexMetaData indexMetaData, Version m return indexMetaData; } }; + + TransportVerifyShardBeforeCloseAction transportVerifyShardBeforeCloseAction = new TransportVerifyShardBeforeCloseAction(SETTINGS, + transportService, clusterService, indicesService, threadPool, null, actionFilters, indexNameExpressionResolver); MetaDataIndexStateService indexStateService = new MetaDataIndexStateService(clusterService, allocationService, - metaDataIndexUpgradeService, indicesService, threadPool); + metaDataIndexUpgradeService, indicesService, threadPool, transportVerifyShardBeforeCloseAction); MetaDataDeleteIndexService deleteIndexService = new MetaDataDeleteIndexService(SETTINGS, clusterService, allocationService); MetaDataUpdateSettingsService metaDataUpdateSettingsService = new MetaDataUpdateSettingsService(clusterService, allocationService, IndexScopedSettings.DEFAULT_SCOPED_SETTINGS, indicesService, threadPool); @@ -210,7 +219,15 @@ public ClusterState createIndex(ClusterState state, CreateIndexRequest request) } public ClusterState closeIndices(ClusterState state, CloseIndexRequest request) { - return execute(transportCloseIndexAction, request, state); + final Index[] concreteIndices = Arrays.stream(request.indices()) + .map(index -> state.metaData().index(index).getIndex()).toArray(Index[]::new); + + final Set blockedIndices = new HashSet<>(); + ClusterState newState = MetaDataIndexStateServiceUtils.addIndexClosedBlocks(concreteIndices, state, blockedIndices); + + newState = MetaDataIndexStateServiceUtils.closeRoutingTable(newState, blockedIndices.stream() + .collect(Collectors.toMap(Function.identity(), index -> new AcknowledgedResponse(true)))); + return allocationService.reroute(newState, "indices closed"); } public ClusterState openIndices(ClusterState state, OpenIndexRequest request) { diff --git a/server/src/test/java/org/elasticsearch/indices/state/CloseIndexIT.java b/server/src/test/java/org/elasticsearch/indices/state/CloseIndexIT.java new file mode 100644 index 0000000000000..c91189972c7b2 --- /dev/null +++ b/server/src/test/java/org/elasticsearch/indices/state/CloseIndexIT.java @@ -0,0 +1,273 @@ +/* + * 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.indices.state; + +import org.elasticsearch.ExceptionsHelper; +import org.elasticsearch.action.ActionRequestValidationException; +import org.elasticsearch.action.support.ActiveShardCount; +import org.elasticsearch.action.support.IndicesOptions; +import org.elasticsearch.cluster.ClusterState; +import org.elasticsearch.cluster.block.ClusterBlockException; +import org.elasticsearch.cluster.metadata.IndexMetaData; +import org.elasticsearch.cluster.metadata.MetaDataIndexStateService; +import org.elasticsearch.cluster.routing.ShardRouting; +import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.index.IndexNotFoundException; +import org.elasticsearch.indices.IndexClosedException; +import org.elasticsearch.test.BackgroundIndexer; +import org.elasticsearch.test.ESIntegTestCase; + +import java.util.ArrayList; +import java.util.List; +import java.util.Locale; +import java.util.concurrent.CountDownLatch; +import java.util.stream.IntStream; + +import static java.util.stream.Collectors.toList; +import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertAcked; +import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertHitCount; +import static org.hamcrest.Matchers.containsString; +import static org.hamcrest.Matchers.equalTo; +import static org.hamcrest.Matchers.hasItem; +import static org.hamcrest.Matchers.hasSize; +import static org.hamcrest.Matchers.is; +import static org.hamcrest.Matchers.notNullValue; +import static org.hamcrest.Matchers.nullValue; + +public class CloseIndexIT extends ESIntegTestCase { + + public void testCloseMissingIndex() { + IndexNotFoundException e = expectThrows(IndexNotFoundException.class, () -> client().admin().indices().prepareClose("test").get()); + assertThat(e.getMessage(), is("no such index [test]")); + } + + public void testCloseOneMissingIndex() { + createIndex("test1"); + final IndexNotFoundException e = expectThrows(IndexNotFoundException.class, + () -> client().admin().indices().prepareClose("test1", "test2").get()); + assertThat(e.getMessage(), is("no such index [test2]")); + } + + public void testCloseOneMissingIndexIgnoreMissing() { + createIndex("test1"); + assertAcked(client().admin().indices().prepareClose("test1", "test2").setIndicesOptions(IndicesOptions.lenientExpandOpen())); + assertIndexIsClosed("test1"); + } + + public void testCloseNoIndex() { + final ActionRequestValidationException e = expectThrows(ActionRequestValidationException.class, + () -> client().admin().indices().prepareClose().get()); + assertThat(e.getMessage(), containsString("index is missing")); + } + + public void testCloseNullIndex() { + final ActionRequestValidationException e = expectThrows(ActionRequestValidationException.class, + () -> client().admin().indices().prepareClose((String[])null).get()); + assertThat(e.getMessage(), containsString("index is missing")); + } + + public void testCloseIndex() throws Exception { + final String indexName = randomAlphaOfLength(10).toLowerCase(Locale.ROOT); + createIndex(indexName); + + final int nbDocs = randomIntBetween(0, 50); + indexRandom(randomBoolean(), false, randomBoolean(), IntStream.range(0, nbDocs) + .mapToObj(i -> client().prepareIndex(indexName, "_doc", String.valueOf(i)).setSource("num", i)).collect(toList())); + + assertAcked(client().admin().indices().prepareClose(indexName)); + assertIndexIsClosed(indexName); + + assertAcked(client().admin().indices().prepareOpen(indexName)); + assertHitCount(client().prepareSearch(indexName).setSize(0).get(), nbDocs); + } + + public void testCloseAlreadyClosedIndex() throws Exception { + final String indexName = randomAlphaOfLength(10).toLowerCase(Locale.ROOT); + createIndex(indexName); + + if (randomBoolean()) { + indexRandom(randomBoolean(), false, randomBoolean(), IntStream.range(0, randomIntBetween(1, 10)) + .mapToObj(i -> client().prepareIndex(indexName, "_doc", String.valueOf(i)).setSource("num", i)).collect(toList())); + } + // First close should be acked + assertAcked(client().admin().indices().prepareClose(indexName)); + assertIndexIsClosed(indexName); + + // Second close should be acked too + assertAcked(client().admin().indices().prepareClose(indexName)); + assertIndexIsClosed(indexName); + } + + public void testCloseUnassignedIndex() { + final String indexName = randomAlphaOfLength(10).toLowerCase(Locale.ROOT); + assertAcked(prepareCreate(indexName) + .setWaitForActiveShards(ActiveShardCount.NONE) + .setSettings(Settings.builder().put("index.routing.allocation.include._name", "nothing").build())); + + final ClusterState clusterState = client().admin().cluster().prepareState().get().getState(); + assertThat(clusterState.metaData().indices().get(indexName).getState(), is(IndexMetaData.State.OPEN)); + assertThat(clusterState.routingTable().allShards().stream().allMatch(ShardRouting::unassigned), is(true)); + + assertAcked(client().admin().indices().prepareClose(indexName)); + assertIndexIsClosed(indexName); + } + + public void testConcurrentClose() throws InterruptedException { + final String indexName = randomAlphaOfLength(10).toLowerCase(Locale.ROOT); + createIndex(indexName); + + final int nbDocs = randomIntBetween(10, 50); + indexRandom(randomBoolean(), false, randomBoolean(), IntStream.range(0, nbDocs) + .mapToObj(i -> client().prepareIndex(indexName, "_doc", String.valueOf(i)).setSource("num", i)).collect(toList())); + + final CountDownLatch startClosing = new CountDownLatch(1); + final Thread[] threads = new Thread[randomIntBetween(2, 5)]; + + for (int i = 0; i < threads.length; i++) { + threads[i] = new Thread(() -> { + try { + startClosing.await(); + } catch (InterruptedException e) { + throw new AssertionError(e); + } + assertAcked(client().admin().indices().prepareClose(indexName)); + }); + threads[i].start(); + } + + startClosing.countDown(); + for (Thread thread : threads) { + thread.join(); + } + assertIndexIsClosed(indexName); + } + + public void testCloseWhileIndexingDocuments() throws Exception { + final String indexName = randomAlphaOfLength(10).toLowerCase(Locale.ROOT); + createIndex(indexName); + + int nbDocs = 0; + try (BackgroundIndexer indexer = new BackgroundIndexer(indexName, "_doc", client())) { + indexer.setAssertNoFailuresOnStop(false); + + waitForDocs(randomIntBetween(10, 50), indexer); + assertAcked(client().admin().indices().prepareClose(indexName)); + indexer.stop(); + nbDocs += indexer.totalIndexedDocs(); + + final Throwable[] failures = indexer.getFailures(); + if (failures != null) { + for (Throwable failure : failures) { + assertException(failure, indexName); + } + } + } + + assertIndexIsClosed(indexName); + assertAcked(client().admin().indices().prepareOpen(indexName)); + assertHitCount(client().prepareSearch(indexName).setSize(0).get(), nbDocs); + } + + public void testCloseWhileDeletingIndices() throws Exception { + final String[] indices = new String[randomIntBetween(3, 10)]; + for (int i = 0; i < indices.length; i++) { + final String indexName = randomAlphaOfLength(10).toLowerCase(Locale.ROOT); + createIndex(indexName); + if (randomBoolean()) { + indexRandom(randomBoolean(), false, randomBoolean(), IntStream.range(0, 10) + .mapToObj(n -> client().prepareIndex(indexName, "_doc", String.valueOf(n)).setSource("num", n)).collect(toList())); + } + indices[i] = indexName; + } + assertThat(client().admin().cluster().prepareState().get().getState().metaData().indices().size(), equalTo(indices.length)); + + final List threads = new ArrayList<>(); + final CountDownLatch latch = new CountDownLatch(1); + + for (final String indexToDelete : indices) { + threads.add(new Thread(() -> { + try { + latch.await(); + } catch (InterruptedException e) { + throw new AssertionError(e); + } + try { + assertAcked(client().admin().indices().prepareDelete(indexToDelete)); + } catch (final Exception e) { + assertException(e, indexToDelete); + } + })); + } + for (final String indexToClose : indices) { + threads.add(new Thread(() -> { + try { + latch.await(); + } catch (InterruptedException e) { + throw new AssertionError(e); + } + try { + client().admin().indices().prepareClose(indexToClose).get(); + } catch (final Exception e) { + assertException(e, indexToClose); + } + })); + } + + for (Thread thread : threads) { + thread.start(); + } + latch.countDown(); + for (Thread thread : threads) { + thread.join(); + } + } + + static void assertIndexIsClosed(final String indexName) { + final ClusterState clusterState = client().admin().cluster().prepareState().get().getState(); + assertThat(clusterState.metaData().indices().get(indexName).getState(), is(IndexMetaData.State.CLOSE)); + assertThat(clusterState.routingTable().index(indexName), nullValue()); + assertThat(clusterState.blocks().hasIndexBlock(indexName, MetaDataIndexStateService.INDEX_CLOSED_BLOCK), is(true)); + } + + static void assertIndexIsOpened(final String indexName) { + final ClusterState clusterState = client().admin().cluster().prepareState().get().getState(); + assertThat(clusterState.metaData().indices().get(indexName).getState(), is(IndexMetaData.State.OPEN)); + assertThat(clusterState.routingTable().index(indexName), notNullValue()); + assertThat(clusterState.blocks().hasIndexBlock(indexName, MetaDataIndexStateService.INDEX_CLOSED_BLOCK), is(false)); + } + + static void assertException(final Throwable throwable, final String indexName) { + final Throwable t = ExceptionsHelper.unwrapCause(throwable); + if (t instanceof ClusterBlockException) { + ClusterBlockException clusterBlockException = (ClusterBlockException) t; + assertThat(clusterBlockException.blocks(), hasSize(1)); + assertThat(clusterBlockException.blocks(), hasItem(MetaDataIndexStateService.INDEX_CLOSED_BLOCK)); + } else if (t instanceof IndexClosedException) { + IndexClosedException indexClosedException = (IndexClosedException) t; + assertThat(indexClosedException.getIndex(), notNullValue()); + assertThat(indexClosedException.getIndex().getName(), equalTo(indexName)); + } else if (t instanceof IndexNotFoundException) { + IndexNotFoundException indexNotFoundException = (IndexNotFoundException) t; + assertThat(indexNotFoundException.getIndex(), notNullValue()); + assertThat(indexNotFoundException.getIndex().getName(), equalTo(indexName)); + } else { + fail("Unexpected exception: " + t); + } + } +} diff --git a/server/src/test/java/org/elasticsearch/indices/state/CloseWhileRelocatingShardsIT.java b/server/src/test/java/org/elasticsearch/indices/state/CloseWhileRelocatingShardsIT.java new file mode 100644 index 0000000000000..c549471074dfc --- /dev/null +++ b/server/src/test/java/org/elasticsearch/indices/state/CloseWhileRelocatingShardsIT.java @@ -0,0 +1,195 @@ +/* + * 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.indices.state; + +import org.elasticsearch.action.index.IndexResponse; +import org.elasticsearch.action.support.master.AcknowledgedResponse; +import org.elasticsearch.cluster.ClusterState; +import org.elasticsearch.cluster.routing.IndexRoutingTable; +import org.elasticsearch.cluster.routing.ShardRouting; +import org.elasticsearch.cluster.routing.allocation.command.MoveAllocationCommand; +import org.elasticsearch.cluster.routing.allocation.decider.EnableAllocationDecider; +import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.common.util.concurrent.ConcurrentCollections; +import org.elasticsearch.rest.RestStatus; +import org.elasticsearch.test.BackgroundIndexer; +import org.elasticsearch.test.ESIntegTestCase; + +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.concurrent.CountDownLatch; + +import static org.elasticsearch.cluster.routing.allocation.decider.ConcurrentRebalanceAllocationDecider.CLUSTER_ROUTING_ALLOCATION_CLUSTER_CONCURRENT_REBALANCE_SETTING; +import static org.elasticsearch.cluster.routing.allocation.decider.EnableAllocationDecider.CLUSTER_ROUTING_REBALANCE_ENABLE_SETTING; +import static org.elasticsearch.cluster.routing.allocation.decider.ThrottlingAllocationDecider.CLUSTER_ROUTING_ALLOCATION_NODE_CONCURRENT_RECOVERIES_SETTING; +import static org.elasticsearch.indices.state.CloseIndexIT.assertException; +import static org.elasticsearch.indices.state.CloseIndexIT.assertIndexIsClosed; +import static org.elasticsearch.indices.state.CloseIndexIT.assertIndexIsOpened; +import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertAcked; +import static org.hamcrest.Matchers.greaterThan; + +@ESIntegTestCase.ClusterScope(minNumDataNodes = 2) +public class CloseWhileRelocatingShardsIT extends ESIntegTestCase { + + @Override + protected Settings nodeSettings(int nodeOrdinal) { + return Settings.builder() + .put(super.nodeSettings(nodeOrdinal)) + .put(CLUSTER_ROUTING_ALLOCATION_NODE_CONCURRENT_RECOVERIES_SETTING.getKey(), 10) + .put(CLUSTER_ROUTING_ALLOCATION_CLUSTER_CONCURRENT_REBALANCE_SETTING.getKey(), -1) + .build(); + } + + @Override + protected int numberOfReplicas() { + return 1; + } + + public void testCloseWhileRelocatingShards() throws Exception { + final String[] indices = new String[randomIntBetween(3, 10)]; + final Map docsPerIndex = new HashMap<>(); + + for (int i = 0; i < indices.length; i++) { + final String indexName = "index-" + i; + createIndex(indexName); + + int nbDocs = 0; + if (randomBoolean()) { + nbDocs = randomIntBetween(1, 20); + for (int j = 0; j < nbDocs; j++) { + IndexResponse indexResponse = client().prepareIndex(indexName, "_doc").setSource("num", j).get(); + assertEquals(RestStatus.CREATED, indexResponse.status()); + } + } + docsPerIndex.put(indexName, (long) nbDocs); + indices[i] = indexName; + } + + ensureGreen(indices); + assertAcked(client().admin().cluster().prepareUpdateSettings() + .setTransientSettings(Settings.builder() + .put(CLUSTER_ROUTING_REBALANCE_ENABLE_SETTING.getKey(), EnableAllocationDecider.Rebalance.NONE.toString()))); + + // start some concurrent indexing threads + final Map indexers = new HashMap<>(); + for (final String index : indices) { + if (randomBoolean()) { + final BackgroundIndexer indexer = new BackgroundIndexer(index, "_doc", client()); + waitForDocs(1, indexer); + indexers.put(index, indexer); + } + } + + final Set acknowledgedCloses = ConcurrentCollections.newConcurrentSet(); + final String newNode = internalCluster().startDataOnlyNode(); + try { + final CountDownLatch latch = new CountDownLatch(1); + final List threads = new ArrayList<>(); + + // start shards relocating threads + final ClusterState clusterState = client().admin().cluster().prepareState().get().getState(); + for (final String indexToRelocate : indices) { + final IndexRoutingTable indexRoutingTable = clusterState.routingTable().index(indexToRelocate); + for (int i = 0; i < getNumShards(indexToRelocate).numPrimaries; i++) { + final int shardId = i; + ShardRouting primary = indexRoutingTable.shard(shardId).primaryShard(); + assertTrue(primary.started()); + ShardRouting replica = indexRoutingTable.shard(shardId).replicaShards().iterator().next(); + assertTrue(replica.started()); + + final String currentNodeId = randomBoolean() ? primary.currentNodeId() : replica.currentNodeId(); + assertNotNull(currentNodeId); + + final Thread thread = new Thread(() -> { + try { + latch.await(); + } catch (InterruptedException e) { + throw new AssertionError(e); + } + assertAcked(client().admin().cluster().prepareReroute() + .add(new MoveAllocationCommand(indexToRelocate, shardId, currentNodeId, newNode))); + }); + threads.add(thread); + thread.start(); + } + } + + // start index closing threads + for (final String indexToClose : indices) { + final Thread thread = new Thread(() -> { + try { + latch.await(); + } catch (InterruptedException e) { + throw new AssertionError(e); + } + AcknowledgedResponse closeResponse = client().admin().indices().prepareClose(indexToClose).get(); + if (closeResponse.isAcknowledged()) { + assertTrue("Index closing should not be acknowledged twice", acknowledgedCloses.add(indexToClose)); + } + }); + threads.add(thread); + thread.start(); + } + + latch.countDown(); + for (Thread thread : threads) { + thread.join(); + } + for (Map.Entry entry : indexers.entrySet()) { + final BackgroundIndexer indexer = entry.getValue(); + indexer.setAssertNoFailuresOnStop(false); + indexer.stop(); + + final String indexName = entry.getKey(); + docsPerIndex.computeIfPresent(indexName, (key, value) -> value + indexer.totalIndexedDocs()); + + final Throwable[] failures = indexer.getFailures(); + if (failures != null) { + for (Throwable failure : failures) { + assertException(failure, indexName); + } + } + } + } finally { + assertAcked(client().admin().cluster().prepareUpdateSettings() + .setTransientSettings(Settings.builder().putNull(CLUSTER_ROUTING_REBALANCE_ENABLE_SETTING.getKey()))); + } + + for (String index : indices) { + if (acknowledgedCloses.contains(index)) { + assertIndexIsClosed(index); + } else { + assertIndexIsOpened(index); + } + } + + assertThat("Consider that the test failed if no indices were successfully closed", acknowledgedCloses.size(), greaterThan(0)); + assertAcked(client().admin().indices().prepareOpen("index-*")); + ensureGreen(indices); + + for (String index : acknowledgedCloses) { + long docsCount = client().prepareSearch(index).setSize(0).get().getHits().getTotalHits().value; + assertEquals("Expected " + docsPerIndex.get(index) + " docs in index " + index + " but got " + docsCount + + " (close acknowledged=" + acknowledgedCloses.contains(index) + ")", (long) docsPerIndex.get(index), docsCount); + } + } +} diff --git a/server/src/test/java/org/elasticsearch/indices/state/OpenCloseIndexIT.java b/server/src/test/java/org/elasticsearch/indices/state/OpenCloseIndexIT.java index 05b2ae1b9cffe..ddbbd0ea73a8f 100644 --- a/server/src/test/java/org/elasticsearch/indices/state/OpenCloseIndexIT.java +++ b/server/src/test/java/org/elasticsearch/indices/state/OpenCloseIndexIT.java @@ -72,13 +72,6 @@ public void testSimpleCloseOpen() { assertIndexIsOpened("test1"); } - public void testSimpleCloseMissingIndex() { - Client client = client(); - Exception e = expectThrows(IndexNotFoundException.class, () -> - client.admin().indices().prepareClose("test1").execute().actionGet()); - assertThat(e.getMessage(), is("no such index [test1]")); - } - public void testSimpleOpenMissingIndex() { Client client = client(); Exception e = expectThrows(IndexNotFoundException.class, () -> @@ -86,27 +79,6 @@ public void testSimpleOpenMissingIndex() { assertThat(e.getMessage(), is("no such index [test1]")); } - public void testCloseOneMissingIndex() { - Client client = client(); - createIndex("test1"); - ClusterHealthResponse healthResponse = client.admin().cluster().prepareHealth().setWaitForGreenStatus().execute().actionGet(); - assertThat(healthResponse.isTimedOut(), equalTo(false)); - Exception e = expectThrows(IndexNotFoundException.class, () -> - client.admin().indices().prepareClose("test1", "test2").execute().actionGet()); - assertThat(e.getMessage(), is("no such index [test2]")); - } - - public void testCloseOneMissingIndexIgnoreMissing() { - Client client = client(); - createIndex("test1"); - ClusterHealthResponse healthResponse = client.admin().cluster().prepareHealth().setWaitForGreenStatus().execute().actionGet(); - assertThat(healthResponse.isTimedOut(), equalTo(false)); - AcknowledgedResponse closeIndexResponse = client.admin().indices().prepareClose("test1", "test2") - .setIndicesOptions(IndicesOptions.lenientExpandOpen()).execute().actionGet(); - assertThat(closeIndexResponse.isAcknowledged(), equalTo(true)); - assertIndexIsClosed("test1"); - } - public void testOpenOneMissingIndex() { Client client = client(); createIndex("test1"); @@ -200,20 +172,6 @@ public void testCloseOpenAllWildcard() { assertIndexIsOpened("test1", "test2", "test3"); } - public void testCloseNoIndex() { - Client client = client(); - Exception e = expectThrows(ActionRequestValidationException.class, () -> - client.admin().indices().prepareClose().execute().actionGet()); - assertThat(e.getMessage(), containsString("index is missing")); - } - - public void testCloseNullIndex() { - Client client = client(); - Exception e = expectThrows(ActionRequestValidationException.class, () -> - client.admin().indices().prepareClose((String[])null).execute().actionGet()); - assertThat(e.getMessage(), containsString("index is missing")); - } - public void testOpenNoIndex() { Client client = client(); Exception e = expectThrows(ActionRequestValidationException.class, () -> @@ -241,23 +199,6 @@ public void testOpenAlreadyOpenedIndex() { assertIndexIsOpened("test1"); } - public void testCloseAlreadyClosedIndex() { - Client client = client(); - createIndex("test1"); - ClusterHealthResponse healthResponse = client.admin().cluster().prepareHealth().setWaitForGreenStatus().execute().actionGet(); - assertThat(healthResponse.isTimedOut(), equalTo(false)); - - //closing the index - AcknowledgedResponse closeIndexResponse = client.admin().indices().prepareClose("test1").execute().actionGet(); - assertThat(closeIndexResponse.isAcknowledged(), equalTo(true)); - assertIndexIsClosed("test1"); - - //no problem if we try to close an index that's already in close state - closeIndexResponse = client.admin().indices().prepareClose("test1").execute().actionGet(); - assertThat(closeIndexResponse.isAcknowledged(), equalTo(true)); - assertIndexIsClosed("test1"); - } - public void testSimpleCloseOpenAlias() { Client client = client(); createIndex("test1"); diff --git a/server/src/test/java/org/elasticsearch/indices/state/SimpleIndexStateIT.java b/server/src/test/java/org/elasticsearch/indices/state/SimpleIndexStateIT.java index 59d04c767d809..1cc2d3e68e2ae 100644 --- a/server/src/test/java/org/elasticsearch/indices/state/SimpleIndexStateIT.java +++ b/server/src/test/java/org/elasticsearch/indices/state/SimpleIndexStateIT.java @@ -26,7 +26,6 @@ import org.elasticsearch.action.admin.indices.create.CreateIndexResponse; import org.elasticsearch.action.admin.indices.open.OpenIndexResponse; import org.elasticsearch.action.support.ActiveShardCount; -import org.elasticsearch.action.support.master.AcknowledgedResponse; import org.elasticsearch.cluster.health.ClusterHealthStatus; import org.elasticsearch.cluster.metadata.IndexMetaData; import org.elasticsearch.cluster.routing.ShardRoutingState; @@ -35,6 +34,7 @@ import org.elasticsearch.indices.IndexClosedException; import org.elasticsearch.test.ESIntegTestCase; +import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertAcked; import static org.hamcrest.Matchers.equalTo; import static org.hamcrest.Matchers.nullValue; @@ -61,8 +61,7 @@ public void testSimpleOpenClose() { client().prepareIndex("test", "type1", "1").setSource("field1", "value1").get(); logger.info("--> closing test index..."); - AcknowledgedResponse closeIndexResponse = client().admin().indices().prepareClose("test").get(); - assertThat(closeIndexResponse.isAcknowledged(), equalTo(true)); + assertAcked(client().admin().indices().prepareClose("test")); stateResponse = client().admin().cluster().prepareState().get(); assertThat(stateResponse.getState().metaData().index("test").getState(), equalTo(IndexMetaData.State.CLOSE)); @@ -103,7 +102,7 @@ public void testFastCloseAfterCreateContinuesCreateAfterOpen() { assertThat(health.isTimedOut(), equalTo(false)); assertThat(health.getStatus(), equalTo(ClusterHealthStatus.RED)); - client().admin().indices().prepareClose("test").get(); + assertAcked(client().admin().indices().prepareClose("test")); logger.info("--> updating test index settings to allow allocation"); client().admin().indices().prepareUpdateSettings("test").setSettings(Settings.builder() diff --git a/server/src/test/java/org/elasticsearch/snapshots/DedicatedClusterSnapshotRestoreIT.java b/server/src/test/java/org/elasticsearch/snapshots/DedicatedClusterSnapshotRestoreIT.java index a3be8cfa15baa..619a09cc18880 100644 --- a/server/src/test/java/org/elasticsearch/snapshots/DedicatedClusterSnapshotRestoreIT.java +++ b/server/src/test/java/org/elasticsearch/snapshots/DedicatedClusterSnapshotRestoreIT.java @@ -594,7 +594,7 @@ public void testRestoreIndexWithMissingShards() throws Exception { equalTo(SnapshotState.PARTIAL)); } - assertAcked(client().admin().indices().prepareClose("test-idx-some", "test-idx-all").execute().actionGet()); + assertAcked(client().admin().indices().prepareClose("test-idx-some", "test-idx-all")); logger.info("--> restore incomplete snapshot - should fail"); assertThrows(client().admin().cluster().prepareRestoreSnapshot("test-repo", "test-snap-2").setRestoreGlobalState(false) diff --git a/test/framework/src/main/java/org/elasticsearch/test/BackgroundIndexer.java b/test/framework/src/main/java/org/elasticsearch/test/BackgroundIndexer.java index 198c02829b171..eabb05a537ca7 100644 --- a/test/framework/src/main/java/org/elasticsearch/test/BackgroundIndexer.java +++ b/test/framework/src/main/java/org/elasticsearch/test/BackgroundIndexer.java @@ -20,12 +20,10 @@ import com.carrotsearch.randomizedtesting.RandomizedTest; import com.carrotsearch.randomizedtesting.generators.RandomNumbers; import com.carrotsearch.randomizedtesting.generators.RandomStrings; - import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; import org.apache.logging.log4j.message.ParameterizedMessage; import org.apache.logging.log4j.util.Supplier; -import org.elasticsearch.ElasticsearchException; import org.elasticsearch.action.bulk.BulkItemResponse; import org.elasticsearch.action.bulk.BulkRequestBuilder; import org.elasticsearch.action.bulk.BulkResponse; @@ -63,6 +61,7 @@ public class BackgroundIndexer implements AutoCloseable { final Semaphore availableBudget = new Semaphore(0); final boolean useAutoGeneratedIDs; private final Set ids = ConcurrentCollections.newConcurrentSet(); + private boolean assertNoFailuresOnStop = true; volatile int minFieldSize = 10; volatile int maxFieldSize = 140; @@ -163,13 +162,11 @@ public void run() { } BulkResponse bulkResponse = bulkRequest.get(); for (BulkItemResponse bulkItemResponse : bulkResponse) { - if (!bulkItemResponse.isFailed()) { + if (bulkItemResponse.isFailed() == false) { boolean add = ids.add(bulkItemResponse.getId()); assert add : "ID: " + bulkItemResponse.getId() + " already used"; } else { - throw new ElasticsearchException("bulk request failure, id: [" - + bulkItemResponse.getFailure().getId() + "] message: " - + bulkItemResponse.getFailure().getMessage()); + failures.add(bulkItemResponse.getFailure().getCause()); } } @@ -283,7 +280,9 @@ public void stop() throws InterruptedException { } stop.set(true); Assert.assertThat("timeout while waiting for indexing threads to stop", stopLatch.await(6, TimeUnit.MINUTES), equalTo(true)); - assertNoFailures(); + if (assertNoFailuresOnStop) { + assertNoFailures(); + } } public long totalIndexedDocs() { @@ -308,6 +307,10 @@ public void setMaxFieldSize(int fieldSize) { maxFieldSize = fieldSize; } + public void setAssertNoFailuresOnStop(final boolean assertNoFailuresOnStop) { + this.assertNoFailuresOnStop = assertNoFailuresOnStop; + } + @Override public void close() throws Exception { stop(); diff --git a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/action/TransportFreezeIndexAction.java b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/action/TransportFreezeIndexAction.java index 1feb84d9b9539..36cce46d47c46 100644 --- a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/action/TransportFreezeIndexAction.java +++ b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/action/TransportFreezeIndexAction.java @@ -5,11 +5,14 @@ */ package org.elasticsearch.xpack.core.action; +import org.apache.logging.log4j.message.ParameterizedMessage; import org.elasticsearch.ResourceNotFoundException; import org.elasticsearch.action.Action; import org.elasticsearch.action.ActionListener; import org.elasticsearch.action.ActionRequestValidationException; import org.elasticsearch.action.IndicesRequest; +import org.elasticsearch.action.admin.indices.close.CloseIndexClusterStateUpdateRequest; +import org.elasticsearch.action.admin.indices.close.TransportCloseIndexAction; import org.elasticsearch.action.admin.indices.open.OpenIndexClusterStateUpdateRequest; import org.elasticsearch.action.admin.indices.open.OpenIndexResponse; import org.elasticsearch.action.support.ActionFilters; @@ -54,17 +57,20 @@ public final class TransportFreezeIndexAction extends private final DestructiveOperations destructiveOperations; private final MetaDataIndexStateService indexStateService; + private final TransportCloseIndexAction transportCloseIndexAction; @Inject public TransportFreezeIndexAction(MetaDataIndexStateService indexStateService, TransportService transportService, ClusterService clusterService, ThreadPool threadPool, ActionFilters actionFilters, IndexNameExpressionResolver indexNameExpressionResolver, - DestructiveOperations destructiveOperations) { + DestructiveOperations destructiveOperations, + TransportCloseIndexAction transportCloseIndexAction) { super(FreezeIndexAction.NAME, transportService, clusterService, threadPool, actionFilters, indexNameExpressionResolver, FreezeRequest::new); this.destructiveOperations = destructiveOperations; this.indexStateService = indexStateService; + this.transportCloseIndexAction = transportCloseIndexAction; } @Override protected String executor() { @@ -108,6 +114,33 @@ protected void masterOperation(FreezeRequest request, ClusterState state, Action listener.onResponse(new FreezeResponse(true, true)); return; } + + final CloseIndexClusterStateUpdateRequest closeRequest = new CloseIndexClusterStateUpdateRequest() + .ackTimeout(request.timeout()) + .masterNodeTimeout(request.masterNodeTimeout()) + .indices(concreteIndices); + + indexStateService.closeIndices(closeRequest, new ActionListener() { + @Override + public void onResponse(final AcknowledgedResponse response) { + if (response.isAcknowledged()) { + toggleFrozenSettings(concreteIndices, request, listener); + } else { + // TODO improve FreezeResponse so that it also reports failures from the close index API + listener.onResponse(new FreezeResponse(false, false)); + } + } + + @Override + public void onFailure(final Exception t) { + logger.debug(() -> new ParameterizedMessage("failed to close indices [{}]", (Object) concreteIndices), t); + listener.onFailure(t); + } + }); + } + + private void toggleFrozenSettings(final Index[] concreteIndices, final FreezeRequest request, + final ActionListener listener) { clusterService.submitStateUpdateTask("toggle-frozen-settings", new AckedClusterStateUpdateTask(Priority.URGENT, request, new ActionListener() { @Override @@ -136,14 +169,6 @@ public void onFailure(Exception e) { }) { @Override public ClusterState execute(ClusterState currentState) { - List toClose = new ArrayList<>(); - for (Index index : concreteIndices) { - IndexMetaData metaData = currentState.metaData().index(index); - if (metaData.getState() != IndexMetaData.State.CLOSE) { - toClose.add(index); - } - } - currentState = indexStateService.closeIndices(currentState, toClose.toArray(new Index[0]), toClose.toString()); final MetaData.Builder builder = MetaData.builder(currentState.metaData()); ClusterBlocks.Builder blocks = ClusterBlocks.builder().blocks(currentState.blocks()); for (Index index : concreteIndices) { diff --git a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/security/authz/privilege/IndexPrivilege.java b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/security/authz/privilege/IndexPrivilege.java index 6281fbb2c8fd6..e7a1a82042344 100644 --- a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/security/authz/privilege/IndexPrivilege.java +++ b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/security/authz/privilege/IndexPrivilege.java @@ -59,7 +59,7 @@ public final class IndexPrivilege extends Privilege { GetIndexAction.NAME, IndicesExistsAction.NAME, GetFieldMappingsAction.NAME + "*", GetMappingsAction.NAME, ClusterSearchShardsAction.NAME, TypesExistsAction.NAME, ValidateQueryAction.NAME + "*", GetSettingsAction.NAME); private static final Automaton MANAGE_FOLLOW_INDEX_AUTOMATON = patterns(PutFollowAction.NAME, UnfollowAction.NAME, - CloseIndexAction.NAME); + CloseIndexAction.NAME + "*"); public static final IndexPrivilege NONE = new IndexPrivilege("none", Automatons.EMPTY); public static final IndexPrivilege ALL = new IndexPrivilege("all", ALL_AUTOMATON); From ef6ae699cf1f626c4f0966cb3618daef8b029136 Mon Sep 17 00:00:00 2001 From: Tanguy Leroux Date: Mon, 17 Dec 2018 10:28:26 +0100 Subject: [PATCH 03/46] [Close Index API] Adapt MetaDataIndexStateServiceTests after merge --- .../cluster/metadata/MetaDataIndexStateServiceTests.java | 7 +++++-- 1 file changed, 5 insertions(+), 2 deletions(-) diff --git a/server/src/test/java/org/elasticsearch/cluster/metadata/MetaDataIndexStateServiceTests.java b/server/src/test/java/org/elasticsearch/cluster/metadata/MetaDataIndexStateServiceTests.java index a5a9e5ee22796..6faaf8e133800 100644 --- a/server/src/test/java/org/elasticsearch/cluster/metadata/MetaDataIndexStateServiceTests.java +++ b/server/src/test/java/org/elasticsearch/cluster/metadata/MetaDataIndexStateServiceTests.java @@ -19,6 +19,7 @@ package org.elasticsearch.cluster.metadata; +import com.google.common.collect.ImmutableList; import org.elasticsearch.Version; import org.elasticsearch.action.support.master.AcknowledgedResponse; import org.elasticsearch.cluster.ClusterName; @@ -267,8 +268,10 @@ private static ClusterState addRestoredIndex(final String index, final int numSh final Snapshot snapshot = new Snapshot(randomAlphaOfLength(10), new SnapshotId(randomAlphaOfLength(5), randomAlphaOfLength(5))); final RestoreInProgress.Entry entry = - new RestoreInProgress.Entry(snapshot, RestoreInProgress.State.INIT, Collections.singletonList(index), shardsBuilder.build()); - return ClusterState.builder(newState).putCustom(RestoreInProgress.TYPE, new RestoreInProgress(entry)).build(); + new RestoreInProgress.Entry("_uuid", snapshot, RestoreInProgress.State.INIT, ImmutableList.of(index), shardsBuilder.build()); + return ClusterState.builder(newState) + .putCustom(RestoreInProgress.TYPE, new RestoreInProgress.Builder().add(entry).build()) + .build(); } private static ClusterState addSnapshotIndex(final String index, final int numShards, final int numReplicas, final ClusterState state) { From 7372529a95393114775c009501bd155abf310d07 Mon Sep 17 00:00:00 2001 From: Tanguy Leroux Date: Tue, 18 Dec 2018 09:25:47 +0100 Subject: [PATCH 04/46] [Tests] Reduce randomization in CloseWhileRelocatingShardsIT (#36694) --- .../indices/state/CloseWhileRelocatingShardsIT.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/server/src/test/java/org/elasticsearch/indices/state/CloseWhileRelocatingShardsIT.java b/server/src/test/java/org/elasticsearch/indices/state/CloseWhileRelocatingShardsIT.java index c549471074dfc..02484eb0de14d 100644 --- a/server/src/test/java/org/elasticsearch/indices/state/CloseWhileRelocatingShardsIT.java +++ b/server/src/test/java/org/elasticsearch/indices/state/CloseWhileRelocatingShardsIT.java @@ -65,7 +65,7 @@ protected int numberOfReplicas() { } public void testCloseWhileRelocatingShards() throws Exception { - final String[] indices = new String[randomIntBetween(3, 10)]; + final String[] indices = new String[randomIntBetween(1, 3)]; final Map docsPerIndex = new HashMap<>(); for (int i = 0; i < indices.length; i++) { @@ -93,7 +93,7 @@ public void testCloseWhileRelocatingShards() throws Exception { final Map indexers = new HashMap<>(); for (final String index : indices) { if (randomBoolean()) { - final BackgroundIndexer indexer = new BackgroundIndexer(index, "_doc", client()); + final BackgroundIndexer indexer = new BackgroundIndexer(index, "_doc", client(), -1, scaledRandomIntBetween(1, 3)); waitForDocs(1, indexer); indexers.put(index, indexer); } From 103c4d4762f52d5a946fd8b4051ca92ba3fcc884 Mon Sep 17 00:00:00 2001 From: Tanguy Leroux Date: Tue, 18 Dec 2018 17:43:14 +0100 Subject: [PATCH 05/46] [Close Index API] Mark unavailable shard copy as stale during verification (#36755) This pull request modifies the TransportVerifyShardBeforeCloseAction so that it marks unavailable shards as stale. --- ...TransportVerifyShardBeforeCloseAction.java | 27 +++ ...portVerifyShardBeforeCloseActionTests.java | 212 +++++++++++++++++- 2 files changed, 233 insertions(+), 6 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/action/admin/indices/close/TransportVerifyShardBeforeCloseAction.java b/server/src/main/java/org/elasticsearch/action/admin/indices/close/TransportVerifyShardBeforeCloseAction.java index de5e372dd1625..1d283cbe004d0 100644 --- a/server/src/main/java/org/elasticsearch/action/admin/indices/close/TransportVerifyShardBeforeCloseAction.java +++ b/server/src/main/java/org/elasticsearch/action/admin/indices/close/TransportVerifyShardBeforeCloseAction.java @@ -21,6 +21,7 @@ import org.elasticsearch.action.ActionListener; import org.elasticsearch.action.admin.indices.flush.FlushRequest; import org.elasticsearch.action.support.ActionFilters; +import org.elasticsearch.action.support.replication.ReplicationOperation; import org.elasticsearch.action.support.replication.ReplicationRequest; import org.elasticsearch.action.support.replication.ReplicationResponse; import org.elasticsearch.action.support.replication.TransportReplicationAction; @@ -39,6 +40,8 @@ import org.elasticsearch.threadpool.ThreadPool; import org.elasticsearch.transport.TransportService; +import java.util.function.Consumer; + public class TransportVerifyShardBeforeCloseAction extends TransportReplicationAction< TransportVerifyShardBeforeCloseAction.ShardRequest, TransportVerifyShardBeforeCloseAction.ShardRequest, ReplicationResponse> { @@ -109,6 +112,30 @@ private void executeShardOperation(final IndexShard indexShard) { logger.debug("{} shard is ready for closing", shardId); } + @Override + protected ReplicationOperation.Replicas newReplicasProxy(final long primaryTerm) { + return new VerifyShardBeforeCloseActionReplicasProxy(primaryTerm); + } + + /** + * A {@link ReplicasProxy} that marks as stale the shards that are unavailable during the verification + * and the flush of the shard. This is done to ensure that such shards won't be later promoted as primary + * or reopened in an unverified state with potential non flushed translog operations. + */ + class VerifyShardBeforeCloseActionReplicasProxy extends ReplicasProxy { + + VerifyShardBeforeCloseActionReplicasProxy(final long primaryTerm) { + super(primaryTerm); + } + + @Override + public void markShardCopyAsStaleIfNeeded(final ShardId shardId, final String allocationId, final Runnable onSuccess, + final Consumer onPrimaryDemoted, final Consumer onIgnoredFailure) { + shardStateAction.remoteShardFailed(shardId, allocationId, primaryTerm, true, "mark copy as stale", null, + createShardActionListener(onSuccess, onPrimaryDemoted, onIgnoredFailure)); + } + } + public static class ShardRequest extends ReplicationRequest { ShardRequest(){ diff --git a/server/src/test/java/org/elasticsearch/action/admin/indices/close/TransportVerifyShardBeforeCloseActionTests.java b/server/src/test/java/org/elasticsearch/action/admin/indices/close/TransportVerifyShardBeforeCloseActionTests.java index de0cc5dfd5a37..2764eee798e6b 100644 --- a/server/src/test/java/org/elasticsearch/action/admin/indices/close/TransportVerifyShardBeforeCloseActionTests.java +++ b/server/src/test/java/org/elasticsearch/action/admin/indices/close/TransportVerifyShardBeforeCloseActionTests.java @@ -18,27 +18,55 @@ */ package org.elasticsearch.action.admin.indices.close; +import org.apache.lucene.util.SetOnce; import org.elasticsearch.action.admin.indices.flush.FlushRequest; import org.elasticsearch.action.support.ActionFilters; +import org.elasticsearch.action.support.PlainActionFuture; +import org.elasticsearch.action.support.replication.ReplicationOperation; +import org.elasticsearch.action.support.replication.ReplicationResponse; +import org.elasticsearch.action.support.replication.TransportReplicationAction; +import org.elasticsearch.action.support.replication.TransportReplicationAction.ConcreteShardRequest; import org.elasticsearch.cluster.ClusterName; import org.elasticsearch.cluster.ClusterState; import org.elasticsearch.cluster.action.shard.ShardStateAction; import org.elasticsearch.cluster.block.ClusterBlocks; +import org.elasticsearch.cluster.metadata.IndexMetaData; import org.elasticsearch.cluster.metadata.IndexNameExpressionResolver; +import org.elasticsearch.cluster.routing.IndexShardRoutingTable; +import org.elasticsearch.cluster.routing.ShardRouting; +import org.elasticsearch.cluster.routing.ShardRoutingState; import org.elasticsearch.cluster.service.ClusterService; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.index.seqno.SeqNoStats; import org.elasticsearch.index.seqno.SequenceNumbers; import org.elasticsearch.index.shard.IndexShard; +import org.elasticsearch.index.shard.ReplicationGroup; import org.elasticsearch.index.shard.ShardId; import org.elasticsearch.indices.IndicesService; 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.TransportResponse; import org.elasticsearch.transport.TransportService; +import org.junit.After; +import org.junit.AfterClass; import org.junit.Before; +import org.junit.BeforeClass; +import java.util.Collections; +import java.util.List; +import java.util.Set; +import java.util.concurrent.TimeUnit; + +import static org.elasticsearch.action.support.replication.ClusterStateCreationUtils.state; import static org.elasticsearch.cluster.metadata.MetaDataIndexStateService.INDEX_CLOSED_BLOCK; +import static org.elasticsearch.test.ClusterServiceUtils.createClusterService; +import static org.elasticsearch.test.ClusterServiceUtils.setState; +import static org.hamcrest.Matchers.arrayWithSize; import static org.hamcrest.Matchers.equalTo; +import static org.hamcrest.Matchers.greaterThan; +import static org.hamcrest.Matchers.instanceOf; import static org.mockito.Matchers.any; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.times; @@ -47,9 +75,17 @@ public class TransportVerifyShardBeforeCloseActionTests extends ESTestCase { + private static ThreadPool threadPool; + private IndexShard indexShard; private TransportVerifyShardBeforeCloseAction action; private ClusterService clusterService; + private CapturingTransport transport; + + @BeforeClass + public static void beforeClass() { + threadPool = new TestThreadPool(getTestClass().getName()); + } @Override @Before @@ -64,13 +100,32 @@ public void setUp() throws Exception { final ShardId shardId = new ShardId("index", "_na_", randomIntBetween(0, 3)); when(indexShard.shardId()).thenReturn(shardId); - clusterService = mock(ClusterService.class); - when(clusterService.state()).thenReturn(new ClusterState.Builder(new ClusterName("test")) + clusterService = createClusterService(threadPool); + setState(clusterService, new ClusterState.Builder(clusterService.state()) .blocks(ClusterBlocks.builder().addIndexBlock("index", INDEX_CLOSED_BLOCK).build()).build()); - action = new TransportVerifyShardBeforeCloseAction(Settings.EMPTY, mock(TransportService.class), clusterService, - mock(IndicesService.class), mock(ThreadPool.class), mock(ShardStateAction.class), mock(ActionFilters.class), - mock(IndexNameExpressionResolver.class)); + transport = new CapturingTransport(); + TransportService transportService = transport.createTransportService(Settings.EMPTY, threadPool, + TransportService.NOOP_TRANSPORT_INTERCEPTOR, x -> clusterService.localNode(), null, Collections.emptySet()); + transportService.start(); + transportService.acceptIncomingRequests(); + + ShardStateAction shardStateAction = new ShardStateAction(clusterService, transportService, null, null, threadPool); + action = new TransportVerifyShardBeforeCloseAction(Settings.EMPTY, transportService, clusterService, mock(IndicesService.class), + mock(ThreadPool.class), shardStateAction, mock(ActionFilters.class), mock(IndexNameExpressionResolver.class)); + } + + @Override + @After + public void tearDown() throws Exception { + super.tearDown(); + clusterService.close(); + } + + @AfterClass + public static void afterClass() { + ThreadPool.terminate(threadPool, 30, TimeUnit.SECONDS); + threadPool = null; } private void executeOnPrimaryOrReplica() throws Exception { @@ -98,7 +153,7 @@ public void testOperationFailsWithOnGoingOps() { } public void testOperationFailsWithNoBlock() { - when(clusterService.state()).thenReturn(new ClusterState.Builder(new ClusterName("test")).build()); + setState(clusterService, new ClusterState.Builder(new ClusterName("test")).build()); IllegalStateException exception = expectThrows(IllegalStateException.class, this::executeOnPrimaryOrReplica); assertThat(exception.getMessage(), @@ -119,4 +174,149 @@ public void testOperationFailsWithGlobalCheckpointNotCaughtUp() { + maxSeqNo + "] on index shard " + indexShard.shardId())); verify(indexShard, times(0)).flush(any(FlushRequest.class)); } + + public void testUnavailableShardsMarkedAsStale() throws Exception { + final String index = "test"; + final ShardId shardId = new ShardId(index, "_na_", 0); + + final int nbReplicas = randomIntBetween(1, 10); + final ShardRoutingState[] replicaStates = new ShardRoutingState[nbReplicas]; + for (int i = 0; i < replicaStates.length; i++) { + replicaStates[i] = ShardRoutingState.STARTED; + } + final ClusterState clusterState = state(index, true, ShardRoutingState.STARTED, replicaStates); + setState(clusterService, clusterState); + + IndexShardRoutingTable shardRoutingTable = clusterState.routingTable().index(index).shard(shardId.id()); + final IndexMetaData indexMetaData = clusterState.getMetaData().index(index); + final ShardRouting primaryRouting = shardRoutingTable.primaryShard(); + final long primaryTerm = indexMetaData.primaryTerm(0); + + final Set inSyncAllocationIds = indexMetaData.inSyncAllocationIds(0); + final Set trackedShards = shardRoutingTable.getAllAllocationIds(); + + List unavailableShards = randomSubsetOf(randomIntBetween(1, nbReplicas), shardRoutingTable.replicaShards()); + IndexShardRoutingTable.Builder shardRoutingTableBuilder = new IndexShardRoutingTable.Builder(shardRoutingTable); + unavailableShards.forEach(shardRoutingTableBuilder::removeShard); + shardRoutingTable = shardRoutingTableBuilder.build(); + + final ReplicationGroup replicationGroup = new ReplicationGroup(shardRoutingTable, inSyncAllocationIds, trackedShards); + assertThat(replicationGroup.getUnavailableInSyncShards().size(), greaterThan(0)); + + final PlainActionFuture listener = new PlainActionFuture<>(); + TransportVerifyShardBeforeCloseAction.ShardRequest request = new TransportVerifyShardBeforeCloseAction.ShardRequest(shardId); + ReplicationOperation.Replicas proxy = action.newReplicasProxy(primaryTerm); + ReplicationOperation operation = + new ReplicationOperation<>(request, createPrimary(primaryRouting, replicationGroup), listener, proxy, logger, "test"); + operation.execute(); + + final CapturingTransport.CapturedRequest[] capturedRequests = transport.getCapturedRequestsAndClear(); + assertThat(capturedRequests.length, equalTo(nbReplicas)); + + for (CapturingTransport.CapturedRequest capturedRequest : capturedRequests) { + final String actionName = capturedRequest.action; + if (actionName.startsWith(ShardStateAction.SHARD_FAILED_ACTION_NAME)) { + assertThat(capturedRequest.request, instanceOf(ShardStateAction.FailedShardEntry.class)); + String allocationId = ((ShardStateAction.FailedShardEntry) capturedRequest.request).getAllocationId(); + assertTrue(unavailableShards.stream().anyMatch(shardRouting -> shardRouting.allocationId().getId().equals(allocationId))); + transport.handleResponse(capturedRequest.requestId, TransportResponse.Empty.INSTANCE); + + } else if (actionName.startsWith(TransportVerifyShardBeforeCloseAction.NAME)) { + assertThat(capturedRequest.request, instanceOf(ConcreteShardRequest.class)); + String allocationId = ((ConcreteShardRequest) capturedRequest.request).getTargetAllocationID(); + assertFalse(unavailableShards.stream().anyMatch(shardRouting -> shardRouting.allocationId().getId().equals(allocationId))); + assertTrue(inSyncAllocationIds.stream().anyMatch(inSyncAllocationId -> inSyncAllocationId.equals(allocationId))); + transport.handleResponse(capturedRequest.requestId, new TransportReplicationAction.ReplicaResponse(0L, 0L)); + + } else { + fail("Test does not support action " + capturedRequest.action); + } + } + + final ReplicationResponse.ShardInfo shardInfo = listener.get().getShardInfo(); + assertThat(shardInfo.getFailed(), equalTo(0)); + assertThat(shardInfo.getFailures(), arrayWithSize(0)); + assertThat(shardInfo.getSuccessful(), equalTo(1 + nbReplicas - unavailableShards.size())); + } + + private static ReplicationOperation.Primary< + TransportVerifyShardBeforeCloseAction.ShardRequest, + TransportVerifyShardBeforeCloseAction.ShardRequest, + PrimaryResult> + createPrimary(final ShardRouting primary, final ReplicationGroup replicationGroup) { + return new ReplicationOperation.Primary< + TransportVerifyShardBeforeCloseAction.ShardRequest, + TransportVerifyShardBeforeCloseAction.ShardRequest, + PrimaryResult>() { + @Override + public ShardRouting routingEntry() { + return primary; + } + + @Override + public ReplicationGroup getReplicationGroup() { + return replicationGroup; + } + + @Override + public PrimaryResult perform(TransportVerifyShardBeforeCloseAction.ShardRequest request) throws Exception { + return new PrimaryResult(request); + } + + @Override + public void failShard(String message, Exception exception) { + + } + + @Override + public void updateLocalCheckpointForShard(String allocationId, long checkpoint) { + } + + @Override + public void updateGlobalCheckpointForShard(String allocationId, long globalCheckpoint) { + } + + @Override + public long localCheckpoint() { + return 0; + } + + @Override + public long globalCheckpoint() { + return 0; + } + + @Override + public long maxSeqNoOfUpdatesOrDeletes() { + return 0; + } + }; + } + + private static class PrimaryResult implements ReplicationOperation.PrimaryResult { + + private final TransportVerifyShardBeforeCloseAction.ShardRequest replicaRequest; + private final SetOnce shardInfo; + + private PrimaryResult(final TransportVerifyShardBeforeCloseAction.ShardRequest replicaRequest) { + this.replicaRequest = replicaRequest; + this.shardInfo = new SetOnce<>(); + } + + @Override + public TransportVerifyShardBeforeCloseAction.ShardRequest replicaRequest() { + return replicaRequest; + } + + @Override + public void setShardInfo(ReplicationResponse.ShardInfo shardInfo) { + this.shardInfo.set(shardInfo); + } + + public ReplicationResponse.ShardInfo getShardInfo() { + return shardInfo.get(); + } + } + } From 19593884eef94392a48d5ba05c0255edb4ea0715 Mon Sep 17 00:00:00 2001 From: Tanguy Leroux Date: Mon, 7 Jan 2019 09:43:50 +0100 Subject: [PATCH 06/46] [Close Index API] Propagate tasks ids between Freeze, Close and Verify Shard actions (#36630) This pull request changes the Freeze Index and Close Index actions so that these actions always requires a Task. The task's id is then propagated from the Freeze action to the Close action, and then to the Verify shard action. This way it is possible to track which Freeze task initiates the closing of an index, and which consecutive verifiy shard are executed for the index closing. --- .../CloseIndexClusterStateUpdateRequest.java | 8 ++++- .../close/TransportCloseIndexAction.java | 8 ++++- ...TransportVerifyShardBeforeCloseAction.java | 4 ++- .../metadata/MetaDataIndexStateService.java | 30 +++++++++---------- ...portVerifyShardBeforeCloseActionTests.java | 6 ++-- .../action/TransportFreezeIndexAction.java | 8 ++++- 6 files changed, 42 insertions(+), 22 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/action/admin/indices/close/CloseIndexClusterStateUpdateRequest.java b/server/src/main/java/org/elasticsearch/action/admin/indices/close/CloseIndexClusterStateUpdateRequest.java index 8ad79f1676eb1..bb0f98ac07b7e 100644 --- a/server/src/main/java/org/elasticsearch/action/admin/indices/close/CloseIndexClusterStateUpdateRequest.java +++ b/server/src/main/java/org/elasticsearch/action/admin/indices/close/CloseIndexClusterStateUpdateRequest.java @@ -25,7 +25,13 @@ */ public class CloseIndexClusterStateUpdateRequest extends IndicesClusterStateUpdateRequest { - public CloseIndexClusterStateUpdateRequest() { + private final long taskId; + public CloseIndexClusterStateUpdateRequest(final long taskId) { + this.taskId = taskId; + } + + public long taskId() { + return taskId; } } diff --git a/server/src/main/java/org/elasticsearch/action/admin/indices/close/TransportCloseIndexAction.java b/server/src/main/java/org/elasticsearch/action/admin/indices/close/TransportCloseIndexAction.java index 605f0ed9217ac..bb3db084b0c53 100644 --- a/server/src/main/java/org/elasticsearch/action/admin/indices/close/TransportCloseIndexAction.java +++ b/server/src/main/java/org/elasticsearch/action/admin/indices/close/TransportCloseIndexAction.java @@ -99,13 +99,19 @@ protected ClusterBlockException checkBlock(CloseIndexRequest request, ClusterSta @Override protected void masterOperation(final CloseIndexRequest request, final ClusterState state, final ActionListener listener) { + throw new UnsupportedOperationException("The task parameter is required"); + } + + @Override + protected void masterOperation(final Task task, final CloseIndexRequest request, final ClusterState state, + final ActionListener listener) throws Exception { final Index[] concreteIndices = indexNameExpressionResolver.concreteIndices(state, request); if (concreteIndices == null || concreteIndices.length == 0) { listener.onResponse(new AcknowledgedResponse(true)); return; } - final CloseIndexClusterStateUpdateRequest closeRequest = new CloseIndexClusterStateUpdateRequest() + final CloseIndexClusterStateUpdateRequest closeRequest = new CloseIndexClusterStateUpdateRequest(task.getId()) .ackTimeout(request.timeout()) .masterNodeTimeout(request.masterNodeTimeout()) .indices(concreteIndices); diff --git a/server/src/main/java/org/elasticsearch/action/admin/indices/close/TransportVerifyShardBeforeCloseAction.java b/server/src/main/java/org/elasticsearch/action/admin/indices/close/TransportVerifyShardBeforeCloseAction.java index 1d283cbe004d0..f603f92a7189e 100644 --- a/server/src/main/java/org/elasticsearch/action/admin/indices/close/TransportVerifyShardBeforeCloseAction.java +++ b/server/src/main/java/org/elasticsearch/action/admin/indices/close/TransportVerifyShardBeforeCloseAction.java @@ -37,6 +37,7 @@ import org.elasticsearch.index.shard.IndexShard; import org.elasticsearch.index.shard.ShardId; import org.elasticsearch.indices.IndicesService; +import org.elasticsearch.tasks.TaskId; import org.elasticsearch.threadpool.ThreadPool; import org.elasticsearch.transport.TransportService; @@ -141,8 +142,9 @@ public static class ShardRequest extends ReplicationRequest { ShardRequest(){ } - public ShardRequest(final ShardId shardId) { + public ShardRequest(final ShardId shardId, final TaskId parentTaskId) { super(shardId); + setParentTask(parentTaskId); } @Override diff --git a/server/src/main/java/org/elasticsearch/cluster/metadata/MetaDataIndexStateService.java b/server/src/main/java/org/elasticsearch/cluster/metadata/MetaDataIndexStateService.java index cda8f9c6f0ac6..6ceda4bf57d13 100644 --- a/server/src/main/java/org/elasticsearch/cluster/metadata/MetaDataIndexStateService.java +++ b/server/src/main/java/org/elasticsearch/cluster/metadata/MetaDataIndexStateService.java @@ -46,7 +46,6 @@ import org.elasticsearch.cluster.routing.RoutingTable; import org.elasticsearch.cluster.routing.allocation.AllocationService; import org.elasticsearch.cluster.service.ClusterService; -import org.elasticsearch.common.Nullable; import org.elasticsearch.common.Priority; import org.elasticsearch.common.ValidationException; import org.elasticsearch.common.collect.ImmutableOpenIntMap; @@ -63,6 +62,7 @@ import org.elasticsearch.rest.RestStatus; import org.elasticsearch.snapshots.RestoreService; import org.elasticsearch.snapshots.SnapshotsService; +import org.elasticsearch.tasks.TaskId; import org.elasticsearch.threadpool.ThreadPool; import java.util.ArrayList; @@ -120,9 +120,6 @@ public void closeIndices(final CloseIndexClusterStateUpdateRequest request, fina throw new IllegalArgumentException("Index name is required"); } - final TimeValue timeout = request.ackTimeout(); - final TimeValue masterTimeout = request.masterNodeTimeout(); - clusterService.submitStateUpdateTask("add-block-index-to-close " + Arrays.toString(concreteIndices), new ClusterStateUpdateTask(Priority.URGENT) { @@ -141,7 +138,7 @@ public void clusterStateProcessed(final String source, final ClusterState oldSta } else { assert blockedIndices.isEmpty() == false : "List of blocked indices is empty but cluster state was changed"; threadPool.executor(ThreadPool.Names.MANAGEMENT) - .execute(new WaitForClosedBlocksApplied(blockedIndices, timeout, + .execute(new WaitForClosedBlocksApplied(blockedIndices, request, ActionListener.wrap(closedBlocksResults -> clusterService.submitStateUpdateTask("close-indices", new ClusterStateUpdateTask(Priority.URGENT) { @Override @@ -176,7 +173,7 @@ public void onFailure(final String source, final Exception e) { @Override public TimeValue timeout() { - return masterTimeout; + return request.masterNodeTimeout(); } } ); @@ -246,18 +243,18 @@ static ClusterState addIndexClosedBlocks(final Index[] indices, final ClusterSta class WaitForClosedBlocksApplied extends AbstractRunnable { private final Set blockedIndices; - private final @Nullable TimeValue timeout; + private final CloseIndexClusterStateUpdateRequest request; private final ActionListener> listener; private WaitForClosedBlocksApplied(final Set blockedIndices, - final @Nullable TimeValue timeout, + final CloseIndexClusterStateUpdateRequest request, final ActionListener> listener) { if (blockedIndices == null || blockedIndices.isEmpty()) { throw new IllegalArgumentException("Cannot wait for closed block to be applied to null or empty list of blocked indices"); } this.blockedIndices = blockedIndices; + this.request = request; this.listener = listener; - this.timeout = timeout; } @Override @@ -271,7 +268,7 @@ protected void doRun() throws Exception { final CountDown countDown = new CountDown(blockedIndices.size()); final ClusterState state = clusterService.state(); for (Index blockedIndex : blockedIndices) { - waitForShardsReadyForClosing(blockedIndex, state, timeout, response -> { + waitForShardsReadyForClosing(blockedIndex, state, response -> { results.put(blockedIndex, response); if (countDown.countDown()) { listener.onResponse(unmodifiableMap(results)); @@ -280,7 +277,7 @@ protected void doRun() throws Exception { } } - private void waitForShardsReadyForClosing(final Index index, final ClusterState state, @Nullable final TimeValue timeout, + private void waitForShardsReadyForClosing(final Index index, final ClusterState state, final Consumer onResponse) { final IndexMetaData indexMetaData = state.metaData().index(index); if (indexMetaData == null) { @@ -302,7 +299,7 @@ private void waitForShardsReadyForClosing(final Index index, final ClusterState for (IntObjectCursor shard : shards) { final IndexShardRoutingTable shardRoutingTable = shard.value; final ShardId shardId = shardRoutingTable.shardId(); - sendVerifyShardBeforeCloseRequest(shardRoutingTable, timeout, new NotifyOnceListener() { + sendVerifyShardBeforeCloseRequest(shardRoutingTable, new NotifyOnceListener() { @Override public void innerOnResponse(final ReplicationResponse replicationResponse) { ReplicationResponse.ShardInfo shardInfo = replicationResponse.getShardInfo(); @@ -326,7 +323,7 @@ private void processIfFinished() { } } - private void sendVerifyShardBeforeCloseRequest(final IndexShardRoutingTable shardRoutingTable, @Nullable final TimeValue timeout, + private void sendVerifyShardBeforeCloseRequest(final IndexShardRoutingTable shardRoutingTable, final ActionListener listener) { final ShardId shardId = shardRoutingTable.shardId(); if (shardRoutingTable.primaryShard().unassigned()) { @@ -336,10 +333,11 @@ private void sendVerifyShardBeforeCloseRequest(final IndexShardRoutingTable shar listener.onResponse(response); return; } + final TaskId parentTaskId = new TaskId(clusterService.localNode().getId(), request.taskId()); final TransportVerifyShardBeforeCloseAction.ShardRequest shardRequest = - new TransportVerifyShardBeforeCloseAction.ShardRequest(shardId); - if (timeout != null) { - shardRequest.timeout(timeout); + new TransportVerifyShardBeforeCloseAction.ShardRequest(shardId, parentTaskId); + if (request.ackTimeout() != null) { + shardRequest.timeout(request.ackTimeout()); } // TODO propagate a task id from the parent CloseIndexRequest to the ShardCloseRequests transportVerifyShardBeforeCloseAction.execute(shardRequest, listener); diff --git a/server/src/test/java/org/elasticsearch/action/admin/indices/close/TransportVerifyShardBeforeCloseActionTests.java b/server/src/test/java/org/elasticsearch/action/admin/indices/close/TransportVerifyShardBeforeCloseActionTests.java index 2764eee798e6b..c0da96ed1efb7 100644 --- a/server/src/test/java/org/elasticsearch/action/admin/indices/close/TransportVerifyShardBeforeCloseActionTests.java +++ b/server/src/test/java/org/elasticsearch/action/admin/indices/close/TransportVerifyShardBeforeCloseActionTests.java @@ -43,6 +43,7 @@ import org.elasticsearch.index.shard.ReplicationGroup; import org.elasticsearch.index.shard.ShardId; import org.elasticsearch.indices.IndicesService; +import org.elasticsearch.tasks.TaskId; import org.elasticsearch.test.ESTestCase; import org.elasticsearch.test.transport.CapturingTransport; import org.elasticsearch.threadpool.TestThreadPool; @@ -130,7 +131,7 @@ public static void afterClass() { private void executeOnPrimaryOrReplica() throws Exception { final TransportVerifyShardBeforeCloseAction.ShardRequest request = - new TransportVerifyShardBeforeCloseAction.ShardRequest(indexShard.shardId()); + new TransportVerifyShardBeforeCloseAction.ShardRequest(indexShard.shardId(), new TaskId("_node_id", randomNonNegativeLong())); if (randomBoolean()) { assertNotNull(action.shardOperationOnPrimary(request, indexShard)); } else { @@ -204,7 +205,8 @@ public void testUnavailableShardsMarkedAsStale() throws Exception { assertThat(replicationGroup.getUnavailableInSyncShards().size(), greaterThan(0)); final PlainActionFuture listener = new PlainActionFuture<>(); - TransportVerifyShardBeforeCloseAction.ShardRequest request = new TransportVerifyShardBeforeCloseAction.ShardRequest(shardId); + TransportVerifyShardBeforeCloseAction.ShardRequest request = + new TransportVerifyShardBeforeCloseAction.ShardRequest(shardId, new TaskId(clusterService.localNode().getId(), 0L)); ReplicationOperation.Replicas proxy = action.newReplicasProxy(primaryTerm); ReplicationOperation operation = diff --git a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/action/TransportFreezeIndexAction.java b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/action/TransportFreezeIndexAction.java index 36cce46d47c46..3031ec5b2a409 100644 --- a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/action/TransportFreezeIndexAction.java +++ b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/action/TransportFreezeIndexAction.java @@ -109,13 +109,19 @@ private Index[] resolveIndices(FreezeRequest request, ClusterState state) { @Override protected void masterOperation(FreezeRequest request, ClusterState state, ActionListener listener) { + throw new UnsupportedOperationException("The task parameter is required"); + } + + @Override + protected void masterOperation(Task task, TransportFreezeIndexAction.FreezeRequest request, ClusterState state, + ActionListener listener) throws Exception { final Index[] concreteIndices = resolveIndices(request, state); if (concreteIndices.length == 0) { listener.onResponse(new FreezeResponse(true, true)); return; } - final CloseIndexClusterStateUpdateRequest closeRequest = new CloseIndexClusterStateUpdateRequest() + final CloseIndexClusterStateUpdateRequest closeRequest = new CloseIndexClusterStateUpdateRequest(task.getId()) .ackTimeout(request.timeout()) .masterNodeTimeout(request.masterNodeTimeout()) .indices(concreteIndices); From e149b0852e206574faad54ab08f8a61827f73e8d Mon Sep 17 00:00:00 2001 From: Tanguy Leroux Date: Mon, 7 Jan 2019 16:44:59 +0100 Subject: [PATCH 07/46] [Close Index API] Add unique UUID to ClusterBlock (#36775) This commit adds a unique id to cluster blocks, so that they can be uniquely identified if needed. This is important for the Close Index API where multiple concurrent closing requests can be executed at the same time. By adding a UUID to the cluster block, we can generate unique "closing block" that can later be verified on shards and then checked again from the cluster state before closing the index. When the verification on shard is done, the closing block is replaced by the regular INDEX_CLOSED_BLOCK instance. If something goes wrong, calling the Open Index API will remove the block. Related to #33888 --- .../test/rest/WaitForRefreshAndCloseIT.java | 2 - ...TransportVerifyShardBeforeCloseAction.java | 39 ++- .../cluster/block/ClusterBlock.java | 61 +++-- .../cluster/block/ClusterBlocks.java | 38 +++ .../metadata/MetaDataIndexStateService.java | 237 +++++++++++------- ...portVerifyShardBeforeCloseActionTests.java | 16 +- .../cluster/block/ClusterBlockTests.java | 161 +++++++++--- .../MetaDataIndexStateServiceTests.java | 133 ++++++---- .../MetaDataIndexStateServiceUtils.java | 17 +- .../indices/cluster/ClusterStateChanges.java | 10 +- .../indices/state/CloseIndexIT.java | 93 ++++++- .../indices/state/OpenCloseIndexIT.java | 21 +- .../indices/state/ReopenWhileClosingIT.java | 167 ++++++++++++ .../DedicatedClusterSnapshotRestoreIT.java | 2 +- .../SharedClusterSnapshotRestoreIT.java | 4 +- .../hamcrest/ElasticsearchAssertions.java | 22 +- 16 files changed, 773 insertions(+), 250 deletions(-) create mode 100644 server/src/test/java/org/elasticsearch/indices/state/ReopenWhileClosingIT.java diff --git a/distribution/archives/integ-test-zip/src/test/java/org/elasticsearch/test/rest/WaitForRefreshAndCloseIT.java b/distribution/archives/integ-test-zip/src/test/java/org/elasticsearch/test/rest/WaitForRefreshAndCloseIT.java index 2292def6d4a3b..52b1a8c52b58e 100644 --- a/distribution/archives/integ-test-zip/src/test/java/org/elasticsearch/test/rest/WaitForRefreshAndCloseIT.java +++ b/distribution/archives/integ-test-zip/src/test/java/org/elasticsearch/test/rest/WaitForRefreshAndCloseIT.java @@ -20,7 +20,6 @@ package org.elasticsearch.test.rest; import org.apache.http.util.EntityUtils; -import org.apache.lucene.util.LuceneTestCase; import org.elasticsearch.action.ActionFuture; import org.elasticsearch.action.support.PlainActionFuture; import org.elasticsearch.client.Request; @@ -41,7 +40,6 @@ /** * Tests that wait for refresh is fired if the index is closed. */ -@LuceneTestCase.AwaitsFix(bugUrl = "to be created") public class WaitForRefreshAndCloseIT extends ESRestTestCase { @Before public void setupIndex() throws IOException { diff --git a/server/src/main/java/org/elasticsearch/action/admin/indices/close/TransportVerifyShardBeforeCloseAction.java b/server/src/main/java/org/elasticsearch/action/admin/indices/close/TransportVerifyShardBeforeCloseAction.java index f603f92a7189e..f08f6ea7dffa2 100644 --- a/server/src/main/java/org/elasticsearch/action/admin/indices/close/TransportVerifyShardBeforeCloseAction.java +++ b/server/src/main/java/org/elasticsearch/action/admin/indices/close/TransportVerifyShardBeforeCloseAction.java @@ -29,9 +29,10 @@ import org.elasticsearch.cluster.block.ClusterBlock; import org.elasticsearch.cluster.block.ClusterBlocks; import org.elasticsearch.cluster.metadata.IndexNameExpressionResolver; -import org.elasticsearch.cluster.metadata.MetaDataIndexStateService; import org.elasticsearch.cluster.service.ClusterService; import org.elasticsearch.common.inject.Inject; +import org.elasticsearch.common.io.stream.StreamInput; +import org.elasticsearch.common.io.stream.StreamOutput; import org.elasticsearch.common.lease.Releasable; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.index.shard.IndexShard; @@ -41,13 +42,14 @@ import org.elasticsearch.threadpool.ThreadPool; import org.elasticsearch.transport.TransportService; +import java.io.IOException; +import java.util.Objects; import java.util.function.Consumer; public class TransportVerifyShardBeforeCloseAction extends TransportReplicationAction< TransportVerifyShardBeforeCloseAction.ShardRequest, TransportVerifyShardBeforeCloseAction.ShardRequest, ReplicationResponse> { public static final String NAME = CloseIndexAction.NAME + "[s]"; - public static final ClusterBlock EXPECTED_BLOCK = MetaDataIndexStateService.INDEX_CLOSED_BLOCK; @Inject public TransportVerifyShardBeforeCloseAction(final Settings settings, final TransportService transportService, @@ -83,25 +85,25 @@ protected void acquireReplicaOperationPermit(final IndexShard replica, @Override protected PrimaryResult shardOperationOnPrimary(final ShardRequest shardRequest, final IndexShard primary) throws Exception { - executeShardOperation(primary); + executeShardOperation(shardRequest, primary); return new PrimaryResult<>(shardRequest, new ReplicationResponse()); } @Override protected ReplicaResult shardOperationOnReplica(final ShardRequest shardRequest, final IndexShard replica) throws Exception { - executeShardOperation(replica); + executeShardOperation(shardRequest, replica); return new ReplicaResult(); } - private void executeShardOperation(final IndexShard indexShard) { + private void executeShardOperation(final ShardRequest request, final IndexShard indexShard) { final ShardId shardId = indexShard.shardId(); if (indexShard.getActiveOperationsCount() != 0) { throw new IllegalStateException("On-going operations in progress while checking index shard " + shardId + " before closing"); } final ClusterBlocks clusterBlocks = clusterService.state().blocks(); - if (clusterBlocks.hasIndexBlock(shardId.getIndexName(), EXPECTED_BLOCK) == false) { - throw new IllegalStateException("Index shard " + shardId + " must be blocked by " + EXPECTED_BLOCK + " before closing"); + if (clusterBlocks.hasIndexBlock(shardId.getIndexName(), request.clusterBlock()) == false) { + throw new IllegalStateException("Index shard " + shardId + " must be blocked by " + request.clusterBlock() + " before closing"); } final long maxSeqNo = indexShard.seqNoStats().getMaxSeqNo(); @@ -139,17 +141,36 @@ public void markShardCopyAsStaleIfNeeded(final ShardId shardId, final String all public static class ShardRequest extends ReplicationRequest { + private ClusterBlock clusterBlock; + ShardRequest(){ } - public ShardRequest(final ShardId shardId, final TaskId parentTaskId) { + public ShardRequest(final ShardId shardId, final ClusterBlock clusterBlock, final TaskId parentTaskId) { super(shardId); + this.clusterBlock = Objects.requireNonNull(clusterBlock); setParentTask(parentTaskId); } @Override public String toString() { - return "verify shard before close {" + shardId + "}"; + return "verify shard " + shardId + " before close with block " + clusterBlock; + } + + @Override + public void readFrom(final StreamInput in) throws IOException { + super.readFrom(in); + clusterBlock = ClusterBlock.readClusterBlock(in); + } + + @Override + public void writeTo(final StreamOutput out) throws IOException { + super.writeTo(out); + clusterBlock.writeTo(out); + } + + public ClusterBlock clusterBlock() { + return clusterBlock; } } } diff --git a/server/src/main/java/org/elasticsearch/cluster/block/ClusterBlock.java b/server/src/main/java/org/elasticsearch/cluster/block/ClusterBlock.java index fafd397722025..5713462b9212f 100644 --- a/server/src/main/java/org/elasticsearch/cluster/block/ClusterBlock.java +++ b/server/src/main/java/org/elasticsearch/cluster/block/ClusterBlock.java @@ -19,6 +19,8 @@ package org.elasticsearch.cluster.block; +import org.elasticsearch.Version; +import org.elasticsearch.common.Nullable; import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.io.stream.StreamOutput; import org.elasticsearch.common.io.stream.Streamable; @@ -30,29 +32,31 @@ import java.util.ArrayList; import java.util.EnumSet; import java.util.Locale; +import java.util.Objects; public class ClusterBlock implements Streamable, ToXContentFragment { private int id; - + private @Nullable String uuid; private String description; - private EnumSet levels; - private boolean retryable; - private boolean disableStatePersistence = false; - private boolean allowReleaseResources; - private RestStatus status; - ClusterBlock() { + private ClusterBlock() { + } + + public ClusterBlock(int id, String description, boolean retryable, boolean disableStatePersistence, + boolean allowReleaseResources, RestStatus status, EnumSet levels) { + this(id, null, description, retryable, disableStatePersistence, allowReleaseResources, status, levels); } - public ClusterBlock(int id, String description, boolean retryable, boolean disableStatePersistence, boolean allowReleaseResources, - RestStatus status, EnumSet levels) { + public ClusterBlock(int id, String uuid, String description, boolean retryable, boolean disableStatePersistence, + boolean allowReleaseResources, RestStatus status, EnumSet levels) { this.id = id; + this.uuid = uuid; this.description = description; this.retryable = retryable; this.disableStatePersistence = disableStatePersistence; @@ -65,6 +69,10 @@ public int id() { return this.id; } + public String uuid() { + return uuid; + } + public String description() { return this.description; } @@ -104,6 +112,9 @@ public boolean disableStatePersistence() { @Override public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException { builder.startObject(Integer.toString(id)); + if (uuid != null) { + builder.field("uuid", uuid); + } builder.field("description", description); builder.field("retryable", retryable); if (disableStatePersistence) { @@ -127,6 +138,11 @@ public static ClusterBlock readClusterBlock(StreamInput in) throws IOException { @Override public void readFrom(StreamInput in) throws IOException { id = in.readVInt(); + if (in.getVersion().onOrAfter(Version.V_7_0_0)) { + uuid = in.readOptionalString(); + } else { + uuid = null; + } description = in.readString(); final int len = in.readVInt(); ArrayList levels = new ArrayList<>(len); @@ -143,6 +159,9 @@ public void readFrom(StreamInput in) throws IOException { @Override public void writeTo(StreamOutput out) throws IOException { out.writeVInt(id); + if (out.getVersion().onOrAfter(Version.V_7_0_0)) { + out.writeOptionalString(uuid); + } out.writeString(description); out.writeVInt(levels.size()); for (ClusterBlockLevel level : levels) { @@ -157,7 +176,11 @@ public void writeTo(StreamOutput out) throws IOException { @Override public String toString() { StringBuilder sb = new StringBuilder(); - sb.append(id).append(",").append(description).append(", blocks "); + sb.append(id).append(","); + if (uuid != null) { + sb.append(uuid).append(','); + } + sb.append(description).append(", blocks "); String delimiter = ""; for (ClusterBlockLevel level : levels) { sb.append(delimiter).append(level.name()); @@ -168,19 +191,19 @@ public String toString() { @Override public boolean equals(Object o) { - if (this == o) return true; - if (o == null || getClass() != o.getClass()) return false; - - ClusterBlock that = (ClusterBlock) o; - - if (id != that.id) return false; - - return true; + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + final ClusterBlock that = (ClusterBlock) o; + return id == that.id && Objects.equals(uuid, that.uuid); } @Override public int hashCode() { - return id; + return Objects.hash(id, uuid); } public boolean isAllowReleaseResources() { diff --git a/server/src/main/java/org/elasticsearch/cluster/block/ClusterBlocks.java b/server/src/main/java/org/elasticsearch/cluster/block/ClusterBlocks.java index 941fee6da7d0d..0de7bce115943 100644 --- a/server/src/main/java/org/elasticsearch/cluster/block/ClusterBlocks.java +++ b/server/src/main/java/org/elasticsearch/cluster/block/ClusterBlocks.java @@ -24,6 +24,7 @@ import org.elasticsearch.cluster.Diff; import org.elasticsearch.cluster.metadata.IndexMetaData; import org.elasticsearch.cluster.metadata.MetaDataIndexStateService; +import org.elasticsearch.common.Nullable; import org.elasticsearch.common.collect.ImmutableOpenMap; import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.io.stream.StreamOutput; @@ -147,6 +148,31 @@ public boolean hasIndexBlock(String index, ClusterBlock block) { return indicesBlocks.containsKey(index) && indicesBlocks.get(index).contains(block); } + public boolean hasIndexBlockWithId(String index, int blockId) { + final Set clusterBlocks = indicesBlocks.get(index); + if (clusterBlocks != null) { + for (ClusterBlock clusterBlock : clusterBlocks) { + if (clusterBlock.id() == blockId) { + return true; + } + } + } + return false; + } + + @Nullable + public ClusterBlock getIndexBlockWithId(final String index, final int blockId) { + final Set clusterBlocks = indicesBlocks.get(index); + if (clusterBlocks != null) { + for (ClusterBlock clusterBlock : clusterBlocks) { + if (clusterBlock.id() == blockId) { + return clusterBlock; + } + } + } + return null; + } + public void globalBlockedRaiseException(ClusterBlockLevel level) throws ClusterBlockException { ClusterBlockException blockException = globalBlockedException(level); if (blockException != null) { @@ -403,6 +429,18 @@ public Builder removeIndexBlock(String index, ClusterBlock block) { return this; } + public Builder removeIndexBlockWithId(String index, int blockId) { + final Set indexBlocks = indices.get(index); + if (indexBlocks == null) { + return this; + } + indexBlocks.removeIf(block -> block.id() == blockId); + if (indexBlocks.isEmpty()) { + indices.remove(index); + } + return this; + } + public ClusterBlocks build() { // We copy the block sets here in case of the builder is modified after build is called ImmutableOpenMap.Builder> indicesBuilder = ImmutableOpenMap.builder(indices.size()); diff --git a/server/src/main/java/org/elasticsearch/cluster/metadata/MetaDataIndexStateService.java b/server/src/main/java/org/elasticsearch/cluster/metadata/MetaDataIndexStateService.java index 6ceda4bf57d13..aa4434a0a74c6 100644 --- a/server/src/main/java/org/elasticsearch/cluster/metadata/MetaDataIndexStateService.java +++ b/server/src/main/java/org/elasticsearch/cluster/metadata/MetaDataIndexStateService.java @@ -47,6 +47,8 @@ import org.elasticsearch.cluster.routing.allocation.AllocationService; import org.elasticsearch.cluster.service.ClusterService; import org.elasticsearch.common.Priority; +import org.elasticsearch.common.Strings; +import org.elasticsearch.common.UUIDs; import org.elasticsearch.common.ValidationException; import org.elasticsearch.common.collect.ImmutableOpenIntMap; import org.elasticsearch.common.inject.Inject; @@ -67,6 +69,8 @@ import java.util.ArrayList; import java.util.Arrays; +import java.util.EnumSet; +import java.util.HashMap; import java.util.HashSet; import java.util.List; import java.util.Map; @@ -83,6 +87,7 @@ public class MetaDataIndexStateService { private static final Logger logger = LogManager.getLogger(MetaDataIndexStateService.class); + public static final int INDEX_CLOSED_BLOCK_ID = 4; public static final ClusterBlock INDEX_CLOSED_BLOCK = new ClusterBlock(4, "index closed", false, false, false, RestStatus.FORBIDDEN, ClusterBlockLevel.READ_WRITE); @@ -123,11 +128,11 @@ public void closeIndices(final CloseIndexClusterStateUpdateRequest request, fina clusterService.submitStateUpdateTask("add-block-index-to-close " + Arrays.toString(concreteIndices), new ClusterStateUpdateTask(Priority.URGENT) { - private final Set blockedIndices = new HashSet<>(); + private final Map blockedIndices = new HashMap<>(); @Override public ClusterState execute(final ClusterState currentState) { - return addIndexClosedBlocks(concreteIndices, currentState, blockedIndices); + return addIndexClosedBlocks(concreteIndices, blockedIndices, currentState); } @Override @@ -139,11 +144,21 @@ public void clusterStateProcessed(final String source, final ClusterState oldSta assert blockedIndices.isEmpty() == false : "List of blocked indices is empty but cluster state was changed"; threadPool.executor(ThreadPool.Names.MANAGEMENT) .execute(new WaitForClosedBlocksApplied(blockedIndices, request, - ActionListener.wrap(closedBlocksResults -> + ActionListener.wrap(results -> clusterService.submitStateUpdateTask("close-indices", new ClusterStateUpdateTask(Priority.URGENT) { + + boolean acknowledged = true; + @Override public ClusterState execute(final ClusterState currentState) throws Exception { - final ClusterState updatedState = closeRoutingTable(currentState, closedBlocksResults); + final ClusterState updatedState = closeRoutingTable(currentState, blockedIndices, results); + for (Map.Entry result : results.entrySet()) { + IndexMetaData updatedMetaData = updatedState.metaData().index(result.getKey()); + if (updatedMetaData != null && updatedMetaData.getState() != IndexMetaData.State.CLOSE) { + acknowledged = false; + break; + } + } return allocationService.reroute(updatedState, "indices closed"); } @@ -155,8 +170,6 @@ public void onFailure(final String source, final Exception e) { @Override public void clusterStateProcessed(final String source, final ClusterState oldState, final ClusterState newState) { - boolean acknowledged = closedBlocksResults.values().stream() - .allMatch(AcknowledgedResponse::isAcknowledged); listener.onResponse(new AcknowledgedResponse(acknowledged)); } }), @@ -182,11 +195,12 @@ public TimeValue timeout() { /** * Step 1 - Start closing indices by adding a write block * - * This step builds the list of indices to close (the ones explicitly requested that are not in CLOSE state) and adds the index block - * {@link #INDEX_CLOSED_BLOCK} to every index to close in the cluster state. After the cluster state is published, the shards should - * start to reject writing operations and we can proceed with step 2. + * This step builds the list of indices to close (the ones explicitly requested that are not in CLOSE state) and adds a unique cluster + * block (or reuses an existing one) to every index to close in the cluster state. After the cluster state is published, the shards + * should start to reject writing operations and we can proceed with step 2. */ - static ClusterState addIndexClosedBlocks(final Index[] indices, final ClusterState currentState, final Set blockedIndices) { + static ClusterState addIndexClosedBlocks(final Index[] indices, final Map blockedIndices, + final ClusterState currentState) { final MetaData.Builder metadata = MetaData.builder(currentState.metaData()); final Set indicesToClose = new HashSet<>(); @@ -196,6 +210,7 @@ static ClusterState addIndexClosedBlocks(final Index[] indices, final ClusterSta indicesToClose.add(indexMetaData); } else { logger.debug("index {} is already closed, ignoring", index); + assert currentState.blocks().hasIndexBlock(index.getName(), INDEX_CLOSED_BLOCK); } } @@ -217,19 +232,37 @@ static ClusterState addIndexClosedBlocks(final Index[] indices, final ClusterSta for (IndexMetaData indexToClose : indicesToClose) { final Index index = indexToClose.getIndex(); - if (currentState.blocks().hasIndexBlock(index.getName(), INDEX_CLOSED_BLOCK) == false) { - blocks.addIndexBlock(index.getName(), INDEX_CLOSED_BLOCK); + + ClusterBlock indexBlock = null; + final Set clusterBlocks = currentState.blocks().indices().get(index.getName()); + if (clusterBlocks != null) { + for (ClusterBlock clusterBlock : clusterBlocks) { + if (clusterBlock.id() == INDEX_CLOSED_BLOCK_ID) { + // Reuse the existing index closed block + indexBlock = clusterBlock; + break; + } + } } if (useDirectClose) { logger.debug("closing index {} directly", index); - metadata.put(IndexMetaData.builder(indexToClose).state(IndexMetaData.State.CLOSE)); + metadata.put(IndexMetaData.builder(indexToClose).state(IndexMetaData.State.CLOSE)); // increment version? + blocks.removeIndexBlockWithId(index.getName(), INDEX_CLOSED_BLOCK_ID); routingTable.remove(index.getName()); + indexBlock = INDEX_CLOSED_BLOCK; + } else { + if (indexBlock == null) { + // Create a new index closed block + indexBlock = createIndexClosingBlock(); + } + assert Strings.hasLength(indexBlock.uuid()) : "Closing block should have a UUID"; } - blockedIndices.add(index); + blocks.addIndexBlock(index.getName(), indexBlock); + blockedIndices.put(index, indexBlock); } logger.info(() -> new ParameterizedMessage("closing indices {}", - blockedIndices.stream().map(Object::toString).collect(Collectors.joining(",")))); + blockedIndices.keySet().stream().map(Object::toString).collect(Collectors.joining(",")))); return ClusterState.builder(currentState).blocks(blocks).metaData(metadata).routingTable(routingTable.build()).build(); } @@ -242,15 +275,15 @@ static ClusterState addIndexClosedBlocks(final Index[] indices, final ClusterSta */ class WaitForClosedBlocksApplied extends AbstractRunnable { - private final Set blockedIndices; + private final Map blockedIndices; private final CloseIndexClusterStateUpdateRequest request; private final ActionListener> listener; - private WaitForClosedBlocksApplied(final Set blockedIndices, + private WaitForClosedBlocksApplied(final Map blockedIndices, final CloseIndexClusterStateUpdateRequest request, final ActionListener> listener) { if (blockedIndices == null || blockedIndices.isEmpty()) { - throw new IllegalArgumentException("Cannot wait for closed block to be applied to null or empty list of blocked indices"); + throw new IllegalArgumentException("Cannot wait for closed blocks to be applied, list of blocked indices is empty or null"); } this.blockedIndices = blockedIndices; this.request = request; @@ -267,18 +300,18 @@ protected void doRun() throws Exception { final Map results = ConcurrentCollections.newConcurrentMap(); final CountDown countDown = new CountDown(blockedIndices.size()); final ClusterState state = clusterService.state(); - for (Index blockedIndex : blockedIndices) { - waitForShardsReadyForClosing(blockedIndex, state, response -> { - results.put(blockedIndex, response); + blockedIndices.forEach((index, block) -> { + waitForShardsReadyForClosing(index, block, state, response -> { + results.put(index, response); if (countDown.countDown()) { listener.onResponse(unmodifiableMap(results)); } }); - } + }); } - private void waitForShardsReadyForClosing(final Index index, final ClusterState state, - final Consumer onResponse) { + private void waitForShardsReadyForClosing(final Index index, final ClusterBlock closingBlock, + final ClusterState state, final Consumer onResponse) { final IndexMetaData indexMetaData = state.metaData().index(index); if (indexMetaData == null) { logger.debug("index {} has been blocked before closing and is now deleted, ignoring", index); @@ -287,6 +320,7 @@ private void waitForShardsReadyForClosing(final Index index, final ClusterState } final IndexRoutingTable indexRoutingTable = state.routingTable().index(index); if (indexRoutingTable == null || indexMetaData.getState() == IndexMetaData.State.CLOSE) { + assert state.blocks().hasIndexBlock(index.getName(), INDEX_CLOSED_BLOCK); logger.debug("index {} has been blocked before closing and is already closed, ignoring", index); onResponse.accept(new AcknowledgedResponse(true)); return; @@ -299,7 +333,7 @@ private void waitForShardsReadyForClosing(final Index index, final ClusterState for (IntObjectCursor shard : shards) { final IndexShardRoutingTable shardRoutingTable = shard.value; final ShardId shardId = shardRoutingTable.shardId(); - sendVerifyShardBeforeCloseRequest(shardRoutingTable, new NotifyOnceListener() { + sendVerifyShardBeforeCloseRequest(shardRoutingTable, closingBlock, new NotifyOnceListener() { @Override public void innerOnResponse(final ReplicationResponse replicationResponse) { ReplicationResponse.ShardInfo shardInfo = replicationResponse.getShardInfo(); @@ -324,6 +358,7 @@ private void processIfFinished() { } private void sendVerifyShardBeforeCloseRequest(final IndexShardRoutingTable shardRoutingTable, + final ClusterBlock closingBlock, final ActionListener listener) { final ShardId shardId = shardRoutingTable.shardId(); if (shardRoutingTable.primaryShard().unassigned()) { @@ -335,11 +370,10 @@ private void sendVerifyShardBeforeCloseRequest(final IndexShardRoutingTable shar } final TaskId parentTaskId = new TaskId(clusterService.localNode().getId(), request.taskId()); final TransportVerifyShardBeforeCloseAction.ShardRequest shardRequest = - new TransportVerifyShardBeforeCloseAction.ShardRequest(shardId, parentTaskId); + new TransportVerifyShardBeforeCloseAction.ShardRequest(shardId, closingBlock, parentTaskId); if (request.ackTimeout() != null) { shardRequest.timeout(request.ackTimeout()); } - // TODO propagate a task id from the parent CloseIndexRequest to the ShardCloseRequests transportVerifyShardBeforeCloseAction.execute(shardRequest, listener); } } @@ -347,7 +381,9 @@ private void sendVerifyShardBeforeCloseRequest(final IndexShardRoutingTable shar /** * Step 3 - Move index states from OPEN to CLOSE in cluster state for indices that are ready for closing. */ - static ClusterState closeRoutingTable(final ClusterState currentState, final Map results) { + static ClusterState closeRoutingTable(final ClusterState currentState, + final Map blockedIndices, + final Map results) { final MetaData.Builder metadata = MetaData.builder(currentState.metaData()); final ClusterBlocks.Builder blocks = ClusterBlocks.builder().blocks(currentState.blocks()); final RoutingTable.Builder routingTable = RoutingTable.builder(currentState.routingTable()); @@ -355,21 +391,29 @@ static ClusterState closeRoutingTable(final ClusterState currentState, final Map final Set closedIndices = new HashSet<>(); for (Map.Entry result : results.entrySet()) { final Index index = result.getKey(); + final boolean acknowledged = result.getValue().isAcknowledged(); try { + if (acknowledged == false) { + logger.debug("verification of shards before closing {} failed", index); + continue; + } final IndexMetaData indexMetaData = metadata.getSafe(index); - if (indexMetaData.getState() != IndexMetaData.State.CLOSE) { - if (result.getValue().isAcknowledged()) { - logger.debug("closing index {} succeed, removing index routing table", index); - metadata.put(IndexMetaData.builder(indexMetaData).state(IndexMetaData.State.CLOSE)); - routingTable.remove(index.getName()); - closedIndices.add(index.getName()); - } else { - logger.debug("closing index {} failed, removing index block because: {}", index, result.getValue()); - blocks.removeIndexBlock(index.getName(), INDEX_CLOSED_BLOCK); - } - } else { - logger.debug("index {} has been closed since it was blocked before closing, ignoring", index); + if (indexMetaData.getState() == IndexMetaData.State.CLOSE) { + logger.debug("verification of shards before closing {} succeeded but index is already closed", index); + assert currentState.blocks().hasIndexBlock(index.getName(), INDEX_CLOSED_BLOCK); + continue; + } + final ClusterBlock closingBlock = blockedIndices.get(index); + if (currentState.blocks().hasIndexBlock(index.getName(), closingBlock) == false) { + logger.debug("verification of shards before closing {} succeeded but block has been removed in the meantime", index); + continue; } + + logger.debug("closing index {} succeeded", index); + blocks.removeIndexBlockWithId(index.getName(), INDEX_CLOSED_BLOCK_ID).addIndexBlock(index.getName(), INDEX_CLOSED_BLOCK); + metadata.put(IndexMetaData.builder(indexMetaData).state(IndexMetaData.State.CLOSE)); + routingTable.remove(index.getName()); + closedIndices.add(index.getName()); } catch (final IndexNotFoundException e) { logger.debug("index {} has been deleted since it was blocked before closing, ignoring", index); } @@ -405,64 +449,73 @@ private void onlyOpenIndex(final OpenIndexClusterStateUpdateRequest request, final String indicesAsString = Arrays.toString(request.indices()); clusterService.submitStateUpdateTask("open-indices " + indicesAsString, - new AckedClusterStateUpdateTask(Priority.URGENT, request, listener) { - @Override - protected ClusterStateUpdateResponse newResponse(boolean acknowledged) { - return new ClusterStateUpdateResponse(acknowledged); - } + new AckedClusterStateUpdateTask(Priority.URGENT, request, listener) { + @Override + protected ClusterStateUpdateResponse newResponse(boolean acknowledged) { + return new ClusterStateUpdateResponse(acknowledged); + } - @Override - public ClusterState execute(ClusterState currentState) { - List indicesToOpen = new ArrayList<>(); - for (Index index : request.indices()) { - final IndexMetaData indexMetaData = currentState.metaData().getIndexSafe(index); - if (indexMetaData.getState() != IndexMetaData.State.OPEN) { - indicesToOpen.add(indexMetaData); - } + @Override + public ClusterState execute(final ClusterState currentState) { + final ClusterState updatedState = openIndices(request.indices(), currentState); + //no explicit wait for other nodes needed as we use AckedClusterStateUpdateTask + return allocationService.reroute(updatedState, "indices opened [" + indicesAsString + "]"); } + } + ); + } - validateShardLimit(currentState, request.indices()); + ClusterState openIndices(final Index[] indices, final ClusterState currentState) { + final List indicesToOpen = new ArrayList<>(); + for (Index index : indices) { + final IndexMetaData indexMetaData = currentState.metaData().getIndexSafe(index); + if (indexMetaData.getState() != IndexMetaData.State.OPEN) { + indicesToOpen.add(indexMetaData); + } else if (currentState.blocks().hasIndexBlockWithId(index.getName(), INDEX_CLOSED_BLOCK_ID)) { + indicesToOpen.add(indexMetaData); + } + } - if (indicesToOpen.isEmpty()) { - return currentState; - } + validateShardLimit(currentState, indices); + if (indicesToOpen.isEmpty()) { + return currentState; + } - logger.info("opening indices [{}]", indicesAsString); - - MetaData.Builder mdBuilder = MetaData.builder(currentState.metaData()); - ClusterBlocks.Builder blocksBuilder = ClusterBlocks.builder() - .blocks(currentState.blocks()); - final Version minIndexCompatibilityVersion = currentState.getNodes().getMaxNodeVersion() - .minimumIndexCompatibilityVersion(); - for (IndexMetaData closedMetaData : indicesToOpen) { - final String indexName = closedMetaData.getIndex().getName(); - IndexMetaData indexMetaData = IndexMetaData.builder(closedMetaData).state(IndexMetaData.State.OPEN).build(); - // The index might be closed because we couldn't import it due to old incompatible version - // We need to check that this index can be upgraded to the current version - indexMetaData = metaDataIndexUpgradeService.upgradeIndexMetaData(indexMetaData, minIndexCompatibilityVersion); - try { - indicesService.verifyIndexMetadata(indexMetaData, indexMetaData); - } catch (Exception e) { - throw new ElasticsearchException("Failed to verify index " + indexMetaData.getIndex(), e); - } + logger.info(() -> new ParameterizedMessage("opening indices [{}]", + String.join(",", indicesToOpen.stream().map(i -> (CharSequence) i.getIndex().toString())::iterator))); - mdBuilder.put(indexMetaData, true); - blocksBuilder.removeIndexBlock(indexName, INDEX_CLOSED_BLOCK); + final MetaData.Builder metadata = MetaData.builder(currentState.metaData()); + final ClusterBlocks.Builder blocks = ClusterBlocks.builder().blocks(currentState.blocks()); + final Version minIndexCompatibilityVersion = currentState.getNodes().getMaxNodeVersion().minimumIndexCompatibilityVersion(); + + for (IndexMetaData indexMetaData : indicesToOpen) { + final Index index = indexMetaData.getIndex(); + if (indexMetaData.getState() != IndexMetaData.State.OPEN) { + IndexMetaData updatedIndexMetaData = IndexMetaData.builder(indexMetaData).state(IndexMetaData.State.OPEN).build(); + // The index might be closed because we couldn't import it due to old incompatible version + // We need to check that this index can be upgraded to the current version + updatedIndexMetaData = metaDataIndexUpgradeService.upgradeIndexMetaData(updatedIndexMetaData, minIndexCompatibilityVersion); + try { + indicesService.verifyIndexMetadata(updatedIndexMetaData, updatedIndexMetaData); + } catch (Exception e) { + throw new ElasticsearchException("Failed to verify index " + index, e); } + metadata.put(updatedIndexMetaData, true); + } - ClusterState updatedState = ClusterState.builder(currentState).metaData(mdBuilder).blocks(blocksBuilder).build(); + // Always removes index closed blocks (note: this can fail on-going close index actions) + blocks.removeIndexBlockWithId(index.getName(), INDEX_CLOSED_BLOCK_ID); + } - RoutingTable.Builder rtBuilder = RoutingTable.builder(updatedState.routingTable()); - for (IndexMetaData index : indicesToOpen) { - rtBuilder.addAsFromCloseToOpen(updatedState.metaData().getIndexSafe(index.getIndex())); - } + ClusterState updatedState = ClusterState.builder(currentState).metaData(metadata).blocks(blocks).build(); - //no explicit wait for other nodes needed as we use AckedClusterStateUpdateTask - return allocationService.reroute( - ClusterState.builder(updatedState).routingTable(rtBuilder.build()).build(), - "indices opened [" + indicesAsString + "]"); + final RoutingTable.Builder routingTable = RoutingTable.builder(updatedState.routingTable()); + for (IndexMetaData previousIndexMetaData : indicesToOpen) { + if (previousIndexMetaData.getState() != IndexMetaData.State.OPEN) { + routingTable.addAsFromCloseToOpen(updatedState.metaData().getIndexSafe(previousIndexMetaData.getIndex())); } - }); + } + return ClusterState.builder(updatedState).routingTable(routingTable.build()).build(); } /** @@ -492,4 +545,14 @@ private static int getTotalShardCount(ClusterState state, Index index) { return indexMetaData.getNumberOfShards() * (1 + indexMetaData.getNumberOfReplicas()); } + /** + * @return Generates a {@link ClusterBlock} that blocks read and write operations on soon-to-be-closed indices. The + * cluster block is generated with the id value equals to {@link #INDEX_CLOSED_BLOCK_ID} and a unique UUID. + */ + public static ClusterBlock createIndexClosingBlock() { + return new ClusterBlock(INDEX_CLOSED_BLOCK_ID, UUIDs.randomBase64UUID(), "index preparing to close. Reopen the index to allow " + + "writes again or retry closing the index to fully close the index.", false, false, false, RestStatus.FORBIDDEN, + EnumSet.of(ClusterBlockLevel.WRITE)); + } + } diff --git a/server/src/test/java/org/elasticsearch/action/admin/indices/close/TransportVerifyShardBeforeCloseActionTests.java b/server/src/test/java/org/elasticsearch/action/admin/indices/close/TransportVerifyShardBeforeCloseActionTests.java index c0da96ed1efb7..6fc744db2f3f3 100644 --- a/server/src/test/java/org/elasticsearch/action/admin/indices/close/TransportVerifyShardBeforeCloseActionTests.java +++ b/server/src/test/java/org/elasticsearch/action/admin/indices/close/TransportVerifyShardBeforeCloseActionTests.java @@ -29,9 +29,11 @@ import org.elasticsearch.cluster.ClusterName; import org.elasticsearch.cluster.ClusterState; import org.elasticsearch.cluster.action.shard.ShardStateAction; +import org.elasticsearch.cluster.block.ClusterBlock; import org.elasticsearch.cluster.block.ClusterBlocks; import org.elasticsearch.cluster.metadata.IndexMetaData; import org.elasticsearch.cluster.metadata.IndexNameExpressionResolver; +import org.elasticsearch.cluster.metadata.MetaDataIndexStateService; import org.elasticsearch.cluster.routing.IndexShardRoutingTable; import org.elasticsearch.cluster.routing.ShardRouting; import org.elasticsearch.cluster.routing.ShardRoutingState; @@ -61,7 +63,6 @@ import java.util.concurrent.TimeUnit; import static org.elasticsearch.action.support.replication.ClusterStateCreationUtils.state; -import static org.elasticsearch.cluster.metadata.MetaDataIndexStateService.INDEX_CLOSED_BLOCK; import static org.elasticsearch.test.ClusterServiceUtils.createClusterService; import static org.elasticsearch.test.ClusterServiceUtils.setState; import static org.hamcrest.Matchers.arrayWithSize; @@ -81,6 +82,7 @@ public class TransportVerifyShardBeforeCloseActionTests extends ESTestCase { private IndexShard indexShard; private TransportVerifyShardBeforeCloseAction action; private ClusterService clusterService; + private ClusterBlock clusterBlock; private CapturingTransport transport; @BeforeClass @@ -102,8 +104,10 @@ public void setUp() throws Exception { when(indexShard.shardId()).thenReturn(shardId); clusterService = createClusterService(threadPool); + + clusterBlock = MetaDataIndexStateService.createIndexClosingBlock(); setState(clusterService, new ClusterState.Builder(clusterService.state()) - .blocks(ClusterBlocks.builder().addIndexBlock("index", INDEX_CLOSED_BLOCK).build()).build()); + .blocks(ClusterBlocks.builder().blocks(clusterService.state().blocks()).addIndexBlock("index", clusterBlock).build()).build()); transport = new CapturingTransport(); TransportService transportService = transport.createTransportService(Settings.EMPTY, threadPool, @@ -130,8 +134,9 @@ public static void afterClass() { } private void executeOnPrimaryOrReplica() throws Exception { + final TaskId taskId = new TaskId("_node_id", randomNonNegativeLong()); final TransportVerifyShardBeforeCloseAction.ShardRequest request = - new TransportVerifyShardBeforeCloseAction.ShardRequest(indexShard.shardId(), new TaskId("_node_id", randomNonNegativeLong())); + new TransportVerifyShardBeforeCloseAction.ShardRequest(indexShard.shardId(), clusterBlock, taskId); if (randomBoolean()) { assertNotNull(action.shardOperationOnPrimary(request, indexShard)); } else { @@ -158,7 +163,7 @@ public void testOperationFailsWithNoBlock() { IllegalStateException exception = expectThrows(IllegalStateException.class, this::executeOnPrimaryOrReplica); assertThat(exception.getMessage(), - equalTo("Index shard " + indexShard.shardId() + " must be blocked by " + INDEX_CLOSED_BLOCK + " before closing")); + equalTo("Index shard " + indexShard.shardId() + " must be blocked by " + clusterBlock + " before closing")); verify(indexShard, times(0)).flush(any(FlushRequest.class)); } @@ -205,8 +210,9 @@ public void testUnavailableShardsMarkedAsStale() throws Exception { assertThat(replicationGroup.getUnavailableInSyncShards().size(), greaterThan(0)); final PlainActionFuture listener = new PlainActionFuture<>(); + TaskId taskId = new TaskId(clusterService.localNode().getId(), 0L); TransportVerifyShardBeforeCloseAction.ShardRequest request = - new TransportVerifyShardBeforeCloseAction.ShardRequest(shardId, new TaskId(clusterService.localNode().getId(), 0L)); + new TransportVerifyShardBeforeCloseAction.ShardRequest(shardId, clusterBlock, taskId); ReplicationOperation.Replicas proxy = action.newReplicasProxy(primaryTerm); ReplicationOperation operation = diff --git a/server/src/test/java/org/elasticsearch/cluster/block/ClusterBlockTests.java b/server/src/test/java/org/elasticsearch/cluster/block/ClusterBlockTests.java index a84d160cf0c95..4cf7b6f9c6d3a 100644 --- a/server/src/test/java/org/elasticsearch/cluster/block/ClusterBlockTests.java +++ b/server/src/test/java/org/elasticsearch/cluster/block/ClusterBlockTests.java @@ -20,36 +20,35 @@ package org.elasticsearch.cluster.block; import org.elasticsearch.Version; +import org.elasticsearch.common.UUIDs; import org.elasticsearch.common.collect.ImmutableOpenMap; import org.elasticsearch.common.io.stream.BytesStreamOutput; import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.rest.RestStatus; import org.elasticsearch.test.ESTestCase; +import java.util.Arrays; import java.util.Collections; -import java.util.EnumSet; +import java.util.List; +import static java.util.EnumSet.copyOf; +import static org.elasticsearch.test.VersionUtils.getPreviousVersion; import static org.elasticsearch.test.VersionUtils.randomVersion; +import static org.elasticsearch.test.VersionUtils.randomVersionBetween; import static org.hamcrest.CoreMatchers.endsWith; import static org.hamcrest.CoreMatchers.equalTo; import static org.hamcrest.CoreMatchers.not; +import static org.hamcrest.Matchers.is; +import static org.hamcrest.Matchers.isOneOf; +import static org.hamcrest.Matchers.nullValue; public class ClusterBlockTests extends ESTestCase { + public void testSerialization() throws Exception { - int iterations = randomIntBetween(10, 100); + int iterations = randomIntBetween(5, 20); for (int i = 0; i < iterations; i++) { - // Get a random version Version version = randomVersion(random()); - - // Get a random list of ClusterBlockLevels - EnumSet levels = EnumSet.noneOf(ClusterBlockLevel.class); - int nbLevels = randomIntBetween(1, ClusterBlockLevel.values().length); - for (int j = 0; j < nbLevels; j++) { - levels.add(randomFrom(ClusterBlockLevel.values())); - } - - ClusterBlock clusterBlock = new ClusterBlock(randomInt(), "cluster block #" + randomInt(), randomBoolean(), - randomBoolean(), false, randomFrom(RestStatus.values()), levels); + ClusterBlock clusterBlock = randomClusterBlock(version); BytesStreamOutput out = new BytesStreamOutput(); out.setVersion(version); @@ -59,37 +58,133 @@ public void testSerialization() throws Exception { in.setVersion(version); ClusterBlock result = ClusterBlock.readClusterBlock(in); - assertThat(result.id(), equalTo(clusterBlock.id())); - assertThat(result.status(), equalTo(clusterBlock.status())); - assertThat(result.description(), equalTo(clusterBlock.description())); - assertThat(result.retryable(), equalTo(clusterBlock.retryable())); - assertThat(result.disableStatePersistence(), equalTo(clusterBlock.disableStatePersistence())); - assertArrayEquals(result.levels().toArray(), clusterBlock.levels().toArray()); + assertClusterBlockEquals(clusterBlock, result); } } - public void testToStringDanglingComma() { - EnumSet levels = EnumSet.noneOf(ClusterBlockLevel.class); - int nbLevels = randomIntBetween(1, ClusterBlockLevel.values().length); - for (int j = 0; j < nbLevels; j++) { - levels.add(randomFrom(ClusterBlockLevel.values())); + public void testBwcSerialization() throws Exception { + for (int runs = 0; runs < randomIntBetween(5, 20); runs++) { + // Generate a random cluster block in version < 7.0.0 + final Version version = randomVersionBetween(random(), Version.V_6_0_0, getPreviousVersion(Version.V_7_0_0)); + final ClusterBlock expected = randomClusterBlock(version); + assertNull(expected.uuid()); + + // Serialize to node in current version + final BytesStreamOutput out = new BytesStreamOutput(); + expected.writeTo(out); + + // Deserialize and check the cluster block + final ClusterBlock actual = ClusterBlock.readClusterBlock(out.bytes().streamInput()); + assertClusterBlockEquals(expected, actual); } - ClusterBlock clusterBlock = new ClusterBlock(randomInt(), "cluster block #" + randomInt(), randomBoolean(), - randomBoolean(), false, randomFrom(RestStatus.values()), levels); + + for (int runs = 0; runs < randomIntBetween(5, 20); runs++) { + // Generate a random cluster block in current version + final ClusterBlock expected = randomClusterBlock(Version.CURRENT); + + // Serialize to node in version < 7.0.0 + final BytesStreamOutput out = new BytesStreamOutput(); + out.setVersion(randomVersionBetween(random(), Version.V_6_0_0, getPreviousVersion(Version.V_7_0_0))); + expected.writeTo(out); + + // Deserialize and check the cluster block + final StreamInput in = out.bytes().streamInput(); + in.setVersion(out.getVersion()); + final ClusterBlock actual = ClusterBlock.readClusterBlock(in); + + assertThat(actual.id(), equalTo(expected.id())); + assertThat(actual.status(), equalTo(expected.status())); + assertThat(actual.description(), equalTo(expected.description())); + assertThat(actual.retryable(), equalTo(expected.retryable())); + assertThat(actual.disableStatePersistence(), equalTo(expected.disableStatePersistence())); + assertArrayEquals(actual.levels().toArray(), expected.levels().toArray()); + } + } + + public void testToStringDanglingComma() { + final ClusterBlock clusterBlock = randomClusterBlock(); assertThat(clusterBlock.toString(), not(endsWith(","))); } public void testGlobalBlocksCheckedIfNoIndicesSpecified() { - EnumSet levels = EnumSet.noneOf(ClusterBlockLevel.class); - int nbLevels = randomIntBetween(1, ClusterBlockLevel.values().length); - for (int j = 0; j < nbLevels; j++) { - levels.add(randomFrom(ClusterBlockLevel.values())); - } - ClusterBlock globalBlock = new ClusterBlock(randomInt(), "cluster block #" + randomInt(), randomBoolean(), - randomBoolean(), false, randomFrom(RestStatus.values()), levels); + ClusterBlock globalBlock = randomClusterBlock(); ClusterBlocks clusterBlocks = new ClusterBlocks(Collections.singleton(globalBlock), ImmutableOpenMap.of()); ClusterBlockException exception = clusterBlocks.indicesBlockedException(randomFrom(globalBlock.levels()), new String[0]); assertNotNull(exception); assertEquals(exception.blocks(), Collections.singleton(globalBlock)); } + + public void testRemoveIndexBlockWithId() { + final ClusterBlocks.Builder builder = ClusterBlocks.builder(); + builder.addIndexBlock("index-1", + new ClusterBlock(1, "uuid", "", true, true, true, RestStatus.OK, copyOf(ClusterBlockLevel.ALL))); + builder.addIndexBlock("index-1", + new ClusterBlock(2, "uuid", "", true, true, true, RestStatus.OK, copyOf(ClusterBlockLevel.ALL))); + builder.addIndexBlock("index-1", + new ClusterBlock(3, "uuid", "", true, true, true, RestStatus.OK, copyOf(ClusterBlockLevel.ALL))); + builder.addIndexBlock("index-1", + new ClusterBlock(3, "other uuid", "", true, true, true, RestStatus.OK, copyOf(ClusterBlockLevel.ALL))); + + builder.addIndexBlock("index-2", + new ClusterBlock(3, "uuid3", "", true, true, true, RestStatus.OK, copyOf(ClusterBlockLevel.ALL))); + + ClusterBlocks clusterBlocks = builder.build(); + assertThat(clusterBlocks.indices().get("index-1").size(), equalTo(4)); + assertThat(clusterBlocks.indices().get("index-2").size(), equalTo(1)); + + builder.removeIndexBlockWithId("index-1", 3); + clusterBlocks = builder.build(); + + assertThat(clusterBlocks.indices().get("index-1").size(), equalTo(2)); + assertThat(clusterBlocks.hasIndexBlockWithId("index-1", 1), is(true)); + assertThat(clusterBlocks.hasIndexBlockWithId("index-1", 2), is(true)); + assertThat(clusterBlocks.indices().get("index-2").size(), equalTo(1)); + assertThat(clusterBlocks.hasIndexBlockWithId("index-2", 3), is(true)); + + builder.removeIndexBlockWithId("index-2", 3); + clusterBlocks = builder.build(); + + assertThat(clusterBlocks.indices().get("index-1").size(), equalTo(2)); + assertThat(clusterBlocks.hasIndexBlockWithId("index-1", 1), is(true)); + assertThat(clusterBlocks.hasIndexBlockWithId("index-1", 2), is(true)); + assertThat(clusterBlocks.indices().get("index-2"), nullValue()); + assertThat(clusterBlocks.hasIndexBlockWithId("index-2", 3), is(false)); + } + + public void testGetIndexBlockWithId() { + final int blockId = randomInt(); + final ClusterBlock[] clusterBlocks = new ClusterBlock[randomIntBetween(1, 5)]; + + final ClusterBlocks.Builder builder = ClusterBlocks.builder(); + for (int i = 0; i < clusterBlocks.length; i++) { + clusterBlocks[i] = new ClusterBlock(blockId, "uuid" + i, "", true, true, true, RestStatus.OK, copyOf(ClusterBlockLevel.ALL)); + builder.addIndexBlock("index", clusterBlocks[i]); + } + + assertThat(builder.build().indices().get("index").size(), equalTo(clusterBlocks.length)); + assertThat(builder.build().getIndexBlockWithId("index", blockId), isOneOf(clusterBlocks)); + assertThat(builder.build().getIndexBlockWithId("index", randomValueOtherThan(blockId, ESTestCase::randomInt)), nullValue()); + } + + private ClusterBlock randomClusterBlock() { + return randomClusterBlock(randomVersion(random())); + } + + private ClusterBlock randomClusterBlock(final Version version) { + final String uuid = (version.onOrAfter(Version.V_7_0_0) && randomBoolean()) ? UUIDs.randomBase64UUID() : null; + final List levels = Arrays.asList(ClusterBlockLevel.values()); + return new ClusterBlock(randomInt(), uuid, "cluster block #" + randomInt(), randomBoolean(), randomBoolean(), randomBoolean(), + randomFrom(RestStatus.values()), copyOf(randomSubsetOf(randomIntBetween(1, levels.size()), levels))); + } + + private void assertClusterBlockEquals(final ClusterBlock expected, final ClusterBlock actual) { + assertEquals(expected, actual); + assertThat(actual.id(), equalTo(expected.id())); + assertThat(actual.uuid(), equalTo(expected.uuid())); + assertThat(actual.status(), equalTo(expected.status())); + assertThat(actual.description(), equalTo(expected.description())); + assertThat(actual.retryable(), equalTo(expected.retryable())); + assertThat(actual.disableStatePersistence(), equalTo(expected.disableStatePersistence())); + assertArrayEquals(actual.levels().toArray(), expected.levels().toArray()); + } } diff --git a/server/src/test/java/org/elasticsearch/cluster/metadata/MetaDataIndexStateServiceTests.java b/server/src/test/java/org/elasticsearch/cluster/metadata/MetaDataIndexStateServiceTests.java index 6faaf8e133800..c30925514bb93 100644 --- a/server/src/test/java/org/elasticsearch/cluster/metadata/MetaDataIndexStateServiceTests.java +++ b/server/src/test/java/org/elasticsearch/cluster/metadata/MetaDataIndexStateServiceTests.java @@ -56,9 +56,14 @@ import java.util.Set; import java.util.stream.Collectors; +import static java.util.Collections.emptyMap; +import static java.util.Collections.emptySet; +import static java.util.Collections.unmodifiableMap; 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; +import static org.elasticsearch.cluster.metadata.MetaDataIndexStateService.INDEX_CLOSED_BLOCK; +import static org.elasticsearch.cluster.metadata.MetaDataIndexStateService.INDEX_CLOSED_BLOCK_ID; import static org.elasticsearch.cluster.routing.TestShardRouting.newShardRouting; import static org.elasticsearch.cluster.shards.ClusterShardLimitIT.ShardCounts.forDataNodeCount; import static org.hamcrest.Matchers.containsString; @@ -73,32 +78,37 @@ public class MetaDataIndexStateServiceTests extends ESTestCase { public void testCloseRoutingTable() { final Set nonBlockedIndices = new HashSet<>(); - final Map blockedIndices = new HashMap<>(); + final Map blockedIndices = new HashMap<>(); + final Map results = new HashMap<>(); ClusterState state = ClusterState.builder(new ClusterName("testCloseRoutingTable")).build(); for (int i = 0; i < randomIntBetween(1, 25); i++) { - final String indexName = randomAlphaOfLengthBetween(5, 15); + final String indexName = "index-" + i; if (randomBoolean()) { state = addOpenedIndex(indexName, randomIntBetween(1, 5), randomIntBetween(0, 5), state); nonBlockedIndices.add(state.metaData().index(indexName).getIndex()); } else { - state = addBlockedIndex(indexName, randomIntBetween(1, 5), randomIntBetween(0, 5), state); - blockedIndices.put(state.metaData().index(indexName).getIndex(), new AcknowledgedResponse(randomBoolean())); + final ClusterBlock closingBlock = MetaDataIndexStateService.createIndexClosingBlock(); + state = addBlockedIndex(indexName, randomIntBetween(1, 5), randomIntBetween(0, 5), state, closingBlock); + blockedIndices.put(state.metaData().index(indexName).getIndex(), closingBlock); + results.put(state.metaData().index(indexName).getIndex(), new AcknowledgedResponse(randomBoolean())); } } - final ClusterState updatedState = MetaDataIndexStateService.closeRoutingTable(state, blockedIndices); + final ClusterState updatedState = MetaDataIndexStateService.closeRoutingTable(state, blockedIndices, results); assertThat(updatedState.metaData().indices().size(), equalTo(nonBlockedIndices.size() + blockedIndices.size())); for (Index nonBlockedIndex : nonBlockedIndices) { assertIsOpened(nonBlockedIndex.getName(), updatedState); + assertThat(updatedState.blocks().hasIndexBlockWithId(nonBlockedIndex.getName(), INDEX_CLOSED_BLOCK_ID), is(false)); } - for (Map.Entry blockedIndex : blockedIndices.entrySet()) { - if (blockedIndex.getValue().isAcknowledged()) { - assertIsClosed(blockedIndex.getKey().getName(), updatedState); + for (Index blockedIndex : blockedIndices.keySet()) { + if (results.get(blockedIndex).isAcknowledged()) { + assertIsClosed(blockedIndex.getName(), updatedState); } else { - assertIsOpened(blockedIndex.getKey().getName(), updatedState); + assertIsOpened(blockedIndex.getName(), updatedState); + assertThat(updatedState.blocks().hasIndexBlockWithId(blockedIndex.getName(), INDEX_CLOSED_BLOCK_ID), is(true)); } } } @@ -106,39 +116,45 @@ public void testCloseRoutingTable() { public void testAddIndexClosedBlocks() { final ClusterState initialState = ClusterState.builder(new ClusterName("testAddIndexClosedBlocks")).build(); { - final Set blockedIndices = new HashSet<>(); + final Map blockedIndices = new HashMap<>(); + Index[] indices = new Index[]{new Index("_name", "_uid")}; expectThrows(IndexNotFoundException.class, () -> - MetaDataIndexStateService.addIndexClosedBlocks(new Index[]{new Index("_name", "_uid")}, initialState, blockedIndices)); + MetaDataIndexStateService.addIndexClosedBlocks(indices, blockedIndices, initialState)); assertTrue(blockedIndices.isEmpty()); } { - final Set blockedIndices = new HashSet<>(); + final Map blockedIndices = new HashMap<>(); Index[] indices = Index.EMPTY_ARRAY; - ClusterState updatedState = MetaDataIndexStateService.addIndexClosedBlocks(indices, initialState, blockedIndices); + ClusterState updatedState = MetaDataIndexStateService.addIndexClosedBlocks(indices, blockedIndices, initialState); assertSame(initialState, updatedState); assertTrue(blockedIndices.isEmpty()); } { - final Set blockedIndices = new HashSet<>(); + final Map blockedIndices = new HashMap<>(); ClusterState state = addClosedIndex("closed", randomIntBetween(1, 3), randomIntBetween(0, 3), initialState); Index[] indices = new Index[]{state.metaData().index("closed").getIndex()}; - ClusterState updatedState = MetaDataIndexStateService.addIndexClosedBlocks(indices, state, blockedIndices); + ClusterState updatedState = MetaDataIndexStateService.addIndexClosedBlocks(indices, blockedIndices, state); assertSame(state, updatedState); assertTrue(blockedIndices.isEmpty()); + } { - final Set blockedIndices = new HashSet<>(); + final Map blockedIndices = new HashMap<>(); ClusterState state = addClosedIndex("closed", randomIntBetween(1, 3), randomIntBetween(0, 3), initialState); state = addOpenedIndex("opened", randomIntBetween(1, 3), randomIntBetween(0, 3), state); Index[] indices = new Index[]{state.metaData().index("opened").getIndex(), state.metaData().index("closed").getIndex()}; - ClusterState updatedState = MetaDataIndexStateService.addIndexClosedBlocks(indices, state, blockedIndices); + ClusterState updatedState = MetaDataIndexStateService.addIndexClosedBlocks(indices, blockedIndices, state); assertNotSame(state, updatedState); - assertTrue(blockedIndices.contains(updatedState.metaData().index("opened").getIndex())); - assertFalse(blockedIndices.contains(updatedState.metaData().index("closed").getIndex())); - assertIsBlocked("opened", updatedState, true); + + Index opened = updatedState.metaData().index("opened").getIndex(); + assertTrue(blockedIndices.containsKey(opened)); + assertHasBlock("opened", updatedState, blockedIndices.get(opened)); + + Index closed = updatedState.metaData().index("closed").getIndex(); + assertFalse(blockedIndices.containsKey(closed)); } { IllegalArgumentException exception = expectThrows(IllegalArgumentException.class, () -> { @@ -150,7 +166,7 @@ public void testAddIndexClosedBlocks() { state = addOpenedIndex("closed", randomIntBetween(1, 3), randomIntBetween(0, 3), state); } Index[] indices = new Index[]{state.metaData().index("restored").getIndex()}; - MetaDataIndexStateService.addIndexClosedBlocks(indices, state, new HashSet<>()); + MetaDataIndexStateService.addIndexClosedBlocks(indices, unmodifiableMap(emptyMap()), state); }); assertThat(exception.getMessage(), containsString("Cannot close indices that are being restored: [[restored]]")); } @@ -164,12 +180,12 @@ public void testAddIndexClosedBlocks() { state = addOpenedIndex("closed", randomIntBetween(1, 3), randomIntBetween(0, 3), state); } Index[] indices = new Index[]{state.metaData().index("snapshotted").getIndex()}; - MetaDataIndexStateService.addIndexClosedBlocks(indices, state, new HashSet<>()); + MetaDataIndexStateService.addIndexClosedBlocks(indices, unmodifiableMap(emptyMap()), state); }); assertThat(exception.getMessage(), containsString("Cannot close indices that are being snapshotted: [[snapshotted]]")); } { - final Set blockedIndices = new HashSet<>(); + final Map blockedIndices = new HashMap<>(); ClusterState state = addOpenedIndex("index-1", randomIntBetween(1, 3), randomIntBetween(0, 3), initialState); state = addOpenedIndex("index-2", randomIntBetween(1, 3), randomIntBetween(0, 3), state); state = addOpenedIndex("index-3", randomIntBetween(1, 3), randomIntBetween(0, 3), state); @@ -177,30 +193,51 @@ public void testAddIndexClosedBlocks() { if (mixedVersions) { state = ClusterState.builder(state) .nodes(DiscoveryNodes.builder(state.nodes()) - .add(new DiscoveryNode("old_node", buildNewFakeTransportAddress(), Collections.emptyMap(), + .add(new DiscoveryNode("old_node", buildNewFakeTransportAddress(), emptyMap(), new HashSet<>(Arrays.asList(DiscoveryNode.Role.values())), Version.V_6_0_0))) .build(); } - Index[] indices = new Index[]{state.metaData().index("index-1").getIndex(), - state.metaData().index("index-2").getIndex(), state.metaData().index("index-3").getIndex()}; - ClusterState updatedState = MetaDataIndexStateService.addIndexClosedBlocks(indices, state, blockedIndices); + Index index1 = state.metaData().index("index-1").getIndex(); + Index index2 = state.metaData().index("index-2").getIndex(); + Index index3 = state.metaData().index("index-3").getIndex(); + Index[] indices = new Index[]{index1, index2, index3}; + + ClusterState updatedState = MetaDataIndexStateService.addIndexClosedBlocks(indices, blockedIndices, state); assertNotSame(state, updatedState); - assertTrue(blockedIndices.contains(updatedState.metaData().index("index-1").getIndex())); - assertTrue(blockedIndices.contains(updatedState.metaData().index("index-2").getIndex())); - assertTrue(blockedIndices.contains(updatedState.metaData().index("index-3").getIndex())); - if (mixedVersions) { - assertIsClosed("index-1", updatedState); - assertIsClosed("index-2", updatedState); - assertIsClosed("index-2", updatedState); - } else { - assertIsBlocked("index-1", updatedState, true); - assertIsBlocked("index-2", updatedState, true); - assertIsBlocked("index-3", updatedState, true); + + for (Index index : indices) { + assertTrue(blockedIndices.containsKey(index)); + if (mixedVersions) { + assertIsClosed(index.getName(), updatedState); + } else { + assertHasBlock(index.getName(), updatedState, blockedIndices.get(index)); + } } } } + public void testAddIndexClosedBlocksReusesBlocks() { + ClusterState state = ClusterState.builder(new ClusterName("testAddIndexClosedBlocksReuseBlocks")).build(); + state = addOpenedIndex("test", randomIntBetween(1, 3), randomIntBetween(0, 3), state); + + Index test = state.metaData().index("test").getIndex(); + Index[] indices = new Index[]{test}; + + final Map blockedIndices = new HashMap<>(); + state = MetaDataIndexStateService.addIndexClosedBlocks(indices, blockedIndices, state); + + assertTrue(blockedIndices.containsKey(test)); + assertHasBlock(test.getName(), state, blockedIndices.get(test)); + + final Map blockedIndices2 = new HashMap<>(); + state = MetaDataIndexStateService.addIndexClosedBlocks(indices, blockedIndices2, state); + + assertTrue(blockedIndices2.containsKey(test)); + assertHasBlock(test.getName(), state, blockedIndices2.get(test)); + assertEquals(blockedIndices.get(test), blockedIndices2.get(test)); + } + public void testValidateShardLimit() { int nodesInCluster = randomIntBetween(2,100); ClusterShardLimitIT.ShardCounts counts = forDataNodeCount(nodesInCluster); @@ -251,11 +288,12 @@ private static ClusterState addOpenedIndex(final String index, final int numShar } private static ClusterState addClosedIndex(final String index, final int numShards, final int numReplicas, final ClusterState state) { - return addIndex(state, index, numShards, numReplicas, IndexMetaData.State.CLOSE, MetaDataIndexStateService.INDEX_CLOSED_BLOCK); + return addIndex(state, index, numShards, numReplicas, IndexMetaData.State.CLOSE, INDEX_CLOSED_BLOCK); } - private static ClusterState addBlockedIndex(final String index, final int numShards, final int numReplicas, final ClusterState state) { - return addIndex(state, index, numShards, numReplicas, IndexMetaData.State.OPEN, MetaDataIndexStateService.INDEX_CLOSED_BLOCK); + private static ClusterState addBlockedIndex(final String index, final int numShards, final int numReplicas, final ClusterState state, + final ClusterBlock closingBlock) { + return addIndex(state, index, numShards, numReplicas, IndexMetaData.State.OPEN, closingBlock); } private static ClusterState addRestoredIndex(final String index, final int numShards, final int numReplicas, final ClusterState state) { @@ -329,16 +367,21 @@ private static ClusterState addIndex(final ClusterState currentState, private static void assertIsOpened(final String indexName, final ClusterState clusterState) { assertThat(clusterState.metaData().index(indexName).getState(), is(IndexMetaData.State.OPEN)); assertThat(clusterState.routingTable().index(indexName), notNullValue()); - assertIsBlocked(indexName, clusterState, false); } private static void assertIsClosed(final String indexName, final ClusterState clusterState) { assertThat(clusterState.metaData().index(indexName).getState(), is(IndexMetaData.State.CLOSE)); assertThat(clusterState.routingTable().index(indexName), nullValue()); - assertIsBlocked(indexName, clusterState, true); + assertThat(clusterState.blocks().hasIndexBlock(indexName, MetaDataIndexStateService.INDEX_CLOSED_BLOCK), is(true)); + assertThat("Index " + indexName + " must have only 1 block with [id=" + MetaDataIndexStateService.INDEX_CLOSED_BLOCK_ID + "]", + clusterState.blocks().indices().getOrDefault(indexName, emptySet()).stream() + .filter(clusterBlock -> clusterBlock.id() == MetaDataIndexStateService.INDEX_CLOSED_BLOCK_ID).count(), equalTo(1L)); } - private static void assertIsBlocked(final String indexName, final ClusterState clusterState, final boolean blocked) { - assertThat(clusterState.blocks().hasIndexBlock(indexName, MetaDataIndexStateService.INDEX_CLOSED_BLOCK), is(blocked)); + private static void assertHasBlock(final String indexName, final ClusterState clusterState, final ClusterBlock closingBlock) { + assertThat(clusterState.blocks().hasIndexBlock(indexName, closingBlock), is(true)); + assertThat("Index " + indexName + " must have only 1 block with [id=" + MetaDataIndexStateService.INDEX_CLOSED_BLOCK_ID + "]", + clusterState.blocks().indices().getOrDefault(indexName, emptySet()).stream() + .filter(clusterBlock -> clusterBlock.id() == MetaDataIndexStateService.INDEX_CLOSED_BLOCK_ID).count(), equalTo(1L)); } } diff --git a/server/src/test/java/org/elasticsearch/cluster/metadata/MetaDataIndexStateServiceUtils.java b/server/src/test/java/org/elasticsearch/cluster/metadata/MetaDataIndexStateServiceUtils.java index a9ffd4c47e161..5ee6a7c60da3d 100644 --- a/server/src/test/java/org/elasticsearch/cluster/metadata/MetaDataIndexStateServiceUtils.java +++ b/server/src/test/java/org/elasticsearch/cluster/metadata/MetaDataIndexStateServiceUtils.java @@ -20,10 +20,10 @@ import org.elasticsearch.action.support.master.AcknowledgedResponse; import org.elasticsearch.cluster.ClusterState; +import org.elasticsearch.cluster.block.ClusterBlock; import org.elasticsearch.index.Index; import java.util.Map; -import java.util.Set; public class MetaDataIndexStateServiceUtils { @@ -31,16 +31,19 @@ private MetaDataIndexStateServiceUtils(){ } /** - * Allows to call {@link MetaDataIndexStateService#addIndexClosedBlocks(Index[], ClusterState, Set)} which is a protected method. + * Allows to call {@link MetaDataIndexStateService#addIndexClosedBlocks(Index[], Map, ClusterState)} which is a protected method. */ - public static ClusterState addIndexClosedBlocks(final Index[] indices, final ClusterState state, final Set blockedIndices) { - return MetaDataIndexStateService.addIndexClosedBlocks(indices, state, blockedIndices); + public static ClusterState addIndexClosedBlocks(final Index[] indices, final Map blockedIndices, + final ClusterState state) { + return MetaDataIndexStateService.addIndexClosedBlocks(indices, blockedIndices, state); } /** - * Allows to call {@link MetaDataIndexStateService#closeRoutingTable(ClusterState, Map)} which is a protected method. + * Allows to call {@link MetaDataIndexStateService#closeRoutingTable(ClusterState, Map, Map)} which is a protected method. */ - public static ClusterState closeRoutingTable(final ClusterState state, final Map results) { - return MetaDataIndexStateService.closeRoutingTable(state, results); + public static ClusterState closeRoutingTable(final ClusterState state, + final Map blockedIndices, + final Map results) { + return MetaDataIndexStateService.closeRoutingTable(state, blockedIndices, results); } } diff --git a/server/src/test/java/org/elasticsearch/indices/cluster/ClusterStateChanges.java b/server/src/test/java/org/elasticsearch/indices/cluster/ClusterStateChanges.java index 1403543078b28..387ba1c3d9653 100644 --- a/server/src/test/java/org/elasticsearch/indices/cluster/ClusterStateChanges.java +++ b/server/src/test/java/org/elasticsearch/indices/cluster/ClusterStateChanges.java @@ -52,6 +52,7 @@ import org.elasticsearch.cluster.action.shard.ShardStateAction; import org.elasticsearch.cluster.action.shard.ShardStateAction.FailedShardEntry; import org.elasticsearch.cluster.action.shard.ShardStateAction.StartedShardEntry; +import org.elasticsearch.cluster.block.ClusterBlock; import org.elasticsearch.cluster.coordination.JoinTaskExecutor; import org.elasticsearch.cluster.coordination.NodeRemovalClusterStateTaskExecutor; import org.elasticsearch.cluster.metadata.AliasValidator; @@ -94,9 +95,10 @@ import java.util.ArrayList; import java.util.Arrays; import java.util.Collections; +import java.util.HashMap; import java.util.HashSet; import java.util.List; -import java.util.Set; +import java.util.Map; import java.util.function.Function; import java.util.stream.Collectors; @@ -222,10 +224,10 @@ public ClusterState closeIndices(ClusterState state, CloseIndexRequest request) final Index[] concreteIndices = Arrays.stream(request.indices()) .map(index -> state.metaData().index(index).getIndex()).toArray(Index[]::new); - final Set blockedIndices = new HashSet<>(); - ClusterState newState = MetaDataIndexStateServiceUtils.addIndexClosedBlocks(concreteIndices, state, blockedIndices); + final Map blockedIndices = new HashMap<>(); + ClusterState newState = MetaDataIndexStateServiceUtils.addIndexClosedBlocks(concreteIndices, blockedIndices, state); - newState = MetaDataIndexStateServiceUtils.closeRoutingTable(newState, blockedIndices.stream() + newState = MetaDataIndexStateServiceUtils.closeRoutingTable(newState, blockedIndices, blockedIndices.keySet().stream() .collect(Collectors.toMap(Function.identity(), index -> new AcknowledgedResponse(true)))); return allocationService.reroute(newState, "indices closed"); } diff --git a/server/src/test/java/org/elasticsearch/indices/state/CloseIndexIT.java b/server/src/test/java/org/elasticsearch/indices/state/CloseIndexIT.java index c91189972c7b2..a0304c96430f0 100644 --- a/server/src/test/java/org/elasticsearch/indices/state/CloseIndexIT.java +++ b/server/src/test/java/org/elasticsearch/indices/state/CloseIndexIT.java @@ -39,12 +39,12 @@ import java.util.concurrent.CountDownLatch; import java.util.stream.IntStream; +import static java.util.Collections.emptySet; import static java.util.stream.Collectors.toList; import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertAcked; import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertHitCount; import static org.hamcrest.Matchers.containsString; import static org.hamcrest.Matchers.equalTo; -import static org.hamcrest.Matchers.hasItem; import static org.hamcrest.Matchers.hasSize; import static org.hamcrest.Matchers.is; import static org.hamcrest.Matchers.notNullValue; @@ -135,6 +135,7 @@ public void testConcurrentClose() throws InterruptedException { final int nbDocs = randomIntBetween(10, 50); indexRandom(randomBoolean(), false, randomBoolean(), IntStream.range(0, nbDocs) .mapToObj(i -> client().prepareIndex(indexName, "_doc", String.valueOf(i)).setSource("num", i)).collect(toList())); + ensureYellowAndNoInitializingShards(indexName); final CountDownLatch startClosing = new CountDownLatch(1); final Thread[] threads = new Thread[randomIntBetween(2, 5)]; @@ -146,7 +147,11 @@ public void testConcurrentClose() throws InterruptedException { } catch (InterruptedException e) { throw new AssertionError(e); } - assertAcked(client().admin().indices().prepareClose(indexName)); + try { + client().admin().indices().prepareClose(indexName).get(); + } catch (final Exception e) { + assertException(e, indexName); + } }); threads[i].start(); } @@ -238,18 +243,84 @@ public void testCloseWhileDeletingIndices() throws Exception { } } - static void assertIndexIsClosed(final String indexName) { + public void testConcurrentClosesAndOpens() throws Exception { + final String indexName = randomAlphaOfLength(10).toLowerCase(Locale.ROOT); + createIndex(indexName); + + final BackgroundIndexer indexer = new BackgroundIndexer(indexName, "_doc", client()); + waitForDocs(1, indexer); + + final CountDownLatch latch = new CountDownLatch(1); + final Runnable waitForLatch = () -> { + try { + latch.await(); + } catch (final InterruptedException e) { + throw new AssertionError(e); + } + }; + + final List threads = new ArrayList<>(); + for (int i = 0; i < randomIntBetween(1, 3); i++) { + threads.add(new Thread(() -> { + try { + waitForLatch.run(); + client().admin().indices().prepareClose(indexName).get(); + } catch (final Exception e) { + throw new AssertionError(e); + } + })); + } + for (int i = 0; i < randomIntBetween(1, 3); i++) { + threads.add(new Thread(() -> { + try { + waitForLatch.run(); + assertAcked(client().admin().indices().prepareOpen(indexName).get()); + } catch (final Exception e) { + throw new AssertionError(e); + } + })); + } + + for (Thread thread : threads) { + thread.start(); + } + latch.countDown(); + for (Thread thread : threads) { + thread.join(); + } + + indexer.setAssertNoFailuresOnStop(false); + indexer.stop(); + + final ClusterState clusterState = client().admin().cluster().prepareState().get().getState(); + if (clusterState.metaData().indices().get(indexName).getState() == IndexMetaData.State.CLOSE) { + assertIndexIsClosed(indexName); + assertAcked(client().admin().indices().prepareOpen(indexName)); + } + refresh(indexName); + assertIndexIsOpened(indexName); + assertHitCount(client().prepareSearch(indexName).setSize(0).get(), indexer.totalIndexedDocs()); + } + + static void assertIndexIsClosed(final String... indices) { final ClusterState clusterState = client().admin().cluster().prepareState().get().getState(); - assertThat(clusterState.metaData().indices().get(indexName).getState(), is(IndexMetaData.State.CLOSE)); - assertThat(clusterState.routingTable().index(indexName), nullValue()); - assertThat(clusterState.blocks().hasIndexBlock(indexName, MetaDataIndexStateService.INDEX_CLOSED_BLOCK), is(true)); + for (String index : indices) { + assertThat(clusterState.metaData().indices().get(index).getState(), is(IndexMetaData.State.CLOSE)); + assertThat(clusterState.routingTable().index(index), nullValue()); + assertThat(clusterState.blocks().hasIndexBlock(index, MetaDataIndexStateService.INDEX_CLOSED_BLOCK), is(true)); + assertThat("Index " + index + " must have only 1 block with [id=" + MetaDataIndexStateService.INDEX_CLOSED_BLOCK_ID + "]", + clusterState.blocks().indices().getOrDefault(index, emptySet()).stream() + .filter(clusterBlock -> clusterBlock.id() == MetaDataIndexStateService.INDEX_CLOSED_BLOCK_ID).count(), equalTo(1L)); + } } - static void assertIndexIsOpened(final String indexName) { + static void assertIndexIsOpened(final String... indices) { final ClusterState clusterState = client().admin().cluster().prepareState().get().getState(); - assertThat(clusterState.metaData().indices().get(indexName).getState(), is(IndexMetaData.State.OPEN)); - assertThat(clusterState.routingTable().index(indexName), notNullValue()); - assertThat(clusterState.blocks().hasIndexBlock(indexName, MetaDataIndexStateService.INDEX_CLOSED_BLOCK), is(false)); + for (String index : indices) { + assertThat(clusterState.metaData().indices().get(index).getState(), is(IndexMetaData.State.OPEN)); + assertThat(clusterState.routingTable().index(index), notNullValue()); + assertThat(clusterState.blocks().hasIndexBlock(index, MetaDataIndexStateService.INDEX_CLOSED_BLOCK), is(false)); + } } static void assertException(final Throwable throwable, final String indexName) { @@ -257,7 +328,7 @@ static void assertException(final Throwable throwable, final String indexName) { if (t instanceof ClusterBlockException) { ClusterBlockException clusterBlockException = (ClusterBlockException) t; assertThat(clusterBlockException.blocks(), hasSize(1)); - assertThat(clusterBlockException.blocks(), hasItem(MetaDataIndexStateService.INDEX_CLOSED_BLOCK)); + assertTrue(clusterBlockException.blocks().stream().allMatch(b -> b.id() == MetaDataIndexStateService.INDEX_CLOSED_BLOCK_ID)); } else if (t instanceof IndexClosedException) { IndexClosedException indexClosedException = (IndexClosedException) t; assertThat(indexClosedException.getIndex(), notNullValue()); diff --git a/server/src/test/java/org/elasticsearch/indices/state/OpenCloseIndexIT.java b/server/src/test/java/org/elasticsearch/indices/state/OpenCloseIndexIT.java index ddbbd0ea73a8f..e9e9108f5e8f1 100644 --- a/server/src/test/java/org/elasticsearch/indices/state/OpenCloseIndexIT.java +++ b/server/src/test/java/org/elasticsearch/indices/state/OpenCloseIndexIT.java @@ -21,7 +21,6 @@ import org.elasticsearch.action.ActionRequestValidationException; import org.elasticsearch.action.admin.cluster.health.ClusterHealthResponse; -import org.elasticsearch.action.admin.cluster.state.ClusterStateResponse; import org.elasticsearch.action.admin.indices.open.OpenIndexResponse; import org.elasticsearch.action.index.IndexRequestBuilder; import org.elasticsearch.action.search.SearchResponse; @@ -46,6 +45,8 @@ import static org.elasticsearch.cluster.metadata.IndexMetaData.SETTING_BLOCKS_WRITE; import static org.elasticsearch.cluster.metadata.IndexMetaData.SETTING_READ_ONLY; import static org.elasticsearch.cluster.metadata.IndexMetaData.SETTING_READ_ONLY_ALLOW_DELETE; +import static org.elasticsearch.indices.state.CloseIndexIT.assertIndexIsClosed; +import static org.elasticsearch.indices.state.CloseIndexIT.assertIndexIsOpened; import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertAcked; import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertBlocked; import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertHitCount; @@ -53,7 +54,6 @@ import static org.hamcrest.Matchers.containsString; import static org.hamcrest.Matchers.equalTo; import static org.hamcrest.Matchers.is; -import static org.hamcrest.Matchers.notNullValue; public class OpenCloseIndexIT extends ESIntegTestCase { public void testSimpleCloseOpen() { @@ -258,23 +258,6 @@ public void testOpenWaitingForActiveShardsFailed() throws Exception { ensureGreen("test"); } - private void assertIndexIsOpened(String... indices) { - checkIndexState(IndexMetaData.State.OPEN, indices); - } - - private void assertIndexIsClosed(String... indices) { - checkIndexState(IndexMetaData.State.CLOSE, indices); - } - - private void checkIndexState(IndexMetaData.State expectedState, String... indices) { - ClusterStateResponse clusterStateResponse = client().admin().cluster().prepareState().execute().actionGet(); - for (String index : indices) { - IndexMetaData indexMetaData = clusterStateResponse.getState().metaData().indices().get(index); - assertThat(indexMetaData, notNullValue()); - assertThat(indexMetaData.getState(), equalTo(expectedState)); - } - } - public void testOpenCloseWithDocs() throws IOException, ExecutionException, InterruptedException { String mapping = Strings.toString(XContentFactory.jsonBuilder(). startObject(). diff --git a/server/src/test/java/org/elasticsearch/indices/state/ReopenWhileClosingIT.java b/server/src/test/java/org/elasticsearch/indices/state/ReopenWhileClosingIT.java new file mode 100644 index 0000000000000..901c4f327af48 --- /dev/null +++ b/server/src/test/java/org/elasticsearch/indices/state/ReopenWhileClosingIT.java @@ -0,0 +1,167 @@ +/* + * 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.indices.state; + +import org.elasticsearch.action.ActionFuture; +import org.elasticsearch.action.admin.indices.close.TransportVerifyShardBeforeCloseAction; +import org.elasticsearch.action.support.master.AcknowledgedResponse; +import org.elasticsearch.cluster.ClusterState; +import org.elasticsearch.cluster.metadata.IndexMetaData; +import org.elasticsearch.cluster.node.DiscoveryNode; +import org.elasticsearch.common.Glob; +import org.elasticsearch.common.Strings; +import org.elasticsearch.common.lease.Releasable; +import org.elasticsearch.common.util.concurrent.RunOnce; +import org.elasticsearch.plugins.Plugin; +import org.elasticsearch.test.ESIntegTestCase; +import org.elasticsearch.test.transport.MockTransportService; +import org.elasticsearch.transport.TransportService; + +import java.util.ArrayList; +import java.util.Collection; +import java.util.List; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.TimeUnit; + +import static java.util.Collections.emptySet; +import static java.util.Collections.singletonList; +import static org.elasticsearch.cluster.metadata.MetaDataIndexStateService.INDEX_CLOSED_BLOCK_ID; +import static org.elasticsearch.indices.state.CloseIndexIT.assertIndexIsClosed; +import static org.elasticsearch.indices.state.CloseIndexIT.assertIndexIsOpened; +import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertAcked; +import static org.hamcrest.Matchers.equalTo; +import static org.hamcrest.Matchers.is; +import static org.hamcrest.Matchers.notNullValue; + +@ESIntegTestCase.ClusterScope(scope = ESIntegTestCase.Scope.TEST, minNumDataNodes = 2) +public class ReopenWhileClosingIT extends ESIntegTestCase { + + @Override + protected Collection> nodePlugins() { + return singletonList(MockTransportService.TestPlugin.class); + } + + public void testReopenDuringClose() throws Exception { + final String indexName = "test"; + createIndexWithDocs(indexName); + + ensureYellowAndNoInitializingShards(indexName); + + final CountDownLatch block = new CountDownLatch(1); + final Releasable releaseBlock = interceptVerifyShardBeforeCloseActions(indexName, block::countDown); + + ActionFuture closeIndexResponse = client().admin().indices().prepareClose(indexName).execute(); + assertTrue("Waiting for index to have a closing blocked", block.await(60, TimeUnit.SECONDS)); + assertIndexIsBlocked(indexName); + assertFalse(closeIndexResponse.isDone()); + + assertAcked(client().admin().indices().prepareOpen(indexName)); + + releaseBlock.close(); + assertFalse(closeIndexResponse.get().isAcknowledged()); + assertIndexIsOpened(indexName); + } + + public void testReopenDuringCloseOnMultipleIndices() throws Exception { + final List indices = new ArrayList<>(); + for (int i = 0; i < randomIntBetween(2, 10); i++) { + indices.add("index-" + i); + createIndexWithDocs(indices.get(i)); + } + + ensureYellowAndNoInitializingShards(indices.toArray(Strings.EMPTY_ARRAY)); + + final CountDownLatch block = new CountDownLatch(1); + final Releasable releaseBlock = interceptVerifyShardBeforeCloseActions(randomFrom(indices), block::countDown); + + ActionFuture closeIndexResponse = client().admin().indices().prepareClose("index-*").execute(); + assertTrue("Waiting for index to have a closing blocked", block.await(60, TimeUnit.SECONDS)); + assertFalse(closeIndexResponse.isDone()); + indices.forEach(ReopenWhileClosingIT::assertIndexIsBlocked); + + final List reopenedIndices = randomSubsetOf(randomIntBetween(1, indices.size()), indices); + assertAcked(client().admin().indices().prepareOpen(reopenedIndices.toArray(Strings.EMPTY_ARRAY))); + + releaseBlock.close(); + assertFalse(closeIndexResponse.get().isAcknowledged()); + + indices.forEach(index -> { + if (reopenedIndices.contains(index)) { + assertIndexIsOpened(index); + } else { + assertIndexIsClosed(index); + } + }); + } + + private void createIndexWithDocs(final String indexName) { + createIndex(indexName); + final int nbDocs = scaledRandomIntBetween(1, 100); + for (int i = 0; i < nbDocs; i++) { + index(indexName, "_doc", String.valueOf(i), "num", i); + } + assertIndexIsOpened(indexName); + } + + /** + * Intercepts and blocks the {@link TransportVerifyShardBeforeCloseAction} executed for the given index pattern. + */ + private Releasable interceptVerifyShardBeforeCloseActions(final String indexPattern, final Runnable onIntercept) { + final MockTransportService mockTransportService = (MockTransportService) internalCluster() + .getInstance(TransportService.class, internalCluster().getMasterName()); + + final CountDownLatch release = new CountDownLatch(1); + for (DiscoveryNode node : internalCluster().clusterService().state().getNodes()) { + mockTransportService.addSendBehavior(internalCluster().getInstance(TransportService.class, node.getName()), + (connection, requestId, action, request, options) -> { + if (action.startsWith(TransportVerifyShardBeforeCloseAction.NAME)) { + if (request instanceof TransportVerifyShardBeforeCloseAction.ShardRequest) { + final String index = ((TransportVerifyShardBeforeCloseAction.ShardRequest) request).shardId().getIndexName(); + if (Glob.globMatch(indexPattern, index)) { + logger.info("request {} intercepted for index {}", requestId, index); + onIntercept.run(); + try { + release.await(); + logger.info("request {} released for index {}", requestId, index); + } catch (final InterruptedException e) { + throw new AssertionError(e); + } + } + } + + } + connection.sendRequest(requestId, action, request, options); + }); + } + final RunOnce releaseOnce = new RunOnce(release::countDown); + return releaseOnce::run; + } + + private static void assertIndexIsBlocked(final String... indices) { + final ClusterState clusterState = client().admin().cluster().prepareState().get().getState(); + for (String index : indices) { + assertThat(clusterState.metaData().indices().get(index).getState(), is(IndexMetaData.State.OPEN)); + assertThat(clusterState.routingTable().index(index), notNullValue()); + assertThat("Index " + index + " must have only 1 block with [id=" + INDEX_CLOSED_BLOCK_ID + "]", + clusterState.blocks().indices().getOrDefault(index, emptySet()).stream() + .filter(clusterBlock -> clusterBlock.id() == INDEX_CLOSED_BLOCK_ID).count(), equalTo(1L)); + } + } +} diff --git a/server/src/test/java/org/elasticsearch/snapshots/DedicatedClusterSnapshotRestoreIT.java b/server/src/test/java/org/elasticsearch/snapshots/DedicatedClusterSnapshotRestoreIT.java index 4c5ed74e3ca5f..afdb14eaf6ba8 100644 --- a/server/src/test/java/org/elasticsearch/snapshots/DedicatedClusterSnapshotRestoreIT.java +++ b/server/src/test/java/org/elasticsearch/snapshots/DedicatedClusterSnapshotRestoreIT.java @@ -544,7 +544,7 @@ public void testRestoreIndexWithMissingShards() throws Exception { logger.info("--> start snapshot with default settings and closed index - should be blocked"); assertBlocked(client().admin().cluster().prepareCreateSnapshot("test-repo", "test-snap-1") .setIndices("test-idx-all", "test-idx-none", "test-idx-some", "test-idx-closed") - .setWaitForCompletion(true), MetaDataIndexStateService.INDEX_CLOSED_BLOCK); + .setWaitForCompletion(true), MetaDataIndexStateService.INDEX_CLOSED_BLOCK_ID); logger.info("--> start snapshot with default settings without a closed index - should fail"); diff --git a/server/src/test/java/org/elasticsearch/snapshots/SharedClusterSnapshotRestoreIT.java b/server/src/test/java/org/elasticsearch/snapshots/SharedClusterSnapshotRestoreIT.java index 7c1cff091300c..3461aac3ec289 100644 --- a/server/src/test/java/org/elasticsearch/snapshots/SharedClusterSnapshotRestoreIT.java +++ b/server/src/test/java/org/elasticsearch/snapshots/SharedClusterSnapshotRestoreIT.java @@ -1562,7 +1562,7 @@ public void testSnapshotClosedIndex() throws Exception { logger.info("--> snapshot with closed index"); assertBlocked(client.admin().cluster().prepareCreateSnapshot("test-repo", "test-snap").setWaitForCompletion(true) - .setIndices("test-idx", "test-idx-closed"), MetaDataIndexStateService.INDEX_CLOSED_BLOCK); + .setIndices("test-idx", "test-idx-closed"), MetaDataIndexStateService.INDEX_CLOSED_BLOCK_ID); } public void testSnapshotSingleClosedIndex() throws Exception { @@ -1580,7 +1580,7 @@ public void testSnapshotSingleClosedIndex() throws Exception { logger.info("--> snapshot"); assertBlocked(client.admin().cluster().prepareCreateSnapshot("test-repo", "test-snap-1") - .setWaitForCompletion(true).setIndices("test-idx"), MetaDataIndexStateService.INDEX_CLOSED_BLOCK); + .setWaitForCompletion(true).setIndices("test-idx"), MetaDataIndexStateService.INDEX_CLOSED_BLOCK_ID); } public void testRenameOnRestore() throws Exception { diff --git a/test/framework/src/main/java/org/elasticsearch/test/hamcrest/ElasticsearchAssertions.java b/test/framework/src/main/java/org/elasticsearch/test/hamcrest/ElasticsearchAssertions.java index 6005e7e6163f1..48d18e096bf17 100644 --- a/test/framework/src/main/java/org/elasticsearch/test/hamcrest/ElasticsearchAssertions.java +++ b/test/framework/src/main/java/org/elasticsearch/test/hamcrest/ElasticsearchAssertions.java @@ -129,7 +129,7 @@ public static void assertAcked(CreateIndexResponse response) { * @param builder the request builder */ public static void assertBlocked(ActionRequestBuilder builder) { - assertBlocked(builder, null); + assertBlocked(builder, (ClusterBlock) null); } /** @@ -155,9 +155,9 @@ public static void assertBlocked(BroadcastResponse replicatedBroadcastResponse) * Executes the request and fails if the request has not been blocked by a specific {@link ClusterBlock}. * * @param builder the request builder - * @param expectedBlock the expected block + * @param expectedBlockId the expected block id */ - public static void assertBlocked(ActionRequestBuilder builder, ClusterBlock expectedBlock) { + public static void assertBlocked(final ActionRequestBuilder builder, @Nullable final Integer expectedBlockId) { try { builder.get(); fail("Request executed with success but a ClusterBlockException was expected"); @@ -165,19 +165,29 @@ public static void assertBlocked(ActionRequestBuilder builder, ClusterBlock expe assertThat(e.blocks().size(), greaterThan(0)); assertThat(e.status(), equalTo(RestStatus.FORBIDDEN)); - if (expectedBlock != null) { + if (expectedBlockId != null) { boolean found = false; for (ClusterBlock clusterBlock : e.blocks()) { - if (clusterBlock.id() == expectedBlock.id()) { + if (clusterBlock.id() == expectedBlockId) { found = true; break; } } - assertThat("Request should have been blocked by [" + expectedBlock + "] instead of " + e.blocks(), found, equalTo(true)); + assertThat("Request should have been blocked by [" + expectedBlockId + "] instead of " + e.blocks(), found, equalTo(true)); } } } + /** + * Executes the request and fails if the request has not been blocked by a specific {@link ClusterBlock}. + * + * @param builder the request builder + * @param expectedBlock the expected block + */ + public static void assertBlocked(final ActionRequestBuilder builder, @Nullable final ClusterBlock expectedBlock) { + assertBlocked(builder, expectedBlock != null ? expectedBlock.id() : null); + } + public static String formatShardStatus(BroadcastResponse response) { StringBuilder msg = new StringBuilder(); msg.append(" Total shards: ").append(response.getTotalShards()) From be2978eacdf11b5a57348a37742f764570e2c876 Mon Sep 17 00:00:00 2001 From: Yuri Astrakhan Date: Mon, 7 Jan 2019 22:53:17 -0500 Subject: [PATCH 08/46] (WIP) Support for tile-based geo grid aggregation --- .../elasticsearch/common/geo/GeoUtils.java | 35 +- .../elasticsearch/search/SearchModule.java | 6 + .../aggregations/bucket/geogrid2/GeoGrid.java | 42 +++ .../geogrid2/GeoGridAggregationBuilder.java | 354 ++++++++++++++++++ .../bucket/geogrid2/GeoGridAggregator.java | 149 ++++++++ .../geogrid2/GeoGridAggregatorFactory.java | 82 ++++ .../bucket/geogrid2/GeoGridType.java | 62 +++ .../bucket/geogrid2/GeoGridTypes.java | 51 +++ .../bucket/geogrid2/GeoHashType.java | 60 +++ .../bucket/geogrid2/InternalGeoGrid.java | 300 +++++++++++++++ .../bucket/geogrid2/ParsedGeoGrid.java | 78 ++++ 11 files changed, 1203 insertions(+), 16 deletions(-) create mode 100644 server/src/main/java/org/elasticsearch/search/aggregations/bucket/geogrid2/GeoGrid.java create mode 100644 server/src/main/java/org/elasticsearch/search/aggregations/bucket/geogrid2/GeoGridAggregationBuilder.java create mode 100644 server/src/main/java/org/elasticsearch/search/aggregations/bucket/geogrid2/GeoGridAggregator.java create mode 100644 server/src/main/java/org/elasticsearch/search/aggregations/bucket/geogrid2/GeoGridAggregatorFactory.java create mode 100644 server/src/main/java/org/elasticsearch/search/aggregations/bucket/geogrid2/GeoGridType.java create mode 100644 server/src/main/java/org/elasticsearch/search/aggregations/bucket/geogrid2/GeoGridTypes.java create mode 100644 server/src/main/java/org/elasticsearch/search/aggregations/bucket/geogrid2/GeoHashType.java create mode 100644 server/src/main/java/org/elasticsearch/search/aggregations/bucket/geogrid2/InternalGeoGrid.java create mode 100644 server/src/main/java/org/elasticsearch/search/aggregations/bucket/geogrid2/ParsedGeoGrid.java diff --git a/server/src/main/java/org/elasticsearch/common/geo/GeoUtils.java b/server/src/main/java/org/elasticsearch/common/geo/GeoUtils.java index 795cc235ce759..befff8bafb6a7 100644 --- a/server/src/main/java/org/elasticsearch/common/geo/GeoUtils.java +++ b/server/src/main/java/org/elasticsearch/common/geo/GeoUtils.java @@ -555,23 +555,26 @@ private static GeoPoint parseGeoHash(GeoPoint point, String geohash, EffectivePo * @return int representing precision */ public static int parsePrecision(XContentParser parser) throws IOException, ElasticsearchParseException { - XContentParser.Token token = parser.currentToken(); - if (token.equals(XContentParser.Token.VALUE_NUMBER)) { - return XContentMapValues.nodeIntegerValue(parser.intValue()); - } else { - String precision = parser.text(); + return parser.currentToken() == Token.VALUE_NUMBER ? parser.intValue() : parsePrecisionString(parser.text()); + } + + /** + * Attempt to parse geohash precision string into an integer value + */ + public static int parsePrecisionString(String precision) { + try { + // we want to treat simple integer strings as precision levels, not distances + return checkPrecisionRange(Integer.parseInt(precision)); + // checkPrecisionRange could also throw IllegalArgumentException, but let it through + // to keep errors somewhat consistent with how they were shown before this change + } catch (NumberFormatException e) { + // try to parse as a distance value + final int parsedPrecision = GeoUtils.geoHashLevelsForPrecision(precision); try { - // we want to treat simple integer strings as precision levels, not distances - return XContentMapValues.nodeIntegerValue(precision); - } catch (NumberFormatException e) { - // try to parse as a distance value - final int parsedPrecision = GeoUtils.geoHashLevelsForPrecision(precision); - try { - return checkPrecisionRange(parsedPrecision); - } catch (IllegalArgumentException e2) { - // this happens when distance too small, so precision > 12. We'd like to see the original string - throw new IllegalArgumentException("precision too high [" + precision + "]", e2); - } + return checkPrecisionRange(parsedPrecision); + } catch (IllegalArgumentException e2) { + // this happens when distance too small, so precision > 12. We'd like to see the original string + throw new IllegalArgumentException("precision too high [" + precision + "]", e2); } } } diff --git a/server/src/main/java/org/elasticsearch/search/SearchModule.java b/server/src/main/java/org/elasticsearch/search/SearchModule.java index 2531685b94557..96cfbfb7f917b 100644 --- a/server/src/main/java/org/elasticsearch/search/SearchModule.java +++ b/server/src/main/java/org/elasticsearch/search/SearchModule.java @@ -110,6 +110,7 @@ import org.elasticsearch.search.aggregations.bucket.filter.InternalFilters; import org.elasticsearch.search.aggregations.bucket.geogrid.GeoGridAggregationBuilder; import org.elasticsearch.search.aggregations.bucket.geogrid.InternalGeoHashGrid; +import org.elasticsearch.search.aggregations.bucket.geogrid2.InternalGeoGrid; import org.elasticsearch.search.aggregations.bucket.global.GlobalAggregationBuilder; import org.elasticsearch.search.aggregations.bucket.global.InternalGlobal; import org.elasticsearch.search.aggregations.bucket.histogram.AutoDateHistogramAggregationBuilder; @@ -421,6 +422,11 @@ private void registerAggregations(List plugins) { GeoDistanceAggregationBuilder::parse).addResultReader(InternalGeoDistance::new)); registerAggregation(new AggregationSpec(GeoGridAggregationBuilder.NAME, GeoGridAggregationBuilder::new, GeoGridAggregationBuilder::parse).addResultReader(InternalGeoHashGrid::new)); + registerAggregation(new AggregationSpec( + org.elasticsearch.search.aggregations.bucket.geogrid2.GeoGridAggregationBuilder.NAME, + org.elasticsearch.search.aggregations.bucket.geogrid2.GeoGridAggregationBuilder::new, + org.elasticsearch.search.aggregations.bucket.geogrid2.GeoGridAggregationBuilder::parse + ).addResultReader(InternalGeoGrid::new)); registerAggregation(new AggregationSpec(NestedAggregationBuilder.NAME, NestedAggregationBuilder::new, NestedAggregationBuilder::parse).addResultReader(InternalNested::new)); registerAggregation(new AggregationSpec(ReverseNestedAggregationBuilder.NAME, ReverseNestedAggregationBuilder::new, diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/geogrid2/GeoGrid.java b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/geogrid2/GeoGrid.java new file mode 100644 index 0000000000000..d06efcef2fb88 --- /dev/null +++ b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/geogrid2/GeoGrid.java @@ -0,0 +1,42 @@ +/* + * 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.aggregations.bucket.geogrid2; + +import org.elasticsearch.search.aggregations.bucket.MultiBucketsAggregation; + +import java.util.List; + +/** + * A {@code geo_grid} aggregation. Defines multiple buckets, each representing a cell in a geo-grid of a specific + * precision. + */ +public interface GeoGrid extends MultiBucketsAggregation { + + /** + * A bucket that is associated with a {@code geohash_grid} cell. The key of the bucket is the {@code geohash} of the cell + */ + interface Bucket extends MultiBucketsAggregation.Bucket { + } + + /** + * @return The buckets of this aggregation (each bucket representing a geohash grid cell) + */ + @Override + List getBuckets(); +} diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/geogrid2/GeoGridAggregationBuilder.java b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/geogrid2/GeoGridAggregationBuilder.java new file mode 100644 index 0000000000000..88cfbf5575f8c --- /dev/null +++ b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/geogrid2/GeoGridAggregationBuilder.java @@ -0,0 +1,354 @@ +/* + * 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.aggregations.bucket.geogrid2; + +import org.apache.lucene.index.LeafReaderContext; +import org.apache.lucene.index.SortedNumericDocValues; +import org.elasticsearch.ElasticsearchException; +import org.elasticsearch.common.ParseField; +import org.elasticsearch.common.geo.GeoPoint; +import org.elasticsearch.common.io.stream.StreamInput; +import org.elasticsearch.common.io.stream.StreamOutput; +import org.elasticsearch.common.xcontent.ConstructingObjectParser; +import org.elasticsearch.common.xcontent.ObjectParser; +import org.elasticsearch.common.xcontent.XContentBuilder; +import org.elasticsearch.common.xcontent.XContentParseException; +import org.elasticsearch.common.xcontent.XContentParser; +import org.elasticsearch.index.fielddata.AbstractSortingNumericDocValues; +import org.elasticsearch.index.fielddata.MultiGeoPointValues; +import org.elasticsearch.index.fielddata.SortedBinaryDocValues; +import org.elasticsearch.index.fielddata.SortedNumericDoubleValues; +import org.elasticsearch.search.aggregations.AggregationBuilder; +import org.elasticsearch.search.aggregations.AggregatorFactories.Builder; +import org.elasticsearch.search.aggregations.AggregatorFactory; +import org.elasticsearch.search.aggregations.bucket.BucketUtils; +import org.elasticsearch.search.aggregations.bucket.MultiBucketAggregationBuilder; +import org.elasticsearch.search.aggregations.support.ValueType; +import org.elasticsearch.search.aggregations.support.ValuesSource; +import org.elasticsearch.search.aggregations.support.ValuesSourceAggregationBuilder; +import org.elasticsearch.search.aggregations.support.ValuesSourceAggregatorFactory; +import org.elasticsearch.search.aggregations.support.ValuesSourceConfig; +import org.elasticsearch.search.aggregations.support.ValuesSourceParserHelper; +import org.elasticsearch.search.aggregations.support.ValuesSourceType; +import org.elasticsearch.search.internal.SearchContext; + +import java.io.IOException; +import java.util.HashMap; +import java.util.Map; +import java.util.Objects; + +import static org.elasticsearch.common.xcontent.ConstructingObjectParser.optionalConstructorArg; + +public class GeoGridAggregationBuilder extends ValuesSourceAggregationBuilder + implements MultiBucketAggregationBuilder { + public static final String NAME = "geo_grid"; + public static final int DEFAULT_MAX_NUM_CELLS = 10000; + + /* recognized field names in JSON */ + static final ParseField FIELD_TYPE = new ParseField("hash_type"); + static final ParseField FIELD_PRECISION = new ParseField("precision"); + static final ParseField FIELD_SIZE = new ParseField("size"); + static final ParseField FIELD_SHARD_SIZE = new ParseField("shard_size"); + + public static GeoGridTypes types; + + private static final ConstructingObjectParser PARSER; + + static { + types = GeoGridTypes.DEFAULT; + + PARSER = new ConstructingObjectParser<>(GeoGridAggregationBuilder.NAME, false, + (a, name) -> new GeoGridAggregationBuilder(name, (GeoGridType) a[0])); + + // TODO: Should we require type param to be always present? + PARSER.declareField( + optionalConstructorArg(), + GeoGridAggregationBuilder::parseType, + FIELD_TYPE, + ObjectParser.ValueType.STRING); + PARSER.declareField( + GeoGridAggregationBuilder::precisionRaw, + GeoGridAggregationBuilder::parsePrecision, + FIELD_PRECISION, + ObjectParser.ValueType.VALUE); + PARSER.declareInt( + GeoGridAggregationBuilder::size, + FIELD_SIZE); + PARSER.declareInt( + GeoGridAggregationBuilder::shardSize, + FIELD_SHARD_SIZE); + + ValuesSourceParserHelper.declareGeoFields(PARSER, false, false); + } + + private static Object parsePrecision(XContentParser parser, String name) + throws IOException { + // Delay actual parsing until builder.precision() + // In some cases, this value cannot be fully parsed until after we know the type + final XContentParser.Token token = parser.currentToken(); + switch (token) { + case VALUE_NUMBER: + return parser.intValue(); + case VALUE_STRING: + return parser.text(); + default: + throw new XContentParseException(parser.getTokenLocation(), + "[geo_grid] failed to parse field [precision] in [" + name + + "]. It must be either an integer or a string"); + } + } + + public static GeoGridAggregationBuilder parse(String aggregationName, XContentParser parser) { + return PARSER.apply(parser, aggregationName); + } + + private final GeoGridType type; + private int precision; + private int requiredSize = DEFAULT_MAX_NUM_CELLS; + private int shardSize = -1; + + public GeoGridAggregationBuilder(String name, GeoGridType type) { + super(name, ValuesSourceType.GEOPOINT, ValueType.GEOPOINT); + + // TODO: FIXME: should there be a default? + this.type = type != null ? type : types.get(GeoHashType.NAME, name); + this.precision = this.type.getDefaultPrecision(); + } + + protected GeoGridAggregationBuilder(GeoGridAggregationBuilder clone, Builder factoriesBuilder, Map metaData) { + super(clone, factoriesBuilder, metaData); + this.type = clone.type; + this.precision = clone.precision; + this.requiredSize = clone.requiredSize; + this.shardSize = clone.shardSize; + } + + @Override + protected AggregationBuilder shallowCopy(Builder factoriesBuilder, Map metaData) { + return new GeoGridAggregationBuilder(this, factoriesBuilder, metaData); + } + + /** + * Read from a stream. + */ + public GeoGridAggregationBuilder(StreamInput in) throws IOException { + super(in, ValuesSourceType.GEOPOINT, ValueType.GEOPOINT); + + // FIXME: better debug name than a class name? + type = types.get(in.readString(), this.getClass().getName()); + precision = in.readVInt(); + requiredSize = in.readVInt(); + shardSize = in.readVInt(); + } + + @Override + protected void innerWriteTo(StreamOutput out) throws IOException { + out.writeString(type.getName()); + out.writeVInt(precision); + out.writeVInt(requiredSize); + out.writeVInt(shardSize); + } + + private static GeoGridType parseType(XContentParser parser, String name) throws IOException { + return types.get(parser.text(), name); + } + + public GeoGridType type() { + return type; + } + + private GeoGridAggregationBuilder precisionRaw(Object precision) { + if (precision == null) { + this.precision(type.getDefaultPrecision()); + } else if (precision instanceof String) { + this.precision(type.parsePrecisionString((String) precision)); + } else { + this.precision((int) precision); + } + return this; + } + + public GeoGridAggregationBuilder precision(int precision) { + this.precision = type.validatePrecision(precision); + return this; + } + + public int precision() { + return precision; + } + + public GeoGridAggregationBuilder size(int size) { + if (size <= 0) { + throw new IllegalArgumentException( + "[size] must be greater than 0. Found [" + size + "] in [" + name + "]"); + } + this.requiredSize = size; + return this; + } + + public int size() { + return requiredSize; + } + + public GeoGridAggregationBuilder shardSize(int shardSize) { + if (shardSize <= 0) { + throw new IllegalArgumentException( + "[shardSize] must be greater than 0. Found [" + shardSize + "] in [" + name + "]"); + } + this.shardSize = shardSize; + return this; + } + + public int shardSize() { + return shardSize; + } + + @Override + protected ValuesSourceAggregatorFactory innerBuild(SearchContext context, + ValuesSourceConfig config, AggregatorFactory parent, Builder subFactoriesBuilder) + throws IOException { + int shardSize = this.shardSize; + int requiredSize = this.requiredSize; + + if (shardSize < 0) { + // Use default heuristic to avoid any wrong-ranking caused by + // distributed counting + shardSize = BucketUtils.suggestShardSideQueueSize(requiredSize); + } + if (requiredSize <= 0 || shardSize <= 0) { + throw new ElasticsearchException( + "parameters [required_size] and [shard_size] must be >0 in geo_grid aggregation [" + name + "]."); + } + if (shardSize < requiredSize) { + shardSize = requiredSize; + } + return new GeoGridAggregatorFactory(name, config, type, precision, requiredSize, shardSize, context, parent, + subFactoriesBuilder, metaData); + } + + @Override + protected XContentBuilder doXContentBody(XContentBuilder builder, Params params) throws IOException { + builder.field(FIELD_TYPE.getPreferredName(), type); + builder.field(FIELD_PRECISION.getPreferredName(), precision); + builder.field(FIELD_SIZE.getPreferredName(), requiredSize); + if (shardSize > -1) { + builder.field(FIELD_SHARD_SIZE.getPreferredName(), shardSize); + } + return builder; + } + + @Override + protected boolean innerEquals(Object obj) { + GeoGridAggregationBuilder other = (GeoGridAggregationBuilder) obj; + if (type != other.type) { + return false; + } + if (precision != other.precision) { + return false; + } + if (requiredSize != other.requiredSize) { + return false; + } + if (shardSize != other.shardSize) { + return false; + } + return true; + } + + @Override + protected int innerHashCode() { + return Objects.hash(type, precision, requiredSize, shardSize); + } + + @Override + public String getType() { + return NAME; + } + + private static class CellValues extends AbstractSortingNumericDocValues { + private MultiGeoPointValues geoValues; + private GeoGridType type; + private int precision; + + protected CellValues(MultiGeoPointValues geoValues, GeoGridType type, int precision) { + this.geoValues = geoValues; + this.type = type; + this.precision = precision; + } + + @Override + public boolean advanceExact(int docId) throws IOException { + if (geoValues.advanceExact(docId)) { + resize(geoValues.docValueCount()); + + for (int i = 0; i < docValueCount(); ++i) { + GeoPoint target = geoValues.nextValue(); + values[i] = type.calculateHash(target.getLon(), target.getLat(), precision); + } + sort(); + return true; + } else { + return false; + } + } + } + + static class CellIdSource extends ValuesSource.Numeric { + private final ValuesSource.GeoPoint valuesSource; + private final GeoGridType type; + private final int precision; + + CellIdSource(ValuesSource.GeoPoint valuesSource, GeoGridType type, int precision) { + this.valuesSource = valuesSource; + //different GeoPoints could map to the same or different geogrid cells. + this.type = type; + this.precision = precision; + } + + public GeoGridType type() { + return type; + } + + public int precision() { + return precision; + } + + @Override + public boolean isFloatingPoint() { + return false; + } + + @Override + public SortedNumericDocValues longValues(LeafReaderContext ctx) { + return new CellValues(valuesSource.geoPointValues(ctx), type, precision); + } + + @Override + public SortedNumericDoubleValues doubleValues(LeafReaderContext ctx) { + throw new UnsupportedOperationException(); + } + + @Override + public SortedBinaryDocValues bytesValues(LeafReaderContext ctx) { + throw new UnsupportedOperationException(); + } + + } +} diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/geogrid2/GeoGridAggregator.java b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/geogrid2/GeoGridAggregator.java new file mode 100644 index 0000000000000..6052142dcdc7f --- /dev/null +++ b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/geogrid2/GeoGridAggregator.java @@ -0,0 +1,149 @@ +/* + * 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.aggregations.bucket.geogrid2; + +import org.apache.lucene.index.LeafReaderContext; +import org.apache.lucene.index.SortedNumericDocValues; +import org.apache.lucene.search.ScoreMode; +import org.elasticsearch.common.lease.Releasables; +import org.elasticsearch.common.util.LongHash; +import org.elasticsearch.search.aggregations.Aggregator; +import org.elasticsearch.search.aggregations.AggregatorFactories; +import org.elasticsearch.search.aggregations.LeafBucketCollector; +import org.elasticsearch.search.aggregations.LeafBucketCollectorBase; +import org.elasticsearch.search.aggregations.bucket.BucketsAggregator; +import org.elasticsearch.search.aggregations.pipeline.PipelineAggregator; +import org.elasticsearch.search.internal.SearchContext; + +import java.io.IOException; +import java.util.Arrays; +import java.util.Collections; +import java.util.List; +import java.util.Map; + +/** + * Aggregates data expressed as Geogrid hash longs (for efficiency's sake) but formats results as geo grid strings. + */ +public class GeoGridAggregator extends BucketsAggregator { + + private final int requiredSize; + private final int shardSize; + private final GeoGridAggregationBuilder.CellIdSource valuesSource; + private final LongHash bucketOrds; + private final GeoGridType type; + + GeoGridAggregator(String name, AggregatorFactories factories, GeoGridAggregationBuilder.CellIdSource valuesSource, + int requiredSize, int shardSize, SearchContext aggregationContext, Aggregator parent, List pipelineAggregators, + Map metaData, GeoGridType type) throws IOException { + super(name, factories, aggregationContext, parent, pipelineAggregators, metaData); + this.valuesSource = valuesSource; + this.requiredSize = requiredSize; + this.shardSize = shardSize; + this.type = type; + bucketOrds = new LongHash(1, aggregationContext.bigArrays()); + } + + @Override + public ScoreMode scoreMode() { + if (valuesSource != null && valuesSource.needsScores()) { + return ScoreMode.COMPLETE; + } + return super.scoreMode(); + } + + @Override + public LeafBucketCollector getLeafCollector(LeafReaderContext ctx, + final LeafBucketCollector sub) throws IOException { + final SortedNumericDocValues values = valuesSource.longValues(ctx); + return new LeafBucketCollectorBase(sub, null) { + @Override + public void collect(int doc, long bucket) throws IOException { + assert bucket == 0; + if (values.advanceExact(doc)) { + final int valuesCount = values.docValueCount(); + + long previous = Long.MAX_VALUE; + for (int i = 0; i < valuesCount; ++i) { + final long val = values.nextValue(); + if (previous != val || i == 0) { + long bucketOrdinal = bucketOrds.add(val); + if (bucketOrdinal < 0) { // already seen + bucketOrdinal = -1 - bucketOrdinal; + collectExistingBucket(sub, doc, bucketOrdinal); + } else { + collectBucket(sub, doc, bucketOrdinal); + } + previous = val; + } + } + } + } + }; + } + + // private impl that stores a bucket ord. This allows for computing the aggregations lazily. + static class OrdinalBucket extends InternalGeoGrid.Bucket { + + long bucketOrd; + + OrdinalBucket() { + super(0, 0, null); + } + + } + + @Override + public InternalGeoGrid buildAggregation(long owningBucketOrdinal) throws IOException { + assert owningBucketOrdinal == 0; + final int size = (int) Math.min(bucketOrds.size(), shardSize); + consumeBucketsAndMaybeBreak(size); + + InternalGeoGrid.BucketPriorityQueue ordered = new InternalGeoGrid.BucketPriorityQueue(size); + OrdinalBucket spare = null; + for (long i = 0; i < bucketOrds.size(); i++) { + if (spare == null) { + spare = new OrdinalBucket(); + } + + spare.hashAsLong = bucketOrds.get(i); + spare.docCount = bucketDocCount(i); + spare.bucketOrd = i; + spare = (OrdinalBucket) ordered.insertWithOverflow(spare); + } + + final InternalGeoGrid.Bucket[] list = new InternalGeoGrid.Bucket[ordered.size()]; + for (int i = ordered.size() - 1; i >= 0; --i) { + final OrdinalBucket bucket = (OrdinalBucket) ordered.pop(); + bucket.aggregations = bucketAggregations(bucket.bucketOrd); + list[i] = bucket; + } + return new InternalGeoGrid(name, type, requiredSize, Arrays.asList(list), pipelineAggregators(), metaData()); + } + + @Override + public InternalGeoGrid buildEmptyAggregation() { + return new InternalGeoGrid(name, type, requiredSize, Collections.emptyList(), pipelineAggregators(), metaData()); + } + + @Override + public void doClose() { + Releasables.close(bucketOrds); + } + +} diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/geogrid2/GeoGridAggregatorFactory.java b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/geogrid2/GeoGridAggregatorFactory.java new file mode 100644 index 0000000000000..94b601eb646d5 --- /dev/null +++ b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/geogrid2/GeoGridAggregatorFactory.java @@ -0,0 +1,82 @@ +/* + * 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.aggregations.bucket.geogrid2; + +import org.elasticsearch.search.aggregations.Aggregator; +import org.elasticsearch.search.aggregations.AggregatorFactories; +import org.elasticsearch.search.aggregations.AggregatorFactory; +import org.elasticsearch.search.aggregations.InternalAggregation; +import org.elasticsearch.search.aggregations.NonCollectingAggregator; +import org.elasticsearch.search.aggregations.bucket.geogrid2.GeoGridAggregationBuilder.CellIdSource; +import org.elasticsearch.search.aggregations.pipeline.PipelineAggregator; +import org.elasticsearch.search.aggregations.support.ValuesSource; +import org.elasticsearch.search.aggregations.support.ValuesSource.GeoPoint; +import org.elasticsearch.search.aggregations.support.ValuesSourceAggregatorFactory; +import org.elasticsearch.search.aggregations.support.ValuesSourceConfig; +import org.elasticsearch.search.internal.SearchContext; + +import java.io.IOException; +import java.util.Collections; +import java.util.List; +import java.util.Map; + +public class GeoGridAggregatorFactory extends ValuesSourceAggregatorFactory { + + private final GeoGridType type; + private final int precision; + private final int requiredSize; + private final int shardSize; + + GeoGridAggregatorFactory(String name, ValuesSourceConfig config, GeoGridType type, int precision, int requiredSize, + int shardSize, SearchContext context, AggregatorFactory parent, AggregatorFactories.Builder subFactoriesBuilder, + Map metaData) throws IOException { + super(name, config, context, parent, subFactoriesBuilder, metaData); + this.type = type; + this.precision = precision; + this.requiredSize = requiredSize; + this.shardSize = shardSize; + } + + @Override + protected Aggregator createUnmapped(Aggregator parent, List pipelineAggregators, Map metaData) + throws IOException { + final InternalAggregation aggregation = new InternalGeoGrid(name, type, requiredSize, + Collections. emptyList(), pipelineAggregators, metaData); + return new NonCollectingAggregator(name, context, parent, pipelineAggregators, metaData) { + @Override + public InternalAggregation buildEmptyAggregation() { + return aggregation; + } + }; + } + + @Override + protected Aggregator doCreateInternal(final ValuesSource.GeoPoint valuesSource, Aggregator parent, boolean collectsFromSingleBucket, + List pipelineAggregators, Map metaData) throws IOException { + if (collectsFromSingleBucket == false) { + return asMultiBucketAggregator(this, context, parent); + } + CellIdSource cellIdSource = new CellIdSource(valuesSource, type, precision); + return new GeoGridAggregator(name, factories, cellIdSource, requiredSize, shardSize, context, parent, + pipelineAggregators, metaData, type); + + } + +} diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/geogrid2/GeoGridType.java b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/geogrid2/GeoGridType.java new file mode 100644 index 0000000000000..fd4185c9f37f3 --- /dev/null +++ b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/geogrid2/GeoGridType.java @@ -0,0 +1,62 @@ +/* + * 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.aggregations.bucket.geogrid2; + +import org.elasticsearch.common.geo.GeoPoint; + +/** + * Instances implement different hashing algorithms for geo-grid aggregations + */ +public interface GeoGridType { + /** + * Returns the name of the grid aggregation, e.g. "geohash" + */ + String getName(); + + /** + * Returns default precision for the type, e.g. 5 for geohash + */ + int getDefaultPrecision(); + + /** + * Parses precision string into an integer, e.g. "100km" into 4 + */ + int parsePrecisionString(String precision); + + /** + * Validates precision for the given geo type, and throws an exception on error + * @param precision value to validate + * @return the original value if everything is ok + */ + int validatePrecision(int precision); + + /** + * Converts longitude/latitude into a bucket identifying hash value with the given precision + * @return hash value + */ + long calculateHash(double longitude, double latitude, int precision); + + /** + * Decodes hash value into a string returned to the user + * @param hash as generated by the {@link #calculateHash} + * @return bucket ID as a string + */ + String hashAsString(long hash); +} diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/geogrid2/GeoGridTypes.java b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/geogrid2/GeoGridTypes.java new file mode 100644 index 0000000000000..b43feda43f980 --- /dev/null +++ b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/geogrid2/GeoGridTypes.java @@ -0,0 +1,51 @@ +/* + * 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.aggregations.bucket.geogrid2; + +import java.util.Map; +import java.util.HashMap; + +/** + * Store for types - needed in multiple classes + */ +public class GeoGridTypes { + + public static GeoGridTypes DEFAULT = new GeoGridTypes(); + + private Map types; + + private GeoGridTypes() { + // TODO: we need to decide how types map is instantiated/stored + // TODO: especially this is important to allow type plugins + types = new HashMap<>(); + final GeoGridType type = new GeoHashType(); + types.put(type.getName(), type); + } + + public GeoGridType get(String typeStr, String name) { + final GeoGridType type = types.get(typeStr); + if (type != null) { + return type; + } + throw new IllegalArgumentException( + "[type] is not valid. Allowed values: " + + String.join(", ", types.keySet()) + + ". Found [" + typeStr + "] in [" + name + "]"); + } +} diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/geogrid2/GeoHashType.java b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/geogrid2/GeoHashType.java new file mode 100644 index 0000000000000..f9a98499896bc --- /dev/null +++ b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/geogrid2/GeoHashType.java @@ -0,0 +1,60 @@ +/* + * 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.aggregations.bucket.geogrid2; + +import org.elasticsearch.common.geo.GeoHashUtils; +import org.elasticsearch.common.geo.GeoUtils; + +/** + * A simple wrapper for GeoUtils handling of the geohash hashing algorithm + */ +public class GeoHashType implements GeoGridType { + static final String NAME = "geohash"; + + @Override + public String getName() { + return NAME; + } + + @Override + public int getDefaultPrecision() { + return 5; + } + + @Override + public int parsePrecisionString(String precision) { + return GeoUtils.parsePrecisionString(precision); + } + + @Override + public int validatePrecision(int precision) { + return GeoUtils.checkPrecisionRange(precision); + } + + @Override + public long calculateHash(double longitude, double latitude, int precision) { + return GeoHashUtils.longEncode(longitude, latitude, precision); + } + + @Override + public String hashAsString(long hash) { + return GeoHashUtils.stringEncode(hash); + } +} diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/geogrid2/InternalGeoGrid.java b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/geogrid2/InternalGeoGrid.java new file mode 100644 index 0000000000000..1d599fc99e9b7 --- /dev/null +++ b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/geogrid2/InternalGeoGrid.java @@ -0,0 +1,300 @@ +/* + * 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.aggregations.bucket.geogrid2; + +import org.apache.lucene.util.PriorityQueue; +import org.elasticsearch.common.geo.GeoPoint; +import org.elasticsearch.common.io.stream.StreamInput; +import org.elasticsearch.common.io.stream.StreamOutput; +import org.elasticsearch.common.util.LongObjectPagedHashMap; +import org.elasticsearch.common.xcontent.XContentBuilder; +import org.elasticsearch.search.aggregations.Aggregations; +import org.elasticsearch.search.aggregations.InternalAggregation; +import org.elasticsearch.search.aggregations.InternalAggregations; +import org.elasticsearch.search.aggregations.InternalMultiBucketAggregation; +import org.elasticsearch.search.aggregations.pipeline.PipelineAggregator; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.List; +import java.util.Map; +import java.util.Objects; + +import static java.util.Collections.unmodifiableList; + +/** + * Represents a grid of cells where each cell's location is determined by a hash. + * All hashes in a grid are of the same precision and held internally as a single long + * for efficiency's sake. + */ +public class InternalGeoGrid extends InternalMultiBucketAggregation implements + GeoGrid { + static class Bucket extends InternalMultiBucketAggregation.InternalBucket implements GeoGrid.Bucket, Comparable { + + protected long hashAsLong; + protected long docCount; + protected InternalAggregations aggregations; + + Bucket(long hashAsLong, long docCount, InternalAggregations aggregations) { + this.docCount = docCount; + this.aggregations = aggregations; + this.hashAsLong = hashAsLong; + } + + /** + * Read from a stream. + */ + private Bucket(StreamInput in) throws IOException { + hashAsLong = in.readLong(); + docCount = in.readVLong(); + aggregations = InternalAggregations.readAggregations(in); + } + + @Override + public void writeTo(StreamOutput out) throws IOException { + out.writeLong(hashAsLong); + out.writeVLong(docCount); + aggregations.writeTo(out); + } + + @Override + public String getKeyAsString() { + throw new IllegalArgumentException(); // FIXME: better ex? Also, any way to structure this better? + } + + @Override + public String getKey() { + throw new IllegalArgumentException(); // FIXME: better ex? Also, any way to structure this better? + } + + @Override + public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException { + throw new IllegalArgumentException(); // FIXME: better ex? Also, any way to structure this better? + } + + public String getKeyAsString2(GeoGridType type) { + try { // for some reason breakpoints don't work for nested classes, hacking around that :) + throw new IOException("DELETEME"); + } catch (IOException ex) { + // ignore + } + // FIXME! + // We either need to have one bucket type per provider, or this method could return hash as a string, + // and we can convert it to a user-facing string before sending it to the user + return type.hashAsString(hashAsLong); +// return type.getHandler().hashAsString(hashAsLong); + } + + @Override + public long getDocCount() { + return docCount; + } + + @Override + public Aggregations getAggregations() { + return aggregations; + } + + @Override + public int compareTo(Bucket other) { + if (this.hashAsLong > other.hashAsLong) { + return 1; + } + if (this.hashAsLong < other.hashAsLong) { + return -1; + } + return 0; + } + + public Bucket reduce(List buckets, ReduceContext context) { + List aggregationsList = new ArrayList<>(buckets.size()); + long docCount = 0; + for (Bucket bucket : buckets) { + docCount += bucket.docCount; + aggregationsList.add(bucket.aggregations); + } + final InternalAggregations aggs = InternalAggregations.reduce(aggregationsList, context); + return new Bucket(hashAsLong, docCount, aggs); + } + + public XContentBuilder toXContent2(XContentBuilder builder, Params params, GeoGridType type) throws IOException { + try { // for some reason breakpoints don't work for nested classes, hacking around that :) + throw new IOException("DELETEME"); + } catch (IOException ex) { + // ignore + } + + builder.startObject(); + builder.field(CommonFields.KEY.getPreferredName(), getKeyAsString2(type)); + builder.field(CommonFields.DOC_COUNT.getPreferredName(), docCount); + aggregations.toXContentInternal(builder, params); + builder.endObject(); + return builder; + } + + @Override + public boolean equals(Object o) { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + Bucket bucket = (Bucket) o; + return hashAsLong == bucket.hashAsLong && + docCount == bucket.docCount && + Objects.equals(aggregations, bucket.aggregations); + } + + @Override + public int hashCode() { + return Objects.hash(hashAsLong, docCount, aggregations); + } + + } + + private final GeoGridType type; + private final int requiredSize; + private final List buckets; + + InternalGeoGrid(String name, GeoGridType type, int requiredSize, List buckets, List pipelineAggregators, + Map metaData) { + super(name, pipelineAggregators, metaData); + this.type = type; + this.requiredSize = requiredSize; + this.buckets = buckets; + } + + /** + * Read from a stream. + */ + public InternalGeoGrid(StreamInput in) throws IOException { + super(in); + type = GeoGridTypes.DEFAULT.get(in.readString(), "internal-worker"); + requiredSize = readSize(in); + buckets = in.readList(Bucket::new); + } + + @Override + protected void doWriteTo(StreamOutput out) throws IOException { + out.writeString(type.getName()); + writeSize(requiredSize, out); + out.writeList(buckets); + } + + @Override + public String getWriteableName() { + return GeoGridAggregationBuilder.NAME; + } + + @Override + public InternalGeoGrid create(List buckets) { + return new InternalGeoGrid(this.name, this.type, this.requiredSize, buckets, this.pipelineAggregators(), this.metaData); + } + + @Override + public Bucket createBucket(InternalAggregations aggregations, Bucket prototype) { + return new Bucket(prototype.hashAsLong, prototype.docCount, aggregations); + } + + @Override + public List getBuckets() { + return unmodifiableList(buckets); + } + + @Override + public InternalGeoGrid doReduce(List aggregations, ReduceContext reduceContext) { + LongObjectPagedHashMap> buckets = null; + for (InternalAggregation aggregation : aggregations) { + InternalGeoGrid grid = (InternalGeoGrid) aggregation; + if (buckets == null) { + buckets = new LongObjectPagedHashMap<>(grid.buckets.size(), reduceContext.bigArrays()); + } + for (Bucket bucket : grid.buckets) { + List existingBuckets = buckets.get(bucket.hashAsLong); + if (existingBuckets == null) { + existingBuckets = new ArrayList<>(aggregations.size()); + buckets.put(bucket.hashAsLong, existingBuckets); + } + existingBuckets.add(bucket); + } + } + + final int size = Math.toIntExact(reduceContext.isFinalReduce() == false ? buckets.size() : Math.min(requiredSize, buckets.size())); + BucketPriorityQueue ordered = new BucketPriorityQueue(size); + for (LongObjectPagedHashMap.Cursor> cursor : buckets) { + List sameCellBuckets = cursor.value; + Bucket removed = ordered.insertWithOverflow(sameCellBuckets.get(0).reduce(sameCellBuckets, reduceContext)); + if (removed != null) { + reduceContext.consumeBucketsAndMaybeBreak(-countInnerBucket(removed)); + } else { + reduceContext.consumeBucketsAndMaybeBreak(1); + } + } + buckets.close(); + Bucket[] list = new Bucket[ordered.size()]; + for (int i = ordered.size() - 1; i >= 0; i--) { + list[i] = ordered.pop(); + } + return new InternalGeoGrid(getName(), type, requiredSize, Arrays.asList(list), pipelineAggregators(), getMetaData()); + } + + @Override + public XContentBuilder doXContentBody(XContentBuilder builder, Params params) throws IOException { + builder.startArray(CommonFields.BUCKETS.getPreferredName()); + for (Bucket bucket : buckets) { + bucket.toXContent2(builder, params, type); + } + builder.endArray(); + return builder; + } + + // package protected for testing + int getRequiredSize() { + return requiredSize; + } + + @Override + protected int doHashCode() { + return Objects.hash(requiredSize, buckets); + } + + @Override + protected boolean doEquals(Object obj) { + InternalGeoGrid other = (InternalGeoGrid) obj; + return Objects.equals(requiredSize, other.requiredSize) && + Objects.equals(buckets, other.buckets); + } + + static class BucketPriorityQueue extends PriorityQueue { + + BucketPriorityQueue(int size) { + super(size); + } + + @Override + protected boolean lessThan(Bucket o1, Bucket o2) { + int cmp = Long.compare(o2.getDocCount(), o1.getDocCount()); + if (cmp == 0) { + cmp = o2.compareTo(o1); + if (cmp == 0) { + cmp = System.identityHashCode(o2) - System.identityHashCode(o1); + } + } + return cmp > 0; + } + } +} diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/geogrid2/ParsedGeoGrid.java b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/geogrid2/ParsedGeoGrid.java new file mode 100644 index 0000000000000..4432b8a165fcd --- /dev/null +++ b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/geogrid2/ParsedGeoGrid.java @@ -0,0 +1,78 @@ +/* + * 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.aggregations.bucket.geogrid2; + +import org.elasticsearch.common.geo.GeoPoint; +import org.elasticsearch.common.xcontent.ObjectParser; +import org.elasticsearch.common.xcontent.XContentBuilder; +import org.elasticsearch.common.xcontent.XContentParser; +import org.elasticsearch.search.aggregations.ParsedMultiBucketAggregation; + +import java.io.IOException; +import java.util.List; + +public class ParsedGeoGrid extends ParsedMultiBucketAggregation implements GeoGrid { + + @Override + public String getType() { + return GeoGridAggregationBuilder.NAME; + } + + @Override + public List getBuckets() { + return buckets; + } + + private static ObjectParser PARSER = + new ObjectParser<>(ParsedGeoGrid.class.getSimpleName(), true, ParsedGeoGrid::new); + static { + declareMultiBucketAggregationFields(PARSER, ParsedBucket::fromXContent, ParsedBucket::fromXContent); + } + + public static ParsedGeoGrid fromXContent(XContentParser parser, String name) throws IOException { + ParsedGeoGrid aggregation = PARSER.parse(parser, null); + aggregation.setName(name); + return aggregation; + } + + public static class ParsedBucket extends ParsedMultiBucketAggregation.ParsedBucket implements GeoGrid.Bucket { + + private String geohashAsString; + + @Override + public GeoPoint getKey() { + return GeoPoint.fromGeohash(geohashAsString); + } + + @Override + public String getKeyAsString() { + return geohashAsString; + } + + @Override + protected XContentBuilder keyToXContent(XContentBuilder builder) throws IOException { + return builder.field(CommonFields.KEY.getPreferredName(), geohashAsString); + } + + static ParsedBucket fromXContent(XContentParser parser) throws IOException { + return parseXContent(parser, false, ParsedBucket::new, (p, bucket) -> bucket.geohashAsString = p.textOrNull()); + } + } +} From 5f2fbedd8c7570bcc2ea74f38dce0b1736a85cee Mon Sep 17 00:00:00 2001 From: Marios Trivyzas Date: Tue, 8 Jan 2019 23:56:00 +0200 Subject: [PATCH 09/46] SQL: Replace String.format() with LoggerMessageFormat.format() (#37216) Fixes: #36532 --- .../sql/jdbc/JdbcPreparedStatementTests.java | 16 +++++++--------- .../xpack/sql/analysis/AnalysisException.java | 4 ++-- .../sql/expression/function/scalar/Cast.java | 6 +++--- .../xpack/sql/expression/gen/script/Param.java | 4 ++-- .../predicate/operator/comparison/In.java | 6 +++--- .../xpack/sql/parser/ParsingException.java | 4 ++-- .../xpack/sql/planner/FoldingException.java | 4 ++-- .../sql/analysis/index/IndexResolverTests.java | 9 +++++---- .../xpack/sql/expression/QuotingTests.java | 4 ++-- .../sql/parser/LikeEscapingParsingTests.java | 8 ++++---- 10 files changed, 32 insertions(+), 33 deletions(-) diff --git a/x-pack/plugin/sql/jdbc/src/test/java/org/elasticsearch/xpack/sql/jdbc/JdbcPreparedStatementTests.java b/x-pack/plugin/sql/jdbc/src/test/java/org/elasticsearch/xpack/sql/jdbc/JdbcPreparedStatementTests.java index ef674d30da965..50143f729370f 100644 --- a/x-pack/plugin/sql/jdbc/src/test/java/org/elasticsearch/xpack/sql/jdbc/JdbcPreparedStatementTests.java +++ b/x-pack/plugin/sql/jdbc/src/test/java/org/elasticsearch/xpack/sql/jdbc/JdbcPreparedStatementTests.java @@ -5,10 +5,8 @@ */ package org.elasticsearch.xpack.sql.jdbc; +import org.elasticsearch.common.logging.LoggerMessageFormat; import org.elasticsearch.test.ESTestCase; -import org.elasticsearch.xpack.sql.jdbc.EsType; -import org.elasticsearch.xpack.sql.jdbc.JdbcConfiguration; -import org.elasticsearch.xpack.sql.jdbc.JdbcPreparedStatement; import java.net.URL; import java.nio.charset.StandardCharsets; @@ -287,12 +285,12 @@ public void testThrownExceptionsWhenSettingFloatValues() throws SQLException { Float floatNotInt = 5_155_000_000f; sqle = expectThrows(SQLException.class, () -> jps.setObject(1, floatNotInt, Types.INTEGER)); - assertEquals(String.format(Locale.ROOT, "Numeric %s out of range", - Long.toString(Math.round(floatNotInt.doubleValue()))), sqle.getMessage()); + assertEquals(LoggerMessageFormat.format("Numeric {} out of range", + Math.round(floatNotInt.doubleValue())), sqle.getMessage()); sqle = expectThrows(SQLException.class, () -> jps.setObject(1, floatNotInt, Types.SMALLINT)); - assertEquals(String.format(Locale.ROOT, "Numeric %s out of range", - Long.toString(Math.round(floatNotInt.doubleValue()))), sqle.getMessage()); + assertEquals(LoggerMessageFormat.format("Numeric {} out of range", + Math.round(floatNotInt.doubleValue())), sqle.getMessage()); } public void testSettingDoubleValues() throws SQLException { @@ -328,8 +326,8 @@ public void testThrownExceptionsWhenSettingDoubleValues() throws SQLException { Double doubleNotInt = 5_155_000_000d; sqle = expectThrows(SQLException.class, () -> jps.setObject(1, doubleNotInt, Types.INTEGER)); - assertEquals(String.format(Locale.ROOT, "Numeric %s out of range", - Long.toString(((Number) doubleNotInt).longValue())), sqle.getMessage()); + assertEquals(LoggerMessageFormat.format("Numeric {} out of range", + ((Number) doubleNotInt).longValue()), sqle.getMessage()); } public void testUnsupportedClasses() throws SQLException { diff --git a/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/analysis/AnalysisException.java b/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/analysis/AnalysisException.java index d85f17f6ab6c3..262d62814e164 100644 --- a/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/analysis/AnalysisException.java +++ b/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/analysis/AnalysisException.java @@ -10,7 +10,7 @@ import org.elasticsearch.xpack.sql.tree.Location; import org.elasticsearch.xpack.sql.tree.Node; -import java.util.Locale; +import static org.elasticsearch.common.logging.LoggerMessageFormat.format; public class AnalysisException extends ClientSqlException { @@ -54,6 +54,6 @@ public RestStatus status() { @Override public String getMessage() { - return String.format(Locale.ROOT, "line %s:%s: %s", getLineNumber(), getColumnNumber(), super.getMessage()); + return format("line {}:{}: {}", getLineNumber(), getColumnNumber(), super.getMessage()); } } diff --git a/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/expression/function/scalar/Cast.java b/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/expression/function/scalar/Cast.java index 7281476b1f9a3..04f9b13ff3329 100644 --- a/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/expression/function/scalar/Cast.java +++ b/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/expression/function/scalar/Cast.java @@ -9,15 +9,15 @@ import org.elasticsearch.xpack.sql.expression.Nullability; import org.elasticsearch.xpack.sql.expression.gen.processor.Processor; import org.elasticsearch.xpack.sql.expression.gen.script.ScriptTemplate; -import org.elasticsearch.xpack.sql.tree.Source; import org.elasticsearch.xpack.sql.tree.NodeInfo; +import org.elasticsearch.xpack.sql.tree.Source; import org.elasticsearch.xpack.sql.type.DataType; import org.elasticsearch.xpack.sql.type.DataTypeConversion; import org.elasticsearch.xpack.sql.type.DataTypes; -import java.util.Locale; import java.util.Objects; +import static org.elasticsearch.common.logging.LoggerMessageFormat.format; import static org.elasticsearch.xpack.sql.expression.gen.script.ParamsBuilder.paramsBuilder; public class Cast extends UnaryScalarFunction { @@ -86,7 +86,7 @@ protected Processor makeProcessor() { public ScriptTemplate asScript() { ScriptTemplate fieldAsScript = asScript(field()); return new ScriptTemplate( - formatTemplate(String.format(Locale.ROOT, "{sql}.cast(%s,{})", fieldAsScript.template())), + formatTemplate(format("{sql}.", "cast({},{})", fieldAsScript.template())), paramsBuilder() .script(fieldAsScript.params()) .variable(dataType.name()) diff --git a/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/expression/gen/script/Param.java b/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/expression/gen/script/Param.java index 579eacd036b0e..e8151ada18a9c 100644 --- a/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/expression/gen/script/Param.java +++ b/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/expression/gen/script/Param.java @@ -5,7 +5,7 @@ */ package org.elasticsearch.xpack.sql.expression.gen.script; -import java.util.Locale; +import static org.elasticsearch.common.logging.LoggerMessageFormat.format; abstract class Param { private final T value; @@ -22,6 +22,6 @@ T value() { @Override public String toString() { - return String.format(Locale.ROOT, "{%s=%s}", prefix(), value); + return format(null, "{{}={}}", prefix(), value); } } diff --git a/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/expression/predicate/operator/comparison/In.java b/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/expression/predicate/operator/comparison/In.java index bf2bddee68b5a..f9e6b72fb5958 100644 --- a/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/expression/predicate/operator/comparison/In.java +++ b/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/expression/predicate/operator/comparison/In.java @@ -12,19 +12,19 @@ import org.elasticsearch.xpack.sql.expression.function.scalar.ScalarFunction; import org.elasticsearch.xpack.sql.expression.gen.pipeline.Pipe; import org.elasticsearch.xpack.sql.expression.gen.script.ScriptTemplate; -import org.elasticsearch.xpack.sql.tree.Source; import org.elasticsearch.xpack.sql.tree.NodeInfo; +import org.elasticsearch.xpack.sql.tree.Source; import org.elasticsearch.xpack.sql.type.DataType; import org.elasticsearch.xpack.sql.util.CollectionUtils; import java.util.ArrayList; import java.util.LinkedHashSet; import java.util.List; -import java.util.Locale; import java.util.Objects; import java.util.StringJoiner; import java.util.stream.Collectors; +import static org.elasticsearch.common.logging.LoggerMessageFormat.format; import static org.elasticsearch.xpack.sql.expression.gen.script.ParamsBuilder.paramsBuilder; public class In extends ScalarFunction { @@ -100,7 +100,7 @@ public ScriptTemplate asScript() { List values = new ArrayList<>(new LinkedHashSet<>(Foldables.valuesOf(list, value.dataType()))); return new ScriptTemplate( - formatTemplate(String.format(Locale.ROOT, "{sql}.in(%s, {})", leftScript.template())), + formatTemplate(format("{sql}.","in({}, {})", leftScript.template())), paramsBuilder() .script(leftScript.params()) .variable(values) diff --git a/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/parser/ParsingException.java b/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/parser/ParsingException.java index 1a6f2ff6e3526..a4a2a8df94089 100644 --- a/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/parser/ParsingException.java +++ b/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/parser/ParsingException.java @@ -9,7 +9,7 @@ import org.elasticsearch.xpack.sql.ClientSqlException; import org.elasticsearch.xpack.sql.tree.Source; -import java.util.Locale; +import static org.elasticsearch.common.logging.LoggerMessageFormat.format; public class ParsingException extends ClientSqlException { private final int line; @@ -56,6 +56,6 @@ public RestStatus status() { @Override public String getMessage() { - return String.format(Locale.ROOT, "line %s:%s: %s", getLineNumber(), getColumnNumber(), getErrorMessage()); + return format("line {}:{}: {}", getLineNumber(), getColumnNumber(), getErrorMessage()); } } diff --git a/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/planner/FoldingException.java b/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/planner/FoldingException.java index 654e20866e095..7cdd26d540432 100644 --- a/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/planner/FoldingException.java +++ b/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/planner/FoldingException.java @@ -10,7 +10,7 @@ import org.elasticsearch.xpack.sql.tree.Location; import org.elasticsearch.xpack.sql.tree.Node; -import java.util.Locale; +import static org.elasticsearch.common.logging.LoggerMessageFormat.format; public class FoldingException extends ClientSqlException { @@ -54,6 +54,6 @@ public RestStatus status() { @Override public String getMessage() { - return String.format(Locale.ROOT, "line %s:%s: %s", getLineNumber(), getColumnNumber(), super.getMessage()); + return format("line {}:{}: {}", getLineNumber(), getColumnNumber(), super.getMessage()); } } diff --git a/x-pack/plugin/sql/src/test/java/org/elasticsearch/xpack/sql/analysis/index/IndexResolverTests.java b/x-pack/plugin/sql/src/test/java/org/elasticsearch/xpack/sql/analysis/index/IndexResolverTests.java index 7ef57972b3131..b53d00cfbb71d 100644 --- a/x-pack/plugin/sql/src/test/java/org/elasticsearch/xpack/sql/analysis/index/IndexResolverTests.java +++ b/x-pack/plugin/sql/src/test/java/org/elasticsearch/xpack/sql/analysis/index/IndexResolverTests.java @@ -15,10 +15,11 @@ import java.util.ArrayList; import java.util.HashMap; import java.util.List; -import java.util.Locale; import java.util.Map; import java.util.Map.Entry; +import static org.elasticsearch.common.logging.LoggerMessageFormat.format; + public class IndexResolverTests extends ESTestCase { public void testMergeSameMapping() throws Exception { @@ -218,14 +219,14 @@ public String[] nonAggregatableIndices() { @Override public String toString() { - return String.format(Locale.ROOT, "%s,%s->%s", getName(), getType(), indices); + return format("{},{}->{}", getName(), getType(), indices); } } private static void assertEqualsMaps(Map left, Map right) { for (Entry entry : left.entrySet()) { V rv = right.get(entry.getKey()); - assertEquals(String.format(Locale.ROOT, "Key [%s] has different values", entry.getKey()), entry.getValue(), rv); + assertEquals(format("Key [{}] has different values", entry.getKey()), entry.getValue(), rv); } } @@ -235,4 +236,4 @@ private void addFieldCaps(Map> fieldCaps, cap.put(name, new FieldCapabilities(name, type, isSearchable, isAggregatable)); fieldCaps.put(name, cap); } -} \ No newline at end of file +} diff --git a/x-pack/plugin/sql/src/test/java/org/elasticsearch/xpack/sql/expression/QuotingTests.java b/x-pack/plugin/sql/src/test/java/org/elasticsearch/xpack/sql/expression/QuotingTests.java index e19f3409c0df1..a7cae7df4312b 100644 --- a/x-pack/plugin/sql/src/test/java/org/elasticsearch/xpack/sql/expression/QuotingTests.java +++ b/x-pack/plugin/sql/src/test/java/org/elasticsearch/xpack/sql/expression/QuotingTests.java @@ -14,7 +14,6 @@ import java.util.ArrayList; import java.util.List; -import java.util.Locale; import static org.hamcrest.Matchers.equalTo; import static org.hamcrest.Matchers.hasSize; @@ -22,6 +21,7 @@ import static org.hamcrest.Matchers.is; import static org.hamcrest.Matchers.nullValue; +import static org.elasticsearch.common.logging.LoggerMessageFormat.format; public class QuotingTests extends ESTestCase { @@ -48,7 +48,7 @@ public void testSingleQuoteLiteral() { public void testMultiSingleQuotedLiteral() { String first = "bucket"; String second = "head"; - Expression exp = new SqlParser().createExpression(String.format(Locale.ROOT, "'%s' '%s'", first, second)); + Expression exp = new SqlParser().createExpression(format(null, "'{}' '{}'", first, second)); assertThat(exp, instanceOf(Literal.class)); Literal l = (Literal) exp; assertThat(l.value(), equalTo(first + second)); diff --git a/x-pack/plugin/sql/src/test/java/org/elasticsearch/xpack/sql/parser/LikeEscapingParsingTests.java b/x-pack/plugin/sql/src/test/java/org/elasticsearch/xpack/sql/parser/LikeEscapingParsingTests.java index 9f8cae9e52918..5221f9695699d 100644 --- a/x-pack/plugin/sql/src/test/java/org/elasticsearch/xpack/sql/parser/LikeEscapingParsingTests.java +++ b/x-pack/plugin/sql/src/test/java/org/elasticsearch/xpack/sql/parser/LikeEscapingParsingTests.java @@ -12,19 +12,19 @@ import org.elasticsearch.xpack.sql.proto.SqlTypedParamValue; import org.elasticsearch.xpack.sql.type.DataType; -import java.util.Locale; - import static java.util.Collections.singletonList; import static org.hamcrest.Matchers.instanceOf; import static org.hamcrest.Matchers.is; +import static org.elasticsearch.common.logging.LoggerMessageFormat.format; + public class LikeEscapingParsingTests extends ESTestCase { private final SqlParser parser = new SqlParser(); private String error(String pattern) { ParsingException ex = expectThrows(ParsingException.class, - () -> parser.createExpression(String.format(Locale.ROOT, "exp LIKE %s", pattern))); + () -> parser.createExpression(format(null, "exp LIKE {}", pattern))); return ex.getMessage(); } @@ -35,7 +35,7 @@ private LikePattern like(String pattern) { if (parameterized) { exp = parser.createExpression("exp LIKE ?", singletonList(new SqlTypedParamValue(DataType.KEYWORD.esType, pattern))); } else { - exp = parser.createExpression(String.format(Locale.ROOT, "exp LIKE '%s'", pattern)); + exp = parser.createExpression(format(null, "exp LIKE '{}'", pattern)); } assertThat(exp, instanceOf(Like.class)); Like l = (Like) exp; From b6a00e02fb95ce109462ac83566fa3cd302e2cc0 Mon Sep 17 00:00:00 2001 From: Yuri Astrakhan Date: Tue, 8 Jan 2019 17:10:23 -0500 Subject: [PATCH 10/46] minor cleanups, renamed "hash_type" -> "type" --- .../elasticsearch/common/geo/GeoUtils.java | 1 - .../geogrid2/GeoGridAggregationBuilder.java | 12 +++---- .../bucket/geogrid2/GeoGridAggregator.java | 5 +-- .../geogrid2/GeoGridAggregatorFactory.java | 7 ++-- .../bucket/geogrid2/GeoGridType.java | 2 -- .../bucket/geogrid2/InternalGeoGrid.java | 36 +++++-------------- 6 files changed, 20 insertions(+), 43 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/common/geo/GeoUtils.java b/server/src/main/java/org/elasticsearch/common/geo/GeoUtils.java index befff8bafb6a7..7079bcc5b4606 100644 --- a/server/src/main/java/org/elasticsearch/common/geo/GeoUtils.java +++ b/server/src/main/java/org/elasticsearch/common/geo/GeoUtils.java @@ -32,7 +32,6 @@ import org.elasticsearch.common.xcontent.XContentParser; import org.elasticsearch.common.xcontent.XContentParser.Token; import org.elasticsearch.common.xcontent.json.JsonXContent; -import org.elasticsearch.common.xcontent.support.XContentMapValues; import org.elasticsearch.index.fielddata.FieldData; import org.elasticsearch.index.fielddata.GeoPointValues; import org.elasticsearch.index.fielddata.MultiGeoPointValues; diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/geogrid2/GeoGridAggregationBuilder.java b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/geogrid2/GeoGridAggregationBuilder.java index 88cfbf5575f8c..9e33a807b0e98 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/geogrid2/GeoGridAggregationBuilder.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/geogrid2/GeoGridAggregationBuilder.java @@ -50,11 +50,10 @@ import org.elasticsearch.search.internal.SearchContext; import java.io.IOException; -import java.util.HashMap; import java.util.Map; import java.util.Objects; -import static org.elasticsearch.common.xcontent.ConstructingObjectParser.optionalConstructorArg; +import static org.elasticsearch.common.xcontent.ConstructingObjectParser.constructorArg; public class GeoGridAggregationBuilder extends ValuesSourceAggregationBuilder implements MultiBucketAggregationBuilder { @@ -62,7 +61,7 @@ public class GeoGridAggregationBuilder extends ValuesSourceAggregationBuilder(GeoGridAggregationBuilder.NAME, false, (a, name) -> new GeoGridAggregationBuilder(name, (GeoGridType) a[0])); - // TODO: Should we require type param to be always present? PARSER.declareField( - optionalConstructorArg(), + constructorArg(), GeoGridAggregationBuilder::parseType, FIELD_TYPE, ObjectParser.ValueType.STRING); @@ -126,9 +124,7 @@ public static GeoGridAggregationBuilder parse(String aggregationName, XContentPa public GeoGridAggregationBuilder(String name, GeoGridType type) { super(name, ValuesSourceType.GEOPOINT, ValueType.GEOPOINT); - - // TODO: FIXME: should there be a default? - this.type = type != null ? type : types.get(GeoHashType.NAME, name); + this.type = type; this.precision = this.type.getDefaultPrecision(); } diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/geogrid2/GeoGridAggregator.java b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/geogrid2/GeoGridAggregator.java index 6052142dcdc7f..e13b5883dcd85 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/geogrid2/GeoGridAggregator.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/geogrid2/GeoGridAggregator.java @@ -49,8 +49,9 @@ public class GeoGridAggregator extends BucketsAggregator { private final GeoGridType type; GeoGridAggregator(String name, AggregatorFactories factories, GeoGridAggregationBuilder.CellIdSource valuesSource, - int requiredSize, int shardSize, SearchContext aggregationContext, Aggregator parent, List pipelineAggregators, - Map metaData, GeoGridType type) throws IOException { + int requiredSize, int shardSize, SearchContext aggregationContext, Aggregator parent, + List pipelineAggregators, Map metaData, GeoGridType type + ) throws IOException { super(name, factories, aggregationContext, parent, pipelineAggregators, metaData); this.valuesSource = valuesSource; this.requiredSize = requiredSize; diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/geogrid2/GeoGridAggregatorFactory.java b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/geogrid2/GeoGridAggregatorFactory.java index 94b601eb646d5..b1d67fc38b2f0 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/geogrid2/GeoGridAggregatorFactory.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/geogrid2/GeoGridAggregatorFactory.java @@ -44,9 +44,10 @@ public class GeoGridAggregatorFactory extends ValuesSourceAggregatorFactory config, GeoGridType type, int precision, int requiredSize, - int shardSize, SearchContext context, AggregatorFactory parent, AggregatorFactories.Builder subFactoriesBuilder, - Map metaData) throws IOException { + GeoGridAggregatorFactory(String name, ValuesSourceConfig config, GeoGridType type, int precision, + int requiredSize, int shardSize, SearchContext context, AggregatorFactory parent, + AggregatorFactories.Builder subFactoriesBuilder, Map metaData + ) throws IOException { super(name, config, context, parent, subFactoriesBuilder, metaData); this.type = type; this.precision = precision; diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/geogrid2/GeoGridType.java b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/geogrid2/GeoGridType.java index fd4185c9f37f3..a5eab70cab134 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/geogrid2/GeoGridType.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/geogrid2/GeoGridType.java @@ -19,8 +19,6 @@ package org.elasticsearch.search.aggregations.bucket.geogrid2; -import org.elasticsearch.common.geo.GeoPoint; - /** * Instances implement different hashing algorithms for geo-grid aggregations */ diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/geogrid2/InternalGeoGrid.java b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/geogrid2/InternalGeoGrid.java index 1d599fc99e9b7..8754759017ff4 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/geogrid2/InternalGeoGrid.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/geogrid2/InternalGeoGrid.java @@ -19,7 +19,6 @@ package org.elasticsearch.search.aggregations.bucket.geogrid2; import org.apache.lucene.util.PriorityQueue; -import org.elasticsearch.common.geo.GeoPoint; import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.io.stream.StreamOutput; import org.elasticsearch.common.util.LongObjectPagedHashMap; @@ -89,17 +88,15 @@ public XContentBuilder toXContent(XContentBuilder builder, Params params) throws throw new IllegalArgumentException(); // FIXME: better ex? Also, any way to structure this better? } - public String getKeyAsString2(GeoGridType type) { - try { // for some reason breakpoints don't work for nested classes, hacking around that :) - throw new IOException("DELETEME"); - } catch (IOException ex) { - // ignore - } - // FIXME! - // We either need to have one bucket type per provider, or this method could return hash as a string, - // and we can convert it to a user-facing string before sending it to the user - return type.hashAsString(hashAsLong); -// return type.getHandler().hashAsString(hashAsLong); + /** + * Serializing to XContent using a specific GeoGridType. + */ + void toXContent2(XContentBuilder builder, Params params, GeoGridType type) throws IOException { + builder.startObject(); + builder.field(CommonFields.KEY.getPreferredName(), type.hashAsString(hashAsLong)); + builder.field(CommonFields.DOC_COUNT.getPreferredName(), docCount); + aggregations.toXContentInternal(builder, params); + builder.endObject(); } @Override @@ -134,21 +131,6 @@ public Bucket reduce(List buckets, ReduceContext context) { return new Bucket(hashAsLong, docCount, aggs); } - public XContentBuilder toXContent2(XContentBuilder builder, Params params, GeoGridType type) throws IOException { - try { // for some reason breakpoints don't work for nested classes, hacking around that :) - throw new IOException("DELETEME"); - } catch (IOException ex) { - // ignore - } - - builder.startObject(); - builder.field(CommonFields.KEY.getPreferredName(), getKeyAsString2(type)); - builder.field(CommonFields.DOC_COUNT.getPreferredName(), docCount); - aggregations.toXContentInternal(builder, params); - builder.endObject(); - return builder; - } - @Override public boolean equals(Object o) { if (this == o) return true; From ec32e66088c0001b0d3475ef06223de51d48b999 Mon Sep 17 00:00:00 2001 From: Mayya Sharipova Date: Tue, 8 Jan 2019 21:46:41 -0500 Subject: [PATCH 11/46] Deprecate reference to _type in lookup queries (#37016) Relates to #35190 --- .../java/org/elasticsearch/client/CrudIT.java | 8 +- .../QueryDSLDocumentationTests.java | 12 +-- .../query-dsl/geo-shape-query.asciidoc | 7 +- docs/java-api/query-dsl/ids-query.asciidoc | 1 - .../query-dsl/geo-shape-query.asciidoc | 4 +- docs/reference/query-dsl/ids-query.asciidoc | 4 - docs/reference/query-dsl/mlt-query.asciidoc | 4 - docs/reference/query-dsl/terms-query.asciidoc | 4 - .../PercolatorFieldMapperTests.java | 2 +- .../test/search/170_terms_query.yml | 4 +- .../search/171_terms_query_with_types.yml | 59 ++++++++++++++ .../index/query/GeoShapeQueryBuilder.java | 53 ++++++++++--- .../index/query/IdsQueryBuilder.java | 22 +++++- .../index/query/MoreLikeThisQueryBuilder.java | 68 ++++++++++++++-- .../index/query/QueryBuilders.java | 41 ++++++++++ .../index/query/TermsQueryBuilder.java | 30 ++++++-- .../elasticsearch/indices/TermsLookup.java | 54 ++++++++++--- .../query/GeoShapeQueryBuilderTests.java | 34 +++++--- .../index/query/IdsQueryBuilderTests.java | 15 +++- .../query/LegacyGeoShapeFieldQueryTests.java | 3 +- .../query/MoreLikeThisQueryBuilderTests.java | 24 +++++- .../index/query/RandomQueryBuilder.java | 5 +- .../index/query/TermsQueryBuilderTests.java | 24 +++++- .../indices/TermsLookupTests.java | 29 +++++++ .../search/geo/GeoShapeQueryTests.java | 77 ++++++++++++++----- .../search/morelikethis/MoreLikeThisIT.java | 56 ++++++++++---- .../test/AbstractQueryTestCase.java | 4 +- 27 files changed, 520 insertions(+), 128 deletions(-) create mode 100644 rest-api-spec/src/main/resources/rest-api-spec/test/search/171_terms_query_with_types.yml diff --git a/client/rest-high-level/src/test/java/org/elasticsearch/client/CrudIT.java b/client/rest-high-level/src/test/java/org/elasticsearch/client/CrudIT.java index 59135204c5be1..bdfc3fe4383ee 100644 --- a/client/rest-high-level/src/test/java/org/elasticsearch/client/CrudIT.java +++ b/client/rest-high-level/src/test/java/org/elasticsearch/client/CrudIT.java @@ -875,7 +875,7 @@ public void testUpdateByQuery() throws Exception { // test1: create one doc in dest UpdateByQueryRequest updateByQueryRequest = new UpdateByQueryRequest(); updateByQueryRequest.indices(sourceIndex); - updateByQueryRequest.setQuery(new IdsQueryBuilder().addIds("1").types("_doc")); + updateByQueryRequest.setQuery(new IdsQueryBuilder().addIds("1")); updateByQueryRequest.setRefresh(true); BulkByScrollResponse bulkResponse = execute(updateByQueryRequest, highLevelClient()::updateByQuery, highLevelClient()::updateByQueryAsync); @@ -917,7 +917,7 @@ public void testUpdateByQuery() throws Exception { // test update-by-query rethrottling UpdateByQueryRequest updateByQueryRequest = new UpdateByQueryRequest(); updateByQueryRequest.indices(sourceIndex); - updateByQueryRequest.setQuery(new IdsQueryBuilder().addIds("1").types("_doc")); + updateByQueryRequest.setQuery(new IdsQueryBuilder().addIds("1")); updateByQueryRequest.setRefresh(true); // this following settings are supposed to halt reindexing after first document @@ -987,7 +987,7 @@ public void testDeleteByQuery() throws Exception { // test1: delete one doc DeleteByQueryRequest deleteByQueryRequest = new DeleteByQueryRequest(); deleteByQueryRequest.indices(sourceIndex); - deleteByQueryRequest.setQuery(new IdsQueryBuilder().addIds("1").types("_doc")); + deleteByQueryRequest.setQuery(new IdsQueryBuilder().addIds("1")); deleteByQueryRequest.setRefresh(true); BulkByScrollResponse bulkResponse = execute(deleteByQueryRequest, highLevelClient()::deleteByQuery, highLevelClient()::deleteByQueryAsync); @@ -1009,7 +1009,7 @@ public void testDeleteByQuery() throws Exception { // test delete-by-query rethrottling DeleteByQueryRequest deleteByQueryRequest = new DeleteByQueryRequest(); deleteByQueryRequest.indices(sourceIndex); - deleteByQueryRequest.setQuery(new IdsQueryBuilder().addIds("2", "3").types("_doc")); + deleteByQueryRequest.setQuery(new IdsQueryBuilder().addIds("2", "3")); deleteByQueryRequest.setRefresh(true); // this following settings are supposed to halt reindexing after first document diff --git a/client/rest-high-level/src/test/java/org/elasticsearch/client/documentation/QueryDSLDocumentationTests.java b/client/rest-high-level/src/test/java/org/elasticsearch/client/documentation/QueryDSLDocumentationTests.java index 789d237c5a3bc..cfe9e98f643e6 100644 --- a/client/rest-high-level/src/test/java/org/elasticsearch/client/documentation/QueryDSLDocumentationTests.java +++ b/client/rest-high-level/src/test/java/org/elasticsearch/client/documentation/QueryDSLDocumentationTests.java @@ -207,11 +207,10 @@ public void testGeoShape() throws IOException { // Using pre-indexed shapes GeoShapeQueryBuilder qb = geoShapeQuery( "pin.location", // <1> - "DEU", // <2> - "countries"); // <3> - qb.relation(ShapeRelation.WITHIN) // <4> - .indexedShapeIndex("shapes") // <5> - .indexedShapePath("location"); // <6> + "DEU"); // <2> + qb.relation(ShapeRelation.WITHIN) // <3> + .indexedShapeIndex("shapes") // <4> + .indexedShapePath("location"); // <5> // end::indexed_geo_shape } } @@ -236,9 +235,6 @@ public void testHasParent() { public void testIds() { // tag::ids - idsQuery("my_type", "type2") - .addIds("1", "4", "100"); - idsQuery() // <1> .addIds("1", "4", "100"); // end::ids diff --git a/docs/java-api/query-dsl/geo-shape-query.asciidoc b/docs/java-api/query-dsl/geo-shape-query.asciidoc index 803f1849b5cdf..c2cd4c14e3adc 100644 --- a/docs/java-api/query-dsl/geo-shape-query.asciidoc +++ b/docs/java-api/query-dsl/geo-shape-query.asciidoc @@ -51,7 +51,6 @@ include-tagged::{query-dsl-test}[indexed_geo_shape] -------------------------------------------------- <1> field <2> The ID of the document that containing the pre-indexed shape. -<3> Index type where the pre-indexed shape is. -<4> relation -<5> Name of the index where the pre-indexed shape is. Defaults to 'shapes'. -<6> The field specified as path containing the pre-indexed shape. Defaults to 'shape'. +<3> relation +<4> Name of the index where the pre-indexed shape is. Defaults to 'shapes'. +<5> The field specified as path containing the pre-indexed shape. Defaults to 'shape'. diff --git a/docs/java-api/query-dsl/ids-query.asciidoc b/docs/java-api/query-dsl/ids-query.asciidoc index 9abc8ed9fed7c..ba12a5df38b0e 100644 --- a/docs/java-api/query-dsl/ids-query.asciidoc +++ b/docs/java-api/query-dsl/ids-query.asciidoc @@ -8,4 +8,3 @@ See {ref}/query-dsl-ids-query.html[Ids Query] -------------------------------------------------- include-tagged::{query-dsl-test}[ids] -------------------------------------------------- -<1> type is optional diff --git a/docs/reference/query-dsl/geo-shape-query.asciidoc b/docs/reference/query-dsl/geo-shape-query.asciidoc index f796881d520c6..059d0db14b51a 100644 --- a/docs/reference/query-dsl/geo-shape-query.asciidoc +++ b/docs/reference/query-dsl/geo-shape-query.asciidoc @@ -81,7 +81,7 @@ GET /example/_search ==== Pre-Indexed Shape The Query also supports using a shape which has already been indexed in -another index and/or index type. This is particularly useful for when +another index. This is particularly useful for when you have a pre-defined list of shapes which are useful to your application and you want to reference this using a logical name (for example 'New Zealand') rather than having to provide their coordinates @@ -90,7 +90,6 @@ each time. In this situation it is only necessary to provide: * `id` - The ID of the document that containing the pre-indexed shape. * `index` - Name of the index where the pre-indexed shape is. Defaults to 'shapes'. -* `type` - Index type where the pre-indexed shape is. * `path` - The field specified as path containing the pre-indexed shape. Defaults to 'shape'. * `routing` - The routing of the shape document if required. @@ -130,7 +129,6 @@ GET /example/_search "location": { "indexed_shape": { "index": "shapes", - "type": "_doc", "id": "deu", "path": "location" } diff --git a/docs/reference/query-dsl/ids-query.asciidoc b/docs/reference/query-dsl/ids-query.asciidoc index 55adcb8f94cf8..8798a2fb093f8 100644 --- a/docs/reference/query-dsl/ids-query.asciidoc +++ b/docs/reference/query-dsl/ids-query.asciidoc @@ -10,13 +10,9 @@ GET /_search { "query": { "ids" : { - "type" : "_doc", "values" : ["1", "4", "100"] } } } -------------------------------------------------- // CONSOLE - -The `type` is optional and can be omitted, and can also accept an array -of values. If no type is specified, all types defined in the index mapping are tried. diff --git a/docs/reference/query-dsl/mlt-query.asciidoc b/docs/reference/query-dsl/mlt-query.asciidoc index 19035d96ae04d..64a2a6052df71 100644 --- a/docs/reference/query-dsl/mlt-query.asciidoc +++ b/docs/reference/query-dsl/mlt-query.asciidoc @@ -42,12 +42,10 @@ GET /_search "like" : [ { "_index" : "imdb", - "_type" : "movies", "_id" : "1" }, { "_index" : "imdb", - "_type" : "movies", "_id" : "2" }, "and potentially some more text here as well" @@ -74,7 +72,6 @@ GET /_search "like" : [ { "_index" : "marvel", - "_type" : "quotes", "doc" : { "name": { "first": "Ben", @@ -85,7 +82,6 @@ GET /_search }, { "_index" : "marvel", - "_type" : "quotes", "_id" : "2" } ], diff --git a/docs/reference/query-dsl/terms-query.asciidoc b/docs/reference/query-dsl/terms-query.asciidoc index c0e94900d7d82..db4597fbea504 100644 --- a/docs/reference/query-dsl/terms-query.asciidoc +++ b/docs/reference/query-dsl/terms-query.asciidoc @@ -36,9 +36,6 @@ The terms lookup mechanism supports the following options: `index`:: The index to fetch the term values from. -`type`:: - The type to fetch the term values from. - `id`:: The id of the document to fetch the term values from. @@ -93,7 +90,6 @@ GET /tweets/_search "terms" : { "user" : { "index" : "users", - "type" : "_doc", "id" : "2", "path" : "followers" } diff --git a/modules/percolator/src/test/java/org/elasticsearch/percolator/PercolatorFieldMapperTests.java b/modules/percolator/src/test/java/org/elasticsearch/percolator/PercolatorFieldMapperTests.java index 3030e48690fe9..f1747d1977561 100644 --- a/modules/percolator/src/test/java/org/elasticsearch/percolator/PercolatorFieldMapperTests.java +++ b/modules/percolator/src/test/java/org/elasticsearch/percolator/PercolatorFieldMapperTests.java @@ -537,7 +537,7 @@ public void testStoringQueries() throws Exception { public void testQueryWithRewrite() throws Exception { addQueryFieldMappings(); client().prepareIndex("remote", "doc", "1").setSource("field", "value").get(); - QueryBuilder queryBuilder = termsLookupQuery("field", new TermsLookup("remote", "doc", "1", "field")); + QueryBuilder queryBuilder = termsLookupQuery("field", new TermsLookup("remote", "1", "field")); ParsedDocument doc = mapperService.documentMapper("doc").parse(new SourceToParse("test", "doc", "1", BytesReference.bytes(XContentFactory .jsonBuilder() diff --git a/rest-api-spec/src/main/resources/rest-api-spec/test/search/170_terms_query.yml b/rest-api-spec/src/main/resources/rest-api-spec/test/search/170_terms_query.yml index 515dcfe463069..3966a6a182a62 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/test/search/170_terms_query.yml +++ b/rest-api-spec/src/main/resources/rest-api-spec/test/search/170_terms_query.yml @@ -48,7 +48,7 @@ search: rest_total_hits_as_int: true index: test_index - body: {"query" : {"terms" : {"user" : {"index" : "test_index", "type" : "test_type", "id" : "u1", "path" : "followers"}}}} + body: {"query" : {"terms" : {"user" : {"index" : "test_index", "id" : "u1", "path" : "followers"}}}} - match: { hits.total: 2 } - do: @@ -56,4 +56,4 @@ search: rest_total_hits_as_int: true index: test_index - body: {"query" : {"terms" : {"user" : {"index" : "test_index", "type" : "test_type", "id" : "u2", "path" : "followers"}}}} + body: {"query" : {"terms" : {"user" : {"index" : "test_index", "id" : "u2", "path" : "followers"}}}} diff --git a/rest-api-spec/src/main/resources/rest-api-spec/test/search/171_terms_query_with_types.yml b/rest-api-spec/src/main/resources/rest-api-spec/test/search/171_terms_query_with_types.yml new file mode 100644 index 0000000000000..515dcfe463069 --- /dev/null +++ b/rest-api-spec/src/main/resources/rest-api-spec/test/search/171_terms_query_with_types.yml @@ -0,0 +1,59 @@ +--- +"Terms Query with No.of terms exceeding index.max_terms_count should FAIL": + - skip: + version: " - 6.99.99" + reason: index.max_terms_count setting has been added in 7.0.0 + - do: + indices.create: + index: test_index + body: + settings: + number_of_shards: 1 + index.max_terms_count: 2 + mappings: + test_type: + properties: + user: + type: keyword + followers: + type: keyword + - do: + bulk: + refresh: true + body: + - '{"index": {"_index": "test_index", "_type": "test_type", "_id": "u1"}}' + - '{"user": "u1", "followers": ["u2", "u3"]}' + - '{"index": {"_index": "test_index", "_type": "test_type", "_id": "u2"}}' + - '{"user": "u2", "followers": ["u1", "u3", "u4"]}' + - '{"index": {"_index": "test_index", "_type": "test_type", "_id": "u3"}}' + - '{"user": "u3", "followers": ["u1"]}' + - '{"index": {"_index": "test_index", "_type": "test_type", "_id": "u4"}}' + - '{"user": "u4", "followers": ["u3"]}' + + - do: + search: + rest_total_hits_as_int: true + index: test_index + body: {"query" : {"terms" : {"user" : ["u1", "u2"]}}} + - match: { hits.total: 2 } + + - do: + catch: bad_request + search: + rest_total_hits_as_int: true + index: test_index + body: {"query" : {"terms" : {"user" : ["u1", "u2", "u3"]}}} + + - do: + search: + rest_total_hits_as_int: true + index: test_index + body: {"query" : {"terms" : {"user" : {"index" : "test_index", "type" : "test_type", "id" : "u1", "path" : "followers"}}}} + - match: { hits.total: 2 } + + - do: + catch: bad_request + search: + rest_total_hits_as_int: true + index: test_index + body: {"query" : {"terms" : {"user" : {"index" : "test_index", "type" : "test_type", "id" : "u2", "path" : "followers"}}}} diff --git a/server/src/main/java/org/elasticsearch/index/query/GeoShapeQueryBuilder.java b/server/src/main/java/org/elasticsearch/index/query/GeoShapeQueryBuilder.java index 6ee0f3f10ddcc..f8ffcfdc05bcc 100644 --- a/server/src/main/java/org/elasticsearch/index/query/GeoShapeQueryBuilder.java +++ b/server/src/main/java/org/elasticsearch/index/query/GeoShapeQueryBuilder.java @@ -19,6 +19,7 @@ package org.elasticsearch.index.query; +import org.apache.logging.log4j.LogManager; import org.apache.lucene.document.LatLonShape; import org.apache.lucene.geo.Line; import org.apache.lucene.geo.Polygon; @@ -38,6 +39,7 @@ import org.elasticsearch.action.get.GetRequest; import org.elasticsearch.action.get.GetResponse; import org.elasticsearch.client.Client; +import org.elasticsearch.common.Nullable; import org.elasticsearch.common.ParseField; import org.elasticsearch.common.ParsingException; import org.elasticsearch.common.geo.GeoPoint; @@ -48,6 +50,7 @@ import org.elasticsearch.common.geo.parsers.ShapeParser; import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.io.stream.StreamOutput; +import org.elasticsearch.common.logging.DeprecationLogger; import org.elasticsearch.common.xcontent.LoggingDeprecationHandler; import org.elasticsearch.common.xcontent.NamedXContentRegistry; import org.elasticsearch.common.xcontent.XContentBuilder; @@ -66,6 +69,10 @@ */ public class GeoShapeQueryBuilder extends AbstractQueryBuilder { public static final String NAME = "geo_shape"; + private static final DeprecationLogger deprecationLogger = new DeprecationLogger( + LogManager.getLogger(GeoShapeQueryBuilder.class)); + static final String TYPES_DEPRECATION_MESSAGE = "[types removal] Types are deprecated in [geo_shape] queries. " + + "The type should no longer be specified in the [indexed_shape] section."; public static final String DEFAULT_SHAPE_INDEX_NAME = "shapes"; public static final String DEFAULT_SHAPE_FIELD_NAME = "shape"; @@ -119,6 +126,19 @@ public GeoShapeQueryBuilder(String fieldName, ShapeBuilder shape) { this(fieldName, shape, null, null); } + /** + * Creates a new GeoShapeQueryBuilder whose Query will be against the given + * field name and will use the Shape found with the given ID + * + * @param fieldName + * Name of the field that will be filtered + * @param indexedShapeId + * ID of the indexed Shape that will be used in the Query + */ + public GeoShapeQueryBuilder(String fieldName, String indexedShapeId) { + this(fieldName, (ShapeBuilder) null, indexedShapeId, null); + } + /** * Creates a new GeoShapeQueryBuilder whose Query will be against the given * field name and will use the Shape found with the given ID in the given @@ -130,20 +150,19 @@ public GeoShapeQueryBuilder(String fieldName, ShapeBuilder shape) { * ID of the indexed Shape that will be used in the Query * @param indexedShapeType * Index type of the indexed Shapes + * @deprecated use {@link #GeoShapeQueryBuilder(String, String)} instead */ + @Deprecated public GeoShapeQueryBuilder(String fieldName, String indexedShapeId, String indexedShapeType) { this(fieldName, (ShapeBuilder) null, indexedShapeId, indexedShapeType); } - private GeoShapeQueryBuilder(String fieldName, ShapeBuilder shape, String indexedShapeId, String indexedShapeType) { + private GeoShapeQueryBuilder(String fieldName, ShapeBuilder shape, String indexedShapeId, @Nullable String indexedShapeType) { if (fieldName == null) { throw new IllegalArgumentException("fieldName is required"); } if (shape == null && indexedShapeId == null) { - throw new IllegalArgumentException("either shapeBytes or indexedShapeId and indexedShapeType are required"); - } - if (indexedShapeId != null && indexedShapeType == null) { - throw new IllegalArgumentException("indexedShapeType is required if indexedShapeId is specified"); + throw new IllegalArgumentException("either shape or indexedShapeId is required"); } this.fieldName = fieldName; this.shape = shape; @@ -152,7 +171,8 @@ private GeoShapeQueryBuilder(String fieldName, ShapeBuilder shape, String indexe this.supplier = null; } - private GeoShapeQueryBuilder(String fieldName, Supplier supplier, String indexedShapeId, String indexedShapeType) { + private GeoShapeQueryBuilder(String fieldName, Supplier supplier, String indexedShapeId, + @Nullable String indexedShapeType) { this.fieldName = fieldName; this.shape = null; this.supplier = supplier; @@ -238,7 +258,10 @@ public String indexedShapeId() { /** * @return the document type of the indexed Shape that will be used in the * Query + * + * @deprecated Types are in the process of being removed. */ + @Deprecated public String indexedShapeType() { return indexedShapeType; } @@ -566,8 +589,10 @@ protected void doXContent(XContentBuilder builder, Params params) throws IOExcep shape.toXContent(builder, params); } else { builder.startObject(INDEXED_SHAPE_FIELD.getPreferredName()) - .field(SHAPE_ID_FIELD.getPreferredName(), indexedShapeId) - .field(SHAPE_TYPE_FIELD.getPreferredName(), indexedShapeType); + .field(SHAPE_ID_FIELD.getPreferredName(), indexedShapeId); + if (indexedShapeType != null) { + builder.field(SHAPE_TYPE_FIELD.getPreferredName(), indexedShapeType); + } if (indexedShapeIndex != null) { builder.field(SHAPE_INDEX_FIELD.getPreferredName(), indexedShapeIndex); } @@ -677,6 +702,11 @@ public static GeoShapeQueryBuilder fromXContent(XContentParser parser) throws IO } } GeoShapeQueryBuilder builder; + if (type != null) { + deprecationLogger.deprecatedAndMaybeLog( + "geo_share_query_with_types", TYPES_DEPRECATION_MESSAGE); + } + if (shape != null) { builder = new GeoShapeQueryBuilder(fieldName, shape); } else { @@ -739,7 +769,12 @@ protected QueryBuilder doRewrite(QueryRewriteContext queryRewriteContext) throws } else if (this.shape == null) { SetOnce supplier = new SetOnce<>(); queryRewriteContext.registerAsyncAction((client, listener) -> { - GetRequest getRequest = new GetRequest(indexedShapeIndex, indexedShapeType, indexedShapeId); + GetRequest getRequest; + if (indexedShapeType == null) { + getRequest = new GetRequest(indexedShapeIndex, indexedShapeId); + } else { + getRequest = new GetRequest(indexedShapeIndex, indexedShapeType, indexedShapeId); + } getRequest.routing(indexedShapeRouting); fetch(client, getRequest, indexedShapePath, ActionListener.wrap(builder-> { supplier.set(builder); diff --git a/server/src/main/java/org/elasticsearch/index/query/IdsQueryBuilder.java b/server/src/main/java/org/elasticsearch/index/query/IdsQueryBuilder.java index 7cbd38f3398fd..358a2fccff108 100644 --- a/server/src/main/java/org/elasticsearch/index/query/IdsQueryBuilder.java +++ b/server/src/main/java/org/elasticsearch/index/query/IdsQueryBuilder.java @@ -19,6 +19,7 @@ package org.elasticsearch.index.query; +import org.apache.logging.log4j.LogManager; import org.apache.lucene.search.MatchNoDocsQuery; import org.apache.lucene.search.Query; import org.elasticsearch.cluster.metadata.MetaData; @@ -27,6 +28,7 @@ import org.elasticsearch.common.Strings; import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.io.stream.StreamOutput; +import org.elasticsearch.common.logging.DeprecationLogger; import org.elasticsearch.common.lucene.search.Queries; import org.elasticsearch.common.xcontent.ObjectParser; import org.elasticsearch.common.xcontent.XContentBuilder; @@ -51,6 +53,9 @@ */ public class IdsQueryBuilder extends AbstractQueryBuilder { public static final String NAME = "ids"; + private static final DeprecationLogger deprecationLogger = new DeprecationLogger( + LogManager.getLogger(IdsQueryBuilder.class)); + static final String TYPES_DEPRECATION_MESSAGE = "[types removal] Types are deprecated in [ids] queries."; private static final ParseField TYPE_FIELD = new ParseField("type"); private static final ParseField VALUES_FIELD = new ParseField("values"); @@ -83,8 +88,10 @@ protected void doWriteTo(StreamOutput out) throws IOException { /** * Add types to query + * + * @deprecated Types are in the process of being removed, prefer to filter on a field instead. */ - // TODO: Remove + @Deprecated public IdsQueryBuilder types(String... types) { if (types == null) { throw new IllegalArgumentException("[" + NAME + "] types cannot be null"); @@ -95,7 +102,10 @@ public IdsQueryBuilder types(String... types) { /** * Returns the types used in this query + * + * @deprecated Types are in the process of being removed, prefer to filter on a field instead. */ + @Deprecated public String[] types() { return this.types; } @@ -121,7 +131,9 @@ public Set ids() { @Override protected void doXContent(XContentBuilder builder, Params params) throws IOException { builder.startObject(NAME); - builder.array(TYPE_FIELD.getPreferredName(), types); + if (types.length > 0) { + builder.array(TYPE_FIELD.getPreferredName(), types); + } builder.startArray(VALUES_FIELD.getPreferredName()); for (String value : ids) { builder.value(value); @@ -142,7 +154,11 @@ protected void doXContent(XContentBuilder builder, Params params) throws IOExcep public static IdsQueryBuilder fromXContent(XContentParser parser) { try { - return PARSER.apply(parser, null); + IdsQueryBuilder builder = PARSER.apply(parser, null); + if (builder.types().length > 0) { + deprecationLogger.deprecatedAndMaybeLog("ids_query_with_types", TYPES_DEPRECATION_MESSAGE); + } + return builder; } catch (IllegalArgumentException e) { throw new ParsingException(parser.getTokenLocation(), e.getMessage(), e); } diff --git a/server/src/main/java/org/elasticsearch/index/query/MoreLikeThisQueryBuilder.java b/server/src/main/java/org/elasticsearch/index/query/MoreLikeThisQueryBuilder.java index 6f87ac0fda79b..b90a1e60ffa0b 100644 --- a/server/src/main/java/org/elasticsearch/index/query/MoreLikeThisQueryBuilder.java +++ b/server/src/main/java/org/elasticsearch/index/query/MoreLikeThisQueryBuilder.java @@ -19,6 +19,7 @@ package org.elasticsearch.index.query; +import org.apache.logging.log4j.LogManager; import org.apache.lucene.analysis.Analyzer; import org.apache.lucene.index.Fields; import org.apache.lucene.search.BooleanClause; @@ -41,6 +42,7 @@ 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.logging.DeprecationLogger; import org.elasticsearch.common.lucene.search.MoreLikeThisQuery; import org.elasticsearch.common.lucene.search.XMoreLikeThis; import org.elasticsearch.common.lucene.uid.Versions; @@ -53,6 +55,7 @@ import org.elasticsearch.index.mapper.IdFieldMapper; import org.elasticsearch.index.mapper.KeywordFieldMapper.KeywordFieldType; import org.elasticsearch.index.mapper.MappedFieldType; +import org.elasticsearch.index.mapper.MapperService; import org.elasticsearch.index.mapper.TextFieldMapper.TextFieldType; import java.io.IOException; @@ -66,6 +69,7 @@ import java.util.Objects; import java.util.Optional; import java.util.Set; +import java.util.stream.Stream; import static org.elasticsearch.common.xcontent.XContentFactory.jsonBuilder; @@ -76,6 +80,11 @@ */ public class MoreLikeThisQueryBuilder extends AbstractQueryBuilder { public static final String NAME = "more_like_this"; + private static final DeprecationLogger deprecationLogger = new DeprecationLogger( + LogManager.getLogger(MoreLikeThisQueryBuilder.class)); + static final String TYPES_DEPRECATION_MESSAGE = "[types removal] Types are deprecated in [more_like_this] " + + "queries. The type should no longer be specified in the [like] and [unlike] sections."; + public static final int DEFAULT_MAX_QUERY_TERMS = XMoreLikeThis.DEFAULT_MAX_QUERY_TERMS; public static final int DEFAULT_MIN_TERM_FREQ = XMoreLikeThis.DEFAULT_MIN_TERM_FREQ; @@ -178,13 +187,45 @@ public Item() { this.versionType = copy.versionType; } + /** + * Constructor for a given item / document request + * + * @param index the index where the document is located + * @param id and its id + */ + public Item(@Nullable String index, String id) { + if (id == null) { + throw new IllegalArgumentException("Item requires id to be non-null"); + } + this.index = index; + this.id = id; + } + + /** + * Constructor for an artificial document request, that is not present in the index. + * + * @param index the index to be used for parsing the doc + * @param doc the document specification + */ + public Item(@Nullable String index, XContentBuilder doc) { + if (doc == null) { + throw new IllegalArgumentException("Item requires doc to be non-null"); + } + this.index = index; + this.doc = BytesReference.bytes(doc); + this.xContentType = doc.contentType(); + } + /** * Constructor for a given item / document request * * @param index the index where the document is located * @param type the type of the document * @param id and its id + * + * @deprecated Types are in the process of being removed, use {@link Item(String, String)} instead. */ + @Deprecated public Item(@Nullable String index, @Nullable String type, String id) { if (id == null) { throw new IllegalArgumentException("Item requires id to be non-null"); @@ -200,7 +241,10 @@ public Item(@Nullable String index, @Nullable String type, String id) { * @param index the index to be used for parsing the doc * @param type the type to be used for parsing the doc * @param doc the document specification + * + * @deprecated Types are in the process of being removed, use {@link Item(String, XContentBuilder)} instead. */ + @Deprecated public Item(@Nullable String index, @Nullable String type, XContentBuilder doc) { if (doc == null) { throw new IllegalArgumentException("Item requires doc to be non-null"); @@ -257,10 +301,18 @@ public Item index(String index) { return this; } + /** + * @deprecated Types are in the process of being removed. + */ + @Deprecated public String type() { return type; } + /** + * @deprecated Types are in the process of being removed. + */ + @Deprecated public Item type(String type) { this.type = type; return this; @@ -912,9 +964,18 @@ public static MoreLikeThisQueryBuilder fromXContent(XContentParser parser) throw if (stopWords != null) { moreLikeThisQueryBuilder.stopWords(stopWords); } + + if (moreLikeThisQueryBuilder.isTypeless() == false) { + deprecationLogger.deprecatedAndMaybeLog("more_like_this_query_with_types", TYPES_DEPRECATION_MESSAGE); + } return moreLikeThisQueryBuilder; } + public boolean isTypeless() { + return Stream.concat(Arrays.stream(likeItems), Arrays.stream(unlikeItems)) + .allMatch(item -> item.type == null); + } + private static void parseLikeField(XContentParser parser, List texts, List items) throws IOException { if (parser.currentToken().isValue()) { texts.add(parser.text()); @@ -1065,12 +1126,7 @@ private static void setDefaultIndexTypeFields(QueryShardContext context, Item it item.index(context.index().getName()); } if (item.type() == null) { - if (context.queryTypes().size() > 1) { - throw new QueryShardException(context, - "ambiguous type for item with id: " + item.id() + " and index: " + item.index()); - } else { - item.type(context.queryTypes().iterator().next()); - } + item.type(MapperService.SINGLE_MAPPING_NAME); } // default fields if not present but don't override for artificial docs if ((item.fields() == null || item.fields().length == 0) && item.doc() == null) { diff --git a/server/src/main/java/org/elasticsearch/index/query/QueryBuilders.java b/server/src/main/java/org/elasticsearch/index/query/QueryBuilders.java index f5cf2d5da66be..5ac70781286a4 100644 --- a/server/src/main/java/org/elasticsearch/index/query/QueryBuilders.java +++ b/server/src/main/java/org/elasticsearch/index/query/QueryBuilders.java @@ -121,7 +121,10 @@ public static IdsQueryBuilder idsQuery() { * Constructs a query that will match only specific ids within types. * * @param types The mapping/doc type + * + * @deprecated Types are in the process of being removed, use {@link #idsQuery()} instead. */ + @Deprecated public static IdsQueryBuilder idsQuery(String... types) { return new IdsQueryBuilder().types(types); } @@ -646,6 +649,14 @@ public static GeoShapeQueryBuilder geoShapeQuery(String name, ShapeBuilder shape return new GeoShapeQueryBuilder(name, shape); } + public static GeoShapeQueryBuilder geoShapeQuery(String name, String indexedShapeId) { + return new GeoShapeQueryBuilder(name, indexedShapeId); + } + + /** + * @deprecated Types are in the process of being removed, use {@link #geoShapeQuery(String, String)} instead. + */ + @Deprecated public static GeoShapeQueryBuilder geoShapeQuery(String name, String indexedShapeId, String indexedShapeType) { return new GeoShapeQueryBuilder(name, indexedShapeId, indexedShapeType); } @@ -662,6 +673,16 @@ public static GeoShapeQueryBuilder geoIntersectionQuery(String name, ShapeBuilde return builder; } + public static GeoShapeQueryBuilder geoIntersectionQuery(String name, String indexedShapeId) { + GeoShapeQueryBuilder builder = geoShapeQuery(name, indexedShapeId); + builder.relation(ShapeRelation.INTERSECTS); + return builder; + } + + /** + * @deprecated Types are in the process of being removed, use {@link #geoIntersectionQuery(String, String)} instead. + */ + @Deprecated public static GeoShapeQueryBuilder geoIntersectionQuery(String name, String indexedShapeId, String indexedShapeType) { GeoShapeQueryBuilder builder = geoShapeQuery(name, indexedShapeId, indexedShapeType); builder.relation(ShapeRelation.INTERSECTS); @@ -680,6 +701,16 @@ public static GeoShapeQueryBuilder geoWithinQuery(String name, ShapeBuilder shap return builder; } + public static GeoShapeQueryBuilder geoWithinQuery(String name, String indexedShapeId) { + GeoShapeQueryBuilder builder = geoShapeQuery(name, indexedShapeId); + builder.relation(ShapeRelation.WITHIN); + return builder; + } + + /** + * @deprecated Types are in the process of being removed, use {@link #geoWithinQuery(String, String)} instead. + */ + @Deprecated public static GeoShapeQueryBuilder geoWithinQuery(String name, String indexedShapeId, String indexedShapeType) { GeoShapeQueryBuilder builder = geoShapeQuery(name, indexedShapeId, indexedShapeType); builder.relation(ShapeRelation.WITHIN); @@ -698,6 +729,16 @@ public static GeoShapeQueryBuilder geoDisjointQuery(String name, ShapeBuilder sh return builder; } + public static GeoShapeQueryBuilder geoDisjointQuery(String name, String indexedShapeId) { + GeoShapeQueryBuilder builder = geoShapeQuery(name, indexedShapeId); + builder.relation(ShapeRelation.DISJOINT); + return builder; + } + + /** + * @deprecated Types are in the process of being removed, use {@link #geoDisjointQuery(String, String)} instead. + */ + @Deprecated public static GeoShapeQueryBuilder geoDisjointQuery(String name, String indexedShapeId, String indexedShapeType) { GeoShapeQueryBuilder builder = geoShapeQuery(name, indexedShapeId, indexedShapeType); builder.relation(ShapeRelation.DISJOINT); diff --git a/server/src/main/java/org/elasticsearch/index/query/TermsQueryBuilder.java b/server/src/main/java/org/elasticsearch/index/query/TermsQueryBuilder.java index a144cbf088a01..ae7bbae63018b 100644 --- a/server/src/main/java/org/elasticsearch/index/query/TermsQueryBuilder.java +++ b/server/src/main/java/org/elasticsearch/index/query/TermsQueryBuilder.java @@ -19,6 +19,7 @@ package org.elasticsearch.index.query; +import org.apache.logging.log4j.LogManager; import org.apache.lucene.search.Query; import org.apache.lucene.search.TermInSetQuery; import org.apache.lucene.util.BytesRef; @@ -34,6 +35,7 @@ import org.elasticsearch.common.io.stream.BytesStreamOutput; import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.io.stream.StreamOutput; +import org.elasticsearch.common.logging.DeprecationLogger; import org.elasticsearch.common.lucene.BytesRefs; import org.elasticsearch.common.lucene.search.Queries; import org.elasticsearch.common.xcontent.XContentBuilder; @@ -63,6 +65,11 @@ public class TermsQueryBuilder extends AbstractQueryBuilder { public static final String NAME = "terms"; + private static final DeprecationLogger deprecationLogger = new DeprecationLogger( + LogManager.getLogger(TermsQueryBuilder.class)); + static final String TYPES_DEPRECATION_MESSAGE = "[types removal] Types are deprecated " + + "in [terms] lookup queries."; + private final String fieldName; private final List values; private final TermsLookup termsLookup; @@ -211,6 +218,10 @@ public TermsLookup termsLookup() { return this.termsLookup; } + public boolean isTypeless() { + return termsLookup == null || termsLookup.type() == null; + } + private static final Set> INTEGER_TYPES = new HashSet<>( Arrays.asList(Byte.class, Short.class, Integer.class, Long.class)); private static final Set> STRING_TYPES = new HashSet<>( @@ -391,9 +402,16 @@ public static TermsQueryBuilder fromXContent(XContentParser parser) throws IOExc throw new ParsingException(parser.getTokenLocation(), "[" + TermsQueryBuilder.NAME + "] query requires a field name, " + "followed by array of terms or a document lookup specification"); } - return new TermsQueryBuilder(fieldName, values, termsLookup) - .boost(boost) - .queryName(queryName); + + TermsQueryBuilder builder = new TermsQueryBuilder(fieldName, values, termsLookup) + .boost(boost) + .queryName(queryName); + + if (builder.isTypeless() == false) { + deprecationLogger.deprecatedAndMaybeLog("terms_lookup_with_types", TYPES_DEPRECATION_MESSAGE); + } + + return builder; } static List parseValues(XContentParser parser) throws IOException { @@ -442,8 +460,10 @@ protected Query doToQuery(QueryShardContext context) throws IOException { } private void fetch(TermsLookup termsLookup, Client client, ActionListener> actionListener) { - GetRequest getRequest = new GetRequest(termsLookup.index(), termsLookup.type(), termsLookup.id()) - .preference("_local").routing(termsLookup.routing()); + GetRequest getRequest = termsLookup.type() == null + ? new GetRequest(termsLookup.index(), termsLookup.id()) + : new GetRequest(termsLookup.index(), termsLookup.type(), termsLookup.id()); + getRequest.preference("_local").routing(termsLookup.routing()); client.get(getRequest, new ActionListener() { @Override public void onResponse(GetResponse getResponse) { diff --git a/server/src/main/java/org/elasticsearch/indices/TermsLookup.java b/server/src/main/java/org/elasticsearch/indices/TermsLookup.java index c1acce072b166..077116e2fd54e 100644 --- a/server/src/main/java/org/elasticsearch/indices/TermsLookup.java +++ b/server/src/main/java/org/elasticsearch/indices/TermsLookup.java @@ -20,11 +20,11 @@ package org.elasticsearch.indices; import org.elasticsearch.Version; +import org.elasticsearch.common.Nullable; import org.elasticsearch.common.ParsingException; 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.xcontent.ToXContent.Params; import org.elasticsearch.common.xcontent.ToXContentFragment; import org.elasticsearch.common.xcontent.XContentBuilder; import org.elasticsearch.common.xcontent.XContentParser; @@ -38,18 +38,24 @@ */ public class TermsLookup implements Writeable, ToXContentFragment { private final String index; - private final String type; + private @Nullable String type; private final String id; private final String path; private String routing; + + public TermsLookup(String index, String id, String path) { + this(index, null, id, path); + } + + /** + * @deprecated Types are in the process of being removed, use {@link TermsLookup(String, String, String)} instead. + */ + @Deprecated public TermsLookup(String index, String type, String id, String path) { if (id == null) { throw new IllegalArgumentException("[" + TermsQueryBuilder.NAME + "] query lookup element requires specifying the id."); } - if (type == null) { - throw new IllegalArgumentException("[" + TermsQueryBuilder.NAME + "] query lookup element requires specifying the type."); - } if (path == null) { throw new IllegalArgumentException("[" + TermsQueryBuilder.NAME + "] query lookup element requires specifying the path."); } @@ -66,7 +72,12 @@ public TermsLookup(String index, String type, String id, String path) { * Read from a stream. */ public TermsLookup(StreamInput in) throws IOException { - type = in.readString(); + if (in.getVersion().onOrAfter(Version.V_7_0_0)) { + type = in.readOptionalString(); + } else { + // Before 7.0, the type parameter was always non-null and serialized as a (non-optional) string. + type = in.readString(); + } id = in.readString(); path = in.readString(); if (in.getVersion().onOrAfter(Version.V_6_0_0_beta1)) { @@ -82,7 +93,16 @@ public TermsLookup(StreamInput in) throws IOException { @Override public void writeTo(StreamOutput out) throws IOException { - out.writeString(type); + if (out.getVersion().onOrAfter(Version.V_7_0_0)) { + out.writeOptionalString(type); + } else { + if (type == null) { + throw new IllegalArgumentException("Typeless [terms] lookup queries are not supported if any " + + "node is running a version before 7.0."); + + } + out.writeString(type); + } out.writeString(id); out.writeString(path); if (out.getVersion().onOrAfter(Version.V_6_0_0_beta1)) { @@ -97,6 +117,10 @@ public String index() { return index; } + /** + * @deprecated Types are in the process of being removed. + */ + @Deprecated public String type() { return type; } @@ -155,18 +179,28 @@ public static TermsLookup parseTermsLookup(XContentParser parser) throws IOExcep + token + "] after [" + currentFieldName + "]"); } } - return new TermsLookup(index, type, id, path).routing(routing); + if (type == null) { + return new TermsLookup(index, id, path).routing(routing); + } else { + return new TermsLookup(index, type, id, path).routing(routing); + } } @Override public String toString() { - return index + "/" + type + "/" + id + "/" + path; + if (type == null) { + return index + "/" + id + "/" + path; + } else { + return index + "/" + type + "/" + id + "/" + path; + } } @Override public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException { builder.field("index", index); - builder.field("type", type); + if (type != null) { + builder.field("type", type); + } builder.field("id", id); builder.field("path", path); if (routing != null) { diff --git a/server/src/test/java/org/elasticsearch/index/query/GeoShapeQueryBuilderTests.java b/server/src/test/java/org/elasticsearch/index/query/GeoShapeQueryBuilderTests.java index 66f955dac7b00..22f9705dcc5f9 100644 --- a/server/src/test/java/org/elasticsearch/index/query/GeoShapeQueryBuilderTests.java +++ b/server/src/test/java/org/elasticsearch/index/query/GeoShapeQueryBuilderTests.java @@ -36,8 +36,10 @@ import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.xcontent.XContentBuilder; import org.elasticsearch.common.xcontent.XContentFactory; +import org.elasticsearch.common.xcontent.XContentParser; import org.elasticsearch.common.xcontent.json.JsonXContent; import org.elasticsearch.index.get.GetResult; +import org.elasticsearch.index.mapper.MapperService; import org.elasticsearch.search.internal.SearchContext; import org.elasticsearch.test.AbstractQueryTestCase; import org.elasticsearch.test.VersionUtils; @@ -94,7 +96,7 @@ protected GeoShapeQueryBuilder doCreateTestQueryBuilder(boolean indexedShape) { } else { indexedShapeToReturn = shape; indexedShapeId = randomAlphaOfLengthBetween(3, 20); - indexedShapeType = randomAlphaOfLengthBetween(3, 20); + indexedShapeType = randomBoolean() ? randomAlphaOfLengthBetween(3, 20) : null; builder = new GeoShapeQueryBuilder(fieldName(), indexedShapeId, indexedShapeType); if (randomBoolean()) { indexedShapeIndex = randomAlphaOfLengthBetween(3, 20); @@ -126,15 +128,17 @@ protected GeoShapeQueryBuilder doCreateTestQueryBuilder(boolean indexedShape) { @Override protected GetResponse executeGet(GetRequest getRequest) { + String indexedType = indexedShapeType != null ? indexedShapeType : MapperService.SINGLE_MAPPING_NAME; + assertThat(indexedShapeToReturn, notNullValue()); assertThat(indexedShapeId, notNullValue()); - assertThat(indexedShapeType, notNullValue()); assertThat(getRequest.id(), equalTo(indexedShapeId)); - assertThat(getRequest.type(), equalTo(indexedShapeType)); + assertThat(getRequest.type(), equalTo(indexedType)); assertThat(getRequest.routing(), equalTo(indexedShapeRouting)); String expectedShapeIndex = indexedShapeIndex == null ? GeoShapeQueryBuilder.DEFAULT_SHAPE_INDEX_NAME : indexedShapeIndex; assertThat(getRequest.index(), equalTo(expectedShapeIndex)); String expectedShapePath = indexedShapePath == null ? GeoShapeQueryBuilder.DEFAULT_SHAPE_FIELD_NAME : indexedShapePath; + String json; try { XContentBuilder builder = XContentFactory.jsonBuilder().prettyPrint(); @@ -146,7 +150,7 @@ protected GetResponse executeGet(GetRequest getRequest) { } catch (IOException ex) { throw new ElasticsearchException("boom", ex); } - return new GetResponse(new GetResult(indexedShapeIndex, indexedShapeType, indexedShapeId, 0, 1, 0, true, new BytesArray(json), + return new GetResponse(new GetResult(indexedShapeIndex, indexedType, indexedShapeId, 0, 1, 0, true, new BytesArray(json), null)); } @@ -176,19 +180,13 @@ public void testNoFieldName() throws Exception { } public void testNoShape() throws IOException { - expectThrows(IllegalArgumentException.class, () -> new GeoShapeQueryBuilder(fieldName(), null)); + expectThrows(IllegalArgumentException.class, () -> new GeoShapeQueryBuilder(fieldName(), (ShapeBuilder) null)); } public void testNoIndexedShape() throws IOException { IllegalArgumentException e = expectThrows(IllegalArgumentException.class, () -> new GeoShapeQueryBuilder(fieldName(), null, "type")); - assertEquals("either shapeBytes or indexedShapeId and indexedShapeType are required", e.getMessage()); - } - - public void testNoIndexedShapeType() throws IOException { - IllegalArgumentException e = expectThrows(IllegalArgumentException.class, - () -> new GeoShapeQueryBuilder(fieldName(), "id", null)); - assertEquals("indexedShapeType is required if indexedShapeId is specified", e.getMessage()); + assertEquals("either shape or indexedShapeId is required", e.getMessage()); } public void testNoRelation() throws IOException { @@ -286,4 +284,16 @@ public void testSerializationFailsUnlessFetched() throws IOException { builder = rewriteAndFetch(builder, createShardContext()); builder.writeTo(new BytesStreamOutput(10)); } + + @Override + protected QueryBuilder parseQuery(XContentParser parser) throws IOException { + QueryBuilder query = super.parseQuery(parser); + assertThat(query, instanceOf(GeoShapeQueryBuilder.class)); + + GeoShapeQueryBuilder shapeQuery = (GeoShapeQueryBuilder) query; + if (shapeQuery.indexedShapeType() != null) { + assertWarnings(GeoShapeQueryBuilder.TYPES_DEPRECATION_MESSAGE); + } + return query; + } } diff --git a/server/src/test/java/org/elasticsearch/index/query/IdsQueryBuilderTests.java b/server/src/test/java/org/elasticsearch/index/query/IdsQueryBuilderTests.java index c146df73019c2..2aed8202dd698 100644 --- a/server/src/test/java/org/elasticsearch/index/query/IdsQueryBuilderTests.java +++ b/server/src/test/java/org/elasticsearch/index/query/IdsQueryBuilderTests.java @@ -25,6 +25,7 @@ import org.apache.lucene.search.TermInSetQuery; import org.elasticsearch.cluster.metadata.MetaData; import org.elasticsearch.common.ParsingException; +import org.elasticsearch.common.xcontent.XContentParser; import org.elasticsearch.index.mapper.IdFieldMapper; import org.elasticsearch.search.internal.SearchContext; import org.elasticsearch.test.AbstractQueryTestCase; @@ -48,7 +49,7 @@ protected IdsQueryBuilder doCreateTestQueryBuilder() { type = randomAlphaOfLengthBetween(1, 10); } } else if (randomBoolean()) { - type = MetaData.ALL; + type = MetaData.ALL; } else { type = null; } @@ -152,4 +153,16 @@ public void testFromJson() throws IOException { assertThat(parsed.ids(), contains("1","100","4")); assertEquals(json, 0, parsed.types().length); } + + @Override + protected QueryBuilder parseQuery(XContentParser parser) throws IOException { + QueryBuilder query = super.parseQuery(parser); + assertThat(query, instanceOf(IdsQueryBuilder.class)); + + IdsQueryBuilder idsQuery = (IdsQueryBuilder) query; + if (idsQuery.types().length > 0) { + assertWarnings(IdsQueryBuilder.TYPES_DEPRECATION_MESSAGE); + } + return query; + } } diff --git a/server/src/test/java/org/elasticsearch/index/query/LegacyGeoShapeFieldQueryTests.java b/server/src/test/java/org/elasticsearch/index/query/LegacyGeoShapeFieldQueryTests.java index f549d17977dc1..2dcf3245dfe15 100644 --- a/server/src/test/java/org/elasticsearch/index/query/LegacyGeoShapeFieldQueryTests.java +++ b/server/src/test/java/org/elasticsearch/index/query/LegacyGeoShapeFieldQueryTests.java @@ -59,8 +59,7 @@ protected GeoShapeQueryBuilder doCreateTestQueryBuilder(boolean indexedShape) { } else { indexedShapeToReturn = shape; indexedShapeId = randomAlphaOfLengthBetween(3, 20); - indexedShapeType = randomAlphaOfLengthBetween(3, 20); - builder = new GeoShapeQueryBuilder(fieldName(), indexedShapeId, indexedShapeType); + builder = new GeoShapeQueryBuilder(fieldName(), indexedShapeId); if (randomBoolean()) { indexedShapeIndex = randomAlphaOfLengthBetween(3, 20); builder.indexedShapeIndex(indexedShapeIndex); diff --git a/server/src/test/java/org/elasticsearch/index/query/MoreLikeThisQueryBuilderTests.java b/server/src/test/java/org/elasticsearch/index/query/MoreLikeThisQueryBuilderTests.java index d6a45a165d19b..62613139b50fd 100644 --- a/server/src/test/java/org/elasticsearch/index/query/MoreLikeThisQueryBuilderTests.java +++ b/server/src/test/java/org/elasticsearch/index/query/MoreLikeThisQueryBuilderTests.java @@ -93,14 +93,20 @@ private static String[] randomStringFields() { private Item generateRandomItem() { String index = randomBoolean() ? getIndex().getName() : null; - String type = "doc"; // indexed item or artificial document Item item; + if (randomBoolean()) { - item = new Item(index, type, randomAlphaOfLength(10)); + item = randomBoolean() + ? new Item(index, randomAlphaOfLength(10)) + : new Item(index, randomArtificialDoc()); } else { - item = new Item(index, type, randomArtificialDoc()); + String type = "doc"; + item = randomBoolean() + ? new Item(index, type, randomAlphaOfLength(10)) + : new Item(index, type, randomArtificialDoc()); } + // if no field is specified MLT uses all mapped fields for this item if (randomBoolean()) { item.fields(randomFrom(randomFields)); @@ -372,4 +378,16 @@ public void testFromJson() throws IOException { assertEquals(json, 2, parsed.fields().length); assertEquals(json, "and potentially some more text here as well", parsed.likeTexts()[0]); } + + @Override + protected QueryBuilder parseQuery(XContentParser parser) throws IOException { + QueryBuilder query = super.parseQuery(parser); + assertThat(query, instanceOf(MoreLikeThisQueryBuilder.class)); + + MoreLikeThisQueryBuilder mltQuery = (MoreLikeThisQueryBuilder) query; + if (mltQuery.isTypeless() == false) { + assertWarnings(MoreLikeThisQueryBuilder.TYPES_DEPRECATION_MESSAGE); + } + return query; + } } diff --git a/server/src/test/java/org/elasticsearch/index/query/RandomQueryBuilder.java b/server/src/test/java/org/elasticsearch/index/query/RandomQueryBuilder.java index ecd767b9d657f..04d2d2c347bbf 100644 --- a/server/src/test/java/org/elasticsearch/index/query/RandomQueryBuilder.java +++ b/server/src/test/java/org/elasticsearch/index/query/RandomQueryBuilder.java @@ -21,6 +21,7 @@ import com.carrotsearch.randomizedtesting.generators.RandomNumbers; import com.carrotsearch.randomizedtesting.generators.RandomStrings; +import org.elasticsearch.common.Strings; import java.util.Random; @@ -47,7 +48,9 @@ public static QueryBuilder createQuery(Random r) { case 1: return new TermQueryBuilderTests().createTestQueryBuilder(); case 2: - return new IdsQueryBuilderTests().createTestQueryBuilder(); + // We make sure this query has no types to avoid deprecation warnings in the + // tests that use this method. + return new IdsQueryBuilderTests().createTestQueryBuilder().types(Strings.EMPTY_ARRAY); case 3: return createMultiTermQuery(r); default: diff --git a/server/src/test/java/org/elasticsearch/index/query/TermsQueryBuilderTests.java b/server/src/test/java/org/elasticsearch/index/query/TermsQueryBuilderTests.java index bc38880651086..d1e0de67369dc 100644 --- a/server/src/test/java/org/elasticsearch/index/query/TermsQueryBuilderTests.java +++ b/server/src/test/java/org/elasticsearch/index/query/TermsQueryBuilderTests.java @@ -35,10 +35,12 @@ import org.elasticsearch.common.io.stream.BytesStreamOutput; import org.elasticsearch.common.xcontent.XContentBuilder; import org.elasticsearch.common.xcontent.XContentFactory; +import org.elasticsearch.common.xcontent.XContentParser; import org.elasticsearch.index.get.GetResult; import org.elasticsearch.indices.TermsLookup; import org.elasticsearch.search.internal.SearchContext; import org.elasticsearch.test.AbstractQueryTestCase; +import org.hamcrest.CoreMatchers; import org.junit.Before; import java.io.IOException; @@ -97,8 +99,13 @@ protected TermsQueryBuilder doCreateTestQueryBuilder() { } private TermsLookup randomTermsLookup() { - return new TermsLookup(randomAlphaOfLength(10), randomAlphaOfLength(10), randomAlphaOfLength(10), - termsPath).routing(randomBoolean() ? randomAlphaOfLength(10) : null); + // Randomly choose between a typeless terms lookup and one with an explicit type to make sure we are + // testing both cases. + TermsLookup lookup = randomBoolean() + ? new TermsLookup(randomAlphaOfLength(10), randomAlphaOfLength(10), termsPath) + : new TermsLookup(randomAlphaOfLength(10), randomAlphaOfLength(10), randomAlphaOfLength(10), termsPath); + lookup.routing(randomBoolean() ? randomAlphaOfLength(10) : null); + return lookup; } @Override @@ -316,5 +323,16 @@ public void testTypeField() throws IOException { builder.doToQuery(createShardContext()); assertWarnings(QueryShardContext.TYPES_DEPRECATION_MESSAGE); } -} + @Override + protected QueryBuilder parseQuery(XContentParser parser) throws IOException { + QueryBuilder query = super.parseQuery(parser); + assertThat(query, CoreMatchers.instanceOf(TermsQueryBuilder.class)); + + TermsQueryBuilder termsQuery = (TermsQueryBuilder) query; + if (termsQuery.isTypeless() == false) { + assertWarnings(TermsQueryBuilder.TYPES_DEPRECATION_MESSAGE); + } + return query; + } +} diff --git a/server/src/test/java/org/elasticsearch/indices/TermsLookupTests.java b/server/src/test/java/org/elasticsearch/indices/TermsLookupTests.java index ecb169ddaf7dc..a3743f55d028b 100644 --- a/server/src/test/java/org/elasticsearch/indices/TermsLookupTests.java +++ b/server/src/test/java/org/elasticsearch/indices/TermsLookupTests.java @@ -19,6 +19,7 @@ package org.elasticsearch.indices; +import org.elasticsearch.Version; import org.elasticsearch.common.io.stream.BytesStreamOutput; import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.test.ESTestCase; @@ -82,9 +83,37 @@ public void testSerialization() throws IOException { assertNotSame(deserializedLookup, termsLookup); } } + + try (BytesStreamOutput output = new BytesStreamOutput()) { + output.setVersion(Version.V_6_7_0); + IllegalArgumentException e = expectThrows(IllegalArgumentException.class, () -> termsLookup.writeTo(output)); + assertEquals("Typeless [terms] lookup queries are not supported if any " + + "node is running a version before 7.0.", e.getMessage()); + } + } + + public void testSerializationWithTypes() throws IOException { + TermsLookup termsLookup = randomTermsLookupWithTypes(); + try (BytesStreamOutput output = new BytesStreamOutput()) { + termsLookup.writeTo(output); + try (StreamInput in = output.bytes().streamInput()) { + TermsLookup deserializedLookup = new TermsLookup(in); + assertEquals(deserializedLookup, termsLookup); + assertEquals(deserializedLookup.hashCode(), termsLookup.hashCode()); + assertNotSame(deserializedLookup, termsLookup); + } + } } public static TermsLookup randomTermsLookup() { + return new TermsLookup( + randomAlphaOfLength(10), + randomAlphaOfLength(10), + randomAlphaOfLength(10).replace('.', '_') + ).routing(randomBoolean() ? randomAlphaOfLength(10) : null); + } + + public static TermsLookup randomTermsLookupWithTypes() { return new TermsLookup( randomAlphaOfLength(10), randomAlphaOfLength(10), diff --git a/server/src/test/java/org/elasticsearch/search/geo/GeoShapeQueryTests.java b/server/src/test/java/org/elasticsearch/search/geo/GeoShapeQueryTests.java index a64f98df5a6eb..35beb10934e3d 100644 --- a/server/src/test/java/org/elasticsearch/search/geo/GeoShapeQueryTests.java +++ b/server/src/test/java/org/elasticsearch/search/geo/GeoShapeQueryTests.java @@ -195,7 +195,44 @@ public void testIndexedShapeReference() throws Exception { .endObject() .endObject()).setRefreshPolicy(IMMEDIATE).get(); - SearchResponse searchResponse = client().prepareSearch("test").setTypes("type1") + SearchResponse searchResponse = client().prepareSearch("test") + .setQuery(geoIntersectionQuery("location", "Big_Rectangle")) + .get(); + + assertSearchResponse(searchResponse); + assertThat(searchResponse.getHits().getTotalHits().value, equalTo(1L)); + assertThat(searchResponse.getHits().getHits().length, equalTo(1)); + assertThat(searchResponse.getHits().getAt(0).getId(), equalTo("1")); + + searchResponse = client().prepareSearch("test") + .setQuery(geoShapeQuery("location", "Big_Rectangle")) + .get(); + + assertSearchResponse(searchResponse); + assertThat(searchResponse.getHits().getTotalHits().value, equalTo(1L)); + assertThat(searchResponse.getHits().getHits().length, equalTo(1)); + assertThat(searchResponse.getHits().getAt(0).getId(), equalTo("1")); + } + + public void testIndexedShapeReferenceWithTypes() throws Exception { + String mapping = Strings.toString(createMapping()); + client().admin().indices().prepareCreate("test").addMapping("type1", mapping, XContentType.JSON).get(); + createIndex("shapes"); + ensureGreen(); + + EnvelopeBuilder shape = new EnvelopeBuilder(new Coordinate(-45, 45), new Coordinate(45, -45)); + + client().prepareIndex("shapes", "shape_type", "Big_Rectangle").setSource(jsonBuilder().startObject() + .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()).setRefreshPolicy(IMMEDIATE).get(); + + SearchResponse searchResponse = client().prepareSearch("test") .setQuery(geoIntersectionQuery("location", "Big_Rectangle", "shape_type")) .get(); @@ -225,8 +262,8 @@ public void testIndexedShapeReferenceSourceDisabled() throws Exception { client().prepareIndex("shapes", "shape_type", "Big_Rectangle").setSource(jsonBuilder().startObject() .field("shape", shape).endObject()).setRefreshPolicy(IMMEDIATE).get(); - IllegalArgumentException e = expectThrows(IllegalArgumentException.class, () -> client().prepareSearch("test").setTypes("type1") - .setQuery(geoIntersectionQuery("location", "Big_Rectangle", "shape_type")).get()); + IllegalArgumentException e = expectThrows(IllegalArgumentException.class, () -> client().prepareSearch("test") + .setQuery(geoIntersectionQuery("location", "Big_Rectangle")).get()); assertThat(e.getMessage(), containsString("source disabled")); } @@ -273,28 +310,28 @@ public void testShapeFetchingPath() throws Exception { .endArray().endArray() .endObject().endObject()).setRefreshPolicy(IMMEDIATE).get(); - GeoShapeQueryBuilder filter = QueryBuilders.geoShapeQuery("location", "1", "type").relation(ShapeRelation.INTERSECTS) + GeoShapeQueryBuilder filter = QueryBuilders.geoShapeQuery("location", "1").relation(ShapeRelation.INTERSECTS) .indexedShapeIndex("shapes") .indexedShapePath("location"); SearchResponse result = client().prepareSearch("test").setQuery(QueryBuilders.matchAllQuery()) .setPostFilter(filter).get(); assertSearchResponse(result); assertHitCount(result, 1); - filter = QueryBuilders.geoShapeQuery("location", "1", "type").relation(ShapeRelation.INTERSECTS) + filter = QueryBuilders.geoShapeQuery("location", "1").relation(ShapeRelation.INTERSECTS) .indexedShapeIndex("shapes") .indexedShapePath("1.location"); result = client().prepareSearch("test").setQuery(QueryBuilders.matchAllQuery()) .setPostFilter(filter).get(); assertSearchResponse(result); assertHitCount(result, 1); - filter = QueryBuilders.geoShapeQuery("location", "1", "type").relation(ShapeRelation.INTERSECTS) + filter = QueryBuilders.geoShapeQuery("location", "1").relation(ShapeRelation.INTERSECTS) .indexedShapeIndex("shapes") .indexedShapePath("1.2.location"); result = client().prepareSearch("test").setQuery(QueryBuilders.matchAllQuery()) .setPostFilter(filter).get(); assertSearchResponse(result); assertHitCount(result, 1); - filter = QueryBuilders.geoShapeQuery("location", "1", "type").relation(ShapeRelation.INTERSECTS) + filter = QueryBuilders.geoShapeQuery("location", "1").relation(ShapeRelation.INTERSECTS) .indexedShapeIndex("shapes") .indexedShapePath("1.2.3.location"); result = client().prepareSearch("test").setQuery(QueryBuilders.matchAllQuery()) @@ -303,25 +340,25 @@ public void testShapeFetchingPath() throws Exception { assertHitCount(result, 1); // now test the query variant - GeoShapeQueryBuilder query = QueryBuilders.geoShapeQuery("location", "1", "type") + GeoShapeQueryBuilder query = QueryBuilders.geoShapeQuery("location", "1") .indexedShapeIndex("shapes") .indexedShapePath("location"); result = client().prepareSearch("test").setQuery(query).get(); assertSearchResponse(result); assertHitCount(result, 1); - query = QueryBuilders.geoShapeQuery("location", "1", "type") + query = QueryBuilders.geoShapeQuery("location", "1") .indexedShapeIndex("shapes") .indexedShapePath("1.location"); result = client().prepareSearch("test").setQuery(query).get(); assertSearchResponse(result); assertHitCount(result, 1); - query = QueryBuilders.geoShapeQuery("location", "1", "type") + query = QueryBuilders.geoShapeQuery("location", "1") .indexedShapeIndex("shapes") .indexedShapePath("1.2.location"); result = client().prepareSearch("test").setQuery(query).get(); assertSearchResponse(result); assertHitCount(result, 1); - query = QueryBuilders.geoShapeQuery("location", "1", "type") + query = QueryBuilders.geoShapeQuery("location", "1") .indexedShapeIndex("shapes") .indexedShapePath("1.2.3.location"); result = client().prepareSearch("test").setQuery(query).get(); @@ -356,7 +393,7 @@ public void testQueryRandomGeoCollection() throws Exception { GeoShapeQueryBuilder geoShapeQueryBuilder = QueryBuilders.geoShapeQuery("location", filterShape); geoShapeQueryBuilder.relation(ShapeRelation.INTERSECTS); - SearchResponse result = client().prepareSearch("test").setTypes("type").setQuery(geoShapeQueryBuilder).get(); + SearchResponse result = client().prepareSearch("test").setQuery(geoShapeQueryBuilder).get(); assertSearchResponse(result); assertHitCount(result, 1); } @@ -405,7 +442,7 @@ public void testRandomGeoCollectionQuery() throws Exception { GeoShapeQueryBuilder geoShapeQueryBuilder = QueryBuilders.geoShapeQuery("location", queryCollection); geoShapeQueryBuilder.relation(ShapeRelation.INTERSECTS); - SearchResponse result = client().prepareSearch("test").setTypes("type").setQuery(geoShapeQueryBuilder).get(); + SearchResponse result = client().prepareSearch("test").setQuery(geoShapeQueryBuilder).get(); assertSearchResponse(result); assertTrue(result.getHits().getTotalHits().value > 0); } @@ -429,7 +466,7 @@ public void testPointQuery() throws Exception { GeoShapeQueryBuilder geoShapeQueryBuilder = QueryBuilders.geoShapeQuery("location", pb); geoShapeQueryBuilder.relation(ShapeRelation.INTERSECTS); - SearchResponse result = client().prepareSearch("test").setTypes("type").setQuery(geoShapeQueryBuilder).get(); + SearchResponse result = client().prepareSearch("test").setQuery(geoShapeQueryBuilder).get(); assertSearchResponse(result); assertHitCount(result, 1); } @@ -454,7 +491,7 @@ public void testContainsShapeQuery() throws Exception { ShapeBuilder filterShape = (gcb.getShapeAt(randomIntBetween(0, gcb.numShapes() - 1))); GeoShapeQueryBuilder filter = QueryBuilders.geoShapeQuery("location", filterShape) .relation(ShapeRelation.CONTAINS); - SearchResponse response = client().prepareSearch("test").setTypes("type").setQuery(QueryBuilders.matchAllQuery()) + SearchResponse response = client().prepareSearch("test").setQuery(QueryBuilders.matchAllQuery()) .setPostFilter(filter).get(); assertSearchResponse(response); @@ -478,7 +515,7 @@ public void testExistsQuery() throws Exception { client().prepareIndex("test", "type", "1").setSource(docSource).setRefreshPolicy(IMMEDIATE).get(); ExistsQueryBuilder eqb = QueryBuilders.existsQuery("location"); - SearchResponse result = client().prepareSearch("test").setTypes("type").setQuery(eqb).get(); + SearchResponse result = client().prepareSearch("test").setQuery(eqb).get(); assertSearchResponse(result); assertHitCount(result, 1); } @@ -520,7 +557,7 @@ public void testShapeFilterWithDefinedGeoCollection() throws Exception { new PolygonBuilder(new CoordinatesBuilder().coordinate(99.0, -1.0).coordinate(99.0, 3.0) .coordinate(103.0, 3.0).coordinate(103.0, -1.0) .coordinate(99.0, -1.0)))).relation(ShapeRelation.INTERSECTS); - SearchResponse result = client().prepareSearch("test").setTypes("type").setQuery(QueryBuilders.matchAllQuery()) + SearchResponse result = client().prepareSearch("test").setQuery(QueryBuilders.matchAllQuery()) .setPostFilter(filter).get(); assertSearchResponse(result); assertHitCount(result, 1); @@ -530,7 +567,7 @@ public void testShapeFilterWithDefinedGeoCollection() throws Exception { new PolygonBuilder(new CoordinatesBuilder().coordinate(199.0, -11.0).coordinate(199.0, 13.0) .coordinate(193.0, 13.0).coordinate(193.0, -11.0) .coordinate(199.0, -11.0)))).relation(ShapeRelation.INTERSECTS); - result = client().prepareSearch("test").setTypes("type").setQuery(QueryBuilders.matchAllQuery()) + result = client().prepareSearch("test").setQuery(QueryBuilders.matchAllQuery()) .setPostFilter(filter).get(); assertSearchResponse(result); assertHitCount(result, 0); @@ -541,13 +578,13 @@ public void testShapeFilterWithDefinedGeoCollection() throws Exception { new PolygonBuilder(new CoordinatesBuilder().coordinate(199.0, -11.0).coordinate(199.0, 13.0) .coordinate(193.0, 13.0).coordinate(193.0, -11.0) .coordinate(199.0, -11.0)))).relation(ShapeRelation.INTERSECTS); - result = client().prepareSearch("test").setTypes("type").setQuery(QueryBuilders.matchAllQuery()) + result = client().prepareSearch("test").setQuery(QueryBuilders.matchAllQuery()) .setPostFilter(filter).get(); assertSearchResponse(result); assertHitCount(result, 1); // no shape filter = QueryBuilders.geoShapeQuery("location", new GeometryCollectionBuilder()); - result = client().prepareSearch("test").setTypes("type").setQuery(QueryBuilders.matchAllQuery()) + result = client().prepareSearch("test").setQuery(QueryBuilders.matchAllQuery()) .setPostFilter(filter).get(); assertSearchResponse(result); assertHitCount(result, 0); diff --git a/server/src/test/java/org/elasticsearch/search/morelikethis/MoreLikeThisIT.java b/server/src/test/java/org/elasticsearch/search/morelikethis/MoreLikeThisIT.java index 62684f811643d..2e29c7c5a3815 100644 --- a/server/src/test/java/org/elasticsearch/search/morelikethis/MoreLikeThisIT.java +++ b/server/src/test/java/org/elasticsearch/search/morelikethis/MoreLikeThisIT.java @@ -89,10 +89,34 @@ public void testSimpleMoreLikeThis() throws Exception { logger.info("Running moreLikeThis"); SearchResponse response = client().prepareSearch().setQuery( - new MoreLikeThisQueryBuilder(null, new Item[] {new Item("test", "type1", "1")}).minTermFreq(1).minDocFreq(1)).get(); + new MoreLikeThisQueryBuilder(null, new Item[] {new Item("test", "1")}).minTermFreq(1).minDocFreq(1)).get(); assertHitCount(response, 1L); } + public void testSimpleMoreLikeThisWithTypes() throws Exception { + logger.info("Creating index test"); + assertAcked(prepareCreate("test").addMapping("type1", + jsonBuilder().startObject().startObject("type1").startObject("properties") + .startObject("text").field("type", "text").endObject() + .endObject().endObject().endObject())); + + logger.info("Running Cluster Health"); + assertThat(ensureGreen(), equalTo(ClusterHealthStatus.GREEN)); + + logger.info("Indexing..."); + client().index(indexRequest("test").type("type1").id("1").source(jsonBuilder().startObject().field("text", "lucene").endObject())) + .actionGet(); + client().index(indexRequest("test").type("type1").id("2") + .source(jsonBuilder().startObject().field("text", "lucene release").endObject())).actionGet(); + client().admin().indices().refresh(refreshRequest()).actionGet(); + + logger.info("Running moreLikeThis"); + SearchResponse response = client().prepareSearch().setQuery( + new MoreLikeThisQueryBuilder(null, new Item[] {new Item("test", "type1", "1")}).minTermFreq(1).minDocFreq(1)).get(); + assertHitCount(response, 1L); + } + + //Issue #30148 public void testMoreLikeThisForZeroTokensInOneOfTheAnalyzedFields() throws Exception { CreateIndexRequestBuilder createIndexRequestBuilder = prepareCreate("test") @@ -116,7 +140,7 @@ public void testMoreLikeThisForZeroTokensInOneOfTheAnalyzedFields() throws Excep client().admin().indices().refresh(refreshRequest()).actionGet(); SearchResponse searchResponse = client().prepareSearch().setQuery( - moreLikeThisQuery(new String[]{"myField", "empty"}, null, new Item[]{new Item("test", "type", "1")}) + moreLikeThisQuery(new String[]{"myField", "empty"}, null, new Item[]{new Item("test", "1")}) .minTermFreq(1).minDocFreq(1) ).get(); @@ -142,7 +166,7 @@ public void testSimpleMoreLikeOnLongField() throws Exception { logger.info("Running moreLikeThis"); SearchResponse response = client().prepareSearch().setQuery( - new MoreLikeThisQueryBuilder(null, new Item[] {new Item("test", "type1", "1")}).minTermFreq(1).minDocFreq(1)).get(); + new MoreLikeThisQueryBuilder(null, new Item[] {new Item("test", "1")}).minTermFreq(1).minDocFreq(1)).get(); assertHitCount(response, 0L); } @@ -173,24 +197,24 @@ public void testMoreLikeThisWithAliases() throws Exception { logger.info("Running moreLikeThis on index"); SearchResponse response = client().prepareSearch().setQuery( - new MoreLikeThisQueryBuilder(null, new Item[] {new Item("test", "type1", "1")}).minTermFreq(1).minDocFreq(1)).get(); + new MoreLikeThisQueryBuilder(null, new Item[] {new Item("test", "1")}).minTermFreq(1).minDocFreq(1)).get(); assertHitCount(response, 2L); logger.info("Running moreLikeThis on beta shard"); response = client().prepareSearch("beta").setQuery( - new MoreLikeThisQueryBuilder(null, new Item[] {new Item("test", "type1", "1")}).minTermFreq(1).minDocFreq(1)).get(); + new MoreLikeThisQueryBuilder(null, new Item[] {new Item("test", "1")}).minTermFreq(1).minDocFreq(1)).get(); assertHitCount(response, 1L); assertThat(response.getHits().getAt(0).getId(), equalTo("3")); logger.info("Running moreLikeThis on release shard"); response = client().prepareSearch("release").setQuery( - new MoreLikeThisQueryBuilder(null, new Item[] {new Item("test", "type1", "1")}).minTermFreq(1).minDocFreq(1)).get(); + new MoreLikeThisQueryBuilder(null, new Item[] {new Item("test", "1")}).minTermFreq(1).minDocFreq(1)).get(); assertHitCount(response, 1L); assertThat(response.getHits().getAt(0).getId(), equalTo("2")); logger.info("Running moreLikeThis on alias with node client"); response = internalCluster().coordOnlyNodeClient().prepareSearch("beta").setQuery( - new MoreLikeThisQueryBuilder(null, new Item[] {new Item("test", "type1", "1")}).minTermFreq(1).minDocFreq(1)).get(); + new MoreLikeThisQueryBuilder(null, new Item[] {new Item("test", "1")}).minTermFreq(1).minDocFreq(1)).get(); assertHitCount(response, 1L); assertThat(response.getHits().getAt(0).getId(), equalTo("3")); } @@ -311,13 +335,13 @@ public void testNumericField() throws Exception { // Implicit list of fields -> ignore numeric fields SearchResponse searchResponse = client().prepareSearch().setQuery( - new MoreLikeThisQueryBuilder(null, new Item[] {new Item("test", "type", "1")}).minTermFreq(1).minDocFreq(1)).get(); + new MoreLikeThisQueryBuilder(null, new Item[] {new Item("test", "1")}).minTermFreq(1).minDocFreq(1)).get(); assertHitCount(searchResponse, 1L); // Explicit list of fields including numeric fields -> fail assertThrows(client().prepareSearch().setQuery( new MoreLikeThisQueryBuilder(new String[] {"string_value", "int_value"}, null, - new Item[] {new Item("test", "type", "1")}).minTermFreq(1).minDocFreq(1)), SearchPhaseExecutionException.class); + new Item[] {new Item("test", "1")}).minTermFreq(1).minDocFreq(1)), SearchPhaseExecutionException.class); // mlt query with no field -> No results (because _all is not enabled) searchResponse = client().prepareSearch().setQuery(moreLikeThisQuery(new String[] {"index"}).minTermFreq(1).minDocFreq(1)) @@ -417,7 +441,7 @@ public void testSimpleMoreLikeInclude() throws Exception { logger.info("Running More Like This with include true"); SearchResponse response = client().prepareSearch().setQuery( - new MoreLikeThisQueryBuilder(null, new Item[] {new Item("test", "type1", "1")}).minTermFreq(1).minDocFreq(1).include(true) + new MoreLikeThisQueryBuilder(null, new Item[] {new Item("test", "1")}).minTermFreq(1).minDocFreq(1).include(true) .minimumShouldMatch("0%")).get(); assertOrderedSearchHits(response, "1", "2"); @@ -428,7 +452,7 @@ public void testSimpleMoreLikeInclude() throws Exception { logger.info("Running More Like This with include false"); response = client().prepareSearch().setQuery( - new MoreLikeThisQueryBuilder(null, new Item[] {new Item("test", "type1", "1")}).minTermFreq(1).minDocFreq(1) + new MoreLikeThisQueryBuilder(null, new Item[] {new Item("test", "1")}).minTermFreq(1).minDocFreq(1) .minimumShouldMatch("0%")).get(); assertSearchHits(response, "2"); } @@ -673,7 +697,7 @@ public void testSelectFields() throws IOException, ExecutionException, Interrupt .field("text1", "elasticsearch") .endObject())); - MoreLikeThisQueryBuilder mltQuery = moreLikeThisQuery(new Item[] {new Item("test", "type1", "1")}) + MoreLikeThisQueryBuilder mltQuery = moreLikeThisQuery(new Item[] {new Item("test", "1")}) .minTermFreq(0) .minDocFreq(0) .include(true) @@ -683,7 +707,7 @@ public void testSelectFields() throws IOException, ExecutionException, Interrupt assertSearchResponse(response); assertHitCount(response, 2); - mltQuery = moreLikeThisQuery(new String[] {"text"}, null, new Item[] {new Item("test", "type1", "1")}) + mltQuery = moreLikeThisQuery(new String[] {"text"}, null, new Item[] {new Item("test", "1")}) .minTermFreq(0) .minDocFreq(0) .include(true) @@ -724,19 +748,19 @@ public void testWithMissingRouting() throws IOException { logger.info("Running moreLikeThis with one item without routing attribute"); SearchPhaseExecutionException exception = expectThrows(SearchPhaseExecutionException.class, () -> client().prepareSearch().setQuery(new MoreLikeThisQueryBuilder(null, new Item[]{ - new Item("test", "type1", "1") + new Item("test", "1") }).minTermFreq(1).minDocFreq(1)).get()); Throwable cause = exception.getCause(); assertThat(cause, instanceOf(RoutingMissingException.class)); - assertThat(cause.getMessage(), equalTo("routing is required for [test]/[type1]/[1]")); + assertThat(cause.getMessage(), equalTo("routing is required for [test]/[_doc]/[1]")); } { logger.info("Running moreLikeThis with one item with routing attribute and two items without routing attribute"); SearchPhaseExecutionException exception = expectThrows(SearchPhaseExecutionException.class, () -> client().prepareSearch().setQuery(new MoreLikeThisQueryBuilder(null, new Item[]{ - new Item("test", "type1", "1").routing("1"), + new Item("test", "1").routing("1"), new Item("test", "type1", "2"), new Item("test", "type1", "3") }).minTermFreq(1).minDocFreq(1)).get()); diff --git a/test/framework/src/main/java/org/elasticsearch/test/AbstractQueryTestCase.java b/test/framework/src/main/java/org/elasticsearch/test/AbstractQueryTestCase.java index c90cd3725ecab..a16f55e04d74a 100644 --- a/test/framework/src/main/java/org/elasticsearch/test/AbstractQueryTestCase.java +++ b/test/framework/src/main/java/org/elasticsearch/test/AbstractQueryTestCase.java @@ -379,7 +379,7 @@ protected void assertParsedQuery(String queryAsString, QueryBuilder expectedQuer /** * Parses the query provided as bytes argument and compares it with the expected result provided as argument as a {@link QueryBuilder} */ - private static void assertParsedQuery(XContentParser parser, QueryBuilder expectedQuery) throws IOException { + private void assertParsedQuery(XContentParser parser, QueryBuilder expectedQuery) throws IOException { QueryBuilder newQuery = parseQuery(parser); assertNotSame(newQuery, expectedQuery); assertEquals(expectedQuery, newQuery); @@ -396,7 +396,7 @@ protected QueryBuilder parseQuery(String queryAsString) throws IOException { return parseQuery(parser); } - protected static QueryBuilder parseQuery(XContentParser parser) throws IOException { + protected QueryBuilder parseQuery(XContentParser parser) throws IOException { QueryBuilder parseInnerQueryBuilder = parseInnerQueryBuilder(parser); assertNull(parser.nextToken()); return parseInnerQueryBuilder; From 38b698d4554d196ecc4f6fcfb52ae5bdeb24ff5b Mon Sep 17 00:00:00 2001 From: Jun Ohtani Date: Wed, 9 Jan 2019 12:42:00 +0900 Subject: [PATCH 12/46] [Analysis] Deprecate Standard Html Strip Analyzer in master (#26719) * [Analysis] Deprecate Standard Html Strip Analyzer Deprecate only Standard Html Strip Analyzer If user create index with the analyzer since 7.0, es throws an exception. If an index was created before 7.0, es issue deprecation log We will remove it in 8.0 Related #4704 --- .../migration/migrate_7_0/analysis.asciidoc | 8 ++++ .../analysis/common/CommonAnalysisPlugin.java | 3 ++ .../common/StandardHtmlStripAnalyzer.java | 5 +- .../StandardHtmlStripAnalyzerProvider.java | 18 ++++++++ .../common/CommonAnalysisPluginTests.java | 46 +++++++++++++++++++ .../test/analysis-common/20_analyzers.yml | 7 +-- .../index/analysis/AnalysisRegistry.java | 7 +++ 7 files changed, 90 insertions(+), 4 deletions(-) diff --git a/docs/reference/migration/migrate_7_0/analysis.asciidoc b/docs/reference/migration/migrate_7_0/analysis.asciidoc index e4b27def9f8cf..36ad41be09aa1 100644 --- a/docs/reference/migration/migrate_7_0/analysis.asciidoc +++ b/docs/reference/migration/migrate_7_0/analysis.asciidoc @@ -31,3 +31,11 @@ instead. ==== `standard` filter has been removed The `standard` token filter has been removed because it doesn't change anything in the stream. + +[float] +==== Deprecated standard_html_strip analyzer + +The `standard_html_strip` analyzer has been deprecated, and should be replaced +with a combination of the `standard` tokenizer and `html_strip` char_filter. +Indexes created using this analyzer will still be readable in elasticsearch 7.0, +but it will not be possible to create new indexes using it. \ No newline at end of file diff --git a/modules/analysis-common/src/main/java/org/elasticsearch/analysis/common/CommonAnalysisPlugin.java b/modules/analysis-common/src/main/java/org/elasticsearch/analysis/common/CommonAnalysisPlugin.java index ad0e59c276779..0059f7460a873 100644 --- a/modules/analysis-common/src/main/java/org/elasticsearch/analysis/common/CommonAnalysisPlugin.java +++ b/modules/analysis-common/src/main/java/org/elasticsearch/analysis/common/CommonAnalysisPlugin.java @@ -171,6 +171,8 @@ public List> getContexts() { public Map>> getAnalyzers() { Map>> analyzers = new TreeMap<>(); analyzers.put("fingerprint", FingerprintAnalyzerProvider::new); + + // TODO remove in 8.0 analyzers.put("standard_html_strip", StandardHtmlStripAnalyzerProvider::new); analyzers.put("pattern", PatternAnalyzerProvider::new); analyzers.put("snowball", SnowballAnalyzerProvider::new); @@ -320,6 +322,7 @@ public Map> getTokenizers() { @Override public List getPreBuiltAnalyzerProviderFactories() { List analyzers = new ArrayList<>(); + // TODO remove in 8.0 analyzers.add(new PreBuiltAnalyzerProviderFactory("standard_html_strip", CachingStrategy.ELASTICSEARCH, () -> new StandardHtmlStripAnalyzer(CharArraySet.EMPTY_SET))); analyzers.add(new PreBuiltAnalyzerProviderFactory("pattern", CachingStrategy.ELASTICSEARCH, diff --git a/modules/analysis-common/src/main/java/org/elasticsearch/analysis/common/StandardHtmlStripAnalyzer.java b/modules/analysis-common/src/main/java/org/elasticsearch/analysis/common/StandardHtmlStripAnalyzer.java index e2ee540fe3e70..a35a0ea2a4a0b 100644 --- a/modules/analysis-common/src/main/java/org/elasticsearch/analysis/common/StandardHtmlStripAnalyzer.java +++ b/modules/analysis-common/src/main/java/org/elasticsearch/analysis/common/StandardHtmlStripAnalyzer.java @@ -37,7 +37,10 @@ public class StandardHtmlStripAnalyzer extends StopwordAnalyzerBase { public StandardHtmlStripAnalyzer() { super(EnglishAnalyzer.ENGLISH_STOP_WORDS_SET); } - + /** + * @deprecated in 6.5, can not create in 7.0, and we remove this in 8.0 + */ + @Deprecated StandardHtmlStripAnalyzer(CharArraySet stopwords) { super(stopwords); } diff --git a/modules/analysis-common/src/main/java/org/elasticsearch/analysis/common/StandardHtmlStripAnalyzerProvider.java b/modules/analysis-common/src/main/java/org/elasticsearch/analysis/common/StandardHtmlStripAnalyzerProvider.java index 89535b78962b0..5dd475cc5e408 100644 --- a/modules/analysis-common/src/main/java/org/elasticsearch/analysis/common/StandardHtmlStripAnalyzerProvider.java +++ b/modules/analysis-common/src/main/java/org/elasticsearch/analysis/common/StandardHtmlStripAnalyzerProvider.java @@ -19,7 +19,10 @@ package org.elasticsearch.analysis.common; +import org.apache.logging.log4j.LogManager; import org.apache.lucene.analysis.CharArraySet; +import org.elasticsearch.Version; +import org.elasticsearch.common.logging.DeprecationLogger; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.env.Environment; import org.elasticsearch.index.IndexSettings; @@ -28,14 +31,29 @@ public class StandardHtmlStripAnalyzerProvider extends AbstractIndexAnalyzerProvider { + private static final DeprecationLogger DEPRECATION_LOGGER = + new DeprecationLogger(LogManager.getLogger(StandardHtmlStripAnalyzerProvider.class)); + private final StandardHtmlStripAnalyzer analyzer; + /** + * @deprecated in 6.5, can not create in 7.0, and we remove this in 8.0 + */ + @Deprecated StandardHtmlStripAnalyzerProvider(IndexSettings indexSettings, Environment env, String name, Settings settings) { super(indexSettings, name, settings); final CharArraySet defaultStopwords = CharArraySet.EMPTY_SET; CharArraySet stopWords = Analysis.parseStopWords(env, settings, defaultStopwords); analyzer = new StandardHtmlStripAnalyzer(stopWords); analyzer.setVersion(version); + if (indexSettings.getIndexVersionCreated().onOrAfter(Version.V_7_0_0)) { + throw new IllegalArgumentException("[standard_html_strip] analyzer is not supported for new indices, " + + "use a custom analyzer using [standard] tokenizer and [html_strip] char_filter, plus [lowercase] filter"); + } else { + DEPRECATION_LOGGER.deprecatedAndMaybeLog("standard_html_strip_deprecation", + "Deprecated analyzer [standard_html_strip] used, " + + "replace it with a custom analyzer using [standard] tokenizer and [html_strip] char_filter, plus [lowercase] filter"); + } } @Override diff --git a/modules/analysis-common/src/test/java/org/elasticsearch/analysis/common/CommonAnalysisPluginTests.java b/modules/analysis-common/src/test/java/org/elasticsearch/analysis/common/CommonAnalysisPluginTests.java index b5dc23fbdb893..c52c78ffe27e3 100644 --- a/modules/analysis-common/src/test/java/org/elasticsearch/analysis/common/CommonAnalysisPluginTests.java +++ b/modules/analysis-common/src/test/java/org/elasticsearch/analysis/common/CommonAnalysisPluginTests.java @@ -19,6 +19,7 @@ package org.elasticsearch.analysis.common; +import org.apache.lucene.analysis.Analyzer; import org.apache.lucene.analysis.MockTokenizer; import org.apache.lucene.analysis.Tokenizer; import org.elasticsearch.Version; @@ -26,6 +27,8 @@ import org.elasticsearch.common.settings.Settings; import org.elasticsearch.env.Environment; import org.elasticsearch.index.IndexSettings; +import org.elasticsearch.index.analysis.IndexAnalyzers; +import org.elasticsearch.index.analysis.NamedAnalyzer; import org.elasticsearch.index.analysis.TokenFilterFactory; import org.elasticsearch.test.ESTestCase; import org.elasticsearch.test.IndexSettingsModule; @@ -116,4 +119,47 @@ public void testEdgeNGramNoDeprecationWarningPre6_4() throws IOException { assertNotNull(tokenFilterFactory.create(tokenizer)); } } + + + /** + * Check that the deprecated analyzer name "standard_html_strip" throws exception for indices created since 7.0.0 + */ + public void testStandardHtmlStripAnalyzerDeprecationError() throws IOException { + Settings settings = Settings.builder().put(Environment.PATH_HOME_SETTING.getKey(), createTempDir()) + .put(IndexMetaData.SETTING_VERSION_CREATED, + VersionUtils.randomVersionBetween(random(), Version.V_7_0_0, Version.CURRENT)) + .put("index.analysis.analyzer.custom_analyzer.type", "standard_html_strip") + .putList("index.analysis.analyzer.custom_analyzer.stopwords", "a", "b") + .build(); + + IndexSettings idxSettings = IndexSettingsModule.newIndexSettings("index", settings); + CommonAnalysisPlugin commonAnalysisPlugin = new CommonAnalysisPlugin(); + IllegalArgumentException ex = expectThrows(IllegalArgumentException.class, + () -> createTestAnalysis(idxSettings, settings, commonAnalysisPlugin)); + assertEquals("[standard_html_strip] analyzer is not supported for new indices, " + + "use a custom analyzer using [standard] tokenizer and [html_strip] char_filter, plus [lowercase] filter", ex.getMessage()); + } + + /** + * Check that the deprecated analyzer name "standard_html_strip" issues a deprecation warning for indices created since 6.5.0 until 7 + */ + public void testStandardHtmlStripAnalyzerDeprecationWarning() throws IOException { + Settings settings = Settings.builder().put(Environment.PATH_HOME_SETTING.getKey(), createTempDir()) + .put(IndexMetaData.SETTING_VERSION_CREATED, + VersionUtils.randomVersionBetween(random(), Version.V_6_0_0, + VersionUtils.getPreviousVersion(Version.V_7_0_0))) + .put("index.analysis.analyzer.custom_analyzer.type", "standard_html_strip") + .putList("index.analysis.analyzer.custom_analyzer.stopwords", "a", "b") + .build(); + + IndexSettings idxSettings = IndexSettingsModule.newIndexSettings("index", settings); + try (CommonAnalysisPlugin commonAnalysisPlugin = new CommonAnalysisPlugin()) { + IndexAnalyzers analyzers = createTestAnalysis(idxSettings, settings, commonAnalysisPlugin).indexAnalyzers; + Analyzer analyzer = analyzers.get("custom_analyzer"); + assertNotNull(((NamedAnalyzer) analyzer).analyzer()); + assertWarnings( + "Deprecated analyzer [standard_html_strip] used, " + + "replace it with a custom analyzer using [standard] tokenizer and [html_strip] char_filter, plus [lowercase] filter"); + } + } } diff --git a/modules/analysis-common/src/test/resources/rest-api-spec/test/analysis-common/20_analyzers.yml b/modules/analysis-common/src/test/resources/rest-api-spec/test/analysis-common/20_analyzers.yml index fa8f6eef8b924..fe5b997974ad1 100644 --- a/modules/analysis-common/src/test/resources/rest-api-spec/test/analysis-common/20_analyzers.yml +++ b/modules/analysis-common/src/test/resources/rest-api-spec/test/analysis-common/20_analyzers.yml @@ -69,14 +69,15 @@ --- "standard_html_strip": + - skip: + version: " - 6.99.99" + reason: only starting from version 7.x this throws an error - do: + catch: /\[standard_html_strip\] analyzer is not supported for new indices, use a custom analyzer using \[standard\] tokenizer and \[html_strip\] char_filter, plus \[lowercase\] filter/ indices.analyze: body: text: analyzer: standard_html_strip - - length: { tokens: 2 } - - match: { tokens.0.token: bold } - - match: { tokens.1.token: italic } --- "pattern": diff --git a/server/src/main/java/org/elasticsearch/index/analysis/AnalysisRegistry.java b/server/src/main/java/org/elasticsearch/index/analysis/AnalysisRegistry.java index 55e2a1836fda3..165256940bb81 100644 --- a/server/src/main/java/org/elasticsearch/index/analysis/AnalysisRegistry.java +++ b/server/src/main/java/org/elasticsearch/index/analysis/AnalysisRegistry.java @@ -20,6 +20,7 @@ import org.apache.lucene.analysis.Analyzer; import org.apache.lucene.analysis.core.WhitespaceTokenizer; +import org.elasticsearch.Version; import org.elasticsearch.core.internal.io.IOUtils; import org.elasticsearch.ElasticsearchException; import org.elasticsearch.cluster.metadata.IndexMetaData; @@ -130,7 +131,13 @@ public Analyzer getAnalyzer(String analyzer) throws IOException { throw new ElasticsearchException("failed to load analyzer for name " + key, ex); }} ); + } else if ("standard_html_strip".equals(analyzer)) { + if (Version.CURRENT.onOrAfter(Version.V_7_0_0)) { + throw new IllegalArgumentException("[standard_html_strip] analyzer is not supported for new indices, " + + "use a custom analyzer using [standard] tokenizer and [html_strip] char_filter, plus [lowercase] filter"); + } } + return analyzerProvider.get(environment, analyzer).get(); } From d6ab3708ce9c0bb826341b60d04cfd65422f583f Mon Sep 17 00:00:00 2001 From: Yuri Astrakhan Date: Wed, 9 Jan 2019 00:18:58 -0500 Subject: [PATCH 13/46] added tests, mv to GeoGridAggregationBuilder2 --- .../client/RestHighLevelClient.java | 3 + .../elasticsearch/search/SearchModule.java | 8 +- .../aggregations/AggregationBuilders.java | 10 + ...r.java => GeoGridAggregationBuilder2.java} | 38 +- .../bucket/geogrid2/GeoGridAggregator.java | 4 +- .../geogrid2/GeoGridAggregatorFactory.java | 2 +- .../bucket/geogrid2/InternalGeoGrid.java | 2 +- .../bucket/geogrid2/ParsedGeoGrid.java | 2 +- .../aggregations/AggregationsTests.java | 2 + .../search/aggregations/bucket/GeoGridIT.java | 329 ++++++++++++++++++ .../aggregations/bucket/GeoGridTests.java | 73 ++++ .../aggregations/bucket/ShardReduceIT.java | 17 + .../geogrid2/GeoGridAggregatorTests.java | 134 +++++++ .../bucket/geogrid2/GeoGridParserTests.java | 149 ++++++++ .../geogrid2/InternalGeoGridGeoHashTests.java | 144 ++++++++ .../aggregations/metrics/GeoCentroidIT.java | 25 ++ .../test/InternalAggregationTestCase.java | 3 + 17 files changed, 916 insertions(+), 29 deletions(-) rename server/src/main/java/org/elasticsearch/search/aggregations/bucket/geogrid2/{GeoGridAggregationBuilder.java => GeoGridAggregationBuilder2.java} (89%) create mode 100644 server/src/test/java/org/elasticsearch/search/aggregations/bucket/GeoGridIT.java create mode 100644 server/src/test/java/org/elasticsearch/search/aggregations/bucket/GeoGridTests.java create mode 100644 server/src/test/java/org/elasticsearch/search/aggregations/bucket/geogrid2/GeoGridAggregatorTests.java create mode 100644 server/src/test/java/org/elasticsearch/search/aggregations/bucket/geogrid2/GeoGridParserTests.java create mode 100644 server/src/test/java/org/elasticsearch/search/aggregations/bucket/geogrid2/InternalGeoGridGeoHashTests.java diff --git a/client/rest-high-level/src/main/java/org/elasticsearch/client/RestHighLevelClient.java b/client/rest-high-level/src/main/java/org/elasticsearch/client/RestHighLevelClient.java index a9c6901d9820a..03368636555ac 100644 --- a/client/rest-high-level/src/main/java/org/elasticsearch/client/RestHighLevelClient.java +++ b/client/rest-high-level/src/main/java/org/elasticsearch/client/RestHighLevelClient.java @@ -95,6 +95,8 @@ import org.elasticsearch.search.aggregations.bucket.filter.ParsedFilters; import org.elasticsearch.search.aggregations.bucket.geogrid.GeoGridAggregationBuilder; import org.elasticsearch.search.aggregations.bucket.geogrid.ParsedGeoHashGrid; +import org.elasticsearch.search.aggregations.bucket.geogrid2.GeoGridAggregationBuilder2; +import org.elasticsearch.search.aggregations.bucket.geogrid2.ParsedGeoGrid; import org.elasticsearch.search.aggregations.bucket.global.GlobalAggregationBuilder; import org.elasticsearch.search.aggregations.bucket.global.ParsedGlobal; import org.elasticsearch.search.aggregations.bucket.histogram.AutoDateHistogramAggregationBuilder; @@ -1759,6 +1761,7 @@ static List getDefaultNamedXContents() { map.put(FilterAggregationBuilder.NAME, (p, c) -> ParsedFilter.fromXContent(p, (String) c)); map.put(InternalSampler.PARSER_NAME, (p, c) -> ParsedSampler.fromXContent(p, (String) c)); map.put(GeoGridAggregationBuilder.NAME, (p, c) -> ParsedGeoHashGrid.fromXContent(p, (String) c)); + map.put(GeoGridAggregationBuilder2.NAME, (p, c) -> ParsedGeoGrid.fromXContent(p, (String) c)); map.put(RangeAggregationBuilder.NAME, (p, c) -> ParsedRange.fromXContent(p, (String) c)); map.put(DateRangeAggregationBuilder.NAME, (p, c) -> ParsedDateRange.fromXContent(p, (String) c)); map.put(GeoDistanceAggregationBuilder.NAME, (p, c) -> ParsedGeoDistance.fromXContent(p, (String) c)); diff --git a/server/src/main/java/org/elasticsearch/search/SearchModule.java b/server/src/main/java/org/elasticsearch/search/SearchModule.java index 96cfbfb7f917b..0d7b673d70815 100644 --- a/server/src/main/java/org/elasticsearch/search/SearchModule.java +++ b/server/src/main/java/org/elasticsearch/search/SearchModule.java @@ -110,6 +110,7 @@ import org.elasticsearch.search.aggregations.bucket.filter.InternalFilters; import org.elasticsearch.search.aggregations.bucket.geogrid.GeoGridAggregationBuilder; import org.elasticsearch.search.aggregations.bucket.geogrid.InternalGeoHashGrid; +import org.elasticsearch.search.aggregations.bucket.geogrid2.GeoGridAggregationBuilder2; import org.elasticsearch.search.aggregations.bucket.geogrid2.InternalGeoGrid; import org.elasticsearch.search.aggregations.bucket.global.GlobalAggregationBuilder; import org.elasticsearch.search.aggregations.bucket.global.InternalGlobal; @@ -422,11 +423,8 @@ private void registerAggregations(List plugins) { GeoDistanceAggregationBuilder::parse).addResultReader(InternalGeoDistance::new)); registerAggregation(new AggregationSpec(GeoGridAggregationBuilder.NAME, GeoGridAggregationBuilder::new, GeoGridAggregationBuilder::parse).addResultReader(InternalGeoHashGrid::new)); - registerAggregation(new AggregationSpec( - org.elasticsearch.search.aggregations.bucket.geogrid2.GeoGridAggregationBuilder.NAME, - org.elasticsearch.search.aggregations.bucket.geogrid2.GeoGridAggregationBuilder::new, - org.elasticsearch.search.aggregations.bucket.geogrid2.GeoGridAggregationBuilder::parse - ).addResultReader(InternalGeoGrid::new)); + registerAggregation(new AggregationSpec(GeoGridAggregationBuilder2.NAME, GeoGridAggregationBuilder2::new, + GeoGridAggregationBuilder2::parse).addResultReader(InternalGeoGrid::new)); registerAggregation(new AggregationSpec(NestedAggregationBuilder.NAME, NestedAggregationBuilder::new, NestedAggregationBuilder::parse).addResultReader(InternalNested::new)); registerAggregation(new AggregationSpec(ReverseNestedAggregationBuilder.NAME, ReverseNestedAggregationBuilder::new, diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/AggregationBuilders.java b/server/src/main/java/org/elasticsearch/search/aggregations/AggregationBuilders.java index 6d8c8a94f3e6f..aded69584db87 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/AggregationBuilders.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/AggregationBuilders.java @@ -30,6 +30,9 @@ import org.elasticsearch.search.aggregations.bucket.filter.FiltersAggregator.KeyedFilter; import org.elasticsearch.search.aggregations.bucket.geogrid.GeoGridAggregationBuilder; import org.elasticsearch.search.aggregations.bucket.geogrid.GeoHashGrid; +import org.elasticsearch.search.aggregations.bucket.geogrid2.GeoGrid; +import org.elasticsearch.search.aggregations.bucket.geogrid2.GeoGridAggregationBuilder2; +import org.elasticsearch.search.aggregations.bucket.geogrid2.GeoGridType; import org.elasticsearch.search.aggregations.bucket.global.Global; import org.elasticsearch.search.aggregations.bucket.global.GlobalAggregationBuilder; import org.elasticsearch.search.aggregations.bucket.histogram.DateHistogramAggregationBuilder; @@ -250,6 +253,13 @@ public static GeoGridAggregationBuilder geohashGrid(String name) { return new GeoGridAggregationBuilder(name); } + /** + * Create a new {@link GeoGrid} aggregation with the given name. + */ + public static GeoGridAggregationBuilder2 geoGrid(String name, GeoGridType type) { + return new GeoGridAggregationBuilder2(name, type); + } + /** * Create a new {@link SignificantTerms} aggregation with the given name. */ diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/geogrid2/GeoGridAggregationBuilder.java b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/geogrid2/GeoGridAggregationBuilder2.java similarity index 89% rename from server/src/main/java/org/elasticsearch/search/aggregations/bucket/geogrid2/GeoGridAggregationBuilder.java rename to server/src/main/java/org/elasticsearch/search/aggregations/bucket/geogrid2/GeoGridAggregationBuilder2.java index 9e33a807b0e98..56e36f7e6ced6 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/geogrid2/GeoGridAggregationBuilder.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/geogrid2/GeoGridAggregationBuilder2.java @@ -55,7 +55,7 @@ import static org.elasticsearch.common.xcontent.ConstructingObjectParser.constructorArg; -public class GeoGridAggregationBuilder extends ValuesSourceAggregationBuilder +public class GeoGridAggregationBuilder2 extends ValuesSourceAggregationBuilder implements MultiBucketAggregationBuilder { public static final String NAME = "geo_grid"; public static final int DEFAULT_MAX_NUM_CELLS = 10000; @@ -68,29 +68,29 @@ public class GeoGridAggregationBuilder extends ValuesSourceAggregationBuilder PARSER; + private static final ConstructingObjectParser PARSER; static { types = GeoGridTypes.DEFAULT; - PARSER = new ConstructingObjectParser<>(GeoGridAggregationBuilder.NAME, false, - (a, name) -> new GeoGridAggregationBuilder(name, (GeoGridType) a[0])); + PARSER = new ConstructingObjectParser<>(GeoGridAggregationBuilder2.NAME, false, + (a, name) -> new GeoGridAggregationBuilder2(name, (GeoGridType) a[0])); PARSER.declareField( constructorArg(), - GeoGridAggregationBuilder::parseType, + GeoGridAggregationBuilder2::parseType, FIELD_TYPE, ObjectParser.ValueType.STRING); PARSER.declareField( - GeoGridAggregationBuilder::precisionRaw, - GeoGridAggregationBuilder::parsePrecision, + GeoGridAggregationBuilder2::precisionRaw, + GeoGridAggregationBuilder2::parsePrecision, FIELD_PRECISION, ObjectParser.ValueType.VALUE); PARSER.declareInt( - GeoGridAggregationBuilder::size, + GeoGridAggregationBuilder2::size, FIELD_SIZE); PARSER.declareInt( - GeoGridAggregationBuilder::shardSize, + GeoGridAggregationBuilder2::shardSize, FIELD_SHARD_SIZE); ValuesSourceParserHelper.declareGeoFields(PARSER, false, false); @@ -113,7 +113,7 @@ private static Object parsePrecision(XContentParser parser, String name) } } - public static GeoGridAggregationBuilder parse(String aggregationName, XContentParser parser) { + public static GeoGridAggregationBuilder2 parse(String aggregationName, XContentParser parser) { return PARSER.apply(parser, aggregationName); } @@ -122,13 +122,13 @@ public static GeoGridAggregationBuilder parse(String aggregationName, XContentPa private int requiredSize = DEFAULT_MAX_NUM_CELLS; private int shardSize = -1; - public GeoGridAggregationBuilder(String name, GeoGridType type) { + public GeoGridAggregationBuilder2(String name, GeoGridType type) { super(name, ValuesSourceType.GEOPOINT, ValueType.GEOPOINT); this.type = type; this.precision = this.type.getDefaultPrecision(); } - protected GeoGridAggregationBuilder(GeoGridAggregationBuilder clone, Builder factoriesBuilder, Map metaData) { + protected GeoGridAggregationBuilder2(GeoGridAggregationBuilder2 clone, Builder factoriesBuilder, Map metaData) { super(clone, factoriesBuilder, metaData); this.type = clone.type; this.precision = clone.precision; @@ -138,13 +138,13 @@ protected GeoGridAggregationBuilder(GeoGridAggregationBuilder clone, Builder fac @Override protected AggregationBuilder shallowCopy(Builder factoriesBuilder, Map metaData) { - return new GeoGridAggregationBuilder(this, factoriesBuilder, metaData); + return new GeoGridAggregationBuilder2(this, factoriesBuilder, metaData); } /** * Read from a stream. */ - public GeoGridAggregationBuilder(StreamInput in) throws IOException { + public GeoGridAggregationBuilder2(StreamInput in) throws IOException { super(in, ValuesSourceType.GEOPOINT, ValueType.GEOPOINT); // FIXME: better debug name than a class name? @@ -170,7 +170,7 @@ public GeoGridType type() { return type; } - private GeoGridAggregationBuilder precisionRaw(Object precision) { + private GeoGridAggregationBuilder2 precisionRaw(Object precision) { if (precision == null) { this.precision(type.getDefaultPrecision()); } else if (precision instanceof String) { @@ -181,7 +181,7 @@ private GeoGridAggregationBuilder precisionRaw(Object precision) { return this; } - public GeoGridAggregationBuilder precision(int precision) { + public GeoGridAggregationBuilder2 precision(int precision) { this.precision = type.validatePrecision(precision); return this; } @@ -190,7 +190,7 @@ public int precision() { return precision; } - public GeoGridAggregationBuilder size(int size) { + public GeoGridAggregationBuilder2 size(int size) { if (size <= 0) { throw new IllegalArgumentException( "[size] must be greater than 0. Found [" + size + "] in [" + name + "]"); @@ -203,7 +203,7 @@ public int size() { return requiredSize; } - public GeoGridAggregationBuilder shardSize(int shardSize) { + public GeoGridAggregationBuilder2 shardSize(int shardSize) { if (shardSize <= 0) { throw new IllegalArgumentException( "[shardSize] must be greater than 0. Found [" + shardSize + "] in [" + name + "]"); @@ -252,7 +252,7 @@ protected XContentBuilder doXContentBody(XContentBuilder builder, Params params) @Override protected boolean innerEquals(Object obj) { - GeoGridAggregationBuilder other = (GeoGridAggregationBuilder) obj; + GeoGridAggregationBuilder2 other = (GeoGridAggregationBuilder2) obj; if (type != other.type) { return false; } diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/geogrid2/GeoGridAggregator.java b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/geogrid2/GeoGridAggregator.java index e13b5883dcd85..cc02119ac2e55 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/geogrid2/GeoGridAggregator.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/geogrid2/GeoGridAggregator.java @@ -44,11 +44,11 @@ public class GeoGridAggregator extends BucketsAggregator { private final int requiredSize; private final int shardSize; - private final GeoGridAggregationBuilder.CellIdSource valuesSource; + private final GeoGridAggregationBuilder2.CellIdSource valuesSource; private final LongHash bucketOrds; private final GeoGridType type; - GeoGridAggregator(String name, AggregatorFactories factories, GeoGridAggregationBuilder.CellIdSource valuesSource, + GeoGridAggregator(String name, AggregatorFactories factories, GeoGridAggregationBuilder2.CellIdSource valuesSource, int requiredSize, int shardSize, SearchContext aggregationContext, Aggregator parent, List pipelineAggregators, Map metaData, GeoGridType type ) throws IOException { diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/geogrid2/GeoGridAggregatorFactory.java b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/geogrid2/GeoGridAggregatorFactory.java index b1d67fc38b2f0..ca26cf91274cf 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/geogrid2/GeoGridAggregatorFactory.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/geogrid2/GeoGridAggregatorFactory.java @@ -24,7 +24,7 @@ import org.elasticsearch.search.aggregations.AggregatorFactory; import org.elasticsearch.search.aggregations.InternalAggregation; import org.elasticsearch.search.aggregations.NonCollectingAggregator; -import org.elasticsearch.search.aggregations.bucket.geogrid2.GeoGridAggregationBuilder.CellIdSource; +import org.elasticsearch.search.aggregations.bucket.geogrid2.GeoGridAggregationBuilder2.CellIdSource; import org.elasticsearch.search.aggregations.pipeline.PipelineAggregator; import org.elasticsearch.search.aggregations.support.ValuesSource; import org.elasticsearch.search.aggregations.support.ValuesSource.GeoPoint; diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/geogrid2/InternalGeoGrid.java b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/geogrid2/InternalGeoGrid.java index 8754759017ff4..6b81de510871e 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/geogrid2/InternalGeoGrid.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/geogrid2/InternalGeoGrid.java @@ -179,7 +179,7 @@ protected void doWriteTo(StreamOutput out) throws IOException { @Override public String getWriteableName() { - return GeoGridAggregationBuilder.NAME; + return GeoGridAggregationBuilder2.NAME; } @Override diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/geogrid2/ParsedGeoGrid.java b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/geogrid2/ParsedGeoGrid.java index 4432b8a165fcd..0f78e78132b69 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/geogrid2/ParsedGeoGrid.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/geogrid2/ParsedGeoGrid.java @@ -32,7 +32,7 @@ public class ParsedGeoGrid extends ParsedMultiBucketAggregation> getAggsTests() { aggsTests.add(new InternalFilterTests()); aggsTests.add(new InternalSamplerTests()); aggsTests.add(new InternalGeoHashGridTests()); + aggsTests.add(new InternalGeoGridGeoHashTests()); aggsTests.add(new InternalRangeTests()); aggsTests.add(new InternalDateRangeTests()); aggsTests.add(new InternalGeoDistanceTests()); diff --git a/server/src/test/java/org/elasticsearch/search/aggregations/bucket/GeoGridIT.java b/server/src/test/java/org/elasticsearch/search/aggregations/bucket/GeoGridIT.java new file mode 100644 index 0000000000000..e7072f402f092 --- /dev/null +++ b/server/src/test/java/org/elasticsearch/search/aggregations/bucket/GeoGridIT.java @@ -0,0 +1,329 @@ +/* + * 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.aggregations.bucket; + +import com.carrotsearch.hppc.ObjectIntHashMap; +import com.carrotsearch.hppc.ObjectIntMap; +import com.carrotsearch.hppc.cursors.ObjectIntCursor; +import org.elasticsearch.Version; +import org.elasticsearch.action.index.IndexRequestBuilder; +import org.elasticsearch.action.search.SearchResponse; +import org.elasticsearch.cluster.metadata.IndexMetaData; +import org.elasticsearch.common.geo.GeoPoint; +import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.common.xcontent.XContentBuilder; +import org.elasticsearch.index.query.GeoBoundingBoxQueryBuilder; +import org.elasticsearch.search.aggregations.AggregationBuilders; +import org.elasticsearch.search.aggregations.InternalAggregation; +import org.elasticsearch.search.aggregations.bucket.filter.Filter; +import org.elasticsearch.search.aggregations.bucket.geogrid2.GeoGrid; +import org.elasticsearch.search.aggregations.bucket.geogrid2.GeoGrid.Bucket; +import org.elasticsearch.test.ESIntegTestCase; +import org.elasticsearch.test.VersionUtils; + +import java.util.ArrayList; +import java.util.Arrays; +import java.util.HashSet; +import java.util.List; +import java.util.Random; +import java.util.Set; + +import static org.elasticsearch.common.geo.GeoHashUtils.PRECISION; +import static org.elasticsearch.common.geo.GeoHashUtils.stringEncode; +import static org.elasticsearch.common.xcontent.XContentFactory.jsonBuilder; +import static org.elasticsearch.search.aggregations.AggregationBuilders.geoGrid; +import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertAcked; +import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertSearchResponse; +import static org.hamcrest.Matchers.containsString; +import static org.hamcrest.Matchers.equalTo; + +@ESIntegTestCase.SuiteScopeTestCase +public class GeoGridIT extends ESIntegTestCase { + + @Override + protected boolean forbidPrivateIndexSettings() { + return false; + } + + private Version version = VersionUtils.randomVersionBetween(random(), Version.V_6_0_0, + Version.CURRENT); + + static ObjectIntMap expectedDocCountsForGeoHash = null; + static ObjectIntMap multiValuedExpectedDocCountsForGeoHash = null; + static int numDocs = 100; + + static String smallestGeoHash = null; + + private static IndexRequestBuilder indexCity(String index, String name, List latLon) throws Exception { + XContentBuilder source = jsonBuilder().startObject().field("city", name); + if (latLon != null) { + source = source.field("location", latLon); + } + source = source.endObject(); + return client().prepareIndex(index, "type").setSource(source); + } + + private static IndexRequestBuilder indexCity(String index, String name, String latLon) throws Exception { + return indexCity(index, name, Arrays.asList(latLon)); + } + + @Override + public void setupSuiteScopeCluster() throws Exception { + createIndex("idx_unmapped"); + + Settings settings = Settings.builder().put(IndexMetaData.SETTING_VERSION_CREATED, version).build(); + + assertAcked(prepareCreate("idx").setSettings(settings) + .addMapping("type", "location", "type=geo_point", "city", "type=keyword")); + + List cities = new ArrayList<>(); + Random random = random(); + expectedDocCountsForGeoHash = new ObjectIntHashMap<>(numDocs * 2); + for (int i = 0; i < numDocs; i++) { + //generate random point + double lat = (180d * random.nextDouble()) - 90d; + double lng = (360d * random.nextDouble()) - 180d; + String randomGeoHash = stringEncode(lng, lat, PRECISION); + //Index at the highest resolution + cities.add(indexCity("idx", randomGeoHash, lat + ", " + lng)); + expectedDocCountsForGeoHash.put(randomGeoHash, expectedDocCountsForGeoHash.getOrDefault(randomGeoHash, 0) + 1); + //Update expected doc counts for all resolutions.. + for (int precision = PRECISION - 1; precision > 0; precision--) { + String hash = stringEncode(lng, lat, precision); + if ((smallestGeoHash == null) || (hash.length() < smallestGeoHash.length())) { + smallestGeoHash = hash; + } + expectedDocCountsForGeoHash.put(hash, expectedDocCountsForGeoHash.getOrDefault(hash, 0) + 1); + } + } + indexRandom(true, cities); + + assertAcked(prepareCreate("multi_valued_idx").setSettings(settings) + .addMapping("type", "location", "type=geo_point", "city", "type=keyword")); + + cities = new ArrayList<>(); + multiValuedExpectedDocCountsForGeoHash = new ObjectIntHashMap<>(numDocs * 2); + for (int i = 0; i < numDocs; i++) { + final int numPoints = random.nextInt(4); + List points = new ArrayList<>(); + Set geoHashes = new HashSet<>(); + for (int j = 0; j < numPoints; ++j) { + double lat = (180d * random.nextDouble()) - 90d; + double lng = (360d * random.nextDouble()) - 180d; + points.add(lat + "," + lng); + // Update expected doc counts for all resolutions.. + for (int precision = PRECISION; precision > 0; precision--) { + final String geoHash = stringEncode(lng, lat, precision); + geoHashes.add(geoHash); + } + } + cities.add(indexCity("multi_valued_idx", Integer.toString(i), points)); + for (String hash : geoHashes) { + multiValuedExpectedDocCountsForGeoHash.put(hash, multiValuedExpectedDocCountsForGeoHash.getOrDefault(hash, 0) + 1); + } + } + indexRandom(true, cities); + + ensureSearchable(); + } + + public void testSimple() throws Exception { + for (int precision = 1; precision <= PRECISION; precision++) { + SearchResponse response = client().prepareSearch("idx") + .addAggregation(geoGrid("geogrid", GeoGridTests.GEOHASH_TYPE) + .field("location") + .precision(precision) + ) + .get(); + + assertSearchResponse(response); + + GeoGrid geoGrid = response.getAggregations().get("geogrid"); + List buckets = geoGrid.getBuckets(); + Object[] propertiesKeys = (Object[]) ((InternalAggregation)geoGrid).getProperty("_key"); + Object[] propertiesDocCounts = (Object[]) ((InternalAggregation)geoGrid).getProperty("_count"); + for (int i = 0; i < buckets.size(); i++) { + Bucket cell = buckets.get(i); + String geohash = cell.getKeyAsString(); + + long bucketCount = cell.getDocCount(); + int expectedBucketCount = expectedDocCountsForGeoHash.get(geohash); + assertNotSame(bucketCount, 0); + assertEquals("Geohash " + geohash + " has wrong doc count ", + expectedBucketCount, bucketCount); + GeoPoint geoPoint = (GeoPoint) propertiesKeys[i]; + assertThat(stringEncode(geoPoint.lon(), geoPoint.lat(), precision), equalTo(geohash)); + assertThat((long) propertiesDocCounts[i], equalTo(bucketCount)); + } + } + } + + public void testMultivalued() throws Exception { + for (int precision = 1; precision <= PRECISION; precision++) { + SearchResponse response = client().prepareSearch("multi_valued_idx") + .addAggregation(geoGrid("geogrid", GeoGridTests.GEOHASH_TYPE) + .field("location") + .precision(precision) + ) + .get(); + + assertSearchResponse(response); + + GeoGrid geoGrid = response.getAggregations().get("geogrid"); + for (Bucket cell : geoGrid.getBuckets()) { + String geohash = cell.getKeyAsString(); + + long bucketCount = cell.getDocCount(); + int expectedBucketCount = multiValuedExpectedDocCountsForGeoHash.get(geohash); + assertNotSame(bucketCount, 0); + assertEquals("Geogrid " + geohash + " has wrong doc count ", + expectedBucketCount, bucketCount); + } + } + } + + public void testFiltered() throws Exception { + GeoBoundingBoxQueryBuilder bbox = new GeoBoundingBoxQueryBuilder("location"); + bbox.setCorners(smallestGeoHash).queryName("bbox"); + for (int precision = 1; precision <= PRECISION; precision++) { + SearchResponse response = client().prepareSearch("idx") + .addAggregation( + AggregationBuilders.filter("filtered", bbox) + .subAggregation( + geoGrid("geogrid", GeoGridTests.GEOHASH_TYPE) + .field("location") + .precision(precision) + ) + ) + .get(); + + assertSearchResponse(response); + + Filter filter = response.getAggregations().get("filtered"); + + GeoGrid geoGrid = filter.getAggregations().get("geogrid"); + for (Bucket cell : geoGrid.getBuckets()) { + String geohash = cell.getKeyAsString(); + long bucketCount = cell.getDocCount(); + int expectedBucketCount = expectedDocCountsForGeoHash.get(geohash); + assertNotSame(bucketCount, 0); + assertTrue("Buckets must be filtered", geohash.startsWith(smallestGeoHash)); + assertEquals("Geohash " + geohash + " has wrong doc count ", + expectedBucketCount, bucketCount); + + } + } + } + + public void testUnmapped() throws Exception { + for (int precision = 1; precision <= PRECISION; precision++) { + SearchResponse response = client().prepareSearch("idx_unmapped") + .addAggregation(geoGrid("geogrid", GeoGridTests.GEOHASH_TYPE) + .field("location") + .precision(precision) + ) + .get(); + + assertSearchResponse(response); + + GeoGrid geoGrid = response.getAggregations().get("geogrid"); + assertThat(geoGrid.getBuckets().size(), equalTo(0)); + } + + } + + public void testPartiallyUnmapped() throws Exception { + for (int precision = 1; precision <= PRECISION; precision++) { + SearchResponse response = client().prepareSearch("idx", "idx_unmapped") + .addAggregation(geoGrid("geogrid", GeoGridTests.GEOHASH_TYPE) + .field("location") + .precision(precision) + ) + .get(); + + assertSearchResponse(response); + + GeoGrid geoGrid = response.getAggregations().get("geogrid"); + for (Bucket cell : geoGrid.getBuckets()) { + String geohash = cell.getKeyAsString(); + + long bucketCount = cell.getDocCount(); + int expectedBucketCount = expectedDocCountsForGeoHash.get(geohash); + assertNotSame(bucketCount, 0); + assertEquals("Geohash " + geohash + " has wrong doc count ", + expectedBucketCount, bucketCount); + } + } + } + + public void testTopMatch() throws Exception { + for (int precision = 1; precision <= PRECISION; precision++) { + SearchResponse response = client().prepareSearch("idx") + .addAggregation(geoGrid("geogrid", GeoGridTests.GEOHASH_TYPE) + .field("location") + .size(1) + .shardSize(100) + .precision(precision) + ) + .get(); + + assertSearchResponse(response); + + GeoGrid geoGrid = response.getAggregations().get("geogrid"); + //Check we only have one bucket with the best match for that resolution + assertThat(geoGrid.getBuckets().size(), equalTo(1)); + for (Bucket cell : geoGrid.getBuckets()) { + String geohash = cell.getKeyAsString(); + long bucketCount = cell.getDocCount(); + int expectedBucketCount = 0; + for (ObjectIntCursor cursor : expectedDocCountsForGeoHash) { + if (cursor.key.length() == precision) { + expectedBucketCount = Math.max(expectedBucketCount, cursor.value); + } + } + assertNotSame(bucketCount, 0); + assertEquals("Geohash " + geohash + " has wrong doc count ", + expectedBucketCount, bucketCount); + } + } + } + + public void testSizeIsZero() { + final int size = 0; + final int shardSize = 10000; + IllegalArgumentException exception = expectThrows(IllegalArgumentException.class, + () -> client() + .prepareSearch("idx") + .addAggregation( + geoGrid("geogrid", GeoGridTests.GEOHASH_TYPE) + .field("location").size(size).shardSize(shardSize)).get()); + assertThat(exception.getMessage(), containsString("[size] must be greater than 0. Found [0] in [geogrid]")); + } + + public void testShardSizeIsZero() { + final int size = 100; + final int shardSize = 0; + IllegalArgumentException exception = expectThrows(IllegalArgumentException.class, + () -> client().prepareSearch("idx") + .addAggregation(geoGrid("geogrid", GeoGridTests.GEOHASH_TYPE).field("location").size(size).shardSize(shardSize)) + .get()); + assertThat(exception.getMessage(), containsString("[shardSize] must be greater than 0. Found [0] in [geogrid]")); + } + +} diff --git a/server/src/test/java/org/elasticsearch/search/aggregations/bucket/GeoGridTests.java b/server/src/test/java/org/elasticsearch/search/aggregations/bucket/GeoGridTests.java new file mode 100644 index 0000000000000..3e84acefaeb08 --- /dev/null +++ b/server/src/test/java/org/elasticsearch/search/aggregations/bucket/GeoGridTests.java @@ -0,0 +1,73 @@ +/* + * 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.aggregations.bucket; + +import org.elasticsearch.search.aggregations.BaseAggregationTestCase; +import org.elasticsearch.search.aggregations.bucket.geogrid2.GeoGridAggregationBuilder2; +import org.elasticsearch.search.aggregations.bucket.geogrid2.GeoGridType; +import org.elasticsearch.search.aggregations.bucket.geogrid2.GeoHashType; + +public class GeoGridTests extends BaseAggregationTestCase { + + public static final GeoGridType GEOHASH_TYPE = new GeoHashType(); + + /** + * Pick a random hash type + */ + public static GeoGridType randomType() { + // With more types, will use randomIntBetween() to pick one + return GEOHASH_TYPE; + } + + /** + * Pick a random precision for the given hash type. + */ + public static int randomPrecision(final GeoGridType type) { + if (type.getClass() == GeoHashType.class) { + return randomIntBetween(1, 12); + } + throw new IllegalArgumentException(type.getClass() + " was not added to the test"); + } + + public static int maxPrecision(GeoGridType type) { + if (type.getClass() == GeoHashType.class) { + return 12; + } + throw new IllegalArgumentException(type.getClass() + " was not added to the test"); + } + + @Override + protected GeoGridAggregationBuilder2 createTestAggregatorBuilder() { + String name = randomAlphaOfLengthBetween(3, 20); + GeoGridType type = randomType(); + GeoGridAggregationBuilder2 factory = new GeoGridAggregationBuilder2(name, type); + if (randomBoolean()) { + factory.precision(randomPrecision(factory.type())); + } + if (randomBoolean()) { + factory.size(randomIntBetween(1, Integer.MAX_VALUE)); + } + if (randomBoolean()) { + factory.shardSize(randomIntBetween(1, Integer.MAX_VALUE)); + } + return factory; + } + +} diff --git a/server/src/test/java/org/elasticsearch/search/aggregations/bucket/ShardReduceIT.java b/server/src/test/java/org/elasticsearch/search/aggregations/bucket/ShardReduceIT.java index 18ab80305dd15..8fbb6f6edd80b 100644 --- a/server/src/test/java/org/elasticsearch/search/aggregations/bucket/ShardReduceIT.java +++ b/server/src/test/java/org/elasticsearch/search/aggregations/bucket/ShardReduceIT.java @@ -25,6 +25,7 @@ import org.elasticsearch.search.aggregations.Aggregator.SubAggCollectionMode; import org.elasticsearch.search.aggregations.bucket.filter.Filter; import org.elasticsearch.search.aggregations.bucket.geogrid.GeoHashGrid; +import org.elasticsearch.search.aggregations.bucket.geogrid2.GeoGrid; import org.elasticsearch.search.aggregations.bucket.global.Global; import org.elasticsearch.search.aggregations.bucket.histogram.DateHistogramInterval; import org.elasticsearch.search.aggregations.bucket.histogram.Histogram; @@ -39,6 +40,7 @@ import static org.elasticsearch.search.aggregations.AggregationBuilders.dateRange; import static org.elasticsearch.search.aggregations.AggregationBuilders.filter; import static org.elasticsearch.search.aggregations.AggregationBuilders.geohashGrid; +import static org.elasticsearch.search.aggregations.AggregationBuilders.geoGrid; import static org.elasticsearch.search.aggregations.AggregationBuilders.global; import static org.elasticsearch.search.aggregations.AggregationBuilders.histogram; import static org.elasticsearch.search.aggregations.AggregationBuilders.ipRange; @@ -307,4 +309,19 @@ public void testGeoHashGrid() throws Exception { } + public void testGeoGrid() throws Exception { + SearchResponse response = client().prepareSearch("idx") + .setQuery(QueryBuilders.matchAllQuery()) + .addAggregation(geoGrid("grid", GeoGridTests.GEOHASH_TYPE).field("location") + .subAggregation(dateHistogram("histo").field("date").dateHistogramInterval(DateHistogramInterval.DAY) + .minDocCount(0))) + .get(); + + assertSearchResponse(response); + + GeoGrid grid = response.getAggregations().get("grid"); + Histogram histo = grid.getBuckets().iterator().next().getAggregations().get("histo"); + assertThat(histo.getBuckets().size(), equalTo(4)); + } + } diff --git a/server/src/test/java/org/elasticsearch/search/aggregations/bucket/geogrid2/GeoGridAggregatorTests.java b/server/src/test/java/org/elasticsearch/search/aggregations/bucket/geogrid2/GeoGridAggregatorTests.java new file mode 100644 index 0000000000000..9b7f5e67721af --- /dev/null +++ b/server/src/test/java/org/elasticsearch/search/aggregations/bucket/geogrid2/GeoGridAggregatorTests.java @@ -0,0 +1,134 @@ +/* + * 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.aggregations.bucket.geogrid2; + +import org.apache.lucene.document.LatLonDocValuesField; +import org.apache.lucene.geo.GeoEncodingUtils; +import org.apache.lucene.index.DirectoryReader; +import org.apache.lucene.index.IndexReader; +import org.apache.lucene.index.RandomIndexWriter; +import org.apache.lucene.search.IndexSearcher; +import org.apache.lucene.search.MatchAllDocsQuery; +import org.apache.lucene.search.Query; +import org.apache.lucene.store.Directory; +import org.elasticsearch.common.CheckedConsumer; +import org.elasticsearch.index.mapper.GeoPointFieldMapper; +import org.elasticsearch.index.mapper.MappedFieldType; +import org.elasticsearch.search.aggregations.Aggregator; +import org.elasticsearch.search.aggregations.AggregatorTestCase; +import org.elasticsearch.search.aggregations.bucket.GeoGridTests; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Collections; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.function.Consumer; + +public class GeoGridAggregatorTests extends AggregatorTestCase { + + private static final String FIELD_NAME = "location"; + + public void testNoDocs() throws IOException { + final GeoGridType type = GeoGridTests.randomType(); + testCase(new MatchAllDocsQuery(), FIELD_NAME, type, GeoGridTests.randomPrecision(type), iw -> { + // Intentionally not writing any docs + }, geoHashGrid -> assertEquals(0, geoHashGrid.getBuckets().size())); + } + + public void testFieldMissing() throws IOException { + final GeoGridType type = GeoGridTests.randomType(); + testCase(new MatchAllDocsQuery(), "wrong_field", type, GeoGridTests.randomPrecision(type), iw -> { + iw.addDocument(Collections.singleton(new LatLonDocValuesField(FIELD_NAME, 10D, 10D))); + }, geoHashGrid -> assertEquals(0, geoHashGrid.getBuckets().size())); + } + + public void testHashcodeWithSeveralDocs() throws IOException { + final GeoGridType type = GeoGridTests.randomType(); + testWithSeveralDocs(type, GeoGridTests.randomPrecision(type)); + } + + private void testWithSeveralDocs(GeoGridType type, int precision) + throws IOException { + int numPoints = randomIntBetween(8, 128); + Map expectedCountPerGeoHash = new HashMap<>(); + testCase(new MatchAllDocsQuery(), FIELD_NAME, type, precision, iw -> { + List points = new ArrayList<>(); + Set distinctHashesPerDoc = new HashSet<>(); + for (int pointId = 0; pointId < numPoints; pointId++) { + double lat = (180d * randomDouble()) - 90d; + double lng = (360d * randomDouble()) - 180d; + + // Precision-adjust longitude/latitude to avoid wrong bucket placement + lng = GeoEncodingUtils.decodeLongitude(GeoEncodingUtils.encodeLongitude(lng)); + lat = GeoEncodingUtils.decodeLatitude(GeoEncodingUtils.encodeLatitude(lat)); + + points.add(new LatLonDocValuesField(FIELD_NAME, lat, lng)); + String hash = type.hashAsString(type.calculateHash(lng, lat, precision)); + if (distinctHashesPerDoc.contains(hash) == false) { + expectedCountPerGeoHash.put(hash, expectedCountPerGeoHash.getOrDefault(hash, 0) + 1); + } + distinctHashesPerDoc.add(hash); + if (usually()) { + iw.addDocument(points); + points.clear(); + distinctHashesPerDoc.clear(); + } + } + if (points.size() != 0) { + iw.addDocument(points); + } + }, geoHashGrid -> { + assertEquals(expectedCountPerGeoHash.size(), geoHashGrid.getBuckets().size()); + for (GeoGrid.Bucket bucket : geoHashGrid.getBuckets()) { + assertEquals((long) expectedCountPerGeoHash.get(bucket.getKeyAsString()), bucket.getDocCount()); + } + }); + } + + private void testCase(Query query, String field, GeoGridType type, int precision, + CheckedConsumer buildIndex, + Consumer verify) throws IOException { + Directory directory = newDirectory(); + RandomIndexWriter indexWriter = new RandomIndexWriter(random(), directory); + buildIndex.accept(indexWriter); + indexWriter.close(); + + IndexReader indexReader = DirectoryReader.open(directory); + IndexSearcher indexSearcher = newSearcher(indexReader, true, true); + + GeoGridAggregationBuilder2 aggregationBuilder = new GeoGridAggregationBuilder2("_name", type).field(field); + aggregationBuilder.precision(precision); + MappedFieldType fieldType = new GeoPointFieldMapper.GeoPointFieldType(); + fieldType.setHasDocValues(true); + fieldType.setName(FIELD_NAME); + + Aggregator aggregator = createAggregator(aggregationBuilder, indexSearcher, fieldType); + aggregator.preCollection(); + indexSearcher.search(query, aggregator); + aggregator.postCollection(); + verify.accept((InternalGeoGrid) aggregator.buildAggregation(0L)); + + indexReader.close(); + directory.close(); + } +} diff --git a/server/src/test/java/org/elasticsearch/search/aggregations/bucket/geogrid2/GeoGridParserTests.java b/server/src/test/java/org/elasticsearch/search/aggregations/bucket/geogrid2/GeoGridParserTests.java new file mode 100644 index 0000000000000..f3793028265fb --- /dev/null +++ b/server/src/test/java/org/elasticsearch/search/aggregations/bucket/geogrid2/GeoGridParserTests.java @@ -0,0 +1,149 @@ +/* + * 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.aggregations.bucket.geogrid2; + +import org.elasticsearch.common.unit.DistanceUnit; +import org.elasticsearch.common.xcontent.XContentParseException; +import org.elasticsearch.common.xcontent.XContentParser; +import org.elasticsearch.common.xcontent.json.JsonXContent; +import org.elasticsearch.search.aggregations.bucket.GeoGridTests; +import org.elasticsearch.test.ESTestCase; + +import static org.hamcrest.Matchers.containsString; +import static org.hamcrest.Matchers.greaterThanOrEqualTo; +import static org.hamcrest.Matchers.instanceOf; +import static org.hamcrest.Matchers.lessThanOrEqualTo; + +public class GeoGridParserTests extends ESTestCase { + public void testParseValidFromInts() throws Exception { + GeoGridType type = GeoGridTests.randomType(); + int precision = GeoGridTests.randomPrecision(type); + XContentParser stParser = createParser(JsonXContent.jsonXContent, + "{\"field\":\"my_loc\", \"hash_type\":\"" + type + "\", \"precision\":" + precision + + ", \"size\": 500, \"shard_size\": 550}"); + XContentParser.Token token = stParser.nextToken(); + assertSame(XContentParser.Token.START_OBJECT, token); + // can create a factory + assertNotNull(GeoGridAggregationBuilder2.parse("geohash_grid", stParser)); + } + + public void testParseValidFromStrings() throws Exception { + GeoGridType type = GeoGridTests.randomType(); + int precision = GeoGridTests.randomPrecision(type); + XContentParser stParser = createParser(JsonXContent.jsonXContent, + "{\"field\":\"my_loc\", \"hash_type\":\"" + type + "\", \"precision\":\"" + precision + + "\", \"size\": \"500\", \"shard_size\": \"550\"}"); + XContentParser.Token token = stParser.nextToken(); + assertSame(XContentParser.Token.START_OBJECT, token); + // can create a factory + assertNotNull(GeoGridAggregationBuilder2.parse("geohash_grid", stParser)); + } + + public void testParseDistanceUnitPrecision() throws Exception { + double distance = randomDoubleBetween(10.0, 100.00, true); + DistanceUnit unit = randomFrom(DistanceUnit.values()); + if (unit.equals(DistanceUnit.MILLIMETERS)) { + distance = 5600 + randomDouble(); // 5.6cm is approx. smallest distance represented by precision 12 + } + String distanceString = distance + unit.toString(); + XContentParser stParser = createParser(JsonXContent.jsonXContent, + "{\"field\":\"my_loc\", \"precision\": \"" + distanceString + "\", \"size\": \"500\", \"shard_size\": \"550\"}"); + XContentParser.Token token = stParser.nextToken(); + assertSame(XContentParser.Token.START_OBJECT, token); + // can create a factory + GeoGridAggregationBuilder2 builder = GeoGridAggregationBuilder2.parse("geohash_grid", stParser); + assertNotNull(builder); + assertThat(builder.precision(), greaterThanOrEqualTo(0)); + assertThat(builder.precision(), lessThanOrEqualTo(12)); + } + + public void testParseInvalidUnitPrecision() throws Exception { + XContentParser stParser = createParser(JsonXContent.jsonXContent, + "{\"field\":\"my_loc\", \"precision\": \"10kg\", \"size\": \"500\", \"shard_size\": \"550\"}"); + XContentParser.Token token = stParser.nextToken(); + assertSame(XContentParser.Token.START_OBJECT, token); + XContentParseException ex = expectThrows(XContentParseException.class, + () -> GeoGridAggregationBuilder2.parse("geohash_grid", stParser)); + assertThat(ex.getMessage(), containsString("failed to build [geohash_grid] after last required field arrived")); + + Throwable cause = ex.getCause(); + assertThat(cause, instanceOf(XContentParseException.class)); + assertThat(cause.getMessage(), containsString("[geohash_grid] failed to parse field [precision]")); + + cause = cause.getCause(); + assertThat(cause, instanceOf(NumberFormatException.class)); + assertThat(cause.getMessage(), containsString("For input string: \"10kg\"")); + } + + public void testParseDistanceUnitPrecisionTooSmall() throws Exception { + XContentParser stParser = createParser(JsonXContent.jsonXContent, + "{\"field\":\"my_loc\", \"precision\": \"1cm\", \"size\": \"500\", \"shard_size\": \"550\"}"); + XContentParser.Token token = stParser.nextToken(); + assertSame(XContentParser.Token.START_OBJECT, token); + XContentParseException ex = expectThrows(XContentParseException.class, + () -> GeoGridAggregationBuilder2.parse("geohash_grid", stParser)); + assertThat(ex.getMessage(), containsString("failed to build [geohash_grid] after last required field arrived")); + + Throwable cause = ex.getCause(); + assertThat(cause, instanceOf(XContentParseException.class)); + assertThat(cause.getMessage(), containsString("[geohash_grid] failed to parse field [precision]")); + + cause = cause.getCause(); + assertThat(cause, instanceOf(IllegalArgumentException.class)); + assertEquals("precision too high [1cm]", cause.getMessage()); + } + + public void testParseErrorOnBooleanPrecision() throws Exception { + GeoGridType type = GeoGridTests.randomType(); + XContentParser stParser = createParser(JsonXContent.jsonXContent, + "{\"field\":\"my_loc\", \"hash_type\":\"" + type + "\", \"precision\":false}"); + XContentParser.Token token = stParser.nextToken(); + assertSame(XContentParser.Token.START_OBJECT, token); + XContentParseException ex = expectThrows(XContentParseException.class, + () -> GeoGridAggregationBuilder2.parse("geohash_grid", stParser)); + assertThat(ex.getMessage(), containsString("[geohash_grid] failed to parse field [precision]")); + + Throwable cause = ex.getCause(); + assertThat(cause, instanceOf(XContentParseException.class)); + assertThat(cause.getMessage(), containsString("[geohash_grid] failed to parse field [precision]" + + " in [geohash_grid]. It must be either an integer or a string")); + } + + public void testParseErrorOnPrecisionOutOfRange() throws Exception { + final GeoGridType type = GeoGridTests.randomType(); + final int precision = GeoGridTests.maxPrecision(type) + 1; + XContentParser stParser = createParser(JsonXContent.jsonXContent, + "{\"field\":\"my_loc\", \"hash_type\":\"" + type + "\", \"precision\":\""+ precision +"\"}"); + XContentParser.Token token = stParser.nextToken(); + assertSame(XContentParser.Token.START_OBJECT, token); + try { + GeoGridAggregationBuilder2.parse("geohash_grid", stParser); + fail(); + } catch (XContentParseException ex) { + assertThat(ex.getCause(), instanceOf(IllegalArgumentException.class)); + String expectedMsg; + if (type.getClass() == GeoHashType.class) { + expectedMsg = "Invalid geohash aggregation precision of 13. Must be between 1 and 12."; + } else { + throw new IllegalArgumentException(type.getClass() + " was not added to the test"); + } + assertEquals(expectedMsg, ex.getCause().getMessage()); + } + } +} diff --git a/server/src/test/java/org/elasticsearch/search/aggregations/bucket/geogrid2/InternalGeoGridGeoHashTests.java b/server/src/test/java/org/elasticsearch/search/aggregations/bucket/geogrid2/InternalGeoGridGeoHashTests.java new file mode 100644 index 0000000000000..8acc6d1780776 --- /dev/null +++ b/server/src/test/java/org/elasticsearch/search/aggregations/bucket/geogrid2/InternalGeoGridGeoHashTests.java @@ -0,0 +1,144 @@ +/* + * 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.aggregations.bucket.geogrid2; + +import org.apache.lucene.index.IndexWriter; +import org.elasticsearch.common.geo.GeoHashUtils; +import org.elasticsearch.common.io.stream.Writeable; +import org.elasticsearch.search.aggregations.InternalAggregations; +import org.elasticsearch.search.aggregations.bucket.GeoGridTests; +import org.elasticsearch.test.InternalMultiBucketAggregationTestCase; +import org.elasticsearch.search.aggregations.ParsedMultiBucketAggregation; +import org.elasticsearch.search.aggregations.bucket.geogrid2.InternalGeoGrid.Bucket; +import org.elasticsearch.search.aggregations.pipeline.PipelineAggregator; + +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +public class InternalGeoGridGeoHashTests extends InternalMultiBucketAggregationTestCase { + + @Override + protected int minNumberOfBuckets() { + return 1; + } + + @Override + protected int maxNumberOfBuckets() { + return 3; + } + + @Override + protected InternalGeoGrid createTestInstance(String name, + List pipelineAggregators, + Map metaData, + InternalAggregations aggregations) { + int size = randomNumberOfBuckets(); + List buckets = new ArrayList<>(size); + for (int i = 0; i < size; i++) { + double latitude = randomDoubleBetween(-90.0, 90.0, false); + double longitude = randomDoubleBetween(-180.0, 180.0, false); + + long geoHashAsLong = GeoHashUtils.longEncode(longitude, latitude, 4); + buckets.add(new InternalGeoGrid.Bucket(geoHashAsLong, randomInt(IndexWriter.MAX_DOCS), aggregations)); + } + return new InternalGeoGrid(name, GeoGridTests.GEOHASH_TYPE, size, buckets, pipelineAggregators, metaData); + } + + @Override + protected Writeable.Reader instanceReader() { + return InternalGeoGrid::new; + } + + @Override + protected void assertReduced(InternalGeoGrid reduced, List inputs) { + Map> map = new HashMap<>(); + for (InternalGeoGrid input : inputs) { + for (GeoGrid.Bucket bucket : input.getBuckets()) { + InternalGeoGrid.Bucket internalBucket = (InternalGeoGrid.Bucket) bucket; + List buckets = map.computeIfAbsent(internalBucket.hashAsLong, k -> new ArrayList<>()); + buckets.add(internalBucket); + } + } + List expectedBuckets = new ArrayList<>(); + for (Map.Entry> entry : map.entrySet()) { + long docCount = 0; + for (InternalGeoGrid.Bucket bucket : entry.getValue()) { + docCount += bucket.docCount; + } + expectedBuckets.add(new InternalGeoGrid.Bucket(entry.getKey(), docCount, InternalAggregations.EMPTY)); + } + expectedBuckets.sort((first, second) -> { + int cmp = Long.compare(second.docCount, first.docCount); + if (cmp == 0) { + return second.compareTo(first); + } + return cmp; + }); + int requestedSize = inputs.get(0).getRequiredSize(); + expectedBuckets = expectedBuckets.subList(0, Math.min(requestedSize, expectedBuckets.size())); + assertEquals(expectedBuckets.size(), reduced.getBuckets().size()); + for (int i = 0; i < reduced.getBuckets().size(); i++) { + GeoGrid.Bucket expected = expectedBuckets.get(i); + GeoGrid.Bucket actual = reduced.getBuckets().get(i); + assertEquals(expected.getDocCount(), actual.getDocCount()); + assertEquals(expected.getKey(), actual.getKey()); + } + } + + @Override + protected Class implementationClass() { + return ParsedGeoGrid.class; + } + + @Override + protected InternalGeoGrid mutateInstance(InternalGeoGrid instance) { + String name = instance.getName(); + int size = instance.getRequiredSize(); + List buckets = instance.getBuckets(); + List pipelineAggregators = instance.pipelineAggregators(); + Map metaData = instance.getMetaData(); + switch (between(0, 3)) { + case 0: + name += randomAlphaOfLength(5); + break; + case 1: + buckets = new ArrayList<>(buckets); + buckets.add( + new InternalGeoGrid.Bucket(randomNonNegativeLong(), randomInt(IndexWriter.MAX_DOCS), InternalAggregations.EMPTY)); + break; + case 2: + size = size + between(1, 10); + break; + case 3: + if (metaData == null) { + metaData = new HashMap<>(1); + } else { + metaData = new HashMap<>(instance.getMetaData()); + } + metaData.put(randomAlphaOfLength(15), randomInt()); + break; + default: + throw new AssertionError("Illegal randomisation branch"); + } + return new InternalGeoGrid(name, GeoGridTests.GEOHASH_TYPE, size, buckets, pipelineAggregators, metaData); + } + +} diff --git a/server/src/test/java/org/elasticsearch/search/aggregations/metrics/GeoCentroidIT.java b/server/src/test/java/org/elasticsearch/search/aggregations/metrics/GeoCentroidIT.java index dfc503219ed74..bd28d13d01682 100644 --- a/server/src/test/java/org/elasticsearch/search/aggregations/metrics/GeoCentroidIT.java +++ b/server/src/test/java/org/elasticsearch/search/aggregations/metrics/GeoCentroidIT.java @@ -22,7 +22,9 @@ import org.elasticsearch.action.search.SearchResponse; import org.elasticsearch.common.geo.GeoPoint; import org.elasticsearch.search.aggregations.InternalAggregation; +import org.elasticsearch.search.aggregations.bucket.GeoGridTests; import org.elasticsearch.search.aggregations.bucket.geogrid.GeoHashGrid; +import org.elasticsearch.search.aggregations.bucket.geogrid2.GeoGrid; import org.elasticsearch.search.aggregations.bucket.global.Global; import org.elasticsearch.test.ESIntegTestCase; @@ -31,6 +33,7 @@ import static org.elasticsearch.index.query.QueryBuilders.matchAllQuery; import static org.elasticsearch.search.aggregations.AggregationBuilders.geoCentroid; import static org.elasticsearch.search.aggregations.AggregationBuilders.geohashGrid; +import static org.elasticsearch.search.aggregations.AggregationBuilders.geoGrid; import static org.elasticsearch.search.aggregations.AggregationBuilders.global; import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertSearchResponse; import static org.hamcrest.Matchers.closeTo; @@ -173,4 +176,26 @@ public void testSingleValueFieldAsSubAggToGeohashGrid() throws Exception { closeTo(centroidAgg.centroid().lon(), GEOHASH_TOLERANCE)); } } + + public void testSingleValueFieldAsSubAggToGeoGrid() throws Exception { + SearchResponse response = client().prepareSearch(HIGH_CARD_IDX_NAME) + .addAggregation(geoGrid("geoGrid", GeoGridTests.GEOHASH_TYPE).field(SINGLE_VALUED_FIELD_NAME) + .subAggregation(geoCentroid(aggName).field(SINGLE_VALUED_FIELD_NAME))) + .get(); + assertSearchResponse(response); + + GeoGrid grid = response.getAggregations().get("geoGrid"); + assertThat(grid, notNullValue()); + assertThat(grid.getName(), equalTo("geoGrid")); + List buckets = grid.getBuckets(); + for (GeoGrid.Bucket cell : buckets) { + String geohash = cell.getKeyAsString(); + GeoPoint expectedCentroid = expectedCentroidsForGeoHash.get(geohash); + GeoCentroid centroidAgg = cell.getAggregations().get(aggName); + assertThat("Geohash " + geohash + " has wrong centroid latitude ", expectedCentroid.lat(), + closeTo(centroidAgg.centroid().lat(), GEOHASH_TOLERANCE)); + assertThat("Geohash " + geohash + " has wrong centroid longitude", expectedCentroid.lon(), + closeTo(centroidAgg.centroid().lon(), GEOHASH_TOLERANCE)); + } + } } diff --git a/test/framework/src/main/java/org/elasticsearch/test/InternalAggregationTestCase.java b/test/framework/src/main/java/org/elasticsearch/test/InternalAggregationTestCase.java index 551110ca2520a..f9f197b7d5c3f 100644 --- a/test/framework/src/main/java/org/elasticsearch/test/InternalAggregationTestCase.java +++ b/test/framework/src/main/java/org/elasticsearch/test/InternalAggregationTestCase.java @@ -51,6 +51,8 @@ import org.elasticsearch.search.aggregations.bucket.filter.ParsedFilters; import org.elasticsearch.search.aggregations.bucket.geogrid.GeoGridAggregationBuilder; import org.elasticsearch.search.aggregations.bucket.geogrid.ParsedGeoHashGrid; +import org.elasticsearch.search.aggregations.bucket.geogrid2.GeoGridAggregationBuilder2; +import org.elasticsearch.search.aggregations.bucket.geogrid2.ParsedGeoGrid; import org.elasticsearch.search.aggregations.bucket.global.GlobalAggregationBuilder; import org.elasticsearch.search.aggregations.bucket.global.ParsedGlobal; import org.elasticsearch.search.aggregations.bucket.histogram.AutoDateHistogramAggregationBuilder; @@ -212,6 +214,7 @@ public abstract class InternalAggregationTestCase map.put(FilterAggregationBuilder.NAME, (p, c) -> ParsedFilter.fromXContent(p, (String) c)); map.put(InternalSampler.PARSER_NAME, (p, c) -> ParsedSampler.fromXContent(p, (String) c)); map.put(GeoGridAggregationBuilder.NAME, (p, c) -> ParsedGeoHashGrid.fromXContent(p, (String) c)); + map.put(GeoGridAggregationBuilder2.NAME, (p, c) -> ParsedGeoGrid.fromXContent(p, (String) c)); map.put(RangeAggregationBuilder.NAME, (p, c) -> ParsedRange.fromXContent(p, (String) c)); map.put(DateRangeAggregationBuilder.NAME, (p, c) -> ParsedDateRange.fromXContent(p, (String) c)); map.put(GeoDistanceAggregationBuilder.NAME, (p, c) -> ParsedGeoDistance.fromXContent(p, (String) c)); From 2a79c468f8624c3ccb452884dfd99e639d2d47da Mon Sep 17 00:00:00 2001 From: Ioannis Kakavas Date: Wed, 9 Jan 2019 07:19:34 +0200 Subject: [PATCH 14/46] Ensure that ActionListener is called exactly once This bug was introduced in #36893 and had the effect that execution would continue after calling onFailure on the the listener in checkIfTokenIsValid in the case that the token is expired. In a case of many consecutive requests this could lead to the unwelcome side effect of an expired access token producing a successful authentication response. --- .../org/elasticsearch/xpack/security/authc/TokenService.java | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/authc/TokenService.java b/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/authc/TokenService.java index b7bf96119a2eb..52c1081367451 100644 --- a/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/authc/TokenService.java +++ b/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/authc/TokenService.java @@ -1017,8 +1017,7 @@ private void checkIfTokenIsValid(UserToken userToken, ActionListener Instant currentTime = clock.instant(); if (currentTime.isAfter(userToken.getExpirationTime())) { listener.onFailure(traceLog("validate token", userToken.getId(), expiredTokenException())); - } - if (securityIndex.indexExists() == false) { + } else if (securityIndex.indexExists() == false) { // index doesn't exist so the token is considered invalid as we cannot verify its validity logger.warn("failed to validate token [{}] since the security index doesn't exist", userToken.getId()); listener.onResponse(null); From 9049263c2c08f4e5a02188d36997049b7668649d Mon Sep 17 00:00:00 2001 From: Ioannis Kakavas Date: Wed, 9 Jan 2019 07:29:22 +0200 Subject: [PATCH 15/46] Handle malformed license signatures (#37137) This commit adds a more user friendly error message when a license signature is malformed/truncated in a way that it cannot be meaningfully parsed. --- .../org/elasticsearch/license/License.java | 14 ++- .../elasticsearch/license/LicenseTests.java | 118 ++++++++++++++++++ 2 files changed, 129 insertions(+), 3 deletions(-) create mode 100644 x-pack/plugin/core/src/test/java/org/elasticsearch/license/LicenseTests.java diff --git a/x-pack/plugin/core/src/main/java/org/elasticsearch/license/License.java b/x-pack/plugin/core/src/main/java/org/elasticsearch/license/License.java index b2130ac9f4b81..158c0eb7b2e63 100644 --- a/x-pack/plugin/core/src/main/java/org/elasticsearch/license/License.java +++ b/x-pack/plugin/core/src/main/java/org/elasticsearch/license/License.java @@ -469,9 +469,17 @@ public static License fromXContent(XContentParser parser) throws IOException { } // not a license spec if (builder.signature != null) { - byte[] signatureBytes = Base64.getDecoder().decode(builder.signature); - ByteBuffer byteBuffer = ByteBuffer.wrap(signatureBytes); - int version = byteBuffer.getInt(); + int version; + // In case the signature is truncated/malformed we might end up with fewer than 4 bytes in the byteBuffer + // or with a string that cannot be base64 decoded. In either case return a more friendly error instead of + // just throwing the BufferUnderflowException or the IllegalArgumentException + try { + byte[] signatureBytes = Base64.getDecoder().decode(builder.signature); + ByteBuffer byteBuffer = ByteBuffer.wrap(signatureBytes); + version = byteBuffer.getInt(); + } catch (Exception e) { + throw new ElasticsearchException("malformed signature for license [" + builder.uid + "]", e); + } // we take the absolute version, because negative versions // mean that the license was generated by the cluster (see TrialLicense) // and positive version means that the license was signed diff --git a/x-pack/plugin/core/src/test/java/org/elasticsearch/license/LicenseTests.java b/x-pack/plugin/core/src/test/java/org/elasticsearch/license/LicenseTests.java new file mode 100644 index 0000000000000..e6cc8f2bd89f2 --- /dev/null +++ b/x-pack/plugin/core/src/test/java/org/elasticsearch/license/LicenseTests.java @@ -0,0 +1,118 @@ +/* + * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one + * or more contributor license agreements. Licensed under the Elastic License; + * you may not use this file except in compliance with the Elastic License. + */ +package org.elasticsearch.license; + +import org.elasticsearch.ElasticsearchException; +import org.elasticsearch.common.bytes.BytesArray; +import org.elasticsearch.common.xcontent.XContentType; +import org.elasticsearch.test.ESTestCase; + +import java.nio.BufferUnderflowException; +import java.nio.charset.StandardCharsets; + +import static org.hamcrest.CoreMatchers.containsString; +import static org.hamcrest.CoreMatchers.equalTo; +import static org.hamcrest.CoreMatchers.instanceOf; + +public class LicenseTests extends ESTestCase { + + public void testFromXContent() throws Exception { + + String licenseString = "{\"license\":" + + "{\"uid\":\"4056779d-b823-4c12-a9cb-efa4a8d8c422\"," + + "\"type\":\"gold\"," + + "\"issue_date_in_millis\":1546589020459," + + "\"expiry_date_in_millis\":1546596340459," + + "\"max_nodes\":5," + + "\"issued_to\":\"customer\"," + + "\"issuer\":\"elasticsearch\"," + + "\"signature\":\"AAAAAgAAAA34V2kfTJVtvdL2LttwAAABmFJ6NGRnbEM3WVQrZVQwNkdKQmR1VytlMTMyM1J0dTZ1WGwyY2ZCVFhqMGtJU2gzZ3pnNTVpOW" + + "F5Y1NaUkwyN2VsTEtCYnlZR2c5WWtjQ0phaDlhRjlDUXViUmUwMWhjSkE2TFcwSGdneTJHbUV4N2RHUWJxV20ybjRsZHRzV2xkN0ZmdDlYblJmNVcxMlBWeU81" + + "V1hLUm1EK0V1dmF3cFdlSGZzTU5SZE1qUmFra3JkS1hCanBWVmVTaFFwV3BVZERzeG9Sci9rYnlJK2toODZXY09tNmFHUVNUL3IyUHExV3VSTlBneWNJcFQ0bX" + + "l0cmhNNnRwbE1CWE4zWjJ5eGFuWFo0NGhsb3B5WFd1eTdYbFFWQkxFVFFPSlBERlB0eVVJYXVSZ0lsR2JpRS9rN1h4MSsvNUpOcGN6cU1NOHN1cHNtSTFIUGN1" + + "bWNGNEcxekhrblhNOXZ2VEQvYmRzQUFwbytUZEpRR3l6QU5oS2ZFSFdSbGxxNDZyZ0xvUHIwRjdBL2JqcnJnNGFlK09Cek9pYlJ5Umc9PQAAAQAth77fQLF7CC" + + "EL7wA6Z0/UuRm/weECcsjW/50kBnPLO8yEs+9/bPa5LSU0bF6byEXOVeO0ebUQfztpjulbXh8TrBDSG+6VdxGtohPo2IYPBaXzGs3LOOor6An/lhptxBWdwYmf" + + "bcp0m8mnXZh1vN9rmbTsZXnhBIoPTaRDwUBi3vJ3Ms3iLaEm4S8Slrfmtht2jUjgGZ2vAeZ9OHU2YsGtrSpz6f\"}"; + License license = License.fromSource(new BytesArray(licenseString.getBytes(StandardCharsets.UTF_8)), + XContentType.JSON); + assertThat(license.type(), equalTo("gold")); + assertThat(license.uid(), equalTo("4056779d-b823-4c12-a9cb-efa4a8d8c422")); + assertThat(license.issuer(), equalTo("elasticsearch")); + assertThat(license.issuedTo(), equalTo("customer")); + assertThat(license.expiryDate(), equalTo(1546596340459L)); + assertThat(license.issueDate(), equalTo(1546589020459L)); + } + + public void testNotEnoughBytesFromXContent() throws Exception { + + String licenseString = "{\"license\": " + + "{\"uid\":\"4056779d-b823-4c12-a9cb-efa4a8d8c422\"," + + "\"type\":\"gold\"," + + "\"issue_date_in_millis\":1546589020459," + + "\"expiry_date_in_millis\":1546596340459," + + "\"max_nodes\":5," + + "\"issued_to\":\"customer\"," + + "\"issuer\":\"elasticsearch\"," + + "\"signature\":\"AA\"}" + + "}"; + ElasticsearchException exception = + expectThrows(ElasticsearchException.class, + () -> { + License.fromSource(new BytesArray(licenseString.getBytes(StandardCharsets.UTF_8)), + XContentType.JSON); + }); + assertThat(exception.getMessage(), containsString("malformed signature for license [4056779d-b823-4c12-a9cb-efa4a8d8c422]")); + assertThat(exception.getCause(), instanceOf(BufferUnderflowException.class)); + } + + public void testMalformedSignatureFromXContent() throws Exception { + + String licenseString = "{\"license\": " + + "{\"uid\":\"4056779d-b823-4c12-a9cb-efa4a8d8c422\"," + + "\"type\":\"gold\"," + + "\"issue_date_in_millis\":1546589020459," + + "\"expiry_date_in_millis\":1546596340459," + + "\"max_nodes\":5," + + "\"issued_to\":\"customer\"," + + "\"issuer\":\"elasticsearch\"," + + "\"signature\":\"" + randomAlphaOfLength(10) + "\"}" + + "}"; + ElasticsearchException exception = + expectThrows(ElasticsearchException.class, + () -> { + License.fromSource(new BytesArray(licenseString.getBytes(StandardCharsets.UTF_8)), + XContentType.JSON); + }); + } + + public void testUnableToBase64DecodeFromXContent() throws Exception { + + String licenseString = "{\"license\":" + + "{\"uid\":\"4056779d-b823-4c12-a9cb-efa4a8d8c422\"," + + "\"type\":\"gold\"," + + "\"issue_date_in_millis\":1546589020459," + + "\"expiry_date_in_millis\":1546596340459," + + "\"max_nodes\":5," + + "\"issued_to\":\"customer\"," + + "\"issuer\":\"elasticsearch\"," + + "\"signature\":\"AAAAAgAAAA34V2kfTJVtvdL2LttwAAABmFJ6NGRnbEM3WVQrZVQwNkdKQmR1VytlMTMyM1J0dTZ1WGwyY2ZCVFhqMGtJU2gzZ3pnNTVpOW" + + "F5Y1NaUkwyN2VsTEtCYnlZR2c5WWtjQ0phaDlhRjlDUXViUmUwMWhjSkE2TFcwSGdneTJHbUV4N2RHUWJxV20ybjRsZHRzV2xkN0ZmdDlYblJmNVcxMlBWeU81" + + "V1hLUm1EK0V1dmF3cFdlSGZzTU5SZE1qUmFra3JkS1hCanBWVmVTaFFwV3BVZERzeG9Sci9rYnlJK2toODZXY09tNmFHUVNUL3IyUHExV3VSTlBneWNJcFQ0bX" + + "l0cmhNNnRwbE1CWE4zWjJ5eGFuWFo0NGhsb3B5WFd1eTdYbFFWQkxFVFFPSlBERlB0eVVJYXVSZ0lsR2JpRS9rN1h4MSsvNUpOcGN6cU1NOHN1cHNtSTFIUGN1" + + "bWNGNEcxekhrblhNOXZ2VEQvYmRzQUFwbytUZEpRR3l6QU5oS2ZFSFdSbGxxNDZyZ0xvUHIwRjdBL2JqcnJnNGFlK09Cek9pYlJ5Umc9PQAAAQAth77fQLF7CC" + + "EL7wA6Z0/UuRm/weECcsjW/50kBnPLO8yEs+9/bPa5LSU0bF6byEXOVeO0ebUQfztpjulbXh8TrBDSG+6VdxGtohPo2IYPBaXzGs3LOOor6An/lhptxBWdwYmf" + + "+xHAQ8tyvRqP5G+PRU7tiluEwR/eyHGZV2exdJNzmoGzdPSWwueBM5HK2GexORICH+UFI4cuGz444/hL2MMM1RdpVWQkT0SJ6D9x/VuSmHuYPdtX59Pp41LXvl" + + "bcp0m8mnXZh1vN9rmbTsZXnhBIoPTaRDwUBi3vJ3Ms3iLaEm4S8Slrfmtht2jUjgGZ2vAeZ9OHU2YsGtrSpz6fd\"}"; + ElasticsearchException exception = + expectThrows(ElasticsearchException.class, + () -> { + License.fromSource(new BytesArray(licenseString.getBytes(StandardCharsets.UTF_8)), + XContentType.JSON); + }); + assertThat(exception.getMessage(), containsString("malformed signature for license [4056779d-b823-4c12-a9cb-efa4a8d8c422]")); + assertThat(exception.getCause(), instanceOf(IllegalArgumentException.class)); + } +} From 751dc1db085375edc1983f3e41270a715254e8d2 Mon Sep 17 00:00:00 2001 From: Yuri Astrakhan Date: Wed, 9 Jan 2019 00:56:47 -0500 Subject: [PATCH 16/46] Implement bucket per hash type --- .../bucket/geogrid2/GeoGridAggregator.java | 21 +-- .../geogrid2/GeoGridAggregatorFactory.java | 2 +- .../bucket/geogrid2/GeoGridBucket.java | 137 +++++++++++++++++ .../bucket/geogrid2/GeoGridType.java | 15 ++ .../bucket/geogrid2/GeoHashBucket.java | 50 ++++++ .../bucket/geogrid2/GeoHashType.java | 14 ++ .../bucket/geogrid2/InternalGeoGrid.java | 145 +++--------------- .../geogrid2/InternalGeoGridGeoHashTests.java | 23 ++- 8 files changed, 254 insertions(+), 153 deletions(-) create mode 100644 server/src/main/java/org/elasticsearch/search/aggregations/bucket/geogrid2/GeoGridBucket.java create mode 100644 server/src/main/java/org/elasticsearch/search/aggregations/bucket/geogrid2/GeoHashBucket.java diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/geogrid2/GeoGridAggregator.java b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/geogrid2/GeoGridAggregator.java index cc02119ac2e55..9d410944aabf6 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/geogrid2/GeoGridAggregator.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/geogrid2/GeoGridAggregator.java @@ -98,17 +98,6 @@ public void collect(int doc, long bucket) throws IOException { }; } - // private impl that stores a bucket ord. This allows for computing the aggregations lazily. - static class OrdinalBucket extends InternalGeoGrid.Bucket { - - long bucketOrd; - - OrdinalBucket() { - super(0, 0, null); - } - - } - @Override public InternalGeoGrid buildAggregation(long owningBucketOrdinal) throws IOException { assert owningBucketOrdinal == 0; @@ -116,21 +105,21 @@ public InternalGeoGrid buildAggregation(long owningBucketOrdinal) throws IOExcep consumeBucketsAndMaybeBreak(size); InternalGeoGrid.BucketPriorityQueue ordered = new InternalGeoGrid.BucketPriorityQueue(size); - OrdinalBucket spare = null; + GeoGridBucket spare = null; for (long i = 0; i < bucketOrds.size(); i++) { if (spare == null) { - spare = new OrdinalBucket(); + spare = type.createBucket(0, 0, null); } spare.hashAsLong = bucketOrds.get(i); spare.docCount = bucketDocCount(i); spare.bucketOrd = i; - spare = (OrdinalBucket) ordered.insertWithOverflow(spare); + spare = ordered.insertWithOverflow(spare); } - final InternalGeoGrid.Bucket[] list = new InternalGeoGrid.Bucket[ordered.size()]; + final GeoGridBucket[] list = new GeoGridBucket[ordered.size()]; for (int i = ordered.size() - 1; i >= 0; --i) { - final OrdinalBucket bucket = (OrdinalBucket) ordered.pop(); + final GeoGridBucket bucket = ordered.pop(); bucket.aggregations = bucketAggregations(bucket.bucketOrd); list[i] = bucket; } diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/geogrid2/GeoGridAggregatorFactory.java b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/geogrid2/GeoGridAggregatorFactory.java index ca26cf91274cf..bea23d17df6d8 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/geogrid2/GeoGridAggregatorFactory.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/geogrid2/GeoGridAggregatorFactory.java @@ -59,7 +59,7 @@ public class GeoGridAggregatorFactory extends ValuesSourceAggregatorFactory pipelineAggregators, Map metaData) throws IOException { final InternalAggregation aggregation = new InternalGeoGrid(name, type, requiredSize, - Collections. emptyList(), pipelineAggregators, metaData); + Collections. emptyList(), pipelineAggregators, metaData); return new NonCollectingAggregator(name, context, parent, pipelineAggregators, metaData) { @Override public InternalAggregation buildEmptyAggregation() { diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/geogrid2/GeoGridBucket.java b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/geogrid2/GeoGridBucket.java new file mode 100644 index 0000000000000..75ffcac5d342e --- /dev/null +++ b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/geogrid2/GeoGridBucket.java @@ -0,0 +1,137 @@ +/* + * 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.aggregations.bucket.geogrid2; + +import org.elasticsearch.common.io.stream.StreamInput; +import org.elasticsearch.common.io.stream.StreamOutput; +import org.elasticsearch.common.xcontent.XContentBuilder; +import org.elasticsearch.search.aggregations.Aggregation.CommonFields; +import org.elasticsearch.search.aggregations.Aggregations; +import org.elasticsearch.search.aggregations.InternalAggregation; +import org.elasticsearch.search.aggregations.InternalAggregations; +import org.elasticsearch.search.aggregations.InternalMultiBucketAggregation; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; +import java.util.Objects; + + +abstract class GeoGridBucket extends InternalMultiBucketAggregation.InternalBucket implements GeoGrid.Bucket, Comparable { + + // This is only used by the private impl that stores a bucket ord. This allows for computing the aggregations lazily. + // This field is used temporarily, and will not be used in serialization or comparison. + long bucketOrd; + + protected long hashAsLong; + protected long docCount; + protected InternalAggregations aggregations; + + /** + * Factory method to instantiate a new bucket from inside the bucket. + * Derived buckets should do return new Bucket(hashAsLong, docCount, aggregations); + */ + protected abstract GeoGridBucket newBucket(long hashAsLong, long docCount, InternalAggregations aggregations); + + protected GeoGridBucket(long hashAsLong, long docCount, InternalAggregations aggregations) { + this.docCount = docCount; + this.aggregations = aggregations; + this.hashAsLong = hashAsLong; + } + + /** + * Read from a stream. + */ + protected GeoGridBucket(StreamInput in) throws IOException { + hashAsLong = in.readLong(); + docCount = in.readVLong(); + aggregations = InternalAggregations.readAggregations(in); + } + + @Override + public void writeTo(StreamOutput out) throws IOException { + out.writeLong(hashAsLong); + out.writeVLong(docCount); + aggregations.writeTo(out); + } + + @Override + public String getKey() { + return getKeyAsString(); + } + + @Override + public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException { + builder.startObject(); + builder.field(CommonFields.KEY.getPreferredName(), getKeyAsString()); + builder.field(CommonFields.DOC_COUNT.getPreferredName(), docCount); + aggregations.toXContentInternal(builder, params); + builder.endObject(); + return builder; + } + + @Override + public long getDocCount() { + return docCount; + } + + @Override + public Aggregations getAggregations() { + return aggregations; + } + + @Override + public int compareTo(GeoGridBucket other) { + if (this.hashAsLong > other.hashAsLong) { + return 1; + } + if (this.hashAsLong < other.hashAsLong) { + return -1; + } + return 0; + } + + public GeoGridBucket reduce(List buckets, InternalAggregation.ReduceContext context) { + List aggregationsList = new ArrayList<>(buckets.size()); + long docCount = 0; + for (GeoGridBucket bucket : buckets) { + docCount += bucket.docCount; + aggregationsList.add(bucket.aggregations); + } + final InternalAggregations aggs = InternalAggregations.reduce(aggregationsList, context); + return newBucket(hashAsLong, docCount, aggs); + } + + @Override + public boolean equals(Object o) { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + GeoGridBucket bucket = (GeoGridBucket) o; + return hashAsLong == bucket.hashAsLong && + docCount == bucket.docCount && + Objects.equals(aggregations, bucket.aggregations); + } + + @Override + public int hashCode() { + return Objects.hash(hashAsLong, docCount, aggregations); + } + +} diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/geogrid2/GeoGridType.java b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/geogrid2/GeoGridType.java index a5eab70cab134..cdbe52f640e83 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/geogrid2/GeoGridType.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/geogrid2/GeoGridType.java @@ -19,6 +19,11 @@ package org.elasticsearch.search.aggregations.bucket.geogrid2; +import org.elasticsearch.common.io.stream.StreamInput; +import org.elasticsearch.search.aggregations.InternalAggregations; + +import java.io.IOException; + /** * Instances implement different hashing algorithms for geo-grid aggregations */ @@ -57,4 +62,14 @@ public interface GeoGridType { * @return bucket ID as a string */ String hashAsString(long hash); + + /** + * Factory method to create a new bucket. + */ + GeoGridBucket createBucket(long hashAsLong, long docCount, InternalAggregations aggregations); + + /** + * Factory method to create a new bucket from a stream. + */ + GeoGridBucket createBucket(StreamInput reader) throws IOException; } diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/geogrid2/GeoHashBucket.java b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/geogrid2/GeoHashBucket.java new file mode 100644 index 0000000000000..79262129f5d34 --- /dev/null +++ b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/geogrid2/GeoHashBucket.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.search.aggregations.bucket.geogrid2; + +import org.elasticsearch.common.geo.GeoHashUtils; +import org.elasticsearch.common.io.stream.StreamInput; +import org.elasticsearch.search.aggregations.InternalAggregations; + +import java.io.IOException; + +/** + * A bucket to store GeoHash geogrid aggregation + */ +public class GeoHashBucket extends GeoGridBucket { + + protected GeoHashBucket(long hashAsLong, long docCount, InternalAggregations aggregations) { + super(hashAsLong, docCount, aggregations); + } + + protected GeoHashBucket(StreamInput in) throws IOException { + super(in); + } + + @Override + protected GeoGridBucket newBucket(long hashAsLong, long docCount, InternalAggregations aggregations) { + return new GeoHashBucket(hashAsLong, docCount, aggregations); + } + + @Override + public String getKeyAsString() { + return GeoHashUtils.stringEncode(this.hashAsLong); + } +} diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/geogrid2/GeoHashType.java b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/geogrid2/GeoHashType.java index f9a98499896bc..03de33233a26c 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/geogrid2/GeoHashType.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/geogrid2/GeoHashType.java @@ -21,6 +21,10 @@ import org.elasticsearch.common.geo.GeoHashUtils; import org.elasticsearch.common.geo.GeoUtils; +import org.elasticsearch.common.io.stream.StreamInput; +import org.elasticsearch.search.aggregations.InternalAggregations; + +import java.io.IOException; /** * A simple wrapper for GeoUtils handling of the geohash hashing algorithm @@ -57,4 +61,14 @@ public long calculateHash(double longitude, double latitude, int precision) { public String hashAsString(long hash) { return GeoHashUtils.stringEncode(hash); } + + @Override + public GeoGridBucket createBucket(long hashAsLong, long docCount, InternalAggregations aggregations) { + return new GeoHashBucket(hashAsLong, docCount, aggregations); + } + + @Override + public GeoGridBucket createBucket(StreamInput reader) throws IOException { + return new GeoHashBucket(reader); + } } diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/geogrid2/InternalGeoGrid.java b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/geogrid2/InternalGeoGrid.java index 6b81de510871e..7c2dd9154bf9e 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/geogrid2/InternalGeoGrid.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/geogrid2/InternalGeoGrid.java @@ -16,6 +16,7 @@ * specific language governing permissions and limitations * under the License. */ + package org.elasticsearch.search.aggregations.bucket.geogrid2; import org.apache.lucene.util.PriorityQueue; @@ -23,7 +24,6 @@ import org.elasticsearch.common.io.stream.StreamOutput; import org.elasticsearch.common.util.LongObjectPagedHashMap; import org.elasticsearch.common.xcontent.XContentBuilder; -import org.elasticsearch.search.aggregations.Aggregations; import org.elasticsearch.search.aggregations.InternalAggregation; import org.elasticsearch.search.aggregations.InternalAggregations; import org.elasticsearch.search.aggregations.InternalMultiBucketAggregation; @@ -43,117 +43,14 @@ * All hashes in a grid are of the same precision and held internally as a single long * for efficiency's sake. */ -public class InternalGeoGrid extends InternalMultiBucketAggregation implements - GeoGrid { - static class Bucket extends InternalMultiBucketAggregation.InternalBucket implements GeoGrid.Bucket, Comparable { - - protected long hashAsLong; - protected long docCount; - protected InternalAggregations aggregations; - - Bucket(long hashAsLong, long docCount, InternalAggregations aggregations) { - this.docCount = docCount; - this.aggregations = aggregations; - this.hashAsLong = hashAsLong; - } - - /** - * Read from a stream. - */ - private Bucket(StreamInput in) throws IOException { - hashAsLong = in.readLong(); - docCount = in.readVLong(); - aggregations = InternalAggregations.readAggregations(in); - } - - @Override - public void writeTo(StreamOutput out) throws IOException { - out.writeLong(hashAsLong); - out.writeVLong(docCount); - aggregations.writeTo(out); - } - - @Override - public String getKeyAsString() { - throw new IllegalArgumentException(); // FIXME: better ex? Also, any way to structure this better? - } - - @Override - public String getKey() { - throw new IllegalArgumentException(); // FIXME: better ex? Also, any way to structure this better? - } - - @Override - public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException { - throw new IllegalArgumentException(); // FIXME: better ex? Also, any way to structure this better? - } - - /** - * Serializing to XContent using a specific GeoGridType. - */ - void toXContent2(XContentBuilder builder, Params params, GeoGridType type) throws IOException { - builder.startObject(); - builder.field(CommonFields.KEY.getPreferredName(), type.hashAsString(hashAsLong)); - builder.field(CommonFields.DOC_COUNT.getPreferredName(), docCount); - aggregations.toXContentInternal(builder, params); - builder.endObject(); - } - - @Override - public long getDocCount() { - return docCount; - } - - @Override - public Aggregations getAggregations() { - return aggregations; - } - - @Override - public int compareTo(Bucket other) { - if (this.hashAsLong > other.hashAsLong) { - return 1; - } - if (this.hashAsLong < other.hashAsLong) { - return -1; - } - return 0; - } - - public Bucket reduce(List buckets, ReduceContext context) { - List aggregationsList = new ArrayList<>(buckets.size()); - long docCount = 0; - for (Bucket bucket : buckets) { - docCount += bucket.docCount; - aggregationsList.add(bucket.aggregations); - } - final InternalAggregations aggs = InternalAggregations.reduce(aggregationsList, context); - return new Bucket(hashAsLong, docCount, aggs); - } - - @Override - public boolean equals(Object o) { - if (this == o) return true; - if (o == null || getClass() != o.getClass()) return false; - Bucket bucket = (Bucket) o; - return hashAsLong == bucket.hashAsLong && - docCount == bucket.docCount && - Objects.equals(aggregations, bucket.aggregations); - } - - @Override - public int hashCode() { - return Objects.hash(hashAsLong, docCount, aggregations); - } - - } +public class InternalGeoGrid extends InternalMultiBucketAggregation implements GeoGrid { private final GeoGridType type; private final int requiredSize; - private final List buckets; + private final List buckets; - InternalGeoGrid(String name, GeoGridType type, int requiredSize, List buckets, List pipelineAggregators, - Map metaData) { + InternalGeoGrid(String name, GeoGridType type, int requiredSize, List buckets, + List pipelineAggregators, Map metaData) { super(name, pipelineAggregators, metaData); this.type = type; this.requiredSize = requiredSize; @@ -167,7 +64,7 @@ public InternalGeoGrid(StreamInput in) throws IOException { super(in); type = GeoGridTypes.DEFAULT.get(in.readString(), "internal-worker"); requiredSize = readSize(in); - buckets = in.readList(Bucket::new); + buckets = in.readList(type::createBucket); } @Override @@ -183,30 +80,30 @@ public String getWriteableName() { } @Override - public InternalGeoGrid create(List buckets) { + public InternalGeoGrid create(List buckets) { return new InternalGeoGrid(this.name, this.type, this.requiredSize, buckets, this.pipelineAggregators(), this.metaData); } @Override - public Bucket createBucket(InternalAggregations aggregations, Bucket prototype) { - return new Bucket(prototype.hashAsLong, prototype.docCount, aggregations); + public GeoGridBucket createBucket(InternalAggregations aggregations, GeoGridBucket prototype) { + return type.createBucket(prototype.hashAsLong, prototype.docCount, aggregations); } @Override - public List getBuckets() { + public List getBuckets() { return unmodifiableList(buckets); } @Override public InternalGeoGrid doReduce(List aggregations, ReduceContext reduceContext) { - LongObjectPagedHashMap> buckets = null; + LongObjectPagedHashMap> buckets = null; for (InternalAggregation aggregation : aggregations) { InternalGeoGrid grid = (InternalGeoGrid) aggregation; if (buckets == null) { buckets = new LongObjectPagedHashMap<>(grid.buckets.size(), reduceContext.bigArrays()); } - for (Bucket bucket : grid.buckets) { - List existingBuckets = buckets.get(bucket.hashAsLong); + for (GeoGridBucket bucket : grid.buckets) { + List existingBuckets = buckets.get(bucket.hashAsLong); if (existingBuckets == null) { existingBuckets = new ArrayList<>(aggregations.size()); buckets.put(bucket.hashAsLong, existingBuckets); @@ -217,9 +114,9 @@ public InternalGeoGrid doReduce(List aggregations, ReduceCo final int size = Math.toIntExact(reduceContext.isFinalReduce() == false ? buckets.size() : Math.min(requiredSize, buckets.size())); BucketPriorityQueue ordered = new BucketPriorityQueue(size); - for (LongObjectPagedHashMap.Cursor> cursor : buckets) { - List sameCellBuckets = cursor.value; - Bucket removed = ordered.insertWithOverflow(sameCellBuckets.get(0).reduce(sameCellBuckets, reduceContext)); + for (LongObjectPagedHashMap.Cursor> cursor : buckets) { + List sameCellBuckets = cursor.value; + GeoGridBucket removed = ordered.insertWithOverflow(sameCellBuckets.get(0).reduce(sameCellBuckets, reduceContext)); if (removed != null) { reduceContext.consumeBucketsAndMaybeBreak(-countInnerBucket(removed)); } else { @@ -227,7 +124,7 @@ public InternalGeoGrid doReduce(List aggregations, ReduceCo } } buckets.close(); - Bucket[] list = new Bucket[ordered.size()]; + GeoGridBucket[] list = new GeoGridBucket[ordered.size()]; for (int i = ordered.size() - 1; i >= 0; i--) { list[i] = ordered.pop(); } @@ -237,8 +134,8 @@ public InternalGeoGrid doReduce(List aggregations, ReduceCo @Override public XContentBuilder doXContentBody(XContentBuilder builder, Params params) throws IOException { builder.startArray(CommonFields.BUCKETS.getPreferredName()); - for (Bucket bucket : buckets) { - bucket.toXContent2(builder, params, type); + for (GeoGridBucket bucket : buckets) { + bucket.toXContent(builder, params); } builder.endArray(); return builder; @@ -261,14 +158,14 @@ protected boolean doEquals(Object obj) { Objects.equals(buckets, other.buckets); } - static class BucketPriorityQueue extends PriorityQueue { + static class BucketPriorityQueue extends PriorityQueue { BucketPriorityQueue(int size) { super(size); } @Override - protected boolean lessThan(Bucket o1, Bucket o2) { + protected boolean lessThan(GeoGridBucket o1, GeoGridBucket o2) { int cmp = Long.compare(o2.getDocCount(), o1.getDocCount()); if (cmp == 0) { cmp = o2.compareTo(o1); diff --git a/server/src/test/java/org/elasticsearch/search/aggregations/bucket/geogrid2/InternalGeoGridGeoHashTests.java b/server/src/test/java/org/elasticsearch/search/aggregations/bucket/geogrid2/InternalGeoGridGeoHashTests.java index 8acc6d1780776..dd15486c00341 100644 --- a/server/src/test/java/org/elasticsearch/search/aggregations/bucket/geogrid2/InternalGeoGridGeoHashTests.java +++ b/server/src/test/java/org/elasticsearch/search/aggregations/bucket/geogrid2/InternalGeoGridGeoHashTests.java @@ -25,7 +25,6 @@ import org.elasticsearch.search.aggregations.bucket.GeoGridTests; import org.elasticsearch.test.InternalMultiBucketAggregationTestCase; import org.elasticsearch.search.aggregations.ParsedMultiBucketAggregation; -import org.elasticsearch.search.aggregations.bucket.geogrid2.InternalGeoGrid.Bucket; import org.elasticsearch.search.aggregations.pipeline.PipelineAggregator; import java.util.ArrayList; @@ -51,13 +50,13 @@ protected InternalGeoGrid createTestInstance(String name, Map metaData, InternalAggregations aggregations) { int size = randomNumberOfBuckets(); - List buckets = new ArrayList<>(size); + List buckets = new ArrayList<>(size); for (int i = 0; i < size; i++) { double latitude = randomDoubleBetween(-90.0, 90.0, false); double longitude = randomDoubleBetween(-180.0, 180.0, false); long geoHashAsLong = GeoHashUtils.longEncode(longitude, latitude, 4); - buckets.add(new InternalGeoGrid.Bucket(geoHashAsLong, randomInt(IndexWriter.MAX_DOCS), aggregations)); + buckets.add(new GeoHashBucket(geoHashAsLong, randomInt(IndexWriter.MAX_DOCS), aggregations)); } return new InternalGeoGrid(name, GeoGridTests.GEOHASH_TYPE, size, buckets, pipelineAggregators, metaData); } @@ -69,21 +68,21 @@ protected Writeable.Reader instanceReader() { @Override protected void assertReduced(InternalGeoGrid reduced, List inputs) { - Map> map = new HashMap<>(); + Map> map = new HashMap<>(); for (InternalGeoGrid input : inputs) { for (GeoGrid.Bucket bucket : input.getBuckets()) { - InternalGeoGrid.Bucket internalBucket = (InternalGeoGrid.Bucket) bucket; - List buckets = map.computeIfAbsent(internalBucket.hashAsLong, k -> new ArrayList<>()); + GeoHashBucket internalBucket = (GeoHashBucket) bucket; + List buckets = map.computeIfAbsent(internalBucket.hashAsLong, k -> new ArrayList<>()); buckets.add(internalBucket); } } - List expectedBuckets = new ArrayList<>(); - for (Map.Entry> entry : map.entrySet()) { + List expectedBuckets = new ArrayList<>(); + for (Map.Entry> entry : map.entrySet()) { long docCount = 0; - for (InternalGeoGrid.Bucket bucket : entry.getValue()) { + for (GeoHashBucket bucket : entry.getValue()) { docCount += bucket.docCount; } - expectedBuckets.add(new InternalGeoGrid.Bucket(entry.getKey(), docCount, InternalAggregations.EMPTY)); + expectedBuckets.add(new GeoHashBucket(entry.getKey(), docCount, InternalAggregations.EMPTY)); } expectedBuckets.sort((first, second) -> { int cmp = Long.compare(second.docCount, first.docCount); @@ -112,7 +111,7 @@ protected Class implementationClass() { protected InternalGeoGrid mutateInstance(InternalGeoGrid instance) { String name = instance.getName(); int size = instance.getRequiredSize(); - List buckets = instance.getBuckets(); + List buckets = instance.getBuckets(); List pipelineAggregators = instance.pipelineAggregators(); Map metaData = instance.getMetaData(); switch (between(0, 3)) { @@ -122,7 +121,7 @@ protected InternalGeoGrid mutateInstance(InternalGeoGrid instance) { case 1: buckets = new ArrayList<>(buckets); buckets.add( - new InternalGeoGrid.Bucket(randomNonNegativeLong(), randomInt(IndexWriter.MAX_DOCS), InternalAggregations.EMPTY)); + new GeoHashBucket(randomNonNegativeLong(), randomInt(IndexWriter.MAX_DOCS), InternalAggregations.EMPTY)); break; case 2: size = size + between(1, 10); From 2d6843398fda84b9cb91120538c7ef09e72bcf59 Mon Sep 17 00:00:00 2001 From: Yuri Astrakhan Date: Wed, 9 Jan 2019 01:50:12 -0500 Subject: [PATCH 17/46] fixed most of the tests --- .../geogrid2/GeoGridAggregationBuilder2.java | 4 +- .../bucket/geogrid2/GeoGridTypes.java | 3 +- .../bucket/geogrid2/GeoHashType.java | 10 +++- .../search/aggregations/bucket/GeoGridIT.java | 17 ++++--- .../aggregations/bucket/GeoGridTests.java | 4 +- .../aggregations/bucket/ShardReduceIT.java | 3 +- .../bucket/geogrid2/GeoGridParserTests.java | 49 +++++++++---------- .../geogrid2/InternalGeoGridGeoHashTests.java | 5 +- .../aggregations/metrics/GeoCentroidIT.java | 4 +- 9 files changed, 50 insertions(+), 49 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/geogrid2/GeoGridAggregationBuilder2.java b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/geogrid2/GeoGridAggregationBuilder2.java index 56e36f7e6ced6..7097b6c577d28 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/geogrid2/GeoGridAggregationBuilder2.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/geogrid2/GeoGridAggregationBuilder2.java @@ -61,7 +61,7 @@ public class GeoGridAggregationBuilder2 extends ValuesSourceAggregationBuilder -1) { diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/geogrid2/GeoGridTypes.java b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/geogrid2/GeoGridTypes.java index b43feda43f980..34b381741fad0 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/geogrid2/GeoGridTypes.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/geogrid2/GeoGridTypes.java @@ -34,8 +34,7 @@ private GeoGridTypes() { // TODO: we need to decide how types map is instantiated/stored // TODO: especially this is important to allow type plugins types = new HashMap<>(); - final GeoGridType type = new GeoHashType(); - types.put(type.getName(), type); + types.put(GeoHashType.SINGLETON.getName(), GeoHashType.SINGLETON); } public GeoGridType get(String typeStr, String name) { diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/geogrid2/GeoHashType.java b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/geogrid2/GeoHashType.java index 03de33233a26c..bc5747f0f7f7f 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/geogrid2/GeoHashType.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/geogrid2/GeoHashType.java @@ -30,7 +30,15 @@ * A simple wrapper for GeoUtils handling of the geohash hashing algorithm */ public class GeoHashType implements GeoGridType { - static final String NAME = "geohash"; + + /** + * GeoGridType must be a singleton because bucket does a reference compare for equality + */ + private GeoHashType() {} + + public static final String NAME = "geohash"; + + public static final GeoGridType SINGLETON = new GeoHashType(); @Override public String getName() { diff --git a/server/src/test/java/org/elasticsearch/search/aggregations/bucket/GeoGridIT.java b/server/src/test/java/org/elasticsearch/search/aggregations/bucket/GeoGridIT.java index e7072f402f092..ea818d929ff8c 100644 --- a/server/src/test/java/org/elasticsearch/search/aggregations/bucket/GeoGridIT.java +++ b/server/src/test/java/org/elasticsearch/search/aggregations/bucket/GeoGridIT.java @@ -34,6 +34,7 @@ import org.elasticsearch.search.aggregations.bucket.filter.Filter; import org.elasticsearch.search.aggregations.bucket.geogrid2.GeoGrid; import org.elasticsearch.search.aggregations.bucket.geogrid2.GeoGrid.Bucket; +import org.elasticsearch.search.aggregations.bucket.geogrid2.GeoHashType; import org.elasticsearch.test.ESIntegTestCase; import org.elasticsearch.test.VersionUtils; @@ -146,7 +147,7 @@ public void setupSuiteScopeCluster() throws Exception { public void testSimple() throws Exception { for (int precision = 1; precision <= PRECISION; precision++) { SearchResponse response = client().prepareSearch("idx") - .addAggregation(geoGrid("geogrid", GeoGridTests.GEOHASH_TYPE) + .addAggregation(geoGrid("geogrid", GeoHashType.SINGLETON) .field("location") .precision(precision) ) @@ -177,7 +178,7 @@ public void testSimple() throws Exception { public void testMultivalued() throws Exception { for (int precision = 1; precision <= PRECISION; precision++) { SearchResponse response = client().prepareSearch("multi_valued_idx") - .addAggregation(geoGrid("geogrid", GeoGridTests.GEOHASH_TYPE) + .addAggregation(geoGrid("geogrid", GeoHashType.SINGLETON) .field("location") .precision(precision) ) @@ -206,7 +207,7 @@ public void testFiltered() throws Exception { .addAggregation( AggregationBuilders.filter("filtered", bbox) .subAggregation( - geoGrid("geogrid", GeoGridTests.GEOHASH_TYPE) + geoGrid("geogrid", GeoHashType.SINGLETON) .field("location") .precision(precision) ) @@ -234,7 +235,7 @@ public void testFiltered() throws Exception { public void testUnmapped() throws Exception { for (int precision = 1; precision <= PRECISION; precision++) { SearchResponse response = client().prepareSearch("idx_unmapped") - .addAggregation(geoGrid("geogrid", GeoGridTests.GEOHASH_TYPE) + .addAggregation(geoGrid("geogrid", GeoHashType.SINGLETON) .field("location") .precision(precision) ) @@ -251,7 +252,7 @@ public void testUnmapped() throws Exception { public void testPartiallyUnmapped() throws Exception { for (int precision = 1; precision <= PRECISION; precision++) { SearchResponse response = client().prepareSearch("idx", "idx_unmapped") - .addAggregation(geoGrid("geogrid", GeoGridTests.GEOHASH_TYPE) + .addAggregation(geoGrid("geogrid", GeoHashType.SINGLETON) .field("location") .precision(precision) ) @@ -275,7 +276,7 @@ public void testPartiallyUnmapped() throws Exception { public void testTopMatch() throws Exception { for (int precision = 1; precision <= PRECISION; precision++) { SearchResponse response = client().prepareSearch("idx") - .addAggregation(geoGrid("geogrid", GeoGridTests.GEOHASH_TYPE) + .addAggregation(geoGrid("geogrid", GeoHashType.SINGLETON) .field("location") .size(1) .shardSize(100) @@ -311,7 +312,7 @@ public void testSizeIsZero() { () -> client() .prepareSearch("idx") .addAggregation( - geoGrid("geogrid", GeoGridTests.GEOHASH_TYPE) + geoGrid("geogrid", GeoHashType.SINGLETON) .field("location").size(size).shardSize(shardSize)).get()); assertThat(exception.getMessage(), containsString("[size] must be greater than 0. Found [0] in [geogrid]")); } @@ -321,7 +322,7 @@ public void testShardSizeIsZero() { final int shardSize = 0; IllegalArgumentException exception = expectThrows(IllegalArgumentException.class, () -> client().prepareSearch("idx") - .addAggregation(geoGrid("geogrid", GeoGridTests.GEOHASH_TYPE).field("location").size(size).shardSize(shardSize)) + .addAggregation(geoGrid("geogrid", GeoHashType.SINGLETON).field("location").size(size).shardSize(shardSize)) .get()); assertThat(exception.getMessage(), containsString("[shardSize] must be greater than 0. Found [0] in [geogrid]")); } diff --git a/server/src/test/java/org/elasticsearch/search/aggregations/bucket/GeoGridTests.java b/server/src/test/java/org/elasticsearch/search/aggregations/bucket/GeoGridTests.java index 3e84acefaeb08..065510d3aa8f6 100644 --- a/server/src/test/java/org/elasticsearch/search/aggregations/bucket/GeoGridTests.java +++ b/server/src/test/java/org/elasticsearch/search/aggregations/bucket/GeoGridTests.java @@ -26,14 +26,12 @@ public class GeoGridTests extends BaseAggregationTestCase { - public static final GeoGridType GEOHASH_TYPE = new GeoHashType(); - /** * Pick a random hash type */ public static GeoGridType randomType() { // With more types, will use randomIntBetween() to pick one - return GEOHASH_TYPE; + return GeoHashType.SINGLETON; } /** diff --git a/server/src/test/java/org/elasticsearch/search/aggregations/bucket/ShardReduceIT.java b/server/src/test/java/org/elasticsearch/search/aggregations/bucket/ShardReduceIT.java index 8fbb6f6edd80b..73aadb17c3905 100644 --- a/server/src/test/java/org/elasticsearch/search/aggregations/bucket/ShardReduceIT.java +++ b/server/src/test/java/org/elasticsearch/search/aggregations/bucket/ShardReduceIT.java @@ -26,6 +26,7 @@ import org.elasticsearch.search.aggregations.bucket.filter.Filter; import org.elasticsearch.search.aggregations.bucket.geogrid.GeoHashGrid; import org.elasticsearch.search.aggregations.bucket.geogrid2.GeoGrid; +import org.elasticsearch.search.aggregations.bucket.geogrid2.GeoHashType; import org.elasticsearch.search.aggregations.bucket.global.Global; import org.elasticsearch.search.aggregations.bucket.histogram.DateHistogramInterval; import org.elasticsearch.search.aggregations.bucket.histogram.Histogram; @@ -312,7 +313,7 @@ public void testGeoHashGrid() throws Exception { public void testGeoGrid() throws Exception { SearchResponse response = client().prepareSearch("idx") .setQuery(QueryBuilders.matchAllQuery()) - .addAggregation(geoGrid("grid", GeoGridTests.GEOHASH_TYPE).field("location") + .addAggregation(geoGrid("grid", GeoHashType.SINGLETON).field("location") .subAggregation(dateHistogram("histo").field("date").dateHistogramInterval(DateHistogramInterval.DAY) .minDocCount(0))) .get(); diff --git a/server/src/test/java/org/elasticsearch/search/aggregations/bucket/geogrid2/GeoGridParserTests.java b/server/src/test/java/org/elasticsearch/search/aggregations/bucket/geogrid2/GeoGridParserTests.java index f3793028265fb..df09b207c0cbf 100644 --- a/server/src/test/java/org/elasticsearch/search/aggregations/bucket/geogrid2/GeoGridParserTests.java +++ b/server/src/test/java/org/elasticsearch/search/aggregations/bucket/geogrid2/GeoGridParserTests.java @@ -35,24 +35,24 @@ public void testParseValidFromInts() throws Exception { GeoGridType type = GeoGridTests.randomType(); int precision = GeoGridTests.randomPrecision(type); XContentParser stParser = createParser(JsonXContent.jsonXContent, - "{\"field\":\"my_loc\", \"hash_type\":\"" + type + "\", \"precision\":" + precision + + "{\"hash_type\":\"" + type.getName() + "\", \"field\":\"my_loc\", \"precision\":" + precision + ", \"size\": 500, \"shard_size\": 550}"); XContentParser.Token token = stParser.nextToken(); assertSame(XContentParser.Token.START_OBJECT, token); // can create a factory - assertNotNull(GeoGridAggregationBuilder2.parse("geohash_grid", stParser)); + assertNotNull(GeoGridAggregationBuilder2.parse("geo_grid", stParser)); } public void testParseValidFromStrings() throws Exception { GeoGridType type = GeoGridTests.randomType(); int precision = GeoGridTests.randomPrecision(type); XContentParser stParser = createParser(JsonXContent.jsonXContent, - "{\"field\":\"my_loc\", \"hash_type\":\"" + type + "\", \"precision\":\"" + precision + + "{\"hash_type\":\"" + type.getName() + "\", \"field\":\"my_loc\", \"precision\":\"" + precision + "\", \"size\": \"500\", \"shard_size\": \"550\"}"); XContentParser.Token token = stParser.nextToken(); assertSame(XContentParser.Token.START_OBJECT, token); // can create a factory - assertNotNull(GeoGridAggregationBuilder2.parse("geohash_grid", stParser)); + assertNotNull(GeoGridAggregationBuilder2.parse("geo_grid", stParser)); } public void testParseDistanceUnitPrecision() throws Exception { @@ -63,11 +63,12 @@ public void testParseDistanceUnitPrecision() throws Exception { } String distanceString = distance + unit.toString(); XContentParser stParser = createParser(JsonXContent.jsonXContent, - "{\"field\":\"my_loc\", \"precision\": \"" + distanceString + "\", \"size\": \"500\", \"shard_size\": \"550\"}"); + "{\"hash_type\":\"geohash\", \"field\":\"my_loc\", \"precision\": \"" + distanceString + + "\", \"size\": \"500\", \"shard_size\": \"550\"}"); XContentParser.Token token = stParser.nextToken(); assertSame(XContentParser.Token.START_OBJECT, token); // can create a factory - GeoGridAggregationBuilder2 builder = GeoGridAggregationBuilder2.parse("geohash_grid", stParser); + GeoGridAggregationBuilder2 builder = GeoGridAggregationBuilder2.parse("geo_grid", stParser); assertNotNull(builder); assertThat(builder.precision(), greaterThanOrEqualTo(0)); assertThat(builder.precision(), lessThanOrEqualTo(12)); @@ -75,36 +76,30 @@ public void testParseDistanceUnitPrecision() throws Exception { public void testParseInvalidUnitPrecision() throws Exception { XContentParser stParser = createParser(JsonXContent.jsonXContent, - "{\"field\":\"my_loc\", \"precision\": \"10kg\", \"size\": \"500\", \"shard_size\": \"550\"}"); + "{\"hash_type\":\"geohash\", \"field\":\"my_loc\", \"precision\": \"10kg\", " + + "\"size\": \"500\", \"shard_size\": \"550\"}"); XContentParser.Token token = stParser.nextToken(); assertSame(XContentParser.Token.START_OBJECT, token); XContentParseException ex = expectThrows(XContentParseException.class, - () -> GeoGridAggregationBuilder2.parse("geohash_grid", stParser)); - assertThat(ex.getMessage(), containsString("failed to build [geohash_grid] after last required field arrived")); + () -> GeoGridAggregationBuilder2.parse("geo_grid", stParser)); + assertThat(ex.getMessage(), containsString("[geo_grid] failed to parse field [precision]")); Throwable cause = ex.getCause(); - assertThat(cause, instanceOf(XContentParseException.class)); - assertThat(cause.getMessage(), containsString("[geohash_grid] failed to parse field [precision]")); - - cause = cause.getCause(); assertThat(cause, instanceOf(NumberFormatException.class)); assertThat(cause.getMessage(), containsString("For input string: \"10kg\"")); } public void testParseDistanceUnitPrecisionTooSmall() throws Exception { XContentParser stParser = createParser(JsonXContent.jsonXContent, - "{\"field\":\"my_loc\", \"precision\": \"1cm\", \"size\": \"500\", \"shard_size\": \"550\"}"); + "{\"hash_type\":\"geohash\", \"field\":\"my_loc\", " + + "\"precision\": \"1cm\", \"size\": \"500\", \"shard_size\": \"550\"}"); XContentParser.Token token = stParser.nextToken(); assertSame(XContentParser.Token.START_OBJECT, token); XContentParseException ex = expectThrows(XContentParseException.class, - () -> GeoGridAggregationBuilder2.parse("geohash_grid", stParser)); - assertThat(ex.getMessage(), containsString("failed to build [geohash_grid] after last required field arrived")); + () -> GeoGridAggregationBuilder2.parse("geo_grid", stParser)); + assertThat(ex.getMessage(), containsString("[geo_grid] failed to parse field [precision]")); Throwable cause = ex.getCause(); - assertThat(cause, instanceOf(XContentParseException.class)); - assertThat(cause.getMessage(), containsString("[geohash_grid] failed to parse field [precision]")); - - cause = cause.getCause(); assertThat(cause, instanceOf(IllegalArgumentException.class)); assertEquals("precision too high [1cm]", cause.getMessage()); } @@ -112,28 +107,28 @@ public void testParseDistanceUnitPrecisionTooSmall() throws Exception { public void testParseErrorOnBooleanPrecision() throws Exception { GeoGridType type = GeoGridTests.randomType(); XContentParser stParser = createParser(JsonXContent.jsonXContent, - "{\"field\":\"my_loc\", \"hash_type\":\"" + type + "\", \"precision\":false}"); + "{\"hash_type\":\"" + type.getName() + "\", \"field\":\"my_loc\", \"precision\":false}"); XContentParser.Token token = stParser.nextToken(); assertSame(XContentParser.Token.START_OBJECT, token); XContentParseException ex = expectThrows(XContentParseException.class, - () -> GeoGridAggregationBuilder2.parse("geohash_grid", stParser)); - assertThat(ex.getMessage(), containsString("[geohash_grid] failed to parse field [precision]")); + () -> GeoGridAggregationBuilder2.parse("geo_grid", stParser)); + assertThat(ex.getMessage(), containsString("[geo_grid] failed to parse field [precision]")); Throwable cause = ex.getCause(); assertThat(cause, instanceOf(XContentParseException.class)); - assertThat(cause.getMessage(), containsString("[geohash_grid] failed to parse field [precision]" + - " in [geohash_grid]. It must be either an integer or a string")); + assertThat(cause.getMessage(), containsString("[geo_grid] failed to parse field [precision]" + + " in [geo_grid]. It must be either an integer or a string")); } public void testParseErrorOnPrecisionOutOfRange() throws Exception { final GeoGridType type = GeoGridTests.randomType(); final int precision = GeoGridTests.maxPrecision(type) + 1; XContentParser stParser = createParser(JsonXContent.jsonXContent, - "{\"field\":\"my_loc\", \"hash_type\":\"" + type + "\", \"precision\":\""+ precision +"\"}"); + "{\"hash_type\":\"" + type.getName() + "\", \"field\":\"my_loc\", \"precision\":\""+ precision +"\"}"); XContentParser.Token token = stParser.nextToken(); assertSame(XContentParser.Token.START_OBJECT, token); try { - GeoGridAggregationBuilder2.parse("geohash_grid", stParser); + GeoGridAggregationBuilder2.parse("geo_grid", stParser); fail(); } catch (XContentParseException ex) { assertThat(ex.getCause(), instanceOf(IllegalArgumentException.class)); diff --git a/server/src/test/java/org/elasticsearch/search/aggregations/bucket/geogrid2/InternalGeoGridGeoHashTests.java b/server/src/test/java/org/elasticsearch/search/aggregations/bucket/geogrid2/InternalGeoGridGeoHashTests.java index dd15486c00341..6324d3133664d 100644 --- a/server/src/test/java/org/elasticsearch/search/aggregations/bucket/geogrid2/InternalGeoGridGeoHashTests.java +++ b/server/src/test/java/org/elasticsearch/search/aggregations/bucket/geogrid2/InternalGeoGridGeoHashTests.java @@ -22,7 +22,6 @@ import org.elasticsearch.common.geo.GeoHashUtils; import org.elasticsearch.common.io.stream.Writeable; import org.elasticsearch.search.aggregations.InternalAggregations; -import org.elasticsearch.search.aggregations.bucket.GeoGridTests; import org.elasticsearch.test.InternalMultiBucketAggregationTestCase; import org.elasticsearch.search.aggregations.ParsedMultiBucketAggregation; import org.elasticsearch.search.aggregations.pipeline.PipelineAggregator; @@ -58,7 +57,7 @@ protected InternalGeoGrid createTestInstance(String name, long geoHashAsLong = GeoHashUtils.longEncode(longitude, latitude, 4); buckets.add(new GeoHashBucket(geoHashAsLong, randomInt(IndexWriter.MAX_DOCS), aggregations)); } - return new InternalGeoGrid(name, GeoGridTests.GEOHASH_TYPE, size, buckets, pipelineAggregators, metaData); + return new InternalGeoGrid(name, GeoHashType.SINGLETON, size, buckets, pipelineAggregators, metaData); } @Override @@ -137,7 +136,7 @@ protected InternalGeoGrid mutateInstance(InternalGeoGrid instance) { default: throw new AssertionError("Illegal randomisation branch"); } - return new InternalGeoGrid(name, GeoGridTests.GEOHASH_TYPE, size, buckets, pipelineAggregators, metaData); + return new InternalGeoGrid(name, GeoHashType.SINGLETON, size, buckets, pipelineAggregators, metaData); } } diff --git a/server/src/test/java/org/elasticsearch/search/aggregations/metrics/GeoCentroidIT.java b/server/src/test/java/org/elasticsearch/search/aggregations/metrics/GeoCentroidIT.java index bd28d13d01682..aa117bd1f1279 100644 --- a/server/src/test/java/org/elasticsearch/search/aggregations/metrics/GeoCentroidIT.java +++ b/server/src/test/java/org/elasticsearch/search/aggregations/metrics/GeoCentroidIT.java @@ -22,9 +22,9 @@ import org.elasticsearch.action.search.SearchResponse; import org.elasticsearch.common.geo.GeoPoint; import org.elasticsearch.search.aggregations.InternalAggregation; -import org.elasticsearch.search.aggregations.bucket.GeoGridTests; import org.elasticsearch.search.aggregations.bucket.geogrid.GeoHashGrid; import org.elasticsearch.search.aggregations.bucket.geogrid2.GeoGrid; +import org.elasticsearch.search.aggregations.bucket.geogrid2.GeoHashType; import org.elasticsearch.search.aggregations.bucket.global.Global; import org.elasticsearch.test.ESIntegTestCase; @@ -179,7 +179,7 @@ public void testSingleValueFieldAsSubAggToGeohashGrid() throws Exception { public void testSingleValueFieldAsSubAggToGeoGrid() throws Exception { SearchResponse response = client().prepareSearch(HIGH_CARD_IDX_NAME) - .addAggregation(geoGrid("geoGrid", GeoGridTests.GEOHASH_TYPE).field(SINGLE_VALUED_FIELD_NAME) + .addAggregation(geoGrid("geoGrid", GeoHashType.SINGLETON).field(SINGLE_VALUED_FIELD_NAME) .subAggregation(geoCentroid(aggName).field(SINGLE_VALUED_FIELD_NAME))) .get(); assertSearchResponse(response); From d9e2ebca674c8834d83594deb24fdced4f720061 Mon Sep 17 00:00:00 2001 From: David Turner Date: Wed, 9 Jan 2019 08:18:25 +0000 Subject: [PATCH 18/46] Add more detail to recovery bandwidth limit docs (#37156) --- .../modules/indices/recovery.asciidoc | 20 ++++++++++++++++--- 1 file changed, 17 insertions(+), 3 deletions(-) diff --git a/docs/reference/modules/indices/recovery.asciidoc b/docs/reference/modules/indices/recovery.asciidoc index 84352cfe6898e..d9e034941f80e 100644 --- a/docs/reference/modules/indices/recovery.asciidoc +++ b/docs/reference/modules/indices/recovery.asciidoc @@ -1,10 +1,24 @@ [[recovery]] === Indices Recovery -The following _expert_ setting can be set to manage the recovery policy. +<> is the process used to build a new copy of a +shard on a node by copying data from the primary. {es} uses this peer recovery +process to rebuild shard copies that were lost if a node has failed, and uses +the same process when migrating a shard copy between nodes to rebalance the +cluster or to honor any changes to the <>. + +The following _expert_ setting can be set to manage the resources consumed by +peer recoveries: `indices.recovery.max_bytes_per_sec`:: - Defaults to `40mb`. + Limits the total inbound and outbound peer recovery traffic on each node. + Since this limit applies on each node, but there may be many nodes + performing peer recoveries concurrently, the total amount of peer recovery + traffic within a cluster may be much higher than this limit. If you set + this limit too high then there is a risk that ongoing peer recoveries will + consume an excess of bandwidth (or other resources) which could destabilize + the cluster. Defaults to `40mb`. This setting can be dynamically updated on a live cluster with the -<> API: +<> API. From 0a40564068d024fd97051c5839c86f92a95b357f Mon Sep 17 00:00:00 2001 From: Tanguy Leroux Date: Wed, 9 Jan 2019 09:24:56 +0100 Subject: [PATCH 19/46] [Tests] Change cluster scope in CorruptedFileIT and FlushIT (#37229) These tests failed on CI multiple times in the past weeks because they use a test cluster with a SUITE scope that recreates nodes between tests. With such a scope, nodes can be recreated in between test executions and can inherit a node id from a previous test execution, while they are assigned a random data path. With the successive node recreations it is possible that a newly recreated node shares the same node id (but different data path) as a non recreated node. This commit changes the cluster scope of the CorruptedFileIT and FlushIT tests which often fail. The failure is reproducable with : ./gradlew :server:integTest -Dtests.seed=EF3A50C225CF377 -Dtests.class=org.elasticsearch.index.store.CorruptedFileIT -Dtests.security.manager=true -Dtests.locale=th-TH-u-nu-thai-x-lvariant-TH -Dtests.timezone=America/Rio_Branco -Dcompiler.java=11 -Druntime.java=8 --- .../java/org/elasticsearch/index/store/CorruptedFileIT.java | 2 +- .../src/test/java/org/elasticsearch/indices/flush/FlushIT.java | 1 + 2 files changed, 2 insertions(+), 1 deletion(-) diff --git a/server/src/test/java/org/elasticsearch/index/store/CorruptedFileIT.java b/server/src/test/java/org/elasticsearch/index/store/CorruptedFileIT.java index 966495faa1e13..cd0c90f50779c 100644 --- a/server/src/test/java/org/elasticsearch/index/store/CorruptedFileIT.java +++ b/server/src/test/java/org/elasticsearch/index/store/CorruptedFileIT.java @@ -107,7 +107,7 @@ import static org.hamcrest.Matchers.not; import static org.hamcrest.Matchers.notNullValue; -@ESIntegTestCase.ClusterScope(scope = ESIntegTestCase.Scope.SUITE) +@ESIntegTestCase.ClusterScope(scope = ESIntegTestCase.Scope.TEST) public class CorruptedFileIT extends ESIntegTestCase { @Override diff --git a/server/src/test/java/org/elasticsearch/indices/flush/FlushIT.java b/server/src/test/java/org/elasticsearch/indices/flush/FlushIT.java index f7d920d1729a6..5535a947d9ef8 100644 --- a/server/src/test/java/org/elasticsearch/indices/flush/FlushIT.java +++ b/server/src/test/java/org/elasticsearch/indices/flush/FlushIT.java @@ -64,6 +64,7 @@ import static org.hamcrest.Matchers.not; import static org.hamcrest.Matchers.nullValue; +@ESIntegTestCase.ClusterScope(scope = ESIntegTestCase.Scope.TEST) public class FlushIT extends ESIntegTestCase { public void testWaitIfOngoing() throws InterruptedException { From 9040a96daf5e052556ec7140d3fbb03f1ece1b4f Mon Sep 17 00:00:00 2001 From: Yannick Welsch Date: Wed, 9 Jan 2019 09:32:24 +0100 Subject: [PATCH 20/46] Zen2: Add test that checks that CoordinatorTests are reproducible (#37225) Ensures reproducibility of the tests in CoordinatorTests. --- .../coordination/CoordinatorTests.java | 20 +++++++++++++++++++ 1 file changed, 20 insertions(+) diff --git a/server/src/test/java/org/elasticsearch/cluster/coordination/CoordinatorTests.java b/server/src/test/java/org/elasticsearch/cluster/coordination/CoordinatorTests.java index 17bf36b411b52..a264015261de5 100644 --- a/server/src/test/java/org/elasticsearch/cluster/coordination/CoordinatorTests.java +++ b/server/src/test/java/org/elasticsearch/cluster/coordination/CoordinatorTests.java @@ -18,6 +18,7 @@ */ package org.elasticsearch.cluster.coordination; +import com.carrotsearch.randomizedtesting.RandomizedContext; import org.apache.logging.log4j.CloseableThreadContext; import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; @@ -70,6 +71,7 @@ import java.util.Map; import java.util.Optional; import java.util.Set; +import java.util.concurrent.Callable; import java.util.concurrent.atomic.AtomicBoolean; import java.util.function.Consumer; import java.util.function.Function; @@ -128,6 +130,24 @@ public void resetPortCounterBeforeEachTest() { resetPortCounter(); } + // check that runRandomly leads to reproducible results + public void testRepeatableTests() throws Exception { + final Callable test = () -> { + final Cluster cluster = new Cluster(randomIntBetween(1, 5)); + cluster.runRandomly(); + final long afterRunRandomly = value(cluster.getAnyNode().getLastAppliedClusterState()); + cluster.stabilise(); + final long afterStabilisation = value(cluster.getAnyNode().getLastAppliedClusterState()); + return afterRunRandomly ^ afterStabilisation; + }; + final long seed = randomLong(); + logger.info("First run with seed [{}]", seed); + final long result1 = RandomizedContext.current().runWithPrivateRandomness(seed, test); + logger.info("Second run with seed [{}]", seed); + final long result2 = RandomizedContext.current().runWithPrivateRandomness(seed, test); + assertEquals(result1, result2); + } + public void testCanUpdateClusterStateAfterStabilisation() { final Cluster cluster = new Cluster(randomIntBetween(1, 5)); cluster.runRandomly(); From 224475e6aa65828e94912803b4aa066de5c1f7ed Mon Sep 17 00:00:00 2001 From: Armin Braun Date: Wed, 9 Jan 2019 09:45:37 +0100 Subject: [PATCH 21/46] MINOR: Some Cleanups in Snapshots (#37175) * Removed pre v6 Bwc logic + comments * Removed some dead code --- .../snapshots/SnapshotException.java | 4 --- .../elasticsearch/snapshots/SnapshotId.java | 28 +++++++------------ .../snapshots/SnapshotShardsService.java | 2 +- .../snapshots/SnapshotsService.java | 6 ++-- 4 files changed, 13 insertions(+), 27 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/snapshots/SnapshotException.java b/server/src/main/java/org/elasticsearch/snapshots/SnapshotException.java index d389ed634f3af..05db85d6f7211 100644 --- a/server/src/main/java/org/elasticsearch/snapshots/SnapshotException.java +++ b/server/src/main/java/org/elasticsearch/snapshots/SnapshotException.java @@ -51,10 +51,6 @@ public SnapshotException(final Snapshot snapshot, final String msg, final Throwa } } - public SnapshotException(final String repositoryName, final SnapshotId snapshotId, final String msg) { - this(repositoryName, snapshotId, msg, null); - } - public SnapshotException(final String repositoryName, final SnapshotId snapshotId, final String msg, final Throwable cause) { super("[" + repositoryName + ":" + snapshotId + "] " + msg, cause); this.repositoryName = repositoryName; diff --git a/server/src/main/java/org/elasticsearch/snapshots/SnapshotId.java b/server/src/main/java/org/elasticsearch/snapshots/SnapshotId.java index 7a8848618c25c..59e1d960bcbfc 100644 --- a/server/src/main/java/org/elasticsearch/snapshots/SnapshotId.java +++ b/server/src/main/java/org/elasticsearch/snapshots/SnapshotId.java @@ -131,25 +131,17 @@ public XContentBuilder toXContent(XContentBuilder builder, Params params) throws } public static SnapshotId fromXContent(XContentParser parser) throws IOException { - // the new format from 5.0 which contains the snapshot name and uuid - if (parser.currentToken() == XContentParser.Token.START_OBJECT) { - String name = null; - String uuid = null; - while (parser.nextToken() != XContentParser.Token.END_OBJECT) { - String currentFieldName = parser.currentName(); - parser.nextToken(); - if (NAME.equals(currentFieldName)) { - name = parser.text(); - } else if (UUID.equals(currentFieldName)) { - uuid = parser.text(); - } + String name = null; + String uuid = null; + while (parser.nextToken() != XContentParser.Token.END_OBJECT) { + String currentFieldName = parser.currentName(); + parser.nextToken(); + if (NAME.equals(currentFieldName)) { + name = parser.text(); + } else if (UUID.equals(currentFieldName)) { + uuid = parser.text(); } - return new SnapshotId(name, uuid); - } else { - // the old format pre 5.0 that only contains the snapshot name, use the name as the uuid too - final String name = parser.text(); - return new SnapshotId(name, name); } + return new SnapshotId(name, uuid); } - } diff --git a/server/src/main/java/org/elasticsearch/snapshots/SnapshotShardsService.java b/server/src/main/java/org/elasticsearch/snapshots/SnapshotShardsService.java index 40c89f10ccbc5..c0e196f1f4eb3 100644 --- a/server/src/main/java/org/elasticsearch/snapshots/SnapshotShardsService.java +++ b/server/src/main/java/org/elasticsearch/snapshots/SnapshotShardsService.java @@ -94,7 +94,7 @@ public class SnapshotShardsService extends AbstractLifecycleComponent implements ClusterStateListener, IndexEventListener { private static final Logger logger = LogManager.getLogger(SnapshotShardsService.class); - public static final String UPDATE_SNAPSHOT_STATUS_ACTION_NAME = "internal:cluster/snapshot/update_snapshot_status"; + private static final String UPDATE_SNAPSHOT_STATUS_ACTION_NAME = "internal:cluster/snapshot/update_snapshot_status"; private final ClusterService clusterService; diff --git a/server/src/main/java/org/elasticsearch/snapshots/SnapshotsService.java b/server/src/main/java/org/elasticsearch/snapshots/SnapshotsService.java index 8c505d20d17ff..65802377be032 100644 --- a/server/src/main/java/org/elasticsearch/snapshots/SnapshotsService.java +++ b/server/src/main/java/org/elasticsearch/snapshots/SnapshotsService.java @@ -515,7 +515,7 @@ public void onFailure(Exception e) { cleanupAfterError(e); } - public void onNoLongerMaster(String source) { + public void onNoLongerMaster() { userCreateSnapshotListener.onFailure(e); } @@ -1073,7 +1073,7 @@ public void onFailure(String source, Exception e) { @Override public void onNoLongerMaster(String source) { if (listener != null) { - listener.onNoLongerMaster(source); + listener.onNoLongerMaster(); } } @@ -1423,8 +1423,6 @@ private ImmutableOpenMap shard builder.put(shardId, new SnapshotsInProgress.ShardSnapshotStatus(null, State.MISSING, "primary shard is not allocated")); } else if (primary.relocating() || primary.initializing()) { - // The WAITING state was introduced in V1.2.0 - - // don't use it if there are nodes with older version in the cluster builder.put(shardId, new SnapshotsInProgress.ShardSnapshotStatus(primary.currentNodeId(), State.WAITING)); } else if (!primary.started()) { builder.put(shardId, From f14cff2102704b627b18a641f38000ef319e746b Mon Sep 17 00:00:00 2001 From: David Roberts Date: Wed, 9 Jan 2019 08:51:00 +0000 Subject: [PATCH 22/46] [TEST] Ensure interrupted flag reset after test that sets it (#37230) Test fix to stop a problem in one test leaking into a different test and causing that other test to spuriously fail. --- .../xpack/ml/filestructurefinder/TimeoutCheckerTests.java | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/filestructurefinder/TimeoutCheckerTests.java b/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/filestructurefinder/TimeoutCheckerTests.java index 8518096d64478..ea581f663462f 100644 --- a/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/filestructurefinder/TimeoutCheckerTests.java +++ b/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/filestructurefinder/TimeoutCheckerTests.java @@ -72,6 +72,9 @@ public void testWatchdog() { } finally { TimeoutChecker.watchdog.unregister(); } + } finally { + // ensure the interrupted flag is cleared to stop it making subsequent tests fail + Thread.interrupted(); } } @@ -89,6 +92,9 @@ public void testGrokCaptures() throws Exception { assertEquals("Aborting grok captures test during [should timeout] as it has taken longer than the timeout of [" + timeout + "]", e.getMessage()); }); + } finally { + // ensure the interrupted flag is cleared to stop it making subsequent tests fail + Thread.interrupted(); } } } From e0ce73713f7292954ff66691919abf62d71442cd Mon Sep 17 00:00:00 2001 From: David Roberts Date: Wed, 9 Jan 2019 10:42:47 +0000 Subject: [PATCH 23/46] [ML] Stop datafeeds running when their jobs are stale (#37227) We already had logic to stop datafeeds running against jobs that were OPENING, but a job that relocates from one node to another while OPENED stays OPENED, and this could cause the datafeed to fail when it sent data to the OPENED job on its new node before it had a corresponding autodetect process. This change extends the check to stop datafeeds running when their job is OPENING _or_ stale (i.e. has not had its status reset since relocating to a different node). Relates #36810 --- .../elasticsearch/xpack/core/ml/MlTasks.java | 35 ++++++++++++++ .../core/ml/job/config/JobTaskState.java | 11 +++++ .../ml/action/TransportOpenJobAction.java | 26 ++-------- .../xpack/ml/datafeed/DatafeedManager.java | 8 ++-- .../action/TransportOpenJobActionTests.java | 10 +++- .../ml/datafeed/DatafeedManagerTests.java | 47 +++++++++++++++++-- 6 files changed, 104 insertions(+), 33 deletions(-) diff --git a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/ml/MlTasks.java b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/ml/MlTasks.java index e78649d152296..b81a1f7d7b9c0 100644 --- a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/ml/MlTasks.java +++ b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/ml/MlTasks.java @@ -55,6 +55,11 @@ public static PersistentTasksCustomMetaData.PersistentTask getDatafeedTask(St return tasks == null ? null : tasks.getTask(datafeedTaskId(datafeedId)); } + /** + * Note that the return value of this method does NOT take node relocations into account. + * Use {@link #getJobStateModifiedForReassignments} to return a value adjusted to the most + * appropriate value following relocations. + */ public static JobState getJobState(String jobId, @Nullable PersistentTasksCustomMetaData tasks) { PersistentTasksCustomMetaData.PersistentTask task = getJobTask(jobId, tasks); if (task != null) { @@ -68,6 +73,36 @@ public static JobState getJobState(String jobId, @Nullable PersistentTasksCustom return JobState.CLOSED; } + public static JobState getJobStateModifiedForReassignments(String jobId, @Nullable PersistentTasksCustomMetaData tasks) { + return getJobStateModifiedForReassignments(getJobTask(jobId, tasks)); + } + + public static JobState getJobStateModifiedForReassignments(@Nullable PersistentTasksCustomMetaData.PersistentTask task) { + if (task == null) { + // A closed job has no persistent task + return JobState.CLOSED; + } + JobTaskState jobTaskState = (JobTaskState) task.getState(); + if (jobTaskState == null) { + return JobState.OPENING; + } + JobState jobState = jobTaskState.getState(); + if (jobTaskState.isStatusStale(task)) { + // the job is re-locating + if (jobState == JobState.CLOSING) { + // previous executor node failed while the job was closing - it won't + // be reopened on another node, so consider it CLOSED for most purposes + return JobState.CLOSED; + } + if (jobState != JobState.FAILED) { + // previous executor node failed and current executor node didn't + // have the chance to set job status to OPENING + return JobState.OPENING; + } + } + return jobState; + } + public static DatafeedState getDatafeedState(String datafeedId, @Nullable PersistentTasksCustomMetaData tasks) { PersistentTasksCustomMetaData.PersistentTask task = getDatafeedTask(datafeedId, tasks); if (task != null && task.getState() != null) { diff --git a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/ml/job/config/JobTaskState.java b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/ml/job/config/JobTaskState.java index 2e6cc4b99c4bb..d979b897ad43a 100644 --- a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/ml/job/config/JobTaskState.java +++ b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/ml/job/config/JobTaskState.java @@ -67,6 +67,17 @@ public JobState getState() { return state; } + /** + * The job state stores the allocation ID at the time it was last set. + * This method compares the allocation ID in the state with the allocation + * ID in the task. If the two are different then the task has been relocated + * to a different node after the last time the state was set. This in turn + * means that the state is not necessarily correct. For example, a job that + * has a state of OPENED but is stale must be considered to be OPENING, because + * it won't yet have a corresponding autodetect process. + * @param task The job task to check. + * @return Has the task been relocated to another node and not had its status set since then? + */ public boolean isStatusStale(PersistentTask task) { return allocationId != task.getAllocationId(); } diff --git a/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/action/TransportOpenJobAction.java b/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/action/TransportOpenJobAction.java index fad24247834d5..c81a539fb0ea4 100644 --- a/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/action/TransportOpenJobAction.java +++ b/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/action/TransportOpenJobAction.java @@ -225,31 +225,13 @@ static PersistentTasksCustomMetaData.Assignment selectLeastLoadedMlNode(String j Collection> assignedTasks = persistentTasks.findTasks( MlTasks.JOB_TASK_NAME, task -> node.getId().equals(task.getExecutorNode())); for (PersistentTasksCustomMetaData.PersistentTask assignedTask : assignedTasks) { - JobTaskState jobTaskState = (JobTaskState) assignedTask.getState(); - JobState jobState; - if (jobTaskState == null) { - // executor node didn't have the chance to set job status to OPENING - ++numberOfAllocatingJobs; - jobState = JobState.OPENING; - } else { - jobState = jobTaskState.getState(); - if (jobTaskState.isStatusStale(assignedTask)) { - // the job is re-locating - if (jobState == JobState.CLOSING) { - // previous executor node failed while the job was closing - it won't - // be reopened, so consider it CLOSED for resource usage purposes - jobState = JobState.CLOSED; - } else if (jobState != JobState.FAILED) { - // previous executor node failed and current executor node didn't - // have the chance to set job status to OPENING - ++numberOfAllocatingJobs; - jobState = JobState.OPENING; - } - } - } + JobState jobState = MlTasks.getJobStateModifiedForReassignments(assignedTask); if (jobState.isAnyOf(JobState.CLOSED, JobState.FAILED) == false) { // Don't count CLOSED or FAILED jobs, as they don't consume native memory ++numberOfAssignedJobs; + if (jobState == JobState.OPENING) { + ++numberOfAllocatingJobs; + } OpenJobAction.JobParams params = (OpenJobAction.JobParams) assignedTask.getParams(); Long jobMemoryRequirement = memoryTracker.getJobMemoryRequirement(params.getJobId()); if (jobMemoryRequirement == null) { diff --git a/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/datafeed/DatafeedManager.java b/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/datafeed/DatafeedManager.java index 4a0f3da060d02..6367a13100ed0 100644 --- a/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/datafeed/DatafeedManager.java +++ b/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/datafeed/DatafeedManager.java @@ -161,7 +161,7 @@ public void onFailure(Exception e) { protected void doRun() { Long next = null; try { - next = holder.executeLoopBack(startTime, endTime); + next = holder.executeLookBack(startTime, endTime); } catch (DatafeedJob.ExtractionProblemException e) { if (endTime == null) { next = e.nextDelayInMsSinceEpoch; @@ -253,7 +253,7 @@ private String getJobId(TransportStartDatafeedAction.DatafeedTask task) { } private JobState getJobState(PersistentTasksCustomMetaData tasks, TransportStartDatafeedAction.DatafeedTask datafeedTask) { - return MlTasks.getJobState(getJobId(datafeedTask), tasks); + return MlTasks.getJobStateModifiedForReassignments(getJobId(datafeedTask), tasks); } private TimeValue computeNextDelay(long next) { @@ -272,7 +272,7 @@ public class Holder { private final TransportStartDatafeedAction.DatafeedTask task; private final long allocationId; private final String datafeedId; - // To ensure that we wait until loopback / realtime search has completed before we stop the datafeed + // To ensure that we wait until lookback / realtime search has completed before we stop the datafeed private final ReentrantLock datafeedJobLock = new ReentrantLock(true); private final DatafeedJob datafeedJob; private final boolean autoCloseJob; @@ -352,7 +352,7 @@ public void setRelocating() { isRelocating = true; } - private Long executeLoopBack(long startTime, Long endTime) throws Exception { + private Long executeLookBack(long startTime, Long endTime) throws Exception { datafeedJobLock.lock(); try { if (isRunning() && !isIsolated()) { diff --git a/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/action/TransportOpenJobActionTests.java b/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/action/TransportOpenJobActionTests.java index cfb16254a9dde..04dfa5f27502d 100644 --- a/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/action/TransportOpenJobActionTests.java +++ b/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/action/TransportOpenJobActionTests.java @@ -575,10 +575,16 @@ public void testJobTaskMatcherMatch() { } public static void addJobTask(String jobId, String nodeId, JobState jobState, PersistentTasksCustomMetaData.Builder builder) { + addJobTask(jobId, nodeId, jobState, builder, false); + } + + public static void addJobTask(String jobId, String nodeId, JobState jobState, PersistentTasksCustomMetaData.Builder builder, + boolean isStale) { builder.addTask(MlTasks.jobTaskId(jobId), MlTasks.JOB_TASK_NAME, new OpenJobAction.JobParams(jobId), - new Assignment(nodeId, "test assignment")); + new Assignment(nodeId, "test assignment")); if (jobState != null) { - builder.updateTaskState(MlTasks.jobTaskId(jobId), new JobTaskState(jobState, builder.getLastAllocationId())); + builder.updateTaskState(MlTasks.jobTaskId(jobId), + new JobTaskState(jobState, builder.getLastAllocationId() - (isStale ? 1 : 0))); } } diff --git a/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/datafeed/DatafeedManagerTests.java b/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/datafeed/DatafeedManagerTests.java index edf734544091c..9bf883232c623 100644 --- a/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/datafeed/DatafeedManagerTests.java +++ b/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/datafeed/DatafeedManagerTests.java @@ -222,7 +222,7 @@ public void testRealTime_GivenNonStoppingAnalysisProblem() throws Exception { assertThat(datafeedManager.isRunning(task.getAllocationId()), is(true)); } - public void testStart_GivenNewlyCreatedJobLoopBackAndRealtime() throws Exception { + public void testStart_GivenNewlyCreatedJobLookBackAndRealtime() throws Exception { when(datafeedJob.runLookBack(anyLong(), anyLong())).thenReturn(1L); when(datafeedJob.runRealtime()).thenReturn(1L); @@ -282,8 +282,45 @@ public void testDatafeedTaskWaitsUntilJobIsOpened() { verify(threadPool, times(1)).executor(MachineLearning.DATAFEED_THREAD_POOL_NAME); } + public void testDatafeedTaskWaitsUntilJobIsNotStale() { + PersistentTasksCustomMetaData.Builder tasksBuilder = PersistentTasksCustomMetaData.builder(); + addJobTask("job_id", "node_id", JobState.OPENED, tasksBuilder, true); + ClusterState.Builder cs = ClusterState.builder(clusterService.state()) + .metaData(new MetaData.Builder().putCustom(PersistentTasksCustomMetaData.TYPE, tasksBuilder.build())); + when(clusterService.state()).thenReturn(cs.build()); + + Consumer handler = mockConsumer(); + DatafeedTask task = createDatafeedTask("datafeed_id", 0L, 60000L); + datafeedManager.run(task, handler); + + // Verify datafeed has not started running yet as job is stale (i.e. even though opened it is part way through relocating) + verify(threadPool, never()).executor(MachineLearning.DATAFEED_THREAD_POOL_NAME); + + tasksBuilder = PersistentTasksCustomMetaData.builder(); + addJobTask("job_id", "node_id", JobState.OPENED, tasksBuilder, true); + addJobTask("another_job", "node_id", JobState.OPENED, tasksBuilder); + ClusterState.Builder anotherJobCs = ClusterState.builder(clusterService.state()) + .metaData(new MetaData.Builder().putCustom(PersistentTasksCustomMetaData.TYPE, tasksBuilder.build())); + + capturedClusterStateListener.getValue().clusterChanged(new ClusterChangedEvent("_source", anotherJobCs.build(), cs.build())); + + // Still no run + verify(threadPool, never()).executor(MachineLearning.DATAFEED_THREAD_POOL_NAME); + + tasksBuilder = PersistentTasksCustomMetaData.builder(); + addJobTask("job_id", "node_id", JobState.OPENED, tasksBuilder); + ClusterState.Builder jobOpenedCs = ClusterState.builder(clusterService.state()) + .metaData(new MetaData.Builder().putCustom(PersistentTasksCustomMetaData.TYPE, tasksBuilder.build())); + + capturedClusterStateListener.getValue().clusterChanged( + new ClusterChangedEvent("_source", jobOpenedCs.build(), anotherJobCs.build())); + + // Now it should run as the job state chanded to OPENED + verify(threadPool, times(1)).executor(MachineLearning.DATAFEED_THREAD_POOL_NAME); + } + public void testDatafeedTaskStopsBecauseJobFailedWhileOpening() { - PersistentTasksCustomMetaData.Builder tasksBuilder = PersistentTasksCustomMetaData.builder(); + PersistentTasksCustomMetaData.Builder tasksBuilder = PersistentTasksCustomMetaData.builder(); addJobTask("job_id", "node_id", JobState.OPENING, tasksBuilder); ClusterState.Builder cs = ClusterState.builder(clusterService.state()) .metaData(new MetaData.Builder().putCustom(PersistentTasksCustomMetaData.TYPE, tasksBuilder.build())); @@ -296,7 +333,7 @@ public void testDatafeedTaskStopsBecauseJobFailedWhileOpening() { // Verify datafeed has not started running yet as job is still opening verify(threadPool, never()).executor(MachineLearning.DATAFEED_THREAD_POOL_NAME); - tasksBuilder = PersistentTasksCustomMetaData.builder(); + tasksBuilder = PersistentTasksCustomMetaData.builder(); addJobTask("job_id", "node_id", JobState.FAILED, tasksBuilder); ClusterState.Builder updatedCs = ClusterState.builder(clusterService.state()) .metaData(new MetaData.Builder().putCustom(PersistentTasksCustomMetaData.TYPE, tasksBuilder.build())); @@ -309,7 +346,7 @@ public void testDatafeedTaskStopsBecauseJobFailedWhileOpening() { } public void testDatafeedGetsStoppedWhileWaitingForJobToOpen() { - PersistentTasksCustomMetaData.Builder tasksBuilder = PersistentTasksCustomMetaData.builder(); + PersistentTasksCustomMetaData.Builder tasksBuilder = PersistentTasksCustomMetaData.builder(); addJobTask("job_id", "node_id", JobState.OPENING, tasksBuilder); ClusterState.Builder cs = ClusterState.builder(clusterService.state()) .metaData(new MetaData.Builder().putCustom(PersistentTasksCustomMetaData.TYPE, tasksBuilder.build())); @@ -326,7 +363,7 @@ public void testDatafeedGetsStoppedWhileWaitingForJobToOpen() { datafeedManager.stopDatafeed(task, "test", StopDatafeedAction.DEFAULT_TIMEOUT); // Update job state to opened - tasksBuilder = PersistentTasksCustomMetaData.builder(); + tasksBuilder = PersistentTasksCustomMetaData.builder(); addJobTask("job_id", "node_id", JobState.OPENED, tasksBuilder); ClusterState.Builder updatedCs = ClusterState.builder(clusterService.state()) .metaData(new MetaData.Builder().putCustom(PersistentTasksCustomMetaData.TYPE, tasksBuilder.build())); From 91225853591a8e35eaecdee4a67fde673d83052b Mon Sep 17 00:00:00 2001 From: Martijn van Groningen Date: Wed, 9 Jan 2019 12:17:47 +0100 Subject: [PATCH 24/46] [CCR] Added more logging. --- .../elasticsearch/xpack/ccr/action/AutoFollowCoordinator.java | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/AutoFollowCoordinator.java b/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/AutoFollowCoordinator.java index 0a7900d004b7b..55e24abc86c26 100644 --- a/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/AutoFollowCoordinator.java +++ b/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/AutoFollowCoordinator.java @@ -246,6 +246,7 @@ public void clusterStateProcessed(String source, ClusterState oldState, ClusterS }; newAutoFollowers.put(remoteCluster, autoFollower); + LOGGER.info("starting auto follower for remote cluster [{}]", remoteCluster); autoFollower.start(); } @@ -256,9 +257,10 @@ public void clusterStateProcessed(String source, ClusterState oldState, ClusterS boolean exist = autoFollowMetadata.getPatterns().values().stream() .anyMatch(pattern -> pattern.getRemoteCluster().equals(remoteCluster)); if (exist == false) { + LOGGER.info("removing auto follower for remote cluster [{}]", remoteCluster); removedRemoteClusters.add(remoteCluster); } else if (autoFollower.remoteClusterConnectionMissing) { - LOGGER.info("Retrying auto follower [{}] after remote cluster connection was missing", remoteCluster); + LOGGER.info("retrying auto follower [{}] after remote cluster connection was missing", remoteCluster); autoFollower.remoteClusterConnectionMissing = false; autoFollower.start(); } From 7ff846ffb24a51f27676a4f1ea39cde1af272318 Mon Sep 17 00:00:00 2001 From: Alpar Torok Date: Wed, 9 Jan 2019 13:43:21 +0200 Subject: [PATCH 25/46] mute failing test: org.elasticsearch.index.shard.RefreshListenersTests.testConcurrentRefresh --- .../org/elasticsearch/index/shard/RefreshListenersTests.java | 1 + 1 file changed, 1 insertion(+) diff --git a/server/src/test/java/org/elasticsearch/index/shard/RefreshListenersTests.java b/server/src/test/java/org/elasticsearch/index/shard/RefreshListenersTests.java index b52a3dbf17c02..ab383f6ef49d6 100644 --- a/server/src/test/java/org/elasticsearch/index/shard/RefreshListenersTests.java +++ b/server/src/test/java/org/elasticsearch/index/shard/RefreshListenersTests.java @@ -264,6 +264,7 @@ public void testClose() throws Exception { * 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. */ + @AwaitsFix(bugUrl = "https://github.com/elastic/elasticsearch/issues/37261") public void testConcurrentRefresh() throws Exception { AtomicBoolean run = new AtomicBoolean(true); Thread refresher = new Thread(() -> { From dc371ef59352ac7d2d5e59ad393f817bf44e2fd0 Mon Sep 17 00:00:00 2001 From: Tanguy Leroux Date: Wed, 9 Jan 2019 13:34:38 +0100 Subject: [PATCH 26/46] [Tests] Fix ReopenWhileClosingIT with correct min num shards The test intercepts TransportVerifyShardBeforeCloseAction shard requests, so it needs a minimum of 2 primary shards on 2 different nodes to correctly intercepts requests. --- .../elasticsearch/indices/state/ReopenWhileClosingIT.java | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/server/src/test/java/org/elasticsearch/indices/state/ReopenWhileClosingIT.java b/server/src/test/java/org/elasticsearch/indices/state/ReopenWhileClosingIT.java index 901c4f327af48..083c5ab1f5510 100644 --- a/server/src/test/java/org/elasticsearch/indices/state/ReopenWhileClosingIT.java +++ b/server/src/test/java/org/elasticsearch/indices/state/ReopenWhileClosingIT.java @@ -58,6 +58,11 @@ protected Collection> nodePlugins() { return singletonList(MockTransportService.TestPlugin.class); } + @Override + protected int minimumNumberOfShards() { + return 2; + } + public void testReopenDuringClose() throws Exception { final String indexName = "test"; createIndexWithDocs(indexName); From 84d520b0e59099ce7bff2714bf82773554e55f69 Mon Sep 17 00:00:00 2001 From: Jim Ferenczi Date: Wed, 9 Jan 2019 13:35:41 +0100 Subject: [PATCH 27/46] Throw an ISE rather than an hard assertion in SearchPhaseController#getTotalHits This change turns an assertion into an IllegalStateException in SearchPhaseController#getTotalHits. The goal is to help identify the cause of the failures in https://github.com/elastic/elasticsearch/issues/37179 which seems to fail only in CI. The assertion will be restored when the issue is solved (NORELEASE). --- .../action/search/SearchPhaseController.java | 8 +++++++- 1 file changed, 7 insertions(+), 1 deletion(-) diff --git a/server/src/main/java/org/elasticsearch/action/search/SearchPhaseController.java b/server/src/main/java/org/elasticsearch/action/search/SearchPhaseController.java index 418d95b2077a9..8fada8938f854 100644 --- a/server/src/main/java/org/elasticsearch/action/search/SearchPhaseController.java +++ b/server/src/main/java/org/elasticsearch/action/search/SearchPhaseController.java @@ -760,7 +760,13 @@ TotalHits getTotalHits() { if (trackTotalHitsUpTo == SearchContext.TRACK_TOTAL_HITS_DISABLED) { return null; } else if (trackTotalHitsUpTo == SearchContext.TRACK_TOTAL_HITS_ACCURATE) { - assert totalHitsRelation == Relation.EQUAL_TO; + // NORELEASE The assertion below has been replaced by a runtime exception in order to debug + // https://github.com/elastic/elasticsearch/issues/37179. + // The assertion should be restored and the exception removed when this issue is solved. + // assert totalHitsRelation == Relation.EQUAL_TO; + if (totalHitsRelation != Relation.EQUAL_TO) { + throw new IllegalStateException("Expected accurate total hits but got " + new TotalHits(totalHits, totalHitsRelation)); + } return new TotalHits(totalHits, totalHitsRelation); } else { if (totalHits < trackTotalHitsUpTo) { From c149bb8cc2544c02997358de762c231c48040704 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Christoph=20B=C3=BCscher?= Date: Wed, 9 Jan 2019 14:17:17 +0100 Subject: [PATCH 28/46] Support 'include_type_name' in RestGetIndicesAction (#37149) This change adds support for the 'include_type_name' parameter for the indices.get API. This parameter, which defaults to `false` starting in 7.0, changes the response to not include the indices type names any longer. If the parameter is set in the request, we additionally emit a deprecation warning since using the parameter should be only temporarily necessary while adapting to the new response format and we will remove it with the next major version. --- .../elasticsearch/client/IndicesClientIT.java | 23 +++---- .../IndicesClientDocumentationIT.java | 8 +-- docs/reference/indices/get-index.asciidoc | 60 ++++++++++++++++++ .../rest-api-spec/api/indices.get.json | 4 ++ .../test/indices.get/10_basic.yml | 29 +++++++++ .../admin/indices/get/GetIndexResponse.java | 44 ++++++++----- .../elasticsearch/rest/BaseRestHandler.java | 2 +- .../admin/indices/RestGetIndicesAction.java | 23 ++++++- .../indices/get/GetIndexResponseTests.java | 16 ++++- .../mapping/get/GetMappingsResponseTests.java | 23 ++++--- .../indices/RestGetIndicesActionTests.java | 61 +++++++++++++++++++ 11 files changed, 251 insertions(+), 42 deletions(-) create mode 100644 server/src/test/java/org/elasticsearch/rest/action/admin/indices/RestGetIndicesActionTests.java diff --git a/client/rest-high-level/src/test/java/org/elasticsearch/client/IndicesClientIT.java b/client/rest-high-level/src/test/java/org/elasticsearch/client/IndicesClientIT.java index a639a09b3cc53..92d7e94394594 100644 --- a/client/rest-high-level/src/test/java/org/elasticsearch/client/IndicesClientIT.java +++ b/client/rest-high-level/src/test/java/org/elasticsearch/client/IndicesClientIT.java @@ -89,6 +89,7 @@ import org.elasticsearch.common.xcontent.json.JsonXContent; import org.elasticsearch.common.xcontent.support.XContentMapValues; import org.elasticsearch.index.IndexSettings; +import org.elasticsearch.index.mapper.MapperService; import org.elasticsearch.index.query.QueryBuilder; import org.elasticsearch.index.query.QueryBuilders; import org.elasticsearch.rest.RestStatus; @@ -209,7 +210,7 @@ public void testCreateIndex() throws IOException { mappingBuilder.startObject().startObject("properties").startObject("field"); mappingBuilder.field("type", "text"); mappingBuilder.endObject().endObject().endObject(); - createIndexRequest.mapping("type_name", mappingBuilder); + createIndexRequest.mapping(MapperService.SINGLE_MAPPING_NAME, mappingBuilder); CreateIndexResponse createIndexResponse = execute(createIndexRequest, highLevelClient().indices()::create, highLevelClient().indices()::createAsync); @@ -226,7 +227,7 @@ public void testCreateIndex() throws IOException { Map term = (Map) filter.get("term"); assertEquals(2016, term.get("year")); - assertEquals("text", XContentMapValues.extractValue(indexName + ".mappings.type_name.properties.field.type", getIndexResponse)); + assertEquals("text", XContentMapValues.extractValue(indexName + ".mappings.properties.field.type", getIndexResponse)); } } @@ -340,7 +341,7 @@ public void testGetIndex() throws IOException { .put(SETTING_NUMBER_OF_SHARDS, 1) .put(SETTING_NUMBER_OF_REPLICAS, 0) .build(); - String mappings = "\"type-1\":{\"properties\":{\"field-1\":{\"type\":\"integer\"}}}"; + String mappings = "\"_doc\":{\"properties\":{\"field-1\":{\"type\":\"integer\"}}}"; createIndex(indexName, basicSettings, mappings); GetIndexRequest getIndexRequest = new GetIndexRequest() @@ -353,8 +354,8 @@ public void testGetIndex() throws IOException { assertEquals("1", getIndexResponse.getSetting(indexName, SETTING_NUMBER_OF_SHARDS)); assertEquals("0", getIndexResponse.getSetting(indexName, SETTING_NUMBER_OF_REPLICAS)); assertNotNull(getIndexResponse.getMappings().get(indexName)); - assertNotNull(getIndexResponse.getMappings().get(indexName).get("type-1")); - Object o = getIndexResponse.getMappings().get(indexName).get("type-1").getSourceAsMap().get("properties"); + assertNotNull(getIndexResponse.getMappings().get(indexName).get("_doc")); + Object o = getIndexResponse.getMappings().get(indexName).get("_doc").getSourceAsMap().get("properties"); assertThat(o, instanceOf(Map.class)); //noinspection unchecked assertThat(((Map) o).get("field-1"), instanceOf(Map.class)); @@ -370,7 +371,7 @@ public void testGetIndexWithDefaults() throws IOException { .put(SETTING_NUMBER_OF_SHARDS, 1) .put(SETTING_NUMBER_OF_REPLICAS, 0) .build(); - String mappings = "\"type-1\":{\"properties\":{\"field-1\":{\"type\":\"integer\"}}}"; + String mappings = "\"_doc\":{\"properties\":{\"field-1\":{\"type\":\"integer\"}}}"; createIndex(indexName, basicSettings, mappings); GetIndexRequest getIndexRequest = new GetIndexRequest() @@ -384,8 +385,8 @@ public void testGetIndexWithDefaults() throws IOException { assertEquals("1", getIndexResponse.getSetting(indexName, SETTING_NUMBER_OF_SHARDS)); assertEquals("0", getIndexResponse.getSetting(indexName, SETTING_NUMBER_OF_REPLICAS)); assertNotNull(getIndexResponse.getMappings().get(indexName)); - assertNotNull(getIndexResponse.getMappings().get(indexName).get("type-1")); - Object o = getIndexResponse.getMappings().get(indexName).get("type-1").getSourceAsMap().get("properties"); + assertNotNull(getIndexResponse.getMappings().get(indexName).get("_doc")); + Object o = getIndexResponse.getMappings().get(indexName).get("_doc").getSourceAsMap().get("properties"); assertThat(o, instanceOf(Map.class)); assertThat(((Map) o).get("field-1"), instanceOf(Map.class)); Map fieldMapping = (Map) ((Map) o).get("field-1"); @@ -408,7 +409,7 @@ public void testPutMapping() throws IOException { createIndex(indexName, Settings.EMPTY); PutMappingRequest putMappingRequest = new PutMappingRequest(indexName); - putMappingRequest.type("type_name"); + putMappingRequest.type("_doc"); XContentBuilder mappingBuilder = JsonXContent.contentBuilder(); mappingBuilder.startObject().startObject("properties").startObject("field"); mappingBuilder.field("type", "text"); @@ -420,7 +421,7 @@ public void testPutMapping() throws IOException { assertTrue(putMappingResponse.isAcknowledged()); Map getIndexResponse = getAsMap(indexName); - assertEquals("text", XContentMapValues.extractValue(indexName + ".mappings.type_name.properties.field.type", getIndexResponse)); + assertEquals("text", XContentMapValues.extractValue(indexName + ".mappings.properties.field.type", getIndexResponse)); } public void testGetMapping() throws IOException { @@ -440,7 +441,7 @@ public void testGetMapping() throws IOException { assertTrue(putMappingResponse.isAcknowledged()); Map getIndexResponse = getAsMap(indexName); - assertEquals("text", XContentMapValues.extractValue(indexName + ".mappings._doc.properties.field.type", getIndexResponse)); + assertEquals("text", XContentMapValues.extractValue(indexName + ".mappings.properties.field.type", getIndexResponse)); GetMappingsRequest request = new GetMappingsRequest() .indices(indexName) diff --git a/client/rest-high-level/src/test/java/org/elasticsearch/client/documentation/IndicesClientDocumentationIT.java b/client/rest-high-level/src/test/java/org/elasticsearch/client/documentation/IndicesClientDocumentationIT.java index 3ff4466649ff2..8f9d8a069fd48 100644 --- a/client/rest-high-level/src/test/java/org/elasticsearch/client/documentation/IndicesClientDocumentationIT.java +++ b/client/rest-high-level/src/test/java/org/elasticsearch/client/documentation/IndicesClientDocumentationIT.java @@ -70,15 +70,15 @@ import org.elasticsearch.action.support.IndicesOptions; import org.elasticsearch.action.support.master.AcknowledgedResponse; import org.elasticsearch.client.ESRestHighLevelClientTestCase; -import org.elasticsearch.client.indices.FreezeIndexRequest; import org.elasticsearch.client.GetAliasesResponse; import org.elasticsearch.client.RequestOptions; import org.elasticsearch.client.RestHighLevelClient; import org.elasticsearch.client.SyncedFlushResponse; +import org.elasticsearch.client.core.ShardsAcknowledgedResponse; +import org.elasticsearch.client.indices.FreezeIndexRequest; import org.elasticsearch.client.indices.GetIndexTemplatesRequest; import org.elasticsearch.client.indices.IndexTemplatesExistRequest; import org.elasticsearch.client.indices.UnfreezeIndexRequest; -import org.elasticsearch.client.core.ShardsAcknowledgedResponse; import org.elasticsearch.cluster.metadata.AliasMetaData; import org.elasticsearch.cluster.metadata.IndexTemplateMetaData; import org.elasticsearch.cluster.metadata.MappingMetaData; @@ -1249,7 +1249,7 @@ public void testGetIndex() throws Exception { Settings settings = Settings.builder().put("number_of_shards", 3).build(); String mappings = "{\"properties\":{\"field-1\":{\"type\":\"integer\"}}}"; CreateIndexResponse createIndexResponse = client.indices().create( - new CreateIndexRequest("index", settings).mapping("doc", mappings, XContentType.JSON), + new CreateIndexRequest("index", settings).mapping("_doc", mappings, XContentType.JSON), RequestOptions.DEFAULT); assertTrue(createIndexResponse.isAcknowledged()); } @@ -1272,7 +1272,7 @@ public void testGetIndex() throws Exception { // tag::get-index-response ImmutableOpenMap indexMappings = getIndexResponse.getMappings().get("index"); // <1> - Map indexTypeMappings = indexMappings.get("doc").getSourceAsMap(); // <2> + Map indexTypeMappings = indexMappings.get("_doc").getSourceAsMap(); // <2> List indexAliases = getIndexResponse.getAliases().get("index"); // <3> String numberOfShardsString = getIndexResponse.getSetting("index", "index.number_of_shards"); // <4> Settings indexSettings = getIndexResponse.getSettings().get("index"); // <5> diff --git a/docs/reference/indices/get-index.asciidoc b/docs/reference/indices/get-index.asciidoc index ffa512dca6363..8e254a576c11d 100644 --- a/docs/reference/indices/get-index.asciidoc +++ b/docs/reference/indices/get-index.asciidoc @@ -15,3 +15,63 @@ alias or wildcard expression is required. The get index API can also be applied to more than one index, or on all indices by using `_all` or `*` as index. + +[float] +=== Skipping types + +Types are scheduled to be fully removed in Elasticsearch 8.0 and will not appear +in requests or responses anymore. You can opt in for this future behaviour by +setting `include_type_name=false` in the request, which will return mappings +directly under `mappings` without keying by the type name. + +Here is an example: + +[source,js] +-------------------------------------------------- +PUT test?include_type_name=false +{ + "mappings": { + "properties": { + "foo": { + "type": "keyword" + } + } + } +} + +GET test?include_type_name=false +-------------------------------------------------- +// CONSOLE + +which returns + +[source,js] +-------------------------------------------------- +{ + "test": { + "aliases": {}, + "mappings": { + "properties": { + "foo": { + "type": "keyword" + } + } + }, + "settings": { + "index": { + "creation_date": "1547028674905", + "number_of_shards": "1", + "number_of_replicas": "1", + "uuid": "u1YpkPqLSqGIn3kNAvY8cA", + "version": { + "created": ... + }, + "provided_name": "test" + } + } + } +} +-------------------------------------------------- +// TESTRESPONSE[s/1547028674905/$body.test.settings.index.creation_date/] +// TESTRESPONSE[s/u1YpkPqLSqGIn3kNAvY8cA/$body.test.settings.index.uuid/] +// TESTRESPONSE[s/"created": \.\.\./"created": $body.test.settings.index.version.created/] diff --git a/rest-api-spec/src/main/resources/rest-api-spec/api/indices.get.json b/rest-api-spec/src/main/resources/rest-api-spec/api/indices.get.json index 6474b8acf5298..76e6ed00feb26 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/api/indices.get.json +++ b/rest-api-spec/src/main/resources/rest-api-spec/api/indices.get.json @@ -13,6 +13,10 @@ } }, "params":{ + "include_type_name": { + "type" : "boolean", + "description" : "Whether to add the type name to the response (default: false)" + }, "local":{ "type":"boolean", "description":"Return local information, do not retrieve the state from master node (default: false)" diff --git a/rest-api-spec/src/main/resources/rest-api-spec/test/indices.get/10_basic.yml b/rest-api-spec/src/main/resources/rest-api-spec/test/indices.get/10_basic.yml index 6301087f48902..ef028dcdf67e8 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/test/indices.get/10_basic.yml +++ b/rest-api-spec/src/main/resources/rest-api-spec/test/indices.get/10_basic.yml @@ -52,6 +52,35 @@ setup: - is_true: test_index.settings - is_true: test_index.mappings +--- +"Test include_type_name": + - skip: + version: " - 6.99.99" + reason: the include_type_name parameter is not backported to pre 7.0 versions yet + + - do: + indices.get: + include_type_name: true + index: test_index + + - is_true: test_index.mappings + - is_true: test_index.mappings.type_1 + + - do: + indices.get: + include_type_name: false + index: test_index + + - is_true: test_index.mappings + - is_false: test_index.mappings.type_1 + + - do: + indices.get: + index: test_index + + - is_true: test_index.mappings + - is_false: test_index.mappings.type_1 + --- "Get index infos should work for wildcards": diff --git a/server/src/main/java/org/elasticsearch/action/admin/indices/get/GetIndexResponse.java b/server/src/main/java/org/elasticsearch/action/admin/indices/get/GetIndexResponse.java index e2b72077b7f21..c7e64143f9c9a 100644 --- a/server/src/main/java/org/elasticsearch/action/admin/indices/get/GetIndexResponse.java +++ b/server/src/main/java/org/elasticsearch/action/admin/indices/get/GetIndexResponse.java @@ -20,6 +20,7 @@ package org.elasticsearch.action.admin.indices.get; import com.carrotsearch.hppc.cursors.ObjectObjectCursor; + import org.apache.lucene.util.CollectionUtil; import org.elasticsearch.Version; import org.elasticsearch.action.ActionResponse; @@ -34,6 +35,7 @@ import org.elasticsearch.common.xcontent.XContentBuilder; import org.elasticsearch.common.xcontent.XContentParser; import org.elasticsearch.common.xcontent.XContentParser.Token; +import org.elasticsearch.index.mapper.MapperService; import java.io.IOException; import java.util.ArrayList; @@ -41,9 +43,11 @@ import java.util.Collections; import java.util.Comparator; import java.util.List; +import java.util.Map; import java.util.Objects; import static org.elasticsearch.common.xcontent.XContentParserUtils.ensureExpectedToken; +import static org.elasticsearch.rest.BaseRestHandler.INCLUDE_TYPE_NAME_PARAMETER; /** * A response for a get index action. @@ -249,15 +253,32 @@ public XContentBuilder toXContent(XContentBuilder builder, Params params) throws } builder.endObject(); - builder.startObject("mappings"); ImmutableOpenMap indexMappings = mappings.get(index); - if (indexMappings != null) { + boolean includeTypeName = params.paramAsBoolean(INCLUDE_TYPE_NAME_PARAMETER, false); + if (includeTypeName) { + builder.startObject("mappings"); + if (indexMappings != null) { + for (final ObjectObjectCursor typeEntry : indexMappings) { + builder.field(typeEntry.key); + builder.map(typeEntry.value.sourceAsMap()); + } + } + builder.endObject(); + } else { + MappingMetaData mappings = null; for (final ObjectObjectCursor typeEntry : indexMappings) { - builder.field(typeEntry.key); - builder.map(typeEntry.value.sourceAsMap()); + if (typeEntry.key.equals(MapperService.DEFAULT_MAPPING) == false) { + assert mappings == null; + mappings = typeEntry.value; + } + } + if (mappings == null) { + // no mappings yet + builder.startObject("mappings").endObject(); + } else { + builder.field("mappings", mappings.sourceAsMap()); } } - builder.endObject(); builder.startObject("settings"); Settings indexSettings = settings.get(index); @@ -292,16 +313,9 @@ private static List parseAliases(XContentParser parser) throws IO private static ImmutableOpenMap parseMappings(XContentParser parser) throws IOException { ImmutableOpenMap.Builder indexMappings = ImmutableOpenMap.builder(); - // We start at START_OBJECT since parseIndexEntry ensures that - while (parser.nextToken() != Token.END_OBJECT) { - ensureExpectedToken(Token.FIELD_NAME, parser.currentToken(), parser::getTokenLocation); - parser.nextToken(); - if (parser.currentToken() == Token.START_OBJECT) { - String mappingType = parser.currentName(); - indexMappings.put(mappingType, new MappingMetaData(mappingType, parser.map())); - } else if (parser.currentToken() == Token.START_ARRAY) { - parser.skipChildren(); - } + Map map = parser.map(); + if (map.isEmpty() == false) { + indexMappings.put(MapperService.SINGLE_MAPPING_NAME, new MappingMetaData(MapperService.SINGLE_MAPPING_NAME, map)); } return indexMappings.build(); } diff --git a/server/src/main/java/org/elasticsearch/rest/BaseRestHandler.java b/server/src/main/java/org/elasticsearch/rest/BaseRestHandler.java index 4611fd69b2bf8..963c8089f342b 100644 --- a/server/src/main/java/org/elasticsearch/rest/BaseRestHandler.java +++ b/server/src/main/java/org/elasticsearch/rest/BaseRestHandler.java @@ -62,7 +62,7 @@ public abstract class BaseRestHandler extends AbstractComponent implements RestH * Parameter that controls whether certain REST apis should include type names in their requests or responses. * Note: Support for this parameter will be removed after the transition perido to typeless APIs. */ - protected static final String INCLUDE_TYPE_NAME_PARAMETER = "include_type_name"; + public static final String INCLUDE_TYPE_NAME_PARAMETER = "include_type_name"; protected BaseRestHandler(Settings settings) { // TODO drop settings from ctor diff --git a/server/src/main/java/org/elasticsearch/rest/action/admin/indices/RestGetIndicesAction.java b/server/src/main/java/org/elasticsearch/rest/action/admin/indices/RestGetIndicesAction.java index 04fae0f30f6bf..f38df9326949f 100644 --- a/server/src/main/java/org/elasticsearch/rest/action/admin/indices/RestGetIndicesAction.java +++ b/server/src/main/java/org/elasticsearch/rest/action/admin/indices/RestGetIndicesAction.java @@ -20,10 +20,12 @@ package org.elasticsearch.rest.action.admin.indices; +import org.apache.logging.log4j.LogManager; import org.elasticsearch.action.admin.indices.get.GetIndexRequest; import org.elasticsearch.action.support.IndicesOptions; import org.elasticsearch.client.node.NodeClient; import org.elasticsearch.common.Strings; +import org.elasticsearch.common.logging.DeprecationLogger; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.rest.BaseRestHandler; import org.elasticsearch.rest.RestController; @@ -31,7 +33,10 @@ import org.elasticsearch.rest.action.RestToXContentListener; import java.io.IOException; +import java.util.Collections; import java.util.Set; +import java.util.stream.Collectors; +import java.util.stream.Stream; import static org.elasticsearch.rest.RestRequest.Method.GET; import static org.elasticsearch.rest.RestRequest.Method.HEAD; @@ -41,6 +46,13 @@ */ public class RestGetIndicesAction extends BaseRestHandler { + private static final DeprecationLogger deprecationLogger = new DeprecationLogger(LogManager.getLogger(RestGetIndicesAction.class)); + static final String TYPES_DEPRECATION_MESSAGE = "[types removal] Using `include_type_name` in get indices requests is deprecated. " + + "The parameter will be removed in the next major version."; + + private static final Set allowedResponseParameters = Collections + .unmodifiableSet(Stream.concat(Collections.singleton(INCLUDE_TYPE_NAME_PARAMETER).stream(), Settings.FORMAT_PARAMS.stream()) + .collect(Collectors.toSet())); public RestGetIndicesAction( final Settings settings, @@ -58,6 +70,10 @@ public String getName() { @Override public RestChannelConsumer prepareRequest(final RestRequest request, final NodeClient client) throws IOException { String[] indices = Strings.splitStringByCommaToArray(request.param("index")); + // starting with 7.0 we don't include types by default in the response + if (request.hasParam(INCLUDE_TYPE_NAME_PARAMETER)) { + deprecationLogger.deprecatedAndMaybeLog("get_indices_with_types", TYPES_DEPRECATION_MESSAGE); + } final GetIndexRequest getIndexRequest = new GetIndexRequest(); getIndexRequest.indices(indices); getIndexRequest.indicesOptions(IndicesOptions.fromRequest(request, getIndexRequest.indicesOptions())); @@ -68,9 +84,12 @@ public RestChannelConsumer prepareRequest(final RestRequest request, final NodeC return channel -> client.admin().indices().getIndex(getIndexRequest, new RestToXContentListener<>(channel)); } + /** + * Parameters used for controlling the response and thus might not be consumed during + * preparation of the request execution in {@link BaseRestHandler#prepareRequest(RestRequest, NodeClient)}. + */ @Override protected Set responseParams() { - return Settings.FORMAT_PARAMS; + return allowedResponseParameters; } - } diff --git a/server/src/test/java/org/elasticsearch/action/admin/indices/get/GetIndexResponseTests.java b/server/src/test/java/org/elasticsearch/action/admin/indices/get/GetIndexResponseTests.java index 3991442fd5b87..af3ab33e915db 100644 --- a/server/src/test/java/org/elasticsearch/action/admin/indices/get/GetIndexResponseTests.java +++ b/server/src/test/java/org/elasticsearch/action/admin/indices/get/GetIndexResponseTests.java @@ -32,6 +32,7 @@ import org.elasticsearch.common.settings.IndexScopedSettings; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.xcontent.XContentParser; +import org.elasticsearch.common.xcontent.XContentType; import org.elasticsearch.index.RandomCreateIndexGenerator; import org.elasticsearch.test.AbstractStreamableXContentTestCase; import org.junit.Assert; @@ -72,6 +73,10 @@ protected GetIndexResponse createBlankInstance() { @Override protected GetIndexResponse createTestInstance() { + return createTestInstance(randomBoolean()); + } + + private GetIndexResponse createTestInstance(boolean randomTypeName) { String[] indices = generateRandomStringArray(5, 5, false, false); ImmutableOpenMap.Builder> mappings = ImmutableOpenMap.builder(); ImmutableOpenMap.Builder> aliases = ImmutableOpenMap.builder(); @@ -80,7 +85,9 @@ protected GetIndexResponse createTestInstance() { IndexScopedSettings indexScopedSettings = IndexScopedSettings.DEFAULT_SCOPED_SETTINGS; boolean includeDefaults = randomBoolean(); for (String index: indices) { - mappings.put(index, GetMappingsResponseTests.createMappingsForIndex()); + // rarely have no types + int typeCount = rarely() ? 0 : 1; + mappings.put(index, GetMappingsResponseTests.createMappingsForIndex(typeCount, randomTypeName)); List aliasMetaDataList = new ArrayList<>(); int aliasesNum = randomIntBetween(0, 3); @@ -103,6 +110,12 @@ protected GetIndexResponse createTestInstance() { ); } + @Override + protected GetIndexResponse createXContextTestInstance(XContentType xContentType) { + // don't use random type names for XContent roundtrip tests because we cannot parse them back anymore + return createTestInstance(false); + } + @Override protected Predicate getRandomFieldsExcludeFilter() { //we do not want to add new fields at the root (index-level), or inside the blocks @@ -190,5 +203,4 @@ public void testCanOutput622Response() throws IOException { Assert.assertEquals(TEST_6_3_0_RESPONSE_BYTES, base64OfResponse); } - } diff --git a/server/src/test/java/org/elasticsearch/action/admin/indices/mapping/get/GetMappingsResponseTests.java b/server/src/test/java/org/elasticsearch/action/admin/indices/mapping/get/GetMappingsResponseTests.java index beae91df77e3d..481d05ea8cc07 100644 --- a/server/src/test/java/org/elasticsearch/action/admin/indices/mapping/get/GetMappingsResponseTests.java +++ b/server/src/test/java/org/elasticsearch/action/admin/indices/mapping/get/GetMappingsResponseTests.java @@ -24,6 +24,7 @@ import org.elasticsearch.cluster.metadata.MappingMetaData; import org.elasticsearch.common.collect.ImmutableOpenMap; import org.elasticsearch.common.xcontent.XContentParser; +import org.elasticsearch.index.mapper.MapperService; import org.elasticsearch.test.AbstractStreamableXContentTestCase; import org.elasticsearch.test.EqualsHashCodeTestUtils; @@ -84,22 +85,30 @@ protected GetMappingsResponse mutateInstance(GetMappingsResponse instance) throw public static ImmutableOpenMap createMappingsForIndex() { // rarely have no types int typeCount = rarely() ? 0 : scaledRandomIntBetween(1, 3); + return createMappingsForIndex(typeCount, true); + } + + public static ImmutableOpenMap createMappingsForIndex(int typeCount, boolean randomTypeName) { List typeMappings = new ArrayList<>(typeCount); for (int i = 0; i < typeCount; i++) { - Map mappings = new HashMap<>(); if (rarely() == false) { // rarely have no fields + Map mappings = new HashMap<>(); mappings.put("field-" + i, randomFieldMapping()); if (randomBoolean()) { mappings.put("field2-" + i, randomFieldMapping()); } - } - try { - MappingMetaData mmd = new MappingMetaData("type-" + randomAlphaOfLength(5), mappings); - typeMappings.add(mmd); - } catch (IOException e) { - fail("shouldn't have failed " + e); + try { + String typeName = MapperService.SINGLE_MAPPING_NAME; + if (randomTypeName) { + typeName = "type-" + randomAlphaOfLength(5); + } + MappingMetaData mmd = new MappingMetaData(typeName, mappings); + typeMappings.add(mmd); + } catch (IOException e) { + fail("shouldn't have failed " + e); + } } } ImmutableOpenMap.Builder typeBuilder = ImmutableOpenMap.builder(); diff --git a/server/src/test/java/org/elasticsearch/rest/action/admin/indices/RestGetIndicesActionTests.java b/server/src/test/java/org/elasticsearch/rest/action/admin/indices/RestGetIndicesActionTests.java new file mode 100644 index 0000000000000..5f157cd298d4d --- /dev/null +++ b/server/src/test/java/org/elasticsearch/rest/action/admin/indices/RestGetIndicesActionTests.java @@ -0,0 +1,61 @@ +/* + * 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.rest.action.admin.indices; + +import org.elasticsearch.client.node.NodeClient; +import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.rest.RestController; +import org.elasticsearch.rest.RestRequest; +import org.elasticsearch.test.ESTestCase; +import org.elasticsearch.test.rest.FakeRestRequest; + +import java.io.IOException; +import java.util.HashMap; +import java.util.Map; + +import static org.elasticsearch.rest.BaseRestHandler.INCLUDE_TYPE_NAME_PARAMETER; +import static org.mockito.Mockito.mock; + +public class RestGetIndicesActionTests extends ESTestCase { + + /** + * Test that setting the "include_type_name" parameter raises a warning + */ + public void testIncludeTypeNamesWarning() throws IOException { + Map params = new HashMap<>(); + params.put(INCLUDE_TYPE_NAME_PARAMETER, randomFrom("true", "false")); + RestRequest request = new FakeRestRequest.Builder(xContentRegistry()) + .withMethod(RestRequest.Method.GET) + .withPath("/some_index") + .withParams(params) + .build(); + + RestGetIndicesAction handler = new RestGetIndicesAction(Settings.EMPTY, mock(RestController.class)); + handler.prepareRequest(request, mock(NodeClient.class)); + assertWarnings(RestGetIndicesAction.TYPES_DEPRECATION_MESSAGE); + + // the same request without the parameter should pass without warning + request = new FakeRestRequest.Builder(xContentRegistry()) + .withMethod(RestRequest.Method.GET) + .withPath("/some_index") + .build(); + handler.prepareRequest(request, mock(NodeClient.class)); + } +} From 7de4d2cb0fb448c2dd4e5d60efc8281e1d5aa92f Mon Sep 17 00:00:00 2001 From: Alpar Torok Date: Wed, 9 Jan 2019 16:43:24 +0200 Subject: [PATCH 29/46] Mute failing test ChildQuerySearchIT Tracked in #37266 --- .../java/org/elasticsearch/join/query/ChildQuerySearchIT.java | 2 ++ 1 file changed, 2 insertions(+) diff --git a/modules/parent-join/src/test/java/org/elasticsearch/join/query/ChildQuerySearchIT.java b/modules/parent-join/src/test/java/org/elasticsearch/join/query/ChildQuerySearchIT.java index 8520b63205678..d92ff102fe8f8 100644 --- a/modules/parent-join/src/test/java/org/elasticsearch/join/query/ChildQuerySearchIT.java +++ b/modules/parent-join/src/test/java/org/elasticsearch/join/query/ChildQuerySearchIT.java @@ -59,6 +59,7 @@ import java.util.Map; import java.util.Set; +import static org.apache.lucene.util.LuceneTestCase.AwaitsFix; 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; @@ -85,6 +86,7 @@ import static org.hamcrest.Matchers.equalTo; import static org.hamcrest.Matchers.is; +@AwaitsFix(bugUrl = "https://github.com/elastic/elasticsearch/issues/37266") public class ChildQuerySearchIT extends ParentChildTestCase { public void testMultiLevelChild() throws Exception { From ae086ebcc4467eb58e14ed3fcea4d02b382a37be Mon Sep 17 00:00:00 2001 From: Alpar Torok Date: Wed, 9 Jan 2019 16:55:11 +0200 Subject: [PATCH 30/46] Muting SnapshotDisruptionIT Tracked in #36779 --- .../java/org/elasticsearch/discovery/SnapshotDisruptionIT.java | 1 + 1 file changed, 1 insertion(+) diff --git a/server/src/test/java/org/elasticsearch/discovery/SnapshotDisruptionIT.java b/server/src/test/java/org/elasticsearch/discovery/SnapshotDisruptionIT.java index bc699c905c014..29fa597aec63f 100644 --- a/server/src/test/java/org/elasticsearch/discovery/SnapshotDisruptionIT.java +++ b/server/src/test/java/org/elasticsearch/discovery/SnapshotDisruptionIT.java @@ -75,6 +75,7 @@ protected Settings nodeSettings(int nodeOrdinal) { .build(); } + @AwaitsFix(bugUrl = "https://github.com/elastic/elasticsearch/issues/36779") public void testDisruptionOnSnapshotInitialization() throws Exception { final String idxName = "test"; final List allMasterEligibleNodes = internalCluster().startMasterOnlyNodes(3); From eacc63b032cb0cb055c41830e2d2eb29e61be030 Mon Sep 17 00:00:00 2001 From: Armin Braun Date: Wed, 9 Jan 2019 16:53:49 +0100 Subject: [PATCH 31/46] TESTS: Real Coordinator in SnapshotServiceTests (#37162) * TESTS: Real Coordinator in SnapshotServiceTests * Introduce real coordinator in SnapshotServiceTests to be able to test network disruptions realistically * Make adjustments to cluster applier service so that we can pass a mocked single threaded executor for tests --- .../cluster/coordination/Coordinator.java | 25 ++- .../service/ClusterApplierService.java | 16 +- .../cluster/service/ClusterService.java | 15 +- .../snapshots/SnapshotsServiceTests.java | 180 ++++++++++++------ .../MockSinglePrioritizingExecutor.java | 58 ++++++ .../DeterministicTaskQueueTests.java | 2 +- .../MockSinglePrioritizingExecutorTests.java | 59 ++++++ 7 files changed, 273 insertions(+), 82 deletions(-) create mode 100644 test/framework/src/main/java/org/elasticsearch/cluster/coordination/MockSinglePrioritizingExecutor.java create mode 100644 test/framework/src/test/java/org/elasticsearch/cluster/coordination/MockSinglePrioritizingExecutorTests.java diff --git a/server/src/main/java/org/elasticsearch/cluster/coordination/Coordinator.java b/server/src/main/java/org/elasticsearch/cluster/coordination/Coordinator.java index 5f131f83b270f..f5ef24a17ade5 100644 --- a/server/src/main/java/org/elasticsearch/cluster/coordination/Coordinator.java +++ b/server/src/main/java/org/elasticsearch/cluster/coordination/Coordinator.java @@ -56,6 +56,8 @@ import org.elasticsearch.common.unit.TimeValue; import org.elasticsearch.common.util.concurrent.EsExecutors; import org.elasticsearch.common.util.concurrent.ListenableFuture; +import org.elasticsearch.common.xcontent.XContentHelper; +import org.elasticsearch.common.xcontent.json.JsonXContent; import org.elasticsearch.discovery.Discovery; import org.elasticsearch.discovery.DiscoverySettings; import org.elasticsearch.discovery.DiscoveryStats; @@ -872,12 +874,7 @@ public void publish(ClusterChangedEvent clusterChangedEvent, ActionListener ActionListener wrapWithMutex(ActionListener listener) { return new ActionListener() { @Override diff --git a/server/src/main/java/org/elasticsearch/cluster/service/ClusterApplierService.java b/server/src/main/java/org/elasticsearch/cluster/service/ClusterApplierService.java index c331a9a137e0f..5bd441419d136 100644 --- a/server/src/main/java/org/elasticsearch/cluster/service/ClusterApplierService.java +++ b/server/src/main/java/org/elasticsearch/cluster/service/ClusterApplierService.java @@ -89,7 +89,7 @@ public class ClusterApplierService extends AbstractLifecycleComponent implements private final Collection clusterStateListeners = new CopyOnWriteArrayList<>(); private final Collection timeoutClusterStateListeners = - Collections.newSetFromMap(new ConcurrentHashMap()); + Collections.newSetFromMap(new ConcurrentHashMap<>()); private final LocalNodeMasterListeners localNodeMasterListeners; @@ -134,11 +134,15 @@ protected synchronized void doStart() { Objects.requireNonNull(nodeConnectionsService, "please set the node connection service before starting"); Objects.requireNonNull(state.get(), "please set initial state before starting"); addListener(localNodeMasterListeners); - threadPoolExecutor = EsExecutors.newSinglePrioritizing( - nodeName + "/" + CLUSTER_UPDATE_THREAD_NAME, - daemonThreadFactory(nodeName, CLUSTER_UPDATE_THREAD_NAME), - threadPool.getThreadContext(), - threadPool.scheduler()); + threadPoolExecutor = createThreadPoolExecutor(); + } + + protected PrioritizedEsThreadPoolExecutor createThreadPoolExecutor() { + return EsExecutors.newSinglePrioritizing( + nodeName + "/" + CLUSTER_UPDATE_THREAD_NAME, + daemonThreadFactory(nodeName, CLUSTER_UPDATE_THREAD_NAME), + threadPool.getThreadContext(), + threadPool.scheduler()); } class UpdateTask extends SourcePrioritizedRunnable implements Function { diff --git a/server/src/main/java/org/elasticsearch/cluster/service/ClusterService.java b/server/src/main/java/org/elasticsearch/cluster/service/ClusterService.java index fa4f6acc65d23..12d45c4fb88f4 100644 --- a/server/src/main/java/org/elasticsearch/cluster/service/ClusterService.java +++ b/server/src/main/java/org/elasticsearch/cluster/service/ClusterService.java @@ -71,8 +71,13 @@ public class ClusterService extends AbstractLifecycleComponent { private final String nodeName; - public ClusterService(Settings settings, ClusterSettings clusterSettings, ThreadPool threadPool, - MasterService masterService) { + public ClusterService(Settings settings, ClusterSettings clusterSettings, ThreadPool threadPool) { + this(settings, clusterSettings, new MasterService(Node.NODE_NAME_SETTING.get(settings), settings, threadPool), + new ClusterApplierService(Node.NODE_NAME_SETTING.get(settings), settings, clusterSettings, threadPool)); + } + + public ClusterService(Settings settings, ClusterSettings clusterSettings, MasterService masterService, + ClusterApplierService clusterApplierService) { super(settings); this.settings = settings; this.nodeName = Node.NODE_NAME_SETTING.get(settings); @@ -84,11 +89,7 @@ public ClusterService(Settings settings, ClusterSettings clusterSettings, Thread this::setSlowTaskLoggingThreshold); // Add a no-op update consumer so changes are logged this.clusterSettings.addAffixUpdateConsumer(USER_DEFINED_META_DATA, (first, second) -> {}, (first, second) -> {}); - this.clusterApplierService = new ClusterApplierService(nodeName, settings, clusterSettings, threadPool); - } - - public ClusterService(Settings settings, ClusterSettings clusterSettings, ThreadPool threadPool) { - this(settings, clusterSettings, threadPool, new MasterService(Node.NODE_NAME_SETTING.get(settings), settings, threadPool)); + this.clusterApplierService = clusterApplierService; } private void setSlowTaskLoggingThreshold(TimeValue slowTaskLoggingThreshold) { diff --git a/server/src/test/java/org/elasticsearch/snapshots/SnapshotsServiceTests.java b/server/src/test/java/org/elasticsearch/snapshots/SnapshotsServiceTests.java index e213979996806..291d6bf63a95b 100644 --- a/server/src/test/java/org/elasticsearch/snapshots/SnapshotsServiceTests.java +++ b/server/src/test/java/org/elasticsearch/snapshots/SnapshotsServiceTests.java @@ -24,6 +24,7 @@ import org.elasticsearch.Version; import org.elasticsearch.action.Action; import org.elasticsearch.action.ActionListener; +import org.elasticsearch.action.admin.cluster.bootstrap.BootstrapConfiguration; import org.elasticsearch.action.admin.cluster.repositories.put.PutRepositoryAction; import org.elasticsearch.action.admin.cluster.repositories.put.TransportPutRepositoryAction; import org.elasticsearch.action.admin.cluster.snapshots.create.CreateSnapshotAction; @@ -36,7 +37,7 @@ import org.elasticsearch.action.support.ActiveShardCount; import org.elasticsearch.action.support.TransportAction; import org.elasticsearch.client.node.NodeClient; -import org.elasticsearch.cluster.ClusterChangedEvent; +import org.elasticsearch.cluster.ClusterModule; import org.elasticsearch.cluster.ClusterName; import org.elasticsearch.cluster.ClusterState; import org.elasticsearch.cluster.ESAllocationTestCase; @@ -44,8 +45,13 @@ import org.elasticsearch.cluster.SnapshotsInProgress; import org.elasticsearch.cluster.action.index.NodeMappingRefreshAction; import org.elasticsearch.cluster.action.shard.ShardStateAction; +import org.elasticsearch.cluster.coordination.ClusterBootstrapService; +import org.elasticsearch.cluster.coordination.CoordinationState; +import org.elasticsearch.cluster.coordination.Coordinator; import org.elasticsearch.cluster.coordination.CoordinatorTests; import org.elasticsearch.cluster.coordination.DeterministicTaskQueue; +import org.elasticsearch.cluster.coordination.InMemoryPersistedState; +import org.elasticsearch.cluster.coordination.MockSinglePrioritizingExecutor; import org.elasticsearch.cluster.metadata.AliasValidator; import org.elasticsearch.cluster.metadata.IndexMetaData; import org.elasticsearch.cluster.metadata.IndexNameExpressionResolver; @@ -55,15 +61,16 @@ import org.elasticsearch.cluster.node.DiscoveryNodes; import org.elasticsearch.cluster.routing.RoutingService; import org.elasticsearch.cluster.routing.allocation.AllocationService; +import org.elasticsearch.cluster.service.ClusterApplierService; import org.elasticsearch.cluster.service.ClusterService; import org.elasticsearch.cluster.service.MasterService; import org.elasticsearch.common.io.stream.NamedWriteableRegistry; import org.elasticsearch.common.settings.ClusterSettings; import org.elasticsearch.common.settings.IndexScopedSettings; import org.elasticsearch.common.settings.Settings; -import org.elasticsearch.common.unit.TimeValue; import org.elasticsearch.common.util.BigArrays; import org.elasticsearch.common.util.PageCacheRecycler; +import org.elasticsearch.common.util.concurrent.PrioritizedEsThreadPoolExecutor; import org.elasticsearch.common.xcontent.NamedXContentRegistry; import org.elasticsearch.env.Environment; import org.elasticsearch.env.NodeEnvironment; @@ -102,21 +109,23 @@ import java.util.Collections; import java.util.HashMap; import java.util.LinkedHashMap; +import java.util.List; import java.util.Map; import java.util.Optional; +import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; -import java.util.concurrent.atomic.AtomicInteger; -import java.util.concurrent.atomic.AtomicReference; +import java.util.function.Predicate; +import java.util.function.Supplier; import java.util.stream.Collectors; import static java.util.Collections.emptyMap; import static java.util.Collections.emptySet; import static org.elasticsearch.env.Environment.PATH_HOME_SETTING; import static org.elasticsearch.node.Node.NODE_NAME_SETTING; +import static org.elasticsearch.transport.TransportService.HANDSHAKE_ACTION_NAME; import static org.elasticsearch.transport.TransportService.NOOP_TRANSPORT_INTERCEPTOR; import static org.hamcrest.Matchers.containsInAnyOrder; import static org.hamcrest.Matchers.hasSize; -import static org.hamcrest.Matchers.lessThan; import static org.mockito.Mockito.mock; public class SnapshotsServiceTests extends ESTestCase { @@ -132,8 +141,6 @@ public void createServices() { tempDir = createTempDir(); deterministicTaskQueue = new DeterministicTaskQueue(Settings.builder().put(NODE_NAME_SETTING.getKey(), "shared").build(), random()); - // TODO: Random number of master nodes and simulate master failover states - testClusterNodes = new TestClusterNodes(1, randomIntBetween(2, 10)); } @After @@ -144,41 +151,40 @@ public void stopServices() { n.clusterService.close(); n.indicesClusterStateService.close(); n.nodeEnv.close(); + n.coordinator.close(); } ); } public void testSuccessfulSnapshot() { + setupTestCluster(randomFrom(1, 3, 5), randomIntBetween(2, 10)); + String repoName = "repo"; String snapshotName = "snapshot"; final String index = "test"; final int shards = randomIntBetween(1, 10); - ClusterState initialClusterState = - new ClusterState.Builder(ClusterName.DEFAULT).nodes(testClusterNodes.randomDiscoveryNodes()).build(); - testClusterNodes.nodes.values().forEach(testClusterNode -> testClusterNode.start(initialClusterState)); - - TestClusterNode masterNode = testClusterNodes.currentMaster(initialClusterState); - - final AtomicBoolean createdSnapshot = new AtomicBoolean(false); + TestClusterNode masterNode = + testClusterNodes.currentMaster(testClusterNodes.nodes.values().iterator().next().clusterService.state()); + final AtomicBoolean createdSnapshot = new AtomicBoolean(); masterNode.client.admin().cluster().preparePutRepository(repoName) .setType(FsRepository.TYPE).setSettings(Settings.builder().put("location", randomAlphaOfLength(10))) .execute( - assertingListener( + assertNoFailureListener( () -> masterNode.client.admin().indices().create( new CreateIndexRequest(index).waitForActiveShards(ActiveShardCount.ALL).settings( Settings.builder() .put(IndexMetaData.INDEX_NUMBER_OF_SHARDS_SETTING.getKey(), shards) .put(IndexMetaData.INDEX_NUMBER_OF_REPLICAS_SETTING.getKey(), 0)), - assertingListener( + assertNoFailureListener( () -> masterNode.client.admin().cluster().prepareCreateSnapshot(repoName, snapshotName) - .execute(assertingListener(() -> createdSnapshot.set(true))))))); + .execute(assertNoFailureListener(() -> createdSnapshot.set(true))))))); deterministicTaskQueue.runAllRunnableTasks(); assertTrue(createdSnapshot.get()); - SnapshotsInProgress finalSnapshotsInProgress = masterNode.currentState.get().custom(SnapshotsInProgress.TYPE); + SnapshotsInProgress finalSnapshotsInProgress = masterNode.clusterService.state().custom(SnapshotsInProgress.TYPE); assertFalse(finalSnapshotsInProgress.entries().stream().anyMatch(entry -> entry.state().completed() == false)); final Repository repository = masterNode.repositoriesService.repository(repoName); Collection snapshotIds = repository.getRepositoryData().getSnapshotIds(); @@ -191,7 +197,52 @@ public void testSuccessfulSnapshot() { assertEquals(0, snapshotInfo.failedShards()); } - private static ActionListener assertingListener(Runnable r) { + private void startCluster() { + final ClusterState initialClusterState = + new ClusterState.Builder(ClusterName.DEFAULT).nodes(testClusterNodes.randomDiscoveryNodes()).build(); + testClusterNodes.nodes.values().forEach(testClusterNode -> testClusterNode.start(initialClusterState)); + + deterministicTaskQueue.advanceTime(); + deterministicTaskQueue.runAllRunnableTasks(); + + final BootstrapConfiguration bootstrapConfiguration = new BootstrapConfiguration( + testClusterNodes.nodes.values().stream().filter(n -> n.node.isMasterNode()) + .map(node -> new BootstrapConfiguration.NodeDescription(node.node)) + .distinct() + .collect(Collectors.toList())); + testClusterNodes.nodes.values().stream().filter(n -> n.node.isMasterNode()).forEach( + testClusterNode -> testClusterNode.coordinator.setInitialConfiguration(bootstrapConfiguration) + ); + + runUntil( + () -> { + List masterNodeIds = testClusterNodes.nodes.values().stream() + .map(node -> node.clusterService.state().nodes().getMasterNodeId()) + .distinct().collect(Collectors.toList()); + return masterNodeIds.size() == 1 && masterNodeIds.contains(null) == false; + }, + TimeUnit.SECONDS.toMillis(30L) + ); + } + + private void runUntil(Supplier fulfilled, long timeout) { + final long start = deterministicTaskQueue.getCurrentTimeMillis(); + while (timeout > deterministicTaskQueue.getCurrentTimeMillis() - start) { + deterministicTaskQueue.runAllRunnableTasks(); + if (fulfilled.get()) { + return; + } + deterministicTaskQueue.advanceTime(); + } + fail("Condition wasn't fulfilled."); + } + + private void setupTestCluster(int masterNodes, int dataNodes) { + testClusterNodes = new TestClusterNodes(masterNodes, dataNodes); + startCluster(); + } + + private static ActionListener assertNoFailureListener(Runnable r) { return new ActionListener() { @Override public void onResponse(final T t) { @@ -213,6 +264,8 @@ private Environment createEnvironment(String nodeName) { .put(NODE_NAME_SETTING.getKey(), nodeName) .put(PATH_HOME_SETTING.getKey(), tempDir.resolve(nodeName).toAbsolutePath()) .put(Environment.PATH_REPO_SETTING.getKey(), tempDir.resolve("repo").toAbsolutePath()) + .putList(ClusterBootstrapService.INITIAL_MASTER_NODES_SETTING.getKey(), + ClusterBootstrapService.INITIAL_MASTER_NODES_SETTING.get(Settings.EMPTY)) .build()); } @@ -235,10 +288,6 @@ private static ClusterState stateForNode(ClusterState state, DiscoveryNode node) return ClusterState.builder(state).nodes(DiscoveryNodes.builder(state.nodes()).localNodeId(node.getId())).build(); } - private static ClusterChangedEvent changeEventForNode(ClusterChangedEvent event, DiscoveryNode node) { - return new ClusterChangedEvent(event.source(), stateForNode(event.state(), node), stateForNode(event.previousState(), node)); - } - private final class TestClusterNodes { // LinkedHashMap so we have deterministic ordering when iterating over the map in tests @@ -254,8 +303,8 @@ private final class TestClusterNodes { } }); } - for (int i = masterNodes; i < dataNodes + masterNodes; ++i) { - nodes.computeIfAbsent("node" + i, nodeName -> { + for (int i = 0; i < dataNodes; ++i) { + nodes.computeIfAbsent("data-node" + i, nodeName -> { try { return SnapshotsServiceTests.this.newDataNode(nodeName); } catch (IOException e) { @@ -273,10 +322,7 @@ private final class TestClusterNodes { public DiscoveryNodes randomDiscoveryNodes() { DiscoveryNodes.Builder builder = DiscoveryNodes.builder(); nodes.values().forEach(node -> builder.add(node.node)); - String masterId = randomFrom(nodes.values().stream().map(node -> node.node).filter(DiscoveryNode::isMasterNode) - .map(DiscoveryNode::getId) - .collect(Collectors.toList())); - return builder.localNodeId(masterId).masterNodeId(masterId).build(); + return builder.build(); } /** @@ -296,6 +342,8 @@ private final class TestClusterNode { private final Logger logger = LogManager.getLogger(TestClusterNode.class); + private final NamedWriteableRegistry namedWriteableRegistry = new NamedWriteableRegistry(ClusterModule.getNamedWriteables()); + private final TransportService transportService; private final ClusterService clusterService; @@ -316,23 +364,30 @@ private final class TestClusterNode { private final AllocationService allocationService; - private final AtomicReference currentState = new AtomicReference<>(); - private final NodeClient client; private final NodeEnvironment nodeEnv; private final DisruptableMockTransport mockTransport; + private final ThreadPool threadPool; + + private Coordinator coordinator; + TestClusterNode(DiscoveryNode node) throws IOException { this.node = node; final Environment environment = createEnvironment(node.getName()); masterService = new FakeThreadPoolMasterService(node.getName(), "test", deterministicTaskQueue::scheduleNow); final Settings settings = environment.settings(); - allocationService = ESAllocationTestCase.createAllocationService(settings); final ClusterSettings clusterSettings = new ClusterSettings(settings, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS); - final ThreadPool threadPool = deterministicTaskQueue.getThreadPool(); - clusterService = new ClusterService(settings, clusterSettings, threadPool, masterService); + threadPool = deterministicTaskQueue.getThreadPool(); + clusterService = new ClusterService(settings, clusterSettings, masterService, + new ClusterApplierService(node.getName(), settings, clusterSettings, threadPool) { + @Override + protected PrioritizedEsThreadPoolExecutor createThreadPoolExecutor() { + return new MockSinglePrioritizingExecutor(node.getName(), deterministicTaskQueue); + } + }); mockTransport = new DisruptableMockTransport(logger) { @Override protected DiscoveryNode getLocalNode() { @@ -346,12 +401,24 @@ protected ConnectionStatus getConnectionStatus(DiscoveryNode sender, DiscoveryNo @Override protected Optional getDisruptedCapturingTransport(DiscoveryNode node, String action) { - return Optional.ofNullable(testClusterNodes.nodes.get(node.getName()).mockTransport); + final Predicate matchesDestination; + if (action.equals(HANDSHAKE_ACTION_NAME)) { + matchesDestination = n -> n.transportService.getLocalNode().getAddress().equals(node.getAddress()); + } else { + matchesDestination = n -> n.transportService.getLocalNode().equals(node); + } + return testClusterNodes.nodes.values().stream().filter(matchesDestination).findAny().map(cn -> cn.mockTransport); } @Override protected void handle(DiscoveryNode sender, DiscoveryNode destination, String action, Runnable doDelivery) { - deterministicTaskQueue.scheduleNow(CoordinatorTests.onNode(destination, doDelivery)); + // handshake needs to run inline as the caller blockingly waits on the result + final Runnable runnable = CoordinatorTests.onNode(destination, doDelivery); + if (action.equals(HANDSHAKE_ACTION_NAME)) { + runnable.run(); + } else { + deterministicTaskQueue.scheduleNow(runnable); + } } }; transportService = mockTransport.createTransportService( @@ -382,6 +449,7 @@ protected void assertSnapshotOrGenericThread() { final NamedXContentRegistry namedXContentRegistry = new NamedXContentRegistry(Collections.emptyList()); final ScriptService scriptService = new ScriptService(settings, emptyMap(), emptyMap()); client = new NodeClient(settings, threadPool); + allocationService = ESAllocationTestCase.createAllocationService(settings); final IndexScopedSettings indexScopedSettings = new IndexScopedSettings(settings, IndexScopedSettings.BUILT_IN_INDEX_SETTINGS); indicesService = new IndicesService( @@ -393,7 +461,7 @@ protected void assertSnapshotOrGenericThread() { emptyMap(), emptyMap(), emptyMap(), emptyMap()), indexNameExpressionResolver, new MapperRegistry(emptyMap(), emptyMap(), MapperPlugin.NOOP_FIELD_FILTER), - new NamedWriteableRegistry(Collections.emptyList()), + namedWriteableRegistry, threadPool, indexScopedSettings, new NoneCircuitBreakerService(), @@ -464,35 +532,23 @@ public void start(ClusterState initialState) { transportService.acceptIncomingRequests(); snapshotsService.start(); snapshotShardsService.start(); - // Mock publisher that invokes other cluster change listeners directly - masterService.setClusterStatePublisher((clusterChangedEvent, publishListener, ackListener) -> { - final AtomicInteger applyCounter = new AtomicInteger(testClusterNodes.nodes.size()); - testClusterNodes.nodes.values().forEach( - n -> - deterministicTaskQueue.scheduleNow(() -> { - assertThat(n.currentState.get().version(), lessThan(clusterChangedEvent.state().version())); - ClusterChangedEvent adjustedEvent = changeEventForNode(clusterChangedEvent, n.node); - n.repositoriesService.applyClusterState(adjustedEvent); - n.snapshotsService.applyClusterState(adjustedEvent); - n.snapshotShardsService.clusterChanged(adjustedEvent); - n.indicesClusterStateService.applyClusterState(adjustedEvent); - n.currentState.set(adjustedEvent.state()); - if (applyCounter.decrementAndGet() == 0) { - publishListener.onResponse(null); - ackListener.onCommit(TimeValue.timeValueMillis(deterministicTaskQueue.getLatestDeferredExecutionTime())); - } - })); - }); - masterService.setClusterStateSupplier(currentState::get); + final CoordinationState.PersistedState persistedState = + new InMemoryPersistedState(0L, stateForNode(initialState, node)); + coordinator = new Coordinator(node.getName(), clusterService.getSettings(), + clusterService.getClusterSettings(), transportService, namedWriteableRegistry, + allocationService, masterService, () -> persistedState, + hostsResolver -> testClusterNodes.nodes.values().stream().filter(n -> n.node.isMasterNode()) + .map(n -> n.node.getAddress()).collect(Collectors.toList()), + clusterService.getClusterApplierService(), random()); + masterService.setClusterStatePublisher(coordinator); + coordinator.start(); masterService.start(); - ClusterState stateForNode = stateForNode(initialState, node); - currentState.set(stateForNode); - clusterService.getClusterApplierService().setInitialState(stateForNode); - clusterService.getClusterApplierService().setNodeConnectionsService(new NodeConnectionsService(clusterService.getSettings(), - deterministicTaskQueue.getThreadPool(), transportService)); + clusterService.getClusterApplierService().setNodeConnectionsService( + new NodeConnectionsService(clusterService.getSettings(), threadPool, transportService)); clusterService.getClusterApplierService().start(); indicesService.start(); indicesClusterStateService.start(); + coordinator.startInitialJoin(); } } } diff --git a/test/framework/src/main/java/org/elasticsearch/cluster/coordination/MockSinglePrioritizingExecutor.java b/test/framework/src/main/java/org/elasticsearch/cluster/coordination/MockSinglePrioritizingExecutor.java new file mode 100644 index 0000000000000..cc21fef5f5559 --- /dev/null +++ b/test/framework/src/main/java/org/elasticsearch/cluster/coordination/MockSinglePrioritizingExecutor.java @@ -0,0 +1,58 @@ +/* + * 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.coordination; + +import org.elasticsearch.common.util.concurrent.EsExecutors; +import org.elasticsearch.common.util.concurrent.PrioritizedEsThreadPoolExecutor; + +import java.util.concurrent.TimeUnit; + +/** + * Mock single threaded {@link PrioritizedEsThreadPoolExecutor} based on {@link DeterministicTaskQueue}, + * simulating the behaviour of an executor returned by {@link EsExecutors#newSinglePrioritizing}. + */ +public class MockSinglePrioritizingExecutor extends PrioritizedEsThreadPoolExecutor { + + public MockSinglePrioritizingExecutor(String name, DeterministicTaskQueue deterministicTaskQueue) { + super(name, 0, 1, 0L, TimeUnit.MILLISECONDS, + r -> new Thread() { + @Override + public void start() { + deterministicTaskQueue.scheduleNow(() -> { + try { + r.run(); + } catch (KillWorkerError kwe) { + // hacks everywhere + } + }); + } + }, + deterministicTaskQueue.getThreadPool().getThreadContext(), deterministicTaskQueue.getThreadPool().scheduler()); + } + + @Override + protected void afterExecute(Runnable r, Throwable t) { + super.afterExecute(r, t); + // kill worker so that next one will be scheduled + throw new KillWorkerError(); + } + + private static final class KillWorkerError extends Error { + } +} diff --git a/test/framework/src/test/java/org/elasticsearch/cluster/coordination/DeterministicTaskQueueTests.java b/test/framework/src/test/java/org/elasticsearch/cluster/coordination/DeterministicTaskQueueTests.java index 5238cfe8ecdbc..c13c840377f5e 100644 --- a/test/framework/src/test/java/org/elasticsearch/cluster/coordination/DeterministicTaskQueueTests.java +++ b/test/framework/src/test/java/org/elasticsearch/cluster/coordination/DeterministicTaskQueueTests.java @@ -422,7 +422,7 @@ public void testThreadPoolSchedulesPeriodicFutureTasks() { assertThat(strings, contains("periodic-0", "periodic-1", "periodic-2")); } - private static DeterministicTaskQueue newTaskQueue() { + static DeterministicTaskQueue newTaskQueue() { return newTaskQueue(random()); } diff --git a/test/framework/src/test/java/org/elasticsearch/cluster/coordination/MockSinglePrioritizingExecutorTests.java b/test/framework/src/test/java/org/elasticsearch/cluster/coordination/MockSinglePrioritizingExecutorTests.java new file mode 100644 index 0000000000000..427f0d32c1270 --- /dev/null +++ b/test/framework/src/test/java/org/elasticsearch/cluster/coordination/MockSinglePrioritizingExecutorTests.java @@ -0,0 +1,59 @@ +/* + * 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.coordination; + +import org.elasticsearch.common.Priority; +import org.elasticsearch.common.util.concurrent.PrioritizedEsThreadPoolExecutor; +import org.elasticsearch.common.util.concurrent.PrioritizedRunnable; +import org.elasticsearch.test.ESTestCase; + +import java.util.concurrent.atomic.AtomicBoolean; + +public class MockSinglePrioritizingExecutorTests extends ESTestCase { + + public void testPrioritizedEsThreadPoolExecutor() { + final DeterministicTaskQueue taskQueue = DeterministicTaskQueueTests.newTaskQueue(); + final PrioritizedEsThreadPoolExecutor executor = new MockSinglePrioritizingExecutor("test", taskQueue); + final AtomicBoolean called1 = new AtomicBoolean(); + final AtomicBoolean called2 = new AtomicBoolean(); + executor.execute(new PrioritizedRunnable(Priority.NORMAL) { + @Override + public void run() { + assertTrue(called1.compareAndSet(false, true)); // check that this is only called once + } + + }); + executor.execute(new PrioritizedRunnable(Priority.HIGH) { + @Override + public void run() { + assertTrue(called2.compareAndSet(false, true)); // check that this is only called once + } + }); + assertFalse(called1.get()); + assertFalse(called2.get()); + taskQueue.runRandomTask(); + assertFalse(called1.get()); + assertTrue(called2.get()); + taskQueue.runRandomTask(); + assertTrue(called1.get()); + assertTrue(called2.get()); + taskQueue.runRandomTask(); + assertFalse(taskQueue.hasRunnableTasks()); + } +} From 85a603ee6192055332e7b002b2a170a38bb48774 Mon Sep 17 00:00:00 2001 From: Evangelos Chatzikalymnios Date: Wed, 9 Jan 2019 17:01:39 +0100 Subject: [PATCH 32/46] Use List instead of priority queue for stable sorting in bucket sort aggregator (#36748) Update BucketSortPipelineAggregator to use a List and Collections.sort() for sorting instead of a priority queue. This preserves the order for equal values. Closes #36322. --- .../BucketSortPipelineAggregator.java | 39 +++++----------- .../aggregations/pipeline/BucketSortIT.java | 44 +++++++++++++++++++ 2 files changed, 56 insertions(+), 27 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/pipeline/BucketSortPipelineAggregator.java b/server/src/main/java/org/elasticsearch/search/aggregations/pipeline/BucketSortPipelineAggregator.java index e98fdec992722..b639a384c7691 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/pipeline/BucketSortPipelineAggregator.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/pipeline/BucketSortPipelineAggregator.java @@ -19,7 +19,6 @@ package org.elasticsearch.search.aggregations.pipeline; -import org.apache.lucene.util.PriorityQueue; import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.io.stream.StreamOutput; import org.elasticsearch.search.aggregations.InternalAggregation; @@ -34,7 +33,6 @@ import java.util.ArrayList; import java.util.Collections; import java.util.HashMap; -import java.util.LinkedList; import java.util.List; import java.util.Map; @@ -95,22 +93,22 @@ public InternalAggregation reduce(InternalAggregation aggregation, ReduceContext return originalAgg.create(new ArrayList<>(buckets.subList(from, Math.min(from + currentSize, bucketsCount)))); } - int queueSize = Math.min(from + currentSize, bucketsCount); - PriorityQueue ordered = new TopNPriorityQueue(queueSize); + List ordered = new ArrayList<>(); for (InternalMultiBucketAggregation.InternalBucket bucket : buckets) { ComparableBucket comparableBucket = new ComparableBucket(originalAgg, bucket); if (comparableBucket.skip() == false) { - ordered.insertWithOverflow(new ComparableBucket(originalAgg, bucket)); + ordered.add(comparableBucket); } } - int resultSize = Math.max(ordered.size() - from, 0); + Collections.sort(ordered); - // Popping from the priority queue returns the least element. The elements we want to skip due to offset would pop last. - // Thus, we just have to pop as many elements as we expect in results and store them in reverse order. - LinkedList newBuckets = new LinkedList<>(); - for (int i = 0; i < resultSize; ++i) { - newBuckets.addFirst(ordered.pop().internalBucket); + // We just have to get as many elements as we expect in results and store them in the same order starting from + // the specified offset and taking currentSize into consideration. + int limit = Math.min(from + currentSize, ordered.size()); + List newBuckets = new ArrayList<>(); + for (int i = from; i < limit; ++i) { + newBuckets.add(ordered.get(i).internalBucket); } return originalAgg.create(newBuckets); } @@ -160,11 +158,11 @@ public int compareTo(ComparableBucket that) { if (thisValue == null && thatValue == null) { continue; } else if (thisValue == null) { - return -1; - } else if (thatValue == null) { return 1; + } else if (thatValue == null) { + return -1; } else { - compareResult = sort.order() == SortOrder.DESC ? thisValue.compareTo(thatValue) : -thisValue.compareTo(thatValue); + compareResult = sort.order() == SortOrder.DESC ? -thisValue.compareTo(thatValue) : thisValue.compareTo(thatValue); } if (compareResult != 0) { break; @@ -173,17 +171,4 @@ public int compareTo(ComparableBucket that) { return compareResult; } } - - - private static class TopNPriorityQueue extends PriorityQueue { - - private TopNPriorityQueue(int n) { - super(n); - } - - @Override - protected boolean lessThan(ComparableBucket a, ComparableBucket b) { - return a.compareTo(b) < 0; - } - } } diff --git a/server/src/test/java/org/elasticsearch/search/aggregations/pipeline/BucketSortIT.java b/server/src/test/java/org/elasticsearch/search/aggregations/pipeline/BucketSortIT.java index 8618d5a34ba5c..7cb4371354c3b 100644 --- a/server/src/test/java/org/elasticsearch/search/aggregations/pipeline/BucketSortIT.java +++ b/server/src/test/java/org/elasticsearch/search/aggregations/pipeline/BucketSortIT.java @@ -43,6 +43,7 @@ import static org.elasticsearch.search.aggregations.AggregationBuilders.avg; import static org.elasticsearch.search.aggregations.AggregationBuilders.dateHistogram; import static org.elasticsearch.search.aggregations.AggregationBuilders.histogram; +import static org.elasticsearch.search.aggregations.AggregationBuilders.max; import static org.elasticsearch.search.aggregations.AggregationBuilders.terms; import static org.elasticsearch.search.aggregations.PipelineAggregatorBuilders.bucketSort; import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertSearchResponse; @@ -191,6 +192,26 @@ public void testSortTermsOnKey() { } } + public void testSortTermsOnKeyWithSize() { + SearchResponse response = client().prepareSearch(INDEX) + .setSize(0) + .addAggregation(terms("foos").field(TERM_FIELD) + .subAggregation(bucketSort("bucketSort", Arrays.asList(new FieldSortBuilder("_key"))).size(3))) + .get(); + + assertSearchResponse(response); + + Terms terms = response.getAggregations().get("foos"); + assertThat(terms, notNullValue()); + List termsBuckets = terms.getBuckets(); + assertEquals(3, termsBuckets.size()); + String previousKey = (String) termsBuckets.get(0).getKey(); + for (Terms.Bucket termBucket : termsBuckets) { + assertThat(previousKey, lessThanOrEqualTo((String) termBucket.getKey())); + previousKey = (String) termBucket.getKey(); + } + } + public void testSortTermsOnSubAggregation() { SearchResponse response = client().prepareSearch(INDEX) .setSize(0) @@ -231,6 +252,29 @@ public void testSortTermsOnSubAggregation() { } } + public void testSortTermsOnSubAggregationPreservesOrderOnEquals() { + SearchResponse response = client().prepareSearch(INDEX) + .setSize(0) + .addAggregation(terms("foos").field(TERM_FIELD) + .subAggregation(bucketSort("keyBucketSort", Arrays.asList(new FieldSortBuilder("_key")))) + .subAggregation(max("max").field("missingValue").missing(1)) + .subAggregation(bucketSort("maxBucketSort", Arrays.asList(new FieldSortBuilder("max"))))) + .get(); + + assertSearchResponse(response); + + Terms terms = response.getAggregations().get("foos"); + assertThat(terms, notNullValue()); + List termsBuckets = terms.getBuckets(); + + // Since all max values are equal, we expect the order of keyBucketSort to have been preserved + String previousKey = (String) termsBuckets.get(0).getKey(); + for (Terms.Bucket termBucket : termsBuckets) { + assertThat(previousKey, lessThanOrEqualTo((String) termBucket.getKey())); + previousKey = (String) termBucket.getKey(); + } + } + public void testSortTermsOnCountWithSecondarySort() { SearchResponse response = client().prepareSearch(INDEX) .setSize(0) From 4efbda1bc0d13ada5f0706fd6a5897356acc4506 Mon Sep 17 00:00:00 2001 From: Albert Zaharovits Date: Wed, 9 Jan 2019 18:14:22 +0200 Subject: [PATCH 33/46] [DOCS] Rolling upgrade with old internal indices (#37184) Upgrading the Elastic Stack perfectly documents the process to upgrade ES from 5 to 6 when internal indices are present. However, the rolling upgrade docs do not mention anything about internal indices. This adds a warning in the rolling upgrade procedure, highlighting that internal indices should be upgraded before the rolling upgrade procedure can be started. --- docs/reference/upgrade/rolling_upgrade.asciidoc | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/docs/reference/upgrade/rolling_upgrade.asciidoc b/docs/reference/upgrade/rolling_upgrade.asciidoc index 86a21627a8901..dff3895ac4c1d 100644 --- a/docs/reference/upgrade/rolling_upgrade.asciidoc +++ b/docs/reference/upgrade/rolling_upgrade.asciidoc @@ -18,6 +18,12 @@ you can do a rolling upgrade you must encrypt the internode-communication with SSL/TLS, which requires a full cluster restart. For more information about this requirement and the associated bootstrap check, see <>. +WARNING: The format used for the internal indices used by Kibana and {xpack} +has changed in 6.x. When upgrading from 5.6 to 6.x, these internal indices have +to be {stack-ref}/upgrading-elastic-stack.html#upgrade-internal-indices[upgraded] +before the rolling upgrade procedure can start. Otherwise the upgraded node will +refuse to join the cluster. + To perform a rolling upgrade: . *Disable shard allocation*. From d68d6aa00cb64f74485b69e081d7432fa887b358 Mon Sep 17 00:00:00 2001 From: Alpar Torok Date: Wed, 9 Jan 2019 18:26:08 +0200 Subject: [PATCH 34/46] Compatability mode for forbidden APIs for Java 12 (#37199) --- .../gradle/precommit/PrecommitTasks.groovy | 12 ++++++++++-- 1 file changed, 10 insertions(+), 2 deletions(-) diff --git a/buildSrc/src/main/groovy/org/elasticsearch/gradle/precommit/PrecommitTasks.groovy b/buildSrc/src/main/groovy/org/elasticsearch/gradle/precommit/PrecommitTasks.groovy index 6fc65d930b0ab..28c86a28f713c 100644 --- a/buildSrc/src/main/groovy/org/elasticsearch/gradle/precommit/PrecommitTasks.groovy +++ b/buildSrc/src/main/groovy/org/elasticsearch/gradle/precommit/PrecommitTasks.groovy @@ -133,8 +133,16 @@ class PrecommitTasks { project.tasks.withType(CheckForbiddenApis) { dependsOn(buildResources) targetCompatibility = project.runtimeJavaVersion >= JavaVersion.VERSION_1_9 ? - project.runtimeJavaVersion.getMajorVersion() : - project.runtimeJavaVersion + project.runtimeJavaVersion.getMajorVersion() : project.runtimeJavaVersion + if (project.runtimeJavaVersion > JavaVersion.VERSION_11) { + doLast { + project.logger.info( + "Forbidden APIs does not support java version past 11. Will use the signatures from 11 for ", + project.runtimeJavaVersion + ) + } + targetCompatibility = JavaVersion.VERSION_11.getMajorVersion() + } bundledSignatures = [ "jdk-unsafe", "jdk-deprecated", "jdk-non-portable", "jdk-system-out" ] From 6a5f3f05f405095ef97c6e40eca4cbebcc4bb82f Mon Sep 17 00:00:00 2001 From: Alpar Torok Date: Wed, 9 Jan 2019 19:27:01 +0200 Subject: [PATCH 35/46] Fix build on Fips testing convetions need to be disabled if the test task is for fips. --- x-pack/plugin/security/cli/build.gradle | 2 ++ 1 file changed, 2 insertions(+) diff --git a/x-pack/plugin/security/cli/build.gradle b/x-pack/plugin/security/cli/build.gradle index 8515b538bd562..1c684809a3203 100644 --- a/x-pack/plugin/security/cli/build.gradle +++ b/x-pack/plugin/security/cli/build.gradle @@ -24,6 +24,7 @@ dependencyLicenses { if (project.inFipsJvm) { unitTest.enabled = false + testingConventions.enabled = false // Forbiden APIs non-portable checks fail because bouncy castle classes being used from the FIPS JDK since those are // not part of the Java specification - all of this is as designed, so we have to relax this check for FIPS. tasks.withType(CheckForbiddenApis) { @@ -32,4 +33,5 @@ if (project.inFipsJvm) { // FIPS JVM includes many classes from bouncycastle which count as jar hell for the third party audit, // rather than provide a long list of exclusions, disable the check on FIPS. thirdPartyAudit.enabled = false + } From 13b8bad2b86c92510a05bd0be1e35c510769d1c2 Mon Sep 17 00:00:00 2001 From: Morris Schreibman Date: Wed, 9 Jan 2019 19:12:31 +0200 Subject: [PATCH 36/46] [DOCS] Clarify client settings (#31469) --- docs/plugins/repository-s3.asciidoc | 39 ++++++++++++++++------------- 1 file changed, 21 insertions(+), 18 deletions(-) diff --git a/docs/plugins/repository-s3.asciidoc b/docs/plugins/repository-s3.asciidoc index ddbe0b16cc6ad..b0b87dda792fe 100644 --- a/docs/plugins/repository-s3.asciidoc +++ b/docs/plugins/repository-s3.asciidoc @@ -34,10 +34,10 @@ PUT _snapshot/my_s3_repository [[repository-s3-client]] ==== Client Settings -The client used to connect to S3 has a number of settings available. Client setting names are of -the form `s3.client.CLIENT_NAME.SETTING_NAME`. The default client name, which is looked up by -an `s3` repository, is called `default`. It can be modified using the -<> `client`. For example: +The client that you use to connect to S3 has a number of settings available. The +settings have the form `s3.client.CLIENT_NAME.SETTING_NAME`. The default client +name that is looked up by an `s3` repository is `default`. It can be modified +using the <> `client`. For example: [source,js] ---- @@ -53,11 +53,13 @@ PUT _snapshot/my_s3_repository // CONSOLE // TEST[skip:we don't have s3 setup while testing this] -Most client settings are specified inside `elasticsearch.yml`, but some are -sensitive and must be stored in the {ref}/secure-settings.html[elasticsearch keystore]. +Most client settings can be added to the `elasticsearch.yml` configuration file +with the exception of the secure settings, which you add to the {es} keystore. +For more information about creating and updating the {es} keystore, see +{ref}/secure-settings.html[Secure settings]. -For example, before you start the node, run these commands to add AWS access -key settings to the keystore: +For example, before you start the node, run these commands to add AWS access key +settings to the keystore: [source,sh] ---- @@ -76,16 +78,17 @@ NOTE: In progress snapshot/restore tasks will not be preempted by a *reload* of the client's secure settings. The task will complete using the client as it was built when the operation started. -The following is the list of all the available client settings. -Those that must be stored in the keystore are marked as `Secure` and are *reloadable*. +The following list contains the available client settings. Those that must be +stored in the keystore are marked as "secure" and are *reloadable*; the other +settings belong in the `elasticsearch.yml` file. -`access_key`:: +`access_key` ({ref}/secure-settings.html[Secure]):: - An s3 access key. The `secret_key` setting must also be specified. (Secure) + An s3 access key. The `secret_key` setting must also be specified. -`secret_key`:: +`secret_key` ({ref}/secure-settings.html[Secure]):: - An s3 secret key. The `access_key` setting must also be specified. (Secure) + An s3 secret key. The `access_key` setting must also be specified. `session_token`:: An s3 session token. The `access_key` and `secret_key` settings must also @@ -110,13 +113,13 @@ Those that must be stored in the keystore are marked as `Secure` and are *reload The port of a proxy to connect to s3 through. -`proxy.username`:: +`proxy.username` ({ref}/secure-settings.html[Secure]):: - The username to connect to the `proxy.host` with. (Secure) + The username to connect to the `proxy.host` with. -`proxy.password`:: +`proxy.password` ({ref}/secure-settings.html[Secure]):: - The password to connect to the `proxy.host` with. (Secure) + The password to connect to the `proxy.host` with. `read_timeout`:: From cfa58a51afd1cd41d8aebb024d6f8987373b21d1 Mon Sep 17 00:00:00 2001 From: Tim Brooks Date: Wed, 9 Jan 2019 11:46:24 -0700 Subject: [PATCH 37/46] Add TLS/SSL channel close timeouts (#37246) Closing a channel using TLS/SSL requires reading and writing a CLOSE_NOTIFY message (for pre-1.3 TLS versions). Many implementations do not actually send the CLOSE_NOTIFY message, which means we are depending on the TCP close from the other side to ensure channels are closed. In case there is an issue with this, we need a timeout. This commit adds a timeout to the channel close process for TLS secured channels. As part of this change, we need a timer service. We could use the generic Elasticsearch timeout threadpool. However, it would be nice to have a local to the nio event loop timer service dedicated to network needs. In the future this service could support read timeouts, connect timeouts, request timeouts, etc. This commit adds a basic priority queue backed service. Since our timeout volume (channel closes) is very low, this should be fine. However, this can be updated to something more efficient in the future if needed (timer wheel). Everything being local to the event loop thread makes the logic simple as no locking or synchronization is necessary. --- .../org/elasticsearch/nio/EventHandler.java | 4 +- .../org/elasticsearch/nio/NioSelector.java | 41 ++++++- .../nio/SocketChannelContext.java | 3 + .../org/elasticsearch/nio/TaskScheduler.java | 92 ++++++++++++++++ .../elasticsearch/nio/EventHandlerTests.java | 2 +- .../elasticsearch/nio/NioSelectorTests.java | 41 ++++++- .../elasticsearch/nio/TaskSchedulerTests.java | 104 ++++++++++++++++++ .../transport/nio/SSLChannelContext.java | 15 +++ .../transport/nio/SSLChannelContextTests.java | 44 ++++++++ 9 files changed, 335 insertions(+), 11 deletions(-) create mode 100644 libs/nio/src/main/java/org/elasticsearch/nio/TaskScheduler.java create mode 100644 libs/nio/src/test/java/org/elasticsearch/nio/TaskSchedulerTests.java diff --git a/libs/nio/src/main/java/org/elasticsearch/nio/EventHandler.java b/libs/nio/src/main/java/org/elasticsearch/nio/EventHandler.java index 87a2489fdbc27..7eebfe24665a2 100644 --- a/libs/nio/src/main/java/org/elasticsearch/nio/EventHandler.java +++ b/libs/nio/src/main/java/org/elasticsearch/nio/EventHandler.java @@ -150,11 +150,11 @@ protected void writeException(SocketChannelContext context, Exception exception) } /** - * This method is called when a listener attached to a channel operation throws an exception. + * This method is called when a task or listener attached to a channel operation throws an exception. * * @param exception that occurred */ - protected void listenerException(Exception exception) { + protected void taskException(Exception exception) { exceptionHandler.accept(exception); } diff --git a/libs/nio/src/main/java/org/elasticsearch/nio/NioSelector.java b/libs/nio/src/main/java/org/elasticsearch/nio/NioSelector.java index 6820b6a07188f..cacd06bde5fa3 100644 --- a/libs/nio/src/main/java/org/elasticsearch/nio/NioSelector.java +++ b/libs/nio/src/main/java/org/elasticsearch/nio/NioSelector.java @@ -33,6 +33,7 @@ import java.util.concurrent.ConcurrentLinkedQueue; import java.util.concurrent.CountDownLatch; import java.util.concurrent.Future; +import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicReference; import java.util.concurrent.locks.ReentrantLock; @@ -54,6 +55,7 @@ public class NioSelector implements Closeable { private final Selector selector; private final ByteBuffer ioBuffer; + private final TaskScheduler taskScheduler = new TaskScheduler(); private final ReentrantLock runLock = new ReentrantLock(); private final CountDownLatch exitedLoop = new CountDownLatch(1); private final AtomicBoolean isClosed = new AtomicBoolean(false); @@ -81,6 +83,10 @@ public ByteBuffer getIoBuffer() { return ioBuffer; } + public TaskScheduler getTaskScheduler() { + return taskScheduler; + } + public Selector rawSelector() { return selector; } @@ -145,8 +151,16 @@ void singleLoop() { try { closePendingChannels(); preSelect(); - - int ready = selector.select(300); + long nanosUntilNextTask = taskScheduler.nanosUntilNextTask(System.nanoTime()); + int ready; + if (nanosUntilNextTask == 0) { + ready = selector.selectNow(); + } else { + long millisUntilNextTask = TimeUnit.NANOSECONDS.toMillis(nanosUntilNextTask); + // Only select until the next task needs to be run. Do not select with a value of 0 because + // that blocks without a timeout. + ready = selector.select(Math.min(300, Math.max(millisUntilNextTask, 1))); + } if (ready > 0) { Set selectionKeys = selector.selectedKeys(); Iterator keyIterator = selectionKeys.iterator(); @@ -164,6 +178,8 @@ void singleLoop() { } } } + + handleScheduledTasks(System.nanoTime()); } catch (ClosedSelectorException e) { if (isOpen()) { throw e; @@ -245,6 +261,17 @@ void preSelect() { handleQueuedWrites(); } + private void handleScheduledTasks(long nanoTime) { + Runnable task; + while ((task = taskScheduler.pollTask(nanoTime)) != null) { + try { + task.run(); + } catch (Exception e) { + eventHandler.taskException(e); + } + } + } + /** * Queues a write operation to be handled by the event loop. This can be called by any thread and is the * api available for non-selector threads to schedule writes. @@ -267,8 +294,10 @@ public void queueChannelClose(NioChannel channel) { ChannelContext context = channel.getContext(); assert context.getSelector() == this : "Must schedule a channel for closure with its selector"; channelsToClose.offer(context); - ensureSelectorOpenForEnqueuing(channelsToClose, context); - wakeup(); + if (isOnCurrentThread() == false) { + ensureSelectorOpenForEnqueuing(channelsToClose, context); + wakeup(); + } } /** @@ -324,7 +353,7 @@ public void executeListener(BiConsumer listener, V value) { try { listener.accept(value, null); } catch (Exception e) { - eventHandler.listenerException(e); + eventHandler.taskException(e); } } @@ -340,7 +369,7 @@ public void executeFailedListener(BiConsumer listener, Excepti try { listener.accept(null, exception); } catch (Exception e) { - eventHandler.listenerException(e); + eventHandler.taskException(e); } } diff --git a/libs/nio/src/main/java/org/elasticsearch/nio/SocketChannelContext.java b/libs/nio/src/main/java/org/elasticsearch/nio/SocketChannelContext.java index 864fe793fdf73..661c55cc7280a 100644 --- a/libs/nio/src/main/java/org/elasticsearch/nio/SocketChannelContext.java +++ b/libs/nio/src/main/java/org/elasticsearch/nio/SocketChannelContext.java @@ -234,6 +234,9 @@ protected boolean closeNow() { return closeNow; } + protected void setCloseNow() { + closeNow = true; + } // When you read or write to a nio socket in java, the heap memory passed down must be copied to/from // direct memory. The JVM internally does some buffering of the direct memory, however we can save space diff --git a/libs/nio/src/main/java/org/elasticsearch/nio/TaskScheduler.java b/libs/nio/src/main/java/org/elasticsearch/nio/TaskScheduler.java new file mode 100644 index 0000000000000..e197230147c8b --- /dev/null +++ b/libs/nio/src/main/java/org/elasticsearch/nio/TaskScheduler.java @@ -0,0 +1,92 @@ +/* + * 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.nio; + +import java.util.Comparator; +import java.util.PriorityQueue; + +/** + * A basic priority queue backed timer service. The service is thread local and should only be used by a + * single nio selector event loop thread. + */ +public class TaskScheduler { + + private final PriorityQueue tasks = new PriorityQueue<>(Comparator.comparingLong(DelayedTask::getDeadline)); + + /** + * Schedule a task at the defined relative nanotime. When {@link #pollTask(long)} is called with a + * relative nanotime after the scheduled time, the task will be returned. This method returns a + * {@link Runnable} that can be run to cancel the scheduled task. + * + * @param task to schedule + * @param relativeNanos defining when to execute the task + * @return runnable that will cancel the task + */ + public Runnable scheduleAtRelativeTime(Runnable task, long relativeNanos) { + DelayedTask delayedTask = new DelayedTask(relativeNanos, task); + tasks.offer(delayedTask); + return delayedTask; + } + + Runnable pollTask(long relativeNanos) { + DelayedTask task; + while ((task = tasks.peek()) != null) { + if (relativeNanos - task.deadline >= 0) { + tasks.remove(); + if (task.cancelled == false) { + return task.runnable; + } + } else { + return null; + } + } + return null; + } + + long nanosUntilNextTask(long relativeNanos) { + DelayedTask nextTask = tasks.peek(); + if (nextTask == null) { + return Long.MAX_VALUE; + } else { + return Math.max(nextTask.deadline - relativeNanos, 0); + } + } + + private static class DelayedTask implements Runnable { + + private final long deadline; + private final Runnable runnable; + private boolean cancelled = false; + + private DelayedTask(long deadline, Runnable runnable) { + this.deadline = deadline; + this.runnable = runnable; + } + + private long getDeadline() { + return deadline; + } + + @Override + public void run() { + cancelled = true; + } + } +} diff --git a/libs/nio/src/test/java/org/elasticsearch/nio/EventHandlerTests.java b/libs/nio/src/test/java/org/elasticsearch/nio/EventHandlerTests.java index 6e1e34ec1f572..f3ffab1baef67 100644 --- a/libs/nio/src/test/java/org/elasticsearch/nio/EventHandlerTests.java +++ b/libs/nio/src/test/java/org/elasticsearch/nio/EventHandlerTests.java @@ -245,7 +245,7 @@ public void testPostHandlingWillRemoveWriteIfNecessary() throws IOException { public void testListenerExceptionCallsGenericExceptionHandler() throws IOException { RuntimeException listenerException = new RuntimeException(); - handler.listenerException(listenerException); + handler.taskException(listenerException); verify(genericExceptionHandler).accept(listenerException); } diff --git a/libs/nio/src/test/java/org/elasticsearch/nio/NioSelectorTests.java b/libs/nio/src/test/java/org/elasticsearch/nio/NioSelectorTests.java index bd5f1c1eb346f..8cde769cca3a2 100644 --- a/libs/nio/src/test/java/org/elasticsearch/nio/NioSelectorTests.java +++ b/libs/nio/src/test/java/org/elasticsearch/nio/NioSelectorTests.java @@ -19,8 +19,10 @@ package org.elasticsearch.nio; +import org.elasticsearch.common.unit.TimeValue; import org.elasticsearch.test.ESTestCase; import org.junit.Before; +import org.mockito.ArgumentCaptor; import java.io.IOException; import java.nio.ByteBuffer; @@ -31,6 +33,8 @@ import java.nio.channels.Selector; import java.util.Collections; import java.util.HashSet; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicBoolean; import java.util.function.BiConsumer; import static org.mockito.Matchers.any; @@ -98,6 +102,39 @@ public void testQueueChannelForClosed() throws IOException { verify(eventHandler).handleClose(context); } + public void testNioDelayedTasksAreExecuted() throws IOException { + AtomicBoolean isRun = new AtomicBoolean(false); + long nanoTime = System.nanoTime() - 1; + selector.getTaskScheduler().scheduleAtRelativeTime(() -> isRun.set(true), nanoTime); + + assertFalse(isRun.get()); + selector.singleLoop(); + verify(rawSelector).selectNow(); + assertTrue(isRun.get()); + } + + public void testDefaultSelectorTimeoutIsUsedIfNoTaskSooner() throws IOException { + long delay = new TimeValue(15, TimeUnit.MINUTES).nanos(); + selector.getTaskScheduler().scheduleAtRelativeTime(() -> {}, System.nanoTime() + delay); + + selector.singleLoop(); + verify(rawSelector).select(300); + } + + public void testSelectorTimeoutWillBeReducedIfTaskSooner() throws Exception { + // As this is a timing based test, we must assertBusy in the very small chance that the loop is + // delayed for 50 milliseconds (causing a selectNow()) + assertBusy(() -> { + ArgumentCaptor captor = ArgumentCaptor.forClass(Long.class); + long delay = new TimeValue(50, TimeUnit.MILLISECONDS).nanos(); + selector.getTaskScheduler().scheduleAtRelativeTime(() -> {}, System.nanoTime() + delay); + selector.singleLoop(); + verify(rawSelector).select(captor.capture()); + assertTrue(captor.getValue() > 0); + assertTrue(captor.getValue() < 300); + }); + } + public void testSelectorClosedExceptionIsNotCaughtWhileRunning() throws IOException { boolean closedSelectorExceptionCaught = false; when(rawSelector.select(anyInt())).thenThrow(new ClosedSelectorException()); @@ -425,7 +462,7 @@ public void testExecuteListenerWillHandleException() throws Exception { selector.executeListener(listener, null); - verify(eventHandler).listenerException(exception); + verify(eventHandler).taskException(exception); } public void testExecuteFailedListenerWillHandleException() throws Exception { @@ -435,6 +472,6 @@ public void testExecuteFailedListenerWillHandleException() throws Exception { selector.executeFailedListener(listener, ioException); - verify(eventHandler).listenerException(exception); + verify(eventHandler).taskException(exception); } } diff --git a/libs/nio/src/test/java/org/elasticsearch/nio/TaskSchedulerTests.java b/libs/nio/src/test/java/org/elasticsearch/nio/TaskSchedulerTests.java new file mode 100644 index 0000000000000..4f5c074826b25 --- /dev/null +++ b/libs/nio/src/test/java/org/elasticsearch/nio/TaskSchedulerTests.java @@ -0,0 +1,104 @@ +/* + * 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.nio; + +import org.elasticsearch.test.ESTestCase; + +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.locks.LockSupport; + +public class TaskSchedulerTests extends ESTestCase { + + private TaskScheduler scheduler = new TaskScheduler(); + + public void testScheduleTask() { + AtomicBoolean complete = new AtomicBoolean(false); + + long executeTime = System.nanoTime() + TimeUnit.MILLISECONDS.toNanos(10); + scheduler.scheduleAtRelativeTime(() -> complete.set(true), executeTime); + + while (true) { + long nanoTime = System.nanoTime(); + Runnable runnable = scheduler.pollTask(nanoTime); + if (nanoTime - executeTime >= 0) { + runnable.run(); + assertTrue(complete.get()); + break; + } else { + assertNull(runnable); + LockSupport.parkNanos(TimeUnit.MILLISECONDS.toNanos(1)); + } + } + } + + public void testPollScheduleTaskAtExactTime() { + long executeTime = System.nanoTime() + TimeUnit.MILLISECONDS.toNanos(10); + scheduler.scheduleAtRelativeTime(() -> {}, executeTime); + + assertNull(scheduler.pollTask(executeTime - 1)); + assertNotNull(scheduler.pollTask(executeTime)); + } + + public void testTaskOrdering() { + AtomicBoolean first = new AtomicBoolean(false); + AtomicBoolean second = new AtomicBoolean(false); + AtomicBoolean third = new AtomicBoolean(false); + long executeTime = System.nanoTime() + TimeUnit.MILLISECONDS.toNanos(10); + scheduler.scheduleAtRelativeTime(() -> third.set(true), executeTime + 2); + scheduler.scheduleAtRelativeTime(() -> first.set(true), executeTime); + scheduler.scheduleAtRelativeTime(() -> second.set(true), executeTime + 1); + + scheduler.pollTask(executeTime + 10).run(); + assertTrue(first.get()); + assertFalse(second.get()); + assertFalse(third.get()); + scheduler.pollTask(executeTime + 10).run(); + assertTrue(first.get()); + assertTrue(second.get()); + assertFalse(third.get()); + scheduler.pollTask(executeTime + 10).run(); + assertTrue(first.get()); + assertTrue(second.get()); + assertTrue(third.get()); + } + + public void testTaskCancel() { + AtomicBoolean first = new AtomicBoolean(false); + AtomicBoolean second = new AtomicBoolean(false); + long executeTime = System.nanoTime() + TimeUnit.MILLISECONDS.toNanos(10); + Runnable cancellable = scheduler.scheduleAtRelativeTime(() -> first.set(true), executeTime); + scheduler.scheduleAtRelativeTime(() -> second.set(true), executeTime + 1); + + cancellable.run(); + scheduler.pollTask(executeTime + 10).run(); + assertFalse(first.get()); + assertTrue(second.get()); + assertNull(scheduler.pollTask(executeTime + 10)); + } + + public void testNanosUntilNextTask() { + long nanoTime = System.nanoTime(); + long executeTime = nanoTime + TimeUnit.MILLISECONDS.toNanos(10); + scheduler.scheduleAtRelativeTime(() -> {}, executeTime); + assertEquals(TimeUnit.MILLISECONDS.toNanos(10), scheduler.nanosUntilNextTask(nanoTime)); + assertEquals(TimeUnit.MILLISECONDS.toNanos(5), scheduler.nanosUntilNextTask(nanoTime + TimeUnit.MILLISECONDS.toNanos(5))); + } +} diff --git a/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/transport/nio/SSLChannelContext.java b/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/transport/nio/SSLChannelContext.java index c83bd16ca95e1..b5d5db2166c1f 100644 --- a/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/transport/nio/SSLChannelContext.java +++ b/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/transport/nio/SSLChannelContext.java @@ -5,6 +5,7 @@ */ package org.elasticsearch.xpack.security.transport.nio; +import org.elasticsearch.common.unit.TimeValue; import org.elasticsearch.core.internal.io.IOUtils; import org.elasticsearch.nio.FlushOperation; import org.elasticsearch.nio.InboundChannelBuffer; @@ -16,6 +17,7 @@ import javax.net.ssl.SSLEngine; import java.io.IOException; +import java.util.concurrent.TimeUnit; import java.util.function.BiConsumer; import java.util.function.Consumer; import java.util.function.Predicate; @@ -28,7 +30,11 @@ */ public final class SSLChannelContext extends SocketChannelContext { + private static final long CLOSE_TIMEOUT_NANOS = new TimeValue(10, TimeUnit.SECONDS).nanos(); + private static final Runnable DEFAULT_TIMEOUT_CANCELLER = () -> {}; + private final SSLDriver sslDriver; + private Runnable closeTimeoutCanceller = DEFAULT_TIMEOUT_CANCELLER; SSLChannelContext(NioSocketChannel channel, NioSelector selector, Consumer exceptionHandler, SSLDriver sslDriver, ReadWriteHandler readWriteHandler, InboundChannelBuffer channelBuffer) { @@ -53,6 +59,8 @@ public void queueWriteOperation(WriteOperation writeOperation) { getSelector().assertOnSelectorThread(); if (writeOperation instanceof CloseNotifyOperation) { sslDriver.initiateClose(); + long relativeNanos = CLOSE_TIMEOUT_NANOS + System.nanoTime(); + closeTimeoutCanceller = getSelector().getTaskScheduler().scheduleAtRelativeTime(this::channelCloseTimeout, relativeNanos); } else { super.queueWriteOperation(writeOperation); } @@ -161,6 +169,7 @@ public void closeChannel() { public void closeFromSelector() throws IOException { getSelector().assertOnSelectorThread(); if (channel.isOpen()) { + closeTimeoutCanceller.run(); IOUtils.close(super::closeFromSelector, sslDriver::close); } } @@ -169,6 +178,12 @@ public SSLEngine getSSLEngine() { return sslDriver.getSSLEngine(); } + private void channelCloseTimeout() { + closeTimeoutCanceller = DEFAULT_TIMEOUT_CANCELLER; + setCloseNow(); + getSelector().queueChannelClose(channel); + } + private static class CloseNotifyOperation implements WriteOperation { private static final BiConsumer LISTENER = (v, t) -> {}; diff --git a/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/transport/nio/SSLChannelContextTests.java b/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/transport/nio/SSLChannelContextTests.java index 4fdfb196d034e..0870124022850 100644 --- a/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/transport/nio/SSLChannelContextTests.java +++ b/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/transport/nio/SSLChannelContextTests.java @@ -12,6 +12,7 @@ import org.elasticsearch.nio.InboundChannelBuffer; import org.elasticsearch.nio.NioSelector; import org.elasticsearch.nio.NioSocketChannel; +import org.elasticsearch.nio.TaskScheduler; import org.elasticsearch.nio.WriteOperation; import org.elasticsearch.test.ESTestCase; import org.junit.Before; @@ -26,9 +27,11 @@ import java.util.function.Consumer; import static org.mockito.Matchers.any; +import static org.mockito.Matchers.anyLong; import static org.mockito.Matchers.same; import static org.mockito.Mockito.doAnswer; import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.never; import static org.mockito.Mockito.times; import static org.mockito.Mockito.verify; import static org.mockito.Mockito.when; @@ -41,6 +44,7 @@ public class SSLChannelContextTests extends ESTestCase { private SSLChannelContext context; private InboundChannelBuffer channelBuffer; private NioSelector selector; + private TaskScheduler nioTimer; private BiConsumer listener; private Consumer exceptionHandler; private SSLDriver sslDriver; @@ -56,6 +60,7 @@ public void init() { messageLength = randomInt(96) + 20; selector = mock(NioSelector.class); + nioTimer = mock(TaskScheduler.class); listener = mock(BiConsumer.class); channel = mock(NioSocketChannel.class); rawChannel = mock(SocketChannel.class); @@ -66,6 +71,7 @@ public void init() { context = new SSLChannelContext(channel, selector, exceptionHandler, sslDriver, readWriteHandler, channelBuffer); when(selector.isOnCurrentThread()).thenReturn(true); + when(selector.getTaskScheduler()).thenReturn(nioTimer); when(sslDriver.getNetworkReadBuffer()).thenReturn(readBuffer); when(sslDriver.getNetworkWriteBuffer()).thenReturn(writeBuffer); ByteBuffer buffer = ByteBuffer.allocate(1 << 14); @@ -334,6 +340,44 @@ public void testReadyToCloseIfDriverIndicateClosed() { assertTrue(context.selectorShouldClose()); } + public void testCloseTimeout() { + context.closeChannel(); + + ArgumentCaptor captor = ArgumentCaptor.forClass(WriteOperation.class); + verify(selector).writeToChannel(captor.capture()); + + ArgumentCaptor taskCaptor = ArgumentCaptor.forClass(Runnable.class); + Runnable cancellable = mock(Runnable.class); + when(nioTimer.scheduleAtRelativeTime(taskCaptor.capture(), anyLong())).thenReturn(cancellable); + context.queueWriteOperation(captor.getValue()); + verify(nioTimer).scheduleAtRelativeTime(taskCaptor.capture(), anyLong()); + assertFalse(context.selectorShouldClose()); + taskCaptor.getValue().run(); + assertTrue(context.selectorShouldClose()); + verify(selector).queueChannelClose(channel); + verify(cancellable, never()).run(); + } + + @SuppressWarnings("unchecked") + public void testCloseTimeoutIsCancelledOnClose() throws IOException { + try (SocketChannel realChannel = SocketChannel.open()) { + when(channel.getRawChannel()).thenReturn(realChannel); + TestReadWriteHandler readWriteHandler = new TestReadWriteHandler(readConsumer); + context = new SSLChannelContext(channel, selector, exceptionHandler, sslDriver, readWriteHandler, channelBuffer); + context.closeChannel(); + ArgumentCaptor captor = ArgumentCaptor.forClass(WriteOperation.class); + verify(selector).writeToChannel(captor.capture()); + ArgumentCaptor taskCaptor = ArgumentCaptor.forClass(Runnable.class); + Runnable cancellable = mock(Runnable.class); + when(nioTimer.scheduleAtRelativeTime(taskCaptor.capture(), anyLong())).thenReturn(cancellable); + context.queueWriteOperation(captor.getValue()); + + when(channel.isOpen()).thenReturn(true); + context.closeFromSelector(); + verify(cancellable).run(); + } + } + public void testInitiateCloseFromDifferentThreadSchedulesCloseNotify() { when(selector.isOnCurrentThread()).thenReturn(false, true); context.closeChannel(); From 95eef77ad4eacb90d20ffc140a1c030eb5d354b6 Mon Sep 17 00:00:00 2001 From: Jack Conradson Date: Wed, 9 Jan 2019 10:55:52 -0800 Subject: [PATCH 38/46] [Style] Fix line length violations for threadpool, indexing, and script packages (#37205) --- .../src/main/resources/checkstyle_suppressions.xml | 13 +++++-------- .../org/elasticsearch/threadpool/ThreadPool.java | 12 +++++++----- .../org/elasticsearch/indexing/IndexActionIT.java | 13 ++++++++----- .../elasticsearch/script/ScriptServiceTests.java | 9 ++++++--- 4 files changed, 26 insertions(+), 21 deletions(-) diff --git a/buildSrc/src/main/resources/checkstyle_suppressions.xml b/buildSrc/src/main/resources/checkstyle_suppressions.xml index 923e9e882b824..4d5872639480b 100644 --- a/buildSrc/src/main/resources/checkstyle_suppressions.xml +++ b/buildSrc/src/main/resources/checkstyle_suppressions.xml @@ -21,6 +21,9 @@ configuration of classes that aren't in packages. --> + + + - - - + + diff --git a/server/src/main/java/org/elasticsearch/threadpool/ThreadPool.java b/server/src/main/java/org/elasticsearch/threadpool/ThreadPool.java index bae801492e14f..e4eaf20725b63 100644 --- a/server/src/main/java/org/elasticsearch/threadpool/ThreadPool.java +++ b/server/src/main/java/org/elasticsearch/threadpool/ThreadPool.java @@ -189,9 +189,11 @@ public ThreadPool(final Settings settings, final ExecutorBuilder... customBui 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.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))); + 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"); @@ -335,9 +337,9 @@ public ExecutorService executor(String name) { * it to this method. * * @param delay delay before the task executes - * @param executor the name of the thread pool on which to execute this task. SAME means "execute on the scheduler thread" which changes the - * meaning of the ScheduledFuture returned by this method. In that case the ScheduledFuture will complete only when the command - * completes. + * @param executor the name of the thread pool on which to execute this task. SAME means "execute on the scheduler thread" which changes + * the meaning of the ScheduledFuture returned by this method. In that case the ScheduledFuture will complete only when the + * command completes. * @param command the command to run * @return a ScheduledFuture who's get will return when the task is has been added to its target thread pool and throw an exception if * the task is canceled before it was added to its target thread pool. Once the task has been added to its target thread pool diff --git a/server/src/test/java/org/elasticsearch/indexing/IndexActionIT.java b/server/src/test/java/org/elasticsearch/indexing/IndexActionIT.java index 0bf8fa698f44e..36488addb3737 100644 --- a/server/src/test/java/org/elasticsearch/indexing/IndexActionIT.java +++ b/server/src/test/java/org/elasticsearch/indexing/IndexActionIT.java @@ -70,8 +70,9 @@ public void testAutoGenerateIdNoDuplicates() throws Exception { logger.debug("running search with all types"); SearchResponse response = client().prepareSearch("test").get(); if (response.getHits().getTotalHits().value != numOfDocs) { - final String message = "Count is " + response.getHits().getTotalHits().value + " but " + numOfDocs + " was expected. " - + ElasticsearchAssertions.formatShardStatus(response); + final String message = + "Count is " + response.getHits().getTotalHits().value + " but " + numOfDocs + " was expected. " + + ElasticsearchAssertions.formatShardStatus(response); logger.error("{}. search response: \n{}", message, response); fail(message); } @@ -85,8 +86,9 @@ public void testAutoGenerateIdNoDuplicates() throws Exception { logger.debug("running search with a specific type"); SearchResponse response = client().prepareSearch("test").setTypes("type").get(); if (response.getHits().getTotalHits().value != numOfDocs) { - final String message = "Count is " + response.getHits().getTotalHits().value + " but " + numOfDocs + " was expected. " - + ElasticsearchAssertions.formatShardStatus(response); + final String message = + "Count is " + response.getHits().getTotalHits().value + " but " + numOfDocs + " was expected. " + + ElasticsearchAssertions.formatShardStatus(response); logger.error("{}. search response: \n{}", message, response); fail(message); } @@ -183,7 +185,8 @@ public void testCreateFlagWithBulk() { createIndex("test"); ensureGreen(); - BulkResponse bulkResponse = client().prepareBulk().add(client().prepareIndex("test", "type", "1").setSource("field1", "value1_1")).execute().actionGet(); + BulkResponse bulkResponse = client().prepareBulk().add( + client().prepareIndex("test", "type", "1").setSource("field1", "value1_1")).execute().actionGet(); assertThat(bulkResponse.hasFailures(), equalTo(false)); assertThat(bulkResponse.getItems().length, equalTo(1)); IndexResponse indexResponse = bulkResponse.getItems()[0].getResponse(); diff --git a/server/src/test/java/org/elasticsearch/script/ScriptServiceTests.java b/server/src/test/java/org/elasticsearch/script/ScriptServiceTests.java index 271007f99787d..f49bb70dc7ffc 100644 --- a/server/src/test/java/org/elasticsearch/script/ScriptServiceTests.java +++ b/server/src/test/java/org/elasticsearch/script/ScriptServiceTests.java @@ -152,10 +152,12 @@ private void assertException(String rate, Class clazz, Stri public void testNotSupportedDisableDynamicSetting() throws IOException { try { - buildScriptService(Settings.builder().put(ScriptService.DISABLE_DYNAMIC_SCRIPTING_SETTING, randomUnicodeOfLength(randomIntBetween(1, 10))).build()); + buildScriptService(Settings.builder().put( + ScriptService.DISABLE_DYNAMIC_SCRIPTING_SETTING, randomUnicodeOfLength(randomIntBetween(1, 10))).build()); fail("script service should have thrown exception due to non supported script.disable_dynamic setting"); } catch(IllegalArgumentException e) { - assertThat(e.getMessage(), containsString(ScriptService.DISABLE_DYNAMIC_SCRIPTING_SETTING + " is not a supported setting, replace with fine-grained script settings")); + assertThat(e.getMessage(), containsString(ScriptService.DISABLE_DYNAMIC_SCRIPTING_SETTING + + " is not a supported setting, replace with fine-grained script settings")); } } @@ -338,7 +340,8 @@ public void testMaxSizeLimit() throws Exception { private void assertCompileRejected(String lang, String script, ScriptType scriptType, ScriptContext scriptContext) { try { scriptService.compile(new Script(scriptType, lang, script, Collections.emptyMap()), scriptContext); - fail("compile should have been rejected for lang [" + lang + "], script_type [" + scriptType + "], scripted_op [" + scriptContext + "]"); + fail("compile should have been rejected for lang [" + lang + "], " + + "script_type [" + scriptType + "], scripted_op [" + scriptContext + "]"); } catch (IllegalArgumentException | IllegalStateException e) { // pass } From 29c895b55c4d52fa5649fcfd122f97c58113cb93 Mon Sep 17 00:00:00 2001 From: Ryan Ernst Date: Wed, 9 Jan 2019 10:56:30 -0800 Subject: [PATCH 39/46] Build: Quiet naming convention logging (#37244) This commit moves log statements related to classification of naming convention checks for tests to debug level. At info level they emit an enormous amount of output in CI, while these are not generally useful for debugging normal build failures. --- .../gradle/precommit/TestingConventionsTasks.java | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/buildSrc/src/main/java/org/elasticsearch/gradle/precommit/TestingConventionsTasks.java b/buildSrc/src/main/java/org/elasticsearch/gradle/precommit/TestingConventionsTasks.java index efa2684690703..0f207ad3fe1af 100644 --- a/buildSrc/src/main/java/org/elasticsearch/gradle/precommit/TestingConventionsTasks.java +++ b/buildSrc/src/main/java/org/elasticsearch/gradle/precommit/TestingConventionsTasks.java @@ -310,18 +310,18 @@ private boolean seemsLikeATest(Class clazz) { Class junitTest = loadClassWithoutInitializing("org.junit.Assert", classLoader); if (junitTest.isAssignableFrom(clazz)) { - getLogger().info("{} is a test because it extends {}", clazz.getName(), junitTest.getName()); + getLogger().debug("{} is a test because it extends {}", clazz.getName(), junitTest.getName()); return true; } Class junitAnnotation = loadClassWithoutInitializing("org.junit.Test", classLoader); for (Method method : clazz.getMethods()) { if (matchesTestMethodNamingConvention(method)) { - getLogger().info("{} is a test because it has method named '{}'", clazz.getName(), method.getName()); + getLogger().debug("{} is a test because it has method named '{}'", clazz.getName(), method.getName()); return true; } if (isAnnotated(method, junitAnnotation)) { - getLogger().info("{} is a test because it has method '{}' annotated with '{}'", + getLogger().debug("{} is a test because it has method '{}' annotated with '{}'", clazz.getName(), method.getName(), junitAnnotation.getName()); return true; } @@ -340,7 +340,7 @@ private boolean implementsNamingConvention(Class clazz) { if (naming.stream() .map(TestingConventionRule::getSuffix) .anyMatch(suffix -> clazz.getName().endsWith(suffix))) { - getLogger().info("{} is a test because it matches the naming convention", clazz.getName()); + getLogger().debug("{} is a test because it matches the naming convention", clazz.getName()); return true; } return false; From 722b850efd1cd269233bfba22d2897d0749d8aee Mon Sep 17 00:00:00 2001 From: Ryan Ernst Date: Wed, 9 Jan 2019 10:57:26 -0800 Subject: [PATCH 40/46] Build: Make assemble a noop in bwc projects (#37245) This commit makes the assemble tasks in the bwc projects noops by setting the dependsOn directly. While we can not remove things from dependsOn, we can still completely override the dependencies. closes #33581 --- distribution/bwc/build.gradle | 24 +++++++++--------------- 1 file changed, 9 insertions(+), 15 deletions(-) diff --git a/distribution/bwc/build.gradle b/distribution/bwc/build.gradle index ec7d84d7a3084..47f3b78c43f5d 100644 --- a/distribution/bwc/build.gradle +++ b/distribution/bwc/build.gradle @@ -197,21 +197,15 @@ bwcVersions.forPreviousUnreleased { VersionCollection.UnreleasedVersionInfo unre } } - if (gradle.startParameter.taskNames == ["assemble"]) { - // Gradle needs the `artifacts` declaration, including `builtBy` bellow to make projects dependencies on this - // project work, but it will also trigger the build of these for the `assemble` task. - // Since these are only used for testing, we don't want to assemble them if `assemble` is the single command being - // ran. - logger.info("Skipping BWC builds since `assemble` is the only task name provided on the command line") - } else { - artifacts { - for (File artifactFile : artifactFiles) { - String artifactName = artifactFile.name.contains('oss') ? 'elasticsearch-oss' : 'elasticsearch' - String suffix = artifactFile.toString()[-3..-1] - 'default' file: artifactFile, name: artifactName, type: suffix, builtBy: buildBwcVersion - } - } + artifacts { + for (File artifactFile : artifactFiles) { + String artifactName = artifactFile.name.contains('oss') ? 'elasticsearch-oss' : 'elasticsearch' + String suffix = artifactFile.toString()[-3..-1] + 'default' file: artifactFile, name: artifactName, type: suffix, builtBy: buildBwcVersion + } } + // make sure no dependencies were added to assemble; we want it to be a no-op + assemble.dependsOn = [] }} class IndentingOutputStream extends OutputStream { @@ -237,4 +231,4 @@ class IndentingOutputStream extends OutputStream { } } } -} \ No newline at end of file +} From c71060fa0118ac9de2b292fe0c2ef587e0cb35ce Mon Sep 17 00:00:00 2001 From: jaymode Date: Wed, 9 Jan 2019 12:17:43 -0700 Subject: [PATCH 41/46] Test: fix race in auth result propagation test This commit fixes a race condition in a test introduced by #36900 that verifies concurrent authentications get a result propagated from the first thread that attempts to authenticate. Previously, a thread may be in a state where it had not attempted to authenticate when the first thread that authenticates finishes the authentication, which would cause the test to fail as there would be an additional authentication attempt. This change adds additional latches to ensure all threads have attempted to authenticate before a result gets returned in the thread that is performing authentication. --- .../CachingUsernamePasswordRealmTests.java | 22 +++++++++++++++++-- 1 file changed, 20 insertions(+), 2 deletions(-) diff --git a/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/authc/support/CachingUsernamePasswordRealmTests.java b/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/authc/support/CachingUsernamePasswordRealmTests.java index 4ed04864041d6..2fed720e23c09 100644 --- a/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/authc/support/CachingUsernamePasswordRealmTests.java +++ b/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/authc/support/CachingUsernamePasswordRealmTests.java @@ -484,13 +484,27 @@ public void testUnauthenticatedResultPropagatesWithSameCreds() throws Exception final int numberOfProcessors = Runtime.getRuntime().availableProcessors(); final int numberOfThreads = scaledRandomIntBetween((numberOfProcessors + 1) / 2, numberOfProcessors * 3); - final CountDownLatch latch = new CountDownLatch(1 + numberOfThreads); List threads = new ArrayList<>(numberOfThreads); final SecureString credsToUse = new SecureString(randomAlphaOfLength(12).toCharArray()); + + // we use a bunch of different latches here, the first `latch` is used to ensure all threads have been started + // before they start to execute. The `authWaitLatch` is there to ensure we have all threads waiting on the + // listener before we auth otherwise we may run into a race condition where we auth and one of the threads is + // not waiting on auth yet. Finally, the completedLatch is used to signal that each thread received a response! + final CountDownLatch latch = new CountDownLatch(1 + numberOfThreads); + final CountDownLatch authWaitLatch = new CountDownLatch(numberOfThreads); + final CountDownLatch completedLatch = new CountDownLatch(numberOfThreads); final CachingUsernamePasswordRealm realm = new CachingUsernamePasswordRealm(config, threadPool) { @Override protected void doAuthenticate(UsernamePasswordToken token, ActionListener listener) { authCounter.incrementAndGet(); + authWaitLatch.countDown(); + try { + authWaitLatch.await(); + } catch (InterruptedException e) { + logger.info("authentication was interrupted", e); + Thread.currentThread().interrupt(); + } // do something slow if (pwdHasher.verify(token.credentials(), passwordHash.toCharArray())) { listener.onFailure(new IllegalStateException("password auth should never succeed")); @@ -513,14 +527,17 @@ protected void doLookupUser(String username, ActionListener listener) { realm.authenticate(token, ActionListener.wrap((result) -> { if (result.isAuthenticated()) { + completedLatch.countDown(); throw new IllegalStateException("invalid password led to an authenticated result: " + result); } assertThat(result.getMessage(), containsString("password verification failed")); + completedLatch.countDown(); }, (e) -> { logger.error("caught exception", e); + completedLatch.countDown(); fail("unexpected exception - " + e); })); - + authWaitLatch.countDown(); } catch (InterruptedException e) { logger.error("thread was interrupted", e); Thread.currentThread().interrupt(); @@ -535,6 +552,7 @@ protected void doLookupUser(String username, ActionListener listener) { for (Thread thread : threads) { thread.join(); } + completedLatch.await(); assertEquals(1, authCounter.get()); } From dec436eace40d1cd2a7e523f02f3eaea7afca8ba Mon Sep 17 00:00:00 2001 From: Yuri Astrakhan Date: Wed, 9 Jan 2019 14:41:10 -0500 Subject: [PATCH 42/46] fix equality --- .../geogrid2/GeoGridAggregationBuilder2.java | 17 ++++------------- .../bucket/geogrid2/GeoHashBucket.java | 3 +-- 2 files changed, 5 insertions(+), 15 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/geogrid2/GeoGridAggregationBuilder2.java b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/geogrid2/GeoGridAggregationBuilder2.java index 7097b6c577d28..88386bd508b3f 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/geogrid2/GeoGridAggregationBuilder2.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/geogrid2/GeoGridAggregationBuilder2.java @@ -253,19 +253,10 @@ protected XContentBuilder doXContentBody(XContentBuilder builder, Params params) @Override protected boolean innerEquals(Object obj) { GeoGridAggregationBuilder2 other = (GeoGridAggregationBuilder2) obj; - if (type != other.type) { - return false; - } - if (precision != other.precision) { - return false; - } - if (requiredSize != other.requiredSize) { - return false; - } - if (shardSize != other.shardSize) { - return false; - } - return true; + return Objects.equals(type, other.type) && + Objects.equals(precision, other.precision) && + Objects.equals(requiredSize, other.requiredSize) && + Objects.equals(shardSize, other.shardSize); } @Override diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/geogrid2/GeoHashBucket.java b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/geogrid2/GeoHashBucket.java index 79262129f5d34..718366586bb82 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/geogrid2/GeoHashBucket.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/geogrid2/GeoHashBucket.java @@ -19,7 +19,6 @@ package org.elasticsearch.search.aggregations.bucket.geogrid2; -import org.elasticsearch.common.geo.GeoHashUtils; import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.search.aggregations.InternalAggregations; @@ -45,6 +44,6 @@ protected GeoGridBucket newBucket(long hashAsLong, long docCount, InternalAggreg @Override public String getKeyAsString() { - return GeoHashUtils.stringEncode(this.hashAsLong); + return GeoHashType.SINGLETON.hashAsString(this.hashAsLong); } } From df3b58cb04c0443142221884380d67d86d7da6e0 Mon Sep 17 00:00:00 2001 From: Benjamin Trent Date: Wed, 9 Jan 2019 14:25:35 -0600 Subject: [PATCH 43/46] ML: add migrate anomalies assistant (#36643) * ML: add migrate anomalies assistant * adjusting failure handling for reindex * Fixing request and tests * Adding tests to blacklist * adjusting test * test fix: posting data directly to the job instead of relying on datafeed * adjusting API usage * adding Todos and adjusting endpoint * Adding types to reindexRequest * removing unreliable "live" data test * adding index refresh to test * adding index refresh to test * adding index refresh to yaml test * fixing bad exists call * removing todo * Addressing remove comments * Adjusting rest endpoint name * making service have its own logger * adjusting validity check for newindex names * fixing typos * fixing renaming --- .../xpack/core/XPackClientPlugin.java | 2 + .../xpack/core/ml/action/MlUpgradeAction.java | 160 ++++++ .../core/ml/action/MlUpgradeRequestTests.java | 28 + .../ml/qa/ml-with-security/build.gradle | 4 +- .../MlNativeAutodetectIntegTestCase.java | 3 +- .../xpack/ml/integration/MlUpgradeIT.java | 378 +++++++++++++ .../xpack/ml/MachineLearning.java | 9 +- .../xpack/ml/MlConfigMigrator.java | 8 +- .../xpack/ml/ResultsIndexUpgradeService.java | 513 ++++++++++++++++++ .../TransportFinalizeJobExecutionAction.java | 10 +- .../ml/action/TransportMlUpgradeAction.java | 79 +++ .../xpack/ml/job/JobManager.java | 18 +- .../ml/rest/results/RestUpgradeMlAction.java | 76 +++ .../xpack/ml/utils/ChainTaskExecutor.java | 60 -- .../ml/utils/TypedChainTaskExecutor.java | 125 +++++ .../xpack/ml/utils/VoidChainTaskExecutor.java | 26 + ...s.java => VoidChainTaskExecutorTests.java} | 58 +- .../rest-api-spec/api/ml.upgrade.json | 21 + .../rest-api-spec/test/ml/ml_upgrade.yml | 70 +++ .../mixed_cluster/80_ml_results_upgrade.yml | 11 + .../old_cluster/80_ml_results_upgrade.yml | 120 ++++ .../80_ml_results_upgrade.yml | 158 ++++++ 22 files changed, 1826 insertions(+), 111 deletions(-) create mode 100644 x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/ml/action/MlUpgradeAction.java create mode 100644 x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/ml/action/MlUpgradeRequestTests.java create mode 100644 x-pack/plugin/ml/qa/native-multi-node-tests/src/test/java/org/elasticsearch/xpack/ml/integration/MlUpgradeIT.java create mode 100644 x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/ResultsIndexUpgradeService.java create mode 100644 x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/action/TransportMlUpgradeAction.java create mode 100644 x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/rest/results/RestUpgradeMlAction.java delete mode 100644 x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/utils/ChainTaskExecutor.java create mode 100644 x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/utils/TypedChainTaskExecutor.java create mode 100644 x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/utils/VoidChainTaskExecutor.java rename x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/utils/{ChainTaskExecutorTests.java => VoidChainTaskExecutorTests.java} (62%) create mode 100644 x-pack/plugin/src/test/resources/rest-api-spec/api/ml.upgrade.json create mode 100644 x-pack/plugin/src/test/resources/rest-api-spec/test/ml/ml_upgrade.yml create mode 100644 x-pack/qa/rolling-upgrade/src/test/resources/rest-api-spec/test/mixed_cluster/80_ml_results_upgrade.yml create mode 100644 x-pack/qa/rolling-upgrade/src/test/resources/rest-api-spec/test/old_cluster/80_ml_results_upgrade.yml create mode 100644 x-pack/qa/rolling-upgrade/src/test/resources/rest-api-spec/test/upgraded_cluster/80_ml_results_upgrade.yml diff --git a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/XPackClientPlugin.java b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/XPackClientPlugin.java index 85751a320c585..d7abe9a1f0f03 100644 --- a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/XPackClientPlugin.java +++ b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/XPackClientPlugin.java @@ -105,6 +105,7 @@ import org.elasticsearch.xpack.core.ml.action.PutDatafeedAction; import org.elasticsearch.xpack.core.ml.action.PutFilterAction; import org.elasticsearch.xpack.core.ml.action.PutJobAction; +import org.elasticsearch.xpack.core.ml.action.MlUpgradeAction; import org.elasticsearch.xpack.core.ml.action.RevertModelSnapshotAction; import org.elasticsearch.xpack.core.ml.action.StartDatafeedAction; import org.elasticsearch.xpack.core.ml.action.StopDatafeedAction; @@ -289,6 +290,7 @@ public List> getClientActions() { PostCalendarEventsAction.INSTANCE, PersistJobAction.INSTANCE, FindFileStructureAction.INSTANCE, + MlUpgradeAction.INSTANCE, // security ClearRealmCacheAction.INSTANCE, ClearRolesCacheAction.INSTANCE, diff --git a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/ml/action/MlUpgradeAction.java b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/ml/action/MlUpgradeAction.java new file mode 100644 index 0000000000000..404f15d4f6270 --- /dev/null +++ b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/ml/action/MlUpgradeAction.java @@ -0,0 +1,160 @@ +/* + * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one + * or more contributor license agreements. Licensed under the Elastic License; + * you may not use this file except in compliance with the Elastic License. + */ +package org.elasticsearch.xpack.core.ml.action; + +import org.elasticsearch.action.Action; +import org.elasticsearch.action.ActionRequestValidationException; +import org.elasticsearch.action.support.IndicesOptions; +import org.elasticsearch.action.support.master.AcknowledgedResponse; +import org.elasticsearch.action.support.master.MasterNodeReadOperationRequestBuilder; +import org.elasticsearch.action.support.master.MasterNodeReadRequest; +import org.elasticsearch.client.ElasticsearchClient; +import org.elasticsearch.common.ParseField; +import org.elasticsearch.common.io.stream.StreamInput; +import org.elasticsearch.common.io.stream.StreamOutput; +import org.elasticsearch.common.xcontent.ObjectParser; +import org.elasticsearch.common.xcontent.ToXContentObject; +import org.elasticsearch.common.xcontent.XContentBuilder; +import org.elasticsearch.tasks.CancellableTask; +import org.elasticsearch.tasks.Task; +import org.elasticsearch.tasks.TaskId; +import org.elasticsearch.xpack.core.ml.job.persistence.AnomalyDetectorsIndexFields; + +import java.io.IOException; +import java.util.Map; +import java.util.Objects; + + +public class MlUpgradeAction extends Action { + public static final MlUpgradeAction INSTANCE = new MlUpgradeAction(); + public static final String NAME = "cluster:admin/xpack/ml/upgrade"; + + private MlUpgradeAction() { + super(NAME); + } + + @Override + public AcknowledgedResponse newResponse() { + return new AcknowledgedResponse(); + } + + public static class Request extends MasterNodeReadRequest implements ToXContentObject { + + private static final ParseField REINDEX_BATCH_SIZE = new ParseField("reindex_batch_size"); + + public static ObjectParser PARSER = new ObjectParser<>("ml_upgrade", true, Request::new); + static { + PARSER.declareInt(Request::setReindexBatchSize, REINDEX_BATCH_SIZE); + } + + static final String INDEX = AnomalyDetectorsIndexFields.RESULTS_INDEX_PREFIX + "*"; + private int reindexBatchSize = 1000; + + /** + * Should this task store its result? + */ + private boolean shouldStoreResult; + + // for serialization + public Request() { + } + + public Request(StreamInput in) throws IOException { + super(in); + reindexBatchSize = in.readInt(); + } + + @Override + public void writeTo(StreamOutput out) throws IOException { + super.writeTo(out); + out.writeInt(reindexBatchSize); + } + + public String[] indices() { + return new String[]{INDEX}; + } + + public IndicesOptions indicesOptions() { + return IndicesOptions.strictExpandOpenAndForbidClosed(); + } + + /** + * Should this task store its result after it has finished? + */ + public Request setShouldStoreResult(boolean shouldStoreResult) { + this.shouldStoreResult = shouldStoreResult; + return this; + } + + @Override + public boolean getShouldStoreResult() { + return shouldStoreResult; + } + + public Request setReindexBatchSize(int reindexBatchSize) { + this.reindexBatchSize = reindexBatchSize; + return this; + } + + public int getReindexBatchSize() { + return reindexBatchSize; + } + + @Override + public ActionRequestValidationException validate() { + if (reindexBatchSize <= 0) { + ActionRequestValidationException validationException = new ActionRequestValidationException(); + validationException.addValidationError("["+ REINDEX_BATCH_SIZE.getPreferredName()+"] must be greater than 0."); + return validationException; + } + return null; + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + + Request request = (Request) o; + return Objects.equals(reindexBatchSize, request.reindexBatchSize); + } + + @Override + public int hashCode() { + return Objects.hash(reindexBatchSize); + } + + @Override + public Task createTask(long id, String type, String action, TaskId parentTaskId, Map headers) { + return new CancellableTask(id, type, action, "ml-upgrade", parentTaskId, headers) { + @Override + public boolean shouldCancelChildrenOnCancellation() { + return true; + } + }; + } + + @Override + public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException { + builder.startObject(); + builder.field(REINDEX_BATCH_SIZE.getPreferredName(), reindexBatchSize); + builder.endObject(); + return builder; + } + } + + public static class RequestBuilder extends MasterNodeReadOperationRequestBuilder { + + public RequestBuilder(ElasticsearchClient client) { + super(client, INSTANCE, new Request()); + } + } + +} diff --git a/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/ml/action/MlUpgradeRequestTests.java b/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/ml/action/MlUpgradeRequestTests.java new file mode 100644 index 0000000000000..227fc20ec9688 --- /dev/null +++ b/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/ml/action/MlUpgradeRequestTests.java @@ -0,0 +1,28 @@ +/* + * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one + * or more contributor license agreements. Licensed under the Elastic License; + * you may not use this file except in compliance with the Elastic License. + */ +package org.elasticsearch.xpack.core.ml.action; + +import org.elasticsearch.common.io.stream.Writeable; +import org.elasticsearch.test.AbstractWireSerializingTestCase; + + +public class MlUpgradeRequestTests extends AbstractWireSerializingTestCase { + + @Override + protected MlUpgradeAction.Request createTestInstance() { + MlUpgradeAction.Request request = new MlUpgradeAction.Request(); + if (randomBoolean()) { + request.setReindexBatchSize(randomIntBetween(1, 10_000)); + } + return request; + } + + @Override + protected Writeable.Reader instanceReader() { + return MlUpgradeAction.Request::new; + } + +} diff --git a/x-pack/plugin/ml/qa/ml-with-security/build.gradle b/x-pack/plugin/ml/qa/ml-with-security/build.gradle index abfed3fd878d0..6e0127f614c9a 100644 --- a/x-pack/plugin/ml/qa/ml-with-security/build.gradle +++ b/x-pack/plugin/ml/qa/ml-with-security/build.gradle @@ -93,7 +93,9 @@ integTestRunner { 'ml/validate/Test job config that is invalid only because of the job ID', 'ml/validate_detector/Test invalid detector', 'ml/delete_forecast/Test delete on _all forecasts not allow no forecasts', - 'ml/delete_forecast/Test delete forecast on missing forecast' + 'ml/delete_forecast/Test delete forecast on missing forecast', + 'ml/ml_upgrade/Upgrade results when there is nothing to upgrade', + 'ml/ml_upgrade/Upgrade results when there is nothing to upgrade not waiting for results' ].join(',') } diff --git a/x-pack/plugin/ml/qa/native-multi-node-tests/src/test/java/org/elasticsearch/xpack/ml/integration/MlNativeAutodetectIntegTestCase.java b/x-pack/plugin/ml/qa/native-multi-node-tests/src/test/java/org/elasticsearch/xpack/ml/integration/MlNativeAutodetectIntegTestCase.java index c06810bbf2a0e..cd33e1d80769e 100644 --- a/x-pack/plugin/ml/qa/native-multi-node-tests/src/test/java/org/elasticsearch/xpack/ml/integration/MlNativeAutodetectIntegTestCase.java +++ b/x-pack/plugin/ml/qa/native-multi-node-tests/src/test/java/org/elasticsearch/xpack/ml/integration/MlNativeAutodetectIntegTestCase.java @@ -26,6 +26,7 @@ import org.elasticsearch.common.xcontent.XContentType; import org.elasticsearch.common.xcontent.json.JsonXContent; import org.elasticsearch.index.query.QueryBuilders; +import org.elasticsearch.index.reindex.ReindexPlugin; import org.elasticsearch.persistent.PersistentTaskParams; import org.elasticsearch.persistent.PersistentTaskState; import org.elasticsearch.plugins.Plugin; @@ -120,7 +121,7 @@ protected Collection> nodePlugins() { @Override protected Collection> transportClientPlugins() { - return Arrays.asList(XPackClientPlugin.class, Netty4Plugin.class); + return Arrays.asList(XPackClientPlugin.class, Netty4Plugin.class, ReindexPlugin.class); } @Override diff --git a/x-pack/plugin/ml/qa/native-multi-node-tests/src/test/java/org/elasticsearch/xpack/ml/integration/MlUpgradeIT.java b/x-pack/plugin/ml/qa/native-multi-node-tests/src/test/java/org/elasticsearch/xpack/ml/integration/MlUpgradeIT.java new file mode 100644 index 0000000000000..a2a05ea1686fa --- /dev/null +++ b/x-pack/plugin/ml/qa/native-multi-node-tests/src/test/java/org/elasticsearch/xpack/ml/integration/MlUpgradeIT.java @@ -0,0 +1,378 @@ +/* + * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one + * or more contributor license agreements. Licensed under the Elastic License; + * you may not use this file except in compliance with the Elastic License. + */ +package org.elasticsearch.xpack.ml.integration; + +import org.elasticsearch.Version; +import org.elasticsearch.action.ActionListener; +import org.elasticsearch.action.admin.cluster.state.ClusterStateRequest; +import org.elasticsearch.action.search.SearchResponse; +import org.elasticsearch.action.support.IndicesOptions; +import org.elasticsearch.action.support.PlainActionFuture; +import org.elasticsearch.action.support.master.AcknowledgedResponse; +import org.elasticsearch.cluster.ClusterState; +import org.elasticsearch.cluster.metadata.IndexNameExpressionResolver; +import org.elasticsearch.common.collect.Tuple; +import org.elasticsearch.common.unit.TimeValue; +import org.elasticsearch.index.query.QueryBuilders; +import org.elasticsearch.index.reindex.ReindexAction; +import org.elasticsearch.index.reindex.ReindexRequest; +import org.elasticsearch.test.ESIntegTestCase; +import org.elasticsearch.test.ESTestCase; +import org.elasticsearch.threadpool.ThreadPool; +import org.elasticsearch.xpack.core.ml.action.MlUpgradeAction; +import org.elasticsearch.xpack.core.ml.datafeed.DatafeedConfig; +import org.elasticsearch.xpack.core.ml.job.config.Job; +import org.elasticsearch.xpack.core.ml.job.config.JobState; +import org.elasticsearch.xpack.core.ml.job.persistence.AnomalyDetectorsIndex; +import org.elasticsearch.xpack.core.ml.job.persistence.AnomalyDetectorsIndexFields; +import org.elasticsearch.xpack.ml.ResultsIndexUpgradeService; +import org.junit.After; +import org.junit.Assert; + +import java.util.Arrays; +import java.util.Collections; +import java.util.List; + +import static org.elasticsearch.xpack.ml.support.BaseMlIntegTestCase.createDatafeedBuilder; +import static org.elasticsearch.xpack.ml.support.BaseMlIntegTestCase.createScheduledJob; +import static org.elasticsearch.xpack.ml.support.BaseMlIntegTestCase.indexDocs; +import static org.hamcrest.Matchers.equalTo; +import static org.hamcrest.Matchers.greaterThan; +import static org.hamcrest.Matchers.instanceOf; +import static org.hamcrest.core.Is.is; + +public class MlUpgradeIT extends MlNativeAutodetectIntegTestCase { + + @After + public void cleanup() throws Exception { + cleanUp(); + } + + public void testMigrationWhenItIsNotNecessary() throws Exception { + String jobId1 = "no-migration-test1"; + String jobId2 = "no-migration-test2"; + String jobId3 = "no-migration-test3"; + + String dataIndex = createDataIndex().v2(); + List jobs = createJobsWithData(jobId1, jobId2, jobId3, dataIndex); + Job job1 = jobs.get(0); + Job job2 = jobs.get(1); + Job job3 = jobs.get(2); + + String job1Index = job1.getResultsIndexName(); + String job2Index = job2.getResultsIndexName(); + String job3Index = job3.getResultsIndexName(); + + assertThat(indexExists(job1Index), is(true)); + assertThat(indexExists(job2Index), is(true)); + assertThat(indexExists(job3Index), is(true)); + + long job1Total = getTotalDocCount(job1Index); + long job2Total = getTotalDocCount(job2Index); + long job3Total = getTotalDocCount(job3Index); + + AcknowledgedResponse resp = ESIntegTestCase.client().execute(MlUpgradeAction.INSTANCE, + new MlUpgradeAction.Request()).actionGet(); + assertThat(resp.isAcknowledged(), is(true)); + + // Migration should have done nothing + assertThat(indexExists(job1Index), is(true)); + assertThat(indexExists(job2Index), is(true)); + assertThat(indexExists(job3Index), is(true)); + + assertThat(getTotalDocCount(job1Index), equalTo(job1Total)); + assertThat(getTotalDocCount(job2Index), equalTo(job2Total)); + assertThat(getTotalDocCount(job3Index), equalTo(job3Total)); + + ClusterState state = admin().cluster().state(new ClusterStateRequest()).actionGet().getState(); + IndexNameExpressionResolver indexNameExpressionResolver = new IndexNameExpressionResolver(); + String[] indices = indexNameExpressionResolver.concreteIndexNames(state, + IndicesOptions.strictExpandOpenAndForbidClosed(), + AnomalyDetectorsIndex.jobResultsIndexPrefix() + "*"); + + // Our backing index size should be two as we have a shared and custom index + assertThat(indices.length, equalTo(2)); + } + + public void testMigration() throws Exception { + String jobId1 = "migration-test1"; + String jobId2 = "migration-test2"; + String jobId3 = "migration-test3"; + + String dataIndex = createDataIndex().v2(); + List jobs = createJobsWithData(jobId1, jobId2, jobId3, dataIndex); + Job job1 = jobs.get(0); + Job job2 = jobs.get(1); + Job job3 = jobs.get(2); + + String job1Index = job1.getResultsIndexName(); + String job2Index = job2.getResultsIndexName(); + String job3Index = job3.getResultsIndexName(); + + assertThat(indexExists(job1Index), is(true)); + assertThat(indexExists(job2Index), is(true)); + assertThat(indexExists(job3Index), is(true)); + + long job1Total = getJobResultsCount(job1.getId()); + long job2Total = getJobResultsCount(job2.getId()); + long job3Total = getJobResultsCount(job3.getId()); + + IndexNameExpressionResolver indexNameExpressionResolver = new IndexNameExpressionResolver(); + + ResultsIndexUpgradeService resultsIndexUpgradeService = new ResultsIndexUpgradeService(indexNameExpressionResolver, + ThreadPool.Names.SAME, + indexMetaData -> true); + + PlainActionFuture future = PlainActionFuture.newFuture(); + + resultsIndexUpgradeService.upgrade(ESIntegTestCase.client(), + new MlUpgradeAction.Request(), + ESIntegTestCase.client().admin().cluster().prepareState().get().getState(), + future); + + AcknowledgedResponse response = future.get(); + assertThat(response.isAcknowledged(), is(true)); + + assertThat(indexExists(job1Index), is(false)); + assertThat(indexExists(job2Index), is(false)); + assertThat(indexExists(job3Index), is(false)); + + ClusterState state = admin().cluster().state(new ClusterStateRequest()).actionGet().getState(); + String[] indices = indexNameExpressionResolver.concreteIndexNames(state, + IndicesOptions.strictExpandOpenAndForbidClosed(), + AnomalyDetectorsIndex.jobResultsIndexPrefix() + "*"); + + // Our backing index size should be four as we have a shared and custom index and upgrading doubles the number of indices + Assert.assertThat(indices.length, equalTo(4)); + + refresh(indices); + assertThat(getJobResultsCount(job1.getId()), equalTo(job1Total)); + assertThat(getJobResultsCount(job2.getId()), equalTo(job2Total)); + assertThat(getJobResultsCount(job3.getId()), equalTo(job3Total)); + + + // WE should still be able to write, and the aliases should allow to read from the appropriate indices + postDataToJob(jobId1); + postDataToJob(jobId2); + postDataToJob(jobId3); + // We should also be able to create new jobs and old jobs should be unaffected. + String jobId4 = "migration-test4"; + Job job4 = createAndOpenJobAndStartDataFeedWithData(jobId4, dataIndex, false); + waitUntilJobIsClosed(jobId4); + + indices = indexNameExpressionResolver.concreteIndexNames(state, + IndicesOptions.strictExpandOpenAndForbidClosed(), + AnomalyDetectorsIndex.jobResultsIndexPrefix() + "*"); + refresh(indices); + + long newJob1Total = getJobResultsCount(job1.getId()); + assertThat(newJob1Total, greaterThan(job1Total)); + + long newJob2Total = getJobResultsCount(job2.getId()); + assertThat(newJob2Total, greaterThan(job2Total)); + + long newJob3Total = getJobResultsCount(job3.getId()); + assertThat(newJob3Total, greaterThan(job3Total)); + + assertThat(getJobResultsCount(jobId4), greaterThan(0L)); + assertThat(getJobResultsCount(jobId1), equalTo(newJob1Total)); + assertThat(getJobResultsCount(jobId2), equalTo(newJob2Total)); + assertThat(getJobResultsCount(jobId3), equalTo(newJob3Total)); + } + + //I think this test name could be a little bit longer.... + public void testMigrationWithManuallyCreatedIndexThatNeedsMigrating() throws Exception { + String jobId1 = "migration-failure-test1"; + String jobId2 = "migration-failure-test2"; + String jobId3 = "migration-failure-test3"; + + String dataIndex = createDataIndex().v2(); + List jobs = createJobsWithData(jobId1, jobId2, jobId3, dataIndex); + Job job1 = jobs.get(0); + Job job2 = jobs.get(1); + Job job3 = jobs.get(2); + + String job1Index = job1.getResultsIndexName(); + String job2Index = job2.getResultsIndexName(); + String job3Index = job3.getResultsIndexName(); + + // This index name should match one of the automatically created migration indices + String manuallyCreatedIndex = job1Index + "-" + Version.CURRENT.major; + client().admin().indices().prepareCreate(manuallyCreatedIndex).execute().actionGet(); + + IndexNameExpressionResolver indexNameExpressionResolver = new IndexNameExpressionResolver(); + + ResultsIndexUpgradeService resultsIndexUpgradeService = new ResultsIndexUpgradeService(indexNameExpressionResolver, + ThreadPool.Names.SAME, + indexMetaData -> true); //indicates that this manually created index needs migrated + + resultsIndexUpgradeService.upgrade(ESIntegTestCase.client(), + new MlUpgradeAction.Request(), + ESIntegTestCase.client().admin().cluster().prepareState().get().getState(), + ActionListener.wrap( + resp -> fail(), + exception -> { + assertThat(exception, instanceOf(IllegalStateException.class)); + assertThat(exception.getMessage(), + equalTo("Index [" + manuallyCreatedIndex + "] already exists and is not the current version.")); + } + )); + } + + public void testMigrationWithExistingIndexWithData() throws Exception { + String jobId1 = "partial-migration-test1"; + String jobId2 = "partial-migration-test2"; + String jobId3 = "partial-migration-test3"; + + String dataIndex = createDataIndex().v2(); + List jobs = createJobsWithData(jobId1, jobId2, jobId3, dataIndex); + Job job1 = jobs.get(0); + Job job2 = jobs.get(1); + Job job3 = jobs.get(2); + + String job1Index = job1.getResultsIndexName(); + String job2Index = job2.getResultsIndexName(); + String job3Index = job3.getResultsIndexName(); + + assertThat(indexExists(job1Index), is(true)); + assertThat(indexExists(job2Index), is(true)); + assertThat(indexExists(job3Index), is(true)); + + long job1Total = getJobResultsCount(job1.getId()); + long job2Total = getJobResultsCount(job2.getId()); + long job3Total = getJobResultsCount(job3.getId()); + + //lets manually create a READ index with reindexed data already + // Should still get aliased appropriately without any additional/duplicate data. + String alreadyMigratedIndex = job1Index + "-" + Version.CURRENT.major + "r"; + ReindexRequest reindexRequest = new ReindexRequest(); + reindexRequest.setSourceIndices(job1Index); + reindexRequest.setDestIndex(alreadyMigratedIndex); + client().execute(ReindexAction.INSTANCE, reindexRequest).actionGet(); + + //New write index as well, should still get aliased appropriately + String alreadyMigratedWriteIndex = job1Index + "-" + Version.CURRENT.major; + client().admin().indices().prepareCreate(alreadyMigratedWriteIndex).execute().actionGet(); + + IndexNameExpressionResolver indexNameExpressionResolver = new IndexNameExpressionResolver(); + + ResultsIndexUpgradeService resultsIndexUpgradeService = new ResultsIndexUpgradeService(indexNameExpressionResolver, + ThreadPool.Names.SAME, + //indicates that this manually created index is already migrated and should not be included in our migration steps + indexMetaData -> !(indexMetaData.getIndex().getName().equals(alreadyMigratedIndex) || + indexMetaData.getIndex().getName().equals(alreadyMigratedWriteIndex))); + + PlainActionFuture future = PlainActionFuture.newFuture(); + + resultsIndexUpgradeService.upgrade(ESIntegTestCase.client(), + new MlUpgradeAction.Request(), + ESIntegTestCase.client().admin().cluster().prepareState().get().getState(), + future); + + AcknowledgedResponse response = future.get(); + assertThat(response.isAcknowledged(), is(true)); + + assertThat(indexExists(job1Index), is(false)); + assertThat(indexExists(job2Index), is(false)); + assertThat(indexExists(job3Index), is(false)); + + ClusterState state = admin().cluster().state(new ClusterStateRequest()).actionGet().getState(); + String[] indices = indexNameExpressionResolver.concreteIndexNames(state, + IndicesOptions.strictExpandOpenAndForbidClosed(), + AnomalyDetectorsIndex.jobResultsIndexPrefix() + "*"); + + // Our backing index size should be four as we have a shared and custom index and upgrading doubles the number of indices + Assert.assertThat(indices.length, equalTo(4)); + refresh(indices); + + assertThat(getJobResultsCount(job1.getId()), equalTo(job1Total)); + assertThat(getJobResultsCount(job2.getId()), equalTo(job2Total)); + assertThat(getJobResultsCount(job3.getId()), equalTo(job3Total)); + + // WE should still be able to write, and the aliases should allow to read from the appropriate indices + postDataToJob(jobId1); + postDataToJob(jobId2); + postDataToJob(jobId3); + + refresh(indices); + + long newJob1Total = getJobResultsCount(job1.getId()); + assertThat(newJob1Total, greaterThan(job1Total)); + + long newJob2Total = getJobResultsCount(job2.getId()); + assertThat(newJob2Total, greaterThan(job2Total)); + + long newJob3Total = getJobResultsCount(job3.getId()); + assertThat(newJob3Total, greaterThan(job3Total)); + } + + private long getTotalDocCount(String indexName) { + SearchResponse searchResponse = ESIntegTestCase.client().prepareSearch(indexName) + .setSize(10_000) + .setTrackTotalHits(true) + .setQuery(QueryBuilders.matchAllQuery()) + .execute().actionGet(); + return searchResponse.getHits().getTotalHits().value; + } + + private long getJobResultsCount(String jobId) { + String index = AnomalyDetectorsIndexFields.RESULTS_INDEX_PREFIX + jobId; + return getTotalDocCount(index); + } + + private void postDataToJob(String jobId) throws Exception { + openJob(jobId); + ESTestCase.assertBusy(() -> Assert.assertEquals(getJobStats(jobId).get(0).getState(), JobState.OPENED)); + startDatafeed(jobId + "-datafeed", 0L, System.currentTimeMillis()); + waitUntilJobIsClosed(jobId); + } + + private Job createAndOpenJobAndStartDataFeedWithData(String jobId, String dataIndex, boolean isCustom) throws Exception { + Job.Builder jobbuilder = createScheduledJob(jobId); + if (isCustom) { + jobbuilder.setResultsIndexName(jobId); + } + registerJob(jobbuilder); + + Job job = putJob(jobbuilder).getResponse(); + + openJob(job.getId()); + ESTestCase.assertBusy(() -> Assert.assertEquals(getJobStats(job.getId()).get(0).getState(), JobState.OPENED)); + + DatafeedConfig.Builder builder = createDatafeedBuilder(job.getId() + "-datafeed", + job.getId(), + Collections.singletonList(dataIndex)); + builder.setQueryDelay(TimeValue.timeValueSeconds(5)); + builder.setFrequency(TimeValue.timeValueSeconds(5)); + DatafeedConfig datafeedConfig = builder.build(); + registerDatafeed(datafeedConfig); + putDatafeed(datafeedConfig); + startDatafeed(datafeedConfig.getId(), 0L, System.currentTimeMillis()); + waitUntilJobIsClosed(jobId); + return job; + } + + private Tuple createDataIndex() { + ESIntegTestCase.client().admin().indices().prepareCreate("data-for-migration-1") + .addMapping("type", "time", "type=date") + .get(); + long numDocs = ESTestCase.randomIntBetween(32, 512); + long now = System.currentTimeMillis(); + long oneWeekAgo = now - 604800000; + long twoWeeksAgo = oneWeekAgo - 604800000; + indexDocs(logger, "data-for-migration-1", numDocs, twoWeeksAgo, oneWeekAgo); + return new Tuple<>(numDocs, "data-for-migration-1"); + } + + private List createJobsWithData(String sharedJobId1, String sharedJobId2, String customJobId, String dataIndex) throws Exception { + + Job job1 = createAndOpenJobAndStartDataFeedWithData(sharedJobId1, dataIndex, false); + Job job2 = createAndOpenJobAndStartDataFeedWithData(sharedJobId2, dataIndex, false); + Job job3 = createAndOpenJobAndStartDataFeedWithData(customJobId, dataIndex, true); + + return Arrays.asList(job1, job2, job3); + } +} diff --git a/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/MachineLearning.java b/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/MachineLearning.java index 7060e87fac0bb..418add2757fbc 100644 --- a/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/MachineLearning.java +++ b/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/MachineLearning.java @@ -96,6 +96,7 @@ import org.elasticsearch.xpack.core.ml.action.PutDatafeedAction; import org.elasticsearch.xpack.core.ml.action.PutFilterAction; import org.elasticsearch.xpack.core.ml.action.PutJobAction; +import org.elasticsearch.xpack.core.ml.action.MlUpgradeAction; import org.elasticsearch.xpack.core.ml.action.RevertModelSnapshotAction; import org.elasticsearch.xpack.core.ml.action.StartDatafeedAction; import org.elasticsearch.xpack.core.ml.action.StopDatafeedAction; @@ -150,6 +151,7 @@ import org.elasticsearch.xpack.ml.action.TransportPutDatafeedAction; import org.elasticsearch.xpack.ml.action.TransportPutFilterAction; import org.elasticsearch.xpack.ml.action.TransportPutJobAction; +import org.elasticsearch.xpack.ml.action.TransportMlUpgradeAction; import org.elasticsearch.xpack.ml.action.TransportRevertModelSnapshotAction; import org.elasticsearch.xpack.ml.action.TransportStartDatafeedAction; import org.elasticsearch.xpack.ml.action.TransportStopDatafeedAction; @@ -229,6 +231,7 @@ import org.elasticsearch.xpack.ml.rest.results.RestGetInfluencersAction; import org.elasticsearch.xpack.ml.rest.results.RestGetOverallBucketsAction; import org.elasticsearch.xpack.ml.rest.results.RestGetRecordsAction; +import org.elasticsearch.xpack.ml.rest.results.RestUpgradeMlAction; import org.elasticsearch.xpack.ml.rest.validate.RestValidateDetectorAction; import org.elasticsearch.xpack.ml.rest.validate.RestValidateJobConfigAction; @@ -541,7 +544,8 @@ public List getRestHandlers(Settings settings, RestController restC new RestPutCalendarJobAction(settings, restController), new RestGetCalendarEventsAction(settings, restController), new RestPostCalendarEventAction(settings, restController), - new RestFindFileStructureAction(settings, restController) + new RestFindFileStructureAction(settings, restController), + new RestUpgradeMlAction(settings, restController) ); } @@ -599,7 +603,8 @@ public List getRestHandlers(Settings settings, RestController restC new ActionHandler<>(GetCalendarEventsAction.INSTANCE, TransportGetCalendarEventsAction.class), new ActionHandler<>(PostCalendarEventsAction.INSTANCE, TransportPostCalendarEventsAction.class), new ActionHandler<>(PersistJobAction.INSTANCE, TransportPersistJobAction.class), - new ActionHandler<>(FindFileStructureAction.INSTANCE, TransportFindFileStructureAction.class) + new ActionHandler<>(FindFileStructureAction.INSTANCE, TransportFindFileStructureAction.class), + new ActionHandler<>(MlUpgradeAction.INSTANCE, TransportMlUpgradeAction.class) ); } @Override diff --git a/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/MlConfigMigrator.java b/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/MlConfigMigrator.java index 184ee44cf376c..bb3735f8aa3f1 100644 --- a/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/MlConfigMigrator.java +++ b/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/MlConfigMigrator.java @@ -44,7 +44,7 @@ import org.elasticsearch.xpack.core.ml.utils.ToXContentParams; import org.elasticsearch.xpack.ml.datafeed.persistence.DatafeedConfigProvider; import org.elasticsearch.xpack.ml.job.persistence.JobConfigProvider; -import org.elasticsearch.xpack.ml.utils.ChainTaskExecutor; +import org.elasticsearch.xpack.ml.utils.VoidChainTaskExecutor; import java.io.IOException; import java.util.ArrayList; @@ -179,9 +179,9 @@ public void migrateConfigsWithoutTasks(ClusterState clusterState, ActionListener } private void migrateBatches(List batches, ActionListener listener) { - ChainTaskExecutor chainTaskExecutor = new ChainTaskExecutor(EsExecutors.newDirectExecutorService(), true); + VoidChainTaskExecutor voidChainTaskExecutor = new VoidChainTaskExecutor(EsExecutors.newDirectExecutorService(), true); for (JobsAndDatafeeds batch : batches) { - chainTaskExecutor.add(chainedListener -> writeConfigToIndex(batch.datafeedConfigs, batch.jobs, ActionListener.wrap( + voidChainTaskExecutor.add(chainedListener -> writeConfigToIndex(batch.datafeedConfigs, batch.jobs, ActionListener.wrap( failedDocumentIds -> { List successfulJobWrites = filterFailedJobConfigWrites(failedDocumentIds, batch.jobs); List successfulDatafeedWrites = @@ -191,7 +191,7 @@ private void migrateBatches(List batches, ActionListener listener.onResponse(true), listener::onFailure)); + voidChainTaskExecutor.execute(ActionListener.wrap(aVoids -> listener.onResponse(true), listener::onFailure)); } // Exposed for testing diff --git a/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/ResultsIndexUpgradeService.java b/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/ResultsIndexUpgradeService.java new file mode 100644 index 0000000000000..ccbaed13feca0 --- /dev/null +++ b/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/ResultsIndexUpgradeService.java @@ -0,0 +1,513 @@ +/* + * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one + * or more contributor license agreements. Licensed under the Elastic License; + * you may not use this file except in compliance with the Elastic License. + */ +package org.elasticsearch.xpack.ml; + +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; +import org.elasticsearch.ElasticsearchException; +import org.elasticsearch.ElasticsearchTimeoutException; +import org.elasticsearch.ExceptionsHelper; +import org.elasticsearch.ResourceAlreadyExistsException; +import org.elasticsearch.Version; +import org.elasticsearch.action.ActionListener; +import org.elasticsearch.action.DocWriteRequest; +import org.elasticsearch.action.admin.indices.alias.Alias; +import org.elasticsearch.action.admin.indices.alias.IndicesAliasesRequest; +import org.elasticsearch.action.admin.indices.create.CreateIndexRequest; +import org.elasticsearch.action.admin.indices.create.CreateIndexResponse; +import org.elasticsearch.action.admin.indices.delete.DeleteIndexRequest; +import org.elasticsearch.action.admin.indices.settings.put.UpdateSettingsRequest; +import org.elasticsearch.action.bulk.BulkItemResponse; +import org.elasticsearch.action.support.IndicesOptions; +import org.elasticsearch.action.support.master.AcknowledgedResponse; +import org.elasticsearch.client.Client; +import org.elasticsearch.cluster.ClusterState; +import org.elasticsearch.cluster.metadata.IndexMetaData; +import org.elasticsearch.cluster.metadata.IndexNameExpressionResolver; +import org.elasticsearch.cluster.metadata.MetaData; +import org.elasticsearch.common.Strings; +import org.elasticsearch.common.collect.Tuple; +import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.index.reindex.BulkByScrollResponse; +import org.elasticsearch.index.reindex.ReindexAction; +import org.elasticsearch.index.reindex.ReindexRequest; +import org.elasticsearch.index.reindex.ScrollableHitSource; +import org.elasticsearch.rest.RestStatus; +import org.elasticsearch.xpack.core.ml.action.MlUpgradeAction; +import org.elasticsearch.xpack.core.ml.job.persistence.ElasticsearchMappings; +import org.elasticsearch.xpack.ml.utils.TypedChainTaskExecutor; + +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collection; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.function.Function; +import java.util.function.Predicate; +import java.util.stream.Collectors; + +import static org.elasticsearch.xpack.core.ClientHelper.ML_ORIGIN; +import static org.elasticsearch.xpack.core.ClientHelper.executeAsyncWithOrigin; + +/** + * ML Job results index upgrade service + */ +public class ResultsIndexUpgradeService { + + private static final Logger logger = LogManager.getLogger(ResultsIndexUpgradeService.class); + + // Adjust the following constants as necessary for various versions and backports. + private static final int INDEX_VERSION = Version.CURRENT.major; + private static final Version MIN_REQUIRED_VERSION = Version.CURRENT.minimumCompatibilityVersion(); + + private final IndexNameExpressionResolver indexNameExpressionResolver; + private final Predicate shouldUpgrade; + private final String executor; + + /** + * Construct a new upgrade service + * + * @param indexNameExpressionResolver Index expression resolver for the request + * @param executor Where to execute client calls + * @param shouldUpgrade Given IndexMetadata indicate if it should be upgraded or not + * {@code true} indicates that it SHOULD upgrade + */ + public ResultsIndexUpgradeService(IndexNameExpressionResolver indexNameExpressionResolver, + String executor, + Predicate shouldUpgrade) { + this.indexNameExpressionResolver = indexNameExpressionResolver; + this.shouldUpgrade = shouldUpgrade; + this.executor = executor; + } + + public static boolean wasIndexCreatedInCurrentMajorVersion(IndexMetaData indexMetaData) { + return indexMetaData.getCreationVersion().major == INDEX_VERSION; + } + + /** + * There are two reasons for these indices to exist: + * 1. The upgrade process has ran before and either failed for some reason, or the end user is simply running it again. + * Either way, it should be ok to proceed as this action SHOULD be idempotent, + * unless the shouldUpgrade predicate is poorly formed + * 2. This index was created manually by the user. If the index was created manually and actually needs upgrading, then + * we consider the "new index" to be invalid as the passed predicate indicates that it still needs upgrading. + * + * @param metaData Cluster metadata + * @param newIndexName The index to check + * @param shouldUpgrade Should be index be upgraded + * @return {@code true} if the "new index" is valid + */ + private static boolean validNewIndex(MetaData metaData, String newIndexName, Predicate shouldUpgrade) { + return (metaData.hasIndex(newIndexName) && shouldUpgrade.test(metaData.index(newIndexName))) == false; + } + + private static void validateMinNodeVersion(ClusterState clusterState) { + if (clusterState.nodes().getMinNodeVersion().before(MIN_REQUIRED_VERSION)) { + throw new IllegalStateException("All nodes should have at least version [" + MIN_REQUIRED_VERSION + "] to upgrade"); + } + } + + // This method copies the behavior of the normal {index}/_upgrade rest response handler + private static Tuple getStatusAndCause(BulkByScrollResponse response) { + /* + * Return the highest numbered rest status under the assumption that higher numbered statuses are "more error" + * and thus more interesting to the user. + */ + RestStatus status = RestStatus.OK; + Throwable cause = null; + if (response.isTimedOut()) { + status = RestStatus.REQUEST_TIMEOUT; + cause = new ElasticsearchTimeoutException("Reindex request timed out"); + } + for (BulkItemResponse.Failure failure : response.getBulkFailures()) { + if (failure.getStatus().getStatus() > status.getStatus()) { + status = failure.getStatus(); + cause = failure.getCause(); + } + } + for (ScrollableHitSource.SearchFailure failure : response.getSearchFailures()) { + RestStatus failureStatus = ExceptionsHelper.status(failure.getReason()); + if (failureStatus.getStatus() > status.getStatus()) { + status = failureStatus; + cause = failure.getReason(); + } + } + return new Tuple<>(status, cause); + } + + /** + * Upgrade the indices given in the request. + * + * @param client The client to use when making calls + * @param request The upgrade request + * @param state The current cluster state + * @param listener The listener to alert when actions have completed + */ + public void upgrade(Client client, MlUpgradeAction.Request request, ClusterState state, + ActionListener listener) { + try { + validateMinNodeVersion(state); + String[] concreteIndices = indexNameExpressionResolver.concreteIndexNames(state, request.indicesOptions(), request.indices()); + MetaData metaData = state.getMetaData(); + + List indicesToUpgrade = Arrays.stream(concreteIndices) + .filter(indexName -> shouldUpgrade.test(metaData.index(indexName))) + .collect(Collectors.toList()); + + // All the internal indices are up to date + if (indicesToUpgrade.isEmpty()) { + listener.onResponse(new AcknowledgedResponse(true)); + return; + } + + IndexNameAndAliasProvider indexNameAndAliasProvider = new IndexNameAndAliasProvider(indicesToUpgrade, metaData); + Exception validationException = indexNameAndAliasProvider.validate(metaData, shouldUpgrade); + if (validationException != null) { + listener.onFailure(validationException); + return; + } + + // <7> Now that we have deleted the old indices, we are complete, alert the user + ActionListener deleteIndicesListener = ActionListener.wrap( + listener::onResponse, + error -> { + String msg = "Failed to delete old indices: " + Strings.collectionToCommaDelimitedString(indicesToUpgrade); + logger.error(msg, error); + listener.onFailure(new ElasticsearchException(msg, error)); + } + ); + + // <6> Now that aliases are moved, need to delete the old indices + ActionListener readAliasListener = ActionListener.wrap( + resp -> deleteOldIndices(client, indicesToUpgrade, deleteIndicesListener), + error -> { + String msg = "Failed adjusting aliases from old indices to new."; + logger.error(msg, error); + listener.onFailure(new ElasticsearchException(msg, error)); + } + ); + + // <5> Documents are now reindexed, time to move read aliases + ActionListener reindexListener = ActionListener.wrap( + resp -> + // Need to make indices writable again so that the aliases can be removed from them + removeReadOnlyBlock(client, indicesToUpgrade, + ActionListener.wrap( + rrob -> adjustAliases(client, + indexNameAndAliasProvider.oldIndicesWithReadAliases(), + indexNameAndAliasProvider.newReadIndicesWithReadAliases(), + readAliasListener), + rrobFailure -> { + String msg = "Failed making old indices writable again so that aliases can be moved."; + logger.error(msg, rrobFailure); + listener.onFailure(new ElasticsearchException(msg, rrobFailure)); + }) + ), + error -> { + logger.error("Failed to reindex old read-only indices", error); + removeReadOnlyBlock(client, indicesToUpgrade, ActionListener.wrap( + empty -> listener.onFailure(error), + removeReadOnlyBlockError -> { + String msg = "Failed making old indices read/write again after failing to reindex: " + error.getMessage(); + logger.error(msg, removeReadOnlyBlockError); + listener.onFailure(new ElasticsearchException(msg, removeReadOnlyBlockError)); + } + )); + } + ); + + // <4> Old indexes are now readOnly, Time to reindex + ActionListener readOnlyListener = ActionListener.wrap( + ack -> reindexOldReadIndicesToNewIndices(client, indexNameAndAliasProvider.needsReindex(), request, reindexListener), + listener::onFailure + ); + + // <3> Set old indices to readOnly + ActionListener writeAliasesMovedListener = ActionListener.wrap( + resp -> setReadOnlyBlock(client, indicesToUpgrade, readOnlyListener), + listener::onFailure + ); + + // <2> Move write index alias to new write indices + ActionListener createWriteIndicesAndSetReadAliasListener = ActionListener.wrap( + resp -> adjustAliases(client, + indexNameAndAliasProvider.oldIndicesWithWriteAliases(), + indexNameAndAliasProvider.newWriteIndicesWithWriteAliases(), + writeAliasesMovedListener), + listener::onFailure + ); + + // <1> Create the new write indices and set the read aliases to include them + createNewWriteIndicesIfNecessary(client, metaData, indexNameAndAliasProvider.newWriteIndices(), + ActionListener.wrap( + indicesCreated -> adjustAliases(client, + Collections.emptyMap(), + indexNameAndAliasProvider.newWriteIndicesWithReadAliases(), + createWriteIndicesAndSetReadAliasListener), + listener::onFailure + )); + + } catch (Exception e) { + listener.onFailure(e); + } + + } + + private void createNewWriteIndicesIfNecessary(Client client, + MetaData metaData, + Collection newWriteIndices, + ActionListener createIndexListener) { + TypedChainTaskExecutor chainTaskExecutor = + new TypedChainTaskExecutor<>( + client.threadPool().executor(executor), + (createIndexResponse -> true), //We always want to complete all our tasks + (exception -> + // Short circuit execution IF the exception is NOT a ResourceAlreadyExistsException + // This should be rare, as it requires the index to be created between our previous check and this exception + exception instanceof ResourceAlreadyExistsException == false + )); + newWriteIndices.forEach((index) -> { + // If the index already exists, don't try and created it + // We have already verified that IF this index exists, that it does not require upgrading + // So, if it was created between that check and this one, we can assume it is the correct version as it was JUST created + if (metaData.hasIndex(index) == false) { + CreateIndexRequest request = new CreateIndexRequest(index); + chainTaskExecutor.add(listener -> + executeAsyncWithOrigin(client.threadPool().getThreadContext(), + ML_ORIGIN, + request, + listener, + client.admin().indices()::create)); + } + }); + + chainTaskExecutor.execute(ActionListener.wrap( + createIndexResponses -> createIndexListener.onResponse(true), + createIndexListener::onFailure + )); + } + + /** + * Makes the indices readonly if it's not set as a readonly yet + */ + private void setReadOnlyBlock(Client client, List indices, ActionListener listener) { + Settings settings = Settings.builder().put(IndexMetaData.INDEX_READ_ONLY_SETTING.getKey(), true).build(); + UpdateSettingsRequest request = new UpdateSettingsRequest(indices.toArray(new String[0])); + request.settings(settings); + executeAsyncWithOrigin(client.threadPool().getThreadContext(), + ML_ORIGIN, + request, + listener, + client.admin().indices()::updateSettings); + } + + private void removeReadOnlyBlock(Client client, List indices, + ActionListener listener) { + Settings settings = Settings.builder().put(IndexMetaData.INDEX_READ_ONLY_SETTING.getKey(), false).build(); + UpdateSettingsRequest request = new UpdateSettingsRequest(indices.toArray(new String[0])); + request.settings(settings); + executeAsyncWithOrigin(client.threadPool().getThreadContext(), + ML_ORIGIN, + request, + listener, + client.admin().indices()::updateSettings); + } + + private void reindexOldReadIndicesToNewIndices(Client client, + Map reindexIndices, + MlUpgradeAction.Request request, + ActionListener listener) { + TypedChainTaskExecutor chainTaskExecutor = + new TypedChainTaskExecutor<>( + client.threadPool().executor(executor), + (createIndexResponse) -> { // If there are errors in the reindex, we should stop + Tuple status = getStatusAndCause(createIndexResponse); + return status.v1().equals(RestStatus.OK); + }, + (exception -> true)); // Short circuit and call onFailure for any exception + + List newIndices = new ArrayList<>(reindexIndices.size()); + reindexIndices.forEach((oldIndex, newIndex) -> { + ReindexRequest reindexRequest = new ReindexRequest(); + reindexRequest.setSourceBatchSize(request.getReindexBatchSize()); + reindexRequest.setSourceIndices(oldIndex); + reindexRequest.setDestIndex(newIndex); + reindexRequest.setSourceDocTypes(ElasticsearchMappings.DOC_TYPE); + reindexRequest.setDestDocType(ElasticsearchMappings.DOC_TYPE); + // Don't worry if these indices already exist, we validated settings.index.created.version earlier + reindexRequest.setAbortOnVersionConflict(false); + // If the document exists already in the new index, don't want to update or overwrite as we are pulling from "old data" + reindexRequest.setDestOpType(DocWriteRequest.OpType.CREATE.getLowercase()); + newIndices.add(newIndex); + chainTaskExecutor.add(chainedListener -> + executeAsyncWithOrigin(client, + ML_ORIGIN, + ReindexAction.INSTANCE, + reindexRequest, + chainedListener)); + }); + + chainTaskExecutor.execute(ActionListener.wrap( + bulkScrollingResponses -> { + BulkByScrollResponse response = bulkScrollingResponses.get(bulkScrollingResponses.size() - 1); + Tuple status = getStatusAndCause(response); + if (status.v1().equals(RestStatus.OK)) { + listener.onResponse(true); + } else { + logger.error("Failed to reindex old results indices.", status.v2()); + listener.onFailure(new ElasticsearchException("Failed to reindex old results indices.",status.v2())); + } + }, + failure -> { + List createdIndices = newIndices.subList(0, chainTaskExecutor.getCollectedResponses().size()); + logger.error( + "Failed to reindex all old read indices. Successfully reindexed: [" + + Strings.collectionToCommaDelimitedString(createdIndices) + "]", + failure); + listener.onFailure(failure); + } + )); + + } + + private void deleteOldIndices(Client client, + List oldIndices, + ActionListener deleteIndicesListener) { + DeleteIndexRequest request = new DeleteIndexRequest(oldIndices.toArray(new String[0])); + request.indicesOptions(IndicesOptions.LENIENT_EXPAND_OPEN); + executeAsyncWithOrigin(client.threadPool().getThreadContext(), + ML_ORIGIN, + request, + deleteIndicesListener, + client.admin().indices()::delete); + } + + private void adjustAliases(Client client, + Map> oldAliases, + Map> newAliases, + ActionListener indicesAliasListener) { + IndicesAliasesRequest indicesAliasesRequest = new IndicesAliasesRequest(); + oldAliases.forEach((oldIndex, aliases) -> + { + if (aliases.isEmpty() == false) { //if the aliases are empty, that means there are none to remove + indicesAliasesRequest.addAliasAction(IndicesAliasesRequest + .AliasActions + .remove() + .index(oldIndex) + .aliases(aliases.stream().map(Alias::name).toArray(String[]::new))); + } + } + ); + newAliases.forEach((newIndex, aliases) -> + aliases.forEach(alias -> { + IndicesAliasesRequest.AliasActions action = IndicesAliasesRequest.AliasActions.add().index(newIndex); + if (alias.filter() != null) { + action.filter(alias.filter()); + } + action.alias(alias.name()); + indicesAliasesRequest.addAliasAction(action); + }) + ); + executeAsyncWithOrigin(client.threadPool().getThreadContext(), + ML_ORIGIN, + indicesAliasesRequest, + indicesAliasListener, + client.admin().indices()::aliases); + } + + + private static class IndexNameAndAliasProvider { + + private final List oldIndices; + private final Map> writeAliases = new HashMap<>(); + private final Map> readAliases = new HashMap<>(); + + private IndexNameAndAliasProvider(List oldIndices, MetaData metaData) { + this.oldIndices = oldIndices; + oldIndices.forEach(index -> { + IndexMetaData indexMetaData = metaData.index(index); + List writes = new ArrayList<>(); + List reads = new ArrayList<>(); + indexMetaData.getAliases().forEach(aliasCursor -> { + Alias alias = new Alias(aliasCursor.value.alias()); + if (aliasCursor.value.filteringRequired()) { + alias.filter(aliasCursor.value.getFilter().string()); //Set the read alias jobId filter + } + if (alias.name().contains(".write-")) { + writes.add(alias); + } else { + reads.add(alias); + } + }); + + writeAliases.put(index, writes); + readAliases.put(index, reads); + }); + } + + private Exception validate(MetaData metaData, Predicate shouldUpgrade) { + for (String index : oldIndices) { + String newWriteName = newWriteName(index); + // If the "new" indices exist, either they were created from a previous run of the upgrade process or the end user + if (validNewIndex(metaData, newWriteName, shouldUpgrade) == false) { + return new IllegalStateException("Index [" + newWriteName + "] already exists and is not the current version."); + } + + String newReadName = newReadName(index); + if (validNewIndex(metaData, newReadName, shouldUpgrade) == false) { + return new IllegalStateException("Index [" + newReadName + "] already exists and is not the current version."); + } + } + return null; + } + + private String newReadName(String oldIndexName) { + return oldIndexName + "-" + INDEX_VERSION + "r"; + } + + private String newWriteName(String oldIndexName) { + return oldIndexName + "-" + INDEX_VERSION; + } + + private List newWriteIndices() { + return oldIndices.stream().map(this::newWriteName).collect(Collectors.toList()); + } + + private List readAliases(String oldIndex) { + return readAliases.get(oldIndex); + } + + private List writeAliases(String oldIndex) { + return writeAliases.get(oldIndex); + } + + private Map> newWriteIndicesWithReadAliases() { + return oldIndices.stream().collect(Collectors.toMap(this::newWriteName, this::readAliases)); + } + + private Map> oldIndicesWithWriteAliases() { + return writeAliases; + } + + private Map> newWriteIndicesWithWriteAliases() { + return oldIndices.stream().collect(Collectors.toMap(this::newWriteName, this::writeAliases)); + } + + private Map> oldIndicesWithReadAliases() { + return readAliases; + } + + private Map> newReadIndicesWithReadAliases() { + return oldIndices.stream().collect(Collectors.toMap(this::newReadName, this::readAliases)); + } + + private Map needsReindex() { + return oldIndices.stream().collect(Collectors.toMap(Function.identity(), this::newReadName)); + } + } +} diff --git a/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/action/TransportFinalizeJobExecutionAction.java b/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/action/TransportFinalizeJobExecutionAction.java index d6c03d6c93fbf..9d76844121cbb 100644 --- a/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/action/TransportFinalizeJobExecutionAction.java +++ b/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/action/TransportFinalizeJobExecutionAction.java @@ -26,7 +26,7 @@ import org.elasticsearch.xpack.core.ml.job.persistence.AnomalyDetectorsIndex; import org.elasticsearch.xpack.core.ml.job.persistence.ElasticsearchMappings; import org.elasticsearch.xpack.ml.MachineLearning; -import org.elasticsearch.xpack.ml.utils.ChainTaskExecutor; +import org.elasticsearch.xpack.ml.utils.VoidChainTaskExecutor; import java.util.Collections; import java.util.Date; @@ -65,7 +65,7 @@ protected void masterOperation(FinalizeJobExecutionAction.Request request, Clust String jobIdString = String.join(",", request.getJobIds()); logger.debug("finalizing jobs [{}]", jobIdString); - ChainTaskExecutor chainTaskExecutor = new ChainTaskExecutor(threadPool.executor( + VoidChainTaskExecutor voidChainTaskExecutor = new VoidChainTaskExecutor(threadPool.executor( MachineLearning.UTILITY_THREAD_POOL_NAME), true); Map update = Collections.singletonMap(Job.FINISHED_TIME.getPreferredName(), new Date()); @@ -77,7 +77,7 @@ protected void masterOperation(FinalizeJobExecutionAction.Request request, Clust updateRequest.doc(update); updateRequest.setRefreshPolicy(WriteRequest.RefreshPolicy.IMMEDIATE); - chainTaskExecutor.add(chainedListener -> { + voidChainTaskExecutor.add(chainedListener -> { executeAsyncWithOrigin(client, ML_ORIGIN, UpdateAction.INSTANCE, updateRequest, ActionListener.wrap( updateResponse -> chainedListener.onResponse(null), chainedListener::onFailure @@ -85,8 +85,8 @@ protected void masterOperation(FinalizeJobExecutionAction.Request request, Clust }); } - chainTaskExecutor.execute(ActionListener.wrap( - aVoid -> { + voidChainTaskExecutor.execute(ActionListener.wrap( + aVoids -> { logger.debug("finalized job [{}]", jobIdString); listener.onResponse(new AcknowledgedResponse(true)); }, diff --git a/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/action/TransportMlUpgradeAction.java b/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/action/TransportMlUpgradeAction.java new file mode 100644 index 0000000000000..2b676277aa690 --- /dev/null +++ b/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/action/TransportMlUpgradeAction.java @@ -0,0 +1,79 @@ +/* + * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one + * or more contributor license agreements. Licensed under the Elastic License; + * you may not use this file except in compliance with the Elastic License. + */ +package org.elasticsearch.xpack.ml.action; + +import org.elasticsearch.action.ActionListener; +import org.elasticsearch.action.support.ActionFilters; +import org.elasticsearch.action.support.master.AcknowledgedResponse; +import org.elasticsearch.action.support.master.TransportMasterNodeReadAction; +import org.elasticsearch.client.Client; +import org.elasticsearch.client.ParentTaskAssigningClient; +import org.elasticsearch.cluster.ClusterState; +import org.elasticsearch.cluster.block.ClusterBlockException; +import org.elasticsearch.cluster.block.ClusterBlockLevel; +import org.elasticsearch.cluster.metadata.IndexNameExpressionResolver; +import org.elasticsearch.cluster.service.ClusterService; +import org.elasticsearch.common.inject.Inject; +import org.elasticsearch.tasks.Task; +import org.elasticsearch.tasks.TaskId; +import org.elasticsearch.threadpool.ThreadPool; +import org.elasticsearch.transport.TransportService; +import org.elasticsearch.xpack.core.ml.action.MlUpgradeAction; +import org.elasticsearch.xpack.ml.ResultsIndexUpgradeService; + +import static org.elasticsearch.xpack.ml.ResultsIndexUpgradeService.wasIndexCreatedInCurrentMajorVersion; + +public class TransportMlUpgradeAction + extends TransportMasterNodeReadAction { + + private final Client client; + private final ResultsIndexUpgradeService resultsIndexUpgradeService; + + @Inject + public TransportMlUpgradeAction(TransportService transportService, ClusterService clusterService, + ThreadPool threadPool, ActionFilters actionFilters, Client client, + IndexNameExpressionResolver indexNameExpressionResolver) { + super(MlUpgradeAction.NAME, transportService, clusterService, threadPool, + actionFilters, MlUpgradeAction.Request::new, indexNameExpressionResolver); + this.client = client; + this.resultsIndexUpgradeService = new ResultsIndexUpgradeService(indexNameExpressionResolver, + executor(), + indexMetadata -> wasIndexCreatedInCurrentMajorVersion(indexMetadata) == false); + } + + @Override + protected void masterOperation(Task task, MlUpgradeAction.Request request, ClusterState state, + ActionListener listener) { + TaskId taskId = new TaskId(clusterService.localNode().getId(), task.getId()); + ParentTaskAssigningClient parentAwareClient = new ParentTaskAssigningClient(client, taskId); + try { + resultsIndexUpgradeService.upgrade(parentAwareClient, request, state, listener); + } catch (Exception e) { + listener.onFailure(e); + } + } + + @Override + protected final void masterOperation(MlUpgradeAction.Request request, ClusterState state, + ActionListener listener) { + throw new UnsupportedOperationException("the task parameter is required"); + } + + @Override + protected String executor() { + return ThreadPool.Names.SAME; + } + + @Override + protected AcknowledgedResponse newResponse() { + return new AcknowledgedResponse(); + } + + @Override + protected ClusterBlockException checkBlock(MlUpgradeAction.Request request, ClusterState state) { + return state.blocks().globalBlockedException(ClusterBlockLevel.METADATA_READ); + } +} diff --git a/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/job/JobManager.java b/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/job/JobManager.java index 33047c1fca39a..53559aee4701b 100644 --- a/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/job/JobManager.java +++ b/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/job/JobManager.java @@ -55,7 +55,7 @@ import org.elasticsearch.xpack.ml.job.persistence.JobResultsProvider; import org.elasticsearch.xpack.ml.job.process.autodetect.UpdateParams; import org.elasticsearch.xpack.ml.notifications.Auditor; -import org.elasticsearch.xpack.ml.utils.ChainTaskExecutor; +import org.elasticsearch.xpack.ml.utils.VoidChainTaskExecutor; import java.io.IOException; import java.util.ArrayList; @@ -397,16 +397,16 @@ private void postJobUpdate(UpdateJobAction.Request request, Job updatedJob, Acti } private void validate(Job job, JobUpdate jobUpdate, ActionListener handler) { - ChainTaskExecutor chainTaskExecutor = new ChainTaskExecutor(client.threadPool().executor( + VoidChainTaskExecutor voidChainTaskExecutor = new VoidChainTaskExecutor(client.threadPool().executor( MachineLearning.UTILITY_THREAD_POOL_NAME), true); - validateModelSnapshotIdUpdate(job, jobUpdate.getModelSnapshotId(), chainTaskExecutor); - validateAnalysisLimitsUpdate(job, jobUpdate.getAnalysisLimits(), chainTaskExecutor); - chainTaskExecutor.execute(handler); + validateModelSnapshotIdUpdate(job, jobUpdate.getModelSnapshotId(), voidChainTaskExecutor); + validateAnalysisLimitsUpdate(job, jobUpdate.getAnalysisLimits(), voidChainTaskExecutor); + voidChainTaskExecutor.execute(ActionListener.wrap(aVoids -> handler.onResponse(null), handler::onFailure)); } - private void validateModelSnapshotIdUpdate(Job job, String modelSnapshotId, ChainTaskExecutor chainTaskExecutor) { + private void validateModelSnapshotIdUpdate(Job job, String modelSnapshotId, VoidChainTaskExecutor voidChainTaskExecutor) { if (modelSnapshotId != null) { - chainTaskExecutor.add(listener -> { + voidChainTaskExecutor.add(listener -> { jobResultsProvider.getModelSnapshot(job.getId(), modelSnapshotId, newModelSnapshot -> { if (newModelSnapshot == null) { String message = Messages.getMessage(Messages.REST_NO_SUCH_MODEL_SNAPSHOT, modelSnapshotId, @@ -428,12 +428,12 @@ private void validateModelSnapshotIdUpdate(Job job, String modelSnapshotId, Chai } } - private void validateAnalysisLimitsUpdate(Job job, AnalysisLimits newLimits, ChainTaskExecutor chainTaskExecutor) { + private void validateAnalysisLimitsUpdate(Job job, AnalysisLimits newLimits, VoidChainTaskExecutor voidChainTaskExecutor) { if (newLimits == null || newLimits.getModelMemoryLimit() == null) { return; } Long newModelMemoryLimit = newLimits.getModelMemoryLimit(); - chainTaskExecutor.add(listener -> { + voidChainTaskExecutor.add(listener -> { if (isJobOpen(clusterService.state(), job.getId())) { listener.onFailure(ExceptionsHelper.badRequestException("Cannot update " + Job.ANALYSIS_LIMITS.getPreferredName() + " while the job is open")); diff --git a/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/rest/results/RestUpgradeMlAction.java b/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/rest/results/RestUpgradeMlAction.java new file mode 100644 index 0000000000000..cad82ce325c27 --- /dev/null +++ b/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/rest/results/RestUpgradeMlAction.java @@ -0,0 +1,76 @@ +/* + * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one + * or more contributor license agreements. Licensed under the Elastic License; + * you may not use this file except in compliance with the Elastic License. + */ +package org.elasticsearch.xpack.ml.rest.results; + +import org.apache.logging.log4j.LogManager; +import org.elasticsearch.client.node.NodeClient; +import org.elasticsearch.common.logging.DeprecationLogger; +import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.common.xcontent.XContentBuilder; +import org.elasticsearch.common.xcontent.XContentParser; +import org.elasticsearch.rest.BaseRestHandler; +import org.elasticsearch.rest.BytesRestResponse; +import org.elasticsearch.rest.RestController; +import org.elasticsearch.rest.RestRequest; +import org.elasticsearch.rest.RestStatus; +import org.elasticsearch.rest.action.RestToXContentListener; +import org.elasticsearch.tasks.LoggingTaskListener; +import org.elasticsearch.tasks.Task; +import org.elasticsearch.xpack.core.ml.action.MlUpgradeAction; +import org.elasticsearch.xpack.ml.MachineLearning; + +import java.io.IOException; + +import static org.elasticsearch.rest.RestRequest.Method.POST; + +public class RestUpgradeMlAction extends BaseRestHandler { + + private static final DeprecationLogger deprecationLogger = + new DeprecationLogger(LogManager.getLogger(RestUpgradeMlAction.class)); + + public RestUpgradeMlAction(Settings settings, RestController controller) { + super(settings); + controller.registerWithDeprecatedHandler( + POST, + MachineLearning.BASE_PATH + "_upgrade", + this, + POST, + MachineLearning.PRE_V7_BASE_PATH + "_upgrade", + deprecationLogger); + } + + @Override + public String getName() { + return "xpack_ml_upgrade_action"; + } + + @Override + protected RestChannelConsumer prepareRequest(RestRequest restRequest, NodeClient client) throws IOException { + MlUpgradeAction.Request parsedRequest = new MlUpgradeAction.Request(); + if (restRequest.hasContent()) { + XContentParser parser = restRequest.contentParser(); + parsedRequest = MlUpgradeAction.Request.PARSER.apply(parser, null); + } + final MlUpgradeAction.Request upgradeRequest = parsedRequest; + + if (restRequest.paramAsBoolean("wait_for_completion", false)) { + return channel -> client.execute(MlUpgradeAction.INSTANCE, upgradeRequest, new RestToXContentListener<>(channel)); + } else { + upgradeRequest.setShouldStoreResult(true); + + Task task = client.executeLocally(MlUpgradeAction.INSTANCE, upgradeRequest, LoggingTaskListener.instance()); + // Send task description id instead of waiting for the message + return channel -> { + try (XContentBuilder builder = channel.newBuilder()) { + builder.startObject(); + builder.field("task", client.getLocalNodeId() + ":" + task.getId()); + builder.endObject(); + channel.sendResponse(new BytesRestResponse(RestStatus.OK, builder)); + } + }; + } + } +} diff --git a/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/utils/ChainTaskExecutor.java b/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/utils/ChainTaskExecutor.java deleted file mode 100644 index 9a0ddb5dd4add..0000000000000 --- a/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/utils/ChainTaskExecutor.java +++ /dev/null @@ -1,60 +0,0 @@ -/* - * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one - * or more contributor license agreements. Licensed under the Elastic License; - * you may not use this file except in compliance with the Elastic License. - */ -package org.elasticsearch.xpack.ml.utils; - -import org.elasticsearch.action.ActionListener; -import org.elasticsearch.common.util.concurrent.AbstractRunnable; - -import java.util.LinkedList; -import java.util.Objects; -import java.util.concurrent.ExecutorService; - -/** - * A utility that allows chained (serial) execution of a number of tasks - * in async manner. - */ -public class ChainTaskExecutor { - - public interface ChainTask { - void run(ActionListener listener); - } - - private final ExecutorService executorService; - private final boolean shortCircuit; - private final LinkedList tasks = new LinkedList<>(); - - public ChainTaskExecutor(ExecutorService executorService, boolean shortCircuit) { - this.executorService = Objects.requireNonNull(executorService); - this.shortCircuit = shortCircuit; - } - - public synchronized void add(ChainTask task) { - tasks.add(task); - } - - public synchronized void execute(ActionListener listener) { - if (tasks.isEmpty()) { - listener.onResponse(null); - return; - } - ChainTask task = tasks.pop(); - executorService.execute(new AbstractRunnable() { - @Override - public void onFailure(Exception e) { - if (shortCircuit) { - listener.onFailure(e); - } else { - execute(listener); - } - } - - @Override - protected void doRun() { - task.run(ActionListener.wrap(nullValue -> execute(listener), this::onFailure)); - } - }); - } -} diff --git a/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/utils/TypedChainTaskExecutor.java b/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/utils/TypedChainTaskExecutor.java new file mode 100644 index 0000000000000..5af9c53649853 --- /dev/null +++ b/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/utils/TypedChainTaskExecutor.java @@ -0,0 +1,125 @@ +/* + * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one + * or more contributor license agreements. Licensed under the Elastic License; + * you may not use this file except in compliance with the Elastic License. + */ +package org.elasticsearch.xpack.ml.utils; + +import org.elasticsearch.action.ActionListener; +import org.elasticsearch.common.util.concurrent.AbstractRunnable; + +import java.util.ArrayList; +import java.util.Collections; +import java.util.LinkedList; +import java.util.List; +import java.util.Objects; +import java.util.concurrent.ExecutorService; +import java.util.function.Predicate; + +/** + * A utility that allows chained (serial) execution of a number of tasks + * in async manner. + */ +public class TypedChainTaskExecutor { + + public interface ChainTask { + void run(ActionListener listener); + } + + private final ExecutorService executorService; + private final LinkedList> tasks = new LinkedList<>(); + private final Predicate failureShortCircuitPredicate; + private final Predicate continuationPredicate; + private final List collectedResponses; + + /** + * Creates a new TypedChainTaskExecutor. + * Each chainedTask is executed in order serially and after each execution the continuationPredicate is tested. + * + * On failures the failureShortCircuitPredicate is tested. + * + * @param executorService The service where to execute the tasks + * @param continuationPredicate The predicate to test on whether to execute the next task or not. + * {@code true} means continue on to the next task. + * Must be able to handle null values. + * @param failureShortCircuitPredicate The predicate on whether to short circuit execution on a give exception. + * {@code true} means that no more tasks should execute and the the listener::onFailure should be + * called. + */ + public TypedChainTaskExecutor(ExecutorService executorService, + Predicate continuationPredicate, + Predicate failureShortCircuitPredicate) { + this.executorService = Objects.requireNonNull(executorService); + this.continuationPredicate = continuationPredicate; + this.failureShortCircuitPredicate = failureShortCircuitPredicate; + this.collectedResponses = new ArrayList<>(); + } + + public synchronized void add(ChainTask task) { + tasks.add(task); + } + + private synchronized void execute(T previousValue, ActionListener> listener) { + collectedResponses.add(previousValue); + if (continuationPredicate.test(previousValue)) { + if (tasks.isEmpty()) { + listener.onResponse(Collections.unmodifiableList(new ArrayList<>(collectedResponses))); + return; + } + ChainTask task = tasks.pop(); + executorService.execute(new AbstractRunnable() { + @Override + public void onFailure(Exception e) { + if (failureShortCircuitPredicate.test(e)) { + listener.onFailure(e); + } else { + execute(null, listener); + } + } + + @Override + protected void doRun() { + task.run(ActionListener.wrap(value -> execute(value, listener), this::onFailure)); + } + }); + } else { + listener.onResponse(Collections.unmodifiableList(new ArrayList<>(collectedResponses))); + } + } + + /** + * Execute all the chained tasks serially, notify listener when completed + * + * @param listener The ActionListener to notify when all executions have been completed, + * or when no further tasks should be executed. + * The resulting list COULD contain null values depending on if execution is continued + * on exceptions or not. + */ + public synchronized void execute(ActionListener> listener) { + if (tasks.isEmpty()) { + listener.onResponse(Collections.emptyList()); + return; + } + collectedResponses.clear(); + ChainTask task = tasks.pop(); + executorService.execute(new AbstractRunnable() { + @Override + public void onFailure(Exception e) { + if (failureShortCircuitPredicate.test(e)) { + listener.onFailure(e); + } else { + execute(null, listener); + } + } + + @Override + protected void doRun() { + task.run(ActionListener.wrap(value -> execute(value, listener), this::onFailure)); + } + }); + } + + public synchronized List getCollectedResponses() { + return Collections.unmodifiableList(new ArrayList<>(collectedResponses)); + } +} diff --git a/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/utils/VoidChainTaskExecutor.java b/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/utils/VoidChainTaskExecutor.java new file mode 100644 index 0000000000000..8351c0a81aaf6 --- /dev/null +++ b/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/utils/VoidChainTaskExecutor.java @@ -0,0 +1,26 @@ +/* + * Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one + * or more contributor license agreements. Licensed under the Elastic License; + * you may not use this file except in compliance with the Elastic License. + */ +package org.elasticsearch.xpack.ml.utils; + +import java.util.concurrent.ExecutorService; +import java.util.function.Predicate; + +/** + * A utility that allows chained (serial) execution of a number of tasks + * in async manner. + */ +public class VoidChainTaskExecutor extends TypedChainTaskExecutor { + + public VoidChainTaskExecutor(ExecutorService executorService, boolean shortCircuit) { + this(executorService, (a) -> true, (e) -> shortCircuit); + } + + VoidChainTaskExecutor(ExecutorService executorService, + Predicate continuationPredicate, + Predicate failureShortCircuitPredicate) { + super(executorService, continuationPredicate, failureShortCircuitPredicate); + } +} diff --git a/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/utils/ChainTaskExecutorTests.java b/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/utils/VoidChainTaskExecutorTests.java similarity index 62% rename from x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/utils/ChainTaskExecutorTests.java rename to x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/utils/VoidChainTaskExecutorTests.java index 87b83852ff56c..44bf4cf75aa13 100644 --- a/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/utils/ChainTaskExecutorTests.java +++ b/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/utils/VoidChainTaskExecutorTests.java @@ -19,7 +19,7 @@ import static org.hamcrest.Matchers.contains; import static org.hamcrest.Matchers.equalTo; -public class ChainTaskExecutorTests extends ESTestCase { +public class VoidChainTaskExecutorTests extends ESTestCase { private final ThreadPool threadPool = new TestThreadPool(getClass().getName()); private final CountDownLatch latch = new CountDownLatch(1); @@ -36,18 +36,18 @@ public void tearDown() throws Exception { public void testExecute() throws InterruptedException { final List strings = new ArrayList<>(); - ActionListener finalListener = createBlockingListener(() -> strings.add("last"), e -> fail()); - ChainTaskExecutor chainTaskExecutor = new ChainTaskExecutor(threadPool.generic(), false); - chainTaskExecutor.add(listener -> { + ActionListener> finalListener = createBlockingListener(() -> strings.add("last"), e -> fail()); + VoidChainTaskExecutor voidChainTaskExecutor = new VoidChainTaskExecutor(threadPool.generic(), false); + voidChainTaskExecutor.add(listener -> { strings.add("first"); listener.onResponse(null); }); - chainTaskExecutor.add(listener -> { + voidChainTaskExecutor.add(listener -> { strings.add("second"); listener.onResponse(null); }); - chainTaskExecutor.execute(finalListener); + voidChainTaskExecutor.execute(finalListener); latch.await(); @@ -56,22 +56,22 @@ public void testExecute() throws InterruptedException { public void testExecute_GivenSingleFailureAndShortCircuit() throws InterruptedException { final List strings = new ArrayList<>(); - ActionListener finalListener = createBlockingListener(() -> fail(), + ActionListener> finalListener = createBlockingListener(() -> fail(), e -> assertThat(e.getMessage(), equalTo("some error"))); - ChainTaskExecutor chainTaskExecutor = new ChainTaskExecutor(threadPool.generic(), true); - chainTaskExecutor.add(listener -> { + VoidChainTaskExecutor voidChainTaskExecutor = new VoidChainTaskExecutor(threadPool.generic(), true); + voidChainTaskExecutor.add(listener -> { strings.add("before"); listener.onResponse(null); }); - chainTaskExecutor.add(listener -> { + voidChainTaskExecutor.add(listener -> { throw new RuntimeException("some error"); }); - chainTaskExecutor.add(listener -> { + voidChainTaskExecutor.add(listener -> { strings.add("after"); listener.onResponse(null); }); - chainTaskExecutor.execute(finalListener); + voidChainTaskExecutor.execute(finalListener); latch.await(); @@ -80,21 +80,21 @@ public void testExecute_GivenSingleFailureAndShortCircuit() throws InterruptedEx public void testExecute_GivenMultipleFailuresAndShortCircuit() throws InterruptedException { final List strings = new ArrayList<>(); - ActionListener finalListener = createBlockingListener(() -> fail(), + ActionListener> finalListener = createBlockingListener(() -> fail(), e -> assertThat(e.getMessage(), equalTo("some error 1"))); - ChainTaskExecutor chainTaskExecutor = new ChainTaskExecutor(threadPool.generic(), true); - chainTaskExecutor.add(listener -> { + VoidChainTaskExecutor voidChainTaskExecutor = new VoidChainTaskExecutor(threadPool.generic(), true); + voidChainTaskExecutor.add(listener -> { strings.add("before"); listener.onResponse(null); }); - chainTaskExecutor.add(listener -> { + voidChainTaskExecutor.add(listener -> { throw new RuntimeException("some error 1"); }); - chainTaskExecutor.add(listener -> { + voidChainTaskExecutor.add(listener -> { throw new RuntimeException("some error 2"); }); - chainTaskExecutor.execute(finalListener); + voidChainTaskExecutor.execute(finalListener); latch.await(); @@ -103,21 +103,21 @@ public void testExecute_GivenMultipleFailuresAndShortCircuit() throws Interrupte public void testExecute_GivenFailureAndNoShortCircuit() throws InterruptedException { final List strings = new ArrayList<>(); - ActionListener finalListener = createBlockingListener(() -> strings.add("last"), e -> fail()); - ChainTaskExecutor chainTaskExecutor = new ChainTaskExecutor(threadPool.generic(), false); - chainTaskExecutor.add(listener -> { + ActionListener> finalListener = createBlockingListener(() -> strings.add("last"), e -> fail()); + VoidChainTaskExecutor voidChainTaskExecutor = new VoidChainTaskExecutor(threadPool.generic(), false); + voidChainTaskExecutor.add(listener -> { strings.add("before"); listener.onResponse(null); }); - chainTaskExecutor.add(listener -> { + voidChainTaskExecutor.add(listener -> { throw new RuntimeException("some error"); }); - chainTaskExecutor.add(listener -> { + voidChainTaskExecutor.add(listener -> { strings.add("after"); listener.onResponse(null); }); - chainTaskExecutor.execute(finalListener); + voidChainTaskExecutor.execute(finalListener); latch.await(); @@ -126,17 +126,17 @@ public void testExecute_GivenFailureAndNoShortCircuit() throws InterruptedExcept public void testExecute_GivenNoTasksAdded() throws InterruptedException { final List strings = new ArrayList<>(); - ActionListener finalListener = createBlockingListener(() -> strings.add("last"), e -> fail()); - ChainTaskExecutor chainTaskExecutor = new ChainTaskExecutor(threadPool.generic(), false); + ActionListener> finalListener = createBlockingListener(() -> strings.add("last"), e -> fail()); + VoidChainTaskExecutor voidChainTaskExecutor = new VoidChainTaskExecutor(threadPool.generic(), false); - chainTaskExecutor.execute(finalListener); + voidChainTaskExecutor.execute(finalListener); latch.await(); assertThat(strings, contains("last")); } - private ActionListener createBlockingListener(Runnable runnable, Consumer errorHandler) { + private ActionListener> createBlockingListener(Runnable runnable, Consumer errorHandler) { return ActionListener.wrap(nullValue -> { runnable.run(); latch.countDown(); @@ -145,4 +145,4 @@ private ActionListener createBlockingListener(Runnable runnable, Consumer< latch.countDown(); }); } -} \ No newline at end of file +} diff --git a/x-pack/plugin/src/test/resources/rest-api-spec/api/ml.upgrade.json b/x-pack/plugin/src/test/resources/rest-api-spec/api/ml.upgrade.json new file mode 100644 index 0000000000000..b67b125bb692a --- /dev/null +++ b/x-pack/plugin/src/test/resources/rest-api-spec/api/ml.upgrade.json @@ -0,0 +1,21 @@ +{ + "ml.upgrade": { + "documentation": "TODO", + "methods": [ "POST" ], + "url": { + "path": "/_ml/_upgrade", + "paths": [ "/_ml/_upgrade" ], + "params": { + "wait_for_completion": { + "type": "boolean", + "description": "Should this request wait until the operation has completed before returning", + "default": false + } + } + }, + "body": { + "description" : "Upgrade options", + "required" : false + } + } +} diff --git a/x-pack/plugin/src/test/resources/rest-api-spec/test/ml/ml_upgrade.yml b/x-pack/plugin/src/test/resources/rest-api-spec/test/ml/ml_upgrade.yml new file mode 100644 index 0000000000000..ee1f9f77f9325 --- /dev/null +++ b/x-pack/plugin/src/test/resources/rest-api-spec/test/ml/ml_upgrade.yml @@ -0,0 +1,70 @@ +setup: + - skip: + features: headers + - do: + headers: + Authorization: "Basic eF9wYWNrX3Jlc3RfdXNlcjp4LXBhY2stdGVzdC1wYXNzd29yZA==" # run as x_pack_rest_user, i.e. the test setup superuser + ml.put_job: + job_id: jobs-upgrade-results + body: > + { + "analysis_config" : { + "detectors" :[{"function":"metric","field_name":"responsetime","by_field_name":"airline"}] + }, + "data_description" : { + "format":"xcontent", + "time_field":"time" + } + } + + - do: + headers: + Authorization: "Basic eF9wYWNrX3Jlc3RfdXNlcjp4LXBhY2stdGVzdC1wYXNzd29yZA==" # run as x_pack_rest_user, i.e. the test setup superuser + Content-Type: application/json + index: + index: .ml-anomalies-jobs-upgrade-results + type: doc + id: "jobs-upgrade-results_1464739200000_1" + body: + { + "job_id": "jobs-upgrade-results", + "result_type": "bucket", + "timestamp": "2016-06-01T00:00:00Z", + "anomaly_score": 90.0, + "bucket_span":1 + } + + - do: + headers: + Authorization: "Basic eF9wYWNrX3Jlc3RfdXNlcjp4LXBhY2stdGVzdC1wYXNzd29yZA==" # run as x_pack_rest_user, i.e. the test setup superuser + indices.refresh: + index: .ml-anomalies-jobs-upgrade-results + +--- +"Upgrade results when there is nothing to upgrade": + - do: + ml.upgrade: + wait_for_completion: true + + - match: { acknowledged: true } + + - do: + indices.exists: + index: .ml-anomalies-shared + + - is_true: '' +--- +"Upgrade results when there is nothing to upgrade not waiting for results": + - do: + ml.upgrade: + wait_for_completion: false + + - match: {task: '/.+:\d+/'} + - set: {task: task} + + - do: + tasks.get: + wait_for_completion: true + task_id: $task + - match: {completed: true} + - match: {response.acknowledged: true} diff --git a/x-pack/qa/rolling-upgrade/src/test/resources/rest-api-spec/test/mixed_cluster/80_ml_results_upgrade.yml b/x-pack/qa/rolling-upgrade/src/test/resources/rest-api-spec/test/mixed_cluster/80_ml_results_upgrade.yml new file mode 100644 index 0000000000000..73478be65597e --- /dev/null +++ b/x-pack/qa/rolling-upgrade/src/test/resources/rest-api-spec/test/mixed_cluster/80_ml_results_upgrade.yml @@ -0,0 +1,11 @@ +--- +"Verify jobs exist": + - do: + ml.get_jobs: + job_id: old-cluster-job-to-upgrade + - match: { count: 1 } + + - do: + ml.get_jobs: + job_id: old-cluster-job-to-upgrade-custom + - match: { count: 1 } diff --git a/x-pack/qa/rolling-upgrade/src/test/resources/rest-api-spec/test/old_cluster/80_ml_results_upgrade.yml b/x-pack/qa/rolling-upgrade/src/test/resources/rest-api-spec/test/old_cluster/80_ml_results_upgrade.yml new file mode 100644 index 0000000000000..d21b5e6def61d --- /dev/null +++ b/x-pack/qa/rolling-upgrade/src/test/resources/rest-api-spec/test/old_cluster/80_ml_results_upgrade.yml @@ -0,0 +1,120 @@ +--- +"Put job on the old cluster and post some data": + + - do: + ml.put_job: + job_id: old-cluster-job-to-upgrade + body: > + { + "description":"Cluster upgrade", + "analysis_config" : { + "bucket_span": "60s", + "detectors" :[{"function":"metric","field_name":"responsetime","by_field_name":"airline"}] + }, + "analysis_limits" : { + "model_memory_limit": "50mb" + }, + "data_description" : { + "format":"xcontent", + "time_field":"time", + "time_format":"epoch" + } + } + - match: { job_id: old-cluster-job-to-upgrade } + + - do: + ml.open_job: + job_id: old-cluster-job-to-upgrade + + - do: + ml.post_data: + job_id: old-cluster-job-to-upgrade + body: + - airline: AAL + responsetime: 132.2046 + sourcetype: post-data-job + time: 1403481600 + - airline: JZA + responsetime: 990.4628 + sourcetype: post-data-job + time: 1403481700 + - match: { processed_record_count: 2 } + + - do: + ml.close_job: + job_id: old-cluster-job-to-upgrade + + - do: + ml.get_buckets: + job_id: old-cluster-job-to-upgrade + - match: { count: 1 } + +# Wait for indices to be fully allocated before +# killing the node + - do: + cluster.health: + index: [".ml-state", ".ml-anomalies-shared"] + wait_for_status: green + +--- +"Put job on the old cluster with a custom index": + - do: + ml.put_job: + job_id: old-cluster-job-to-upgrade-custom + body: > + { + "description":"Cluster upgrade", + "analysis_config" : { + "bucket_span": "60s", + "detectors" :[{"function":"metric","field_name":"responsetime","by_field_name":"airline"}] + }, + "analysis_limits" : { + "model_memory_limit": "50mb" + }, + "data_description" : { + "format":"xcontent", + "time_field":"time", + "time_format":"epoch" + }, + "results_index_name": "old-cluster-job-to-upgrade-custom" + } + - match: { job_id: old-cluster-job-to-upgrade-custom } + + - do: + ml.open_job: + job_id: old-cluster-job-to-upgrade-custom + + - do: + ml.post_data: + job_id: old-cluster-job-to-upgrade-custom + body: + - airline: AAL + responsetime: 132.2046 + sourcetype: post-data-job + time: 1403481600 + - airline: JZA + responsetime: 990.4628 + sourcetype: post-data-job + time: 1403481700 + - airline: JZA + responsetime: 423.0000 + sourcetype: post-data-job + time: 1403481800 + - match: { processed_record_count: 3 } + + - do: + ml.close_job: + job_id: old-cluster-job-to-upgrade-custom + + - do: + ml.get_buckets: + job_id: old-cluster-job-to-upgrade-custom + - match: { count: 3 } + +# Wait for indices to be fully allocated before +# killing the node + - do: + cluster.health: + index: [".ml-state", ".ml-anomalies-old-cluster-job-to-upgrade-custom"] + wait_for_status: green + diff --git a/x-pack/qa/rolling-upgrade/src/test/resources/rest-api-spec/test/upgraded_cluster/80_ml_results_upgrade.yml b/x-pack/qa/rolling-upgrade/src/test/resources/rest-api-spec/test/upgraded_cluster/80_ml_results_upgrade.yml new file mode 100644 index 0000000000000..f049b9c073ad8 --- /dev/null +++ b/x-pack/qa/rolling-upgrade/src/test/resources/rest-api-spec/test/upgraded_cluster/80_ml_results_upgrade.yml @@ -0,0 +1,158 @@ +--- +"Migrate results data to latest index binary version": + # Verify that all the results are there and the typical indices exist + - do: + ml.get_buckets: + job_id: old-cluster-job-to-upgrade + - match: { count: 1 } + + - do: + ml.get_buckets: + job_id: old-cluster-job-to-upgrade-custom + - match: { count: 3 } + + - do: + indices.exists: + index: .ml-anomalies-shared + + - is_true: '' + + - do: + indices.get_settings: + index: .ml-anomalies-shared + name: index.version.created + + - match: { \.ml-anomalies-shared.settings.index.version.created: '/6\d+/' } + + - do: + indices.exists: + index: .ml-anomalies-custom-old-cluster-job-to-upgrade-custom + + - is_true: '' + + # Do the upgrade + - do: + ml.upgrade: + wait_for_completion: true + + - match: { acknowledged: true } + + # Verify that old indices are gone + - do: + indices.exists: + index: .ml-anomalies-shared + + - is_false: '' + + - do: + indices.exists: + index: .ml-anomalies-custom-old-cluster-job-to-upgrade-custom + + - is_false: '' + + # Verify that results can still be retrieved + + - do: + indices.refresh: {} + + - do: + ml.get_buckets: + job_id: old-cluster-job-to-upgrade + - match: { count: 1 } + + - do: + ml.get_buckets: + job_id: old-cluster-job-to-upgrade-custom + - match: { count: 3 } + + # Verify the created version is correct + + - do: + indices.get_settings: + index: .ml-anomalies-old-cluster-job-to-upgrade + name: index.version.created + - match: { \.ml-anomalies-shared-7.settings.index.version.created: '/7\d+/' } + - match: { \.ml-anomalies-shared-7r.settings.index.version.created: '/7\d+/' } + + - do: + indices.get_settings: + index: .ml-anomalies-old-cluster-job-to-upgrade-custom + name: index.version.created + - match: { \.ml-anomalies-custom-old-cluster-job-to-upgrade-custom-7.settings.index.version.created: '/7\d+/' } + - match: { \.ml-anomalies-custom-old-cluster-job-to-upgrade-custom-7r.settings.index.version.created: '/7\d+/' } + + # Create a new job to verify that the .ml-anomalies-shared index gets created again without issues + + - do: + ml.put_job: + job_id: upgraded-cluster-job-should-not-upgrade + body: > + { + "description":"Cluster upgrade", + "analysis_config" : { + "bucket_span": "60s", + "detectors" :[{"function":"metric","field_name":"responsetime","by_field_name":"airline"}] + }, + "analysis_limits" : { + "model_memory_limit": "50mb" + }, + "data_description" : { + "format":"xcontent", + "time_field":"time", + "time_format":"epoch" + } + } + - match: { job_id: upgraded-cluster-job-should-not-upgrade } + + - do: + ml.open_job: + job_id: upgraded-cluster-job-should-not-upgrade + + - do: + ml.post_data: + job_id: upgraded-cluster-job-should-not-upgrade + body: + - airline: AAL + responsetime: 132.2046 + sourcetype: post-data-job + time: 1403481600 + - airline: JZA + responsetime: 990.4628 + sourcetype: post-data-job + time: 1403481700 + - match: { processed_record_count: 2 } + + - do: + ml.close_job: + job_id: upgraded-cluster-job-should-not-upgrade + + - do: + ml.get_buckets: + job_id: upgraded-cluster-job-should-not-upgrade + - match: { count: 1 } + + - do: + indices.exists: + index: .ml-anomalies-shared + + - is_true: '' + + - do: + indices.get_settings: + index: .ml-anomalies-shared + name: index.version.created + + - match: { \.ml-anomalies-shared.settings.index.version.created: '/7\d+/' } + + # Do the upgrade Again as nothing needs upgraded now + - do: + ml.upgrade: + wait_for_completion: true + + - match: { acknowledged: true } + + - do: + indices.exists: + index: .ml-anomalies-shared + + - is_true: '' From 78f91393abcddff1303a53068d5ad44c0d083512 Mon Sep 17 00:00:00 2001 From: Yuri Astrakhan Date: Wed, 9 Jan 2019 15:47:05 -0500 Subject: [PATCH 44/46] Use string for ParsedGeoGrid.getKey() --- .../search/aggregations/bucket/geogrid2/ParsedGeoGrid.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/geogrid2/ParsedGeoGrid.java b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/geogrid2/ParsedGeoGrid.java index 0f78e78132b69..cc60edaad85a3 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/geogrid2/ParsedGeoGrid.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/geogrid2/ParsedGeoGrid.java @@ -57,8 +57,8 @@ public static class ParsedBucket extends ParsedMultiBucketAggregation.ParsedBuck private String geohashAsString; @Override - public GeoPoint getKey() { - return GeoPoint.fromGeohash(geohashAsString); + public String getKey() { + return getKeyAsString(); } @Override From 195873002bd3724c839a4a211c3aa1428f104a3a Mon Sep 17 00:00:00 2001 From: Jake Landis Date: Wed, 9 Jan 2019 14:47:47 -0600 Subject: [PATCH 45/46] ingest: compile mustache template only if field includes '{{'' (#37207) * ingest: compile mustache template only if field includes '{{'' Prior to this change, any field in an ingest node processor that supports script templates would be compiled as mustache template regardless if they contain a template or not. Compiling normal text as mustache templates is harmless. However, each compilation counts against the script compilation circuit breaker. A large number of processors without any templates or scripts could un-intuitively trip the too many script compilations circuit breaker. This change simple checks for '{{' in the text before it attempts to compile. fixes #37120 --- .../common/AppendProcessorFactoryTests.java | 2 +- .../common/FailProcessorFactoryTests.java | 2 +- .../common/RemoveProcessorFactoryTests.java | 2 +- .../common/SetProcessorFactoryTests.java | 2 +- .../ingest/ConfigurationUtils.java | 2 +- .../org/elasticsearch/ingest/ValueSource.java | 2 +- .../ingest/ConfigurationUtilsTests.java | 29 +++++++++++++++++++ .../ingest/ValueSourceTests.java | 27 +++++++++++++++++ 8 files changed, 62 insertions(+), 6 deletions(-) diff --git a/modules/ingest-common/src/test/java/org/elasticsearch/ingest/common/AppendProcessorFactoryTests.java b/modules/ingest-common/src/test/java/org/elasticsearch/ingest/common/AppendProcessorFactoryTests.java index d51cb368e4317..4544140737612 100644 --- a/modules/ingest-common/src/test/java/org/elasticsearch/ingest/common/AppendProcessorFactoryTests.java +++ b/modules/ingest-common/src/test/java/org/elasticsearch/ingest/common/AppendProcessorFactoryTests.java @@ -95,7 +95,7 @@ public void testCreateNullValue() throws Exception { public void testInvalidMustacheTemplate() throws Exception { AppendProcessor.Factory factory = new AppendProcessor.Factory(TestTemplateService.instance(true)); Map config = new HashMap<>(); - config.put("field", "field1"); + config.put("field", "{{field1}}"); config.put("value", "value1"); String processorTag = randomAlphaOfLength(10); ElasticsearchException exception = expectThrows(ElasticsearchException.class, () -> factory.create(null, processorTag, config)); diff --git a/modules/ingest-common/src/test/java/org/elasticsearch/ingest/common/FailProcessorFactoryTests.java b/modules/ingest-common/src/test/java/org/elasticsearch/ingest/common/FailProcessorFactoryTests.java index 3c89778f0e825..78891e0a02a64 100644 --- a/modules/ingest-common/src/test/java/org/elasticsearch/ingest/common/FailProcessorFactoryTests.java +++ b/modules/ingest-common/src/test/java/org/elasticsearch/ingest/common/FailProcessorFactoryTests.java @@ -62,7 +62,7 @@ public void testCreateMissingMessageField() throws Exception { public void testInvalidMustacheTemplate() throws Exception { FailProcessor.Factory factory = new FailProcessor.Factory(TestTemplateService.instance(true)); Map config = new HashMap<>(); - config.put("message", "error"); + config.put("message", "{{error}}"); String processorTag = randomAlphaOfLength(10); ElasticsearchException exception = expectThrows(ElasticsearchException.class, () -> factory.create(null, processorTag, config)); assertThat(exception.getMessage(), equalTo("java.lang.RuntimeException: could not compile script")); diff --git a/modules/ingest-common/src/test/java/org/elasticsearch/ingest/common/RemoveProcessorFactoryTests.java b/modules/ingest-common/src/test/java/org/elasticsearch/ingest/common/RemoveProcessorFactoryTests.java index bebe780276208..61c1f73142798 100644 --- a/modules/ingest-common/src/test/java/org/elasticsearch/ingest/common/RemoveProcessorFactoryTests.java +++ b/modules/ingest-common/src/test/java/org/elasticsearch/ingest/common/RemoveProcessorFactoryTests.java @@ -75,7 +75,7 @@ public void testCreateMissingField() throws Exception { public void testInvalidMustacheTemplate() throws Exception { RemoveProcessor.Factory factory = new RemoveProcessor.Factory(TestTemplateService.instance(true)); Map config = new HashMap<>(); - config.put("field", "field1"); + config.put("field", "{{field1}}"); String processorTag = randomAlphaOfLength(10); ElasticsearchException exception = expectThrows(ElasticsearchException.class, () -> factory.create(null, processorTag, config)); assertThat(exception.getMessage(), equalTo("java.lang.RuntimeException: could not compile script")); diff --git a/modules/ingest-common/src/test/java/org/elasticsearch/ingest/common/SetProcessorFactoryTests.java b/modules/ingest-common/src/test/java/org/elasticsearch/ingest/common/SetProcessorFactoryTests.java index 9602f34f698f7..b3e183a8ab9ce 100644 --- a/modules/ingest-common/src/test/java/org/elasticsearch/ingest/common/SetProcessorFactoryTests.java +++ b/modules/ingest-common/src/test/java/org/elasticsearch/ingest/common/SetProcessorFactoryTests.java @@ -103,7 +103,7 @@ public void testCreateNullValue() throws Exception { public void testInvalidMustacheTemplate() throws Exception { SetProcessor.Factory factory = new SetProcessor.Factory(TestTemplateService.instance(true)); Map config = new HashMap<>(); - config.put("field", "field1"); + config.put("field", "{{field1}}"); config.put("value", "value1"); String processorTag = randomAlphaOfLength(10); ElasticsearchException exception = expectThrows(ElasticsearchException.class, () -> factory.create(null, processorTag, config)); diff --git a/server/src/main/java/org/elasticsearch/ingest/ConfigurationUtils.java b/server/src/main/java/org/elasticsearch/ingest/ConfigurationUtils.java index d4f27f47eb8f2..29ae578a64371 100644 --- a/server/src/main/java/org/elasticsearch/ingest/ConfigurationUtils.java +++ b/server/src/main/java/org/elasticsearch/ingest/ConfigurationUtils.java @@ -335,7 +335,7 @@ public static TemplateScript.Factory compileTemplate(String processorType, Strin // installed for use by REST tests. `propertyValue` will not be // modified if templating is not available so a script that simply returns an unmodified `propertyValue` // is returned. - if (scriptService.isLangSupported(DEFAULT_TEMPLATE_LANG)) { + if (scriptService.isLangSupported(DEFAULT_TEMPLATE_LANG) && propertyValue.contains("{{")) { Script script = new Script(ScriptType.INLINE, DEFAULT_TEMPLATE_LANG, propertyValue, Collections.emptyMap()); return scriptService.compile(script, TemplateScript.CONTEXT); } else { diff --git a/server/src/main/java/org/elasticsearch/ingest/ValueSource.java b/server/src/main/java/org/elasticsearch/ingest/ValueSource.java index 4e2787c023539..4dda3e86ba27e 100644 --- a/server/src/main/java/org/elasticsearch/ingest/ValueSource.java +++ b/server/src/main/java/org/elasticsearch/ingest/ValueSource.java @@ -75,7 +75,7 @@ static ValueSource wrap(Object value, ScriptService scriptService) { // This check is here because the DEFAULT_TEMPLATE_LANG(mustache) is not // installed for use by REST tests. `value` will not be // modified if templating is not available - if (scriptService.isLangSupported(DEFAULT_TEMPLATE_LANG)) { + if (scriptService.isLangSupported(DEFAULT_TEMPLATE_LANG) && ((String) value).contains("{{")) { Script script = new Script(ScriptType.INLINE, DEFAULT_TEMPLATE_LANG, (String) value, Collections.emptyMap()); return new TemplatedValue(scriptService.compile(script, TemplateScript.CONTEXT)); } else { diff --git a/server/src/test/java/org/elasticsearch/ingest/ConfigurationUtilsTests.java b/server/src/test/java/org/elasticsearch/ingest/ConfigurationUtilsTests.java index 8979ac0a289d5..20f67fd10a36d 100644 --- a/server/src/test/java/org/elasticsearch/ingest/ConfigurationUtilsTests.java +++ b/server/src/test/java/org/elasticsearch/ingest/ConfigurationUtilsTests.java @@ -21,6 +21,7 @@ import org.elasticsearch.ElasticsearchParseException; import org.elasticsearch.script.ScriptService; +import org.elasticsearch.script.TemplateScript; import org.elasticsearch.test.ESTestCase; import org.junit.Before; @@ -36,7 +37,12 @@ import static org.hamcrest.Matchers.is; import static org.hamcrest.Matchers.nullValue; import static org.hamcrest.Matchers.sameInstance; +import static org.mockito.Matchers.any; +import static org.mockito.Matchers.anyString; import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.times; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.when; public class ConfigurationUtilsTests extends ESTestCase { @@ -181,4 +187,27 @@ public void testReadProcessorFromObjectOrMap() throws Exception { assertThat(ex.getMessage(), equalTo("property isn't a map, but of type [" + invalidConfig.getClass().getName() + "]")); } + public void testNoScriptCompilation() { + ScriptService scriptService = mock(ScriptService.class); + when(scriptService.isLangSupported(anyString())).thenReturn(true); + String propertyValue = randomAlphaOfLength(10); + TemplateScript.Factory result; + result = ConfigurationUtils.compileTemplate(randomAlphaOfLength(10), randomAlphaOfLength(10), randomAlphaOfLength(10), + propertyValue, scriptService); + assertThat(result.newInstance(null).execute(), equalTo(propertyValue)); + verify(scriptService, times(0)).compile(any(), any()); + } + + public void testScriptShouldCompile() { + ScriptService scriptService = mock(ScriptService.class); + when(scriptService.isLangSupported(anyString())).thenReturn(true); + String propertyValue = "{{" + randomAlphaOfLength(10) + "}}"; + String compiledValue = randomAlphaOfLength(10); + when(scriptService.compile(any(), any())).thenReturn(new TestTemplateService.MockTemplateScript.Factory(compiledValue)); + TemplateScript.Factory result; + result = ConfigurationUtils.compileTemplate(randomAlphaOfLength(10), randomAlphaOfLength(10), randomAlphaOfLength(10), + propertyValue, scriptService); + assertThat(result.newInstance(null).execute(), equalTo(compiledValue)); + verify(scriptService, times(1)).compile(any(), any()); + } } diff --git a/server/src/test/java/org/elasticsearch/ingest/ValueSourceTests.java b/server/src/test/java/org/elasticsearch/ingest/ValueSourceTests.java index 72238d3b59656..37b9956119334 100644 --- a/server/src/test/java/org/elasticsearch/ingest/ValueSourceTests.java +++ b/server/src/test/java/org/elasticsearch/ingest/ValueSourceTests.java @@ -19,6 +19,7 @@ package org.elasticsearch.ingest; +import org.elasticsearch.script.ScriptService; import org.elasticsearch.test.ESTestCase; import java.util.ArrayList; @@ -30,6 +31,12 @@ import static org.hamcrest.Matchers.equalTo; import static org.hamcrest.Matchers.not; import static org.hamcrest.Matchers.sameInstance; +import static org.mockito.Matchers.any; +import static org.mockito.Matchers.anyString; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.times; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.when; public class ValueSourceTests extends ESTestCase { @@ -69,4 +76,24 @@ public void testCopyDoesNotChangeProvidedList() { assertThat(myPreciousList.size(), equalTo(1)); assertThat(myPreciousList.get(0), equalTo("value")); } + + public void testNoScriptCompilation() { + ScriptService scriptService = mock(ScriptService.class); + when(scriptService.isLangSupported(anyString())).thenReturn(true); + String propertyValue = randomAlphaOfLength(10); + ValueSource result = ValueSource.wrap(propertyValue, scriptService); + assertThat(result.copyAndResolve(null), equalTo(propertyValue)); + verify(scriptService, times(0)).compile(any(), any()); + } + + public void testScriptShouldCompile() { + ScriptService scriptService = mock(ScriptService.class); + when(scriptService.isLangSupported(anyString())).thenReturn(true); + String propertyValue = "{{" + randomAlphaOfLength(10) + "}}"; + String compiledValue = randomAlphaOfLength(10); + when(scriptService.compile(any(), any())).thenReturn(new TestTemplateService.MockTemplateScript.Factory(compiledValue)); + ValueSource result = ValueSource.wrap(propertyValue, scriptService); + assertThat(result.copyAndResolve(Collections.emptyMap()), equalTo(compiledValue)); + verify(scriptService, times(1)).compile(any(), any()); + } } From 2d0fcaef0edb93b653bd1832f25e8c36b0a08055 Mon Sep 17 00:00:00 2001 From: Yuri Astrakhan Date: Wed, 9 Jan 2019 18:07:32 -0500 Subject: [PATCH 46/46] fix styling --- .../search/aggregations/bucket/geogrid2/ParsedGeoGrid.java | 1 - 1 file changed, 1 deletion(-) diff --git a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/geogrid2/ParsedGeoGrid.java b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/geogrid2/ParsedGeoGrid.java index cc60edaad85a3..04c3c057c3c80 100644 --- a/server/src/main/java/org/elasticsearch/search/aggregations/bucket/geogrid2/ParsedGeoGrid.java +++ b/server/src/main/java/org/elasticsearch/search/aggregations/bucket/geogrid2/ParsedGeoGrid.java @@ -19,7 +19,6 @@ package org.elasticsearch.search.aggregations.bucket.geogrid2; -import org.elasticsearch.common.geo.GeoPoint; import org.elasticsearch.common.xcontent.ObjectParser; import org.elasticsearch.common.xcontent.XContentBuilder; import org.elasticsearch.common.xcontent.XContentParser;