From 3ca885e9ac6b031b896e62bece07170abe0ac85c Mon Sep 17 00:00:00 2001 From: Tanguy Leroux Date: Fri, 7 Dec 2018 09:30:27 +0100 Subject: [PATCH 001/186] [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 002/186] [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 003/186] [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 004/186] [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 005/186] [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 006/186] [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 007/186] [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 6b376a1ff4d6ea49f9fa42f828627e5a0444f51c Mon Sep 17 00:00:00 2001 From: Benjamin Trent Date: Mon, 7 Jan 2019 15:18:38 -0600 Subject: [PATCH 008/186] ML: fix delayed data annotations on secured cluster (#37193) * changing executing context for writing annotation * adjusting user * removing unused import --- .../xpack/ml/datafeed/DatafeedJob.java | 22 +++++++++++-------- .../xpack/ml/datafeed/DatafeedJobTests.java | 8 +++---- 2 files changed, 17 insertions(+), 13 deletions(-) diff --git a/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/datafeed/DatafeedJob.java b/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/datafeed/DatafeedJob.java index 64e8512baa5b0..35878f1199586 100644 --- a/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/datafeed/DatafeedJob.java +++ b/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/datafeed/DatafeedJob.java @@ -32,7 +32,7 @@ import org.elasticsearch.xpack.core.ml.job.messages.Messages; import org.elasticsearch.xpack.core.ml.job.process.autodetect.state.DataCounts; import org.elasticsearch.xpack.core.ml.job.results.Bucket; -import org.elasticsearch.xpack.core.security.user.SystemUser; +import org.elasticsearch.xpack.core.security.user.XPackUser; import org.elasticsearch.xpack.ml.datafeed.delayeddatacheck.DelayedDataDetector; import org.elasticsearch.xpack.ml.datafeed.delayeddatacheck.DelayedDataDetectorFactory.BucketWithMissingData; import org.elasticsearch.xpack.ml.datafeed.extractor.DataExtractorFactory; @@ -225,12 +225,12 @@ private Annotation createAnnotation(Date startTime, Date endTime, String msg) { Date currentTime = new Date(currentTimeSupplier.get()); return new Annotation(msg, currentTime, - SystemUser.NAME, + XPackUser.NAME, startTime, endTime, jobId, currentTime, - SystemUser.NAME, + XPackUser.NAME, "annotation"); } @@ -238,9 +238,11 @@ private String addAndSetDelayedDataAnnotation(Annotation annotation) { try (XContentBuilder xContentBuilder = annotation.toXContent(XContentFactory.jsonBuilder(), ToXContent.EMPTY_PARAMS)) { IndexRequest request = new IndexRequest(AnnotationIndex.WRITE_ALIAS_NAME); request.source(xContentBuilder); - IndexResponse response = client.index(request).actionGet(); - lastDataCheckAnnotation = annotation; - return response.getId(); + try (ThreadContext.StoredContext ignore = stashWithOrigin(client.threadPool().getThreadContext(), ML_ORIGIN)) { + IndexResponse response = client.index(request).actionGet(); + lastDataCheckAnnotation = annotation; + return response.getId(); + } } catch (IOException ex) { String errorMessage = "[" + jobId + "] failed to create annotation for delayed data checker."; LOGGER.error(errorMessage, ex); @@ -251,7 +253,7 @@ private String addAndSetDelayedDataAnnotation(Annotation annotation) { private void updateAnnotation(Annotation annotation) { Annotation updatedAnnotation = new Annotation(lastDataCheckAnnotation); - updatedAnnotation.setModifiedUsername(SystemUser.NAME); + updatedAnnotation.setModifiedUsername(XPackUser.NAME); updatedAnnotation.setModifiedTime(new Date(currentTimeSupplier.get())); updatedAnnotation.setAnnotation(annotation.getAnnotation()); updatedAnnotation.setTimestamp(annotation.getTimestamp()); @@ -260,8 +262,10 @@ private void updateAnnotation(Annotation annotation) { IndexRequest indexRequest = new IndexRequest(AnnotationIndex.WRITE_ALIAS_NAME); indexRequest.id(lastDataCheckAnnotationId); indexRequest.source(xContentBuilder); - client.index(indexRequest).actionGet(); - lastDataCheckAnnotation = updatedAnnotation; + try (ThreadContext.StoredContext ignore = stashWithOrigin(client.threadPool().getThreadContext(), ML_ORIGIN)) { + client.index(indexRequest).actionGet(); + lastDataCheckAnnotation = updatedAnnotation; + } } catch (IOException ex) { String errorMessage = "[" + jobId + "] failed to update annotation for delayed data checker."; LOGGER.error(errorMessage, ex); diff --git a/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/datafeed/DatafeedJobTests.java b/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/datafeed/DatafeedJobTests.java index 534681ff3c86a..2540ab8cde8ef 100644 --- a/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/datafeed/DatafeedJobTests.java +++ b/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/datafeed/DatafeedJobTests.java @@ -30,7 +30,7 @@ import org.elasticsearch.xpack.core.ml.datafeed.extractor.DataExtractor; import org.elasticsearch.xpack.core.ml.job.messages.Messages; import org.elasticsearch.xpack.core.ml.job.results.Bucket; -import org.elasticsearch.xpack.core.security.user.SystemUser; +import org.elasticsearch.xpack.core.security.user.XPackUser; import org.elasticsearch.xpack.ml.datafeed.delayeddatacheck.DelayedDataDetector; import org.elasticsearch.xpack.ml.datafeed.delayeddatacheck.DelayedDataDetectorFactory.BucketWithMissingData; import org.elasticsearch.xpack.ml.datafeed.extractor.DataExtractorFactory; @@ -271,12 +271,12 @@ public void testRealtimeRun() throws Exception { Annotation expectedAnnotation = new Annotation(msg, new Date(currentTime), - SystemUser.NAME, + XPackUser.NAME, bucket.getTimestamp(), new Date((bucket.getEpoch() + bucket.getBucketSpan()) * 1000), jobId, new Date(currentTime), - SystemUser.NAME, + XPackUser.NAME, "annotation"); IndexRequest request = new IndexRequest(AnnotationIndex.WRITE_ALIAS_NAME); @@ -312,7 +312,7 @@ public void testRealtimeRun() throws Exception { Annotation updatedAnnotation = new Annotation(expectedAnnotation); updatedAnnotation.setAnnotation(msg); updatedAnnotation.setModifiedTime(new Date(currentTime)); - updatedAnnotation.setModifiedUsername(SystemUser.NAME); + updatedAnnotation.setModifiedUsername(XPackUser.NAME); updatedAnnotation.setEndTimestamp(new Date((bucket2.getEpoch() + bucket2.getBucketSpan()) * 1000)); try (XContentBuilder xContentBuilder = updatedAnnotation.toXContent(XContentFactory.jsonBuilder(), ToXContent.EMPTY_PARAMS)) { indexRequest.source(xContentBuilder); From 382e4d39efa560fed16a2b5f3d98416920c70e8d Mon Sep 17 00:00:00 2001 From: lcawl Date: Mon, 7 Jan 2019 14:32:36 -0800 Subject: [PATCH 009/186] [DOCS] Cleans up xpackml attributes --- docs/reference/ml/aggregations.asciidoc | 2 +- docs/reference/ml/apis/resultsresource.asciidoc | 2 +- docs/reference/ml/configuring.asciidoc | 4 ++-- docs/reference/ml/functions.asciidoc | 2 +- docs/reference/ml/functions/count.asciidoc | 2 +- docs/reference/ml/functions/geo.asciidoc | 4 ++-- docs/reference/ml/functions/info.asciidoc | 2 +- docs/reference/ml/functions/metric.asciidoc | 2 +- docs/reference/ml/functions/rare.asciidoc | 4 ++-- docs/reference/ml/functions/sum.asciidoc | 2 +- docs/reference/ml/functions/time.asciidoc | 2 +- docs/reference/ml/transforms.asciidoc | 2 +- docs/reference/modules/ml-node.asciidoc | 11 +++++------ 13 files changed, 20 insertions(+), 21 deletions(-) diff --git a/docs/reference/ml/aggregations.asciidoc b/docs/reference/ml/aggregations.asciidoc index 47db536db014b..3f09022d17eaa 100644 --- a/docs/reference/ml/aggregations.asciidoc +++ b/docs/reference/ml/aggregations.asciidoc @@ -8,7 +8,7 @@ and to configure your jobs to analyze aggregated data. One of the benefits of aggregating data this way is that {es} automatically distributes these calculations across your cluster. You can then feed this -aggregated data into {xpackml} instead of raw results, which +aggregated data into the {ml-features} instead of raw results, which reduces the volume of data that must be considered while detecting anomalies. There are some limitations to using aggregations in {dfeeds}, however. diff --git a/docs/reference/ml/apis/resultsresource.asciidoc b/docs/reference/ml/apis/resultsresource.asciidoc index 8962129c73966..f2533bbd07345 100644 --- a/docs/reference/ml/apis/resultsresource.asciidoc +++ b/docs/reference/ml/apis/resultsresource.asciidoc @@ -269,7 +269,7 @@ probability of this occurrence. There can be many anomaly records depending on the characteristics and size of the input data. In practice, there are often too many to be able to manually -process them. The {xpackml} features therefore perform a sophisticated +process them. The {ml-features} therefore perform a sophisticated aggregation of the anomaly records into buckets. The number of record results depends on the number of anomalies found in each diff --git a/docs/reference/ml/configuring.asciidoc b/docs/reference/ml/configuring.asciidoc index a7773b5681f89..9304a93d360c7 100644 --- a/docs/reference/ml/configuring.asciidoc +++ b/docs/reference/ml/configuring.asciidoc @@ -2,12 +2,12 @@ [[ml-configuring]] == Configuring machine learning -If you want to use {xpackml} features, there must be at least one {ml} node in +If you want to use {ml-features}, there must be at least one {ml} node in your cluster and all master-eligible nodes must have {ml} enabled. By default, all nodes are {ml} nodes. For more information about these settings, see {ref}/modules-node.html#modules-node-xpack[{ml} nodes]. -To use the {xpackml} features to analyze your data, you must create a job and +To use the {ml-features} to analyze your data, you must create a job and send your data to that job. * If your data is stored in {es}: diff --git a/docs/reference/ml/functions.asciidoc b/docs/reference/ml/functions.asciidoc index e32470c6827b6..48e56bb4627ee 100644 --- a/docs/reference/ml/functions.asciidoc +++ b/docs/reference/ml/functions.asciidoc @@ -2,7 +2,7 @@ [[ml-functions]] == Function reference -The {xpackml} features include analysis functions that provide a wide variety of +The {ml-features} include analysis functions that provide a wide variety of flexible ways to analyze data for anomalies. When you create jobs, you specify one or more detectors, which define the type of diff --git a/docs/reference/ml/functions/count.asciidoc b/docs/reference/ml/functions/count.asciidoc index 3365a0923a8b0..404ed7f2d94a3 100644 --- a/docs/reference/ml/functions/count.asciidoc +++ b/docs/reference/ml/functions/count.asciidoc @@ -14,7 +14,7 @@ in one field is unusual, as opposed to the total count. Use high-sided functions if you want to monitor unusually high event rates. Use low-sided functions if you want to look at drops in event rate. -The {xpackml} features include the following count functions: +The {ml-features} include the following count functions: * xref:ml-count[`count`, `high_count`, `low_count`] * xref:ml-nonzero-count[`non_zero_count`, `high_non_zero_count`, `low_non_zero_count`] diff --git a/docs/reference/ml/functions/geo.asciidoc b/docs/reference/ml/functions/geo.asciidoc index 3698ab7c0590e..130e17d85dcfe 100644 --- a/docs/reference/ml/functions/geo.asciidoc +++ b/docs/reference/ml/functions/geo.asciidoc @@ -5,7 +5,7 @@ The geographic functions detect anomalies in the geographic location of the input data. -The {xpackml} features include the following geographic function: `lat_long`. +The {ml-features} include the following geographic function: `lat_long`. NOTE: You cannot create forecasts for jobs that contain geographic functions. You also cannot add rules with conditions to detectors that use geographic @@ -72,7 +72,7 @@ For example, JSON data might contain the following transaction coordinates: In {es}, location data is likely to be stored in `geo_point` fields. For more information, see {ref}/geo-point.html[Geo-point datatype]. This data type is not -supported natively in {xpackml} features. You can, however, use Painless scripts +supported natively in {ml-features}. You can, however, use Painless scripts in `script_fields` in your {dfeed} to transform the data into an appropriate format. For example, the following Painless script transforms `"coords": {"lat" : 41.44, "lon":90.5}` into `"lat-lon": "41.44,90.5"`: diff --git a/docs/reference/ml/functions/info.asciidoc b/docs/reference/ml/functions/info.asciidoc index 2c3117e0e5644..c75440f238ff5 100644 --- a/docs/reference/ml/functions/info.asciidoc +++ b/docs/reference/ml/functions/info.asciidoc @@ -6,7 +6,7 @@ that is contained in strings within a bucket. These functions can be used as a more sophisticated method to identify incidences of data exfiltration or C2C activity, when analyzing the size in bytes of the data might not be sufficient. -The {xpackml} features include the following information content functions: +The {ml-features} include the following information content functions: * `info_content`, `high_info_content`, `low_info_content` diff --git a/docs/reference/ml/functions/metric.asciidoc b/docs/reference/ml/functions/metric.asciidoc index 9d6f3515a029c..7868d4b780a40 100644 --- a/docs/reference/ml/functions/metric.asciidoc +++ b/docs/reference/ml/functions/metric.asciidoc @@ -6,7 +6,7 @@ The metric functions include functions such as mean, min and max. These values are calculated for each bucket. Field values that cannot be converted to double precision floating point numbers are ignored. -The {xpackml} features include the following metric functions: +The {ml-features} include the following metric functions: * <> * <> diff --git a/docs/reference/ml/functions/rare.asciidoc b/docs/reference/ml/functions/rare.asciidoc index 1531285a7add2..87c212fbd1275 100644 --- a/docs/reference/ml/functions/rare.asciidoc +++ b/docs/reference/ml/functions/rare.asciidoc @@ -27,7 +27,7 @@ with shorter bucket spans typically being measured in minutes, not hours. for typical data. ==== -The {xpackml} features include the following rare functions: +The {ml-features} include the following rare functions: * <> * <> @@ -85,7 +85,7 @@ different rare status codes compared to the population is regarded as highly anomalous. This analysis is based on the number of different status code values, not the count of occurrences. -NOTE: To define a status code as rare the {xpackml} features look at the number +NOTE: To define a status code as rare the {ml-features} look at the number of distinct status codes that occur, not the number of times the status code occurs. If a single client IP experiences a single unique status code, this is rare, even if it occurs for that client IP in every bucket. diff --git a/docs/reference/ml/functions/sum.asciidoc b/docs/reference/ml/functions/sum.asciidoc index 7a95ad63fccee..9313a60a01a6c 100644 --- a/docs/reference/ml/functions/sum.asciidoc +++ b/docs/reference/ml/functions/sum.asciidoc @@ -11,7 +11,7 @@ If want to look at drops in totals, use low-sided functions. If your data is sparse, use `non_null_sum` functions. Buckets without values are ignored; buckets with a zero value are analyzed. -The {xpackml} features include the following sum functions: +The {ml-features} include the following sum functions: * xref:ml-sum[`sum`, `high_sum`, `low_sum`] * xref:ml-nonnull-sum[`non_null_sum`, `high_non_null_sum`, `low_non_null_sum`] diff --git a/docs/reference/ml/functions/time.asciidoc b/docs/reference/ml/functions/time.asciidoc index ac8199307f130..026d29d85d3d7 100644 --- a/docs/reference/ml/functions/time.asciidoc +++ b/docs/reference/ml/functions/time.asciidoc @@ -6,7 +6,7 @@ The time functions detect events that happen at unusual times, either of the day or of the week. These functions can be used to find unusual patterns of behavior, typically associated with suspicious user activity. -The {xpackml} features include the following time functions: +The {ml-features} include the following time functions: * <> * <> diff --git a/docs/reference/ml/transforms.asciidoc b/docs/reference/ml/transforms.asciidoc index 66c55d72b14f2..6fc67fa7c4e4b 100644 --- a/docs/reference/ml/transforms.asciidoc +++ b/docs/reference/ml/transforms.asciidoc @@ -569,7 +569,7 @@ GET _ml/datafeeds/datafeed-test4/_preview // TEST[skip:needs-licence] In {es}, location data can be stored in `geo_point` fields but this data type is -not supported natively in {xpackml} analytics. This example of a script field +not supported natively in {ml} analytics. This example of a script field transforms the data into an appropriate format. For more information, see <>. diff --git a/docs/reference/modules/ml-node.asciidoc b/docs/reference/modules/ml-node.asciidoc index 9e4413e3a0c7e..5a907adfbbf3a 100644 --- a/docs/reference/modules/ml-node.asciidoc +++ b/docs/reference/modules/ml-node.asciidoc @@ -9,10 +9,9 @@ If {xpack} is installed, there is an additional node type: <>:: A node that has `xpack.ml.enabled` and `node.ml` set to `true`, which is the -default behavior when {xpack} is installed. If you want to use {xpackml} -features, there must be at least one {ml} node in your cluster. For more -information about {xpackml} features, -see {xpack-ref}/xpack-ml.html[Machine Learning in the Elastic Stack]. +default behavior when {xpack} is installed. If you want to use {ml-features}, there must be at least one {ml} node in your cluster. For more +information about {ml-features}, +see {stack-ov}/xpack-ml.html[Machine learning in the {stack}]. IMPORTANT: Do not set use the `node.ml` setting unless {xpack} is installed. Otherwise, the node fails to start. @@ -88,11 +87,11 @@ node.ml: false <5> [[ml-node]] === [xpack]#Machine learning node# -The {xpackml} features provide {ml} nodes, which run jobs and handle {ml} API +The {ml-features} provide {ml} nodes, which run jobs and handle {ml} API requests. If `xpack.ml.enabled` is set to true and `node.ml` is set to `false`, the node can service API requests but it cannot run jobs. -If you want to use {xpackml} features in your cluster, you must enable {ml} +If you want to use {ml-features} in your cluster, you must enable {ml} (set `xpack.ml.enabled` to `true`) on all master-eligible nodes. Do not use these settings if you do not have {xpack} installed. From 3b48b9986110a3dbb89d3dbf5ad35f9e8b98ef00 Mon Sep 17 00:00:00 2001 From: Jason Tedor Date: Mon, 7 Jan 2019 14:59:24 -0800 Subject: [PATCH 010/186] Fix setting by time unit (#37192) This commit fixes an issue with a settings builder method that allows setting a duration by time unit. In particular, this method can suffer from a loss of precision. For example, if the input duration is 1500 microseconds then internally we are converting this to "1ms", demonstrating the loss of precision. Instead, we should internally convert this to a TimeValue that correctly represents the input duration, and then convert this to a string using a method that does not lose the unit. That is what this commit does. --- .../elasticsearch/common/settings/Settings.java | 4 ++-- .../common/settings/SettingsTests.java | 15 +++++++++++++++ .../snapshots/SharedClusterSnapshotRestoreIT.java | 2 +- 3 files changed, 18 insertions(+), 3 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/common/settings/Settings.java b/server/src/main/java/org/elasticsearch/common/settings/Settings.java index e8ba6d383d55e..ac43a1800b40f 100644 --- a/server/src/main/java/org/elasticsearch/common/settings/Settings.java +++ b/server/src/main/java/org/elasticsearch/common/settings/Settings.java @@ -1019,8 +1019,8 @@ public Builder put(String setting, double value) { * @param value The time value * @return The builder */ - public Builder put(String setting, long value, TimeUnit timeUnit) { - put(setting, timeUnit.toMillis(value) + "ms"); + public Builder put(final String setting, final long value, final TimeUnit timeUnit) { + put(setting, new TimeValue(value, timeUnit)); return this; } diff --git a/server/src/test/java/org/elasticsearch/common/settings/SettingsTests.java b/server/src/test/java/org/elasticsearch/common/settings/SettingsTests.java index 27a9b00204203..802bceaa90812 100644 --- a/server/src/test/java/org/elasticsearch/common/settings/SettingsTests.java +++ b/server/src/test/java/org/elasticsearch/common/settings/SettingsTests.java @@ -47,6 +47,7 @@ import java.util.Map; import java.util.NoSuchElementException; import java.util.Set; +import java.util.concurrent.TimeUnit; import static org.hamcrest.Matchers.contains; import static org.hamcrest.Matchers.containsInAnyOrder; @@ -744,4 +745,18 @@ public void testFractionalByteSizeValue() { assertThat(actual, equalTo(expected)); } + public void testSetByTimeUnit() { + final Setting setting = + Setting.timeSetting("key", TimeValue.parseTimeValue(randomTimeValue(0, 24, "h"), "key"), TimeValue.ZERO); + final TimeValue expected = new TimeValue(1500, TimeUnit.MICROSECONDS); + final Settings settings = Settings.builder().put("key", expected.getMicros(), TimeUnit.MICROSECONDS).build(); + /* + * Previously we would internally convert the duration to a string by converting to milliseconds which could lose precision (e.g., + * 1500 microseconds would be converted to 1ms). Effectively this test is then asserting that we no longer make this mistake when + * doing the internal string conversion. Instead, we convert to a duration using a method that does not lose the original unit. + */ + final TimeValue actual = setting.get(settings); + assertThat(actual, equalTo(expected)); + } + } diff --git a/server/src/test/java/org/elasticsearch/snapshots/SharedClusterSnapshotRestoreIT.java b/server/src/test/java/org/elasticsearch/snapshots/SharedClusterSnapshotRestoreIT.java index d7ca90a90a3d9..c90d9319df30b 100644 --- a/server/src/test/java/org/elasticsearch/snapshots/SharedClusterSnapshotRestoreIT.java +++ b/server/src/test/java/org/elasticsearch/snapshots/SharedClusterSnapshotRestoreIT.java @@ -445,7 +445,7 @@ public void testRestoreWithDifferentMappingsAndSettings() throws Exception { logger.info("--> assert that old settings are restored"); GetSettingsResponse getSettingsResponse = client.admin().indices().prepareGetSettings("test-idx").execute().actionGet(); - assertThat(getSettingsResponse.getSetting("test-idx", "index.refresh_interval"), equalTo("10000ms")); + assertThat(getSettingsResponse.getSetting("test-idx", "index.refresh_interval"), equalTo("10s")); } public void testEmptySnapshot() throws Exception { From 56e472bfbcf6b16e2fb518d9b183d7dfcaea476d Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Christoph=20B=C3=BCscher?= Date: Tue, 8 Jan 2019 00:14:50 +0100 Subject: [PATCH 011/186] Removing unused methods in Numbers (#37186) Remove several unused helper methods. Most of them are one-liners and should be easier to be used from the corresponding primitive wrapper classes. The bytes array conversion methods are unused as well, it should be easy to re-create them if needed. --- .../org/elasticsearch/common/Numbers.java | 86 ------------------- 1 file changed, 86 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/common/Numbers.java b/server/src/main/java/org/elasticsearch/common/Numbers.java index 7561175f3fe35..27c1dd18e97b8 100644 --- a/server/src/main/java/org/elasticsearch/common/Numbers.java +++ b/server/src/main/java/org/elasticsearch/common/Numbers.java @@ -33,48 +33,6 @@ public final class Numbers { private static final BigInteger MIN_LONG_VALUE = BigInteger.valueOf(Long.MIN_VALUE); private Numbers() { - - } - - /** - * Converts a byte array to an short. - * - * @param arr The byte array to convert to an short - * @return The int converted - */ - public static short bytesToShort(byte[] arr) { - return (short) (((arr[0] & 0xff) << 8) | (arr[1] & 0xff)); - } - - public static short bytesToShort(BytesRef bytes) { - return (short) (((bytes.bytes[bytes.offset] & 0xff) << 8) | (bytes.bytes[bytes.offset + 1] & 0xff)); - } - - /** - * Converts a byte array to an int. - * - * @param arr The byte array to convert to an int - * @return The int converted - */ - public static int bytesToInt(byte[] arr) { - return (arr[0] << 24) | ((arr[1] & 0xff) << 16) | ((arr[2] & 0xff) << 8) | (arr[3] & 0xff); - } - - public static int bytesToInt(BytesRef bytes) { - return (bytes.bytes[bytes.offset] << 24) | ((bytes.bytes[bytes.offset + 1] & 0xff) << 16) | - ((bytes.bytes[bytes.offset + 2] & 0xff) << 8) | (bytes.bytes[bytes.offset + 3] & 0xff); - } - - /** - * Converts a byte array to a long. - * - * @param arr The byte array to convert to a long - * @return The long converter - */ - public static long bytesToLong(byte[] arr) { - int high = (arr[0] << 24) | ((arr[1] & 0xff) << 16) | ((arr[2] & 0xff) << 8) | (arr[3] & 0xff); - int low = (arr[4] << 24) | ((arr[5] & 0xff) << 16) | ((arr[6] & 0xff) << 8) | (arr[7] & 0xff); - return (((long) high) << 32) | (low & 0x0ffffffffL); } public static long bytesToLong(BytesRef bytes) { @@ -85,40 +43,6 @@ public static long bytesToLong(BytesRef bytes) { return (((long) high) << 32) | (low & 0x0ffffffffL); } - /** - * Converts a byte array to float. - * - * @param arr The byte array to convert to a float - * @return The float converted - */ - public static float bytesToFloat(byte[] arr) { - return Float.intBitsToFloat(bytesToInt(arr)); - } - - public static float bytesToFloat(BytesRef bytes) { - return Float.intBitsToFloat(bytesToInt(bytes)); - } - - /** - * Converts a byte array to double. - * - * @param arr The byte array to convert to a double - * @return The double converted - */ - public static double bytesToDouble(byte[] arr) { - return Double.longBitsToDouble(bytesToLong(arr)); - } - - public static double bytesToDouble(BytesRef bytes) { - return Double.longBitsToDouble(bytesToLong(bytes)); - } - - /** - * Converts an int to a byte array. - * - * @param val The int to convert to a byte array - * @return The byte array converted - */ public static byte[] intToBytes(int val) { byte[] arr = new byte[4]; arr[0] = (byte) (val >>> 24); @@ -160,16 +84,6 @@ public static byte[] longToBytes(long val) { return arr; } - /** - * Converts a float to a byte array. - * - * @param val The float to convert to a byte array - * @return The byte array converted - */ - public static byte[] floatToBytes(float val) { - return intToBytes(Float.floatToRawIntBits(val)); - } - /** * Converts a double to a byte array. * From 55d3ca3aa853e50fc72d6fbe701881540818af51 Mon Sep 17 00:00:00 2001 From: Ryan Ernst Date: Mon, 7 Jan 2019 16:15:30 -0800 Subject: [PATCH 012/186] Core: Rework epoch time parsing for java time (#36914) This commit converts the epoch time parsing implementation which uses the java time api to create DateTimeFormatters instead of DateFormatter implementations. This will allow multi formats for java time to be implemented in a single DateTimeFormatter in a future change. --- .../common/time/DateFormatters.java | 4 +- .../time/EpochSecondsDateFormatter.java | 113 --------- .../elasticsearch/common/time/EpochTime.java | 219 ++++++++++++++++++ .../joda/JavaJodaTimeDuellingTests.java | 11 +- .../elasticsearch/common/joda/JodaTests.java | 1 - .../common/time/DateFormattersTests.java | 55 +---- .../search/fields/SearchFieldsIT.java | 4 +- 7 files changed, 239 insertions(+), 168 deletions(-) delete mode 100644 server/src/main/java/org/elasticsearch/common/time/EpochSecondsDateFormatter.java create mode 100644 server/src/main/java/org/elasticsearch/common/time/EpochTime.java diff --git a/server/src/main/java/org/elasticsearch/common/time/DateFormatters.java b/server/src/main/java/org/elasticsearch/common/time/DateFormatters.java index 0f1234dde0214..a9d953248a2ce 100644 --- a/server/src/main/java/org/elasticsearch/common/time/DateFormatters.java +++ b/server/src/main/java/org/elasticsearch/common/time/DateFormatters.java @@ -1366,9 +1366,9 @@ public static DateFormatter forPattern(String input) { } else if ("yearMonthDay".equals(input) || "year_month_day".equals(input)) { return YEAR_MONTH_DAY; } else if ("epoch_second".equals(input)) { - return EpochSecondsDateFormatter.INSTANCE; + return EpochTime.SECONDS_FORMATTER; } else if ("epoch_millis".equals(input)) { - return EpochMillisDateFormatter.INSTANCE; + return EpochTime.MILLIS_FORMATTER; // strict date formats here, must be at least 4 digits for year and two for months and two for day } else if ("strictBasicWeekDate".equals(input) || "strict_basic_week_date".equals(input)) { return STRICT_BASIC_WEEK_DATE; diff --git a/server/src/main/java/org/elasticsearch/common/time/EpochSecondsDateFormatter.java b/server/src/main/java/org/elasticsearch/common/time/EpochSecondsDateFormatter.java deleted file mode 100644 index 3d33d083ec6af..0000000000000 --- a/server/src/main/java/org/elasticsearch/common/time/EpochSecondsDateFormatter.java +++ /dev/null @@ -1,113 +0,0 @@ -/* - * Licensed to Elasticsearch under one or more contributor - * license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright - * ownership. Elasticsearch licenses this file to you under - * the Apache License, Version 2.0 (the "License"); you may - * not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ - -package org.elasticsearch.common.time; - -import java.math.BigDecimal; -import java.time.Instant; -import java.time.ZoneId; -import java.time.ZoneOffset; -import java.time.format.DateTimeParseException; -import java.time.temporal.TemporalAccessor; -import java.util.Locale; -import java.util.regex.Pattern; - -public class EpochSecondsDateFormatter implements DateFormatter { - - public static DateFormatter INSTANCE = new EpochSecondsDateFormatter(); - static final DateMathParser DATE_MATH_INSTANCE = new JavaDateMathParser(INSTANCE, INSTANCE); - private static final Pattern SPLIT_BY_DOT_PATTERN = Pattern.compile("\\."); - - private EpochSecondsDateFormatter() {} - - @Override - public TemporalAccessor parse(String input) { - try { - if (input.contains(".")) { - String[] inputs = SPLIT_BY_DOT_PATTERN.split(input, 2); - Long seconds = Long.valueOf(inputs[0]); - if (inputs[1].length() == 0) { - // this is BWC compatible to joda time, nothing after the dot is allowed - return Instant.ofEpochSecond(seconds, 0).atZone(ZoneOffset.UTC); - } - // scientific notation it is! - if (inputs[1].contains("e")) { - return Instant.ofEpochSecond(Double.valueOf(input).longValue()).atZone(ZoneOffset.UTC); - } - if (inputs[1].length() > 9) { - throw new DateTimeParseException("too much granularity after dot [" + input + "]", input, 0); - } - Long nanos = new BigDecimal(inputs[1]).movePointRight(9 - inputs[1].length()).longValueExact(); - if (seconds < 0) { - nanos = nanos * -1; - } - return Instant.ofEpochSecond(seconds, nanos).atZone(ZoneOffset.UTC); - } else { - return Instant.ofEpochSecond(Long.valueOf(input)).atZone(ZoneOffset.UTC); - } - } catch (NumberFormatException e) { - throw new DateTimeParseException("invalid number [" + input + "]", input, 0, e); - } - } - - @Override - public String format(TemporalAccessor accessor) { - Instant instant = Instant.from(accessor); - if (instant.getNano() != 0) { - return String.valueOf(instant.getEpochSecond()) + "." + String.valueOf(instant.getNano()).replaceAll("0*$", ""); - } - return String.valueOf(instant.getEpochSecond()); - } - - @Override - public String pattern() { - return "epoch_second"; - } - - @Override - public Locale locale() { - return Locale.ROOT; - } - - @Override - public ZoneId zone() { - return ZoneOffset.UTC; - } - - @Override - public DateMathParser toDateMathParser() { - return DATE_MATH_INSTANCE; - } - - @Override - public DateFormatter withZone(ZoneId zoneId) { - if (zoneId.equals(ZoneOffset.UTC) == false) { - throw new IllegalArgumentException(pattern() + " date formatter can only be in zone offset UTC"); - } - return this; - } - - @Override - public DateFormatter withLocale(Locale locale) { - if (Locale.ROOT.equals(locale) == false) { - throw new IllegalArgumentException(pattern() + " date formatter can only be in locale ROOT"); - } - return this; - } -} diff --git a/server/src/main/java/org/elasticsearch/common/time/EpochTime.java b/server/src/main/java/org/elasticsearch/common/time/EpochTime.java new file mode 100644 index 0000000000000..518957cd2eb9d --- /dev/null +++ b/server/src/main/java/org/elasticsearch/common/time/EpochTime.java @@ -0,0 +1,219 @@ +/* + * 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.common.time; + +import java.time.format.DateTimeFormatter; +import java.time.format.DateTimeFormatterBuilder; +import java.time.format.ResolverStyle; +import java.time.format.SignStyle; +import java.time.temporal.ChronoField; +import java.time.temporal.ChronoUnit; +import java.time.temporal.Temporal; +import java.time.temporal.TemporalAccessor; +import java.time.temporal.TemporalField; +import java.time.temporal.TemporalUnit; +import java.time.temporal.ValueRange; +import java.util.Locale; +import java.util.Map; + +/** + * This class provides {@link DateTimeFormatter}s capable of parsing epoch seconds and milliseconds. + *

+ * The seconds formatter is provided by {@link #SECONDS_FORMATTER}. + * The milliseconds formatter is provided by {@link #MILLIS_FORMATTER}. + *

+ * Both formatters support fractional time, up to nanosecond precision. Values must be positive numbers. + */ +class EpochTime { + + private static final ValueRange LONG_POSITIVE_RANGE = ValueRange.of(0, Long.MAX_VALUE); + + private static final EpochField SECONDS = new EpochField(ChronoUnit.SECONDS, ChronoUnit.FOREVER, LONG_POSITIVE_RANGE) { + @Override + public boolean isSupportedBy(TemporalAccessor temporal) { + return temporal.isSupported(ChronoField.INSTANT_SECONDS); + } + @Override + public long getFrom(TemporalAccessor temporal) { + return temporal.getLong(ChronoField.INSTANT_SECONDS); + } + @Override + public TemporalAccessor resolve(Map fieldValues, + TemporalAccessor partialTemporal, ResolverStyle resolverStyle) { + long seconds = fieldValues.remove(this); + fieldValues.put(ChronoField.INSTANT_SECONDS, seconds); + Long nanos = fieldValues.remove(NANOS_OF_SECOND); + if (nanos != null) { + fieldValues.put(ChronoField.NANO_OF_SECOND, nanos); + } + return null; + } + }; + + private static final EpochField NANOS_OF_SECOND = new EpochField(ChronoUnit.NANOS, ChronoUnit.SECONDS, ValueRange.of(0, 999_999_999)) { + @Override + public boolean isSupportedBy(TemporalAccessor temporal) { + return temporal.isSupported(ChronoField.NANO_OF_SECOND) && temporal.getLong(ChronoField.NANO_OF_SECOND) != 0; + } + @Override + public long getFrom(TemporalAccessor temporal) { + return temporal.getLong(ChronoField.NANO_OF_SECOND); + } + }; + + private static final EpochField MILLIS = new EpochField(ChronoUnit.MILLIS, ChronoUnit.FOREVER, LONG_POSITIVE_RANGE) { + @Override + public boolean isSupportedBy(TemporalAccessor temporal) { + return temporal.isSupported(ChronoField.INSTANT_SECONDS) && temporal.isSupported(ChronoField.MILLI_OF_SECOND); + } + @Override + public long getFrom(TemporalAccessor temporal) { + return temporal.getLong(ChronoField.INSTANT_SECONDS) * 1_000 + temporal.getLong(ChronoField.MILLI_OF_SECOND); + } + @Override + public TemporalAccessor resolve(Map fieldValues, + TemporalAccessor partialTemporal, ResolverStyle resolverStyle) { + long secondsAndMillis = fieldValues.remove(this); + long seconds = secondsAndMillis / 1_000; + long nanos = secondsAndMillis % 1000 * 1_000_000; + Long nanosOfMilli = fieldValues.remove(NANOS_OF_MILLI); + if (nanosOfMilli != null) { + nanos += nanosOfMilli; + } + fieldValues.put(ChronoField.INSTANT_SECONDS, seconds); + fieldValues.put(ChronoField.NANO_OF_SECOND, nanos); + return null; + } + }; + + private static final EpochField NANOS_OF_MILLI = new EpochField(ChronoUnit.NANOS, ChronoUnit.MILLIS, ValueRange.of(0, 999_999)) { + @Override + public boolean isSupportedBy(TemporalAccessor temporal) { + return temporal.isSupported(ChronoField.NANO_OF_SECOND) && temporal.getLong(ChronoField.NANO_OF_SECOND) % 1_000_000 != 0; + } + @Override + public long getFrom(TemporalAccessor temporal) { + return temporal.getLong(ChronoField.NANO_OF_SECOND); + } + }; + + // this supports seconds without any fraction + private static final DateTimeFormatter SECONDS_FORMATTER1 = new DateTimeFormatterBuilder() + .appendValue(SECONDS, 1, 19, SignStyle.NORMAL) + .toFormatter(Locale.ROOT); + + // this supports seconds ending in dot + private static final DateTimeFormatter SECONDS_FORMATTER2 = new DateTimeFormatterBuilder() + .append(SECONDS_FORMATTER1) + .appendLiteral('.') + .toFormatter(Locale.ROOT); + + // this supports seconds with a fraction and is also used for printing + private static final DateTimeFormatter SECONDS_FORMATTER3 = new DateTimeFormatterBuilder() + .append(SECONDS_FORMATTER1) + .optionalStart() // optional is used so isSupported will be called when printing + .appendFraction(NANOS_OF_SECOND, 1, 9, true) + .optionalEnd() + .toFormatter(Locale.ROOT); + + // this supports milliseconds without any fraction + private static final DateTimeFormatter MILLISECONDS_FORMATTER1 = new DateTimeFormatterBuilder() + .appendValue(MILLIS, 1, 19, SignStyle.NORMAL) + .toFormatter(Locale.ROOT); + + // this supports milliseconds ending in dot + private static final DateTimeFormatter MILLISECONDS_FORMATTER2 = new DateTimeFormatterBuilder() + .append(MILLISECONDS_FORMATTER1) + .appendLiteral('.') + .toFormatter(Locale.ROOT); + + // this supports milliseconds with a fraction and is also used for printing + private static final DateTimeFormatter MILLISECONDS_FORMATTER3 = new DateTimeFormatterBuilder() + .append(MILLISECONDS_FORMATTER1) + .optionalStart() // optional is used so isSupported will be called when printing + .appendFraction(NANOS_OF_MILLI, 1, 6, true) + .optionalEnd() + .toFormatter(Locale.ROOT); + + static final DateFormatter SECONDS_FORMATTER = new JavaDateFormatter("epoch_second", SECONDS_FORMATTER3, + SECONDS_FORMATTER1, SECONDS_FORMATTER2, SECONDS_FORMATTER3); + + static final DateFormatter MILLIS_FORMATTER = new JavaDateFormatter("epoch_millis", MILLISECONDS_FORMATTER3, + MILLISECONDS_FORMATTER1, MILLISECONDS_FORMATTER2, MILLISECONDS_FORMATTER3); + + private abstract static class EpochField implements TemporalField { + + private final TemporalUnit baseUnit; + private final TemporalUnit rangeUnit; + private final ValueRange range; + + private EpochField(TemporalUnit baseUnit, TemporalUnit rangeUnit, ValueRange range) { + this.baseUnit = baseUnit; + this.rangeUnit = rangeUnit; + this.range = range; + } + + @Override + public String getDisplayName(Locale locale) { + return toString(); + } + + @Override + public String toString() { + return "Epoch" + baseUnit.toString() + (rangeUnit != ChronoUnit.FOREVER ? "Of" + rangeUnit.toString() : ""); + } + + @Override + public TemporalUnit getBaseUnit() { + return baseUnit; + } + + @Override + public TemporalUnit getRangeUnit() { + return rangeUnit; + } + + @Override + public ValueRange range() { + return range; + } + + @Override + public boolean isDateBased() { + return false; + } + + @Override + public boolean isTimeBased() { + return true; + } + + @Override + public ValueRange rangeRefinedBy(TemporalAccessor temporal) { + return range(); + } + + @SuppressWarnings("unchecked") + @Override + public R adjustInto(R temporal, long newValue) { + return (R) temporal.with(this, newValue); + } + } +} diff --git a/server/src/test/java/org/elasticsearch/common/joda/JavaJodaTimeDuellingTests.java b/server/src/test/java/org/elasticsearch/common/joda/JavaJodaTimeDuellingTests.java index 5840f7ef60191..23674ec85b44f 100644 --- a/server/src/test/java/org/elasticsearch/common/joda/JavaJodaTimeDuellingTests.java +++ b/server/src/test/java/org/elasticsearch/common/joda/JavaJodaTimeDuellingTests.java @@ -19,6 +19,7 @@ package org.elasticsearch.common.joda; +import org.elasticsearch.bootstrap.JavaVersion; import org.elasticsearch.common.time.DateFormatter; import org.elasticsearch.common.time.DateFormatters; import org.elasticsearch.test.ESTestCase; @@ -384,6 +385,7 @@ public void testSamePrinterOutput() { ZonedDateTime javaDate = ZonedDateTime.of(year, month, day, hour, minute, second, 0, ZoneOffset.UTC); DateTime jodaDate = new DateTime(year, month, day, hour, minute, second, DateTimeZone.UTC); + assertSamePrinterOutput("epoch_second", javaDate, jodaDate); assertSamePrinterOutput("basicDate", javaDate, jodaDate); assertSamePrinterOutput("basicDateTime", javaDate, jodaDate); @@ -428,7 +430,7 @@ public void testSamePrinterOutput() { assertSamePrinterOutput("year", javaDate, jodaDate); assertSamePrinterOutput("yearMonth", javaDate, jodaDate); assertSamePrinterOutput("yearMonthDay", javaDate, jodaDate); - assertSamePrinterOutput("epoch_second", javaDate, jodaDate); + assertSamePrinterOutput("epoch_millis", javaDate, jodaDate); assertSamePrinterOutput("strictBasicWeekDate", javaDate, jodaDate); assertSamePrinterOutput("strictBasicWeekDateTime", javaDate, jodaDate); @@ -476,6 +478,12 @@ private void assertSamePrinterOutput(String format, ZonedDateTime javaDate, Date assertThat(jodaDate.getMillis(), is(javaDate.toInstant().toEpochMilli())); String javaTimeOut = DateFormatters.forPattern(format).format(javaDate); String jodaTimeOut = DateFormatter.forPattern(format).formatJoda(jodaDate); + if (JavaVersion.current().getVersion().get(0) == 8 && javaTimeOut.endsWith(".0") + && (format.equals("epoch_second") || format.equals("epoch_millis"))) { + // java 8 has a bug in DateTimeFormatter usage when printing dates that rely on isSupportedBy for fields, which is + // what we use for epoch time. This change accounts for that bug. It should be removed when java 8 support is removed + jodaTimeOut += ".0"; + } String message = String.format(Locale.ROOT, "expected string representation to be equal for format [%s]: joda [%s], java [%s]", format, jodaTimeOut, javaTimeOut); assertThat(message, javaTimeOut, is(jodaTimeOut)); @@ -484,7 +492,6 @@ private void assertSamePrinterOutput(String format, ZonedDateTime javaDate, Date private void assertSameDate(String input, String format) { DateFormatter jodaFormatter = Joda.forPattern(format); DateFormatter javaFormatter = DateFormatters.forPattern(format); - assertSameDate(input, format, jodaFormatter, javaFormatter); } diff --git a/server/src/test/java/org/elasticsearch/common/joda/JodaTests.java b/server/src/test/java/org/elasticsearch/common/joda/JodaTests.java index a00bffe2bf6f2..fde9d73fae892 100644 --- a/server/src/test/java/org/elasticsearch/common/joda/JodaTests.java +++ b/server/src/test/java/org/elasticsearch/common/joda/JodaTests.java @@ -29,7 +29,6 @@ public class JodaTests extends ESTestCase { - public void testBasicTTimePattern() { DateFormatter formatter1 = DateFormatter.forPattern("basic_t_time"); assertEquals(formatter1.pattern(), "basic_t_time"); diff --git a/server/src/test/java/org/elasticsearch/common/time/DateFormattersTests.java b/server/src/test/java/org/elasticsearch/common/time/DateFormattersTests.java index 0f58e30f7a2bf..b60e6b27eca03 100644 --- a/server/src/test/java/org/elasticsearch/common/time/DateFormattersTests.java +++ b/server/src/test/java/org/elasticsearch/common/time/DateFormattersTests.java @@ -23,7 +23,6 @@ import java.time.Instant; import java.time.ZoneId; -import java.time.ZoneOffset; import java.time.format.DateTimeParseException; import java.time.temporal.TemporalAccessor; import java.util.Locale; @@ -58,21 +57,6 @@ public void testEpochMillisParser() { assertThat(instant.getEpochSecond(), is(12L)); assertThat(instant.getNano(), is(345_000_000)); } - { - Instant instant = Instant.from(formatter.parse("-12345.6789")); - assertThat(instant.getEpochSecond(), is(-13L)); - assertThat(instant.getNano(), is(1_000_000_000 - 345_678_900)); - } - { - Instant instant = Instant.from(formatter.parse("-436134.241272")); - assertThat(instant.getEpochSecond(), is(-437L)); - assertThat(instant.getNano(), is(1_000_000_000 - 134_241_272)); - } - { - Instant instant = Instant.from(formatter.parse("-12345")); - assertThat(instant.getEpochSecond(), is(-13L)); - assertThat(instant.getNano(), is(1_000_000_000 - 345_000_000)); - } { Instant instant = Instant.from(formatter.parse("0")); assertThat(instant.getEpochSecond(), is(0L)); @@ -83,10 +67,10 @@ public void testEpochMillisParser() { public void testEpochMilliParser() { DateFormatter formatter = DateFormatters.forPattern("epoch_millis"); DateTimeParseException e = expectThrows(DateTimeParseException.class, () -> formatter.parse("invalid")); - assertThat(e.getMessage(), containsString("invalid number")); + assertThat(e.getMessage(), containsString("could not be parsed")); e = expectThrows(DateTimeParseException.class, () -> formatter.parse("123.1234567")); - assertThat(e.getMessage(), containsString("too much granularity after dot [123.1234567]")); + assertThat(e.getMessage(), containsString("unparsed text found at index 3")); } // this is not in the duelling tests, because the epoch second parser in joda time drops the milliseconds after the comma @@ -108,17 +92,14 @@ public void testEpochSecondParser() { assertThat(Instant.from(formatter.parse("1234.12345678")).getNano(), is(123_456_780)); assertThat(Instant.from(formatter.parse("1234.123456789")).getNano(), is(123_456_789)); - assertThat(Instant.from(formatter.parse("-1234.567")).toEpochMilli(), is(-1234567L)); - assertThat(Instant.from(formatter.parse("-1234")).getNano(), is(0)); - DateTimeParseException e = expectThrows(DateTimeParseException.class, () -> formatter.parse("1234.1234567890")); - assertThat(e.getMessage(), is("too much granularity after dot [1234.1234567890]")); + assertThat(e.getMessage(), is("Text '1234.1234567890' could not be parsed, unparsed text found at index 4")); e = expectThrows(DateTimeParseException.class, () -> formatter.parse("1234.123456789013221")); - assertThat(e.getMessage(), is("too much granularity after dot [1234.123456789013221]")); + assertThat(e.getMessage(), is("Text '1234.123456789013221' could not be parsed, unparsed text found at index 4")); e = expectThrows(DateTimeParseException.class, () -> formatter.parse("abc")); - assertThat(e.getMessage(), is("invalid number [abc]")); + assertThat(e.getMessage(), is("Text 'abc' could not be parsed at index 0")); e = expectThrows(DateTimeParseException.class, () -> formatter.parse("1234.abc")); - assertThat(e.getMessage(), is("invalid number [1234.abc]")); + assertThat(e.getMessage(), is("Text '1234.abc' could not be parsed, unparsed text found at index 4")); } public void testEpochMilliParsersWithDifferentFormatters() { @@ -132,18 +113,6 @@ public void testLocales() { assertThat(DateFormatters.forPattern("strict_date_optional_time").locale(), is(Locale.ROOT)); Locale locale = randomLocale(random()); assertThat(DateFormatters.forPattern("strict_date_optional_time").withLocale(locale).locale(), is(locale)); - if (locale.equals(Locale.ROOT)) { - DateFormatter millisFormatter = DateFormatters.forPattern("epoch_millis"); - assertThat(millisFormatter.withLocale(locale), is(millisFormatter)); - DateFormatter secondFormatter = DateFormatters.forPattern("epoch_second"); - assertThat(secondFormatter.withLocale(locale), is(secondFormatter)); - } else { - IllegalArgumentException e = - expectThrows(IllegalArgumentException.class, () -> DateFormatters.forPattern("epoch_millis").withLocale(locale)); - assertThat(e.getMessage(), is("epoch_millis date formatter can only be in locale ROOT")); - e = expectThrows(IllegalArgumentException.class, () -> DateFormatters.forPattern("epoch_second").withLocale(locale)); - assertThat(e.getMessage(), is("epoch_second date formatter can only be in locale ROOT")); - } } public void testTimeZones() { @@ -151,18 +120,6 @@ public void testTimeZones() { assertThat(DateFormatters.forPattern("strict_date_optional_time").zone(), is(nullValue())); ZoneId zoneId = randomZone(); assertThat(DateFormatters.forPattern("strict_date_optional_time").withZone(zoneId).zone(), is(zoneId)); - if (zoneId.equals(ZoneOffset.UTC)) { - DateFormatter millisFormatter = DateFormatters.forPattern("epoch_millis"); - assertThat(millisFormatter.withZone(zoneId), is(millisFormatter)); - DateFormatter secondFormatter = DateFormatters.forPattern("epoch_second"); - assertThat(secondFormatter.withZone(zoneId), is(secondFormatter)); - } else { - IllegalArgumentException e = - expectThrows(IllegalArgumentException.class, () -> DateFormatters.forPattern("epoch_millis").withZone(zoneId)); - assertThat(e.getMessage(), is("epoch_millis date formatter can only be in zone offset UTC")); - e = expectThrows(IllegalArgumentException.class, () -> DateFormatters.forPattern("epoch_second").withZone(zoneId)); - assertThat(e.getMessage(), is("epoch_second date formatter can only be in zone offset UTC")); - } } public void testEqualsAndHashcode() { diff --git a/server/src/test/java/org/elasticsearch/search/fields/SearchFieldsIT.java b/server/src/test/java/org/elasticsearch/search/fields/SearchFieldsIT.java index 5f5f742bfd630..8a90ca0b8ca47 100644 --- a/server/src/test/java/org/elasticsearch/search/fields/SearchFieldsIT.java +++ b/server/src/test/java/org/elasticsearch/search/fields/SearchFieldsIT.java @@ -28,6 +28,7 @@ import org.elasticsearch.common.bytes.BytesReference; import org.elasticsearch.common.collect.MapBuilder; import org.elasticsearch.common.document.DocumentField; +import org.elasticsearch.common.joda.Joda; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.time.DateFormatters; import org.elasticsearch.common.xcontent.XContentBuilder; @@ -897,8 +898,9 @@ public void testDocValueFields() throws Exception { assertThat(searchResponse.getHits().getAt(0).getFields().get("long_field").getValue(), equalTo("4.0")); assertThat(searchResponse.getHits().getAt(0).getFields().get("float_field").getValue(), equalTo("5.0")); assertThat(searchResponse.getHits().getAt(0).getFields().get("double_field").getValue(), equalTo("6.0")); + // TODO: switch to java date formatter, but will require special casing java 8 as there is a bug with epoch formatting there assertThat(searchResponse.getHits().getAt(0).getFields().get("date_field").getValue(), - equalTo(DateFormatters.forPattern("epoch_millis").format(date))); + equalTo(Joda.forPattern("epoch_millis").format(date))); } public void testScriptFields() throws Exception { From c5aac4705d7e5afff701830f1a9e1e82765db6eb Mon Sep 17 00:00:00 2001 From: Julie Tibshirani Date: Mon, 7 Jan 2019 17:56:40 -0800 Subject: [PATCH 013/186] Revert "Stop automatically nesting mappings in index creation requests. (#36924)" This reverts commit ac1c6940d20b9ac993422b279d8f3a8bd73a3527. --- .../indices/create/CreateIndexRequest.java | 7 +- .../mapping/put/PutMappingRequest.java | 12 -- .../admin/indices/create/CreateIndexIT.java | 16 +- .../create/CreateIndexRequestTests.java | 149 ++++++++++-------- .../index/RandomCreateIndexGenerator.java | 20 +-- 5 files changed, 93 insertions(+), 111 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/action/admin/indices/create/CreateIndexRequest.java b/server/src/main/java/org/elasticsearch/action/admin/indices/create/CreateIndexRequest.java index bf82d70521648..1bbce19ee8dec 100644 --- a/server/src/main/java/org/elasticsearch/action/admin/indices/create/CreateIndexRequest.java +++ b/server/src/main/java/org/elasticsearch/action/admin/indices/create/CreateIndexRequest.java @@ -33,6 +33,7 @@ import org.elasticsearch.common.Strings; import org.elasticsearch.common.bytes.BytesArray; import org.elasticsearch.common.bytes.BytesReference; +import org.elasticsearch.common.collect.MapBuilder; import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.io.stream.StreamOutput; import org.elasticsearch.common.settings.Settings; @@ -255,6 +256,10 @@ public CreateIndexRequest mapping(String type, Map source) { if (mappings.containsKey(type)) { throw new IllegalStateException("mappings for type \"" + type + "\" were already defined"); } + // wrap it in a type map if its not + if (source.size() != 1 || !source.containsKey(type)) { + source = MapBuilder.newMapBuilder().put(type, source).map(); + } try { XContentBuilder builder = XContentFactory.contentBuilder(XContentType.JSON); builder.map(source); @@ -269,7 +274,7 @@ public CreateIndexRequest mapping(String type, Map source) { * ("field1", "type=string,store=true"). */ public CreateIndexRequest mapping(String type, Object... source) { - mapping(type, PutMappingRequest.buildFromSimplifiedDef(source)); + mapping(type, PutMappingRequest.buildFromSimplifiedDef(type, source)); return this; } diff --git a/server/src/main/java/org/elasticsearch/action/admin/indices/mapping/put/PutMappingRequest.java b/server/src/main/java/org/elasticsearch/action/admin/indices/mapping/put/PutMappingRequest.java index 237e720ae2a58..926ae175d65ad 100644 --- a/server/src/main/java/org/elasticsearch/action/admin/indices/mapping/put/PutMappingRequest.java +++ b/server/src/main/java/org/elasticsearch/action/admin/indices/mapping/put/PutMappingRequest.java @@ -184,18 +184,6 @@ public PutMappingRequest source(Object... source) { return source(buildFromSimplifiedDef(type, source)); } - /** - * @param source - * consisting of field/properties pairs (e.g. "field1", - * "type=string,store=true") - * @throws IllegalArgumentException - * if the number of the source arguments is not divisible by two - * @return the mappings definition - */ - public static XContentBuilder buildFromSimplifiedDef(Object... source) { - return buildFromSimplifiedDef(null, source); - } - /** * @param type * the mapping type diff --git a/server/src/test/java/org/elasticsearch/action/admin/indices/create/CreateIndexIT.java b/server/src/test/java/org/elasticsearch/action/admin/indices/create/CreateIndexIT.java index ada78767ca501..05da57cc5da45 100644 --- a/server/src/test/java/org/elasticsearch/action/admin/indices/create/CreateIndexIT.java +++ b/server/src/test/java/org/elasticsearch/action/admin/indices/create/CreateIndexIT.java @@ -143,7 +143,7 @@ public void testNonNestedMappings() throws Exception { assertFalse(metadata.sourceAsMap().isEmpty()); } - public void testNonNestedEmptyMappings() throws Exception { + public void testEmptyNestedMappings() throws Exception { assertAcked(prepareCreate("test") .addMapping("_doc", XContentFactory.jsonBuilder().startObject().endObject())); @@ -173,20 +173,6 @@ public void testEmptyMappings() throws Exception { assertTrue(metadata.sourceAsMap().isEmpty()); } - public void testFlatMappingFormat() throws Exception { - assertAcked(prepareCreate("test") - .addMapping("_doc", "field", "type=keyword")); - - GetMappingsResponse response = client().admin().indices().prepareGetMappings("test").get(); - - ImmutableOpenMap mappings = response.mappings().get("test"); - assertNotNull(mappings); - - MappingMetaData metadata = mappings.get("_doc"); - assertNotNull(metadata); - assertFalse(metadata.sourceAsMap().isEmpty()); - } - public void testInvalidShardCountSettings() throws Exception { int value = randomIntBetween(-10, 0); try { diff --git a/server/src/test/java/org/elasticsearch/action/admin/indices/create/CreateIndexRequestTests.java b/server/src/test/java/org/elasticsearch/action/admin/indices/create/CreateIndexRequestTests.java index 64b77d0dc349b..1c27934927413 100644 --- a/server/src/test/java/org/elasticsearch/action/admin/indices/create/CreateIndexRequestTests.java +++ b/server/src/test/java/org/elasticsearch/action/admin/indices/create/CreateIndexRequestTests.java @@ -22,86 +22,31 @@ import org.elasticsearch.ElasticsearchParseException; import org.elasticsearch.action.admin.indices.alias.Alias; import org.elasticsearch.common.Strings; +import org.elasticsearch.common.bytes.BytesReference; import org.elasticsearch.common.io.stream.BytesStreamOutput; import org.elasticsearch.common.io.stream.StreamInput; +import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.xcontent.LoggingDeprecationHandler; import org.elasticsearch.common.xcontent.NamedXContentRegistry; import org.elasticsearch.common.xcontent.XContentParser; import org.elasticsearch.common.xcontent.XContentType; import org.elasticsearch.common.xcontent.json.JsonXContent; import org.elasticsearch.index.RandomCreateIndexGenerator; -import org.elasticsearch.test.AbstractXContentTestCase; +import org.elasticsearch.test.ESTestCase; +import org.elasticsearch.test.hamcrest.ElasticsearchAssertions; import java.io.IOException; import java.util.Map; import java.util.Set; -public class CreateIndexRequestTests extends AbstractXContentTestCase { +import static org.elasticsearch.cluster.metadata.IndexMetaData.SETTING_NUMBER_OF_SHARDS; +import static org.elasticsearch.common.xcontent.ToXContent.EMPTY_PARAMS; - @Override - protected CreateIndexRequest createTestInstance() { - try { - return RandomCreateIndexGenerator.randomCreateIndexRequest(); - } catch (IOException e) { - throw new RuntimeException(e); - } - } - - @Override - protected CreateIndexRequest doParseInstance(XContentParser parser) throws IOException { - CreateIndexRequest request = new CreateIndexRequest(); - request.source(parser.map(), LoggingDeprecationHandler.INSTANCE); - return request; - } - - @Override - protected void assertEqualInstances(CreateIndexRequest expectedInstance, CreateIndexRequest newInstance) { - assertEquals(expectedInstance.settings(), newInstance.settings()); - assertAliasesEqual(expectedInstance.aliases(), newInstance.aliases()); - assertMappingsEqual(expectedInstance.mappings(), newInstance.mappings()); - } - - @Override - protected boolean supportsUnknownFields() { - return false; - } - - public static void assertMappingsEqual(Map expected, Map actual) { - assertEquals(expected.keySet(), actual.keySet()); - - for (Map.Entry expectedEntry : expected.entrySet()) { - String expectedValue = expectedEntry.getValue(); - String actualValue = actual.get(expectedEntry.getKey()); - try (XContentParser expectedJson = JsonXContent.jsonXContent.createParser(NamedXContentRegistry.EMPTY, - LoggingDeprecationHandler.INSTANCE, expectedValue); - XContentParser actualJson = JsonXContent.jsonXContent.createParser(NamedXContentRegistry.EMPTY, - LoggingDeprecationHandler.INSTANCE, actualValue)) { - assertEquals(expectedJson.map(), actualJson.map()); - } catch (IOException e) { - throw new RuntimeException(e); - } - } - } - - public static void assertAliasesEqual(Set expected, Set actual) { - assertEquals(expected, actual); - - for (Alias expectedAlias : expected) { - for (Alias actualAlias : actual) { - if (expectedAlias.equals(actualAlias)) { - // As Alias#equals only looks at name, we check the equality of the other Alias parameters here. - assertEquals(expectedAlias.filter(), actualAlias.filter()); - assertEquals(expectedAlias.indexRouting(), actualAlias.indexRouting()); - assertEquals(expectedAlias.searchRouting(), actualAlias.searchRouting()); - } - } - } - } +public class CreateIndexRequestTests extends ESTestCase { public void testSerialization() throws IOException { CreateIndexRequest request = new CreateIndexRequest("foo"); - String mapping = Strings.toString(JsonXContent.contentBuilder().startObject() - .startObject("type").endObject().endObject()); + String mapping = Strings.toString(JsonXContent.contentBuilder().startObject().startObject("type").endObject().endObject()); request.mapping("my_type", mapping, XContentType.JSON); try (BytesStreamOutput output = new BytesStreamOutput()) { @@ -118,7 +63,7 @@ public void testSerialization() throws IOException { public void testTopLevelKeys() { String createIndex = - "{\n" + "{\n" + " \"FOO_SHOULD_BE_ILLEGAL_HERE\": {\n" + " \"BAR_IS_THE_SAME\": 42\n" + " },\n" @@ -135,7 +80,81 @@ public void testTopLevelKeys() { CreateIndexRequest request = new CreateIndexRequest(); ElasticsearchParseException e = expectThrows(ElasticsearchParseException.class, - () -> {request.source(createIndex, XContentType.JSON);}); + () -> {request.source(createIndex, XContentType.JSON);}); assertEquals("unknown key [FOO_SHOULD_BE_ILLEGAL_HERE] for create index", e.getMessage()); } + + public void testToXContent() throws IOException { + CreateIndexRequest request = new CreateIndexRequest("foo"); + + String mapping = Strings.toString(JsonXContent.contentBuilder().startObject().startObject("type").endObject().endObject()); + request.mapping("my_type", mapping, XContentType.JSON); + + Alias alias = new Alias("test_alias"); + alias.routing("1"); + alias.filter("{\"term\":{\"year\":2016}}"); + alias.writeIndex(true); + request.alias(alias); + + Settings.Builder settings = Settings.builder(); + settings.put(SETTING_NUMBER_OF_SHARDS, 10); + request.settings(settings); + + String actualRequestBody = Strings.toString(request); + + String expectedRequestBody = "{\"settings\":{\"index\":{\"number_of_shards\":\"10\"}}," + + "\"mappings\":{\"my_type\":{\"type\":{}}}," + + "\"aliases\":{\"test_alias\":{\"filter\":{\"term\":{\"year\":2016}},\"routing\":\"1\",\"is_write_index\":true}}}"; + + assertEquals(expectedRequestBody, actualRequestBody); + } + + public void testToAndFromXContent() throws IOException { + + final CreateIndexRequest createIndexRequest = RandomCreateIndexGenerator.randomCreateIndexRequest(); + + boolean humanReadable = randomBoolean(); + final XContentType xContentType = randomFrom(XContentType.values()); + BytesReference originalBytes = toShuffledXContent(createIndexRequest, xContentType, EMPTY_PARAMS, humanReadable); + + CreateIndexRequest parsedCreateIndexRequest = new CreateIndexRequest(); + parsedCreateIndexRequest.source(originalBytes, xContentType); + + assertMappingsEqual(createIndexRequest.mappings(), parsedCreateIndexRequest.mappings()); + assertAliasesEqual(createIndexRequest.aliases(), parsedCreateIndexRequest.aliases()); + assertEquals(createIndexRequest.settings(), parsedCreateIndexRequest.settings()); + + BytesReference finalBytes = toShuffledXContent(parsedCreateIndexRequest, xContentType, EMPTY_PARAMS, humanReadable); + ElasticsearchAssertions.assertToXContentEquivalent(originalBytes, finalBytes, xContentType); + } + + public static void assertMappingsEqual(Map expected, Map actual) throws IOException { + assertEquals(expected.keySet(), actual.keySet()); + + for (Map.Entry expectedEntry : expected.entrySet()) { + String expectedValue = expectedEntry.getValue(); + String actualValue = actual.get(expectedEntry.getKey()); + try (XContentParser expectedJson = JsonXContent.jsonXContent.createParser(NamedXContentRegistry.EMPTY, + LoggingDeprecationHandler.INSTANCE, expectedValue); + XContentParser actualJson = JsonXContent.jsonXContent.createParser(NamedXContentRegistry.EMPTY, + LoggingDeprecationHandler.INSTANCE, actualValue)){ + assertEquals(expectedJson.map(), actualJson.map()); + } + } + } + + public static void assertAliasesEqual(Set expected, Set actual) throws IOException { + assertEquals(expected, actual); + + for (Alias expectedAlias : expected) { + for (Alias actualAlias : actual) { + if (expectedAlias.equals(actualAlias)) { + // As Alias#equals only looks at name, we check the equality of the other Alias parameters here. + assertEquals(expectedAlias.filter(), actualAlias.filter()); + assertEquals(expectedAlias.indexRouting(), actualAlias.indexRouting()); + assertEquals(expectedAlias.searchRouting(), actualAlias.searchRouting()); + } + } + } + } } diff --git a/test/framework/src/main/java/org/elasticsearch/index/RandomCreateIndexGenerator.java b/test/framework/src/main/java/org/elasticsearch/index/RandomCreateIndexGenerator.java index 27b7db3d36a03..e88a9f0a38d2c 100644 --- a/test/framework/src/main/java/org/elasticsearch/index/RandomCreateIndexGenerator.java +++ b/test/framework/src/main/java/org/elasticsearch/index/RandomCreateIndexGenerator.java @@ -29,7 +29,6 @@ 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.test.ESTestCase.frequently; import static org.elasticsearch.test.ESTestCase.randomAlphaOfLength; import static org.elasticsearch.test.ESTestCase.randomBoolean; import static org.elasticsearch.test.ESTestCase.randomIntBetween; @@ -46,14 +45,9 @@ public static CreateIndexRequest randomCreateIndexRequest() throws IOException { String index = randomAlphaOfLength(5); CreateIndexRequest request = new CreateIndexRequest(index); randomAliases(request); - if (frequently()) { + if (randomBoolean()) { String type = randomAlphaOfLength(5); - if (randomBoolean()) { - request.mapping(type, randomMapping()); - } else { - request.mapping(type, randomMapping(type)); - - } + request.mapping(type, randomMapping(type)); } if (randomBoolean()) { request.settings(randomIndexSettings()); @@ -82,16 +76,6 @@ public static Settings randomIndexSettings() { return builder.build(); } - public static XContentBuilder randomMapping() throws IOException { - XContentBuilder builder = XContentFactory.jsonBuilder(); - builder.startObject(); - - randomMappingFields(builder, true); - - builder.endObject(); - return builder; - } - public static XContentBuilder randomMapping(String type) throws IOException { XContentBuilder builder = XContentFactory.jsonBuilder(); builder.startObject().startObject(type); From c8c596cead495c7cd7c1834ef5a871e4bf2d6251 Mon Sep 17 00:00:00 2001 From: Jason Tedor Date: Mon, 7 Jan 2019 22:03:52 -0800 Subject: [PATCH 014/186] Introduce retention lease expiration (#37195) This commit implements a straightforward approach to retention lease expiration. Namely, we inspect which leases are expired when obtaining the current leases through the replication tracker. At that moment, we clean the map that persists the retention leases in memory. --- .../common/settings/IndexScopedSettings.java | 1 + .../elasticsearch/index/IndexSettings.java | 24 ++++++ .../index/seqno/ReplicationTracker.java | 26 ++++-- .../index/seqno/RetentionLease.java | 22 ++++- .../elasticsearch/index/shard/IndexShard.java | 8 +- .../index/engine/SoftDeletesPolicyTests.java | 2 +- ...ReplicationTrackerRetentionLeaseTests.java | 71 +++++++++++++++- .../seqno/ReplicationTrackerTestCase.java | 9 +- .../index/seqno/ReplicationTrackerTests.java | 10 +-- .../shard/IndexShardRetentionLeaseTests.java | 84 ++++++++++++++++++- .../index/engine/EngineTestCase.java | 4 +- .../index/shard/IndexShardTestCase.java | 12 ++- .../action/TransportResumeFollowAction.java | 1 + 13 files changed, 245 insertions(+), 29 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/common/settings/IndexScopedSettings.java b/server/src/main/java/org/elasticsearch/common/settings/IndexScopedSettings.java index c4e0257c95395..1d2e54ae86d7c 100644 --- a/server/src/main/java/org/elasticsearch/common/settings/IndexScopedSettings.java +++ b/server/src/main/java/org/elasticsearch/common/settings/IndexScopedSettings.java @@ -133,6 +133,7 @@ public final class IndexScopedSettings extends AbstractScopedSettings { IndexSettings.INDEX_GC_DELETES_SETTING, IndexSettings.INDEX_SOFT_DELETES_SETTING, IndexSettings.INDEX_SOFT_DELETES_RETENTION_OPERATIONS_SETTING, + IndexSettings.INDEX_SOFT_DELETES_RETENTION_LEASE_SETTING, IndicesRequestCache.INDEX_CACHE_REQUEST_ENABLED_SETTING, UnassignedInfo.INDEX_DELAYED_NODE_LEFT_TIMEOUT_SETTING, EnableAllocationDecider.INDEX_ROUTING_REBALANCE_ENABLE_SETTING, diff --git a/server/src/main/java/org/elasticsearch/index/IndexSettings.java b/server/src/main/java/org/elasticsearch/index/IndexSettings.java index ead9e7597fd73..4d9a8f7d37b70 100644 --- a/server/src/main/java/org/elasticsearch/index/IndexSettings.java +++ b/server/src/main/java/org/elasticsearch/index/IndexSettings.java @@ -256,6 +256,17 @@ public final class IndexSettings { Setting.longSetting("index.soft_deletes.retention.operations", 0, 0, Property.IndexScope, Property.Dynamic); + /** + * Controls the maximum length of time since a retention lease is created or renewed before it is considered expired. + */ + public static final Setting INDEX_SOFT_DELETES_RETENTION_LEASE_SETTING = + Setting.timeSetting( + "index.soft_deletes.retention.lease", + TimeValue.timeValueHours(12), + TimeValue.ZERO, + Property.Dynamic, + Property.IndexScope); + /** * The maximum number of refresh listeners allows on this shard. */ @@ -316,6 +327,18 @@ public final class IndexSettings { private long gcDeletesInMillis = DEFAULT_GC_DELETES.millis(); private final boolean softDeleteEnabled; private volatile long softDeleteRetentionOperations; + + private volatile long retentionLeaseMillis; + + /** + * The maximum age of a retention lease before it is considered expired. + * + * @return the maximum age + */ + public long getRetentionLeaseMillis() { + return retentionLeaseMillis; + } + private volatile boolean warmerEnabled; private volatile int maxResultWindow; private volatile int maxInnerResultWindow; @@ -431,6 +454,7 @@ public IndexSettings(final IndexMetaData indexMetaData, final Settings nodeSetti gcDeletesInMillis = scopedSettings.get(INDEX_GC_DELETES_SETTING).getMillis(); softDeleteEnabled = version.onOrAfter(Version.V_6_5_0) && scopedSettings.get(INDEX_SOFT_DELETES_SETTING); softDeleteRetentionOperations = scopedSettings.get(INDEX_SOFT_DELETES_RETENTION_OPERATIONS_SETTING); + retentionLeaseMillis = scopedSettings.get(INDEX_SOFT_DELETES_RETENTION_LEASE_SETTING).millis(); warmerEnabled = scopedSettings.get(INDEX_WARMER_ENABLED_SETTING); maxResultWindow = scopedSettings.get(MAX_RESULT_WINDOW_SETTING); maxInnerResultWindow = scopedSettings.get(MAX_INNER_RESULT_WINDOW_SETTING); diff --git a/server/src/main/java/org/elasticsearch/index/seqno/ReplicationTracker.java b/server/src/main/java/org/elasticsearch/index/seqno/ReplicationTracker.java index 7168bb772dd5b..4298e5712bfc6 100644 --- a/server/src/main/java/org/elasticsearch/index/seqno/ReplicationTracker.java +++ b/server/src/main/java/org/elasticsearch/index/seqno/ReplicationTracker.java @@ -35,7 +35,6 @@ import org.elasticsearch.index.shard.ShardId; import java.io.IOException; -import java.util.ArrayList; import java.util.Collection; import java.util.Collections; import java.util.HashMap; @@ -137,6 +136,12 @@ public class ReplicationTracker extends AbstractIndexShardComponent implements L */ private final LongConsumer onGlobalCheckpointUpdated; + /** + * A supplier of the current time. This supplier is used to add a timestamp to retention leases, and to determine retention lease + * expiration. + */ + private final LongSupplier currentTimeMillisSupplier; + /** * This set contains allocation IDs for which there is a thread actively waiting for the local checkpoint to advance to at least the * current global checkpoint. @@ -151,12 +156,21 @@ public class ReplicationTracker extends AbstractIndexShardComponent implements L private final Map retentionLeases = new HashMap<>(); /** - * Get all retention leases tracker on this shard. An unmodifiable copy of the retention leases is returned. + * Get all non-expired retention leases tracker on this shard. An unmodifiable copy of the retention leases is returned. * * @return the retention leases */ public synchronized Collection getRetentionLeases() { - return Collections.unmodifiableCollection(new ArrayList<>(retentionLeases.values())); + final long currentTimeMillis = currentTimeMillisSupplier.getAsLong(); + final long retentionLeaseMillis = indexSettings.getRetentionLeaseMillis(); + final Collection nonExpiredRetentionLeases = retentionLeases + .values() + .stream() + .filter(retentionLease -> currentTimeMillis - retentionLease.timestamp() <= retentionLeaseMillis) + .collect(Collectors.toList()); + retentionLeases.clear(); + retentionLeases.putAll(nonExpiredRetentionLeases.stream().collect(Collectors.toMap(RetentionLease::id, lease -> lease))); + return Collections.unmodifiableCollection(nonExpiredRetentionLeases); } /** @@ -168,7 +182,7 @@ public synchronized Collection getRetentionLeases() { */ public synchronized void addOrUpdateRetentionLease(final String id, final long retainingSequenceNumber, final String source) { assert primaryMode; - retentionLeases.put(id, new RetentionLease(id, retainingSequenceNumber, source)); + retentionLeases.put(id, new RetentionLease(id, retainingSequenceNumber, currentTimeMillisSupplier.getAsLong(), source)); } public static class CheckpointState implements Writeable { @@ -425,7 +439,8 @@ public ReplicationTracker( final String allocationId, final IndexSettings indexSettings, final long globalCheckpoint, - final LongConsumer onGlobalCheckpointUpdated) { + final LongConsumer onGlobalCheckpointUpdated, + final LongSupplier currentTimeMillisSupplier) { super(shardId, indexSettings); assert globalCheckpoint >= SequenceNumbers.UNASSIGNED_SEQ_NO : "illegal initial global checkpoint: " + globalCheckpoint; this.shardAllocationId = allocationId; @@ -435,6 +450,7 @@ public ReplicationTracker( this.checkpoints = new HashMap<>(1 + indexSettings.getNumberOfReplicas()); checkpoints.put(allocationId, new CheckpointState(SequenceNumbers.UNASSIGNED_SEQ_NO, globalCheckpoint, false, false)); this.onGlobalCheckpointUpdated = Objects.requireNonNull(onGlobalCheckpointUpdated); + this.currentTimeMillisSupplier = Objects.requireNonNull(currentTimeMillisSupplier); this.pendingInSync = new HashSet<>(); this.routingTable = null; this.replicationGroup = null; diff --git a/server/src/main/java/org/elasticsearch/index/seqno/RetentionLease.java b/server/src/main/java/org/elasticsearch/index/seqno/RetentionLease.java index 68e73219dc8a5..2881aac73c2c3 100644 --- a/server/src/main/java/org/elasticsearch/index/seqno/RetentionLease.java +++ b/server/src/main/java/org/elasticsearch/index/seqno/RetentionLease.java @@ -23,9 +23,9 @@ * A "shard history retention lease" (or "retention lease" for short) is conceptually a marker containing a retaining sequence number such * that all operations with sequence number at least that retaining sequence number will be retained during merge operations (which could * otherwise merge away operations that have been soft deleted). Each retention lease contains a unique identifier, the retaining sequence - * number, and the source of the retention lease (e.g., "ccr"). + * number, the timestamp of when the lease was created or renewed, and the source of the retention lease (e.g., "ccr"). */ -public class RetentionLease { +public final class RetentionLease { private final String id; @@ -50,6 +50,17 @@ public long retainingSequenceNumber() { return retainingSequenceNumber; } + private final long timestamp; + + /** + * The timestamp of when this retention lease was created or renewed. + * + * @return the timestamp used as a basis for determining lease expiration + */ + public long timestamp() { + return timestamp; + } + private final String source; /** @@ -66,19 +77,22 @@ public String source() { * * @param id the identifier of the retention lease * @param retainingSequenceNumber the retaining sequence number + * @param timestamp the timestamp of when the retention lease was created or renewed * @param source the source of the retention lease */ - public RetentionLease(final String id, final long retainingSequenceNumber, final String source) { + public RetentionLease(final String id, final long retainingSequenceNumber, final long timestamp, final String source) { this.id = id; this.retainingSequenceNumber = retainingSequenceNumber; + this.timestamp = timestamp; this.source = source; } @Override public String toString() { - return "ShardHistoryRetentionLease{" + + return "RetentionLease{" + "id='" + id + '\'' + ", retainingSequenceNumber=" + retainingSequenceNumber + + ", timestamp=" + timestamp + ", source='" + source + '\'' + '}'; } diff --git a/server/src/main/java/org/elasticsearch/index/shard/IndexShard.java b/server/src/main/java/org/elasticsearch/index/shard/IndexShard.java index ae4c3e0c0bcb8..66b8e607b5c76 100644 --- a/server/src/main/java/org/elasticsearch/index/shard/IndexShard.java +++ b/server/src/main/java/org/elasticsearch/index/shard/IndexShard.java @@ -305,7 +305,13 @@ public IndexShard( this.globalCheckpointListeners = new GlobalCheckpointListeners(shardId, threadPool.executor(ThreadPool.Names.LISTENER), threadPool.scheduler(), logger); this.replicationTracker = - new ReplicationTracker(shardId, aId, indexSettings, UNASSIGNED_SEQ_NO, globalCheckpointListeners::globalCheckpointUpdated); + new ReplicationTracker( + shardId, + aId, + indexSettings, + UNASSIGNED_SEQ_NO, + globalCheckpointListeners::globalCheckpointUpdated, + threadPool::absoluteTimeInMillis); // the query cache is a node-level thing, however we want the most popular filters // to be computed on a per-shard basis diff --git a/server/src/test/java/org/elasticsearch/index/engine/SoftDeletesPolicyTests.java b/server/src/test/java/org/elasticsearch/index/engine/SoftDeletesPolicyTests.java index 17ad6750cf50e..310e83e9d2cef 100644 --- a/server/src/test/java/org/elasticsearch/index/engine/SoftDeletesPolicyTests.java +++ b/server/src/test/java/org/elasticsearch/index/engine/SoftDeletesPolicyTests.java @@ -55,7 +55,7 @@ public void testSoftDeletesRetentionLock() { () -> { final Set leases = new HashSet<>(retainingSequenceNumbers.length); for (int i = 0; i < retainingSequenceNumbers.length; i++) { - leases.add(new RetentionLease(Integer.toString(i), retainingSequenceNumbers[i].get(), "test")); + leases.add(new RetentionLease(Integer.toString(i), retainingSequenceNumbers[i].get(), 0L, "test")); } return leases; }; diff --git a/server/src/test/java/org/elasticsearch/index/seqno/ReplicationTrackerRetentionLeaseTests.java b/server/src/test/java/org/elasticsearch/index/seqno/ReplicationTrackerRetentionLeaseTests.java index 83dbb3194ae12..2854cc87d8695 100644 --- a/server/src/test/java/org/elasticsearch/index/seqno/ReplicationTrackerRetentionLeaseTests.java +++ b/server/src/test/java/org/elasticsearch/index/seqno/ReplicationTrackerRetentionLeaseTests.java @@ -21,6 +21,8 @@ import org.elasticsearch.cluster.routing.AllocationId; import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.common.unit.TimeValue; +import org.elasticsearch.index.IndexSettings; import org.elasticsearch.index.shard.ShardId; import org.elasticsearch.test.IndexSettingsModule; @@ -28,11 +30,14 @@ import java.util.Collections; import java.util.HashMap; import java.util.Map; +import java.util.concurrent.atomic.AtomicLong; +import java.util.function.LongSupplier; import static org.elasticsearch.index.seqno.SequenceNumbers.UNASSIGNED_SEQ_NO; import static org.hamcrest.Matchers.equalTo; import static org.hamcrest.Matchers.hasItem; import static org.hamcrest.Matchers.hasSize; +import static org.hamcrest.Matchers.lessThanOrEqualTo; public class ReplicationTrackerRetentionLeaseTests extends ReplicationTrackerTestCase { @@ -43,7 +48,8 @@ public void testAddOrUpdateRetentionLease() { id.getId(), IndexSettingsModule.newIndexSettings("test", Settings.EMPTY), UNASSIGNED_SEQ_NO, - value -> {}); + value -> {}, + () -> 0L); replicationTracker.updateFromMaster( randomNonNegativeLong(), Collections.singleton(id.getId()), @@ -55,19 +61,73 @@ public void testAddOrUpdateRetentionLease() { for (int i = 0; i < length; i++) { minimumRetainingSequenceNumbers[i] = randomLongBetween(SequenceNumbers.NO_OPS_PERFORMED, Long.MAX_VALUE); replicationTracker.addOrUpdateRetentionLease(Integer.toString(i), minimumRetainingSequenceNumbers[i], "test-" + i); - assertRetentionLeases(replicationTracker, i + 1, minimumRetainingSequenceNumbers); + assertRetentionLeases(replicationTracker, i + 1, minimumRetainingSequenceNumbers, () -> 0L); } for (int i = 0; i < length; i++) { minimumRetainingSequenceNumbers[i] = randomLongBetween(minimumRetainingSequenceNumbers[i], Long.MAX_VALUE); replicationTracker.addOrUpdateRetentionLease(Integer.toString(i), minimumRetainingSequenceNumbers[i], "test-" + i); - assertRetentionLeases(replicationTracker, length, minimumRetainingSequenceNumbers); + assertRetentionLeases(replicationTracker, length, minimumRetainingSequenceNumbers, () -> 0L); } } + public void testExpiration() { + final AllocationId id = AllocationId.newInitializing(); + final AtomicLong currentTimeMillis = new AtomicLong(randomLongBetween(0, 1024)); + final long retentionLeaseMillis = randomLongBetween(1, TimeValue.timeValueHours(12).millis()); + final Settings settings = Settings + .builder() + .put(IndexSettings.INDEX_SOFT_DELETES_RETENTION_LEASE_SETTING.getKey(), TimeValue.timeValueMillis(retentionLeaseMillis)) + .build(); + final ReplicationTracker replicationTracker = new ReplicationTracker( + new ShardId("test", "_na", 0), + id.getId(), + IndexSettingsModule.newIndexSettings("test", settings), + UNASSIGNED_SEQ_NO, + value -> {}, + currentTimeMillis::get); + replicationTracker.updateFromMaster( + randomNonNegativeLong(), + Collections.singleton(id.getId()), + routingTable(Collections.emptySet(), id), + Collections.emptySet()); + replicationTracker.activatePrimaryMode(SequenceNumbers.NO_OPS_PERFORMED); + final long[] retainingSequenceNumbers = new long[1]; + retainingSequenceNumbers[0] = randomLongBetween(SequenceNumbers.NO_OPS_PERFORMED, Long.MAX_VALUE); + replicationTracker.addOrUpdateRetentionLease("0", retainingSequenceNumbers[0], "test-0"); + + { + final Collection retentionLeases = replicationTracker.getRetentionLeases(); + assertThat(retentionLeases, hasSize(1)); + final RetentionLease retentionLease = retentionLeases.iterator().next(); + assertThat(retentionLease.timestamp(), equalTo(currentTimeMillis.get())); + assertRetentionLeases(replicationTracker, 1, retainingSequenceNumbers, currentTimeMillis::get); + } + + // renew the lease + currentTimeMillis.set(currentTimeMillis.get() + randomLongBetween(0, 1024)); + retainingSequenceNumbers[0] = randomLongBetween(retainingSequenceNumbers[0], Long.MAX_VALUE); + replicationTracker.addOrUpdateRetentionLease("0", retainingSequenceNumbers[0], "test-0"); + + { + final Collection retentionLeases = replicationTracker.getRetentionLeases(); + assertThat(retentionLeases, hasSize(1)); + final RetentionLease retentionLease = retentionLeases.iterator().next(); + assertThat(retentionLease.timestamp(), equalTo(currentTimeMillis.get())); + assertRetentionLeases(replicationTracker, 1, retainingSequenceNumbers, currentTimeMillis::get); + } + + // now force the lease to expire + currentTimeMillis.set(currentTimeMillis.get() + randomLongBetween(retentionLeaseMillis, Long.MAX_VALUE - currentTimeMillis.get())); + assertRetentionLeases(replicationTracker, 0, retainingSequenceNumbers, currentTimeMillis::get); + } + private void assertRetentionLeases( - final ReplicationTracker replicationTracker, final int size, final long[] minimumRetainingSequenceNumbers) { + final ReplicationTracker replicationTracker, + final int size, + final long[] minimumRetainingSequenceNumbers, + final LongSupplier currentTimeMillisSupplier) { final Collection retentionLeases = replicationTracker.getRetentionLeases(); final Map idToRetentionLease = new HashMap<>(); for (final RetentionLease retentionLease : retentionLeases) { @@ -79,6 +139,9 @@ private void assertRetentionLeases( assertThat(idToRetentionLease.keySet(), hasItem(Integer.toString(i))); final RetentionLease retentionLease = idToRetentionLease.get(Integer.toString(i)); assertThat(retentionLease.retainingSequenceNumber(), equalTo(minimumRetainingSequenceNumbers[i])); + assertThat( + currentTimeMillisSupplier.getAsLong() - retentionLease.timestamp(), + lessThanOrEqualTo(replicationTracker.indexSettings().getRetentionLeaseMillis())); assertThat(retentionLease.source(), equalTo("test-" + i)); } } diff --git a/server/src/test/java/org/elasticsearch/index/seqno/ReplicationTrackerTestCase.java b/server/src/test/java/org/elasticsearch/index/seqno/ReplicationTrackerTestCase.java index 62e43af7d0e01..9b1f951a030fe 100644 --- a/server/src/test/java/org/elasticsearch/index/seqno/ReplicationTrackerTestCase.java +++ b/server/src/test/java/org/elasticsearch/index/seqno/ReplicationTrackerTestCase.java @@ -31,18 +31,23 @@ import java.util.Set; import java.util.function.LongConsumer; +import java.util.function.LongSupplier; import static org.elasticsearch.index.seqno.SequenceNumbers.UNASSIGNED_SEQ_NO; public abstract class ReplicationTrackerTestCase extends ESTestCase { - ReplicationTracker newTracker(final AllocationId allocationId, final LongConsumer updatedGlobalCheckpoint) { + ReplicationTracker newTracker( + final AllocationId allocationId, + final LongConsumer updatedGlobalCheckpoint, + final LongSupplier currentTimeMillisSupplier) { return new ReplicationTracker( new ShardId("test", "_na_", 0), allocationId.getId(), IndexSettingsModule.newIndexSettings("test", Settings.EMPTY), UNASSIGNED_SEQ_NO, - updatedGlobalCheckpoint); + updatedGlobalCheckpoint, + currentTimeMillisSupplier); } static IndexShardRoutingTable routingTable(final Set initializingIds, final AllocationId primaryId) { diff --git a/server/src/test/java/org/elasticsearch/index/seqno/ReplicationTrackerTests.java b/server/src/test/java/org/elasticsearch/index/seqno/ReplicationTrackerTests.java index e3fad3182a7a4..001e50af57c79 100644 --- a/server/src/test/java/org/elasticsearch/index/seqno/ReplicationTrackerTests.java +++ b/server/src/test/java/org/elasticsearch/index/seqno/ReplicationTrackerTests.java @@ -406,7 +406,7 @@ public void testWaitForAllocationIdToBeInSync() throws Exception { private AtomicLong updatedGlobalCheckpoint = new AtomicLong(UNASSIGNED_SEQ_NO); private ReplicationTracker newTracker(final AllocationId allocationId) { - return newTracker(allocationId, updatedGlobalCheckpoint::set); + return newTracker(allocationId, updatedGlobalCheckpoint::set, () -> 0L); } public void testWaitForAllocationIdToBeInSyncCanBeInterrupted() throws BrokenBarrierException, InterruptedException { @@ -683,10 +683,10 @@ public void testPrimaryContextHandoff() throws IOException { final AllocationId primaryAllocationId = clusterState.routingTable.primaryShard().allocationId(); final LongConsumer onUpdate = updatedGlobalCheckpoint -> {}; final long globalCheckpoint = UNASSIGNED_SEQ_NO; - ReplicationTracker oldPrimary = - new ReplicationTracker(shardId, primaryAllocationId.getId(), indexSettings, globalCheckpoint, onUpdate); - ReplicationTracker newPrimary = - new ReplicationTracker(shardId, primaryAllocationId.getRelocationId(), indexSettings, globalCheckpoint, onUpdate); + ReplicationTracker oldPrimary = new ReplicationTracker( + shardId, primaryAllocationId.getId(), indexSettings, globalCheckpoint, onUpdate, () -> 0L); + ReplicationTracker newPrimary = new ReplicationTracker( + shardId, primaryAllocationId.getRelocationId(), indexSettings, globalCheckpoint, onUpdate, () -> 0L); Set allocationIds = new HashSet<>(Arrays.asList(oldPrimary.shardAllocationId, newPrimary.shardAllocationId)); diff --git a/server/src/test/java/org/elasticsearch/index/shard/IndexShardRetentionLeaseTests.java b/server/src/test/java/org/elasticsearch/index/shard/IndexShardRetentionLeaseTests.java index 1214ac0ec9fb8..bd2a33617eecf 100644 --- a/server/src/test/java/org/elasticsearch/index/shard/IndexShardRetentionLeaseTests.java +++ b/server/src/test/java/org/elasticsearch/index/shard/IndexShardRetentionLeaseTests.java @@ -19,20 +19,50 @@ package org.elasticsearch.index.shard; +import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.common.unit.TimeValue; +import org.elasticsearch.index.IndexSettings; +import org.elasticsearch.index.engine.InternalEngineFactory; import org.elasticsearch.index.seqno.RetentionLease; import org.elasticsearch.index.seqno.SequenceNumbers; +import org.elasticsearch.threadpool.ThreadPool; import java.io.IOException; import java.util.Collection; import java.util.HashMap; import java.util.Map; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.atomic.AtomicLong; +import java.util.function.LongSupplier; import static org.hamcrest.Matchers.equalTo; import static org.hamcrest.Matchers.hasItem; import static org.hamcrest.Matchers.hasSize; +import static org.hamcrest.Matchers.lessThanOrEqualTo; +import static org.mockito.Matchers.anyString; +import static org.mockito.Mockito.doAnswer; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; public class IndexShardRetentionLeaseTests extends IndexShardTestCase { + private final AtomicLong currentTimeMillis = new AtomicLong(); + + @Override + protected ThreadPool setUpThreadPool() { + final ThreadPool threadPool = mock(ThreadPool.class); + doAnswer(invocationOnMock -> currentTimeMillis.get()).when(threadPool).absoluteTimeInMillis(); + when(threadPool.executor(anyString())).thenReturn(mock(ExecutorService.class)); + when(threadPool.scheduler()).thenReturn(mock(ScheduledExecutorService.class)); + return threadPool; + } + + @Override + protected void tearDownThreadPool() { + + } + public void testAddOrUpdateRetentionLease() throws IOException { final IndexShard indexShard = newStartedShard(true); try { @@ -41,22 +71,67 @@ public void testAddOrUpdateRetentionLease() throws IOException { for (int i = 0; i < length; i++) { minimumRetainingSequenceNumbers[i] = randomLongBetween(SequenceNumbers.NO_OPS_PERFORMED, Long.MAX_VALUE); indexShard.addOrUpdateRetentionLease(Integer.toString(i), minimumRetainingSequenceNumbers[i], "test-" + i); - assertRetentionLeases(indexShard, i + 1, minimumRetainingSequenceNumbers); + assertRetentionLeases(indexShard, i + 1, minimumRetainingSequenceNumbers, () -> 0L); } for (int i = 0; i < length; i++) { minimumRetainingSequenceNumbers[i] = randomLongBetween(minimumRetainingSequenceNumbers[i], Long.MAX_VALUE); indexShard.addOrUpdateRetentionLease(Integer.toString(i), minimumRetainingSequenceNumbers[i], "test-" + i); - assertRetentionLeases(indexShard, length, minimumRetainingSequenceNumbers); + assertRetentionLeases(indexShard, length, minimumRetainingSequenceNumbers, () -> 0L); } } finally { closeShards(indexShard); } + } + + public void testExpiration() throws IOException { + final long retentionLeaseMillis = randomLongBetween(1, TimeValue.timeValueHours(12).millis()); + final Settings settings = Settings + .builder() + .put(IndexSettings.INDEX_SOFT_DELETES_RETENTION_LEASE_SETTING.getKey(), TimeValue.timeValueMillis(retentionLeaseMillis)) + .build(); + // current time is mocked through the thread pool + final IndexShard indexShard = newStartedShard(true, settings, new InternalEngineFactory()); + try { + final long[] retainingSequenceNumbers = new long[1]; + retainingSequenceNumbers[0] = randomLongBetween(SequenceNumbers.NO_OPS_PERFORMED, Long.MAX_VALUE); + indexShard.addOrUpdateRetentionLease("0", retainingSequenceNumbers[0], "test-0"); + { + final Collection retentionLeases = indexShard.getEngine().config().retentionLeasesSupplier().get(); + assertThat(retentionLeases, hasSize(1)); + final RetentionLease retentionLease = retentionLeases.iterator().next(); + assertThat(retentionLease.timestamp(), equalTo(currentTimeMillis.get())); + assertRetentionLeases(indexShard, 1, retainingSequenceNumbers, currentTimeMillis::get); + } + + // renew the lease + currentTimeMillis.set(currentTimeMillis.get() + randomLongBetween(0, 1024)); + retainingSequenceNumbers[0] = randomLongBetween(retainingSequenceNumbers[0], Long.MAX_VALUE); + indexShard.addOrUpdateRetentionLease("0", retainingSequenceNumbers[0], "test-0"); + + { + final Collection retentionLeases = indexShard.getEngine().config().retentionLeasesSupplier().get(); + assertThat(retentionLeases, hasSize(1)); + final RetentionLease retentionLease = retentionLeases.iterator().next(); + assertThat(retentionLease.timestamp(), equalTo(currentTimeMillis.get())); + assertRetentionLeases(indexShard, 1, retainingSequenceNumbers, currentTimeMillis::get); + } + + // now force the lease to expire + currentTimeMillis.set( + currentTimeMillis.get() + randomLongBetween(retentionLeaseMillis, Long.MAX_VALUE - currentTimeMillis.get())); + assertRetentionLeases(indexShard, 0, retainingSequenceNumbers, currentTimeMillis::get); + } finally { + closeShards(indexShard); + } } private void assertRetentionLeases( - final IndexShard indexShard, final int size, final long[] minimumRetainingSequenceNumbers) { + final IndexShard indexShard, + final int size, + final long[] minimumRetainingSequenceNumbers, + final LongSupplier currentTimeMillisSupplier) { final Collection retentionLeases = indexShard.getEngine().config().retentionLeasesSupplier().get(); final Map idToRetentionLease = new HashMap<>(); for (final RetentionLease retentionLease : retentionLeases) { @@ -68,6 +143,9 @@ private void assertRetentionLeases( assertThat(idToRetentionLease.keySet(), hasItem(Integer.toString(i))); final RetentionLease retentionLease = idToRetentionLease.get(Integer.toString(i)); assertThat(retentionLease.retainingSequenceNumber(), equalTo(minimumRetainingSequenceNumbers[i])); + assertThat( + currentTimeMillisSupplier.getAsLong() - retentionLease.timestamp(), + lessThanOrEqualTo(indexShard.indexSettings().getRetentionLeaseMillis())); assertThat(retentionLease.source(), equalTo("test-" + i)); } } diff --git a/test/framework/src/main/java/org/elasticsearch/index/engine/EngineTestCase.java b/test/framework/src/main/java/org/elasticsearch/index/engine/EngineTestCase.java index 3786cc9591d3b..043cb85a091ca 100644 --- a/test/framework/src/main/java/org/elasticsearch/index/engine/EngineTestCase.java +++ b/test/framework/src/main/java/org/elasticsearch/index/engine/EngineTestCase.java @@ -602,8 +602,8 @@ public void onFailedEngine(String reason, @Nullable Exception e) { final LongSupplier globalCheckpointSupplier; final Supplier> retentionLeasesSupplier; if (maybeGlobalCheckpointSupplier == null) { - final ReplicationTracker replicationTracker = - new ReplicationTracker(shardId, allocationId.getId(), indexSettings, SequenceNumbers.NO_OPS_PERFORMED, update -> {}); + final ReplicationTracker replicationTracker = new ReplicationTracker( + shardId, allocationId.getId(), indexSettings, SequenceNumbers.NO_OPS_PERFORMED, update -> {}, () -> 0L); globalCheckpointSupplier = replicationTracker; retentionLeasesSupplier = replicationTracker::getRetentionLeases; } else { diff --git a/test/framework/src/main/java/org/elasticsearch/index/shard/IndexShardTestCase.java b/test/framework/src/main/java/org/elasticsearch/index/shard/IndexShardTestCase.java index 84e88b6accc5b..2e13cd6e66543 100644 --- a/test/framework/src/main/java/org/elasticsearch/index/shard/IndexShardTestCase.java +++ b/test/framework/src/main/java/org/elasticsearch/index/shard/IndexShardTestCase.java @@ -133,20 +133,28 @@ public void onRecoveryFailure(RecoveryState state, RecoveryFailedException e, bo @Override public void setUp() throws Exception { super.setUp(); - threadPool = new TestThreadPool(getClass().getName(), threadPoolSettings()); + threadPool = setUpThreadPool(); primaryTerm = randomIntBetween(1, 100); // use random but fixed term for creating shards failOnShardFailures(); } + protected ThreadPool setUpThreadPool() { + return new TestThreadPool(getClass().getName(), threadPoolSettings()); + } + @Override public void tearDown() throws Exception { try { - ThreadPool.terminate(threadPool, 30, TimeUnit.SECONDS); + tearDownThreadPool(); } finally { super.tearDown(); } } + protected void tearDownThreadPool() { + ThreadPool.terminate(threadPool, 30, TimeUnit.SECONDS); + } + /** * by default, tests will fail if any shard created by this class fails. Tests that cause failures by design * can call this method to ignore those failures diff --git a/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/TransportResumeFollowAction.java b/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/TransportResumeFollowAction.java index 4fc3efedd82c6..0a1a22215a04b 100644 --- a/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/TransportResumeFollowAction.java +++ b/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/TransportResumeFollowAction.java @@ -385,6 +385,7 @@ static String[] extractLeaderShardHistoryUUIDs(Map ccrIndexMetaD nonReplicatedSettings.add(IndexSettings.ALLOW_UNMAPPED); nonReplicatedSettings.add(IndexSettings.INDEX_SEARCH_IDLE_AFTER); nonReplicatedSettings.add(IndexSettings.INDEX_SOFT_DELETES_RETENTION_OPERATIONS_SETTING); + nonReplicatedSettings.add(IndexSettings.INDEX_SOFT_DELETES_RETENTION_LEASE_SETTING); nonReplicatedSettings.add(IndexSettings.MAX_SCRIPT_FIELDS_SETTING); nonReplicatedSettings.add(IndexSettings.MAX_REGEX_LENGTH_SETTING); nonReplicatedSettings.add(IndexSettings.MAX_TERMS_COUNT_SETTING); From b0665963e8b4f6ab1264ffc1168d0a20adb81ea2 Mon Sep 17 00:00:00 2001 From: Tim Brooks Date: Mon, 7 Jan 2019 23:35:22 -0700 Subject: [PATCH 015/186] Make remote ping and compress settings dynamic (#37200) Traditionally remote clusters can be configured dynamically. However, the compress and ping settings are not currently set to be configured dynamically. This commit changes that. --- .../org/elasticsearch/transport/RemoteClusterService.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/transport/RemoteClusterService.java b/server/src/main/java/org/elasticsearch/transport/RemoteClusterService.java index cb802f13fdb50..ff9dea8fe45a7 100644 --- a/server/src/main/java/org/elasticsearch/transport/RemoteClusterService.java +++ b/server/src/main/java/org/elasticsearch/transport/RemoteClusterService.java @@ -174,13 +174,13 @@ public String getKey(final String key) { public static final Setting.AffixSetting REMOTE_CLUSTER_PING_SCHEDULE = Setting.affixKeySetting( "cluster.remote.", "transport.ping_schedule", - key -> timeSetting(key, TransportSettings.PING_SCHEDULE, Setting.Property.NodeScope), + key -> timeSetting(key, TransportSettings.PING_SCHEDULE, Setting.Property.Dynamic, Setting.Property.NodeScope), REMOTE_CLUSTERS_SEEDS); public static final Setting.AffixSetting REMOTE_CLUSTER_COMPRESS = Setting.affixKeySetting( "cluster.remote.", "transport.compress", - key -> boolSetting(key, TransportSettings.TRANSPORT_COMPRESS, Setting.Property.NodeScope), + key -> boolSetting(key, TransportSettings.TRANSPORT_COMPRESS, Setting.Property.Dynamic, Setting.Property.NodeScope), REMOTE_CLUSTERS_SEEDS); private static final Predicate DEFAULT_NODE_PREDICATE = (node) -> Version.CURRENT.isCompatible(node.getVersion()) From 3fad9d25f622be5857854c2cb677426393172871 Mon Sep 17 00:00:00 2001 From: Andrei Stefan Date: Tue, 8 Jan 2019 08:47:35 +0200 Subject: [PATCH 016/186] SQL: fix COUNT DISTINCT filtering (#37176) * Use `_count` aggregation value only for not-DISTINCT COUNT function calls * COUNT DISTINCT will use the _exact_ version of a field (the `keyword` sub-field for example), if there is one --- .../sql/qa/src/main/resources/agg.sql-spec | 14 ++++++++ .../expression/function/aggregate/Count.java | 5 ++- .../xpack/sql/planner/QueryTranslator.java | 8 ++++- .../sql/planner/QueryTranslatorTests.java | 35 ++++++++++++++++++- 4 files changed, 59 insertions(+), 3 deletions(-) diff --git a/x-pack/plugin/sql/qa/src/main/resources/agg.sql-spec b/x-pack/plugin/sql/qa/src/main/resources/agg.sql-spec index 149e23f771349..39775fc13aed3 100644 --- a/x-pack/plugin/sql/qa/src/main/resources/agg.sql-spec +++ b/x-pack/plugin/sql/qa/src/main/resources/agg.sql-spec @@ -110,6 +110,8 @@ aggCountWithAlias SELECT gender g, COUNT(*) c FROM "test_emp" GROUP BY g ORDER BY gender; countDistinct SELECT COUNT(DISTINCT hire_date) AS count FROM test_emp; +countDistinctAndCountSimpleWithAlias +SELECT COUNT(*) cnt, COUNT(DISTINCT first_name) as names, gender FROM test_emp GROUP BY gender ORDER BY gender; aggCountAliasAndWhereClauseMultiGroupBy SELECT gender g, languages l, COUNT(*) c FROM "test_emp" WHERE emp_no < 10020 GROUP BY gender, languages ORDER BY gender, languages; @@ -121,6 +123,8 @@ aggCountWithAliasMultiGroupBy SELECT gender g, languages l, COUNT(*) c FROM "test_emp" GROUP BY g, l ORDER BY gender, languages; aggCountWithAliasMultiGroupByDifferentOrder SELECT gender g, languages l, COUNT(*) c FROM "test_emp" GROUP BY g, l ORDER BY languages ASC, gender DESC; +aggCountDistinctWithAliasAndGroupBy +SELECT COUNT(*) cnt, COUNT(DISTINCT first_name) as names, gender FROM test_emp GROUP BY gender ORDER BY gender; @@ -161,12 +165,20 @@ aggCountStarAndHavingBetween SELECT gender g, COUNT(*) c FROM "test_emp" GROUP BY g HAVING c BETWEEN 10 AND 70 ORDER BY gender ASC; aggCountStarAndHavingBetweenWithLimit SELECT gender g, COUNT(*) c FROM "test_emp" GROUP BY g HAVING c BETWEEN 10 AND 70 ORDER BY gender LIMIT 1; +aggCountDistinctAndHavingBetweenWithLimit +SELECT gender g, COUNT(DISTINCT first_name) c FROM "test_emp" GROUP BY g HAVING c BETWEEN 40 AND 50 ORDER BY gender LIMIT 1; aggCountOnColumnAndHavingOnAliasAndFunction SELECT gender g, COUNT(gender) c FROM "test_emp" GROUP BY g HAVING c > 10 AND COUNT(gender) < 70 ORDER BY gender; aggCountOnColumnAndHavingOnAliasAndFunctionWildcard -> COUNT(*/1) vs COUNT(gender) SELECT gender g, COUNT(gender) c FROM "test_emp" GROUP BY g HAVING c > 10 AND COUNT(*) < 70 ORDER BY gender; aggCountOnColumnAndHavingOnAliasAndFunctionConstant SELECT gender g, COUNT(gender) c FROM "test_emp" GROUP BY g HAVING c > 10 AND COUNT(1) < 70 ORDER BY gender; +aggDistinctCountWithAliasAndHaving +SELECT COUNT(*) c, COUNT(DISTINCT first_name) AS names, gender FROM test_emp GROUP BY gender HAVING names > 40 ORDER BY gender; +aggDistinctCountWithFunctionWildcardAndHaving +SELECT COUNT(*) c, COUNT(DISTINCT first_name) AS names, gender FROM test_emp GROUP BY gender HAVING names < 50 AND c < 50 ORDER BY gender; +aggDistinctCountWithFunctionWildcardAndFunctionConstantAndHaving +SELECT COUNT(*) c, COUNT(DISTINCT first_name) AS names, COUNT(123) AS c123, gender FROM test_emp GROUP BY gender HAVING names < 50 AND c < 50 AND c123 < 50 ORDER BY gender; aggCountAndHavingMultiGroupBy SELECT gender g, languages l, COUNT(*) c FROM "test_emp" GROUP BY g, l HAVING COUNT(*) > 10 ORDER BY gender, l; @@ -195,6 +207,8 @@ aggCountOnColumnAndHavingOnAliasAndFunctionWildcardMultiGroupBy -> COUNT(*/1) vs SELECT gender g, languages l, COUNT(gender) c FROM "test_emp" GROUP BY g, l HAVING c > 10 AND COUNT(*) < 70 ORDER BY gender, languages; aggCountOnColumnAndHavingOnAliasAndFunctionConstantMultiGroupBy SELECT gender g, languages l, COUNT(gender) c FROM "test_emp" GROUP BY g, l HAVING c > 10 AND COUNT(1) < 70 ORDER BY gender, languages; +aggCountOnDistinctColumnAndHavingOnAliasAndFunctionConstantMultiGroupBy +SELECT gender g, languages l, COUNT(DISTINCT last_name) c FROM "test_emp" GROUP BY g, l HAVING c > 5 AND COUNT(1) < 70 ORDER BY gender, languages; // MIN diff --git a/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/expression/function/aggregate/Count.java b/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/expression/function/aggregate/Count.java index 9d4e0ea0dae01..b4afa6762dea4 100644 --- a/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/expression/function/aggregate/Count.java +++ b/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/expression/function/aggregate/Count.java @@ -61,6 +61,9 @@ public String functionId() { @Override public AggregateFunctionAttribute toAttribute() { - return new AggregateFunctionAttribute(source(), name(), dataType(), id(), functionId(), "_count"); + if (!distinct()) { + return new AggregateFunctionAttribute(source(), name(), dataType(), id(), functionId(), "_count"); + } + return super.toAttribute(); } } diff --git a/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/planner/QueryTranslator.java b/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/planner/QueryTranslator.java index 130187a9e6db7..a34b3325ee763 100644 --- a/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/planner/QueryTranslator.java +++ b/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/planner/QueryTranslator.java @@ -429,7 +429,13 @@ static String dateFormat(Expression e) { static String field(AggregateFunction af) { Expression arg = af.field(); if (arg instanceof FieldAttribute) { - return ((FieldAttribute) arg).name(); + FieldAttribute field = (FieldAttribute) arg; + // COUNT(DISTINCT) uses cardinality aggregation which works on exact values (not changed by analyzers or normalizers) + if (af instanceof Count && ((Count) af).distinct()) { + // use the `keyword` version of the field, if there is one + return field.isInexact() ? field.exactAttribute().name() : field.name(); + } + return field.name(); } if (arg instanceof Literal) { return String.valueOf(((Literal) arg).value()); diff --git a/x-pack/plugin/sql/src/test/java/org/elasticsearch/xpack/sql/planner/QueryTranslatorTests.java b/x-pack/plugin/sql/src/test/java/org/elasticsearch/xpack/sql/planner/QueryTranslatorTests.java index 559d676f1b95f..861c201440285 100644 --- a/x-pack/plugin/sql/src/test/java/org/elasticsearch/xpack/sql/planner/QueryTranslatorTests.java +++ b/x-pack/plugin/sql/src/test/java/org/elasticsearch/xpack/sql/planner/QueryTranslatorTests.java @@ -5,6 +5,8 @@ */ package org.elasticsearch.xpack.sql.planner; +import org.elasticsearch.search.aggregations.AggregationBuilder; +import org.elasticsearch.search.aggregations.metrics.CardinalityAggregationBuilder; import org.elasticsearch.test.ESTestCase; import org.elasticsearch.xpack.sql.SqlIllegalArgumentException; import org.elasticsearch.xpack.sql.TestUtils; @@ -19,11 +21,14 @@ import org.elasticsearch.xpack.sql.expression.function.grouping.Histogram; import org.elasticsearch.xpack.sql.expression.function.scalar.math.MathProcessor.MathOperation; import org.elasticsearch.xpack.sql.expression.gen.script.ScriptTemplate; +import org.elasticsearch.xpack.sql.optimizer.Optimizer; import org.elasticsearch.xpack.sql.parser.SqlParser; import org.elasticsearch.xpack.sql.plan.logical.Aggregate; import org.elasticsearch.xpack.sql.plan.logical.Filter; import org.elasticsearch.xpack.sql.plan.logical.LogicalPlan; import org.elasticsearch.xpack.sql.plan.logical.Project; +import org.elasticsearch.xpack.sql.plan.physical.EsQueryExec; +import org.elasticsearch.xpack.sql.plan.physical.PhysicalPlan; import org.elasticsearch.xpack.sql.planner.QueryTranslator.QueryTranslation; import org.elasticsearch.xpack.sql.querydsl.agg.AggFilter; import org.elasticsearch.xpack.sql.querydsl.query.ExistsQuery; @@ -41,6 +46,7 @@ import org.junit.AfterClass; import org.junit.BeforeClass; +import java.util.Collection; import java.util.List; import java.util.Locale; import java.util.Map; @@ -55,6 +61,8 @@ public class QueryTranslatorTests extends ESTestCase { private static SqlParser parser; private static Analyzer analyzer; + private static Optimizer optimizer; + private static Planner planner; @BeforeClass public static void init() { @@ -64,6 +72,8 @@ public static void init() { EsIndex test = new EsIndex("test", mapping); IndexResolution getIndexResult = IndexResolution.valid(test); analyzer = new Analyzer(TestUtils.TEST_CFG, new FunctionRegistry(), getIndexResult, new Verifier(new Metrics())); + optimizer = new Optimizer(); + planner = new Planner(); } @AfterClass @@ -75,6 +85,10 @@ public static void destroy() { private LogicalPlan plan(String sql) { return analyzer.analyze(parser.createStatement(sql), true); } + + private PhysicalPlan optimizeAndPlan(String sql) { + return planner.plan(optimizer.optimize(plan(sql)), true); + } public void testTermEqualityAnalyzer() { LogicalPlan p = plan("SELECT some.string FROM test WHERE some.string = 'value'"); @@ -433,6 +447,7 @@ public void testTranslateNullIf_GroupBy_Painless() { scriptTemplate.toString()); assertEquals("[{v=int}, {v=10}]", scriptTemplate.params().toString()); } + public void testGroupByDateHistogram() { LogicalPlan p = plan("SELECT MAX(int) FROM test GROUP BY HISTOGRAM(int, 1000)"); assertTrue(p instanceof Aggregate); @@ -448,7 +463,6 @@ public void testGroupByDateHistogram() { assertEquals(DataType.INTEGER, field.dataType()); } - public void testGroupByHistogram() { LogicalPlan p = plan("SELECT MAX(int) FROM test GROUP BY HISTOGRAM(date, INTERVAL 2 YEARS)"); assertTrue(p instanceof Aggregate); @@ -463,4 +477,23 @@ public void testGroupByHistogram() { assertEquals(FieldAttribute.class, field.getClass()); assertEquals(DataType.DATE, field.dataType()); } + + public void testCountDistinctCardinalityFolder() { + PhysicalPlan p = optimizeAndPlan("SELECT COUNT(DISTINCT keyword) cnt FROM test GROUP BY bool HAVING cnt = 0"); + assertEquals(EsQueryExec.class, p.getClass()); + EsQueryExec ee = (EsQueryExec) p; + assertEquals(1, ee.output().size()); + assertThat(ee.output().get(0).toString(), startsWith("cnt{a->")); + + Collection subAggs = ee.queryContainer().aggs().asAggBuilder().getSubAggregations(); + assertEquals(1, subAggs.size()); + assertTrue(subAggs.toArray()[0] instanceof CardinalityAggregationBuilder); + + CardinalityAggregationBuilder cardinalityAgg = (CardinalityAggregationBuilder) subAggs.toArray()[0]; + assertEquals("keyword", cardinalityAgg.field()); + assertThat(ee.queryContainer().aggs().asAggBuilder().toString().replaceAll("\\s+", ""), + endsWith("{\"buckets_path\":{\"a0\":\"" + cardinalityAgg.getName() +"\"},\"script\":{" + + "\"source\":\"InternalSqlScriptUtils.nullSafeFilter(InternalSqlScriptUtils.eq(params.a0,params.v0))\"," + + "\"lang\":\"painless\",\"params\":{\"v0\":0}},\"gap_policy\":\"skip\"}}}}}")); + } } From 67f97e36861ce3e20c14735f26e2c6d09e18792c Mon Sep 17 00:00:00 2001 From: Luca Cavanna Date: Tue, 8 Jan 2019 09:53:20 +0100 Subject: [PATCH 017/186] Re-enable bwc tests Relates to #37187 --- build.gradle | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/build.gradle b/build.gradle index 3b5519628fd2e..439bd32727d18 100644 --- a/build.gradle +++ b/build.gradle @@ -159,8 +159,8 @@ task verifyVersions { * the enabled state of every bwc task. It should be set back to true * after the backport of the backcompat code is complete. */ -final boolean bwc_tests_enabled = false -final String bwc_tests_disabled_issue = "https://github.com/elastic/elasticsearch/pull/37142" /* place a PR link here when committing bwc changes */ +final boolean bwc_tests_enabled = true +final String bwc_tests_disabled_issue = "" /* place a PR link here when committing bwc changes */ if (bwc_tests_enabled == false) { if (bwc_tests_disabled_issue.isEmpty()) { throw new GradleException("bwc_tests_disabled_issue must be set when bwc_tests_enabled == false") From 1d371a427faaf5c6ed066d7592dc0fa912400669 Mon Sep 17 00:00:00 2001 From: Andrew Banchich Date: Tue, 8 Jan 2019 04:22:50 -0500 Subject: [PATCH 018/186] =?UTF-8?q?[Docs]=C2=A0Fix=20wrong=20math=20in=20o?= =?UTF-8?q?verview.asciidoc=20(#37209)?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- docs/reference/rollup/overview.asciidoc | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/reference/rollup/overview.asciidoc b/docs/reference/rollup/overview.asciidoc index b2570f647e72b..90c5e20a850c1 100644 --- a/docs/reference/rollup/overview.asciidoc +++ b/docs/reference/rollup/overview.asciidoc @@ -6,7 +6,7 @@ experimental[] Time-based data (documents that are predominantly identified by their timestamp) often have associated retention policies -to manage data growth. For example, your system may be generating 500,000 documents every second. That will generate +to manage data growth. For example, your system may be generating 500 documents every second. That will generate 43 million documents per day, and nearly 16 billion documents a year. While your analysts and data scientists may wish you stored that data indefinitely for analysis, time is never-ending and From c980cc12df569651ff2c83a6e61c4a4ae1216132 Mon Sep 17 00:00:00 2001 From: Martijn van Groningen Date: Tue, 8 Jan 2019 11:05:18 +0100 Subject: [PATCH 019/186] Added CCR rolling upgrade tests (#36648) Added CCR rolling upgrade tests. --- .../org/elasticsearch/upgrades/CCRIT.java | 280 ++++++++++++++++++ 1 file changed, 280 insertions(+) create mode 100644 x-pack/qa/rolling-upgrade/src/test/java/org/elasticsearch/upgrades/CCRIT.java diff --git a/x-pack/qa/rolling-upgrade/src/test/java/org/elasticsearch/upgrades/CCRIT.java b/x-pack/qa/rolling-upgrade/src/test/java/org/elasticsearch/upgrades/CCRIT.java new file mode 100644 index 0000000000000..45936c81a98dc --- /dev/null +++ b/x-pack/qa/rolling-upgrade/src/test/java/org/elasticsearch/upgrades/CCRIT.java @@ -0,0 +1,280 @@ +/* + * 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.upgrades; + +import org.apache.http.util.EntityUtils; +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; +import org.elasticsearch.Version; +import org.elasticsearch.client.Request; +import org.elasticsearch.client.Response; +import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.common.xcontent.ObjectPath; +import org.elasticsearch.common.xcontent.XContentHelper; +import org.elasticsearch.common.xcontent.json.JsonXContent; + +import java.io.IOException; +import java.util.Map; + +import static org.hamcrest.Matchers.equalTo; + +public class CCRIT extends AbstractUpgradeTestCase { + + private static final Logger LOGGER = LogManager.getLogger(CCRIT.class); + + private static final Version UPGRADE_FROM_VERSION = + Version.fromString(System.getProperty("tests.upgrade_from_version")); + + private static final boolean SECOND_ROUND = "false".equals(System.getProperty("tests.first_round")); + + @Override + protected boolean preserveClusterSettings() { + return true; + } + + public void testIndexFollowing() throws Exception { + assumeTrue("CCR became available in 6.5 and test relies on a fix that was shipped with 6.5.4", + UPGRADE_FROM_VERSION.onOrAfter(Version.V_6_5_4)); + setupRemoteCluster(); + + final String leaderIndex = "my-leader-index"; + final String followerIndex = "my-follower-index"; + + switch (CLUSTER_TYPE) { + case OLD: + Settings indexSettings = Settings.builder() + .put("index.soft_deletes.enabled", true) + .put("index.number_of_shards", 1) + .build(); + createIndex(leaderIndex, indexSettings); + followIndex(leaderIndex, followerIndex); + index(leaderIndex, "1"); + assertDocumentExists(leaderIndex, "1"); + assertBusy(() -> { + assertFollowerGlobalCheckpoint(followerIndex, 0); + assertDocumentExists(followerIndex, "1"); + }); + break; + case MIXED: + if (SECOND_ROUND == false) { + index(leaderIndex, "2"); + assertDocumentExists(leaderIndex, "1", "2"); + assertBusy(() -> { + assertFollowerGlobalCheckpoint(followerIndex, 1); + assertDocumentExists(followerIndex, "1", "2"); + }); + } else { + index(leaderIndex, "3"); + assertDocumentExists(leaderIndex, "1", "2", "3"); + assertBusy(() -> { + assertFollowerGlobalCheckpoint(followerIndex, 2); + assertDocumentExists(followerIndex, "1", "2", "3"); + }); + } + break; + case UPGRADED: + index(leaderIndex, "4"); + assertDocumentExists(leaderIndex, "1", "2", "3", "4"); + assertBusy(() -> { + assertFollowerGlobalCheckpoint(followerIndex, 3); + assertDocumentExists(followerIndex, "1", "2", "3", "4"); + }); + stopIndexFollowing(followerIndex); + break; + default: + throw new UnsupportedOperationException("Unknown cluster type [" + CLUSTER_TYPE + "]"); + } + } + + public void testAutoFollowing() throws Exception { + assumeTrue("CCR became available in 6.5 and test relies on a fix that was shipped with 6.5.4", + UPGRADE_FROM_VERSION.onOrAfter(Version.V_6_5_4)); + setupRemoteCluster(); + + final Settings indexSettings = Settings.builder() + .put("index.soft_deletes.enabled", true) + .put("index.number_of_shards", 1) + .build(); + + String leaderIndex1 = "logs-20200101"; + String leaderIndex2 = "logs-20200102"; + String leaderIndex3 = "logs-20200103"; + + switch (CLUSTER_TYPE) { + case OLD: + putAutoFollowPattern("test_pattern", "logs-*"); + createIndex(leaderIndex1, indexSettings); + index(leaderIndex1, "1"); + assertBusy(() -> { + String followerIndex = "copy-" + leaderIndex1; + assertThat(getNumberOfSuccessfulFollowedIndices(), equalTo(1)); + assertFollowerGlobalCheckpoint(followerIndex, 0); + assertDocumentExists(followerIndex, "1"); + }); + break; + case MIXED: + if (SECOND_ROUND == false) { + index(leaderIndex1, "2"); + assertBusy(() -> { + String followerIndex = "copy-" + leaderIndex1; + assertFollowerGlobalCheckpoint(followerIndex, 1); + assertDocumentExists(followerIndex, "2"); + }); + // Auto follow stats are kept in-memory on master elected node + // and if this node get updated then auto follow stats are reset + int previousNumberOfSuccessfulFollowedIndices = getNumberOfSuccessfulFollowedIndices(); + createIndex(leaderIndex2, indexSettings); + index(leaderIndex2, "1"); + assertBusy(() -> { + String followerIndex = "copy-" + leaderIndex2; + assertThat(getNumberOfSuccessfulFollowedIndices(), equalTo(previousNumberOfSuccessfulFollowedIndices + 1)); + assertFollowerGlobalCheckpoint(followerIndex, 0); + assertDocumentExists(followerIndex, "1"); + }); + } else { + index(leaderIndex1, "3"); + assertBusy(() -> { + String followerIndex = "copy-" + leaderIndex1; + assertFollowerGlobalCheckpoint(followerIndex, 2); + assertDocumentExists(followerIndex, "3"); + }); + index(leaderIndex2, "2"); + assertBusy(() -> { + String followerIndex = "copy-" + leaderIndex2; + assertFollowerGlobalCheckpoint(followerIndex, 1); + assertDocumentExists(followerIndex, "2"); + }); + + // Auto follow stats are kept in-memory on master elected node + // and if this node get updated then auto follow stats are reset + int previousNumberOfSuccessfulFollowedIndices = getNumberOfSuccessfulFollowedIndices(); + createIndex(leaderIndex3, indexSettings); + index(leaderIndex3, "1"); + assertBusy(() -> { + String followerIndex = "copy-" + leaderIndex3; + assertThat(getNumberOfSuccessfulFollowedIndices(), equalTo(previousNumberOfSuccessfulFollowedIndices + 1)); + assertFollowerGlobalCheckpoint(followerIndex, 0); + assertDocumentExists(followerIndex, "1"); + }); + } + break; + case UPGRADED: + index(leaderIndex1, "4"); + assertBusy(() -> { + String followerIndex = "copy-" + leaderIndex1; + assertFollowerGlobalCheckpoint(followerIndex, 3); + assertDocumentExists(followerIndex, "4"); + }); + index(leaderIndex2, "3"); + assertBusy(() -> { + String followerIndex = "copy-" + leaderIndex2; + assertFollowerGlobalCheckpoint(followerIndex, 2); + assertDocumentExists(followerIndex, "3"); + }); + index(leaderIndex3, "2"); + assertBusy(() -> { + String followerIndex = "copy-" + leaderIndex3; + assertFollowerGlobalCheckpoint(followerIndex, 1); + assertDocumentExists(followerIndex, "2"); + }); + + deleteAutoFollowPattern("test_pattern"); + + stopIndexFollowing("copy-" + leaderIndex1); + stopIndexFollowing("copy-" + leaderIndex2); + stopIndexFollowing("copy-" + leaderIndex3); + break; + default: + throw new UnsupportedOperationException("Unknown cluster type [" + CLUSTER_TYPE + "]"); + } + } + + private static void stopIndexFollowing(String followerIndex) throws IOException { + pauseFollow(followerIndex); + closeIndex(followerIndex); + unfollow(followerIndex); + } + + private static void followIndex(String leaderIndex, String followIndex) throws IOException { + final Request request = new Request("PUT", "/" + followIndex + "/_ccr/follow"); + request.setJsonEntity("{\"remote_cluster\": \"local\", \"leader_index\": \"" + leaderIndex + + "\", \"read_poll_timeout\": \"10ms\"}"); + assertOK(client().performRequest(request)); + } + + private static void pauseFollow(String followIndex) throws IOException { + assertOK(client().performRequest(new Request("POST", "/" + followIndex + "/_ccr/pause_follow"))); + } + + private static void unfollow(String followIndex) throws IOException { + assertOK(client().performRequest(new Request("POST", "/" + followIndex + "/_ccr/unfollow"))); + } + + private static void putAutoFollowPattern(String name, String pattern) throws IOException { + Request request = new Request("PUT", "/_ccr/auto_follow/" + name); + request.setJsonEntity("{\"leader_index_patterns\": [\"" + pattern + "\"], \"remote_cluster\": \"local\"," + + "\"follow_index_pattern\": \"copy-{{leader_index}}\", \"read_poll_timeout\": \"10ms\"}"); + assertOK(client().performRequest(request)); + } + + private static void deleteAutoFollowPattern(String patternName) throws IOException { + Request request = new Request("DELETE", "/_ccr/auto_follow/" + patternName); + assertOK(client().performRequest(request)); + } + + private static void index(String index, String id) throws IOException { + Request request = new Request("POST", "/" + index + "/_doc/" + id); + request.setJsonEntity("{}"); + assertOK(client().performRequest(request)); + } + + private static void assertDocumentExists(String index, String... ids) throws IOException { + for (String id : ids) { + Request request = new Request("HEAD", "/" + index + "/_doc/" + id); + Response response = client().performRequest(request); + assertThat(response.getStatusLine().getStatusCode(), equalTo(200)); + } + } + + private static void setupRemoteCluster() throws IOException { + Request request = new Request("GET", "/_nodes"); + Map nodesResponse = (Map) toMap(client().performRequest(request)).get("nodes"); + // Select node info of first node (we don't know the node id): + nodesResponse = (Map) nodesResponse.get(nodesResponse.keySet().iterator().next()); + String transportAddress = (String) nodesResponse.get("transport_address"); + + LOGGER.info("Configuring local remote cluster [{}]", transportAddress); + request = new Request("PUT", "/_cluster/settings"); + request.setJsonEntity("{\"persistent\": {\"cluster.remote.local.seeds\": \"" + transportAddress + "\"}}"); + assertThat(client().performRequest(request).getStatusLine().getStatusCode(), equalTo(200)); + } + + private int getNumberOfSuccessfulFollowedIndices() throws IOException { + Request statsRequest = new Request("GET", "/_ccr/stats"); + Map response = toMap(client().performRequest(statsRequest)); + Integer actualSuccessfulFollowedIndices = ObjectPath.eval("auto_follow_stats.number_of_successful_follow_indices", response); + if (actualSuccessfulFollowedIndices != null) { + return actualSuccessfulFollowedIndices; + } else { + return -1; + } + } + + private void assertFollowerGlobalCheckpoint(String followerIndex, int expectedFollowerCheckpoint) throws IOException { + Request statsRequest = new Request("GET", "/" + followerIndex + "/_stats"); + statsRequest.addParameter("level", "shards"); + Map response = toMap(client().performRequest(statsRequest)); + LOGGER.info("INDEX STATS={}", response); + assertThat(((Map) response.get("indices")).size(), equalTo(1)); + Integer actualFollowerCheckpoint = ObjectPath.eval("indices." + followerIndex + ".shards.0.0.seq_no.global_checkpoint", response); + assertThat(actualFollowerCheckpoint, equalTo(expectedFollowerCheckpoint)); + } + + private static Map toMap(Response response) throws IOException { + return XContentHelper.convertToMap(JsonXContent.jsonXContent, EntityUtils.toString(response.getEntity()), false); + } + +} From 0c68eead519cc09c8f9ef657f550379336d5b326 Mon Sep 17 00:00:00 2001 From: Alpar Torok Date: Tue, 8 Jan 2019 13:00:55 +0200 Subject: [PATCH 020/186] Mute failing test clusters test Tracking issue: #37218 --- .../elasticsearch/gradle/testclusters/TestClustersPluginIT.java | 2 ++ 1 file changed, 2 insertions(+) diff --git a/buildSrc/src/test/java/org/elasticsearch/gradle/testclusters/TestClustersPluginIT.java b/buildSrc/src/test/java/org/elasticsearch/gradle/testclusters/TestClustersPluginIT.java index ee366ac7b7c65..514f75eaa86e9 100644 --- a/buildSrc/src/test/java/org/elasticsearch/gradle/testclusters/TestClustersPluginIT.java +++ b/buildSrc/src/test/java/org/elasticsearch/gradle/testclusters/TestClustersPluginIT.java @@ -21,9 +21,11 @@ import org.elasticsearch.gradle.test.GradleIntegrationTestCase; import org.gradle.testkit.runner.BuildResult; import org.gradle.testkit.runner.GradleRunner; +import org.junit.Ignore; import java.util.Arrays; +@Ignore // https://github.com/elastic/elasticsearch/issues/37218 public class TestClustersPluginIT extends GradleIntegrationTestCase { public void testListClusters() { From 6344e9a3ce485d6deafb78082dc5f757788e83ec Mon Sep 17 00:00:00 2001 From: Alpar Torok Date: Tue, 8 Jan 2019 13:39:03 +0200 Subject: [PATCH 021/186] Testing conventions: add support for checking base classes (#36650) --- buildSrc/build.gradle | 12 ++ .../gradle/plugin/PluginBuildPlugin.groovy | 13 ++ .../gradle/precommit/PrecommitTasks.groovy | 12 +- .../precommit/TestingConventionRule.java | 99 ++++++++++ .../precommit/TestingConventionsTasks.java | 186 +++++++++++------- .../testfixtures/TestFixturesPlugin.java | 5 + .../resources/checkstyle_suppressions.xml | 5 +- .../precommit/TestingConventionsTasksIT.java | 108 ++++++++++ .../test/GradleIntegrationTestCase.java | 2 +- .../gradle/testkit/NamingConventionIT.java | 23 +++ .../gradle/testkit/NamingConventionTests.java | 23 +++ .../testKit/testingConventions/build.gradle | 86 ++++++++ .../empty_test_task/.gitignore | 0 ...ooksLikeATestWithoutNamingConvention1.java | 30 +++ ...ooksLikeATestWithoutNamingConvention2.java | 25 +++ ...ooksLikeATestWithoutNamingConvention3.java | 30 +++ .../testkit/LooksLikeTestsButAbstract.java | 30 +++ .../gradle/testkit/NamingConventionIT.java | 23 +++ .../gradle/testkit/NamingConventionTests.java | 23 +++ .../gradle/testkit/NastyInnerClasses.java | 64 ++++++ .../gradle/testkit/AbstractIT.java | 23 +++ .../gradle/testkit/Integration.java | 23 +++ .../gradle/testkit/NamingConventionIT.java | 23 +++ .../testkit/NamingConventionMissmatchIT.java | 23 +++ .../NamingConventionMissmatchTests.java | 23 +++ .../gradle/testkit/NamingConventionTests.java | 23 +++ .../elasticsearch/gradle/testkit/Unit.java | 23 +++ .../testingConventions/settings.gradle | 7 + .../gradle/testkit/NamingConventionIT.java | 23 +++ .../gradle/testkit/NamingConventionTests.java | 23 +++ .../gradle/testkit/Integration.java | 23 +++ .../gradle/testkit/NamingConventionIT.java | 23 +++ .../gradle/testkit/NamingConventionTests.java | 23 +++ .../elasticsearch/gradle/testkit/Unit.java | 23 +++ client/rest/build.gradle | 9 + .../RestClientDocumentation.java | 4 +- client/sniffer/build.gradle | 10 + .../documentation/SnifferDocumentation.java | 2 +- client/transport/build.gradle | 9 + distribution/tools/launchers/build.gradle | 9 + libs/secure-sm/build.gradle | 9 + plugins/examples/rest-handler/build.gradle | 6 + qa/smoke-test-client/build.gradle | 10 + .../build.gradle | 8 + qa/wildfly/build.gradle | 11 ++ server/build.gradle | 13 ++ x-pack/plugin/ccr/build.gradle | 6 + x-pack/plugin/sql/build.gradle | 8 + x-pack/plugin/sql/qa/security/build.gradle | 3 + .../xpack/sql/action/SqlActionIT.java | 1 + .../sql/action/SqlClearCursorActionIT.java | 2 + .../xpack/sql/action/SqlDisabledIT.java | 1 + .../xpack/sql/action/SqlLicenseIT.java | 2 + .../sql/action/SqlTranslateActionIT.java | 1 + x-pack/qa/full-cluster-restart/build.gradle | 4 + x-pack/qa/rolling-upgrade/build.gradle | 3 + x-pack/qa/security-migrate-tests/build.gradle | 9 + x-pack/qa/smoke-test-watcher/build.gradle | 4 + ...plateIT.java => WatcherTemplateTests.java} | 2 +- x-pack/qa/transport-client-tests/build.gradle | 10 + x-pack/transport-client/build.gradle | 9 + 61 files changed, 1192 insertions(+), 78 deletions(-) create mode 100644 buildSrc/src/main/java/org/elasticsearch/gradle/precommit/TestingConventionRule.java create mode 100644 buildSrc/src/test/java/org/elasticsearch/gradle/precommit/TestingConventionsTasksIT.java create mode 100644 buildSrc/src/testKit/testingConventions/all_classes_in_tasks/src/test/java/org/elasticsearch/gradle/testkit/NamingConventionIT.java create mode 100644 buildSrc/src/testKit/testingConventions/all_classes_in_tasks/src/test/java/org/elasticsearch/gradle/testkit/NamingConventionTests.java create mode 100644 buildSrc/src/testKit/testingConventions/build.gradle create mode 100644 buildSrc/src/testKit/testingConventions/empty_test_task/.gitignore create mode 100644 buildSrc/src/testKit/testingConventions/incorrect_naming_conventions/src/test/java/org/elasticsearch/gradle/testkit/LooksLikeATestWithoutNamingConvention1.java create mode 100644 buildSrc/src/testKit/testingConventions/incorrect_naming_conventions/src/test/java/org/elasticsearch/gradle/testkit/LooksLikeATestWithoutNamingConvention2.java create mode 100644 buildSrc/src/testKit/testingConventions/incorrect_naming_conventions/src/test/java/org/elasticsearch/gradle/testkit/LooksLikeATestWithoutNamingConvention3.java create mode 100644 buildSrc/src/testKit/testingConventions/incorrect_naming_conventions/src/test/java/org/elasticsearch/gradle/testkit/LooksLikeTestsButAbstract.java create mode 100644 buildSrc/src/testKit/testingConventions/incorrect_naming_conventions/src/test/java/org/elasticsearch/gradle/testkit/NamingConventionIT.java create mode 100644 buildSrc/src/testKit/testingConventions/incorrect_naming_conventions/src/test/java/org/elasticsearch/gradle/testkit/NamingConventionTests.java create mode 100644 buildSrc/src/testKit/testingConventions/no_tests_in_inner_classes/src/test/java/org/elasticsearch/gradle/testkit/NastyInnerClasses.java create mode 100644 buildSrc/src/testKit/testingConventions/not_implementing_base/src/test/java/org/elasticsearch/gradle/testkit/AbstractIT.java create mode 100644 buildSrc/src/testKit/testingConventions/not_implementing_base/src/test/java/org/elasticsearch/gradle/testkit/Integration.java create mode 100644 buildSrc/src/testKit/testingConventions/not_implementing_base/src/test/java/org/elasticsearch/gradle/testkit/NamingConventionIT.java create mode 100644 buildSrc/src/testKit/testingConventions/not_implementing_base/src/test/java/org/elasticsearch/gradle/testkit/NamingConventionMissmatchIT.java create mode 100644 buildSrc/src/testKit/testingConventions/not_implementing_base/src/test/java/org/elasticsearch/gradle/testkit/NamingConventionMissmatchTests.java create mode 100644 buildSrc/src/testKit/testingConventions/not_implementing_base/src/test/java/org/elasticsearch/gradle/testkit/NamingConventionTests.java create mode 100644 buildSrc/src/testKit/testingConventions/not_implementing_base/src/test/java/org/elasticsearch/gradle/testkit/Unit.java create mode 100644 buildSrc/src/testKit/testingConventions/settings.gradle create mode 100644 buildSrc/src/testKit/testingConventions/valid_setup_no_base/src/test/java/org/elasticsearch/gradle/testkit/NamingConventionIT.java create mode 100644 buildSrc/src/testKit/testingConventions/valid_setup_no_base/src/test/java/org/elasticsearch/gradle/testkit/NamingConventionTests.java create mode 100644 buildSrc/src/testKit/testingConventions/valid_setup_with_base/src/test/java/org/elasticsearch/gradle/testkit/Integration.java create mode 100644 buildSrc/src/testKit/testingConventions/valid_setup_with_base/src/test/java/org/elasticsearch/gradle/testkit/NamingConventionIT.java create mode 100644 buildSrc/src/testKit/testingConventions/valid_setup_with_base/src/test/java/org/elasticsearch/gradle/testkit/NamingConventionTests.java create mode 100644 buildSrc/src/testKit/testingConventions/valid_setup_with_base/src/test/java/org/elasticsearch/gradle/testkit/Unit.java rename x-pack/qa/smoke-test-watcher/src/test/java/org/elasticsearch/smoketest/{WatcherTemplateIT.java => WatcherTemplateTests.java} (99%) diff --git a/buildSrc/build.gradle b/buildSrc/build.gradle index 029727df389e1..b9ff1fd485e05 100644 --- a/buildSrc/build.gradle +++ b/buildSrc/build.gradle @@ -223,6 +223,18 @@ if (project != rootProject) { integTestClass = 'org.elasticsearch.gradle.test.GradleIntegrationTestCase' } + testingConventions { + naming.clear() + naming { + Tests { + baseClass 'org.elasticsearch.gradle.test.GradleUnitTestCase' + } + IT { + baseClass 'org.elasticsearch.gradle.test.GradleIntegrationTestCase' + } + } + } + /* * We alread configure publication and we don't need or want this one that * comes from the java-gradle-plugin. diff --git a/buildSrc/src/main/groovy/org/elasticsearch/gradle/plugin/PluginBuildPlugin.groovy b/buildSrc/src/main/groovy/org/elasticsearch/gradle/plugin/PluginBuildPlugin.groovy index 28d18e9b876f5..d4f7325093218 100644 --- a/buildSrc/src/main/groovy/org/elasticsearch/gradle/plugin/PluginBuildPlugin.groovy +++ b/buildSrc/src/main/groovy/org/elasticsearch/gradle/plugin/PluginBuildPlugin.groovy @@ -78,6 +78,19 @@ public class PluginBuildPlugin extends BuildPlugin { skipIntegTestInDisguise = true } } + project.testingConventions { + naming.clear() + naming { + Tests { + baseClass 'org.apache.lucene.util.LuceneTestCase' + } + IT { + baseClass 'org.elasticsearch.test.ESIntegTestCase' + baseClass 'org.elasticsearch.test.rest.ESRestTestCase' + baseClass 'org.elasticsearch.test.ESSingleNodeTestCase' + } + } + } createIntegTestTask(project) createBundleTask(project) project.configurations.getByName('default').extendsFrom(project.configurations.getByName('runtime')) 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 9fdb1b41ec0d2..6fc65d930b0ab 100644 --- a/buildSrc/src/main/groovy/org/elasticsearch/gradle/precommit/PrecommitTasks.groovy +++ b/buildSrc/src/main/groovy/org/elasticsearch/gradle/precommit/PrecommitTasks.groovy @@ -91,7 +91,17 @@ class PrecommitTasks { } static Task configureTestingConventions(Project project) { - project.getTasks().create("testingConventions", TestingConventionsTasks.class) + TestingConventionsTasks task = project.getTasks().create("testingConventions", TestingConventionsTasks.class) + task.naming { + Tests { + baseClass "org.apache.lucene.util.LuceneTestCase" + } + IT { + baseClass "org.elasticsearch.test.ESIntegTestCase" + baseClass 'org.elasticsearch.test.rest.ESRestTestCase' + } + } + return task } private static Task configureJarHell(Project project) { diff --git a/buildSrc/src/main/java/org/elasticsearch/gradle/precommit/TestingConventionRule.java b/buildSrc/src/main/java/org/elasticsearch/gradle/precommit/TestingConventionRule.java new file mode 100644 index 0000000000000..5fec05d945ed5 --- /dev/null +++ b/buildSrc/src/main/java/org/elasticsearch/gradle/precommit/TestingConventionRule.java @@ -0,0 +1,99 @@ +/* + * 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.gradle.precommit; + +import java.io.Serializable; +import java.util.Collection; +import java.util.HashSet; +import java.util.Objects; +import java.util.Set; +import java.util.regex.Pattern; + +/** + * Represent rules for tests enforced by the @{link {@link TestingConventionsTasks}} + * + * Rules are identified by name, tests must have this name as a suffix and implement one of the base classes + * and be part of all the specified tasks. + */ +public class TestingConventionRule implements Serializable { + + private final String suffix; + + private Set baseClasses = new HashSet<>(); + + private Set taskNames = new HashSet<>(); + + public TestingConventionRule(String suffix) { + this.suffix = suffix; + } + + public String getSuffix() { + return suffix; + } + + /** + * Alias for @{link getSuffix} as Gradle requires a name property + * + */ + public String getName() { + return suffix; + } + + public void baseClass(String clazz) { + baseClasses.add(clazz); + } + + public void setBaseClasses(Collection baseClasses) { + this.baseClasses.clear(); + this.baseClasses.addAll(baseClasses); + } + + public void taskName(Pattern expression) { + taskNames.add(expression); + } + public void taskName(String expression) { + taskNames.add(Pattern.compile(expression)); + } + + public void setTaskNames(Collection expressions) { + taskNames.clear(); + taskNames.addAll(expressions); + } + + public Set getBaseClasses() { + return baseClasses; + } + + public Set getTaskNames() { + return taskNames; + } + + @Override + public boolean equals(Object o) { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + TestingConventionRule that = (TestingConventionRule) o; + return Objects.equals(suffix, that.suffix); + } + + @Override + public int hashCode() { + return Objects.hash(suffix); + } +} 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 105deabfd40fd..efa2684690703 100644 --- a/buildSrc/src/main/java/org/elasticsearch/gradle/precommit/TestingConventionsTasks.java +++ b/buildSrc/src/main/java/org/elasticsearch/gradle/precommit/TestingConventionsTasks.java @@ -18,8 +18,10 @@ */ package org.elasticsearch.gradle.precommit; +import groovy.lang.Closure; import org.elasticsearch.gradle.tool.Boilerplate; import org.gradle.api.DefaultTask; +import org.gradle.api.NamedDomainObjectContainer; import org.gradle.api.Task; import org.gradle.api.file.FileCollection; import org.gradle.api.file.FileTree; @@ -54,50 +56,37 @@ public class TestingConventionsTasks extends DefaultTask { - private static final String TEST_CLASS_SUFIX = "Tests"; - private static final String INTEG_TEST_CLASS_SUFIX = "IT"; private static final String TEST_METHOD_PREFIX = "test"; - /** - * Are there tests to execute ? Accounts for @Ignore and @AwaitsFix - */ - private Boolean activeTestsExists; - private Map testClassNames; + private final NamedDomainObjectContainer naming; + public TestingConventionsTasks() { setDescription("Tests various testing conventions"); // Run only after everything is compiled Boilerplate.getJavaSourceSets(getProject()).all(sourceSet -> dependsOn(sourceSet.getClassesTaskName())); + naming = getProject().container(TestingConventionRule.class); } - + @Input - public Map> classFilesPerTask(FileTree testClassFiles) { + public Map> classFilesPerEnabledTask(FileTree testClassFiles) { Map> collector = new HashMap<>(); + // RandomizedTestingTask collector.putAll( - Stream.concat( - getProject().getTasks().withType(getRandomizedTestingTask()).stream(), - // Look at sub-projects too. As sometimes tests are implemented in parent but ran in sub-projects against - // different configurations - getProject().getSubprojects().stream().flatMap(subproject -> - subproject.getTasks().withType(getRandomizedTestingTask()).stream() - ) - ) + getProject().getTasks().withType(getRandomizedTestingTask()).stream() .filter(Task::getEnabled) .collect(Collectors.toMap( Task::getPath, task -> testClassFiles.matching(getRandomizedTestingPatternSet(task)).getFiles() - )) + ) + ) ); + // Gradle Test collector.putAll( - Stream.concat( - getProject().getTasks().withType(Test.class).stream(), - getProject().getSubprojects().stream().flatMap(subproject -> - subproject.getTasks().withType(Test.class).stream() - ) - ) + getProject().getTasks().withType(Test.class).stream() .filter(Task::getEnabled) .collect(Collectors.toMap( Task::getPath, @@ -119,14 +108,22 @@ public Map getTestClassNames() { return testClassNames; } + @Input + public NamedDomainObjectContainer getNaming() { + return naming; + } + @OutputFile public File getSuccessMarker() { return new File(getProject().getBuildDir(), "markers/" + getName()); } + public void naming(Closure action) { + naming.configure(action); + } + @TaskAction public void doCheck() throws IOException { - activeTestsExists = false; final String problems; try (URLClassLoader isolatedClassLoader = new URLClassLoader( @@ -134,62 +131,83 @@ public void doCheck() throws IOException { )) { Predicate> isStaticClass = clazz -> Modifier.isStatic(clazz.getModifiers()); Predicate> isPublicClass = clazz -> Modifier.isPublic(clazz.getModifiers()); - Predicate> implementsNamingConvention = clazz -> - clazz.getName().endsWith(TEST_CLASS_SUFIX) || - clazz.getName().endsWith(INTEG_TEST_CLASS_SUFIX); + Predicate> isAbstractClass = clazz -> Modifier.isAbstract(clazz.getModifiers()); - Map> classes = getTestClassNames().entrySet().stream() + final Map> classes = getTestClassNames().entrySet().stream() .collect(Collectors.toMap( Map.Entry::getValue, entry -> loadClassWithoutInitializing(entry.getKey(), isolatedClassLoader)) ); - FileTree allTestClassFiles = getProject().files( + final FileTree allTestClassFiles = getProject().files( classes.values().stream() .filter(isStaticClass.negate()) .filter(isPublicClass) - .filter(implementsNamingConvention) + .filter((Predicate>) this::implementsNamingConvention) .map(clazz -> testClassNames.get(clazz.getName())) .collect(Collectors.toList()) ).getAsFileTree(); - final Map> classFilesPerTask = classFilesPerTask(allTestClassFiles); + final Map> classFilesPerTask = classFilesPerEnabledTask(allTestClassFiles); - Map>> testClassesPerTask = classFilesPerTask.entrySet().stream() + final Map>> testClassesPerTask = classFilesPerTask.entrySet().stream() .collect( Collectors.toMap( Map.Entry::getKey, entry -> entry.getValue().stream() .map(classes::get) - .filter(implementsNamingConvention) + .filter(this::implementsNamingConvention) .collect(Collectors.toSet()) ) ); + final Map>> suffixToBaseClass; + if (classes.isEmpty()) { + // Don't load base classes if we don't have any tests. + // This allows defaults to be configured for projects that don't have any tests + // + suffixToBaseClass = Collections.emptyMap(); + } else { + suffixToBaseClass = naming.stream() + .collect( + Collectors.toMap( + TestingConventionRule::getSuffix, + rule -> rule.getBaseClasses().stream() + .map(each -> loadClassWithoutInitializing(each, isolatedClassLoader)) + .collect(Collectors.toSet()) + )); + } + problems = collectProblems( checkNoneExists( "Test classes implemented by inner classes will not run", classes.values().stream() .filter(isStaticClass) - .filter(implementsNamingConvention.or(this::seemsLikeATest)) + .filter(isPublicClass) + .filter(((Predicate>) this::implementsNamingConvention).or(this::seemsLikeATest)) ), checkNoneExists( "Seem like test classes but don't match naming convention", classes.values().stream() .filter(isStaticClass.negate()) .filter(isPublicClass) - .filter(this::seemsLikeATest) - .filter(implementsNamingConvention.negate()) + .filter(isAbstractClass.negate()) + .filter(this::seemsLikeATest) // TODO when base classes are set, check for classes that extend them + .filter(((Predicate>) this::implementsNamingConvention).negate()) ), + // TODO: check for non public classes that seem like tests + // TODO: check for abstract classes that implement the naming conventions + // No empty enabled tasks collectProblems( testClassesPerTask.entrySet().stream() - .map( entry -> + .map(entry -> checkAtLeastOneExists( - "test class in " + entry.getKey(), + "test class included in task " + entry.getKey(), entry.getValue().stream() ) ) - .collect(Collectors.joining()) + .sorted() + .collect(Collectors.joining("\n")) ), checkNoneExists( "Test classes are not included in any enabled task (" + @@ -201,25 +219,43 @@ public void doCheck() throws IOException { .anyMatch(fileSet -> fileSet.contains(testFile)) == false ) .map(classes::get) + ), + collectProblems( + suffixToBaseClass.entrySet().stream() + .filter(entry -> entry.getValue().isEmpty() == false) + .map(entry -> { + return checkNoneExists( + "Tests classes with suffix `" + entry.getKey() + "` should extend " + + entry.getValue().stream().map(Class::getName).collect(Collectors.joining(" or ")) + + " but the following classes do not", + classes.values().stream() + .filter(clazz -> clazz.getName().endsWith(entry.getKey())) + .filter(clazz -> entry.getValue().stream() + .anyMatch(test -> test.isAssignableFrom(clazz)) == false) + ); + }).sorted() + .collect(Collectors.joining("\n")) ) + // TODO: check that the testing tasks are included in the right task based on the name ( from the rule ) + // TODO: check to make sure that the main source set doesn't have classes that match + // the naming convention (just the names, don't load classes) ); } if (problems.isEmpty()) { - getLogger().error(problems); - throw new IllegalStateException("Testing conventions are not honored"); - } else { getSuccessMarker().getParentFile().mkdirs(); Files.write(getSuccessMarker().toPath(), new byte[]{}, StandardOpenOption.CREATE); + } else { + getLogger().error(problems); + throw new IllegalStateException("Testing conventions are not honored"); } } private String collectProblems(String... problems) { return Stream.of(problems) .map(String::trim) - .filter(String::isEmpty) - .map(each -> each + "\n") - .collect(Collectors.joining()); + .filter(s -> s.isEmpty() == false) + .collect(Collectors.joining("\n")); } @SuppressWarnings("unchecked") @@ -251,10 +287,11 @@ private Class getRandomizedTestingTask() { private String checkNoneExists(String message, Stream> stream) { String problem = stream .map(each -> " * " + each.getName()) + .sorted() .collect(Collectors.joining("\n")); if (problem.isEmpty() == false) { return message + ":\n" + problem; - } else{ + } else { return ""; } } @@ -263,28 +300,33 @@ private String checkAtLeastOneExists(String message, Stream> if (stream.findAny().isPresent()) { return ""; } else { - return "Expected at least one " + message + ", but found none.\n"; + return "Expected at least one " + message + ", but found none."; } } private boolean seemsLikeATest(Class clazz) { try { ClassLoader classLoader = clazz.getClassLoader(); - Class junitTest; - try { - junitTest = classLoader.loadClass("junit.framework.Test"); - } catch (ClassNotFoundException e) { - throw new IllegalStateException("Could not load junit.framework.Test. It's expected that this class is " + - "available on the tests classpath"); - } + + Class junitTest = loadClassWithoutInitializing("org.junit.Assert", classLoader); if (junitTest.isAssignableFrom(clazz)) { - getLogger().info("{} is a test because it extends junit.framework.Test", clazz.getName()); + getLogger().info("{} 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(clazz, method)) return true; - if (isAnnotated(clazz, method, junitTest)) return true; + if (matchesTestMethodNamingConvention(method)) { + getLogger().info("{} 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 '{}'", + clazz.getName(), method.getName(), junitAnnotation.getName()); + return true; + } } + return false; } catch (NoClassDefFoundError e) { // Include the message to get more info to get more a more useful message when running Gradle without -s @@ -294,23 +336,25 @@ private boolean seemsLikeATest(Class clazz) { } } - private boolean matchesTestMethodNamingConvention(Class clazz, Method method) { - if (method.getName().startsWith(TEST_METHOD_PREFIX) && - Modifier.isStatic(method.getModifiers()) == false && - method.getReturnType().equals(Void.class) - ) { - getLogger().info("{} is a test because it has method: {}", clazz.getName(), method.getName()); + 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()); return true; } return false; } - private boolean isAnnotated(Class clazz, Method method, Class annotation) { + private boolean matchesTestMethodNamingConvention(Method method) { + return method.getName().startsWith(TEST_METHOD_PREFIX) && + Modifier.isStatic(method.getModifiers()) == false + ; + } + + private boolean isAnnotated(Method method, Class annotation) { for (Annotation presentAnnotation : method.getAnnotations()) { if (annotation.isAssignableFrom(presentAnnotation.getClass())) { - getLogger().info("{} is a test because {} is annotated with junit.framework.Test", - clazz.getName(), method.getName() - ); return true; } } @@ -380,14 +424,14 @@ public FileVisitResult visitFileFailed(Path file, IOException exc) throws IOExce private Class loadClassWithoutInitializing(String name, ClassLoader isolatedClassLoader) { try { - return Class.forName(name, + return Class.forName( + name, // Don't initialize the class to save time. Not needed for this test and this doesn't share a VM with any other tests. false, isolatedClassLoader ); } catch (ClassNotFoundException e) { - // Will not get here as the exception will be loaded by isolatedClassLoader - throw new RuntimeException("Failed to load class " + name, e); + throw new RuntimeException("Failed to load class " + name + ". Incorrect test runtime classpath?", e); } } diff --git a/buildSrc/src/main/java/org/elasticsearch/gradle/testfixtures/TestFixturesPlugin.java b/buildSrc/src/main/java/org/elasticsearch/gradle/testfixtures/TestFixturesPlugin.java index 73b3baf66ddeb..c13bcc02cbe89 100644 --- a/buildSrc/src/main/java/org/elasticsearch/gradle/testfixtures/TestFixturesPlugin.java +++ b/buildSrc/src/main/java/org/elasticsearch/gradle/testfixtures/TestFixturesPlugin.java @@ -21,6 +21,7 @@ import com.avast.gradle.dockercompose.ComposeExtension; import com.avast.gradle.dockercompose.DockerComposePlugin; import org.elasticsearch.gradle.precommit.JarHellTask; +import org.elasticsearch.gradle.precommit.TestingConventionsTasks; import org.elasticsearch.gradle.precommit.ThirdPartyAuditTask; import org.gradle.api.DefaultTask; import org.gradle.api.Plugin; @@ -100,6 +101,10 @@ public void apply(Project project) { tasks.withType(getTaskClass("com.carrotsearch.gradle.junit4.RandomizedTestingTask"), task -> task.setEnabled(false) ); + // conventions are not honored when the tasks are disabled + tasks.withType(TestingConventionsTasks.class, task -> + task.setEnabled(false) + ); return; } tasks.withType(getTaskClass("com.carrotsearch.gradle.junit4.RandomizedTestingTask"), task -> diff --git a/buildSrc/src/main/resources/checkstyle_suppressions.xml b/buildSrc/src/main/resources/checkstyle_suppressions.xml index c793e227a7be6..3233d66ddd58a 100644 --- a/buildSrc/src/main/resources/checkstyle_suppressions.xml +++ b/buildSrc/src/main/resources/checkstyle_suppressions.xml @@ -67,5 +67,8 @@ - + + + + diff --git a/buildSrc/src/test/java/org/elasticsearch/gradle/precommit/TestingConventionsTasksIT.java b/buildSrc/src/test/java/org/elasticsearch/gradle/precommit/TestingConventionsTasksIT.java new file mode 100644 index 0000000000000..dbe06287782f7 --- /dev/null +++ b/buildSrc/src/test/java/org/elasticsearch/gradle/precommit/TestingConventionsTasksIT.java @@ -0,0 +1,108 @@ +/* + * 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.gradle.precommit; + +import org.elasticsearch.gradle.test.GradleIntegrationTestCase; +import org.gradle.testkit.runner.BuildResult; +import org.gradle.testkit.runner.GradleRunner; +import org.junit.Before; + +public class TestingConventionsTasksIT extends GradleIntegrationTestCase { + + @Before + public void setUp() { + } + + public void testInnerClasses() { + GradleRunner runner = getGradleRunner("testingConventions") + .withArguments("clean", ":no_tests_in_inner_classes:testingConventions", "-i", "-s"); + BuildResult result = runner.buildAndFail(); + assertOutputContains(result.getOutput(), + "Test classes implemented by inner classes will not run:", + " * org.elasticsearch.gradle.testkit.NastyInnerClasses$LooksLikeATestWithoutNamingConvention1", + " * org.elasticsearch.gradle.testkit.NastyInnerClasses$LooksLikeATestWithoutNamingConvention2", + " * org.elasticsearch.gradle.testkit.NastyInnerClasses$LooksLikeATestWithoutNamingConvention3", + " * org.elasticsearch.gradle.testkit.NastyInnerClasses$NamingConventionIT", + " * org.elasticsearch.gradle.testkit.NastyInnerClasses$NamingConventionTests" + ); + } + + public void testNamingConvention() { + GradleRunner runner = getGradleRunner("testingConventions") + .withArguments("clean", ":incorrect_naming_conventions:testingConventions", "-i", "-s"); + BuildResult result = runner.buildAndFail(); + assertOutputContains(result.getOutput(), + "Seem like test classes but don't match naming convention:", + " * org.elasticsearch.gradle.testkit.LooksLikeATestWithoutNamingConvention1", + " * org.elasticsearch.gradle.testkit.LooksLikeATestWithoutNamingConvention2", + " * org.elasticsearch.gradle.testkit.LooksLikeATestWithoutNamingConvention3" + ); + assertOutputDoesNotContain(result.getOutput(), "LooksLikeTestsButAbstract"); + } + + public void testNoEmptyTasks() { + GradleRunner runner = getGradleRunner("testingConventions") + .withArguments("clean", ":empty_test_task:testingConventions", "-i", "-s"); + BuildResult result = runner.buildAndFail(); + assertOutputContains(result.getOutput(), + "Expected at least one test class included in task :empty_test_task:emptyTest, but found none.", + "Expected at least one test class included in task :empty_test_task:emptyTestRandomized, but found none." + ); + } + + public void testAllTestTasksIncluded() { + GradleRunner runner = getGradleRunner("testingConventions") + .withArguments("clean", ":all_classes_in_tasks:testingConventions", "-i", "-s"); + BuildResult result = runner.buildAndFail(); + assertOutputContains(result.getOutput(), + "Test classes are not included in any enabled task (:all_classes_in_tasks:emptyTestRandomized):", + " * org.elasticsearch.gradle.testkit.NamingConventionIT", + " * org.elasticsearch.gradle.testkit.NamingConventionTests" + ); + } + + public void testTaskNotImplementBaseClass() { + GradleRunner runner = getGradleRunner("testingConventions") + .withArguments("clean", ":not_implementing_base:testingConventions", "-i", "-s"); + BuildResult result = runner.buildAndFail(); + assertOutputContains(result.getOutput(), + "Tests classes with suffix `IT` should extend org.elasticsearch.gradle.testkit.Integration but the following classes do not:", + " * org.elasticsearch.gradle.testkit.NamingConventionIT", + " * org.elasticsearch.gradle.testkit.NamingConventionMissmatchIT", + "Tests classes with suffix `Tests` should extend org.elasticsearch.gradle.testkit.Unit but the following classes do not:", + " * org.elasticsearch.gradle.testkit.NamingConventionMissmatchTests", + " * org.elasticsearch.gradle.testkit.NamingConventionTests" + ); + } + + public void testValidSetupWithoutBaseClass() { + GradleRunner runner = getGradleRunner("testingConventions") + .withArguments("clean", ":valid_setup_no_base:testingConventions", "-i", "-s"); + BuildResult result = runner.build(); + assertTaskSuccessful(result, ":valid_setup_no_base:testingConventions"); + } + + public void testValidSetupWithBaseClass() { + GradleRunner runner = getGradleRunner("testingConventions") + .withArguments("clean", ":valid_setup_with_base:testingConventions", "-i", "-s"); + BuildResult result = runner.build(); + assertTaskSuccessful(result, ":valid_setup_with_base:testingConventions"); + } + +} diff --git a/buildSrc/src/test/java/org/elasticsearch/gradle/test/GradleIntegrationTestCase.java b/buildSrc/src/test/java/org/elasticsearch/gradle/test/GradleIntegrationTestCase.java index ab29a33a62eda..3e1d0b176b011 100644 --- a/buildSrc/src/test/java/org/elasticsearch/gradle/test/GradleIntegrationTestCase.java +++ b/buildSrc/src/test/java/org/elasticsearch/gradle/test/GradleIntegrationTestCase.java @@ -43,7 +43,7 @@ protected void assertOutputContains(String output, String... lines) { if (index.equals(index.stream().sorted().collect(Collectors.toList())) == false) { fail("Expected the following lines to appear in this order:\n" + Stream.of(lines).map(line -> " - `" + line + "`").collect(Collectors.joining("\n")) + - "\nBut they did not. Output is:\n\n```" + output + "\n```\n" + "\nTBut the order was different. Output is:\n\n```" + output + "\n```\n" ); } } diff --git a/buildSrc/src/testKit/testingConventions/all_classes_in_tasks/src/test/java/org/elasticsearch/gradle/testkit/NamingConventionIT.java b/buildSrc/src/testKit/testingConventions/all_classes_in_tasks/src/test/java/org/elasticsearch/gradle/testkit/NamingConventionIT.java new file mode 100644 index 0000000000000..48a4f7adfd99e --- /dev/null +++ b/buildSrc/src/testKit/testingConventions/all_classes_in_tasks/src/test/java/org/elasticsearch/gradle/testkit/NamingConventionIT.java @@ -0,0 +1,23 @@ +/* + * 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.gradle.testkit; + +public class NamingConventionIT { + +} \ No newline at end of file diff --git a/buildSrc/src/testKit/testingConventions/all_classes_in_tasks/src/test/java/org/elasticsearch/gradle/testkit/NamingConventionTests.java b/buildSrc/src/testKit/testingConventions/all_classes_in_tasks/src/test/java/org/elasticsearch/gradle/testkit/NamingConventionTests.java new file mode 100644 index 0000000000000..6afb89ddf56b0 --- /dev/null +++ b/buildSrc/src/testKit/testingConventions/all_classes_in_tasks/src/test/java/org/elasticsearch/gradle/testkit/NamingConventionTests.java @@ -0,0 +1,23 @@ +/* + * 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.gradle.testkit; + +public class NamingConventionTests { + +} diff --git a/buildSrc/src/testKit/testingConventions/build.gradle b/buildSrc/src/testKit/testingConventions/build.gradle new file mode 100644 index 0000000000000..d1a21a1ead0e7 --- /dev/null +++ b/buildSrc/src/testKit/testingConventions/build.gradle @@ -0,0 +1,86 @@ +plugins { + id 'elasticsearch.build' apply false +} + +allprojects { + apply plugin: 'java' + apply plugin: 'elasticsearch.build' + + repositories { + jcenter() + } + dependencies { + testCompile "junit:junit:4.12" + } + + ext.licenseFile = file("$buildDir/dummy/license") + ext.noticeFile = file("$buildDir/dummy/notice") + + testingConventions.naming { + // Reset default to no baseClass checks + Tests { + baseClasses = [] + } + IT { + baseClasses = [] + } + } + + unitTest.enabled = false +} + +project(':empty_test_task') { + task emptyTest(type: Test) { + + } + + task emptyTestRandomized(type: com.carrotsearch.gradle.junit4.RandomizedTestingTask) { + + } +} + +project(':all_classes_in_tasks') { + task emptyTestRandomized(type: com.carrotsearch.gradle.junit4.RandomizedTestingTask) { + include "**/Convention*" + } +} + +project(':not_implementing_base') { + testingConventions.naming { + Tests { + baseClass 'org.elasticsearch.gradle.testkit.Unit' + } + IT { + baseClass 'org.elasticsearch.gradle.testkit.Integration' + } + } + task randomized(type: com.carrotsearch.gradle.junit4.RandomizedTestingTask) { + include "**/*IT.class" + include "**/*Tests.class" + } +} + +project(':valid_setup_no_base') { + task randomized(type: com.carrotsearch.gradle.junit4.RandomizedTestingTask) { + include "**/*IT.class" + include "**/*Tests.class" + } +} + +project (':valid_setup_with_base') { + task randomized(type: com.carrotsearch.gradle.junit4.RandomizedTestingTask) { + include "**/*IT.class" + include "**/*Tests.class" + } + testingConventions.naming { + Tests { + baseClass 'org.elasticsearch.gradle.testkit.Unit' + } + IT { + baseClass 'org.elasticsearch.gradle.testkit.Integration' + } + } +} + + + diff --git a/buildSrc/src/testKit/testingConventions/empty_test_task/.gitignore b/buildSrc/src/testKit/testingConventions/empty_test_task/.gitignore new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/buildSrc/src/testKit/testingConventions/incorrect_naming_conventions/src/test/java/org/elasticsearch/gradle/testkit/LooksLikeATestWithoutNamingConvention1.java b/buildSrc/src/testKit/testingConventions/incorrect_naming_conventions/src/test/java/org/elasticsearch/gradle/testkit/LooksLikeATestWithoutNamingConvention1.java new file mode 100644 index 0000000000000..35d60d8a56b53 --- /dev/null +++ b/buildSrc/src/testKit/testingConventions/incorrect_naming_conventions/src/test/java/org/elasticsearch/gradle/testkit/LooksLikeATestWithoutNamingConvention1.java @@ -0,0 +1,30 @@ +/* + * 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.gradle.testkit; + +import org.junit.Test; + +public class LooksLikeATestWithoutNamingConvention1 { + + @Test + public void annotatedTestMethod() { + + } + +} \ No newline at end of file diff --git a/buildSrc/src/testKit/testingConventions/incorrect_naming_conventions/src/test/java/org/elasticsearch/gradle/testkit/LooksLikeATestWithoutNamingConvention2.java b/buildSrc/src/testKit/testingConventions/incorrect_naming_conventions/src/test/java/org/elasticsearch/gradle/testkit/LooksLikeATestWithoutNamingConvention2.java new file mode 100644 index 0000000000000..1de116d21538d --- /dev/null +++ b/buildSrc/src/testKit/testingConventions/incorrect_naming_conventions/src/test/java/org/elasticsearch/gradle/testkit/LooksLikeATestWithoutNamingConvention2.java @@ -0,0 +1,25 @@ +/* + * 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.gradle.testkit; + +import org.junit.Assert; + +public class LooksLikeATestWithoutNamingConvention2 extends Assert { + +} diff --git a/buildSrc/src/testKit/testingConventions/incorrect_naming_conventions/src/test/java/org/elasticsearch/gradle/testkit/LooksLikeATestWithoutNamingConvention3.java b/buildSrc/src/testKit/testingConventions/incorrect_naming_conventions/src/test/java/org/elasticsearch/gradle/testkit/LooksLikeATestWithoutNamingConvention3.java new file mode 100644 index 0000000000000..4a946c3aeb0ae --- /dev/null +++ b/buildSrc/src/testKit/testingConventions/incorrect_naming_conventions/src/test/java/org/elasticsearch/gradle/testkit/LooksLikeATestWithoutNamingConvention3.java @@ -0,0 +1,30 @@ +/* + * 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.gradle.testkit; + +import org.junit.Assert; +import org.junit.Test; + +public class LooksLikeATestWithoutNamingConvention3 { + + public void testMethod() { + + } + +} \ No newline at end of file diff --git a/buildSrc/src/testKit/testingConventions/incorrect_naming_conventions/src/test/java/org/elasticsearch/gradle/testkit/LooksLikeTestsButAbstract.java b/buildSrc/src/testKit/testingConventions/incorrect_naming_conventions/src/test/java/org/elasticsearch/gradle/testkit/LooksLikeTestsButAbstract.java new file mode 100644 index 0000000000000..15718cff841e1 --- /dev/null +++ b/buildSrc/src/testKit/testingConventions/incorrect_naming_conventions/src/test/java/org/elasticsearch/gradle/testkit/LooksLikeTestsButAbstract.java @@ -0,0 +1,30 @@ +/* + * 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.gradle.testkit; + +import org.junit.Assert; +import org.junit.Test; + +public abstract class LooksLikeTestsButAbstract { + + public void testMethod() { + + } + +} diff --git a/buildSrc/src/testKit/testingConventions/incorrect_naming_conventions/src/test/java/org/elasticsearch/gradle/testkit/NamingConventionIT.java b/buildSrc/src/testKit/testingConventions/incorrect_naming_conventions/src/test/java/org/elasticsearch/gradle/testkit/NamingConventionIT.java new file mode 100644 index 0000000000000..48a4f7adfd99e --- /dev/null +++ b/buildSrc/src/testKit/testingConventions/incorrect_naming_conventions/src/test/java/org/elasticsearch/gradle/testkit/NamingConventionIT.java @@ -0,0 +1,23 @@ +/* + * 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.gradle.testkit; + +public class NamingConventionIT { + +} \ No newline at end of file diff --git a/buildSrc/src/testKit/testingConventions/incorrect_naming_conventions/src/test/java/org/elasticsearch/gradle/testkit/NamingConventionTests.java b/buildSrc/src/testKit/testingConventions/incorrect_naming_conventions/src/test/java/org/elasticsearch/gradle/testkit/NamingConventionTests.java new file mode 100644 index 0000000000000..95152520a3f2d --- /dev/null +++ b/buildSrc/src/testKit/testingConventions/incorrect_naming_conventions/src/test/java/org/elasticsearch/gradle/testkit/NamingConventionTests.java @@ -0,0 +1,23 @@ +/* + * 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.gradle.testkit; + +public class NamingConventionTests { + +} \ No newline at end of file diff --git a/buildSrc/src/testKit/testingConventions/no_tests_in_inner_classes/src/test/java/org/elasticsearch/gradle/testkit/NastyInnerClasses.java b/buildSrc/src/testKit/testingConventions/no_tests_in_inner_classes/src/test/java/org/elasticsearch/gradle/testkit/NastyInnerClasses.java new file mode 100644 index 0000000000000..c09cd9292926d --- /dev/null +++ b/buildSrc/src/testKit/testingConventions/no_tests_in_inner_classes/src/test/java/org/elasticsearch/gradle/testkit/NastyInnerClasses.java @@ -0,0 +1,64 @@ +/* + * 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.gradle.testkit; + +import org.junit.Assert; +import org.junit.Test; + +public class NastyInnerClasses { + + public static class NamingConventionTests { + + } + + public static class NamingConventionIT { + + } + + public static class LooksLikeATestWithoutNamingConvention1 { + @Test + public void annotatedTestMethod() { + + } + } + + public static class LooksLikeATestWithoutNamingConvention2 extends Assert { + + } + + public static class LooksLikeATestWithoutNamingConvention3 { + + public void testMethod() { + + } + + } + + static abstract public class NonOffendingAbstractTests { + + } + + private static class NonOffendingPrivateTests { + + } + + static class NonOffendingPackageTests { + + } +} diff --git a/buildSrc/src/testKit/testingConventions/not_implementing_base/src/test/java/org/elasticsearch/gradle/testkit/AbstractIT.java b/buildSrc/src/testKit/testingConventions/not_implementing_base/src/test/java/org/elasticsearch/gradle/testkit/AbstractIT.java new file mode 100644 index 0000000000000..6abba7fd52776 --- /dev/null +++ b/buildSrc/src/testKit/testingConventions/not_implementing_base/src/test/java/org/elasticsearch/gradle/testkit/AbstractIT.java @@ -0,0 +1,23 @@ +/* + * 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.gradle.testkit; + +public abstract class AbstractIT { + +} diff --git a/buildSrc/src/testKit/testingConventions/not_implementing_base/src/test/java/org/elasticsearch/gradle/testkit/Integration.java b/buildSrc/src/testKit/testingConventions/not_implementing_base/src/test/java/org/elasticsearch/gradle/testkit/Integration.java new file mode 100644 index 0000000000000..80522be3fb5ab --- /dev/null +++ b/buildSrc/src/testKit/testingConventions/not_implementing_base/src/test/java/org/elasticsearch/gradle/testkit/Integration.java @@ -0,0 +1,23 @@ +/* + * 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.gradle.testkit; + +public class Integration { + +} diff --git a/buildSrc/src/testKit/testingConventions/not_implementing_base/src/test/java/org/elasticsearch/gradle/testkit/NamingConventionIT.java b/buildSrc/src/testKit/testingConventions/not_implementing_base/src/test/java/org/elasticsearch/gradle/testkit/NamingConventionIT.java new file mode 100644 index 0000000000000..48a4f7adfd99e --- /dev/null +++ b/buildSrc/src/testKit/testingConventions/not_implementing_base/src/test/java/org/elasticsearch/gradle/testkit/NamingConventionIT.java @@ -0,0 +1,23 @@ +/* + * 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.gradle.testkit; + +public class NamingConventionIT { + +} \ No newline at end of file diff --git a/buildSrc/src/testKit/testingConventions/not_implementing_base/src/test/java/org/elasticsearch/gradle/testkit/NamingConventionMissmatchIT.java b/buildSrc/src/testKit/testingConventions/not_implementing_base/src/test/java/org/elasticsearch/gradle/testkit/NamingConventionMissmatchIT.java new file mode 100644 index 0000000000000..ea7f921372f9b --- /dev/null +++ b/buildSrc/src/testKit/testingConventions/not_implementing_base/src/test/java/org/elasticsearch/gradle/testkit/NamingConventionMissmatchIT.java @@ -0,0 +1,23 @@ +/* + * 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.gradle.testkit; + +public class NamingConventionMissmatchIT extends Unit { + +} diff --git a/buildSrc/src/testKit/testingConventions/not_implementing_base/src/test/java/org/elasticsearch/gradle/testkit/NamingConventionMissmatchTests.java b/buildSrc/src/testKit/testingConventions/not_implementing_base/src/test/java/org/elasticsearch/gradle/testkit/NamingConventionMissmatchTests.java new file mode 100644 index 0000000000000..76e8b25c69f73 --- /dev/null +++ b/buildSrc/src/testKit/testingConventions/not_implementing_base/src/test/java/org/elasticsearch/gradle/testkit/NamingConventionMissmatchTests.java @@ -0,0 +1,23 @@ +/* + * 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.gradle.testkit; + +public class NamingConventionMissmatchTests extends Integration { + +} diff --git a/buildSrc/src/testKit/testingConventions/not_implementing_base/src/test/java/org/elasticsearch/gradle/testkit/NamingConventionTests.java b/buildSrc/src/testKit/testingConventions/not_implementing_base/src/test/java/org/elasticsearch/gradle/testkit/NamingConventionTests.java new file mode 100644 index 0000000000000..6afb89ddf56b0 --- /dev/null +++ b/buildSrc/src/testKit/testingConventions/not_implementing_base/src/test/java/org/elasticsearch/gradle/testkit/NamingConventionTests.java @@ -0,0 +1,23 @@ +/* + * 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.gradle.testkit; + +public class NamingConventionTests { + +} diff --git a/buildSrc/src/testKit/testingConventions/not_implementing_base/src/test/java/org/elasticsearch/gradle/testkit/Unit.java b/buildSrc/src/testKit/testingConventions/not_implementing_base/src/test/java/org/elasticsearch/gradle/testkit/Unit.java new file mode 100644 index 0000000000000..6a8ca7f758876 --- /dev/null +++ b/buildSrc/src/testKit/testingConventions/not_implementing_base/src/test/java/org/elasticsearch/gradle/testkit/Unit.java @@ -0,0 +1,23 @@ +/* + * 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.gradle.testkit; + +public class Unit { + +} diff --git a/buildSrc/src/testKit/testingConventions/settings.gradle b/buildSrc/src/testKit/testingConventions/settings.gradle new file mode 100644 index 0000000000000..2baec09d27c8e --- /dev/null +++ b/buildSrc/src/testKit/testingConventions/settings.gradle @@ -0,0 +1,7 @@ +include 'no_tests_in_inner_classes' +include 'incorrect_naming_conventions' +include 'empty_test_task' +include 'all_classes_in_tasks' +include 'not_implementing_base' +include 'valid_setup_no_base' +include 'valid_setup_with_base' \ No newline at end of file diff --git a/buildSrc/src/testKit/testingConventions/valid_setup_no_base/src/test/java/org/elasticsearch/gradle/testkit/NamingConventionIT.java b/buildSrc/src/testKit/testingConventions/valid_setup_no_base/src/test/java/org/elasticsearch/gradle/testkit/NamingConventionIT.java new file mode 100644 index 0000000000000..48a4f7adfd99e --- /dev/null +++ b/buildSrc/src/testKit/testingConventions/valid_setup_no_base/src/test/java/org/elasticsearch/gradle/testkit/NamingConventionIT.java @@ -0,0 +1,23 @@ +/* + * 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.gradle.testkit; + +public class NamingConventionIT { + +} \ No newline at end of file diff --git a/buildSrc/src/testKit/testingConventions/valid_setup_no_base/src/test/java/org/elasticsearch/gradle/testkit/NamingConventionTests.java b/buildSrc/src/testKit/testingConventions/valid_setup_no_base/src/test/java/org/elasticsearch/gradle/testkit/NamingConventionTests.java new file mode 100644 index 0000000000000..6afb89ddf56b0 --- /dev/null +++ b/buildSrc/src/testKit/testingConventions/valid_setup_no_base/src/test/java/org/elasticsearch/gradle/testkit/NamingConventionTests.java @@ -0,0 +1,23 @@ +/* + * 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.gradle.testkit; + +public class NamingConventionTests { + +} diff --git a/buildSrc/src/testKit/testingConventions/valid_setup_with_base/src/test/java/org/elasticsearch/gradle/testkit/Integration.java b/buildSrc/src/testKit/testingConventions/valid_setup_with_base/src/test/java/org/elasticsearch/gradle/testkit/Integration.java new file mode 100644 index 0000000000000..508adc24bc73a --- /dev/null +++ b/buildSrc/src/testKit/testingConventions/valid_setup_with_base/src/test/java/org/elasticsearch/gradle/testkit/Integration.java @@ -0,0 +1,23 @@ +/* + * 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.gradle.testkit; + +public class Integration { + +} diff --git a/buildSrc/src/testKit/testingConventions/valid_setup_with_base/src/test/java/org/elasticsearch/gradle/testkit/NamingConventionIT.java b/buildSrc/src/testKit/testingConventions/valid_setup_with_base/src/test/java/org/elasticsearch/gradle/testkit/NamingConventionIT.java new file mode 100644 index 0000000000000..cdb7ff1f1a8f3 --- /dev/null +++ b/buildSrc/src/testKit/testingConventions/valid_setup_with_base/src/test/java/org/elasticsearch/gradle/testkit/NamingConventionIT.java @@ -0,0 +1,23 @@ +/* + * 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.gradle.testkit; + +public class NamingConventionIT extends Integration { + +} diff --git a/buildSrc/src/testKit/testingConventions/valid_setup_with_base/src/test/java/org/elasticsearch/gradle/testkit/NamingConventionTests.java b/buildSrc/src/testKit/testingConventions/valid_setup_with_base/src/test/java/org/elasticsearch/gradle/testkit/NamingConventionTests.java new file mode 100644 index 0000000000000..12060909b1df4 --- /dev/null +++ b/buildSrc/src/testKit/testingConventions/valid_setup_with_base/src/test/java/org/elasticsearch/gradle/testkit/NamingConventionTests.java @@ -0,0 +1,23 @@ +/* + * 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.gradle.testkit; + +public class NamingConventionTests extends Unit { + +} diff --git a/buildSrc/src/testKit/testingConventions/valid_setup_with_base/src/test/java/org/elasticsearch/gradle/testkit/Unit.java b/buildSrc/src/testKit/testingConventions/valid_setup_with_base/src/test/java/org/elasticsearch/gradle/testkit/Unit.java new file mode 100644 index 0000000000000..6a8ca7f758876 --- /dev/null +++ b/buildSrc/src/testKit/testingConventions/valid_setup_with_base/src/test/java/org/elasticsearch/gradle/testkit/Unit.java @@ -0,0 +1,23 @@ +/* + * 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.gradle.testkit; + +public class Unit { + +} diff --git a/client/rest/build.gradle b/client/rest/build.gradle index 1adea057a953a..a6d8eb8467dab 100644 --- a/client/rest/build.gradle +++ b/client/rest/build.gradle @@ -77,6 +77,15 @@ namingConventions { skipIntegTestInDisguise = true } +testingConventions { + naming.clear() + naming { + Tests { + baseClass 'org.elasticsearch.client.RestClientTestCase' + } + } +} + thirdPartyAudit.ignoreMissingClasses ( //commons-logging optional dependencies 'org.apache.avalon.framework.logger.Logger', diff --git a/client/rest/src/test/java/org/elasticsearch/client/documentation/RestClientDocumentation.java b/client/rest/src/test/java/org/elasticsearch/client/documentation/RestClientDocumentation.java index 9191f5025581b..7eae17d83cf2b 100644 --- a/client/rest/src/test/java/org/elasticsearch/client/documentation/RestClientDocumentation.java +++ b/client/rest/src/test/java/org/elasticsearch/client/documentation/RestClientDocumentation.java @@ -91,7 +91,7 @@ public class RestClientDocumentation { // end::rest-client-options-singleton @SuppressWarnings("unused") - public void testUsage() throws IOException, InterruptedException { + public void usage() throws IOException, InterruptedException { //tag::rest-client-init RestClient restClient = RestClient.builder( @@ -291,7 +291,7 @@ public void onFailure(Exception exception) { } @SuppressWarnings("unused") - public void testCommonConfiguration() throws Exception { + public void commonConfiguration() throws Exception { { //tag::rest-client-config-timeouts RestClientBuilder builder = RestClient.builder( diff --git a/client/sniffer/build.gradle b/client/sniffer/build.gradle index 283b564b40a6f..9f2dd73c5c8e7 100644 --- a/client/sniffer/build.gradle +++ b/client/sniffer/build.gradle @@ -78,6 +78,16 @@ namingConventions { skipIntegTestInDisguise = true } +testingConventions { + naming.clear() + naming { + Tests { + baseClass 'org.elasticsearch.client.RestClientTestCase' + } + } +} + + dependencyLicenses { dependencies = project.configurations.runtime.fileCollection { it.group.startsWith('org.elasticsearch') == false diff --git a/client/sniffer/src/test/java/org/elasticsearch/client/sniff/documentation/SnifferDocumentation.java b/client/sniffer/src/test/java/org/elasticsearch/client/sniff/documentation/SnifferDocumentation.java index 70d7373dfc9eb..24d0d404988ec 100644 --- a/client/sniffer/src/test/java/org/elasticsearch/client/sniff/documentation/SnifferDocumentation.java +++ b/client/sniffer/src/test/java/org/elasticsearch/client/sniff/documentation/SnifferDocumentation.java @@ -52,7 +52,7 @@ public class SnifferDocumentation { @SuppressWarnings("unused") - public void testUsage() throws IOException { + public void usage() throws IOException { { //tag::sniffer-init RestClient restClient = RestClient.builder( diff --git a/client/transport/build.gradle b/client/transport/build.gradle index 269a37105fb19..7516e5eb89cce 100644 --- a/client/transport/build.gradle +++ b/client/transport/build.gradle @@ -52,3 +52,12 @@ namingConventions { //we don't have integration tests skipIntegTestInDisguise = true } + +testingConventions { + naming.clear() + naming { + Tests { + baseClass 'com.carrotsearch.randomizedtesting.RandomizedTest' + } + } +} diff --git a/distribution/tools/launchers/build.gradle b/distribution/tools/launchers/build.gradle index f933c04278e7b..4c7d171663a0f 100644 --- a/distribution/tools/launchers/build.gradle +++ b/distribution/tools/launchers/build.gradle @@ -38,6 +38,15 @@ namingConventions { skipIntegTestInDisguise = true } +testingConventions { + naming.clear() + naming { + Tests { + baseClass 'org.elasticsearch.tools.launchers.LaunchersTestCase' + } + } +} + javadoc.enabled = false loggerUsageCheck.enabled = false jarHell.enabled = false diff --git a/libs/secure-sm/build.gradle b/libs/secure-sm/build.gradle index 3baf3513b1206..97b6652fc12a1 100644 --- a/libs/secure-sm/build.gradle +++ b/libs/secure-sm/build.gradle @@ -66,3 +66,12 @@ jarHell.enabled = false namingConventions { testClass = 'junit.framework.TestCase' } + +testingConventions { + naming.clear() + naming { + Tests { + baseClass 'junit.framework.TestCase' + } + } +} diff --git a/plugins/examples/rest-handler/build.gradle b/plugins/examples/rest-handler/build.gradle index bae98c247d134..025e570bedea4 100644 --- a/plugins/examples/rest-handler/build.gradle +++ b/plugins/examples/rest-handler/build.gradle @@ -41,4 +41,10 @@ integTestCluster { } integTestRunner { systemProperty 'external.address', "${ -> exampleFixture.addressAndPort }" +} + +testingConventions.naming { + IT { + baseClass 'org.elasticsearch.test.ESTestCase' + } } \ No newline at end of file diff --git a/qa/smoke-test-client/build.gradle b/qa/smoke-test-client/build.gradle index a575a131d8759..637913083b48c 100644 --- a/qa/smoke-test-client/build.gradle +++ b/qa/smoke-test-client/build.gradle @@ -40,3 +40,13 @@ singleNodeIntegTestCluster { integTestCluster.dependsOn(singleNodeIntegTestRunner, 'singleNodeIntegTestCluster#stop') check.dependsOn(integTest) + + +testingConventions { + naming.clear() + naming { + IT { + baseClass 'org.elasticsearch.smoketest.ESSmokeClientTestCase' + } + } +} \ No newline at end of file diff --git a/qa/smoke-test-ingest-with-all-dependencies/build.gradle b/qa/smoke-test-ingest-with-all-dependencies/build.gradle index b2295f535ba38..9267f90cd7e0a 100644 --- a/qa/smoke-test-ingest-with-all-dependencies/build.gradle +++ b/qa/smoke-test-ingest-with-all-dependencies/build.gradle @@ -27,3 +27,11 @@ dependencies { testCompile project(path: ':modules:lang-painless', configuration: 'runtime') testCompile project(path: ':modules:reindex', configuration: 'runtime') } + +testingConventions { + naming { + IT { + baseClass 'org.elasticsearch.ingest.AbstractScriptTestCase' + } + } +} diff --git a/qa/wildfly/build.gradle b/qa/wildfly/build.gradle index 0ed4f8c067911..fce27e6ab8a36 100644 --- a/qa/wildfly/build.gradle +++ b/qa/wildfly/build.gradle @@ -217,3 +217,14 @@ dependencyLicenses.enabled = false dependenciesInfo.enabled = false thirdPartyAudit.enabled = false + + +testingConventions { + naming.clear() + // We only have one "special" integration test here to connect to wildfly + naming { + IT { + baseClass 'org.apache.lucene.util.LuceneTestCase' + } + } +} diff --git a/server/build.gradle b/server/build.gradle index 3c1ed1b2abfe1..a3197acde4ad8 100644 --- a/server/build.gradle +++ b/server/build.gradle @@ -161,6 +161,19 @@ forbiddenPatterns { exclude '**/*.st' } +testingConventions { + naming.clear() + naming { + Tests { + baseClass "org.apache.lucene.util.LuceneTestCase" + } + IT { + baseClass "org.elasticsearch.test.ESIntegTestCase" + baseClass "org.elasticsearch.test.ESSingleNodeTestCase" + } + } +} + task generateModulesList { List modules = project(':modules').subprojects.collect { it.name } modules.add('x-pack') diff --git a/x-pack/plugin/ccr/build.gradle b/x-pack/plugin/ccr/build.gradle index 97a25d1110d89..b8ed9f55932cc 100644 --- a/x-pack/plugin/ccr/build.gradle +++ b/x-pack/plugin/ccr/build.gradle @@ -54,3 +54,9 @@ dependencyLicenses { run { plugin xpackModule('core') } + +testingConventions.naming { + IT { + baseClass "org.elasticsearch.xpack.CcrIntegTestCase" + } +} diff --git a/x-pack/plugin/sql/build.gradle b/x-pack/plugin/sql/build.gradle index 3cde541d304d5..f5dc3175d4162 100644 --- a/x-pack/plugin/sql/build.gradle +++ b/x-pack/plugin/sql/build.gradle @@ -1,3 +1,5 @@ +import com.carrotsearch.gradle.junit4.RandomizedTestingTask + evaluationDependsOn(xpackModule('core')) apply plugin: 'elasticsearch.esplugin' @@ -18,6 +20,12 @@ archivesBaseName = 'x-pack-sql' // All integration tests live in qa modules integTest.enabled = false +task internalClusterTest(type: RandomizedTestingTask, + group: JavaBasePlugin.VERIFICATION_GROUP +) { + include '**/*IT.class' +} + dependencies { // "org.elasticsearch.plugin:x-pack-core:${version}" doesn't work with idea because the testArtifacts are also here compileOnly project(path: xpackModule('core'), configuration: 'default') diff --git a/x-pack/plugin/sql/qa/security/build.gradle b/x-pack/plugin/sql/qa/security/build.gradle index 9e6cc4eab2352..d239518df81fe 100644 --- a/x-pack/plugin/sql/qa/security/build.gradle +++ b/x-pack/plugin/sql/qa/security/build.gradle @@ -6,6 +6,9 @@ Project mainProject = project group = "${group}.x-pack.qa.sql.security" +// Tests are pushed down to subprojects and will be checked there. +testingConventions.enabled = false + subprojects { // Use resources from the parent project in subprojects sourceSets { diff --git a/x-pack/plugin/sql/src/test/java/org/elasticsearch/xpack/sql/action/SqlActionIT.java b/x-pack/plugin/sql/src/test/java/org/elasticsearch/xpack/sql/action/SqlActionIT.java index 09e106b17bf6c..c71d7c274947f 100644 --- a/x-pack/plugin/sql/src/test/java/org/elasticsearch/xpack/sql/action/SqlActionIT.java +++ b/x-pack/plugin/sql/src/test/java/org/elasticsearch/xpack/sql/action/SqlActionIT.java @@ -18,6 +18,7 @@ public class SqlActionIT extends AbstractSqlIntegTestCase { + @AwaitsFix(bugUrl = "https://github.com/elastic/elasticsearch/issues/37191") public void testSqlAction() throws Exception { assertAcked(client().admin().indices().prepareCreate("test").get()); client().prepareBulk() diff --git a/x-pack/plugin/sql/src/test/java/org/elasticsearch/xpack/sql/action/SqlClearCursorActionIT.java b/x-pack/plugin/sql/src/test/java/org/elasticsearch/xpack/sql/action/SqlClearCursorActionIT.java index de55d486555ad..952104b49eebb 100644 --- a/x-pack/plugin/sql/src/test/java/org/elasticsearch/xpack/sql/action/SqlClearCursorActionIT.java +++ b/x-pack/plugin/sql/src/test/java/org/elasticsearch/xpack/sql/action/SqlClearCursorActionIT.java @@ -5,6 +5,7 @@ */ package org.elasticsearch.xpack.sql.action; +import org.apache.lucene.util.LuceneTestCase.AwaitsFix; import org.elasticsearch.action.bulk.BulkRequestBuilder; import org.elasticsearch.action.index.IndexRequest; import org.elasticsearch.action.support.WriteRequest; @@ -16,6 +17,7 @@ import static org.hamcrest.Matchers.not; import static org.hamcrest.Matchers.notNullValue; +@AwaitsFix(bugUrl = "https://github.com/elastic/elasticsearch/issues/37191") public class SqlClearCursorActionIT extends AbstractSqlIntegTestCase { public void testSqlClearCursorAction() throws Exception { diff --git a/x-pack/plugin/sql/src/test/java/org/elasticsearch/xpack/sql/action/SqlDisabledIT.java b/x-pack/plugin/sql/src/test/java/org/elasticsearch/xpack/sql/action/SqlDisabledIT.java index c50f1095164df..0a56e804a00c6 100644 --- a/x-pack/plugin/sql/src/test/java/org/elasticsearch/xpack/sql/action/SqlDisabledIT.java +++ b/x-pack/plugin/sql/src/test/java/org/elasticsearch/xpack/sql/action/SqlDisabledIT.java @@ -29,6 +29,7 @@ protected Settings transportClientSettings() { .build(); } + @AwaitsFix(bugUrl = "https://github.com/elastic/elasticsearch/issues/37191") public void testSqlAction() throws Exception { Throwable throwable = expectThrows(Throwable.class, () -> new SqlQueryRequestBuilder(client(), SqlQueryAction.INSTANCE).query("SHOW tables").get()); diff --git a/x-pack/plugin/sql/src/test/java/org/elasticsearch/xpack/sql/action/SqlLicenseIT.java b/x-pack/plugin/sql/src/test/java/org/elasticsearch/xpack/sql/action/SqlLicenseIT.java index df0875690a2c2..62f7b42c69944 100644 --- a/x-pack/plugin/sql/src/test/java/org/elasticsearch/xpack/sql/action/SqlLicenseIT.java +++ b/x-pack/plugin/sql/src/test/java/org/elasticsearch/xpack/sql/action/SqlLicenseIT.java @@ -5,6 +5,7 @@ */ package org.elasticsearch.xpack.sql.action; +import org.apache.lucene.util.LuceneTestCase; import org.elasticsearch.ElasticsearchSecurityException; import org.elasticsearch.action.index.IndexRequest; import org.elasticsearch.action.support.WriteRequest; @@ -34,6 +35,7 @@ import static org.elasticsearch.license.XPackLicenseStateTests.randomTrialOrPlatinumMode; import static org.hamcrest.Matchers.equalTo; +@LuceneTestCase.AwaitsFix(bugUrl = "https://github.com/elastic/elasticsearch/issues/37191") public class SqlLicenseIT extends AbstractLicensesIntegrationTestCase { @Override protected boolean ignoreExternalCluster() { diff --git a/x-pack/plugin/sql/src/test/java/org/elasticsearch/xpack/sql/action/SqlTranslateActionIT.java b/x-pack/plugin/sql/src/test/java/org/elasticsearch/xpack/sql/action/SqlTranslateActionIT.java index e413a590dfaf1..d86245dcbfae7 100644 --- a/x-pack/plugin/sql/src/test/java/org/elasticsearch/xpack/sql/action/SqlTranslateActionIT.java +++ b/x-pack/plugin/sql/src/test/java/org/elasticsearch/xpack/sql/action/SqlTranslateActionIT.java @@ -17,6 +17,7 @@ public class SqlTranslateActionIT extends AbstractSqlIntegTestCase { + @AwaitsFix(bugUrl = "https://github.com/elastic/elasticsearch/issues/37191") public void testSqlTranslateAction() throws Exception { assertAcked(client().admin().indices().prepareCreate("test").get()); client().prepareBulk() diff --git a/x-pack/qa/full-cluster-restart/build.gradle b/x-pack/qa/full-cluster-restart/build.gradle index 5ac7c0befb71c..716289359faa2 100644 --- a/x-pack/qa/full-cluster-restart/build.gradle +++ b/x-pack/qa/full-cluster-restart/build.gradle @@ -88,6 +88,10 @@ licenseHeaders { forbiddenPatterns { exclude '**/system_key' } + +// tests are pushed down to subprojects +testingConventions.enabled = false + /** * Subdirectories of this project are test rolling upgrades with various * configuration options based on their name. diff --git a/x-pack/qa/rolling-upgrade/build.gradle b/x-pack/qa/rolling-upgrade/build.gradle index f59dd6bc3e451..0636f943c6d32 100644 --- a/x-pack/qa/rolling-upgrade/build.gradle +++ b/x-pack/qa/rolling-upgrade/build.gradle @@ -76,6 +76,9 @@ forbiddenPatterns { exclude '**/system_key' } +// Tests are pushed down to subprojects +testingConventions.enabled = false + /** * Subdirectories of this project are test rolling upgrades with various * configuration options based on their name. diff --git a/x-pack/qa/security-migrate-tests/build.gradle b/x-pack/qa/security-migrate-tests/build.gradle index abc3564ca13f2..88006a38bd5e7 100644 --- a/x-pack/qa/security-migrate-tests/build.gradle +++ b/x-pack/qa/security-migrate-tests/build.gradle @@ -31,3 +31,12 @@ integTestCluster { return tmpFile.exists() } } + +testingConventions { + naming.clear() + naming { + IT { + baseClass 'org.elasticsearch.xpack.security.MigrateToolTestCase' + } + } +} diff --git a/x-pack/qa/smoke-test-watcher/build.gradle b/x-pack/qa/smoke-test-watcher/build.gradle index 6a9b9415588a9..fb2e4c06ced88 100644 --- a/x-pack/qa/smoke-test-watcher/build.gradle +++ b/x-pack/qa/smoke-test-watcher/build.gradle @@ -16,3 +16,7 @@ integTestCluster { setting 'xpack.license.self_generated.type', 'trial' setting 'logger.org.elasticsearch.xpack.watcher', 'DEBUG' } + +integTestRunner { + include "**/*Tests.class" +} \ No newline at end of file diff --git a/x-pack/qa/smoke-test-watcher/src/test/java/org/elasticsearch/smoketest/WatcherTemplateIT.java b/x-pack/qa/smoke-test-watcher/src/test/java/org/elasticsearch/smoketest/WatcherTemplateTests.java similarity index 99% rename from x-pack/qa/smoke-test-watcher/src/test/java/org/elasticsearch/smoketest/WatcherTemplateIT.java rename to x-pack/qa/smoke-test-watcher/src/test/java/org/elasticsearch/smoketest/WatcherTemplateTests.java index e9c5106d44e87..df98e73118711 100644 --- a/x-pack/qa/smoke-test-watcher/src/test/java/org/elasticsearch/smoketest/WatcherTemplateIT.java +++ b/x-pack/qa/smoke-test-watcher/src/test/java/org/elasticsearch/smoketest/WatcherTemplateTests.java @@ -30,7 +30,7 @@ import static org.hamcrest.Matchers.equalTo; import static org.hamcrest.Matchers.notNullValue; -public class WatcherTemplateIT extends ESTestCase { +public class WatcherTemplateTests extends ESTestCase { private TextTemplateEngine textTemplateEngine; diff --git a/x-pack/qa/transport-client-tests/build.gradle b/x-pack/qa/transport-client-tests/build.gradle index 3ece6dd1147c4..5ca96eb0d7a87 100644 --- a/x-pack/qa/transport-client-tests/build.gradle +++ b/x-pack/qa/transport-client-tests/build.gradle @@ -10,3 +10,13 @@ integTestCluster { setting 'xpack.security.enabled', 'false' setting 'xpack.license.self_generated.type', 'trial' } + + +testingConventions { + naming.clear() + naming { + IT { + baseClass 'org.elasticsearch.xpack.ml.client.ESXPackSmokeClientTestCase' + } + } +} \ No newline at end of file diff --git a/x-pack/transport-client/build.gradle b/x-pack/transport-client/build.gradle index a96f4146fbf67..87a626be65d42 100644 --- a/x-pack/transport-client/build.gradle +++ b/x-pack/transport-client/build.gradle @@ -29,6 +29,15 @@ namingConventions { skipIntegTestInDisguise = true } +testingConventions { + naming.clear() + naming { + Tests { + baseClass 'com.carrotsearch.randomizedtesting.RandomizedTest' + } + } +} + publishing { publications { nebula(MavenPublication) { From 20c2c439e7eaaf7faa8e6e6ae4af1e1316037523 Mon Sep 17 00:00:00 2001 From: Alpar Torok Date: Tue, 8 Jan 2019 13:43:10 +0200 Subject: [PATCH 022/186] Upgrade to Gradle 5.1 (#36767) --- .../src/main/resources/minimumGradleVersion | 2 +- gradle/wrapper/gradle-wrapper.jar | Bin 55741 -> 55190 bytes gradle/wrapper/gradle-wrapper.properties | 4 ++-- 3 files changed, 3 insertions(+), 3 deletions(-) diff --git a/buildSrc/src/main/resources/minimumGradleVersion b/buildSrc/src/main/resources/minimumGradleVersion index 6e6366051638f..83364075e010b 100644 --- a/buildSrc/src/main/resources/minimumGradleVersion +++ b/buildSrc/src/main/resources/minimumGradleVersion @@ -1 +1 @@ -5.0 \ No newline at end of file +5.1 \ No newline at end of file diff --git a/gradle/wrapper/gradle-wrapper.jar b/gradle/wrapper/gradle-wrapper.jar index 457aad0d98108420a977756b7145c93c8910b076..87b738cbd051603d91cc39de6cb000dd98fe6b02 100644 GIT binary patch delta 19106 zcmYhiV|1WRv^1JbCbn(cwr$&-Sd(OuJQLeaCbn(cwr$(VeeXHzyZ8Iot9$kSwO8%x zs_M>qu;gkma3xu=pD=N~=y5oxs4yTPP*5NsAi^MawF3M?|E&b$gyd0a0x`)k&S!}+H#rV|80M?e0?nlx-o1DiJI!RW3n`) z2Xl{LZ$cb2h?YF@lpJ8b;U4?f!E|s4Kls+t81L*D4qC8#FqUpS-Ka$i+Vr-qvz>GS zw+mHypnQZTrl|#HvXxNoOkyQ(BRKxgW@D3ms1^{SUR7w~XgE}>bQs#KKB^rK z_G_fCoP55qQPV7y7jk6BcC4u^{BhVO3;J%C3%&@eRSipvlV>&a8VDVJG^$^pGf6S- zn-G^4Ib-OrGsOODg+BDC@?{&kJBqX;sKQ!oY9=!798k_)$dxAb_Ik-8YA+RbS(kS6 z5X!-=M9j-g~fi0dG;rCg0>Lg2jSBzWDlcr_PL`C zPuIJGM7i!bARg*)nidc{!)3thd37c^MuuOZTvc@al0$h1n`@G6)hS$PEm`wMaNSk zh|0hhk_YCqP0khwtqW1_G(3()tx^0u6Y_Ib%{in}no_#Lu+n)JtPgQ7tdu%8WOF#m za5!rJX03nSXf}9*`UM6|TO?0?b=SeCD}=cX^K-Jh4e!n40~qjqBtIP5mVdkcgT z`BH{%4_<@c!ucAq134aFO6&mI0K!MDA0yUF0#Wk5VgS_cw7&;|q1wEm%4BxNO9atY zOt;#kA@)n%+m{>C^sa;di>|jgJD@3U4LIx(`oBniB=`Xww_ZpYw~W3ez$5TCtQz9B z`lZ?CgZGOx)bci);!9TfOZW1DUHmoOubS59OOpM&JpEgBl2h$I*ruyA-GMsOK3$~# zOR$l7w&mCxQLtG6HLiTC_$6*GMcrtku6*lc;$)F6^P#(H zQ~F#E$BKniKC%3da@sh@jj5JQ8E$Ifl4*H^EbiDRip)Sv+{VrV+7jFg?ydPhWyc#c zxnEYA4v4F5)9aX*O)Z(JRuNwIbj-DiO7m%Q1$?}_@T<2CrZV+8P1Qj6v($I4)q~!# zvOP&`mCWHUVYlI(-TsibY4LUi!m^(@QZNGvsHg~smFEg=sCKAT{20I27+Rkm8upvq z&ZWbdxvpx9@!nE8iHGF}VN~D=gWMfZPo_PCuHF`ZGiqEyP`TKm%Mz5y&PKLl6>$? zV15JoRcKTP58~5(IFGutS(jocQ1@%Y8qVqD;4h!XAO_V?A1`KMDKAd{u^(wERO}=N zUvxna|H8ev9+M6uUq=|;i%vFLF*TK8FOsFEtv8#H_N*0C(69qebtrmr$MdHe(hv6D zIwqt+TTZhOV%}1RF@oO^o2f?oKMWmoPZW8`EJpAX z<+FdUl&+lgdDzq9Rt-v&^ngpkng;7}H_|d^krhR*P+LA%iH0fbgEn&}j#fB>s0i+Q ztcqVj4`;~DrTmI`(X-@Ju(n&}zSE7in7YSV&DDib>?8$_TcyPHXRj{Y5LfN_idr-< zoX}CIr(23cnR7_Tr_S=waah_Pf9BW8)+7J>tSVTR8;Ogq;`@hl9FPs6>1bL)a%<^v2&(6R$CuLm)gV_L900qbO5nqiFWOX= zOu};E&0Ya2Z3tmgZcA(k>UT)Yd;2|MtrztM$YHl*hEKYrr!(v`HH>oEI1dzL-SJlxoEy&=;-fL+_$7_)x07VIy^5B#jIu3(37X@ zH<_TyQh6s-k`E!zbe`!8xiN-iG>&Fa3U9W`)^!G?5OkfzqVM!{`aopc4pPa73)f>& zlOLq2$$ezjDG$j^QB)mwi;DRzCt`hN^H_+q3`%acM82T=Ug=3wF4PLFjLTJ$$JIU& zby&!X;XEVA`<>U~S)DaNWUo>{WeJpwYr;{d<6Tng4}=$2WTuf9an&cWaIem_MvOFe zG-m->Z59Spf7)V%hiBJEyfC<;RI9jrFfbqFD4aNzl~6K99{T|WrS(MCv3wGi7&v4` z>`_vezz7n_1~5`nzR-;oDV^H}iMKrjW>I?CK`rGtS7D=uB<+#`^39cg?pU498VH`b z+Lo#rAr1b+Olerbm>4Y-RgJ6^*yMqrS0$i&sY%7QqfxX!ksR8%6ymryM5sGwiasql z#JN?i!`;PL+z!JVB3?Vl**Tne1Zg=wu_pX804zi|b=kUAt1rc{tpkn$nk3w-_3=)v z!;LH%>leS(pC(S$OQgh8A0Hh0j3WpGjYb55j4L=~SKoSH0-ajwVyo%*`_`fkeF}i7 zbfhqqvsG-zv{Z`ZR9xls^pwmED;uQpqqbI<7^YA7(RR1cn`1LSm(L_!Ptrip0<+rpyoLt+e z?5gZ?+g^Cg6dB*UvvP@MGJl)Qy+iPCQJHcJYNQzJOwRo28uG~{O+U{+7#;Zj)Wf_d zk?W^isDS>glU85xbBVHvUCKbk`PqHQ=0^}ClbbQ!QPD%=Jsa$(tPOz1bqrj4jN@P! zvh~9sI_TL{B_C0QgVnnCW|3pIAXT3ICXl&z=eaJaG{53Lyk2N*XdvqOEwWhu`5hKfps9omOG^nC~A0l@Q`7TRTgh`UGLMq>#`k6=~3`t*1ckp|1Og{Y+Aqjrb@!hF^LDD zHZ(0J)5L#;|JHtJ(7Mk&IvLj5Gr9XA+T9#UIGyNZ*E$^PtTh~2?CY$pJ|*&qV5i3; z2{V%IQ^e`YfKHp_a}P|%_jt@@)^&nYB;KlFqdM!Y1U{bl_4xTQnQ%>ifrFrjn$j1= zQbi`=rY4$+ZzAv(+po~v66=lzjU3%a5n<9ZmP?XmB#Ba_edj zalE69x7lBCGpdPupiK>~ByiwuiyG)RuH6?*Xf<;DqE~|97er3$XxmMNQ=-82Ic!*I zU8m>UO0wq0BxwkhxhuUIlY7ErJ@M5!m3TvTY9{jDc<2gc3zrHtsvU9>b!zd}8*ORZ z87SW`(>0S5UH~u;NO7}Hol?vtc1xuM;mgvYq9g5v$@SFDXN0Tvh)>?fuDFYMe|fhS zf5L^jvm4R12*HW2m_(}g>oM5!jljQOtJT4X`R>5H+k$0H1wT{B%ne>LqUue!cEtrW zB#YMK)h}C^cZ=F|2E%##K%ZR4l@TMmm6YB#_wPCgZ(z)pbJ42*%4jj`tc;)C7+9Gw z+P}Ff3qNLd!*4c2@`Rxk43^K;Wo@xT$OsYDwQzbLFSpUq(64=7s3=?7K2u+$t#Kog_4q8I4 zwx@r#rYyYAM1rk^Axyc}@rN?tNf1WQ>`F2Z%wk2?zD2WUwxwLm_;!RlBb{Jp0r5IgGFFND|1@QkHVqp!+Gyez*hOr%6$=H zuWS_Xn?KXj;HiGL#FfwOKjhI=%e$;FZ>07r7`s4zyWd_bd|e)GXK8P^H7YfOcJ7EE zBiQy$&lTR22uMvK?)Gydcc%s{e@4mI11szEcR1dR!D|^*v;jVf{d3aZoCz)5uG4r= zh|_ypGd|dOR%B?uZ!&}_%UuX!>Q+(<2yx7X5;A6H+3JxEwPByK#N?Fqv*P>SE6=;fpV0xdN!L`ze0TP^Hq3))8q(xW8snx-qM@_l4v0p z#(L+HjI$H>?Vl%&t+-reKv#4Pioz)(cE|$O6lDnyInVPYydWA@&BCjVBtVN>mPb>0OyY%obmv`~itUUp9OSigKpi}Y<052rTFvYL#f!=2tOx)hleO^O*) z25&*ukrcR#q*ZFjLfDLT6aOTqBW#?#K7*mh0A7j1^p2b?hhY^<`;-AaU^YSHqtcZE z>K)(K9b^80W_*>FC%39R;~O$%ZeUz;>*kZE`!ikRK>?+4>di>e)VGREZ)Hn~uX72MN11PDkV90&+&g0KQcLI*ZGphLr3n*<>fZYzSf ztZ*p%Zzx(fCDxTba|O5Beb5GdctbT^M3<_$X-3J&BNq{l^-`{!%p>@rbb_fL7Y+c8 z@!1W+#$xKOr{(S#cU>3%AtC+1GBIu<=lkL^E7) zRFK4*F6mEMxatT7U?)8c?BvD&smcn|i#JNaV`eZ{H^K7o7@aP-OZS_w+Tu$OW{F8l z5=6H-xh38nT6oA97GnGUP|PheMN4s`+m(SkE501K{h54=4sW=4!|wNVJDOVyOZs}d z3#)1=DRDOq;~H(q!kD4G?s`emI*MxGY1nx~+_NE&mb|7dilU zPt;$Z;_Z51$Y>%Nf}|V1i-8_sdG&sd6mlJ*SE;eva6Txtu~zV!Kl(?X#NTYR##&@q z>XM1|h)HKN8r{C-?Y_kt(KNx}Y0*m8d2y5EuV1N^au{kp-YX)kL4#UkIwczuJv8~3 zh$$6%zhRG<$H#}qD}MAf>*OJR|ZNyAbv!v6jhBlu%-%%G$ozt(5&?j)^T z&*oXK_$0aSMYJho$u@J_JH0-KKT^W z7oK=r4H5_2R)=}Jsq@pkDf7E7J$&~_>S;B-ms?b>?9?Fh>ld|OWGKR4R>b79eq+(+ zP@kJ0gut4|)MuY3idTJLMa9Z2s&`-z)7~^{RsCYA$%jVu{>}r**ffZ z;5(>eQxPoX);d!){Js}Q*8m=L-4yc}3JIr@@s%&0b z{hFP0`WfLqP&Bn?c!jGYntWJ)fIn@ZI!C>?H>#4huOmdl8^|$wT&a}AYh^+sd-IF{ zZmAVda?eNBd2xq{+Ixpna?kDoyj3&1Xklp+OXrm+ZZ)O1`|ZTiJ3USOEyBp1Q7IT< zp*6EvuolJYx1MrZNp7a)Q3!}r>U!;GbBJ57fBN(bzQtbLbs-S-oKcqqk`&APBz80xr!0^;G$&OuK#qC z=+hRMH9QGj37AhP3xu{D0n)z|1+l`MKfOG{iTo7Ug<7uFSOcf2uHRx0y;6&{@7PK) zeWu|~P0# zE&*AF^soPe$4&?~L1`{xO4!X-m(W%ZIT9J%bDVv)Syf?z(v{GM*7yy-cnrZI-OrbDm*f_{(A7%a0Hs^XVnP0h~Y=%2rZ?&UAG z!@vr+E7?`~a(0t$Z-1)uS z{hWo!?|g{!w^F4rl9GBQf|6FrF$OD(B)E9YBi$cGn4x#xoKJKJ=rrJ)kFbR9t4HU9 zoPON%H29Zt>kS_9bDyL`h^|YiH^MPH5*8+z=Kzz|1KNPb$dDp9C1MvHhGEYayZT{I z-JnCqFK~Ipqtc(ZE zSn3)Ex3Pri)RdYja$z(*$xy*8;K%J)GY3;wl3+!j>W7^`Z(htRh`Ht#AkUFVva!sHxKG3vy)~UI$y!aKqH0#D<)|S!(SlGzG*wz6?P)_HF54yF?dLV&}7R2jv z+b!aOGjwyae4y(v9(K05W5e#QH>Z>q{s_F(mE3@a6B)OR=5g)vVTg;ePOdF>lHU&w zhd^y=*Ksp3xn`WVKRm4z%2Q={CfBX0h4N+>$m&bKtaIlf8<}}?&Ak=+mU=dbX=nDE zJnhl}dI6d0I404ygmzijaLan65v+Eh2Auuc&|LWPlObYsN#ioE0A}>x#*sdzRf(Fn zp5A^_lhs@lBi?1{z7)R-``o|5ybx`1bU`jSF-m!^Hba=pF`T zpP?MraE{?l3d+I-U!L^T)Ff-CT+f%xP__C8M7R1aFY?CresH+p4#cl-g`=QW4H0c% zDp_z((PTT@{6l=u@Rk!B(7Z+bP#x-4`@lsUe~Af~yfHkQr#lo@oIDUyB(Vs}_p*q4 zSultX#K=leGf2Xa5=VJzvZ_rrUW7YmV;MtQ3MA3t6OaHF|FKP^Hz?Yy4E$0^muR3E zxIYTSSpH?RJot#niI2psJ5VoFkv8_1Hm@rvn-dYH!`!t{ zy%)5Jf)0ggs>c4HlU=kG3?;w}!`lC>oo8Oscy?tp#4X8~230XB5dPgp;8HK-s@Pnp z9p1zXZ@^JDm@yeArWT{^QkvY0Zn-{sBh_bCRY_~59W)f$rEi;^b>BGj=rIngSuFFs zUxnJ6H$NTDS9ad?B+MVz2KwAv)oD}NLdm;%xhLmCQ!I|; z+D;#)I%mhsPUi-Eq}U&8Ln)#GR)sHpKTq*x_u>4Shx>4)*iBRH2lq<0j~95+e+Jh3T90nDRsQnS%XA#ZG!0Ej78n6XYbmX0Jd?aD ziR%*qwnoXuoR548PU3o5mla*g^H`Y6LhHOF-krJ1n_VX7mMA)b2s-hT*LaHZZjz%t zb1^n>8}^vjh^-e+5gRBeo$mCjVP;Vn1hC+jF*f&FsRAT=prw}toNxas*vPDEr_SL5LnOK^~{SF`v9#x_?=w0<>L{?gO;% zVW!&aj*@QfGM-JML!_;Dst30B-q0}gnzX&Rl2-;6PYz3X;7&vZ0a>S?U$*>-!A|2_=nJzL@HIy*h38Z5!%n@y7sMXm>j2YSP~#4B4%HwKwuW7g*fZ?J->D}n z3PvT#pKJ`KVGNKqnWFNXRQIp=4;#})C*+wzQq-V68*yHVqfYH>=o$U1GS0qiV;<%| zAZ#$@92_1&+SCpwyE<6@A=5HpIf@icbu7FQWqQC9i8agK_Q7>biDbg%dWCmHrMh(4 zu?o)|eOw!b`Kp|!pJQ+Q&xA{1-I+}6WirE;XN=f{eg|-%$L9{}G3u;l2zrzL?~yTI z0Iywu9g?WZoCB}EqS)N9EW26MVV)%Pif_l#<38RHHQ^P!RTjF@Tu;YLyHc8LC z$xtYi6+85ZZJR}RmyF}qp}xAenLKQ`poX2WiWb&h8zS1Fn{5NVP2ptoR5gAjdEkl6 zo$PjnmjLML+d*@Ox0f=0b_=(9U?;CV_NBuV+pf$iU*XkK+vW{hS84h5(n&c&d(y8H zS7$PHpOY4evk3dN`!_f1@QRqXpt)l!w{9n=) z%W(Q%#=?s+5Ay#CZK##d0rY4fAg&3FoP1~@_|Tz&WKwL|u zGFyFN8fzrNonFv!t4CKKtl5wV=j4HE0T35Kd(f)2qEBL4+(sRf()PmU$uNE-V{mmu zQ{}#2DY1)guM^tIm+ zR<)&HUbrf$NR`5!xI!i}SIzL;oud1@sYcFztZEI{S$Kj~1(rb^WF23Tk%g`WL=&L-8l3+u! z*PmePph1bo)>+Z22H$UsU^I4yAg!$QO0+i7CtK6m!dd(f?(85fYitJ2XYY7ikZAS4 zq$IhpW!7!6(YwOF4Jc(7n=lvH?U23fSw@zV)O#STM!iAKoA6_4{ zDhyzMQ8*l?Q7z!PfsSye{OkiYrJipEHzZk3k#)9hf?Z}^Ehq8Kv^LWxs^AK~x8Yl)|{x$p8o012(l+Xo9&Pa5!td(biy4vh2slxVhBeGzkAEWApIEM~s`l++8 zJP9S%7XeG34@q*Ehsqqp=<~1irbSwqM6f9aB7Og^lL5yJ$2bo0T;jEjy)iXDL!hu$ zK7$nyHsu>&B(OMMWg|#^O#CZt2=PgTFP(Jf9V9~>P8=^9E|gC@-`X6&ueMu@TTdALCcVpt;p8a z#;J$R7fN${Qr!eNkr2JNt6W}_|3~dz+PAH7B0yBB&Pmh%_2|VmGnZ5~z#a$H!~QFF zNA!@uzqG?OU-O*VkZs0(b?ypLjQE)97x+-Xp&O_w=sv`oh~}R;!j6C=2JDvgxj`7i z;KOygC)6zM#iIA7p#l4Q*kBfKIlr9-nUOlAhw|6R77R8(ITk-+AYWi};Ek!XUjG@p zZ5z)CoW4bv$HAiHLALznkZ2Lh3jUV_Um4=!?GC!AS#Y6goANv~7j|CJq8A%l=-|&j z$GqMJB8B9hU*2WgB0*P%`M-Q=afaXkGW!A|EBj6o6nN`kV}%+>D8qy6qYZh*-PmCOcGXY#{ zC}n&ZVl|Ivhu6YXq8hRU-78`g8PNez=%W-{pJEfhL*Ta+YgfTwZj35C43%bQArXc- zk{SfLjUkIuxBf&q4pbA0_c6e!O?TugDE`qnX*PbfOs-3{p5VvmaB@Wv0%Mqa^BcVj zg?J0o`dS87Le3l0x}UzXA-%3HDw<~*mmS1f_bsdE&!hNb?=*p!6!~4K{Z>8y|weqXMS1H?bX-7u2`m5ywgtIX4 zJ;r`pgA8?AnmXBV=&Fj~;Kjd_nMRm#`Y8F3%8AF^K_v)JkMv}-%ODp)k}E-$p5*xp zWp(kFSO-1Sfk7u~7}v^SW@h@bdaz0!@U1?1oU417egBzP45_x&|IB2^LxURDX}UtP z2k*mhh%_EM!JaT zq%_N~7yZd_sH1#vFzu%+AoNXTxc4Qb)a`Jn9BnQ^nqDq>Hb_f+qbLhXB^3IBNF+Q+jmfLz}}u&l*_a z?ojXG@(J-5i_o_*Q=gR?TzVNHf^2%F26 zpm9A+p2mUKGM0VmQ8Czf)}1iZ6lDYOqn}V<4Ehzeh@c_DLaP2Xh$vH2j$4d;Lx&Og zEBCNn91M&V;J!$HJlf=xpH_+ori#yf>AfmVC)(kM{hk$r0>59$VnbP;UC< zh_5|`#6dJG#tK6;j4oUpzd3cWufU$SEmG2fyR%3$!cp?R2FEZ-Z%b|jEu377ehJG? zCdWb518&gbPSLj>AB8CA!YqTKaBKJ+FIbT?c9ad zRP;rJ13EOWOe^=RXhl4GpKf5r)GGSkr@PVHeVSDhh z)yYRJO6GaVD}ta#B;;sW?wL45*J83r^l^u@THMTY-G<1J58BCH*u7Tfa;8(kef z;5TUQoSP`s>~nXPgw#cNjcz8g+OqMli=GMK7^ro`B<98Y9MI8hN|HXM@_J)l{&ddE z1cH+caAYY1LQdI&?ru2(ss^Ou#1pFWu94Fm`cN|*`qYzODZE!_-hJLcy#g~n4=BK= z3kNsZ>If!Fg7ZSGei}$*kY}e`Ies)A?843*4~-OZ?>{FLccoCUp+CHQkm(k>rK9$!6`xzw9f!O-W7=LXRn;u(kMXFU=`m`ZrNE!)ru07g)h) zan>g|&gHdQvO2rxa?y8T+)$)cH~Ul9Zk_(gfperRF?m*$!0CZ@n%-p`67l=7b4R(2 z+Hk>yb+OK0B7gH|({y261NO|y4^$hrq;zYF*BmOrOp!_@$-fH4_g3(BgONmCBCZ1B zC9po2$_r}7us;(Qn?P(g0A z{6DpQ2spxT)SHlv41-new?_jm)Sr>Q2&|ikJcS1*{cOb|>*!;uC!j%3H6nk)ioAvM zZ}Vo<1VmWT%04245o<;R9+ZBVcK=`&wiHE{+qI(N7h=V-z~7c-&2Wafhyj$06DkXw z1bd0Y2?wp9`$j9ncISHgo50kP#i1uw=b4uMnov#LL5~d~_ZV_$JgmAM*a{NojZDHO zn|+i_Ff^(P?9cq1pYk=?(JA>MXGm>EiYiFi^m7ALAwpMV5h=yIp?T0KY-F;F%JSj~ zXsT;c!nY^d;*Nf()N>w8Cp&R!QJ+Cip_lc-`-&DX`?e^hFy zS!%tZIW~m*jTUM{sR#xVv2V6pJ+gjmYeRIG250M8dnKUy(D45##nY+{Mc9=usFs#C z6pcxrmE9UhsGDdWmWyfMj&%5NK$1(jck&`3J}&nN2h)6-6X6=6Mgf>HE(=;c)YuXU zbc~wm9S`~juZSG1t4tMw=Jzj(26?6FZ@Ca>;_23E`az#P>I=5LROS46L3ePCJ2Of^ z^d|eBB5r(fgt8ipK=n=(=I?8LWYWh1>Ge|?F(zcRXU@3)62#e%tiH_@{PkAKrNov= z{hsT)nM z$Ag$G_2)KTJR2e?tKB<-8IZ?%c*tzXOVQwXRIy9&&Yl&ud{W93OZB3v$rV9{ue2$1 z`S1Wzlir7ufoD_He;Upt{fvIww=s)9Tuv`YL$uN_eadll2O0 z#9bA9uh&^D5MJp3V=Z8$0aN*qxZ)bT(B7&X((wYk@e^tT(kMEBX}v$^3Cht0>gqey zdcYmT84T}U`wbv>Z3iDc=VoPVm^I^pdugp%$MMQC_*+YRBIh;xP)5u2Z2rdnRu!Uc zSfnY(MW_I}!^A`lf)$VnnD%jFetd9%ALBT;zSUIIT6=y}t68OqR8u#om=w*BDqAA_wK zjBu<=i3LKkin(X9YPb;B4^5W{PZg0&e)g3eVB;=L_yU}{M)IpkLKZXWWj%9mo<;8z}SMUKqWhgvghZ(ASNOgvTHZ>Ua|P#OD>Yq&7DM3)g^Xhj0ad(A*zSKMq#)75fIa9zesP&Yq_V63{4l zgi4}CyU1THDA?7Ln5q1WZT4ScR$^7#r>2dR9h|W8NQsc{7^?wAmyA`MbmfKelQxdp zULw1P9caKe*A#W49;++@f6A~DlP=Y^bpx)>-l2LfL`v3{vTrN&zP7Et5pTa#)&6`- z3`QKtW)}-V=TXZam|pQ0QQVyZIn1Mns*`y`5F{rDPUI{KL( z@N*au^t-H;;c6;1$q-?8s`3%Mxbb*^KtBHG>pS~9f`*J=vlfBr(X>aieEdLSV z?}D}3uPX|8vVulL?&l2`-{TDz;CL(J^L<*ozg7if~i+->J?G{9hPU?mCQENbjpJ6W?ZQMDo#lPIQ z1GndohhaEkK?}t!;gTs@x<9(#(TlS;F^`X(y{~&T2@i6+$o@jQyC`fiJ1cfHZ<8?v z=@0n*dBgbLk;abRk!+lp!rE_-^j!Swtwk2E}R_9eeC*bN!Of zRVTh=L4M?X<%Fq0$QY^t*UCshGV|(2OFsD$_U@5#tWr(c-!V78ww5?Hc1lZD;NetH z@ZQU>C8-|O%da4s($!ljxXc5z7(D%HDQ%&Sx~dQ$2}|smwLyb!;Ln|8$;>r@Kc|^v zxs9Xw#IV{f_Y$z=84B*2Ue#zVl;tMvEL;Dfue{!LF*TWJ?_NZ2m((@Y8SHLgNuN){ zVI*T*I1q0YIYykVBN9}pR9;)wYDhmq>&r0GbDE`N!Nz*hiHbG3Dh zl#~!wI=ZKpfB7qe3b{*cZEQ?a;%b%5oyhfu>=BO%ofo;Z2b1w+{>FEhCe@=r1kz9J zL8H{D42ItRv(cMzuZ^_1`#X9Dje37=@FfaB@vJnz87^RvkhWgM{-l^Sn8zP0r{fon zN@BmOyIpO{yoleu>1t5hkFzV2Ketf%vC@de=)=(*Pe0TgXrc~%OJ;s+`4w`aVou<$ zwd0Nc!b;wA4|QhM5IHKu)2S;xl?Vn%$OfYcGP>@l##14U+rv?jv zJmjR9;=j!(q6G{Z-k?B#G)1Hz{<)ERJlg$4?&@l9znN9y?SwT?OZ#N~1oK?D#as#I z*-Mg{`2*l)vbHOw{GJth$UW-!WMdImC71+zc0BL}2qb^ckT9{<1Zep2w#XIA8x7z- z9643A^2Z!s<#e;1j_kjLGM?l=q3Qr{{sPNx{#*F}9*nHtXBD>fJw_xsA+%qJrrJ!A z`mu!hn@KqZWAYP+cZZ?zhNRhAZQYsA*giMOhz3gRJ0_e`>ycGZiMeI5{>o3&*Lh{6 z0X^IwA`8`6&SC!!G{>kLkl06XNKS~Z=o;5A#<;z_BYlGm`59{w;>SjaqkZK*0A=>MS|-haLd)&C$K-86c_njta(hdlH* z))ww36m7vu>fVxQqira0CAX~QyU-GKG3pLl`s|!*0j*tn?3L?GS}KMu4reavn(KAv zyXC?Bljjz1K>`=mr|`8`*7N(tF@NXXHKrj%4fuEbZD^#VsE3>|`WDT;9gII$u~6Uo z>wy#YHPmh<%U0nI+*V}{h9VV^8sfsTm2X5mwTN>3$J2ul@%lJ|aA#M#$d@S)@%95P z+dEZLG2{kr`qZ$spe$+h~ya z6JfZ>Ez`I+>)`nrjZ1-iuU-z8%dkD}((arAx1N#MFIYTDv zsOA`59iIH=?79ZL&0m5S6PfiQO}o zRKeRMjQkW><NbHYy*ujD-5(S*aGBW+ zjSFE0^v7G}tEOh~f2so?(RJS1?fZ$LQS{57$;6jW6SG|Sk!mR6oV~|0(^}XpPq`=O zo^d^8DrSEKz-b2uiqFDF|E;iZMuc~D5xTSJl}Qp&&!Z9tzhgNX5tCh>`!R=C$KpzC zYI7P$NiQOijG_%JQ$)DO&Tg*i^}0v@9)AgrOfl<_yCPh+tM)3u%}fM05aE0KHVdnO zMU17E%$IavU=S>@XK0ceN?9^a=Qu4s^kQ$4t+(Aoae}d0;?mYVCJjhr!jpoQZ$_mu z9xXxqq95ziM2Cs(cc1x3mT?^SzG0+fR}ryQu>m3Iquvad6u+@atnRv8E5yycrD}|9 zoapghL!E{)s*TJM(kbN@mr_MbL1mdqVQW%XQ`O3kU$NhLn$WM7lZj*SHntSxk#N*O zK~H|SyN|#))XIcv-0-e*{BEyZxKe}L%OG2;ZcBTjCkR&a#@)^x(y#N(iEWIM?$okD z(b3N&A-@4eu&7bByE(cA;yJ7{*e^qU)5|PLljNbxfkfV<@aw%lbeA~n*}N45=+*t_ z>UcfB)bYfcVWQml{qt$6_TS0KE~yt0=&Q?LKn;}RhNaO3&7{ropeM7GZb^?`m{HF= z2Mz)y7?taoO7|!chOE&A%{d;4NmB}_PND;|nI9AOO^JWtt|ma35r75xkS0)?u>)ga z9svO4Z~A*#xFrtb#5H)<7PeL89=od4fBp$-Zs!TR+;%_vm;d{{Y? z$0sRoBZ3`46|qpPFL<0=CfLd$chd{limh#&F89iFytqkpTP8n@t@pKIYy&ajFru~T zU&IsnfJmcNkHLmy`|?Z`OomnWbiJX(JHoUt_$4p@yL!X+^(yCl*Lb~Q1XJ@bosd@3 zh%_&*$$M#cM6a;=828MQT=W^H|EG*A0c+~Y!T~~9lzn}&n59&PhCo3ow1^<$KnVyG zkWCap1F{GNM^G@P1);JOxE2%zmCX()Tfn`dgJCm}gq<>CQK*87g`qf88JT+@OiJp@ z_a*QB=Rf~B_q^nN@8)~=C`2Wz`n5I2|3;T(E2;|159-c0pL}^m92SyE%&lQ79aa~$ z=$p#sIqNnLjjZ1;+|T)s;i1g7FMl=>7}VvN)H{~)mul&Be?~=6XMm%%IxWYxMD$C+ z;7mz@_FW$>kA;1oZ?Aecz5S$TbvQ4;@%}ocWbpxdUQ6gsTiw>2+qO|`<6Z5+_50jj zX?h>@-u6w^ek`=Te|?d#JdBjvJF5Krs@sKYNxlAdtfK{)RO^7(9K+AP$c!e2XS1&l z*#&Rv9op*{elfm{)S;5_H|H_>{@oUZ?=|c*Sn5ZNL|m86an$G`(K5d$VEghpgBNNC zh-0SJW8WsK@zWG$J3N{D^^Ey(w_A(V7k{>&<*(CTPLB^ex#*stn3>^zWxCyx?ElKu z?W(PFO}cw=`OG$=j|b-g@HXjh^)IPN>aMrXH4MtOtc{GLJm(+pthhF8ci5E0xEx1{ zO=>DgWRYI?&-u8vQLReTZ*E*t5;Nzu@^-W*{-*qtVSjQ#z-zSpX} z=w!FBLcd*XT93zM$PkU16g+*b-A1$K1U8*>zI8RuKdwEg&Gc3f@#zuk|Xl|6d!bn2Py#P4R#Bze2VYk3Qg z@5si^k-BrX>SeonI(p`RXC1)2CCCac4d#caXvO-tD$&+`6~WZEvv)nJ<1^fD4+(g3L@3ti!mL+Dz}G z$yuohr|7oXQ^{uwr!~sA{W2Hs?p<6|&M>cI+%2IJ7sF}eOqNVRgmTSZmw@(j`^*zc z&5v0-MG0<{?4aG*@`CbN(*$AoeeU(piQqSoX*l8IDaSMr;k>U*gz*OAQ_;N7J8>`iXk@#Ppy6%G419kt#(3|J<;p?) z+Y|H1FO?ev{dmI}4d^HK zwtkeRh~I}~4Ms6=x5GwWMpj!6bs}+r_id}2I^new+VybpbuxGJ+-c$EjPw;j*$Fn`Sn0uGL?BS4obD*$PItStP!i&9l*Da) z924jpe;_2am`O^g23rxr$O@~`LBS)W)<^~#0s|m^gncac$Eql)2gjv^jsSIm6-p+b zfq^{&RH+3z)Yb(Y0iNU;0g@dzhuB9SBq(_@>JtKiETzk}fewbiYYRz|Ajchpi?UX> zO71H+d;~E)8Ynua2Squ{#IcRNDxkdweL27sj#qlF;`*D`3`*)xlm83D&r0rlKb-&L zZX=N2E01)_LNL(Ki$F#AVsnve2NJ;ou|hM5qaHnV zLS+*A61K6xJ)tflbD}WtMu;NMvj8Oq1NBiGBqgyCp<$;Tf`+XUunY$ohry*2hTw_F z5UF@4t_}Gx9=AD&wUMonje%IPqFg2SBR~-w0lN{rq(6v}_Mv1*rGa0?>k;)mj`^iP z%oMoy;iF;oA>L9d#B3xvq;l8Bd&9v+)ceV6>)&iv8{`3h8*kumW2aQPzD&pY$NEuO zKy^O~@a`0j&Gu6erZIr9pgbH493Ug?IF1DtfJ*~A(DBRfF|a&%yv|0TN3a zLgM}x@#KsyYk(&Mc+kf+nB7!iWVV07lfkBFL1PkeDnkyR8%w&5GW`R5D*~{-p0FmzW-*b34(G+#$yA`nL$nW z&Y&Rw-)>US|2eXT{y$?9I4C|SS|bUl90Vv@k|X3k!vDQ!d8R`!$?&%$AX;|rw;yt0 zq(C2`S(ge7#~j*Lcut1T?4Os-_6&pG-XCyB!gzdH!{xweEnFAsDW0~-P&5n`O_mKE zgjA;uA+VlUbX*Utk-Si+vOHF@&rb}I20ELdJ!A5i;fv?%I7Nq7c~RtAl1kq7eHykC zf2ML=qiAPb>1(%@^$ucV0KR&AYE-`;_NZ_Q>9#fW02;0GQgFw9vm`kFaF$#_5M~3a zMB+;73`{Z>@Lbpdshd>IjtFxi-DrMRWBUP-HzE`vELfW3nuYF_6t*^1c4b*V!RQ)~ zD7cY&qk?8xWYOoHOO1(Nx0qQ9=fu-E-PUr8j!mxDK}UG4?W8aO5WsNfu%A8&l0>(z z>1Ne|D4P(@LBNdmeZzv?Ksh2$ZZ6I-{r3!+lwc)z+FmAFFASO$lN`Fd6K>Mq45UyE zYenX*kvNP}B;ZGKIy)`_ntl?#W;uD^l;rHBbrP5dMdCD>pw)E{`LrR>zJ}k)-d)oUQ1=|+)gOj^^h2j^)SNs zfmH6$<_K~!8JR7?W|(IxF#pCXb%`i>5?2hJKB8*yPF=0OgNb`h>jQrJ!P0q=em!sh zc+1+hKx?OT;iVfEYy+7p^#%Graq|C16Y9Af40|yG2#5nd2ncb~Us8-Dz;7r3gNdz? zi%X6cv?1Dx*SEh_GplxNh~~`Ey>T$TD^5&51_uX9njlnx7zymiGf`}D=u#Xn4=k13 zJ{Ns;R!Ec80U1SH&X9ceHWvpiUgD~o_9n2}htn>Jz$z=(-n&FDi%Y5Z<$12ANht@+ z`yuY9`{#T6XDF@bWsMPJl|%(VT-8rY{J0n0eEAh+?XSkZqV`as{l$*jOE8djiwo(` z8iM3c8?th{LChPsqcnX(lewcZeG^`X4KUnAg#T26lDR!X8hAZG;*Szgngmp2)~3H& zLlO4pl^@T-e}=w)n?Wh;+7N%De`~V?RAlT8IPY4CKk>h}`vlToN09&=yLs06-)vwM z`iOgZcd^``u;1(<@Cke##Hu7?*ry*ng@sf0U+M)gJJdx(ZC+ zYa(w#nF6p?45P4=(Ff#x=aa4d$pul$mAZ+h$;6@fj5C#S2OdyZN@kE`#tzH-w+F=WjF3BRy*=nT%*IeuTNR|F#9**PN z{1YRu5x9_^?Zr}2!Cc_w^Jm^KXZa=fDl8OjN_jf zaw0*3tvtfxm`?x-WqAYh{p17uW(D$QhU>ZNy&YyApfw{bCAsh9a z39f9v2MMs13YmcuQrO(2^6Lu4_gy#>FDxL>*@W&COC$hJP7KfIV&74jH7S_1yF?fH ztD}}M+4Vd3*gtPfB?k&-;u2~@syU~Hyya6v0Y#>@rPocVI~T<>K zghc3Pu;%VV?Vy)|>~hshCTDvhOHa`D3qFQFUvi9X}fOtre`@ z=l`0EXu|=l*o&wCx{GjAIjj@7L3W22ZsF-TWZEd;kQt3tAIfEUtep+r>H_R5PHWzz zsYXM?qD?w|tv(zEWu@KcOqu7nL+D_!Wahd0z7 z>L37^Nf-n{7R1#!gxIh*eS>T}Z8r)n3N&?!iMS2Ji5s_8LLXqaO-A47BK!?Ye>dpY zC^qQZMXsBQRopc_(QX=U(fG~B^_)3DEB;dIaYyDPEZ7?3q#BE#C2|8wt*`43_1Dby ztawfKX!MJrlXvabk5gTxTFE=j5E&~W(CPrDwU>>iy^mPVeTl&fHsi$PS0)P*d1Pq1 zo*7a**vyA@;n8=L5omESxs8kSiLqDGn)WJI0)|quaX2iN?e5_=`LYY1HR%-x( zJ-I!8^ZjhU=sSK+=fBIuV=e#Kif%Q=VK%EIb6mkaR7A{>i2U;?f-sG9R&mYaFmw&U zOi1HOQDVEA|1K7pQruX8Hdkd+F~zd6K%;T5N%OQJC{~}E4y=(SDQ9I%M^(@ot-qz? zw;L~8aF!jRmm0GQido-~ZyA4PfL8&eW+QK-6lmQN&&`gz$WV}_#Fs2|<3{n|IPbdr zn}$w4QGpQ(YfF`#5NW#Go$t2u(A<~mh^eV_lY-Di5nZ}`dwxJ2e^!b7C;4H&(fU;` z6`n79aj{`}M}~OoQs{QK{s@f7Zw!aOU67XLBDct;8_H~OhZrp`Izjn<1&aeH6cjon8a7q9-H!PjeW*x4BBbTe<&GvnK>r*9Yh*wLJ6JIl|Y%Fvj4&R!eS zH@XvpZ|IJ-^DZM*({A0UcHaY1w4;8PqueiuUem+I6&>XIFlabdGfY3Ps!p`drnPx# z4ck=mJ_9u|6Ls(x12ko)KdJoy^koeUzs#R99z>x^@Byu?))t8|K}+NsM%X$0Mf&y4 zeDcS#E1GSm=F#1|Lm3GnY0Km%P810@);?&|78jkvau3l{k}5~mGSvWf#VyrxQuXOS zja8noDwUuo1#q;BS|VG@1taCU3Y?0IEI&0=O3h9GCYMCEj?k&xPFQJo&+eOmXQF91 z9I2tAJS@)lTKu zWfP97OR_pg?HGZ!SvUaqiKr62OzmOr600}R?LWyijvu$N`xXnP2{hj}_+0s> zHGfXXeIfIOHl82LGOT;l{s?5dE>@TpDi{wQ#0_8ve|_1n&py3F2fXE0>zXxqkR4FL zsY^AlG>%fZB6Ge0jm^GgXMY^B7KkOY*2f?5G>&9UQsnF$D#QV^28|WwPtlk;3o1oX zEN8%W^!qs7p|pN^Q+}qb23;@7xG7k znM~7HFKPnVDow68>3Lr`o?FjdirVSLaxnJ7??^LONOb$+oloNH33p{{A&-`$Q_ZGJA?zne8lYbFSR7q4gy+ z{j^g<8R%*O$n-)@VpHd4&00or9gtgZafS=^Xv`keo~ zP3HkJvuF7}@U@GkwFkKGSuSS^%vAwAZQD{&otEF7*hD)LWeDoEAs`DzIws!dMWn`pRXW|ra9pm3`6_E zQ{K64Dp2pgVw`OELGPIeT08b?)-`|ZNJs&2Zdv^CZh#`KyU<<$Wy4p*`roSTb%TAW{`9n0b^0saF;rz;i=e6HW42;L}9O#pPhr?w>2+Q7UzIaq9?^mV}K%!D4)`!t`Qp2{9B z#A5(bh0CbOE&L@=M_Ma{8|{p2ZN%zsj3DNT#%e&#D-O0I3;Ql*W9U&ia9hGHD~eCW zNLdvm={ay=qLyNTcRnJSuIGv73)dd>XDsE zO;*xsgHU5NHK;a8D!=W5Ia`IhNfLC5LiJA7(eF$7gm8-_J(`-{<6#qrxF_X0G?g{0Pz!?3PksSfmf__e1%JlVL??(bs#L)TPAEV{}F(tkF%G^(Q4i zP)olur?L}TnB78~_bO=X zH)|k4Kzx(>u@wL?lJd4Cdx<1jNTbavZ;5U72g~GSdNP#HBb9pf+QZB*Xu`0ejy8c| zV1uxJhUo>Ys#3Rbdl2b+*kQc^qdm6+aB%56Ybkg;C_sXb%Rf7B*E=^K?q4TA-_IT( zaeEfr{2?bB`3DfqQ&yfz5@>k~jzSW6vz3FTOV@>A|AD=yl%1SdwuX!nJ$bVve40*u zeoHiWhvC^jcdCpXpG40TUwgJyJpu-QVO37m!--GMgTdFjEw zad7opE%>~}Td%;G$i~5gG;vr|(lp>~d>n>8(rIp+u|B&?%NgAv&~Mfm0}^~aT(9&z zD^9>>ogF|ucdTP}w){C>5kLiXS<%*_=jeN;$8IMUgQy*Ml!p{y1`~Q&CFr_JuT*C; z;Cz^&?WOO{dfY{y!q;lK-UzfPcL~ON!lbtzmuuS=w(n<^X_=z;F)tzM-W+Q#)T`9i zI4U!nEHGA5uS^1(Fi6J7jZ6m+F(pGE)NN+(;{u=**moLS#8#-6c@?<*?N^)Z%$$fi ziO(A!G)(6;CJvjUj;fi4m^(~6hsguGiHk(Khs741iuK^(n4-v2sM_awD9jr%X;>Pi z*?)h>3xHKm=$92&w)>6TpJr6*o zj02=gv{ESvub+0Oruv{$LacV^800cj1Z4*?*?g4-)7X5q1TEf7hg%~k2Q1gM9c6ou zm0k{aW&M4S<-2NG!VpZ`7a}3fW|2ygQtfEznoS$3j5XGDGtvfIbW&!Q=*~7|&F*_* zcS*p(nK#sJ2_Oj80>kHJJK1Yj%bo&%lL8tIpm9$}-F}5T4G9v)L1E#Fu->xtkld~= zU-PInUM4=m`J!{e`NG#iJ$BdD5FNi??I#9mqy+i+P+%>l=`mIlW%kH!SpDJ@5!vBz#83f?7XUz| zf+0Js^?BUt{n{CPf#(f(!O`cuh4d%-n+|OuyWX9x9{BV3fk+N9D!L(ApMySc99p)v zRhQfLyXgip_mKLTw99H2qsHd7KCqRkWRDWkKk9<+^edlJbfXnPQLX7n@>6%lz#yOj^~krW zmBR06EI5>|P)d1j2;*QyQUiJ_%q5vOBp_YW(qb+WgX5Xc?z-dEULdPRW z27kn^DevsSjz2SMx&8tpS9vy$GeL27M(0#ZY(-b3r87(T(7%|v^CNd`RFr>4k>KhJ zo`iGiAaks>Wc@pYSWL#cwm0;kQgY_WO~4&D{=}qQ6TjS-7rpi%G63LoRa^dwe$=Wu1O zmnn{2HUeq}CU$YX532hQeIiu^9~SBdGXugM|KvweI!v8&;8^)CLp3jaZ0bc*3T3RP z<$ic4s5W~4H7b|TCtmT4LgI2z%xN)aaLnA$&gmj`VR=ZeAr@fv_P{M8mKpLBxAdJJ{=nvvz)pgnY1K)V?foP;9S-LZ z?&o#F`f;It7s0YSIJncP?6dNTBI@_!gAhy1-%{kZnr$Qaw`gDSApK9$b)TF4UtkWT z+;<4$N-QmlwlcK&43V019iNPWkI`N2eLcQb!S4to0)wR!Az@?1;#&IIh`gC!g#U|W zb>rBLvqTkOX#bbTk80c@e}Vx4X~Rm2C1nNpqOM^72+(~TJ{aLIL#l)pY_j-)>X*)MwcYf#|yT4)l zPlw~l?&8%x!39z#XH`G(dKK?vKBfC>)gBKx!C>8=Tiimp5fL8Uo(VRW8cP8WhO^t> zHM#bn2B#7Jx(<=&^$Dr`v;>i{wkYXa_9G~%1dWhSmJ3KwXbB#!&#i8u+&m4w)(cc0 zkR99~jdR0Zb0mcg(+QeG>m-WGJ4{r&R-EC~)ih|4*jwZEBH~>J0%+{TiRPN|2PCr1Ydtep~?SrtFA|g9Gs{v(oOZo6G0Z1gzNmm^)gg-fzMB@fKG; zeqn!hhEXu<%!d?j-Ta0Ah;(`$5~bkMlmn-vaP=Wc$%p_HPPs}_;mrmozFekP6Rn!u zJ^L_ zbC}C%gnkZ{p$z7qXYy*vCNms{D_2i=#WlTStSh?cOj1(iwdl=4qy|JOjg&CgT($wJ zNwRMk&d;wm&4sHR%0cmwY#cw&wz=P6m@6j|qyWlamE6X7u{N-CxG*MvD68daV$R=5 z*c_6nsnI8aB2#tRkluh!PYNmvDT2Bi)u{yq^VkIPWs9@DBA-Vmo-FjwRNsZf-?5$p z=DeJwE&1H>JW%qHik)oBZ{>rexfkHX*TNZ7E`n zYYMcbyB;w9jCH{8fj5=Iv}#roZi9w0zZPbB;%UZ;Lvw1yL3Ti!`XH43vLmYfKSyxR z206X50k%(2K=B|<&V+Ba6snPFr1n)lJNbB>RkBpcp2BNmP_N}H?YDTg30TQ!B62W- z{s$nrF0A)}5a&A*K>gYp_Osh7J8>4YSwbnE!GV)I!itkS?1F>RrZPy)s}LO8i6*WE z#Vli(5G|+IPz#VKqSH(pM0gOTvpyfZB%T?EhG!hHb8Vp-f3XiUuC~Vj*UqnI_4+P~=c(RU2QfjBu7KD-=e>`Ne3i!mbp# zu^Lumq&}nprFxhcL8C-}c1$%;8i~WKf&9S6Y(S?tBn*&y9VQXeEsh*EhVFLCH7+47 zeJJ6?ZX_`ge^$J&T^brzhMhH{sf`TKq^T>>iyxMyWwMiuOZqu}WEyqGRS7Ui!>j-|6gQ%Y#+N~BDdsmQTXV<4 zYIkl{GdSX<`Dl6_{_xn4%k~hZ#5a#YC>4jzd?8$6_BH{1;Al#QT^dUp?d?|2d=Ez7 zY>}R!eZXyc_;-!|Z|iD=(b0B2ENhzH-D*K`DM{cJYnz^PTb+WvHPz~50-aU!IKgQp z;}D>vev*MVT~TZK_Iuz@|C!!J6l4sx9QRu)x^C34z6YVv3EBk3Wyb@HdnXCYO2CDc z{%9)%RhS)woWO1>VGT*P&e+q`RchT4od^CSj2@D!k(TGhwQ+^$(V^-fUe~RtR<@A) z^|{k>4(h}GrM`Yn*i!YRl1eo0fs&p)%N+n5kz;w5D#jkwU~5X!%72NiNl%|EF1BfA zAAJB}&WO7<-z>vghROXd!mXkNDcsm*Hh%9_sAnxZGIq9`q-L@7u=Q5g;?nF8Z9@Lv zn%9QaR)O4GHSbWWX@x9+-CoZ$$=J#x#{~JF;S6dVg+hAWbtxIG(cf|J2WyN8rr$ueM8tdR8Ph5@E zj{!n!hMUE9FsEszMAE^d=hqT^uqGhN*%ov+kT7@^F|e{N3nJyCAlnb(MdRd>wq4+F zV=W0GF(I37EopJ6LBE5QBiOzW?h(-n3$AALMY}Zum@D3XE1nS?YV^=PK~yc#<(19S1G?Dg${=%ugqwB<%HaKvg)j{u|XKV*X|2K_E$YVr&m2ju;`7 zWrHg-Za}8|;uS|Gal`7|1CJ6bEvW@BkneN)M4Nll=st7U0iTaM^c_%;Uuy)`FU7Wy zVWnt-V7bnmq6&8^2+w_%iQl61fUn*bDBbt%HpC&F!g$!!_jLAIVMwIG^D~2Hs-+>H zs1%1U(p#sf z`--8jfZ^8_r^Ii%5Z_yuSS2d>a$N_(XoR}R;MAz&EeQhMSHHW#;LYpA8M%h=B>qD$ zu7dMS@;xmsRU0&!E8ZTY^9znWZ!8Yq7<*7hIvLcUy_PRDZVS*)$+WTDyr|^(yV1v@ zEJ6F)`q8b#@hEo*J7x>T#EmmvSk6M*EFttWx3Tl8(cV2Mx6xF8(sx@Nbfa$TI|#lk z`g&_b%^zP!Cj)1fk+FJVmGqc()HWnsHb^%dH z8Yc1F{oMkiy$P^K?pb~{DBcz0#Soz-wA00Vl_2h!JK_uB!5drx<4$$UPx(t4!~+RN z{|>KR@H??QuP^7U4RP&Aa6~9Cb6~rZ6{2st2hH1zh;p8?v7C(ZfF6<}QjD(rtg50} zW?rcj+!%RSvyGPC+(cXU2qN_ zo#`TmZnmaZw`s1{q1Rriuq;tiL?Mc@*^1I>RdfMa(ot3$a3Uds?JznTZhiC4Pizbf>Q`*@k z?|qJLujrW)tS`ArmrqN?*1@r)7buQ_%j9| zMrOF&kKmm}&n<)*-TX%ZV`pN5UZh6tMBA>p<=k?unToyjYS`5w zxp}jH4`KW}3wwAa%sdSN1IUu9h7WzTbQ6poe#A<$@S2XQaC5WaWURH*?F727bu>3u z;&9Z~Xm=)3mtN!9WvLU)3ELgTivI%;TI&q4b#dcGVQY_R)v65CZ(}TJgL!*Ns78xET6*RDiAbqQb(Ek&TGmbnnr3Qt@-j1Yin0}T!_Yxy{Sx-l z20~OWo*2%9cQ0Z_jDBuBqJIbIA3dDaYj$HS$Zk)tDwX}&2m9;7@Era*f&v6I!uAN) z)TSPu+G5sNG-!nDxB?U0zW;6`A#z;IH{(DKgBj&vXx55--RGP(;a>S`+AELb408m6 zw0V3Kc9@?|elZ7I8Hq!4XewYeE$v{$TZakYX{%adZxJ$J=0s5Yo*l@Bj>^KIG#h@D z_5NYlxhMn$B4fP$wNI|qQHcgnx1XutY&1uCJ>VV)0p~%H{k+!m2X=%!ws!4-?Gj7( zkHnRAjU+sbFA^%UPm@)dS$RkCW1Ozg*{8@_7%HB>r-FnGctM_k%)HfxL@5KZ+a@+E zJQ=I;z1o@$OJsTIAu_Tpz)@EN-gC~5iHkrn5U2KDxp?e`2DO;ntaU&U2U;X`@7>Upv8r*cb9->N5ra%{i_?-H=yN z|AZ5r$v&UaRV?v)7azdpao4w%N~ipaLbqdHpLaWT?!MK^PT`)V!IZpL1Or~G7L*#A zZ!m2!)Ep2Qbg9Wj@N`D89DbIAyk&7pe;h$;*bl$p#VS%_pdHExQ7#V9k!0Oh@+*t` zU5iD+N{7D~nKNQTqFdbqd*Eng$F`9HGu0MsC;42JOWbLJC=>b2Gi$J;&7>< zGIeU9B!V-I(oru#MH%NJ*+@}J%Wd7*=$zcNHVNyfoIk~iq3wbpc% za0gQ7J;5vF^7dPV`z9QbT*Gt-m-Y-5Vf%YktDq!M^-FGF@(lpC!2%)0;Qe+Gp%y|T zj#WZQ!)x^3f`GJL?CE?uSyf{vR;>Ut-*(@tzw{NqdQ3RXAH4tBVq?&jPDhVnA! z%@(Hj=0wDsXw8sSHZ-;4(p5=GQHl-YRN;_*BvG_dmKmMoCEv)`PD+p~p@)OLX%shP zTQ2E4OkvG@SZlUba^ES<#8}KS%KnDtyXoVHEpTysb%!^Y`oYi(MN z*uw(et&R0FWL3_jyroM#&iGQi&xBI8^99NtsrY~kVvqPK48;zWt^$1d3njh~9Q-xP zDi~jaB{vJ?G*?8XTZ9##q&><c`bmrW&=#&n+Z>eV*jW_65^C+^zLBu%#ovzckqMg6#n* zfbo~8+-nst#GT=$vj|zT=QyM2I?mT8E=4cKo!W(I*XRkvTmF>J0(5)MO6uNOxV~-2 zN={V7Y8=@`XJLOTfq&qZ=GmzKI=ZgbIkq;cE{JOjAV?Eiy7B#sy6FT=F%IYRS1ecH z*g8DSN1T0toG9=n1@lLtz)n*}SVKSwbU+IE2yu%>lzOcI8QB(9Hc59`8u??#$S9YYws^xo#PW1v^# z4gNcL!r`8fdD#fvm!@~vq4JZ`|2Kri2#Q1r;;UZD4l4L(#!n*K^j`qC?9mJP!k5}D zvsK-bIy}|fNa^p*M`Gs9-^Qc8<&O&QX%mF^=rlh$4q~^w&Q%hTS(nIp;$9j~pQF2E z2(k|)!$*+@PFQlI^4?pV#)emc3yuf7`!CLmov}gHCe1@oH zdQn!o%fG0NnbB>j{=g>9;8(Nz)PPwHY>xK6uUg7ov1uX1IQH#O{UlW_qzhOGXc~ih zOL)K_Q9iZM*l%bH3$7bXf}{Ikgv(~A9u5ZDDhQ7|$!K#)sZam{tn*B2h{Gx@%+b5q zV;`2az7PWI0PTl^u`{eStMOPSK6^q9Xrh4^6{>+udl9E=WJ0-qUmsXxQMuPsu_p}Vti zE5dQ=fjax>Kwy%9ARYiYp*BEQ>@3HBF+Vl%&P;sGwtW*4H`hYvd7;ye6c0h1_4sV*7)RpJ58FoiC9{H4o6GD8ou_+;!$Xh+32T zNr2^9y)EWknnb5Lb+E(SE_$y&;WBj4y(ts2Q6<>GtwhB-%|-DZow%S8`$s7E475mMJF@_<_wvrdjf*k9uyQOI- zMkS)eQ)U1`2s{Af(}=ixQPLNILwq6&y=<~W!3DBa`=uk-9}B)6l13rTtxsWo=drdaz&XbuAJ8vOL6V zqGMisTLD^&mMC9n4)Ptyl~UT(-T+3e;weR*CSb|v*eGk@8naijHk(oMTm>pFm@VL3I!i%>w_>?AKLG?vA{=IZ@{8!)TxJ0g-H(RCLzo33MuosfsR{;~vjV`q zv`_i#bk&6w0uj59hGh+WB?-2*~+ozH~w4??fOv>>1Lw=zC`JBm3ij zAa^LoOFil9GxXTzGyOX91KIEThY(OQisIZ8RLbS3#z-F!oK1#ovL~MzEMX>cR`BM& z;VZ$Qd&zO1ddy8jlS5#^Nxv>-9XaY1fMGEohL6xTszd&@oMnu%2@PU^c(El2v1^Ud zVi5770&-sr0lO>Qnw1-WrLv$(ZhJQ1TW_d75u`o~`WT^#I=&oE0dW`%!v~mA95_@U zzC|^EeMkS!EiSEM%3;UDH7ojIGW1|Ep3Bvru%VgJS`aDy=OV&LZ|BS}swUTxZN_9N z&priR$>4BVfJ19vBe)qNq_8DL>uqJG!nnjS{$j=F>W#Q06;d7!hD(D!KUV>iUP($m zQF2z-@qB5+={$>pkB~r-Tn4aR7D{2PHG7JgZYx}LS_QjiuYruAlW@>>{{xS*II_J2 zIzSEFo&x2zTwCCdZAT^vB-3HB^rT~tAFQ?0K-Sli!=Dk-_vNXs8H%eqnGXm8Y&q3P z2w&%;{0K}PizH{$#JK#sYf8u_5C5dol_;k)1Q>EbaU ze{pxy6&6|Nt}w`IEvDz@rc25TEZBGAq6O7apFP=dxRx%83rk!%0c4C31Q{M0;>T;p@xameOu%`UL9Y5*dR5ls$GkW_QS>GcDW zx+8cW@zex;2KIGr)8e%p76^%lphDEfSvjgQgdFBKfFQ8S?wW!CmMbDKILZx7HsJyf z&wC4^L~;Q;vik?tRo%_=3VrLOsZIO-bw;f{ym1)+E?YA}3d28&j+94{j^y{qL7rP^ zVFq2BGRRy@?MYJ-_yEM4^GWx$NfX|ZORbwt4lYT zleSHmLW(s!$a}064QoU816%x@cTd)VA11zX@DiR)&T$; zonHY(UjmjdDgaflAMujtVi|+tKBz)!xkHm{Js`#X86cl|d{1TCcm%TSc>X67Vx}AD zK&Q39Be3;hQEr%E*N94lO|A+WhTzkIh>p%N z4^dd;qP?83oT-}H@NNF%AjwKuH?Vd&Qw%pF`vEnPPe(xUpd}Vc(@qTAzw+*ZuK>(L z`uY7|r~N>~eo85N^et8YDWjlc?lmUeLe8#m4n!P5No&cWy zc3Q0`%||f|Nm^3$jrnU~*=aHQly-;UVL&<_bosS)X>r8b4Q-CfjlCi9nfVoSyJ*zW zIhUkNsn%(fb>$9~EA&$Y^hx%r+mM8L(yw8wZDy7|EBZ;UUL$ARjur}}s=kpa+imI& zC{((t?#uXTQ3vv;Dici#zg*mRNC2vpJ54VzOP*YUb+XcXf%sgoMK?`WLBz}oES7u- zk~Ya!Ee&VvdTniw?|KVgqPQq%G4>Ln)`A8#cV}npE7fUjGo*;zfoWj6h^X993^e5e z3?7=)%vyfi{QGpV-T(Lr44T$~@`*%BPvP`60#5A?OdhJ~vWB(%XhBy-3;<&lDAG)Y za_T9ZWyUQsOk-a*Ha29({!TyFuJfe3?Ha@G7vEUqEra?i%DE| zcKX7Y*I&e^siYGS`dw*rXm-$O$I~u0xaVh8J50Kvnm0e43|?o$tpBd-|3@Rr0dd$G zWnXH1U*Q{Z{%C^tvaYWk=hi|x-~{=w7pKofq+M03L+IPM=@I zj-Di;S+@-wiD)?nDByGyWE2w4ID!f?T40YH<<$2dhWvzmG923OlHH9o?-t=^zv=VL!D8+&-Q7{Mgml^ys{}7n(nw2y4gue#6>i8S3LpLI6_HI~MC$nZKyy z6iJxTqIXuzw$U=7BTnzG*=H0hLuAsY@SiyMLh&|#aJ`|NLicd|KXUQ$^R>6pC*;e5 z!q{G6MT7#2-cqUXQ2RdFgOz0dP5%-L8HKO9KB^A^As7%Q@fquthS?Qhc>`vlK`R6U z03{)l6;d{;wp&CB0+z)K`%XrdL2`}G&;QjLHKOXggZ)3E*~CV*IF$cV9aa*Nz92xB zBnFgt4GRiMJ<-@CN(XVNfjF$`m@en)msV-EykY5FOe&C6PiisDwKRBUsL61#4LWBI z=yS6rK$_Pw`>;)%GbD6|=)LmLwPPW0bCv?=ePs!#eoc#!Fy+M=FoCJKRmXNvK^oG6 zK-r$5jGy$BfVxS(V}K~VvxjK*6bAHzWvAQ`KtwyWq;l;{4t{ug3K)EXAS$ptl_T+w zJuLd(6YKkNg;;p-AwF&H(KIFD#a|=6exQJ4JPqn)#sXKtm|M>z1`*NAa?&w=B_V2Sx zPS|9291f}L#bM!COtDc?MLD^_8alT4+2*D}$Fm@4)nL>j3(?TWCetZV>Gfn_C0)iQ z=0ao2OHgy0ps=;1O#ymYbn8Om9u_VT8-KP*4{%xZ6Yp5s0lKZ-=Lz<%${3!|cc(u| zRDuW_LN1B6`hwRk!9vaT{N6>*xL=QITY#vX>qRWH(%J3}G3g5v zZDN;hm#Yr?V$vQ(Vi`c4U}!F|kc))aGm+R#x1g(Vh=aa!W$Yxf*|athP3!XCenV8$ z>JSMT+t1B9F>Ti{4Y79rjqS$f6;QcTg*n0@(X#j4e7)5ZOZm>uHyp=f{5Vi}nSCENeR0@fjw$UTnAAMEZ`VHcLU1g`)mLMg6 zAHGt_6|hpl6=9E#bKwmxp{6_dA|<)5Qh~3nuW|RQtB@o zV%aBh96DCN(9}qI`2W;#E#OdQe|VJK!XUR{Oa>K2uH}|WR>`f9T-N1}#N;*%8?{Uq zw<5(+k~>RV5t$8zCHI6G?7BYLznGktSG}QnnBOn*S!H>ku@S?d z-RpwcA2oxvt^+KZ!HvgXY8!@Jvzq=2GMtObN;;y}<*-x3qTr=Ek=AFw!z44}P_vD@ zO<2UPF>}7IZrO;e9{Od8nr3Ht+b=~*F@bWLAxU~0riS&zd7l*OWek=K1!=My*52g? zmR7GmuTmX79{syqc|{LbLXKczP}ScGC)p1>#a(kXcmnKtAF$i;+EF_9#2&W)Iu*x7 zv#Xpl=zg;09aQ@8*kL+iIf@j(mCte14xF$@|_nF4zIhuSGk!0=4 zbJiJgd$O;3fOT6SgHp$2xEp^GW=2U&8XA2v^sRfw2r7z9^QjZ|2psrLnAR$(Jx9GJ zmGYJ}E9SF)(!j=KORfirJ-y>*tk?4ieeN+)Vzx;qag@L{aG;>Eio`EtKQ8uEKG#q4 zAU82i)wkr1Q*?GSy*;8=IPP-2g`(PPNv)esv2c{}tBt0rM)8LQv&V?piIr|G<%6dA z9eGibkD9e=&RlmY&NOLMmC0h*cR2yQA15=1pJxv5C!O!`yg;W7z9rt4NL1^3Y%uO$ zt2aa?U5IQDl9khKdMp$`i+8=Z`|o|zI@!j(VY_*+u&UCFoh-geRsN6gk56Fku2fO~ z>sQ~&w-1ZYXH~7Q);KpDE`IXWm)htusc|aaE%(Ew=(VeVb5AnSQ*|*g(T(N0D{ww*?F}gr@;^y=(ciF^3yQr6xQI>PGlPPVjzWP%WQb27>go!>)G>qX} z0IX6C8NNB^()P$mCcjo{`e)=o{Lz#gs*=`~TMi<}UvSkV&w2*zx#VWbu>7g++)rEv za^!bjrdA{o9}zNNoG#Pz(Kh8f(_{MnMmsK6$2Nxy5i1;y9#*H{V8t2O=B7INxkxzM%xxp$Rr{jOY@JP~Vm_sX6- z@k%lK4;yt_jXL&|6=t0$kLs4{DJPZ-7MMje+A*I$A8a$Dh0k8gzv(qwP|8!uQ_1eQ z5|gR(#MP;>bw_`k3;oRo%X;cmW%c~opX=VguNn_23;(ERT@}@S$=EpQbOeA#U5y)*${^Y_Bk$?Hbct(dKbFe|7#3 z*YWBDr)mR>Vm*UZb~igyOuE47y!<8+f&8;t?EO-&ab$Z#vyvz?dc~%AU4=g<#7k~+ zkEJGjBu<{mPf|X4!?)tCunza_{35bO&k2bK{po$<2W1oNw3udv+<_ZaMPkoTCIs=x z>egV5Pnj6rPRnS|Z0n9Ff0YTgZEs<7UyMUfbbXY6Q^spm`4mgzxm$dX^yG>++}1V? z+W2er`;v{_P4D>cTltgp>ci-@g$^HH249TJIDcQ~-hH$1gc16Ox`julDL!rsHBp9r z2r!wO_c;&KJI0w~W!x~!tL-8%EQ5ZASsoxV2_Q5_2DmcujJ3gXDDhJXG39|wQ1}G= zF2_PV8F;f~7`G#AS3{mF@;&mJr8^@pn{EQbH#!X><#BGj5YS)~7~x4TQHWbIHxhWI z;?WOU+Gk`>P5%TrpxH|R$CEdKvUDC0(<{Pg$h?Z+%w7T`G{*p`OY`)R?Da2BVvwlq zBQRVlXHXBK{R9T3P5_d?4<24B9q zpP&$*B@fsW*xD)}gQ%KC$|uC*h`;x;2t1}lAy^wnxjD@0v7rLcN`^VR3>QdNlmOOT zG8}@+Ll9h&;BdY?SOA@uUt4pb8p`*xK=pHESK)A$&`#%@5FT0ZE^5iD--}?qm2c%% zR*0(j7e#zREBb@4ub}~FNH5V~Mq2VezMMDO;1u9>TGkQ>m~e<7&zWi2L*#f~9ul%( z`XONxx(bVnmQ(WQYPkDt;s{3L*FR<-ghkSfG(Rkka zf>S;lmIjH|a%ZOi2}vy|3(I_QEqnP>mewb0QN3TZRB*rUp8vVdup(EXE~75Y#iV$p(cJzzCCXH4gV47_*cB zo+XIFO-7*_Y!v$9;zDSWEN5zN$;)rh=P6jg)np~T;r@7!k zuRc02&09VktpY&q+lz2YAJY35LA?Sf>syPkS`2Hc0?w7noQ_2MQD9F!1Qo!t%ixFM z^NrwAzcCW`VCRpTfqRo7M=aa2y!Y%@;69+}jO}ooeSY~yjrRF(ZL0PE@A?NV*1$Jo Q;k^UD+~FQ?)`rgh1-4RH;Q#;t diff --git a/gradle/wrapper/gradle-wrapper.properties b/gradle/wrapper/gradle-wrapper.properties index 5333afa71b6f5..e73574cc2a21c 100644 --- a/gradle/wrapper/gradle-wrapper.properties +++ b/gradle/wrapper/gradle-wrapper.properties @@ -1,6 +1,6 @@ distributionBase=GRADLE_USER_HOME distributionPath=wrapper/dists -distributionUrl=https\://services.gradle.org/distributions/gradle-5.0-all.zip +distributionUrl=https\://services.gradle.org/distributions/gradle-5.1-all.zip zipStoreBase=GRADLE_USER_HOME zipStorePath=wrapper/dists -distributionSha256Sum=17847c8e12b2bcfce26a79f425f082c31d4ded822f99a66127eee2d96bf18216 +distributionSha256Sum=7b8a8b9cce0406733d2d3fab3874386c530657c73c3f4e9a3837d081e26060d8 From 054c3bb04f281cdc71a98e2ac138c4e908943924 Mon Sep 17 00:00:00 2001 From: Jim Ferenczi Date: Tue, 8 Jan 2019 13:31:53 +0100 Subject: [PATCH 023/186] Fix topDocs.totalHits assignment on scroll queries (#37180) This change fixes an unreleased bug that assigns the wrong totalHits to scroll queries. Closes #37179 --- .../org/elasticsearch/search/query/QueryPhase.java | 2 -- .../search/query/TopDocsCollectorContext.java | 13 ++++++------- .../elasticsearch/search/stats/SearchStatsIT.java | 1 - 3 files changed, 6 insertions(+), 10 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/search/query/QueryPhase.java b/server/src/main/java/org/elasticsearch/search/query/QueryPhase.java index ce27f03daea06..9ccde2468227a 100644 --- a/server/src/main/java/org/elasticsearch/search/query/QueryPhase.java +++ b/server/src/main/java/org/elasticsearch/search/query/QueryPhase.java @@ -166,7 +166,6 @@ static boolean execute(SearchContext searchContext, } // ... and stop collecting after ${size} matches searchContext.terminateAfter(searchContext.size()); - searchContext.trackTotalHitsUpTo(SearchContext.TRACK_TOTAL_HITS_DISABLED); } else if (canEarlyTerminate(reader, searchContext.sort())) { // now this gets interesting: since the search sort is a prefix of the index sort, we can directly // skip to the desired doc @@ -177,7 +176,6 @@ static boolean execute(SearchContext searchContext, .build(); query = bq; } - searchContext.trackTotalHitsUpTo(SearchContext.TRACK_TOTAL_HITS_DISABLED); } } } diff --git a/server/src/main/java/org/elasticsearch/search/query/TopDocsCollectorContext.java b/server/src/main/java/org/elasticsearch/search/query/TopDocsCollectorContext.java index 44c1b35ad1458..4f648666f550d 100644 --- a/server/src/main/java/org/elasticsearch/search/query/TopDocsCollectorContext.java +++ b/server/src/main/java/org/elasticsearch/search/query/TopDocsCollectorContext.java @@ -217,8 +217,6 @@ private SimpleTopDocsCollectorContext(IndexReader reader, super(REASON_SEARCH_TOP_HITS, numHits); this.sortAndFormats = sortAndFormats; - // implicit total hit counts are valid only when there is no filter collector in the chain - final int hitCount = hasFilterCollector ? -1 : shortcutTotalHitCount(reader, query); final TopDocsCollector topDocsCollector; if (trackTotalHitsUpTo == SearchContext.TRACK_TOTAL_HITS_DISABLED) { // don't compute hit counts via the collector @@ -226,6 +224,8 @@ private SimpleTopDocsCollectorContext(IndexReader reader, topDocsSupplier = new CachedSupplier<>(topDocsCollector::topDocs); totalHitsSupplier = () -> new TotalHits(0, TotalHits.Relation.GREATER_THAN_OR_EQUAL_TO); } else { + // implicit total hit counts are valid only when there is no filter collector in the chain + final int hitCount = hasFilterCollector ? -1 : shortcutTotalHitCount(reader, query); if (hitCount == -1) { topDocsCollector = createCollector(sortAndFormats, numHits, searchAfter, trackTotalHitsUpTo); topDocsSupplier = new CachedSupplier<>(topDocsCollector::topDocs); @@ -293,12 +293,11 @@ private ScrollingTopDocsCollectorContext(IndexReader reader, @Override void postProcess(QuerySearchResult result) throws IOException { final TopDocs topDocs = topDocsSupplier.get(); - topDocs.totalHits = totalHitsSupplier.get(); - float maxScore = maxScoreSupplier.get(); + final float maxScore; if (scrollContext.totalHits == null) { // first round - scrollContext.totalHits = topDocs.totalHits; - scrollContext.maxScore = maxScore; + topDocs.totalHits = scrollContext.totalHits = totalHitsSupplier.get(); + maxScore = scrollContext.maxScore = maxScoreSupplier.get(); } else { // subsequent round: the total number of hits and // the maximum score were computed on the first round @@ -367,7 +366,7 @@ static TopDocsCollectorContext createTopDocsCollectorContext(SearchContext searc // we can disable the tracking of total hits after the initial scroll query // since the total hits is preserved in the scroll context. int trackTotalHitsUpTo = searchContext.scrollContext().totalHits != null ? - SearchContext.TRACK_TOTAL_HITS_DISABLED : searchContext.trackTotalHitsUpTo(); + SearchContext.TRACK_TOTAL_HITS_DISABLED : SearchContext.TRACK_TOTAL_HITS_ACCURATE; // no matter what the value of from is int numDocs = Math.min(searchContext.size(), totalNumDocs); return new ScrollingTopDocsCollectorContext(reader, query, searchContext.scrollContext(), diff --git a/server/src/test/java/org/elasticsearch/search/stats/SearchStatsIT.java b/server/src/test/java/org/elasticsearch/search/stats/SearchStatsIT.java index 7c2bef2193858..a575d1ad63eb7 100644 --- a/server/src/test/java/org/elasticsearch/search/stats/SearchStatsIT.java +++ b/server/src/test/java/org/elasticsearch/search/stats/SearchStatsIT.java @@ -178,7 +178,6 @@ private Set nodeIdsWithIndex(String... indices) { return nodes; } - @AwaitsFix(bugUrl="https://github.com/elastic/elasticsearch/issues/37179") public void testOpenContexts() { String index = "test1"; createIndex(index); From dd69553d4d6d57b64664349067b839aca585bbca Mon Sep 17 00:00:00 2001 From: Michael Basnight Date: Tue, 8 Jan 2019 09:11:58 -0600 Subject: [PATCH 024/186] HLRC: Use nonblocking entity for requests (#32249) Previously the HLRC used a blocking ByteArrayEntity, but the Request class also allows to set a NByteArrayEntity, and defaults to nonblocking when calling the createJsonEntity method. This commit cleans up all the uses of ByteArrayEntity in the RequestConverters to use the nonblocking entity. --- .../client/MLRequestConverters.java | 6 ++-- .../client/RequestConverters.java | 12 +++---- .../client/WatcherRequestConverters.java | 4 +-- .../forbidden/rest-high-level-signatures.txt | 4 +++ .../CustomRestHighLevelClientTests.java | 4 +-- .../client/RequestConvertersTests.java | 8 ++--- .../client/RestHighLevelClientExtTests.java | 6 ++-- .../client/RestHighLevelClientTests.java | 35 +++++++++---------- 8 files changed, 41 insertions(+), 38 deletions(-) diff --git a/client/rest-high-level/src/main/java/org/elasticsearch/client/MLRequestConverters.java b/client/rest-high-level/src/main/java/org/elasticsearch/client/MLRequestConverters.java index f24f12dd6b7a0..6b0a5d2642f02 100644 --- a/client/rest-high-level/src/main/java/org/elasticsearch/client/MLRequestConverters.java +++ b/client/rest-high-level/src/main/java/org/elasticsearch/client/MLRequestConverters.java @@ -24,7 +24,7 @@ import org.apache.http.client.methods.HttpGet; import org.apache.http.client.methods.HttpPost; import org.apache.http.client.methods.HttpPut; -import org.apache.http.entity.ByteArrayEntity; +import org.apache.http.nio.entity.NByteArrayEntity; import org.apache.lucene.util.BytesRef; import org.elasticsearch.client.RequestConverters.EndpointBuilder; import org.elasticsearch.client.ml.CloseJobRequest; @@ -462,7 +462,7 @@ static Request postData(PostDataRequest postDataRequest) { BytesReference content = postDataRequest.getContent(); if (content != null) { BytesRef source = postDataRequest.getContent().toBytesRef(); - HttpEntity byteEntity = new ByteArrayEntity(source.bytes, + HttpEntity byteEntity = new NByteArrayEntity(source.bytes, source.offset, source.length, createContentType(postDataRequest.getXContentType())); @@ -686,7 +686,7 @@ static Request findFileStructure(FindFileStructureRequest findFileStructureReque BytesReference sample = findFileStructureRequest.getSample(); BytesRef source = sample.toBytesRef(); - HttpEntity byteEntity = new ByteArrayEntity(source.bytes, source.offset, source.length, createContentType(XContentType.JSON)); + HttpEntity byteEntity = new NByteArrayEntity(source.bytes, source.offset, source.length, createContentType(XContentType.JSON)); request.setEntity(byteEntity); return request; } diff --git a/client/rest-high-level/src/main/java/org/elasticsearch/client/RequestConverters.java b/client/rest-high-level/src/main/java/org/elasticsearch/client/RequestConverters.java index 63620bd987712..d4f67ba5e9e8c 100644 --- a/client/rest-high-level/src/main/java/org/elasticsearch/client/RequestConverters.java +++ b/client/rest-high-level/src/main/java/org/elasticsearch/client/RequestConverters.java @@ -25,8 +25,8 @@ import org.apache.http.client.methods.HttpHead; import org.apache.http.client.methods.HttpPost; import org.apache.http.client.methods.HttpPut; -import org.apache.http.entity.ByteArrayEntity; import org.apache.http.entity.ContentType; +import org.apache.http.nio.entity.NByteArrayEntity; import org.apache.lucene.util.BytesRef; import org.elasticsearch.action.DocWriteRequest; import org.elasticsearch.action.admin.cluster.health.ClusterHealthRequest; @@ -239,7 +239,7 @@ static Request bulk(BulkRequest bulkRequest) throws IOException { content.write(separator); } } - request.setEntity(new ByteArrayEntity(content.toByteArray(), 0, content.size(), requestContentType)); + request.setEntity(new NByteArrayEntity(content.toByteArray(), 0, content.size(), requestContentType)); return request; } @@ -322,7 +322,7 @@ static Request index(IndexRequest indexRequest) { BytesRef source = indexRequest.source().toBytesRef(); ContentType contentType = createContentType(indexRequest.getContentType()); - request.setEntity(new ByteArrayEntity(source.bytes, source.offset, source.length, contentType)); + request.setEntity(new NByteArrayEntity(source.bytes, source.offset, source.length, contentType)); return request; } @@ -431,7 +431,7 @@ static Request multiSearch(MultiSearchRequest multiSearchRequest) throws IOExcep XContent xContent = REQUEST_BODY_CONTENT_TYPE.xContent(); byte[] source = MultiSearchRequest.writeMultiLineFormat(multiSearchRequest, xContent); - request.setEntity(new ByteArrayEntity(source, createContentType(xContent.type()))); + request.setEntity(new NByteArrayEntity(source, createContentType(xContent.type()))); return request; } @@ -464,7 +464,7 @@ static Request multiSearchTemplate(MultiSearchTemplateRequest multiSearchTemplat XContent xContent = REQUEST_BODY_CONTENT_TYPE.xContent(); byte[] source = MultiSearchTemplateRequest.writeMultiLineFormat(multiSearchTemplateRequest, xContent); - request.setEntity(new ByteArrayEntity(source, createContentType(xContent.type()))); + request.setEntity(new NByteArrayEntity(source, createContentType(xContent.type()))); return request; } @@ -694,7 +694,7 @@ static HttpEntity createEntity(ToXContent toXContent, XContentType xContentType) static HttpEntity createEntity(ToXContent toXContent, XContentType xContentType, ToXContent.Params toXContentParams) throws IOException { BytesRef source = XContentHelper.toXContent(toXContent, xContentType, toXContentParams, false).toBytesRef(); - return new ByteArrayEntity(source.bytes, source.offset, source.length, createContentType(xContentType)); + return new NByteArrayEntity(source.bytes, source.offset, source.length, createContentType(xContentType)); } static String endpoint(String index, String type, String id) { diff --git a/client/rest-high-level/src/main/java/org/elasticsearch/client/WatcherRequestConverters.java b/client/rest-high-level/src/main/java/org/elasticsearch/client/WatcherRequestConverters.java index 33051ae0e432a..a5c6d0dd1810e 100644 --- a/client/rest-high-level/src/main/java/org/elasticsearch/client/WatcherRequestConverters.java +++ b/client/rest-high-level/src/main/java/org/elasticsearch/client/WatcherRequestConverters.java @@ -23,8 +23,8 @@ import org.apache.http.client.methods.HttpGet; import org.apache.http.client.methods.HttpPost; import org.apache.http.client.methods.HttpPut; -import org.apache.http.entity.ByteArrayEntity; import org.apache.http.entity.ContentType; +import org.apache.http.nio.entity.NByteArrayEntity; import org.elasticsearch.client.watcher.AckWatchRequest; import org.elasticsearch.client.watcher.ActivateWatchRequest; import org.elasticsearch.client.watcher.DeactivateWatchRequest; @@ -75,7 +75,7 @@ static Request putWatch(PutWatchRequest putWatchRequest) { } ContentType contentType = RequestConverters.createContentType(putWatchRequest.xContentType()); BytesReference source = putWatchRequest.getSource(); - request.setEntity(new ByteArrayEntity(source.toBytesRef().bytes, 0, source.length(), contentType)); + request.setEntity(new NByteArrayEntity(source.toBytesRef().bytes, 0, source.length(), contentType)); return request; } diff --git a/client/rest-high-level/src/main/resources/forbidden/rest-high-level-signatures.txt b/client/rest-high-level/src/main/resources/forbidden/rest-high-level-signatures.txt index a9214e9333c4e..b9224ffe64971 100644 --- a/client/rest-high-level/src/main/resources/forbidden/rest-high-level-signatures.txt +++ b/client/rest-high-level/src/main/resources/forbidden/rest-high-level-signatures.txt @@ -30,3 +30,7 @@ org.elasticsearch.common.logging.PrefixLogger @defaultMessage We can't rely on log4j2 being on the classpath so don't log deprecations! org.elasticsearch.common.xcontent.LoggingDeprecationHandler + +@defaultMessage Use Nonblocking org.apache.http.nio.entity.NByteArrayEntity +org.apache.http.entity.ByteArrayEntity +org.apache.http.entity.StringEntity diff --git a/client/rest-high-level/src/test/java/org/elasticsearch/client/CustomRestHighLevelClientTests.java b/client/rest-high-level/src/test/java/org/elasticsearch/client/CustomRestHighLevelClientTests.java index 3b69f10344a0d..316de885fa136 100644 --- a/client/rest-high-level/src/test/java/org/elasticsearch/client/CustomRestHighLevelClientTests.java +++ b/client/rest-high-level/src/test/java/org/elasticsearch/client/CustomRestHighLevelClientTests.java @@ -24,10 +24,10 @@ import org.apache.http.ProtocolVersion; import org.apache.http.RequestLine; import org.apache.http.client.methods.HttpGet; -import org.apache.http.entity.ByteArrayEntity; import org.apache.http.entity.ContentType; import org.apache.http.message.BasicRequestLine; import org.apache.http.message.BasicStatusLine; +import org.apache.http.nio.entity.NByteArrayEntity; import org.apache.lucene.util.BytesRef; import org.elasticsearch.Build; import org.elasticsearch.Version; @@ -166,7 +166,7 @@ private Response mockPerformRequest(Request request) throws IOException { MainResponse response = new MainResponse(httpHeader.getValue(), Version.CURRENT, ClusterName.DEFAULT, "_na", Build.CURRENT); BytesRef bytesRef = XContentHelper.toXContent(response, XContentType.JSON, false).toBytesRef(); - when(mockResponse.getEntity()).thenReturn(new ByteArrayEntity(bytesRef.bytes, ContentType.APPLICATION_JSON)); + when(mockResponse.getEntity()).thenReturn(new NByteArrayEntity(bytesRef.bytes, ContentType.APPLICATION_JSON)); RequestLine requestLine = new BasicRequestLine(HttpGet.METHOD_NAME, ENDPOINT, protocol); when(mockResponse.getRequestLine()).thenReturn(requestLine); diff --git a/client/rest-high-level/src/test/java/org/elasticsearch/client/RequestConvertersTests.java b/client/rest-high-level/src/test/java/org/elasticsearch/client/RequestConvertersTests.java index 698f7557c1306..468bcf2bf8a1d 100644 --- a/client/rest-high-level/src/test/java/org/elasticsearch/client/RequestConvertersTests.java +++ b/client/rest-high-level/src/test/java/org/elasticsearch/client/RequestConvertersTests.java @@ -25,7 +25,7 @@ import org.apache.http.client.methods.HttpHead; import org.apache.http.client.methods.HttpPost; import org.apache.http.client.methods.HttpPut; -import org.apache.http.entity.ByteArrayEntity; +import org.apache.http.nio.entity.NByteArrayEntity; import org.apache.http.util.EntityUtils; import org.elasticsearch.action.DocWriteRequest; import org.elasticsearch.action.admin.cluster.storedscripts.DeleteStoredScriptRequest; @@ -671,7 +671,7 @@ public void testIndex() throws IOException { assertEquals(method, request.getMethod()); HttpEntity entity = request.getEntity(); - assertTrue(entity instanceof ByteArrayEntity); + assertTrue(entity instanceof NByteArrayEntity); assertEquals(indexRequest.getContentType().mediaTypeWithoutParameters(), entity.getContentType().getValue()); try (XContentParser parser = createParser(xContentType.xContent(), entity.getContent())) { assertEquals(nbFields, parser.map().size()); @@ -714,7 +714,7 @@ public void testIndexWithType() throws IOException { assertEquals(method, request.getMethod()); HttpEntity entity = request.getEntity(); - assertTrue(entity instanceof ByteArrayEntity); + assertTrue(entity instanceof NByteArrayEntity); assertEquals(indexRequest.getContentType().mediaTypeWithoutParameters(), entity.getContentType().getValue()); try (XContentParser parser = createParser(xContentType.xContent(), entity.getContent())) { assertEquals(nbFields, parser.map().size()); @@ -787,7 +787,7 @@ public void testUpdate() throws IOException { assertEquals(HttpPost.METHOD_NAME, request.getMethod()); HttpEntity entity = request.getEntity(); - assertTrue(entity instanceof ByteArrayEntity); + assertTrue(entity instanceof NByteArrayEntity); UpdateRequest parsedUpdateRequest = new UpdateRequest(); diff --git a/client/rest-high-level/src/test/java/org/elasticsearch/client/RestHighLevelClientExtTests.java b/client/rest-high-level/src/test/java/org/elasticsearch/client/RestHighLevelClientExtTests.java index 970ffb15f083b..130692a5fd8bc 100644 --- a/client/rest-high-level/src/test/java/org/elasticsearch/client/RestHighLevelClientExtTests.java +++ b/client/rest-high-level/src/test/java/org/elasticsearch/client/RestHighLevelClientExtTests.java @@ -21,7 +21,7 @@ import org.apache.http.HttpEntity; import org.apache.http.entity.ContentType; -import org.apache.http.entity.StringEntity; +import org.apache.http.nio.entity.NStringEntity; import org.elasticsearch.common.ParseField; import org.elasticsearch.common.xcontent.NamedXContentRegistry; import org.elasticsearch.common.xcontent.XContentParser; @@ -51,14 +51,14 @@ public void initClient() { public void testParseEntityCustomResponseSection() throws IOException { { - HttpEntity jsonEntity = new StringEntity("{\"custom1\":{ \"field\":\"value\"}}", ContentType.APPLICATION_JSON); + HttpEntity jsonEntity = new NStringEntity("{\"custom1\":{ \"field\":\"value\"}}", ContentType.APPLICATION_JSON); BaseCustomResponseSection customSection = restHighLevelClient.parseEntity(jsonEntity, BaseCustomResponseSection::fromXContent); assertThat(customSection, instanceOf(CustomResponseSection1.class)); CustomResponseSection1 customResponseSection1 = (CustomResponseSection1) customSection; assertEquals("value", customResponseSection1.value); } { - HttpEntity jsonEntity = new StringEntity("{\"custom2\":{ \"array\": [\"item1\", \"item2\"]}}", ContentType.APPLICATION_JSON); + HttpEntity jsonEntity = new NStringEntity("{\"custom2\":{ \"array\": [\"item1\", \"item2\"]}}", ContentType.APPLICATION_JSON); BaseCustomResponseSection customSection = restHighLevelClient.parseEntity(jsonEntity, BaseCustomResponseSection::fromXContent); assertThat(customSection, instanceOf(CustomResponseSection2.class)); CustomResponseSection2 customResponseSection2 = (CustomResponseSection2) customSection; diff --git a/client/rest-high-level/src/test/java/org/elasticsearch/client/RestHighLevelClientTests.java b/client/rest-high-level/src/test/java/org/elasticsearch/client/RestHighLevelClientTests.java index f9d2977b64e58..a94ab4541f0f9 100644 --- a/client/rest-high-level/src/test/java/org/elasticsearch/client/RestHighLevelClientTests.java +++ b/client/rest-high-level/src/test/java/org/elasticsearch/client/RestHighLevelClientTests.java @@ -27,12 +27,11 @@ import org.apache.http.RequestLine; import org.apache.http.StatusLine; import org.apache.http.client.methods.HttpGet; -import org.apache.http.entity.ByteArrayEntity; import org.apache.http.entity.ContentType; -import org.apache.http.entity.StringEntity; import org.apache.http.message.BasicHttpResponse; import org.apache.http.message.BasicRequestLine; import org.apache.http.message.BasicStatusLine; +import org.apache.http.nio.entity.NByteArrayEntity; import org.apache.http.nio.entity.NStringEntity; import org.elasticsearch.Build; import org.elasticsearch.ElasticsearchException; @@ -243,11 +242,11 @@ public void testParseEntity() throws IOException { } { IllegalStateException ise = expectThrows(IllegalStateException.class, - () -> restHighLevelClient.parseEntity(new StringEntity("", (ContentType) null), null)); + () -> restHighLevelClient.parseEntity(new NStringEntity("", (ContentType) null), null)); assertEquals("Elasticsearch didn't return the [Content-Type] header, unable to parse response body", ise.getMessage()); } { - StringEntity entity = new StringEntity("", ContentType.APPLICATION_SVG_XML); + NStringEntity entity = new NStringEntity("", ContentType.APPLICATION_SVG_XML); IllegalStateException ise = expectThrows(IllegalStateException.class, () -> restHighLevelClient.parseEntity(entity, null)); assertEquals("Unsupported Content-Type: " + entity.getContentType().getValue(), ise.getMessage()); } @@ -260,9 +259,9 @@ public void testParseEntity() throws IOException { assertEquals(XContentParser.Token.END_OBJECT, parser.nextToken()); return value; }; - HttpEntity jsonEntity = new StringEntity("{\"field\":\"value\"}", ContentType.APPLICATION_JSON); + HttpEntity jsonEntity = new NStringEntity("{\"field\":\"value\"}", ContentType.APPLICATION_JSON); assertEquals("value", restHighLevelClient.parseEntity(jsonEntity, entityParser)); - HttpEntity yamlEntity = new StringEntity("---\nfield: value\n", ContentType.create("application/yaml")); + HttpEntity yamlEntity = new NStringEntity("---\nfield: value\n", ContentType.create("application/yaml")); assertEquals("value", restHighLevelClient.parseEntity(yamlEntity, entityParser)); HttpEntity smileEntity = createBinaryEntity(SmileXContent.contentBuilder(), ContentType.create("application/smile")); assertEquals("value", restHighLevelClient.parseEntity(smileEntity, entityParser)); @@ -276,7 +275,7 @@ private static HttpEntity createBinaryEntity(XContentBuilder xContentBuilder, Co builder.startObject(); builder.field("field", "value"); builder.endObject(); - return new ByteArrayEntity(BytesReference.bytes(builder).toBytesRef().bytes, contentType); + return new NByteArrayEntity(BytesReference.bytes(builder).toBytesRef().bytes, contentType); } } @@ -302,7 +301,7 @@ public void testParseResponseException() throws IOException { { RestStatus restStatus = randomFrom(RestStatus.values()); HttpResponse httpResponse = new BasicHttpResponse(newStatusLine(restStatus)); - httpResponse.setEntity(new StringEntity("{\"error\":\"test error message\",\"status\":" + restStatus.getStatus() + "}", + httpResponse.setEntity(new NStringEntity("{\"error\":\"test error message\",\"status\":" + restStatus.getStatus() + "}", ContentType.APPLICATION_JSON)); Response response = new Response(REQUEST_LINE, new HttpHost("localhost", 9200), httpResponse); ResponseException responseException = new ResponseException(response); @@ -314,7 +313,7 @@ public void testParseResponseException() throws IOException { { RestStatus restStatus = randomFrom(RestStatus.values()); HttpResponse httpResponse = new BasicHttpResponse(newStatusLine(restStatus)); - httpResponse.setEntity(new StringEntity("{\"error\":", ContentType.APPLICATION_JSON)); + httpResponse.setEntity(new NStringEntity("{\"error\":", ContentType.APPLICATION_JSON)); Response response = new Response(REQUEST_LINE, new HttpHost("localhost", 9200), httpResponse); ResponseException responseException = new ResponseException(response); ElasticsearchException elasticsearchException = restHighLevelClient.parseResponseException(responseException); @@ -326,7 +325,7 @@ public void testParseResponseException() throws IOException { { RestStatus restStatus = randomFrom(RestStatus.values()); HttpResponse httpResponse = new BasicHttpResponse(newStatusLine(restStatus)); - httpResponse.setEntity(new StringEntity("{\"status\":" + restStatus.getStatus() + "}", ContentType.APPLICATION_JSON)); + httpResponse.setEntity(new NStringEntity("{\"status\":" + restStatus.getStatus() + "}", ContentType.APPLICATION_JSON)); Response response = new Response(REQUEST_LINE, new HttpHost("localhost", 9200), httpResponse); ResponseException responseException = new ResponseException(response); ElasticsearchException elasticsearchException = restHighLevelClient.parseResponseException(responseException); @@ -378,7 +377,7 @@ public void testPerformRequestOnResponseExceptionWithEntity() throws IOException CheckedFunction requestConverter = request -> new Request(HttpGet.METHOD_NAME, "/"); RestStatus restStatus = randomFrom(RestStatus.values()); HttpResponse httpResponse = new BasicHttpResponse(newStatusLine(restStatus)); - httpResponse.setEntity(new StringEntity("{\"error\":\"test error message\",\"status\":" + restStatus.getStatus() + "}", + httpResponse.setEntity(new NStringEntity("{\"error\":\"test error message\",\"status\":" + restStatus.getStatus() + "}", ContentType.APPLICATION_JSON)); Response mockResponse = new Response(REQUEST_LINE, new HttpHost("localhost", 9200), httpResponse); ResponseException responseException = new ResponseException(mockResponse); @@ -396,7 +395,7 @@ public void testPerformRequestOnResponseExceptionWithBrokenEntity() throws IOExc CheckedFunction requestConverter = request -> new Request(HttpGet.METHOD_NAME, "/"); RestStatus restStatus = randomFrom(RestStatus.values()); HttpResponse httpResponse = new BasicHttpResponse(newStatusLine(restStatus)); - httpResponse.setEntity(new StringEntity("{\"error\":", ContentType.APPLICATION_JSON)); + httpResponse.setEntity(new NStringEntity("{\"error\":", ContentType.APPLICATION_JSON)); Response mockResponse = new Response(REQUEST_LINE, new HttpHost("localhost", 9200), httpResponse); ResponseException responseException = new ResponseException(mockResponse); when(restClient.performRequest(any(Request.class))).thenThrow(responseException); @@ -414,7 +413,7 @@ public void testPerformRequestOnResponseExceptionWithBrokenEntity2() throws IOEx CheckedFunction requestConverter = request -> new Request(HttpGet.METHOD_NAME, "/"); RestStatus restStatus = randomFrom(RestStatus.values()); HttpResponse httpResponse = new BasicHttpResponse(newStatusLine(restStatus)); - httpResponse.setEntity(new StringEntity("{\"status\":" + restStatus.getStatus() + "}", ContentType.APPLICATION_JSON)); + httpResponse.setEntity(new NStringEntity("{\"status\":" + restStatus.getStatus() + "}", ContentType.APPLICATION_JSON)); Response mockResponse = new Response(REQUEST_LINE, new HttpHost("localhost", 9200), httpResponse); ResponseException responseException = new ResponseException(mockResponse); when(restClient.performRequest(any(Request.class))).thenThrow(responseException); @@ -458,7 +457,7 @@ public void testPerformRequestOnResponseExceptionWithIgnoresErrorValidBody() thr MainRequest mainRequest = new MainRequest(); CheckedFunction requestConverter = request -> new Request(HttpGet.METHOD_NAME, "/"); HttpResponse httpResponse = new BasicHttpResponse(newStatusLine(RestStatus.NOT_FOUND)); - httpResponse.setEntity(new StringEntity("{\"error\":\"test error message\",\"status\":404}", + httpResponse.setEntity(new NStringEntity("{\"error\":\"test error message\",\"status\":404}", ContentType.APPLICATION_JSON)); Response mockResponse = new Response(REQUEST_LINE, new HttpHost("localhost", 9200), httpResponse); ResponseException responseException = new ResponseException(mockResponse); @@ -528,7 +527,7 @@ public void testWrapResponseListenerOnResponseExceptionWithEntity() throws IOExc response -> response.getStatusLine().getStatusCode(), trackingActionListener, Collections.emptySet()); RestStatus restStatus = randomFrom(RestStatus.values()); HttpResponse httpResponse = new BasicHttpResponse(newStatusLine(restStatus)); - httpResponse.setEntity(new StringEntity("{\"error\":\"test error message\",\"status\":" + restStatus.getStatus() + "}", + httpResponse.setEntity(new NStringEntity("{\"error\":\"test error message\",\"status\":" + restStatus.getStatus() + "}", ContentType.APPLICATION_JSON)); Response response = new Response(REQUEST_LINE, new HttpHost("localhost", 9200), httpResponse); ResponseException responseException = new ResponseException(response); @@ -547,7 +546,7 @@ public void testWrapResponseListenerOnResponseExceptionWithBrokenEntity() throws response -> response.getStatusLine().getStatusCode(), trackingActionListener, Collections.emptySet()); RestStatus restStatus = randomFrom(RestStatus.values()); HttpResponse httpResponse = new BasicHttpResponse(newStatusLine(restStatus)); - httpResponse.setEntity(new StringEntity("{\"error\":", ContentType.APPLICATION_JSON)); + httpResponse.setEntity(new NStringEntity("{\"error\":", ContentType.APPLICATION_JSON)); Response response = new Response(REQUEST_LINE, new HttpHost("localhost", 9200), httpResponse); ResponseException responseException = new ResponseException(response); responseListener.onFailure(responseException); @@ -564,7 +563,7 @@ public void testWrapResponseListenerOnResponseExceptionWithBrokenEntity() throws response -> response.getStatusLine().getStatusCode(), trackingActionListener, Collections.emptySet()); RestStatus restStatus = randomFrom(RestStatus.values()); HttpResponse httpResponse = new BasicHttpResponse(newStatusLine(restStatus)); - httpResponse.setEntity(new StringEntity("{\"status\":" + restStatus.getStatus() + "}", ContentType.APPLICATION_JSON)); + httpResponse.setEntity(new NStringEntity("{\"status\":" + restStatus.getStatus() + "}", ContentType.APPLICATION_JSON)); Response response = new Response(REQUEST_LINE, new HttpHost("localhost", 9200), httpResponse); ResponseException responseException = new ResponseException(response); responseListener.onFailure(responseException); @@ -614,7 +613,7 @@ public void testWrapResponseListenerOnResponseExceptionWithIgnoresErrorValidBody ResponseListener responseListener = restHighLevelClient.wrapResponseListener( response -> { throw new IllegalStateException(); }, trackingActionListener, Collections.singleton(404)); HttpResponse httpResponse = new BasicHttpResponse(newStatusLine(RestStatus.NOT_FOUND)); - httpResponse.setEntity(new StringEntity("{\"error\":\"test error message\",\"status\":404}", + httpResponse.setEntity(new NStringEntity("{\"error\":\"test error message\",\"status\":404}", ContentType.APPLICATION_JSON)); Response response = new Response(REQUEST_LINE, new HttpHost("localhost", 9200), httpResponse); ResponseException responseException = new ResponseException(response); From 1514bbcdde36d2f64c4391ec6874e7d0aa75ab5f Mon Sep 17 00:00:00 2001 From: Jay Modi Date: Tue, 8 Jan 2019 08:52:12 -0700 Subject: [PATCH 025/186] Security: propagate auth result to listeners (#36900) After #30794, our caching realms limit each principal to a single auth attempt at a time. This prevents hammering of external servers but can cause a significant performance hit when requests need to go through a realm that takes a long time to attempt to authenticate in order to get to the realm that actually authenticates. In order to address this, this change will propagate failed results to listeners if they use the same set of credentials that the authentication attempt used. This does prevent these stalled requests from retrying the authentication attempt but the implementation does allow for new requests to retry the attempt. --- .../support/CachingUsernamePasswordRealm.java | 117 +++++++++--------- .../CachingUsernamePasswordRealmTests.java | 98 ++++++++++----- 2 files changed, 131 insertions(+), 84 deletions(-) diff --git a/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/authc/support/CachingUsernamePasswordRealm.java b/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/authc/support/CachingUsernamePasswordRealm.java index faec4b8c66511..fed4e1fb13ee6 100644 --- a/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/authc/support/CachingUsernamePasswordRealm.java +++ b/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/authc/support/CachingUsernamePasswordRealm.java @@ -6,6 +6,7 @@ package org.elasticsearch.xpack.security.authc.support; import org.elasticsearch.action.ActionListener; +import org.elasticsearch.common.Nullable; import org.elasticsearch.common.cache.Cache; import org.elasticsearch.common.cache.CacheBuilder; import org.elasticsearch.common.settings.SecureString; @@ -29,7 +30,7 @@ public abstract class CachingUsernamePasswordRealm extends UsernamePasswordRealm implements CachingRealm { - private final Cache> cache; + private final Cache> cache; private final ThreadPool threadPool; private final boolean authenticationEnabled; final Hasher cacheHasher; @@ -40,7 +41,7 @@ protected CachingUsernamePasswordRealm(RealmConfig config, ThreadPool threadPool this.threadPool = threadPool; final TimeValue ttl = this.config.getSetting(CachingUsernamePasswordRealmSettings.CACHE_TTL_SETTING); if (ttl.getNanos() > 0) { - cache = CacheBuilder.>builder() + cache = CacheBuilder.>builder() .setExpireAfterWrite(ttl) .setMaximumWeight(this.config.getSetting(CachingUsernamePasswordRealmSettings.CACHE_MAX_USERS_SETTING)) .build(); @@ -122,58 +123,61 @@ private void authenticateWithCache(UsernamePasswordToken token, ActionListener listenableCacheEntry = cache.computeIfAbsent(token.principal(), k -> { + final ListenableFuture listenableCacheEntry = cache.computeIfAbsent(token.principal(), k -> { authenticationInCache.set(false); return new ListenableFuture<>(); }); if (authenticationInCache.get()) { // there is a cached or an inflight authenticate request - listenableCacheEntry.addListener(ActionListener.wrap(authenticatedUserWithHash -> { - if (authenticatedUserWithHash != null && authenticatedUserWithHash.verify(token.credentials())) { - // cached credential hash matches the credential hash for this forestalled request - handleCachedAuthentication(authenticatedUserWithHash.user, ActionListener.wrap(cacheResult -> { - if (cacheResult.isAuthenticated()) { - logger.debug("realm [{}] authenticated user [{}], with roles [{}]", - name(), token.principal(), cacheResult.getUser().roles()); - } else { - logger.debug("realm [{}] authenticated user [{}] from cache, but then failed [{}]", - name(), token.principal(), cacheResult.getMessage()); - } - listener.onResponse(cacheResult); - }, listener::onFailure)); + listenableCacheEntry.addListener(ActionListener.wrap(cachedResult -> { + final boolean credsMatch = cachedResult.verify(token.credentials()); + if (cachedResult.authenticationResult.isAuthenticated()) { + if (credsMatch) { + // cached credential hash matches the credential hash for this forestalled request + handleCachedAuthentication(cachedResult.user, ActionListener.wrap(cacheResult -> { + if (cacheResult.isAuthenticated()) { + logger.debug("realm [{}] authenticated user [{}], with roles [{}]", + name(), token.principal(), cacheResult.getUser().roles()); + } else { + logger.debug("realm [{}] authenticated user [{}] from cache, but then failed [{}]", + name(), token.principal(), cacheResult.getMessage()); + } + listener.onResponse(cacheResult); + }, listener::onFailure)); + } else { + // its credential hash does not match the + // hash of the credential for this forestalled request. + // clear cache and try to reach the authentication source again because password + // might have changed there and the local cached hash got stale + cache.invalidate(token.principal(), listenableCacheEntry); + authenticateWithCache(token, listener); + } + } else if (credsMatch) { + // not authenticated but instead of hammering reuse the result. a new + // request will trigger a retried auth + listener.onResponse(cachedResult.authenticationResult); } else { - // The inflight request has failed or its credential hash does not match the - // hash of the credential for this forestalled request. - // clear cache and try to reach the authentication source again because password - // might have changed there and the local cached hash got stale cache.invalidate(token.principal(), listenableCacheEntry); authenticateWithCache(token, listener); } - }, e -> { - // the inflight request failed, so try again, but first (always) make sure cache - // is cleared of the failed authentication - cache.invalidate(token.principal(), listenableCacheEntry); - authenticateWithCache(token, listener); - }), threadPool.executor(ThreadPool.Names.GENERIC), threadPool.getThreadContext()); + }, listener::onFailure), threadPool.executor(ThreadPool.Names.GENERIC), threadPool.getThreadContext()); } else { // attempt authentication against the authentication source doAuthenticate(token, ActionListener.wrap(authResult -> { - if (authResult.isAuthenticated() && authResult.getUser().enabled()) { - // compute the credential hash of this successful authentication request - final UserWithHash userWithHash = new UserWithHash(authResult.getUser(), token.credentials(), cacheHasher); - // notify any forestalled request listeners; they will not reach to the - // authentication request and instead will use this hash for comparison - listenableCacheEntry.onResponse(userWithHash); - } else { - // notify any forestalled request listeners; they will retry the request - listenableCacheEntry.onResponse(null); + if (authResult.isAuthenticated() == false || authResult.getUser().enabled() == false) { + // a new request should trigger a new authentication + cache.invalidate(token.principal(), listenableCacheEntry); } - // notify the listener of the inflight authentication request; this request is not retried + // notify any forestalled request listeners; they will not reach to the + // authentication request and instead will use this result if they contain + // the same credentials + listenableCacheEntry.onResponse(new CachedResult(authResult, cacheHasher, authResult.getUser(), token.credentials())); listener.onResponse(authResult); }, e -> { - // notify any staved off listeners; they will retry the request + cache.invalidate(token.principal(), listenableCacheEntry); + // notify any staved off listeners; they will propagate this error listenableCacheEntry.onFailure(e); - // notify the listener of the inflight authentication request; this request is not retried + // notify the listener of the inflight authentication request listener.onFailure(e); })); } @@ -225,25 +229,21 @@ private void lookupWithCache(String username, ActionListener listener) { assert cache != null; try { final AtomicBoolean lookupInCache = new AtomicBoolean(true); - final ListenableFuture listenableCacheEntry = cache.computeIfAbsent(username, key -> { + final ListenableFuture listenableCacheEntry = cache.computeIfAbsent(username, key -> { lookupInCache.set(false); return new ListenableFuture<>(); }); if (false == lookupInCache.get()) { // attempt lookup against the user directory doLookupUser(username, ActionListener.wrap(user -> { - if (user != null) { - // user found - final UserWithHash userWithHash = new UserWithHash(user, null, null); - // notify forestalled request listeners - listenableCacheEntry.onResponse(userWithHash); - } else { + final CachedResult result = new CachedResult(AuthenticationResult.notHandled(), cacheHasher, user, null); + if (user == null) { // user not found, invalidate cache so that subsequent requests are forwarded to // the user directory cache.invalidate(username, listenableCacheEntry); - // notify forestalled request listeners - listenableCacheEntry.onResponse(null); } + // notify forestalled request listeners + listenableCacheEntry.onResponse(result); }, e -> { // the next request should be forwarded, not halted by a failed lookup attempt cache.invalidate(username, listenableCacheEntry); @@ -251,9 +251,9 @@ private void lookupWithCache(String username, ActionListener listener) { listenableCacheEntry.onFailure(e); })); } - listenableCacheEntry.addListener(ActionListener.wrap(userWithHash -> { - if (userWithHash != null) { - listener.onResponse(userWithHash.user); + listenableCacheEntry.addListener(ActionListener.wrap(cachedResult -> { + if (cachedResult.user != null) { + listener.onResponse(cachedResult.user); } else { listener.onResponse(null); } @@ -265,16 +265,21 @@ private void lookupWithCache(String username, ActionListener listener) { protected abstract void doLookupUser(String username, ActionListener listener); - private static class UserWithHash { - final User user; - final char[] hash; + private static class CachedResult { + private final AuthenticationResult authenticationResult; + private final User user; + private final char[] hash; - UserWithHash(User user, SecureString password, Hasher hasher) { - this.user = Objects.requireNonNull(user); + private CachedResult(AuthenticationResult result, Hasher hasher, @Nullable User user, @Nullable SecureString password) { + this.authenticationResult = Objects.requireNonNull(result); + if (authenticationResult.isAuthenticated() && user == null) { + throw new IllegalArgumentException("authentication cannot be successful with a null user"); + } + this.user = user; this.hash = password == null ? null : hasher.hash(password); } - boolean verify(SecureString password) { + private boolean verify(SecureString password) { return hash != null && Hasher.verifyHash(password, hash); } } 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 5bce054b42f62..4ed04864041d6 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 @@ -58,13 +58,13 @@ public void setup() { } @After - public void stop() throws InterruptedException { + public void stop() { if (threadPool != null) { terminate(threadPool); } } - public void testCacheSettings() throws Exception { + public void testCacheSettings() { String cachingHashAlgo = Hasher.values()[randomIntBetween(0, Hasher.values().length - 1)].name().toLowerCase(Locale.ROOT); int maxUsers = randomIntBetween(10, 100); TimeValue ttl = TimeValue.timeValueMinutes(randomIntBetween(10, 20)); @@ -352,7 +352,7 @@ private void sleepUntil(long until) throws InterruptedException { } } - public void testAuthenticateContract() throws Exception { + public void testAuthenticateContract() { Realm realm = new FailingAuthenticationRealm(globalSettings, threadPool); PlainActionFuture future = new PlainActionFuture<>(); realm.authenticate(new UsernamePasswordToken("user", new SecureString("pass")), future); @@ -366,7 +366,7 @@ public void testAuthenticateContract() throws Exception { assertThat(e.getMessage(), containsString("whatever exception")); } - public void testLookupContract() throws Exception { + public void testLookupContract() { Realm realm = new FailingAuthenticationRealm(globalSettings, threadPool); PlainActionFuture future = new PlainActionFuture<>(); realm.lookupUser("user", future); @@ -380,7 +380,7 @@ public void testLookupContract() throws Exception { assertThat(e.getMessage(), containsString("lookup exception")); } - public void testReturnDifferentObjectFromCache() throws Exception { + public void testReturnDifferentObjectFromCache() { final AtomicReference userArg = new AtomicReference<>(); final AtomicReference result = new AtomicReference<>(); Realm realm = new AlwaysAuthenticateCachingRealm(globalSettings, threadPool) { @@ -473,6 +473,71 @@ protected void doLookupUser(String username, ActionListener listener) { assertEquals(1, authCounter.get()); } + public void testUnauthenticatedResultPropagatesWithSameCreds() throws Exception { + final String username = "username"; + final SecureString password = SecuritySettingsSourceField.TEST_PASSWORD_SECURE_STRING; + final AtomicInteger authCounter = new AtomicInteger(0); + final Hasher pwdHasher = Hasher.resolve(randomFrom("pbkdf2", "pbkdf2_1000", "bcrypt", "bcrypt9")); + final String passwordHash = new String(pwdHasher.hash(password)); + RealmConfig config = new RealmConfig(new RealmConfig.RealmIdentifier("caching", "test_realm"), globalSettings, + TestEnvironment.newEnvironment(globalSettings), new ThreadContext(Settings.EMPTY)); + + 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()); + final CachingUsernamePasswordRealm realm = new CachingUsernamePasswordRealm(config, threadPool) { + @Override + protected void doAuthenticate(UsernamePasswordToken token, ActionListener listener) { + authCounter.incrementAndGet(); + // do something slow + if (pwdHasher.verify(token.credentials(), passwordHash.toCharArray())) { + listener.onFailure(new IllegalStateException("password auth should never succeed")); + } else { + listener.onResponse(AuthenticationResult.unsuccessful("password verification failed", null)); + } + } + + @Override + protected void doLookupUser(String username, ActionListener listener) { + listener.onFailure(new UnsupportedOperationException("this method should not be called")); + } + }; + for (int i = 0; i < numberOfThreads; i++) { + threads.add(new Thread(() -> { + try { + latch.countDown(); + latch.await(); + final UsernamePasswordToken token = new UsernamePasswordToken(username, credsToUse); + + realm.authenticate(token, ActionListener.wrap((result) -> { + if (result.isAuthenticated()) { + throw new IllegalStateException("invalid password led to an authenticated result: " + result); + } + assertThat(result.getMessage(), containsString("password verification failed")); + }, (e) -> { + logger.error("caught exception", e); + fail("unexpected exception - " + e); + })); + + } catch (InterruptedException e) { + logger.error("thread was interrupted", e); + Thread.currentThread().interrupt(); + } + })); + } + + for (Thread thread : threads) { + thread.start(); + } + latch.countDown(); + for (Thread thread : threads) { + thread.join(); + } + assertEquals(1, authCounter.get()); + } + public void testCacheConcurrency() throws Exception { final String username = "username"; final SecureString password = SecuritySettingsSourceField.TEST_PASSWORD_SECURE_STRING; @@ -704,27 +769,4 @@ protected void doLookupUser(String username, ActionListener listener) { listener.onResponse(new User(username, new String[]{"lookupRole1", "lookupRole2"})); } } - - static class LookupNotSupportedRealm extends CachingUsernamePasswordRealm { - - public final AtomicInteger authInvocationCounter = new AtomicInteger(0); - public final AtomicInteger lookupInvocationCounter = new AtomicInteger(0); - - LookupNotSupportedRealm(Settings globalSettings, ThreadPool threadPool) { - super(new RealmConfig(new RealmConfig.RealmIdentifier("caching", "lookup-notsupported-test"), globalSettings, - TestEnvironment.newEnvironment(globalSettings), threadPool.getThreadContext()), threadPool); - } - - @Override - protected void doAuthenticate(UsernamePasswordToken token, ActionListener listener) { - authInvocationCounter.incrementAndGet(); - listener.onResponse(AuthenticationResult.success(new User(token.principal(), new String[]{"testRole1", "testRole2"}))); - } - - @Override - protected void doLookupUser(String username, ActionListener listener) { - lookupInvocationCounter.incrementAndGet(); - listener.onFailure(new UnsupportedOperationException("don't call lookup if lookup isn't supported!!!")); - } - } } From d6608caf55af64dc74eeaf2f1aa67b6ef9ca34bf Mon Sep 17 00:00:00 2001 From: Martijn van Groningen Date: Tue, 8 Jan 2019 16:51:52 +0100 Subject: [PATCH 026/186] Muted rolling upgrade tests. Relates to #37231 --- .../src/test/java/org/elasticsearch/upgrades/CCRIT.java | 2 ++ 1 file changed, 2 insertions(+) diff --git a/x-pack/qa/rolling-upgrade/src/test/java/org/elasticsearch/upgrades/CCRIT.java b/x-pack/qa/rolling-upgrade/src/test/java/org/elasticsearch/upgrades/CCRIT.java index 45936c81a98dc..443fddce52d4a 100644 --- a/x-pack/qa/rolling-upgrade/src/test/java/org/elasticsearch/upgrades/CCRIT.java +++ b/x-pack/qa/rolling-upgrade/src/test/java/org/elasticsearch/upgrades/CCRIT.java @@ -8,6 +8,7 @@ import org.apache.http.util.EntityUtils; import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; +import org.apache.lucene.util.LuceneTestCase; import org.elasticsearch.Version; import org.elasticsearch.client.Request; import org.elasticsearch.client.Response; @@ -21,6 +22,7 @@ import static org.hamcrest.Matchers.equalTo; +@LuceneTestCase.AwaitsFix(bugUrl = "https://github.com/elastic/elasticsearch/issues/37231") public class CCRIT extends AbstractUpgradeTestCase { private static final Logger LOGGER = LogManager.getLogger(CCRIT.class); From 86b71930f6a01bf33171651ac951a70626f8a835 Mon Sep 17 00:00:00 2001 From: Ryan Ernst Date: Tue, 8 Jan 2019 11:04:54 -0800 Subject: [PATCH 027/186] Remove more checkstyle suppressions (#36735) This commit removes several more line length violations from checkstyle suppression. relates #34884 --- .../resources/checkstyle_suppressions.xml | 10 -- .../repositories/RepositoriesService.java | 103 +++++++------- .../repositories/Repository.java | 3 +- .../VerifyNodeRepositoryAction.java | 39 ++--- .../blobstore/BlobStoreRepository.java | 48 ++++--- .../repositories/fs/FsRepository.java | 3 +- .../rest/action/cat/RestIndicesAction.java | 134 ++++++++++++------ .../rest/action/cat/RestShardsAction.java | 28 ++-- .../rest/action/cat/RestThreadPoolAction.java | 3 +- .../explain/ExplainActionIT.java | 3 +- .../rest/BytesRestResponseTests.java | 8 +- 11 files changed, 228 insertions(+), 154 deletions(-) diff --git a/buildSrc/src/main/resources/checkstyle_suppressions.xml b/buildSrc/src/main/resources/checkstyle_suppressions.xml index 3233d66ddd58a..923e9e882b824 100644 --- a/buildSrc/src/main/resources/checkstyle_suppressions.xml +++ b/buildSrc/src/main/resources/checkstyle_suppressions.xml @@ -48,20 +48,10 @@ - - - - - - - - - - diff --git a/server/src/main/java/org/elasticsearch/repositories/RepositoriesService.java b/server/src/main/java/org/elasticsearch/repositories/RepositoriesService.java index 7ab6ae0a1f4c9..033080c2c38e6 100644 --- a/server/src/main/java/org/elasticsearch/repositories/RepositoriesService.java +++ b/server/src/main/java/org/elasticsearch/repositories/RepositoriesService.java @@ -111,62 +111,63 @@ public void registerRepository(final RegisterRepositoryRequest request, final Ac return; } - clusterService.submitStateUpdateTask(request.cause, new AckedClusterStateUpdateTask(request, registrationListener) { - @Override - protected ClusterStateUpdateResponse newResponse(boolean acknowledged) { - return new ClusterStateUpdateResponse(acknowledged); - } - - @Override - public ClusterState execute(ClusterState currentState) { - ensureRepositoryNotInUse(currentState, request.name); - MetaData metaData = currentState.metaData(); - MetaData.Builder mdBuilder = MetaData.builder(currentState.metaData()); - RepositoriesMetaData repositories = metaData.custom(RepositoriesMetaData.TYPE); - if (repositories == null) { - logger.info("put repository [{}]", request.name); - repositories = new RepositoriesMetaData( - Collections.singletonList(new RepositoryMetaData(request.name, request.type, request.settings))); - } else { - boolean found = false; - List repositoriesMetaData = new ArrayList<>(repositories.repositories().size() + 1); + clusterService.submitStateUpdateTask(request.cause, + new AckedClusterStateUpdateTask(request, registrationListener) { + @Override + protected ClusterStateUpdateResponse newResponse(boolean acknowledged) { + return new ClusterStateUpdateResponse(acknowledged); + } - for (RepositoryMetaData repositoryMetaData : repositories.repositories()) { - if (repositoryMetaData.name().equals(newRepositoryMetaData.name())) { - if (newRepositoryMetaData.equals(repositoryMetaData)) { - // Previous version is the same as this one no update is needed. - return currentState; + @Override + public ClusterState execute(ClusterState currentState) { + ensureRepositoryNotInUse(currentState, request.name); + MetaData metaData = currentState.metaData(); + MetaData.Builder mdBuilder = MetaData.builder(currentState.metaData()); + RepositoriesMetaData repositories = metaData.custom(RepositoriesMetaData.TYPE); + if (repositories == null) { + logger.info("put repository [{}]", request.name); + repositories = new RepositoriesMetaData( + Collections.singletonList(new RepositoryMetaData(request.name, request.type, request.settings))); + } else { + boolean found = false; + List repositoriesMetaData = new ArrayList<>(repositories.repositories().size() + 1); + + for (RepositoryMetaData repositoryMetaData : repositories.repositories()) { + if (repositoryMetaData.name().equals(newRepositoryMetaData.name())) { + if (newRepositoryMetaData.equals(repositoryMetaData)) { + // Previous version is the same as this one no update is needed. + return currentState; + } + found = true; + repositoriesMetaData.add(newRepositoryMetaData); + } else { + repositoriesMetaData.add(repositoryMetaData); } - found = true; - repositoriesMetaData.add(newRepositoryMetaData); + } + if (!found) { + logger.info("put repository [{}]", request.name); + repositoriesMetaData.add(new RepositoryMetaData(request.name, request.type, request.settings)); } else { - repositoriesMetaData.add(repositoryMetaData); + logger.info("update repository [{}]", request.name); } + repositories = new RepositoriesMetaData(repositoriesMetaData); } - if (!found) { - logger.info("put repository [{}]", request.name); - repositoriesMetaData.add(new RepositoryMetaData(request.name, request.type, request.settings)); - } else { - logger.info("update repository [{}]", request.name); - } - repositories = new RepositoriesMetaData(repositoriesMetaData); + mdBuilder.putCustom(RepositoriesMetaData.TYPE, repositories); + return ClusterState.builder(currentState).metaData(mdBuilder).build(); } - mdBuilder.putCustom(RepositoriesMetaData.TYPE, repositories); - return ClusterState.builder(currentState).metaData(mdBuilder).build(); - } - @Override - public void onFailure(String source, Exception e) { - logger.warn(() -> new ParameterizedMessage("failed to create repository [{}]", request.name), e); - super.onFailure(source, e); - } + @Override + public void onFailure(String source, Exception e) { + logger.warn(() -> new ParameterizedMessage("failed to create repository [{}]", request.name), e); + super.onFailure(source, e); + } - @Override - public boolean mustAck(DiscoveryNode discoveryNode) { - // repository is created on both master and data nodes - return discoveryNode.isMasterNode() || discoveryNode.isDataNode(); - } - }); + @Override + public boolean mustAck(DiscoveryNode discoveryNode) { + // repository is created on both master and data nodes + return discoveryNode.isMasterNode() || discoveryNode.isDataNode(); + } + }); } /** * Unregisters repository in the cluster @@ -323,7 +324,8 @@ public void applyClusterState(ClusterChangedEvent event) { } catch (RepositoryException ex) { // TODO: this catch is bogus, it means the old repo is already closed, // but we have nothing to replace it - logger.warn(() -> new ParameterizedMessage("failed to change repository [{}]", repositoryMetaData.name()), ex); + logger.warn(() -> new ParameterizedMessage("failed to change repository [{}]", + repositoryMetaData.name()), ex); } } } else { @@ -411,7 +413,8 @@ private Repository createRepository(RepositoryMetaData repositoryMetaData, Map listener) { @@ -90,28 +92,31 @@ public void verify(String repository, String verificationToken, final ActionList finishVerification(listener, nodes, errors); } } else { - transportService.sendRequest(node, ACTION_NAME, new VerifyNodeRepositoryRequest(repository, verificationToken), new EmptyTransportResponseHandler(ThreadPool.Names.SAME) { - @Override - public void handleResponse(TransportResponse.Empty response) { - if (counter.decrementAndGet() == 0) { - finishVerification(listener, nodes, errors); + transportService.sendRequest(node, ACTION_NAME, new VerifyNodeRepositoryRequest(repository, verificationToken), + new EmptyTransportResponseHandler(ThreadPool.Names.SAME) { + @Override + public void handleResponse(TransportResponse.Empty response) { + if (counter.decrementAndGet() == 0) { + finishVerification(listener, nodes, errors); + } } - } - @Override - public void handleException(TransportException exp) { - errors.add(new VerificationFailure(node.getId(), exp)); - if (counter.decrementAndGet() == 0) { - finishVerification(listener, nodes, errors); + @Override + public void handleException(TransportException exp) { + errors.add(new VerificationFailure(node.getId(), exp)); + if (counter.decrementAndGet() == 0) { + finishVerification(listener, nodes, errors); + } } - } - }); + }); } } } - public void finishVerification(ActionListener listener, List nodes, CopyOnWriteArrayList errors) { - listener.onResponse(new RepositoriesService.VerifyResponse(nodes.toArray(new DiscoveryNode[nodes.size()]), errors.toArray(new VerificationFailure[errors.size()]))); + public void finishVerification(ActionListener listener, List nodes, + CopyOnWriteArrayList errors) { + listener.onResponse(new RepositoriesService.VerifyResponse(nodes.toArray(new DiscoveryNode[nodes.size()]), + errors.toArray(new VerificationFailure[errors.size()]))); } private void doVerify(String repositoryName, String verificationToken, DiscoveryNode localNode) { diff --git a/server/src/main/java/org/elasticsearch/repositories/blobstore/BlobStoreRepository.java b/server/src/main/java/org/elasticsearch/repositories/blobstore/BlobStoreRepository.java index 09eb557fe9cad..99cb1db3e3652 100644 --- a/server/src/main/java/org/elasticsearch/repositories/blobstore/BlobStoreRepository.java +++ b/server/src/main/java/org/elasticsearch/repositories/blobstore/BlobStoreRepository.java @@ -479,12 +479,12 @@ public void deleteSnapshot(SnapshotId snapshotId, long repositoryStateId) { // we'll ignore that and accept that cleanup didn't fully succeed. // since we are using UUIDs for path names, this won't be an issue for // snapshotting indices of the same name - logger.debug(() -> new ParameterizedMessage("[{}] index [{}] no longer part of any snapshots in the repository, but failed to clean up " + - "its index folder due to the directory not being empty.", metadata.name(), indexId), dnee); + logger.debug(() -> new ParameterizedMessage("[{}] index [{}] no longer part of any snapshots in the repository, " + + "but failed to clean up its index folder due to the directory not being empty.", metadata.name(), indexId), dnee); } catch (IOException ioe) { // a different IOException occurred while trying to delete - will just log the issue for now - logger.debug(() -> new ParameterizedMessage("[{}] index [{}] no longer part of any snapshots in the repository, but failed to clean up " + - "its index folder.", metadata.name(), indexId), ioe); + logger.debug(() -> new ParameterizedMessage("[{}] index [{}] no longer part of any snapshots in the repository, " + + "but failed to clean up its index folder.", metadata.name(), indexId), ioe); } } } catch (IOException | ResourceNotFoundException ex) { @@ -524,7 +524,8 @@ private void deleteIndexMetaDataBlobIgnoringErrors(final SnapshotInfo snapshotIn try { indexMetaDataFormat.delete(indexMetaDataBlobContainer, snapshotId.getUUID()); } catch (IOException ex) { - logger.warn(() -> new ParameterizedMessage("[{}] failed to delete metadata for index [{}]", snapshotId, indexId.getName()), ex); + logger.warn(() -> new ParameterizedMessage("[{}] failed to delete metadata for index [{}]", + snapshotId, indexId.getName()), ex); } } @@ -861,7 +862,8 @@ public void snapshotShard(IndexShard shard, Store store, SnapshotId snapshotId, } @Override - public void restoreShard(IndexShard shard, SnapshotId snapshotId, Version version, IndexId indexId, ShardId snapshotShardId, RecoveryState recoveryState) { + public void restoreShard(IndexShard shard, SnapshotId snapshotId, Version version, IndexId indexId, ShardId snapshotShardId, + RecoveryState recoveryState) { final RestoreContext snapshotContext = new RestoreContext(shard, snapshotId, indexId, snapshotShardId, recoveryState); try { snapshotContext.restore(); @@ -898,12 +900,14 @@ public void verify(String seed, DiscoveryNode localNode) { testBlobContainer.writeBlob("data-" + localNode.getId() + ".dat", stream, bytes.length(), true); } } catch (IOException exp) { - throw new RepositoryVerificationException(metadata.name(), "store location [" + blobStore() + "] is not accessible on the node [" + localNode + "]", exp); + throw new RepositoryVerificationException(metadata.name(), "store location [" + blobStore() + + "] is not accessible on the node [" + localNode + "]", exp); } } else { - throw new RepositoryVerificationException(metadata.name(), "a file written by master to the store [" + blobStore() + "] cannot be accessed on the node [" + localNode + "]. " - + "This might indicate that the store [" + blobStore() + "] is not shared between this node and the master node or " - + "that permissions on the store don't allow reading files written by the master node"); + throw new RepositoryVerificationException(metadata.name(), "a file written by master to the store [" + blobStore() + + "] cannot be accessed on the node [" + localNode + "]. " + + "This might indicate that the store [" + blobStore() + "] is not shared between this node and the master node or " + + "that permissions on the store don't allow reading files written by the master node"); } } } @@ -945,7 +949,8 @@ private class Context { Context(SnapshotId snapshotId, IndexId indexId, ShardId shardId, ShardId snapshotShardId) { this.snapshotId = snapshotId; this.shardId = shardId; - blobContainer = blobStore().blobContainer(basePath().add("indices").add(indexId.getId()).add(Integer.toString(snapshotShardId.getId()))); + blobContainer = blobStore().blobContainer(basePath().add("indices").add(indexId.getId()) + .add(Integer.toString(snapshotShardId.getId()))); } /** @@ -1235,7 +1240,8 @@ public void snapshot(final IndexCommit snapshotIndexCommit) { // in a bwc compatible way. maybeRecalculateMetadataHash(blobContainer, fileInfo, metadata); } catch (Exception e) { - logger.warn(() -> new ParameterizedMessage("{} Can't calculate hash from blob for file [{}] [{}]", shardId, fileInfo.physicalName(), fileInfo.metadata()), e); + logger.warn(() -> new ParameterizedMessage("{} Can't calculate hash from blob for file [{}] [{}]", + shardId, fileInfo.physicalName(), fileInfo.metadata()), e); } if (fileInfo.isSame(md) && snapshotFileExistsInBlobs(fileInfo, blobs)) { // a commit point file with the same name, size and checksum was already copied to repository @@ -1253,7 +1259,8 @@ public void snapshot(final IndexCommit snapshotIndexCommit) { indexIncrementalFileCount++; indexIncrementalSize += md.length(); // create a new FileInfo - BlobStoreIndexShardSnapshot.FileInfo snapshotFileInfo = new BlobStoreIndexShardSnapshot.FileInfo(fileNameFromGeneration(++generation), md, chunkSize()); + BlobStoreIndexShardSnapshot.FileInfo snapshotFileInfo = + new BlobStoreIndexShardSnapshot.FileInfo(fileNameFromGeneration(++generation), md, chunkSize()); indexCommitPointFiles.add(snapshotFileInfo); filesToSnapshot.add(snapshotFileInfo); } else { @@ -1411,7 +1418,8 @@ private void checkAborted() { * The new logic for StoreFileMetaData reads the entire {@code .si} and {@code segments.n} files to strengthen the * comparison of the files on a per-segment / per-commit level. */ - private static void maybeRecalculateMetadataHash(final BlobContainer blobContainer, final BlobStoreIndexShardSnapshot.FileInfo fileInfo, Store.MetadataSnapshot snapshot) throws Exception { + private static void maybeRecalculateMetadataHash(final BlobContainer blobContainer, final BlobStoreIndexShardSnapshot.FileInfo fileInfo, + Store.MetadataSnapshot snapshot) throws Exception { final StoreFileMetaData metadata; if (fileInfo != null && (metadata = snapshot.get(fileInfo.physicalName())) != null) { if (metadata.hash().length > 0 && fileInfo.metadata().hash().length == 0) { @@ -1509,7 +1517,8 @@ public void restore() throws IOException { logger.trace("[{}] [{}] restoring from to an empty shard", shardId, snapshotId); recoveryTargetMetadata = Store.MetadataSnapshot.EMPTY; } catch (IOException e) { - logger.warn(() -> new ParameterizedMessage("{} Can't read metadata from store, will not reuse any local file while restoring", shardId), e); + logger.warn(() -> new ParameterizedMessage("{} Can't read metadata from store, will not reuse any " + + "local file while restoring", shardId), e); recoveryTargetMetadata = Store.MetadataSnapshot.EMPTY; } @@ -1525,7 +1534,8 @@ public void restore() throws IOException { maybeRecalculateMetadataHash(blobContainer, fileInfo, recoveryTargetMetadata); } catch (Exception e) { // if the index is broken we might not be able to read it - logger.warn(() -> new ParameterizedMessage("{} Can't calculate hash from blog for file [{}] [{}]", shardId, fileInfo.physicalName(), fileInfo.metadata()), e); + logger.warn(() -> new ParameterizedMessage("{} Can't calculate hash from blog for file [{}] [{}]", + shardId, fileInfo.physicalName(), fileInfo.metadata()), e); } snapshotMetaData.put(fileInfo.metadata().name(), fileInfo.metadata()); fileInfos.put(fileInfo.metadata().name(), fileInfo); @@ -1543,7 +1553,8 @@ public void restore() throws IOException { BlobStoreIndexShardSnapshot.FileInfo fileInfo = fileInfos.get(md.name()); recoveryState.getIndex().addFileDetail(fileInfo.name(), fileInfo.length(), true); if (logger.isTraceEnabled()) { - logger.trace("[{}] [{}] not_recovering [{}] from [{}], exists in local store and is same", shardId, snapshotId, fileInfo.physicalName(), fileInfo.name()); + logger.trace("[{}] [{}] not_recovering [{}] from [{}], exists in local store and is same", + shardId, snapshotId, fileInfo.physicalName(), fileInfo.name()); } } @@ -1634,7 +1645,8 @@ private void restoreFile(final BlobStoreIndexShardSnapshot.FileInfo fileInfo, fi stream = new RateLimitingInputStream(partSliceStream, restoreRateLimiter, restoreRateLimitingTimeInNanos::inc); } - try (IndexOutput indexOutput = store.createVerifyingOutput(fileInfo.physicalName(), fileInfo.metadata(), IOContext.DEFAULT)) { + try (IndexOutput indexOutput = store.createVerifyingOutput(fileInfo.physicalName(), + fileInfo.metadata(), IOContext.DEFAULT)) { final byte[] buffer = new byte[BUFFER_SIZE]; int length; while ((length = stream.read(buffer)) > 0) { diff --git a/server/src/main/java/org/elasticsearch/repositories/fs/FsRepository.java b/server/src/main/java/org/elasticsearch/repositories/fs/FsRepository.java index 3db5195097145..01c08fbce0044 100644 --- a/server/src/main/java/org/elasticsearch/repositories/fs/FsRepository.java +++ b/server/src/main/java/org/elasticsearch/repositories/fs/FsRepository.java @@ -43,7 +43,8 @@ *

*
{@code location}
Path to the root of repository. This is mandatory parameter.
*
{@code concurrent_streams}
Number of concurrent read/write stream (per repository on each node). Defaults to 5.
- *
{@code chunk_size}
Large file can be divided into chunks. This parameter specifies the chunk size. Defaults to not chucked.
+ *
{@code chunk_size}
Large file can be divided into chunks. This parameter specifies the chunk size. + * Defaults to not chucked.
*
{@code compress}
If set to true metadata files will be stored compressed. Defaults to false.
*
*/ diff --git a/server/src/main/java/org/elasticsearch/rest/action/cat/RestIndicesAction.java b/server/src/main/java/org/elasticsearch/rest/action/cat/RestIndicesAction.java index fc2fa9a34e5f6..676f2bbdc7b2e 100644 --- a/server/src/main/java/org/elasticsearch/rest/action/cat/RestIndicesAction.java +++ b/server/src/main/java/org/elasticsearch/rest/action/cat/RestIndicesAction.java @@ -122,7 +122,8 @@ public void processResponse(final ClusterHealthResponse clusterHealthResponse) { client.admin().indices().stats(indicesStatsRequest, new RestResponseListener(channel) { @Override public RestResponse buildResponse(IndicesStatsResponse indicesStatsResponse) throws Exception { - Table tab = buildTable(request, concreteIndices, clusterHealthResponse, indicesStatsResponse, state.metaData()); + Table tab = buildTable(request, concreteIndices, clusterHealthResponse, + indicesStatsResponse, state.metaData()); return RestTable.buildResponse(tab, channel); } }); @@ -168,28 +169,36 @@ protected Table getTableWithHeader(final RestRequest request) { table.addCell("completion.size", "sibling:pri;alias:cs,completionSize;default:false;text-align:right;desc:size of completion"); table.addCell("pri.completion.size", "default:false;text-align:right;desc:size of completion"); - table.addCell("fielddata.memory_size", "sibling:pri;alias:fm,fielddataMemory;default:false;text-align:right;desc:used fielddata cache"); + table.addCell("fielddata.memory_size", + "sibling:pri;alias:fm,fielddataMemory;default:false;text-align:right;desc:used fielddata cache"); table.addCell("pri.fielddata.memory_size", "default:false;text-align:right;desc:used fielddata cache"); - table.addCell("fielddata.evictions", "sibling:pri;alias:fe,fielddataEvictions;default:false;text-align:right;desc:fielddata evictions"); + table.addCell("fielddata.evictions", + "sibling:pri;alias:fe,fielddataEvictions;default:false;text-align:right;desc:fielddata evictions"); table.addCell("pri.fielddata.evictions", "default:false;text-align:right;desc:fielddata evictions"); - table.addCell("query_cache.memory_size", "sibling:pri;alias:qcm,queryCacheMemory;default:false;text-align:right;desc:used query cache"); + table.addCell("query_cache.memory_size", + "sibling:pri;alias:qcm,queryCacheMemory;default:false;text-align:right;desc:used query cache"); table.addCell("pri.query_cache.memory_size", "default:false;text-align:right;desc:used query cache"); - table.addCell("query_cache.evictions", "sibling:pri;alias:qce,queryCacheEvictions;default:false;text-align:right;desc:query cache evictions"); + table.addCell("query_cache.evictions", + "sibling:pri;alias:qce,queryCacheEvictions;default:false;text-align:right;desc:query cache evictions"); table.addCell("pri.query_cache.evictions", "default:false;text-align:right;desc:query cache evictions"); - table.addCell("request_cache.memory_size", "sibling:pri;alias:rcm,requestCacheMemory;default:false;text-align:right;desc:used request cache"); + table.addCell("request_cache.memory_size", + "sibling:pri;alias:rcm,requestCacheMemory;default:false;text-align:right;desc:used request cache"); table.addCell("pri.request_cache.memory_size", "default:false;text-align:right;desc:used request cache"); - table.addCell("request_cache.evictions", "sibling:pri;alias:rce,requestCacheEvictions;default:false;text-align:right;desc:request cache evictions"); + table.addCell("request_cache.evictions", + "sibling:pri;alias:rce,requestCacheEvictions;default:false;text-align:right;desc:request cache evictions"); table.addCell("pri.request_cache.evictions", "default:false;text-align:right;desc:request cache evictions"); - table.addCell("request_cache.hit_count", "sibling:pri;alias:rchc,requestCacheHitCount;default:false;text-align:right;desc:request cache hit count"); + table.addCell("request_cache.hit_count", + "sibling:pri;alias:rchc,requestCacheHitCount;default:false;text-align:right;desc:request cache hit count"); table.addCell("pri.request_cache.hit_count", "default:false;text-align:right;desc:request cache hit count"); - table.addCell("request_cache.miss_count", "sibling:pri;alias:rcmc,requestCacheMissCount;default:false;text-align:right;desc:request cache miss count"); + table.addCell("request_cache.miss_count", + "sibling:pri;alias:rcmc,requestCacheMissCount;default:false;text-align:right;desc:request cache miss count"); table.addCell("pri.request_cache.miss_count", "default:false;text-align:right;desc:request cache miss count"); table.addCell("flush.total", "sibling:pri;alias:ft,flushTotal;default:false;text-align:right;desc:number of flushes"); @@ -207,49 +216,64 @@ protected Table getTableWithHeader(final RestRequest request) { table.addCell("get.total", "sibling:pri;alias:gto,getTotal;default:false;text-align:right;desc:number of get ops"); table.addCell("pri.get.total", "default:false;text-align:right;desc:number of get ops"); - table.addCell("get.exists_time", "sibling:pri;alias:geti,getExistsTime;default:false;text-align:right;desc:time spent in successful gets"); + table.addCell("get.exists_time", + "sibling:pri;alias:geti,getExistsTime;default:false;text-align:right;desc:time spent in successful gets"); table.addCell("pri.get.exists_time", "default:false;text-align:right;desc:time spent in successful gets"); - table.addCell("get.exists_total", "sibling:pri;alias:geto,getExistsTotal;default:false;text-align:right;desc:number of successful gets"); + table.addCell("get.exists_total", + "sibling:pri;alias:geto,getExistsTotal;default:false;text-align:right;desc:number of successful gets"); table.addCell("pri.get.exists_total", "default:false;text-align:right;desc:number of successful gets"); - table.addCell("get.missing_time", "sibling:pri;alias:gmti,getMissingTime;default:false;text-align:right;desc:time spent in failed gets"); + table.addCell("get.missing_time", + "sibling:pri;alias:gmti,getMissingTime;default:false;text-align:right;desc:time spent in failed gets"); table.addCell("pri.get.missing_time", "default:false;text-align:right;desc:time spent in failed gets"); - table.addCell("get.missing_total", "sibling:pri;alias:gmto,getMissingTotal;default:false;text-align:right;desc:number of failed gets"); + table.addCell("get.missing_total", + "sibling:pri;alias:gmto,getMissingTotal;default:false;text-align:right;desc:number of failed gets"); table.addCell("pri.get.missing_total", "default:false;text-align:right;desc:number of failed gets"); - table.addCell("indexing.delete_current", "sibling:pri;alias:idc,indexingDeleteCurrent;default:false;text-align:right;desc:number of current deletions"); + table.addCell("indexing.delete_current", + "sibling:pri;alias:idc,indexingDeleteCurrent;default:false;text-align:right;desc:number of current deletions"); table.addCell("pri.indexing.delete_current", "default:false;text-align:right;desc:number of current deletions"); - table.addCell("indexing.delete_time", "sibling:pri;alias:idti,indexingDeleteTime;default:false;text-align:right;desc:time spent in deletions"); + table.addCell("indexing.delete_time", + "sibling:pri;alias:idti,indexingDeleteTime;default:false;text-align:right;desc:time spent in deletions"); table.addCell("pri.indexing.delete_time", "default:false;text-align:right;desc:time spent in deletions"); - table.addCell("indexing.delete_total", "sibling:pri;alias:idto,indexingDeleteTotal;default:false;text-align:right;desc:number of delete ops"); + table.addCell("indexing.delete_total", + "sibling:pri;alias:idto,indexingDeleteTotal;default:false;text-align:right;desc:number of delete ops"); table.addCell("pri.indexing.delete_total", "default:false;text-align:right;desc:number of delete ops"); - table.addCell("indexing.index_current", "sibling:pri;alias:iic,indexingIndexCurrent;default:false;text-align:right;desc:number of current indexing ops"); + table.addCell("indexing.index_current", + "sibling:pri;alias:iic,indexingIndexCurrent;default:false;text-align:right;desc:number of current indexing ops"); table.addCell("pri.indexing.index_current", "default:false;text-align:right;desc:number of current indexing ops"); - table.addCell("indexing.index_time", "sibling:pri;alias:iiti,indexingIndexTime;default:false;text-align:right;desc:time spent in indexing"); + table.addCell("indexing.index_time", + "sibling:pri;alias:iiti,indexingIndexTime;default:false;text-align:right;desc:time spent in indexing"); table.addCell("pri.indexing.index_time", "default:false;text-align:right;desc:time spent in indexing"); - table.addCell("indexing.index_total", "sibling:pri;alias:iito,indexingIndexTotal;default:false;text-align:right;desc:number of indexing ops"); + table.addCell("indexing.index_total", + "sibling:pri;alias:iito,indexingIndexTotal;default:false;text-align:right;desc:number of indexing ops"); table.addCell("pri.indexing.index_total", "default:false;text-align:right;desc:number of indexing ops"); - table.addCell("indexing.index_failed", "sibling:pri;alias:iif,indexingIndexFailed;default:false;text-align:right;desc:number of failed indexing ops"); + table.addCell("indexing.index_failed", + "sibling:pri;alias:iif,indexingIndexFailed;default:false;text-align:right;desc:number of failed indexing ops"); table.addCell("pri.indexing.index_failed", "default:false;text-align:right;desc:number of failed indexing ops"); - table.addCell("merges.current", "sibling:pri;alias:mc,mergesCurrent;default:false;text-align:right;desc:number of current merges"); + table.addCell("merges.current", + "sibling:pri;alias:mc,mergesCurrent;default:false;text-align:right;desc:number of current merges"); table.addCell("pri.merges.current", "default:false;text-align:right;desc:number of current merges"); - table.addCell("merges.current_docs", "sibling:pri;alias:mcd,mergesCurrentDocs;default:false;text-align:right;desc:number of current merging docs"); + table.addCell("merges.current_docs", + "sibling:pri;alias:mcd,mergesCurrentDocs;default:false;text-align:right;desc:number of current merging docs"); table.addCell("pri.merges.current_docs", "default:false;text-align:right;desc:number of current merging docs"); - table.addCell("merges.current_size", "sibling:pri;alias:mcs,mergesCurrentSize;default:false;text-align:right;desc:size of current merges"); + table.addCell("merges.current_size", + "sibling:pri;alias:mcs,mergesCurrentSize;default:false;text-align:right;desc:size of current merges"); table.addCell("pri.merges.current_size", "default:false;text-align:right;desc:size of current merges"); - table.addCell("merges.total", "sibling:pri;alias:mt,mergesTotal;default:false;text-align:right;desc:number of completed merge ops"); + table.addCell("merges.total", + "sibling:pri;alias:mt,mergesTotal;default:false;text-align:right;desc:number of completed merge ops"); table.addCell("pri.merges.total", "default:false;text-align:right;desc:number of completed merge ops"); table.addCell("merges.total_docs", "sibling:pri;alias:mtd,mergesTotalDocs;default:false;text-align:right;desc:docs merged"); @@ -258,7 +282,8 @@ protected Table getTableWithHeader(final RestRequest request) { table.addCell("merges.total_size", "sibling:pri;alias:mts,mergesTotalSize;default:false;text-align:right;desc:size merged"); table.addCell("pri.merges.total_size", "default:false;text-align:right;desc:size merged"); - table.addCell("merges.total_time", "sibling:pri;alias:mtt,mergesTotalTime;default:false;text-align:right;desc:time spent in merges"); + table.addCell("merges.total_time", + "sibling:pri;alias:mtt,mergesTotalTime;default:false;text-align:right;desc:time spent in merges"); table.addCell("pri.merges.total_time", "default:false;text-align:right;desc:time spent in merges"); table.addCell("refresh.total", "sibling:pri;alias:rto,refreshTotal;default:false;text-align:right;desc:total refreshes"); @@ -267,37 +292,48 @@ protected Table getTableWithHeader(final RestRequest request) { table.addCell("refresh.time", "sibling:pri;alias:rti,refreshTime;default:false;text-align:right;desc:time spent in refreshes"); table.addCell("pri.refresh.time", "default:false;text-align:right;desc:time spent in refreshes"); - table.addCell("refresh.listeners", "sibling:pri;alias:rli,refreshListeners;default:false;text-align:right;desc:number of pending refresh listeners"); + table.addCell("refresh.listeners", + "sibling:pri;alias:rli,refreshListeners;default:false;text-align:right;desc:number of pending refresh listeners"); table.addCell("pri.refresh.listeners", "default:false;text-align:right;desc:number of pending refresh listeners"); - table.addCell("search.fetch_current", "sibling:pri;alias:sfc,searchFetchCurrent;default:false;text-align:right;desc:current fetch phase ops"); + table.addCell("search.fetch_current", + "sibling:pri;alias:sfc,searchFetchCurrent;default:false;text-align:right;desc:current fetch phase ops"); table.addCell("pri.search.fetch_current", "default:false;text-align:right;desc:current fetch phase ops"); - table.addCell("search.fetch_time", "sibling:pri;alias:sfti,searchFetchTime;default:false;text-align:right;desc:time spent in fetch phase"); + table.addCell("search.fetch_time", + "sibling:pri;alias:sfti,searchFetchTime;default:false;text-align:right;desc:time spent in fetch phase"); table.addCell("pri.search.fetch_time", "default:false;text-align:right;desc:time spent in fetch phase"); - table.addCell("search.fetch_total", "sibling:pri;alias:sfto,searchFetchTotal;default:false;text-align:right;desc:total fetch ops"); + table.addCell("search.fetch_total", + "sibling:pri;alias:sfto,searchFetchTotal;default:false;text-align:right;desc:total fetch ops"); table.addCell("pri.search.fetch_total", "default:false;text-align:right;desc:total fetch ops"); - table.addCell("search.open_contexts", "sibling:pri;alias:so,searchOpenContexts;default:false;text-align:right;desc:open search contexts"); + table.addCell("search.open_contexts", + "sibling:pri;alias:so,searchOpenContexts;default:false;text-align:right;desc:open search contexts"); table.addCell("pri.search.open_contexts", "default:false;text-align:right;desc:open search contexts"); - table.addCell("search.query_current", "sibling:pri;alias:sqc,searchQueryCurrent;default:false;text-align:right;desc:current query phase ops"); + table.addCell("search.query_current", + "sibling:pri;alias:sqc,searchQueryCurrent;default:false;text-align:right;desc:current query phase ops"); table.addCell("pri.search.query_current", "default:false;text-align:right;desc:current query phase ops"); - table.addCell("search.query_time", "sibling:pri;alias:sqti,searchQueryTime;default:false;text-align:right;desc:time spent in query phase"); + table.addCell("search.query_time", + "sibling:pri;alias:sqti,searchQueryTime;default:false;text-align:right;desc:time spent in query phase"); table.addCell("pri.search.query_time", "default:false;text-align:right;desc:time spent in query phase"); - table.addCell("search.query_total", "sibling:pri;alias:sqto,searchQueryTotal;default:false;text-align:right;desc:total query phase ops"); + table.addCell("search.query_total", + "sibling:pri;alias:sqto,searchQueryTotal;default:false;text-align:right;desc:total query phase ops"); table.addCell("pri.search.query_total", "default:false;text-align:right;desc:total query phase ops"); - table.addCell("search.scroll_current", "sibling:pri;alias:scc,searchScrollCurrent;default:false;text-align:right;desc:open scroll contexts"); + table.addCell("search.scroll_current", + "sibling:pri;alias:scc,searchScrollCurrent;default:false;text-align:right;desc:open scroll contexts"); table.addCell("pri.search.scroll_current", "default:false;text-align:right;desc:open scroll contexts"); - table.addCell("search.scroll_time", "sibling:pri;alias:scti,searchScrollTime;default:false;text-align:right;desc:time scroll contexts held open"); + table.addCell("search.scroll_time", + "sibling:pri;alias:scti,searchScrollTime;default:false;text-align:right;desc:time scroll contexts held open"); table.addCell("pri.search.scroll_time", "default:false;text-align:right;desc:time scroll contexts held open"); - table.addCell("search.scroll_total", "sibling:pri;alias:scto,searchScrollTotal;default:false;text-align:right;desc:completed scroll contexts"); + table.addCell("search.scroll_total", + "sibling:pri;alias:scto,searchScrollTotal;default:false;text-align:right;desc:completed scroll contexts"); table.addCell("pri.search.scroll_total", "default:false;text-align:right;desc:completed scroll contexts"); table.addCell("segments.count", "sibling:pri;alias:sc,segmentsCount;default:false;text-align:right;desc:number of segments"); @@ -306,14 +342,20 @@ protected Table getTableWithHeader(final RestRequest request) { table.addCell("segments.memory", "sibling:pri;alias:sm,segmentsMemory;default:false;text-align:right;desc:memory used by segments"); table.addCell("pri.segments.memory", "default:false;text-align:right;desc:memory used by segments"); - table.addCell("segments.index_writer_memory", "sibling:pri;alias:siwm,segmentsIndexWriterMemory;default:false;text-align:right;desc:memory used by index writer"); + table.addCell("segments.index_writer_memory", + "sibling:pri;alias:siwm,segmentsIndexWriterMemory;default:false;text-align:right;desc:memory used by index writer"); table.addCell("pri.segments.index_writer_memory", "default:false;text-align:right;desc:memory used by index writer"); - table.addCell("segments.version_map_memory", "sibling:pri;alias:svmm,segmentsVersionMapMemory;default:false;text-align:right;desc:memory used by version map"); + table.addCell("segments.version_map_memory", + "sibling:pri;alias:svmm,segmentsVersionMapMemory;default:false;text-align:right;desc:memory used by version map"); table.addCell("pri.segments.version_map_memory", "default:false;text-align:right;desc:memory used by version map"); - table.addCell("segments.fixed_bitset_memory", "sibling:pri;alias:sfbm,fixedBitsetMemory;default:false;text-align:right;desc:memory used by fixed bit sets for nested object field types and type filters for types referred in _parent fields"); - table.addCell("pri.segments.fixed_bitset_memory", "default:false;text-align:right;desc:memory used by fixed bit sets for nested object field types and type filters for types referred in _parent fields"); + table.addCell("segments.fixed_bitset_memory", + "sibling:pri;alias:sfbm,fixedBitsetMemory;default:false;text-align:right;desc:memory used by fixed bit sets for" + + " nested object field types and type filters for types referred in _parent fields"); + table.addCell("pri.segments.fixed_bitset_memory", + "default:false;text-align:right;desc:memory used by fixed bit sets for nested object" + + " field types and type filters for types referred in _parent fields"); table.addCell("warmer.current", "sibling:pri;alias:wc,warmerCurrent;default:false;text-align:right;desc:current warmer ops"); table.addCell("pri.warmer.current", "default:false;text-align:right;desc:current warmer ops"); @@ -321,10 +363,12 @@ protected Table getTableWithHeader(final RestRequest request) { table.addCell("warmer.total", "sibling:pri;alias:wto,warmerTotal;default:false;text-align:right;desc:total warmer ops"); table.addCell("pri.warmer.total", "default:false;text-align:right;desc:total warmer ops"); - table.addCell("warmer.total_time", "sibling:pri;alias:wtt,warmerTotalTime;default:false;text-align:right;desc:time spent in warmers"); + table.addCell("warmer.total_time", + "sibling:pri;alias:wtt,warmerTotalTime;default:false;text-align:right;desc:time spent in warmers"); table.addCell("pri.warmer.total_time", "default:false;text-align:right;desc:time spent in warmers"); - table.addCell("suggest.current", "sibling:pri;alias:suc,suggestCurrent;default:false;text-align:right;desc:number of current suggest ops"); + table.addCell("suggest.current", + "sibling:pri;alias:suc,suggestCurrent;default:false;text-align:right;desc:number of current suggest ops"); table.addCell("pri.suggest.current", "default:false;text-align:right;desc:number of current suggest ops"); table.addCell("suggest.time", "sibling:pri;alias:suti,suggestTime;default:false;text-align:right;desc:time spend in suggest"); @@ -343,7 +387,8 @@ protected Table getTableWithHeader(final RestRequest request) { } // package private for testing - Table buildTable(RestRequest request, Index[] indices, ClusterHealthResponse response, IndicesStatsResponse stats, MetaData indexMetaDatas) { + Table buildTable(RestRequest request, Index[] indices, ClusterHealthResponse response, + IndicesStatsResponse stats, MetaData indexMetaDatas) { final String healthParam = request.param("health"); final ClusterHealthStatus status; if (healthParam != null) { @@ -374,7 +419,8 @@ Table buildTable(RestRequest request, Index[] indices, ClusterHealthResponse res final CommonStats totalStats = indexStats == null ? new CommonStats() : indexStats.getTotal(); table.startRow(); - table.addCell(state == IndexMetaData.State.OPEN ? (indexHealth == null ? "red*" : indexHealth.getStatus().toString().toLowerCase(Locale.ROOT)) : null); + table.addCell(state == IndexMetaData.State.OPEN ? + (indexHealth == null ? "red*" : indexHealth.getStatus().toString().toLowerCase(Locale.ROOT)) : null); table.addCell(state.toString().toLowerCase(Locale.ROOT)); table.addCell(indexName); table.addCell(index.getUUID()); diff --git a/server/src/main/java/org/elasticsearch/rest/action/cat/RestShardsAction.java b/server/src/main/java/org/elasticsearch/rest/action/cat/RestShardsAction.java index e4e8ea121dd9d..eb82f7da58c5c 100644 --- a/server/src/main/java/org/elasticsearch/rest/action/cat/RestShardsAction.java +++ b/server/src/main/java/org/elasticsearch/rest/action/cat/RestShardsAction.java @@ -142,16 +142,20 @@ protected Table getTableWithHeader(final RestRequest request) { table.addCell("get.missing_time", "alias:gmti,getMissingTime;default:false;text-align:right;desc:time spent in failed gets"); table.addCell("get.missing_total", "alias:gmto,getMissingTotal;default:false;text-align:right;desc:number of failed gets"); - table.addCell("indexing.delete_current", "alias:idc,indexingDeleteCurrent;default:false;text-align:right;desc:number of current deletions"); + table.addCell("indexing.delete_current", + "alias:idc,indexingDeleteCurrent;default:false;text-align:right;desc:number of current deletions"); table.addCell("indexing.delete_time", "alias:idti,indexingDeleteTime;default:false;text-align:right;desc:time spent in deletions"); table.addCell("indexing.delete_total", "alias:idto,indexingDeleteTotal;default:false;text-align:right;desc:number of delete ops"); - table.addCell("indexing.index_current", "alias:iic,indexingIndexCurrent;default:false;text-align:right;desc:number of current indexing ops"); + table.addCell("indexing.index_current", + "alias:iic,indexingIndexCurrent;default:false;text-align:right;desc:number of current indexing ops"); table.addCell("indexing.index_time", "alias:iiti,indexingIndexTime;default:false;text-align:right;desc:time spent in indexing"); table.addCell("indexing.index_total", "alias:iito,indexingIndexTotal;default:false;text-align:right;desc:number of indexing ops"); - table.addCell("indexing.index_failed", "alias:iif,indexingIndexFailed;default:false;text-align:right;desc:number of failed indexing ops"); + table.addCell("indexing.index_failed", + "alias:iif,indexingIndexFailed;default:false;text-align:right;desc:number of failed indexing ops"); table.addCell("merges.current", "alias:mc,mergesCurrent;default:false;text-align:right;desc:number of current merges"); - table.addCell("merges.current_docs", "alias:mcd,mergesCurrentDocs;default:false;text-align:right;desc:number of current merging docs"); + table.addCell("merges.current_docs", + "alias:mcd,mergesCurrentDocs;default:false;text-align:right;desc:number of current merging docs"); table.addCell("merges.current_size", "alias:mcs,mergesCurrentSize;default:false;text-align:right;desc:size of current merges"); table.addCell("merges.total", "alias:mt,mergesTotal;default:false;text-align:right;desc:number of completed merge ops"); table.addCell("merges.total_docs", "alias:mtd,mergesTotalDocs;default:false;text-align:right;desc:docs merged"); @@ -160,7 +164,8 @@ protected Table getTableWithHeader(final RestRequest request) { table.addCell("refresh.total", "alias:rto,refreshTotal;default:false;text-align:right;desc:total refreshes"); table.addCell("refresh.time", "alias:rti,refreshTime;default:false;text-align:right;desc:time spent in refreshes"); - table.addCell("refresh.listeners", "alias:rli,refreshListeners;default:false;text-align:right;desc:number of pending refresh listeners"); + table.addCell("refresh.listeners", + "alias:rli,refreshListeners;default:false;text-align:right;desc:number of pending refresh listeners"); table.addCell("search.fetch_current", "alias:sfc,searchFetchCurrent;default:false;text-align:right;desc:current fetch phase ops"); table.addCell("search.fetch_time", "alias:sfti,searchFetchTime;default:false;text-align:right;desc:time spent in fetch phase"); @@ -170,14 +175,19 @@ protected Table getTableWithHeader(final RestRequest request) { table.addCell("search.query_time", "alias:sqti,searchQueryTime;default:false;text-align:right;desc:time spent in query phase"); table.addCell("search.query_total", "alias:sqto,searchQueryTotal;default:false;text-align:right;desc:total query phase ops"); table.addCell("search.scroll_current", "alias:scc,searchScrollCurrent;default:false;text-align:right;desc:open scroll contexts"); - table.addCell("search.scroll_time", "alias:scti,searchScrollTime;default:false;text-align:right;desc:time scroll contexts held open"); + table.addCell("search.scroll_time", + "alias:scti,searchScrollTime;default:false;text-align:right;desc:time scroll contexts held open"); table.addCell("search.scroll_total", "alias:scto,searchScrollTotal;default:false;text-align:right;desc:completed scroll contexts"); table.addCell("segments.count", "alias:sc,segmentsCount;default:false;text-align:right;desc:number of segments"); table.addCell("segments.memory", "alias:sm,segmentsMemory;default:false;text-align:right;desc:memory used by segments"); - table.addCell("segments.index_writer_memory", "alias:siwm,segmentsIndexWriterMemory;default:false;text-align:right;desc:memory used by index writer"); - table.addCell("segments.version_map_memory", "alias:svmm,segmentsVersionMapMemory;default:false;text-align:right;desc:memory used by version map"); - table.addCell("segments.fixed_bitset_memory", "alias:sfbm,fixedBitsetMemory;default:false;text-align:right;desc:memory used by fixed bit sets for nested object field types and type filters for types referred in _parent fields"); + table.addCell("segments.index_writer_memory", + "alias:siwm,segmentsIndexWriterMemory;default:false;text-align:right;desc:memory used by index writer"); + table.addCell("segments.version_map_memory", + "alias:svmm,segmentsVersionMapMemory;default:false;text-align:right;desc:memory used by version map"); + table.addCell("segments.fixed_bitset_memory", + "alias:sfbm,fixedBitsetMemory;default:false;text-align:right;desc:memory used by fixed bit sets for nested object" + + " field types and type filters for types referred in _parent fields"); table.addCell("seq_no.max", "alias:sqm,maxSeqNo;default:false;text-align:right;desc:max sequence number"); table.addCell("seq_no.local_checkpoint", "alias:sql,localCheckpoint;default:false;text-align:right;desc:local checkpoint"); diff --git a/server/src/main/java/org/elasticsearch/rest/action/cat/RestThreadPoolAction.java b/server/src/main/java/org/elasticsearch/rest/action/cat/RestThreadPoolAction.java index 3df270c8f6c80..e420dfb9843b8 100644 --- a/server/src/main/java/org/elasticsearch/rest/action/cat/RestThreadPoolAction.java +++ b/server/src/main/java/org/elasticsearch/rest/action/cat/RestThreadPoolAction.java @@ -88,7 +88,8 @@ public void processResponse(final NodesInfoResponse nodesInfoResponse) { client.admin().cluster().nodesStats(nodesStatsRequest, new RestResponseListener(channel) { @Override public RestResponse buildResponse(NodesStatsResponse nodesStatsResponse) throws Exception { - return RestTable.buildResponse(buildTable(request, clusterStateResponse, nodesInfoResponse, nodesStatsResponse), channel); + return RestTable.buildResponse( + buildTable(request, clusterStateResponse, nodesInfoResponse, nodesStatsResponse), channel); } }); } diff --git a/server/src/test/java/org/elasticsearch/explain/ExplainActionIT.java b/server/src/test/java/org/elasticsearch/explain/ExplainActionIT.java index 229cb99fbfbda..6616f93ded24b 100644 --- a/server/src/test/java/org/elasticsearch/explain/ExplainActionIT.java +++ b/server/src/test/java/org/elasticsearch/explain/ExplainActionIT.java @@ -257,7 +257,8 @@ public void testExplainDateRangeInQueryString() { refresh(); - ExplainResponse explainResponse = client().prepareExplain("test", "type", "1").setQuery(queryStringQuery("past:[now-2M/d TO now/d]")).get(); + ExplainResponse explainResponse = client().prepareExplain("test", "type", "1") + .setQuery(queryStringQuery("past:[now-2M/d TO now/d]")).get(); assertThat(explainResponse.isExists(), equalTo(true)); assertThat(explainResponse.isMatch(), equalTo(true)); } diff --git a/server/src/test/java/org/elasticsearch/rest/BytesRestResponseTests.java b/server/src/test/java/org/elasticsearch/rest/BytesRestResponseTests.java index a80c3b1bd4238..29a7944f58792 100644 --- a/server/src/test/java/org/elasticsearch/rest/BytesRestResponseTests.java +++ b/server/src/test/java/org/elasticsearch/rest/BytesRestResponseTests.java @@ -153,10 +153,14 @@ public void testConvert() throws IOException { new SearchShardTarget("node_1", new Index("foo", "_na_"), 1, null)); ShardSearchFailure failure1 = new ShardSearchFailure(new ParsingException(1, 2, "foobar", null), new SearchShardTarget("node_1", new Index("foo", "_na_"), 2, null)); - SearchPhaseExecutionException ex = new SearchPhaseExecutionException("search", "all shards failed", new ShardSearchFailure[] {failure, failure1}); + SearchPhaseExecutionException ex = new SearchPhaseExecutionException("search", "all shards failed", + new ShardSearchFailure[] {failure, failure1}); BytesRestResponse response = new BytesRestResponse(channel, new RemoteTransportException("foo", ex)); String text = response.content().utf8ToString(); - String expected = "{\"error\":{\"root_cause\":[{\"type\":\"parsing_exception\",\"reason\":\"foobar\",\"line\":1,\"col\":2}],\"type\":\"search_phase_execution_exception\",\"reason\":\"all shards failed\",\"phase\":\"search\",\"grouped\":true,\"failed_shards\":[{\"shard\":1,\"index\":\"foo\",\"node\":\"node_1\",\"reason\":{\"type\":\"parsing_exception\",\"reason\":\"foobar\",\"line\":1,\"col\":2}}]},\"status\":400}"; + String expected = "{\"error\":{\"root_cause\":[{\"type\":\"parsing_exception\",\"reason\":\"foobar\",\"line\":1,\"col\":2}]," + + "\"type\":\"search_phase_execution_exception\",\"reason\":\"all shards failed\",\"phase\":\"search\",\"grouped\":true," + + "\"failed_shards\":[{\"shard\":1,\"index\":\"foo\",\"node\":\"node_1\",\"reason\":{\"type\":\"parsing_exception\"," + + "\"reason\":\"foobar\",\"line\":1,\"col\":2}}]},\"status\":400}"; assertEquals(expected.trim(), text.trim()); String stackTrace = ExceptionsHelper.stackTrace(ex); assertTrue(stackTrace.contains("Caused by: ParsingException[foobar]")); From 87ac3103f426e4ab5c66078d5d41166885ef4100 Mon Sep 17 00:00:00 2001 From: Nhat Nguyen Date: Tue, 8 Jan 2019 16:12:18 -0500 Subject: [PATCH 028/186] Do not mutate RecoveryResponse (#37204) Today we create a global instance of RecoveryResponse then mutate it when executing each recovery step. This is okay for the current sequential recovery flow but not suitable for an asynchronous recovery which we are targeting. With this commit, we return the result of each step separately, then construct a RecoveryResponse at the end. Relates #37174 --- .../recovery/PeerRecoveryTargetService.java | 4 +- .../indices/recovery/RecoveryResponse.java | 93 +++---- .../recovery/RecoverySourceHandler.java | 240 +++++++++--------- .../recovery/RecoverySourceHandlerTests.java | 18 +- 4 files changed, 170 insertions(+), 185 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/indices/recovery/PeerRecoveryTargetService.java b/server/src/main/java/org/elasticsearch/indices/recovery/PeerRecoveryTargetService.java index 8598917b82313..8e52a05e2ac30 100644 --- a/server/src/main/java/org/elasticsearch/indices/recovery/PeerRecoveryTargetService.java +++ b/server/src/main/java/org/elasticsearch/indices/recovery/PeerRecoveryTargetService.java @@ -304,9 +304,7 @@ public String executor() { @Override public RecoveryResponse read(StreamInput in) throws IOException { - RecoveryResponse recoveryResponse = new RecoveryResponse(); - recoveryResponse.readFrom(in); - return recoveryResponse; + return new RecoveryResponse(in); } }) ); diff --git a/server/src/main/java/org/elasticsearch/indices/recovery/RecoveryResponse.java b/server/src/main/java/org/elasticsearch/indices/recovery/RecoveryResponse.java index 9018f6f0be199..02d4ff5dbc13b 100644 --- a/server/src/main/java/org/elasticsearch/indices/recovery/RecoveryResponse.java +++ b/server/src/main/java/org/elasticsearch/indices/recovery/RecoveryResponse.java @@ -24,53 +24,46 @@ import org.elasticsearch.transport.TransportResponse; import java.io.IOException; -import java.util.ArrayList; import java.util.List; -class RecoveryResponse extends TransportResponse { +final class RecoveryResponse extends TransportResponse { - List phase1FileNames = new ArrayList<>(); - List phase1FileSizes = new ArrayList<>(); - List phase1ExistingFileNames = new ArrayList<>(); - List phase1ExistingFileSizes = new ArrayList<>(); - long phase1TotalSize; - long phase1ExistingTotalSize; - long phase1Time; - long phase1ThrottlingWaitTime; + final List phase1FileNames; + final List phase1FileSizes; + final List phase1ExistingFileNames; + final List phase1ExistingFileSizes; + final long phase1TotalSize; + final long phase1ExistingTotalSize; + final long phase1Time; + final long phase1ThrottlingWaitTime; - long startTime; + final long startTime; - int phase2Operations; - long phase2Time; + final int phase2Operations; + final long phase2Time; - RecoveryResponse() { + RecoveryResponse(List phase1FileNames, List phase1FileSizes, List phase1ExistingFileNames, + List phase1ExistingFileSizes, long phase1TotalSize, long phase1ExistingTotalSize, + long phase1Time, long phase1ThrottlingWaitTime, long startTime, int phase2Operations, long phase2Time) { + this.phase1FileNames = phase1FileNames; + this.phase1FileSizes = phase1FileSizes; + this.phase1ExistingFileNames = phase1ExistingFileNames; + this.phase1ExistingFileSizes = phase1ExistingFileSizes; + this.phase1TotalSize = phase1TotalSize; + this.phase1ExistingTotalSize = phase1ExistingTotalSize; + this.phase1Time = phase1Time; + this.phase1ThrottlingWaitTime = phase1ThrottlingWaitTime; + this.startTime = startTime; + this.phase2Operations = phase2Operations; + this.phase2Time = phase2Time; } - @Override - public void readFrom(StreamInput in) throws IOException { - super.readFrom(in); - int size = in.readVInt(); - phase1FileNames = new ArrayList<>(size); - for (int i = 0; i < size; i++) { - phase1FileNames.add(in.readString()); - } - size = in.readVInt(); - phase1FileSizes = new ArrayList<>(size); - for (int i = 0; i < size; i++) { - phase1FileSizes.add(in.readVLong()); - } - - size = in.readVInt(); - phase1ExistingFileNames = new ArrayList<>(size); - for (int i = 0; i < size; i++) { - phase1ExistingFileNames.add(in.readString()); - } - size = in.readVInt(); - phase1ExistingFileSizes = new ArrayList<>(size); - for (int i = 0; i < size; i++) { - phase1ExistingFileSizes.add(in.readVLong()); - } - + RecoveryResponse(StreamInput in) throws IOException { + super(in); + phase1FileNames = in.readList(StreamInput::readString); + phase1FileSizes = in.readList(StreamInput::readVLong); + phase1ExistingFileNames = in.readList(StreamInput::readString); + phase1ExistingFileSizes = in.readList(StreamInput::readVLong); phase1TotalSize = in.readVLong(); phase1ExistingTotalSize = in.readVLong(); phase1Time = in.readVLong(); @@ -83,24 +76,10 @@ public void readFrom(StreamInput in) throws IOException { @Override public void writeTo(StreamOutput out) throws IOException { super.writeTo(out); - out.writeVInt(phase1FileNames.size()); - for (String name : phase1FileNames) { - out.writeString(name); - } - out.writeVInt(phase1FileSizes.size()); - for (long size : phase1FileSizes) { - out.writeVLong(size); - } - - out.writeVInt(phase1ExistingFileNames.size()); - for (String name : phase1ExistingFileNames) { - out.writeString(name); - } - out.writeVInt(phase1ExistingFileSizes.size()); - for (long size : phase1ExistingFileSizes) { - out.writeVLong(size); - } - + out.writeStringList(phase1FileNames); + out.writeCollection(phase1FileSizes, StreamOutput::writeVLong); + out.writeStringList(phase1ExistingFileNames); + out.writeCollection(phase1ExistingFileSizes, StreamOutput::writeVLong); out.writeVLong(phase1TotalSize); out.writeVLong(phase1ExistingTotalSize); out.writeVLong(phase1Time); diff --git a/server/src/main/java/org/elasticsearch/indices/recovery/RecoverySourceHandler.java b/server/src/main/java/org/elasticsearch/indices/recovery/RecoverySourceHandler.java index f6196e36ba2d3..315af6b4ae084 100644 --- a/server/src/main/java/org/elasticsearch/indices/recovery/RecoverySourceHandler.java +++ b/server/src/main/java/org/elasticsearch/indices/recovery/RecoverySourceHandler.java @@ -41,6 +41,7 @@ import org.elasticsearch.common.logging.Loggers; import org.elasticsearch.common.lucene.store.InputStreamIndexInput; import org.elasticsearch.common.unit.ByteSizeValue; +import org.elasticsearch.common.unit.TimeValue; import org.elasticsearch.common.util.CancellableThreads; import org.elasticsearch.common.util.concurrent.FutureUtils; import org.elasticsearch.core.internal.io.IOUtils; @@ -64,6 +65,7 @@ import java.io.IOException; import java.io.OutputStream; import java.util.ArrayList; +import java.util.Collections; import java.util.Comparator; import java.util.List; import java.util.Locale; @@ -95,8 +97,6 @@ public class RecoverySourceHandler { private final int chunkSizeInBytes; private final RecoveryTargetHandler recoveryTarget; - protected final RecoveryResponse response; - private final CancellableThreads cancellableThreads = new CancellableThreads() { @Override protected void onCancel(String reason, @Nullable Exception suppressedException) { @@ -122,7 +122,6 @@ public RecoverySourceHandler(final IndexShard shard, RecoveryTargetHandler recov this.shardId = this.request.shardId().id(); this.logger = Loggers.getLogger(getClass(), request.shardId(), "recover to " + request.targetNode().getName()); this.chunkSizeInBytes = fileChunkSizeInBytes; - this.response = new RecoveryResponse(); } public StartRecoveryRequest getRequest() { @@ -149,10 +148,12 @@ public RecoveryResponse recoverToTarget() throws IOException { final long requiredSeqNoRangeStart; final boolean isSequenceNumberBasedRecovery = request.startingSeqNo() != SequenceNumbers.UNASSIGNED_SEQ_NO && isTargetSameHistory() && shard.hasCompleteHistoryOperations("peer-recovery", request.startingSeqNo()); + final SendFileResult sendFileResult; if (isSequenceNumberBasedRecovery) { logger.trace("performing sequence numbers based recovery. starting at [{}]", request.startingSeqNo()); startingSeqNo = request.startingSeqNo(); requiredSeqNoRangeStart = startingSeqNo; + sendFileResult = SendFileResult.EMPTY; } else { final Engine.IndexCommitRef phase1Snapshot; try { @@ -169,7 +170,7 @@ public RecoveryResponse recoverToTarget() throws IOException { startingSeqNo = shard.indexSettings().isSoftDeleteEnabled() ? requiredSeqNoRangeStart : 0; try { final int estimateNumOps = shard.estimateNumberOfHistoryOperations("peer-recovery", startingSeqNo); - phase1(phase1Snapshot.getIndexCommit(), () -> estimateNumOps); + sendFileResult = phase1(phase1Snapshot.getIndexCommit(), () -> estimateNumOps); } catch (final Exception e) { throw new RecoveryEngineException(shard.shardId(), 1, "phase1 failed", e); } finally { @@ -184,9 +185,10 @@ public RecoveryResponse recoverToTarget() throws IOException { assert requiredSeqNoRangeStart >= startingSeqNo : "requiredSeqNoRangeStart [" + requiredSeqNoRangeStart + "] is lower than [" + startingSeqNo + "]"; + final TimeValue prepareEngineTime; try { // For a sequence based recovery, the target can keep its local translog - prepareTargetForTranslog(isSequenceNumberBasedRecovery == false, + prepareEngineTime = prepareTargetForTranslog(isSequenceNumberBasedRecovery == false, shard.estimateNumberOfHistoryOperations("peer-recovery", startingSeqNo)); } catch (final Exception e) { throw new RecoveryEngineException(shard.shardId(), 1, "prepare target for translog failed", e); @@ -213,21 +215,25 @@ public RecoveryResponse recoverToTarget() throws IOException { logger.trace("snapshot translog for recovery; current size is [{}]", shard.estimateNumberOfHistoryOperations("peer-recovery", startingSeqNo)); } - final long targetLocalCheckpoint; + final SendSnapshotResult sendSnapshotResult; try (Translog.Snapshot snapshot = shard.getHistoryOperations("peer-recovery", startingSeqNo)) { // we have to capture the max_seen_auto_id_timestamp and the max_seq_no_of_updates to make sure that these values // are at least as high as the corresponding values on the primary when any of these operations were executed on it. final long maxSeenAutoIdTimestamp = shard.getMaxSeenAutoIdTimestamp(); final long maxSeqNoOfUpdatesOrDeletes = shard.getMaxSeqNoOfUpdatesOrDeletes(); - targetLocalCheckpoint = phase2(startingSeqNo, requiredSeqNoRangeStart, endingSeqNo, snapshot, + sendSnapshotResult = phase2(startingSeqNo, requiredSeqNoRangeStart, endingSeqNo, snapshot, maxSeenAutoIdTimestamp, maxSeqNoOfUpdatesOrDeletes); } catch (Exception e) { throw new RecoveryEngineException(shard.shardId(), 2, "phase2 failed", e); } - finalizeRecovery(targetLocalCheckpoint); + finalizeRecovery(sendSnapshotResult.targetLocalCheckpoint); + final long phase1ThrottlingWaitTime = 0L; // TODO: return the actual throttle time + return new RecoveryResponse(sendFileResult.phase1FileNames, sendFileResult.phase1FileSizes, + sendFileResult.phase1ExistingFileNames, sendFileResult.phase1ExistingFileSizes, sendFileResult.totalSize, + sendFileResult.existingTotalSize, sendFileResult.took.millis(), phase1ThrottlingWaitTime, prepareEngineTime.millis(), + sendSnapshotResult.totalOperations, sendSnapshotResult.tookTime.millis()); } - return response; } private boolean isTargetSameHistory() { @@ -276,6 +282,32 @@ public void onFailure(Exception e) { }); } + static final class SendFileResult { + final List phase1FileNames; + final List phase1FileSizes; + final long totalSize; + + final List phase1ExistingFileNames; + final List phase1ExistingFileSizes; + final long existingTotalSize; + + final TimeValue took; + + SendFileResult(List phase1FileNames, List phase1FileSizes, long totalSize, + List phase1ExistingFileNames, List phase1ExistingFileSizes, long existingTotalSize, TimeValue took) { + this.phase1FileNames = phase1FileNames; + this.phase1FileSizes = phase1FileSizes; + this.totalSize = totalSize; + this.phase1ExistingFileNames = phase1ExistingFileNames; + this.phase1ExistingFileSizes = phase1ExistingFileSizes; + this.existingTotalSize = existingTotalSize; + this.took = took; + } + + static final SendFileResult EMPTY = new SendFileResult(Collections.emptyList(), Collections.emptyList(), 0L, + Collections.emptyList(), Collections.emptyList(), 0L, TimeValue.ZERO); + } + /** * Perform phase1 of the recovery operations. Once this {@link IndexCommit} * snapshot has been performed no commit operations (files being fsync'd) @@ -285,12 +317,16 @@ public void onFailure(Exception e) { * segments that are missing. Only segments that have the same size and * checksum can be reused */ - public void phase1(final IndexCommit snapshot, final Supplier translogOps) { + public SendFileResult phase1(final IndexCommit snapshot, final Supplier translogOps) { cancellableThreads.checkForCancel(); // Total size of segment files that are recovered long totalSize = 0; // Total size of segment files that were able to be re-used long existingTotalSize = 0; + final List phase1FileNames = new ArrayList<>(); + final List phase1FileSizes = new ArrayList<>(); + final List phase1ExistingFileNames = new ArrayList<>(); + final List phase1ExistingFileSizes = new ArrayList<>(); final Store store = shard.store(); store.incRef(); try { @@ -331,8 +367,8 @@ public void phase1(final IndexCommit snapshot, final Supplier translogO } else { final Store.RecoveryDiff diff = recoverySourceMetadata.recoveryDiff(request.metadataSnapshot()); for (StoreFileMetaData md : diff.identical) { - response.phase1ExistingFileNames.add(md.name()); - response.phase1ExistingFileSizes.add(md.length()); + phase1ExistingFileNames.add(md.name()); + phase1ExistingFileSizes.add(md.length()); existingTotalSize += md.length(); if (logger.isTraceEnabled()) { logger.trace("recovery [phase1]: not recovering [{}], exist in local store and has checksum [{}]," + @@ -350,20 +386,16 @@ public void phase1(final IndexCommit snapshot, final Supplier translogO } else { logger.trace("recovery [phase1]: recovering [{}], does not exist in remote", md.name()); } - response.phase1FileNames.add(md.name()); - response.phase1FileSizes.add(md.length()); + phase1FileNames.add(md.name()); + phase1FileSizes.add(md.length()); totalSize += md.length(); } - response.phase1TotalSize = totalSize; - response.phase1ExistingTotalSize = existingTotalSize; - logger.trace("recovery [phase1]: recovering_files [{}] with total_size [{}], reusing_files [{}] with total_size [{}]", - response.phase1FileNames.size(), - new ByteSizeValue(totalSize), response.phase1ExistingFileNames.size(), new ByteSizeValue(existingTotalSize)); - cancellableThreads.execute(() -> - recoveryTarget.receiveFileInfo(response.phase1FileNames, response.phase1FileSizes, response.phase1ExistingFileNames, - response.phase1ExistingFileSizes, translogOps.get())); + phase1FileNames.size(), new ByteSizeValue(totalSize), + phase1ExistingFileNames.size(), new ByteSizeValue(existingTotalSize)); + cancellableThreads.execute(() -> recoveryTarget.receiveFileInfo( + phase1FileNames, phase1FileSizes, phase1ExistingFileNames, phase1ExistingFileSizes, translogOps.get())); // How many bytes we've copied since we last called RateLimiter.pause final Function outputStreamFactories = md -> new BufferedOutputStream(new RecoveryOutputStream(md, translogOps), chunkSizeInBytes); @@ -417,27 +449,27 @@ public void phase1(final IndexCommit snapshot, final Supplier translogO } } } - - logger.trace("recovery [phase1]: took [{}]", stopWatch.totalTime()); - response.phase1Time = stopWatch.totalTime().millis(); + final TimeValue took = stopWatch.totalTime(); + logger.trace("recovery [phase1]: took [{}]", took); + return new SendFileResult(phase1FileNames, phase1FileSizes, totalSize, phase1ExistingFileNames, + phase1ExistingFileSizes, existingTotalSize, took); } catch (Exception e) { - throw new RecoverFilesRecoveryException(request.shardId(), response.phase1FileNames.size(), new ByteSizeValue(totalSize), e); + throw new RecoverFilesRecoveryException(request.shardId(), phase1FileNames.size(), new ByteSizeValue(totalSize), e); } finally { store.decRef(); } } - void prepareTargetForTranslog(final boolean fileBasedRecovery, final int totalTranslogOps) throws IOException { + TimeValue prepareTargetForTranslog(final boolean fileBasedRecovery, final int totalTranslogOps) throws IOException { StopWatch stopWatch = new StopWatch().start(); logger.trace("recovery [phase1]: prepare remote engine for translog"); - final long startEngineStart = stopWatch.totalTime().millis(); // Send a request preparing the new shard's translog to receive operations. This ensures the shard engine is started and disables // garbage collection (not the JVM's GC!) of tombstone deletes. cancellableThreads.executeIO(() -> recoveryTarget.prepareForTranslogOperations(fileBasedRecovery, totalTranslogOps)); stopWatch.stop(); - - response.startTime = stopWatch.totalTime().millis() - startEngineStart; - logger.trace("recovery [phase1]: remote engine start took [{}]", stopWatch.totalTime()); + final TimeValue tookTime = stopWatch.totalTime(); + logger.trace("recovery [phase1]: remote engine start took [{}]", tookTime); + return tookTime; } /** @@ -454,102 +486,23 @@ void prepareTargetForTranslog(final boolean fileBasedRecovery, final int totalTr * @param snapshot a snapshot of the translog * @param maxSeenAutoIdTimestamp the max auto_id_timestamp of append-only requests on the primary * @param maxSeqNoOfUpdatesOrDeletes the max seq_no of updates or deletes on the primary after these operations were executed on it. - * @return the local checkpoint on the target + * @return the send snapshot result */ - long phase2(final long startingSeqNo, long requiredSeqNoRangeStart, long endingSeqNo, final Translog.Snapshot snapshot, - final long maxSeenAutoIdTimestamp, final long maxSeqNoOfUpdatesOrDeletes) - throws IOException { + SendSnapshotResult phase2(long startingSeqNo, long requiredSeqNoRangeStart, long endingSeqNo, Translog.Snapshot snapshot, + long maxSeenAutoIdTimestamp, long maxSeqNoOfUpdatesOrDeletes) throws IOException { + assert requiredSeqNoRangeStart <= endingSeqNo + 1: + "requiredSeqNoRangeStart " + requiredSeqNoRangeStart + " is larger than endingSeqNo " + endingSeqNo; + assert startingSeqNo <= requiredSeqNoRangeStart : + "startingSeqNo " + startingSeqNo + " is larger than requiredSeqNoRangeStart " + requiredSeqNoRangeStart; if (shard.state() == IndexShardState.CLOSED) { throw new IndexShardClosedException(request.shardId()); } - cancellableThreads.checkForCancel(); final StopWatch stopWatch = new StopWatch().start(); logger.trace("recovery [phase2]: sending transaction log operations (seq# from [" + startingSeqNo + "], " + "required [" + requiredSeqNoRangeStart + ":" + endingSeqNo + "]"); - // send all the snapshot's translog operations to the target - final SendSnapshotResult result = sendSnapshot( - startingSeqNo, requiredSeqNoRangeStart, endingSeqNo, snapshot, maxSeenAutoIdTimestamp, maxSeqNoOfUpdatesOrDeletes); - - stopWatch.stop(); - logger.trace("recovery [phase2]: took [{}]", stopWatch.totalTime()); - response.phase2Time = stopWatch.totalTime().millis(); - response.phase2Operations = result.totalOperations; - return result.targetLocalCheckpoint; - } - - /* - * finalizes the recovery process - */ - public void finalizeRecovery(final long targetLocalCheckpoint) throws IOException { - if (shard.state() == IndexShardState.CLOSED) { - throw new IndexShardClosedException(request.shardId()); - } - cancellableThreads.checkForCancel(); - StopWatch stopWatch = new StopWatch().start(); - logger.trace("finalizing recovery"); - /* - * Before marking the shard as in-sync we acquire an operation permit. We do this so that there is a barrier between marking a - * shard as in-sync and relocating a shard. If we acquire the permit then no relocation handoff can complete before we are done - * marking the shard as in-sync. If the relocation handoff holds all the permits then after the handoff completes and we acquire - * the permit then the state of the shard will be relocated and this recovery will fail. - */ - runUnderPrimaryPermit(() -> shard.markAllocationIdAsInSync(request.targetAllocationId(), targetLocalCheckpoint), - shardId + " marking " + request.targetAllocationId() + " as in sync", shard, cancellableThreads, logger); - final long globalCheckpoint = shard.getGlobalCheckpoint(); - cancellableThreads.executeIO(() -> recoveryTarget.finalizeRecovery(globalCheckpoint)); - runUnderPrimaryPermit(() -> shard.updateGlobalCheckpointForShard(request.targetAllocationId(), globalCheckpoint), - shardId + " updating " + request.targetAllocationId() + "'s global checkpoint", shard, cancellableThreads, logger); - - if (request.isPrimaryRelocation()) { - logger.trace("performing relocation hand-off"); - // this acquires all IndexShard operation permits and will thus delay new recoveries until it is done - cancellableThreads.execute(() -> shard.relocated(recoveryTarget::handoffPrimaryContext)); - /* - * if the recovery process fails after disabling primary mode on the source shard, both relocation source and - * target are failed (see {@link IndexShard#updateRoutingEntry}). - */ - } - stopWatch.stop(); - logger.trace("finalizing recovery took [{}]", stopWatch.totalTime()); - } - - static class SendSnapshotResult { - - final long targetLocalCheckpoint; - final int totalOperations; - - SendSnapshotResult(final long targetLocalCheckpoint, final int totalOperations) { - this.targetLocalCheckpoint = targetLocalCheckpoint; - this.totalOperations = totalOperations; - } - - } - - /** - * Send the given snapshot's operations with a sequence number greater than the specified staring sequence number to this handler's - * target node. - *

- * Operations are bulked into a single request depending on an operation count limit or size-in-bytes limit. - * - * @param startingSeqNo the sequence number for which only operations with a sequence number greater than this will be sent - * @param requiredSeqNoRangeStart the lower sequence number of the required range - * @param endingSeqNo the upper bound of the sequence number range to be sent (inclusive) - * @param snapshot the translog snapshot to replay operations from @return the local checkpoint on the target and the - * total number of operations sent - * @param maxSeenAutoIdTimestamp the max auto_id_timestamp of append-only requests on the primary - * @param maxSeqNoOfUpdatesOrDeletes the max seq_no of updates or deletes on the primary after these operations were executed on it. - * @throws IOException if an I/O exception occurred reading the translog snapshot - */ - protected SendSnapshotResult sendSnapshot(final long startingSeqNo, long requiredSeqNoRangeStart, long endingSeqNo, - final Translog.Snapshot snapshot, final long maxSeenAutoIdTimestamp, - final long maxSeqNoOfUpdatesOrDeletes) throws IOException { - assert requiredSeqNoRangeStart <= endingSeqNo + 1: - "requiredSeqNoRangeStart " + requiredSeqNoRangeStart + " is larger than endingSeqNo " + endingSeqNo; - assert startingSeqNo <= requiredSeqNoRangeStart : - "startingSeqNo " + startingSeqNo + " is larger than requiredSeqNoRangeStart " + requiredSeqNoRangeStart; int ops = 0; long size = 0; int skippedOps = 0; @@ -615,7 +568,58 @@ protected SendSnapshotResult sendSnapshot(final long startingSeqNo, long require logger.trace("sent final batch of [{}][{}] (total: [{}]) translog operations", ops, new ByteSizeValue(size), expectedTotalOps); - return new SendSnapshotResult(targetLocalCheckpoint.get(), totalSentOps); + stopWatch.stop(); + final TimeValue tookTime = stopWatch.totalTime(); + logger.trace("recovery [phase2]: took [{}]", tookTime); + return new SendSnapshotResult(targetLocalCheckpoint.get(), totalSentOps, tookTime); + } + + /* + * finalizes the recovery process + */ + public void finalizeRecovery(final long targetLocalCheckpoint) throws IOException { + if (shard.state() == IndexShardState.CLOSED) { + throw new IndexShardClosedException(request.shardId()); + } + cancellableThreads.checkForCancel(); + StopWatch stopWatch = new StopWatch().start(); + logger.trace("finalizing recovery"); + /* + * Before marking the shard as in-sync we acquire an operation permit. We do this so that there is a barrier between marking a + * shard as in-sync and relocating a shard. If we acquire the permit then no relocation handoff can complete before we are done + * marking the shard as in-sync. If the relocation handoff holds all the permits then after the handoff completes and we acquire + * the permit then the state of the shard will be relocated and this recovery will fail. + */ + runUnderPrimaryPermit(() -> shard.markAllocationIdAsInSync(request.targetAllocationId(), targetLocalCheckpoint), + shardId + " marking " + request.targetAllocationId() + " as in sync", shard, cancellableThreads, logger); + final long globalCheckpoint = shard.getGlobalCheckpoint(); + cancellableThreads.executeIO(() -> recoveryTarget.finalizeRecovery(globalCheckpoint)); + runUnderPrimaryPermit(() -> shard.updateGlobalCheckpointForShard(request.targetAllocationId(), globalCheckpoint), + shardId + " updating " + request.targetAllocationId() + "'s global checkpoint", shard, cancellableThreads, logger); + + if (request.isPrimaryRelocation()) { + logger.trace("performing relocation hand-off"); + // this acquires all IndexShard operation permits and will thus delay new recoveries until it is done + cancellableThreads.execute(() -> shard.relocated(recoveryTarget::handoffPrimaryContext)); + /* + * if the recovery process fails after disabling primary mode on the source shard, both relocation source and + * target are failed (see {@link IndexShard#updateRoutingEntry}). + */ + } + stopWatch.stop(); + logger.trace("finalizing recovery took [{}]", stopWatch.totalTime()); + } + + static final class SendSnapshotResult { + final long targetLocalCheckpoint; + final int totalOperations; + final TimeValue tookTime; + + SendSnapshotResult(final long targetLocalCheckpoint, final int totalOperations, final TimeValue tookTime) { + this.targetLocalCheckpoint = targetLocalCheckpoint; + this.totalOperations = totalOperations; + this.tookTime = tookTime; + } } /** diff --git a/server/src/test/java/org/elasticsearch/indices/recovery/RecoverySourceHandlerTests.java b/server/src/test/java/org/elasticsearch/indices/recovery/RecoverySourceHandlerTests.java index af4dc59ca1a76..3f6a8072d86d5 100644 --- a/server/src/test/java/org/elasticsearch/indices/recovery/RecoverySourceHandlerTests.java +++ b/server/src/test/java/org/elasticsearch/indices/recovery/RecoverySourceHandlerTests.java @@ -46,6 +46,7 @@ import org.elasticsearch.common.lucene.uid.Versions; import org.elasticsearch.common.settings.ClusterSettings; import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.common.unit.TimeValue; import org.elasticsearch.common.util.CancellableThreads; import org.elasticsearch.common.xcontent.XContentType; import org.elasticsearch.core.internal.io.IOUtils; @@ -190,7 +191,7 @@ public void testSendSnapshotSendsOps() throws IOException { final long startingSeqNo = randomIntBetween(0, numberOfDocsWithValidSequenceNumbers - 1); final long requiredStartingSeqNo = randomIntBetween((int) startingSeqNo, numberOfDocsWithValidSequenceNumbers - 1); final long endingSeqNo = randomIntBetween((int) requiredStartingSeqNo - 1, numberOfDocsWithValidSequenceNumbers - 1); - RecoverySourceHandler.SendSnapshotResult result = handler.sendSnapshot(startingSeqNo, requiredStartingSeqNo, + RecoverySourceHandler.SendSnapshotResult result = handler.phase2(startingSeqNo, requiredStartingSeqNo, endingSeqNo, new Translog.Snapshot() { @Override public void close() { @@ -229,7 +230,7 @@ public Translog.Operation next() throws IOException { .filter(o -> o.seqNo() >= requiredStartingSeqNo && o.seqNo() <= endingSeqNo).collect(Collectors.toList()); List opsToSkip = randomSubsetOf(randomIntBetween(1, requiredOps.size()), requiredOps); expectThrows(IllegalStateException.class, () -> - handler.sendSnapshot(startingSeqNo, requiredStartingSeqNo, + handler.phase2(startingSeqNo, requiredStartingSeqNo, endingSeqNo, new Translog.Snapshot() { @Override public void close() { @@ -412,20 +413,23 @@ public void testThrowExceptionOnPrimaryRelocatedBeforePhase1Started() throws IOE recoverySettings.getChunkSize().bytesAsInt()) { @Override - public void phase1(final IndexCommit snapshot, final Supplier translogOps) { + public SendFileResult phase1(final IndexCommit snapshot, final Supplier translogOps) { phase1Called.set(true); + return super.phase1(snapshot, translogOps); } @Override - void prepareTargetForTranslog(final boolean fileBasedRecovery, final int totalTranslogOps) throws IOException { + TimeValue prepareTargetForTranslog(final boolean fileBasedRecovery, final int totalTranslogOps) throws IOException { prepareTargetForTranslogCalled.set(true); + return super.prepareTargetForTranslog(fileBasedRecovery, totalTranslogOps); } @Override - long phase2(long startingSeqNo, long requiredSeqNoRangeStart, long endingSeqNo, Translog.Snapshot snapshot, - long maxSeenAutoIdTimestamp, long maxSeqNoOfUpdatesOrDeletes) { + SendSnapshotResult phase2(long startingSeqNo, long requiredSeqNoRangeStart, long endingSeqNo, Translog.Snapshot snapshot, + long maxSeenAutoIdTimestamp, long maxSeqNoOfUpdatesOrDeletes) throws IOException { phase2Called.set(true); - return SequenceNumbers.UNASSIGNED_SEQ_NO; + return super.phase2(startingSeqNo, requiredSeqNoRangeStart, endingSeqNo, snapshot, + maxSeenAutoIdTimestamp, maxSeqNoOfUpdatesOrDeletes); } }; From 5f2fbedd8c7570bcc2ea74f38dce0b1736a85cee Mon Sep 17 00:00:00 2001 From: Marios Trivyzas Date: Tue, 8 Jan 2019 23:56:00 +0200 Subject: [PATCH 029/186] 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 ec32e66088c0001b0d3475ef06223de51d48b999 Mon Sep 17 00:00:00 2001 From: Mayya Sharipova Date: Tue, 8 Jan 2019 21:46:41 -0500 Subject: [PATCH 030/186] 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 031/186] [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 2a79c468f8624c3ccb452884dfd99e639d2d47da Mon Sep 17 00:00:00 2001 From: Ioannis Kakavas Date: Wed, 9 Jan 2019 07:19:34 +0200 Subject: [PATCH 032/186] 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 033/186] 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 d9e2ebca674c8834d83594deb24fdced4f720061 Mon Sep 17 00:00:00 2001 From: David Turner Date: Wed, 9 Jan 2019 08:18:25 +0000 Subject: [PATCH 034/186] 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 035/186] [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 036/186] 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 037/186] 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 038/186] [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 039/186] [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 040/186] [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 041/186] 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 042/186] [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 043/186] 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 044/186] 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 045/186] 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 046/186] 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 047/186] 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 048/186] 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 049/186] [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 050/186] 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 051/186] 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 052/186] [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 053/186] 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 054/186] [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 055/186] 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 056/186] 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 057/186] 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 df3b58cb04c0443142221884380d67d86d7da6e0 Mon Sep 17 00:00:00 2001 From: Benjamin Trent Date: Wed, 9 Jan 2019 14:25:35 -0600 Subject: [PATCH 058/186] 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 195873002bd3724c839a4a211c3aa1428f104a3a Mon Sep 17 00:00:00 2001 From: Jake Landis Date: Wed, 9 Jan 2019 14:47:47 -0600 Subject: [PATCH 059/186] 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 95479f17667c7d27fc480c260a1c380d681d8b93 Mon Sep 17 00:00:00 2001 From: Jim Ferenczi Date: Wed, 9 Jan 2019 22:57:34 +0100 Subject: [PATCH 060/186] Ensure that a non static top docs is created during the search phase This change fixes an unreleased bug that trips an assertion because a static instance shared among threads is modified during the search. This commit copies the static instance in order to ensure that each thread can modify the value without modifying the other instances. Closes #37179 Closes #37266 --- .../org/elasticsearch/join/query/ChildQuerySearchIT.java | 2 -- .../action/search/SearchPhaseController.java | 8 +------- .../search/query/TopDocsCollectorContext.java | 7 +++---- 3 files changed, 4 insertions(+), 13 deletions(-) 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 d92ff102fe8f8..8520b63205678 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,7 +59,6 @@ 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; @@ -86,7 +85,6 @@ 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 { 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 8fada8938f854..418d95b2077a9 100644 --- a/server/src/main/java/org/elasticsearch/action/search/SearchPhaseController.java +++ b/server/src/main/java/org/elasticsearch/action/search/SearchPhaseController.java @@ -760,13 +760,7 @@ TotalHits getTotalHits() { if (trackTotalHitsUpTo == SearchContext.TRACK_TOTAL_HITS_DISABLED) { return null; } else if (trackTotalHitsUpTo == SearchContext.TRACK_TOTAL_HITS_ACCURATE) { - // 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)); - } + assert totalHitsRelation == Relation.EQUAL_TO; return new TotalHits(totalHits, totalHitsRelation); } else { if (totalHits < trackTotalHitsUpTo) { diff --git a/server/src/main/java/org/elasticsearch/search/query/TopDocsCollectorContext.java b/server/src/main/java/org/elasticsearch/search/query/TopDocsCollectorContext.java index 4f648666f550d..6312fd3856b66 100644 --- a/server/src/main/java/org/elasticsearch/search/query/TopDocsCollectorContext.java +++ b/server/src/main/java/org/elasticsearch/search/query/TopDocsCollectorContext.java @@ -264,8 +264,7 @@ Collector create(Collector in) { @Override void postProcess(QuerySearchResult result) throws IOException { - final TopDocs topDocs = topDocsSupplier.get(); - topDocs.totalHits = totalHitsSupplier.get(); + final TopDocs topDocs = new TopDocs(totalHitsSupplier.get(), topDocsSupplier.get().scoreDocs); result.topDocs(new TopDocsAndMaxScore(topDocs, maxScoreSupplier.get()), sortAndFormats == null ? null : sortAndFormats.formats); } @@ -292,11 +291,11 @@ private ScrollingTopDocsCollectorContext(IndexReader reader, @Override void postProcess(QuerySearchResult result) throws IOException { - final TopDocs topDocs = topDocsSupplier.get(); + final TopDocs topDocs = new TopDocs(totalHitsSupplier.get(), topDocsSupplier.get().scoreDocs); final float maxScore; if (scrollContext.totalHits == null) { // first round - topDocs.totalHits = scrollContext.totalHits = totalHitsSupplier.get(); + scrollContext.totalHits = topDocs.totalHits; maxScore = scrollContext.maxScore = maxScoreSupplier.get(); } else { // subsequent round: the total number of hits and From 234059d2c091f192a1be6393c60f2a10e902dc54 Mon Sep 17 00:00:00 2001 From: Simon Willnauer Date: Wed, 9 Jan 2019 23:46:31 +0100 Subject: [PATCH 061/186] Enable Bulk-Merge if all source remains (#37269) Today we still wrap recovery source readers on merge even if we keep all documents recovery source. This basically disables bulk merging for stored fields. This change skips wrapping if all docs sources are kept anyway. --- .../RecoverySourcePruneMergePolicy.java | 16 ++++----- .../RecoverySourcePruneMergePolicyTests.java | 36 +++++++++++++++++++ 2 files changed, 44 insertions(+), 8 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/index/engine/RecoverySourcePruneMergePolicy.java b/server/src/main/java/org/elasticsearch/index/engine/RecoverySourcePruneMergePolicy.java index 7faed37b2fd36..42276f4ca2108 100644 --- a/server/src/main/java/org/elasticsearch/index/engine/RecoverySourcePruneMergePolicy.java +++ b/server/src/main/java/org/elasticsearch/index/engine/RecoverySourcePruneMergePolicy.java @@ -33,11 +33,8 @@ import org.apache.lucene.index.SortedNumericDocValues; import org.apache.lucene.index.SortedSetDocValues; import org.apache.lucene.index.StoredFieldVisitor; -import org.apache.lucene.search.BooleanClause; -import org.apache.lucene.search.BooleanQuery; import org.apache.lucene.search.ConjunctionDISI; import org.apache.lucene.search.DocIdSetIterator; -import org.apache.lucene.search.DocValuesFieldExistsQuery; import org.apache.lucene.search.IndexSearcher; import org.apache.lucene.search.Query; import org.apache.lucene.search.ScoreMode; @@ -68,15 +65,18 @@ static CodecReader wrapReader(String recoverySourceField, CodecReader reader, Su if (recoverySource == null || recoverySource.nextDoc() == DocIdSetIterator.NO_MORE_DOCS) { return reader; // early terminate - nothing to do here since non of the docs has a recovery source anymore. } - BooleanQuery.Builder builder = new BooleanQuery.Builder(); - builder.add(new DocValuesFieldExistsQuery(recoverySourceField), BooleanClause.Occur.FILTER); - builder.add(retainSourceQuerySupplier.get(), BooleanClause.Occur.FILTER); IndexSearcher s = new IndexSearcher(reader); s.setQueryCache(null); - Weight weight = s.createWeight(s.rewrite(builder.build()), ScoreMode.COMPLETE_NO_SCORES, 1.0f); + Weight weight = s.createWeight(s.rewrite(retainSourceQuerySupplier.get()), ScoreMode.COMPLETE_NO_SCORES, 1.0f); Scorer scorer = weight.scorer(reader.getContext()); if (scorer != null) { - return new SourcePruningFilterCodecReader(recoverySourceField, reader, BitSet.of(scorer.iterator(), reader.maxDoc())); + BitSet recoverySourceToKeep = BitSet.of(scorer.iterator(), reader.maxDoc()); + // calculating the cardinality is significantly cheaper than skipping all bulk-merging we might do + // if retentions are high we keep most of it + if (recoverySourceToKeep.cardinality() == reader.maxDoc()) { + return reader; // keep all source + } + return new SourcePruningFilterCodecReader(recoverySourceField, reader, recoverySourceToKeep); } else { return new SourcePruningFilterCodecReader(recoverySourceField, reader, null); } diff --git a/server/src/test/java/org/elasticsearch/index/engine/RecoverySourcePruneMergePolicyTests.java b/server/src/test/java/org/elasticsearch/index/engine/RecoverySourcePruneMergePolicyTests.java index c46b47b87d06e..ef895e1a4ce8e 100644 --- a/server/src/test/java/org/elasticsearch/index/engine/RecoverySourcePruneMergePolicyTests.java +++ b/server/src/test/java/org/elasticsearch/index/engine/RecoverySourcePruneMergePolicyTests.java @@ -37,6 +37,7 @@ import org.apache.lucene.index.StandardDirectoryReader; import org.apache.lucene.index.Term; import org.apache.lucene.search.DocIdSetIterator; +import org.apache.lucene.search.MatchAllDocsQuery; import org.apache.lucene.search.MatchNoDocsQuery; import org.apache.lucene.search.TermQuery; import org.apache.lucene.store.Directory; @@ -158,4 +159,39 @@ public void testPruneSome() throws IOException { } } } + + public void testPruneNone() throws IOException { + try (Directory dir = newDirectory()) { + IndexWriterConfig iwc = newIndexWriterConfig(); + iwc.setMergePolicy(new RecoverySourcePruneMergePolicy("extra_source", + () -> new MatchAllDocsQuery(), iwc.getMergePolicy())); + try (IndexWriter writer = new IndexWriter(dir, iwc)) { + for (int i = 0; i < 20; i++) { + if (i > 0 && randomBoolean()) { + writer.flush(); + } + Document doc = new Document(); + doc.add(new StoredField("source", "hello world")); + doc.add(new StoredField("extra_source", "hello world")); + doc.add(new NumericDocValuesField("extra_source", 1)); + writer.addDocument(doc); + } + writer.forceMerge(1); + writer.commit(); + try (DirectoryReader reader = DirectoryReader.open(writer)) { + assertEquals(1, reader.leaves().size()); + NumericDocValues extra_source = reader.leaves().get(0).reader().getNumericDocValues("extra_source"); + assertNotNull(extra_source); + for (int i = 0; i < reader.maxDoc(); i++) { + Document document = reader.document(i); + Set collect = document.getFields().stream().map(IndexableField::name).collect(Collectors.toSet()); + assertTrue(collect.contains("source")); + assertTrue(collect.contains("extra_source")); + assertEquals(i, extra_source.nextDoc()); + } + assertEquals(DocIdSetIterator.NO_MORE_DOCS, extra_source.nextDoc()); + } + } + } + } } From 586093ec5e96bf657bc89040b0b6c822fd4f4678 Mon Sep 17 00:00:00 2001 From: Jim Ferenczi Date: Thu, 10 Jan 2019 00:26:35 +0100 Subject: [PATCH 062/186] Handle TopFieldDocs copy in TopDocsCollectorContext This commit fixes the clone of TopFieldDocs. Relates #37179 Relates #37266 --- .../search/query/TopDocsCollectorContext.java | 45 ++++++++++++------- 1 file changed, 28 insertions(+), 17 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/search/query/TopDocsCollectorContext.java b/server/src/main/java/org/elasticsearch/search/query/TopDocsCollectorContext.java index 6312fd3856b66..2314d11e7e387 100644 --- a/server/src/main/java/org/elasticsearch/search/query/TopDocsCollectorContext.java +++ b/server/src/main/java/org/elasticsearch/search/query/TopDocsCollectorContext.java @@ -35,6 +35,7 @@ import org.apache.lucene.search.TopDocs; import org.apache.lucene.search.TopDocsCollector; import org.apache.lucene.search.TopFieldCollector; +import org.apache.lucene.search.TopFieldDocs; import org.apache.lucene.search.TopScoreDocCollector; import org.apache.lucene.search.TotalHitCountCollector; import org.apache.lucene.search.TotalHits; @@ -189,11 +190,11 @@ private static TopDocsCollector createCollector(@Nullable SortAndFormats sort } } - private final Collector collector; protected final @Nullable SortAndFormats sortAndFormats; - protected final Supplier totalHitsSupplier; - protected final Supplier topDocsSupplier; - protected final Supplier maxScoreSupplier; + private final Collector collector; + private final Supplier totalHitsSupplier; + private final Supplier topDocsSupplier; + private final Supplier maxScoreSupplier; /** * Ctr @@ -262,11 +263,23 @@ Collector create(Collector in) { return collector; } + TopDocsAndMaxScore newTopDocs() { + TopDocs in = topDocsSupplier.get(); + float maxScore = maxScoreSupplier.get(); + final TopDocs newTopDocs; + if (in instanceof TopFieldDocs) { + TopFieldDocs fieldDocs = (TopFieldDocs) in; + newTopDocs = new TopFieldDocs(totalHitsSupplier.get(), fieldDocs.scoreDocs, fieldDocs.fields); + } else { + newTopDocs = new TopDocs(totalHitsSupplier.get(), in.scoreDocs); + } + return new TopDocsAndMaxScore(newTopDocs, maxScore); + } + @Override void postProcess(QuerySearchResult result) throws IOException { - final TopDocs topDocs = new TopDocs(totalHitsSupplier.get(), topDocsSupplier.get().scoreDocs); - result.topDocs(new TopDocsAndMaxScore(topDocs, maxScoreSupplier.get()), - sortAndFormats == null ? null : sortAndFormats.formats); + final TopDocsAndMaxScore topDocs = newTopDocs(); + result.topDocs(topDocs, sortAndFormats == null ? null : sortAndFormats.formats); } } @@ -291,27 +304,25 @@ private ScrollingTopDocsCollectorContext(IndexReader reader, @Override void postProcess(QuerySearchResult result) throws IOException { - final TopDocs topDocs = new TopDocs(totalHitsSupplier.get(), topDocsSupplier.get().scoreDocs); - final float maxScore; + final TopDocsAndMaxScore topDocs = newTopDocs(); if (scrollContext.totalHits == null) { // first round - scrollContext.totalHits = topDocs.totalHits; - maxScore = scrollContext.maxScore = maxScoreSupplier.get(); + scrollContext.totalHits = topDocs.topDocs.totalHits; + scrollContext.maxScore = topDocs.maxScore; } else { // subsequent round: the total number of hits and // the maximum score were computed on the first round - topDocs.totalHits = scrollContext.totalHits; - maxScore = scrollContext.maxScore; + topDocs.topDocs.totalHits = scrollContext.totalHits; + topDocs.maxScore = scrollContext.maxScore; } if (numberOfShards == 1) { // if we fetch the document in the same roundtrip, we already know the last emitted doc - if (topDocs.scoreDocs.length > 0) { + if (topDocs.topDocs.scoreDocs.length > 0) { // set the last emitted doc - scrollContext.lastEmittedDoc = topDocs.scoreDocs[topDocs.scoreDocs.length - 1]; + scrollContext.lastEmittedDoc = topDocs.topDocs.scoreDocs[topDocs.topDocs.scoreDocs.length - 1]; } } - result.topDocs(new TopDocsAndMaxScore(topDocs, maxScore), - sortAndFormats == null ? null : sortAndFormats.formats); + result.topDocs(topDocs, sortAndFormats == null ? null : sortAndFormats.formats); } } From d625b79df2b1bb3a950038dfd39d27e93d4b57b0 Mon Sep 17 00:00:00 2001 From: Michael Basnight Date: Wed, 9 Jan 2019 22:09:34 -0600 Subject: [PATCH 063/186] Add getZone to JodaCompatibleZonedDateTime (#37084) The ZonedDateTime#getZone() was not accessible via the Joda shim. This commit adds getZone() and exposes it through painless. --- .../org/elasticsearch/painless/spi/org.elasticsearch.txt | 1 + .../org/elasticsearch/script/JodaCompatibleZonedDateTime.java | 4 ++++ .../script/JodaCompatibleZonedDateTimeTests.java | 4 ++++ 3 files changed, 9 insertions(+) diff --git a/modules/lang-painless/src/main/resources/org/elasticsearch/painless/spi/org.elasticsearch.txt b/modules/lang-painless/src/main/resources/org/elasticsearch/painless/spi/org.elasticsearch.txt index 2b4946d1ca852..88cb4d73adf6f 100644 --- a/modules/lang-painless/src/main/resources/org/elasticsearch/painless/spi/org.elasticsearch.txt +++ b/modules/lang-painless/src/main/resources/org/elasticsearch/painless/spi/org.elasticsearch.txt @@ -87,6 +87,7 @@ class org.elasticsearch.script.JodaCompatibleZonedDateTime { int getNano() int getSecond() int getYear() + ZoneId getZone() ZonedDateTime minus(TemporalAmount) ZonedDateTime minus(long,TemporalUnit) ZonedDateTime minusYears(long) diff --git a/server/src/main/java/org/elasticsearch/script/JodaCompatibleZonedDateTime.java b/server/src/main/java/org/elasticsearch/script/JodaCompatibleZonedDateTime.java index ec6492c9d5ec9..546deb3a24b68 100644 --- a/server/src/main/java/org/elasticsearch/script/JodaCompatibleZonedDateTime.java +++ b/server/src/main/java/org/elasticsearch/script/JodaCompatibleZonedDateTime.java @@ -151,6 +151,10 @@ public int getYear() { return dt.getYear(); } + public ZoneId getZone() { + return dt.getZone(); + } + public ZonedDateTime minus(TemporalAmount delta) { return dt.minus(delta); } diff --git a/server/src/test/java/org/elasticsearch/script/JodaCompatibleZonedDateTimeTests.java b/server/src/test/java/org/elasticsearch/script/JodaCompatibleZonedDateTimeTests.java index f01079d092fa7..4750ee36b0bd6 100644 --- a/server/src/test/java/org/elasticsearch/script/JodaCompatibleZonedDateTimeTests.java +++ b/server/src/test/java/org/elasticsearch/script/JodaCompatibleZonedDateTimeTests.java @@ -152,6 +152,10 @@ public void testYear() { assertThat(javaTime.getYear(), equalTo(jodaTime.getYear())); } + public void testZone() { + assertThat(javaTime.getZone().getId(), equalTo(jodaTime.getZone().getID())); + } + public void testMillis() { assertMethodDeprecation(() -> assertThat(javaTime.getMillis(), equalTo(jodaTime.getMillis())), "getMillis()", "toInstant().toEpochMilli()"); From 4a92de214a0811bfde65311aae65979866b0db5e Mon Sep 17 00:00:00 2001 From: Andrei Stefan Date: Thu, 10 Jan 2019 09:51:51 +0200 Subject: [PATCH 064/186] SQL: Proper handling of COUNT(field_name) and COUNT(DISTINCT field_name) (#37254) * provide overriden `hashCode` and toString methods to account for `DISTINCT` * change the analyzer for scenarios where `COUNT ` and `COUNT DISTINCT` have different paths * defined a new `filter` aggregation encapsulating an `exists` query to filter out null or missing values --- docs/reference/sql/functions/aggs.asciidoc | 33 ++++++- .../sql/qa/src/main/resources/agg.csv-spec | 57 +++++++++++- .../sql/qa/src/main/resources/agg.sql-spec | 12 +++ .../sql/qa/src/main/resources/docs.csv-spec | 18 +++- .../xpack/sql/analysis/analyzer/Analyzer.java | 21 ++++- .../search/extractor/MetricAggExtractor.java | 4 + .../function/UnresolvedFunction.java | 11 ++- .../expression/function/aggregate/Count.java | 43 ++++++++- .../function/grouping/GroupingFunction.java | 2 +- .../xpack/sql/planner/QueryFolder.java | 11 ++- .../xpack/sql/planner/QueryTranslator.java | 14 +-- .../sql/querydsl/agg/CardinalityAgg.java | 3 +- .../sql/querydsl/agg/FilterExistsAgg.java | 26 ++++++ .../sql/planner/QueryTranslatorTests.java | 91 +++++++++++++++++-- 14 files changed, 313 insertions(+), 33 deletions(-) create mode 100644 x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/querydsl/agg/FilterExistsAgg.java diff --git a/docs/reference/sql/functions/aggs.asciidoc b/docs/reference/sql/functions/aggs.asciidoc index aecb557d8eba0..b23b4ebbc9af4 100644 --- a/docs/reference/sql/functions/aggs.asciidoc +++ b/docs/reference/sql/functions/aggs.asciidoc @@ -53,11 +53,42 @@ COUNT(expression<1>) Returns the total number (count) of input values. +In case of `COUNT(*)` or `COUNT()`, _all_ values are considered (including `null` or missing ones). + +In case of `COUNT()` `null` values are not considered. + + ["source","sql",subs="attributes,macros"] -------------------------------------------------- include-tagged::{sql-specs}/docs.csv-spec[aggCountStar] -------------------------------------------------- + +[[sql-functions-aggs-count-all]] +===== `COUNT(ALL)` + +.Synopsis: +[source, sql] +-------------------------------------------------- +COUNT(ALL field_name<1>) +-------------------------------------------------- + +*Input*: + +<1> a field name + +*Output*: numeric value + +.Description: + +Returns the total number (count) of all _non-null_ input values. `COUNT()` and `COUNT(ALL )` are equivalent. + +["source","sql",subs="attributes,macros"] +-------------------------------------------------- +include-tagged::{sql-specs}/docs.csv-spec[aggCountAll] +-------------------------------------------------- + + [[sql-functions-aggs-count-distinct]] ===== `COUNT(DISTINCT)` @@ -75,7 +106,7 @@ COUNT(DISTINCT field_name<1>) .Description: -Returns the total number of _distinct_ values in input values. +Returns the total number of _distinct non-null_ values in input values. ["source","sql",subs="attributes,macros"] -------------------------------------------------- diff --git a/x-pack/plugin/sql/qa/src/main/resources/agg.csv-spec b/x-pack/plugin/sql/qa/src/main/resources/agg.csv-spec index f9576c7b859a6..bdb94321b76d5 100644 --- a/x-pack/plugin/sql/qa/src/main/resources/agg.csv-spec +++ b/x-pack/plugin/sql/qa/src/main/resources/agg.csv-spec @@ -309,4 +309,59 @@ SELECT HISTOGRAM(emp_no % 100, 10) AS h, COUNT(*) as c FROM test_emp GROUP BY h 20 |10 10 |10 0 |10 -; \ No newline at end of file +; + +countAll +schema::all_names:l|c:l +SELECT COUNT(ALL first_name) all_names, COUNT(*) c FROM test_emp; + + all_names | c +---------------+--------------- +90 |100 +; + +countAllCountTypesWithHaving +schema::ln:l|dln:l|fn:l|dfn:l|ccc:l +SELECT COUNT(last_name) ln, COUNT(distinct last_name) dln, COUNT(first_name) fn, COUNT(distinct first_name) dfn, COUNT(*) ccc FROM test_emp GROUP BY gender HAVING dln>5 AND ln>32 AND dfn>1 AND fn>1 AND ccc>5; + + ln | dln | fn | dfn | ccc +---------------+-------------+---------------+------------+------------- +33 |32 |32 |32 |33 +57 |54 |48 |48 |57 +; + +aggCountEqualityFalse +schema::areEqual:b|ln:l|dln:l +SELECT COUNT(last_name)=COUNT(DISTINCT last_name) AS areEqual, COUNT(last_name) ln, COUNT(DISTINCT last_name) dln FROM test_emp; + + areEqual | ln | dln +---------------+---------------+--------------- +false |100 |96 +; + +aggCountEqualityTrue +schema::areEqual:b|fn:l|dfn:l +SELECT COUNT(first_name)=COUNT(DISTINCT first_name) AS areEqual, COUNT(first_name) fn, COUNT(DISTINCT first_name) dfn FROM test_emp; + + areEqual | fn | dfn +---------------+---------------+--------------- +true |90 |90 +; + +aggCountAllEquality +schema::areEqual:b|afn:l +SELECT COUNT(first_name)=COUNT(ALL first_name) AS areEqual, COUNT(ALL first_name) afn FROM test_emp; + + areEqual | afn +---------------+--------------- +true |90 +; + +aggCountAllDifferentFields +schema::areEqual:b|afn:l|aln:l +SELECT COUNT(ALL last_name)=COUNT(ALL first_name) AS areEqual, COUNT(ALL first_name) afn, COUNT(ALL last_name) aln FROM test_emp; + + areEqual | afn | aln +---------------+---------------+--------------- +false |90 |100 +; diff --git a/x-pack/plugin/sql/qa/src/main/resources/agg.sql-spec b/x-pack/plugin/sql/qa/src/main/resources/agg.sql-spec index 39775fc13aed3..21dd7bf530e3d 100644 --- a/x-pack/plugin/sql/qa/src/main/resources/agg.sql-spec +++ b/x-pack/plugin/sql/qa/src/main/resources/agg.sql-spec @@ -210,6 +210,18 @@ SELECT gender g, languages l, COUNT(gender) c FROM "test_emp" GROUP BY g, l HAVI aggCountOnDistinctColumnAndHavingOnAliasAndFunctionConstantMultiGroupBy SELECT gender g, languages l, COUNT(DISTINCT last_name) c FROM "test_emp" GROUP BY g, l HAVING c > 5 AND COUNT(1) < 70 ORDER BY gender, languages; +aggCount +SELECT COUNT(last_name) c FROM test_emp; +aggCountAndCountDistinct +SELECT COUNT(last_name) c, COUNT(DISTINCT last_name) distinct_names FROM test_emp; +aggCountAndCountDistinctWithHaving +SELECT COUNT(last_name) c, COUNT(DISTINCT last_name) distinct_names, gender FROM test_emp GROUP BY gender HAVING distinct_names > 10 ORDER BY gender; +aggCountMultiComparisonWithHaving +SELECT COUNT(last_name) ln, COUNT(distinct last_name) dln, COUNT(first_name) fn, COUNT(distinct first_name) dfn, COUNT(*) ccc FROM test_emp GROUP BY gender HAVING dln>5 AND ln>32 AND dfn>1 AND fn>1 AND ccc>5 ORDER BY gender DESC; +aggCountMultiComparisonWithHavingAndNullGrouping +SELECT gender, COUNT(last_name) ln, COUNT(distinct last_name) dln, COUNT(first_name) fn, COUNT(distinct first_name) dfn, COUNT(*) ccc FROM test_emp GROUP BY gender HAVING dln>1 AND ln>1 AND dfn>1 AND fn>1 AND ccc>1 ORDER BY gender DESC; +aggCountWithHavingAndWhere +SELECT COUNT(last_name) c, COUNT(DISTINCT last_name) distinct_names, gender FROM test_emp WHERE salary > 65000 GROUP BY gender HAVING distinct_names > 10 ORDER BY gender; // MIN aggMinImplicit diff --git a/x-pack/plugin/sql/qa/src/main/resources/docs.csv-spec b/x-pack/plugin/sql/qa/src/main/resources/docs.csv-spec index e04a02558b9ac..2903292b1adff 100644 --- a/x-pack/plugin/sql/qa/src/main/resources/docs.csv-spec +++ b/x-pack/plugin/sql/qa/src/main/resources/docs.csv-spec @@ -977,14 +977,24 @@ SELECT COUNT(*) AS count FROM emp; // end::aggCountStar ; +aggCountAll +// tag::aggCountAll +SELECT COUNT(ALL last_name) AS count_all, COUNT(DISTINCT last_name) count_distinct FROM emp; + + count_all | count_distinct +---------------+------------------ +100 |96 +// end::aggCountAll +; + aggCountDistinct // tag::aggCountDistinct -SELECT COUNT(DISTINCT hire_date) AS hires FROM emp; +SELECT COUNT(DISTINCT hire_date) unique_hires, COUNT(hire_date) AS hires FROM emp; - hires ---------------- -99 + unique_hires | hires +----------------+--------------- +99 |100 // end::aggCountDistinct ; diff --git a/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/analysis/analyzer/Analyzer.java b/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/analysis/analyzer/Analyzer.java index dc94db5a74a55..090fea80b14c4 100644 --- a/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/analysis/analyzer/Analyzer.java +++ b/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/analysis/analyzer/Analyzer.java @@ -28,6 +28,7 @@ import org.elasticsearch.xpack.sql.expression.function.FunctionRegistry; import org.elasticsearch.xpack.sql.expression.function.Functions; import org.elasticsearch.xpack.sql.expression.function.UnresolvedFunction; +import org.elasticsearch.xpack.sql.expression.function.aggregate.Count; import org.elasticsearch.xpack.sql.expression.function.scalar.Cast; import org.elasticsearch.xpack.sql.expression.predicate.operator.arithmetic.ArithmeticOperation; import org.elasticsearch.xpack.sql.plan.TableIdentifier; @@ -770,7 +771,15 @@ private Expression collectResolvedAndReplace(Expression e, Map list = getList(seen, fName); for (Function seenFunction : list) { if (seenFunction != f && f.arguments().equals(seenFunction.arguments())) { - return seenFunction; + // Special check for COUNT: an already seen COUNT function will be returned only if its DISTINCT property + // matches the one from the unresolved function to be checked. + if (seenFunction instanceof Count) { + if (seenFunction.equals(f)){ + return seenFunction; + } + } else { + return seenFunction; + } } } list.add(f); @@ -808,7 +817,15 @@ protected LogicalPlan resolve(LogicalPlan plan, Map> seen if (!list.isEmpty()) { for (Function seenFunction : list) { if (uf.arguments().equals(seenFunction.arguments())) { - return seenFunction; + // Special check for COUNT: an already seen COUNT function will be returned only if its DISTINCT property + // matches the one from the unresolved function to be checked. + if (seenFunction instanceof Count) { + if (uf.sameAs((Count) seenFunction)) { + return seenFunction; + } + } else { + return seenFunction; + } } } } diff --git a/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/execution/search/extractor/MetricAggExtractor.java b/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/execution/search/extractor/MetricAggExtractor.java index fc3f6753525d4..e24bf4d0adaa5 100644 --- a/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/execution/search/extractor/MetricAggExtractor.java +++ b/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/execution/search/extractor/MetricAggExtractor.java @@ -9,6 +9,7 @@ import org.elasticsearch.common.io.stream.StreamOutput; import org.elasticsearch.search.aggregations.InternalAggregation; import org.elasticsearch.search.aggregations.bucket.MultiBucketsAggregation.Bucket; +import org.elasticsearch.search.aggregations.bucket.filter.InternalFilter; import org.elasticsearch.search.aggregations.matrix.stats.MatrixStats; import org.elasticsearch.search.aggregations.metrics.InternalNumericMetricsAggregation; import org.elasticsearch.search.aggregations.metrics.InternalNumericMetricsAggregation.SingleValue; @@ -83,6 +84,9 @@ public Object extract(Bucket bucket) { // throw new SqlIllegalArgumentException("Invalid innerKey {} specified for aggregation {}", innerKey, name); //} return ((InternalNumericMetricsAggregation.MultiValue) agg).value(property); + } else if (agg instanceof InternalFilter) { + // COUNT(expr) and COUNT(ALL expr) uses this type of aggregation to account for non-null values only + return ((InternalFilter) agg).getDocCount(); } Object v = agg.getProperty(property); diff --git a/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/expression/function/UnresolvedFunction.java b/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/expression/function/UnresolvedFunction.java index 13e038977ae1e..82b4d95a9b47b 100644 --- a/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/expression/function/UnresolvedFunction.java +++ b/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/expression/function/UnresolvedFunction.java @@ -11,10 +11,11 @@ import org.elasticsearch.xpack.sql.expression.Expression; import org.elasticsearch.xpack.sql.expression.Literal; import org.elasticsearch.xpack.sql.expression.Nullability; +import org.elasticsearch.xpack.sql.expression.function.aggregate.Count; import org.elasticsearch.xpack.sql.expression.gen.script.ScriptTemplate; import org.elasticsearch.xpack.sql.session.Configuration; -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.StringUtils; @@ -129,6 +130,14 @@ ResolutionType resolutionType() { public boolean analyzed() { return analyzed; } + + public boolean sameAs(Count count) { + if (this.resolutionType == ResolutionType.DISTINCT && count.distinct() + || this.resolutionType == ResolutionType.STANDARD && count.distinct() == false) { + return true; + } + return false; + } @Override public DataType dataType() { diff --git a/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/expression/function/aggregate/Count.java b/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/expression/function/aggregate/Count.java index b4afa6762dea4..429b4e7ba0774 100644 --- a/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/expression/function/aggregate/Count.java +++ b/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/expression/function/aggregate/Count.java @@ -5,14 +5,16 @@ */ package org.elasticsearch.xpack.sql.expression.function.aggregate; -import java.util.List; - import org.elasticsearch.xpack.sql.expression.Expression; +import org.elasticsearch.xpack.sql.expression.Literal; import org.elasticsearch.xpack.sql.expression.NamedExpression; -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 java.util.List; +import java.util.Objects; + /** * Count the number of documents matched ({@code COUNT}) * OR count the number of distinct values @@ -53,17 +55,48 @@ public DataType dataType() { public String functionId() { String functionId = id().toString(); // if count works against a given expression, use its id (to identify the group) - if (field() instanceof NamedExpression) { + // in case of COUNT DISTINCT don't use the expression id to avoid possible duplicate IDs when COUNT and COUNT DISTINCT is used + // in the same query + if (!distinct() && field() instanceof NamedExpression) { functionId = ((NamedExpression) field()).id().toString(); } return functionId; } + @Override + public String name() { + if (distinct()) { + StringBuilder sb = new StringBuilder(super.name()); + sb.insert(sb.indexOf("(") + 1, "DISTINCT "); + return sb.toString(); + } + return super.name(); + } + @Override public AggregateFunctionAttribute toAttribute() { - if (!distinct()) { + // COUNT(*) gets its value from the parent aggregation on which _count is called + if (field() instanceof Literal) { return new AggregateFunctionAttribute(source(), name(), dataType(), id(), functionId(), "_count"); } + // COUNT(column) gets its value from a sibling aggregation (an exists filter agg) by calling its id and then _count on it + if (!distinct()) { + return new AggregateFunctionAttribute(source(), name(), dataType(), id(), functionId(), functionId() + "._count"); + } return super.toAttribute(); } + + @Override + public boolean equals(Object obj) { + if (false == super.equals(obj)) { + return false; + } + Count other = (Count) obj; + return Objects.equals(other.distinct(), distinct()); + } + + @Override + public int hashCode() { + return Objects.hash(super.hashCode(), distinct()); + } } diff --git a/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/expression/function/grouping/GroupingFunction.java b/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/expression/function/grouping/GroupingFunction.java index a158394a52e46..0595e29176a93 100644 --- a/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/expression/function/grouping/GroupingFunction.java +++ b/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/expression/function/grouping/GroupingFunction.java @@ -51,7 +51,7 @@ public List parameters() { @Override public GroupingFunctionAttribute toAttribute() { if (lazyAttribute == null) { - // this is highly correlated with QueryFolder$FoldAggregate#addFunction (regarding the function name within the querydsl) + // this is highly correlated with QueryFolder$FoldAggregate#addAggFunction (regarding the function name within the querydsl) lazyAttribute = new GroupingFunctionAttribute(source(), name(), dataType(), id(), functionId()); } return lazyAttribute; diff --git a/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/planner/QueryFolder.java b/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/planner/QueryFolder.java index f55221e5fd5d4..46380a9de2afd 100644 --- a/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/planner/QueryFolder.java +++ b/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/planner/QueryFolder.java @@ -13,6 +13,7 @@ import org.elasticsearch.xpack.sql.expression.Expression; import org.elasticsearch.xpack.sql.expression.Expressions; import org.elasticsearch.xpack.sql.expression.Foldables; +import org.elasticsearch.xpack.sql.expression.Literal; import org.elasticsearch.xpack.sql.expression.NamedExpression; import org.elasticsearch.xpack.sql.expression.Order; import org.elasticsearch.xpack.sql.expression.function.Function; @@ -380,7 +381,8 @@ private Tuple addAggFunction(GroupByKey groupingAg // handle count as a special case agg if (f instanceof Count) { Count c = (Count) f; - if (!c.distinct()) { + // COUNT(*) or COUNT() + if (c.field() instanceof Literal) { AggRef ref = groupingAgg == null ? GlobalCountRef.INSTANCE : new GroupByRef(groupingAgg.id(), Property.COUNT, null); @@ -388,7 +390,14 @@ private Tuple addAggFunction(GroupByKey groupingAg Map pseudoFunctions = new LinkedHashMap<>(queryC.pseudoFunctions()); pseudoFunctions.put(functionId, groupingAgg); return new Tuple<>(queryC.withPseudoFunctions(pseudoFunctions), new AggPathInput(f, ref)); + // COUNT() + } else if (!c.distinct()) { + LeafAgg leafAgg = toAgg(functionId, f); + AggPathInput a = new AggPathInput(f, new MetricAggRef(leafAgg.id(), "doc_count", "_count")); + queryC = queryC.with(queryC.aggs().addAgg(leafAgg)); + return new Tuple<>(queryC, a); } + // the only variant left - COUNT(DISTINCT) - will be covered by the else branch below } AggPathInput aggInput = null; diff --git a/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/planner/QueryTranslator.java b/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/planner/QueryTranslator.java index a34b3325ee763..e0472f27131f0 100644 --- a/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/planner/QueryTranslator.java +++ b/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/planner/QueryTranslator.java @@ -62,6 +62,7 @@ import org.elasticsearch.xpack.sql.querydsl.agg.AvgAgg; import org.elasticsearch.xpack.sql.querydsl.agg.CardinalityAgg; import org.elasticsearch.xpack.sql.querydsl.agg.ExtendedStatsAgg; +import org.elasticsearch.xpack.sql.querydsl.agg.FilterExistsAgg; import org.elasticsearch.xpack.sql.querydsl.agg.GroupByDateHistogram; import org.elasticsearch.xpack.sql.querydsl.agg.GroupByKey; import org.elasticsearch.xpack.sql.querydsl.agg.GroupByNumericHistogram; @@ -135,7 +136,7 @@ private QueryTranslator(){} new MatrixStatsAggs(), new PercentilesAggs(), new PercentileRanksAggs(), - new DistinctCounts(), + new CountAggs(), new DateTimes() ); @@ -778,15 +779,16 @@ protected QueryTranslation asQuery(ScalarFunction f, boolean onAggs) { // // Agg translators // - - static class DistinctCounts extends SingleValueAggTranslator { + + static class CountAggs extends SingleValueAggTranslator { @Override protected LeafAgg toAgg(String id, Count c) { - if (!c.distinct()) { - return null; + if (c.distinct()) { + return new CardinalityAgg(id, field(c)); + } else { + return new FilterExistsAgg(id, field(c)); } - return new CardinalityAgg(id, field(c)); } } diff --git a/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/querydsl/agg/CardinalityAgg.java b/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/querydsl/agg/CardinalityAgg.java index f4fb20428c59f..847509125ba49 100644 --- a/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/querydsl/agg/CardinalityAgg.java +++ b/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/querydsl/agg/CardinalityAgg.java @@ -15,7 +15,8 @@ public CardinalityAgg(String id, String fieldName) { super(id, fieldName); } - @Override AggregationBuilder toBuilder() { + @Override + AggregationBuilder toBuilder() { return cardinality(id()).field(fieldName()); } } diff --git a/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/querydsl/agg/FilterExistsAgg.java b/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/querydsl/agg/FilterExistsAgg.java new file mode 100644 index 0000000000000..4c14fa6dad82a --- /dev/null +++ b/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/querydsl/agg/FilterExistsAgg.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.sql.querydsl.agg; + +import org.elasticsearch.index.query.QueryBuilders; +import org.elasticsearch.search.aggregations.AggregationBuilder; + +import static org.elasticsearch.search.aggregations.AggregationBuilders.filter; + +/** + * Aggregation builder for a "filter" aggregation encapsulating an "exists" query. + */ +public class FilterExistsAgg extends LeafAgg { + + public FilterExistsAgg(String id, String fieldName) { + super(id, fieldName); + } + + @Override + AggregationBuilder toBuilder() { + return filter(id(), QueryBuilders.existsQuery(fieldName())); + } +} diff --git a/x-pack/plugin/sql/src/test/java/org/elasticsearch/xpack/sql/planner/QueryTranslatorTests.java b/x-pack/plugin/sql/src/test/java/org/elasticsearch/xpack/sql/planner/QueryTranslatorTests.java index 861c201440285..8ca1d64bd12fd 100644 --- a/x-pack/plugin/sql/src/test/java/org/elasticsearch/xpack/sql/planner/QueryTranslatorTests.java +++ b/x-pack/plugin/sql/src/test/java/org/elasticsearch/xpack/sql/planner/QueryTranslatorTests.java @@ -5,7 +5,10 @@ */ package org.elasticsearch.xpack.sql.planner; +import org.elasticsearch.index.query.ExistsQueryBuilder; import org.elasticsearch.search.aggregations.AggregationBuilder; +import org.elasticsearch.search.aggregations.bucket.filter.FilterAggregationBuilder; +import org.elasticsearch.search.aggregations.metrics.AvgAggregationBuilder; import org.elasticsearch.search.aggregations.metrics.CardinalityAggregationBuilder; import org.elasticsearch.test.ESTestCase; import org.elasticsearch.xpack.sql.SqlIllegalArgumentException; @@ -478,22 +481,90 @@ public void testGroupByHistogram() { assertEquals(DataType.DATE, field.dataType()); } - public void testCountDistinctCardinalityFolder() { - PhysicalPlan p = optimizeAndPlan("SELECT COUNT(DISTINCT keyword) cnt FROM test GROUP BY bool HAVING cnt = 0"); + public void testCountAndCountDistinctFolding() { + PhysicalPlan p = optimizeAndPlan("SELECT COUNT(DISTINCT keyword) dkey, COUNT(keyword) key FROM test"); assertEquals(EsQueryExec.class, p.getClass()); EsQueryExec ee = (EsQueryExec) p; - assertEquals(1, ee.output().size()); - assertThat(ee.output().get(0).toString(), startsWith("cnt{a->")); + assertEquals(2, ee.output().size()); + assertThat(ee.output().get(0).toString(), startsWith("dkey{a->")); + assertThat(ee.output().get(1).toString(), startsWith("key{a->")); Collection subAggs = ee.queryContainer().aggs().asAggBuilder().getSubAggregations(); - assertEquals(1, subAggs.size()); + assertEquals(2, subAggs.size()); assertTrue(subAggs.toArray()[0] instanceof CardinalityAggregationBuilder); + assertTrue(subAggs.toArray()[1] instanceof FilterAggregationBuilder); + + CardinalityAggregationBuilder cardinalityKeyword = (CardinalityAggregationBuilder) subAggs.toArray()[0]; + assertEquals("keyword", cardinalityKeyword.field()); + + FilterAggregationBuilder existsKeyword = (FilterAggregationBuilder) subAggs.toArray()[1]; + assertTrue(existsKeyword.getFilter() instanceof ExistsQueryBuilder); + assertEquals("keyword", ((ExistsQueryBuilder) existsKeyword.getFilter()).fieldName()); + + assertThat(ee.queryContainer().aggs().asAggBuilder().toString().replaceAll("\\s+", ""), + endsWith("{\"filter\":{\"exists\":{\"field\":\"keyword\",\"boost\":1.0}}}}}}")); + } + + public void testAllCountVariantsWithHavingGenerateCorrectAggregations() { + PhysicalPlan p = optimizeAndPlan("SELECT AVG(int), COUNT(keyword) ln, COUNT(distinct keyword) dln, COUNT(some.dotted.field) fn," + + "COUNT(distinct some.dotted.field) dfn, COUNT(*) ccc FROM test GROUP BY bool " + + "HAVING dln > 3 AND ln > 32 AND dfn > 1 AND fn > 2 AND ccc > 5 AND AVG(int) > 50000"); + assertEquals(EsQueryExec.class, p.getClass()); + EsQueryExec ee = (EsQueryExec) p; + assertEquals(6, ee.output().size()); + assertThat(ee.output().get(0).toString(), startsWith("AVG(int){a->")); + assertThat(ee.output().get(1).toString(), startsWith("ln{a->")); + assertThat(ee.output().get(2).toString(), startsWith("dln{a->")); + assertThat(ee.output().get(3).toString(), startsWith("fn{a->")); + assertThat(ee.output().get(4).toString(), startsWith("dfn{a->")); + assertThat(ee.output().get(5).toString(), startsWith("ccc{a->")); - CardinalityAggregationBuilder cardinalityAgg = (CardinalityAggregationBuilder) subAggs.toArray()[0]; - assertEquals("keyword", cardinalityAgg.field()); + Collection subAggs = ee.queryContainer().aggs().asAggBuilder().getSubAggregations(); + assertEquals(5, subAggs.size()); + assertTrue(subAggs.toArray()[0] instanceof AvgAggregationBuilder); + assertTrue(subAggs.toArray()[1] instanceof FilterAggregationBuilder); + assertTrue(subAggs.toArray()[2] instanceof CardinalityAggregationBuilder); + assertTrue(subAggs.toArray()[3] instanceof FilterAggregationBuilder); + assertTrue(subAggs.toArray()[4] instanceof CardinalityAggregationBuilder); + + AvgAggregationBuilder avgInt = (AvgAggregationBuilder) subAggs.toArray()[0]; + assertEquals("int", avgInt.field()); + + FilterAggregationBuilder existsKeyword = (FilterAggregationBuilder) subAggs.toArray()[1]; + assertTrue(existsKeyword.getFilter() instanceof ExistsQueryBuilder); + assertEquals("keyword", ((ExistsQueryBuilder) existsKeyword.getFilter()).fieldName()); + + CardinalityAggregationBuilder cardinalityKeyword = (CardinalityAggregationBuilder) subAggs.toArray()[2]; + assertEquals("keyword", cardinalityKeyword.field()); + + FilterAggregationBuilder existsDottedField = (FilterAggregationBuilder) subAggs.toArray()[3]; + assertTrue(existsDottedField.getFilter() instanceof ExistsQueryBuilder); + assertEquals("some.dotted.field", ((ExistsQueryBuilder) existsDottedField.getFilter()).fieldName()); + + CardinalityAggregationBuilder cardinalityDottedField = (CardinalityAggregationBuilder) subAggs.toArray()[4]; + assertEquals("some.dotted.field", cardinalityDottedField.field()); + assertThat(ee.queryContainer().aggs().asAggBuilder().toString().replaceAll("\\s+", ""), - endsWith("{\"buckets_path\":{\"a0\":\"" + cardinalityAgg.getName() +"\"},\"script\":{" - + "\"source\":\"InternalSqlScriptUtils.nullSafeFilter(InternalSqlScriptUtils.eq(params.a0,params.v0))\"," - + "\"lang\":\"painless\",\"params\":{\"v0\":0}},\"gap_policy\":\"skip\"}}}}}")); + endsWith("{\"buckets_path\":{" + + "\"a0\":\"" + cardinalityKeyword.getName() + "\"," + + "\"a1\":\"" + existsKeyword.getName() + "._count\"," + + "\"a2\":\"" + cardinalityDottedField.getName() + "\"," + + "\"a3\":\"" + existsDottedField.getName() + "._count\"," + + "\"a4\":\"_count\"," + + "\"a5\":\"" + avgInt.getName() + "\"}," + + "\"script\":{\"source\":\"" + + "InternalSqlScriptUtils.nullSafeFilter(InternalSqlScriptUtils.and(" + + "InternalSqlScriptUtils.nullSafeFilter(InternalSqlScriptUtils.and(" + + "InternalSqlScriptUtils.nullSafeFilter(InternalSqlScriptUtils.and(" + + "InternalSqlScriptUtils.nullSafeFilter(InternalSqlScriptUtils.and(" + + "InternalSqlScriptUtils.nullSafeFilter(InternalSqlScriptUtils.and(" + + "InternalSqlScriptUtils.nullSafeFilter(InternalSqlScriptUtils.gt(params.a0,params.v0))," + + "InternalSqlScriptUtils.nullSafeFilter(InternalSqlScriptUtils.gt(params.a1,params.v1))))," + + "InternalSqlScriptUtils.nullSafeFilter(InternalSqlScriptUtils.gt(params.a2,params.v2))))," + + "InternalSqlScriptUtils.nullSafeFilter(InternalSqlScriptUtils.gt(params.a3,params.v3))))," + + "InternalSqlScriptUtils.nullSafeFilter(InternalSqlScriptUtils.gt(params.a4,params.v4))))," + + "InternalSqlScriptUtils.nullSafeFilter(InternalSqlScriptUtils.gt(params.a5,params.v5))))\"," + + "\"lang\":\"painless\",\"params\":{\"v0\":3,\"v1\":32,\"v2\":1,\"v3\":2,\"v4\":5,\"v5\":50000}}," + + "\"gap_policy\":\"skip\"}}}}}")); } } From b2e8437424660a7552f638a7749dc652e606ae49 Mon Sep 17 00:00:00 2001 From: Alexander Reelsen Date: Thu, 10 Jan 2019 09:25:36 +0100 Subject: [PATCH 065/186] Tests: Add ElasticsearchAssertions.awaitLatch method (#36777) * Tests: Add ElasticsearchAssertions.awaitLatch method Some tests are using assertTrue(latch.await(...)) in their code. This leads to an assertion error without any error message. This adds a method which has a nicer error message and can be used in tests. * fix forbidden apis * fix spaces --- .../transport/RemoteClusterServiceTests.java | 11 +++++----- .../hamcrest/ElasticsearchAssertions.java | 20 +++++++++++++++++++ .../IndexLifecycleRunnerTests.java | 11 +++++----- 3 files changed, 32 insertions(+), 10 deletions(-) diff --git a/server/src/test/java/org/elasticsearch/transport/RemoteClusterServiceTests.java b/server/src/test/java/org/elasticsearch/transport/RemoteClusterServiceTests.java index 258d99a347b3d..d5671eec21961 100644 --- a/server/src/test/java/org/elasticsearch/transport/RemoteClusterServiceTests.java +++ b/server/src/test/java/org/elasticsearch/transport/RemoteClusterServiceTests.java @@ -60,6 +60,7 @@ import java.util.function.Supplier; import java.util.stream.Collectors; +import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.awaitLatch; import static org.hamcrest.CoreMatchers.containsString; import static org.hamcrest.CoreMatchers.instanceOf; import static org.hamcrest.Matchers.anyOf; @@ -742,7 +743,7 @@ public void testCollectSearchShards() throws Exception { AtomicReference failure = new AtomicReference<>(); remoteClusterService.collectSearchShards(IndicesOptions.lenientExpandOpen(), null, null, remoteIndicesByCluster, new LatchedActionListener<>(ActionListener.wrap(response::set, failure::set), latch)); - assertTrue(latch.await(5, TimeUnit.SECONDS)); + awaitLatch(latch, 5, TimeUnit.SECONDS); assertNull(failure.get()); assertNotNull(response.get()); Map map = response.get(); @@ -761,7 +762,7 @@ public void testCollectSearchShards() throws Exception { remoteClusterService.collectSearchShards(IndicesOptions.lenientExpandOpen(), "index_not_found", null, remoteIndicesByCluster, new LatchedActionListener<>(ActionListener.wrap(response::set, failure::set), latch)); - assertTrue(latch.await(5, TimeUnit.SECONDS)); + awaitLatch(latch, 5, TimeUnit.SECONDS); assertNull(response.get()); assertNotNull(failure.get()); assertThat(failure.get(), instanceOf(RemoteTransportException.class)); @@ -800,7 +801,7 @@ public void onNodeDisconnected(DiscoveryNode node) { AtomicReference failure = new AtomicReference<>(); remoteClusterService.collectSearchShards(IndicesOptions.lenientExpandOpen(), null, null, remoteIndicesByCluster, new LatchedActionListener<>(ActionListener.wrap(response::set, failure::set), latch)); - assertTrue(latch.await(5, TimeUnit.SECONDS)); + awaitLatch(latch, 5, TimeUnit.SECONDS); assertNull(response.get()); assertNotNull(failure.get()); assertThat(failure.get(), instanceOf(RemoteTransportException.class)); @@ -818,7 +819,7 @@ public void onNodeDisconnected(DiscoveryNode node) { AtomicReference failure = new AtomicReference<>(); remoteClusterService.collectSearchShards(IndicesOptions.lenientExpandOpen(), null, null, remoteIndicesByCluster, new LatchedActionListener<>(ActionListener.wrap(response::set, failure::set), latch)); - assertTrue(latch.await(5, TimeUnit.SECONDS)); + awaitLatch(latch, 5, TimeUnit.SECONDS); assertNull(failure.get()); assertNotNull(response.get()); Map map = response.get(); @@ -854,7 +855,7 @@ public void onNodeDisconnected(DiscoveryNode node) { AtomicReference failure = new AtomicReference<>(); remoteClusterService.collectSearchShards(IndicesOptions.lenientExpandOpen(), null, null, remoteIndicesByCluster, new LatchedActionListener<>(ActionListener.wrap(response::set, failure::set), latch)); - assertTrue(latch.await(5, TimeUnit.SECONDS)); + awaitLatch(latch, 5, TimeUnit.SECONDS); assertNull(failure.get()); assertNotNull(response.get()); Map map = response.get(); 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 48d18e096bf17..f18824b27e6c7 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 @@ -47,6 +47,7 @@ import org.elasticsearch.cluster.metadata.IndexTemplateMetaData; import org.elasticsearch.common.Nullable; import org.elasticsearch.common.bytes.BytesReference; +import org.elasticsearch.common.unit.TimeValue; import org.elasticsearch.common.xcontent.DeprecationHandler; import org.elasticsearch.common.xcontent.NamedXContentRegistry; import org.elasticsearch.common.xcontent.ToXContent; @@ -72,6 +73,8 @@ import java.util.Locale; import java.util.Map; import java.util.Set; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.TimeUnit; import java.util.function.Function; import static org.hamcrest.CoreMatchers.equalTo; @@ -693,6 +696,23 @@ public static void assertToXContentEquivalent(BytesReference expected, BytesRefe } } + /** + * Wait for a latch to countdown and provide a useful error message if it does not + * Often latches are called as assertTrue(latch.await(1, TimeUnit.SECONDS)); + * In case of a failure this will just throw an assertion error without any further message + * + * @param latch The latch to wait for + * @param timeout The value of the timeout + * @param unit The unit of the timeout + * @throws InterruptedException An exception if the waiting is interrupted + */ + public static void awaitLatch(CountDownLatch latch, long timeout, TimeUnit unit) throws InterruptedException { + TimeValue timeValue = new TimeValue(timeout, unit); + String message = String.format(Locale.ROOT, "expected latch to be counted down after %s, but was not", timeValue); + boolean isCountedDown = latch.await(timeout, unit); + assertThat(message, isCountedDown, is(true)); + } + /** * Compares two maps recursively, using arrays comparisons for byte[] through Arrays.equals(byte[], byte[]) */ diff --git a/x-pack/plugin/ilm/src/test/java/org/elasticsearch/xpack/indexlifecycle/IndexLifecycleRunnerTests.java b/x-pack/plugin/ilm/src/test/java/org/elasticsearch/xpack/indexlifecycle/IndexLifecycleRunnerTests.java index 7693a752b28a1..f4ab15a30e880 100644 --- a/x-pack/plugin/ilm/src/test/java/org/elasticsearch/xpack/indexlifecycle/IndexLifecycleRunnerTests.java +++ b/x-pack/plugin/ilm/src/test/java/org/elasticsearch/xpack/indexlifecycle/IndexLifecycleRunnerTests.java @@ -71,6 +71,7 @@ import java.util.function.Function; import java.util.stream.Collectors; +import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.awaitLatch; import static org.elasticsearch.xpack.core.indexlifecycle.LifecycleExecutionState.ILM_CUSTOM_METADATA_KEY; import static org.elasticsearch.xpack.core.indexlifecycle.LifecyclePolicyTestsUtils.newTestLifecyclePolicy; import static org.hamcrest.Matchers.containsString; @@ -203,7 +204,7 @@ public void testRunStateChangePolicyWithNoNextStep() throws Exception { step.setLatch(latch); runner.runPolicyAfterStateChange(policyName, indexMetaData); - latch.await(5, TimeUnit.SECONDS); + awaitLatch(latch, 5, TimeUnit.SECONDS); ClusterState after = clusterService.state(); assertEquals(before, after); @@ -264,7 +265,7 @@ public void testRunStateChangePolicyWithNextStep() throws Exception { nextStep.setLatch(latch); runner.runPolicyAfterStateChange(policyName, indexMetaData); - assertTrue(latch.await(5, TimeUnit.SECONDS)); + awaitLatch(latch, 5, TimeUnit.SECONDS); // The cluster state can take a few extra milliseconds to update after the steps are executed assertBusy(() -> assertNotEquals(before, clusterService.state())); @@ -373,13 +374,13 @@ public void testRunStateChangePolicyWithAsyncActionNextStep() throws Exception { runner.runPolicyAfterStateChange(policyName, indexMetaData); // Wait for the cluster state action step - latch.await(5, TimeUnit.SECONDS); + awaitLatch(latch, 5, TimeUnit.SECONDS); CountDownLatch asyncLatch = new CountDownLatch(1); nextStep.setLatch(asyncLatch); // Wait for the async action step - asyncLatch.await(5, TimeUnit.SECONDS); + awaitLatch(asyncLatch, 5, TimeUnit.SECONDS); ClusterState after = clusterService.state(); assertNotEquals(before, after); @@ -440,7 +441,7 @@ public void testRunPeriodicStep() throws Exception { CountDownLatch latch = new CountDownLatch(1); step.setLatch(latch); runner.runPeriodicStep(policyName, indexMetaData); - latch.await(5, TimeUnit.SECONDS); + awaitLatch(latch, 5, TimeUnit.SECONDS); ClusterState after = clusterService.state(); From eb12de550aeae0a4d88927183c7c8f3af8facf31 Mon Sep 17 00:00:00 2001 From: Alexander Reelsen Date: Thu, 10 Jan 2019 09:26:01 +0100 Subject: [PATCH 066/186] Java Time: Fix timezone parsing (#37262) * Java Time: Fix timezone parsing An independent test uncovered an issue when parsing a timezone containing a colon like `01:00` - some formats did not properly support this. This commit adds test for all formats in the dueling tests and fixes a few issues with existing date formatters. * fix tests, so they run under java8 --- .../common/time/DateFormatters.java | 69 ++++++++-- .../joda/JavaJodaTimeDuellingTests.java | 122 ++++++++++++++++++ .../common/time/DateFormattersTests.java | 12 ++ 3 files changed, 189 insertions(+), 14 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/common/time/DateFormatters.java b/server/src/main/java/org/elasticsearch/common/time/DateFormatters.java index a9d953248a2ce..d5a8f4b7a5112 100644 --- a/server/src/main/java/org/elasticsearch/common/time/DateFormatters.java +++ b/server/src/main/java/org/elasticsearch/common/time/DateFormatters.java @@ -38,6 +38,7 @@ import java.time.temporal.IsoFields; import java.time.temporal.TemporalAccessor; import java.time.temporal.TemporalAdjusters; +import java.time.temporal.TemporalQueries; import java.time.temporal.WeekFields; import java.util.Collections; import java.util.List; @@ -122,20 +123,19 @@ public class DateFormatters { .optionalStart() .appendZoneOrOffsetId() .optionalEnd() + .optionalStart() + .appendOffset("+HHmm", "Z") + .optionalEnd() .optionalEnd() .toFormatter(Locale.ROOT); - private static final DateTimeFormatter STRICT_DATE_OPTIONAL_TIME_FORMATTER_WITH_NANOS_2 = new DateTimeFormatterBuilder() + private static final DateTimeFormatter STRICT_DATE_OPTIONAL_TIME_PRINTER = new DateTimeFormatterBuilder() .append(STRICT_YEAR_MONTH_DAY_FORMATTER) - .optionalStart() .appendLiteral('T') .append(STRICT_HOUR_MINUTE_SECOND_FORMATTER) - .optionalStart() .appendFraction(NANO_OF_SECOND, 3, 9, true) - .optionalEnd() .optionalStart() - .appendOffset("+HHmm", "Z") - .optionalEnd() + .appendZoneOrOffsetId() .optionalEnd() .toFormatter(Locale.ROOT); @@ -143,9 +143,7 @@ public class DateFormatters { * Returns a generic ISO datetime parser where the date is mandatory and the time is optional with nanosecond resolution. */ private static final DateFormatter STRICT_DATE_OPTIONAL_TIME_NANOS = new JavaDateFormatter("strict_date_optional_time_nanos", - STRICT_DATE_OPTIONAL_TIME_FORMATTER_WITH_NANOS_1, - STRICT_DATE_OPTIONAL_TIME_FORMATTER_WITH_NANOS_1, STRICT_DATE_OPTIONAL_TIME_FORMATTER_WITH_NANOS_2); - + STRICT_DATE_OPTIONAL_TIME_PRINTER, STRICT_DATE_OPTIONAL_TIME_FORMATTER_WITH_NANOS_1); ///////////////////////////////////////// // // BEGIN basic time formatters @@ -279,6 +277,8 @@ public class DateFormatters { * digit year and three digit dayOfYear (yyyyDDD'T'HHmmss.SSSZ). */ private static final DateFormatter BASIC_ORDINAL_DATE_TIME = new JavaDateFormatter("basic_ordinal_date_time", + new DateTimeFormatterBuilder().appendPattern("yyyyDDD").append(BASIC_T_TIME_PRINTER) + .appendZoneOrOffsetId().toFormatter(Locale.ROOT), new DateTimeFormatterBuilder().appendPattern("yyyyDDD").append(BASIC_T_TIME_PRINTER) .appendZoneOrOffsetId().toFormatter(Locale.ROOT), new DateTimeFormatterBuilder().appendPattern("yyyyDDD").append(BASIC_T_TIME_FORMATTER) @@ -346,10 +346,28 @@ public class DateFormatters { */ private static final DateFormatter STRICT_BASIC_WEEK_DATE_TIME_NO_MILLIS = new JavaDateFormatter("strict_basic_week_date_no_millis", new DateTimeFormatterBuilder() - .append(STRICT_BASIC_WEEK_DATE_PRINTER).append(DateTimeFormatter.ofPattern("'T'HHmmssX", Locale.ROOT)) + .append(STRICT_BASIC_WEEK_DATE_PRINTER) + .appendLiteral("T") + .appendValue(HOUR_OF_DAY, 2, 2, SignStyle.NOT_NEGATIVE) + .appendValue(MINUTE_OF_HOUR, 2, 2, SignStyle.NOT_NEGATIVE) + .appendValue(SECOND_OF_MINUTE, 2, 2, SignStyle.NOT_NEGATIVE) + .appendZoneOrOffsetId() .toFormatter(Locale.ROOT), new DateTimeFormatterBuilder() - .append(STRICT_BASIC_WEEK_DATE_FORMATTER).append(DateTimeFormatter.ofPattern("'T'HHmmssX", Locale.ROOT)) + .append(STRICT_BASIC_WEEK_DATE_PRINTER) + .appendLiteral("T") + .appendValue(HOUR_OF_DAY, 2, 2, SignStyle.NOT_NEGATIVE) + .appendValue(MINUTE_OF_HOUR, 2, 2, SignStyle.NOT_NEGATIVE) + .appendValue(SECOND_OF_MINUTE, 2, 2, SignStyle.NOT_NEGATIVE) + .appendZoneOrOffsetId() + .toFormatter(Locale.ROOT), + new DateTimeFormatterBuilder() + .append(STRICT_BASIC_WEEK_DATE_PRINTER) + .appendLiteral("T") + .appendValue(HOUR_OF_DAY, 2, 2, SignStyle.NOT_NEGATIVE) + .appendValue(MINUTE_OF_HOUR, 2, 2, SignStyle.NOT_NEGATIVE) + .appendValue(SECOND_OF_MINUTE, 2, 2, SignStyle.NOT_NEGATIVE) + .append(TIME_ZONE_FORMATTER_NO_COLON) .toFormatter(Locale.ROOT) ); @@ -363,9 +381,23 @@ public class DateFormatters { .append(DateTimeFormatter.ofPattern("'T'HHmmss.SSSX", Locale.ROOT)) .toFormatter(Locale.ROOT), new DateTimeFormatterBuilder() - .append(STRICT_BASIC_WEEK_DATE_FORMATTER) - .append(DateTimeFormatter.ofPattern("'T'HHmmss.SSSX", Locale.ROOT)) - .toFormatter(Locale.ROOT) + .append(STRICT_BASIC_WEEK_DATE_FORMATTER) + .appendLiteral("T") + .appendValue(HOUR_OF_DAY, 2, 2, SignStyle.NOT_NEGATIVE) + .appendValue(MINUTE_OF_HOUR, 2, 2, SignStyle.NOT_NEGATIVE) + .appendValue(SECOND_OF_MINUTE, 2, 2, SignStyle.NOT_NEGATIVE) + .appendFraction(MILLI_OF_SECOND, 3, 3, true) + .appendZoneOrOffsetId() + .toFormatter(Locale.ROOT), + new DateTimeFormatterBuilder() + .append(STRICT_BASIC_WEEK_DATE_FORMATTER) + .appendLiteral("T") + .appendValue(HOUR_OF_DAY, 2, 2, SignStyle.NOT_NEGATIVE) + .appendValue(MINUTE_OF_HOUR, 2, 2, SignStyle.NOT_NEGATIVE) + .appendValue(SECOND_OF_MINUTE, 2, 2, SignStyle.NOT_NEGATIVE) + .appendFraction(MILLI_OF_SECOND, 3, 3, true) + .append(TIME_ZONE_FORMATTER_NO_COLON) + .toFormatter(Locale.ROOT) ); /* @@ -447,6 +479,8 @@ public class DateFormatters { * using a four digit year and three digit dayOfYear (yyyy-DDD'T'HH:mm:ssZZ). */ private static final DateFormatter STRICT_ORDINAL_DATE_TIME_NO_MILLIS = new JavaDateFormatter("strict_ordinal_date_time_no_millis", + new DateTimeFormatterBuilder().append(STRICT_ORDINAL_DATE_TIME_NO_MILLIS_BASE) + .appendZoneOrOffsetId().toFormatter(Locale.ROOT), new DateTimeFormatterBuilder().append(STRICT_ORDINAL_DATE_TIME_NO_MILLIS_BASE) .appendZoneOrOffsetId().toFormatter(Locale.ROOT), new DateTimeFormatterBuilder().append(STRICT_ORDINAL_DATE_TIME_NO_MILLIS_BASE) @@ -551,6 +585,8 @@ public class DateFormatters { * digit year and three digit dayOfYear (yyyy-DDD'T'HH:mm:ss.SSSZZ). */ private static final DateFormatter STRICT_ORDINAL_DATE_TIME = new JavaDateFormatter("strict_ordinal_date_time", + new DateTimeFormatterBuilder().append(STRICT_ORDINAL_DATE_TIME_FORMATTER_BASE) + .appendZoneOrOffsetId().toFormatter(Locale.ROOT), new DateTimeFormatterBuilder().append(STRICT_ORDINAL_DATE_TIME_FORMATTER_BASE) .appendZoneOrOffsetId().toFormatter(Locale.ROOT), new DateTimeFormatterBuilder().append(STRICT_ORDINAL_DATE_TIME_FORMATTER_BASE) @@ -1627,6 +1663,11 @@ public static ZonedDateTime toZonedDateTime(TemporalAccessor accessor, ZonedDate result = result.with(ChronoField.NANO_OF_SECOND, accessor.getLong(ChronoField.NANO_OF_SECOND)); } + ZoneId zoneOffset = accessor.query(TemporalQueries.zone()); + if (zoneOffset != null) { + result = result.withZoneSameLocal(zoneOffset); + } + return result; } } diff --git a/server/src/test/java/org/elasticsearch/common/joda/JavaJodaTimeDuellingTests.java b/server/src/test/java/org/elasticsearch/common/joda/JavaJodaTimeDuellingTests.java index 23674ec85b44f..a96fee5b6333e 100644 --- a/server/src/test/java/org/elasticsearch/common/joda/JavaJodaTimeDuellingTests.java +++ b/server/src/test/java/org/elasticsearch/common/joda/JavaJodaTimeDuellingTests.java @@ -86,18 +86,36 @@ public void testDuellingFormatsValidParsing() { assertSameDate("20181126T121212.123-0800", "basic_date_time"); assertSameDate("20181126T121212Z", "basic_date_time_no_millis"); + assertSameDate("20181126T121212+01:00", "basic_date_time_no_millis"); + assertSameDate("20181126T121212+0100", "basic_date_time_no_millis"); assertSameDate("2018363", "basic_ordinal_date"); assertSameDate("2018363T121212.123Z", "basic_ordinal_date_time"); + assertSameDate("2018363T121212.123+0100", "basic_ordinal_date_time"); + assertSameDate("2018363T121212.123+01:00", "basic_ordinal_date_time"); assertSameDate("2018363T121212Z", "basic_ordinal_date_time_no_millis"); + assertSameDate("2018363T121212+0100", "basic_ordinal_date_time_no_millis"); + assertSameDate("2018363T121212+01:00", "basic_ordinal_date_time_no_millis"); assertSameDate("121212.123Z", "basic_time"); + assertSameDate("121212.123+0100", "basic_time"); + assertSameDate("121212.123+01:00", "basic_time"); assertSameDate("121212Z", "basic_time_no_millis"); + assertSameDate("121212+0100", "basic_time_no_millis"); + assertSameDate("121212+01:00", "basic_time_no_millis"); assertSameDate("T121212.123Z", "basic_t_time"); + assertSameDate("T121212.123+0100", "basic_t_time"); + assertSameDate("T121212.123+01:00", "basic_t_time"); assertSameDate("T121212Z", "basic_t_time_no_millis"); + assertSameDate("T121212+0100", "basic_t_time_no_millis"); + assertSameDate("T121212+01:00", "basic_t_time_no_millis"); assertSameDate("2018W313", "basic_week_date"); assertSameDate("1W313", "basic_week_date"); assertSameDate("18W313", "basic_week_date"); assertSameDate("2018W313T121212.123Z", "basic_week_date_time"); + assertSameDate("2018W313T121212.123+0100", "basic_week_date_time"); + assertSameDate("2018W313T121212.123+01:00", "basic_week_date_time"); assertSameDate("2018W313T121212Z", "basic_week_date_time_no_millis"); + assertSameDate("2018W313T121212+0100", "basic_week_date_time_no_millis"); + assertSameDate("2018W313T121212+01:00", "basic_week_date_time_no_millis"); assertSameDate("2018-12-31", "date"); assertSameDate("18-5-6", "date"); @@ -127,6 +145,9 @@ public void testDuellingFormatsValidParsing() { assertSameDate("2018-05-30T20:21", "date_optional_time"); assertSameDate("2018-05-30T20:21:23", "date_optional_time"); assertSameDate("2018-05-30T20:21:23.123", "date_optional_time"); + assertSameDate("2018-05-30T20:21:23.123Z", "date_optional_time"); + assertSameDate("2018-05-30T20:21:23.123+0100", "date_optional_time"); + assertSameDate("2018-05-30T20:21:23.123+01:00", "date_optional_time"); assertSameDate("2018-12-1", "date_optional_time"); assertSameDate("2018-12-31T10:15:30", "date_optional_time"); assertSameDate("2018-12-31T10:15:3", "date_optional_time"); @@ -134,13 +155,27 @@ public void testDuellingFormatsValidParsing() { assertSameDate("2018-12-31T1:15:30", "date_optional_time"); assertSameDate("2018-12-31T10:15:30.123Z", "date_time"); + assertSameDate("2018-12-31T10:15:30.123+0100", "date_time"); + assertSameDate("2018-12-31T10:15:30.123+01:00", "date_time"); assertSameDate("2018-12-31T10:15:30.11Z", "date_time"); + assertSameDate("2018-12-31T10:15:30.11+0100", "date_time"); + assertSameDate("2018-12-31T10:15:30.11+01:00", "date_time"); assertSameDate("2018-12-31T10:15:3.123Z", "date_time"); + assertSameDate("2018-12-31T10:15:3.123+0100", "date_time"); + assertSameDate("2018-12-31T10:15:3.123+01:00", "date_time"); assertSameDate("2018-12-31T10:15:30Z", "date_time_no_millis"); + assertSameDate("2018-12-31T10:15:30+0100", "date_time_no_millis"); + assertSameDate("2018-12-31T10:15:30+01:00", "date_time_no_millis"); assertSameDate("2018-12-31T10:5:30Z", "date_time_no_millis"); + assertSameDate("2018-12-31T10:5:30+0100", "date_time_no_millis"); + assertSameDate("2018-12-31T10:5:30+01:00", "date_time_no_millis"); assertSameDate("2018-12-31T10:15:3Z", "date_time_no_millis"); + assertSameDate("2018-12-31T10:15:3+0100", "date_time_no_millis"); + assertSameDate("2018-12-31T10:15:3+01:00", "date_time_no_millis"); assertSameDate("2018-12-31T1:15:30Z", "date_time_no_millis"); + assertSameDate("2018-12-31T1:15:30+0100", "date_time_no_millis"); + assertSameDate("2018-12-31T1:15:30+01:00", "date_time_no_millis"); assertSameDate("12", "hour"); assertSameDate("01", "hour"); @@ -165,36 +200,78 @@ public void testDuellingFormatsValidParsing() { assertSameDate("2018-1", "ordinal_date"); assertSameDate("2018-128T10:15:30.123Z", "ordinal_date_time"); + assertSameDate("2018-128T10:15:30.123+0100", "ordinal_date_time"); + assertSameDate("2018-128T10:15:30.123+01:00", "ordinal_date_time"); assertSameDate("2018-1T10:15:30.123Z", "ordinal_date_time"); + assertSameDate("2018-1T10:15:30.123+0100", "ordinal_date_time"); + assertSameDate("2018-1T10:15:30.123+01:00", "ordinal_date_time"); assertSameDate("2018-128T10:15:30Z", "ordinal_date_time_no_millis"); + assertSameDate("2018-128T10:15:30+0100", "ordinal_date_time_no_millis"); + assertSameDate("2018-128T10:15:30+01:00", "ordinal_date_time_no_millis"); assertSameDate("2018-1T10:15:30Z", "ordinal_date_time_no_millis"); + assertSameDate("2018-1T10:15:30+0100", "ordinal_date_time_no_millis"); + assertSameDate("2018-1T10:15:30+01:00", "ordinal_date_time_no_millis"); assertSameDate("10:15:30.123Z", "time"); + assertSameDate("10:15:30.123+0100", "time"); + assertSameDate("10:15:30.123+01:00", "time"); assertSameDate("1:15:30.123Z", "time"); + assertSameDate("1:15:30.123+0100", "time"); + assertSameDate("1:15:30.123+01:00", "time"); assertSameDate("10:1:30.123Z", "time"); + assertSameDate("10:1:30.123+0100", "time"); + assertSameDate("10:1:30.123+01:00", "time"); assertSameDate("10:15:3.123Z", "time"); + assertSameDate("10:15:3.123+0100", "time"); + assertSameDate("10:15:3.123+01:00", "time"); assertParseException("10:15:3.1", "time"); assertParseException("10:15:3Z", "time"); assertSameDate("10:15:30Z", "time_no_millis"); + assertSameDate("10:15:30+0100", "time_no_millis"); + assertSameDate("10:15:30+01:00", "time_no_millis"); assertSameDate("01:15:30Z", "time_no_millis"); + assertSameDate("01:15:30+0100", "time_no_millis"); + assertSameDate("01:15:30+01:00", "time_no_millis"); assertSameDate("1:15:30Z", "time_no_millis"); + assertSameDate("1:15:30+0100", "time_no_millis"); + assertSameDate("1:15:30+01:00", "time_no_millis"); assertSameDate("10:5:30Z", "time_no_millis"); + assertSameDate("10:5:30+0100", "time_no_millis"); + assertSameDate("10:5:30+01:00", "time_no_millis"); assertSameDate("10:15:3Z", "time_no_millis"); + assertSameDate("10:15:3+0100", "time_no_millis"); + assertSameDate("10:15:3+01:00", "time_no_millis"); assertParseException("10:15:3", "time_no_millis"); assertSameDate("T10:15:30.123Z", "t_time"); + assertSameDate("T10:15:30.123+0100", "t_time"); + assertSameDate("T10:15:30.123+01:00", "t_time"); assertSameDate("T1:15:30.123Z", "t_time"); + assertSameDate("T1:15:30.123+0100", "t_time"); + assertSameDate("T1:15:30.123+01:00", "t_time"); assertSameDate("T10:1:30.123Z", "t_time"); + assertSameDate("T10:1:30.123+0100", "t_time"); + assertSameDate("T10:1:30.123+01:00", "t_time"); assertSameDate("T10:15:3.123Z", "t_time"); + assertSameDate("T10:15:3.123+0100", "t_time"); + assertSameDate("T10:15:3.123+01:00", "t_time"); assertParseException("T10:15:3.1", "t_time"); assertParseException("T10:15:3Z", "t_time"); assertSameDate("T10:15:30Z", "t_time_no_millis"); + assertSameDate("T10:15:30+0100", "t_time_no_millis"); + assertSameDate("T10:15:30+01:00", "t_time_no_millis"); assertSameDate("T1:15:30Z", "t_time_no_millis"); + assertSameDate("T1:15:30+0100", "t_time_no_millis"); + assertSameDate("T1:15:30+01:00", "t_time_no_millis"); assertSameDate("T10:1:30Z", "t_time_no_millis"); + assertSameDate("T10:1:30+0100", "t_time_no_millis"); + assertSameDate("T10:1:30+01:00", "t_time_no_millis"); assertSameDate("T10:15:3Z", "t_time_no_millis"); + assertSameDate("T10:15:3+0100", "t_time_no_millis"); + assertSameDate("T10:15:3+01:00", "t_time_no_millis"); assertParseException("T10:15:3", "t_time_no_millis"); assertSameDate("2012-W48-6", "week_date"); @@ -206,10 +283,18 @@ public void testDuellingFormatsValidParsing() { assertJavaTimeParseException("2012-W1-8", "week_date", "Text '2012-W1-8' could not be parsed"); assertSameDate("2012-W48-6T10:15:30.123Z", "week_date_time"); + assertSameDate("2012-W48-6T10:15:30.123+0100", "week_date_time"); + assertSameDate("2012-W48-6T10:15:30.123+01:00", "week_date_time"); assertSameDate("2012-W1-6T10:15:30.123Z", "week_date_time"); + assertSameDate("2012-W1-6T10:15:30.123+0100", "week_date_time"); + assertSameDate("2012-W1-6T10:15:30.123+01:00", "week_date_time"); assertSameDate("2012-W48-6T10:15:30Z", "week_date_time_no_millis"); + assertSameDate("2012-W48-6T10:15:30+0100", "week_date_time_no_millis"); + assertSameDate("2012-W48-6T10:15:30+01:00", "week_date_time_no_millis"); assertSameDate("2012-W1-6T10:15:30Z", "week_date_time_no_millis"); + assertSameDate("2012-W1-6T10:15:30+0100", "week_date_time_no_millis"); + assertSameDate("2012-W1-6T10:15:30+01:00", "week_date_time_no_millis"); assertSameDate("2012", "year"); assertSameDate("1", "year"); @@ -238,14 +323,24 @@ public void testDuelingStrictParsing() { assertSameDate("2018W313", "strict_basic_week_date"); assertParseException("18W313", "strict_basic_week_date"); assertSameDate("2018W313T121212.123Z", "strict_basic_week_date_time"); + assertSameDate("2018W313T121212.123+0100", "strict_basic_week_date_time"); + assertSameDate("2018W313T121212.123+01:00", "strict_basic_week_date_time"); assertParseException("2018W313T12128.123Z", "strict_basic_week_date_time"); assertParseException("2018W313T81212.123Z", "strict_basic_week_date_time"); assertParseException("2018W313T12812.123Z", "strict_basic_week_date_time"); assertParseException("2018W313T12812.1Z", "strict_basic_week_date_time"); assertSameDate("2018W313T121212Z", "strict_basic_week_date_time_no_millis"); + assertSameDate("2018W313T121212+0100", "strict_basic_week_date_time_no_millis"); + assertSameDate("2018W313T121212+01:00", "strict_basic_week_date_time_no_millis"); assertParseException("2018W313T12128Z", "strict_basic_week_date_time_no_millis"); + assertParseException("2018W313T12128+0100", "strict_basic_week_date_time_no_millis"); + assertParseException("2018W313T12128+01:00", "strict_basic_week_date_time_no_millis"); assertParseException("2018W313T81212Z", "strict_basic_week_date_time_no_millis"); + assertParseException("2018W313T81212+0100", "strict_basic_week_date_time_no_millis"); + assertParseException("2018W313T81212+01:00", "strict_basic_week_date_time_no_millis"); assertParseException("2018W313T12812Z", "strict_basic_week_date_time_no_millis"); + assertParseException("2018W313T12812+0100", "strict_basic_week_date_time_no_millis"); + assertParseException("2018W313T12812+01:00", "strict_basic_week_date_time_no_millis"); assertSameDate("2018-12-31", "strict_date"); assertParseException("10000-12-31", "strict_date"); assertParseException("2018-8-31", "strict_date"); @@ -266,15 +361,24 @@ public void testDuelingStrictParsing() { assertParseException("2018-1-31", "strict_date_optional_time"); assertParseException("10000-01-31", "strict_date_optional_time"); assertSameDate("2018-12-31T10:15:30", "strict_date_optional_time"); + assertSameDate("2018-12-31T10:15:30Z", "strict_date_optional_time"); + assertSameDate("2018-12-31T10:15:30+0100", "strict_date_optional_time"); + assertSameDate("2018-12-31T10:15:30+01:00", "strict_date_optional_time"); assertParseException("2018-12-31T10:15:3", "strict_date_optional_time"); assertParseException("2018-12-31T10:5:30", "strict_date_optional_time"); assertParseException("2018-12-31T9:15:30", "strict_date_optional_time"); assertSameDate("2018-12-31T10:15:30.123Z", "strict_date_time"); + assertSameDate("2018-12-31T10:15:30.123+0100", "strict_date_time"); + assertSameDate("2018-12-31T10:15:30.123+01:00", "strict_date_time"); assertSameDate("2018-12-31T10:15:30.11Z", "strict_date_time"); + assertSameDate("2018-12-31T10:15:30.11+0100", "strict_date_time"); + assertSameDate("2018-12-31T10:15:30.11+01:00", "strict_date_time"); assertParseException("2018-12-31T10:15:3.123Z", "strict_date_time"); assertParseException("2018-12-31T10:5:30.123Z", "strict_date_time"); assertParseException("2018-12-31T1:15:30.123Z", "strict_date_time"); assertSameDate("2018-12-31T10:15:30Z", "strict_date_time_no_millis"); + assertSameDate("2018-12-31T10:15:30+0100", "strict_date_time_no_millis"); + assertSameDate("2018-12-31T10:15:30+01:00", "strict_date_time_no_millis"); assertParseException("2018-12-31T10:5:30Z", "strict_date_time_no_millis"); assertParseException("2018-12-31T10:15:3Z", "strict_date_time_no_millis"); assertParseException("2018-12-31T1:15:30Z", "strict_date_time_no_millis"); @@ -297,12 +401,18 @@ public void testDuelingStrictParsing() { assertParseException("2018-1", "strict_ordinal_date"); assertSameDate("2018-128T10:15:30.123Z", "strict_ordinal_date_time"); + assertSameDate("2018-128T10:15:30.123+0100", "strict_ordinal_date_time"); + assertSameDate("2018-128T10:15:30.123+01:00", "strict_ordinal_date_time"); assertParseException("2018-1T10:15:30.123Z", "strict_ordinal_date_time"); assertSameDate("2018-128T10:15:30Z", "strict_ordinal_date_time_no_millis"); + assertSameDate("2018-128T10:15:30+0100", "strict_ordinal_date_time_no_millis"); + assertSameDate("2018-128T10:15:30+01:00", "strict_ordinal_date_time_no_millis"); assertParseException("2018-1T10:15:30Z", "strict_ordinal_date_time_no_millis"); assertSameDate("10:15:30.123Z", "strict_time"); + assertSameDate("10:15:30.123+0100", "strict_time"); + assertSameDate("10:15:30.123+01:00", "strict_time"); assertParseException("1:15:30.123Z", "strict_time"); assertParseException("10:1:30.123Z", "strict_time"); assertParseException("10:15:3.123Z", "strict_time"); @@ -310,13 +420,19 @@ public void testDuelingStrictParsing() { assertParseException("10:15:3Z", "strict_time"); assertSameDate("10:15:30Z", "strict_time_no_millis"); + assertSameDate("10:15:30+0100", "strict_time_no_millis"); + assertSameDate("10:15:30+01:00", "strict_time_no_millis"); assertSameDate("01:15:30Z", "strict_time_no_millis"); + assertSameDate("01:15:30+0100", "strict_time_no_millis"); + assertSameDate("01:15:30+01:00", "strict_time_no_millis"); assertParseException("1:15:30Z", "strict_time_no_millis"); assertParseException("10:5:30Z", "strict_time_no_millis"); assertParseException("10:15:3Z", "strict_time_no_millis"); assertParseException("10:15:3", "strict_time_no_millis"); assertSameDate("T10:15:30.123Z", "strict_t_time"); + assertSameDate("T10:15:30.123+0100", "strict_t_time"); + assertSameDate("T10:15:30.123+01:00", "strict_t_time"); assertParseException("T1:15:30.123Z", "strict_t_time"); assertParseException("T10:1:30.123Z", "strict_t_time"); assertParseException("T10:15:3.123Z", "strict_t_time"); @@ -324,6 +440,8 @@ public void testDuelingStrictParsing() { assertParseException("T10:15:3Z", "strict_t_time"); assertSameDate("T10:15:30Z", "strict_t_time_no_millis"); + assertSameDate("T10:15:30+0100", "strict_t_time_no_millis"); + assertSameDate("T10:15:30+01:00", "strict_t_time_no_millis"); assertParseException("T1:15:30Z", "strict_t_time_no_millis"); assertParseException("T10:1:30Z", "strict_t_time_no_millis"); assertParseException("T10:15:3Z", "strict_t_time_no_millis"); @@ -343,9 +461,13 @@ public void testDuelingStrictParsing() { assertJavaTimeParseException("2012-W01-8", "strict_week_date", "Text '2012-W01-8' could not be parsed"); assertSameDate("2012-W48-6T10:15:30.123Z", "strict_week_date_time"); + assertSameDate("2012-W48-6T10:15:30.123+0100", "strict_week_date_time"); + assertSameDate("2012-W48-6T10:15:30.123+01:00", "strict_week_date_time"); assertParseException("2012-W1-6T10:15:30.123Z", "strict_week_date_time"); assertSameDate("2012-W48-6T10:15:30Z", "strict_week_date_time_no_millis"); + assertSameDate("2012-W48-6T10:15:30+0100", "strict_week_date_time_no_millis"); + assertSameDate("2012-W48-6T10:15:30+01:00", "strict_week_date_time_no_millis"); assertParseException("2012-W1-6T10:15:30Z", "strict_week_date_time_no_millis"); assertSameDate("2012", "strict_year"); diff --git a/server/src/test/java/org/elasticsearch/common/time/DateFormattersTests.java b/server/src/test/java/org/elasticsearch/common/time/DateFormattersTests.java index b60e6b27eca03..8d79f9d3600e7 100644 --- a/server/src/test/java/org/elasticsearch/common/time/DateFormattersTests.java +++ b/server/src/test/java/org/elasticsearch/common/time/DateFormattersTests.java @@ -162,4 +162,16 @@ public void testForceJava8() { assertThat(mergedFormatter.formatters.get(0), instanceOf(JavaDateFormatter.class)); assertThat(mergedFormatter.formatters.get(1), instanceOf(JavaDateFormatter.class)); } + + public void testParsingStrictNanoDates() { + DateFormatter formatter = DateFormatters.forPattern("strict_date_optional_time_nanos"); + formatter.format(formatter.parse("2016-01-01T00:00:00.000")); + formatter.format(formatter.parse("2018-05-15T17:14:56")); + formatter.format(formatter.parse("2018-05-15T17:14:56Z")); + formatter.format(formatter.parse("2018-05-15T17:14:56+0100")); + formatter.format(formatter.parse("2018-05-15T17:14:56+01:00")); + formatter.format(formatter.parse("2018-05-15T17:14:56.123456789Z")); + formatter.format(formatter.parse("2018-05-15T17:14:56.123456789+0100")); + formatter.format(formatter.parse("2018-05-15T17:14:56.123456789+01:00")); + } } From 1a41d84536aa6eca84d6c929af595c91edf3a525 Mon Sep 17 00:00:00 2001 From: Martijn van Groningen Date: Thu, 10 Jan 2019 09:48:26 +0100 Subject: [PATCH 067/186] [CCR] Resume follow Api should not require a request body (#37217) Closes #37022 --- .../rest-api-spec/test/ccr/follow_and_unfollow.yml | 1 - .../xpack/ccr/rest/RestResumeFollowAction.java | 10 ++++++++-- .../resources/rest-api-spec/api/ccr.resume_follow.json | 2 +- 3 files changed, 9 insertions(+), 4 deletions(-) diff --git a/x-pack/plugin/ccr/qa/rest/src/test/resources/rest-api-spec/test/ccr/follow_and_unfollow.yml b/x-pack/plugin/ccr/qa/rest/src/test/resources/rest-api-spec/test/ccr/follow_and_unfollow.yml index d50bc52bc3620..f73f5c6dfb2d3 100644 --- a/x-pack/plugin/ccr/qa/rest/src/test/resources/rest-api-spec/test/ccr/follow_and_unfollow.yml +++ b/x-pack/plugin/ccr/qa/rest/src/test/resources/rest-api-spec/test/ccr/follow_and_unfollow.yml @@ -52,7 +52,6 @@ - do: ccr.resume_follow: index: bar - body: {} - is_true: acknowledged - do: diff --git a/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/rest/RestResumeFollowAction.java b/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/rest/RestResumeFollowAction.java index 62b3f6323ab88..ce2eab52e0cab 100644 --- a/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/rest/RestResumeFollowAction.java +++ b/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/rest/RestResumeFollowAction.java @@ -37,8 +37,14 @@ protected RestChannelConsumer prepareRequest(RestRequest restRequest, NodeClient } static Request createRequest(RestRequest restRequest) throws IOException { - try (XContentParser parser = restRequest.contentOrSourceParamParser()) { - return Request.fromXContent(parser, restRequest.param("index")); + if (restRequest.hasContentOrSourceParam()) { + try (XContentParser parser = restRequest.contentOrSourceParamParser()) { + return Request.fromXContent(parser, restRequest.param("index")); + } + } else { + Request request = new Request(); + request.setFollowerIndex(restRequest.param("index")); + return request; } } } diff --git a/x-pack/plugin/src/test/resources/rest-api-spec/api/ccr.resume_follow.json b/x-pack/plugin/src/test/resources/rest-api-spec/api/ccr.resume_follow.json index 61bdf82372fc0..61e3b8580fc2c 100644 --- a/x-pack/plugin/src/test/resources/rest-api-spec/api/ccr.resume_follow.json +++ b/x-pack/plugin/src/test/resources/rest-api-spec/api/ccr.resume_follow.json @@ -15,7 +15,7 @@ }, "body": { "description" : "The name of the leader index and other optional ccr related parameters", - "required" : true + "required" : false } } } From 3d66764660debae71a059136a36ea5412c905c2c Mon Sep 17 00:00:00 2001 From: Alpar Torok Date: Wed, 9 Jan 2019 20:49:42 +0200 Subject: [PATCH 068/186] Mute watcher SingleNodeTests Tracking: #36782 --- .../xpack/watcher/test/integration/SingleNodeTests.java | 2 ++ 1 file changed, 2 insertions(+) diff --git a/x-pack/plugin/watcher/src/test/java/org/elasticsearch/xpack/watcher/test/integration/SingleNodeTests.java b/x-pack/plugin/watcher/src/test/java/org/elasticsearch/xpack/watcher/test/integration/SingleNodeTests.java index 6d2d63d4f4158..b03d75af113af 100644 --- a/x-pack/plugin/watcher/src/test/java/org/elasticsearch/xpack/watcher/test/integration/SingleNodeTests.java +++ b/x-pack/plugin/watcher/src/test/java/org/elasticsearch/xpack/watcher/test/integration/SingleNodeTests.java @@ -5,6 +5,7 @@ */ package org.elasticsearch.xpack.watcher.test.integration; +import org.apache.lucene.util.LuceneTestCase; import org.elasticsearch.action.admin.cluster.state.ClusterStateResponse; import org.elasticsearch.action.search.SearchResponse; import org.elasticsearch.cluster.metadata.IndexMetaData; @@ -27,6 +28,7 @@ import static org.hamcrest.Matchers.greaterThanOrEqualTo; import static org.hamcrest.Matchers.is; +@LuceneTestCase.AwaitsFix(bugUrl = "https://github.com/elastic/elasticsearch/issues/36782") @ClusterScope(scope = SUITE, numClientNodes = 0, transportClientRatio = 0, maxNumDataNodes = 1, supportsDedicatedMasters = false) public class SingleNodeTests extends AbstractWatcherIntegrationTestCase { From cd608848e7fb6645816df9044b0b6419f16dfe5a Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Christoph=20B=C3=BCscher?= Date: Thu, 10 Jan 2019 12:50:07 +0100 Subject: [PATCH 069/186] Remove deprecated QUERY_AND_FETCH SearchType (#37257) This SearchType was deprecated since at least 6.0 and according to the documentation is only kept around for pre-5.3 requests. Removing and leaving a comment as placeholder so we don't reuse the byte value associated with it without further consideration. --- .../org/elasticsearch/action/search/SearchType.java | 11 +++-------- .../action/search/TransportSearchAction.java | 4 ++-- 2 files changed, 5 insertions(+), 10 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/action/search/SearchType.java b/server/src/main/java/org/elasticsearch/action/search/SearchType.java index 910bc3d676a05..1a382fa9caf27 100644 --- a/server/src/main/java/org/elasticsearch/action/search/SearchType.java +++ b/server/src/main/java/org/elasticsearch/action/search/SearchType.java @@ -36,14 +36,9 @@ public enum SearchType { * document content. The return number of hits is exactly as specified in size, since they are the only ones that * are fetched. This is very handy when the index has a lot of shards (not replicas, shard id groups). */ - QUERY_THEN_FETCH((byte) 1), + QUERY_THEN_FETCH((byte) 1); // 2 used to be DFS_QUERY_AND_FETCH - - /** - * Only used for pre 5.3 request where this type is still needed - */ - @Deprecated - QUERY_AND_FETCH((byte) 3); + // 3 used to be QUERY_AND_FETCH /** * The default search type ({@link #QUERY_THEN_FETCH}. @@ -99,5 +94,5 @@ public static SearchType fromString(String searchType) { throw new IllegalArgumentException("No search type for [" + searchType + "]"); } } - + } diff --git a/server/src/main/java/org/elasticsearch/action/search/TransportSearchAction.java b/server/src/main/java/org/elasticsearch/action/search/TransportSearchAction.java index 463d1487d0307..88e2764982cb4 100644 --- a/server/src/main/java/org/elasticsearch/action/search/TransportSearchAction.java +++ b/server/src/main/java/org/elasticsearch/action/search/TransportSearchAction.java @@ -42,6 +42,7 @@ import org.elasticsearch.index.Index; import org.elasticsearch.index.query.Rewriteable; import org.elasticsearch.index.shard.ShardId; +import org.elasticsearch.search.SearchPhaseResult; import org.elasticsearch.search.SearchService; import org.elasticsearch.search.builder.SearchSourceBuilder; import org.elasticsearch.search.internal.AliasFilter; @@ -413,14 +414,13 @@ public void run() { }; }, clusters); } else { - AbstractSearchAsyncAction searchAsyncAction; + AbstractSearchAsyncAction searchAsyncAction; switch (searchRequest.searchType()) { case DFS_QUERY_THEN_FETCH: searchAsyncAction = new SearchDfsQueryThenFetchAsyncAction(logger, searchTransportService, connectionLookup, aliasFilter, concreteIndexBoosts, indexRoutings, searchPhaseController, executor, searchRequest, listener, shardIterators, timeProvider, clusterStateVersion, task, clusters); break; - case QUERY_AND_FETCH: case QUERY_THEN_FETCH: searchAsyncAction = new SearchQueryThenFetchAsyncAction(logger, searchTransportService, connectionLookup, aliasFilter, concreteIndexBoosts, indexRoutings, searchPhaseController, executor, searchRequest, listener, From d499233068a840a632b42b5f890708241666912f Mon Sep 17 00:00:00 2001 From: Yannick Welsch Date: Thu, 10 Jan 2019 12:57:50 +0100 Subject: [PATCH 070/186] Zen2: Add join validation (#37203) Adds join validation to Zen2, which prevents a node from joining a cluster when the node does not have the right ES version or does not satisfy any other of the join validation constraints. --- .../cluster/coordination/Coordinator.java | 50 ++++++++- .../cluster/coordination/JoinHelper.java | 45 +++++++- .../coordination/JoinTaskExecutor.java | 15 +++ .../PublicationTransportHandler.java | 28 +++-- .../discovery/DiscoveryModule.java | 4 +- .../discovery/zen/MembershipAction.java | 6 +- .../discovery/zen/ZenDiscovery.java | 14 +-- .../TransportBootstrapClusterActionTests.java | 3 +- ...ransportGetDiscoveredNodesActionTests.java | 2 +- .../coordination/CoordinatorTests.java | 106 +++++++++++++++--- .../cluster/coordination/JoinHelperTests.java | 3 +- .../cluster/coordination/NodeJoinTests.java | 3 + .../discovery/zen/ZenDiscoveryUnitTests.java | 3 +- .../test/discovery/TestZenDiscovery.java | 2 +- 14 files changed, 234 insertions(+), 50 deletions(-) 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 f5ef24a17ade5..233423a391c19 100644 --- a/server/src/main/java/org/elasticsearch/cluster/coordination/Coordinator.java +++ b/server/src/main/java/org/elasticsearch/cluster/coordination/Coordinator.java @@ -71,12 +71,14 @@ import org.elasticsearch.transport.TransportService; import java.util.ArrayList; +import java.util.Collection; import java.util.HashSet; import java.util.List; import java.util.Optional; import java.util.Random; import java.util.Set; import java.util.concurrent.atomic.AtomicBoolean; +import java.util.function.BiConsumer; import java.util.function.Supplier; import java.util.stream.Collectors; import java.util.stream.StreamSupport; @@ -119,6 +121,7 @@ public class Coordinator extends AbstractLifecycleComponent implements Discovery private final LeaderChecker leaderChecker; private final FollowersChecker followersChecker; private final ClusterApplier clusterApplier; + private final Collection> onJoinValidators; @Nullable private Releasable electionScheduler; @Nullable @@ -141,13 +144,14 @@ public class Coordinator extends AbstractLifecycleComponent implements Discovery public Coordinator(String nodeName, Settings settings, ClusterSettings clusterSettings, TransportService transportService, NamedWriteableRegistry namedWriteableRegistry, AllocationService allocationService, MasterService masterService, Supplier persistedStateSupplier, UnicastHostsProvider unicastHostsProvider, - ClusterApplier clusterApplier, Random random) { + ClusterApplier clusterApplier, Collection> onJoinValidators, Random random) { super(settings); this.settings = settings; this.transportService = transportService; this.masterService = masterService; + this.onJoinValidators = JoinTaskExecutor.addBuiltInJoinValidators(onJoinValidators); this.joinHelper = new JoinHelper(settings, allocationService, masterService, transportService, - this::getCurrentTerm, this::handleJoinRequest, this::joinLeaderInTerm); + this::getCurrentTerm, this::handleJoinRequest, this::joinLeaderInTerm, this.onJoinValidators); this.persistedStateSupplier = persistedStateSupplier; this.discoverySettings = new DiscoverySettings(settings, clusterSettings); this.lastKnownLeader = Optional.empty(); @@ -279,6 +283,11 @@ PublishWithJoinResponse handlePublishRequest(PublishRequest publishRequest) { + lastKnownLeader + ", rejecting"); } + if (publishRequest.getAcceptedState().term() > coordinationState.get().getLastAcceptedState().term()) { + // only do join validation if we have not accepted state from this master yet + onJoinValidators.forEach(a -> a.accept(getLocalNode(), publishRequest.getAcceptedState())); + } + ensureTermAtLeast(sourceNode, publishRequest.getAcceptedState().term()); final PublishResponse publishResponse = coordinationState.get().handlePublishRequest(publishRequest); @@ -391,6 +400,41 @@ private void handleJoinRequest(JoinRequest joinRequest, JoinHelper.JoinCallback logger.trace("handleJoinRequest: as {}, handling {}", mode, joinRequest); transportService.connectToNode(joinRequest.getSourceNode()); + final ClusterState stateForJoinValidation = getStateForMasterService(); + + if (stateForJoinValidation.nodes().isLocalNodeElectedMaster()) { + onJoinValidators.forEach(a -> a.accept(joinRequest.getSourceNode(), stateForJoinValidation)); + if (stateForJoinValidation.getBlocks().hasGlobalBlock(STATE_NOT_RECOVERED_BLOCK) == false) { + // we do this in a couple of places including the cluster update thread. This one here is really just best effort + // to ensure we fail as fast as possible. + JoinTaskExecutor.ensureMajorVersionBarrier(joinRequest.getSourceNode().getVersion(), + stateForJoinValidation.getNodes().getMinNodeVersion()); + } + + // validate the join on the joining node, will throw a failure if it fails the validation + joinHelper.sendValidateJoinRequest(joinRequest.getSourceNode(), stateForJoinValidation, new ActionListener() { + @Override + public void onResponse(Empty empty) { + try { + processJoinRequest(joinRequest, joinCallback); + } catch (Exception e) { + joinCallback.onFailure(e); + } + } + + @Override + public void onFailure(Exception e) { + logger.warn(() -> new ParameterizedMessage("failed to validate incoming join request from node [{}]", + joinRequest.getSourceNode()), e); + joinCallback.onFailure(new IllegalStateException("failure when sending a validation request to node", e)); + } + }); + } else { + processJoinRequest(joinRequest, joinCallback); + } + } + + private void processJoinRequest(JoinRequest joinRequest, JoinHelper.JoinCallback joinCallback) { final Optional optionalJoin = joinRequest.getOptionalJoin(); synchronized (mutex) { final CoordinationState coordState = coordinationState.get(); @@ -516,7 +560,7 @@ Mode getMode() { } // visible for testing - public DiscoveryNode getLocalNode() { + DiscoveryNode getLocalNode() { return transportService.getLocalNode(); } diff --git a/server/src/main/java/org/elasticsearch/cluster/coordination/JoinHelper.java b/server/src/main/java/org/elasticsearch/cluster/coordination/JoinHelper.java index 0154919de74ff..8c41d7b2eaa52 100644 --- a/server/src/main/java/org/elasticsearch/cluster/coordination/JoinHelper.java +++ b/server/src/main/java/org/elasticsearch/cluster/coordination/JoinHelper.java @@ -21,6 +21,7 @@ import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; import org.apache.logging.log4j.message.ParameterizedMessage; +import org.elasticsearch.action.ActionListener; import org.elasticsearch.cluster.ClusterState; import org.elasticsearch.cluster.ClusterStateTaskConfig; import org.elasticsearch.cluster.ClusterStateTaskListener; @@ -40,15 +41,18 @@ import org.elasticsearch.discovery.zen.ZenDiscovery; import org.elasticsearch.threadpool.ThreadPool; import org.elasticsearch.threadpool.ThreadPool.Names; +import org.elasticsearch.transport.EmptyTransportResponseHandler; import org.elasticsearch.transport.TransportChannel; import org.elasticsearch.transport.TransportException; import org.elasticsearch.transport.TransportRequest; import org.elasticsearch.transport.TransportRequestOptions; +import org.elasticsearch.transport.TransportResponse; import org.elasticsearch.transport.TransportResponse.Empty; import org.elasticsearch.transport.TransportResponseHandler; import org.elasticsearch.transport.TransportService; import java.io.IOException; +import java.util.Collection; import java.util.HashMap; import java.util.LinkedHashMap; import java.util.List; @@ -64,6 +68,7 @@ public class JoinHelper { private static final Logger logger = LogManager.getLogger(JoinHelper.class); public static final String JOIN_ACTION_NAME = "internal:cluster/coordination/join"; + public static final String VALIDATE_JOIN_ACTION_NAME = "internal:cluster/coordination/join/validate"; public static final String START_JOIN_ACTION_NAME = "internal:cluster/coordination/start_join"; // the timeout for each join attempt @@ -80,7 +85,8 @@ public class JoinHelper { public JoinHelper(Settings settings, AllocationService allocationService, MasterService masterService, TransportService transportService, LongSupplier currentTermSupplier, - BiConsumer joinHandler, Function joinLeaderInTerm) { + BiConsumer joinHandler, Function joinLeaderInTerm, + Collection> joinValidators) { this.masterService = masterService; this.transportService = transportService; this.joinTimeout = JOIN_TIMEOUT_SETTING.get(settings); @@ -123,9 +129,19 @@ public ClusterTasksResult execute(ClusterState currentSta channel.sendResponse(Empty.INSTANCE); }); + transportService.registerRequestHandler(VALIDATE_JOIN_ACTION_NAME, + MembershipAction.ValidateJoinRequest::new, ThreadPool.Names.GENERIC, + (request, channel, task) -> { + joinValidators.forEach(action -> action.accept(transportService.getLocalNode(), request.getState())); + channel.sendResponse(Empty.INSTANCE); + }); + transportService.registerRequestHandler(MembershipAction.DISCOVERY_JOIN_VALIDATE_ACTION_NAME, - () -> new MembershipAction.ValidateJoinRequest(), ThreadPool.Names.GENERIC, - (request, channel, task) -> channel.sendResponse(Empty.INSTANCE)); // TODO: implement join validation + MembershipAction.ValidateJoinRequest::new, ThreadPool.Names.GENERIC, + (request, channel, task) -> { + joinValidators.forEach(action -> action.accept(transportService.getLocalNode(), request.getState())); + channel.sendResponse(Empty.INSTANCE); + }); transportService.registerRequestHandler( ZenDiscovery.DISCOVERY_REJOIN_ACTION_NAME, ZenDiscovery.RejoinClusterRequest::new, ThreadPool.Names.SAME, @@ -244,6 +260,29 @@ public String executor() { }); } + public void sendValidateJoinRequest(DiscoveryNode node, ClusterState state, ActionListener listener) { + final String actionName; + if (Coordinator.isZen1Node(node)) { + actionName = MembershipAction.DISCOVERY_JOIN_VALIDATE_ACTION_NAME; + } else { + actionName = VALIDATE_JOIN_ACTION_NAME; + } + transportService.sendRequest(node, actionName, + new MembershipAction.ValidateJoinRequest(state), + TransportRequestOptions.builder().withTimeout(joinTimeout).build(), + new EmptyTransportResponseHandler(ThreadPool.Names.GENERIC) { + @Override + public void handleResponse(TransportResponse.Empty response) { + listener.onResponse(response); + } + + @Override + public void handleException(TransportException exp) { + listener.onFailure(exp); + } + }); + } + public interface JoinCallback { void onSuccess(); diff --git a/server/src/main/java/org/elasticsearch/cluster/coordination/JoinTaskExecutor.java b/server/src/main/java/org/elasticsearch/cluster/coordination/JoinTaskExecutor.java index 9544cf15a0c4e..c4c76d8a8fe74 100644 --- a/server/src/main/java/org/elasticsearch/cluster/coordination/JoinTaskExecutor.java +++ b/server/src/main/java/org/elasticsearch/cluster/coordination/JoinTaskExecutor.java @@ -31,7 +31,11 @@ import org.elasticsearch.cluster.routing.allocation.AllocationService; import org.elasticsearch.discovery.DiscoverySettings; +import java.util.ArrayList; +import java.util.Collection; +import java.util.Collections; import java.util.List; +import java.util.function.BiConsumer; import static org.elasticsearch.gateway.GatewayService.STATE_NOT_RECOVERED_BLOCK; @@ -259,4 +263,15 @@ public static void ensureMajorVersionBarrier(Version joiningNodeVersion, Version "All nodes in the cluster are of a higher major [" + clusterMajor + "]."); } } + + public static Collection> addBuiltInJoinValidators( + Collection> onJoinValidators) { + final Collection> validators = new ArrayList<>(); + validators.add((node, state) -> { + ensureNodesCompatibility(node.getVersion(), state.getNodes()); + ensureIndexCompatibility(node.getVersion(), state.getMetaData()); + }); + validators.addAll(onJoinValidators); + return Collections.unmodifiableCollection(validators); + } } diff --git a/server/src/main/java/org/elasticsearch/cluster/coordination/PublicationTransportHandler.java b/server/src/main/java/org/elasticsearch/cluster/coordination/PublicationTransportHandler.java index 87ce488345db0..b0b91cd0980f2 100644 --- a/server/src/main/java/org/elasticsearch/cluster/coordination/PublicationTransportHandler.java +++ b/server/src/main/java/org/elasticsearch/cluster/coordination/PublicationTransportHandler.java @@ -389,7 +389,13 @@ private PublishWithJoinResponse handleIncomingPublishRequest(BytesTransportReque in.setVersion(request.version()); // If true we received full cluster state - otherwise diffs if (in.readBoolean()) { - final ClusterState incomingState = ClusterState.readFrom(in, transportService.getLocalNode()); + final ClusterState incomingState; + try { + incomingState = ClusterState.readFrom(in, transportService.getLocalNode()); + } catch (Exception e){ + logger.warn("unexpected error while deserializing an incoming cluster state", e); + throw e; + } fullClusterStateReceivedCount.incrementAndGet(); logger.debug("received full cluster state version [{}] with size [{}]", incomingState.version(), request.bytes().length()); @@ -400,10 +406,20 @@ private PublishWithJoinResponse handleIncomingPublishRequest(BytesTransportReque final ClusterState lastSeen = lastSeenClusterState.get(); if (lastSeen == null) { logger.debug("received diff for but don't have any local cluster state - requesting full state"); + incompatibleClusterStateDiffReceivedCount.incrementAndGet(); throw new IncompatibleClusterStateVersionException("have no local cluster state"); } else { - Diff diff = ClusterState.readDiffFrom(in, lastSeen.nodes().getLocalNode()); - final ClusterState incomingState = diff.apply(lastSeen); // might throw IncompatibleClusterStateVersionException + final ClusterState incomingState; + try { + Diff diff = ClusterState.readDiffFrom(in, lastSeen.nodes().getLocalNode()); + incomingState = diff.apply(lastSeen); // might throw IncompatibleClusterStateVersionException + } catch (IncompatibleClusterStateVersionException e) { + incompatibleClusterStateDiffReceivedCount.incrementAndGet(); + throw e; + } catch (Exception e){ + logger.warn("unexpected error while deserializing an incoming cluster state", e); + throw e; + } compatibleClusterStateDiffReceivedCount.incrementAndGet(); logger.debug("received diff cluster state version [{}] with uuid [{}], diff size [{}]", incomingState.version(), incomingState.stateUUID(), request.bytes().length()); @@ -412,12 +428,6 @@ private PublishWithJoinResponse handleIncomingPublishRequest(BytesTransportReque return response; } } - } catch (IncompatibleClusterStateVersionException e) { - incompatibleClusterStateDiffReceivedCount.incrementAndGet(); - throw e; - } catch (Exception e) { - logger.warn("unexpected error while deserializing an incoming cluster state", e); - throw e; } finally { IOUtils.close(in); } diff --git a/server/src/main/java/org/elasticsearch/discovery/DiscoveryModule.java b/server/src/main/java/org/elasticsearch/discovery/DiscoveryModule.java index 1572548b1b1fc..042eb9daa0d9d 100644 --- a/server/src/main/java/org/elasticsearch/discovery/DiscoveryModule.java +++ b/server/src/main/java/org/elasticsearch/discovery/DiscoveryModule.java @@ -127,11 +127,11 @@ public DiscoveryModule(Settings settings, ThreadPool threadPool, TransportServic Map> discoveryTypes = new HashMap<>(); discoveryTypes.put(ZEN_DISCOVERY_TYPE, () -> new ZenDiscovery(settings, threadPool, transportService, namedWriteableRegistry, masterService, clusterApplier, - clusterSettings, hostsProvider, allocationService, Collections.unmodifiableCollection(joinValidators), gatewayMetaState)); + clusterSettings, hostsProvider, allocationService, joinValidators, gatewayMetaState)); discoveryTypes.put(ZEN2_DISCOVERY_TYPE, () -> new Coordinator(NODE_NAME_SETTING.get(settings), settings, clusterSettings, transportService, namedWriteableRegistry, allocationService, masterService, () -> gatewayMetaState.getPersistedState(settings, (ClusterApplierService) clusterApplier), hostsProvider, clusterApplier, - Randomness.get())); + joinValidators, Randomness.get())); discoveryTypes.put("single-node", () -> new SingleNodeDiscovery(settings, transportService, masterService, clusterApplier, gatewayMetaState)); for (DiscoveryPlugin plugin : plugins) { diff --git a/server/src/main/java/org/elasticsearch/discovery/zen/MembershipAction.java b/server/src/main/java/org/elasticsearch/discovery/zen/MembershipAction.java index a136d28305252..550b25083fb96 100644 --- a/server/src/main/java/org/elasticsearch/discovery/zen/MembershipAction.java +++ b/server/src/main/java/org/elasticsearch/discovery/zen/MembershipAction.java @@ -164,7 +164,7 @@ public static class ValidateJoinRequest extends TransportRequest { public ValidateJoinRequest() {} - ValidateJoinRequest(ClusterState state) { + public ValidateJoinRequest(ClusterState state) { this.state = state; } @@ -179,6 +179,10 @@ public void writeTo(StreamOutput out) throws IOException { super.writeTo(out); this.state.writeTo(out); } + + public ClusterState getState() { + return state; + } } static class ValidateJoinRequestRequestHandler implements TransportRequestHandler { diff --git a/server/src/main/java/org/elasticsearch/discovery/zen/ZenDiscovery.java b/server/src/main/java/org/elasticsearch/discovery/zen/ZenDiscovery.java index 0657f9e80278a..05d0bfa27188a 100644 --- a/server/src/main/java/org/elasticsearch/discovery/zen/ZenDiscovery.java +++ b/server/src/main/java/org/elasticsearch/discovery/zen/ZenDiscovery.java @@ -73,7 +73,6 @@ import java.io.IOException; import java.util.ArrayList; import java.util.Collection; -import java.util.Collections; import java.util.List; import java.util.Locale; import java.util.Set; @@ -163,7 +162,7 @@ public ZenDiscovery(Settings settings, ThreadPool threadPool, TransportService t ClusterSettings clusterSettings, UnicastHostsProvider hostsProvider, AllocationService allocationService, Collection> onJoinValidators, GatewayMetaState gatewayMetaState) { super(settings); - this.onJoinValidators = addBuiltInJoinValidators(onJoinValidators); + this.onJoinValidators = JoinTaskExecutor.addBuiltInJoinValidators(onJoinValidators); this.masterService = masterService; this.clusterApplier = clusterApplier; this.transportService = transportService; @@ -235,17 +234,6 @@ public ZenDiscovery(Settings settings, ThreadPool threadPool, TransportService t } } - static Collection> addBuiltInJoinValidators( - Collection> onJoinValidators) { - Collection> validators = new ArrayList<>(); - validators.add((node, state) -> { - JoinTaskExecutor.ensureNodesCompatibility(node.getVersion(), state.getNodes()); - JoinTaskExecutor.ensureIndexCompatibility(node.getVersion(), state.getMetaData()); - }); - validators.addAll(onJoinValidators); - return Collections.unmodifiableCollection(validators); - } - // protected to allow overriding in tests protected ZenPing newZenPing(Settings settings, ThreadPool threadPool, TransportService transportService, UnicastHostsProvider hostsProvider) { diff --git a/server/src/test/java/org/elasticsearch/action/admin/cluster/bootstrap/TransportBootstrapClusterActionTests.java b/server/src/test/java/org/elasticsearch/action/admin/cluster/bootstrap/TransportBootstrapClusterActionTests.java index cf814482f6da0..31486a52bd08f 100644 --- a/server/src/test/java/org/elasticsearch/action/admin/cluster/bootstrap/TransportBootstrapClusterActionTests.java +++ b/server/src/test/java/org/elasticsearch/action/admin/cluster/bootstrap/TransportBootstrapClusterActionTests.java @@ -47,6 +47,7 @@ import org.junit.BeforeClass; import java.io.IOException; +import java.util.Collections; import java.util.Random; import java.util.concurrent.CountDownLatch; import java.util.concurrent.TimeUnit; @@ -96,7 +97,7 @@ public void setupTest() { ESAllocationTestCase.createAllocationService(Settings.EMPTY), new MasterService("local", Settings.EMPTY, threadPool), () -> new InMemoryPersistedState(0, ClusterState.builder(new ClusterName("cluster")).build()), r -> emptyList(), - new NoOpClusterApplier(), new Random(random().nextLong())); + new NoOpClusterApplier(), Collections.emptyList(), new Random(random().nextLong())); } public void testHandlesNonstandardDiscoveryImplementation() throws InterruptedException { diff --git a/server/src/test/java/org/elasticsearch/action/admin/cluster/bootstrap/TransportGetDiscoveredNodesActionTests.java b/server/src/test/java/org/elasticsearch/action/admin/cluster/bootstrap/TransportGetDiscoveredNodesActionTests.java index b41f7f882920e..6d94dcf6eca14 100644 --- a/server/src/test/java/org/elasticsearch/action/admin/cluster/bootstrap/TransportGetDiscoveredNodesActionTests.java +++ b/server/src/test/java/org/elasticsearch/action/admin/cluster/bootstrap/TransportGetDiscoveredNodesActionTests.java @@ -133,7 +133,7 @@ protected void onSendRequest(long requestId, String action, TransportRequest req ESAllocationTestCase.createAllocationService(settings), new MasterService("local", settings, threadPool), () -> new InMemoryPersistedState(0, ClusterState.builder(new ClusterName(clusterName)).build()), r -> emptyList(), - new NoOpClusterApplier(), new Random(random().nextLong())); + new NoOpClusterApplier(), Collections.emptyList(), new Random(random().nextLong())); } public void testHandlesNonstandardDiscoveryImplementation() throws InterruptedException { 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 a264015261de5..36879df5ac06d 100644 --- a/server/src/test/java/org/elasticsearch/cluster/coordination/CoordinatorTests.java +++ b/server/src/test/java/org/elasticsearch/cluster/coordination/CoordinatorTests.java @@ -63,6 +63,7 @@ import java.io.UncheckedIOException; import java.util.ArrayList; import java.util.Arrays; +import java.util.Collection; import java.util.Collections; import java.util.EnumSet; import java.util.HashMap; @@ -73,6 +74,7 @@ import java.util.Set; import java.util.concurrent.Callable; import java.util.concurrent.atomic.AtomicBoolean; +import java.util.function.BiConsumer; import java.util.function.Consumer; import java.util.function.Function; import java.util.function.Predicate; @@ -974,6 +976,67 @@ private void testAppliesNoMasterBlock(String noMasterBlockSetting, ClusterBlock // TODO reboot the leader and verify that the same block is applied when it restarts } + public void testNodeCannotJoinIfJoinValidationFailsOnMaster() { + final Cluster cluster = new Cluster(randomIntBetween(1, 3)); + cluster.runRandomly(); + cluster.stabilise(); + + // check that if node join validation fails on master, the nodes can't join + List addedNodes = cluster.addNodes(randomIntBetween(1, 2)); + final Set validatedNodes = new HashSet<>(); + cluster.getAnyLeader().extraJoinValidators.add((discoveryNode, clusterState) -> { + validatedNodes.add(discoveryNode); + throw new IllegalArgumentException("join validation failed"); + }); + final long previousClusterStateVersion = cluster.getAnyLeader().getLastAppliedClusterState().version(); + cluster.runFor(10000, "failing join validation"); + assertEquals(validatedNodes, addedNodes.stream().map(ClusterNode::getLocalNode).collect(Collectors.toSet())); + assertTrue(addedNodes.stream().allMatch(ClusterNode::isCandidate)); + final long newClusterStateVersion = cluster.getAnyLeader().getLastAppliedClusterState().version(); + assertEquals(previousClusterStateVersion, newClusterStateVersion); + + cluster.getAnyLeader().extraJoinValidators.clear(); + cluster.stabilise(); + } + + public void testNodeCannotJoinIfJoinValidationFailsOnJoiningNode() { + final Cluster cluster = new Cluster(randomIntBetween(1, 3)); + cluster.runRandomly(); + cluster.stabilise(); + + // check that if node join validation fails on joining node, the nodes can't join + List addedNodes = cluster.addNodes(randomIntBetween(1, 2)); + final Set validatedNodes = new HashSet<>(); + addedNodes.stream().forEach(cn -> cn.extraJoinValidators.add((discoveryNode, clusterState) -> { + validatedNodes.add(discoveryNode); + throw new IllegalArgumentException("join validation failed"); + })); + final long previousClusterStateVersion = cluster.getAnyLeader().getLastAppliedClusterState().version(); + cluster.runFor(10000, "failing join validation"); + assertEquals(validatedNodes, addedNodes.stream().map(ClusterNode::getLocalNode).collect(Collectors.toSet())); + assertTrue(addedNodes.stream().allMatch(ClusterNode::isCandidate)); + final long newClusterStateVersion = cluster.getAnyLeader().getLastAppliedClusterState().version(); + assertEquals(previousClusterStateVersion, newClusterStateVersion); + + addedNodes.stream().forEach(cn -> cn.extraJoinValidators.clear()); + cluster.stabilise(); + } + + public void testClusterCannotFormWithFailingJoinValidation() { + final Cluster cluster = new Cluster(randomIntBetween(1, 5)); + // fail join validation on a majority of nodes in the initial configuration + randomValueOtherThanMany(nodes -> + cluster.initialConfiguration.hasQuorum( + nodes.stream().map(ClusterNode::getLocalNode).map(DiscoveryNode::getId).collect(Collectors.toSet())) == false, + () -> randomSubsetOf(cluster.clusterNodes)) + .forEach(cn -> cn.extraJoinValidators.add((discoveryNode, clusterState) -> { + throw new IllegalArgumentException("join validation failed"); + })); + cluster.bootstrapIfNecessary(); + cluster.runFor(10000, "failing join validation"); + assertTrue(cluster.clusterNodes.stream().allMatch(cn -> cn.getLastAppliedClusterState().version() == 0)); + } + private static long defaultMillis(Setting setting) { return setting.get(Settings.EMPTY).millis() + Cluster.DEFAULT_DELAY_VARIABILITY; } @@ -1061,8 +1124,8 @@ class Cluster { initialNodeCount, masterEligibleNodeIds, initialConfiguration); } - void addNodesAndStabilise(int newNodesCount) { - addNodes(newNodesCount); + List addNodesAndStabilise(int newNodesCount) { + final List addedNodes = addNodes(newNodesCount); stabilise( // The first pinging discovers the master defaultMillis(DISCOVERY_FIND_PEERS_INTERVAL_SETTING) @@ -1072,16 +1135,20 @@ void addNodesAndStabilise(int newNodesCount) { // followup reconfiguration + newNodesCount * 2 * DEFAULT_CLUSTER_STATE_UPDATE_DELAY); // TODO Investigate whether 4 publications is sufficient due to batching? A bound linear in the number of nodes isn't great. + return addedNodes; } - void addNodes(int newNodesCount) { + List addNodes(int newNodesCount) { logger.info("--> adding {} nodes", newNodesCount); final int nodeSizeAtStart = clusterNodes.size(); + final List addedNodes = new ArrayList<>(); for (int i = 0; i < newNodesCount; i++) { final ClusterNode clusterNode = new ClusterNode(nodeSizeAtStart + i, true); - clusterNodes.add(clusterNode); + addedNodes.add(clusterNode); } + clusterNodes.addAll(addedNodes); + return addedNodes; } int size() { @@ -1219,15 +1286,7 @@ void stabilise(long stabilisationDurationMillis) { deterministicTaskQueue.getExecutionDelayVariabilityMillis(), lessThanOrEqualTo(DEFAULT_DELAY_VARIABILITY)); assertFalse("stabilisation requires stable storage", disruptStorage); - if (clusterNodes.stream().allMatch(ClusterNode::isNotUsefullyBootstrapped)) { - assertThat("setting initial configuration may fail with disconnected nodes", disconnectedNodes, empty()); - assertThat("setting initial configuration may fail with blackholed nodes", blackholedNodes, empty()); - runFor(defaultMillis(DISCOVERY_FIND_PEERS_INTERVAL_SETTING) * 2, "discovery prior to setting initial configuration"); - final ClusterNode bootstrapNode = getAnyMasterEligibleNode(); - bootstrapNode.applyInitialConfiguration(); - } else { - logger.info("setting initial configuration not required"); - } + bootstrapIfNecessary(); runFor(stabilisationDurationMillis, "stabilising"); @@ -1293,6 +1352,18 @@ void stabilise(long stabilisationDurationMillis) { leader.improveConfiguration(lastAcceptedState), sameInstance(lastAcceptedState)); } + void bootstrapIfNecessary() { + if (clusterNodes.stream().allMatch(ClusterNode::isNotUsefullyBootstrapped)) { + assertThat("setting initial configuration may fail with disconnected nodes", disconnectedNodes, empty()); + assertThat("setting initial configuration may fail with blackholed nodes", blackholedNodes, empty()); + runFor(defaultMillis(DISCOVERY_FIND_PEERS_INTERVAL_SETTING) * 2, "discovery prior to setting initial configuration"); + final ClusterNode bootstrapNode = getAnyMasterEligibleNode(); + bootstrapNode.applyInitialConfiguration(); + } else { + logger.info("setting initial configuration not required"); + } + } + void runFor(long runDurationMillis, String description) { final long endTime = deterministicTaskQueue.getCurrentTimeMillis() + runDurationMillis; logger.info("--> runFor({}ms) running until [{}ms]: {}", runDurationMillis, endTime, description); @@ -1421,6 +1492,7 @@ class ClusterNode { private AckedFakeThreadPoolMasterService masterService; private TransportService transportService; private DisruptableMockTransport mockTransport; + private List> extraJoinValidators = new ArrayList<>(); private ClusterStateApplyResponse clusterStateApplyResponse = ClusterStateApplyResponse.SUCCEED; ClusterNode(int nodeIndex, boolean masterEligible) { @@ -1495,9 +1567,11 @@ protected void onBlackholedDuringSend(long requestId, String action, DiscoveryNo transportService = mockTransport.createTransportService( settings, deterministicTaskQueue.getThreadPool(runnable -> onNode(localNode, runnable)), NOOP_TRANSPORT_INTERCEPTOR, a -> localNode, null, emptySet()); + final Collection> onJoinValidators = + Collections.singletonList((dn, cs) -> extraJoinValidators.forEach(validator -> validator.accept(dn, cs))); coordinator = new Coordinator("test_node", settings, clusterSettings, transportService, writableRegistry(), ESAllocationTestCase.createAllocationService(Settings.EMPTY), masterService, this::getPersistedState, - Cluster.this::provideUnicastHosts, clusterApplier, Randomness.get()); + Cluster.this::provideUnicastHosts, clusterApplier, onJoinValidators, Randomness.get()); masterService.setClusterStatePublisher(coordinator); transportService.start(); @@ -1523,6 +1597,10 @@ boolean isLeader() { return coordinator.getMode() == LEADER; } + boolean isCandidate() { + return coordinator.getMode() == CANDIDATE; + } + ClusterState improveConfiguration(ClusterState currentState) { synchronized (coordinator.mutex) { return coordinator.improveConfiguration(currentState); diff --git a/server/src/test/java/org/elasticsearch/cluster/coordination/JoinHelperTests.java b/server/src/test/java/org/elasticsearch/cluster/coordination/JoinHelperTests.java index 6e7965f896f4c..ef843717fb469 100644 --- a/server/src/test/java/org/elasticsearch/cluster/coordination/JoinHelperTests.java +++ b/server/src/test/java/org/elasticsearch/cluster/coordination/JoinHelperTests.java @@ -44,7 +44,8 @@ public void testJoinDeduplication() { deterministicTaskQueue.getThreadPool(), TransportService.NOOP_TRANSPORT_INTERCEPTOR, x -> localNode, null, Collections.emptySet()); JoinHelper joinHelper = new JoinHelper(Settings.EMPTY, null, null, transportService, () -> 0L, - (joinRequest, joinCallback) -> { throw new AssertionError(); }, startJoinRequest -> { throw new AssertionError(); }); + (joinRequest, joinCallback) -> { throw new AssertionError(); }, startJoinRequest -> { throw new AssertionError(); }, + Collections.emptyList()); transportService.start(); DiscoveryNode node1 = new DiscoveryNode("node1", buildNewFakeTransportAddress(), Version.CURRENT); diff --git a/server/src/test/java/org/elasticsearch/cluster/coordination/NodeJoinTests.java b/server/src/test/java/org/elasticsearch/cluster/coordination/NodeJoinTests.java index 214174d2a52fa..108b9e6dd7f1e 100644 --- a/server/src/test/java/org/elasticsearch/cluster/coordination/NodeJoinTests.java +++ b/server/src/test/java/org/elasticsearch/cluster/coordination/NodeJoinTests.java @@ -157,6 +157,8 @@ protected void onSendRequest(long requestId, String action, TransportRequest req if (action.equals(HANDSHAKE_ACTION_NAME)) { handleResponse(requestId, new TransportService.HandshakeResponse(destination, initialState.getClusterName(), destination.getVersion())); + } else if (action.equals(JoinHelper.VALIDATE_JOIN_ACTION_NAME)) { + handleResponse(requestId, new TransportResponse.Empty()); } else { super.onSendRequest(requestId, action, request, destination); } @@ -173,6 +175,7 @@ transportService, writableRegistry(), masterService, () -> new InMemoryPersistedState(term, initialState), r -> emptyList(), new NoOpClusterApplier(), + Collections.emptyList(), random); transportService.start(); transportService.acceptIncomingRequests(); diff --git a/server/src/test/java/org/elasticsearch/discovery/zen/ZenDiscoveryUnitTests.java b/server/src/test/java/org/elasticsearch/discovery/zen/ZenDiscoveryUnitTests.java index 99155799fb199..c9a2f7dc58388 100644 --- a/server/src/test/java/org/elasticsearch/discovery/zen/ZenDiscoveryUnitTests.java +++ b/server/src/test/java/org/elasticsearch/discovery/zen/ZenDiscoveryUnitTests.java @@ -28,6 +28,7 @@ import org.elasticsearch.cluster.ClusterState; import org.elasticsearch.cluster.ClusterStateTaskExecutor; import org.elasticsearch.cluster.ESAllocationTestCase; +import org.elasticsearch.cluster.coordination.JoinTaskExecutor; import org.elasticsearch.cluster.coordination.NodeRemovalClusterStateTaskExecutor; import org.elasticsearch.cluster.metadata.IndexMetaData; import org.elasticsearch.cluster.metadata.MetaData; @@ -388,7 +389,7 @@ public void testValidateOnUnsupportedIndexVersionCreated() throws Exception { final DiscoveryNode localNode = new DiscoveryNode("other_node", buildNewFakeTransportAddress(), emptyMap(), EnumSet.allOf(DiscoveryNode.Role.class), Version.CURRENT); MembershipAction.ValidateJoinRequestRequestHandler request = new MembershipAction.ValidateJoinRequestRequestHandler - (() -> localNode, ZenDiscovery.addBuiltInJoinValidators(Collections.emptyList())); + (() -> localNode, JoinTaskExecutor.addBuiltInJoinValidators(Collections.emptyList())); final boolean incompatible = randomBoolean(); IndexMetaData indexMetaData = IndexMetaData.builder("test").settings(Settings.builder() .put(SETTING_VERSION_CREATED, diff --git a/test/framework/src/main/java/org/elasticsearch/test/discovery/TestZenDiscovery.java b/test/framework/src/main/java/org/elasticsearch/test/discovery/TestZenDiscovery.java index 783dc6325c4a2..53be34c0b40c8 100644 --- a/test/framework/src/main/java/org/elasticsearch/test/discovery/TestZenDiscovery.java +++ b/test/framework/src/main/java/org/elasticsearch/test/discovery/TestZenDiscovery.java @@ -81,7 +81,7 @@ public Map> getDiscoveryTypes(ThreadPool threadPool, return new Coordinator("test_node", fixedSettings, clusterSettings, transportService, namedWriteableRegistry, allocationService, masterService, () -> gatewayMetaState.getPersistedState(settings, (ClusterApplierService) clusterApplier), hostsProvider, - clusterApplier, new Random(Randomness.get().nextLong())); + clusterApplier, Collections.emptyList(), new Random(Randomness.get().nextLong())); } else { return new TestZenDiscovery(fixedSettings, threadPool, transportService, namedWriteableRegistry, masterService, clusterApplier, clusterSettings, hostsProvider, allocationService, gatewayMetaState); From 71287b0759d4a1562206446c6196bcec6cae663a Mon Sep 17 00:00:00 2001 From: Alexander Reelsen Date: Thu, 10 Jan 2019 13:20:24 +0100 Subject: [PATCH 071/186] Remove unused EpochMillisDateFormatter (#37293) This class has been superceded by a custom java time epoch millis date parser. --- .../common/time/EpochMillisDateFormatter.java | 119 ------------------ 1 file changed, 119 deletions(-) delete mode 100644 server/src/main/java/org/elasticsearch/common/time/EpochMillisDateFormatter.java diff --git a/server/src/main/java/org/elasticsearch/common/time/EpochMillisDateFormatter.java b/server/src/main/java/org/elasticsearch/common/time/EpochMillisDateFormatter.java deleted file mode 100644 index b7276e4fd1466..0000000000000 --- a/server/src/main/java/org/elasticsearch/common/time/EpochMillisDateFormatter.java +++ /dev/null @@ -1,119 +0,0 @@ -/* - * Licensed to Elasticsearch under one or more contributor - * license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright - * ownership. Elasticsearch licenses this file to you under - * the Apache License, Version 2.0 (the "License"); you may - * not use this file except in compliance with the License. - * You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ - -package org.elasticsearch.common.time; - -import java.math.BigDecimal; -import java.time.Instant; -import java.time.ZoneId; -import java.time.ZoneOffset; -import java.time.format.DateTimeParseException; -import java.time.temporal.TemporalAccessor; -import java.util.Locale; -import java.util.regex.Pattern; - -/** - * This is a special formatter to parse the milliseconds since the epoch. - * There is no way using a native java time date formatter to resemble - * the required behaviour to parse negative milliseconds as well. - * - * This implementation simply tries to convert the input to a long and uses - * this as the milliseconds since the epoch without involving any other - * java time code - */ -class EpochMillisDateFormatter implements DateFormatter { - - private static final Pattern SPLIT_BY_DOT_PATTERN = Pattern.compile("\\."); - static final DateFormatter INSTANCE = new EpochMillisDateFormatter(); - static final DateMathParser DATE_MATH_INSTANCE = new JavaDateMathParser(INSTANCE, INSTANCE); - - private EpochMillisDateFormatter() { - } - - @Override - public TemporalAccessor parse(String input) { - try { - if (input.contains(".")) { - String[] inputs = SPLIT_BY_DOT_PATTERN.split(input, 2); - Long milliSeconds = Long.valueOf(inputs[0]); - if (inputs[1].length() == 0) { - // this is BWC compatible to joda time, nothing after the dot is allowed - return Instant.ofEpochMilli(milliSeconds).atZone(ZoneOffset.UTC); - } - // scientific notation it is! - if (inputs[1].contains("e")) { - return Instant.ofEpochMilli(Double.valueOf(input).longValue()).atZone(ZoneOffset.UTC); - } - - if (inputs[1].length() > 6) { - throw new DateTimeParseException("too much granularity after dot [" + input + "]", input, 0); - } - Long nanos = new BigDecimal(inputs[1]).movePointRight(6 - inputs[1].length()).longValueExact(); - if (milliSeconds < 0) { - nanos = nanos * -1; - } - return Instant.ofEpochMilli(milliSeconds).plusNanos(nanos).atZone(ZoneOffset.UTC); - } else { - return Instant.ofEpochMilli(Long.valueOf(input)).atZone(ZoneOffset.UTC); - } - } catch (NumberFormatException e) { - throw new DateTimeParseException("invalid number [" + input + "]", input, 0, e); - } - } - @Override - public DateFormatter withZone(ZoneId zoneId) { - if (ZoneOffset.UTC.equals(zoneId) == false) { - throw new IllegalArgumentException(pattern() + " date formatter can only be in zone offset UTC"); - } - return INSTANCE; - } - - @Override - public DateFormatter withLocale(Locale locale) { - if (Locale.ROOT.equals(locale) == false) { - throw new IllegalArgumentException(pattern() + " date formatter can only be in locale ROOT"); - } - return this; - } - - @Override - public String format(TemporalAccessor accessor) { - return String.valueOf(Instant.from(accessor).toEpochMilli()); - } - - @Override - public String pattern() { - return "epoch_millis"; - } - - @Override - public Locale locale() { - return Locale.ROOT; - } - - @Override - public ZoneId zone() { - return ZoneOffset.UTC; - } - - @Override - public DateMathParser toDateMathParser() { - return DATE_MATH_INSTANCE; - } -} From 61b54196c4d2a79ea9fa416cf436cac053c3c271 Mon Sep 17 00:00:00 2001 From: Luca Cavanna Date: Thu, 10 Jan 2019 13:38:39 +0100 Subject: [PATCH 072/186] [TEST] Fixed compile issue in SnapshotsServiceTests Relates to #37203 --- .../java/org/elasticsearch/snapshots/SnapshotsServiceTests.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/server/src/test/java/org/elasticsearch/snapshots/SnapshotsServiceTests.java b/server/src/test/java/org/elasticsearch/snapshots/SnapshotsServiceTests.java index 291d6bf63a95b..34160e901006f 100644 --- a/server/src/test/java/org/elasticsearch/snapshots/SnapshotsServiceTests.java +++ b/server/src/test/java/org/elasticsearch/snapshots/SnapshotsServiceTests.java @@ -539,7 +539,7 @@ public void start(ClusterState initialState) { allocationService, masterService, () -> persistedState, hostsResolver -> testClusterNodes.nodes.values().stream().filter(n -> n.node.isMasterNode()) .map(n -> n.node.getAddress()).collect(Collectors.toList()), - clusterService.getClusterApplierService(), random()); + clusterService.getClusterApplierService(), Collections.emptyList(), random()); masterService.setClusterStatePublisher(coordinator); coordinator.start(); masterService.start(); From 44acb016a693221d77687851db9ba764a1ea71be Mon Sep 17 00:00:00 2001 From: Martijn van Groningen Date: Thu, 10 Jan 2019 14:04:55 +0100 Subject: [PATCH 073/186] [HLRC] Ignore unknown fields in responses of CCR APIs (#36821) Otherwise hlrc fails if new fields are introduced in ccr api responses in future versions. --- .../java/org/elasticsearch/client/ccr/AutoFollowStats.java | 6 +++++- .../org/elasticsearch/client/ccr/CcrStatsResponse.java | 4 +++- .../client/ccr/GetAutoFollowPatternResponse.java | 6 +++--- .../org/elasticsearch/client/ccr/IndicesFollowStats.java | 7 ++++++- .../org/elasticsearch/client/ccr/PutFollowResponse.java | 2 +- .../elasticsearch/client/ccr/CcrStatsResponseTests.java | 2 +- .../elasticsearch/client/ccr/FollowStatsResponseTests.java | 2 +- .../client/ccr/GetAutoFollowPatternResponseTests.java | 2 +- .../elasticsearch/client/ccr/PutFollowRequestTests.java | 2 +- .../elasticsearch/client/ccr/PutFollowResponseTests.java | 2 +- 10 files changed, 23 insertions(+), 12 deletions(-) diff --git a/client/rest-high-level/src/main/java/org/elasticsearch/client/ccr/AutoFollowStats.java b/client/rest-high-level/src/main/java/org/elasticsearch/client/ccr/AutoFollowStats.java index bb286b6e5d59b..394adb3b19182 100644 --- a/client/rest-high-level/src/main/java/org/elasticsearch/client/ccr/AutoFollowStats.java +++ b/client/rest-high-level/src/main/java/org/elasticsearch/client/ccr/AutoFollowStats.java @@ -47,7 +47,9 @@ public final class AutoFollowStats { static final ParseField LAST_SEEN_METADATA_VERSION = new ParseField("last_seen_metadata_version"); @SuppressWarnings("unchecked") - static final ConstructingObjectParser STATS_PARSER = new ConstructingObjectParser<>("auto_follow_stats", + static final ConstructingObjectParser STATS_PARSER = new ConstructingObjectParser<>( + "auto_follow_stats", + true, args -> new AutoFollowStats( (Long) args[0], (Long) args[1], @@ -65,11 +67,13 @@ public final class AutoFollowStats { static final ConstructingObjectParser>, Void> AUTO_FOLLOW_EXCEPTIONS_PARSER = new ConstructingObjectParser<>( "auto_follow_stats_errors", + true, args -> new AbstractMap.SimpleEntry<>((String) args[0], Tuple.tuple((Long) args[1], (ElasticsearchException) args[2]))); private static final ConstructingObjectParser, Void> AUTO_FOLLOWED_CLUSTERS_PARSER = new ConstructingObjectParser<>( "auto_followed_clusters", + true, args -> new AbstractMap.SimpleEntry<>((String) args[0], new AutoFollowedCluster((Long) args[1], (Long) args[2]))); static { diff --git a/client/rest-high-level/src/main/java/org/elasticsearch/client/ccr/CcrStatsResponse.java b/client/rest-high-level/src/main/java/org/elasticsearch/client/ccr/CcrStatsResponse.java index 889a96683bfb3..54f79892c1d08 100644 --- a/client/rest-high-level/src/main/java/org/elasticsearch/client/ccr/CcrStatsResponse.java +++ b/client/rest-high-level/src/main/java/org/elasticsearch/client/ccr/CcrStatsResponse.java @@ -28,7 +28,9 @@ public final class CcrStatsResponse { static final ParseField AUTO_FOLLOW_STATS_FIELD = new ParseField("auto_follow_stats"); static final ParseField FOLLOW_STATS_FIELD = new ParseField("follow_stats"); - private static final ConstructingObjectParser PARSER = new ConstructingObjectParser<>("indices", + private static final ConstructingObjectParser PARSER = new ConstructingObjectParser<>( + "indices", + true, args -> { AutoFollowStats autoFollowStats = (AutoFollowStats) args[0]; IndicesFollowStats indicesFollowStats = (IndicesFollowStats) args[1]; diff --git a/client/rest-high-level/src/main/java/org/elasticsearch/client/ccr/GetAutoFollowPatternResponse.java b/client/rest-high-level/src/main/java/org/elasticsearch/client/ccr/GetAutoFollowPatternResponse.java index ce42c98e57c41..d05ab3f3ee363 100644 --- a/client/rest-high-level/src/main/java/org/elasticsearch/client/ccr/GetAutoFollowPatternResponse.java +++ b/client/rest-high-level/src/main/java/org/elasticsearch/client/ccr/GetAutoFollowPatternResponse.java @@ -42,7 +42,7 @@ public final class GetAutoFollowPatternResponse { static final ParseField PATTERN_FIELD = new ParseField("pattern"); private static final ConstructingObjectParser, Void> ENTRY_PARSER = new ConstructingObjectParser<>( - "get_auto_follow_pattern_response", args -> new AbstractMap.SimpleEntry<>((String) args[0], (Pattern) args[1])); + "get_auto_follow_pattern_response", true, args -> new AbstractMap.SimpleEntry<>((String) args[0], (Pattern) args[1])); static { ENTRY_PARSER.declareString(ConstructingObjectParser.constructorArg(), NAME_FIELD); @@ -50,7 +50,7 @@ public final class GetAutoFollowPatternResponse { } private static final ConstructingObjectParser PARSER = new ConstructingObjectParser<>( - "get_auto_follow_pattern_response", args -> { + "get_auto_follow_pattern_response", true, args -> { @SuppressWarnings("unchecked") List> entries = (List>) args[0]; return new GetAutoFollowPatternResponse(new TreeMap<>(entries.stream() @@ -92,7 +92,7 @@ public static class Pattern extends FollowConfig { @SuppressWarnings("unchecked") private static final ConstructingObjectParser PARSER = new ConstructingObjectParser<>( - "pattern", args -> new Pattern((String) args[0], (List) args[1], (String) args[2])); + "pattern", true, args -> new Pattern((String) args[0], (List) args[1], (String) args[2])); static { PARSER.declareString(ConstructingObjectParser.constructorArg(), PutFollowRequest.REMOTE_CLUSTER_FIELD); diff --git a/client/rest-high-level/src/main/java/org/elasticsearch/client/ccr/IndicesFollowStats.java b/client/rest-high-level/src/main/java/org/elasticsearch/client/ccr/IndicesFollowStats.java index 02e2fc4f4ed18..7d3af08577b16 100644 --- a/client/rest-high-level/src/main/java/org/elasticsearch/client/ccr/IndicesFollowStats.java +++ b/client/rest-high-level/src/main/java/org/elasticsearch/client/ccr/IndicesFollowStats.java @@ -41,6 +41,7 @@ public final class IndicesFollowStats { private static final ConstructingObjectParser>, Void> ENTRY_PARSER = new ConstructingObjectParser<>( "entry", + true, args -> { String index = (String) args[0]; @SuppressWarnings("unchecked") @@ -54,7 +55,9 @@ public final class IndicesFollowStats { ENTRY_PARSER.declareObjectArray(ConstructingObjectParser.constructorArg(), ShardFollowStats.PARSER, SHARDS_FIELD); } - static final ConstructingObjectParser PARSER = new ConstructingObjectParser<>("indices", + static final ConstructingObjectParser PARSER = new ConstructingObjectParser<>( + "indices", + true, args -> { @SuppressWarnings("unchecked") List>> entries = (List>>) args[0]; @@ -116,6 +119,7 @@ public static final class ShardFollowStats { static final ConstructingObjectParser PARSER = new ConstructingObjectParser<>( "shard-follow-stats", + true, args -> new ShardFollowStats( (String) args[0], (String) args[1], @@ -152,6 +156,7 @@ public static final class ShardFollowStats { static final ConstructingObjectParser>, Void> READ_EXCEPTIONS_ENTRY_PARSER = new ConstructingObjectParser<>( "shard-follow-stats-read-exceptions-entry", + true, args -> new AbstractMap.SimpleEntry<>((long) args[0], Tuple.tuple((Integer) args[1], (ElasticsearchException)args[2]))); static { diff --git a/client/rest-high-level/src/main/java/org/elasticsearch/client/ccr/PutFollowResponse.java b/client/rest-high-level/src/main/java/org/elasticsearch/client/ccr/PutFollowResponse.java index 2d928b859882a..3841b868e73ab 100644 --- a/client/rest-high-level/src/main/java/org/elasticsearch/client/ccr/PutFollowResponse.java +++ b/client/rest-high-level/src/main/java/org/elasticsearch/client/ccr/PutFollowResponse.java @@ -33,7 +33,7 @@ public final class PutFollowResponse { static final ParseField INDEX_FOLLOWING_STARTED = new ParseField("index_following_started"); private static final ConstructingObjectParser PARSER = new ConstructingObjectParser<>( - "put_follow_response", args -> new PutFollowResponse((boolean) args[0], (boolean) args[1], (boolean) args[2])); + "put_follow_response", true, args -> new PutFollowResponse((boolean) args[0], (boolean) args[1], (boolean) args[2])); static { PARSER.declareBoolean(ConstructingObjectParser.constructorArg(), FOLLOW_INDEX_CREATED); diff --git a/client/rest-high-level/src/test/java/org/elasticsearch/client/ccr/CcrStatsResponseTests.java b/client/rest-high-level/src/test/java/org/elasticsearch/client/ccr/CcrStatsResponseTests.java index b818b7ae4219c..cb8072f6bafb3 100644 --- a/client/rest-high-level/src/test/java/org/elasticsearch/client/ccr/CcrStatsResponseTests.java +++ b/client/rest-high-level/src/test/java/org/elasticsearch/client/ccr/CcrStatsResponseTests.java @@ -51,7 +51,7 @@ public void testFromXContent() throws IOException { CcrStatsResponseTests::createTestInstance, CcrStatsResponseTests::toXContent, CcrStatsResponse::fromXContent) - .supportsUnknownFields(false) + .supportsUnknownFields(true) .assertEqualsConsumer(CcrStatsResponseTests::assertEqualInstances) .assertToXContentEquivalence(false) .test(); diff --git a/client/rest-high-level/src/test/java/org/elasticsearch/client/ccr/FollowStatsResponseTests.java b/client/rest-high-level/src/test/java/org/elasticsearch/client/ccr/FollowStatsResponseTests.java index c8d15bc508503..5ec3cb4edcf07 100644 --- a/client/rest-high-level/src/test/java/org/elasticsearch/client/ccr/FollowStatsResponseTests.java +++ b/client/rest-high-level/src/test/java/org/elasticsearch/client/ccr/FollowStatsResponseTests.java @@ -48,7 +48,7 @@ public void testFromXContent() throws IOException { FollowStatsResponseTests::createTestInstance, FollowStatsResponseTests::toXContent, FollowStatsResponse::fromXContent) - .supportsUnknownFields(false) + .supportsUnknownFields(true) .assertEqualsConsumer(FollowStatsResponseTests::assertEqualInstances) .assertToXContentEquivalence(false) .test(); diff --git a/client/rest-high-level/src/test/java/org/elasticsearch/client/ccr/GetAutoFollowPatternResponseTests.java b/client/rest-high-level/src/test/java/org/elasticsearch/client/ccr/GetAutoFollowPatternResponseTests.java index b4a37286b4ace..f6f0f1747e2a2 100644 --- a/client/rest-high-level/src/test/java/org/elasticsearch/client/ccr/GetAutoFollowPatternResponseTests.java +++ b/client/rest-high-level/src/test/java/org/elasticsearch/client/ccr/GetAutoFollowPatternResponseTests.java @@ -43,7 +43,7 @@ public void testFromXContent() throws IOException { this::createTestInstance, GetAutoFollowPatternResponseTests::toXContent, GetAutoFollowPatternResponse::fromXContent) - .supportsUnknownFields(false) + .supportsUnknownFields(true) .test(); } diff --git a/client/rest-high-level/src/test/java/org/elasticsearch/client/ccr/PutFollowRequestTests.java b/client/rest-high-level/src/test/java/org/elasticsearch/client/ccr/PutFollowRequestTests.java index 0814278a0cf59..35353ce4a96f9 100644 --- a/client/rest-high-level/src/test/java/org/elasticsearch/client/ccr/PutFollowRequestTests.java +++ b/client/rest-high-level/src/test/java/org/elasticsearch/client/ccr/PutFollowRequestTests.java @@ -31,7 +31,7 @@ public class PutFollowRequestTests extends AbstractXContentTestCase { private static final ConstructingObjectParser PARSER = new ConstructingObjectParser<>("test_parser", - (args) -> new PutFollowRequest((String) args[0], (String) args[1], (String) args[2])); + true, (args) -> new PutFollowRequest((String) args[0], (String) args[1], (String) args[2])); static { PARSER.declareString(ConstructingObjectParser.constructorArg(), PutFollowRequest.REMOTE_CLUSTER_FIELD); diff --git a/client/rest-high-level/src/test/java/org/elasticsearch/client/ccr/PutFollowResponseTests.java b/client/rest-high-level/src/test/java/org/elasticsearch/client/ccr/PutFollowResponseTests.java index 48eb15717c599..00bcf535f08af 100644 --- a/client/rest-high-level/src/test/java/org/elasticsearch/client/ccr/PutFollowResponseTests.java +++ b/client/rest-high-level/src/test/java/org/elasticsearch/client/ccr/PutFollowResponseTests.java @@ -33,7 +33,7 @@ public void testFromXContent() throws IOException { this::createTestInstance, PutFollowResponseTests::toXContent, PutFollowResponse::fromXContent) - .supportsUnknownFields(false) + .supportsUnknownFields(true) .test(); } From 26cb7466ef93c8b915741a379cb614a3600a5226 Mon Sep 17 00:00:00 2001 From: Armin Braun Date: Thu, 10 Jan 2019 14:13:20 +0100 Subject: [PATCH 074/186] SNAPSHOT+TESTS: Stabilize SnapshotDisruptionIT (#37289) * Ensure retry by busy assert on SnapshotMissingException * Closes #36739 --- .../org/elasticsearch/discovery/SnapshotDisruptionIT.java | 7 +++++-- 1 file changed, 5 insertions(+), 2 deletions(-) diff --git a/server/src/test/java/org/elasticsearch/discovery/SnapshotDisruptionIT.java b/server/src/test/java/org/elasticsearch/discovery/SnapshotDisruptionIT.java index 29fa597aec63f..f6a95a3ed5b35 100644 --- a/server/src/test/java/org/elasticsearch/discovery/SnapshotDisruptionIT.java +++ b/server/src/test/java/org/elasticsearch/discovery/SnapshotDisruptionIT.java @@ -75,7 +75,6 @@ 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); @@ -180,7 +179,11 @@ public void clusterChanged(ClusterChangedEvent event) { logger.info("--> verify that snapshot eventually will be created due to retries"); assertBusy(() -> { - assertSnapshotExists("test-repo", "test-snap-2"); + try { + assertSnapshotExists("test-repo", "test-snap-2"); + } catch (SnapshotMissingException ex) { + throw new AssertionError(ex); + } }, 1, TimeUnit.MINUTES); } From df488720e02ad4bf7398cf35a98a5abd171ecad0 Mon Sep 17 00:00:00 2001 From: Martijn van Groningen Date: Thu, 10 Jan 2019 15:02:30 +0100 Subject: [PATCH 075/186] [CCR] Make shard follow tasks more resilient for restarts (#37239) If a running shard follow task needs to be restarted and the remote connection seeds have changed then a shard follow task currently fails with a fatal error. The change creates the remote client lazily and adjusts the errors a shard follow task should retry. This issue was found in test failures in the recently added ccr rolling upgrade tests. The reason why this issue occurs more frequently in the rolling upgrade test is because ccr is setup in local mode (so remote connection seed will become stale) and all nodes are restarted, which forces the shard follow tasks to get restarted at some point during the test. Note that these tests cannot be enabled yet, because this change will need to be backported to 6.x first. (otherwise the issue still occurs on non upgraded nodes) I also changed the RestartIndexFollowingIT to setup remote cluster via persistent settings and to also restart the leader cluster. This way what happens during the ccr rolling upgrade qa tests, also happens in this test. Relates to #37231 --- .../xpack/ccr/action/ShardFollowNodeTask.java | 12 ++++--- .../ccr/action/ShardFollowTasksExecutor.java | 28 +++++++++------- .../elasticsearch/xpack/CcrIntegTestCase.java | 6 +++- .../xpack/ccr/RestartIndexFollowingIT.java | 32 ++++++++++++++++++- 4 files changed, 60 insertions(+), 18 deletions(-) diff --git a/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/ShardFollowNodeTask.java b/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/ShardFollowNodeTask.java index 13b12d4b96f2b..b1d6467168c90 100644 --- a/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/ShardFollowNodeTask.java +++ b/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/ShardFollowNodeTask.java @@ -29,8 +29,7 @@ import org.elasticsearch.node.NodeClosedException; import org.elasticsearch.persistent.AllocatedPersistentTask; import org.elasticsearch.tasks.TaskId; -import org.elasticsearch.transport.NodeDisconnectedException; -import org.elasticsearch.transport.NodeNotConnectedException; +import org.elasticsearch.transport.ConnectTransportException; import org.elasticsearch.xpack.ccr.action.bulk.BulkShardOperationsResponse; import org.elasticsearch.xpack.core.ccr.ShardFollowNodeTaskStatus; @@ -448,7 +447,10 @@ static boolean shouldRetry(String remoteCluster, Exception e) { return true; } + // This is thrown when using a Client and its remote cluster alias went MIA String noSuchRemoteClusterMessage = "no such remote cluster: " + remoteCluster; + // This is thrown when creating a Client and the remote cluster does not exist: + String unknownClusterMessage = "unknown cluster alias [" + remoteCluster + "]"; final Throwable actual = ExceptionsHelper.unwrapCause(e); return actual instanceof ShardNotFoundException || actual instanceof IllegalIndexShardStateException || @@ -458,11 +460,11 @@ static boolean shouldRetry(String remoteCluster, Exception e) { actual instanceof ElasticsearchSecurityException || // If user does not have sufficient privileges actual instanceof ClusterBlockException || // If leader index is closed or no elected master actual instanceof IndexClosedException || // If follow index is closed - actual instanceof NodeDisconnectedException || - actual instanceof NodeNotConnectedException || + actual instanceof ConnectTransportException || actual instanceof NodeClosedException || (actual.getMessage() != null && actual.getMessage().contains("TransportService is closed")) || - (actual instanceof IllegalArgumentException && noSuchRemoteClusterMessage.equals(actual.getMessage())); + (actual instanceof IllegalArgumentException && (noSuchRemoteClusterMessage.equals(actual.getMessage()) || + unknownClusterMessage.equals(actual.getMessage()))); } // These methods are protected for testing purposes: diff --git a/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/ShardFollowTasksExecutor.java b/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/ShardFollowTasksExecutor.java index 14ec147a5366f..97308126ffb3f 100644 --- a/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/ShardFollowTasksExecutor.java +++ b/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/ShardFollowTasksExecutor.java @@ -94,12 +94,6 @@ protected AllocatedPersistentTask createTask(long id, String type, String action PersistentTasksCustomMetaData.PersistentTask taskInProgress, Map headers) { ShardFollowTask params = taskInProgress.getParams(); - final Client remoteClient; - if (params.getRemoteCluster() != null) { - remoteClient = wrapClient(client.getRemoteClusterClient(params.getRemoteCluster()), params.getHeaders()); - } else { - remoteClient = wrapClient(client, params.getHeaders()); - } Client followerClient = wrapClient(client, params.getHeaders()); BiConsumer scheduler = (delay, command) -> { try { @@ -123,8 +117,7 @@ protected void innerUpdateMapping(LongConsumer handler, Consumer erro Index followIndex = params.getFollowShardId().getIndex(); ClusterStateRequest clusterStateRequest = CcrRequests.metaDataRequest(leaderIndex.getName()); - - remoteClient.admin().cluster().state(clusterStateRequest, ActionListener.wrap(clusterStateResponse -> { + CheckedConsumer onResponse = clusterStateResponse -> { IndexMetaData indexMetaData = clusterStateResponse.getState().metaData().getIndexSafe(leaderIndex); if (indexMetaData.getMappings().isEmpty()) { assert indexMetaData.getMappingVersion() == 1; @@ -140,7 +133,12 @@ protected void innerUpdateMapping(LongConsumer handler, Consumer erro followerClient.admin().indices().putMapping(putMappingRequest, ActionListener.wrap( putMappingResponse -> handler.accept(indexMetaData.getMappingVersion()), errorHandler)); - }, errorHandler)); + }; + try { + remoteClient(params).admin().cluster().state(clusterStateRequest, ActionListener.wrap(onResponse, errorHandler)); + } catch (Exception e) { + errorHandler.accept(e); + } } @Override @@ -181,7 +179,11 @@ protected void innerUpdateSettings(final LongConsumer finalHandler, final Consum } } }; - remoteClient.admin().cluster().state(clusterStateRequest, ActionListener.wrap(onResponse, errorHandler)); + try { + remoteClient(params).admin().cluster().state(clusterStateRequest, ActionListener.wrap(onResponse, errorHandler)); + } catch (Exception e) { + errorHandler.accept(e); + } } private void closeIndexUpdateSettingsAndOpenIndex(String followIndex, @@ -236,7 +238,7 @@ protected void innerSendShardChangesRequest(long from, int maxOperationCount, Co request.setMaxBatchSize(params.getMaxReadRequestSize()); request.setPollTimeout(params.getReadPollTimeout()); try { - remoteClient.execute(ShardChangesAction.INSTANCE, request, ActionListener.wrap(handler::accept, errorHandler)); + remoteClient(params).execute(ShardChangesAction.INSTANCE, request, ActionListener.wrap(handler::accept, errorHandler)); } catch (Exception e) { errorHandler.accept(e); } @@ -251,6 +253,10 @@ private String getLeaderShardHistoryUUID(ShardFollowTask params) { return recordedLeaderShardHistoryUUIDs[params.getLeaderShardId().id()]; } + private Client remoteClient(ShardFollowTask params) { + return wrapClient(client.getRemoteClusterClient(params.getRemoteCluster()), params.getHeaders()); + } + interface FollowerStatsInfoHandler { void accept(String followerHistoryUUID, long globalCheckpoint, long maxSeqNo); } diff --git a/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/CcrIntegTestCase.java b/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/CcrIntegTestCase.java index 01e51ea94f255..05b20050ee3c9 100644 --- a/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/CcrIntegTestCase.java +++ b/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/CcrIntegTestCase.java @@ -202,7 +202,7 @@ private NodeConfigurationSource createNodeConfigurationSource(String leaderSeedA builder.put(LicenseService.SELF_GENERATED_LICENSE_TYPE.getKey(), "trial"); // Let cluster state api return quickly in order to speed up auto follow tests: builder.put(CcrSettings.CCR_AUTO_FOLLOW_WAIT_FOR_METADATA_TIMEOUT.getKey(), TimeValue.timeValueMillis(100)); - if (leaderSeedAddress != null) { + if (configureRemoteClusterViaNodeSettings() && leaderSeedAddress != null) { builder.put("cluster.remote.leader_cluster.seeds", leaderSeedAddress); } return new NodeConfigurationSource() { @@ -247,6 +247,10 @@ protected boolean reuseClusters() { return true; } + protected boolean configureRemoteClusterViaNodeSettings() { + return true; + } + protected final Client leaderClient() { return clusterGroup.leaderCluster.client(); } diff --git a/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/RestartIndexFollowingIT.java b/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/RestartIndexFollowingIT.java index d72eca17fdb1b..b8649aaa4320c 100644 --- a/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/RestartIndexFollowingIT.java +++ b/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/RestartIndexFollowingIT.java @@ -6,8 +6,11 @@ package org.elasticsearch.xpack.ccr; +import org.elasticsearch.action.admin.cluster.settings.ClusterUpdateSettingsRequest; +import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.xcontent.XContentType; import org.elasticsearch.index.IndexSettings; +import org.elasticsearch.transport.TransportService; import org.elasticsearch.xpack.CcrIntegTestCase; import org.elasticsearch.xpack.core.ccr.action.PutFollowAction; @@ -24,12 +27,17 @@ protected int numberOfNodesPerCluster() { return 1; } + @Override + protected boolean configureRemoteClusterViaNodeSettings() { + return false; + } + public void testFollowIndex() throws Exception { final String leaderIndexSettings = getIndexSettings(1, 0, singletonMap(IndexSettings.INDEX_SOFT_DELETES_SETTING.getKey(), "true")); - singletonMap(IndexSettings.INDEX_SOFT_DELETES_SETTING.getKey(), "true"); assertAcked(leaderClient().admin().indices().prepareCreate("index1").setSource(leaderIndexSettings, XContentType.JSON)); ensureLeaderGreen("index1"); + setupRemoteCluster(); final PutFollowAction.Request followRequest = putFollow("index1", "index2"); followerClient().execute(PutFollowAction.INSTANCE, followRequest).get(); @@ -57,6 +65,28 @@ public void testFollowIndex() throws Exception { assertThat(followerClient().prepareSearch("index2").get().getHits().getTotalHits().value, equalTo(firstBatchNumDocs + secondBatchNumDocs)); }); + + getLeaderCluster().fullRestart(); + ensureLeaderGreen("index1"); + // Remote connection needs to be re-configured, because all the nodes in leader cluster have been restarted: + setupRemoteCluster(); + + final long thirdBatchNumDocs = randomIntBetween(2, 64); + for (int i = 0; i < thirdBatchNumDocs; i++) { + leaderClient().prepareIndex("index1", "doc").setSource("{}", XContentType.JSON).get(); + } + + assertBusy(() -> { + assertThat(followerClient().prepareSearch("index2").get().getHits().getTotalHits().value, + equalTo(firstBatchNumDocs + secondBatchNumDocs + thirdBatchNumDocs)); + }); + } + + private void setupRemoteCluster() { + ClusterUpdateSettingsRequest updateSettingsRequest = new ClusterUpdateSettingsRequest(); + String address = getLeaderCluster().getMasterNodeInstance(TransportService.class).boundAddress().publishAddress().toString(); + updateSettingsRequest.persistentSettings(Settings.builder().put("cluster.remote.leader_cluster.seeds", address)); + assertAcked(followerClient().admin().cluster().updateSettings(updateSettingsRequest).actionGet()); } } From c812e6aea6a66092188d56090ff5a77a48a81a8b Mon Sep 17 00:00:00 2001 From: Przemyslaw Gomulka Date: Thu, 10 Jan 2019 15:23:34 +0100 Subject: [PATCH 076/186] Fix line length in org.elasticsearch.routing (#37253) Remove the line length suppression for this package and fix offending lines relates: #34884 --- .../resources/checkstyle_suppressions.xml | 2 - .../elasticsearch/routing/AliasRoutingIT.java | 372 +++++++++++++++--- .../routing/SimpleRoutingIT.java | 340 +++++++++++++--- 3 files changed, 588 insertions(+), 126 deletions(-) diff --git a/buildSrc/src/main/resources/checkstyle_suppressions.xml b/buildSrc/src/main/resources/checkstyle_suppressions.xml index 4d5872639480b..370a6bbcac901 100644 --- a/buildSrc/src/main/resources/checkstyle_suppressions.xml +++ b/buildSrc/src/main/resources/checkstyle_suppressions.xml @@ -53,8 +53,6 @@ - - diff --git a/server/src/test/java/org/elasticsearch/routing/AliasRoutingIT.java b/server/src/test/java/org/elasticsearch/routing/AliasRoutingIT.java index eceef54ccac8a..1919b7e6a9ae4 100644 --- a/server/src/test/java/org/elasticsearch/routing/AliasRoutingIT.java +++ b/server/src/test/java/org/elasticsearch/routing/AliasRoutingIT.java @@ -71,7 +71,12 @@ public void testAliasCrudRouting() throws Exception { .execute().actionGet(); for (int i = 0; i < 5; i++) { assertThat(client().prepareGet("alias0", "type1", "1").execute().actionGet().isExists(), equalTo(true)); - assertThat(client().prepareGet("alias0", "type1", "1").execute().actionGet().getSourceAsMap().get("field").toString(), equalTo("value2")); + assertThat(client().prepareGet("alias0", "type1", "1") + .execute() + .actionGet() + .getSourceAsMap() + .get("field") + .toString(), equalTo("value2")); } @@ -108,10 +113,10 @@ public void testAliasSearchRouting() throws Exception { createIndex("test"); ensureGreen(); assertAcked(admin().indices().prepareAliases() - .addAliasAction(AliasActions.add().index("test").alias("alias")) - .addAliasAction(AliasActions.add().index("test").alias("alias0").routing("0")) - .addAliasAction(AliasActions.add().index("test").alias("alias1").routing("1")) - .addAliasAction(AliasActions.add().index("test").alias("alias01").searchRouting("0,1"))); + .addAliasAction(AliasActions.add().index("test").alias("alias")) + .addAliasAction(AliasActions.add().index("test").alias("alias0").routing("0")) + .addAliasAction(AliasActions.add().index("test").alias("alias1").routing("1")) + .addAliasAction(AliasActions.add().index("test").alias("alias01").searchRouting("0,1"))); logger.info("--> indexing with id [1], and routing [0] using alias"); client().prepareIndex("alias0", "type1", "1").setSource("field", "value1").setRefreshPolicy(RefreshPolicy.IMMEDIATE).get(); @@ -126,23 +131,80 @@ public void testAliasSearchRouting() throws Exception { logger.info("--> search with no routing, should fine one"); for (int i = 0; i < 5; i++) { - assertThat(client().prepareSearch().setQuery(QueryBuilders.matchAllQuery()).execute().actionGet().getHits().getTotalHits().value, equalTo(1L)); + assertThat(client().prepareSearch() + .setQuery(QueryBuilders.matchAllQuery()) + .execute() + .actionGet() + .getHits() + .getTotalHits().value, equalTo(1L)); } logger.info("--> search with wrong routing, should not find"); for (int i = 0; i < 5; i++) { - assertThat(client().prepareSearch().setRouting("1").setQuery(QueryBuilders.matchAllQuery()).execute().actionGet().getHits().getTotalHits().value, equalTo(0L)); - assertThat(client().prepareSearch().setSize(0).setRouting("1").setQuery(QueryBuilders.matchAllQuery()).execute().actionGet().getHits().getTotalHits().value, equalTo(0L)); - assertThat(client().prepareSearch("alias1").setQuery(QueryBuilders.matchAllQuery()).execute().actionGet().getHits().getTotalHits().value, equalTo(0L)); - assertThat(client().prepareSearch("alias1").setSize(0).setQuery(QueryBuilders.matchAllQuery()).execute().actionGet().getHits().getTotalHits().value, equalTo(0L)); + assertThat(client().prepareSearch() + .setRouting("1") + .setQuery(QueryBuilders.matchAllQuery()) + .execute() + .actionGet() + .getHits() + .getTotalHits().value, equalTo(0L)); + + assertThat(client().prepareSearch() + .setSize(0) + .setRouting("1") + .setQuery(QueryBuilders.matchAllQuery()) + .execute() + .actionGet() + .getHits() + .getTotalHits().value, equalTo(0L)); + + assertThat(client().prepareSearch("alias1") + .setQuery(QueryBuilders.matchAllQuery()) + .execute() + .actionGet() + .getHits() + .getTotalHits().value, equalTo(0L)); + + assertThat(client().prepareSearch("alias1") + .setSize(0) + .setQuery(QueryBuilders.matchAllQuery()) + .execute() + .actionGet() + .getHits() + .getTotalHits().value, equalTo(0L)); } logger.info("--> search with correct routing, should find"); for (int i = 0; i < 5; i++) { - assertThat(client().prepareSearch().setRouting("0").setQuery(QueryBuilders.matchAllQuery()).execute().actionGet().getHits().getTotalHits().value, equalTo(1L)); - assertThat(client().prepareSearch().setSize(0).setRouting("0").setQuery(QueryBuilders.matchAllQuery()).execute().actionGet().getHits().getTotalHits().value, equalTo(1L)); - assertThat(client().prepareSearch("alias0").setQuery(QueryBuilders.matchAllQuery()).execute().actionGet().getHits().getTotalHits().value, equalTo(1L)); - assertThat(client().prepareSearch("alias0").setSize(0).setQuery(QueryBuilders.matchAllQuery()).execute().actionGet().getHits().getTotalHits().value, equalTo(1L)); + + assertThat(client().prepareSearch() + .setRouting("0") + .setQuery(QueryBuilders.matchAllQuery()) + .execute() + .actionGet() + .getHits() + .getTotalHits().value, equalTo(1L)); + assertThat(client().prepareSearch() + .setSize(0) + .setRouting("0") + .setQuery(QueryBuilders.matchAllQuery()) + .execute() + .actionGet() + .getHits() + .getTotalHits().value, equalTo(1L)); + assertThat(client().prepareSearch("alias0") + .setQuery(QueryBuilders.matchAllQuery()) + .execute() + .actionGet() + .getHits() + .getTotalHits().value, equalTo(1L)); + assertThat(client().prepareSearch("alias0") + .setSize(0) + .setQuery(QueryBuilders.matchAllQuery()) + .execute() + .actionGet() + .getHits() + .getTotalHits().value, equalTo(1L)); } logger.info("--> indexing with id [2], and routing [1] using alias"); @@ -150,50 +212,166 @@ public void testAliasSearchRouting() throws Exception { logger.info("--> search with no routing, should fine two"); for (int i = 0; i < 5; i++) { - assertThat(client().prepareSearch().setQuery(QueryBuilders.matchAllQuery()).execute().actionGet().getHits().getTotalHits().value, equalTo(2L)); - assertThat(client().prepareSearch().setSize(0).setQuery(QueryBuilders.matchAllQuery()).execute().actionGet().getHits().getTotalHits().value, equalTo(2L)); + assertThat(client().prepareSearch() + .setQuery(QueryBuilders.matchAllQuery()) + .execute() + .actionGet() + .getHits() + .getTotalHits().value, equalTo(2L)); + assertThat(client().prepareSearch() + .setSize(0) + .setQuery(QueryBuilders.matchAllQuery()) + .execute() + .actionGet() + .getHits() + .getTotalHits().value, equalTo(2L)); } logger.info("--> search with 0 routing, should find one"); for (int i = 0; i < 5; i++) { - assertThat(client().prepareSearch().setRouting("0").setQuery(QueryBuilders.matchAllQuery()).execute().actionGet().getHits().getTotalHits().value, equalTo(1L)); - assertThat(client().prepareSearch().setSize(0).setRouting("0").setQuery(QueryBuilders.matchAllQuery()).execute().actionGet().getHits().getTotalHits().value, equalTo(1L)); - assertThat(client().prepareSearch("alias0").setQuery(QueryBuilders.matchAllQuery()).execute().actionGet().getHits().getTotalHits().value, equalTo(1L)); - assertThat(client().prepareSearch("alias0").setSize(0).setQuery(QueryBuilders.matchAllQuery()).execute().actionGet().getHits().getTotalHits().value, equalTo(1L)); + assertThat(client().prepareSearch() + .setRouting("0") + .setQuery(QueryBuilders.matchAllQuery()) + .execute() + .actionGet() + .getHits() + .getTotalHits().value, equalTo(1L)); + assertThat(client().prepareSearch() + .setSize(0) + .setRouting("0") + .setQuery(QueryBuilders.matchAllQuery()) + .execute() + .actionGet() + .getHits() + .getTotalHits().value, equalTo(1L)); + assertThat(client().prepareSearch("alias0") + .setQuery(QueryBuilders.matchAllQuery()) + .execute() + .actionGet() + .getHits() + .getTotalHits().value, equalTo(1L)); + assertThat(client().prepareSearch("alias0") + .setSize(0) + .setQuery(QueryBuilders.matchAllQuery()) + .execute() + .actionGet() + .getHits() + .getTotalHits().value, equalTo(1L)); } logger.info("--> search with 1 routing, should find one"); for (int i = 0; i < 5; i++) { - assertThat(client().prepareSearch().setRouting("1").setQuery(QueryBuilders.matchAllQuery()).execute().actionGet().getHits().getTotalHits().value, equalTo(1L)); - assertThat(client().prepareSearch().setSize(0).setRouting("1").setQuery(QueryBuilders.matchAllQuery()).execute().actionGet().getHits().getTotalHits().value, equalTo(1L)); - assertThat(client().prepareSearch("alias1").setQuery(QueryBuilders.matchAllQuery()).execute().actionGet().getHits().getTotalHits().value, equalTo(1L)); - assertThat(client().prepareSearch("alias1").setSize(0).setQuery(QueryBuilders.matchAllQuery()).execute().actionGet().getHits().getTotalHits().value, equalTo(1L)); + assertThat(client().prepareSearch() + .setRouting("1") + .setQuery(QueryBuilders.matchAllQuery()) + .execute() + .actionGet() + .getHits() + .getTotalHits().value, equalTo(1L)); + assertThat(client().prepareSearch() + .setSize(0) + .setRouting("1") + .setQuery(QueryBuilders.matchAllQuery()) + .execute() + .actionGet() + .getHits() + .getTotalHits().value, equalTo(1L)); + assertThat(client().prepareSearch("alias1") + .setQuery(QueryBuilders.matchAllQuery()) + .execute() + .actionGet() + .getHits() + .getTotalHits().value, equalTo(1L)); + assertThat(client().prepareSearch("alias1") + .setSize(0) + .setQuery(QueryBuilders.matchAllQuery()) + .execute() + .actionGet() + .getHits() + .getTotalHits().value, equalTo(1L)); } logger.info("--> search with 0,1 indexRoutings , should find two"); for (int i = 0; i < 5; i++) { - assertThat(client().prepareSearch().setRouting("0", "1").setQuery(QueryBuilders.matchAllQuery()).execute().actionGet().getHits().getTotalHits().value, equalTo(2L)); - assertThat(client().prepareSearch().setSize(0).setRouting("0", "1").setQuery(QueryBuilders.matchAllQuery()).execute().actionGet().getHits().getTotalHits().value, equalTo(2L)); - assertThat(client().prepareSearch("alias01").setQuery(QueryBuilders.matchAllQuery()).execute().actionGet().getHits().getTotalHits().value, equalTo(2L)); - assertThat(client().prepareSearch("alias01").setSize(0).setQuery(QueryBuilders.matchAllQuery()).execute().actionGet().getHits().getTotalHits().value, equalTo(2L)); + assertThat(client().prepareSearch() + .setRouting("0", "1") + .setQuery(QueryBuilders.matchAllQuery()) + .execute() + .actionGet() + .getHits() + .getTotalHits().value, equalTo(2L)); + assertThat(client().prepareSearch() + .setSize(0) + .setRouting("0", "1") + .setQuery(QueryBuilders.matchAllQuery()) + .execute() + .actionGet() + .getHits() + .getTotalHits().value, equalTo(2L)); + assertThat(client().prepareSearch("alias01") + .setQuery(QueryBuilders.matchAllQuery()) + .execute() + .actionGet() + .getHits() + .getTotalHits().value, equalTo(2L)); + assertThat(client().prepareSearch("alias01") + .setSize(0) + .setQuery(QueryBuilders.matchAllQuery()) + .execute() + .actionGet() + .getHits() + .getTotalHits().value, equalTo(2L)); } logger.info("--> search with two routing aliases , should find two"); for (int i = 0; i < 5; i++) { - assertThat(client().prepareSearch("alias0", "alias1").setQuery(QueryBuilders.matchAllQuery()).execute().actionGet().getHits().getTotalHits().value, equalTo(2L)); - assertThat(client().prepareSearch("alias0", "alias1").setSize(0).setQuery(QueryBuilders.matchAllQuery()).execute().actionGet().getHits().getTotalHits().value, equalTo(2L)); + assertThat(client().prepareSearch("alias0", "alias1") + .setQuery(QueryBuilders.matchAllQuery()) + .execute() + .actionGet() + .getHits() + .getTotalHits().value, equalTo(2L)); + assertThat(client().prepareSearch("alias0", "alias1") + .setSize(0) + .setQuery(QueryBuilders.matchAllQuery()) + .execute() + .actionGet() + .getHits() + .getTotalHits().value, equalTo(2L)); } logger.info("--> search with alias0, alias1 and alias01, should find two"); for (int i = 0; i < 5; i++) { - assertThat(client().prepareSearch("alias0", "alias1", "alias01").setQuery(QueryBuilders.matchAllQuery()).execute().actionGet().getHits().getTotalHits().value, equalTo(2L)); - assertThat(client().prepareSearch("alias0", "alias1", "alias01").setSize(0).setQuery(QueryBuilders.matchAllQuery()).execute().actionGet().getHits().getTotalHits().value, equalTo(2L)); + assertThat(client().prepareSearch("alias0", "alias1", "alias01") + .setQuery(QueryBuilders.matchAllQuery()) + .execute() + .actionGet() + .getHits() + .getTotalHits().value, equalTo(2L)); + assertThat(client().prepareSearch("alias0", "alias1", "alias01") + .setSize(0) + .setQuery(QueryBuilders.matchAllQuery()) + .execute() + .actionGet() + .getHits() + .getTotalHits().value, equalTo(2L)); } logger.info("--> search with test, alias0 and alias1, should find two"); for (int i = 0; i < 5; i++) { - assertThat(client().prepareSearch("test", "alias0", "alias1").setQuery(QueryBuilders.matchAllQuery()).execute().actionGet().getHits().getTotalHits().value, equalTo(2L)); - assertThat(client().prepareSearch("test", "alias0", "alias1").setSize(0).setQuery(QueryBuilders.matchAllQuery()).execute().actionGet().getHits().getTotalHits().value, equalTo(2L)); + assertThat(client().prepareSearch("test", "alias0", "alias1") + .setQuery(QueryBuilders.matchAllQuery()) + .execute() + .actionGet() + .getHits() + .getTotalHits().value, equalTo(2L)); + assertThat(client().prepareSearch("test", "alias0", "alias1") + .setSize(0) + .setQuery(QueryBuilders.matchAllQuery()) + .execute() + .actionGet() + .getHits() + .getTotalHits().value, equalTo(2L)); } } @@ -210,12 +388,12 @@ public void testAliasSearchRoutingWithTwoIndices() throws Exception { createIndex("test-b"); ensureGreen(); assertAcked(admin().indices().prepareAliases() - .addAliasAction(AliasActions.add().index("test-a").alias("alias-a0").routing("0")) - .addAliasAction(AliasActions.add().index("test-a").alias("alias-a1").routing("1")) - .addAliasAction(AliasActions.add().index("test-b").alias("alias-b0").routing("0")) - .addAliasAction(AliasActions.add().index("test-b").alias("alias-b1").routing("1")) - .addAliasAction(AliasActions.add().index("test-a").alias("alias-ab").searchRouting("0")) - .addAliasAction(AliasActions.add().index("test-b").alias("alias-ab").searchRouting("1"))); + .addAliasAction(AliasActions.add().index("test-a").alias("alias-a0").routing("0")) + .addAliasAction(AliasActions.add().index("test-a").alias("alias-a1").routing("1")) + .addAliasAction(AliasActions.add().index("test-b").alias("alias-b0").routing("0")) + .addAliasAction(AliasActions.add().index("test-b").alias("alias-b1").routing("1")) + .addAliasAction(AliasActions.add().index("test-a").alias("alias-ab").searchRouting("0")) + .addAliasAction(AliasActions.add().index("test-b").alias("alias-ab").searchRouting("1"))); ensureGreen(); // wait for events again to make sure we got the aliases on all nodes logger.info("--> indexing with id [1], and routing [0] using alias to test-a"); client().prepareIndex("alias-a0", "type1", "1").setSource("field", "value1").setRefreshPolicy(RefreshPolicy.IMMEDIATE).get(); @@ -242,20 +420,53 @@ public void testAliasSearchRoutingWithTwoIndices() throws Exception { logger.info("--> search with alias-a1,alias-b0, should not find"); for (int i = 0; i < 5; i++) { - assertThat(client().prepareSearch("alias-a1", "alias-b0").setQuery(QueryBuilders.matchAllQuery()).execute().actionGet().getHits().getTotalHits().value, equalTo(0L)); - assertThat(client().prepareSearch("alias-a1", "alias-b0").setSize(0).setQuery(QueryBuilders.matchAllQuery()).execute().actionGet().getHits().getTotalHits().value, equalTo(0L)); + assertThat(client().prepareSearch("alias-a1", "alias-b0") + .setQuery(QueryBuilders.matchAllQuery()) + .execute() + .actionGet() + .getHits() + .getTotalHits().value, equalTo(0L)); + assertThat(client().prepareSearch("alias-a1", "alias-b0") + .setSize(0) + .setQuery(QueryBuilders.matchAllQuery()) + .execute() + .actionGet() + .getHits() + .getTotalHits().value, equalTo(0L)); } logger.info("--> search with alias-ab, should find two"); for (int i = 0; i < 5; i++) { - assertThat(client().prepareSearch("alias-ab").setQuery(QueryBuilders.matchAllQuery()).execute().actionGet().getHits().getTotalHits().value, equalTo(2L)); - assertThat(client().prepareSearch("alias-ab").setSize(0).setQuery(QueryBuilders.matchAllQuery()).execute().actionGet().getHits().getTotalHits().value, equalTo(2L)); + assertThat(client().prepareSearch("alias-ab") + .setQuery(QueryBuilders.matchAllQuery()) + .execute() + .actionGet() + .getHits() + .getTotalHits().value, equalTo(2L)); + assertThat(client().prepareSearch("alias-ab") + .setSize(0) + .setQuery(QueryBuilders.matchAllQuery()) + .execute() + .actionGet() + .getHits() + .getTotalHits().value, equalTo(2L)); } logger.info("--> search with alias-a0,alias-b1 should find two"); for (int i = 0; i < 5; i++) { - assertThat(client().prepareSearch("alias-a0", "alias-b1").setQuery(QueryBuilders.matchAllQuery()).execute().actionGet().getHits().getTotalHits().value, equalTo(2L)); - assertThat(client().prepareSearch("alias-a0", "alias-b1").setSize(0).setQuery(QueryBuilders.matchAllQuery()).execute().actionGet().getHits().getTotalHits().value, equalTo(2L)); + assertThat(client().prepareSearch("alias-a0", "alias-b1") + .setQuery(QueryBuilders.matchAllQuery()) + .execute() + .actionGet() + .getHits() + .getTotalHits().value, equalTo(2L)); + assertThat(client().prepareSearch("alias-a0", "alias-b1") + .setSize(0) + .setQuery(QueryBuilders.matchAllQuery()) + .execute() + .actionGet() + .getHits() + .getTotalHits().value, equalTo(2L)); } } @@ -269,7 +480,7 @@ public void testAliasSearchRoutingWithConcreteAndAliasedIndices_issue2682() thro createIndex("index", "index_2"); ensureGreen(); assertAcked(admin().indices().prepareAliases() - .addAliasAction(AliasActions.add().index("index").alias("index_1").routing("1"))); + .addAliasAction(AliasActions.add().index("index").alias("index_1").routing("1"))); logger.info("--> indexing on index_1 which is an alias for index with routing [1]"); client().prepareIndex("index_1", "type1", "1").setSource("field", "value1").setRefreshPolicy(RefreshPolicy.IMMEDIATE).get(); @@ -279,7 +490,12 @@ public void testAliasSearchRoutingWithConcreteAndAliasedIndices_issue2682() thro logger.info("--> search all on index_* should find two"); for (int i = 0; i < 5; i++) { - assertThat(client().prepareSearch("index_*").setQuery(QueryBuilders.matchAllQuery()).execute().actionGet().getHits().getTotalHits().value, equalTo(2L)); + assertThat(client().prepareSearch("index_*") + .setQuery(QueryBuilders.matchAllQuery()) + .execute() + .actionGet() + .getHits() + .getTotalHits().value, equalTo(2L)); } } @@ -294,14 +510,19 @@ public void testAliasSearchRoutingWithConcreteAndAliasedIndices_issue3268() thro createIndex("index", "index_2"); ensureGreen(); assertAcked(admin().indices().prepareAliases() - .addAliasAction(AliasActions.add().index("index").alias("index_1").routing("1"))); + .addAliasAction(AliasActions.add().index("index").alias("index_1").routing("1"))); logger.info("--> indexing on index_1 which is an alias for index with routing [1]"); client().prepareIndex("index_1", "type1", "1").setSource("field", "value1").setRefreshPolicy(RefreshPolicy.IMMEDIATE).get(); logger.info("--> indexing on index_2 which is a concrete index"); client().prepareIndex("index_2", "type2", "2").setSource("field", "value2").setRefreshPolicy(RefreshPolicy.IMMEDIATE).get(); - SearchResponse searchResponse = client().prepareSearch("index_*").setSearchType(SearchType.QUERY_THEN_FETCH).setSize(1).setQuery(QueryBuilders.matchAllQuery()).execute().actionGet(); + SearchResponse searchResponse = client().prepareSearch("index_*") + .setSearchType(SearchType.QUERY_THEN_FETCH) + .setSize(1) + .setQuery(QueryBuilders.matchAllQuery()) + .execute() + .actionGet(); logger.info("--> search all on index_* should find two"); assertThat(searchResponse.getHits().getTotalHits().value, equalTo(2L)); @@ -315,7 +536,7 @@ public void testIndexingAliasesOverTime() throws Exception { ensureGreen(); logger.info("--> creating alias with routing [3]"); assertAcked(admin().indices().prepareAliases() - .addAliasAction(AliasActions.add().index("test").alias("alias").routing("3"))); + .addAliasAction(AliasActions.add().index("test").alias("alias").routing("3"))); logger.info("--> indexing with id [0], and routing [3]"); client().prepareIndex("alias", "type1", "0").setSource("field", "value1").setRefreshPolicy(RefreshPolicy.IMMEDIATE).get(); @@ -324,23 +545,45 @@ public void testIndexingAliasesOverTime() throws Exception { logger.info("--> verifying get and search with routing, should find"); for (int i = 0; i < 5; i++) { assertThat(client().prepareGet("test", "type1", "0").setRouting("3").execute().actionGet().isExists(), equalTo(true)); - assertThat(client().prepareSearch("alias").setQuery(QueryBuilders.matchAllQuery()).execute().actionGet().getHits().getTotalHits().value, equalTo(1L)); - assertThat(client().prepareSearch("alias").setSize(0).setQuery(QueryBuilders.matchAllQuery()).execute().actionGet().getHits().getTotalHits().value, equalTo(1L)); + assertThat(client().prepareSearch("alias") + .setQuery(QueryBuilders.matchAllQuery()) + .execute() + .actionGet() + .getHits() + .getTotalHits().value, equalTo(1L)); + assertThat(client().prepareSearch("alias") + .setSize(0) + .setQuery(QueryBuilders.matchAllQuery()) + .execute() + .actionGet() + .getHits() + .getTotalHits().value, equalTo(1L)); } logger.info("--> creating alias with routing [4]"); assertAcked(admin().indices().prepareAliases() - .addAliasAction(AliasActions.add().index("test").alias("alias").routing("4"))); + .addAliasAction(AliasActions.add().index("test").alias("alias").routing("4"))); logger.info("--> verifying search with wrong routing should not find"); for (int i = 0; i < 5; i++) { - assertThat(client().prepareSearch("alias").setQuery(QueryBuilders.matchAllQuery()).execute().actionGet().getHits().getTotalHits().value, equalTo(0L)); - assertThat(client().prepareSearch("alias").setSize(0).setQuery(QueryBuilders.matchAllQuery()).execute().actionGet().getHits().getTotalHits().value, equalTo(0L)); + assertThat(client().prepareSearch("alias") + .setQuery(QueryBuilders.matchAllQuery()) + .execute() + .actionGet() + .getHits() + .getTotalHits().value, equalTo(0L)); + assertThat(client().prepareSearch("alias") + .setSize(0) + .setQuery(QueryBuilders.matchAllQuery()) + .execute() + .actionGet() + .getHits() + .getTotalHits().value, equalTo(0L)); } logger.info("--> creating alias with search routing [3,4] and index routing 4"); assertAcked(client().admin().indices().prepareAliases() - .addAliasAction(AliasActions.add().index("test").alias("alias").searchRouting("3,4").indexRouting("4"))); + .addAliasAction(AliasActions.add().index("test").alias("alias").searchRouting("3,4").indexRouting("4"))); logger.info("--> indexing with id [1], and routing [4]"); client().prepareIndex("alias", "type1", "1").setSource("field", "value2").setRefreshPolicy(RefreshPolicy.IMMEDIATE).get(); @@ -350,8 +593,19 @@ public void testIndexingAliasesOverTime() throws Exception { for (int i = 0; i < 5; i++) { assertThat(client().prepareGet("test", "type1", "0").setRouting("3").execute().actionGet().isExists(), equalTo(true)); assertThat(client().prepareGet("test", "type1", "1").setRouting("4").execute().actionGet().isExists(), equalTo(true)); - assertThat(client().prepareSearch("alias").setQuery(QueryBuilders.matchAllQuery()).execute().actionGet().getHits().getTotalHits().value, equalTo(2L)); - assertThat(client().prepareSearch("alias").setSize(0).setQuery(QueryBuilders.matchAllQuery()).execute().actionGet().getHits().getTotalHits().value, equalTo(2L)); + assertThat(client().prepareSearch("alias") + .setQuery(QueryBuilders.matchAllQuery()) + .execute() + .actionGet() + .getHits() + .getTotalHits().value, equalTo(2L)); + assertThat(client().prepareSearch("alias") + .setSize(0) + .setQuery(QueryBuilders.matchAllQuery()) + .execute() + .actionGet() + .getHits() + .getTotalHits().value, equalTo(2L)); } } diff --git a/server/src/test/java/org/elasticsearch/routing/SimpleRoutingIT.java b/server/src/test/java/org/elasticsearch/routing/SimpleRoutingIT.java index ad8024e76b4b2..b383a80296247 100644 --- a/server/src/test/java/org/elasticsearch/routing/SimpleRoutingIT.java +++ b/server/src/test/java/org/elasticsearch/routing/SimpleRoutingIT.java @@ -78,7 +78,10 @@ public void testSimpleCrudRouting() throws Exception { ensureGreen(); String routingValue = findNonMatchingRoutingValue("test", "1"); logger.info("--> indexing with id [1], and routing [{}]", routingValue); - client().prepareIndex("test", "type1", "1").setRouting(routingValue).setSource("field", "value1").setRefreshPolicy(RefreshPolicy.IMMEDIATE) + client().prepareIndex("test", "type1", "1") + .setRouting(routingValue) + .setSource("field", "value1") + .setRefreshPolicy(RefreshPolicy.IMMEDIATE) .get(); logger.info("--> verifying get with no routing, should not find anything"); for (int i = 0; i < 5; i++) { @@ -86,25 +89,40 @@ public void testSimpleCrudRouting() throws Exception { } logger.info("--> verifying get with routing, should find"); for (int i = 0; i < 5; i++) { - assertThat(client().prepareGet("test", "type1", "1").setRouting(routingValue).execute().actionGet().isExists(), equalTo(true)); + assertThat(client().prepareGet("test", "type1", "1") + .setRouting(routingValue) + .execute() + .actionGet() + .isExists(), equalTo(true)); } logger.info("--> deleting with no routing, should not delete anything"); client().prepareDelete("test", "type1", "1").setRefreshPolicy(RefreshPolicy.IMMEDIATE).get(); for (int i = 0; i < 5; i++) { assertThat(client().prepareGet("test", "type1", "1").execute().actionGet().isExists(), equalTo(false)); - assertThat(client().prepareGet("test", "type1", "1").setRouting(routingValue).execute().actionGet().isExists(), equalTo(true)); + assertThat(client().prepareGet("test", "type1", "1") + .setRouting(routingValue) + .execute() + .actionGet() + .isExists(), equalTo(true)); } logger.info("--> deleting with routing, should delete"); client().prepareDelete("test", "type1", "1").setRouting(routingValue).setRefreshPolicy(RefreshPolicy.IMMEDIATE).get(); for (int i = 0; i < 5; i++) { assertThat(client().prepareGet("test", "type1", "1").execute().actionGet().isExists(), equalTo(false)); - assertThat(client().prepareGet("test", "type1", "1").setRouting(routingValue).execute().actionGet().isExists(), equalTo(false)); + assertThat(client().prepareGet("test", "type1", "1") + .setRouting(routingValue) + .execute() + .actionGet() + .isExists(), equalTo(false)); } logger.info("--> indexing with id [1], and routing [0]"); - client().prepareIndex("test", "type1", "1").setRouting(routingValue).setSource("field", "value1").setRefreshPolicy(RefreshPolicy.IMMEDIATE) + client().prepareIndex("test", "type1", "1") + .setRouting(routingValue) + .setSource("field", "value1") + .setRefreshPolicy(RefreshPolicy.IMMEDIATE) .get(); logger.info("--> verifying get with no routing, should not find anything"); for (int i = 0; i < 5; i++) { @@ -112,7 +130,11 @@ public void testSimpleCrudRouting() throws Exception { } logger.info("--> verifying get with routing, should find"); for (int i = 0; i < 5; i++) { - assertThat(client().prepareGet("test", "type1", "1").setRouting(routingValue).execute().actionGet().isExists(), equalTo(true)); + assertThat(client().prepareGet("test", "type1", "1") + .setRouting(routingValue) + .execute() + .actionGet() + .isExists(), equalTo(true)); } } @@ -122,7 +144,10 @@ public void testSimpleSearchRouting() { String routingValue = findNonMatchingRoutingValue("test", "1"); logger.info("--> indexing with id [1], and routing [{}]", routingValue); - client().prepareIndex("test", "type1", "1").setRouting(routingValue).setSource("field", "value1").setRefreshPolicy(RefreshPolicy.IMMEDIATE) + client().prepareIndex("test", "type1", "1") + .setRouting(routingValue) + .setSource("field", "value1") + .setRefreshPolicy(RefreshPolicy.IMMEDIATE) .get(); logger.info("--> verifying get with no routing, should not find anything"); for (int i = 0; i < 5; i++) { @@ -130,65 +155,193 @@ public void testSimpleSearchRouting() { } logger.info("--> verifying get with routing, should find"); for (int i = 0; i < 5; i++) { - assertThat(client().prepareGet("test", "type1", "1").setRouting(routingValue).execute().actionGet().isExists(), equalTo(true)); + assertThat(client().prepareGet("test", "type1", "1") + .setRouting(routingValue) + .execute() + .actionGet() + .isExists(), equalTo(true)); } logger.info("--> search with no routing, should fine one"); for (int i = 0; i < 5; i++) { - assertThat(client().prepareSearch().setQuery(QueryBuilders.matchAllQuery()).execute().actionGet().getHits().getTotalHits().value, equalTo(1L)); + assertThat(client().prepareSearch() + .setQuery(QueryBuilders.matchAllQuery()) + .execute() + .actionGet() + .getHits() + .getTotalHits() + .value, equalTo(1L)); } logger.info("--> search with wrong routing, should not find"); for (int i = 0; i < 5; i++) { - assertThat(client().prepareSearch().setRouting("1").setQuery(QueryBuilders.matchAllQuery()).execute().actionGet().getHits().getTotalHits().value, equalTo(0L)); - assertThat(client().prepareSearch().setSize(0).setRouting("1").setQuery(QueryBuilders.matchAllQuery()).execute().actionGet().getHits().getTotalHits().value, equalTo(0L)); + assertThat(client().prepareSearch() + .setRouting("1") + .setQuery(QueryBuilders.matchAllQuery()) + .execute() + .actionGet() + .getHits() + .getTotalHits() + .value, equalTo(0L)); + assertThat(client().prepareSearch() + .setSize(0) + .setRouting("1") + .setQuery(QueryBuilders.matchAllQuery()) + .execute() + .actionGet() + .getHits() + .getTotalHits() + .value, equalTo(0L)); } logger.info("--> search with correct routing, should find"); for (int i = 0; i < 5; i++) { - assertThat(client().prepareSearch().setRouting(routingValue).setQuery(QueryBuilders.matchAllQuery()).execute().actionGet().getHits().getTotalHits().value, equalTo(1L)); - assertThat(client().prepareSearch().setSize(0).setRouting(routingValue).setQuery(QueryBuilders.matchAllQuery()).execute().actionGet().getHits().getTotalHits().value, equalTo(1L)); + assertThat(client().prepareSearch() + .setRouting(routingValue) + .setQuery(QueryBuilders.matchAllQuery()) + .execute() + .actionGet() + .getHits() + .getTotalHits() + .value, equalTo(1L)); + assertThat(client().prepareSearch() + .setSize(0) + .setRouting(routingValue) + .setQuery(QueryBuilders.matchAllQuery()) + .execute() + .actionGet() + .getHits() + .getTotalHits() + .value, equalTo(1L)); } String secondRoutingValue = "1"; logger.info("--> indexing with id [{}], and routing [{}]", routingValue, secondRoutingValue); - client().prepareIndex("test", "type1", routingValue).setRouting(secondRoutingValue).setSource("field", "value1").setRefreshPolicy(RefreshPolicy.IMMEDIATE).get(); + client().prepareIndex("test", "type1", routingValue) + .setRouting(secondRoutingValue) + .setSource("field", "value1") + .setRefreshPolicy(RefreshPolicy.IMMEDIATE) + .get(); logger.info("--> search with no routing, should fine two"); for (int i = 0; i < 5; i++) { - assertThat(client().prepareSearch().setQuery(QueryBuilders.matchAllQuery()).execute().actionGet().getHits().getTotalHits().value, equalTo(2L)); - assertThat(client().prepareSearch().setSize(0).setQuery(QueryBuilders.matchAllQuery()).execute().actionGet().getHits().getTotalHits().value, equalTo(2L)); + assertThat(client().prepareSearch() + .setQuery(QueryBuilders.matchAllQuery()) + .execute() + .actionGet() + .getHits() + .getTotalHits() + .value, equalTo(2L)); + assertThat(client().prepareSearch() + .setSize(0) + .setQuery(QueryBuilders.matchAllQuery()) + .execute() + .actionGet() + .getHits() + .getTotalHits() + .value, equalTo(2L)); } logger.info("--> search with {} routing, should find one", routingValue); for (int i = 0; i < 5; i++) { - assertThat(client().prepareSearch().setRouting(routingValue).setQuery(QueryBuilders.matchAllQuery()).execute().actionGet().getHits().getTotalHits().value, equalTo(1L)); - assertThat(client().prepareSearch().setSize(0).setRouting(routingValue).setQuery(QueryBuilders.matchAllQuery()).execute().actionGet().getHits().getTotalHits().value, equalTo(1L)); + assertThat(client().prepareSearch() + .setRouting(routingValue) + .setQuery(QueryBuilders.matchAllQuery()) + .execute() + .actionGet() + .getHits() + .getTotalHits() + .value, equalTo(1L)); + assertThat(client().prepareSearch() + .setSize(0) + .setRouting(routingValue) + .setQuery(QueryBuilders.matchAllQuery()) + .execute() + .actionGet() + .getHits() + .getTotalHits() + .value, equalTo(1L)); } logger.info("--> search with {} routing, should find one", secondRoutingValue); for (int i = 0; i < 5; i++) { - assertThat(client().prepareSearch().setRouting("1").setQuery(QueryBuilders.matchAllQuery()).execute().actionGet().getHits().getTotalHits().value, equalTo(1L)); - assertThat(client().prepareSearch().setSize(0).setRouting(secondRoutingValue).setQuery(QueryBuilders.matchAllQuery()).execute().actionGet().getHits().getTotalHits().value, equalTo(1L)); + assertThat(client().prepareSearch() + .setRouting("1") + .setQuery(QueryBuilders.matchAllQuery()) + .execute() + .actionGet() + .getHits() + .getTotalHits() + .value, equalTo(1L)); + assertThat(client().prepareSearch() + .setSize(0) + .setRouting(secondRoutingValue) + .setQuery(QueryBuilders.matchAllQuery()) + .execute() + .actionGet() + .getHits() + .getTotalHits() + .value, equalTo(1L)); } logger.info("--> search with {},{} indexRoutings , should find two", routingValue, "1"); for (int i = 0; i < 5; i++) { - assertThat(client().prepareSearch().setRouting(routingValue, secondRoutingValue).setQuery(QueryBuilders.matchAllQuery()).execute().actionGet().getHits().getTotalHits().value, equalTo(2L)); - assertThat(client().prepareSearch().setSize(0).setRouting(routingValue, secondRoutingValue).setQuery(QueryBuilders.matchAllQuery()).execute().actionGet().getHits().getTotalHits().value, equalTo(2L)); + assertThat(client().prepareSearch() + .setRouting(routingValue, secondRoutingValue) + .setQuery(QueryBuilders.matchAllQuery()) + .execute() + .actionGet() + .getHits() + .getTotalHits() + .value, equalTo(2L)); + assertThat(client().prepareSearch() + .setSize(0) + .setRouting(routingValue, secondRoutingValue) + .setQuery(QueryBuilders.matchAllQuery()) + .execute() + .actionGet() + .getHits() + .getTotalHits() + .value, equalTo(2L)); } logger.info("--> search with {},{},{} indexRoutings , should find two", routingValue, secondRoutingValue, routingValue); for (int i = 0; i < 5; i++) { - assertThat(client().prepareSearch().setRouting(routingValue, secondRoutingValue, routingValue).setQuery(QueryBuilders.matchAllQuery()).execute().actionGet().getHits().getTotalHits().value, equalTo(2L)); - assertThat(client().prepareSearch().setSize(0).setRouting(routingValue, secondRoutingValue,routingValue).setQuery(QueryBuilders.matchAllQuery()).execute().actionGet().getHits().getTotalHits().value, equalTo(2L)); + assertThat(client().prepareSearch() + .setRouting(routingValue, secondRoutingValue, routingValue) + .setQuery(QueryBuilders.matchAllQuery()) + .execute() + .actionGet() + .getHits() + .getTotalHits() + .value, equalTo(2L)); + assertThat(client().prepareSearch() + .setSize(0) + .setRouting(routingValue, secondRoutingValue, routingValue) + .setQuery(QueryBuilders.matchAllQuery()) + .execute() + .actionGet() + .getHits() + .getTotalHits() + .value, equalTo(2L)); } } public void testRequiredRoutingCrudApis() throws Exception { - client().admin().indices().prepareCreate("test").addAlias(new Alias("alias")) - .addMapping("type1", XContentFactory.jsonBuilder().startObject().startObject("type1").startObject("_routing").field("required", true).endObject().endObject().endObject()) - .execute().actionGet(); + client().admin() + .indices() + .prepareCreate("test") + .addAlias(new Alias("alias")) + .addMapping("type1", XContentFactory.jsonBuilder() + .startObject() + .startObject("type1") + .startObject("_routing") + .field("required", true) + .endObject() + .endObject() + .endObject()) + .execute() + .actionGet(); ensureGreen(); String routingValue = findNonMatchingRoutingValue("test", "1"); @@ -207,7 +360,12 @@ public void testRequiredRoutingCrudApis() throws Exception { logger.info("--> verifying get with routing, should find"); for (int i = 0; i < 5; i++) { - assertThat(client().prepareGet(indexOrAlias(), "type1", "1").setRouting(routingValue).execute().actionGet().isExists(), equalTo(true)); + assertThat(client() + .prepareGet(indexOrAlias(), "type1", "1") + .setRouting(routingValue) + .execute() + .actionGet() + .isExists(), equalTo(true)); } logger.info("--> deleting with no routing, should fail"); @@ -226,17 +384,28 @@ public void testRequiredRoutingCrudApis() throws Exception { assertThat(e.status(), equalTo(RestStatus.BAD_REQUEST)); assertThat(e.getMessage(), equalTo("routing is required for [test]/[type1]/[1]")); } - assertThat(client().prepareGet(indexOrAlias(), "type1", "1").setRouting(routingValue).execute().actionGet().isExists(), equalTo(true)); + assertThat(client() + .prepareGet(indexOrAlias(), "type1", "1") + .setRouting(routingValue) + .execute() + .actionGet() + .isExists(), equalTo(true)); } try { - client().prepareUpdate(indexOrAlias(), "type1", "1").setDoc(Requests.INDEX_CONTENT_TYPE, "field", "value2").execute().actionGet(); + client().prepareUpdate(indexOrAlias(), "type1", "1") + .setDoc(Requests.INDEX_CONTENT_TYPE, "field", "value2") + .execute() + .actionGet(); fail("update with missing routing when routing is required should fail"); - } catch(ElasticsearchException e) { + } catch (ElasticsearchException e) { assertThat(e.unwrapCause(), instanceOf(RoutingMissingException.class)); } - client().prepareUpdate(indexOrAlias(), "type1", "1").setRouting(routingValue).setDoc(Requests.INDEX_CONTENT_TYPE, "field", "value2").get(); + client().prepareUpdate(indexOrAlias(), "type1", "1") + .setRouting(routingValue) + .setDoc(Requests.INDEX_CONTENT_TYPE, "field", "value2") + .get(); client().admin().indices().prepareRefresh().execute().actionGet(); for (int i = 0; i < 5; i++) { @@ -262,21 +431,32 @@ public void testRequiredRoutingCrudApis() throws Exception { assertThat(e.status(), equalTo(RestStatus.BAD_REQUEST)); assertThat(e.getMessage(), equalTo("routing is required for [test]/[type1]/[1]")); } - assertThat(client().prepareGet(indexOrAlias(), "type1", "1").setRouting(routingValue).execute().actionGet().isExists(), equalTo(false)); + assertThat(client() + .prepareGet(indexOrAlias(), "type1", "1") + .setRouting(routingValue) + .execute() + .actionGet() + .isExists(), equalTo(false)); } } public void testRequiredRoutingBulk() throws Exception { client().admin().indices().prepareCreate("test") - .addAlias(new Alias("alias")) - .addMapping("type1", XContentFactory.jsonBuilder().startObject().startObject("type1") - .startObject("_routing").field("required", true).endObject() - .endObject().endObject()) - .execute().actionGet(); + .addAlias(new Alias("alias")) + .addMapping("type1", XContentFactory.jsonBuilder().startObject().startObject("type1") + .startObject("_routing").field("required", true).endObject() + .endObject().endObject()) + .execute().actionGet(); ensureGreen(); { - BulkResponse bulkResponse = client().prepareBulk().add(Requests.indexRequest(indexOrAlias()).type("type1").id("1") - .source(Requests.INDEX_CONTENT_TYPE, "field", "value")).execute().actionGet(); + BulkResponse bulkResponse = client() + .prepareBulk() + .add(Requests.indexRequest(indexOrAlias()) + .type("type1") + .id("1") + .source(Requests.INDEX_CONTENT_TYPE, "field", "value")) + .execute() + .actionGet(); assertThat(bulkResponse.getItems().length, equalTo(1)); assertThat(bulkResponse.hasFailures(), equalTo(true)); @@ -290,15 +470,22 @@ public void testRequiredRoutingBulk() throws Exception { } { - BulkResponse bulkResponse = client().prepareBulk().add(Requests.indexRequest(indexOrAlias()).type("type1").id("1").routing("0") - .source(Requests.INDEX_CONTENT_TYPE, "field", "value")).execute().actionGet(); + BulkResponse bulkResponse = client() + .prepareBulk() + .add(Requests.indexRequest(indexOrAlias()) + .type("type1") + .id("1") + .routing("0") + .source(Requests.INDEX_CONTENT_TYPE, "field", "value")) + .execute() + .actionGet(); assertThat(bulkResponse.hasFailures(), equalTo(false)); } { BulkResponse bulkResponse = client().prepareBulk().add(new UpdateRequest(indexOrAlias(), "type1", "1") .doc(Requests.INDEX_CONTENT_TYPE, "field", "value2")) - .execute().actionGet(); + .execute().actionGet(); assertThat(bulkResponse.getItems().length, equalTo(1)); assertThat(bulkResponse.hasFailures(), equalTo(true)); @@ -320,7 +507,7 @@ public void testRequiredRoutingBulk() throws Exception { { BulkResponse bulkResponse = client().prepareBulk().add(Requests.deleteRequest(indexOrAlias()).type("type1").id("1")) - .execute().actionGet(); + .execute().actionGet(); assertThat(bulkResponse.getItems().length, equalTo(1)); assertThat(bulkResponse.hasFailures(), equalTo(true)); @@ -335,7 +522,7 @@ public void testRequiredRoutingBulk() throws Exception { { BulkResponse bulkResponse = client().prepareBulk().add(Requests.deleteRequest(indexOrAlias()).type("type1").id("1") - .routing("0")).execute().actionGet(); + .routing("0")).execute().actionGet(); assertThat(bulkResponse.getItems().length, equalTo(1)); assertThat(bulkResponse.hasFailures(), equalTo(false)); } @@ -343,10 +530,20 @@ public void testRequiredRoutingBulk() throws Exception { public void testRequiredRoutingMappingVariousAPIs() throws Exception { - client().admin().indices().prepareCreate("test").addAlias(new Alias("alias")) - .addMapping("type1", XContentFactory.jsonBuilder().startObject().startObject("type1") - .startObject("_routing").field("required", true).endObject().endObject().endObject()) - .execute().actionGet(); + client().admin() + .indices() + .prepareCreate("test") + .addAlias(new Alias("alias")) + .addMapping("type1", XContentFactory.jsonBuilder() + .startObject() + .startObject("type1") + .startObject("_routing") + .field("required", true) + .endObject() + .endObject() + .endObject()) + .execute() + .actionGet(); ensureGreen(); String routingValue = findNonMatchingRoutingValue("test", "1"); logger.info("--> indexing with id [1], and routing [{}]", routingValue); @@ -356,7 +553,11 @@ public void testRequiredRoutingMappingVariousAPIs() throws Exception { .setRefreshPolicy(RefreshPolicy.IMMEDIATE).get(); logger.info("--> verifying get with id [1] with routing [0], should succeed"); - assertThat(client().prepareGet(indexOrAlias(), "type1", "1").setRouting(routingValue).execute().actionGet().isExists(), equalTo(true)); + assertThat(client().prepareGet(indexOrAlias(), "type1", "1") + .setRouting(routingValue) + .execute() + .actionGet() + .isExists(), equalTo(true)); logger.info("--> verifying get with id [1], with no routing, should fail"); try { @@ -368,8 +569,8 @@ public void testRequiredRoutingMappingVariousAPIs() throws Exception { logger.info("--> verifying explain with id [2], with routing [0], should succeed"); ExplainResponse explainResponse = client().prepareExplain(indexOrAlias(), "type1", "2") - .setQuery(QueryBuilders.matchAllQuery()) - .setRouting(routingValue).get(); + .setQuery(QueryBuilders.matchAllQuery()) + .setRouting(routingValue).get(); assertThat(explainResponse.isExists(), equalTo(true)); assertThat(explainResponse.isMatch(), equalTo(true)); @@ -383,7 +584,9 @@ public void testRequiredRoutingMappingVariousAPIs() throws Exception { } logger.info("--> verifying term vector with id [1], with routing [0], should succeed"); - TermVectorsResponse termVectorsResponse = client().prepareTermVectors(indexOrAlias(), "type1", "1").setRouting(routingValue).get(); + TermVectorsResponse termVectorsResponse = client().prepareTermVectors(indexOrAlias(), "type1", "1") + .setRouting(routingValue) + .get(); assertThat(termVectorsResponse.isExists(), equalTo(true)); assertThat(termVectorsResponse.getId(), equalTo("1")); @@ -395,7 +598,7 @@ public void testRequiredRoutingMappingVariousAPIs() throws Exception { } UpdateResponse updateResponse = client().prepareUpdate(indexOrAlias(), "type1", "1").setRouting(routingValue) - .setDoc(Requests.INDEX_CONTENT_TYPE, "field1", "value1").get(); + .setDoc(Requests.INDEX_CONTENT_TYPE, "field1", "value1").get(); assertThat(updateResponse.getId(), equalTo("1")); assertThat(updateResponse.getVersion(), equalTo(2L)); @@ -408,8 +611,8 @@ public void testRequiredRoutingMappingVariousAPIs() throws Exception { logger.info("--> verifying mget with ids [1,2], with routing [0], should succeed"); MultiGetResponse multiGetResponse = client().prepareMultiGet() - .add(new MultiGetRequest.Item(indexOrAlias(), "type1", "1").routing("0")) - .add(new MultiGetRequest.Item(indexOrAlias(), "type1", "2").routing("0")).get(); + .add(new MultiGetRequest.Item(indexOrAlias(), "type1", "1").routing("0")) + .add(new MultiGetRequest.Item(indexOrAlias(), "type1", "2").routing("0")).get(); assertThat(multiGetResponse.getResponses().length, equalTo(2)); assertThat(multiGetResponse.getResponses()[0].isFailed(), equalTo(false)); assertThat(multiGetResponse.getResponses()[0].getResponse().getId(), equalTo("1")); @@ -418,8 +621,8 @@ public void testRequiredRoutingMappingVariousAPIs() throws Exception { logger.info("--> verifying mget with ids [1,2], with no routing, should fail"); multiGetResponse = client().prepareMultiGet() - .add(new MultiGetRequest.Item(indexOrAlias(), "type1", "1")) - .add(new MultiGetRequest.Item(indexOrAlias(), "type1", "2")).get(); + .add(new MultiGetRequest.Item(indexOrAlias(), "type1", "1")) + .add(new MultiGetRequest.Item(indexOrAlias(), "type1", "2")).get(); assertThat(multiGetResponse.getResponses().length, equalTo(2)); assertThat(multiGetResponse.getResponses()[0].isFailed(), equalTo(true)); assertThat(multiGetResponse.getResponses()[0].getFailure().getId(), equalTo("1")); @@ -429,8 +632,11 @@ public void testRequiredRoutingMappingVariousAPIs() throws Exception { assertThat(multiGetResponse.getResponses()[1].getFailure().getMessage(), equalTo("routing is required for [test]/[type1]/[2]")); MultiTermVectorsResponse multiTermVectorsResponse = client().prepareMultiTermVectors() - .add(new TermVectorsRequest(indexOrAlias(), "type1", "1").routing(routingValue)) - .add(new TermVectorsRequest(indexOrAlias(), "type1", "2").routing(routingValue)).get(); + .add(new TermVectorsRequest(indexOrAlias(), "type1", "1") + .routing(routingValue)) + .add(new TermVectorsRequest(indexOrAlias(), "type1", "2") + .routing(routingValue)) + .get(); assertThat(multiTermVectorsResponse.getResponses().length, equalTo(2)); assertThat(multiTermVectorsResponse.getResponses()[0].getId(), equalTo("1")); assertThat(multiTermVectorsResponse.getResponses()[0].isFailed(), equalTo(false)); @@ -442,17 +648,21 @@ public void testRequiredRoutingMappingVariousAPIs() throws Exception { assertThat(multiTermVectorsResponse.getResponses()[1].getResponse().isExists(), equalTo(true)); multiTermVectorsResponse = client().prepareMultiTermVectors() - .add(new TermVectorsRequest(indexOrAlias(), "type1", "1")) - .add(new TermVectorsRequest(indexOrAlias(), "type1", "2")).get(); + .add(new TermVectorsRequest(indexOrAlias(), "type1", "1")) + .add(new TermVectorsRequest(indexOrAlias(), "type1", "2")).get(); assertThat(multiTermVectorsResponse.getResponses().length, equalTo(2)); assertThat(multiTermVectorsResponse.getResponses()[0].getId(), equalTo("1")); assertThat(multiTermVectorsResponse.getResponses()[0].isFailed(), equalTo(true)); - assertThat(multiTermVectorsResponse.getResponses()[0].getFailure().getCause().getMessage(), equalTo("routing is required for [test]/[type1]/[1]")); + assertThat(multiTermVectorsResponse.getResponses()[0].getFailure() + .getCause() + .getMessage(), equalTo("routing is required for [test]/[type1]/[1]")); assertThat(multiTermVectorsResponse.getResponses()[0].getResponse(), nullValue()); assertThat(multiTermVectorsResponse.getResponses()[1].getId(), equalTo("2")); assertThat(multiTermVectorsResponse.getResponses()[1].isFailed(), equalTo(true)); - assertThat(multiTermVectorsResponse.getResponses()[1].getResponse(),nullValue()); - assertThat(multiTermVectorsResponse.getResponses()[1].getFailure().getCause().getMessage(), equalTo("routing is required for [test]/[type1]/[2]")); + assertThat(multiTermVectorsResponse.getResponses()[1].getResponse(), nullValue()); + assertThat(multiTermVectorsResponse.getResponses()[1].getFailure() + .getCause() + .getMessage(), equalTo("routing is required for [test]/[type1]/[2]")); } private static String indexOrAlias() { From 6d81e7c3e7100ddbef1ee812093f4386395e8f2b Mon Sep 17 00:00:00 2001 From: Martijn van Groningen Date: Thu, 10 Jan 2019 15:54:34 +0100 Subject: [PATCH 077/186] [CCR] FollowingEngine should fail with 403 if operation has no seqno assigned (#37213) Fail with a 403 when indexing a document directly into a follower index. In order to test this change, I had to move specific assertions into a dedicated class and disable assertions for that class in the rest qa module. I think that is the right trade off. --- x-pack/plugin/ccr/qa/rest/build.gradle | 4 ++ .../index_directly_into_follower_index.yml | 66 +++++++++++++++++++ .../ccr/index/engine/FollowingEngine.java | 17 ++--- .../engine/FollowingEngineAssertions.java | 37 +++++++++++ 4 files changed, 113 insertions(+), 11 deletions(-) create mode 100644 x-pack/plugin/ccr/qa/rest/src/test/resources/rest-api-spec/test/ccr/index_directly_into_follower_index.yml create mode 100644 x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/index/engine/FollowingEngineAssertions.java diff --git a/x-pack/plugin/ccr/qa/rest/build.gradle b/x-pack/plugin/ccr/qa/rest/build.gradle index c2ca1d499b610..38873262ed273 100644 --- a/x-pack/plugin/ccr/qa/rest/build.gradle +++ b/x-pack/plugin/ccr/qa/rest/build.gradle @@ -13,6 +13,10 @@ task restTest(type: RestIntegTestTask) { restTestCluster { distribution 'zip' + // Disable assertions in FollowingEngineAssertions, otherwise an AssertionError is thrown before + // indexing a document directly in a follower index. In a rest test we like to test the exception + // that is thrown in production when indexing a document directly in a follower index. + environment 'ES_JAVA_OPTS', '-da:org.elasticsearch.xpack.ccr.index.engine.FollowingEngineAssertions' setting 'xpack.ml.enabled', 'false' setting 'xpack.monitoring.enabled', 'false' setting 'xpack.security.enabled', 'true' diff --git a/x-pack/plugin/ccr/qa/rest/src/test/resources/rest-api-spec/test/ccr/index_directly_into_follower_index.yml b/x-pack/plugin/ccr/qa/rest/src/test/resources/rest-api-spec/test/ccr/index_directly_into_follower_index.yml new file mode 100644 index 0000000000000..60c3b404b6f09 --- /dev/null +++ b/x-pack/plugin/ccr/qa/rest/src/test/resources/rest-api-spec/test/ccr/index_directly_into_follower_index.yml @@ -0,0 +1,66 @@ +--- +"Test indexing direcly into a follower index": + - do: + cluster.state: {} + + - set: {master_node: master} + + - do: + nodes.info: {} + + - set: {nodes.$master.transport_address: local_ip} + + - do: + cluster.put_settings: + body: + transient: + cluster.remote.local.seeds: $local_ip + flat_settings: true + + - match: {transient: {cluster.remote.local.seeds: $local_ip}} + + - do: + indices.create: + index: foo + body: + settings: + index: + soft_deletes: + enabled: true + mappings: + doc: + properties: + field: + type: keyword + - is_true: acknowledged + + - do: + ccr.follow: + index: bar + body: + remote_cluster: local + leader_index: foo + - is_true: follow_index_created + - is_true: follow_index_shards_acked + - is_true: index_following_started + + - do: + catch: forbidden + index: + index: bar + body: {} + + - do: + ccr.pause_follow: + index: bar + - is_true: acknowledged + + - do: + indices.close: + index: bar + - is_true: acknowledged + + - do: + ccr.unfollow: + index: bar + - is_true: acknowledged diff --git a/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/index/engine/FollowingEngine.java b/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/index/engine/FollowingEngine.java index c5728516426d8..b7086ed876db7 100644 --- a/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/index/engine/FollowingEngine.java +++ b/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/index/engine/FollowingEngine.java @@ -16,6 +16,7 @@ import org.apache.lucene.search.IndexSearcher; import org.apache.lucene.search.Query; import org.apache.lucene.search.TopDocs; +import org.elasticsearch.ElasticsearchStatusException; import org.elasticsearch.common.lucene.Lucene; import org.elasticsearch.common.metrics.CounterMetric; import org.elasticsearch.index.VersionType; @@ -23,6 +24,7 @@ import org.elasticsearch.index.engine.InternalEngine; import org.elasticsearch.index.mapper.SeqNoFieldMapper; import org.elasticsearch.index.seqno.SequenceNumbers; +import org.elasticsearch.rest.RestStatus; import org.elasticsearch.xpack.ccr.CcrSettings; import java.io.IOException; @@ -56,17 +58,11 @@ private static EngineConfig validateEngineConfig(final EngineConfig engineConfig } private void preFlight(final Operation operation) { - /* - * We assert here so that this goes uncaught in unit tests and fails nodes in standalone tests (we want a harsh failure so that we - * do not have a situation where a shard fails and is recovered elsewhere and a test subsequently passes). We throw an exception so - * that we also prevent issues in production code. - */ - assert operation.seqNo() != SequenceNumbers.UNASSIGNED_SEQ_NO; + assert FollowingEngineAssertions.preFlight(operation); if (operation.seqNo() == SequenceNumbers.UNASSIGNED_SEQ_NO) { - throw new IllegalStateException("a following engine does not accept operations without an assigned sequence number"); + throw new ElasticsearchStatusException("a following engine does not accept operations without an assigned sequence number", + RestStatus.FORBIDDEN); } - assert (operation.origin() == Operation.Origin.PRIMARY) == (operation.versionType() == VersionType.EXTERNAL) : - "invalid version_type in a following engine; version_type=" + operation.versionType() + "origin=" + operation.origin(); } @Override @@ -133,8 +129,7 @@ public int fillSeqNoGaps(long primaryTerm) throws IOException { @Override protected boolean assertPrimaryIncomingSequenceNumber(final Operation.Origin origin, final long seqNo) { - // sequence number should be set when operation origin is primary - assert seqNo != SequenceNumbers.UNASSIGNED_SEQ_NO : "primary operations on a following index must have an assigned sequence number"; + assert FollowingEngineAssertions.assertPrimaryIncomingSequenceNumber(origin, seqNo); return true; } diff --git a/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/index/engine/FollowingEngineAssertions.java b/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/index/engine/FollowingEngineAssertions.java new file mode 100644 index 0000000000000..31b13270ba191 --- /dev/null +++ b/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/index/engine/FollowingEngineAssertions.java @@ -0,0 +1,37 @@ +/* + * 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.ccr.index.engine; + +import org.elasticsearch.index.VersionType; +import org.elasticsearch.index.engine.Engine; +import org.elasticsearch.index.seqno.SequenceNumbers; + +/** + * Moved preflight and assertPrimaryIncomingSequenceNumber check to its own class, + * so that when testing writing directly into a follower index, + * only these assertions here need to be disabled instead of all assertions in FollowingEngine class. + */ +final class FollowingEngineAssertions { + + static boolean preFlight(final Engine.Operation operation) { + /* + * We assert here so that this goes uncaught in unit tests and fails nodes in standalone tests (we want a harsh failure so that we + * do not have a situation where a shard fails and is recovered elsewhere and a test subsequently passes). We throw an exception so + * that we also prevent issues in production code. + */ + assert operation.seqNo() != SequenceNumbers.UNASSIGNED_SEQ_NO; + assert (operation.origin() == Engine.Operation.Origin.PRIMARY) == (operation.versionType() == VersionType.EXTERNAL) : + "invalid version_type in a following engine; version_type=" + operation.versionType() + "origin=" + operation.origin(); + return true; + } + + static boolean assertPrimaryIncomingSequenceNumber(final Engine.Operation.Origin origin, final long seqNo) { + // sequence number should be set when operation origin is primary + assert seqNo != SequenceNumbers.UNASSIGNED_SEQ_NO : "primary operations on a following index must have an assigned sequence number"; + return true; + } + +} From 46237faa97978327be138fceeadfb66329c799ff Mon Sep 17 00:00:00 2001 From: Armin Braun Date: Thu, 10 Jan 2019 16:28:38 +0100 Subject: [PATCH 078/186] Fail Stale Primary Alloc. Req. without Data (#37226) * Get indices shard store status before enqueuing the reallocation state update task to prevent tasks that would fail because a node does not hold a stale copy of the shard on a best effort basis * Closes #37098 --- .../TransportClusterRerouteAction.java | 94 ++++++++++++++++--- .../shards/IndicesShardStoresResponse.java | 6 +- .../cluster/routing/PrimaryAllocationIT.java | 53 +++++++++-- 3 files changed, 134 insertions(+), 19 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/action/admin/cluster/reroute/TransportClusterRerouteAction.java b/server/src/main/java/org/elasticsearch/action/admin/cluster/reroute/TransportClusterRerouteAction.java index 061ec41039b43..d5cc35b2205ac 100644 --- a/server/src/main/java/org/elasticsearch/action/admin/cluster/reroute/TransportClusterRerouteAction.java +++ b/server/src/main/java/org/elasticsearch/action/admin/cluster/reroute/TransportClusterRerouteAction.java @@ -21,7 +21,12 @@ import org.apache.logging.log4j.Logger; import org.apache.logging.log4j.message.ParameterizedMessage; +import org.elasticsearch.ExceptionsHelper; import org.elasticsearch.action.ActionListener; +import org.elasticsearch.action.ActionListenerResponseHandler; +import org.elasticsearch.action.admin.indices.shards.IndicesShardStoresAction; +import org.elasticsearch.action.admin.indices.shards.IndicesShardStoresRequest; +import org.elasticsearch.action.admin.indices.shards.IndicesShardStoresResponse; import org.elasticsearch.action.support.ActionFilters; import org.elasticsearch.action.support.master.TransportMasterNodeAction; import org.elasticsearch.cluster.AckedClusterStateUpdateTask; @@ -29,14 +34,26 @@ import org.elasticsearch.cluster.block.ClusterBlockException; import org.elasticsearch.cluster.block.ClusterBlockLevel; import org.elasticsearch.cluster.metadata.IndexNameExpressionResolver; +import org.elasticsearch.cluster.node.DiscoveryNode; import org.elasticsearch.cluster.routing.allocation.AllocationService; import org.elasticsearch.cluster.routing.allocation.RoutingExplanations; +import org.elasticsearch.cluster.routing.allocation.command.AbstractAllocateAllocationCommand; +import org.elasticsearch.cluster.routing.allocation.command.AllocateStalePrimaryAllocationCommand; +import org.elasticsearch.cluster.routing.allocation.command.AllocationCommand; import org.elasticsearch.cluster.service.ClusterService; import org.elasticsearch.common.Priority; +import org.elasticsearch.common.Strings; +import org.elasticsearch.common.collect.ImmutableOpenIntMap; +import org.elasticsearch.common.collect.ImmutableOpenMap; import org.elasticsearch.common.inject.Inject; import org.elasticsearch.threadpool.ThreadPool; import org.elasticsearch.transport.TransportService; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + public class TransportClusterRerouteAction extends TransportMasterNodeAction { private final AllocationService allocationService; @@ -69,18 +86,71 @@ protected ClusterRerouteResponse newResponse() { @Override protected void masterOperation(final ClusterRerouteRequest request, final ClusterState state, final ActionListener listener) { - ActionListener logWrapper = ActionListener.wrap( - response -> { - if (request.dryRun() == false) { - response.getExplanations().getYesDecisionMessages().forEach(logger::info); - } - listener.onResponse(response); - }, - listener::onFailure - ); - - clusterService.submitStateUpdateTask("cluster_reroute (api)", new ClusterRerouteResponseAckedClusterStateUpdateTask(logger, - allocationService, request, logWrapper)); + Map> stalePrimaryAllocations = new HashMap<>(); + for (AllocationCommand command : request.getCommands().commands()) { + if (command instanceof AllocateStalePrimaryAllocationCommand) { + final AllocateStalePrimaryAllocationCommand cmd = (AllocateStalePrimaryAllocationCommand) command; + stalePrimaryAllocations.computeIfAbsent(cmd.index(), k -> new ArrayList<>()).add(cmd); + } + } + if (stalePrimaryAllocations.isEmpty()) { + submitStateUpdate(request, listener); + } else { + verifyThenSubmitUpdate(request, listener, stalePrimaryAllocations); + } + } + + private void verifyThenSubmitUpdate(ClusterRerouteRequest request, ActionListener listener, + Map> stalePrimaryAllocations) { + transportService.sendRequest(transportService.getLocalNode(), IndicesShardStoresAction.NAME, + new IndicesShardStoresRequest().indices(stalePrimaryAllocations.keySet().toArray(Strings.EMPTY_ARRAY)), + new ActionListenerResponseHandler<>( + ActionListener.wrap( + response -> { + ImmutableOpenMap>> status = + response.getStoreStatuses(); + Exception e = null; + for (Map.Entry> entry : stalePrimaryAllocations.entrySet()) { + final String index = entry.getKey(); + final ImmutableOpenIntMap> indexStatus = status.get(index); + assert indexStatus != null; + for (AbstractAllocateAllocationCommand command : entry.getValue()) { + final List shardStatus = + indexStatus.get(command.shardId()); + if (shardStatus == null || shardStatus.isEmpty()) { + e = ExceptionsHelper.useOrSuppress(e, new IllegalArgumentException( + "No data for shard [" + command.shardId() + "] of index [" + index + "] found on any node") + ); + } else if (shardStatus.stream().noneMatch(storeStatus -> { + final DiscoveryNode node = storeStatus.getNode(); + final String nodeInCommand = command.node(); + return nodeInCommand.equals(node.getName()) || nodeInCommand.equals(node.getId()); + })) { + e = ExceptionsHelper.useOrSuppress(e, new IllegalArgumentException( + "No data for shard [" + command.shardId() + "] of index [" + index + "] found on node [" + + command.node() + ']')); + } + } + } + if (e == null) { + submitStateUpdate(request, listener); + } else { + listener.onFailure(e); + } + }, listener::onFailure + ), IndicesShardStoresResponse::new)); + } + + private void submitStateUpdate(final ClusterRerouteRequest request, final ActionListener listener) { + clusterService.submitStateUpdateTask("cluster_reroute (api)", + new ClusterRerouteResponseAckedClusterStateUpdateTask(logger, allocationService, request, + ActionListener.wrap( + response -> { + if (request.dryRun() == false) { + response.getExplanations().getYesDecisionMessages().forEach(logger::info); + } + listener.onResponse(response); + }, listener::onFailure))); } static class ClusterRerouteResponseAckedClusterStateUpdateTask extends AckedClusterStateUpdateTask { diff --git a/server/src/main/java/org/elasticsearch/action/admin/indices/shards/IndicesShardStoresResponse.java b/server/src/main/java/org/elasticsearch/action/admin/indices/shards/IndicesShardStoresResponse.java index d87de21bc48d8..ed348539d3562 100644 --- a/server/src/main/java/org/elasticsearch/action/admin/indices/shards/IndicesShardStoresResponse.java +++ b/server/src/main/java/org/elasticsearch/action/admin/indices/shards/IndicesShardStoresResponse.java @@ -283,7 +283,11 @@ public IndicesShardStoresResponse(ImmutableOpenMap>>of(), Collections.emptyList()); + this(ImmutableOpenMap.of(), Collections.emptyList()); + } + + public IndicesShardStoresResponse(StreamInput in) throws IOException { + readFrom(in); } /** diff --git a/server/src/test/java/org/elasticsearch/cluster/routing/PrimaryAllocationIT.java b/server/src/test/java/org/elasticsearch/cluster/routing/PrimaryAllocationIT.java index e2777616f426d..213d8175fcd18 100644 --- a/server/src/test/java/org/elasticsearch/cluster/routing/PrimaryAllocationIT.java +++ b/server/src/test/java/org/elasticsearch/cluster/routing/PrimaryAllocationIT.java @@ -34,6 +34,7 @@ import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.util.set.Sets; import org.elasticsearch.gateway.GatewayAllocator; +import org.elasticsearch.index.IndexNotFoundException; import org.elasticsearch.index.engine.Engine; import org.elasticsearch.index.engine.EngineTestCase; import org.elasticsearch.index.shard.IndexShard; @@ -50,6 +51,7 @@ import org.elasticsearch.test.junit.annotations.TestLogging; import org.elasticsearch.test.transport.MockTransportService; +import java.util.ArrayList; import java.util.Arrays; import java.util.Collection; import java.util.Collections; @@ -175,15 +177,17 @@ public void testFailedAllocationOfStalePrimaryToDataNodeWithNoData() throws Exce .getShards().get(0).primaryShard().unassignedInfo().getReason(), equalTo(UnassignedInfo.Reason.NODE_LEFT)); logger.info("--> force allocation of stale copy to node that does not have shard copy"); - client().admin().cluster().prepareReroute().add(new AllocateStalePrimaryAllocationCommand("test", 0, - dataNodeWithNoShardCopy, true)).get(); + Throwable iae = expectThrows( + IllegalArgumentException.class, + () -> client().admin().cluster().prepareReroute().add(new AllocateStalePrimaryAllocationCommand("test", 0, + dataNodeWithNoShardCopy, true)).get()); + assertThat(iae.getMessage(), equalTo("No data for shard [0] of index [test] found on any node")); logger.info("--> wait until shard is failed and becomes unassigned again"); - assertBusy(() -> - assertTrue(client().admin().cluster().prepareState().get().getState().toString(), - client().admin().cluster().prepareState().get().getState().getRoutingTable().index("test").allPrimaryShardsUnassigned())); + assertTrue(client().admin().cluster().prepareState().get().getState().toString(), + client().admin().cluster().prepareState().get().getState().getRoutingTable().index("test").allPrimaryShardsUnassigned()); assertThat(client().admin().cluster().prepareState().get().getState().getRoutingTable().index("test") - .getShards().get(0).primaryShard().unassignedInfo().getReason(), equalTo(UnassignedInfo.Reason.ALLOCATION_FAILED)); + .getShards().get(0).primaryShard().unassignedInfo().getReason(), equalTo(UnassignedInfo.Reason.NODE_LEFT)); } public void testForceStaleReplicaToBePromotedToPrimary() throws Exception { @@ -261,6 +265,43 @@ public void testForceStaleReplicaToBePromotedToPrimary() throws Exception { assertThat(newHistoryUUIds, hasSize(1)); } + public void testForceStaleReplicaToBePromotedToPrimaryOnWrongNode() throws Exception { + String master = internalCluster().startMasterOnlyNode(Settings.EMPTY); + internalCluster().startDataOnlyNodes(2); + final String idxName = "test"; + assertAcked(client().admin().indices().prepareCreate(idxName) + .setSettings(Settings.builder().put("index.number_of_shards", 1) + .put("index.number_of_replicas", 1)).get()); + ensureGreen(); + createStaleReplicaScenario(master); + internalCluster().startDataOnlyNodes(2); + final int shardId = 0; + final List nodeNames = new ArrayList<>(Arrays.asList(internalCluster().getNodeNames())); + nodeNames.remove(master); + client().admin().indices().prepareShardStores(idxName).get().getStoreStatuses().get(idxName) + .get(shardId).forEach(status -> nodeNames.remove(status.getNode().getName())); + assertThat(nodeNames, hasSize(1)); + final String nodeWithoutData = nodeNames.get(0); + Throwable iae = expectThrows( + IllegalArgumentException.class, + () -> client().admin().cluster().prepareReroute() + .add(new AllocateStalePrimaryAllocationCommand(idxName, shardId, nodeWithoutData, true)).get()); + assertThat( + iae.getMessage(), + equalTo("No data for shard [" + shardId + "] of index [" + idxName + "] found on node [" + nodeWithoutData + ']')); + } + + public void testForceStaleReplicaToBePromotedForMissingIndex() { + internalCluster().startMasterOnlyNode(Settings.EMPTY); + final String dataNode = internalCluster().startDataOnlyNode(); + final String idxName = "test"; + IndexNotFoundException ex = expectThrows( + IndexNotFoundException.class, + () -> client().admin().cluster().prepareReroute() + .add(new AllocateStalePrimaryAllocationCommand(idxName, 0, dataNode, true)).get()); + assertThat(ex.getIndex().getName(), equalTo(idxName)); + } + public void testForcePrimaryShardIfAllocationDecidersSayNoAfterIndexCreation() throws ExecutionException, InterruptedException { String node = internalCluster().startNode(); client().admin().indices().prepareCreate("test").setWaitForActiveShards(ActiveShardCount.NONE).setSettings(Settings.builder() From fcf7df3edaad3b346ddfd5b536d8a410996663cf Mon Sep 17 00:00:00 2001 From: Ryan Ernst Date: Thu, 10 Jan 2019 07:44:40 -0800 Subject: [PATCH 079/186] Core: Handle security manager permission for deprecation log rolling (#37281) When the deprecation log is written to within scripting support code like ScriptDocValues, it runs under the reduces privileges of scripts. Sometimes this can trigger log rolling, which then causes uncaught security errors, as was handled in #28485. While doing individual deprecation handling within each deprecation scripting location is possible, there are a growing number of deprecations in scripts. This commit wraps the logging call within the deprecation logger use a doPrivileged block, just was we would within individual logging call sites for scripting utilities. --- .../common/logging/EvilLoggerTests.java | 10 +-- .../common/logging/DeprecationLogger.java | 13 +++- .../logging/DeprecationLoggerTests.java | 63 ++++++++++++++++++- 3 files changed, 77 insertions(+), 9 deletions(-) diff --git a/qa/evil-tests/src/test/java/org/elasticsearch/common/logging/EvilLoggerTests.java b/qa/evil-tests/src/test/java/org/elasticsearch/common/logging/EvilLoggerTests.java index 8438c002c2a4e..e32447c47b092 100644 --- a/qa/evil-tests/src/test/java/org/elasticsearch/common/logging/EvilLoggerTests.java +++ b/qa/evil-tests/src/test/java/org/elasticsearch/common/logging/EvilLoggerTests.java @@ -126,7 +126,7 @@ public void testDeprecationLogger() throws IOException, UserException { assertLogLine( deprecationEvents.get(i), Level.WARN, - "org.elasticsearch.common.logging.DeprecationLogger.deprecated", + "org.elasticsearch.common.logging.DeprecationLogger\\$2\\.run", "This is a deprecation message"); } } @@ -200,7 +200,7 @@ public void testConcurrentDeprecationLogger() throws IOException, UserException, assertLogLine( deprecationEvents.get(i), Level.WARN, - "org.elasticsearch.common.logging.DeprecationLogger.deprecated", + "org.elasticsearch.common.logging.DeprecationLogger\\$2\\.run", "This is a maybe logged deprecation message" + i); } @@ -242,13 +242,13 @@ public void testDeprecationLoggerMaybeLog() throws IOException, UserException { assertLogLine( deprecationEvents.get(0), Level.WARN, - "org.elasticsearch.common.logging.DeprecationLogger.deprecated", + "org.elasticsearch.common.logging.DeprecationLogger\\$2\\.run", "This is a maybe logged deprecation message"); for (int k = 0; k < 128; k++) { assertLogLine( deprecationEvents.get(1 + k), Level.WARN, - "org.elasticsearch.common.logging.DeprecationLogger.deprecated", + "org.elasticsearch.common.logging.DeprecationLogger\\$2\\.run", "This is a maybe logged deprecation message" + k); } } @@ -276,7 +276,7 @@ public void testDeprecatedSettings() throws IOException, UserException { assertLogLine( deprecationEvents.get(0), Level.WARN, - "org.elasticsearch.common.logging.DeprecationLogger.deprecated", + "org.elasticsearch.common.logging.DeprecationLogger\\$2\\.run", "\\[deprecated.foo\\] setting was deprecated in Elasticsearch and will be removed in a future release! " + "See the breaking changes documentation for the next major version."); } diff --git a/server/src/main/java/org/elasticsearch/common/logging/DeprecationLogger.java b/server/src/main/java/org/elasticsearch/common/logging/DeprecationLogger.java index d1ac53fff3b99..e8f06a43a5c13 100644 --- a/server/src/main/java/org/elasticsearch/common/logging/DeprecationLogger.java +++ b/server/src/main/java/org/elasticsearch/common/logging/DeprecationLogger.java @@ -27,6 +27,8 @@ import org.elasticsearch.common.util.concurrent.ThreadContext; import java.nio.charset.Charset; +import java.security.AccessController; +import java.security.PrivilegedAction; import java.time.ZoneId; import java.time.ZonedDateTime; import java.time.format.DateTimeFormatter; @@ -298,7 +300,7 @@ void deprecated(final Set threadContexts, final String message, f deprecated(threadContexts, message, true, params); } - @SuppressLoggerChecks(reason = "safely delegates to logger") + void deprecated(final Set threadContexts, final String message, final boolean log, final Object... params) { final Iterator iterator = threadContexts.iterator(); @@ -318,7 +320,14 @@ void deprecated(final Set threadContexts, final String message, f } if (log) { - logger.warn(message, params); + AccessController.doPrivileged(new PrivilegedAction() { + @SuppressLoggerChecks(reason = "safely delegates to logger") + @Override + public Void run() { + logger.warn(message, params); + return null; + } + }); } } diff --git a/server/src/test/java/org/elasticsearch/common/logging/DeprecationLoggerTests.java b/server/src/test/java/org/elasticsearch/common/logging/DeprecationLoggerTests.java index 537bb3db70aca..740430ac0993b 100644 --- a/server/src/test/java/org/elasticsearch/common/logging/DeprecationLoggerTests.java +++ b/server/src/test/java/org/elasticsearch/common/logging/DeprecationLoggerTests.java @@ -19,8 +19,13 @@ package org.elasticsearch.common.logging; import com.carrotsearch.randomizedtesting.generators.CodepointSetGenerator; - import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; +import org.apache.logging.log4j.simple.SimpleLoggerContext; +import org.apache.logging.log4j.simple.SimpleLoggerContextFactory; +import org.apache.logging.log4j.spi.ExtendedLogger; +import org.apache.logging.log4j.spi.LoggerContext; +import org.apache.logging.log4j.spi.LoggerContextFactory; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.util.concurrent.ThreadContext; import org.elasticsearch.test.ESTestCase; @@ -28,14 +33,21 @@ import org.hamcrest.core.IsSame; import java.io.IOException; +import java.net.URI; +import java.nio.charset.StandardCharsets; +import java.security.AccessControlContext; +import java.security.AccessController; +import java.security.Permissions; +import java.security.PrivilegedAction; +import java.security.ProtectionDomain; import java.util.Collections; import java.util.HashSet; import java.util.List; import java.util.Locale; import java.util.Map; import java.util.Set; +import java.util.concurrent.atomic.AtomicBoolean; import java.util.stream.IntStream; -import java.nio.charset.StandardCharsets; import static org.elasticsearch.common.logging.DeprecationLogger.WARNING_HEADER_PATTERN; import static org.elasticsearch.test.hamcrest.RegexMatcher.matches; @@ -43,6 +55,10 @@ import static org.hamcrest.Matchers.equalTo; import static org.hamcrest.Matchers.hasSize; import static org.hamcrest.Matchers.not; +import static org.hamcrest.core.Is.is; +import static org.mockito.Mockito.doAnswer; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; /** * Tests {@link DeprecationLogger} @@ -303,6 +319,49 @@ public void testWarningHeaderSizeSetting() throws IOException{ } } + public void testLogPermissions() { + AtomicBoolean supplierCalled = new AtomicBoolean(false); + + // mocking the logger used inside DeprecationLogger requires heavy hacking... + Logger parentLogger = mock(Logger.class); + when(parentLogger.getName()).thenReturn("logger"); + ExtendedLogger mockLogger = mock(ExtendedLogger.class); + doAnswer(invocationOnMock -> { + supplierCalled.set(true); + createTempDir(); // trigger file permission, like rolling logs would + return null; + }).when(mockLogger).warn("foo", new Object[] {"bar"}); + final LoggerContext context = new SimpleLoggerContext() { + @Override + public ExtendedLogger getLogger(String name) { + return mockLogger; + } + }; + + final LoggerContextFactory originalFactory = LogManager.getFactory(); + try { + LogManager.setFactory(new SimpleLoggerContextFactory() { + @Override + public LoggerContext getContext(String fqcn, ClassLoader loader, Object externalContext, boolean currentContext, + URI configLocation, String name) { + return context; + } + }); + DeprecationLogger deprecationLogger = new DeprecationLogger(parentLogger); + + AccessControlContext noPermissionsAcc = new AccessControlContext( + new ProtectionDomain[]{new ProtectionDomain(null, new Permissions())} + ); + AccessController.doPrivileged((PrivilegedAction) () -> { + deprecationLogger.deprecated("foo", "bar"); + return null; + }, noPermissionsAcc); + assertThat("supplier called", supplierCalled.get(), is(true)); + } finally { + LogManager.setFactory(originalFactory); + } + } + private String range(int lowerInclusive, int upperInclusive) { return IntStream .range(lowerInclusive, upperInclusive + 1) From 71633775fd1a3cdad497233848a83ccd42020602 Mon Sep 17 00:00:00 2001 From: Jay Modi Date: Thu, 10 Jan 2019 09:06:16 -0700 Subject: [PATCH 080/186] Security: reorder realms based on last success (#36878) This commit reorders the realm list for iteration based on the last successful authentication for the given principal. This is an optimization to prevent unnecessary iteration over realms if we can make a smart guess on which realm to try first. --- .../xpack/security/Security.java | 1 + .../realm/TransportClearRealmCacheAction.java | 19 ++- .../security/authc/AuthenticationService.java | 92 ++++++++++- .../authc/AuthenticationServiceTests.java | 148 ++++++++++++++++++ 4 files changed, 257 insertions(+), 3 deletions(-) diff --git a/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/Security.java b/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/Security.java index ad9f1d7aa948c..714da7cf11c35 100644 --- a/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/Security.java +++ b/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/Security.java @@ -463,6 +463,7 @@ Collection createComponents(Client client, ThreadPool threadPool, Cluste authcService.set(new AuthenticationService(settings, realms, auditTrailService, failureHandler, threadPool, anonymousUser, tokenService)); components.add(authcService.get()); + securityIndex.get().addIndexStateListener(authcService.get()::onSecurityIndexStateChange); final NativePrivilegeStore privilegeStore = new NativePrivilegeStore(settings, client, securityIndex.get()); components.add(privilegeStore); diff --git a/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/action/realm/TransportClearRealmCacheAction.java b/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/action/realm/TransportClearRealmCacheAction.java index 6db95e822101e..b4ee8b677c13b 100644 --- a/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/action/realm/TransportClearRealmCacheAction.java +++ b/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/action/realm/TransportClearRealmCacheAction.java @@ -17,6 +17,7 @@ import org.elasticsearch.xpack.core.security.action.realm.ClearRealmCacheRequest; import org.elasticsearch.xpack.core.security.action.realm.ClearRealmCacheResponse; import org.elasticsearch.xpack.core.security.authc.Realm; +import org.elasticsearch.xpack.security.authc.AuthenticationService; import org.elasticsearch.xpack.security.authc.Realms; import org.elasticsearch.xpack.security.authc.support.CachingRealm; @@ -26,14 +27,16 @@ public class TransportClearRealmCacheAction extends TransportNodesAction { private final Realms realms; + private final AuthenticationService authenticationService; @Inject public TransportClearRealmCacheAction(ThreadPool threadPool, ClusterService clusterService, TransportService transportService, - ActionFilters actionFilters, Realms realms) { + ActionFilters actionFilters, Realms realms, AuthenticationService authenticationService) { super(ClearRealmCacheAction.NAME, threadPool, clusterService, transportService, actionFilters, ClearRealmCacheRequest::new, ClearRealmCacheRequest.Node::new, ThreadPool.Names.MANAGEMENT, ClearRealmCacheResponse.Node.class); this.realms = realms; + this.authenticationService = authenticationService; } @Override @@ -68,9 +71,23 @@ protected ClearRealmCacheResponse.Node nodeOperation(ClearRealmCacheRequest.Node } clearCache(realm, nodeRequest.getUsernames()); } + clearAuthenticationServiceCache(nodeRequest.getUsernames()); return new ClearRealmCacheResponse.Node(clusterService.localNode()); } + private void clearAuthenticationServiceCache(String[] usernames) { + // this is heavy handed since we could also take realm into account but that would add + // complexity since we would need to iterate over the cache under a lock to remove all + // entries that referenced the specific realm + if (usernames != null && usernames.length != 0) { + for (String username : usernames) { + authenticationService.expire(username); + } + } else { + authenticationService.expireAll(); + } + } + private void clearCache(Realm realm, String[] usernames) { if (!(realm instanceof CachingRealm)) { return; diff --git a/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/authc/AuthenticationService.java b/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/authc/AuthenticationService.java index ef04f3d22f854..a6d5cb50a76d6 100644 --- a/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/authc/AuthenticationService.java +++ b/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/authc/AuthenticationService.java @@ -13,9 +13,13 @@ import org.elasticsearch.action.ActionListener; import org.elasticsearch.action.support.ContextPreservingActionListener; import org.elasticsearch.common.Nullable; +import org.elasticsearch.common.cache.Cache; +import org.elasticsearch.common.cache.CacheBuilder; import org.elasticsearch.common.collect.Tuple; import org.elasticsearch.common.settings.Setting; +import org.elasticsearch.common.settings.Setting.Property; import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.common.unit.TimeValue; import org.elasticsearch.common.util.concurrent.ThreadContext; import org.elasticsearch.node.Node; import org.elasticsearch.rest.RestRequest; @@ -38,14 +42,21 @@ import org.elasticsearch.xpack.security.audit.AuditTrailService; import org.elasticsearch.xpack.security.audit.AuditUtil; import org.elasticsearch.xpack.security.authc.support.RealmUserLookup; +import org.elasticsearch.xpack.security.support.SecurityIndexManager; +import java.util.ArrayList; +import java.util.Collections; import java.util.LinkedHashMap; import java.util.List; import java.util.Map; import java.util.Objects; +import java.util.concurrent.atomic.AtomicLong; import java.util.function.BiConsumer; import java.util.function.Consumer; +import static org.elasticsearch.xpack.security.support.SecurityIndexManager.isIndexDeleted; +import static org.elasticsearch.xpack.security.support.SecurityIndexManager.isMoveFromRedToNonRed; + /** * An authentication service that delegates the authentication process to its configured {@link Realm realms}. * This service also supports request level caching of authenticated users (i.e. once a user authenticated @@ -53,6 +64,12 @@ */ public class AuthenticationService { + static final Setting SUCCESS_AUTH_CACHE_ENABLED = + Setting.boolSetting("xpack.security.authc.success_cache.enabled", true, Property.NodeScope); + private static final Setting SUCCESS_AUTH_CACHE_MAX_SIZE = + Setting.intSetting("xpack.security.authc.success_cache.size", 10000, Property.NodeScope); + private static final Setting SUCCESS_AUTH_CACHE_EXPIRE_AFTER_ACCESS = + Setting.timeSetting("xpack.security.authc.success_cache.expire_after_access", TimeValue.timeValueHours(1L), Property.NodeScope); private static final Logger logger = LogManager.getLogger(AuthenticationService.class); private final Realms realms; @@ -62,6 +79,8 @@ public class AuthenticationService { private final String nodeName; private final AnonymousUser anonymousUser; private final TokenService tokenService; + private final Cache lastSuccessfulAuthCache; + private final AtomicLong numInvalidation = new AtomicLong(); private final boolean runAsEnabled; private final boolean isAnonymousUserEnabled; @@ -77,6 +96,14 @@ public AuthenticationService(Settings settings, Realms realms, AuditTrailService this.runAsEnabled = AuthenticationServiceField.RUN_AS_ENABLED.get(settings); this.isAnonymousUserEnabled = AnonymousUser.isAnonymousEnabled(settings); this.tokenService = tokenService; + if (SUCCESS_AUTH_CACHE_ENABLED.get(settings)) { + this.lastSuccessfulAuthCache = CacheBuilder.builder() + .setMaximumWeight(Integer.toUnsignedLong(SUCCESS_AUTH_CACHE_MAX_SIZE.get(settings))) + .setExpireAfterAccess(SUCCESS_AUTH_CACHE_EXPIRE_AFTER_ACCESS.get(settings)) + .build(); + } else { + this.lastSuccessfulAuthCache = null; + } } /** @@ -120,6 +147,28 @@ public void authenticate(String action, TransportMessage message, new Authenticator(action, message, null, listener).authenticateToken(token); } + public void expire(String principal) { + if (lastSuccessfulAuthCache != null) { + numInvalidation.incrementAndGet(); + lastSuccessfulAuthCache.invalidate(principal); + } + } + + public void expireAll() { + if (lastSuccessfulAuthCache != null) { + numInvalidation.incrementAndGet(); + lastSuccessfulAuthCache.invalidateAll(); + } + } + + public void onSecurityIndexStateChange(SecurityIndexManager.State previousState, SecurityIndexManager.State currentState) { + if (lastSuccessfulAuthCache != null) { + if (isMoveFromRedToNonRed(previousState, currentState) || isIndexDeleted(previousState, currentState)) { + expireAll(); + } + } + } + // pkg private method for testing Authenticator createAuthenticator(RestRequest request, ActionListener listener) { return new Authenticator(request, listener); @@ -130,6 +179,11 @@ Authenticator createAuthenticator(String action, TransportMessage message, User return new Authenticator(action, message, fallbackUser, listener); } + // pkg private method for testing + long getNumInvalidation() { + return numInvalidation.get(); + } + /** * This class is responsible for taking a request and executing the authentication. The authentication is executed in an asynchronous * fashion in order to avoid blocking calls on a network thread. This class also performs the auditing necessary around authentication @@ -263,7 +317,8 @@ private void consumeToken(AuthenticationToken token) { handleNullToken(); } else { authenticationToken = token; - final List realmsList = realms.asList(); + final List realmsList = getRealmList(authenticationToken.principal()); + final long startInvalidation = numInvalidation.get(); final Map> messages = new LinkedHashMap<>(); final BiConsumer> realmAuthenticatingConsumer = (realm, userListener) -> { if (realm.supports(authenticationToken)) { @@ -273,6 +328,9 @@ private void consumeToken(AuthenticationToken token) { // user was authenticated, populate the authenticated by information authenticatedBy = new RealmRef(realm.name(), realm.type(), nodeName); authenticationResult = result; + if (lastSuccessfulAuthCache != null && startInvalidation == numInvalidation.get()) { + lastSuccessfulAuthCache.put(authenticationToken.principal(), realm); + } userListener.onResponse(result.getUser()); } else { // the user was not authenticated, call this so we can audit the correct event @@ -313,6 +371,27 @@ private void consumeToken(AuthenticationToken token) { } } + private List getRealmList(String principal) { + final List defaultOrderedRealms = realms.asList(); + if (lastSuccessfulAuthCache != null) { + final Realm lastSuccess = lastSuccessfulAuthCache.get(principal); + if (lastSuccess != null) { + final int index = defaultOrderedRealms.indexOf(lastSuccess); + if (index > 0) { + final List smartOrder = new ArrayList<>(defaultOrderedRealms.size()); + smartOrder.add(lastSuccess); + for (int i = 1; i < defaultOrderedRealms.size(); i++) { + if (i != index) { + smartOrder.add(defaultOrderedRealms.get(i)); + } + } + return Collections.unmodifiableList(smartOrder); + } + } + } + return defaultOrderedRealms; + } + /** * Handles failed extraction of an authentication token. This can happen in a few different scenarios: * @@ -391,7 +470,8 @@ private void consumeUser(User user, Map> message * names of users that exist using a timing attack */ private void lookupRunAsUser(final User user, String runAsUsername, Consumer userConsumer) { - final RealmUserLookup lookup = new RealmUserLookup(realms.asList(), threadContext); + final RealmUserLookup lookup = new RealmUserLookup(getRealmList(runAsUsername), threadContext); + final long startInvalidationNum = numInvalidation.get(); lookup.lookup(runAsUsername, ActionListener.wrap(tuple -> { if (tuple == null) { // the user does not exist, but we still create a User object, which will later be rejected by authz @@ -400,6 +480,11 @@ private void lookupRunAsUser(final User user, String runAsUsername, Consumer realm); + } userConsumer.accept(new User(foundUser, user)); } }, exception -> listener.onFailure(request.exceptionProcessingRequest(exception, authenticationToken)))); @@ -602,5 +687,8 @@ public String toString() { public static void addSettings(List> settings) { settings.add(AuthenticationServiceField.RUN_AS_ENABLED); + settings.add(SUCCESS_AUTH_CACHE_ENABLED); + settings.add(SUCCESS_AUTH_CACHE_MAX_SIZE); + settings.add(SUCCESS_AUTH_CACHE_EXPIRE_AFTER_ACCESS); } } diff --git a/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/authc/AuthenticationServiceTests.java b/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/authc/AuthenticationServiceTests.java index e7354b9b32564..397c68c1b72ed 100644 --- a/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/authc/AuthenticationServiceTests.java +++ b/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/authc/AuthenticationServiceTests.java @@ -22,6 +22,7 @@ import org.elasticsearch.action.update.UpdateAction; import org.elasticsearch.action.update.UpdateRequestBuilder; import org.elasticsearch.client.Client; +import org.elasticsearch.cluster.health.ClusterHealthStatus; import org.elasticsearch.cluster.service.ClusterService; import org.elasticsearch.common.SuppressForbidden; import org.elasticsearch.common.collect.Tuple; @@ -103,6 +104,7 @@ import static org.mockito.Mockito.mock; import static org.mockito.Mockito.never; import static org.mockito.Mockito.reset; +import static org.mockito.Mockito.times; import static org.mockito.Mockito.verify; import static org.mockito.Mockito.verifyNoMoreInteractions; import static org.mockito.Mockito.verifyZeroInteractions; @@ -133,6 +135,7 @@ public class AuthenticationServiceTests extends ESTestCase { @SuppressForbidden(reason = "Allow accessing localhost") public void init() throws Exception { token = mock(AuthenticationToken.class); + when(token.principal()).thenReturn(randomAlphaOfLength(5)); message = new InternalMessage(); remoteAddress = new InetSocketAddress(InetAddress.getLocalHost(), 100); message.remoteAddress(new TransportAddress(remoteAddress)); @@ -258,6 +261,134 @@ public void testAuthenticateBothSupportSecondSucceeds() throws Exception { verify(auditTrail).authenticationFailed(reqId, firstRealm.name(), token, "_action", message); } + public void testAuthenticateSmartRealmOrdering() { + User user = new User("_username", "r1"); + when(firstRealm.supports(token)).thenReturn(true); + mockAuthenticate(firstRealm, token, null); + when(secondRealm.supports(token)).thenReturn(true); + mockAuthenticate(secondRealm, token, user); + when(secondRealm.token(threadContext)).thenReturn(token); + final String reqId = AuditUtil.getOrGenerateRequestId(threadContext); + + final AtomicBoolean completed = new AtomicBoolean(false); + service.authenticate("_action", message, (User)null, ActionListener.wrap(result -> { + assertThat(result, notNullValue()); + assertThat(result.getUser(), is(user)); + assertThat(result.getLookedUpBy(), is(nullValue())); + assertThat(result.getAuthenticatedBy(), is(notNullValue())); // TODO implement equals + assertThreadContextContainsAuthentication(result); + setCompletedToTrue(completed); + }, this::logAndFail)); + assertTrue(completed.get()); + + completed.set(false); + service.authenticate("_action", message, (User)null, ActionListener.wrap(result -> { + assertThat(result, notNullValue()); + assertThat(result.getUser(), is(user)); + assertThat(result.getLookedUpBy(), is(nullValue())); + assertThat(result.getAuthenticatedBy(), is(notNullValue())); // TODO implement equals + assertThreadContextContainsAuthentication(result); + setCompletedToTrue(completed); + }, this::logAndFail)); + verify(auditTrail).authenticationFailed(reqId, firstRealm.name(), token, "_action", message); + verify(auditTrail, times(2)).authenticationSuccess(reqId, secondRealm.name(), user, "_action", message); + verify(firstRealm, times(2)).name(); // used above one time + verify(secondRealm, times(3)).name(); // used above one time + verify(secondRealm, times(2)).type(); // used to create realm ref + verify(firstRealm, times(2)).token(threadContext); + verify(secondRealm, times(2)).token(threadContext); + verify(firstRealm).supports(token); + verify(secondRealm, times(2)).supports(token); + verify(firstRealm).authenticate(eq(token), any(ActionListener.class)); + verify(secondRealm, times(2)).authenticate(eq(token), any(ActionListener.class)); + verifyNoMoreInteractions(auditTrail, firstRealm, secondRealm); + } + + public void testCacheClearOnSecurityIndexChange() { + long expectedInvalidation = 0L; + assertEquals(expectedInvalidation, service.getNumInvalidation()); + + // existing to no longer present + SecurityIndexManager.State previousState = dummyState(randomFrom(ClusterHealthStatus.GREEN, ClusterHealthStatus.YELLOW)); + SecurityIndexManager.State currentState = dummyState(null); + service.onSecurityIndexStateChange(previousState, currentState); + assertEquals(++expectedInvalidation, service.getNumInvalidation()); + + // doesn't exist to exists + previousState = dummyState(null); + currentState = dummyState(randomFrom(ClusterHealthStatus.GREEN, ClusterHealthStatus.YELLOW)); + service.onSecurityIndexStateChange(previousState, currentState); + assertEquals(++expectedInvalidation, service.getNumInvalidation()); + + // green or yellow to red + previousState = dummyState(randomFrom(ClusterHealthStatus.GREEN, ClusterHealthStatus.YELLOW)); + currentState = dummyState(ClusterHealthStatus.RED); + service.onSecurityIndexStateChange(previousState, currentState); + assertEquals(expectedInvalidation, service.getNumInvalidation()); + + // red to non red + previousState = dummyState(ClusterHealthStatus.RED); + currentState = dummyState(randomFrom(ClusterHealthStatus.GREEN, ClusterHealthStatus.YELLOW)); + service.onSecurityIndexStateChange(previousState, currentState); + assertEquals(++expectedInvalidation, service.getNumInvalidation()); + + // green to yellow or yellow to green + previousState = dummyState(randomFrom(ClusterHealthStatus.GREEN, ClusterHealthStatus.YELLOW)); + currentState = dummyState(previousState.indexStatus == ClusterHealthStatus.GREEN ? + ClusterHealthStatus.YELLOW : ClusterHealthStatus.GREEN); + service.onSecurityIndexStateChange(previousState, currentState); + assertEquals(expectedInvalidation, service.getNumInvalidation()); + } + + public void testAuthenticateSmartRealmOrderingDisabled() { + final Settings settings = Settings.builder() + .put(AuthenticationService.SUCCESS_AUTH_CACHE_ENABLED.getKey(), false) + .build(); + service = new AuthenticationService(settings, realms, auditTrail, + new DefaultAuthenticationFailureHandler(Collections.emptyMap()), threadPool, new AnonymousUser(Settings.EMPTY), + tokenService); + User user = new User("_username", "r1"); + when(firstRealm.supports(token)).thenReturn(true); + mockAuthenticate(firstRealm, token, null); + when(secondRealm.supports(token)).thenReturn(true); + mockAuthenticate(secondRealm, token, user); + when(secondRealm.token(threadContext)).thenReturn(token); + final String reqId = AuditUtil.getOrGenerateRequestId(threadContext); + + final AtomicBoolean completed = new AtomicBoolean(false); + service.authenticate("_action", message, (User)null, ActionListener.wrap(result -> { + assertThat(result, notNullValue()); + assertThat(result.getUser(), is(user)); + assertThat(result.getLookedUpBy(), is(nullValue())); + assertThat(result.getAuthenticatedBy(), is(notNullValue())); // TODO implement equals + assertThreadContextContainsAuthentication(result); + setCompletedToTrue(completed); + }, this::logAndFail)); + assertTrue(completed.get()); + + completed.set(false); + service.authenticate("_action", message, (User)null, ActionListener.wrap(result -> { + assertThat(result, notNullValue()); + assertThat(result.getUser(), is(user)); + assertThat(result.getLookedUpBy(), is(nullValue())); + assertThat(result.getAuthenticatedBy(), is(notNullValue())); // TODO implement equals + assertThreadContextContainsAuthentication(result); + setCompletedToTrue(completed); + }, this::logAndFail)); + verify(auditTrail, times(2)).authenticationFailed(reqId, firstRealm.name(), token, "_action", message); + verify(auditTrail, times(2)).authenticationSuccess(reqId, secondRealm.name(), user, "_action", message); + verify(firstRealm, times(3)).name(); // used above one time + verify(secondRealm, times(3)).name(); // used above one time + verify(secondRealm, times(2)).type(); // used to create realm ref + verify(firstRealm, times(2)).token(threadContext); + verify(secondRealm, times(2)).token(threadContext); + verify(firstRealm, times(2)).supports(token); + verify(secondRealm, times(2)).supports(token); + verify(firstRealm, times(2)).authenticate(eq(token), any(ActionListener.class)); + verify(secondRealm, times(2)).authenticate(eq(token), any(ActionListener.class)); + verifyNoMoreInteractions(auditTrail, firstRealm, secondRealm); + } + public void testAuthenticateFirstNotSupportingSecondSucceeds() throws Exception { User user = new User("_username", "r1"); when(firstRealm.supports(token)).thenReturn(false); @@ -614,6 +745,7 @@ public void testRealmTokenThrowingExceptionRest() throws Exception { public void testRealmSupportsMethodThrowingException() throws Exception { AuthenticationToken token = mock(AuthenticationToken.class); + when(token.principal()).thenReturn(randomAlphaOfLength(5)); when(secondRealm.token(threadContext)).thenReturn(token); when(secondRealm.supports(token)).thenThrow(authenticationError("realm doesn't like supports")); final String reqId = AuditUtil.getOrGenerateRequestId(threadContext); @@ -628,6 +760,7 @@ public void testRealmSupportsMethodThrowingException() throws Exception { public void testRealmSupportsMethodThrowingExceptionRest() throws Exception { AuthenticationToken token = mock(AuthenticationToken.class); + when(token.principal()).thenReturn(randomAlphaOfLength(5)); when(secondRealm.token(threadContext)).thenReturn(token); when(secondRealm.supports(token)).thenThrow(authenticationError("realm doesn't like supports")); try { @@ -643,6 +776,7 @@ public void testRealmSupportsMethodThrowingExceptionRest() throws Exception { public void testRealmAuthenticateTerminatingAuthenticationProcess() throws Exception { final String reqId = AuditUtil.getOrGenerateRequestId(threadContext); final AuthenticationToken token = mock(AuthenticationToken.class); + when(token.principal()).thenReturn(randomAlphaOfLength(5)); when(secondRealm.token(threadContext)).thenReturn(token); when(secondRealm.supports(token)).thenReturn(true); final boolean terminateWithNoException = rarely(); @@ -684,6 +818,7 @@ public void testRealmAuthenticateTerminatingAuthenticationProcess() throws Excep public void testRealmAuthenticateThrowingException() throws Exception { AuthenticationToken token = mock(AuthenticationToken.class); + when(token.principal()).thenReturn(randomAlphaOfLength(5)); when(secondRealm.token(threadContext)).thenReturn(token); when(secondRealm.supports(token)).thenReturn(true); doThrow(authenticationError("realm doesn't like authenticate")) @@ -700,6 +835,7 @@ public void testRealmAuthenticateThrowingException() throws Exception { public void testRealmAuthenticateThrowingExceptionRest() throws Exception { AuthenticationToken token = mock(AuthenticationToken.class); + when(token.principal()).thenReturn(randomAlphaOfLength(5)); when(secondRealm.token(threadContext)).thenReturn(token); when(secondRealm.supports(token)).thenReturn(true); doThrow(authenticationError("realm doesn't like authenticate")) @@ -716,6 +852,7 @@ public void testRealmAuthenticateThrowingExceptionRest() throws Exception { public void testRealmLookupThrowingException() throws Exception { AuthenticationToken token = mock(AuthenticationToken.class); + when(token.principal()).thenReturn(randomAlphaOfLength(5)); threadContext.putHeader(AuthenticationServiceField.RUN_AS_USER_HEADER, "run_as"); when(secondRealm.token(threadContext)).thenReturn(token); when(secondRealm.supports(token)).thenReturn(true); @@ -736,6 +873,7 @@ public void testRealmLookupThrowingException() throws Exception { public void testRealmLookupThrowingExceptionRest() throws Exception { AuthenticationToken token = mock(AuthenticationToken.class); + when(token.principal()).thenReturn(randomAlphaOfLength(5)); threadContext.putHeader(AuthenticationServiceField.RUN_AS_USER_HEADER, "run_as"); when(secondRealm.token(threadContext)).thenReturn(token); when(secondRealm.supports(token)).thenReturn(true); @@ -755,6 +893,7 @@ public void testRealmLookupThrowingExceptionRest() throws Exception { public void testRunAsLookupSameRealm() throws Exception { AuthenticationToken token = mock(AuthenticationToken.class); + when(token.principal()).thenReturn(randomAlphaOfLength(5)); threadContext.putHeader(AuthenticationServiceField.RUN_AS_USER_HEADER, "run_as"); when(secondRealm.token(threadContext)).thenReturn(token); when(secondRealm.supports(token)).thenReturn(true); @@ -803,6 +942,7 @@ public void testRunAsLookupSameRealm() throws Exception { @SuppressWarnings("unchecked") public void testRunAsLookupDifferentRealm() throws Exception { AuthenticationToken token = mock(AuthenticationToken.class); + when(token.principal()).thenReturn(randomAlphaOfLength(5)); threadContext.putHeader(AuthenticationServiceField.RUN_AS_USER_HEADER, "run_as"); when(secondRealm.token(threadContext)).thenReturn(token); when(secondRealm.supports(token)).thenReturn(true); @@ -839,6 +979,7 @@ public void testRunAsLookupDifferentRealm() throws Exception { public void testRunAsWithEmptyRunAsUsernameRest() throws Exception { AuthenticationToken token = mock(AuthenticationToken.class); + when(token.principal()).thenReturn(randomAlphaOfLength(5)); User user = new User("lookup user", new String[]{"user"}); threadContext.putHeader(AuthenticationServiceField.RUN_AS_USER_HEADER, ""); when(secondRealm.token(threadContext)).thenReturn(token); @@ -857,6 +998,7 @@ public void testRunAsWithEmptyRunAsUsernameRest() throws Exception { public void testRunAsWithEmptyRunAsUsername() throws Exception { AuthenticationToken token = mock(AuthenticationToken.class); + when(token.principal()).thenReturn(randomAlphaOfLength(5)); User user = new User("lookup user", new String[]{"user"}); threadContext.putHeader(AuthenticationServiceField.RUN_AS_USER_HEADER, ""); final String reqId = AuditUtil.getOrGenerateRequestId(threadContext); @@ -876,6 +1018,7 @@ public void testRunAsWithEmptyRunAsUsername() throws Exception { @SuppressWarnings("unchecked") public void testAuthenticateTransportDisabledRunAsUser() throws Exception { AuthenticationToken token = mock(AuthenticationToken.class); + when(token.principal()).thenReturn(randomAlphaOfLength(5)); threadContext.putHeader(AuthenticationServiceField.RUN_AS_USER_HEADER, "run_as"); final String reqId = AuditUtil.getOrGenerateRequestId(threadContext); when(secondRealm.token(threadContext)).thenReturn(token); @@ -897,6 +1040,7 @@ public void testAuthenticateTransportDisabledRunAsUser() throws Exception { public void testAuthenticateRestDisabledRunAsUser() throws Exception { AuthenticationToken token = mock(AuthenticationToken.class); + when(token.principal()).thenReturn(randomAlphaOfLength(5)); threadContext.putHeader(AuthenticationServiceField.RUN_AS_USER_HEADER, "run_as"); when(secondRealm.token(threadContext)).thenReturn(token); when(secondRealm.supports(token)).thenReturn(true); @@ -1117,4 +1261,8 @@ private void logAndFail(Exception e) { private void setCompletedToTrue(AtomicBoolean completed) { assertTrue(completed.compareAndSet(false, true)); } + + private SecurityIndexManager.State dummyState(ClusterHealthStatus indexStatus) { + return new SecurityIndexManager.State(true, true, true, true, null, indexStatus); + } } From 9de62f126278d1866e381418dec4af1aea7cd0b4 Mon Sep 17 00:00:00 2001 From: Tim Brooks Date: Thu, 10 Jan 2019 09:17:20 -0700 Subject: [PATCH 081/186] Increase IO direct byte buffers to 256KB (#37283) Currently we read and write 64KB at a time in the nio libraries. As a single byte buffer per event loop thread does not consume much memory, there is little reason to not increase it further. This commit increases the buffer to 256KB but still limits a single write to 64KB. The write limit could be increased, but too high of a write limit will lead to copying more data (if all the data is not flushed and needs to be copied on the next call). This is something to explore in the future. --- .../src/main/java/org/elasticsearch/nio/NioSelector.java | 2 +- .../java/org/elasticsearch/nio/SocketChannelContext.java | 6 ++++++ 2 files changed, 7 insertions(+), 1 deletion(-) 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 cacd06bde5fa3..1484ba2198f12 100644 --- a/libs/nio/src/main/java/org/elasticsearch/nio/NioSelector.java +++ b/libs/nio/src/main/java/org/elasticsearch/nio/NioSelector.java @@ -69,7 +69,7 @@ public NioSelector(EventHandler eventHandler) throws IOException { public NioSelector(EventHandler eventHandler, Selector selector) { this.selector = selector; this.eventHandler = eventHandler; - this.ioBuffer = ByteBuffer.allocateDirect(1 << 16); + this.ioBuffer = ByteBuffer.allocateDirect(1 << 18); } /** 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 661c55cc7280a..a43a799423f06 100644 --- a/libs/nio/src/main/java/org/elasticsearch/nio/SocketChannelContext.java +++ b/libs/nio/src/main/java/org/elasticsearch/nio/SocketChannelContext.java @@ -294,9 +294,14 @@ protected int readFromChannel(InboundChannelBuffer channelBuffer) throws IOExcep } } + // Currently we limit to 64KB. This is a trade-off which means more syscalls, in exchange for less + // copying. + private final int WRITE_LIMIT = 1 << 16; + protected int flushToChannel(ByteBuffer buffer) throws IOException { int initialPosition = buffer.position(); ByteBuffer ioBuffer = getSelector().getIoBuffer(); + ioBuffer.limit(Math.min(WRITE_LIMIT, ioBuffer.limit())); copyBytes(buffer, ioBuffer); ioBuffer.flip(); int bytesWritten; @@ -318,6 +323,7 @@ protected int flushToChannel(FlushOperation flushOperation) throws IOException { int totalBytesFlushed = 0; while (continueFlush) { ioBuffer.clear(); + ioBuffer.limit(Math.min(WRITE_LIMIT, ioBuffer.limit())); int j = 0; ByteBuffer[] buffers = flushOperation.getBuffersToWrite(); while (j < buffers.length && ioBuffer.remaining() > 0) { From a433c4012c51263d49e33cb9c708920f84b86f99 Mon Sep 17 00:00:00 2001 From: Julie Tibshirani Date: Thu, 10 Jan 2019 09:24:08 -0800 Subject: [PATCH 082/186] Support include_type_name in the field mapping and index template APIs. (#37210) * Add include_type_name to the get field mappings API. * Make sure the API specification lists include_type_name as a boolean. * Add include_type_name to the get index templates API. * Add include_type_name to the put index templates API. --- .../rest-api-spec/api/indices.create.json | 2 +- .../api/indices.get_field_mapping.json | 4 + .../api/indices.get_mapping.json | 2 +- .../api/indices.get_template.json | 4 + .../api/indices.put_mapping.json | 2 +- .../api/indices.put_template.json | 4 + .../indices.get_field_mapping/10_basic.yml | 53 +++---- .../11_basic_with_types.yml | 76 ++++++++++ .../20_missing_field.yml | 21 +-- .../21_missing_field_with_types.yml | 21 +++ .../40_missing_index.yml | 1 - .../50_field_wildcards.yml | 141 +++++++++--------- .../51_field_wildcards_with_types.yml | 135 +++++++++++++++++ .../60_mix_typeless_typeful.yml | 24 +++ .../test/indices.get_template/10_basic.yml | 31 ++++ .../11_basic_with_types.yml | 45 ++++++ .../test/indices.put_template/10_basic.yml | 45 ++++++ .../11_basic_with_types.yml | 68 +++++++++ .../mapping/get/GetFieldMappingsResponse.java | 37 ++++- .../metadata/IndexTemplateMetaData.java | 52 +++++-- .../elasticsearch/rest/BaseRestHandler.java | 2 +- .../indices/RestGetFieldMappingAction.java | 7 + .../indices/RestGetIndexTemplateAction.java | 7 +- .../indices/RestPutIndexTemplateAction.java | 21 ++- .../RestGetFieldMappingActionTests.java | 61 ++++++++ .../RestPutIndexTemplateActionTests.java | 81 ++++++++++ 26 files changed, 811 insertions(+), 136 deletions(-) create mode 100644 rest-api-spec/src/main/resources/rest-api-spec/test/indices.get_field_mapping/11_basic_with_types.yml create mode 100644 rest-api-spec/src/main/resources/rest-api-spec/test/indices.get_field_mapping/21_missing_field_with_types.yml create mode 100644 rest-api-spec/src/main/resources/rest-api-spec/test/indices.get_field_mapping/51_field_wildcards_with_types.yml create mode 100644 rest-api-spec/src/main/resources/rest-api-spec/test/indices.get_field_mapping/60_mix_typeless_typeful.yml create mode 100644 rest-api-spec/src/main/resources/rest-api-spec/test/indices.get_template/11_basic_with_types.yml create mode 100644 rest-api-spec/src/main/resources/rest-api-spec/test/indices.put_template/11_basic_with_types.yml create mode 100644 server/src/test/java/org/elasticsearch/rest/action/admin/indices/RestGetFieldMappingActionTests.java create mode 100644 server/src/test/java/org/elasticsearch/rest/action/admin/indices/RestPutIndexTemplateActionTests.java diff --git a/rest-api-spec/src/main/resources/rest-api-spec/api/indices.create.json b/rest-api-spec/src/main/resources/rest-api-spec/api/indices.create.json index 2ff9d3f68d9d9..d4a16e576e1b9 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/api/indices.create.json +++ b/rest-api-spec/src/main/resources/rest-api-spec/api/indices.create.json @@ -14,7 +14,7 @@ }, "params": { "include_type_name": { - "type" : "string", + "type" : "boolean", "description" : "Whether a type should be expected in the body of the mappings." }, "wait_for_active_shards": { diff --git a/rest-api-spec/src/main/resources/rest-api-spec/api/indices.get_field_mapping.json b/rest-api-spec/src/main/resources/rest-api-spec/api/indices.get_field_mapping.json index 3d5a629eff08e..3ce610153b5e6 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/api/indices.get_field_mapping.json +++ b/rest-api-spec/src/main/resources/rest-api-spec/api/indices.get_field_mapping.json @@ -21,6 +21,10 @@ } }, "params": { + "include_type_name": { + "type" : "boolean", + "description" : "Whether a type should be returned in the body of the mappings." + }, "include_defaults": { "type" : "boolean", "description" : "Whether the default mapping values should be returned as well" diff --git a/rest-api-spec/src/main/resources/rest-api-spec/api/indices.get_mapping.json b/rest-api-spec/src/main/resources/rest-api-spec/api/indices.get_mapping.json index 9bfb9c76abf82..ccec2ddffdd0c 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/api/indices.get_mapping.json +++ b/rest-api-spec/src/main/resources/rest-api-spec/api/indices.get_mapping.json @@ -17,7 +17,7 @@ }, "params": { "include_type_name": { - "type" : "string", + "type" : "boolean", "description" : "Whether to add the type name to the response" }, "ignore_unavailable": { diff --git a/rest-api-spec/src/main/resources/rest-api-spec/api/indices.get_template.json b/rest-api-spec/src/main/resources/rest-api-spec/api/indices.get_template.json index e3a97ee5c012a..e2aae3b7444aa 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/api/indices.get_template.json +++ b/rest-api-spec/src/main/resources/rest-api-spec/api/indices.get_template.json @@ -16,6 +16,10 @@ } }, "params": { + "include_type_name": { + "type" : "boolean", + "description" : "Whether a type should be returned in the body of the mappings." + }, "flat_settings": { "type": "boolean", "description": "Return settings in flat format (default: false)" diff --git a/rest-api-spec/src/main/resources/rest-api-spec/api/indices.put_mapping.json b/rest-api-spec/src/main/resources/rest-api-spec/api/indices.put_mapping.json index 4efb615329639..cc55ffccdd1ef 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/api/indices.put_mapping.json +++ b/rest-api-spec/src/main/resources/rest-api-spec/api/indices.put_mapping.json @@ -17,7 +17,7 @@ }, "params": { "include_type_name": { - "type" : "string", + "type" : "boolean", "description" : "Whether a type should be expected in the body of the mappings." }, "timeout": { diff --git a/rest-api-spec/src/main/resources/rest-api-spec/api/indices.put_template.json b/rest-api-spec/src/main/resources/rest-api-spec/api/indices.put_template.json index 5bcb2f8a24346..65aa9506ff9f1 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/api/indices.put_template.json +++ b/rest-api-spec/src/main/resources/rest-api-spec/api/indices.put_template.json @@ -13,6 +13,10 @@ } }, "params": { + "include_type_name": { + "type" : "boolean", + "description" : "Whether a type should be returned in the body of the mappings." + }, "order": { "type" : "number", "description" : "The order for this template when merging multiple matching ones (higher numbers are merged later, overriding the lower numbers)" diff --git a/rest-api-spec/src/main/resources/rest-api-spec/test/indices.get_field_mapping/10_basic.yml b/rest-api-spec/src/main/resources/rest-api-spec/test/indices.get_field_mapping/10_basic.yml index 6cf0a0b7cf26c..b77c56d34160c 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/test/indices.get_field_mapping/10_basic.yml +++ b/rest-api-spec/src/main/resources/rest-api-spec/test/indices.get_field_mapping/10_basic.yml @@ -1,54 +1,48 @@ --- setup: + - skip: + version: " - 6.99.99" + reason: include_type_name is not supported before 7.0.0 - do: indices.create: + include_type_name: false index: test_index body: mappings: - test_type: - properties: - text: - type: text + properties: + text: + type: text --- -"Get field mapping with no index and type": +"Get field mapping with no index": - do: indices.get_field_mapping: + include_type_name: false fields: text - - match: {test_index.mappings.test_type.text.mapping.text.type: text} + - match: {test_index.mappings.text.mapping.text.type: text} --- "Get field mapping by index only": - do: indices.get_field_mapping: + include_type_name: false index: test_index fields: text - - match: {test_index.mappings.test_type.text.mapping.text.type: text} + - match: {test_index.mappings.text.mapping.text.type: text} --- -"Get field mapping by type & field": +"Get field mapping by field, with another field that doesn't exist": - do: indices.get_field_mapping: + include_type_name: false index: test_index - type: test_type - fields: text - - - match: {test_index.mappings.test_type.text.mapping.text.type: text} - ---- -"Get field mapping by type & field, with another field that doesn't exist": - - - do: - indices.get_field_mapping: - index: test_index - type: test_type fields: [ text , text1 ] - - match: {test_index.mappings.test_type.text.mapping.text.type: text} + - match: {test_index.mappings.text.mapping.text.type: text} - is_false: test_index.mappings.test_type.text1 --- @@ -56,21 +50,10 @@ setup: - do: indices.get_field_mapping: + include_type_name: false index: test_index - type: test_type fields: text include_defaults: true - - match: {test_index.mappings.test_type.text.mapping.text.type: text} - - match: {test_index.mappings.test_type.text.mapping.text.analyzer: default} - ---- -"Get field mapping should work without index specifying type and fields": - - - do: - indices.get_field_mapping: - type: test_type - fields: text - - - match: {test_index.mappings.test_type.text.mapping.text.type: text} - + - match: {test_index.mappings.text.mapping.text.type: text} + - match: {test_index.mappings.text.mapping.text.analyzer: default} diff --git a/rest-api-spec/src/main/resources/rest-api-spec/test/indices.get_field_mapping/11_basic_with_types.yml b/rest-api-spec/src/main/resources/rest-api-spec/test/indices.get_field_mapping/11_basic_with_types.yml new file mode 100644 index 0000000000000..6cf0a0b7cf26c --- /dev/null +++ b/rest-api-spec/src/main/resources/rest-api-spec/test/indices.get_field_mapping/11_basic_with_types.yml @@ -0,0 +1,76 @@ +--- +setup: + - do: + indices.create: + index: test_index + body: + mappings: + test_type: + properties: + text: + type: text + +--- +"Get field mapping with no index and type": + + - do: + indices.get_field_mapping: + fields: text + + - match: {test_index.mappings.test_type.text.mapping.text.type: text} + +--- +"Get field mapping by index only": + - do: + indices.get_field_mapping: + index: test_index + fields: text + + - match: {test_index.mappings.test_type.text.mapping.text.type: text} + +--- +"Get field mapping by type & field": + + - do: + indices.get_field_mapping: + index: test_index + type: test_type + fields: text + + - match: {test_index.mappings.test_type.text.mapping.text.type: text} + +--- +"Get field mapping by type & field, with another field that doesn't exist": + + - do: + indices.get_field_mapping: + index: test_index + type: test_type + fields: [ text , text1 ] + + - match: {test_index.mappings.test_type.text.mapping.text.type: text} + - is_false: test_index.mappings.test_type.text1 + +--- +"Get field mapping with include_defaults": + + - do: + indices.get_field_mapping: + index: test_index + type: test_type + fields: text + include_defaults: true + + - match: {test_index.mappings.test_type.text.mapping.text.type: text} + - match: {test_index.mappings.test_type.text.mapping.text.analyzer: default} + +--- +"Get field mapping should work without index specifying type and fields": + + - do: + indices.get_field_mapping: + type: test_type + fields: text + + - match: {test_index.mappings.test_type.text.mapping.text.type: text} + diff --git a/rest-api-spec/src/main/resources/rest-api-spec/test/indices.get_field_mapping/20_missing_field.yml b/rest-api-spec/src/main/resources/rest-api-spec/test/indices.get_field_mapping/20_missing_field.yml index 9b8c3efbce81a..61f1f409d2939 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/test/indices.get_field_mapping/20_missing_field.yml +++ b/rest-api-spec/src/main/resources/rest-api-spec/test/indices.get_field_mapping/20_missing_field.yml @@ -1,21 +1,22 @@ --- "Return empty object if field doesn't exist, but type and index do": - + - skip: + version: " - 6.99.99" + reason: types are required in requests before 7.0.0 - do: indices.create: + include_type_name: false index: test_index body: - mappings: - test_type: - properties: - text: - type: text - analyzer: whitespace + mappings: + properties: + text: + type: text + analyzer: whitespace - do: indices.get_field_mapping: index: test_index - type: test_type fields: not_existent - - - match: { '': {}} + + - match: { 'test_index.mappings': {}} diff --git a/rest-api-spec/src/main/resources/rest-api-spec/test/indices.get_field_mapping/21_missing_field_with_types.yml b/rest-api-spec/src/main/resources/rest-api-spec/test/indices.get_field_mapping/21_missing_field_with_types.yml new file mode 100644 index 0000000000000..c760561f09282 --- /dev/null +++ b/rest-api-spec/src/main/resources/rest-api-spec/test/indices.get_field_mapping/21_missing_field_with_types.yml @@ -0,0 +1,21 @@ +--- +"Return empty object if field doesn't exist, but type and index do": + + - do: + indices.create: + index: test_index + body: + mappings: + test_type: + properties: + text: + type: text + analyzer: whitespace + + - do: + indices.get_field_mapping: + index: test_index + type: test_type + fields: not_existent + + - match: { '': {}} diff --git a/rest-api-spec/src/main/resources/rest-api-spec/test/indices.get_field_mapping/40_missing_index.yml b/rest-api-spec/src/main/resources/rest-api-spec/test/indices.get_field_mapping/40_missing_index.yml index 7da516e116c3d..7c7b07b587849 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/test/indices.get_field_mapping/40_missing_index.yml +++ b/rest-api-spec/src/main/resources/rest-api-spec/test/indices.get_field_mapping/40_missing_index.yml @@ -5,7 +5,6 @@ catch: missing indices.get_field_mapping: index: test_index - type: type fields: field diff --git a/rest-api-spec/src/main/resources/rest-api-spec/test/indices.get_field_mapping/50_field_wildcards.yml b/rest-api-spec/src/main/resources/rest-api-spec/test/indices.get_field_mapping/50_field_wildcards.yml index 9d62ab6101fc2..3ffecdcc72618 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/test/indices.get_field_mapping/50_field_wildcards.yml +++ b/rest-api-spec/src/main/resources/rest-api-spec/test/indices.get_field_mapping/50_field_wildcards.yml @@ -1,135 +1,142 @@ --- setup: + - skip: + version: " - 6.99.99" + reason: types are required in requests before 7.0.0 - do: indices.create: + include_type_name: false index: test_index body: mappings: - test_type: - properties: - t1: - type: text - t2: - type: text - obj: - properties: - t1: - type: text - i_t1: - type: text - i_t3: - type: text + properties: + t1: + type: text + t2: + type: text + obj: + properties: + t1: + type: text + i_t1: + type: text + i_t3: + type: text - do: indices.create: + include_type_name: false index: test_index_2 body: mappings: - test_type_2: - properties: - t1: - type: text - t2: - type: text - obj: - properties: - t1: - type: text - i_t1: - type: text - i_t3: - type: text + properties: + t1: + type: text + t2: + type: text + obj: + properties: + t1: + type: text + i_t1: + type: text + i_t3: + type: text --- "Get field mapping with * for fields": - do: indices.get_field_mapping: + include_type_name: false fields: "*" - - match: {test_index.mappings.test_type.t1.full_name: t1 } - - match: {test_index.mappings.test_type.t2.full_name: t2 } - - match: {test_index.mappings.test_type.obj\.t1.full_name: obj.t1 } - - match: {test_index.mappings.test_type.obj\.i_t1.full_name: obj.i_t1 } - - match: {test_index.mappings.test_type.obj\.i_t3.full_name: obj.i_t3 } + - match: {test_index.mappings.t1.full_name: t1 } + - match: {test_index.mappings.t2.full_name: t2 } + - match: {test_index.mappings.obj\.t1.full_name: obj.t1 } + - match: {test_index.mappings.obj\.i_t1.full_name: obj.i_t1 } + - match: {test_index.mappings.obj\.i_t3.full_name: obj.i_t3 } --- "Get field mapping with t* for fields": - do: indices.get_field_mapping: + include_type_name: false index: test_index fields: "t*" - - match: {test_index.mappings.test_type.t1.full_name: t1 } - - match: {test_index.mappings.test_type.t2.full_name: t2 } - - length: {test_index.mappings.test_type: 2} + - match: {test_index.mappings.t1.full_name: t1 } + - match: {test_index.mappings.t2.full_name: t2 } + - length: {test_index.mappings: 2} --- "Get field mapping with *t1 for fields": - do: indices.get_field_mapping: + include_type_name: false index: test_index fields: "*t1" - - match: {test_index.mappings.test_type.t1.full_name: t1 } - - match: {test_index.mappings.test_type.obj\.t1.full_name: obj.t1 } - - match: {test_index.mappings.test_type.obj\.i_t1.full_name: obj.i_t1 } - - length: {test_index.mappings.test_type: 3} + - match: {test_index.mappings.t1.full_name: t1 } + - match: {test_index.mappings.obj\.t1.full_name: obj.t1 } + - match: {test_index.mappings.obj\.i_t1.full_name: obj.i_t1 } + - length: {test_index.mappings: 3} --- "Get field mapping with wildcarded relative names": - do: indices.get_field_mapping: + include_type_name: false index: test_index fields: "obj.i_*" - - match: {test_index.mappings.test_type.obj\.i_t1.full_name: obj.i_t1 } - - match: {test_index.mappings.test_type.obj\.i_t3.full_name: obj.i_t3 } - - length: {test_index.mappings.test_type: 2} + - match: {test_index.mappings.obj\.i_t1.full_name: obj.i_t1 } + - match: {test_index.mappings.obj\.i_t3.full_name: obj.i_t3 } + - length: {test_index.mappings: 2} --- -"Get field mapping should work using '_all' for indices and types": +"Get field mapping should work using '_all' for index": - do: indices.get_field_mapping: + include_type_name: false index: _all - type: _all fields: "t*" - - match: {test_index.mappings.test_type.t1.full_name: t1 } - - match: {test_index.mappings.test_type.t2.full_name: t2 } - - length: {test_index.mappings.test_type: 2} - - match: {test_index_2.mappings.test_type_2.t1.full_name: t1 } - - match: {test_index_2.mappings.test_type_2.t2.full_name: t2 } - - length: {test_index_2.mappings.test_type_2: 2} + - match: {test_index.mappings.t1.full_name: t1 } + - match: {test_index.mappings.t2.full_name: t2 } + - length: {test_index.mappings: 2} + - match: {test_index_2.mappings.t1.full_name: t1 } + - match: {test_index_2.mappings.t2.full_name: t2 } + - length: {test_index_2.mappings: 2} --- -"Get field mapping should work using '*' for indices and types": +"Get field mapping should work using '*' for index": - do: indices.get_field_mapping: + include_type_name: false index: '*' - type: '*' fields: "t*" - - match: {test_index.mappings.test_type.t1.full_name: t1 } - - match: {test_index.mappings.test_type.t2.full_name: t2 } - - length: {test_index.mappings.test_type: 2} - - match: {test_index_2.mappings.test_type_2.t1.full_name: t1 } - - match: {test_index_2.mappings.test_type_2.t2.full_name: t2 } - - length: {test_index_2.mappings.test_type_2: 2} + - match: {test_index.mappings.t1.full_name: t1 } + - match: {test_index.mappings.t2.full_name: t2 } + - length: {test_index.mappings: 2} + - match: {test_index_2.mappings.t1.full_name: t1 } + - match: {test_index_2.mappings.t2.full_name: t2 } + - length: {test_index_2.mappings: 2} --- -"Get field mapping should work using comma_separated values for indices and types": +"Get field mapping should work using comma_separated values for indices": - do: indices.get_field_mapping: + include_type_name: false index: 'test_index,test_index_2' - type: 'test_type,test_type_2' fields: "t*" - - match: {test_index.mappings.test_type.t1.full_name: t1 } - - match: {test_index.mappings.test_type.t2.full_name: t2 } - - length: {test_index.mappings.test_type: 2} - - match: {test_index_2.mappings.test_type_2.t1.full_name: t1 } - - match: {test_index_2.mappings.test_type_2.t2.full_name: t2 } - - length: {test_index_2.mappings.test_type_2: 2} + - match: {test_index.mappings.t1.full_name: t1 } + - match: {test_index.mappings.t2.full_name: t2 } + - length: {test_index.mappings: 2} + - match: {test_index_2.mappings.t1.full_name: t1 } + - match: {test_index_2.mappings.t2.full_name: t2 } + - length: {test_index_2.mappings: 2} diff --git a/rest-api-spec/src/main/resources/rest-api-spec/test/indices.get_field_mapping/51_field_wildcards_with_types.yml b/rest-api-spec/src/main/resources/rest-api-spec/test/indices.get_field_mapping/51_field_wildcards_with_types.yml new file mode 100644 index 0000000000000..9d62ab6101fc2 --- /dev/null +++ b/rest-api-spec/src/main/resources/rest-api-spec/test/indices.get_field_mapping/51_field_wildcards_with_types.yml @@ -0,0 +1,135 @@ +--- +setup: + - do: + indices.create: + index: test_index + body: + mappings: + test_type: + properties: + t1: + type: text + t2: + type: text + obj: + properties: + t1: + type: text + i_t1: + type: text + i_t3: + type: text + + - do: + indices.create: + index: test_index_2 + body: + mappings: + test_type_2: + properties: + t1: + type: text + t2: + type: text + obj: + properties: + t1: + type: text + i_t1: + type: text + i_t3: + type: text + +--- +"Get field mapping with * for fields": + + - do: + indices.get_field_mapping: + fields: "*" + + - match: {test_index.mappings.test_type.t1.full_name: t1 } + - match: {test_index.mappings.test_type.t2.full_name: t2 } + - match: {test_index.mappings.test_type.obj\.t1.full_name: obj.t1 } + - match: {test_index.mappings.test_type.obj\.i_t1.full_name: obj.i_t1 } + - match: {test_index.mappings.test_type.obj\.i_t3.full_name: obj.i_t3 } + +--- +"Get field mapping with t* for fields": + + - do: + indices.get_field_mapping: + index: test_index + fields: "t*" + + - match: {test_index.mappings.test_type.t1.full_name: t1 } + - match: {test_index.mappings.test_type.t2.full_name: t2 } + - length: {test_index.mappings.test_type: 2} + +--- +"Get field mapping with *t1 for fields": + + - do: + indices.get_field_mapping: + index: test_index + fields: "*t1" + - match: {test_index.mappings.test_type.t1.full_name: t1 } + - match: {test_index.mappings.test_type.obj\.t1.full_name: obj.t1 } + - match: {test_index.mappings.test_type.obj\.i_t1.full_name: obj.i_t1 } + - length: {test_index.mappings.test_type: 3} + +--- +"Get field mapping with wildcarded relative names": + + - do: + indices.get_field_mapping: + index: test_index + fields: "obj.i_*" + - match: {test_index.mappings.test_type.obj\.i_t1.full_name: obj.i_t1 } + - match: {test_index.mappings.test_type.obj\.i_t3.full_name: obj.i_t3 } + - length: {test_index.mappings.test_type: 2} + +--- +"Get field mapping should work using '_all' for indices and types": + + - do: + indices.get_field_mapping: + index: _all + type: _all + fields: "t*" + - match: {test_index.mappings.test_type.t1.full_name: t1 } + - match: {test_index.mappings.test_type.t2.full_name: t2 } + - length: {test_index.mappings.test_type: 2} + - match: {test_index_2.mappings.test_type_2.t1.full_name: t1 } + - match: {test_index_2.mappings.test_type_2.t2.full_name: t2 } + - length: {test_index_2.mappings.test_type_2: 2} + +--- +"Get field mapping should work using '*' for indices and types": + + - do: + indices.get_field_mapping: + index: '*' + type: '*' + fields: "t*" + - match: {test_index.mappings.test_type.t1.full_name: t1 } + - match: {test_index.mappings.test_type.t2.full_name: t2 } + - length: {test_index.mappings.test_type: 2} + - match: {test_index_2.mappings.test_type_2.t1.full_name: t1 } + - match: {test_index_2.mappings.test_type_2.t2.full_name: t2 } + - length: {test_index_2.mappings.test_type_2: 2} + +--- +"Get field mapping should work using comma_separated values for indices and types": + + - do: + indices.get_field_mapping: + index: 'test_index,test_index_2' + type: 'test_type,test_type_2' + fields: "t*" + - match: {test_index.mappings.test_type.t1.full_name: t1 } + - match: {test_index.mappings.test_type.t2.full_name: t2 } + - length: {test_index.mappings.test_type: 2} + - match: {test_index_2.mappings.test_type_2.t1.full_name: t1 } + - match: {test_index_2.mappings.test_type_2.t2.full_name: t2 } + - length: {test_index_2.mappings.test_type_2: 2} + diff --git a/rest-api-spec/src/main/resources/rest-api-spec/test/indices.get_field_mapping/60_mix_typeless_typeful.yml b/rest-api-spec/src/main/resources/rest-api-spec/test/indices.get_field_mapping/60_mix_typeless_typeful.yml new file mode 100644 index 0000000000000..d7ea620bb6c58 --- /dev/null +++ b/rest-api-spec/src/main/resources/rest-api-spec/test/indices.get_field_mapping/60_mix_typeless_typeful.yml @@ -0,0 +1,24 @@ +--- +"GET mapping with typeless API on an index that has types": + + - skip: + version: " - 6.99.99" + reason: include_type_name was introduced in 7.0.0 + + - do: + indices.create: # not using include_type_name: false on purpose + index: index + body: + mappings: + not_doc: + properties: + foo: + type: "keyword" + + - do: + indices.get_field_mapping: + include_type_name: false + index: index + fields: foo + + - match: { index.mappings.foo.mapping.foo.type: "keyword" } diff --git a/rest-api-spec/src/main/resources/rest-api-spec/test/indices.get_template/10_basic.yml b/rest-api-spec/src/main/resources/rest-api-spec/test/indices.get_template/10_basic.yml index a03a10c1a5a89..8fe244e0c6323 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/test/indices.get_template/10_basic.yml +++ b/rest-api-spec/src/main/resources/rest-api-spec/test/indices.get_template/10_basic.yml @@ -1,4 +1,7 @@ setup: + - skip: + version: " - 6.99.99" + reason: include_type_name is not supported before 7.0.0 - do: indices.put_template: name: test @@ -7,16 +10,44 @@ setup: settings: number_of_shards: 1 number_of_replicas: 0 + mappings: + _doc: + properties: + field: + type: keyword --- "Get template": - do: indices.get_template: + include_type_name: false name: test - match: {test.index_patterns: ["test-*"]} - match: {test.settings: {index: {number_of_shards: '1', number_of_replicas: '0'}}} + - match: {test.mappings: {properties: {field: {type: keyword}}}} + +--- +"Get template with no mappings": + + - do: + indices.put_template: + name: test_no_mappings + body: + index_patterns: test-* + settings: + number_of_shards: 1 + number_of_replicas: 0 + + - do: + indices.get_template: + include_type_name: false + name: test_no_mappings + + - match: {test_no_mappings.index_patterns: ["test-*"]} + - match: {test_no_mappings.settings: {index: {number_of_shards: '1', number_of_replicas: '0'}}} + - match: {test_no_mappings.mappings: {}} --- "Get all templates": diff --git a/rest-api-spec/src/main/resources/rest-api-spec/test/indices.get_template/11_basic_with_types.yml b/rest-api-spec/src/main/resources/rest-api-spec/test/indices.get_template/11_basic_with_types.yml new file mode 100644 index 0000000000000..c15f5dc6de4f1 --- /dev/null +++ b/rest-api-spec/src/main/resources/rest-api-spec/test/indices.get_template/11_basic_with_types.yml @@ -0,0 +1,45 @@ +setup: + - do: + indices.put_template: + name: test + body: + index_patterns: test-* + settings: + number_of_shards: 1 + number_of_replicas: 0 + mappings: + _doc: + properties: + field: + type: keyword + +--- +"Get template": + + - do: + indices.get_template: + name: test + + - match: {test.index_patterns: ["test-*"]} + - match: {test.settings: {index: {number_of_shards: '1', number_of_replicas: '0'}}} + - match: {test.mappings: {_doc: {properties: {field: {type: keyword}}}}} + +--- +"Get template with no mappings": + + - do: + indices.put_template: + name: test_no_mappings + body: + index_patterns: test-* + settings: + number_of_shards: 1 + number_of_replicas: 0 + + - do: + indices.get_template: + name: test_no_mappings + + - match: {test_no_mappings.index_patterns: ["test-*"]} + - match: {test_no_mappings.settings: {index: {number_of_shards: '1', number_of_replicas: '0'}}} + - match: {test_no_mappings.mappings: {}} diff --git a/rest-api-spec/src/main/resources/rest-api-spec/test/indices.put_template/10_basic.yml b/rest-api-spec/src/main/resources/rest-api-spec/test/indices.put_template/10_basic.yml index b4e66c23c605b..8637b3e6d1864 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/test/indices.put_template/10_basic.yml +++ b/rest-api-spec/src/main/resources/rest-api-spec/test/indices.put_template/10_basic.yml @@ -1,42 +1,87 @@ --- "Put template": + - skip: + version: " - 6.99.99" + reason: include_type_name is not supported before 7.0.0 - do: indices.put_template: + include_type_name: false name: test body: index_patterns: test-* settings: number_of_shards: 1 number_of_replicas: 0 + mappings: + properties: + field: + type: keyword - do: indices.get_template: + include_type_name: false name: test flat_settings: true - match: {test.index_patterns: ["test-*"]} - match: {test.settings: {index.number_of_shards: '1', index.number_of_replicas: '0'}} + - match: {test.mappings: {properties: {field: {type: keyword}}}} --- "Put multiple template": + - skip: + version: " - 6.99.99" + reason: include_type_name was introduced in 7.0.0 - do: indices.put_template: + include_type_name: false name: test body: index_patterns: [test-*, test2-*] settings: number_of_shards: 1 number_of_replicas: 0 + mappings: + properties: + field: + type: text - do: indices.get_template: + include_type_name: false name: test flat_settings: true - match: {test.index_patterns: ["test-*", "test2-*"]} - match: {test.settings: {index.number_of_shards: '1', index.number_of_replicas: '0'}} + - match: {test.mappings: {properties: {field: {type: text}}}} + +--- +"Put template with empty mappings": + - skip: + version: " - 6.99.99" + reason: include_type_name was introduced in 7.0.0 + + - do: + indices.put_template: + include_type_name: false + name: test + body: + index_patterns: test-* + settings: + number_of_shards: 1 + number_of_replicas: 0 + mappings: {} + + - do: + indices.get_template: + include_type_name: false + name: test + flat_settings: true + + - match: {test.mappings: {}} --- "Put template with aliases": diff --git a/rest-api-spec/src/main/resources/rest-api-spec/test/indices.put_template/11_basic_with_types.yml b/rest-api-spec/src/main/resources/rest-api-spec/test/indices.put_template/11_basic_with_types.yml new file mode 100644 index 0000000000000..1e14a9d3895a7 --- /dev/null +++ b/rest-api-spec/src/main/resources/rest-api-spec/test/indices.put_template/11_basic_with_types.yml @@ -0,0 +1,68 @@ +--- +"Put template": + - do: + indices.put_template: + name: test + body: + index_patterns: test-* + settings: + number_of_shards: 1 + number_of_replicas: 0 + mappings: + _doc: + properties: + field: + type: keyword + + - do: + indices.get_template: + name: test + flat_settings: true + + - match: {test.index_patterns: ["test-*"]} + - match: {test.settings: {index.number_of_shards: '1', index.number_of_replicas: '0'}} + - match: {test.mappings: {_doc: {properties: {field: {type: keyword}}}}} + +--- +"Put multiple template": + - do: + indices.put_template: + name: test + body: + index_patterns: [test-*, test2-*] + settings: + number_of_shards: 1 + number_of_replicas: 0 + mappings: + _doc: + properties: + field: + type: text + + - do: + indices.get_template: + name: test + flat_settings: true + + - match: {test.index_patterns: ["test-*", "test2-*"]} + - match: {test.settings: {index.number_of_shards: '1', index.number_of_replicas: '0'}} + - match: {test.mappings: {_doc: {properties: {field: {type: text}}}}} + +--- +"Put template with empty mappings": + - do: + indices.put_template: + name: test + body: + index_patterns: test-* + settings: + number_of_shards: 1 + number_of_replicas: 0 + mappings: {} + + - do: + indices.get_template: + name: test + flat_settings: true + + - match: {test.mappings: {}} diff --git a/server/src/main/java/org/elasticsearch/action/admin/indices/mapping/get/GetFieldMappingsResponse.java b/server/src/main/java/org/elasticsearch/action/admin/indices/mapping/get/GetFieldMappingsResponse.java index 44a66f497c846..2c07ebc68d0ef 100644 --- a/server/src/main/java/org/elasticsearch/action/admin/indices/mapping/get/GetFieldMappingsResponse.java +++ b/server/src/main/java/org/elasticsearch/action/admin/indices/mapping/get/GetFieldMappingsResponse.java @@ -34,6 +34,8 @@ import org.elasticsearch.common.xcontent.XContentParser; import org.elasticsearch.common.xcontent.XContentType; import org.elasticsearch.index.mapper.Mapper; +import org.elasticsearch.index.mapper.MapperService; +import org.elasticsearch.rest.BaseRestHandler; import java.io.IOException; import java.io.InputStream; @@ -112,19 +114,32 @@ public FieldMappingMetaData fieldMappings(String index, String type, String fiel @Override public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException { + boolean includeTypeName = params.paramAsBoolean(BaseRestHandler.INCLUDE_TYPE_NAME_PARAMETER, true); + builder.startObject(); for (Map.Entry>> indexEntry : mappings.entrySet()) { builder.startObject(indexEntry.getKey()); builder.startObject(MAPPINGS.getPreferredName()); - for (Map.Entry> typeEntry : indexEntry.getValue().entrySet()) { - builder.startObject(typeEntry.getKey()); - for (Map.Entry fieldEntry : typeEntry.getValue().entrySet()) { - builder.startObject(fieldEntry.getKey()); - fieldEntry.getValue().toXContent(builder, params); + + if (includeTypeName == false) { + Map mappings = null; + for (Map.Entry> typeEntry : indexEntry.getValue().entrySet()) { + if (typeEntry.getKey().equals(MapperService.DEFAULT_MAPPING) == false) { + assert mappings == null; + mappings = typeEntry.getValue(); + } + } + if (mappings != null) { + addFieldMappingsToBuilder(builder, params, mappings); + } + } else { + for (Map.Entry> typeEntry : indexEntry.getValue().entrySet()) { + builder.startObject(typeEntry.getKey()); + addFieldMappingsToBuilder(builder, params, typeEntry.getValue()); builder.endObject(); } - builder.endObject(); } + builder.endObject(); builder.endObject(); } @@ -132,6 +147,16 @@ public XContentBuilder toXContent(XContentBuilder builder, Params params) throws return builder; } + private void addFieldMappingsToBuilder(XContentBuilder builder, + Params params, + Map mappings) throws IOException { + for (Map.Entry fieldEntry : mappings.entrySet()) { + builder.startObject(fieldEntry.getKey()); + fieldEntry.getValue().toXContent(builder, params); + builder.endObject(); + } + } + public static GetFieldMappingsResponse fromXContent(XContentParser parser) throws IOException { ensureExpectedToken(XContentParser.Token.START_OBJECT, parser.nextToken(), parser::getTokenLocation); diff --git a/server/src/main/java/org/elasticsearch/cluster/metadata/IndexTemplateMetaData.java b/server/src/main/java/org/elasticsearch/cluster/metadata/IndexTemplateMetaData.java index f9ef5786afdca..4055af3e2f460 100644 --- a/server/src/main/java/org/elasticsearch/cluster/metadata/IndexTemplateMetaData.java +++ b/server/src/main/java/org/elasticsearch/cluster/metadata/IndexTemplateMetaData.java @@ -20,7 +20,6 @@ import com.carrotsearch.hppc.cursors.ObjectCursor; import com.carrotsearch.hppc.cursors.ObjectObjectCursor; - import org.apache.logging.log4j.LogManager; import org.elasticsearch.ElasticsearchParseException; import org.elasticsearch.Version; @@ -42,6 +41,8 @@ import org.elasticsearch.common.xcontent.XContentFactory; import org.elasticsearch.common.xcontent.XContentHelper; import org.elasticsearch.common.xcontent.XContentParser; +import org.elasticsearch.index.mapper.MapperService; +import org.elasticsearch.rest.BaseRestHandler; import java.io.IOException; import java.util.ArrayList; @@ -342,6 +343,8 @@ public static void toXContent(IndexTemplateMetaData indexTemplateMetaData, XCont public static void toInnerXContent(IndexTemplateMetaData indexTemplateMetaData, XContentBuilder builder, ToXContent.Params params) throws IOException { + boolean includeTypeName = params.paramAsBoolean(BaseRestHandler.INCLUDE_TYPE_NAME_PARAMETER, true); + builder.field("order", indexTemplateMetaData.order()); if (indexTemplateMetaData.version() != null) { builder.field("version", indexTemplateMetaData.version()); @@ -353,18 +356,35 @@ public static void toInnerXContent(IndexTemplateMetaData indexTemplateMetaData, builder.endObject(); if (params.paramAsBoolean("reduce_mappings", false)) { - builder.startObject("mappings"); - for (ObjectObjectCursor cursor : indexTemplateMetaData.mappings()) { - byte[] mappingSource = cursor.value.uncompressed(); - Map mapping = XContentHelper.convertToMap(new BytesArray(mappingSource), true).v2(); - if (mapping.size() == 1 && mapping.containsKey(cursor.key)) { - // the type name is the root value, reduce it - mapping = (Map) mapping.get(cursor.key); + // The parameter include_type_name is only ever used in the REST API, where reduce_mappings is + // always set to true. We therefore only check for include_type_name in this branch. + if (includeTypeName == false) { + Map documentMapping = null; + for (ObjectObjectCursor cursor : indexTemplateMetaData.mappings()) { + if (!cursor.key.equals(MapperService.DEFAULT_MAPPING)) { + assert documentMapping == null; + byte[] mappingSource = cursor.value.uncompressed(); + Map mapping = XContentHelper.convertToMap(new BytesArray(mappingSource), true).v2(); + documentMapping = reduceMapping(cursor.key, mapping); + } + } + + if (documentMapping != null) { + builder.field("mappings", documentMapping); + } else { + builder.startObject("mappings").endObject(); + } + } else { + builder.startObject("mappings"); + for (ObjectObjectCursor cursor : indexTemplateMetaData.mappings()) { + byte[] mappingSource = cursor.value.uncompressed(); + Map mapping = XContentHelper.convertToMap(new BytesArray(mappingSource), true).v2(); + mapping = reduceMapping(cursor.key, mapping); + builder.field(cursor.key); + builder.map(mapping); } - builder.field(cursor.key); - builder.map(mapping); + builder.endObject(); } - builder.endObject(); } else { builder.startArray("mappings"); for (ObjectObjectCursor cursor : indexTemplateMetaData.mappings()) { @@ -381,6 +401,16 @@ public static void toInnerXContent(IndexTemplateMetaData indexTemplateMetaData, builder.endObject(); } + @SuppressWarnings("unchecked") + private static Map reduceMapping(String type, Map mapping) { + if (mapping.size() == 1 && mapping.containsKey(type)) { + // the type name is the root value, reduce it + return (Map) mapping.get(type); + } else { + return mapping; + } + } + public static IndexTemplateMetaData fromXContent(XContentParser parser, String templateName) throws IOException { Builder builder = new Builder(templateName); diff --git a/server/src/main/java/org/elasticsearch/rest/BaseRestHandler.java b/server/src/main/java/org/elasticsearch/rest/BaseRestHandler.java index 963c8089f342b..97b4e29d9a208 100644 --- a/server/src/main/java/org/elasticsearch/rest/BaseRestHandler.java +++ b/server/src/main/java/org/elasticsearch/rest/BaseRestHandler.java @@ -60,7 +60,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. + * Note: Support for this parameter will be removed after the transition period to typeless APIs. */ public static final String INCLUDE_TYPE_NAME_PARAMETER = "include_type_name"; diff --git a/server/src/main/java/org/elasticsearch/rest/action/admin/indices/RestGetFieldMappingAction.java b/server/src/main/java/org/elasticsearch/rest/action/admin/indices/RestGetFieldMappingAction.java index c43f14dcddf26..f3a73fa29fd98 100644 --- a/server/src/main/java/org/elasticsearch/rest/action/admin/indices/RestGetFieldMappingAction.java +++ b/server/src/main/java/org/elasticsearch/rest/action/admin/indices/RestGetFieldMappingAction.java @@ -62,6 +62,13 @@ public RestChannelConsumer prepareRequest(final RestRequest request, final NodeC final String[] indices = Strings.splitStringByCommaToArray(request.param("index")); final String[] types = request.paramAsStringArrayOrEmptyIfAll("type"); final String[] fields = Strings.splitStringByCommaToArray(request.param("fields")); + + boolean includeTypeName = request.paramAsBoolean(INCLUDE_TYPE_NAME_PARAMETER, true); + if (includeTypeName == false && types.length > 0) { + throw new IllegalArgumentException("Cannot set include_type_name=false and specify" + + " types at the same time."); + } + GetFieldMappingsRequest getMappingsRequest = new GetFieldMappingsRequest(); getMappingsRequest.indices(indices).types(types).fields(fields).includeDefaults(request.paramAsBoolean("include_defaults", false)); getMappingsRequest.indicesOptions(IndicesOptions.fromRequest(request, getMappingsRequest.indicesOptions())); diff --git a/server/src/main/java/org/elasticsearch/rest/action/admin/indices/RestGetIndexTemplateAction.java b/server/src/main/java/org/elasticsearch/rest/action/admin/indices/RestGetIndexTemplateAction.java index 38c1cb76611f4..50370797aa6f2 100644 --- a/server/src/main/java/org/elasticsearch/rest/action/admin/indices/RestGetIndexTemplateAction.java +++ b/server/src/main/java/org/elasticsearch/rest/action/admin/indices/RestGetIndexTemplateAction.java @@ -24,6 +24,7 @@ import org.elasticsearch.client.node.NodeClient; import org.elasticsearch.common.Strings; import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.common.util.set.Sets; import org.elasticsearch.rest.BaseRestHandler; import org.elasticsearch.rest.RestController; import org.elasticsearch.rest.RestRequest; @@ -31,6 +32,7 @@ import org.elasticsearch.rest.action.RestToXContentListener; import java.io.IOException; +import java.util.Collections; import java.util.Set; import static org.elasticsearch.rest.RestRequest.Method.GET; @@ -43,6 +45,9 @@ */ public class RestGetIndexTemplateAction extends BaseRestHandler { + private static final Set RESPONSE_PARAMETERS = Collections.unmodifiableSet(Sets.union( + Collections.singleton(INCLUDE_TYPE_NAME_PARAMETER), Settings.FORMAT_PARAMS)); + public RestGetIndexTemplateAction(final Settings settings, final RestController controller) { super(settings); controller.registerHandler(GET, "/_template", this); @@ -79,7 +84,7 @@ protected RestStatus getStatus(final GetIndexTemplatesResponse response) { @Override protected Set responseParams() { - return Settings.FORMAT_PARAMS; + return RESPONSE_PARAMETERS; } } diff --git a/server/src/main/java/org/elasticsearch/rest/action/admin/indices/RestPutIndexTemplateAction.java b/server/src/main/java/org/elasticsearch/rest/action/admin/indices/RestPutIndexTemplateAction.java index 258bb05a7d66c..f5cc3c6aad26d 100644 --- a/server/src/main/java/org/elasticsearch/rest/action/admin/indices/RestPutIndexTemplateAction.java +++ b/server/src/main/java/org/elasticsearch/rest/action/admin/indices/RestPutIndexTemplateAction.java @@ -25,6 +25,8 @@ import org.elasticsearch.common.Strings; import org.elasticsearch.common.logging.DeprecationLogger; import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.common.xcontent.XContentHelper; +import org.elasticsearch.index.mapper.MapperService; import org.elasticsearch.rest.BaseRestHandler; import org.elasticsearch.rest.RestController; import org.elasticsearch.rest.RestRequest; @@ -33,6 +35,8 @@ import java.io.IOException; import java.util.Arrays; import java.util.Collections; +import java.util.HashMap; +import java.util.Map; public class RestPutIndexTemplateAction extends BaseRestHandler { @@ -63,8 +67,23 @@ public RestChannelConsumer prepareRequest(final RestRequest request, final NodeC putRequest.masterNodeTimeout(request.paramAsTime("master_timeout", putRequest.masterNodeTimeout())); putRequest.create(request.paramAsBoolean("create", false)); putRequest.cause(request.param("cause", "")); - putRequest.source(request.requiredContent(), request.getXContentType()); + + boolean includeTypeName = request.paramAsBoolean(INCLUDE_TYPE_NAME_PARAMETER, true); + Map sourceAsMap = prepareRequestSource(request, includeTypeName); + putRequest.source(sourceAsMap); + return channel -> client.admin().indices().putTemplate(putRequest, new RestToXContentListener<>(channel)); } + Map prepareRequestSource(RestRequest request, boolean includeTypeName) { + Map sourceAsMap = XContentHelper.convertToMap(request.requiredContent(), false, + request.getXContentType()).v2(); + if (includeTypeName == false && sourceAsMap.containsKey("mappings")) { + Map newSourceAsMap = new HashMap<>(sourceAsMap); + newSourceAsMap.put("mappings", Collections.singletonMap(MapperService.SINGLE_MAPPING_NAME, sourceAsMap.get("mappings"))); + return newSourceAsMap; + } else { + return sourceAsMap; + } + } } diff --git a/server/src/test/java/org/elasticsearch/rest/action/admin/indices/RestGetFieldMappingActionTests.java b/server/src/test/java/org/elasticsearch/rest/action/admin/indices/RestGetFieldMappingActionTests.java new file mode 100644 index 0000000000000..dcf4237ae07bf --- /dev/null +++ b/server/src/test/java/org/elasticsearch/rest/action/admin/indices/RestGetFieldMappingActionTests.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.common.settings.Settings; +import org.elasticsearch.common.util.concurrent.ThreadContext; +import org.elasticsearch.rest.RestRequest; +import org.elasticsearch.rest.RestStatus; +import org.elasticsearch.test.rest.FakeRestChannel; +import org.elasticsearch.test.rest.FakeRestRequest; +import org.elasticsearch.test.rest.RestActionTestCase; +import org.junit.Before; + +import java.util.HashMap; +import java.util.Map; + +import static org.elasticsearch.rest.BaseRestHandler.INCLUDE_TYPE_NAME_PARAMETER; + +public class RestGetFieldMappingActionTests extends RestActionTestCase { + + @Before + public void setUpAction() { + new RestGetFieldMappingAction(Settings.EMPTY, controller()); + } + + public void testTypeInPath() { + // Test that specifying a type while setting include_type_name to false + // results in an illegal argument exception. + Map params = new HashMap<>(); + params.put(INCLUDE_TYPE_NAME_PARAMETER, "false"); + RestRequest request = new FakeRestRequest.Builder(xContentRegistry()) + .withMethod(RestRequest.Method.GET) + .withPath("some_index/some_type/_mapping/field/some_field") + .withParams(params) + .build(); + + FakeRestChannel channel = new FakeRestChannel(request, false, 1); + ThreadContext threadContext = new ThreadContext(Settings.EMPTY); + controller().dispatchRequest(request, channel, threadContext); + + assertEquals(1, channel.errors().get()); + assertEquals(RestStatus.BAD_REQUEST, channel.capturedResponse().status()); + } +} diff --git a/server/src/test/java/org/elasticsearch/rest/action/admin/indices/RestPutIndexTemplateActionTests.java b/server/src/test/java/org/elasticsearch/rest/action/admin/indices/RestPutIndexTemplateActionTests.java new file mode 100644 index 0000000000000..ac0eb8f0d81a6 --- /dev/null +++ b/server/src/test/java/org/elasticsearch/rest/action/admin/indices/RestPutIndexTemplateActionTests.java @@ -0,0 +1,81 @@ +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch licenses this file to you under + * the Apache License, Version 2.0 (the "License"); you may + * not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.elasticsearch.rest.action.admin.indices; + +import org.elasticsearch.common.bytes.BytesReference; +import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.common.xcontent.XContentBuilder; +import org.elasticsearch.common.xcontent.XContentFactory; +import org.elasticsearch.common.xcontent.XContentHelper; +import org.elasticsearch.common.xcontent.XContentType; +import org.elasticsearch.rest.RestRequest; +import org.elasticsearch.test.rest.FakeRestRequest; +import org.elasticsearch.test.rest.RestActionTestCase; +import org.junit.Before; + +import java.io.IOException; +import java.util.Map; + +public class RestPutIndexTemplateActionTests extends RestActionTestCase { + private RestPutIndexTemplateAction action; + + @Before + public void setUpAction() { + action = new RestPutIndexTemplateAction(Settings.EMPTY, controller()); + } + + public void testPrepareTypelessRequest() throws IOException { + XContentBuilder content = XContentFactory.jsonBuilder().startObject() + .startObject("mappings") + .startObject("properties") + .startObject("field").field("type", "keyword").endObject() + .endObject() + .endObject() + .startObject("aliases") + .startObject("read_alias").endObject() + .endObject() + .endObject(); + + RestRequest request = new FakeRestRequest.Builder(xContentRegistry()) + .withMethod(RestRequest.Method.PUT) + .withPath("/_template/_some_template") + .withContent(BytesReference.bytes(content), XContentType.JSON) + .build(); + boolean includeTypeName = false; + Map source = action.prepareRequestSource(request, includeTypeName); + + XContentBuilder expectedContent = XContentFactory.jsonBuilder().startObject() + .startObject("mappings") + .startObject("_doc") + .startObject("properties") + .startObject("field").field("type", "keyword").endObject() + .endObject() + .endObject() + .endObject() + .startObject("aliases") + .startObject("read_alias").endObject() + .endObject() + .endObject(); + Map expectedContentAsMap = XContentHelper.convertToMap( + BytesReference.bytes(expectedContent), true, expectedContent.contentType()).v2(); + + assertEquals(expectedContentAsMap, source); + } +} From b65006e8cd558b5e1a91ce9f185dcda5f6135769 Mon Sep 17 00:00:00 2001 From: David Roberts Date: Thu, 10 Jan 2019 17:28:00 +0000 Subject: [PATCH 083/186] [ML] Fix ML memory tracker for old jobs (#37311) Jobs created in version 6.1 or earlier can have a null model_memory_limit. If these are parsed from cluster state following a full cluster restart then we replace the null with 4096mb to make the meaning explicit. But if such jobs are streamed from an old node in a mixed version cluster this does not happen. Therefore we need to account for the possibility of a null model_memory_limit in the ML memory tracker. --- .../core/ml/job/config/AnalysisLimits.java | 2 +- .../xpack/ml/process/MlMemoryTracker.java | 18 ++++++++++-------- .../xpack/ml/process/MlMemoryTrackerTests.java | 9 ++++++--- 3 files changed, 17 insertions(+), 12 deletions(-) diff --git a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/ml/job/config/AnalysisLimits.java b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/ml/job/config/AnalysisLimits.java index 797df5892f82f..1e114ee0f7a46 100644 --- a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/ml/job/config/AnalysisLimits.java +++ b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/ml/job/config/AnalysisLimits.java @@ -36,7 +36,7 @@ public class AnalysisLimits implements ToXContentObject, Writeable { * the old default value should be used. From 6.3 onwards, the value will always be explicit. */ public static final long DEFAULT_MODEL_MEMORY_LIMIT_MB = 1024L; - static final long PRE_6_1_DEFAULT_MODEL_MEMORY_LIMIT_MB = 4096L; + public static final long PRE_6_1_DEFAULT_MODEL_MEMORY_LIMIT_MB = 4096L; public static final long DEFAULT_CATEGORIZATION_EXAMPLES_LIMIT = 4; diff --git a/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/process/MlMemoryTracker.java b/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/process/MlMemoryTracker.java index 54a7400375fe9..441317bcbe207 100644 --- a/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/process/MlMemoryTracker.java +++ b/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/process/MlMemoryTracker.java @@ -20,6 +20,7 @@ import org.elasticsearch.threadpool.ThreadPool; import org.elasticsearch.xpack.core.ml.MlTasks; import org.elasticsearch.xpack.core.ml.action.OpenJobAction; +import org.elasticsearch.xpack.core.ml.job.config.AnalysisLimits; import org.elasticsearch.xpack.core.ml.job.config.Job; import org.elasticsearch.xpack.ml.MachineLearning; import org.elasticsearch.xpack.ml.job.JobManager; @@ -269,15 +270,16 @@ public void refreshJobMemory(String jobId, ActionListener listener) { private void setJobMemoryToLimit(String jobId, ActionListener listener) { jobManager.getJob(jobId, ActionListener.wrap(job -> { - Long memoryLimitMb = job.getAnalysisLimits().getModelMemoryLimit(); - if (memoryLimitMb != null) { - Long memoryRequirementBytes = ByteSizeUnit.MB.toBytes(memoryLimitMb) + Job.PROCESS_MEMORY_OVERHEAD.getBytes(); - memoryRequirementByJob.put(jobId, memoryRequirementBytes); - listener.onResponse(memoryRequirementBytes); - } else { - memoryRequirementByJob.remove(jobId); - listener.onResponse(null); + Long memoryLimitMb = (job.getAnalysisLimits() != null) ? job.getAnalysisLimits().getModelMemoryLimit() : null; + // Although recent versions of the code enforce a non-null model_memory_limit + // when parsing, the job could have been streamed from an older version node in + // a mixed version cluster + if (memoryLimitMb == null) { + memoryLimitMb = AnalysisLimits.PRE_6_1_DEFAULT_MODEL_MEMORY_LIMIT_MB; } + Long memoryRequirementBytes = ByteSizeUnit.MB.toBytes(memoryLimitMb) + Job.PROCESS_MEMORY_OVERHEAD.getBytes(); + memoryRequirementByJob.put(jobId, memoryRequirementBytes); + listener.onResponse(memoryRequirementBytes); }, e -> { if (e instanceof ResourceNotFoundException) { // TODO: does this also happen if the .ml-config index exists but is unavailable? diff --git a/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/process/MlMemoryTrackerTests.java b/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/process/MlMemoryTrackerTests.java index 197fa469bed7c..3e54994ac043b 100644 --- a/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/process/MlMemoryTrackerTests.java +++ b/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/process/MlMemoryTrackerTests.java @@ -123,9 +123,10 @@ public void testRefreshOne() { return null; }).when(jobResultsProvider).getEstablishedMemoryUsage(eq(jobId), any(), any(), any(Consumer.class), any()); - long modelMemoryLimitMb = 2; + boolean simulateVeryOldJob = randomBoolean(); + long recentJobModelMemoryLimitMb = 2; Job job = mock(Job.class); - when(job.getAnalysisLimits()).thenReturn(new AnalysisLimits(modelMemoryLimitMb, 4L)); + when(job.getAnalysisLimits()).thenReturn(simulateVeryOldJob ? null : new AnalysisLimits(recentJobModelMemoryLimitMb, 4L)); doAnswer(invocation -> { @SuppressWarnings("unchecked") ActionListener listener = (ActionListener) invocation.getArguments()[1]; @@ -141,7 +142,9 @@ public void testRefreshOne() { assertEquals(Long.valueOf(modelBytes + Job.PROCESS_MEMORY_OVERHEAD.getBytes()), memoryTracker.getJobMemoryRequirement(jobId)); } else { - assertEquals(Long.valueOf(ByteSizeUnit.MB.toBytes(modelMemoryLimitMb) + Job.PROCESS_MEMORY_OVERHEAD.getBytes()), + long expectedModelMemoryLimit = + simulateVeryOldJob ? AnalysisLimits.PRE_6_1_DEFAULT_MODEL_MEMORY_LIMIT_MB : recentJobModelMemoryLimitMb; + assertEquals(Long.valueOf(ByteSizeUnit.MB.toBytes(expectedModelMemoryLimit) + Job.PROCESS_MEMORY_OVERHEAD.getBytes()), memoryTracker.getJobMemoryRequirement(jobId)); } } else { From a57571045ee404468ea5b311033b8b8aae4b89dd Mon Sep 17 00:00:00 2001 From: Michael Basnight Date: Thu, 10 Jan 2019 12:00:51 -0600 Subject: [PATCH 084/186] Fix rest reindex test for IPv4 addresses (#37310) Some of our CI boxes end up giving out an IPv4 address for this test. This commit allows both v4 and v6 addresses to be used. --- client/rest-high-level/build.gradle | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/client/rest-high-level/build.gradle b/client/rest-high-level/build.gradle index 9b9074b3d9b5e..ed9b4451db350 100644 --- a/client/rest-high-level/build.gradle +++ b/client/rest-high-level/build.gradle @@ -100,7 +100,7 @@ integTestRunner { integTestCluster { systemProperty 'es.scripting.update.ctx_in_params', 'false' - setting 'reindex.remote.whitelist', '"[::1]:*"' + setting 'reindex.remote.whitelist', ['"[::1]:*"', '"127.0.0.1:*"'] setting 'xpack.license.self_generated.type', 'trial' setting 'xpack.security.enabled', 'true' setting 'xpack.security.authc.token.enabled', 'true' From 2d5a8ec59db8e4018615572e7150658b9c37e718 Mon Sep 17 00:00:00 2001 From: lcawl Date: Thu, 10 Jan 2019 11:10:25 -0800 Subject: [PATCH 085/186] [DOCS] Remove unused screenshots --- docs/reference/ml/images/ml-data-dates.jpg | Bin 17402 -> 0 bytes docs/reference/ml/images/ml-data-keywords.jpg | Bin 17540 -> 0 bytes docs/reference/ml/images/ml-data-metrics.jpg | Bin 358164 -> 0 bytes .../reference/ml/images/ml-data-topmetrics.jpg | Bin 101513 -> 0 bytes docs/reference/ml/images/ml-start-feed.jpg | Bin 1346 -> 0 bytes 5 files changed, 0 insertions(+), 0 deletions(-) delete mode 100644 docs/reference/ml/images/ml-data-dates.jpg delete mode 100644 docs/reference/ml/images/ml-data-keywords.jpg delete mode 100644 docs/reference/ml/images/ml-data-metrics.jpg delete mode 100644 docs/reference/ml/images/ml-data-topmetrics.jpg delete mode 100644 docs/reference/ml/images/ml-start-feed.jpg diff --git a/docs/reference/ml/images/ml-data-dates.jpg b/docs/reference/ml/images/ml-data-dates.jpg deleted file mode 100644 index e00b765402aaeff7638ad4a8414d53d2d253e246..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 17402 zcmeIZcUV)~wV6L-Zh;gYIKez&9!{`lmt`cSK?--Y^~ire4QPMd3;uC?ewaIMA`$e3 zeC-AGn@!xEAmUdL6BVeA#EXpN3W$`6gp7%V*a6}N)}$c$qx~@rxR8*NkyD(bq@t#w z1qM`~2a%GHk&%*Xl3%?}Nlkn6HvL0xUjD~|!cRqCDyyn%YU}D7+L0ZdUESY$z7Gx! zkBp9uPfViF3yVw3E30ek*xkMTgTo`-@yTzz0DArpw1DgX!0a!0F#&jyl9Q8>Q~t(_ zgw*FZ;!NZeeAmt~-!`Cp?9OuKdN9?6J4rbeZPffSh8R}c=L0ls0~GZm!0ewP z7V^J^*?%JTKk=FZsewrUXk?_MWE5m%WEAHpfOd}R-0wz3L-j|a`F|VTAC2L6WBkuX z1Rg>HJcFE^oD%pyPfJaE{=eObGXPs&Bu;|p$VdQYB4Yx9L1%~8ZjlIHAQ_%_x9Y_; ze#KTK=(*fCc^E?f<8tW!jE&`~_WIZf1Zk>EZNXb>tqv1?81m%NkKUy5EzqeuB=aIq z3NA4!OR`_EtAv%429_c>Rde*k#71-T5;e+GCV;DRqy$|ys10!|e|V5fVBsZ2{PNHv zg1){%;EKSgYZ&1(O4&I)btDlKQ{EH6s3WZ+HjhBtx+C_fdGXXlP}1QXmIO1m&mx7V zK)gK}AcCk9gTbpeKN3On_rQ2#Ud$UD3n4TeMj%HLL7k?HEtL5ed0rzrI?zPWDl-_z zLIhnVf}$8Bh@kgx5%XgB{?SC-y#V0RxBKLYAkxP~5W0{E3hVxwy?<#4ff3>*Tx}J^ zsEB;s&m{;6vlBtLk^p$Kk16rD0_4xgdx;qW#Id{s{xL701Fz1hH!3g;dFb zC6+UYpkQet2y1*S6}+ZFna_*8rqFsqo=5<`{RT#pp;}jkAUGN#Nai2XLxc#5j_xFa zehDBj!i3vI(0N{Z0Ql|zjt4X5X=ui2i|4Gy}-dv zsM83ftv}A3{(}6!P^WkoLN*x3s002l+wv08L5W2Cy4Q+fB{**U3-U&Pp$;S&$MgD; z9{?N+ODsSoEnxTb@u7svs$7H+dm`vX`5xHfFYNGFZo#JkJ3=;?h z4tn@E9Afs@8N8U60bn91CuWEUA_IDC`S(9W_usQ>@xM&VB``ArlD~h=0x|E0z+*a5ghkAGJ6lMS z=wPML@QaTUtB`Sif$MoUh@ePhIF{lK*?ac2?*C0*vmEtqP|YQsfgWirv!9k8;U6BHW}G;C zR(Y3C@{U6qHLCtGO*{oJz8ZBIblg-Bn<%#Sn?8MV$}@C2NCeGw6G5EdiY;)n)EORu zE!D9y-513f5PzMyo)r>7i|;Inpm7bxGj_qCK+{#JEkOJQ&~t=#X~b&1*xq_;tg<&? zEB1&Wa>T=vmA1rV=5lQBK8?tSR!*=f@bcoXU4<4k7s8Lt%Jn=+nLG4O%xiapmASvV zr1sum$m7uhRzXE<@G&IV=TH(@STQB_bsDwfS&^HX)|YSjDOubduIevMK0rzkKA2l>qdS$OsZOVtPZ;1({j@=?162s(LR}uHNpr$`5*6;~MRC ztr|L3wpOSxIkc$5rq}QFV+TGlBj#MKOayhe&^Ka)qm~AQ-&o%0hb z9$vCb6;Es1=ja~0VTuar7fM_R8;bBcH;}g*a<<3&CJS!mk(rgUhbu{=*(-XZgFwc< z`r;F`Y4z^m==NHtivPMl{OR2JFHyEd2&VjE6N{?X=|g%(=|eT))j0S0evuA>^vv}N zjk;L<%uUwr2X2%X`>{{BH<%g22-0;}o?O(Mt)u>rUyr1&o`c&+XZ*?)xw)rFrIe+_ zc_QH@&`>vgE(5L0g`i#MWcob#RBuwF6`FjW3SPM@{b<6G`)i*a9ASdM=~yBgG~UC6 z1_Ssd^7>u)oE96SwYg~C-AwczdE9ShI%``eFYZymp_w*Y+c4Mc=2~B%RA4Vs4fFBZ z5eu-E6#KzXlWf>%(At#NSaTRBKGr_e+FUs>w|s>8T7}upEnIJPt{f3};|sIsak)~~ zqp>xpb&D$S!mhtDKZ=n}qU5OW!)&ZS?BENRFKGeCal)*^%9>5m6V+UIj7O)3^A|md zUT43c!_2pNV9_q!y}{;>2o_u@fm7+;udW>Fp=1%;Qw5Ku3;z1og7q7;pS()~Xg+Rp zzHRJ~!SJ=wj7YIq<8aE|cV#Ef#U{sAQ&(DJ(E--fCULfTs4ao*{VIOFYA(q#x3T8h zu_udw1kuA|bt@tbhpQShEOq02J#=aFT(jOdfIOx~Q-_c?S6YVhN5wrGi(@Z(w7^l` z(e7j7B$s^l%byN3sh8mr+gn*)S+}Ggvyo@AnM!`T(%*IqZUw|a+!TIUwPb<_Igm_qXwrxv7VDX}>VIL~dUGcILEKhp^-<~W^IEkSUc5qdax^w~qT;TuzWir$I(=?a{DbP?v^ctZ7FH<)$2>8jys;-t`;T@{yAuPF2~O`kuK)t810$U zGG~by=$Ns8H%#U$V5ezt*(NdO`^{Ijw-u-v25xN^nHCB#RG%z0p^vBizbVnJ z%+8h<&N<1oUM$-;Pyhb7&_V>B&}ASVrb@M{C7iVGjQZ+nOf_11Q{(%6U5)*Z>(J5$ zi@pQBsmJ>9R~x?M>oED{vlm$m8yyUuHqJ=Ki){zYX3=89(stLNIkVM^SGmz_yD(#w zi*e5w6jU~@&S{Qp7o<;)cKLp+MH^{lW?jEO_x6^c??M*dvXi zM3AAhM`4am-u_~1LpgLX#cV>jSVy_+wASmfKENCEOD6AvW~#lLu4-hoqzm_XO-Q%} zAc~z7XM?ZaE`uI%1|*j)nlCu@G8)v;jq3St&HQW(#9ZG+uqzKK`&Fsk_y5vmV|3ic zL9ng+>F;=B-WdCqU`!{W6V%gU-l+}o@?Rxv@frEe2yg#u#1YD%#&_E*S(F9`usqvm(pv0*-qUUxC5%l?K>xtae({eh% ziXlP)D|eeN&FfMi{8G0_N^~HgVv8W#@i-BXi{1SXyLF)};iDSz1$q{ zcmua5jstPAC?ZrlY{Zzt!Ygbm{Wxxl|IpJ50)dvjgLQ6o9+noulCcy zSFZN%R!4T3EM?1hS7jy4ywdC*HXR3JdQ7wTWdLjX0`S8U04Ihn#TW25Wf3?k1a2$H z5r{Srz@DuFVUQ3oLLdmZFg*kUs`uIWR0KX6d%$|gh`(;Uaz+PCp64Ti7OD_PK^w%c zK3)PTFF{EYf1AJ$42R&;dGSEhVDt^VDog|kW)neI0s_?o0gC)-N#1{f=+6M~Vp*>< zo=_7(JpgRqvUv?4kBFc@FZdT|I(X6G6AEBL4$vv_>i0qkA&6abB4}6mACQ0-ONmP) z1S58x5WBbkPvRa_@~!Pxiv8sf1}grXbB)?90`QXseMKt*UyHwCqYlSZ%D(VT{_*q^ z83M=EJwgN^ADHt%Tsa9wC6|L&9TI^!6bJ>l%Szs@qvJzgg%#$j$#hEUCiNV}}J;_Qn1PdIXc zph6AsC4i;=#8(x$C@GeQ>7VrJhEqR%YLh)J_~ByOLEw33B^;%v!s(uwP0)eI_jkH% zL@l|awiuLpMpCwu+mwgu9%X4i2*ld>=B-a=g$J_b=UQvq`%q4&3SC9=Q)b^$OgK15 z{CC`Ajca9q*EB5{=AT>Q&X#3u+D{(q_eiAgp1!7WJzTLz)UDoOXQ{+{cl_A@Ar=x{ z{mSR8_*1{9)QrG8rOYqNYR)nYVTBhjxf{qwE8m%H#|2l5g%%0PO&`av&=-Gxe1d6W zbUw-W8w&g49(O)ZQ!eajoFxn@6;}~dhF}o+=}8{`XTzrA&YfbG^Qn_!0ZQhAyVi-X zwhmnH4~jMVTYj>QIrE;rT0whtMZq1fro3l43GQIz!UcU@^OLOga)kCUb@S2GZ`7W) zMu@FR{rf%0;zII>eHbnrcx_=am}inPrM(ptQjV6Q8@YeOxl4PaMBrejME{*^c~sEN z0SUW`L9eMsOJDQMBmsr#6>;Rv%gN8R+WK#(X<#E`0#tn55g(ju*?i;?8=053UkTp~ zSaI(W7EV9$|M#RQ1*5sJUh`+K&-x>el8ypZ&Q$M$&nrM?&lUr>h#(({z`&Ux$G*eX z^8p_A1*)&0WwUBfewmA-hUcu7KKsSJN+xM5@oiEsw<_JWa%TC~kXw>=J5S*L>hn_J zqbL>Xav&%4C+^Pu3p_Dd99X--xYeG#l5#^OS6HHp603fP>c!CK@n#v#ead}qp{LVNKTP%VG!%+1)ifR6zF%4d?S;8K zqpfOretokyG-dMrqm^oZ{eQ=nmXDMb&`l~3SAB*Z7Z(L0NG#^mKX0ay;|UxPor()H zqGKkzVXbeq&$3Fzj#MFIcyl!5LeGV)E5j?poS?fwF59$Ob=>Ft-+5Ra{GAFMW8-g6 z5H6KBEaU#J@4a$vuJ7As4%(4nFeLoyx~S)aWXSWsSHpmP`jZ~y1m6Gus`&q$88RK5 z`_hen&9!rS4oDE@+Lo37g>m;G;RuQXoIQGMcxfc<0lSHZdiCRLt{q8o9?h*OfsUlO zCt34*0dm4jkxOCr-pB7ZTMVBbXUSk$ggWivaVj)N?{e+x?aOCN0=d;M1l>3bL5T4Z z2hY}q5o2bEv2MbU-%v@P=9yRISpZ^7zElw;@*(EWy_^KMfu-$IWsg+HL?WAoHVmg&z6?IUj5L9VgMI)W#vJ{*^zQ5m)>Z9I$G+ivQzDaeiD`vWfy(G5;~gy`8I(O#pm=!^(V z>0Gqu@*scm5VM7{L3Pz9fl8JO@ru3=L#~r8J*N;Of*@n>x6TkY1yjSOU5W)jjx-KJ zJ7qRV5Nb&(w$Ve;KkR7mJPE8C9YFfzU*XtX(oBCSfM|~EzAU0ecabdAe3EyI7A^KW z=)~m)F7VRg*7X^k+HueZ0V$YZbFYX!yX2TD<@cU8*VwNDJR>G^y#&T%YEK}|5KM5K z+BfaZqQSjg*rG(Mmm{-dIa?x&5l>fTTt;gftYA(xJuU)JWe=Yl9>rw7_Ct%9l&dFDM<;tbqalq5b zMj>`>Ti0-apOfun0FlM9bj`8J!lEr;NJQ*IsNd@>{(*v#l5NjMUssb^|C?8IGc>jB z(|+6%)=04Wr0}SBuqsG|@Jp2&Be-3UilwDDH64|8m0o8r|7cddsAy*>cFb%ot?VXZ zb|uoa$gFG@syzWY86sRPlytN|gAgvAu*Dq9xARuT=hq_VK=M%~Uf>J3?haM1$}uTd zPhG|;-AmW+w%GrO$~TmSQX4(+xki5Fo$2F3!eBPT!e`9VTm;{6leIg#_~s2RE)~w6 zZMXP$cg0`n#DBh9xX8!M>I$UjDS}z^=F!vs(6hk)Z9%2t?Ar!52Cmpw?d)toO2D(R z7DhUW^eO6=r4Wb~u4Mn7qur|zh3_4f?;WknL>y*fc%pq+!^_f2H+&48YG(Q1`!g!r zbc*)BY%9M#FKYC@^wEb1^0G{i%&jD}7PEfeLTsFj@M;V|RN*p*sW{(-co*Hls*OOy zuCWfbH1>x3xrir%r3twdMUihy7hh2>skI)(WTY)yB{*Tqx5O}~oeh~~*-&WFcw4ch z)htxLwnmELV=2Xlos`W*Hwg7!;j{2p%MVP`c``yw$4?kyjw{>2)xt<3h+}$H`yoGi^5>XLP+Vu;8&X5|g-Pp6g;mK7@yq!A&iPwh!WU z^knouCk*wapUi7NW0m+hUt*%<4b^2ey2+tz$J$@KTCNOkTCPg$Zg2NVidvoBo`Y!gXWkklCb7H?fk8jB9#@&JWrO*ArzE zQcnU)hX;I32HG$7+-OU9lwW)GoXL|BqfFN`XC+<-1Bep-E;hTP7&x`z^l8@*uT^wQ zfy??XY?4r)$3#e-((?ZKcC|;>xi>Fj}1DSK2%r%sN}M zW%!^mcHF)tgkuwCdH+wHCyb6IOK1pj6lam(J_EvEd%cR=}WqAa=Vw;(H~5DS|#@H~)76#KAq z7F00(+O(Y)Gc>sq!Am&b%CTX=G=?7C+R7jCuCB@CNq)Ko+jg^&d;87vRI4VATGf|> z>*6%!FB2ffy+PF?YmQ^soScCpIyw2BC%L!k1+#QD)f=PV$cVql8_;DDx#UKvaIQFI z>&(^NLkw~<+g&rD&dUF%2=hl8L5*$hpbK_Jqc~L3gIK%<$cmK%kgm>sPp=51HNnK8 z|Aq?gyo7NOgNZHdz zvS5b3HV_)i1B@_3JyVVt(`i7Jb%-zaCwwfaRN(;i!pF_WJdXbxn zQc)nhD#O4~wPE?5wiBySuxXWv&_v6_cTKGAl*2 zaLg-h92Cpg0cMdTT*a**(-~4-m6d#Qrl4|Z`tXL}#^jrpTeXa?%u%8aTfgAeDgQ6k8$)WvK}`N@S@`4PAM#Pe_y3Fh(He!8+WQ9)!* zsz@pNAsxj6Jmj-dY-pF*Hm4|f14{(SiTMy}h#>xD1SyUmJzI%Yx`#``x^WBZsd5bn z4L;qfDt_x0-r2eMvUy~x>N9dF=lqrK^K=?!l{{knfJtNsl7^}(W0#vTqx8InpKQ{t_5+7i0`;9SH?@qe%_-0!IHrHYdi0i<8 zL)Sg8F>Et{^4_a+Ro}nA)YXXE`#wTpGRrBqBn0xMZcBHDxkb7 zEwjJHdR-fc7DnTPKE~ui#tM%47Xy{%g`@i;gXrN1qxu>njB2EBzH|SDIBow@akK5( z@AUJk_bp+U7OI~?sK5@H{$ES;pcZqX`)qZXTcn3P`%KViz~Z|}*uW)$#_VZdJ@c+T z<`u!g=DQO-%Om7~v&jSVs?%0R7eAp&w#LG)!TiDXhLU(uxz9>3rImb4M1B!gKnFAda6*j`^8?q%$6!06@7FrbT)T*XY6|%Z$2e~SQ;23 zHg<@hYf>8+{AK{)iPCi!{4Klvk~Oxb6bm&_&ohmOzv%g#8}2SzdyhyQ`05M!~y3P%Qr+1>!NtkT*OXwH#bNg)5K*VZ`49) z2t%`+V+FX%c^LalYzn42HRF?ckFp{!UckTtV$~$bV|kh5wKj7Db;`x^83hJC;pT{> zL&=H-DJr=4ARus%F>%jc%OAeXGuG;K&lgGMWIofs)V$-YXWH4q`m3~awzgyfin%WS zK{qxEt;Th}s4erwq&A2r?4{VAo2!%|COZPpj#=hbJu<|kv_uwKnK`Eg(Z4brc5G;D ze2H)nEdFFk)uB!kZLB6VkV4}=5x|VoY4}--aa%xx!|m#MxNvZc>u@Qprc1z>+{RMF z_k0ukIB8?$;A9o&)I&l85tN3$mzj33L_h=|=trntV`Q08hdJi-Ewx5f>oYceKyz0h z9DM8R9&N_;@+Fe+o7zuPAK&I5EWw+jTs18kFtgbu=2q*YYVvIERvJgriRM>KB%A$% zA>T2^F1<6=YPp6v z>koHRw$GpG{sc`u+?_s7{rM%r>~$r@RqySKT{3MOkhdq`FeyO{Z$c0&oDt~(FF-zN zOAmn_O-4ot4Bm)U$^%(K2ld3IOE+3{BkwZsNdGX8x^W~K>;E1XJ8$!zKs)F-w6#1~ zhSt3)d+d1(WUcu^r6%$d%YP zE}5jbmp(7Xc!EZ5MXd2EoX!OerU{V11;a{kGAIkDF#{Y|cMT5@maS2&qlY_lHrj=G z*8DrwvBjZQ>Gc9~w2k73H52~vd%Ob98gL1jan3H>SCzPJmt7_Tt^wQ0jcxWy=(#MS zUXdj%v8+}1h;)%Iy?%9{P}&OAB3yr5;)ZYKZrN^W(Ol!n1;S?P)kED+&L=bR@wpws z<^}aJB9m8cF;hB#K8fev?9w`58C;lLjDlla3HYVW{daCbMPmPMgt)%t}kgmSUj|xp5k1!7LZiP!M5EeM)JRT53 zbqvF)KIWAXPA*X~J6v?TM!meey8c}(Q<;vI#qUw$+v!iNdCa&&g&q)7^j z2%>;mgiV%5=Or_4S-M_dc~~=e`Id6;Tia@Fjzy9RIx~{FR6gUv!cu>q+9iT*%-9m5 zFDN22vil=eIIIeG0nXB>P4BG9l^Awiw$gr%nL7S{#H=&O{Eb=d-7{(Ba#TUP<&3jb z$O(IVlb|~|VxRj-Cp(*gvp1gun~BTqNO7q9gpSeqoI+2)h26s*oYK)xHTnpKp>D^+99 zYGMOFWi%?eR1Ix1IW~GcFEo)Q|Bx(?|DjU|s<=#pV?mEa$f}6r{V@G8ExZt1s1?@! z3nQJLV{R|vQccoY64I5T39B(M#bp_m^j$_XF2Z_(^-yppuDs;}q5lLH!-YIaXh!5Y zjfyl_$2a!+-e`+a=m@g-WRJ{#x21G}`F5A6*z=oUsV?9s1%n?+iBGR;~>&;|`v$80TpGYqNBxjN-To-r;k9jb^91?j5CTT$?4OQ~$^3E!3q) zVNE3H88xfV4WXL^eyj>6@|%b9XKoF&kyCEh${q`aTxcu5+jFrPoNIAyJ2;%PysI^4 z)YJeT+nxSSG36}J;5f})HCNm_-7AEj^=57OhqzDt^i2nSB8?l1Ek{Eqo08e`I0Phm z8;EID1=hfH%HVUAg%i{3K8oSfoPpu-$zs#9Hq$lZIdMu?LY|qweRCD?VJLhT!BvT2 zrA=bxE-eCs)e>ALm>E2$@p80)w%muYvB&gza%;2=dWT6@-n@lC$rs~GqYzGw+%{rT zWbi4yA-8HLWe&8Zp5e*lccHA+iHnZasahZIHAP4%YZ>3REqX?cZZX6Kqaf!4bcX|U zu_04xV~;RJxJG2Hjd2q@n_uVX+dT6=S1&IZEUcb(*~`QV>_p*=>(~h zd&NrlmZP%l5JI}@F=j5ov2|d`2wRpY6=IjQCLytPpqrt8-SM^5Yo_<apS*1j*Q4xipp> zpcqKgZXFb;i&P|3V%8DNtI;U`K$D;A+vrhznNU-V()5fP^uno`w(vCH#4QVSP6bw? zHTcR|hGygq2P`LU+B&1PLhtOIYy}XExtz6bWdy`$>GxD3-Yy4Z_>XAxm8Y!{5arM9 zgEW8;6Np6@Tlv?>e?vJdNazPD_YlTBoQeaORV;M^@8(`nv~so>k7M@I@mxI%0Oi}? zP&zKO#?4;B1h!-{B$2m>kKV}C(Yw-ne=v)~A^GqER9Mp|=}FYBs6Nr8h_5#ngCRV$ zyvzkJh7MgJ49?vhPXKv#IfaaWB>d!r9L^oKP7x+*zfCL#WXK<9Irh{x`+;NJWcBzv zXPI&NDBg4HCKkNm0T0V9T`H!a&n1xkg_c#!&QKV0RoeNgHXogs&Xn3mFP#0PHfoPs z*RJ(ytIqvKt2xLq4@iOP!u??MYe$oa$ZE@0W@p8w6H+)_lV+XzlSNJ{e9__ct%~mR zd-qx%>xG-Kyc8R1;SA6kY!O;DRjdxr{F&zSaaO;;nT@^ZlZT_P+pT0#Qkk6jVlgpR zd;3SeTAFBA7p;yCTMQy4NSHdA#zK8u%VS=P%K zm8UZ;EPsqsLUT2<9d!f=jRvTcSXknXvEfLldv#ohnZ%Qd7^-cU!ROI|-a`lb>!4e9 z-#F&uPs*|r8w&uRAg^1}Cb1k<)(Wtp;tpfbAb}f88712&i#aTYqWaG}%dBylw8}n< zrJb63KK~*7^^a)2=ur-y8|F0gUL5-WbHWV_q1X|au>M~r1kQg5=6lPm#u8-*AF&e) z9(G*bIk!d-%xD7x<;RYxnB`I!nwJ8eIvZ%XY$=*MgsK>HOm+86i~B%rvUJZ@P*Wo9 za&Xna{01Y7{(N?LcI6PGgWHq{l+bEcC{g{{+pmM5<^XcH>8{8a2o|G0uDytPvL;n*)0O0VK zzcK8%uKd_etlJ{I4K(5d3!8^~Q<1 zt}YZveN+t#dWj({KYxf31sSMc0GZge?p_3vDP=Nc3LC^v9 z3navme-I`8ak5gv$O6y3jqy$tYDMc-R}7((FJ?K=eO2iwrceRQzihSc(ea_2YTOm`?=N8>;yX|mU zghSx)J*9!!2}5fRu0X413@?v3_vz!TsFRy(p1JKyQlk{FZwi+alw!t#=Oc&ly6>GP zQ`2&tYq1t{F3+{;w&4_RkeNsY+w|qno5)3)bugF@X%*WFFj8 z4hnJnH=o>Y3RHg>cC1M7rDWHU*+ZQ`lX@k)@auVjkmR^3V(CC@H~32X^GlE34}2?A;`%ls{{ zCc&twE#CX1Ft*7RNQ0b-h#HH^*XU&txe$`w1@GQR?qdoUNPZ1if#ZV&15|PDFqqfE z9Fv(7(@cd~(8RNsZ*_L3SfeC@$3P3sx61>JaW1Gt`f*$<1gnwp9D;AGb3D@BC@h^cdKmJJJK~mbVNS6dt+oE0BKZ@C%pw7U zb?*x6JRp^6Yh+3-t5+}Z;4A87TrtD#MHAC-yfq5OwMJczQ!=R@-JjqXaC^F}jdoCr zAo-Q_63(w|0*)gH;}vn9qb>{m%xfoP&J^Kw&GlMz8Q1CxpIZZXg5UCnP$i`#)xPqK6GnFx$}zVYbun;!q7c^r^@iL*T042CYHsWM3 zsle&K2d?F)twFu3_W7E4+86UXLrqC3b{>3&J|Qt`Uh^JCS#x8sUCK(x*h5sHrU90* zy+1O$iZW4|3$3Ez>ek({XI|LQX#TibH}_GZ=aVEIfh0|S)}*dZV-Bv{WeG^Nv%IOh zSd-Ws7gJ8}Y=2n36|~y~OJ9BXevdZvu4?xdUr@-_ryu9a~ zZNvcczG5~i(U-q9Q4`na<@S~AQeUlPmm436(&^V{Fe|=e3Etn?x{0>-K|&~(Fa;qUW3!^z z*iO`zAe%Znv#xiZu80qxnx@;A-sUJ!p+L-j7fuP2)5H%bO%^yhSRca5osIL&^b4R!foQ^Tdl<4d*MeCzjFMTF`wCkM{>-(;EjJ#6Dw9@sGGzyF9 z?x@dJya(|tHl1(Jz;pgFbB^3=;A7_F~eyD+Kuy@U~p{%EayeX6SF@ci9nj@Nlf$wdrfhQ^V16mBrbkEA)m zOc8r{NK&4?qt6n~5}%l=tmJ`Z{K|_d3NW0N9StxDTu`BXko~Y~*ulKiu|85q{#TXs zRgtig;wma^{e3RsbX1cP{m)#)=?x(MHNr&}!ZVDqK?R4Wy-LMzma3skgJpS1Tknl| z9W5Q6FbscfBWZg?ACyoKp?s4ID#i2mY3chzO8}J zsWPo9u3Mls2t%kwxx8%d{xnxDK>iX8C`t}5i~q?nP!ZcnbnA)v|9fs zm=RC;7}Jj~Z`(PC%R1>di0MYE3b^(-op8_o^V8=1n&fGRNmj~Qi)=>MMXyqWUMh}ow9w%B>M$idl-X_Rce8kE z-j-eYEcYP8Y5D1P68}`S+2YcRo}Pnzc2`7c`k9+0P;F2)s(OAnfOme5nb0+>JmxU; zzGbKhj6k1e)8Fj94RbrHrM^(JmidM@TlBMc!h@Jo@B%pgJFH|8m}2IJxZKMZni?suYZOIVh%bO!e=K?ksSK|DzFSf0 zpfD9@TjCtV^UEu3XmO-!Rf?{U-Y_02On>{)@FQ0v%O0_vBsTxkF991<;dj*J*6bXgGQO6kBx7APMYi{z z>fzR!5QqAe@*%QB(3i7XxBkzx8U0)Ijkt-%-o3z@l{nL1X{w~ZV5*nL@j4BAf+JOZ zih*+DY&`c2tiC1x_+HolW?9y7;xzZ-;vl>iRgRpCvi$|$LHhYbsebJ&CCs>RcfYpe zs_~6^#zn@T^R@44)?rI)jLXSs2*a37b~<8gHppn*qB|x)^HdS;*p?-S8EpoKdNPvN ze5ka9?oLlRo9@7*#@WzD-*jC>LUh_%_QRt*6|#7 zq>VncUF=dQbJ#(3zmQr^MV}%Y0y|zob2U8#t6UrP=o>d9FE3x(tK43xw#p9*1b%$F z)%tTbnB5H`h2vfFSdj47#D$jCMj`CEP0fFKQDPZGSL&;V_fO@&_{+cqs_X~Vq*_K_4pE>XPt~=v zw~gVB2F08zA+xDf*>T|e!&Qm2@Q3#XpB{u=v*~n&UeNX6Fbo%*%=H5tcWg)?bW*<^ z5n(|YallKtn#DTwR95e?wbWBob&P_WiTZfEoiWL70i`z_{W|j?2oX;NX^7-i*+2@8 zdHJT1^SzJ@_`uNaw3^A+q9~P1>RyGi&$Zq2B(twqHU1>ypN2Vzw!KU^b$Nlal{Ln+ zb{yiqO0>+*F$P?>C;XaP<_4c$xh*xf0-J2=15V&m-}LvzoUT~ov@x}v^!eZ@ykP!% zd`?%A8CQlv{;90{e354T{Sk`WGz#0EpqJnrP|TIaF;U>!I*&-&rA$0XEwEW8wEe|061s{q!_!Yo2E3hkf%%e5d#_KfE zjrZO8iC9X6q_Ljx$%h1Rdwg>C>--t*D6w7boGL%0Ox5q@!n5`AhnCU;*A&gfr7STS zWHv2<3#WFs1v3_om-@KocfX)<%e&?7Y5Z;>I|B{A?h$+;A1Z1SKyW3b?q83y zNRl%DCo07K;lrnQzcH9`MvrSc}A9$2@sIB`!fD{T!ob|M2r zQStKHg9j3G&eb))SwLPAx%{E5kD0sZcOE^sMT{IAt7pdnxc z8FB0%`qckY{P@>m$ba}=?7tkOZruO#DeDW_Rp2SWPZa+fHJJZ!*?$NAKLBaq|2~lZ NTa~K+-%x4d{{cS0)cdZKM(~6x~mCuvIl|m^+A_FAkaw=Rg@Em7Wf2Q0^$QY2t<=Z1)>4oss1>Z zL;d$}sXTIM{{9(s^v8vi4bav5j&5FVo{nzMMdW2JgRZLU>eK%5G@$+Mv-saWvJ45v zjU|FAX;*?7-mDYVgD9;a){~$kDi0c}vmk0#DjHTQN(V?7_{|eke`|lc4S1oVrlCDS zM}Lyx)M?;?8Ws>W6%7qFEzOA&fBXzpAn-kimh}Xi=#^V^?1oS1&$@HSg(Q7Ac}~5e zjnk+fFDC!gBb4D37dH?yWwx*W0j_w`f2Mi+s|U~p)71T#8@#m&ymFDx!CudEWbcYf{e?Gq0U|KJOd=fAN9y#5}TadPI9OxeW++-I45s}=X~nXe~L>?0VhuQ zgSEeL_SYB-{cmyhAB_FKe2s%NK-7P0G}P2ICunGBPSBkI8r?~{KibJtC;!$?{d;5l zTRZbdWB!ju0UkmHJcE{&mLB+LIn8jI|-(_^P7K5 zKCNz~%j=~ECwwERw*Jba8A#l<5dKM?p@3pC&=gRg z033gwti35rk`QJDFGB|@AiNzn1tgG1W@nBR{-wA=0TJb1Qa}l&xJCK`9--Ipk9}mH zeR|T}5m|EZ90e2t-UdIXfN+ud@r>;jUwAdG(#aG3)7fH0td+FnCQ9!TcP zH!c7a9GgV#AnU=i_7(77^uiMFCO6=Ywz*&~L&5|ISa4&|^Qt zkQMF2a^@xYb9l#(rxeig88DHZ0y?+P{JXFW|6x2O@jEmFam;tX!;Wy3qcE}r6p->w zcM7OA`#`uJ{=2jif8z%d^+xD5KmoZ_<6wG1_^1qSt$+S$t92+>L5HNp-;mri7=>F64Wb+OBb)cGd)0!W+cDqD`oBf0wRZh1HD zq*f^)jxc-oM^p{m@#C{XNqG}J;^>IaDfc2(xe{g_3$Ml5 zY{`Eli!ldlX}ve^u{+WovC%f)~`ZgC+wW#7m3ipR0q@%0HJ^x=T_jWc8y zN%NYt&h3d1F?ms4mRBbyijrR4Lg?VRQT3`0fGy!!i*~gg3IKEFSy+x2!>?_MUrDvY zni2YB&Onn1O~iHs4hT3=vlV0Zq2b7ABU7B)71ybzU%!X;_i2-cKE&^%WMdm!UV7~L zd>%79xfr+MH7zfCNn2#*9c%Z79;bfJD-xRw++Lw zjg61KeZ}rIs?uBGG11Mk&ug>J4a#^m%=4$pJb}GUAEKu@w_>stz&R(VqMg{Gd?>h4 zn=W(8O4seIx`vlZcPFz=7Iby%l(Qi*1=ov`L<(9dNjfK{uMlXAwwp{i`}4Z4Xi6Kb{XjiUiNk(K?}*fm1dT! z%5~fpJ}+g~m(s9KW;}CsD4B(8+b?Z{VM~S1A8njHua=(o-nX8{*r+PgJ?0X8YWZ$% zeSWj7T#=`aAM9(f6)0{ySo?Cm@!~}&1}t&vLT>PX*^lJ+&Qd_h>17nqs5k}mD?pL( zKJe`kEBve82z;H50yHjB->qvz# zhYO$PYL+d6SqN%+b#kwOGx~X>zB1;y_Vr+HctT|0s8g%LzpeskCs-OlR;BB5kA6m6 z5=a$y@R=ZM6B1F6wiJFAylj1Pfa~E5ax6DFye1KFS~GB$0%DXbhWE{JlRCy>Nl;qf<+C6e#~J?WMj0BCZQ0Tc!RxB0{nM??z}7var(C;-5dRxSWG z4*kY`F?ph$0s;eTwi`UlK+ry>LXyq^piS&AaIF1zhHp_oEC#XQWhMY^c~L+I=VmM< zw-A4(5#Qbzb$sRueC9NqunMlbjO-liJ32*&Q-7DU2+V%>nP@2SftE*=_CC znn4(()A}O*;o7Pi`|$aJO>wJ)KDEiUWmF#X-2}-Ega3k+w+?1nWlI1rN+_Z-`nCPX z0o2v6&xU3)emOg!uK%nlCB8JyHpjTmBU>6p3o`5{HgC3mRSO2)I3b=^SGqEy+-MiE zUb~l(oI80t;h}`t&jq#%xX>&uf6N3h{(lnMZzGAUeg0x$aj+i{#B^A*+3IzqNx;wj z0DM!+e!#IC!V?!YQWK*PT}}VxZS|;~GQ+b+2YQl>+zfqJ1ne8j^=1=0Tx@O$@eZqM z2YMF-bF_(l|Mu;=q)-|CV_nYOt=~lVCkdR-e4Mon@dTi*pEWxnUh%@<2eFK*8Xd_` zz0e0lBxPW+BfXf@nl*xSW! zYxxUvNYHE3X~=&iyWzUOPhIWQf1kUo|A)@q>i-W`hv2SH;MsWiyzu|^I_V!R4(2yq zQ##s0NABeY(~%s!OM5>Rev~Z}3qCvk=FY5H<~5mvm4SMBscB}Ta2$gb60)s^K_O0E z4WoIFbBC2FOJBXSv2$z8{Fl+Q!}iz-LXlpA~D&%MQ;xJ97vY!M`*@!F_(iLLt>(mfgTPf`O@y3eFsR4d%w1$^XC z0?AN9jsH9 zG2&0W8I65k?4R!mRpSh~s-7Pl6mJMoTBCpjcW-dd2*(D=JaRO&bZ+5yQ6JHx@vyJs zHc>WkNO^uP?v2ll1_@wh$4@wa4{_VZsX_L z3@k_eU~So6%I)oLE6#$r{E*j`o3u2srFKWRU@x_ZdUN0yjL;Q)U19;A{GNj_=J@$b zMQy|DV5IYl#`-5*W^6A?uw}P?5Z}r1%y~#nyfbcDMUjk zLhB3927phuNEsBYrmP-Ht|^%|S~b^}@{j8aD4QSzl6&eeW+)}x4Mjb-`*|@NFwTyF z^sU+W6rzt##jv*(6m@7hZEa9wrZ`!fzSyzhy8n!szjh1zVN^t_RaT)u4A5NV^e24W z)*wXBl}nNf5N7fK-;3;-Y}&#vPd>u(&>3BkWr~yP)3-k>%hJlX7i`)+ycCU)n2FAB zdK_*XyINy_)x)iGO$n!?xDK=9WK{JZdt0hlvHjRwI--76IQvkXR!3kR^$NZHL62He zw+&r0Zm_m#i=KoWl55n0bDQ4<*y9}so1rB=d&Ma@lB7~u@=X&Rlz9Pp*t)Epn^!PX z5?haC4;pNt#!uzRq-9vwl|8apQNk{C4_5Eja~-<$y)a*R77RK$b{{}^-j?(%zh2l~ zX!_&%+WoIhd#9L>^T|TQUbD(UX_ws_O5>&#^+VM^Gz=?~2YL*vYEBL6xrx2QaSoHc z)$ebD5Q@T)U?+Ig)P<(lDajj__ae!iOGX2mZtFr*(+{hw?{9y#U-uxY3rL3BtMgDd z_e(3YVQiKyBp0_B$U{QZg!rj=y3G8w+C8#FP1c%LW8B++xN=-9@?C8VZ2=69Iz>G{OCS(_BIxXq@U*5^)ny)OSjpcIUjs2J< z2m4TZYH5VSG;=cG?f_u7BCn^Tr6C3_15yf2lBFvfZZk@lq=MY0>(#g4v47r}4?QU| z6N?A)kSvK?IJdAVHUegk?ltD{RYl`SL7KLi)Y+DXP=%UEP^Yf_*j1~AQ0lm@kL7|c z$IkNWcO;i&qlFZRA3g>ch2g;MrSE0$G*tInTDG(dZ7ehyiM<)ej(2f*XvS|Q*m`t1 zhg92Q-oPMp2((+=01SGI_^i<&O3~bl!TbGF7j<4lZo;BugA~^Qt*)pFA52_Y^r<>k zuvuS?DP#n|nBUyVEn7R<lp!n;~n8se4G);4e94L??Z_Ev^`RI$vH1G z9*bl{Nj=L6D7*EON@g1KZ5Fw^Q?c>dP*XI5!|>*bzju3G3P|(~cx?)DeCGOK*;nwq zJpHkM58wuL-Q@E!)C9pPga4||r|E7EoXO^06OzJHK=A=grK3)bkuffC`>!SDJ-&<; z5XB}oT4k>Fvcd_QMUhY#A~T5dD<0W;y|DaKz#aVeY<$HBjCaW=MLmPls~&Gvj~c=~ z+Eicc3pc@d)t@?uUM{FdG%e>aX9cmDPdLL*sow0(3oKWBZ`xt1w0FHs&O&^M+h?Eo zRIdPmTW8UPcbZk@-ixf%nLT)=VkZ(hXW6sCuR(+>)qSXanLq02|LI(oW!x zXBW*D8?$C#)M>>Cw8!wBoGAYK)TvP$B^=GS1whhSL_&aa&M264&U0A_D_t`<*;UL? zU_Lti=u5TU!UC;hC^xojen|zo#&;39Lvn-v2u)uhT1+RaF$3aTk0^r0ZgtdUUFrAQ zqH=2dnUe+MluzEEBDJzmw&vV-w{l~#Ym;~*!hInI4f4v5fpnb}&tlWB_+F5EM}bc_BEb#)+`yjGCHX+bhL`{0(kti`WOTb0@{K zZeOW%upi(Ti9K?TZP2azu{b^LPkt6%Dhao+Bqb5J6N154;{j!y?8f*q|DXD z8ozw0kEtHdUSIh9o>teD?8g$4=hA%(d|Bja)A6~PrPyZ@gm+cC&i9Zr$LDX=p4Iv` zY1ZZ=5JA3oPWrmbIkBBrm*s!$aUQpve=+dLYSP*W2lU@7YMGzLP7^2~`RhR&<%5UE%$W*q z;Xx9E)!88 z1Jy^T<@46Ahg%r;LW`x+Hr#GR49Pv}nECMdLHlZvD}U&jGtd?6g%%1(wpG_+%9LdA z9#ApJm<$K>$sj3`WbL4$Wm7q4wceWGy+uof4p07|;5d|%n%FR|PojS}+M8jqBGNKOW*(_mET&WF663D=AQ9-VJW<-&JcPuA7KHK4Y zrSkXjdc!p%mO)j7G-J$t(P%EVyA{JKW zw_Gd5&Y0Yts2M6>ct21j^PB^tiJxn=SWG2&wr26CT@(LI)SW@BH@(;R2HV)IVbWe- zTe$g%X-&YSV6;TVCpXK3e(bd&*U0opd=L_*2U+_GfcaO3njFDEU<&m2E-7blzzBpW ztha%*ko(B{KbZ~CAOQMlC`13Mo%jzw0U=H7Cm7rzO5%GhiJ9dVU;UkO^Plx~s8_u$ z&@o=T);cj$(I#RWY(PI$O6b}9KpZR6JTjOfohBCH8UplZq`ke?^e=sLQFjqS^)~A4 zxT}8taHKA_k!+SEG!asNpMI!?=lE0h{V#o6e1WIYjoQ(!8`AG!&U#-r6%uRj))H#w z^iRg8eD`LAtZ$`6#IJ3}l3xT!&br9YIH3KrL)a{U1e(nZgv~i^(8s&QKJ9?JjU7K2}So{9tOj)rID}7e5Q3S!<+AM>s>ql&%-dTAs9PR%xz*<-K2u z7at9%gSDzU@N#ZgEJZ5_Gp=lv!cvTC2cFr8D5596(q1x~q2A`aZ{P~WG4ty=%x%R9 z(Gwu3aX7a%Eae_%#=l&}^m9w1#^(yddFu&|<`@@T>eH{0Isy%z+w)ftrpGz!(lf+( zJQ9c+^?eGsHM4Ol@GeB#%}yd$+I92s?W)G)5wWDn`!Ot6wMKt3^LFbVl>m!zURl@| zA-kiJcq+i7nhkxwd{ zOHt4WI|5~pX1wl&l`2kob0_iA`4{c;pwlOCEf(ML>ESKk6}KWG%*{8zG0lpBxMN=v z7YCG)Q)6M0=h?K-VZ>sjSTp;#jx&tG#r7{~6dx+YkL;0rjY(~mncpa&BTq=O+OL72 zi!gYHC(Og`bfq)aAo_8A{ZW82i?2!h$-LK6#RCk=mtWaldjIBNk6)&glUOwpW&X>0yI2Cvpk;6`Uhws6#q6poPei*B8dD$j3f z3AVb8VbNgQeA*W|#e9=aP8Z~%>9S%ik1wpjb(5@$oJT5`x6gjv`yl6giC;iv$vNrO z$@3z3=+0IvnHvuTaI3nagIFOj>P>L0yCdI(^1#G$C0FB+;G^oE5>+kU`0o)ya+PVW zSNWf-mEpA_W)fqCK|s3vOq5Rg-TvM8SAMPaSjMYaDLqKjx$VhL+uu1JCJ+S*MTsU_ zxR~pem+wrIURE|DeGY}DeM?uad&O{l@IO)7oqVTMDmEf7n&w<^huwV^vtPW6&Apuv zUR6olbQHF(LzvkLOU%OPCZEjLU&^}r;w?MOsKtJrHA@nA%&Dd@3DZC$*#b;w zR=mNiM0Sl@{KNS7$0?KAfiZnhz6^^v#qY-23zZ+?<*cdn$&wicr-3z9N33$sS-Bp+ zmtC@=#+`u0sj5gs2(Irnro0V!c{%pSf+(m;2A|h>;^~0?7~GQVJnfsRs|bW%t-$9G zgiE%{mI)|dcky4Rx!&nCdR~eM{up)E_?-ktsEG=u?%Y5;|}~^p?FjjjcPYkM3d6-MajYU0|3CgWf=hy={lqRrFDcH(s<4nlyH$ zI4BMoPUtp;R7yPF$QtDwz$|G;D;UpYzf3z!4D~vi=32uRkRyaj3)j^6GGK+aPbQ-=TBw&fGFgUpDJiZ7cRY_`cYho*h*uFN+bAxz}z6p5zVnt+aCR~&QB zn05%Az?;7G>IvjW^PPx9JaO#Ck56LJs)h;ljw2Ux?-xf? ze=ME>(SxY^9#sD$Q@=C$&yN2J<0JoV#{VM_mz(`_TkJ?}TG?p~sp4Z3H}}rrlO)c7 zj^sofFC-~c4eb(t#lt()E)gCsMO8m$+{2@HHdpH)k4 z;_N8^qBMv@VVEI=o0z^APRyRs=kLr=j{e0Y13(-hj_g}U;4z5|L6!S8m`~_@gQM2% zRziK!Bepq_%VGk|26*}oaCFh^ivoP!Y-&PUMy!(|)I>Q$j&e6>q*PRnQZ9VwQMX1OBz?872C13Qt zYYX(SV8^|68^Q@#Q>?K$ehKZO-2Q4B=3_D7=6$43X9?wQN-?bUuUd!^Pj?%8IQNQ# zcX$q&okU+)9k7DWz%v*eGXueh{%rex2s??N`0}Hp#m}gj@DLdnlI84{pxB_!ciRZB zqpyo`BFnc_S^Unt&XPa==+FNoXMSFzCM{E}<+W%65lxg<+L zaKdF}L8M$<(mfTw^ZVnwk6SP)&`soWVyz=07_(PFKzD`Ajo1u$9hp>Pmil+4{qzFW zt=OifRNUQ}wq941cgH?m<1!16N4ciBG8{?b$PIzB5W1D?dwKqatIFJcF&1;$sfC|k zUs183hrfLt=e?D5XA=HtX1{KCbiFt?_9h45&Ht}D=iBq7$j3mu&lmpd>e{_3F(Bml zU3hm(mK-Gra_ZOQprT;@w`U?EV|>rCFH>uXbOWd#Qv2K(&{%=-Dln}xSFqCjB-Y?m zw!q>hnxDFz!pjZi>|{U`Tqtn;5GWO3N-*8?YDmW+V#?MPr7h7x7YJ$PIcj+&`eIk6 z0@J$ghXe;}2pFoT_4_7#VBd1Jsn6J#1(Aen@our+U_q$R*`a3hS?N0?{LHnh_Mpd= zbx&7l4aYSZoljTzxHP?QI3Hq-)--y&t7_9BOry#_YY-M#3rPF$fCpY$DYni2;B9mCXWg*SE`ykxm;Byg89sG|^6_ zQ1O-*pR6v2f4`nV36Dxr2VroYJK5@u3MpA4l(LMUrju}#js-2 z4XxpFtYaYnw>CdqRFJ!k|NN@hME&L|A6wf+b8?S0X9fO8XCiZqVpZu^#MfCz@7ntC zRi(rH&t-BtaiXjoUX11Jo!t*(=XkmXtBsq;Es*(;wPQaDC{xwQAR_Cs3mcOfIOg!s znbk_~`F^LgV%CFHYG0<)=5J`7>O3}Xw#A4ixLd(d;HV7YcH{{sG97;Hhjd!@(&c1a zYC*DPjJ2<=$H$MSyVpk_KICl$fh3!AUk3l~FZ|ta`1|``@gMsBl?g`EQAiwiQ%0ZO zioh~Q$?m@V!duo^;K^9)-sM>LP3W3x$>zBXo#A^{bMe8;%9lkvvILHcn>lA8Tux-+ znnA>q!J3}&A1Zpog5^KHEKEkf>VC=g-SwOatL}no(xsNcuy2d-Xe6ySkh_MkRJ(CJ zTFuaZpAZ#j7YJ4Lvy6A;tg-Wz&Ku%dc%Y#|EoRJcz?)7VW+r?FZ2vWg9~Q}8Xpm#& z$mL_2uGFg(CcHNf^|JMjUX{E|dwcsi`QCQ$#T(0(&_(5zNJI$SX_C214ra6wAYHb6 z$Q@s()M!!?ltyT~@eL(5Wq$kZf`0pX)+!Oxa|_)f8I4gJrZ|Q7&s0@DtnzLAs^iPm zV3c(NWngsPSU$4KPnSpYM)$tS<*)1h_Pj|8s&%v%lr^^(&>zG&1`8g83h{@ct0!Hg zw?{CC{$;Nw%1#`~&6ET^NXmk~9)a6QtMN2*de0oiOHIQxbX-{Tfb}{23?V&lms;Q$ zSE1^8@XdCI^D&=mze?{yh9!fmgMFyHY&uVB z5jH7}%Dv~^yjKthVL!TA`(wO{G5$;)uA32vk~$*;uYEY-9*(Y=-*&w=H{T_YbWisN zclIKXt=BJt8+?Tjt^xViAbEm80-4>6nBHk@`J-I1v&h9=JYS24w`|Ly)+qWrT5Gh+ zL;`tfdX5_%a*k?F@DXG7L0~2Y6pO$>BFI#3k=C@3lQqjjjLO&Jd*7YguG(8`_*A>^hC%>1VSGNk6ITn{cMf3?d{GzKx9<(1OpSJGG z1)R=VVg-PT(hxueL=~Ly0RG#Fhl@nD*N+cPhXbv%w-NQ^M--4*74zZAZ1M@&og3%K z+}!BILxCDNiRlj#^L{%8r$0o$f%n}8a`YGBZ&!rz^<*iae0HG} zNFzU+COr%^JrafOcakp>UzUu*(jhIwE_nmUl7a9XrS_27=N-9`FzU z01W@YOxFB_o(?}`c(opcMRTrHmTDO# z^m4?SsV@Lr;?%-Dj3CL&M!O+0dNz_KxEhsebvc+WZztOsD>T^K(TcQvE%#4MfN*-l z=>o(FBZ)SJPFT~}8o^6F7IM52*}YATT$OE5;ZwyYU$8Y*w4UO==8OT#8SJ+>VQho9 zzYv|~E&$~uEMq`zL*Vi7X2$v2rbMGA=w7mlV_~>DenLe-}{ZX15R}F60?i z2Kbqmjccjk@j+m`Vl#~`6h1E*~Bnqd>A3|-k~wB-wRdys(3c8KWy zsl)Sg+->F1*$u3dvR_ta5z)i(Hup1yHl@#f4PeK}sojM!TF*k(*mg7O(QD@fl;hy` zfo>h#LNfgd*{o)xMB^^g+H`EVywlS_t;zDIkt}IG)*fe>KFg+ApLf(1KjR)@;0<3R z3ok7{*l+;@N@n%RBnd)9a5Sc)FCf<8IK}~zwW`9Q@)*PW^^skLF&Q#u(cvdIqwNER zsvdrW?;dKSh}2CA$RW^g&a!?Wm+-)8hJTAyd)7bx!kB9_wEC@U#-uB0GIwnA{@vDg z$VYlCbk&c$y$PtKfs~_VW5(TxvcEo88)}N2`kq48xD3ZMBM$|#@thQ2dQ46Ey}?!=(?k8ptivmx)22>m7MfwFABMs**0|X|Ge}9P zm&%^>7zVTzz1rEJK5G@inU`}hb{YDojfTA~^-V^g@zT#+TwJg)T*3$kdUc|eP)HDc zI@558Zh~ROy~b_ra6KrzACfh7hOmToI~5>F^gOyn1j@za*Z9;Bif;&lWwm>s&9F}U zt9_c!Bd?}uUcTE`{pnF8|4)v`rV2rqNWO%~Qrr!;pF8&}g%k&O;q2JLtV6p3tVL#i z;=LNvA@&~I$gyOn*EEvy+v+s>z-yP)O5qX3mEyhM2*!Rpy``$$sRe(o`so=3%1FUGi&p3O4DZBnBkHuu{= z^meMXu5P#HXVu*iuXyL4BR#E_u=~|wA&SkVQnm=g1_l8i#2IjPQ7g;y0L?Sn$JZah z2^sCjS*r4TiiD!X-2NPVL3+H*Ok-sq?siPMw97^ZYmYkzM9ha<#5Y+)_yV>}j}GzG z7d!$!>N>zQrPw|1@bs)LcWK@X5}o6YxjH3$SU0rNZatVq`ihK78>zA)^^QV_PLN>X zUsp5$qmvFMIVb*hj1*i(cph8ikp_-12f{J%j&29=&TKlF228$t91Ub#Q;Af7W#9l4 z8Gy=*&f)T5!@Y?9!NFlXhy8%_sk6nTOj1oS#L(QV2 zBz6Cz(b5|wG#l4!U zTlO%2WY&b~#v0@dTo(`hU^qgg)?lBJ2 zY#u7l8Ip;7Qk!gRo-b6wtCCst1d^3qV-N;-oM?h^mqlWvLL$pJG4pQKOWk3aEiV4a zvzu6Uj{CRxwYJ|1&BQ$Bn&(daJRY>`o7i!zY}!EJ?vUl18g)t!67E-oFe0M2WWpi% zq;ad!HmKN~j!k>#t(VSClJeD4_r-!AC=MQy(~S!FfocEosF%~I8pBz zqgquY;OmQhDmKTfc<6XRZ_o;| zDiUA&)^k?pj-0cc#qIq&{Y=H}Moo1WaQ8gu@93y;k`!tQ&$4dR;y=CRjuDsXEiTB{ z^ZIdi{e@h_g!${#^ViOW>6Hc;5#4i$xtKN!OicBWv>(3n^JtGJ-y@Z~?kp$od5Yqw z7836cmZ@C`xHhc%(0j&3xdZEsiI%E67-600#IPQ~rWhYDT}*2BezX(IoxVFE{I*m! zlypux!XTuu|Jgi#?be=gkpgR`2Hlkf9-Xl&!HU=qvx$)u5HnEQY`)wgi=WCa8+DrJ zca+(%$+&vbDzC@Xh<#yhp-aH4><7&pcY8J`)Sc*tvX&Xx)i<);VZ&lKFXr2gE!sUa zTVmK@xki(shX{B~G{-^BWdpQlC*|*j0yPnWOO5f=lj<~j3_XFwwi8|WurxH)yqhIV z9@Y4WO*V4f<(FQ{6QN8_^gC9uSC$%0-SKh}FVwBxvXtD^J@7r-Oi*Le`1a1-*h z=TX4BI`Wp{#&(4snWtzL3B*}fg!jpA*jX)! z0t#CCx60<<&J1rtN)B5=68o4(}iKsuP zvc*D7TkM2FGwEv$U8E9@ux1l%LJB>}hDTh8F;Y!qvFOn$&1ahbKz{D4Wk2BHF!F#pBN7sh-CZ;yFpa;u=Ntzs{ z-NGZc{#oesk#bxMFG+%0b)VGpqQ2z-?8I8O8=41zhrC?Q8xKpXiTrr_%0LjAej*w2?emAm)NW1pb1qjd zI&`1s*K!_k8Sf>WFY3YHZ~!#Y43h(zD@$pZbggJ=ZSPd68~jGK$D?+ZTtQ`ayXW>E+BZD`DBqKx z+3@c$xegEaX7bU+-&NWd5*cz?2;uss6261tG#%~u=COu4#6(@o4WXgHhd)9l7oZBuEkiE3~vUfC9?p4cl1;t#k{ zPY8gsNZ?1_ktO2QKjn{9@PDRG<;ipE3l`qja$Zpl2MWYIvgr|78bu2V-Tx zq>niGe85oBRHhOTKjjpDubnQPhwTOSwCTWW=@d(vsa|(S0GIz2bMh}$IMK{)@xd&+ z4&j{@vj!1ZE`*(XDP#pQ&TXV$)RW`Pb(vN-KkcHO^79GQnfdq+no{2yCh{|O48pT% zR3G9HY%sV6p)hGy@@@0Q($$QJ!U9fmhL@5u#8L9eP#^UwB1VA47g%SEY?bMCPvXgji1Ur>M+K0_m96<=!wFnp{2nk`{`$q z+7~{$ZU9N$1&3o_`rF9ztz*AzVUjpe8;YOq?1L2^X$MN)>?1K3th}hZUVv`iSF)d4 z+?-ux5RZ>dl;)G$tXBuMi9!cz65}W!re@PTh+_-Q(6b;>XKt3%6y1JrOK@$E^SoFu zuhSgO(^}oBqP`{0&3w8OX0}2n6|E&zbc6#gi@6M5$`J_S(4B^_nY1}>B1^@P8VlKD zbGW^#nR1)1jCf>}OtUaf-^0@U*RkaN#)R!(R${JoK60uay%qMyqh{ZKYPCQAPyu;e z2`_{1C+;AYMPJLMx<-XTRE5V(XP$^8#)c>^g}FZ;I!^pWXTH8=30Ml!v;+RE$`(!d^2y zGf{e(XWk3pnVVud;WSF*$8Cj>^cYmH^a$=uIwds^ zsgA^-6dyIy1 zhR;>@VK4KEO0P>QXG^6Kj)xLh#P|(IKi%AcM^94)IQh?gaWflK6%8yQe&YXej?5~{ zit>kYy)QC%BHS*nOWQhI^=5u&vMtm`{ z{}LrD6JHAJR|Q-<22evCSJFyiu|&xw1x<4QUTV_aOQCZ4EJuVS$O}VFhUN6#N=Q!D zC!ppAJ_H2IZ~lcu%zwYm=ARAhUp@Dy(ohTU|C}#crnv`f>tz1lNYegQBL3NRe}S<7 VACjp5X({|OBL3OAzd+d7{{z7R72*H@ diff --git a/docs/reference/ml/images/ml-data-metrics.jpg b/docs/reference/ml/images/ml-data-metrics.jpg deleted file mode 100644 index eeb83a76e1cb276fe9cb1987ccd4c0d710b8ea44..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 358164 zcmeFZcT^Kw*9Saw>C!u>Nbe3AApMCb;bIzH4%H(9>WEnVXY+z&n zkdXnvHSiBO!2y@`k#J7{Ff|1v0RT7ykR`$aN)Ul8055m~0QnR0zbTZLpHTdb$(K$O zPrd+ZH@pKv146t50?#SQ$^&Y;My8af62Rj(7Woa&&IqKv%>NiAy$th+4@hg8fGjkBL zf_MfFTJ$fx<{fs;2E?cQB=-8EniygiKm#!o#$^sPWTr~`SVC)~gY#Naa|?|_FI zS%VnVf&A5NxUCt8nLzw51Yvcm!>OFjNN>GU`d?**S)S?)K647dF9bxtWl~)93^q8m z7v!f<@rE0n@`GhjTo3iO2HVAooL;WkS4B+`Y_4e-^2w)8?!H<%cQkznf@;Z3E z44#UVYLx1K=R3#zx1QAc)K{smQ5#T8K;bdLf2aSQ3b=z7_=0CJU;^GfK&yhmtN-7+ z|62~#8r38fo@$nAnd&WI3w#3k;D2-dzet1sLo)@IL8*Uv#RK%%e^^H;3Th3Z)TO)x z>JREp$w|oz$WSVSmRzCK0d0eT)PH;JZx8?N&r{vQ{??jSo|c@Ji}o_D`2Vz#Ns%fhA8?dUQ={Yv;*fTdMG+5 zMk!vN;(wKI@Y`=*|CDC=+t&!t2Ke7R@@V-D`NMzl|7|G@Og%0C^x1%ICe#BN3f4zJ zFCZ`?81CgAdhQC?TYH={@^_b&KBp+Jpb7w|eg7#B0Q}(m-KUdr4*#1La~l9INt~RV zbp4xlT^#^gIskxf^5i%(tvQ>4rDjUyvY2? z!pKl$v1F-a56SY$O3A9o8p&Re^^v_MnArzw^xwkdv5Qd6=~o}-ij>+3a28%lS|Aj&ApG|GI+ z=ag-f{gl&`I7$NLF%=^fKa~`f29*(&J(U+#I8_2w4pjwJE7btlrdFx;sHv$rsl}*O z!M5r^?L&QsI*q!Bx}Lh1dXjpDdY6WVhKELq<`RuL4U8s~CV}QLO$|*C%@oZl4e1R1 z8KEg&n3Whh0B90nX8U#it8sgANOT$5AGE1dhS{7 zeV%hX`aB4p2R!XO?|CVCC3!7)!+DE&2Y9#n*!eX1-1$=Yn)nv^$@wMuE&1>8m+_DC z?+OSC7zzXmJQlzTYzuMg?(extfBKqQ@2(5^kh_^_t z$dJfSQ87_F(L~X9(a&PsVn$+-Vra2<;*8=t;z8n1#b+d_CDbGk5=9bkB*`TqlHQU9 zk`q$oQYun#sUoRKX)0-TX@BXb((^J5GFN17%b;a&vYfKlWn*O9WxvW@lyj7OAU7m; zEDw?Qm47C`q`<0RtPrEnsqkGcs4S)IpYB$u<)LyAmsq3mot9PmY(ooR|)o9ZArYWO|(5%+{ ztR<%9q4i8_<2Y*@k@~z7im)EaIT=BV5bLFe9 zyl${=tL{%d4ZXX1y;mu&8eUDgIG2->lsnFgG%PWd8n! z*o~kYe_Bvm+^{IHShG~HL|G15u~|W_s;%~{b*(e4mu$ps!fY_M%(l+9RklCvuG&4c zTeg?8ziU6@!0P~a=y0TQv~{d-+;!4-%63|HhBzlU&$)=XAYER&^1AxEzJxMEVbJEA zR5$HzR^L2?nZusKcHNBJ3f&0qdhWUITemLX%DT1TamnMM$GYbw&xf9$y)Jn@^4jp$ z@y_wyhU>xe;Y6QnJ|#Xs5$1?W#EGw+Z@u3czngw9{8{}G{`~=h0my*KK&illz~vx~ zpzI(*ut{)52w8|rNJl6;7-WxziHF69;cjc+F1Sq!w+(MbvLOADqY;u3$r0<329f1= zDDSx6!Jy8g?xApZuiPz-B8zg1!bA&4$3?IHVfe@Mdvx~@_r_x6V;;ur#M;Gn#tFpz z5w{X=7>`b1N(fGvPt;5-Nuo@GCygg7B|lC+PPvsboGPE1lX{ruo;IAWke-)*lHr*# zo~e>qe4qNh|NXfKIuD*dWO*3*aOIKdqqeMbSt(gN+0g9QIm$UDxpcWrWUO^uWxQ=db)w^q`kT&4t;y~wovFU*tJ8xsMl&O` zX0wxXHggN}&hyI)w-!FX^?gfRyuEm^^v64@cPa1BzR&r<|KaIJiI4TmD$6f%S8=0w zEBw-m`^wg*kWYuJachigIqO2}Ri70WcQCs*c0cU-?H&9~A@Pu&|5E!kv~Rb+eh_|0bC`D|anyNi za{TVZ@8l#977RNL0>BU+4DUU`5%Bj)0AO?g0Jb+^{NVK0*z7k4*{0+z$g2B*) zbaL`T003w@0pKY3 zA*l>eQB_md(7UQ{V0g{Q*vi_*)(#BG+;4e!dU?ZrLPEoCha)2*W8>lz5|ffs9zM#- z&dJStod2}!S$Rd}^Q!8m=9bpB_Kp{w7;OK*;Lxkr!&B2UvvczcZx?a+l~1c{>z_9^ ziQj+h?C$*}{n|gZiwvOn%dCGa`yY0JP9P(vq@aRO-H?jbQVp8$MMD7ICgKh_Z^ zz#oJYfY=FW|2O%+ZuxI@_-`}#|6T9!t6*=cy*+mC^3n6Pm@_x?<=+0pA3iW;Fye2# zR!lbk_+rY{GsIVE6vAa_DPZ^td3R!*CCUn$Vx;_Iv{rd`8tPJ3W^Pd8i5lWl>CjBc z_k&>HVog=@5pw;?(~cFzRkgOCPXHZ-T7;SqQEj%9`B$C6Jp_+CC&19>fAb~&o2&1rwdMqP<8FHb^ksiL0ffpH zU?;$5!Ng(v*>p6_dge9H;Pa=g#qIq|{Silsu z%tP@Sp(V$OBWg&%AkytcME_T(|N7~_^(p1Q?e4#Q z!GHVA|GryJ-_-xhzJTG8;~eUR4n7Edckl$r%l`J${78#={g;)s{7>lijw-TI1g} zrIh|`lpY@)9W}c6&@`Hf$ZL`m^68q$+%L8k7ko@Z{aQ zjm6uQe$UhWzyB(KBP-LYDfO9CCam_tG%vE$B#*Hd>6zRex$nwWnr%1EHcr@8aB$@# z%U@~Cg4-1FJ}i$J39`>s8)uDsQKeyAoxWI*C~T@Jq%_E#_L2Gi{Bd9ZRNopY?Y7Au zy?Df%gbUdkZ!^MkZ~9qgUmKT&z>>0gOmZwMsx|KaT;USRjz*L$l?>&UkevWtC7e-X zk?&mjiDLcl`LIH~sPN~IjH2p^}p)aw~2U^G`%>dRz36m2b@hp_2GGa~ahjMBQY#fVv zqoC-GPQJ}3UXpn@>VqOtCNZ&9z>;$Av7K3`_l2&`soF4cKbQG6MOndw$TEYBo6pxc zs@)`D4Ve^aXru9XE>|IkAYP1C@*s+>Fn&uvl*9SS=@W0M*zzVO%XBH>OpQF}Q6BK+gS)mEnaH&d zrqTt!L>s5vWGT9!VO_KNkxQ5L*3^HI_XgmQps{n zv|uZgABngz6M%B4J+vp5BpynItHyW~k}}LG^}++(ac59hM0V_QzB}|!el{&Hf4O~p z?1x3tbv!nteZ1U2S|EIIDb$ll5MqI2nXwDkqg2=iB%=n4wOcjpYwnbfNQPMcB(l`1 znhB}r1lg)G8q^RUyXvsy8m5I5COEvjZp)R=T0fw+{ij@!^uSId)#{3LCvZ80%ocaAm_(w)9 zqm?^3F+c1~7=^D*AJRPADCX{rHS6wZ>zS_!HMtXlg1od&`l_{v(%F4$m3`d+nvEkA zie#^ox^y#Co_Xo9NO#7+*V#+SR^pu=l~Bi&G-HiBS1Y%xUYlv>h$WZh%+(rjJ5NLU z-8<%0ab1D}CREJs$ym!Fyfe(=$W^(i`fRVQb*01HfYsAq)t%ERX>DP0OY3HYpV(S+ zd!iTSt``={rTi_U_CP%v^K1qc09F(2=wzD9t;94tD4|8ptAS8nc`q7A-<8@w?<(0H z^8w*>W6n@N*9vh*Dv^mTN0=wBGIsR2o8miHi{uHNmhSrG*wR1x+^Uy@f~xMNNxF%o zRc1URgc4>7R~%b5hKt}9Ayup>4}NVLAv?Zow}!VlG+KRx?WWlzjCDF{L;-6tbkl_& zV3yAw0j;=i(5l@gKSk{p;WbT#Ool6L_?#7uXCYo^%zSab^97rvx}(Kn4YXmt#Un=nr# zCFI;WD%XEwfBm*aS^~2`LV^@?MmdiS)?xs+i6bH%jI(_p4ZApzFtb+42WW$rykX}r zhUhfu3`DacSC$QWR!)HA?sL4^9}aOlaF_JS;4D{OKc=&T?rtC#i=>foqQ!B}I$Hcj*wvNBZY`C=ah`UqsRjXbdII zV4l(=$l{?O=g z%n%lMFkNP++B-y_QqG`(cFf^L&}Y=1!8JBv-5S{gN!eUR6601+i0m+p#_x*bn!RiG z@18!y8Yj5)Lx|j{M+!MgG>#Rbp@F-^(AAWY=d(F-6y|(!ST_YVbz{gaoS8B)5aT9g zStbLHwvU&>!+Ti5dJ;ZdB3;#DtL-Dq&zc_#bVv(oz}WOo4Orz+2*sov zzex{(SurLJSi~uWu2{xZ*{)hwTlsTvNLUtnhTp(PbgeLFxi`JOS(#q`*al;r04~o} ztY&M-u^DnDVC;&V1&pTZ>1IU4G$5=+stk%o(}Wn+*QP_wmqZQcJ&jVUsr@oBf znQDWkxN|R(wagFfk+F`1`IJy*PLk@-2|(U0dMJNX8tvHwAhCLc_T(;>70T{uZ8_2s zh6Wq%=#}!dCACP{noyD)T4ae}=2((VFDGT1H@lckn?Ia8eEU%u))ARM;9;L8C`VP` z@vE3VX>%$^HbIGg#7<O3?FJ=3?I zrLZ6!Wt2IL2;3@}Rp6F(M<llg}N9 zUSB@q<(OO!J?3h9JiqZGdT4vRzNN4oDwE2Iib8xOjlSAvZ3S5TPiI1Ooa{S1*M12K?!yJ$uS%}eLBnZyS?e8L`%CcAU#9DjO3tK*#b-&3)ySWHP&#cHB;T@E zDhj64A3%@PUO(ov%QTO3pO)Uqq~E-4x0q3iTjDDbKzdGF%s+e4oHr5?B3tmxhtu)y z`$YpM3}J9;xVtfbst+I1rQ(r3((sKwslN&rT(we~lyjj~%8=QyrfD>Mc0+9@uFOET z)mNvrlJ*09;Crw2xGSGTeFi4*SWK$Vv!gr(3lq?C^>6wG!-0Xq#{tVShIu(;etWr> zs|h2&d1KAWJ|o+!H~l{h&SgI}%s+qbzTWxEw=&1bY_IUKcHgvNpkAg9f{Mu6zi_lm z@t;QI5TWpjozB^cop0q6&56ooT;ktfmyc2YdU0#6(bOq_!lb-zUr6})RS%n10P#;# zQXetZX)ga8u0B;;0a>q0qA3%}j(=I3i|I*JG+Y)xG~t-xoP=&N6KMt(xl&LL?2GYZ zi=q!A%7$*bwc&=1j!bAQD_V&b=#1=n=_**Gje66e){_>?3~F`-#mLta91zi&b5A!n z(gv&&2@T`L%%;{vd!&qTwCeTfG8Imr5-hW%sbs$zuiNN z{MN6-g(JQ%b8;BrtSl#)H%iDYE{knxTIft0^9bL-j5gJ%_`;v}AKRi4B#H7lC< z{4^!sy*7FdCgb2RZ@mlSLH8@l)!NOEDj5@>uhZ(Bj>@RDwZk6|p`MV$#v2UyBML2E=NKgoiIb7SR^bCU2ben$%+I7Jm3e*0=ngK&a9)J&qsy zJeds3jofLM_&MEUuxmCQUgBfk9xSCv#yw(;%S2L#3HQ_Idh-34VkrS$Q)3S6hhcLU z>YKDsC8;3{+B6)gql#TaTsg3gdX3?ri0S%x1U)U$KvlYLCWE!{7Jqf2j@g6Z)h>8#yG#Iz!?_MvETo@CT%g6VzonE1700%^~&0%MCMS zwrNsB90uzCA$hhVQ-vPe>zO%F6GpQZb|1^a=`?)di>{KrilauYAi2~N;Fh1}UNBLa zvzU$8o^(ukXi93Y7cHsqf%=dPiMe>4_$Wan8|FdlQSHuc)o#_!c`g0tS`GAmx{E^? z)rU^+4F=E)xUXCoX>*pTiO*(X^6brBWpZV7tr8Mmk8fK%_S_}1e)^QP;X!h4V(fdN z8$8-D9+YKXf_If47@@_?BK^B0hLq6{y&U2r^N@B~YIpbwRwJ&;j}%##8@IWqjER_W z_-rIShyZu$t(YRe(-^t#HDEbk+w`b*Asu@F9*6UTZ6q7!pnMRTo5 z_afY%dGqc-&wTk-PE19bh8-C!8O^k+e^|0;7WA*^RNi-vh6m6~;68hjG+G0H#n^v_ zh44G}X)!@2_aZzbNhvd%Ym)hxsd_s`-LU6|`JvC=iOAaOXDK9|MrHMny}cpo_Lex9n;WvvoqvpeeA9g_`6 ziV_iO&P}U3W;hfk;hrBdl2VEKS^>+SD;sCGQSeHV#GyW}ozgzQp8o}VN zTrk4%=ZgdkBW!1@-K>j?c6vZhv{HJV)=&Y@P?rPGUKZrzf{4$SIkswWk6Pb+j+3<6 zf9V+N;nPrbHQP|%jJ1SXE^QOGT4Y;xf2C19fN4!}WdtA*Yo5(FKT)pMC zf%C@0oY=T_BL?&RGusL!bp_UkgcgrafMuP`?ND>}L&=or#1K+uZxnwg@skcX245mQ zE*R=;d-TK!p&^G+uP+#k{E%3>6*<;g{6dXWb%Q~5F{7x`M}@OC@Lhx$v!P~1sh-gG z58hPX&&%b13|c?^z!dLmoFObgP8jMf%r?$%X|l z29SLrHjc`(Z;7uoS`+;3eLKr-(gvhu92<~HgY2I+epr`&$&%zZPdNenOU`b%UYvbG zDCrfj8|WsRIsv@DsP5Tmt96nd;i4JdzdN*hIqR#eSt5o}z0I{4L$aK(pAnuOY!%su zx@bVQ9nVcA&Df!-FfByo2G@@&`Y@s>j zo`%LuJ`FxwZ_Xle`O~Nx2C31pt|!*-MsSD<&kg*7vYR6;L}*PY=8zZ3c_maIfg>)2 zC(;qp$Xe+0!#GSFhPV-nmi_`grc+-|m%Bo#4svo_4OjgQd=_IyNL7 z6jC#_^1D~^Yhv*VN|k8?n7ArjwWPh{k3<=b$^2)(ie!BxtA~g!grM;tYf`k*phuxj zK2~B(p+)K@2h`8DjxJs!kDueZjYeLlPn`}uU3Ca{oVAwY%n~3VXeXa8mTfKlRKJDMOOWh{-tFI zM9opc2zJaX==s=l7kKJk^)SCU@zLuOpz(*aHcU@-aE&Lj-?}uZuSQT<5*$stuO?Nh z%{(XU5V%w9JHz0EA!yn=BD!iN?+PZ;++!+>!Pn%f+GuCDVeaa$XM~+CSkMGd+*puJ z1NRwR^;D6D35S(JnKXwZ#DmTnoIQ(bl$I{Opw$%JXQ*td*$3s`jb=xQ&#(=ZP<=gS z(QaL422chBx&Ezame0|=T26k;A89wESc!G_!U9TNKVoZCO1;qDj?W_X-)DbG3*sqn znvvcWeNS-lStxv1Oe17g$E7*2rbcJq<dl#u;46v$4+4cVKMN@UaMXn9n-tg!1JupUTaCQGW$&qxQ(_2ZpdQ7 zA)&{TP`_O%c+D)UdaBm!Sry9ls2v<~GvmH}q45ek&~!$!tw1rE@%hXH0nDEx*E0{L zz>yZMPcaQqyAe$a*5NR(^A2$5`$&R1OxDBZy7;Tx?V3ayrXY8gOme#Jt#&EotOUJa zD=)%wWv`GD60T&_n$RR;@$w5Vh8SJh;ZrqP)s{n+^>j#-+fVOd{|eE#OIaJ45d7f? zSuCD6Cv|#Aui`uSC-@1Ed_Hn4ccFH%lWRC81dOMZT0B@7e=?$GgifTdv8)^u})E zmz|6ry9M`1q?aX@CRdrzC+*L2x0E?8%#?-)L~4)McfSxps?0*ODHoU?o~8YEe*`6ppKf*J`sIl95cF z%Jn0}eYY<{Sy5i3=4ioYoS&edF2W(eZ-t@#*NBH@Cptcb| z{2IC{C{G02mNJwQdHbgm=jfyE%<|31L*9yV208wDm7L&3taR}8y5b3@)xh!Q(Pk;M zt+lw@)(kR#X2!)B=`73Jt;)V3aZuOQK=NrQ9WAGFw0?iBLqY=SY+f+)O6`v7INT!R zLK_7mE$s7*pJi3*fLkDmfwg8ud<-_H#FdE5H8XySt*$Ftdzz`*Mu;9{emjpTmM5(e zb@qx#xy0`2UMK?!vBJ!U(+VoDfDdeE5`W@hKDwBxhK{e-(2JveTeXALh>&00E$ zf=v$%A)P*Us^k)mQKM#>lA}HrBZ^-I`#gG_M&<3bm=so&vO-ubqdv?vWn%BZ|TpNG9~4~qu3aqF#y`w2|k zUDEJj+Ba5169u_P6*ce5#fkYPUvM>_b(p&tSHM|xVD9kzbCfTUGrjDPoKTsjEr!x2 z9Q{mhsWrs>JXV;|vRNr|UEA1^i`Otqd}0szvRa3}TZdq4T`Tc`SBzA>=i}BaIfFeg z;mVmBmtT17p@#Al)9Qyc5~T!oOSo6%5!j^ZxULA!hGy%HHxth z3?B6lbHhWsxsO`DYTY5+u_vW22gJ8U)~BEZh`gZ8K ztB)7HX!Y=zhsc#6yImiONawO3?A9KaSBbWI^BJ^yL#SF-2MANWD+DCU>=KqLZ?#^j zcLkjN@p61AgwSA$9m&#eG+{?i_FEPDsNq(&MXa`D*~%u~#^8It@b?HMVOBShhiGL7 zGxvKr9)H9$K|Ee9Y;djYh(Gh0*TYs?3sho;2l<5Av3 z4D&rA;&u<`@p%-Cz?IRZQ-jha#Mgc(!t2C!ilvv$qEvC;*qq+EdTXRN&KxQp3pWwt z+ZUBzLN7Oib*VkoV$NN~HBQZnDzU0)dQ{2Ij9WTUdOUc4hoIJi#{H1?bTG2(hgF_& zlmCvj8m#6B%ogVDv9GHrm$Zk!_|?23eOt6xU#negy(%5=Jbivl+g5|O;IpY zrN@|5uF>IpJqXgE&~Gs~h2FLmOsg8Es}rT;(>@RGzsV$tVNZawt+l*RO4?&Ck`;ch zh#&_}o@Vx!kUMU5MrVG56scml+BKjUlhjYdx<8li(hq2~7RRGWb?^OFS<8=0D?Tsg z;j!ABE=0Vk;`g+ROHeFyTlrERTbYJ-fvqBKS3kjD_nFkEH@;Y7mKiR? zwl>TtT@PpVSfGn`uB62^vX|c*Ykbsan>D?6c5tS55<9{{Lzjn~Sd!fTF6ExXnpTs#aX4XyMrL*Z>Fz~To!q9JC27o{|{yTll&e&l6E(yWKmL3jBEQ=J z`lShNqGSzzF0y;^*;ETV{{6KPEg|;l(M`e!*|-mE6zL<#W|l%&(Akv!n+h-3Nl`_< zvI-rYEm9uY3K8+r`fA@L-dhJ1+VB|0#2(WnmM(ekXEfu5Vl{FKb3f8)84~Mp#|5P* zOH$a1Ur2v96~XCqeqx%;q0KXr+uq(fZXzwzoCfg#IdshO5_(`j|EH9IxrwOm+6v;2+tS#9pj+MvMgbH%jI$Bh!4O?$k zTViL@;SmprT+G{L$>=OROpoiK=dU#jUiBp*-b zMx?q$X+nA4JjOC8tJ->CYB-47dCK}%7zXb>%>Qm)ZEzqQ)M4gsUc3Zier3B=$$AlrZfCup>Qp>-;>L^EX2MqOIP1t$TV;P~Fj_z%b!f z#?=w2VZt;%Kf>Tb&>*bJc0zo-TudXBC?_`@;M0QEorfbXwHjXJ3mN6%lS;`n^RS(gp)f$+Jma44MjB#JXp+XLV zK9KBgkQ(taY|A=nhwcQaWwA6dg5jW08gFzj?v7zHnuDe z3^bbwVV>1%^fJBkl}SSoX~@fcq)iH%R@c>9#|I2mv;k840sP%Y2`ws&IE2f@KsiZ5 z1D$P+mBY5MF7NKHFDT^yPH}$;7iO@>%N{Rl1C@hK|tBu)AIF-m49o zzm-(>QW{2?Jf55&VT$RchrAY~J2RPAs;t21HtW0|Z3=FbNu^wxQh@_L!I3s#jZCw| z8py9dN^`MuY?V|zQX=#u?PPM35ZDj0Z)Bt1J|R`=O@VLI_gTpFxkt4NPMeWouakR` zqkG}&xp)}I-IYcr@B|Yi}u_szWC-?Xvs;(?Etw5;0Wdnt9&SA^80RUa5|x_J@xP|&YTP-(+Kud=}x%0 zW+KI2DoKhkWJz?5uSDLlo>M*nn2_rO6o&Im1qgh)u=8a1!d!hpg#XS`e8yF)hN@f9 zr8tx*Gf%!5_qGOD!Y{Ksnfoo(bKq=Q({`>@wiMZpI!y#!ic8s@x6QFCxTW6h9y`KmmG?<$WOJ@O zp&uW_Esdqfn@7i|m!_kW>JtWGjc3cpbZRs>Hf;Q?D%DhFeQZKT3z8JmdIa*pt-2oS zhd~RGVU70JMY-fhl%%P-w&sIJKEoPFHPlM<=4{?awrXjRNEWkNp7Je=ny~uKzCE@_fHesvy&g1Y?R9l|C2@ z-p_2>sv*&$ZF<)T!e`~*-+^G>6<=cy33H-16v)9T>u-Ow7y8o4=f?AFf4 zk!y>ORps_p)q(2v-E4bns)1}=X2fk_vCtr!ro$z^-02KN4&GI??Z$JB1%5}&99kNj zhe|4ww5gVy;TO>V;^L!JF1Un+woD9`9;hD5k=pU*_Z-=f^@EY&39@p&0o{;8O@dC` zyvZC&r~>hZG`G@Ue7+D{?tp0jsj8A65cj1vtTw9eF#F|_uV`0F2*V>OERo40+hWYK zp+jSVWKg*mCC8Rzg;^0x%%D?-8|-dL8d>LvT9(H9C^-`gz_lH_lC_RHuHohy0~;fJ zUJbM(=@dS{Y3-Y{-g<_>0U(FELk-KwQp4{tk`dn4LrYOwHb zW54bG)?MEl&Z6TCBeIFiUe*l@pPKnaXjeP-8!H;Qz0ya5tQ+3ORFp!@ITC82R8DV) z3vs+NQbq#)}fy5ACzM(d8A32!r^+U z<&FIha+iK~M78+n9}}%cw43wQpl^rmBaJIBsRx^^%cD&4JUg_hu{dY>w}W@`)VwUR z#jW>q_!8C22H1;K^2;SW?_*cb7Gx^hs)UriwxtW~BZV9Uf{z@!ha}y_36YaZR?PdO zlc7{NQcY*#0CVyQK;O-UI6gOZk*HFM!&Sn=n>QAriHGo&{uhloPGdA!N3+LXm2jC# zsE^&;M?b@;v;ew6`64jzGWY*OrNLPzBSrSp1m<@Y2k%UKaRj(H^G)v zYcavPX;H?TQ8=U2ON_0If7M{`!#(;@ek|}sg>2jp-`jRMw*8hPKe%FxI2?2{*jVz6 zLlT=YFz-QS!eqLyzOhA?m)(~52>?fsg&&yNyO5M@n_nK~b;;m9vXWA$XO%a4aOdwv zOuCn`?RRr~?+7-o)N^cTXck~SDwp8CG0OGoaIUZ~l+_e-7{%Esju33TE%s}roiNk(`HlN?kjd)z8lkxjSyi3E{kkFL zyxo_Gs(bh5eY@-q%)#jg;(WiUYHw#a=G%f(|1o_#p>;5o6a56-OSwSGU13zU?V8*< z0XX1`gRwBxF!^2HK$;E567h^4Artorha0DkIFWK`q%cQ283FL(zVVX=#to$4QdIp zd&utdn^B@AdPM(6L@`W$;hYZhp|DS`#<%`&tVm-i@7(}qlIxVwcId3?PkP8)^YgbG zoH{-)v--Q4*e+u6O{fXgh+1Bj;lMZbspU?zDHeOPn=_SheU|+fH*5ynY}9At)zAj% zlDnca;=_i`nG;(#Lu$~X$HVo>#UZIVLNvAsVhros;x{Ut1TxVHS+kBvE&3`tNVQ5K z+72@vlSNcsjxfgxk;Kuqqbj<4R8F%~pJ6i(KSpxYdBnTP6mlm1{dHf^9pne=_T+eb3>j z9A$H0MhafSvqLEkjR;G)+wD(9PJlQqCwvbl(&wiY(X2e+xk<(5+*~)?oi}&+vvADjiE0_`EEr^`gNhPG4wTI}L6y(xOnFLu{D29)qjF%2 z&s=rG;m&5KeSQXAeRFozbB&Oaxv(FJGgRPDiQJ%BRXyZD>^&t-5~RQd)8_TQk8ejD zr@-BNJ$wD6pQR`YLhgWH^p!^g8g`gVymy6ch&t(75dk>lmwK>mh)#E1Dx2`bnKR$b z*w~eE{jSrz=<0$E@Cqa&{Ye zHq}PfJpqopKLbyg+y_5MG!j%}QBEs#)L@y(}froKa_y~ zv*f@{%A*Z|1ayLXmUDGlOVXo#zb&lymT+~8ecm@qC)ybknKPr<4=tcfRZ#lP585PC z{1>~^_)si|b3 z52{qO&yc*4!WdOS84~!d*66g14gVz&dI&O7iGQw;Bpb^gRK1 zyF&L9lgo(a_qxc?q^98y5nXbGEz8W&g!SvlQPs$Sy+_J|99=UVgyLL}GN-c>w!n7MssV6kqU4Dy>hXO<=roXbk>>!wk5m5oOX=1u6{XqG2Mg< zN+am;=3aM#4e{(8$3?V(QrgNWz7(8e?1OaE95Rzus-xaGiVZPdSX?)6C57Nz`1*>4 zJGy$B11P}pqELn}iURkMePB;>jXS zmGlGC%}aW_939&UIovNw-Eh?)@!$l^`*m2lJSc7LattxVFWl73v@}!pkiKEQk04_RUFA)v&lxl)_&w8ijdY z;;)0-T>X29U~?pO|teLul_W;n=OGc0qBI9#tSUK3*rJ zT>6-#CC4?Gy5-%)7O~>wBC9$hvr{1MFa7P>0i^BO=*7@|>FHFDtTk{3x~7Sq%1Nou zg<_z!$oAW2Y+c%Cr z`3Wy7;4yoP^mg5CZO%eG#vE!1FMV^_3`PDQmfk!b%Ju)_uhS`Io60WCDcQ%qGsdTl zgOGhUA!N&*k%qe(M;c=c#+oG&GdQ-$8VNHKvek?--7@Co<7kIe-skB`PB^)1;Pu@y z#=tuVagkz+VDO)^e*UHNa)dZG+^6hN!EjxC-#$hoqQ8%>AYowR3c4{29FmI+u%(Ca z#PO9DH4-B{M@DEW=`Q3X+Gi@PH^4NvtGJ$Fdk!tj3dh9QrWpN7`PQufN^{a$(!O5y zKd5%sLgMO>cO^^G7|Hgix>wm({w!f1c7qP8bW|qR>CxcqzfB7|D`%M#jg-l8jgNKC zA!&oYTZf^|5tY0u5a&7{%c2J&Ys$8t1}yJsl;yQiJeB&VC!(cZmz?)M`02vWiiy-^ zNg&kuo{VU^!g_FxaG~9U#%_QVq*?2K|)x>+kjAfS9062Jc_H=>%#0|Tq;`dv|hw0&A z8!%B>nG4%dq2Gm`Gon3PG)98`C32nkPd=EDQNtioL)H9w_xk55GyfqDx)e)xD_@!R z%vF9L_X{D0pzX-Gm8?nhxK!A`-|MDV`={Z(Zbus4lgFqlVP_XGq#Y@5?+jh@;xT*OIkQhbiyy2{A_fcai?T(3$Cx3-G>2HHF64BCpMg7z7T+kH z9anUCN89Xlp_-p5v=wHF|K9gKP&OC&cB?A&OG?o62GOvqLF2bnl{zgOc`PzOO`PBB z-ldVo0~_TAzs%aIwEpgZ1oM;pY95%X%Xo>~2ZRhN{h-#eKF%`xBCqF<-}*NJIhu(F z`wrldz;s~;AC+_Az_!y7b{{Fmv6=nWqrFTR^5Plfv11wZc6~qy-Ca2!^ZDAXl71Oz zn)$5z30e?kT@aO_OfUR#|Jnqku)o?04r}Z%H+o(- z*W=}0B#FwtxnVI(KcA-aZF5lzQK0^>Ml4`;iI^WT#Lv=;%p7ik2Lv}se71}~f>{pw z`CS!<;UHy`qztUkDWc<-Tv7XiS4Gupijt-$d+(4J$Y#LX!#`k2XA9^7B-W??GxS~& zG#pIGjJpCJRCZTAj!Fu_ouwoZPfo{p$x`$eUNjI z2PF2t`?Eh~}7L@9aEHc^K42>wCGL=gPC@`u8U$tiP^=)Ei= zY|loJmMQ+7SM&VXvVRucGt8)Cha^<&D+AG}+UGdS)H&7~OjD-%l>JtG_I<6PdP z<6~JktH`;+N@fQ#kgJC=ItzP0Ji8TTQtcB6!*C$jPtXUpr5E=4R+b+OXPkrsGvk1g^-k*)qH0eb{*SK*OevvtNQhRGznfdd9d`l%N@Lh4wACu=d9C~K=JK8#86M*_k zpFi96nEFt0*wD#GV?RdFcG<&BKRDx-eCU zpQ4#V{)VMPUih*9js*l5V%#YE2BthX-xjHH7)5iyPg~Kk-ygDY7~Bzzx@jhZ;n~tL z>ABRPe{Pp%s2o}7a~JG{Dy*6=B+($rV1i8otZAYxfJ47SM@|G#Ntm{>Wx6qIIkyoTK{0vM%%`ZDw)fao|t>JVg{`6pB z_(&j1VHL9Tnf7Ak)BNhMjBZW(iZt2}kOPn1MIlFKVCH_9y{d7a_xVYLIC8TqDOI<^ z*k+25Y&YK>S?{?e_?xtSJjUeGyzRLKckkhv(aRFPsI>IR&cG9a;0nZ!7q0{FCGpQE z@fH&jSpxyjd;PL0N>28Kv{9l1L7&G4D4VW*xn`r&NkG|%(?Z`EfsJui$ISr$(k~wc zRq6}q!C+U(6)x&jJSz5Nz%d6J=pj<}zRf*mLt^0e=GC;eyw(c0tMIfP{RCzSJuaIg zzziA0HMjTWi3B|`!DVzKSso1A%(j)x`(5m3%<2BkBzm4Gs^nJdWOoQ2xUclnOTgv& zhkR9pxLUY}Medeiby*;8mf;#8ai33XLZjQN{C4NXtC$mteUqcXcI4;tY%rT9ncXDr z{xJ;}pk`O$xV|J@+l6k*_JfK&3M>v-z_?I8=AQkx<&0y!`)j`nMw7&V^2gAIG=w@d zn=GYfR4l3SZne_R{%FR@qE=@u$T125YTI=vTO{z9@i6*bqBQa}0M?PtZ);Gd{hN`| zFpYNyxm~c!YW7he0=*NNDx73Mre=v5@lCd1;ufXB~Bp!gtdV? z%i*9m>;uC1BS-G2JW&}P!usAfj##033ul?LR+8aPeye!`Xj2yE?a3JzcT7nQa_co$ zc1BG{Vke-$`f=F*G;Q*|65M{B3pBM$g^8YeZ4H|?rgfLV-iO#=l!XX5P}Mn?P4STv ze%l~eq%@fwVtFf;0G_2Nn|+Tzp`|2JgAKB@K?cI-=4{|tByqnxt#s<`M@RW z6<5!{oF!-qD*18znIMDVsFXs$Uw(yGKO=Lt*&j zylNVDDF}zC2fQR%Dm>xb0!*1gbOp-9<0P4wPhgKXaik1M1r-mZbOZbv&B`N56lm`& z=@sUl+7@*-x|gxq5i9NDF#j_ZDDfwKr0_q;B_xEr3ceqw_Hujobl$hs7T6D!u@+7~ zVGgKn_=7Cy+bE)JdnR=V_6EV8VX2EagC16ym!rWx1UCb_>@9oZze|puv>)*pyQhJM z`hu*>&XDA<>ss{ec1?U5A!lH%7fgoo8e~Ie73T(gH64@M6n#Zys)*@@7c*YrdE-dp zRx3K;kD)Dnpp)&9sqd%nPkuXc^;qpYq(IiSTSt}u_fpH4PH{E2S9n1z=VqGC^vhYX zejXaPRHULewLwRz+eCH$%lQd(ICVrHaKpsnD93TEhx~LrT-2Oew6~*owXq}Uc4WLW z^10n;B+|W{cy!NT$DtzB+GvL5S?CGYx)wjKeckI$!zc$xsrHGN=jM3EonyWh(;18u zZgTy1-IaeuZ!}V(nr|y+a~>tJ+GvN7Nevc>*qmFhJ0h*+8MoR#>TMZCUj>p5g;&tx zQ)=*VM$&@s1U;bg@H$QiMMNi5qy6T&BEe8`ouTQNFS9%;wv^9$DppB+ww1QD1@&*c z__70XaV)jkr()y7yFiZ5H3O2>iF}qEx|i?iLl*&u1!G%Uty)%rg=wQU+>V z2pKV(Hae?B#o;}PfGZSTZ5gpuG>GAUZJOTU!B@&K$e~9*9g8p=1RHYNH--A9QsbUr zdCrLY)HqXSD<|A+-o$!V$s}bb-AOuuHtJ^f?y9g_=y6X%d*nBhTE6$!U!XqVq11cW z&;G;-qDlr6by>MDRzLHG3oU(hDNAfZJg$6L`@_C#Iy9EgF!oIGqI~CdGu{1ekGvsJ ze4FL8zy_?;$yB#}c!OJZ&@|bDCzX&&^^4S^Op(Ma=)wKjLH0z%-Kq8RPKUKX3e(1^ zgq}3#xuM%Pq}?CU4w)Y>b}^c5oQVV$=n)@ba&T*RXpgI)=xTl&#ldAHZ2PeB%K^j zSyo8?Us#N{1m4PQ=DPA9=wewzI=`M4TK0(S>^joz^=L$q#m@wS7{oikPQl@Wsh7lC z{Sz9gy8Etl5%E5y=q{H`^Ch2-xU|SpFF^#CQfaGR)h{s^l1VE=)%nBToWrdd?r4d3 z#d^5|i*8Q-&ISuKZ@HQ1O#E)VzlQ0Fq`<%7eDQ|?4J-qG6vsyWSGrE6JVZ{sH*= z1s&>fynuKb37LuzJ*wminKPhgQEG$R?(;t(e*jgK9-VnOAALJdP}*|aZhnLLd8x}N zq%5&0GCm>c9yu~dc&a#6Y#S9WQfe+nEmiZ+B9=M-`-2}A@qDYX%1S8n-Fo~zs8;{f zK!S3N#&;p}#V&2qY5VMge~x~${e-zf74-Janw@Eg-9&iwu;<_&7!OVDc63+gbl zj;iiKLXHnEWR7Y}QFMnTBPUpHGPY@bej58B>*Bvk9pBNK$$@t&;il?`X8y)L+h*D% z%fc|pz}`~S!y+r);pVXt#5#LPH<=FmA(Z!yUgOg|Mn066C{{O)#d9oOb3=TZ%OX%+ zT7`c5T}diXG?=A)-7HH!N0hxVy22m7HE>F53u7#A1C@#ZZ9 zPp_M>CdbI1DyvQTL|*YaJ#C2i)mIPk5>q6bTPZY?^FbCk1zf}&6gC>Q^j-PA;zp@j zV8-C>AbK&MP%-$Da`+V;$teT3BW*xI2X@0~M!CP<)sb+2$s@Dm;|dtY+lgIG%D^=f zvi6fiP;oAAffd$l;w7s?^xFAJBh%5Tjx*hitUQPBTFCMoBH0O+i5|xih&) z1j8zPXSYCVbZ^OR0JTIi?X&NrD_v9hm$(|a(y8A~CDVb#csn*r=p2Z_#sjsHUfoG5z)>OFW z%TIzgGrH&opY#pzDx( z-j+9b&gkYZquJ$*hbJ5IXuaBYEn!yfZ_XSQ|I)aws_wXC75+WI(<+y;;W}2J^sh>w zYqp?M@_Q90!gE@LnmH#|!oL%?INZJHPB# zyuqq;s48f#X?f}Br3bY+D^n_z^)Z_)@a)d>y2Xa@{Hciu4lt(oM+LtMbwa>fe59yw zR${WI%65spJl8sFfEV7f>hhMprbY>LZMaN_$(virPGGMf3r=K>1^cqJ`-U!Hh>r;7 z3-8;hMOkL!sa4fi`uyqqOdeVJu@i z+&f90oeCz7v66L}<((Eww-rOX1EhMCwl#D9aa&N>&wqb@hTLdkr6tpi2%j#_G_1Hc zA%(Y4`HLU#3O=Ly?|1Z`WeV3Yn&7r&cf?PBsJGOK%6^@zFO)ghXVczVVL^+Ga->}= z)~uAnR4Lu)^mQnKp|L0XlY*>-9`!ME?Wm=%NVKg4^ml#I8#Yn#PX<;oa4O-}HkU<1>5W|cY2YLA_ z`pBsDIJ}@JQ+;d9;&OOVsOe&PJS1RQ3x)H)(ytS)K!qRg?@TT3yuVPcA`nX zL)BK&6YP|L4Ts!sblUCgyzFQxd2doce1N`KaZm@ksyVfN8wiGQ^C7ERdN?wQHA26U=`=# zNdJF=%J$bQrJ|NT195#Wi#9Fv!`-W&8Ply5nz|JxVc+j~JW{L3=RM-n?_+2d2HE#- z=r$WnHEb~=U4vcS=82e8L+>J^kN-MVrdYnru`F7P$>{TS*3I|HD_3%;P^__vK3D(c(;f*teMBU$q| zUdK@BOMSH~dM{!hB=%?UtONYN5MvRg@-9~TwZ+i`hTl${p+2Sf6dQ|C;zdD!2)(9n z>i3HQKjnVZUEjo7;`<`?Fv+lCOJv=;rzyqISs(sBaq#Pqet* zD@+2dBsT{71aXc9WBl&}R}N?&vRV}gyfWMEK>9pP_Xx-6&5L%WWz_%E*!70RPrdTb zQN!|7Ku6YWzaLQC<1`Q-Af`=m8aD{nrTU1*jLrKA^!k;^~r1orJ9QnW-?XKG3>=yVarUAM9J?^-U7#$8$|{!XP?LlbjxT zVDg|;k4GYRz(?t}!hY@Z>{F6$!5z4IpW4ClpwUIAw{oZPN}r=%p~RQWoG9QO!+?h( z5fF#LW7pjhk;y|&W0g|UaJ|6H%BxthEmW6?vSozh*TwFw;XSa5H2k|4l%EFcSQw!<3f3< zyAKBG4=-&62^;%Fu38>c>nl;$4Ke;%pm@(25^}gq%E71MGq{6Q>LgmoY_4PuZ?Nm8 z12NOi4&Az8<&sc=kn@Q`w;T2O8wR*M8EGoL?~syr)^2o*UP~*E&&Zo=G6)VAL!PsP z^;EbUSeOQVcgpd;QywOBjqSyPq?oqHd{0Yaj$p-)UU3z`TxxsT&m{gK`pP+AoliyJ z{KLjANwYi}`BZQh564E$Up+}ZbKlyAK5aftE8Up3ZWZ!$AyUOdtu0kVJ4((`O7HUs zWYp&pG5E(D#LESE_iB|3=I10G?+%q(L3lm)Hx%WZ)t5?g`hhVK$(Fv+ltzwI*}8b0 zG=EpF`nlX3_vr)9#zhrxfrM*VEQgGHn)HTB$yrQXQL$qdn0}5L|rRBQ;{3- zbOEOw)!U{ElIfqK)raaPwCejyPsL(c@`03ZJ-p;OQJ2Cga^}!{ImziAz4pc`YnqY} z+Eu7BOHp11;jeCIx!)2Oa)FE)?@s^RF8u4ML?I_o@-c6DA zIsBS7n85U6QLH>FbH9?;-g~6m5mftnv=b-5)z7DZw_IH2A|#D_*&Q&aaL*i7gO+6d z$sPe<=q5|bolRP`PW)v7I$jvmAo|an}kuGixA zdr2R=(Xmr-iJb8c{?L=6I3D@DoSy2bqw!OjhG^UaoGBkRvIhJ%`?LKmQy4puQADc+ zgyplmoB^~t<)ff1Jyc#u8(&C)jYr*-;k!(o9+)L%<|4#zl-WXZs0j#9c{lflOZ7%T zx}`2NBK7Z?wF*P0-{B0x&c>>B2wgozM;-G2{;FS;-UX8D5usa&2`ZiJTtJqEWzcOp zs{m~`7Ocyr)Q>^6O;1#vM4yYJSIDfZeBGG&nT1ri_2l$L?3rsG6F<;~I0h}b=q3d* z(d2o(BO3brT3R&vrNn(9v0MsGF1=9ecfn}mJ}aMM!>gu4vRW$NkW*B<`_NW2V$K5= z-Z%opVH)8u0H-!e7>HMkcF`TY9U-3WdH2S8%*8T44FFm{*ifW0`NZksX@xVRPl>Dv zwi^>eV)imh(%V)RvJ(gI_x`~(;vks%DLt1*P4o7pp>zmET^v}nJ+Up>o&gruqQSqD zQ7ul)*_1B*Tu3HdLb|x3#^bXs{+f8uqB|zirUz+xrhWe|(WcHf8rQP%j-0&9T%oHHo^-1oY9`K^ zo)0q(vcC2X!QTlMFag`Eqo^0O+!wS9ZZP<@RkFYm`U(?E0^eq|9b9A6Sxef|0zO(Z zMeBa0`w185)-v!VpE^Ug5qmdFTLd^G63#AC>!{Fi#_dYXfJ-iqhz&31j1$p0SSR07 z&>~tFYK)1JD%g@Zg3Ciw4ceh;^iVyoBBu%Mj4v94{MV>MAQ_zx*Li}$Y$Co3&+ zt@UGTDta$F!#S!)*w)Wb+UKFHwWD-dfW-psziolpTp^A<@b_;rRW6x*kBP|>Er`WSC>=saaen$8RlSlPh$#slhEZ^hU6F=%`Qz_4cNHzZrTe>e(s}8>F;U zg5DfUa9j>craeKdAB8X=r@ZRvoZa1Nv*2jQg&cvezypRSHKMN;nE{^HXGV$D3y4n< z_k2_${&kt}kJR(eVr~(H?W$}V!fY=O(tRw2Uu`-MuGTvDkJKq1PC0HEX$>5Tey%(u zj1*O^u1j4kTt(Xv`bAhy5eKR_f2dO4w*@z`XEY3RU}~y98|j`+u|Brv8zToiiYAKR zzAD1A1D+j9bBux4I6=-eRv;VMd6d1|3SJ!;l-`rN(wBzZSjAb@90wqlZlW+C_Bv9X z12ge(=xt%?lGEXa1R#hL_{yZ=S-nEKq&4DmV4H0Vr@Vu#F)HG^)@>!IQJ=ZGGhytg zV0+TJ+T+FTiz!5F^FyOl#X6HROm1Km0>iAnziY>!L zdN(E~41IMfXc9J;#g6iHdG4QC7qu*xC+<8DaX+x4RvZ7M-T^k=c(&iMy6ULnYN4}C z8~jG#zLZswWs+0(qLb` zI_7HGXWyxDvOCyjmjfqm_qLd3B15&=XT2Jzvg`>^C-PJrh*97zo$08*W@@u zj|b;(Y2E#s7$3+sjMXg}vA&Ws;x4j07x&i=*-B+h)C(!(HWmA}$D_=jQ;RMqYmTzS z<(xSMT`#u}%^`6W<|SD8Jbdhvda*yIDlNz9#?)AID5Zk3DBoQ7>11h_XE(E@p%FV^ z^L?T7(EQ9^hZP-M+vj2JIQK5YcVh%i2Ma?_$HW^xJTU3b8{ow;1k)_=#>_IzU{s*xzhi%Hf;p9NVRT4GBZw!Rux5TA z49~zqfvO}r^ep;()&Nd2#-jBMS|vK1`O6%57RR{Y3hqH^n{l*|cv#p@j8&L}b!&G| zV_WB$*huZ|f=E1n1NO$Q7AMxdQ&`iHc#9G+!Ba%%wj;&tT|ow%xW)TwqY9Y6rDYY> z$mwAbk9%PyBoNlZ{gb*Oo_nzfHl$W%o}P%eZv*G2Hg&${b>)|b2-xx@4~<^1tblj5 zGIg4S@cV)8IhG#@SL=eEQ|~@hqM1HQ5>FK4c(Nwa5fU61=EL^BiOgQ%Bw8rIcqgL) z#3wbN+O1fEq`+!7)Z)ZDuFf>t@1@(v%^nrW8x=btWFXqJ;D0^EyLCAO-Ot`(MLa*L z53W6jsrfyk&J2McY)+iW>1PB@R@sBsOHG3tx&1SJbZIv*P7{oWA!K>C?Ho5}x~r7) ze=FWOXn+_D`86+1@bUNRgZ|BhT%HS$i5Vcy229jy7^D@`ztWjal~Mta6KN?#dLuAk z)D)i}pSvK7H z`_pTkY0h5a!M5@DpIIF5)w5U0(j^9dh;RL2T05+Cr(nb+@3l4279&SlURg|#i+ROu z+c^+gJDM;`>ejBOM3V|(a<#w%kNdN$0Dz8WheL*x<`hI7P@bqh_>px! zO*gzzzr>Gl^eX=P4`3AHoi=;tZ9Cr@+<1J~ zenQX2sw7E$J&S5_J*rks3bH+JMGsG#^VVYCe@5(M4*d`COIV$dLYX?v=6@PYl$rba zP}}h^Y@ls9ImutX%0CYOwvAm_@Hy7M+;MNVXxi1ipekiJ8lERf^xeRa(^&UDc+HUppse6G^%%gqun_mM8|~(ZVrc!hgVmnP}qS!uoTu zmcn$Q*`wu|_4R2VLqm7N)O3>Rkn>(!zstkJB6SwWzx|UBcR9OY9wMxFckjS((8pYZ zm{ufKHllO4!_w__;1R()R?glbDrN2hAFs}g=U>FD1#gmxO6`}ccJ9On$g1KqT%UiL zBzkuTir*=wh|mf=x2v{B`sktT&*kuYb2w z7812NuzBuDRN4?yq}!?iWpA7Oe?W!Z_sJ~3fgUB+y_QN36kRz9zyHq`*oy2Jvm?+!bzEs3+~0HFtvqM|RQT&K(d2Yl(8NHf|yt}LKUV>YTTaQ-MaB#)d|j{*Cxtbg!gUQ?+^ zqG_zx^xJ0?wEEE#a2xnViAHNBNpOV2k9|ZVSiKao0O|)Nsb#&Cmb7RzZ^;r%jfI{8 z!tG~z6hER!hX~uVfcwxb=kEC!kh!#g1n|2jw$8gx#K%P(mG+8K^onl!yBofWoCOhI z&I)|+^;jUW9*vMMRyY^hP%Yu;8hVGQ3kOB`Dl=Vzr1ez|^HUamPFjC|c=w^cpu3$9 zJ^H}6DHSh`Xsp(gI)u1TTSgufp7{ZWhOkBLC6oUL()Eg%|A7lU9PKw&C)(>#t}ZF_cGy)9KnB|q%Ax~}Wx zVpXG{z%q}EfpPH@5-&i~u%=oA zRuY-@9v2ia1pC@Nxid3G!QHv_fd)sK`a2(D^(o;C+V4!6C7z}lb4y{TF30k{QrfOsT4 z{gWY?IQP?Aws^}X_F?f{#Z1)*3M?(Jzh$*&b781vQ|ZOcVU{|5Xs3y#rXt4NX}7K# zI5gN=lqbYi4M*tKh_Y=(bEGII_b9<#N*xe6EMcwip%(X2E)S&e7RthrJC-JV)!wPl zr2Y3$@x*b_lID_@-&MG_{CkNDYS%ZB~pO?JMQefMZ&-CEU}%^_y*2}qZ))TIXA4n`sih%0zjclzkZE2g*5^8 zrXorGN-dVycOmCqx5$-nWUaf{#&2~458df=>VF+{(tdoM95X;l@4Hz<>}cGoFbysk zIC9V%SPvWY9zT>p(Vc|LgQGg44@W$ol_v>KNEEXYkNYxgJndjssV-NBZ}&7k^FPqq zNNdZY8g%nfjV)7WZ&-a^V;6ARfoJ-ett*Qv>?go99&qW8>xIG)>YH;byh(v%=|nJc zE}STb${KWD5%^gz+s8e{(#Ig=@491?SI2orhxG2ZBb1w2HB#^IjY-{i-ap^&4hCj+aErGmD(w9p}g_^#n*fTBD|)BG6Rs^s8SEsuXt z$l}%QZB};9)^yE`q?nbPM)xS9LI!2vfCVG(G|GYc6m0@N7I1-jAmMM8<7)`ne04X( z`@_ccyjx~J->y=d(oc(f*sTwuzdy)zwx=%ipEBpfiCq<`@_QH2SXgFIrE^dYNS5it?dTkos=Tf1NMs<8ubk|JROD17ZJXP@Sjd6JbZ)|6L-CJOw z-Y^Ttn<#~f109SM(_?65W+60?IZx?5jY{h4QWkM~wgWf8e+qiKWZRDE>Jt6$Sn!(y zIecfCn_FplORR6mL*KWSOSR|9-;RXP@RlmYMT)!M>1U6YaSk_hLK5w zuug$M13sH-|0h*NG>vnliRbU@K-QdEef(q|Q)A|<|Gu2$tUG9R$Rv!%?{RuUz#TOM zaL*Mxky98II?@xJJ*JTMWi)8A*rIeb$PObPWpyqKs{p<6&^tSTcG|fD8Bi&PFn@<@ z%A<~-MDTD7S>c06vQ%8luN@jAOdTYO2%uF7Xp@5LKu+Dn>dziq%mbc0PAFGtCML|> zj49q@e*P{(X!1M9O1`ujLDZ2`yD^lgkf{;7weD6PRr_?JCA8}`GK<*ZRs2V;!Z|558IfR>*}{VfUM>YE*1949cz||RE&>Qa z0w*ic_g*kkf2GwSM3=sYg&YYc}#T5zl{fKl5fWR;Jg)5W!S%eC_gtPnzanT6x!I1 zRav<*+A*~@+U4?FKkQd`77apeY#k7knSUb>xGL~LpE>8vbwR8wf`|k0+;)RU4;-BZ zUH_#W)dxRU@+}v65JNa^BSYCCXZD`{0u<4|KTeeQx+kCpP3{#^@A=Cs=tofXf5*7z zjT2VVybYNb?ry4oQZ2E$y{u_{rQr+aKt|<^sz8nYB|4;Zl=Y$Snr_P1!@KM8X0pod zTA74h7K-1tEojjD!;RZZ(#Dh*i^z&7yn53Z^Zb~o)3C=!c)LkoAbcs;hA_h{Exzz^ zfH)nqrjX|(;xSsX;PLE-x2Xh5g?na*fP(aKFEV?cv6Sh1URLa45OkG%bw>B0U`P`! zDHVFM`=KAh_812;{SSBX+EM90SB`oKt|5Q8(lf&zBv4)%QB`qUwUKR>KzWsr%ZUqX z2WnxD2M&$8o5xx_KV7DNd0w~P{n^mH&GP)a5xl;{=i5hR#P7sw3}S=;904`1G}kEM zC#Z(Gkio_jqXH+!Gqd<_fRQk3e&!HLCx|?wM( zLynk>j#TB2%);Q=jk@{QY;8Q&=iSHh0AsCs{vlNYW(<|bLoFb|i=5v(GCPa;G;+E%uf@2i1U=mA|u`?f@CJSC#1Job-jgY%qZzwW6jog|94D0(GUdZNf{l0 z!w&}jp@4v>E(L1Jr2Mp5%Y`lb#erAS(ruUDHd|mLAHaP7?ipLwgQWYVj&&eqn7J=$ zOD!i#EK6S7{a%?>qz4(*dXLxg66OpGB}t>B_u``s&6nya4#Xmsjc3!d?2ZD7QaZzJ zGR()cRpY0ka?}&j1Mbkr&l6=Orn;Seqwy#)Dily~$>apmS2A7h6(yVJSx@e?nHbyP4%G~E%1e4f z*ik(?m5=cGhc{0d;X_MKWWIB#yrx$yrcB-^lg&`S56~Lz)Kcj6iAFp+gaa#?bUZSZ%5lL2ZE60$zAsZ|cG+XVQKj!QSg^O^)`o zDbh7{Yd@4jCM$T@7w5YnpoW`xiJ92M6mtZhY%oSwf)xF}(<9HFC=1U?Q~?6Esgip@ z)*wQXnMdv5IJ{Dv+LyK_tDy}aHheY0R?!rC?$+aOJsFwOJy2vhD*Jax$;e9#z|Fjh zll(tmJt`(nJfVK5vTTNpGCBIYa&IvldfeGF~v4}&6d+~Ws*p8IUw zDQfI|9sDEP3HIqC##FPPPI@n*efqYxI73m+T*Ur7$xJ+3H!zv#9$ZMU)JQR|@MB-; zo=p@1-7_7mwN@=)f=z_BJ^>x+&HomQ_A)zvs%Jr&874UPi0fDJEH+gg@c&7q?Iu@B@Mb`D^cXBD=vJ}z zxuW&>MCEn}jgX76JMA)zN;=df`git->D0RjIIG8HPZpT?T!$Ua589+lN49={@N#Ji=Bfc8#z`_PlbJWy@;7xqsCY(Bl8uiw2h zIsUe)*=>a3aB9tYC0SttYj^FWjkv#eHD6{U+*-nhlxl>tr_k=!NmYq=nlD-jjBO9P zWLCT_ShL5G%9%^ibE2?t<>aMl+3otBtbtUgElz zU2{PpSt&unAJ&iI&4EpyOBwJ}8p@Alx(0VxipUCAolzEPGU)6NQ<-TCr#YS-qj&I%7o z){5Di-1;S-ZFgA`x;SSX-lp5_8SzV_&_%M#QNR;zz7wU*f{ zCvIQ8rgr`P!68s?b8&MQZ=*x&ZKC*mou<%g>th;sThZ z7eWoupqbxwt<^1A{zbLd>c2*thlBNl`;@MVYBU8a`(eH4a)z$S3FNfSXYVmX+IRU7pjmvPVyp`4}Qv9#PHS$~jB@BzzkIhw1pBIzLc-wwx$)+5TsYoPrhK zpKoPs1RQva8IQ0I%nYq*+EVbw{kP?X%KhK8@!r{s`50M{x7%d!AXm$Ady1eh5w(Xs z3C@JEViQ?Ol-~3tUamUl0pLde&JR#%VtEM3PuqzyfMy-aZ(#nxMCMQ9oD-I;y!3BS zx-+5j&1Uhu_UGUS@~Krp&WrU+yf*oY%EE(F9s`%Sg^*nJeAea z_A=*6nu+-k;&>B~%=|?)Q|GZ>#hv}Sti=?ps#)aQg(9c!-6>lRG4Y*unbcK;#Y!6i z6T$snQCzuV9{u-`-2U#y&NX-#zZXA33kCO=H0TbKdT8|A^Pzz?=rC&%f1s6(H>;PN zbW)ssNpogyi~*Js7>lQCBRvv+2KQ0CfRv5O3I2P$^=O$t8oZpb+G<35O;XALtx1Gt zVAf{zQ$g9O5lDA@RoL8N>_QCXJp7cnFYWFB@H}R&Kwa_=;d+3_SD%CKmH#4&B0us zw1L}!B|ed;II}1C(s@U@G-|(Ng)(C2?Y>Hx{>~LLpic*}wstfVZ4YnK5b zx!#-HTu7|BYU;vq1+4U`y+Ry*7Qg$ByAekkXzV?|(N0*l?E(nT@f|8p(I~!nOy&g5s%Oht2D^Torh1 zKd-Yhehd{){YI~-#1q&zNXn)Q@!ueZMVnpNMFUS6zf@H0cgjL%cNDJeUa)dOUUP|2 zAtJEz=n)61unS?O(bou>-+z8j!%6kPBV^nRheLb9&mMa2a?aM85C=_Ty7~n~ZWpN4UVVQOFsF7H3{{Xli^QvnZ< zC!^bTg_2aeK1O5_i6&J$s1s!4>TO|{qYFLn3B7g{M5dk{07<=A&+!xI6`av!GYNKC`5)Fj z`|>Kwy_9ZmbwIADZsi0Q!cm{)Y6VN;vbu#>;dZPcDmlHLHfXhyi9H26e-e}Fy_oi5 zfd&itQo^>(Ddx~)G^{;;%c%U8LTcDO_kJJ6Co3Nl|MD;$2>97t=E!W*YD?|+@A=$? z@*KgcA9@Is)n=A$5U&{%vxWw7m%W-|gOoG({vY<-t<7R8c}cuQIn!S=#r_HT4Sn@MR2b?c>>C|`UR49tscd#^z%KGYuWFT-jQ$*P#qxE zKfkAK8#`8M5&O7xz-8Mt7(GdUCzXLUqteB@OZR$u1^G_0=&sa+!rY0ml1;Ml{YD3F zWXb?B0f%OYt86lHW-nT<^D6jrVkKNmV6}wFh4=&sr!?wLuG^K%!Dmhxb@u z8k1EG^0kZfEx4)@(_xOqmRZdc0d6;?d3IN_$HnFbF!a@f~{aS2?8|&xx=l_qz6`n4=1hQUl^Z6Iyz$jxKVx0TrT4 z!C(*5V#A$oP}riHWqAm_*hoGnClniA0s3M9n!zz?mgJ*s%%yDP)S-~v@Ty^F)Nrlu zSK&iKdy;e(lNdk)>xu{=MF-t&AnT&%I3^PT^PA&AeAV^P@|8Xpf&M`N2hOgq(d{Bu z4K3>`MWw1hNcE!U9?!d;!TZ+_ls?E=M}fhaIX7rAy5;sln@6%V>8*>%bLZKiCz>i^ zJuSLD0%xlQ$?FEeRo@XV?9#MwHT8Sv&NexYT&WS_Tc%p}TKRHf>Lo?GM zOj)br{oKPqvHrbB7Oep_75O+_Cy%ctHS4RuusMrI^R>AZo>`i}@L5|cE5Bix=z+Cv zOIQEyH>#L%*vLE)XZ{>+CC+Beb*GF$`K?v)o#vZ|j_^rv9NN#zj=G4r8VAYYRfR)+VqNc@ut2 zB3z#!%Zgz~l{?JCgDQsn%4CN?I2XsrwC%gv-17HJw?+Och}oxdb1*Sq(9Zr^uToLs{&9;wX($Y)?C7YL(|J}Wv+mv$48HC3INPaW<9ZbGFI}`%uGF5{- zUzkFp-(I-^MO2~?<{*wGrdf+;;4wC8Ph0HgXyOnS#D)BdzUzizP1DM%i6|Q=UMk8W zZUI_SMEC*})h*Mj(uYeB)!|4kU{f#E6N(*}9dv21+GjNdl<9Y8&U83#O@7C#2+ab$KfQx~&_wy=#x zbH=4`D+R-Xh^S)ywQt#0a_s4}vC+bL$$g908>JfiV#{e_ru}n`wUU$SDK6gDsa{ec zeQRz!{1FcuILZ?z)(srKC70&8TU3yI)cdn#igp*JB0|Rf(%CG0b^6ff$x}Otc+ffbf!BkEQTmwmG3n_6i&2V20SCL8jU6gcaH>GKhrg|{LJB@Cp?)wK7m}=E z_iQBOB$Qj}+GsF5%wb3_%Q?wxgpgFKqug(qs>WihK9WIcd0HdmC}YKM>1_N;U!&j2 znKTKB?Ipnt*%A)A1&C=l&U(rwD14C@*PPzeL3?X4h|3_|>PQkZa|e?bh(d)zYqp$I9xB0@<0KOnvob|>&F7&N7Nyp#>mWW$UH8f}cqf%UKA&Ixu|}%n2xKTv&^)9p zNevO}Gr)_%v0Y6x|DKSF+i^Kc?J)a8Kh#qO$fZKr6oOm6{jn=PvF?3%~Ea*YOs34@{;PJR$g8Y&aShL*|vP^ zauewkl^Mx8+-|>o=47NBVI{7FHxwA&ttC0SGcvJ_i<>MtE)*&zKZaePO&|rRvS377 zK*O)mrl2x1E=?SYmyu~P^b02hB(kNVRmkBw`bFfu{xk-tV3wH7FP!FF&XoJnZXZ-X zV|0=2Z}y?0MkNBO40liH~*So$YahRV~(3G zXKc#UlNy>|ZEEZH>N$^QhrGyfB!?t?6Bt44( z8ul4Ye}h?Q^%wQ+nbcHdtXnrA{gocFZJBNVAp8Ns||Iy zy7y1k4vLrPICy3N1r~s!_uZ;ATqATH%`e3L99tMKR8PkRu9Oy`pTdRwn!lY--Be z+uP`9G1`n!E>SI+b_K0TK@RL;eZUS1q$3IOm{Y#vexY1=L2U2hAil{}U$vXMN)b*| z8O*J=w^=!^@O$E;R9(;e{ZENM?lY86{jA1KcI?xtnNx6B{(uwh9`B-*OgY?EbD~F)`?fIv?8A z1wfQldzmUwo` z$Ax_~JBjex#AICbhO8a$18f3?WYXC|{72`sJyQWIN^~Ixg%_ceAlA3Im%8)Quqf)2lBre=9%ZSWb{Su{Ch->lHT zUk8W#*ky^Z0B%<$6fwb__q1B0Y*B6#9{?8SpWy7|aG=tl?{)KIAcoChc=kdydX%Uh z&~#AI5UG&svlCx0ZDAVDMA&14{C&pNa4Q1*{X>;LY9ZS^c4c*NcKb2AkCwB7y&@Bo zj#eg3;^SjqCv(Q>;cw^yl>#Ovdv*qcRyB4cJMSFcMM!LGCdQdDlq6lHf7;Dg$WA^& zrn~boTqNw0H0EGVe2C>0#CU>eU_$MedD3 z2nrBPaC^%PeOV_eA2*6ARejo5$hFyZ%idMxi0C@1Bx#pt=%x?SLpA)0xA9(aP03}Q zkaGP1<%IAfEPI6f&c?CQxI&j!l1==yEw_J@zF8)ZrOPW&OH{(G7D(lvbw0}gp)Puv zec~aTjWS0Xs%#1Mz_av*$edVa#8_XmQ^0CI=p1BtWw9x@YV^?muCZdto(jUV!}nKE#dc*^6o=vdLZ?of0se8ZN7eWyyX~h^)m`EyPh40>obu6kp zd182cU>^|U4$oQnCcY)RF{*4$N5q8`uCFdZ)T1zN zzHbvHk`LCUIAZl%?wFq7LN+y@TGJSH*79}w_)F6PVxt_E3|6zHtojfAmgEGDBIhqt7CZjtRTb=i#AC;7!zc_Jas}Cyu4Yx*4Ei=zE zxrqyGG3Gw05&Zrx@Pp*!IXK7IXE!D$)rNU_#!N1|ro@cO5yg-Uakd=uqoKKj!etXM zp>P3l~RTkL^4)!T7oQ$mKF59<=tP|D3q;B!%m% zi}uqmZyjj`bbvG;eFu&y_MUZ~oYbQS&c0V%r5%%OR!)1qN&vs<}bQwk%v z4twGP1Mc1sAb+PQZmya11tjzcv_cJJ1hKq7?ea48g_xQ`^gO5n~+YC=*OSIolse&n0e%31pdnfTUM*HaES6^Hb+#+wR# zO9f};hXW1S{+fOHEjHD-xN2 zJ_zct5`p)b5km#!`UZ439&MH%Da|^qkblt*t$>Zwe zCL{unT;qv(jTtM#-3~-)(SiqCbEVAvWvuF!Nlp_=8z&_}PJg$i4S=;c;y)=?ow+^pemFfi#tl+hIkHC|A?DgFVig)#Qt6 ze{7p=y%G>dxVv}!x~tLfK_m@tK|J9_&ob#r6RCb6wHF;P~AnuK9W*nZ@tA!C&HLD0eHwV;lt4v)lztTKJ! z)t0l|E`9PvDAG#~%vvccLQq$a1b>Fl`D#tJ zsIbY4tx?M+@5Y*pLQnd)nJQTBC@07+zgsexx%6zS%FVd;Y1XDeXL2yLSaD=UV5xPx zD^mJ=yk(}-qn8yj($QrEIsYg7Lm9`_h|9WICJ(qexiN3wfh03-rSx8upGmGXvF!6m zT9bcC72F{Gb!DGU`b6Gw9jv&AC2}4{*CP};(Yg3`^)i|ndJ$3$?VDw75*P7EER6R6 z^OH|6d#L@B;q|F5uBWm%Vekis&kxL+@TSEQXWrpQvw;~E zqlN^K7#v3CxjUr|L-~NsB^}y>WF!KV0L8?~_|+!#WwdF-Y_0ORG=6yio4`TX8+d{v zs3EyOKN7w%vlCWa>1p+tylMwP0SoWvUz|_MklHuvtGKJ1*12sSl~}>B_Z-0zdCMLR zT1%8k!Vs3Kpb71hGPHTgWmOc|sjUbU$-ic{8G`8VXZyD`Tb-P@ysS@L@ z$K=I*sXUj*Lut|7=b53(xT2t=?>#GeQ9MzJ&y6-bOj1(2JN?$h+iKG^1^!coRjH}%wAy8A$w#GoUc>bC1!Z);w&8zQ461@D?@btyK8`65++ zzvz-)x>>eDHfbIb@gkeAxfModi8u*F^L}L33i21A?@rx? zhjC}F`Q(e!#CyrH-aQC%a*pUmc@;>#W5W=x&*^3;*Z7YtTHS4NyzTtWGk2gk;}&X<-Qc}FeL9EeI~=OGLdYk?}Mus!g~24X9}B!en0FzN0orE0i@T>y+5xmISkj06NCbY zJTVWaYg@GJ?Z})xV)OR*_^sO9e5&M^tj=6+N>{t1`BL$Q05s7tAb3Fs-_$>;{&v34 zkUinm=P6(c$GEvz+J$}wnlk75IWMcDxw0+wkNly zM<$FJ9r#>|0?^33AC#5y#OEt5Y8!X_rCNLu^HnoP4MFAR>9cuVLj;dqX>u;X4R{s( zeIwbRXuBp#TVF2>F#D=x_PQeAW>kBm&~~9eh6AOzz8l~|R;_yVG}o|qmSnF{{b{cH z$ic30uRD8s48U8*wp?$18}~FViXBt*9gVfPDLE=um!S*P536a56;|0_*ZH*F49tV_ zA~MyFBlNfdh!%Z8j!#vDJz?O1616Qk{~o&75YfMxYMS0}{wX7PyWKlDxLkeEG|WPy zNX}a0v0wJ?^csTM8=UQ*006Vya^xszO@joZwbowpCXzD7gS_13Q-XG$_XA6fAT)HJ z(vlkORl3!4lmBbxOCTcMvs zh9q^)Wz7-xyv|-=i=(QXY7FLN?XNT#m)cC!Er=0`i``pO6ngnlu+M*FnNE}>*V3OW zbbGhc#PeM0MO=VMLXk0Ft2hVH$Ye6wWW07>VOk>Oc|H+xn71nGgBF1+ke&#^FOfHp zzpp63&S|p(>_#r3Fix2J4U}9%O6ZYB>vTD~nxZ8Rg0MxLgKifuQy=_{}*?h+UkJ z9*ZH{$m@Os^NO)8lkO=HH=A~DYuO`?CW1usFYIBw_byNwo*iL80|ft>8o~)RyVe3maF4I);h%ZhaT&qV7*eP$VQJ_wRD(%N|HC&p1#N@fD$m_|HsBaid#KtIm zLcV(3) z&aP?68K6TM{A2)YF9p<-hk&`q*sGvtP6@lF=9w7Kk{^a?0)G+3}Dpsr}1RPWQ ztYOC_0})BxQyW)IsjrLIxAg7cF?{Ec{1uyEVR zIC%GG$_OKP?x=W--=2S2Zj~plKH{%yPrTq59cS`_s3zqEprkTb_kBfQ}k%F{oPzlf$OwsAj!StDMJD{bZ zDl)?z$mCQ9dP3tCk&maEle)$2lJNQzled~5zvfpgqnx&JHjVY2v&u|w1X5)1i>*z! zuZ#%2*%yXQD9L%xha}$!>IlEAqe(~{l|NfRV&jA@M6+GDbW4O4xr(e>G+SKdy^D7? z`N}#(HrC;$f2_;pGun!&meDH{a*o+}rXq4JLmZ6IzbLSeeLEh>+gg)$_RwR4YpEGn zbQD{0U~jBzoJi9?3e)GB7ZT>cJlS%VQ3{>&IK8Duo?sKY)64EWRSmaUy)4^v9n&8i zDpIOXVW4rm@K{(+X+A%-vK&6keD6)IK$L?(N`-Nmv$-6~jlIkTUk|Z1A$KKoIP`~DcCP*WS>ISG;}v3ea!up#RvK0b-0QPPX4z{jg*&@2 z+^_JkpU`))cj_sw=zq0B34fC&JZ1m;I~M6zzzRx zl1#nJJi2X0n=C~#MbXBKvvEIt=`36VC;8o}+_3I9nU&?jHiI|d{~F!0EY36?R=B*j z?4XfK$a$eNCjT5zQd*v^CRtUDS4JSYy)lvk7vlG-sL{Zd2GBa5%aZ0W7f2l>`3x$* zp5#gzu*~@MdMf7H`HSM()*{=se@`v3ZBBC2)UYR@^w_rCmJ)f+9&|06A4LHrpzj0n zVv06wYlUU1OSEho!BUch6SpBF6J^WDUtiRn)me5f>==!il;`SS`*Tlv)1&YC@%^!z z_s7Z2KY9IF_T8*7w`KpxqnB&u&TsLS-Wmx&LFO$b#yxT5v+*69*C*;uydkaGwecWWZ(Kw&A%Ij+x*+RiLPM#+fL z(`&X!&J-Khcpn1(s$I9;eC1p=e(phrdGPVdRia=WUxr(r)*o)D4j^HNn2mgbOe`TM)eV@9ei8ySIgr&b_D-ClcH1+qXZeOO}p*Ya(xD(*hGSzpd?X6g7v zZ+Yv9-fRs~J3RIWoi&-NkvOI((}ev-y>O)fsVAgnD3pX-%#s&M9U9D#n8h0^3|Qn< z&Q@HKF!fGCXzgyhwGw3(GDjYZ)HJot0}#13}gI=t$|$Cn|eU zZn;Ckyz?dCoLrLK|J}oa?A{v@#&J8u%e83xQLCmmuDp3mwh#Hd?;+YWhA0yWlbyWT zOlg<9Oh~Tm0YReD^B|2Iw#$1TKXo5JFf+7PWRRDW@#Rj*rnsrx!?f1%c5L*%Fz0BK z;QOnC7H>Bk{O}z$Abn+=3_X6we<-njs9+(j7M<7vyY;mhsy?uFEF*`I&DzB^TPIm(*O$<41X=0+dX(hU2nxcfz!CQ`Wd>2fJ4ORY=> zgLhan!IklH{Sb%kNQuZLD+GI}eCjeB!&ax91&xzuk6Zu%8X#t)C}=JD9SV1*&_ac> zPqBTz0fHOjp4HZ*+u+B0@0ESNp0fUPnIJP8~8W026~%_Ntv2ExQHlKZn4ibZwN3h{l51uAt9x=Vt3 zEdpbICVme2xKK4hjPU3K0RWLBcH-FGVrY?!A?_u^y-J`&KM!LH8?G+#L@zqD+vMIQ zH?cy(8S>Y8v}mnKc}xDs8eD7lE93x1hG(bOL*MUedN+Ildu6Wf(x-?r>uB648^7gi zcMU(^hnZn!RasJ*cIuUaALcfVVOCX_%({jB-FC&<@JGi6i^&^pgHd3Ox(opTBr zR(^VP`D`BgGNO-F$j@VHthl^~_;gF8?^+#2PB=|)XlE_<`mfTVeR`nca@s8Nd{`Yf zN@Nhhr(BpS%rYG*fI37=Rs*iAonB1n>Nz7qZnua1v`X(TXlFvX{dQuY-^0{#wRs*m zGyHDM?|gKBaW#5_+{KeR?EGwju(J}&JSscduw*69@qGOUoehpu4^jKed^D?Abl7#a z=L45wZRc@!xI|5sM&Heu8I2(A6DbU`linWfkz#)VNl&5bSa8?krKr&mH5~vXZp>>e zFfxYO4LE*hRwk1h~*z(jIn&h5WTbxT(l{i^z7-t3W8*D2{ADQ0|=_Y;$k;21~fW z@|T#k55}hPgk<2yHM2=hOhsHusd(Y+7H^v=v&d%%drrK@ui|CMmQ;}tYiriEJrmuYaP9Aj@^3bb2G-xB;M)CVRqazV|ne0yfgbH zskV*v!uhkdTh>mqOLml0leE8OLF`1tPRF-eFuhvQDh^-ohy0+cniM%%Z9P&o-N4u$ z2@m%mKR&Tq)fkB!LNuEIv~2N!lLTclIg-PtkR9$8^fCoTm`qe=cfK`-Aw}+QbATWmGUg-0`cVy zsti~wETE2@Mf@+N; z+tULx{c|Ifkd5ShMyVbSfjebV`KnpmVP3<|l`D;AIcf4fguUgEH8yM*GO>p*dTnwb z1vCYQCKXo2yB+T1hVHEN$k2WXde5&ydpmB$ANNl+9 zHU8H69(Kr&c#*1i{fm>BpY{q!DG zk(9yNLHE47$eRQ1Hp@2QSlnQb8p9#KG8u}D7*Y4-I)8CCxunvOz=GWthA9Ngq zeRw!4P|&R(-qb)a5rV1Q*3or|sp33lscf*bqWGHR0o@b{6Rs4zLGbIX=@cpJs2C5g zkWDoFtOSi%n_R{r&t8Wiu|~v*5+KzD93vL8-$L8Zs*ho*Xw#=P-aX87VY!<#{ycUK zWUIQ$_uJ{qx82Hzs~oqo$~2k}MXiDawc;!Kl~GP{F7@*wnQMxP$1$GzBQ_3NQoLxN zVtvb`Qfr>LQmaf2gOYhJ+ty0;Qt?izMM)d`LHM z)*;{NTa`xR=S!_#rSLupzcczL;3=TxtI%Q#J^Yc=J&I6UIEEOI+9LvVtW?q8>r_r7 zOybdc#HWTfP$p^D=?RaK7)@G8irur*y$xOy^!m$vBR5~#y?y&bm%!sx0WO@4V9sUS z!*{ksk(?d}c(I9G-&(Pu{M(tB*(%28$vRP%p6}B+y+fKyx#WDsq$LNP)!URNw3yomSs|Q- zqVx-et)ER0iYCJFsE6cZsBMicVcFw1K)b?VzgsO(whP$FH4Eiy816Rg2Ic23p*>d@ z56W8R_nMcAW*>Ak#qoHddn5>5qX_(Z6K><(JyrU1h-v^R?+Cm1h(jnP=6#Vp4?pUI zbd4cvhnpdjsX;Oc!lH5}1wKCK(3~e}QEC<(tH+A^7NRbA(6Ayr5k^QP)ZR!Hwv;qs zkrLF4t4NID;vMYf|5lZ0UTmf!EjeFmmnUgeQogg=vYt$;qxc*>hwf6|55g`-ilL36 zYgHdndjl}`9)2V$^fQ88MHcbvR3r;ZT&JGA(nS7tVp~(zKHp_X6cW*ILPADgWlFlD zXtn{aal9|>rEAfJar^hu*s|9G`Q}AqN)0t?UUYUm)vy@YSjyXWk#G&1GF$H_ke^r9 z`Y`aFYM4fqikn%AYC39cazFfVH*s7^aP(>=rN!e9LZh3WVhz6ybD@Mq{UI6E@vVZ? zVDo`X-VN?c;OlQb>sH4tNri1ttan++^miaF5-bYfjabpYkh9}xa;UbqPFJChTI#RR zvr<$qK0F78+qHU%)Fr1ybLf(5V@p0$1m0JC^@raEW?HrV_C)R`8~ReX+}#2fCG;EB z$^B$**-y=sDjA#`xJ!i85{3vX+&%JqQc&_;{blSMW-UJRsO+9?ZF>WSJUtu=gnbS} z9wK9bY={E-Q)`}_jf^J7jQS3LN!LIKeqK}EK})&dnyxwCMWa$(Z>jQ8MyXJmAZ>n? z$v;wftM_M`5LyeWE@V;xa+ZsffPphTV(-8(Jct4t0tIrF_eQVKs^p_`*Ag#&#P3w` zvZ06y2m8Ba%acJN)rMXb_s9Yk39jdHw$bC)^4;IY*_bJ1R$*4{Io!f;ATCNsi(hTvE+oRP`1FXJzzQMQ)ybeHrmx6jAF*WAn5*#sq)Dh!h33R!C6vyNGb27w@+56{{N=k2A%q~x` zuNOTJq{q8%CKI}j?`OGMvuByV^=N)QlIlbD+{k>JG54K`)Ns6Cmrl*ndRu2%u;+Gxn;`WZpKNq zc+FQmUmsp5$KqUWY|mwgsHhkh$B^WMemN-Q2g-$~`9ZMDZV>}$ch|%svp5DZUCiwC*Ohw?DArLa0QOcA+112m`& zq3?OSOjT0?UP9IeD*2a)hIr`Tbnh4c!uInkq`tP7g%oPAz|B{?Gnk z<2x&cpQ}Y%LFXt<1ZUBVx4?WUFY^njn?3Z0-gkhRHieHD9baB*5p1RyVMa z-dQx)@d%T-k45cn1hRMgVukc%fFf$>a@pqxAu`-@jip>tsen3RTm!^g86uMHeXq;a z%f5S3P|xJ+asz1*nFL3bgt6c}9R2P2XR)I8T(MQUxfK$F!O5H(c^{m`iYtu@M}GT3 z7sW(detQm-YQ^6QGn}zG-TXnf>5nPgM9v^6d?1@0?SX;rf+On&=@>z~$^!tbQzmSH z78EJ}gD(FS==_7)lwZIJpEezMFr36jdvt)oyA#-sDF9_m;0Djnjs7>UD*~*2p0k$T z&krGbkN~opj5vM|%J=NKF^J7UAPkMM384A2LBBZ!0NyJc0iFW&(jH-{>d2#oA9O{} z*4St~k$=C_06nU%|K%Tag?y_tcIV#Xb_DTUQ198#pzlKC0cE1&Wf+-&d2dgPGx-hV z#zSBU%kPH$uNMYds`2N+e0#tFehVR!z@PQR{Bv3V^IQIZ_H7kPMK1J`4Zya8{wNd* zs{M%mg#!Er0$PT)WHr1jMrXSDsR?l}F+e^PL%+oxt}VH6SP1<>WAm;ez|MQt)qLg; zPe5acZwlE^@;GxSP1h&{pJVQAl}dXQ06+w;!PTP-F}5F>t)%G_afB~Vtzr>wb}g4? z%gqaTH`Rkj^!EoebQcr-Yz~7F^zCotCmQo2RUHuV{SWp9g{3)kzC6z}wmn-aVZy;+$bR>U!GvK4H|o(ik-)D%yq37{V=90HaD(D3J^C3Mk)A&`*6sX6gy z3CaX-WmSVAfON9RgXNww6?V{AD0A_WZe=FsLf7p94+@zZ83cKF|YJM@821|mjBm5FY?b^ z@y`_a&&>XxdcgvXT~_Y1mfJT_X=`-id{2Y$XRfPqJ- z$RBjKbZCO|r#_geVKM{o=JoPo_S=eSB5D^glgWF4ic-Hy{Q#rTwXf2mP#Ew83WmDN z1_k!+*8HH;5q}G#%2Azv&`qqQLCGJk{h-rU>ZM-CVfF(t_@79alS_mjbX0k8;M#9S z5Wp6?we5_#kPbtBL&1W7{TM-13dB+&aH=CG0Z-a^AJ}9wrTn1#EDnB6?gdKui7B8L z0$Zxp14Z;X@Rs)mVbon9P5S4u{`t24xe5Q@Gq(7_c)lI*-to^2)U!2KgL)VR93Coo zf6%@1M-;9bp(guzknI!S!OZkur>RTO`N3V4!vfg6{TXPof_Y%}>T|=gXmn}I@)Vdm9)VOCYuh>42_;#DA zNw;^LuSY8Dmg=~|oW;DLyKBmTndOz_c^O@?8gK6)V|^oCpBIw);-R^X9)~@8(N)R-2T6Ezc$0SUqF2WH$+o*=I>XNkL6P_If{N|MMJ=|J}nmTN1{1 z^7j=DDEcMp7V-PrV?RG2?wnjYHJ?LUllv#lyV6H-NOol{C+S26;8Q1py+kTG?=`?@0a2 zX}>G4F1^=3%LzjW`p{S?bvs`udVPPJj{RUU{IqNe6Cu05O6c(rK`*-N97SXZI2YY} zp-jwKE}3L9+r#~tCe}t#+&V(C`G@_%Hs1@0chRmvSn}u)tOc2{Q*-d7_r20k4rXYT z8RgH^GIxVK*`4_L!ch`>$5h|IHt;A9Mf7zy6rdcm5e! z|I9o8%*y{P68|ir{~zCA{FtI`|A%eb|J@Vlf3^ofVCDbq6`!X|r~5fQkZ#-L<(J`; z$k21yC131Rd)>oW7;I)*#%-h}p4>vJ+!y!Ooms)N5hz)?xhA%PKdg90DUa)yXUk3?eXFXhrDZbY z1FS%v$9ZpNm#rhL$d_m#?5_=!x`A7D zoDRdpk3RQW7l`DqSi-eedLi5|idPwCYr`Ck_nlfS+vgs1wl+1lw7>U}*&n(AtAL#Q zUIcIfMW?Rs_T>32Jr=ImVlxY;>phYSL1EIjEB`1p<`hNE2Ij5b=qTp=&?`WRPHS>v z>RVSwXDK_Uj&CMC8?-Lv|H6XpdvadXAjR0*ti}ozFRIy)aqXhv_u!0Tbs7%_w~7ha zJCU)Ir}0n5A*iEi>&Mo+2nr)gfELRIY_`uq-h*4k0GOIw4o+$G zG~u&T3})J88be+{D}ckDQyLTt2SuhkiK|e^7zSV(h{_P?is*DAAkPuZ@gE4n-fG+T zGj3T>Ph|1H!Dw!OHxkU>t{zC#fGz_Z)S+^4|NNhSKDU4FgMY@)|LOUFtrp|F3Q5Cu z^+^HCCIk$QErov-OS^_rBBmvVPX07Ubu7DY9M*y@zZ$ommS;F70h=H zzRFq5hFP+2y1Bn@ z*-+`ue(O=D&BIy;6Ue8;|H9sTMK!gy?Z2*NK`FB6MIcL&-c&$3c@+TxAyPtz2nZN@ zCrW^&((5Z#sUn0BT0lAi0RmwG0@9@jBoUFGh=wVU_@D1S*n5m!#yo=gD*5*LDA{0IAv%Z#j)@SvP7)2EI9yse0m;w=GNuS)A zj%@?OaL=6RPqE|>PAFdsTHCWQIX`c@Ma|i+&ZezM%Oydnxz^l3MP5=1{{L+3Ug;V;KSRkbyGo=0-3D*FtDfMh494te>8g z0-E4p^=6Q~wT5|V6?q>g8cKA>{CF~6l2QpeIeDgEHCI&YIE*&a&Vk+Dv*94}S^QJX zM@dE}ZJ&rpc2!{(2CyX25N5tE7Y#ygeMiR3us!u}Sv~U!m>w!oYMA*^Yz+=c-Ne`W zYP%L=oPbc%cl4{G64_i8jv-wxuZXO%qaSyz;qsQHbJ&Emj<0T2x`vA8&H@)&w$>uqq?(;AGBvfnAP#AbPiKVZp1(cQ zFqhh5tc)-QK+nY4>+^fDYcHVOP(RqSFYjYM#5QXr#E6{80MOUs`V&^Zh{Z&TL<659f^|CZ> z?Yd>g&2UdS(tV!<5L{uJWtHaZvk%hxLbg)3UcR?=1D&Q1$-$+FltZn?mtM5ywQ?aL zNFZo6ZhV&IlbYKaq-(}VVYafAH%nWD;-L>dAs7IlaV$jW1t|mbSKuEcf{ZAEJq1&m3`kZ;2zRrSdh$eSmtmxLWwKvw3x|`AT3ft(%AQyih5l*~ zt!JD#O4DqA5M=*IVQz4?#%yVn&no=kmxks?ASonwH>C(SZk75hUL`Q0e_d_+1EYPJ z$Ym!E@+iLD@4t_6A(sZ!FJ?~3dq3&$!eJk(huIctSz23D?2dJ=sd}0G2b=57aQij^ z$21=jM$f%kVcOCzq%$Zj)Kt?B?tblKEoWo1b+f{UVyPXvrs>|rDMq-rQJ9aigDJ27 zU|;|BBR0mV`Cq33RWcx)F>I`5LQCK-d~H8r^}MbDt#x$BZneofzDEQQB!c2ADD;uf z7W>CYp{;@8I%c*LIMPtRQU+jZkt^cQO=#eteOoi_=&|73AxXtrz#;?T#3y=))6RK} zvB2*kJzEsyt;;QPKnJ6vja-Z9H$2Alhm0`kk2tY*+7v!5LEI8?HqZ+z9A)|u91YaY z*&5B#-_@L}%AcJD3xX7XQ#eB9BNUu3^#vsx*UQY}Smde@YYf^9>BA?L^62xiCQ1c* zt|GBy!T=-@9BkM}t5g9|eWHwV^28rLi_+=U+aJ=km7a{hmtk~gBlELmn_gPHZC3hB zb$O!6X(6BUFc;#@?LRzbA>gCa&zqh08moD`w(Nqh>~x_?l?KF4DW3gj$!QL!Jj(%h zIg(A$Pfu;vsUjbBZ;_CC6wVaNkrU`Ol~0A8C}jc)AjXnrE-u4jkE)4Tj5$MNbdy(~ zi3$ycGhgk*dc%HmKGCEn66>z4keoi#6VsJBfPSbraC|3YW8kokJUgYv#d~0rcxO`S z```XqTNju6?~exmQG`>xBZc^#E^OhGLpCb8X&=@wBM)5-{g955Z;@XNt*%~LwwY;v zyBMyj8SSbrE>ewZzdz7U3iq|`8~S72iHAERN76CO3)i2cC26+MCSq!3Y)9Xqa-RqY ztzmcUDnR4!{cytKXs-mkCztk)%AEprPuE#9232^nrTvLC@CuLe8K3SI^&U7btih*< zCZ;uaa}Rdf%&*A3>uNJAwotlN$5ng5Co^1bdh|e1f{$+k9HqQQTno-%jRb+ChFM2# zS8R`(36Vl-7|Wp&pYi(z{nZ_Lg58dCZazb7DI_;*3m-E!qwu=qF99nn{MehXD_(!b z8I9qHUo!zeoz4sCsaG}-N>(v}$MfZ4H7W*UN1Juov?C>BXb6D$#!OC+u)Fd`K& zfNvj^TMc^2WdUVo2P9Hm0+g)Iug{)*Zk!Gq4pe&W3{!JHNpZ+;M293XUXURDE+QCf8Di>O&o^mqtkUNWhaeF*syC;eXEW@zZ5rqUb^~*k zhow4!+zD1bd^iy~N5>Mxu{3LeAgX_bH3fk{ALuDCzJXi8{ZFf`@z9e9W=X6+AwmBP z;9CQf_iEU>UTL5WrRs9iRmq15;e50u7rJ1g60@g?Je+FSN!pr=whSomcs4R8A9c^j zu5dQZRfs1C#s&4|-+)1$;n{3`#1<_F42fH_^0&7NYIZ$@Y;$ai_=A9*o^STG`fj2h z{!XUjBYT-82zRFMDPBHd_i{B>$A(UpVwrWWdkzkm*&gZhkx4wQ1K z_{I8|n@+3=9QLauUUT>etf67RQ@x=K8K92Sc&M|Rxskl{LxtZ3`6M~|rc=6?zut@5 zy1H$=dj8C}0^W42W>5yg?$l11*cs$n8G_sNx=HQra-*V#;W?{NYerK@k$ckUH0jHv zb#|Ihu#^w#>w*b2x>fSyT;!qarA0UnT(}kDP|q`ETB@z!{DUKk$YQrU4my!ymVlc8 zcY0|HG*Mp^t>ohDsM4=;1}#97E~{gGIAV*}g;#It%hN$%w1_LwYXwwdpLtL0m+x1+Vg%AOCoH~c!~;BQ)ooFtiA#mS9a$6Ca*i?z!Rcg)mi z+6;jFJKj7tVksbJJctKpFKFhOO9qTO`WQa3M-pT;dmj75e1QHI!v#d^>bwSY?##9z z_2s`pxnuYq6)@Z4YMW(Sgu{+Mg&rFWhz{)ZT%OiD2rxQe_Ih{uq#|D zx=*yUE@q~K)c%)$k9WPsi-v99FO&PG>}XNpF)@PC8lA2&O;dILrVfWyt(&3>UR2Jv z=1C+7rWLDOIZuV+tb(w+SF0`fmx9uYBrOpBG2^jKZ72Lu#1dt|jV{AT0dM_NB!NfC zD=G_8JK|H>*7!K*J-9Qd#1wi$ev3AyGt zY0>FhgZiuZEB9#sw(4KI9#q}PTH;O|@Sxy2UHfb11rltOYuuO_jcHq}PNBz>1vdxB zuMJYKWj*h&7O1xD8+%uKM=Q{PZx=5Qr`pod%>|vQ>nUF{RVriqoMYWpE}aB}kum;4 zYv6!XixQp$S=_gwt?nGtvni#|K<5#`f|;)`dLm3Cy^vZ>>oTCHl3iaYpdu|h=;xBu zqoCYKp+Gn9>eb^AH16v{S#2fC#9_Fr*S;R8`&efQ>2Zx$0kjq48nH56aQks2Z zw~Lm$V_D~^$J(f-Ua*{s97JS_!ggIMSTmi58%YE&@H0EIw3%%Xthe@i^))6zU|q_X z8Ub>`HK}Xl{&uSCH;MsfVYFiTqocf2OE;^~BN~VNeTsoFf>W*8j9O$n%~Y`lm(qx~ z0jO%>MW1GG(di`%v|K48;MwY+kmDrqgJwr245kj1nt>VJUfV;Rtd2LMZyXgPLt~Lko5%{O z0lOwHv@7(<5TbCo^H?=1cvvClcdTk$udw=b9k)6JJ1Q+1sAw)L}>xBiB(=>0*DYHN&9 zx2Y;_G@;C@ts(W9MYrkwO02TeC;KD6Rig7=B9HegWn-iIz=UCi2UWK3ea+bNEyEkz z#ctlnWz%()FM(|H!atYYE=JDgT;A2S}*>NMy7<$!H@Byy~*VpzKU$)C=_xr1>N6g$);r)xK8IgQa1<{*O8Qfo7KmyR`Br zxM3m%AIcQr?84RP4|rK3N6Pw9j|`utRnE0A-%u5AYaw}_yu;~F^c$>ml=oAb9S?3e zo5zABh)6W5`lVxNp3a4_A4L+P#eRlb)L=C&NOSpf`{W_B6U#5l%w&S+yVXS^m6S;v z--`Ayq2-lgw_~mX&ubsmy&0@WI-ZNK*aF+l52xPHxn?o>3{qRw!qTIgH(&xYqS1r< zjD#{DKfgT}$yCjRQqIqsT}yca+lJPdr{(&QAdj<7w%R8YQ;iV~fMc#w$Igd=Zy+dPNOOyDinlCsLScd#zZDIXSHrUS3Dji(>RNLURFoi@eXh zhR&_Bwj3jveoqq2NlF>r@=ySWwo9(42(iCNb5pmiyJ)m;8e&q)fz|<3=$ELF0gj18 z=!GtYDqwUVVs$A$VPqv*o1~z}eXAH8$Z(o#iFVvJpH%Z#9#N6PhwE9fNc|$xDK}s0 zHcCdfpy%2C8o_xjlw8T<=A;3px$H_}e;~dGS_AU9)9gAsf%Eb; z3vClpye=G!%gh?I36&srqBhB52TG`WbL{ilZtZ5!aoz4218{UUpQY{2uyh zkHkAmIq1~FMRR41c#2JPB`tDI4A^X$S+S3b`KTA4(m3t|9@R2~rTky%Kq|3Euyq!# zQ6=3({u#Clvvr9e4v5Ayg_Dz;!>615t+rh5`o?<1D=OcamaH6^O62>jnN}lGJpCRS z>jj4W=|)J0HS(7TVxXR5rgiMiWy$G?)k(|BDy^~El(>fr36jr&Y>pz&pIpxMct*YN zd7IIOnHo3WArXu6+H%{BD`2jspv`_FeheBYY_MQHMjFnmhk6he-RH@Q1Tk;GkRr?y zF?geqa&^9E{&n{^z%alZdm?SnkE>}*UFxg5_C7|@9Av7Sg|xlcQt=uNYEqUbfAzBY zmh6=y`O>Dw>|&Lson?l6mmWq;+}Zf;Q46R4-4jji+4AHK=2B(#3Qv0wf*(HO0shfsmBgcg7_mab`IhLw{)V0of6AElH2g!Lnv00~u==heX=~)@(?-wq_jiTn z+7Ve3)+Hm~I8sr~M7lJXN+JLGqjSG-8u-t1VF!F{f^VRne3YBTCSzUk4K}hPs^xHl zMVpkm(I>himS56fY;*~38Eh%#?ZW*L1veUaAR2j@cGV-9NTuFJIezqk@%%`;BX7)i zYitr@{qbP23#mDgx7xS=ai5U|{3MrCA#tmmDqVKuTlh}^yz6#2I860O+X~1VU%l6v zeM`etntm~*bg_w8yNQK0O%zv~1cA8;d%jUgZ;IxVzbWQ*1$C6Tao&`+kxWj2e5Cl! z6<;u|h_00*f3?jiV>2qRnx~aZ_!Kn;hd5}Q2RRuZN)%*A<=tt`HC^h?X81Uezi4NP zQ8%q~ww2_z;48qJSXmV?nu?IqUy3P?K>D#DYo_%430o9hg<{-?=R=wG9l_V*GdiNH zmcOdtP$AKdTV_g^PhjL4J=gt#l}pXDd;d15Sluqri_fU|wsc9SCcd*38QK>d2zD2@ zv?ACvi0+N3m&4>T6WcJlGXJbk4h={&ajUV#njhGdS=3ffH_DxgHKw>b*%_L^S|2oR)bvgI3Fb5>rmay?Ne82ioq6lIPGD*pj?Tguic3h;FhZ znsV2Z?`=(8c~?GIq~UbQNhKjL0ac_M9D@ul;g10l!ns>3L5{a|A9+mba^U zG@8d7{74}PD)?dat5^69UFV>e$!=~I&OofMP)5EqyL}O~HJoPCXdRp`3M`_YD2+h7 zo#}eq6z&tgKt??cF*xSC_9b~umIWRNOr|jOGa4o5=XEur{Bfv=c}v<}a@F=564mKB ze9BL4H*{bzqBu-_$>rwem4hFl0iAl?gF6xdh@mcxj8SNdkB5Xyf<3+(wmnm0%?QW- z*d)K%s=k4d1Cdh^l04`8Lq5MPhY$Thws5WYC+7>Uzrr)j71fs7`#T#JLg$?$mNaOCky7EqcRzrYOI6hC&|7 zTzfxy`swHl&Hm?x{qli8wxR$8=T6JTnw@<{x}U+p^K5`3l{7G-*}Q%0)@>f&6$`!% zG=T({){R`QJt~3S9d^x3%ASYlFPsKLehF?^#G|Zn`z}jJNoN68L7BsFj%^y_>nyShl1=WEnE$Czh%Q^MY@FD;r41k z3MwKu^?hJ--xsyU$6=vblLPA?BQ3fg6IHQdQfNJPzG^UJhE0x?CaxXOmepBR!MkguhJJpw0sG>ybvb+GP3Jtr>yXiKBg74@&S z&0uMt81JOolE3SjnN9g$GRe1_4pf`2l3`T7T=i*vp&!aca)>~dd|q&crIe+WWe~lE z)BM&AgpBHh*Kls$PiqnxGgYL@OG5H5b9c$?Gu2V&aORq@!d(R*vS3~8`oO{lkK1*v z07*IAi|J8hH6S_@%r`_;ARowUwtuKe!Ws2kES;5?zfmGlr{NN1Z(p4hCW_&U}xC3`7Qr-=@Oo=5z5BJs69p&Hy$^?wb8F z`J>OR=981Xb02oW8F98HpaAVl6evq8=A05J;W+Db4R`(n2$^eWhFsku3pif0Z;vt- zMFWwxJfUgW(Q_LRWo%-X%k9eFOJrnA#Jz-%MEvEm2OnVBHQ(QJ-n1ljY8K#6YgnjO zm8aOO%l4b{?`=XgA?3|gCn^D zrjx_7RJUz?$a_RHHeOcy_%#8_(QHqdbi=(_1@N>O>e3b9_E=xtw{RdTLGg(p7a z3ZWH~-fO_6E5Pa8;Co3sx6!6N0c+cA6|d<0m2zd{{CG-H2kd3cpJ*r@+fNWdJtyxE zz__+tdFc4~z>GpX5NVe#*)tt*QdKYMpo^1!_9#Hf``O;q)*^COMRDWV*4*Q|FB!H+ zoafx0ja>^<(hz@1!=;#F-CC z8dY_cau$Sg$bzL;n~JU$S+w~x=7I%Zys34(X}H|@{i3TF!D-_z7AT~DdrWvc!ORZ? zxiLps_VcNv(r$f8 z_HpSyr{-tFVJ?o^ex-29vPAv z%hlyw6|>|j3yjjN*Z{ZJDC>d67%cV7Wk?jM~K0v}Vx z;lDj;kmnD*ckZS@2_KiiA2rlBYoDB51#bIPp0h{ESrySQnigT;RYr0LCet}ov0%Q? zUXN?DRf;Y?z+*SsT#f8>{~TEHr6yedHj5R#aI@_EaF73$EhBcJ$BSDpa9m0w=Kb81r4ejok=1HSCt?HRQLcfW%@fp>iU#8e>X}Qt)wcN=T7}=P;pzK4u zMs?UHZYz51^B4^lI4Iwsvb(evt;6_jlJHOkgptSaqNrqWf(n)(=qdwT52wOXvVA1D z2+u7Dbe9QYBL-zaan*fPwun9TEV?XE#N+e?pklkkJ~MubuCY2+FB=J6UXJ|>bN}<%8lL}&2%_#+0-Qr zSr@B#Uqn`iRnV5-`d4Sy$qWnkHGNlQ*wZqT^~G852~GJ=brF^NsT4430E|U4Aw&5e zeVg!(ID#desIQr`-JJ_tQ1(++-cc?YvTEvReY~?*?=O8lzpcdHLA?8(|6SWbcllem zX`D{wKGHZ#?vsg(w^xC$OJoOiO>U;_8PTe$NZ$Inm7V$E!SL7F=}9bx^`tbh^bDw_ z6s|Vg#eq_AL1dh+IoZq}x>f{fn%@|a@1xxTwssWC@x7%3!L#x-&u{xovQ7!FDOG=g%Dxo&eb zUF2kn^5ficD<6M7>(nmuIE3Maw8N?2-pdP2>-Bo|cELa(pNzGzsa;je1G@@v%&6RL z8-&uGq6e5gLi6Dck^XM&aCe^qS;|}kAzkHbu%%RT=1^eV3I~9oQ`aK!;j~p#FaZOWbv(%FR*Q%KI@V98aGl>0)UZ3ZJS_8ft5C#Cw@XCWb>d9LmRsYG zuhc4cx%STXdvGzeh9B3y#T6iI>gyavN~&u=e||$2K-_~J-w6`%u)N*+dpug~>o^?@ z6>#EFw89Bp!qDuWLdx+8!bZg$MGS=+VQ|f`MdxVN7YFu7bBW-gVP4CiNVFqVqR4->SPOkJ2yv zP`$*>2;ebqLb<|AEH zer5$(^H|eR3zoGqHsU{Q8sUt)b3|ll0tPiGRfZt5Q2z|tn--n(WHk}$g-?WX>dOM$ zE~2&sEXNY`cY9RiV2{9@;9RuXojJdXD~I<8ca>F|m%fVHI0QV%4vl)e?x(M#I%Hcx zb>eodaCUz4(_6qaCp>NHiK&iB03JuoGinvO5DzO!e`3-tg`CcBDS9*W`crWMa**+n z0%FYv31@VoiWZuJ4d$F9zZ7`4&_W=fmQV){g3eUr4MV{f7_D(0#aAWbwaB|?|*wfOimlUsj` z1~R~qu>GApg)dUacMLSV&~vIhwy4NmNkt!pq?qyRkJFdlwY6f?(p8>n?05dVwIEd4*U@lwDT;vvR_PwB^9yw`?;+e>7s4qj z8S{vy(w&2~)YwyowWtWv9*iH5KO{T%$V?eERr?CMit-{Yfhc>V@+AjBJBYiRcW8)b z8}DX;sb<<@$(6L6uS5Do_fLjlJ$RVlmT4VEKtWdDHzc&B`I7US;m-{-QtZD@y~}4< zcqFbrVI8%y@t79t3>9n0=SuSLl}FZ6vR@Eci-fm!LkirBsti|V2kRXAEb#c)#I8$eU!1;q9=7>P)PYjiC(fPIz?`AMbhy~r7 ztxaP{{nx44HKJOTNceP~%-I5rMMwo%^rItyzOzN9=TNQWX0W#TQt~r<73KW-ij`YL z;c3QCFNEQk8@#0&j+V-!!Ovm_N$e>|G8j_%Bv(ut;K1o>!X6J*7 z_2XNxtR4x=iuIk(TO$=SR%vBYqoPkntipxVyqE*!la=Uv@bUUUGv19nzJV44&y9#w z!dpa&Uzf(3Q|Y4miR7`(BM48-r7gmRF5fa>$c1i~rpqVhDyID%@rPdvGiHB)@R2kZ zeO5`0`1|r!o8TRDUJK=l1H~8p6)|<`{Xsi72IKR48tn`CZP|;i&AQK6<;rKg!#QPf zSr#06TK$m#z+tD0NX-cSbFEQg{Q@43VO;UW_7v|QRqcHp{bAktN&=Q`w(N6pSSX&b zmK}SRB@Jej(_s4xnQIfIOb*yuW^&1IwEZN%E*ElCv4Qa{c5)9Na~Z%pwxpOF5RVe- zDqKXux{XG@s%Mmir8Fgd%Ja-pn=;qd6lPXC1e}XEJZX9KAx++(X+5iDIxDI&M3&(D z(sUxP)%)dq#p)sk!6P?SvbQ{mTv(`Cs3|Sh2M>79e9@5rc*Z+3&TIlh8A~rABM2At zub~Z@tueZ+N$6=j+7S2Xs6JC_03xUhr3;aTlEQsUxe(iQhp#bm{yaMY)ON^Xr+Lf4 z@+$JmUHwrNrMt5UE=2wQDbnYepcd?Hf6|RQ1CjLHdrBsog4J%@twj$vtFtOGnizG6 zZJncls`Jm8oIJLh>Gu@s&WD=cuS+P01X^$%U@clSS8m1|pS83;Eb@fB1HD1e#Z1OG zIw2)S1g!@A0DtQ*^u&HAVqhrt#1q&6D@P((u#J{#_HOdK0z=|5O)&M1PbWS@*UXnP z_GAL~95xxCsJ8jt*WcYCV1JJ~Ewa8_BM;wl_qDPjf?R@m%xyxq-4fwR372-vc4|(M zc;ra=`A>swk00Mp`)Z9bwXKsi%@xu{oPJY~M&}Lr9Gq3z!7MISW6nyAc^s58Hfch; zw~nJvuFb{p0hz4x2UVy`jKaKcTry<*u5Zmr1Rz+#52(~23%dNl*ya!5hlCxx1$Y;W z*_;cyS?)>Z-uVrTNL2J{%X-vCP|^-5c2>R;Z}%78qE({X=i7!gR@f%4k)KkfLvh|D z*=!yt$eGxrq`k>le2cNr>=bZgmdAzB)vE3v!nIjb7u95>9Jkqj7HiF{2aI1V%z3i} zEnEG46LkIRa<37%QQ6Mf11pjrbRpzujsaJx?DxE=Y?sOJI$Tiu`E}k>jhGgklk-;K zMnHb__(SFws>{p8<{Ikt+^^OQH3dqeK-HwJX}V}GR@+6XJb%^fL_oL=fv97WCD>2p16UUk{Q$5_wtS(TvUySUB{2so)uut}W-otZZLxZND zNK&Ki21Hw5(GCVXNgb77qFPRm&}56b%D@zmcz5>U)3DT~Ze=QLhLW!Q`C97{APb*B zg=$kwKL$t)dcQ;;swITy+x5C%Yl!E(*P!XFbWg&)5o{O5Yt9jG$J!^hB)%=G$j%Ca zTH-E>QO3o{ViGDT*POMI$VFJ5tS_2q;)RSFZNNBlz#sV2>i%qVPw@?tORg6r`OJ4WszDQ4g?KmsPaqs*Mg`hoT&br7s2wU!k z6-BY_YRScnC}R80S#7Yde_bG0;6@iIPa7jl3HfT5tDS|jO}a5s?;%ZtZ@ii%fv$dz^_vR&HJ3otXWXr4y1xXd`l=h2he{G;Cojz z05?kD8H4{wa6Ol1>il%=%FM(wYLweNzl??}@|Jgbi9t(hV?5HvZtU!8hJ?MV-36L%1>r|~3d4;r$IlNweB~<9g0`Cpd4ChaWyfO*U+kw`yAf2A zjMEOQhGR_Ax-oXL)mhJjO-8Qg_tG+ApZ5dlVEhnO=yrm@m!Fk7LTQ+U8W^`wDkg*Z zLrh5%UT2zXnmgZAE<6%erhWl^!}|l|?1}lY9&o${-SH>~4jA#Y*gid9#ywzyx+;XC z5?vKQM9`0PkHHc_x^X62idw9^Po@CYw`X_4oem(08&k?^+uMbwk>M(f-Y=ARHQI@4 z8KL=f>8O_v-s*x;wEE=bHYFnLLUP}fy>r0t32yPhLrkAKA=5Zk2YYq5I^r-mI90EKk8Je-2M2tR>#8xwD%hy)sp~P#5-zWJcL(Q zi^63w8xBcesekxx4sq%aUGPi?oF^I_gjPA%wCH8Eap7r8vJ!_nRS6r!0^z^lcvN_Ik)yUk&ONgkpN51 zI|naoT?3!jrH?eC?)jUhR3b9_r^{{2iC)4_(oIJ4_mv@BPtMXZ3PDhFZ0~l>J})pqJ{*8TVulg4b{539=kg5tF&s8GUDi4`3VDvW29A5W)J+;VdRb>-(@AxSgM;e~DrtnbMgf7In-mJ>$IC>yP ze75NgC`O~@+O3r*+Hza2jSG@QZqRN>%? zJRQ*wUGOFw6Ma{o>Yt_KpQ2AwAnYgFKzxm_{#9B+x_AA@x~WB!6LGQ7-WB14$0Co$ zi~hZNWTSDvmxo#OE>TLf=z4N_uGhyWVeMGGi6;Av_GM`uh?tSe6?;eFql0qzV3(10 zx)8f1qXjpYinq=$P}aZbKlmaH)y8F~3oiCF<8wjabd9IFW|Yd%+Kr9GemXy0v#W_) z%uw0aa&+5{k-Pe~+%_IoW1WK$A=$)8u;y0JH$y+R?EUa}N5%R)Tu^(T>J7l}nDAYe}1_Hy9q!7F~Xs|lgbcKrC7 zX;?z&a)5bpFn@QQW;+##5ypt=VC3X;FBe*r*vWsZ#2`Q9V0}X;^dXd!^0R$=mGXop z%8E#@_kspH-pwim_nh*W%-?u@pf6;y#6dwalDE9ouD^;2epdcsyx2Kp z(2c&3aPb`+Nxc?xfEN&*^$Br&o}J_HT>wQ?8t`<8aLe@;Q^@tW<}7J;d62B?mk4o6 zlCvSM$vg+c2d}qVJP`=eoDHlQ9?8|!tsb6FS7-Ail5PXC!PpkKN**(E^V z?qi)|7WJG*>m>F=c$+8qT!72!b!_CDuU-TAyB$+m+eGcwUADouTalCIZjoH+<4wP> z8Y`Fi^p1i>pP8$$vJ`O!?8^^$i&>X?1rI%tzn`E^ZWO`KT|*UwL+0f0fpKwiA+xDP zvg1L)X`jk}ehD9V`Q#;w2hVP=dvM|3G;gkTZk6{)cbTWyRHEl~%uBawtMq$nTiTk7 z+1WLw_$DxGIT&Y^=Pc(Oj-fJtdl@9{k?;0( z?Jbx9K+?Q7rKRiu<1@a%dGS(W^@Cg=i`tr7hE%wAQ~bSqThU$t%^v3s?;ZvU#LObj zNdsV|bXK2^A*8v{rE}?o`+!T;i9}ob^vP{WAf5XeOBwJOIM@tI-TkxpiZEIS@R-|2suUa^c37sulEDHqG;ntoH-WZh@=~0h+^}$l&%#Ka#0K->F;kl^YRqA z+2P>X>dE(V-uCiE*Nw&rP4BM($Pjl5oqvm@PRrRGX=}l0SEt3r5RTKH&!YzO6vS+>rUa zx;UE|-QnQRnT_fAyy9^2vEAmw2;0L;>sk_liiruRSV$J-Parb=+G8*^#IfCEtq~Q6o%t+Sn* znp;o1a6N$|WA7@tcJ=0aRDb;?4pWToXkxoh!tbc+W-3@_kEp)#vM=HX8J9@d;#5Ud zlxah}^mfIDEN(Q_JVeyN_7X&3hiY1l zajmV15>=PC1557QviED_9&0yWtY!WsAy1!pt_E9DWoQG5Yp^8!KR&#nAI8Thpml)` z3gS7Gr+FK~v%0eMNVIE}7H?ehsOLKEc#tjWJJZGj`JaUs?Pi;fVxcz(np0=GD^16r_hpbJe(p_1?LvoDky_5vD z#71DmUSDHvCq_uMx6^({-#_P!8Ydfy znizyGN-wQphg4EpOB>n-5d0{}T1*oa_B-mGQQP#xo@4RDq!VsWWGwPAmXNB5p@R;t zDP4!)dTuTx>$3de6OP4Obq>oUUisaT5XXg9v%DU)-uoo4_9u^vg*&$j@G!Q_@SwSY z+N&!99>F3}6^$X~W5TwFo0>Bg-nk|8i;$}m2lhbhOUgBon@XQ&tCr(Q$tDHAt7#>i6i#okNBk6!y(5o|fq@M@<~tJtUI zI_gOn&MUk@^VPAbX)RXZIo7ACX!Evy6J3ZA&&)mfjkaw{!^i19KGt33(qSwC7wA}X z+QiN(t&MW?=~-;w{;93Sfh-^W8+47dT3dt|`cA`i8t?Y+PNbd$vm~Xu%APD_-t=?H zk@kGPf(P_hdE9z&W&J0&@;!nbR^0xoG4%Rb$(}2~NnndFII48sRLMr{=pmz9vRs^dZk`tY? z6jnra=+7_Rq{Wa#!C5GuKpp~#uwXY-%F`o_hJoC@SF{OItO)uv{W%GGzDtibJ-8xE zckQ4MxXsqyLAW=GHozeE@zI^i33%g77@-MvM%!>}wQjR*u8`8cuB-IyF_d?$!9o9` zh?jrDF#VM+%ES&DI zrVuW&)MMr(^np}b&A=b1^4>*KJZ*9G?%Zyeiy5)O#pLygHjNZtH7J-|v$TZRO!ltY zJhV+k%4^@g%+nVf%8nTznDuJ8dSuf+2RYPBpq#xK^`XhRqAyeIan<%+(T?A}6%7T+ zw(!g9GFBgQ3PW-=P1+$4ifd@@#`3);&Qd;3SB_&L$^PnQ+H6A*yy0=UsKDkBhl_lU zq(!Zi@N_`%Gt=uhbqS9bOz`ac`5@M_^jP^nNZKD)8|zm<8%vf!GGWK~RqU49gbopG zAt{57{}C~8F2D*`3KCxbFKI#l-N;7&=NzZgKli}y@ei~%y`+OYv2aMa3C`b}!NIUM zX(0YJs>XJDSNdJzzq;wFfYE_LKKvW6fu~26a&lFlzzJE&aK>^;r*^vcE$d({0m^3D z6O|8&!kE;-+OudS`K&Z?lpCvseI#EsrHH({Ioz*owI1MHCmpL1efisRgd_(MTo3Aw z9~C*96!mYkXle~{SqsuaiJi-x{l$ifH3C+)&k8I9zFFoU7&a0*SF`QDYn?QCFEINW z3GJ)d_XYF456_WU)EM)BIl7cCU%GUh{!{%oWJGL`U*Mj`=b5vZ|Ic1Jv;;0N3q+#6 zI`H(^J7!{x4CVHD($e<4M+)UlV#yKOnv1{ho+j!G&0LvnF*k$vocG^(^Ua?%_DCi? zYIHl*$t?V_Cmf$nKYgulP18XCX!c1MCUrDeQ&C;ZLLP&*kS@fS7RyI|DeUUNjEczi ztL7BhIOVSZ9=X&52`zHlb0WM>?4obn5LMoYmtpUx%v)V&Tj!Jaxw6A!^di!5yrTR( z?q*Ibi0a*v{rys5lK-CtUnsZ4{W9Prv>yl4D{>7=vmqojTNbrIe*;#OGkTy$^PV!$@~EQ8Tzkt;S)#vKlYE0 zU&ns?@)zfJ>36y9)x*Po{W`Tic1mO(`qg@#u)ha7Ge!$esn*ASz~W<9Kb`({DqroC zHyD&x1_S*Zgif|!r<5O^8t(wgyglN>wS7s$RgEa_I52ft&NWqfzC_+S6q zp3UkUkJbKvYnS$|Y?9Zt0t}q8B#WPLfXt)fSPJ(-xap zuhOvDf*2%o{#>@XTEP>wDP;Ya4)*ZzwtFH&t@OJEs9g7QgJS%*#bySmYH$2NoQOz5 zJ&jptOAyOo1D+l2V?9j3sKve5lT)z=MA7|NiYV0}-rP zIQSk}-2c4WKlkmQC*=R>wcwzrqc-f35PNYMKq3CVoMP8BMQ-gZhOg*&7n*ih|HZ&fQRzA*5G zx&N526&$|IEk|fiBtAW4LrmY`o`I15(#!kr5TN>hTd~y;<)KIFAivQgP-3wc z^)#94u%|EH2+sk)KBPW|cRvFytdlU!oS zL`Y^@T!0v4eEAFdYyN+^mG&S&!YS~W6NvhC3U=gqDoW`Tyt}(~x=vXIFJ-Uf?~bg@ zeH-!SNdDRQP7qG?++*=v9*2)grygYTth94M;=1nr-FrjT_|3oGWB+|mLv%-#bayM; zP%L$F(q7>8S48)7ZbMvxdBtVR40+>uBVa@xW^`dSp1F?*er_OB-fee_s@7Pv z?A91#wdNXj+aD^Ls1QR;TU7}~O%(*Yt)a&38h2wzH6kL0mV_c?6T?PPbIlSpPYFd@ zk^Nl%|FhP!o)`DM*8j!5?l)G}Vl4^Rb)DyVe2?REyaxJ<%qI>s0rkoJ08pFTW!vby zUDcD*=Dzb>4VBI0BMY7#d%j)%94@8!0w*{g(+G_yh4#^ZhI1_c zrexLFqHfmH)2&t``yNbp@^lr&e$Hctrl0q0Z`SslcCP;@`y0WL?0>O#$OsI}0TAAu z0ic1<^s5q)pk9#jY=+-(QS_e#h>B6O)?>VQ7L8zJ-E}?TS|QO0tdn4$g9Wdwm@{g) zY9KB-V!=JK&HR1*)?o!&hMUb27P`yEbbU7yMtLw8Qcdx%_GKXk)||1Ognf}$dGn&- zm}DpUz?b&%66?hx)YL2ITC0@aP2)G$36`Z5!*~}BC2;SWQ;1z@;pK^OUnuB2jZ@fZ_YUDt z(W--vhK45<7c<|uWti7&{NQ!m_%%d2T=cp|x$_yHEB8D~&wo9CYcch|zsKOq0tZ3i zV?TA{Ur*0I5EYmF{q2XT%0U@ThM<--lo#;xeb0Aa9AC!F$KZ$WF%Aq%KN0!7zHYe!fQxV=wblDCQy)zem}^wY_Eb{4O4Qsc`>#XoM&XNI2E-uSU?zVU-k z*BSk3LH<0@4hYn6-{&p?F0K=wDf6NTZubGwT<}aLRTwtVE+XO|&;E!8m7^F8Pj-=cd+Shy?MYGUzWD=5}ye{s{v5AN|S4Im?^ctjH~Ue2~ubJqjY0f zJ+gHb#y2g0i*P*sXYupD6-g!w*QK`#VSIYchV5VS8gYZbeRxdl4o;y{lNl)974yW> z<2=4T68B5BUO~pQD==-@K@wVz?bHFEsCqJT|9zmqHB1+%^vVs_D)`r!Us|M68?KG3 zFK=lgOIV#A@1cyY{8f+Qf$5Af4JX2nGWo@pKH=L&6)IL(=VUT{C6S8N%jB5#y3pv z`pR7eV1%qM^3hCjIP|{{7RJ#h_An{&QhZ6C%+MC?Hpi)(FV2m}&&{R@$?ix5I3n&x z9uiC#NFw1ANpHX1;?1L^VbGkaL;tnP-<$dh6;wNWKv3QXdW({5m@f5M#)NKt20(`E zwaZQ~%??RyBm_zvKn`<@^0o(hra1F#-wd8SN0t#CoQ^)t3KZ#*lNg7&=+ZB6!(h}g zO4`zkD9!5AE&GMFZ3@#f3_WUxS3=(5g}zh&WOZ#IpkgnABY?cRn+vN1v^p0SG zRXHQJ2U)I>X)BKWygnsaBQBNqVO=J+3r$XF7}c!O_7b7C+7$TtTv|HAL8^Y-opmiP zQu{ECh#+-QL4Na{&{7}Yisd@%GJK)0^qyh!?kE}TVGlGTR9N}MA^yw>qCBIyLX0Bx zlzuY&%08o5JZcc=k>WqSdd8F)EXnommeHiAm|Uq@^H63V&m4{)V8>Hpe!-Wy_m_FP zt>v!;#yr_|or%qVKxw4pzI%3CSM=>ec*ZR=XYkw>80ez-R%HCzo@jWOzA*ak+j4p~ z;?_S$_L!-JC1^mU?=Opn*6vewS^hRQdXFi_GQR=fwNqkl)2`GeH@Dj--3K(Q9Fg8B z)j{`RpES1^wQq>m+jQTl9$GJ1Xgq(WkZaP^ytZ1+|5^~6S9z;7bt1NU*l9mdi8^f^ zjI{mjf<;*1tr?5c9b(o<)felA1Z%meuyamA!tGZz&y@)<|FZhk>3YBTU#hF0!fo+^ zDnlQetyLkROJqWj+EVeW-C6{fw&BZlUI_(yMG$1B=;-%WR3E>f!^ zg;#pNE@X$=*?HL?y0wnXEH5?a2^IT4Fd(!3BMFln>&H zGFWjX2R?Od&54xok7z61iTBk0jJG2m3-|L6$NvFRC*SqwJ*HU!uE>Wzks$tlzy0xI z*%UUj%CH)#uIZW(yH>h$_Gp-5FTh{BJ$fjZ zjr-M0=5vi*xYo73u=0&wR%01$MqvjEn!9kga4VrB0v(WPGlZvTQQtJC< z7?$5G_c%IZX{j2AjEIj76E4cgpd9AsN8V7>Kd7%C(w&ep5Hk`GzkUa!QEPPeRpqDO zPL1~3CHNTiSYA)dA*lFV>U8gEcxSCV9dotoMh^ZG5oBH71#<6&Oe!|jPg%#mX`kV_ za=m=nyCP|kac1nI28K|TIdrUw7pvdsC6%x=5)n`kV5%JZn6$hyOUx1sV6w_ zXG(hNh?}kA)3!@WOK*k^+am3(9GZG2wW=nYm71SH)`kp*k}ZE|x~}GKMGpB7k54s6 zfbBp3SGR4LnH&d~h<1r!#mEZ>{0O2(F3>Lk^J9tWwL>A3rZXvlo4UuaSSc8Md5*7n z%WUQ-U@OlhH9R%$|FGyx{~ zG`RO^R!$5ttPBu5PwJLAqL#SgeAJJrO>fkF0C0k?lHeVpaBKErT^H$oeti!e-6dK7D<~+olnKnrc_xJ$^UmdMoj@Lq~fFRQN$w zR)823{uD4}bP=QtdkLoBOQbD2d;E$IYAtqqntD{wruT-c=__N_7Es;wc#T3tR@(T$ za^6c0ktvlLNK=Yp)}$2w!a0q@=v}d~V*AWxuwoVLsT}=Uht&$rJFTmP(K+W>V5Q+3(v&DrSEA9q0Z>X00bV@8>vPB3^~%Q4t;@$%48 zSkqF*J-eJH%c^^g_C)Qa!bM4@bZBEZbj@bO-}3)F&r?1d$41P5zpnji{y>js`0t6) zKAX29EdM`lc_!n8d9j>;&MDhJTBT8UJvf8u)E+t0n~c`WBg>OUif;eWyKcA6Zq<=rr{qG5{LE-rI+hw1B5kBFR#<#Qu&V^s> z9{?XH67gR`sYtgcSL>4)j;qq0qIc^E860<+8@!0y>|tRt4nn5;BPF`qAZHfo&x85D z#`_^$OS9b8B&X0%=Qb|n_+`EceD+va^kb#?8m#|gQ2n)e2Rv?JB{dXjpYC&4!bU_b z6KrSay#AMEVKx|@u^B$%#y$6=P{HV-|0C2tOmFMm(Fkv2An=T^3xc)TcUtyZPqrMG zyO|hNqT)Nuc`s8f1;Fs2Vvuc-Wsa1Yc-3#nGb3J*rvwi=-U?VY+AL|v zK(<1=*;P({YWCmeXCr_Zqz%(EUCYD=Xc9sG7|Tr2PSe|ZcnE>mz{WN8PGKNy{+lwF z&eNq$IgtS(2DTdQtgCWQ9CV__)FOO(m!KIMgA$HKf(AB|UGNWj>!Xo5M5Fd?9l}}J zWyh*F6YVy$jQB_FV{nc)s-618(mp8zyq^=erA%V3*_Ul?cS1JQvR;2jvTGL1w*rqs zdFQ!|qkAazgWu~ohg;8q^LM7a%y!(BxaoXcLlU07$2#-3rwrPNOR8RtJHysaHamBh zDSM=S5~ac%$CWs9<2IOtuSn38}1TA$VGTu?x?(>4<;->iB)Daa+yAme~TovDsZcq0rS!fA__5YliktXQEb$bu>K>36O_T81nS-rYbj=+ z=b9yTuR7Eu2nF6xUprja0sMyIoG_9qRPTBYJ~dZEo3fm1PJYnsrShuE2;*x6S<7NZcf-7zugG}I&EVNORkhVylQn5^H7iQgZ_qGj; z7TKi#%)JGD*;#eYC!Do0jrw2^QBQt_n`Y&PJW`Y}kj z%bVm#fv2)J?KFtrF+&UoiXiQOe!S|0zp9(5!A*)-X5GQ zGAF~yiKP{w7_}QzC9vb`Zz@}+?05&@Y$t9GJx=ARAE~U$Q|O!1M>qMM^XF+!?6d<{ zj^~zjRoo!|#PGhnEEmX=O1)}TACwr#T@CAKpzUnIJNGs~p3;A9g^<2=dmkT3bRSb#|4+Pd(Ub2x$&sB@-7I3UG{^)<1x1OvgXjcO+?zj8V0bqs@hZ0VZFiSxg zukE;kuMWs83e~_h0VGpeMK`9B6HQ{7^d$QGcE35TVTU58cF7+*?PMS@70{|VqA0D8 zO-?c~C9gMa<-Ic+-hOgz6eU?YA*@LJa|CakMX(ZMo%c%YfG6ZA*@~pJu_1!}*6W39 zqv193v0-!PZ-A|y_^3Yqw%(o-Fj~@{!x1Mz3&_%N#smv7T1X-vSwO=V?b@x|(S>!e zbgVIbO61`FF%IBM3(0?xKX`eJ3FcWbwbMP+k1Oeq=`V{FIPiv!U}vc#JN3^fw27)^NoaC%eEOm2nJv(G-NcZV5G-(odls%o+ zxwy-7^pwoKY$lp?M%K}=*gB@S-rY&WqFWT3uRGll*5$F2GfifE*=#2u$Ge#C4K)}- z>A0IUDP8dH-;sU7pRmxp8@bT~s7P6`c#hw$byx;h_(+Hkw|xTV+*8CTIg%cJ~j-WG4LrQgSLdeSd8{;1W8}l-5LO zAtqEk(QR%|NJgV zkG=51)Z|Io-W&U`Q28%BBW>?K6l=-kjOK==*}jj2h&NP=S2*UNM^cfe#V=&wF4Dp{ zHxEv8eMzXj&Z(6Sb-EyTdeN7meGdH}j#lZ^c*`O^nSRbp2Q~fo+)o^>s;Xua?Ce%| zVz}vzfxySWzR#|*b=IWL)+CD_>(Y1eMUT!{i42O?trW9PFljCeV;5qq$6K4HAo(CjjMqBG{FN|Z{MoGd4z?6of|z~(66 z)`s?lRjFs);E}1pPQJX9fw6yIm~WI%6f$n8@-6_g`Iw5XJNRgo*kNC=blhE^c}B#i$k@6{wiJQ_+TNmUW zSG@yA3GUR+#Qqn`W&SiO-h~@kf+FALeve||1bK4YF(xMQZYN-gmq7pafo<_E?i{x< z2CrcZ*yFDi$9i|^PkB5K^sYdq%6_TY_X_hO%qvz4O*`Z)Y*=RXJ~f=tP+kNR^ipe5 zOB}<>MsuB+d2)qFMS{g$F({3++l$lf_0MrvCK>nL?C=VaP>_NZ*?K22CJ2HN+EP&G zL=TT)+rtXy=o~L4)&8QSn#K9Fsr=_a`v1QKYx&ej&nw~D$>tZ#lu&~%T)U&+c&1!? zW)e55lsj`75f_Te&?4@G5r6+azs18j z`g~!by3-QX42(*(%fSPA2&3Gb$4*>xR&W}ywo~XIT3-U79Goam2@kLk9XLA}zM?zY zc)XHkP=i`=V|QoL#F#CD2avcCTP#F*t*dlms+jb)DzWPw&aXD(?fcOn1&{A`o-SRZ z0kb7}PnXqnpR9F0tP_M+IAtg9T+%Ni+-`L}Qv9B)WuGb`|AEndn0r4i`0kWGQJc(E z+qK8&SJ>N3D#7R6D)7)7q)#!JO&*rmiMfQf;}$I9GJ(>cEgob+1b88CCbMS#%eVQ< zj1rWHC42rwbuikT8F;3n__QS1AVtqT-YQl5(p*3^rKZBaZobVlhyBI1Qge9#>T;*^ zBWbML=X()?lM@WhwK9niu){;#?S2qs9u$gF?^(xyvx=r};l=Bu5@9(YFP*4BW3hBB z*%7BK+EfE8Ey(q(av-aPE<;6Md?ots$lr<^h!)8-lLQ*Up7Ryf;-%wM*KeR7a5I>g z$a0R4^D;_N21mxS?#ln3U@ZPN^tEoI@oW3$bgp3ugZ zAX@G>kR|t%q>Sy9ebs9)ZAmZc=)V448|mBa5W24nj40#w-Y$JjI|>{WW*9ZXcBOBZPX@_`A#N{=-kBA@GCqA}sjq{D+XCiu!b1T>x%2THVa^9DwBi z^?Q7+NH^wH{5jffugbq#&+&d6rRlLe&#g;NwPe$xe<1`#n5w{rS3WirZA$y*-6}xN z25xJnNeT-ke(hFd*QHOVtH+$R-ikI%&3QK!1c;h&d5vtR^z8K9(M+14CnE8%Y`u53 z;&Y<4;>a+E3;|g>jceVobwDd{%=WsQJTuLGYbJ3E$1CZSnk0nECcYFk$S{s%01K)N zSE7=ObEl35q75Y9p}nit@>e?Q%`OeSrHlumHA^LIng=gvQCXL2o`_x|)1|ijA4@Za zbJ$q~8(|1#jk##_@F5wR%xfjsez9-w1S)301<<{!lLPIBm2~w2>q;jI$pc~yrR8eq zx!5P#?7_%J57nItAaGJl6{x>&{UiTZwC9kyx2cDS6bdj2O3G6oXB?TJCb~ptT>%a_ zmI5R>3xplVC%XHB%f9DDPCX3>G+TTywn=F;F_&z3(m74U zT)$q=7N{LQt7qnWb}UCn9=;r1nxmxUTI76IJHWCiU_eiur8*RbY|0eNL0X1_)$ECb zcQ-&Q2CTi>^IIO9DOTY8xV90UoswQ)1R&=hm%ut%|nY2nYt|niMK$;A!!}9 zv*{p9v(Aq|6)>jO9JwKqBB#RDBu$|uIn_mgO1>VAOIR79-esl53z)y6Xm9nK?IGCo zWLofXQI$$k1YK&?IHf+O3*mcEt`Bf**;I{CEhENmJT z(=VGd)T*>DUl*So%)b^l5+^W$mO63=&?q=eI7SZb_kO~}YGGc=4Skm5nWDKxk$;5% zwS3@6)J|EK)g(of(VJUS`#PHV7>x_IapU!eLbg)*qeRu-qWk9~F7@S4jxD#BA2~bN z%q~Mdf*%~bCRyru3?1#J!)e!1UmVgh5U5XC__TaV=^Bf?J5pq{(>rFMC*4&@a-_}b zNkRH-fHe6Et3X={mCrJ0$4H_BMkw=%IC(Q^v}ntii3Nw))?~{Fbn5uwn~IiY(q*`w z_qo>swP?2&$?gG=(H+9*evA9&zLEAaZQ`5kUzDnw{&T`d=s3@TEc~0twQtvU_lzDy z1U#q_e31DUBD}&r8-J@QXY}E^B0*Hg=F0--&xDzZh0uST4)SfXN=f!L$s|C|T}lB8 zmC7pa%yqd9sAKK0IWl9dRCWV%hbN9k0EoH1*GxU#u_5BP zJ81$D@RW#W?KR%AXxAAJ1Ty6rUBgLs2&$a(Z+x#kcRORH@pf zd?Vg%ZX(MTgI#H-zobit;{^OC=05~uOzcZ%+&1!Vpu8D#Ms#lC!tBjvJ77z)3II;kYZ0i*xuZYk1YUuf2U&*-K8=faqX&&X+#Atul=9mz? zU*XYP>2EPoI8#V-1q}juiH2(x*{Y5&3=&XrU}S7 zbWZ*Qwp6!ys&Vk<9(#PS%@~@ppq<7)$0Y-A*Y-c3!c}+Gf(5Qf)3gGA znde$Lh9L`-TJS=j*gQ2-Jnc9dLtR3|hWLIYVhsXrJNzghZ%vP6S-1K%f#L2-POoBX zv}>c{t`3KR2L*SHZ8Vo#c21``eAl8>Vslrd(>ZnAhvt(n4|&7NPb9zD<9S-rnE*eK67OffVmJBl z-xCcV;-?_wP&J%g#FfXJMMlcg&{O>}lRM+$QA&C6rGvbN_fZlO`NTk<)9GIiV#yx- z-8TmlTWhQxvD@tU`&cC8?09_(6L!?*76`!{-G67G2?w zb>DvT7FDJ##f?Su*=4z#Lq85}uIRX_R%5Sc8rNIP3{KO`B9r<)G=VjT$y)&njbHae z^Ul|deP^N?60#xI;EU~)Vo{?6l8U9?ZYA`l;7$x2pL}p>z*RZRIo0MO65+Xp=#iw#_pMV$?&_uNU*FMlzCK#4< zO=sA!Lc;wkQ8FMUUG?{kcI*<+ozUoxz6RyDhjZv#D3vj_8)U(czUrRBhznejHMqJc zimheNRWaz?@JzdjGPYjkL~c1rB|X`hhh%q#p?}L>wCz02zQ(njwVsgeGH3s3mNzGF z49l=k*?p9e_vPM3(pAV(;~j?Q21d0vPhf;jaqdneY!*hve$BOs35f5jjxZ1VYvbKo z_w}w=FG9kPt;qD1hB}l5FD~Y&`dj0uHn{*ScZ{hpGviqR#+I~$pOU-sA{($k&w&Ax z*lw!cf!I270lXvJNN0}3{7banqV0v*nf0gc$a+L(h8&4J-&3((zqi%|-;|$cZM5zbjP266F`xRBzg*)n6p>f z0iNV#ZeN%A6%#$qO1GIDHyiKY5O!S50ve?x^QU$gH7Ov;*94n|>D8UZv0!ET2=Dsu zS^T-VbL#^RoL(|43zxQ-u%deoeH;Vyn7jYXpCzEU0~{qnHva=G^Bd6D zhOCafZ2II1ibW3VJ`7D~8hW!2LK5{X^K~oQ;6f znnW$gzsirV$Na4=z#Uu|cwVz^_R9$WC!kzP_@wm^&1_sI=jU%5PCuTNoat~4mh#BU zJcN&h7Zlu7M4YC9T8wqawLxD=Fpp`mf*A1Ju;y^hW%_kb`{Zls-X>xUdxoj_hVlLe z;!DN-6_R66lDG7o>p-0IjHZFG+UGn4A-jj`p1Hl38XP0C@^U0(!ly$^>})=8yKCZx z_}93z0K&<)eO{Vp;le+KHs!iYwWo3U=BT^O&Jy}KO-8S$73iBCror*Ev*<@7`<@rC z4pF1F3%SvAgVv*!qC$a>`L5-S$brhmSDo5Ef`5K8#VAhb%jL@GviBT_$!;{dE736m zj-(oaIv;#>@EII297$t*4sy%A9#%1x4PPHELR+Z7B%U147}OC(e6A=c^~22rP7dwQ zsw|%A*pcRs<+mzxH#nZm{lT`um+@z57R-4;o+>BDSUyP?lOog5c5K^erZ|SZ_oRBh z!}w+Z_Q^G2(e-u6&-^n@#oNQnu4z#VgVK(M(bD5;$v5-Fo(ozr*8f9M4! z7`Z#Z)!=a>MFyud8W*nb4f95u2aP_t~BQybeLw;a!oLAiCHl~bNE!7X8#Cj-xCz{7a@r?EJ3RB|^~ zX{#Dj!}V>Goc{^9zfGmu_?I~non$L!8eN)OF(oWI?*x~jX_n92F0;Kd!@+i?n^yFJ z9Go-=y!@xcwg{x= z@PPt2sw(kHrVnx5hP~pmu06^N%mGwQ7hOX4UaK}*gl*ov^GYT!WnpKk`M`l2#xzfp zk{L^WMT>vCW-b*J9u;>D1}!m>Evet?oQ)X@}{+foz>m3&ES!CQUSGZpGKryBh2d67S2E@g6)y>gq4 z+1y(!I?3`(M9Ll5Ce!Gh)`sVVXq)jBVu_MCkX&xK5$|dJ$d2^OmhHsAb^$|)v>@|l59ZBc(m$&vT~q-_Yx_PkO(HQ4_s zRRvr4ZO!<5bfS$N;h2iGof^UDJz6voC(4Ax_1}3wk&wv@&h3TNgw2mk9SMG`xE+J+ zKACu^rb`+<-d5`+yUnoks;kUg!IIXkk$qpdXWHej@HCHLyf`+dOYZ{`EKq z5U401I!AN%k&z(uB&GOsaULF%Ya6QOFE*$B;u7u^UWmu0B zv5pjPaLomlnri(LOiv}}(XPM!GZ;oA3u4I|D#&eomkGT$T>LYp1vK1K-Zp3tt*zx?+^-C>t}8Xq86BuD}OLUDl02n7)O^UQc*v{vR3|H94*F7ZAG!@!+zp7e8y z&cl_?2Vm!4QGCbEnVMsk7=rM#Mc39>&S!<*9K2n$EVE87TXJ;Ri_Sv#&W#&Ywj`;X zIZLJ{B}w;p8P|XRikG#JFl^%VZ@~R?{CzH`Xvbtl`G6)E4=>NYBad}jW;(qqevI$g_;p0oq3K8Q2OzoXJlP%=-gNLk{ljCtGcrwRClha4%msTV@lELlON_4)s}suwiQ8jUS-Ooo`1+)`a?IN>l1RtzTa+ zzLu6wv;aS?60j#p7(x(sSxvF%%8@$At|z(ggUI-XXU6!AuXy&Ypdh^ ze%l3BE5VF-u44S=U3PL;u^1F@`0A_r2xDbo5NCd?TaGqBDB?DfOwThc@NM zRRARqsE%%DE}7QYoUwku=;|Hv7|60)<`M#Ww>Q6^VM*~svk#QHuQ@_pz~P;$oy<36 zS2u9(I%5EC)wlRW++{NWXwJ6kn7=@Nyy(FJu#qy{!*;$H=X@QhL?<9RV?0ZXI_hiO z(U4`M-Y5R9yu95YyaYZ@yn~N;`mMrx4>E!`^M7ieOE9z)6wQKkPUa#)t7q21`$}M4 zAB5Qw5|$9rC*p zne(T(YrKn`VAdcbLyU!a$y1V<=0-9+B}|MK%w61bp#|1Qj@15A9lwnw#i|`-zj6QZtF7w<9uhYq;4i*;cVq2r9;Aq{h>t zHc9iUcddn;ac$dZ;kxn1voK;})s`~(j;^1H8mt0aWMwQoQn3z(*4!+zTpnHYf9CwL zoH4oB*h;m@lItSsu8lep`g%?a=8y)kW$)gy8lP0BV_wa(A@P$2teT|O3#%SC*fkif z{rTs{knNXj2CJ(8n1~-Fa;9$&=~;Wkdf#i}&cEZ1Ggy%T%lBcFT%fHjM?QrbaEpW~ z{N3fZ=+Hy5@mf=akL4x*%izkv+X|5EMO3P>Hj-W9QLtS=6X-wtIqq87;ux*(9;GJK|0pzu_r_k=JDc8`5Yl zOkjgy)hTir^Ms=5diQy)YH=yd+6@b%IS;EyF} z7m~1UfzVyEPwf|GFy+ ziS=CPo;!c$e2D?5(Mx^t16=23wj+$}bVar6Vr_=89SqX5dno@4eIeiTL*(C>R~|y# zHw(5&e8r{-z7(Uvl_x*Ki{kjQx;u8xqAw-8SbL31F;!l)UgeH-v`Vp0)go+_alw(y zEw$cV1|)d%*o-wksH1nMbYV#g39ej=k$$LVhQ5m*$28=w#~5ipL)y&0c!1LlN?(DB z*}Y8g21~f3KEHVImOQHmw@dJ{ExzIQ8Qr?s!Y%Kyw2)6VXM^cq$gGvjRs}SKBlU*~ zvINjB+sfxWc#fP%Moosfw3&2(`NbzS-`u4bhjP-KbpFqriR8rznw)O`&%NrlLY|$< zX=VLpX(SjGWFXKt4acebuNOGL zNtV}*RXhmRU6rLRl&b(AXiaq-FQ|Jpi6{fVmh-kH*AV1V9d+BGXkFVcwvj!;B>oX6 z%sts|E@UPN6f&~48G}EqJ~NYHCnun9Jd z{kmu0n$@vxw8Fvcf;W~wK!}+>N)=(ag)$oas`WHliW8im``-h zsEIirvtwDG`U&e*2nI9^*Ft$_3s6dk&%J3vvu+g$5?f;ONAd_=IbN~zhPR)y7G7u7nNhY`SF>lwf;aI7p{aZ)w_I97clPC?Z)}~Ak2mk;wf{6I; zm_0kyX7XD4>RouhcFbHp9hwMQKVC;YT1+OY#CqSk;h%+`NrhNF>+HK?I<`Gfb#IkI za^G{rZmHECR_JDDkIirH9mb!G6ZS;kZs68L0pL$3c`?m%fx(MCXmz|n>SC6%hO zQP^gDaKK+rRIt73`ERxr>RwmjUweI5yOtwiaq1me?R5qI9ZBZ%L4-qAlYf;y6Lzrj zG6aj9g6psJ4DK8mKpnnQzlIYBAzg=cL|vlD*WAJr~RY{%E0D$lVg@wla`KlQf8g*mrw*~B+vBrwJoH!y=vX*^N5Hu${Y7*xgEIa4dbt~fNba!e- zfA_&={gJHQo}1gwDH&_k&O0Dg4TI4}$=5#!BR{BF4IgOs2hT$GX=#DIL8V3OHWx3u z7I2HzK;m_2)icXi6KD44h^@GihQ zQjS#CNt~*DY7vNTI80(H!1v^qot`w;IXDt!z7Z`6vqr-XGLSM-MO?2=+MLOs;Oha- zyJhtj_Jt&Ph94597fao0u`>7jdf1$^1Bz^JsqyZmcuoc|4Y_s;A_<;A1yw1}|FfAi zaPIQq-nHWr-ujtc8xNn=n*Kx?)=V5(TYa|(`P`Y8aLBfh7$Ew3ox(mZW|qF*Cus*7 zj+f)r)^k7uHEJ-jYP${zhHwGbrIH+VmJ-Tla^S<6;S;s#Ya>Cr1Xx&x`rLI`A@Qsw zu!`lID=AOt{+!sS%6^7ht zNEmctiESb|^`y!4Z?=A2SXotO3v*idu5HpDq@^{}=qlDu$-)Unn9eEy=GkOH(mCv`jheNaA~=|6WbB zeS#~p8pVxgn12as@9W+}^|3qw|Gh7OXkTGAr0_4z@pQPdY;61)lvTH}Ra%g)%l<8T zO6Z1hS1DgLwI|#s?YGZ3TN`)Kj@I_nk9B5S)iqY0QV%%zqRhpEP~>O)rjAbMol4MZ zvBK1yi}0#5GLbLRmY;OlW#`G}md7Fph}2Shctz8fC@%su%+>B|Vdrv(WHzct?DRe4w+?tUCV!JgA_tyf={mPh~GO1 zJ{K4?R7)b2&mx<0T}gJM5$@Kv2B&Nauta^{?n>@b3>D;Y=nsdeJ#bhfoK;oxzv!4+ zmIW5PrAiRbTJs5f{g3t0E%fPvze9N1`3v(5CvJpfMbl=SQ1CCSDvXqB^*6uN>(bvJ z=O6uEr@v+IJ6+_0M^0WD2bxnLeRJXQ3G=+>mYp76L-R6tVmY#9CaGl}*+-=EHR(%B zje&y=dX3%zb2Am=wQ93*{ql94hpz~nh@Cc(#K>Y5tf{e4#751}sGvoMkW!_b=toJr z?xtl&775WoI0?dM@32#j&pvdkmFf`O{GobEjyH1r!F3(rQdj$kS-8vivSZvZ{9Q$_ zjSD2K@68(MY1)f2R*{~~7ggOxxcuWaHMiE?rw#-)f32R|yJSs;A$PPS_iii;bCElB z$jpDP#dBX2{yA6(Ro4+|CN)GH<3X5xuYPa%IHY&f6DSktdB1oLsD=-)IVhG6+^Sl5 zeg8ohIbYF8&3*}^E!OCd(2Z*u3Ro%0tZ0`GINOW(O3%JuSLeK zuJB25YN(Uj+>Z7;py;a(lk{!N``+ic(>#@7-T4cCz;<_{a?|8AlZ=X}5m^s&p7C78 zmO~7$+yyt`Y(@y#w#~>ubf1uw^UK?!owjo?iJHI9f(%>ia|N1m9*wlDK=ENjgDA`3 zH-;@f>a$tcVLQ0^BZ|MCcMnXR;wutr*HRenZ(Z$*b=q8H3*BMQcE9`<{4!pcLKm9l zCR=iRmJ_`fjzy=TG>$=X(AEw`_}T89x~^-By=61Cq2NC4Bl2;rpZA4(uk_)dfg2@y zAP}=6=7~x6c!nc30!gp4D;vQc;vB!(;soX;%TwZmvB@d`elsn$YSv!EfY9WDgB14^Wgag z&Fpo7Fz;aX=}EVg{?5X4UiEpG2Bx0I^$$4AD-!z$7PlM2s$J@bp?j%zdAy-vXh#pI zOD7AKF79w`Y}CQ0UqbxAE_*rEI7h;o9KUun@MUVllv3TdE3ezIAE)?`-Mcz))=%l%|5 zc2a0J1Id8%(6f1DD@ZRxaKgjJnc5ty@*q#8X@oWQu9lK!XXA3s^`EV`HW?eDnimg! zjI693%1BTfD`S8EcSMzO!BTHCC*lp*?8z}pZMeYEJtE~XAX4Y`x@F}A(>1W`pKLFp zZ*U`6$M-hO)Yt|IJPWdZKC+~qSmh`4n1G(r$JID?9ck}x9HW=}E$_zP6OP?Ig)}e6tV7>H3FJp`HkR8`I2D@s z4P6@oAyc;A&$q}~$9!H%ggRA5t!)TQ&W7D??sBcpYZ){cqCdt>E;3rxJ+JtAo@^Cc zlhcJ*?l{>>tnp8?gz3T3rCH{yaHl?Ozd_x z`HPExlOrL(G!tL8?)csld3fMtM={WktdG>`3yuf~5u(^n4~j8#pA7wzo{*Gc3Fu8M zf)wphHgsyeXgtF+P_AiG*g&$-HO*Cba{})g?>;EYUfJwBy2Z5|US=@lPsK^DIvQZ< z$@@pm5Q8>3Ich`WI9eq_Zzpjd#Im_iJBtcEV8yhYw}OR}Nsm5$*!6ih<_|Fr)Gq=x z<;)(hI)pFTOj&FbH;Kmnk;`^D?+Wchb3S&G%Ch0VuJ!Ack{a9bM5V`tSMJ>jlIiTd z80fAdhws3hF%#o5_@X0#ATf8AbCI)^mqO%DQZeV&q_$Xn=E5dfsAnP+a5d2Z3_(n&+Ys4?A5V&l*Vi#|MrAhk*rZT8v5g+rj@GiKY-z)6=chIaLH?M6?aJ3|HIyU zM>Vy+`@XnbihvU75`ip5jD#i#(gG|&K!OAb2_>|Jh(HkO5+M?mCVhz%r9})OgaA?L zNRfo{6BfM*N=Xorrin_-f`E6fea^V!o_qEl``mHP{cryXi~(;ZbH4L^p6C1hd|~7C zL(ViP}YT31l|()dp?6c+EM<)FbC?iQSM?X0E@n z!S$IHK$twUV;+p1PvUs_EK-@fj^5m!<_x($qHZ# zMrEv<_B|VyJL*<_{P!wUE)H3LZ4#SjR=oxd8ltDA)aN!u8>=4s*0?+%o10hb@Q4bA z)5a3ba@A)dY)am@7yAXnVjdClmqVKRYyWO-4`3TIe~L&2cBpedwsU`U6%O57y5lKG z=mnco+3lxy9C+v3KfHaH8R}WrT%V(C?|Mmp(B#aP1|{dK{_{Gizev~bme&`ba42UZ zB4|i6^&yHKrg*XAFDfdMrfxIjoLl~R`84|4lKPMq4V{)`I~A58tE%{!B6p})-Lw{+ zpYN0z0$QAA`WzMnAh-c3@n8YuBDXY!BVi#Qq@p|yKS!SVv`tKl+WE9)3X%43W8v~w z8CaaI^__#YGCP*0mE@9}@)vFBD+LC|-^_9FdBANphDuz0&BJQ(l26|XYh zhBde&itGrLcEj5{6}WQS^Bcdt{zAhFi0EeEHD`PO(;F3147|?cP^s(n+`n&oMujmB zUk;&apOnh*b-zVF)~oB>FmbiOASX4XMZ>N|#~+ncN`E`m*+2=Psv!dgm1QDA)~j31 z&N1hY18u9L!Lv3A+DN+9GC8f{{`TS6Kt)k8?y^N@gwYZ?3KniA@YBv!aV2H?r5vDrTjI8Hu ze%r!1y4#2sdrVg0A}gPkuEC)Y)8+U0n|O7Q{K?x8sM|8=HKpaVKtqZPZ<&A8T<~$o^PneM`FgezJPGsrT1!s44vjzu~X^97|PnL{|Bh9 z*Xnj=#&d5V(<1U^@NM#+*Y3VOeb>Aa(7D`-pM-$op{o$NYXa7i&jXXGl8QM&+?7#t zb-^QV-b$j-iLGUaG4MGg$hMqnFn-!L(s1hHIa>)2FBbfY*R0ot$X_l!g=nr9KAzfY z55R(NJt1fMxlA-Tl?Ubz`afH>a%wC6wqtmyF6eZygEnhuN1KEj>krvHnWn0`sq;JJ zQao$npMd&;-C{$g&Uo*(fe5ioO(eX-6zG~ACh`HZ$zC})MC6Z=1NhDl*SMtJBm>Z{ zZFH5$R@(nctL{I#rvI)-X8b>D;-NcUG)`21)(lns7E(*rMHD@;bH1;rEZ!EOD#H6$ z216;WHlw;~Vqc=ZGDhT*(2sM#`uG*IrW9xx`7Ssp3TV6ZxjvkE`?WhKa3AscR`#}1 z!vc^#0a$*|O^6VE*CF4HgF6NL+1Yyml_GnrDT_}L(?qT)3nCo8vTrw~nmDoyc+?g_ z$rgW?KHa|q0>+kQ173hi9CDcW&FptkzRHWAB3AjYeu~^RKL%h3WAB1C%>4FJv;aDy z2m#eb9*M6K?aq{(S=jd zfv5h2<^iG2`@bpdo!HrZMEq2>H$oCv2!Lk9&^!Q&P5Hui_$l&?zGF#T{3*g-`6*%= zD`LTcbj}#=r7<=E`U9ZUNg)Y0KLHTg2k}67)WSBbvmUw!+EXEZYkCTb*e&T?%(Qsf zw6V_fU@jbsDhcI~vV@L{MQpmjWxUViM@q#=eNBWBC2JgeUPYP9NLlXa7%#L6r>4K7 zg@iDqH5f13Ml3(=+n)SgDHL$jeb^7#O@SNy6rmj~`u&*xK5w1B72#0858+_-Q^aWw zch~Id|IW$ge`b~a6OQxm+EV||tAPFw$3+7R{9je!fF5?<7|try-6id*2=#f6_d}vg z+id}bMSb7KBw+G)h#y3W_R$uKn& zCYbcNY%X)E`}$3%j1p@T#bmFPSdDL8BRwwhgSJm5(6jkj(xXl^7o%fWvd^n37PDWz z1!z~~2Ls>W7g6oG*Vox+tQ`jOES-@WMWyiU1dBy7NmOXhrT1E>z#X~M{3L-D*OW;* ztZ9B05EjaJXb`kRKwa~2u6CVTdAxEfSGG{)701EPZMyBU!4VT5kig`J?O2-+t*>&G zqjlK+_OFhUE(T|85H+wryZfY)nAITYdA$m#qA`j!LPnD6ldw6}Wj2Ts_c)UVnhiOv zKtKH1l6SePP1K4_6wjceX{anQN$Dhtfvv$7XO$R@uh12YY-gP*E2th zbYJ|4w=STay9BswiWY}^9DfBQSmq`Zd&h*gxf1yOy@m^*HvdT4Tfmc3s9X=Octu&8 zSL&FI3h}UR+cB@%WjqazC2U)y>X)4SDbjy$)sjm;(=UyW9c6KXdc$+MTUC}AbsIuc z2HB+;3c~~r)hK>NJJALeFBWK+7-md_qpMt*6XCdHY3KlCNzjGPSUrkKJqCgCL)6P* zQF-R>y(j^7i6N8?<{TuI5xX1+#}|nS3o;$bf+D`L0JEeV{CYvL;fq+1Nx{7Up2{Px z?Vda){{_Rk(r5mihgBRgI>hk)_nk})R1?o8Ny(&S+pT78Mn{!CTrhBq0y7`hbF=F$ zM?InfPI(2=3ZT5(oGzy&5RQG5nj0W zd7?~H%|x&0QHS~@rTWCMPMNs`vCG0Z)~5d2ctcEyMV|%Tr(i`fb0t3TNYuiG(-~aMYh@InJR-5ipX}8DJn{;xcWBU(7Q&d2JJo7 z_y>(mo2aa^VoE0$v?ngXjpW}4@)A2#KJ7^e z*xVH-Kp*1Z!^!T_@>^11?%KWvB2PEDE58?Aa`5ZvJZvCAe>adm5O>_-C_cucz?x^4 zoz#Y&SDLOT9kqUHo)E(v>7J2-ho&$3uBHu!+mTuwYOU9qIVQ z0NWww{_QQ_CymtP$ka&@U92UsJ0*oqf-itfb5^43$AYxE0$y}cCxOlDOE+L7n;;9G zt!+1Tr~m6q0|OL@g(g&Mwb`+Eq9+>e+?on;#Z1+2LM$65qmB3edNoN!Wt~&FpL=Tk zj}x%R)4whsHc<0olTl>5H#<09iOEr_*km=^BEX^2p|3Rak}H!P9-ixGuWaPZRPV5J zrNb=T@{Hzr>11mf2Z{=#SR=&C7grn{KXwYBO+s1lF#kc_SerUe2hBSnNZ|Q!0yfDt z1aJ|SE5bF)`@+ng`6;rrJAHSfO@{XF4#}~3-24>YkcnMLte7TPRIiS%kImHVQ7hj& z-VPwv1t5x_-@dVmI7iop416Q_Um7%tW@q*-J%8BaJNLK09U@6QTJ_iVf+kkKllSbh zOAa-yr<^JdI#17MuMVB3*>pg(n4ERrbuG_p%bI8!B4o$M*ULE~QzvrvX6IrVQIZi4 z4)RejS0>2rRsy+UGH%o6I;-bf7nnMU&~84RDwN@Q{YYqY<OuoNBVC-6Yk>Qub6&mwz8lH^ zSpPE?GS5{`v1fqjrxW4>5(Vdd@>|Job-MYj$2rmK()c4p5mh_~ALLp*38`jAk&TV& zbh>@Gs4w?!6*e)4C4po>oxX+a4ya(5D{8$B2OJ9@#fNzj=;BgOLa_rrW<3HjyJs^Z~U$I@87RfQQD?RqvhXi`T z9y-eSlpgcr)1hq!zKbP>UD(M@YI4JB8>S}cGQlBSGNHV$Y7M0+cP^2Pjk;Pw47ks_ zZh#wUK%lusZ^!c`WYoR2`sAxvYupTf7y3s;&WB32Q_v`XJib>j`*)Q3WGl^CIjO|$ zDt9CUU0^OZTFH}%KnGbvVdJcwJ||U>#ZHHX3RUf3S5XfIDAzFw?&(xNtgAXegTAhItlfVot4HGl7FI>Us}L z=NS!EmAF}fY~>Ei^2S2`*lRLE(Kh^C-j9bRs~&+(s*2i5zK~Up{d1*0D$}GcQw0%G z68oKgRsD~pejcjKCR)9pQHNSaM60_biQmLEo<>*2i93I2!+)7&?1?*bzp;UEJ2>Fo za7>QZ6QXzwX6M}k3t6}m55$Hrrv<6pg++j3@w&AywL=zAyEV9nhD=3IX=zmpJ+}>0 zqKFD}Zv|E=syw-5($h;fiw@sd^X1NDyxVSj96S6IZCw4{A1Wt@>2| z!TB{(y+x+Jh2XSLG7#4LMGC*}$Ypekb^3#nN%C-8-oen6fu&4C2cF{t_tqlj_`4Y=qs9&J)+71{h9lg7KJKEE+p-IER z@(RB;$BAEM{sy7cA0E?dAvp7+%i?~io*;E`&bg~oIsPP8C^1Gbt&SfhN|{Rw)h7r6 z+%JpD1rNSIhF=IU?9TK)Vl%sYvqq>DE}GFWc)TmtHPaybumFLNUO4KuI)&=d1RQ9& z6qlR!phRj0UZ;kmHjupCvsj|$k-sIUh#@08PU=soWZLx_;Kp!-R?Cur!}@41f&<}X zPqmrc&JUaM23G{XV%-EoNhyQXHUu(}xj1cC9cdm!D2okEfdKK>65PF=F5qxrD%9dG zZ2T12F(L$UB}qGGLTH2F{=6KQl(D&|<%QqroBwTklcDe0G1gFYN1^L0o@kZsJSDWH z^#t6}NhBD?DPilXnJDVvxvNVspP47`pSz`f%gEDI7R_D8KNzoh^Vf>n1KMNEBdU0w z`dH$e54x&d{!8sgS-vkut?kpO)A; zkZ=MP{}r$;{1lm4VOlg?ECeE)AH<6a-V)SXM}+?DJExb>iJwS{aC8+C-+NrsoTd9CBvW2tc&VeTT|^ zSy_YsTI$SU2-FoeUI1&9jei?|WD?kiAV>FDrJ`Jb2xIhi=97%4+^Ep;nO#i~KVG6> z_>$10_F{Yw*=O}u_OFfcCnmpVp0qJkpw*R1^B)IYeOmshx<3n}5R9leC*|&2$t?Cj zhLu7|l2wdwS2DIJsH?=g-(JpM?0lI^?Jte~wo#eB4S9WOEju#a8rug@CVj>X~|661B zlFsHZq=p0xLwq^z+6F!<7sAQ!?lgHcjaWSAfZ?r~-Er2ol2q@o!Yn8}>GCzR_j&-- z|LyxDMIS4l(KIz_z3qSNUP%Ffmo_*lL=hc3k^8LbkFNx4P0jKAlc>MWqQk{{hZpi( zUC!~PQp_PmZ(A@qkvV}x?v>78@Z*aX%;DMNY|?9;***^Ae!TXqpcBm?2KETjMAMb8`SW#30zxG@=;bk6XJ#pfW>8YP0zeIKJ!v`-F^pd3a#(YG}1nb;%rq7{xS==6U zdTSv?)v0bg`i_T7*7o|v@iquFIiMq9Px=tStMXH7kZNs7h!+2aL1U45M{nry2d{gW z(P!(!jumeC8MW!3y`4*EiDhi}Oel+IrOmWO=%nyqThPx@;%_2w@oBXs<{`oo>FEruYC!$P@pjsSIJA%hPXmxhz$^*!;A*nK@; zLAXOi{rdc`bc;Rpa@>n zXrKi*kHs-)Y`npRKd!#%c<}GAndIB|Uo$bcY1OR5b!XV&4Dnf1hMG7&&$2$xA{2&^ z2cvSG@{Olhi*?xC#d+xdM@zbszZK{|Hz3wg(wKLyYgM*%O#oS^Hs;ea%&_PQUv<^Z z;dhSw7oEE~gcIDl?$%JIjtarMMev9>&9?3OMiCrGChy$=zvGCee{Vnd16}M8cTH{5 zd+2kV9Ln>8SB)Ud~Q@6pc!lY=1N3P ztS#uUwf`W@iRxOb&q(U!+|J|X(j1(p3nOYCP*O5Va%&LMl`YGiaGG=G+{L98A8Fzu z#YZ$=aXnsjB-C6!eymjM6`-dCi4hQjpCX2L%|Tv*+G&0WcR1sFGEXuO((A6dm6+4l z?J}>Nw%#2JxaOolS#bMud~E-U$E&ddmpO%s4V-a)*-%5_WbcJD|!0qat5?I}{ zkB>;7)u+oe9jq0sj%@mFRx$ZmI2l~AA2Oob1z-(oD!HDAQ4YA&9K&Q9eBRru5g!4v zqUCN_QFZ(gx16&lh(|jEI}=EU&G+LL>W9Dj9z(%!neU(gzy2<%mN6h)zfQS_|`oD&Id zv0>2~jtVYqVH3~4vn6lh@^ob~&&+)F_8cqzgQ6GqzPXu6QDn^^GhV1nmAB`aM4hmk zqG9Q@fekh?*B?bSL8Ya8*2b@XU3p-!Sl)RUzMso-6wr9VfS=5lOSGTvfB_VRl|;DC z(!#wT>q|dHlS&!#?%d00v%B{+@4ZR7kY!Ps~}Mjf0p zwhRc@1AF2wV{nfP(cJpNk{D+Rt93b5a@1{q2RqA`m7l&j(}z(~RP`rS&l}v|?3l;~ zjxsbIE-qaduW=vsWMH$x@~9@rLfRV>!oytZp*(mv;K$V!I4>S5OTY@$cF^V{V*Q2X>pQL2TGW?L?quJGQh(CkPT)tcn_? zVU;F=yo?8}scH@;-_@sBHf{yjJj6t$QwAlqSlw#Y4u)C|uUv{poC~KLLyuh;(iT-^ zhYHTfr_}6?lT_lxa0}w`s)3Nh9pVH-K{QWgouAk?rlr7U2vz5`fP-Hsc&$U7Cr>+F zmGjA4+yruevy7YkQnR)eexXX|fp5#p@sRp;jiUE|=b0GiU3T84_w(NVb^4@%LDK~H zEn-Xh!%AfJyjs6L?VGyqa&k2V>cz~@p$=-kJa+Sj{a?@Pb@fa1$9{_77$*8C{zGbD z-@~e}kJ6|v2I~Fu{ip(GuIQflegM*$!26P7uDoh44s(UHVWn?O=);JTc*-yug=!EEl|} zW%7`3RIRB#(euYzbDvxa)g`|F9PQWlcV-uw>BkV!XIvUgc+}GmbC-`*VNUMWLZq-1 z@nco4`6>-K^qjk;%*%py8@(Ev$cz&YC}I<3us<`3<5#MCW%u>sfykQt1wMqkuvfYf zxc}cWqyN*^u`DCn%+%q~Y|?IC+P^Mk%zIhgIHi%1_^rfH>S0OwQpS@0J^-&NW}Dpk z>5owhZjk>^k-mT0?TKWT&-b%|B!ND_Z(sT@LS#Cz8QaJ79fe?bB5xZ=?m5Gq-aQP4 z?j6zW6zqyXxViY?z2qY@KSd7a`v~KK_CDgDKDyPxRG^J-ud`^{0BcKAHFq9hJh%TF8Q&vSk! z(`D0K|9A@YMbBD*MaKT0c9s8V2L7Lh`Tc)UC;ZR9Y>DtsHXksK-Y{k}b}#AA{`UyL zH~&Nc;vm~u>*#&!virV)%naPMulQ*cE6gna@Eh43!#nG^llP2UDpI$QRVY=b~V)A=scQn-&l1NLW z0ed>#zA$RtnSN+!lvKyC$z4VwVPU8gea~F-x2#mIGrwQZwQ~Uv0*I_-Z8|$fLeuO; zXxDItx1_+C%V7V=CWP}4eSMwMd*_18kBtNOft2tPui!!aas2+B{p_uz?4&=&z7LCk z>X7FhtUEi|plpwtq@2GMd#9(>En4YLyq#+|=ef_!#M#N)E-Iz+ZfS1)Uu5*F;sA=~ z+Z$%GWE=feC(JdV<0t!D?#KCGa*L>r5?NX19j7xWn^i8!5UIrS=MyBI=jfoTJh|Ic zY;_eT&uFJAho+PMZcogQH_S1-ZxJ+{{55dlV5i?gB1sXz_IfLgb%+pD1?@chB~q6M zV^-#ZfXpVP&N}vdYCF!!{*|CH7B7uQGm?-s!BKP7Ld0ubM9?^JmW1k*pwde%|p zE4w1&Vv_&*3TNrCI7;usp7#f1=&3CDwbS$}_e5{`Y^`HGzVYqh*q|nQd5-?32yw}~QZiIdiP!>DIb&?1Q2cKz0>wrE@ zO~>-fn+gm&Z?^qj>c8M>re@jZ3mb`kAq#_na>DpgK^$2x+(14?V=Vt<81)T8|8J-Z zayrqkKsEB&$^Z(j?s~tfg;UT}syu-X>_T`Xrba(CKBnGujQB1yHYuT-*!b89j z6g8_HUAhzcQN{@iAHk0uMSRf`F)JRV*sv+{I~SB?v>3!)Fcc04J9u zb6LEr?GzkZVP3TRgcv~XTAaJZ`LcufDRLPb5|5CNvG&0VQIy_plgDAKSh4!a{D%Q zc3cRag7qEDEJz-&H}Kl~fWU0$Jl?s>ME{p&y=AEq57dD!qj1j4h66GdtwmwYafS zrOttUJl1v*XX%7uAY?yGoL-r6sh+zGra;Ay!A)tPB=qh4ll7F;Zfp8bAvKwNk2RVL zvmVH)ryvbc1w%fEh`ycgj`N*)(5_IhQ038LN>8sqbP=dw`EFdseRDN{h}@ZQ9J~97 z5W=%!5oO~=a3M=IrtINNZGE3q3-JQh6cN7H@~UU4_xhaE((^7|!WZOgNtmTN@5uke)!FphvECc?{zZ^U9$dP>r8|xjgNEe@wl4(_*_6T+OW=jyI@=$bLX?fR zPuCEOS_c{kiLY2H3c;`5)GWrm&4-<>u~lV`mAH*DKQ!QD&K3WSLA1mJ!^3q<7&lvA z4nMr+wqa7B^RnA_Le=7yf!dXB$Jvj~Y4x<$#~CcKqDA!{nlmMHpezZGh+Z~viX+I8-rU@Gb6V7 zhk6$jeT;IP%9IFazej98ycya5JU5UNXfJA+j(F4#dmQTTtu2%Be1h$w2RO}~l5<$> zm;E7;6HH7uM|VARQ_~^#aV)PJQ&nSXS+A=db*jX;8ohEr@%^?2UvoEY54ty0sx1K3 z;@`P(&FR71x;?QaZZ_Xxs)D=8XGGqP;fcQPFqshA)vgtyW3y7J$=sx^%pTI)&!4rm zw(M#-Uq9DB$dPT17rMZVGHiUYu&`vkQmcW3)34xl2eMex&P}bmLM&e|+fZNj9h&j4 z+jvhUF0ZQ>l{hc&&WUtwTY9_{W|Ssr=)jm`pIWybm@B%EeCEmHL6S%>6gGo*^guH) zoC2o4K#>RKWHUn(fE4EVeL}P#WQnMR@9Q!@%7fgG*ARyKWSWC{hW9#71^IyB?zGj? zQ`}xgMNnhxP(h7Ae93XDe$MvF{V_gs-Laviot;tTOvhIa<*TL$LN&V84Z*qLZD#kj z+hN~}Z|8ZM1#w=C(8UnaAGRB;Rie*Wz3oOCCAnzdETnc9Tb1~ZRaDS@zrFWIJR)Nq zsE7vGQ5I5CJdEElTNII=WR)QRa1_5T2MxRUJ>%ya9J&JBs6<9WT;Im3IoL zHp|SRT7m4>AEPO71Hw?PdFjcJ9pq&j$bn>rO(O{Fz@WjtL&dpy>&U#D(c;d@5p3H}|E60590@V#YJHTedpkeq@n{eF!*-AL6E^qz z`e?UvB3jiObM{5CMsFn5ab3C{R@$pe?*O-jHGI8+F$vFj`Io%Uz3uXoWtPPe8<kE6W|U3u5Zv^6kcEXd?0i2{`Hb6CCib`|gT!MbLO zIVYa1OsxC*8N*}bmff~x!H!1(3d*{z&Vx6XHsz|;~ zwUlChm`PS?2%gQr{S>*nw~N>iVs`TY;Xw_+f@>TG1`Gie zyQGHBWQ(SpS={z;TD%6nsCQ49SMVC5uueJz*9Fj5vn8ZN0N0T%wMtZoQWrwmAv)cq zGNYg2uG~F-U-v@DxIx@{%}KYXAYl>_+_ zQs92ZLp8e)_V?Ki7JtM zz-mR_jJn2l>TRH1pV@o!54LFI9nYh0UUfIG8d!el0|2lex>2;Fzc&b$BlVJ0@H?*o zE5pKr&VXS_{5MvsVRwn+o|I#X%K7o=-F-D}-YIDA4TVA%D&1b82vC@xvV0TmMnm^u zIL4R{-CJr&I z+Q0e-)b{?8Qrg%yf>7GD{fHUt&mJyH|ljE2vf^5!wkIORyCOo)@h7P z+GCjRvNN?1-E9v+W}ug@tzM-&Q*Za~t%m?a?noV>G<4}>3tMOAmmj@8(jCe^1WdD_ zgtH(6O!y53JW8VL7qaHXd0%=-`{PGVrjrSZ;m~AG>qO@ZcpUH)&dYLzDkmC7LLPNB zVL4Bh)FLlivL@NO6QN_3(qlJnR{jQ!&3VH$j`EF-03HsqaqGkM zbTgp9UfA&4@xmvte{5=SSJ++)%uFj_Fru?r7YT4?rn2dTD?Adw$>?5qxwXI~CY8$b zK+HkY-a-iTi;}0n5qLh2wq011AW7mIKM&hXTx}FYSK-1|v8}*#I%)xy1Qox19U_Iu zf=VQHuNbv?iM4#ain|(FeapY#N^uCT8KdN7@^~$THQS%SjGG8X({+&T(I$`kze7!o z3g2qd4|&cQ;qJ-t#O8oy!x8LpVyV zUJWvl1Ppms)m-Q4|DwePY#8o{3*B}L33fYLYaQ@u`2K1Z*Mk2`>u#RVA(CCRdy}(r zh4;Nv0j{z1i2muyTB~xAVdCegI|sv0{g{4^qg5Kzv=tI~8r}?a)p+zd;~vD@HXX4m zPnY{wARBO%J66ZRS-SU~N-hr^`qZp};YX{PM0u+B#v%yRmulK>U}R&U5jl|ySK#<;=+dQ}|XF(*HU^w|*V_5o~pv<5K@;guHecnO0 z%~UJhN8oO7@wNh7Ey$8!yj607^|q~jzCe?b(Y5?{!SvZx8>h4~tn5d&I|8HXKH@zT z)&45#>|a5e=*T{6in9q2-81_v*DtAuqer<*gP!TX*`Pj@0ku0b7Imj?B@<(3NhDh| z{^%m6LBJg{c+26%H(UU#Asv5Ah!@E5rde2Mx0$Ou@DfObyIOP&r$-V*DwPe2o*8oUN4VsA zVjKfoT5hX-g`@#nug|+CR=nme4er2owc0H_4QVn=giFNn#HTsXZXzUkw|Gxyl_*|M zN{EL|5)4jfa;rXCWOeKdmH4!6!*~`{z!jdEI>kF!5_4ePx&lY%UtafBpXyrr8slQW zFTLj~?`SV9e0xIu`eRQmrQQUVx{Wjq>2}+GzsI^~6K!W3r|V<@D8-aqe4Zeswx>iT ztd4BC4NX8b51vza@Oml>Iumz^R&D9lk1B(Hj^|YSpI-S60uGUY@U92Kb-W|P*Wij~ zZAN7u6`BitxUNhR2u|c=p7>Hv1MUUUD_n~Q1jQ-=!}@vg<=N9M41t70cK5B=s4tNr z6+S@u74Y{s>t^=!V+T%nZuo?HkG<%V#X2Qzl^VOR26wy1=>Vv>TA{w4=FHct`L%+A zqnSn&8<~WY6B!8%^=6=Yc`tdABBv6?n}CheuoE$%`NH)I1$iaW%B+p~B>ykB&q$gIfW;CaYY3A-gf!N}r zdf-BwTPv=5xAr8Worb#qtc9$IKv9)|#1NOnPZj~ESMn$Lv?1M+%cp} z7r^>jetQi7U&A43QM)=`+`Ufj01J3W!~K9UvV5TTX?YbbF6n{LG?<2+50ambHE*0M z%-l`mjXtB(T-trT0 zL*6f}Zt%iPH0g+7+MlM^uO2EHPBgy^58?TJSrF=BmdT z*Kkzu+h(Lun$s0J7$eObPcnAGK-KT~SH|}el|P%G6yD-?C$LL?yk^Xf10qcipzG+s zi!wjUlPutwvZ1LRV1kk0#o}#)&O>-h$Ng*Z#|7EsUpYQb7|n)SPRAHH9;2poW zQ+cpQ8Dz(Nj9H032 zL*lxdoNaQ34@{AKf6X~_D|sYH-mvtdb;9lNhQasWE>(&?-YgjEp4~Tk+le@Q=BLP) zB;Y_4CG!9J>dH@%VWa={LyqH?(8Ot%=rjFs-?H^j5jgp$NQVL-gBbb$?@Li*PT+6$ z8us1E`mJ?SQRf>=(3;P8sqQc8n$clZUC@T+LNVtP*E0K?eSiE8M2hbdEgLpkDz$am zf~-(P{#`M@NW#d9sM0NS687J&*_$(s?bDW z?x9{uTr;h?B3garpG@6wmHmReMV-{`_?9Z68Si>JXV;MBeU|6L+Kv5T)n@(m<&Q$ND&6pdxaZsvjCc;}xis#QJ9ItsJ63>UaehN@<{`h5ls`C zjS2%e$M1kp0{Rl(9Drvvlulo14wVv2t+4if<6Sn9do_dg>fx`RmrIi;e7j?6#Z_D56H^C$U zs-+yXHRFW=Ipt~L!jC-avD1(2UiFVUlx&@ltNC){3wTvad$YE*ce$E@n#WUvO}o<+ zXUA5r#+?75aiFJd@Yi>0QjUj{enBK>js>ItuKwYee^a_{2H4#bYvPdF+Cy5RF0FL0 zXwOk+S1%ng8w9`YTIry%`%_T9x%S>H(0l;BCP8#qI%y=?4WaJRnKp%J-f{OAK9cwW zukp6yiyCfQsh|vi_xy+zuRH}e;J#xOSg~d~L4{4((bPpyU?h?h2BLqK?;B3Opq^p1FC{zljx~hjQUN>d5u>Z z1Ghz`FGPmuJ6kR#XHKSIXDP;`G*sTDW;DgvnI_r>6*C&ro(2Mtki%4GATRTT_@6ns zFq?CU`C}Q=`qvsqD+zq5_a@`1X;PJMIo}&_16xWZ+P*s03tDBxn3heP}*Zyw* zY-hl(F~>Eeyx`V%`!7bIB%&#Ker@Wz zgR0vj+s<_K*r~~=cUl*|*C0hMnq(DDhy68W$W8W@o1>+1&UT-o=b*864nam0WVP~~ zyq26eiiXC_x_`8WHfpn{Z=?VH=#F0Fw1=tJ>`s(%HkfdQ3t_dpvOlIS=^TCcDVEVg z1XEd^DN~QIYYHHaqv9L(!cqJSebbni^(|>re(Wz2!P^D128x;+iLp_C7L`RBwCOcN zQT1V5&q=DSjQ3fORyPE)<8+JwMX@ukpPLe5O_NA0Hax`3j4}1Y6noW(@qh#@fm^^dDaejj!&7a! zJ_%8`zo&aScgJpx%>>;t1n)Ia@i8`OoaK8`9pqZv?9C^*L#Ce@C$g94T9dw@$#%ic zQr=nMMDs~co0GT_@L3-m1)Gy`Lhky9&8aaI?kL^G!KOsqAY6_L4IC$17FSKtjZM!m zONiJT-g1G;nj#wKOK{R1h`)%+$+AcX2;AsV&Xa7HFE3 zGT0VJXKX`5iCP;v6>roAtB($bXnq!(NQ(XQM_E#h707FIWBsZ+Z~&6zhYaPZ4;pz+t9 zo}!AWqgmr+*ftr@3EI^FY>9Y|N980frRLB;04&1m7dxgtRc!#g1sfwn8xXlCkd#2% zHrhBH8DSqdq(hq>ms;tR13Xj~0tok8kA>Wxx*+Er69Q_JqgK#3+0tIKYYL6C4v7V~ zrqsi=YV(4Km6vN`VnPqudzhvgA|waPo;c^CY1Z{61e@a$i0w$%-mB`qVH00oMh8z( z91t!Uo{aO)ipjvohqY9NM~0Doq)s)V!|GvIgNFy5%McWsymwCZ&LOHJm;{m6AI=^I zRjU5R-?(D|t0)XH1K$d+^C7PMcw&Qa%6RrgQSD5}DxF%}N}Z@VvXwX3vrK7UuKt0I z`m?9f!z%o6lf2&h3(AviF@KWl^Itj!Yr38-@6Xhfw$qri77>o1r z^729Y$9aJUY8auXXB|iDr`}Q9pWy!dzx{#XrT@SO)S7lvgoZywsIR~xKd1kr^WSa5 zzIFvpXsWp=!52_Zg&xHU9G4EZa9Mq^w#)^Yg`|9NA`tbRt6Y!J|ea_t&rYkk62oRyxpCW12L=PWB@}ky|Bz^S>R65nz=2>w5 z3Q{`CHJ>-5gq$IUq+XHH!8@N)+%RC&ls1wi=m=Q#SLyxtXEFc z_l8l8(v(2gnq%XU{FhYmwx~diTgI>FL;=5C4qeReZ0XivKguC{`PE+Oq(eipowMq` zFLWuWT-oxLpx*Mo79;^>jJr9^T@b);FK~UiOEN#ZP6RYDet=~*n3ysT73lf`Vz2mL zc}HJN0!g@=R_xE6!8{c6o%f{kj=kMDL*GYPU~_A9CJ%PV!10{grI`s^;{+y>{26&F2BpyQJ`ZnleLTK5tcA2dl7H)>M} ze-QoKq-_c<55=08!d; ze0e^6;v_K$0>XCy)#i95K89V8(sqs!sG>04p(Lndjoe9b5u|fGG9v*vdV&Kl>mFSo zb1ALgLTcf?;JRzwc>VhBN=hNU=t@Qz?1?F}zj^^4#F^6T&;Z%wsRd}7W_n7hO)#9$ z$h7jwV;CANY%%$?WT`tUyx5{aUNxt%ekIurXM~-^O_b)Y&6jHDBpqsXq5sw1w%RxA zsQMbhy+WF$^HUP|I^4e3!$8UL3qfOPcs6v=wvUwN4WHbD1(%vk2_f78H^F|M3=?t$ zenIkH1^!Cl_H-uTO2eo7G^g9twM?`b-U;y!_osQUK{pg`DKz&L$aptR9^Wq3)69k^ z;YQoNJ=cn1sIH-6PrpsoO|)0^872%!F;F)~QFLsj5^L_UdN?i6wCtnTpUC0abI(_o ztLXUFNLp-h9XbF9f0;``0Vpe*b5-H9t^k|gC*4P}(-%-r3Ej9oDfb9*T%9Eap==~u zrw>349(V7Y7D6mC_CyGBJh_wy1{MeAaZ*7)W*Ny_?=(yur34WR?oAH1NG}Kdff<#r z{lT;dnh~3pR6}tZ60JQeO9H~ zU{rnf8#>O8a6Cy8haENZOmbhesDxy&II4<~7z+6D4Sz9A`QOW8y^X>fDTJh?v%#Md@!|vkyyLBdo>x$8)w+~Y!KZj z&SFY7-X$1tP9-@#k~y8SGse7HRAwW7_#6Fd(3w$2%1WO2+pPrjM>6blE;}_(Gh`JJ zwd61?N%nq3Y{q=T)&x~OOxc`z_*3LWs2uUF3jiu{+Er;89J>4GzEgKaPUVD&DCO8K zikLjwRX?8-QV*ORKjnBdZlAiR@u?y7Vu3R+0BFbxWm!FYCw5=P%MgH`S7Ims|6=S- zqmo?Pu;KP@m!%nLIix1L%v79mp7qI66B84`0jI5IDpt-#Nl@lIKBZ{eF+)T^qa3o# zAwhJfspXiO1CW_hLMBlR`@5fSy+7Y?t?v&%T&yMHb)VOH9_KM2g-l*M7Z0HI1x|9w z#{!p*{JTrl!dMZr`=T&m3ID&`OXjcAF1}1(2MT*im^qvI_tNm$|M08xmMZe}>9Ux@ zZxj9tm@HUZ&x?BpPNKO_g)fH6{=TI8cGiO8nouhp@{@BWS*N`I(n`aTDp@ZC66#s z;=}5QNn^)9fr4Z`wvCveD{o2nNAWp=&1+j;Y#cvtWUQ4L5MH=; zb-WJluJ*z$nGUb~v%4qS*}l{2wX9{jAEY`oKgqkDQ9qXGKI?hv>zL(RMOYP~hilK* zREzoQ^A?_NE;@~JwsX6LKa1?Y{C|d3oEGrgF9E(UjZW&oYcWoq7uW9z795%CL4#+0 z_#?+UZ}R<^HEIz&b*wI*F!fl_VN}Eg9ROwEY+u}CE@-u-0&nN1(!6 zk=CJ*yRZQwcicO?rg_oq;ib_Hn#c^3%Dz;ih|?0=3jgL4SvY$k@LBdpF5b0Q;c?mZ z^rap-bvy7u%&8yl)14liQ6(XL;Fd#1C4R2lwWc=4_f~aNiiE8DeEcI(c8ALO7~vTf zuNvfly|8?WI!P;_`(}bbu6pin6>A^L0i8P3oDGGTufggf)M#(qK$#6cNq$H3Xc@Jt zGp?2KA21vf>)iG!D5Oa?nc_TF1uZLK?{gp8b`qQgQs!c64q&YQBLpq=?7%GcbyJA(SI`68JS-l zyhLvP6Unb`o31V3a?MwAhD`Tl2ED*q9YDTKK*ox8YCR<3tfgr0Pw9`T)0Ax17#Z^M zJsC#On~1-5u|CmiBjKe4+KePSYBQE6BHRZnb>Quwcm1|Qv#zis6!y1Y+sQv~F2j>o z`}r#rFmYTedsyVUxcbZOb+8@ViJ^ z!b17b zQmSa9Y4Vo|Vzoi!)HtM&)0B=0&^`n4uSO4=5-p4dd8PZqLm)=Dk(>7M)Jmc!r?!fb z65n>IZY_0>b2a$rPVP0Iic-yqNN;48@x;Ho4mvtwZ;b=mIC|z{^WuO!n60#u3y2_Q zDmYOC#>$u;HdHt`^dIR5URl4_KA{}*mS0wDk4nhGnq|keT6Ub03uXQOnw8%V-|9cx zA51ZRabf%I+uVJcB#*hH;0E?LRF0h^`ept$3iEI~=2kMPjQPQ%)C}N5MnAd#2G#gw za%gqd)PMG6V^CXOhP}LfM>49gTjG}AKoyPUe#$!aE$U5(Ch*jx;U>T;hCPv?2e6vL ztGu&b{C{@=2BLw272aM_BlgS$_J-9xARi7dI4{h4f(i8C1C)L{gIkL^^iH?BFYogJ zH*|R~=gi<6h<4@K&t>Gfw?P7nlizmam31<76Esf{VbBcdl0>V3cvJNH z5u0iSUW75)Vm4XBNl^Ht?rD&lmljE`UvsvE_p$TbQ@2-6d_xQ1#CXAP4(Pk- z0CtrHJY%t8kD@h|Tlh~X@cubT;?c0W0XlwApmSJ_@Knvl)=SM05 zU|&Z}UCwlSoG z?L7yYl8p;y+c%Fj#+2jx)bSaBS=nm6u=qT$6<}X~-;x*NS(Tn1>6s#%o z>EdS+nue4!Uq%*M3zw66%*}ETh+yL+kz*y5qZp%r%ybFl zTqoWLBytb#zv-zv(uyYQ6wuT#w?0r+b;sZd1htfp(NNssO5>?W~W0x_19;v_%dSoTN9QuIit@OHRHnC73Xu=wO)of{dYW|mFG z7yFPsb~Kf|x~dEI5isw^5^A5F$oe5^$bPC?f!Z?o6a=>Ln@0v~{)xQ9Ui zo`z@=Qdnhk5o${L6FAK6^gBCVLcQt2N4#mKR}o5PgU?Te0*jE*^aaItI4!6A?{l|3 z*j}%}hhmzd_7*79ZqqS=%xYDB_ba{}XK!&`^e=ir8&D4egbxOK6IhWqYgeBoyrV7? z*y~gA8K{@fjb(h4-oQx)fE{&}hLM&+2tj)@Dw7fRa|LFOK-;B4x0jJonw2h)Sd%KX zpoRWwZ?~hcJc2c{Z=zc6SS~Ru!##04)A(p^!2u_S!1w7I$N%_=APIhKScuLD%K+Z^ zRSrlU*exjGFZbm63UIo&RiIPAE>9hwZL8f~V)$JY&~^++Hc7*sS_s9ozpl0&r;{Jp z6&*ykxTI=UUp4x=v2nR0_WUcV&qbo^^DAVp-@HDdjE{g2kF z_rACA-2Zj6&BlH|^}|{b$G`RJwLq^!!rS!L3hjaY5Lls0_JHRYv8vI+0%EfAi*V(d z;}buCzHT^(T=h5xr0^1~8OaSuq!OyoEIpZKbnEXfgHtVUxVaRcxp{*N@&5ZRV{#Ec z5lB4)jS4afmOw!n)imd}F25F|fX9!4S?}#pml}VOf_O#DV>j46uWM$FT_n|QTNz2w zq3UflvvwRz`;=Ipy_gLE8lAhJVEh>EK)DfHkug>m`3}Pi1#B-M9()l~;le9@)qu9$ z%Fz7bo`x(NtTw`Up;y!}Z?Eg`5jC{?$f&7)gav9uUQ301C*?AsB&hk#$OHQDf`Dl_ zv#c=x>61>CSN4{nnFHItQ`JWPm3s_EtWl6M3n=_o+pT0Sst&CePWorn`GJ5u4{piY z+%H0iZwp2IE!Yd5Td(n6+%SHZ)KTmyRIq$$MB@g>0w8-|lN$qds<>fw_H$N!p8-~T zNn+>GGu=9G7bu;o2z@afV(9suc2&_X@TPH;oN;VM<|%4?3hCh(5vJ0`v+1`pDm;<& zz+=XjbngM;uSzb;tYNJqF~hnO5p3YW9P^}Je2t``>hG5%ybL+4@ntOi_6nSE0oGQ> z^&BsWp(a*F+1&ISh?~Q6Lz#B0VtUT?~kt=9@?Y8d!_gF&} zw?JUR@NS=uUE-!QIXVSQS5j?uBF(o;Hd#*(YT7%QLUxC0Z0A1*%fs3%YaEYGR^7_Q zBWrUpl`d)+`*U3N_gxS~!eP43U~LXrEuN<9RrYBUEXa{O>4}PVS|S0=I5EHtPSZRk z+|6?naQO@EoOzvrjvqxtA%#0^w8B{2D^sx14PJ=;2dlGG!h2HOaOTuqNmnEX)9LZV zh!~S_l|6Kx)76pxyO(^4YwJ%6XB&Gxq}E;fE8hg`Q`kG+H$1|_68I{=I=mBWU51fB zckSsnnlnt$w({G5r?#Mks{g)b(mF{lAA<_{j=ENQY!KP4ihNSLv6eWJ@Pj_*B$f5! zYyrJQS+)W8IA?Jtb^v!+c>4)fW86N4m!KnjMU%eso&z|ER$2B=Po9uCPI9Ydp|UvD z*xBIr9K)0?*)JiMu`lht>jxA+evE#AMx}PYuBJ~PfwPMH7H3u%I+JpDcq z!?{ET$5{-+tWzN5N3o&L$(!0?Kvyw4CBvkrtNCrvMSB<4rkxIny>C-7L%<-#hci7S zmB1dU9~heccSAdb8+018o1)H^1+}(zO?SFApv!Tz7@Gg=hv?FFRLahGz;c~By>5sV5VaYiI);;9?{=&RS7dnIky%>JOx0&D4;!AM5Tc<9DB-NT0c%bx^OOssUsh;9H{ zB&S9=P4Am`e0*6!HeCa@+ko}%+7T0*ufgytz4<8wB=Ya!3-&8;(2Uot&kdTptK%Rx z*R-*c404xr;gp42+<{HLNCpe$E0^)^^@sju)Tu9W7N47t9uuE)I`q|GXBhwb-(9?h z@AF=LsFC~ZVZRISQk)bpfoJ#yq8AMvNBx820R*!H*Xqmya9^X>;Mb=hjecsMg_@d zO~0nkth29YaTe9^j2qmZ+>Z85{aBF=a89C%>;v(au_jA#ndQ$Z=Ck#5;_5G}9IR&q zIiP8$o2I|$Mw;DbB+)!{;1qZ6i@To|{o-oYUg%xM#5HoX^DD#Ge3RM3*<@IL%r_QO z4qivInYdG>J7iK`;J;)Vy6M$q=jxw-?zEBvK|B3kiPqk=Ie9qp=9ad-i!E-%gzONP zx7gmZ874A_?rH7Gz}bt5oSYay*lC0XY5H%7tIFY^&C_QiXK z{2m;MeK%^768{3lD=(>%{cp93lI#REslSSXL_(b5`RJ0vbf|T1>*~9vvK7^#UKsN9 z_M4~wWrZKNW4X5JjnkduDN3%_kzsF+>JFmZ1pftY#7Ym5v(?7y?!A)i4Qx`Qbd)J5 z8xtobG-9uH^2*Z1HgTLXp2H8ud=gVZF8Bs(K^K2`el_+gf49e!Sg+Qtc7TdD!&uN+ zr3H>DWVO)QlAi?V%X6o9qI7lC{DtA&uhdLxaWQ|*RtMy#KzuP78!H*{>15kybv|ga z;3_P@xirSW#oFb?V7)xsS2l&}D~!m29W7MyA$xr2LDRNj=Is4Qk)Pl%NyDF#M}VhJ zrc>MHt_b#MCZ1O=)^<8%?+$n|AIb*&zUJLv&E262$2?e8@hDZSHfaqNDc)+Dp9>sP z&Qk4qnW60m{W7d4Ipf|R5B1m3v@M|xl`0fE{gyRwF(ZcMlxF?VhobA|r_5GLQ9O+s zYx8sMvT-FD;EZ>N95M8TVGL5RN&()UERR`Rfa7KQp!q>QiFlvC_5Y$_`}kV}MYADZ zcS1~ovb4(*KZb* zO3>rf@{5%^zeC=PQ!ZZ06z@DFhEjeNo{Ga=NE1j1Pu1ZFc85 z749^7v473RI?YyJ28dnzMk?5`S0S_0zMhu_TO)-KYwRn>e$JycgI7uxd)#NSt=}GU zW8ekjxnwKzb$1Fv&)1l7ZYm`>44{MPnHNjodI z4DvaV5@n;8SpB;MP^n&x1Y%2l`F^ae%+MAf*w}ckeo*8pG>ci9$8(fPoT4-*1;9?P zm<2xSc<4)w>ern^RZjc6 z9k5v8v`9M!e5hMD7UlT&Qm6{nJCAd%D&mP7MVQAjiS3B|k%83~^c)g$2i~0_gM{Zg zb(TD?nl9W{_bT^y>95_ft9*Ix*k*!l_2{fKta9o0wg`%Q7pl?fRRrDxq*WEo5ScY{D?%dQ?RCvz zoG~VEphhi{?=o-)yBo;kyG~l79}H2oH(45L((eNpW}!KqKK$Kh#?4>t!=8Dx`pYw; zn@mHi_yrnnkaA(jb*|poQj^zzJrxd)RbG-bKEsqhykewQk=LrRd{zP87En(at1F3s z*Kt)}_QDx#{^9(Uoz7&QIYI+o|0S%uEt~Euxaz7%f)!ZD`jUK?Z7yN)-! znArw8f>oI^Z_+={m*zael@8 zw;?u#N+$tZmGJ0EqQ>q03+}yRS*`afNs~Z*h;1{@>5>2L_2F!dfZ<7W ztXvkAJVBWn<{HHI4axh+JRC`@E89v6ILX7!(@ssglGI`nM?i-yfI!zriM9bQF6jtg zqiQolvh|RQhknRp1leWK)caF@O#fDC&=^UsVI;ALGu=y{B?<@zhy61>Rk7BImZV4p?U{12hcg0{Xu)i=6-sSG{>FaRI*#xt;jXKH- zP7yD!M$`wNHCL1GDu}zo95tntoK^R&)OX8zQFJW7Q1gAE@zD~doDV3_JrO30q=l_R z5|{I9J)fWpM$EEH;t?&>hyI>p0Ry10)6GvRSI){;;Gz*qFX8a=;q!L7T-Q*XYG(Kg3Ejt{c#&X<&Mm+F7+rKzpPGeA8QZ zI>AYfMQlj~=Q4%1x_$$fBJaC40C>a9Sm^7g(l_h^Hm?W9BuW3^qYCt)E97F^BqOwx<}})XF-#Z zSxh9QFw?fkq1ao1!C1d(Q}u0n%UN}CRl(v#?hBZ00Rj*whGya{fQF)4Xe{vH(KpNZ zWHu0}etFxQADZW=C7l7wV*_hfHIGho>hb>F6~dpsyAg-=0`8JOI;!cIr}#?&)WD!K zlH!!`U!SWY$R0C#@A^22w!7g$U6K7rVQi@+yc7~6GN9I>-BbcWe37*U^mIba$x>}7KKbZH2<8E#F_VE;tm#0t;y*b8vZ12n!GshxZr||fuoZ`X1|vu zD{P~y^ziSswhyTgL+Zn~K_tK5?qwW|vi+_xZU1E$w)!p1?7%DlD?Z;X@Avz+HKmf3 zO}_sR-y=_(eY{KJfxPxoD;3BSlMJ=OgBJzgiC~bV+9@V9ucozT0|2T1AW4dLK(p&y z`H7xZ{AP>jEXGPu%U?;uokk~!lPg|JWqOK(fe@KC`q3v%6d|2q8DjE&CxRAZS!Qj5$Spl32FRW(p7R4(+Q^6^A^R+;_T)FAQy20j9p}nnM=u;6#W!h}Ha+p}L%v4!39NSGST{~%mt zJITz6y`~JXGEEq%baVQ(R1a7Q3g$eFIp71)mTFoU`&e6qGeG0_IM^tt2#fJ!a=&3B zC=KEIOJ)a>iOvaCYM`SGn`E>T-e$!I|{XsnT{;~MG;fN*={B=ub0v8~N)M-{9x z&^n!bDtY`Bw||)3(kL94s=0T{NAf>NM^x}?#TWk(Vonxu)I6HVz!J}?72NCnliAM= zOd0o$FCoM0-941hNu$z7PsIf;FKk$j{5=iNEJ^VRn*_^bSMRoUpVv#Ei4wZ#CR@P{!Q(h$fY>h)gE4D zU{Lge>bo|Y=k+b*ShA=K&<`TVP-_g<(@OGLgMiXg%9;AcZ%u@1$>k+orE2w7^7pDw zco8ge`~F(Cov2myl6Q8e_)?emweK;AeLa4I;@lnVq2TNFvACc86f^sjbfkm`Q4P9u zf^4Nb`J~*tz?|w)&bzV6=Z*qMC!XgC%D>^bn@8xTixRNDKt9aByZUF(sdNm*bnk!4 z2b9e@IJ?dsUTgFDYjk4+Iu;ZWfi7>cH${caztKH>@>Ex7L+|I-Z#y=rHlHw^lM~B9 zYfsdVXI`Ti-bC6Ui|gh-m$kHn{CdgEphJF!n1OX!Uh+kmil_6gDW(h4Y@F;$3*6yl zZEY`uLvNh_f_<+eKB6TMck{q^y3GNb(18tl67L6xym^;X^kMP!!l5Z~1mEL4pE51e z4?2cb<>6VI_`a#x%alhm7!m`mkYrf$%doWOAm$;jy96uw)Phw?9-Us5;hD$K8j$BCsq(*m@Os(jO@?1hY8S| zt`2{V|3Eye6n?anRk#N)iT{>yB~mSkV&QFd;M_bUAz#ZeivyS(a=G}|pgq#}fL`qV zQc>EktO39itM$f)NaTo#U^&Xa00bl015?ui0`Gb;w?r}|I+Q7q=mjhgnPv@^d?+uxLh*Inu zNBj&9ZCpaT9(gJqvwzsWvF0bh{cPmz#4ju~VMJ&-J;vn^{$VxRw`zjkqL&Xb08iao zuNm&Dm4S1C2U~m7A*(g${^wSZ5R$dI=LwL-^j1?@7{R*-xC<}J;U*Vbyf!x3;G@xi zu^W&_jomA3dW;)uSDq>l%}dVi{u7fJ3dY2G^Y%VU@iZQ=4JYSq{8{5>WqXZuQ<)SO z7uTvcbclEQaaTr=ubFA@1SXRTPd|M$^1|r)#Jrws64BPguU6NY+)6~HLxu;HB8mlw z+9Dd9-bzj%iVsLXrut$KX>zkJakK%kQnJeEXs9Imnw8ux)sh2XIjB#aouE`g@42=p zBmPs;(yc!KQ93G>7jBE}f~A9+x?@s^*shhoEwl>}hKkM!*8m_cE+zs%ycf|OQ+K(; zAOx?QOk9``1`D*Ou$5?zx{g7@`AgZEue7~q49>LbB51<-n{N~`g`1*mY_sZ;B;+tCX~$@Wv%z*ktZ~Ghp@0xYy|j<_AJ($J1wm z6fScD%PBR*XDoCIPTPy_}1e&%u8mKpVu5!hs~_A>V9fE zsbq3&mi}D(_HB}2Z?w1T3St$W054QFZJdWBmE20Q8IsEzCc5U|&K*g_{_#u^>OI8! zP?=V0_@dvVtVX4Pi>2$gs@o$#pd48|NC~j)_5hw@pv(rUxKkGZq@k zq&fbMHJ4E6oqp^Vz9UFRSR;S~UPFK&I<_cR#WrHqwx_^eo$IB(z9?hZc{4R8iIW=h z^-4r%H4vv6lRU##@0(_KdbM5b3t#1TE21-;B6l`U$?VQ2r9VJ)RgntPbuwsy2Pm#R zlT=tb0o788rW+bqDPANu$*STG^)Cgg?SwjwnK#yQL=OdF8+%@IuM%C1MEQyqz86vZ#93b0*@7HIX zK094BwvXNk!ycy2t)jYeF=?@j3STTWO`pF{(SK9h);vc#Nsat(AM9ckvf?sE7m?v~ zd5wKqc!j3m3iocaEhc97b5AEW{qa)TSPQIdArEZO-+Trj=F3(`moXYw(LPCKlYEP3 zH>=g4rsW-(TA+K(aH2cZo4iTVZ5XF{dTcvRBLO3V84VM~b_pwdCJnI2Cn(FZ zx>ZYM(|OeM`U$)N5Ob;l`4$2#(I4)I z`8T5aW)SgFuoK4Xoo7*%4)t3531JBzP$2Zq^uA9osMzn{Q&z)1Q+jY8zlx$3I#8BoYU>D4Yl|`u z&_gBxk{kxdbO-}n^TYm^n{a{zjT)>0V~;fg%3TT2o)-(p*-m-*qseqtjEdl(Fo|1p zaE4rT3NMR_wVh_x%)X9!Kg`a2CLQe@w^vFc_BIIB7#p4{Svww0aH0B}l>P6q4=1a^ z1|ut;PonzFpNhwFOCC_l;^%Mp8j3BNY{DeuE@Ntg_m7Ap%j*{b59(OvGoS-7petA@ zOXhsunyEsrzo#aaAY)uSP}2>w9qFk*0em`DPz01+k+U!rA)z?|9eRdTM(8d&f_o#i z=4EAc%VPxK-v(4zq1g-S!eKlWcr&7>bbm9*t@AP;fH=WWYmnF z-4!| z$-o9t|Dp`Osyyx?4?&RBzEzG~Q9==vUDB&m;Hs>@C!2EM&$U$i_WXVJH!m5r`yMl? z8dgRSaelIWP(6OB8V^YGc}~)OuXs^he33<0)J2{jV2D)F@1$Vw!*P=J zpZh4h=kpG0;gr`l?1AEat?IDTnQT)Jj4^Jd`C=M<*D#9i;cW-k3ynfX_uP}rU$7^Y zd`dpGStRV`VVgl#y6*&ZJX_0?(pX%pO`s*O=RVxKk)C#h% z`8F22fsk!*8zW~^zy zNaNgJA&Q7+s@)oN)t6!07cN^?61^F63h9L^pd6wTmDLlzIy|D#Q1|mC6<%Jl8uTm5 z{pJ4-@@c?&5zdDdh|Y;|fMbF;e_dc9A=%&LYrS4n!TQMLcY?VfU`Z!EDBR5nO=~OK zjd2}t8)FM^Bv|FNRe79|vUs+u0=3e+_^Xk(V{TFd3t+g;1@Z)qZy~FYt`}zdao}hM z;k@l+yfFV@WCqi@E(#aJXfi-Zo@)-~mFRVtH#9U1mFb5D*2qiHSM6~zcNv^z7_`bN zleLtPvgSq@xNqyj92X$xkcOk1k&f4567j&*(T4{Mo!6!N1Nm716!P4S6*>#*uW1jd z)vV!**!X;L0#P{d`d0sa4*0I42|z}OJBqat;_E-0k?H*+{R1O%9G=*W7XiLbS5oYmyU&D`XlU7l^L=V_2D{Q?Y#&0PBzz-4Pr40jT(f=J zNCYeK4wi||1NTRb_;4y`geFRoPW@LnDqu`g0o3- zZ5E|w;tk;1TN!M-jk)!t%f`0=c~(Kq{uu$dkGdX-ukvp&<3K=^Su?DSZzpIJwsCBn zmTFOP8_OaybhU|nf^12^>9m(yW?m%x})~B{j8OYxtb!TME zxWGd}12X)yRUZm@U$UG@ay{5)A&be_6Jmm!y0FfIW3Z=E&n|lVSA+UeAp~Sp7A1%E z7sQWvw2rh7@^Qx@1BQMPGf}mFSY%GT2M|+2TYnU_j(*5mdS1*~%s~|v@$>qfc;3T! z)&rpo7w~zyT$CNEkg}MD2k4L4STJvz{guGa3m7|<>H$6=bo1+;eu7nD+q5no7Cs(e z`j)BOxM8cD(=<2#^JQ{W4lyl>5Q5fFX9|UhutS0mjVT)1_LprN)#4NkP9&g9{GMi9 z0G!0rxw76CZ%Gd8lFXIF$uS8vhsD@D?m!N(>PYLs#T8_hBvwhJhlL2c8o)<{Ng_p#akiRHj@XOl5l0b* zvf5EKS+!$dQ+(=Ie_bb2gU}XzpLj2Q51Zvz1-V>USI?@{*S2*+Fh4)MC7VWW1hBJ8mDp{uOU?oK;QH0 z%+r(Pght@hSi-LrCi7z2x=fD*gt$i-ftHrE=R>1F5Y z?(LNDl3=}8FD94EeafMZxKQ7e<|~n?$_EsmtOWnu;^gr%+o5GRT6s*NVXgqFwB=p0 z8guj?zQjv#xl5P>oZKToiS_``dI6a|QJGkIV9@;~+^^AnN|?&`^*qe+D{LDY&0lN{ z3elYIjN+>~oZ1SU8pxP3^!+^TS(qBE@88pNN0>Q|Y<`;i=7kSk1ALf6)b{jdeY#I_ zHGx)|n191ELIbKSU&g{Nl2uAQpe-@ z0tsG1^Q$^c zl{;l(ltTS%M%>_itq*;TVL?sf(@LWxwCuO>!WhRWSIn|fR;@Aq!_Z8swPT>{kuU7k zziPdwXtFnZl$NSuw0-H#ug91^4BtNO4Q_Cl;U-OIL(+^q=B9ZqTJksWJ+k@rwV+(7 z1NM-xzR4_N)}SE7F?%$p+{Q(S?aDh!|xio zOZRMw4$&UR8aJ+%p^%xQGGBb*eeJ=W6Erpz$vpdja3JM^1uCU>D39q|CF`GyK@%KE zP)xZA(E^gP3Zo?v{HH2U)F7_w(8l0QAB|I~wIky|!jt;RF za}QE)qe|uU=o|VzWZjqWKHeA$+7@6ZLf5$Y+)1 zJ6dDIg>u}uSu=)-^zCA+@#~s9x29gvaAV+u;=?=$K8Fir(j5mf47U#A-j=E1YB}^g zTYQe#n48i*-3C4^C--W7hQ8zV+v3@%M_y*vBcE7TG7U<~M2=B?h;LsQ#DbXVS`Sm{ z(A(o`7k@1b6xLe$cG**Tr72l33~S5Zh@V z_TujZ{aZM;`DD3PGwVB6UQa7O1bS%K8cblr{4b)ox((Dq)0R z@Fx6Jt*k#zb7&j7J~Qb4zShH%5G}IBPU^$+qX!WUXR$7l6+nO_v*eV6(*jm5z-sAg z(M%|iL1V&S6r>9AB-~I>%Szv-Tfe5UgpSh`ql7dbs9)1ic#kUq2TYrf7ACAJ0FB?l zdOj|PJ$kU7v^4J;?BU)l@}I<-8CGRwZ=Qgan-)M-N7+z}#=pDtNzTcu6B+a9P=8A4 zNQ&W3M~eRYT2CL^e!tv4M6I5uuXmx+@eT4l+8g+X4WeZVWrDKb#kHaU5|FM3C-4aI z-}y>ZMkC~QNE`@C*B6+5mS$ghAJN)YV05X|e{;#@Rr$HnK$1VfHHVlMl?6!-tD=gzzTFU@sghy$-}v_P!Hso58RX=jD(?&P1! zphQn~TksqfYQnf6*^i-f?tj?+$fR3bH47Nq+^I5k3#9G0IHm({KHQ(SHuzbQZ;rTE z8H8BzTrh;y7@ICXAan@_4WS>KVMhF7RInM;2l2UXl|hBNGaeV;`cVW?AH5uf~xw#o|(&E2R9KUD7ukg zJ>KWrR5n$53bOOpx5i+5oNSd5E*ehsQirLQmvHyeMouHE{8_Hrw#9j*qgH+(Xtin0 zB-##6*%51B*b5=K9fTG)*&jna#2{9>bP*NgAwB3hKxfr^TCBo}akNZGJiCsOm-g7;9-F)M8d26o#>j_Fa+#f3&M&k4 zx+dZ*j$I%3?oO_<)kbzGHE1hoxJH&-ZoAZ0!MZfbPvNZ@cKctX?2utistvu)CC8MY zjWk%}yUH)VOl10#rw{E1b_2usO(l)NOtp*{kJ8fw=zISZU1TN6a`}xua*3Z2EL|trd)z)>J9N=`fIj56V5lx}lE!ahceH6E~?SA%8HoT&{ zWFGY)-}WBC4`S6LmjHKVxA!7NU$PY>&U z-g&0zF*|fL3uwJtLrT-g(9Gj-@*dFjBd7Ur=;4(fk&TpP1#WR9v4eGbdw>CF6U+gp zg@Rf)LtTU^A_vTzuQ0YrxSL@sFQElQOOJUScH@l=ai+Q+VKm=okngsRDu~Zc5Z^T} zJ#D+cRXe_UKUvUriXH;*n-rLTf$Y?5-Hb`KkS&h?RDfEmQXM0psXmpFaG$P#!AsV? zRvR%)5DZE6BdD3QIijoIj~wz@A=tu5?nne%x7S_!NUhTDjbmC;=LHC00@YL`CH``_ zYQE)J$8dY>O?WRppMWO{CH`C6csjFKyDbkpiheQ2cbO-ECXk z-z-HQ(6%P2HaJ&BH!AI28_&N@OcOjS_s$v4CaSH^usR*VBDf8M=^Z*|zr}Qf$NW%vHcavajCcM6X%3ll z5s43a-l_iO{YDpAn^BT7Mmq0X{A3mlUyi>B9s5EzzBc|~`FTQhl)Ee7pk}j!kIN&| z#V^SgXxWeaLV^zRNwkC#p!k2bc9&yqPZe++jJD7558khjoi(ae&HQ)Q`Kc?07M@1G zl#&VIo#R`NtH(Z@7{vVP{y61e_R_VB(Ys-HY_^Wz|Ix;87_&zu=Ub_}e$oV)k_X8} zg*ZE5BtMl|19+?e>)3H##RYy*W~U+FjeSyjpxEyf&#iCI`#ILaUcT#HKB!k}%ZtnD zw4j6vQ~xuy`PYQl{ztiG7tw#M!)2$AvE+3cPKGx@lS&+0viyS(D*73>4pBBKr$|Rr zZCfGjwic_2lUWYut#o1DMm|0y7~#BLf1%d8|Mic>#NhE{G}Io*GHBD;SQ{%ZIYsmo zHK(7%>;;C0KRSedOIy2yyP4g82;YmGF@78`27q5G10R34Al(bt--Q6gY)lWgYI0|k zRbUqUlCN`TCv*>Hl;LO-?CPHa$WSoE-i{I#*SQO?0v`Ok%cJ?_Jr|vQOJS}P$L|dB zk0Nc8$CB5Ug_T#HeX9*oa;QVwVu~W*gsb{Klq_NZA|*Hg_GgK&(`?AAw+R?5qje1$ z^qv3VinhImiMh7d%nK+Y0(H3lXF~_3{5!$IlX0XCEZq+@FhFXp=Fw&V;2H1|_BKB;;!w{2ixIMx|(H(Dt$esZl-D~TsvKR+o3S&8-6d$NX?oP__$y8BlP z@4F^A*Um0AyQZ}yGOZLW`TMWvaNb0%=gWT50R{>BGCe`IrGYl2?n`V!)jcS0QcKb8 zfDtT!aJW(nzc_`RpJ9=HvnaEJ}JjTZ( zh_PypIac@LI_;LOf_#BMbOb2ww*A;gPlNWYiTIF_D(oqh{kbI1)@O2nxg^v&%j#C^ zGw)KZLtSmM|NWs4lQr=@@upuEZK30KIP`EP&H8N3LH?1|F^a3DZ?&d3$$hW@sjWAb zKbHt|EqYxe)FCrKGXkfXet_D?dUXU#d-O9jMM+<@-~jEw(VPrjxcSE z)MF*nk1ngCvD2TDSNuM%B$W8TOY~inN*1)N0wx0nN8qg6fu>CBg(*a3lI?baM@6-H zlQqMPY-m^XL5J-y%9Iafdk8ByzkjhE-An;*mL=geiH{0GoN`RW*PlrqshmlRU^Anu z<$$5A#VZRclAG}WkXVNO*cr(yNu8e$>{c8Qz#OJc{pYa1$KsD_L74t@3|Zg91bEkt zjc!?wK=+YH^Hx>Xum0V2tv{Z%Fgnvu(oi~@3R=d#P&2gIo*ArP9e7Z9S0pzll;O1J z01@wHf?`Rn^aw^~phh{A8&@d(4daIp#^QnCFYqA?2b5t_{|(E-!X9>e4v=GG=PWFs zotrHx#M&$^s(rZ?Lb;tvUGQhsdr^b!rj+r%75Z0P^t(ahK@8rAVMgAKtKq+ zCkRTD1O+05xS#pIXPxt|wfA0U@BQID=fix;TA}1wcb@0Iuj_aH$`$0M?!|-Xk)RSt z=arf(@$w^9IP1M$uy2;M?^L0~^rzY+1U8~>3V9bC^17ntIy|Naf|Y|4?wZaa*5KKp z{I3Ii4eOS>cLG+z`an;@oTcasfE;d;KH5_ZYPTTFFMF9&d2aLqp~?d|rhmuy_Zx{H zL6)&38mbu5LatWYZQQwKb(aIzk`K!}$+mXD*&v)vD4r*=DP3-L1%s?9)Fo~IdtS5kr zm;q?Gh)(1CzC=!j_|1*&P8(*F zEjORB>J+O0dvXs1(@H?X;|=|gW1sTlN|5-SO6yt;Z*q4(2OQg2IfgU82oN7tMksn} zJz&h|TeCV)Kp;~jInMv}C)@qqS@_M9!u{Gb1Fd{2D3hhkgfkAQ0l;QRE$$&EC9mZ3 zxVW^^5)*cqQj{B97N;r1*r+or9m3F$ja;Ek$sxNuy=b&vi*Q8s$@gL5XK%6+B()tL zTH!816+~4cf-rRLx2}ZgN?Ca`X)x-Jo)nbxLJEF(5U(>BK9m>wxN3b(+9*SabIdxP zlfM>r(ac!KY@)}epu%D>;DyQNg*eNRpKMPJX%#xvO;cCsc+RU1pAS5J%@tGZ5Tz!1hF+?_{Ln8EU<{{@4|wkZ%#!SoCFYUEbsX z#g*2n#^?Cpzp0pe6{*UW>;fjR!?4)P36~yxlx^ zFqiUVS~TuJLtdQ#rZeGISOkC6Opnc29|6nn5Z+YKg+?G8MabX+Q-9JxY>BB&$7lNGn#DTP1sefN5#Lo6 z0mhZx{u~FHLhsdH8YnqR=REP#0D8?~vqejIZ%Ob3NzPRC{fJ8@q~H>3vhf^K)p`~Y zP9G0KaS7GQm9`i4^eWy)I!qQ^lCtrtf*gJD(UKplwdLhjq^sYbH04xxPb@*2_*9Dk z-T@or5@@5hd?Xm*ZV=!h^q7YRq(8>9&b9O=(E4ySo3g9yeVO9SK)Ue3_5uG%Y_FtV zs%a{c91>8safHq;apV!2*`PU}G_%yME_+QV2gM`w2L-Net&mOvo)MEzf>rz0Lv@Nq zb2@daE_;Z>($#;I;dC}^{HyJ!u&sD*MKMR|xGvX+>N}NTA$4__{jh?v@id3HsMCN- zH|g?8c~gha5LI|bkiMi66AhkVa#W&f4?@QPE+DL*_SvqLc7~N*xk0lBMyU+PiYOb} zWJZ1Emk576RUFO3G#dZZ@y*N?;g$=j3n6P9ehpD%rN8?YcolYGJJy-DAW)g zNdv^%Xb&AT2q0@7hB%Las}>oy-M@3{*pzXL;@Ll%awOazkPP~jDN5x{bO?Lel$qKI0!|l=op!6a<`UYp z4ddwvHav6%#04bK$CS}i879Xt@~gLxrQHxMjwHiMbI#@6)u$hP?%RPw9Y2*g*SMbG zU_9^xXoqYHHUhKijFdJ(E^L_D;1UajMj8o354d1V!f9cc?nnnG3bf5oUoU$-fkJX3 zCD<*Har%X~2Qi!7XeR(L0z_aWeZA-jo3^Tm+LT7axz9a+BdWEO(F>NI${92u$Q>AU zxgcq*v>YLa$BWI$1e9slBb8u@owbw011t-tt~KU3;OpxfTLQHc&K=M)LI&>N`#Nc= z*wx<=tQXP#7zXDTNL#ro5MVJviJVZ=ScW{KLan zu2*fnIF3lO80BHAF|fx5Q6^JBy$~?xOQl8+=6R+twP?Iu#r%3^w6D9}MclO9)F|P{ zD>sJ4;gX^ZUrHgmZ!SJF5nuO)9Cep^5^O`iCv>ENt=!z3TiC4IlTR^c1@F-Q7S8c@>+vlQ=;tqkC#uIPDnNg3|GdFqXn)oBn92^I{ z85V8h>mB6KO6wxOpIo0b!RmQ%2VoQ^%Qq<$q2=qJf}99VQa+V)DqO{liWqnb$`xrY*AcraiL=*t4|Jwf+Wjh`G>VQLr>dAnZQEj& zJ%0;ib9*IBSB284#Pf`7WAe7^xSr_>QPO$d*MNV*m277}P`Z(dKKrT?r4l}=#fdsM zPJpG~X%;!RLKj$g+ys8%2S(KGX9qqEz2t>MYPW?7Y~JiO^1m4P1~uYkRIL|9AMdq( zT??sF6T|wr$-S*c@K>GWuYWFIg_ik%pQWvSns%XCsQkicxyhgi_qk*cC0W{0HyIl= zyfH((ifK<@$J4&{_eP&*SiA>CyEcH{R>Y#&k+B9N&vw(#=d(qw@0kv`Ai<2(9Jw zsSF6LaQTpT4%Bm*N*gD>&bvf#Kf0{?lh@YkNI|>4(|9 zrl-V;$iA~46X7}trC}IzVeHd45ErY5crHrkSdh4VIWTgS_d=9tE5n-V65Dc=K&Z5v zXUrXTFM(riVqvX*?m#~Hi7wB5=_zH&!LReCDRZJ%TCUrFNWEfSq+&dPq4zwW%Lek(aFvOui|a z$aKnNwykQIpHx)K?F}Q-Z)m{hl!G%E{Kx0Kfxu|>FX%>gU@qomnMssjNizagI`OpA zH{UTT2~j#Nw1afo_)oUGw6G*sGLUTvID7@KSV)rXTPC7JJ(WHuko{(mX3MuOt0w~ z2p`7FW3&BXBeK<1?E4pZUP0bxg7hT~feI+p%oM93P6%^WJNb*LMcthf14ucERKuPC zUcsjKNjxxn9Y+Pp=(Glo5=W}$G z_*#{!ecrldbB#7#OA<6)uK!vojI&c*&9;Y9(Hd#^pf31*#rHLKXq8OeUd%(Je%czO z0MDU5l7jUaSn=d&6Wlv0n29nru!YaITPQlchH6N`Focn6+3-k1pHkSX^xI3tD%nvI z0DKd~5T}@?b#iYVwe<7Oou(ef8Qzbya3ddhG2nFbo=)D0_KP&vP&DrnLsxv*;4^87 zowl5&7g~w(eSegG7!xI08D&8?O$%sOr3?+dixll#RGqN(fcfJf`(<0yO;_2NA}|hX z@4C77*tN;RD0d1NJW3_^R75XTAzrOxA)V8r@WeM6k_C@5F1opxC_9+fQ4LRX2PeFBA{2uq*`u|)^=%i z6*}561pQVe#QO{`(OsUXgLVLAe^K$O<(Vx9;q-k!jZ#Lw$Ep$<8nb+dH8}ksdieSx z25*CTSRq&uOnK9k)mHm#qBiFiPfYhX$;MyEAFKC{L(V47dnyL0w^GoA4fTK|zrUe=fDsVvi7cr#W6v{~S3DnEa>Wp#bSdP{i0op!Kpy zp8@XJh&8CI((fV!_#*(J2v9~o;BX1pxaryBP_oqiZr|c_KiL9g9i1c|_ffK#)v8vF4KT&aG1m0M!S++2-VU%uXQYJ?gQchF@rm3b4A)dN^>eUWu zgAYAY7ynN-81qt1F7!1V@l50zK@4CH_im}*7~b-53e zvc$uPU0Id{K!ln~_trMnJG2NYo}hf{ZELoa}*i=-3weG)aT8l~aX({CHWSRtRaNTM=Y++ykHA z@r%eaf!1&YS>d4<*GeIA@^aScVj-&{F#I|4nveh88G@qIe&xno(P{Lq_E$wi{C5FYtym0aw z{b2OTTyX;HD(WX2bu<;&XO$Se?ys0OjNaAa~jkbwSDo zP7ofJMMAGPu1b4B5mc-OXJAF}r=x{W*eIxSlh|JG)Ztb)!rcZeu-r{BL-;?$c!c+9 zD1RQ-iTK=I%aA4lz%j#&nph)RH$lcF4{dQO!WOZ2b*Db`vvbhv&1sBx*NDR& zZhHe(UV5$%e8E9SXV68#+mLqiC)>;_$*>{psR0{Gr3WO;JVR9h)>zcPQx8CiGijO~ z6hn^ak4ut_Wc}X5+1kvs@1Qew8xgo1Y?({(`3=oBe4>A`<=A%P%#OcQYkkUk!Dqnz z#S?Np1fN5xq+Et!OBIdv;D8|vs>o{qT7E^floNZUM3!6s+@guHTBY%j>UvC}V!t4_ z(`!oyl>NBN_IFw_H^CYZv;hy=AQ)LOK!8|bqUk5V+|8{v&ae*O^)53O&EenfN)^8w z+-D%oNL?jfiZY#|(zP->uP}e7N9F9HQhPg9rrheu)d3pfUv?CCY;IT>j7%w*YugS) zCcJ%Hv^fDx+#e`*N<3(1!KzhZPw!4MT{<_nOsL!Qaxa} zJJ2f?l@9G|5m^{mn*Pp7Pvc^%4%A{K$a5r*oypfi8BNpx#fu>UZQQXLP;cfixaiZj zPhNI%M33^J{$`Re6?hXnua7aUBdPE13W+x|{D~K$%vK!57j5ktX@~K8@YTD{)7{Nt z*`_beJ@j+(gS`typ%_}ItV#gx0&HA8;@Mq;Qr(H#E57^2o zxWLoijQXnHTv*f@6F8IPMpwnjw9>8mK7Nq6-Fj>a6;urojKIjh9mCoW`pEZP%FhM{ z$EpTGqg0N@14b8p5$%~@n_);B%XN8U#E9)GT>)rqg1l+DRPzsjZEc)5OXyzouenr* z%%p7QwM;4}LnuHfx7|;C`?^6PGB&^K#JKQUKU9XYiIH#lY)?Ad+D∿}`()%5$E2 z@SVuP>T&QS)utP$F^XwJtE2s(Qz%|)SkJKKs&LQp>^!8M5k_E2&=Y%HF3f1)Q$vXi zf`Q}~=t7jL=c+lbc{<9z1ZCRqAe3E+is1;gY;YVvN9V%La|DEh%7kQjI8k#5ujzN` zoT$U3LOVFt(9@cXcj0%un1I)T2q;)CI1ht^LN}pJcGIvBUtDb3=2#ong#flHl_x~r z24|#mYDY9(1_Yia95*aNWnMvON6u%Po_{`siisNDztqx&5x4rPu@oRtQjL zFKKWx=ECE?xUP!8kB0Sp`km$R1189h(h#YyD!#IUqTLwE*Ji`I+U_ABB81c6VlUNF zl9O7F5aIO_lkVGct@i2THrqUEzJ@Frj=!<)2U z%O3?D(}UBu`T^XKrD#eea-gy;KLUtDDisYFId*7~(A>;mT0SE7kMNK}cWCOJgk7u5 zo?AV0{;v-kg7nTPVX2Y0eH{` z;~^Tq0lazW6}1A)%M!BVMeydeN|C{WpiTRGgc`44%9mBTJy!l(pYymNCAH`HHWH^B z6!&@zqvLcRUmb8P>g4MzL7PgaqYttf-{?cuwA_AlJP4!?x`6u364ZPaT<%Ler@{5u zlGG_&Ki&z7Qla@j^`lroj}{gf>1UtM9pKc*2ZY{vB#r61J7(EA{Dp&PT$>EsN1B_B z)K1d`_kNHG$3u}|^+3QqC|=_{YEYbU2F+VDodLA1LV0H#TVl}VOBFfipTz$Uy z9RB^VZ?E%J8gFttkKUZ%ryPST4bx1dGCtyHb~@x;t9rF2f4Q;W!Aw9n#>q<;@fjRs zKg@lA^_riv?x%RIVP0#Ow|YQ$!K8_hP5IzK0A&h-s3zX3S<~e!xGXm|q-t*-yKYU0 z??dz>YDZS(h7nmG6uQ;oC_{o4WjJB^aAI!&Lw2NrPVPE1Z|j!j z>SFr_t&V!dzG7S^>+SUX#DL6LZW~U7&o}`E(QuNK<>kKfr7f#`{j)eK%i=mex z5994kLglEmZ`EG9Mg2mX1Y>z41ZKm!8Z1!J7W}$I1Og9r!>hQSyg|DG+huX{Xcv)z z0P(}E$Eo8@3WgULQoVB=&A^Wmy9{K=)#Z&1lj^%W=lrWxs_!?xEh5>D?VrOnOg8=~ zOMvqXIxIO7AaD#`HXH(lJzkT2^!?3bXX;lO)ify)V~1-gX*M-uDQ|4U{0Ed4j~9OA zzXt{aT=~11AYeM?WjL3;nioA*q{QHNKj4iyGI&ArqH!$pifb$q26xuFzKw!N!`DBT)#Z|H=XN$;pMzmKYH&}CW}-g_LFTQ z*5W0%7+b9{Gp&dppiz{U{A7Fmwdr@s`rjoV(+w$F-GETA=t2h#5auTqaWhI*9-B}{ z_71z9^=M1kKr}ezy2_9&U!t~V@RRKtM?1_#UR6d zCG%-C7elzW2$%@7#F>{FH>!Nv%$qX(#<6gs5*QV6Sca|BZASQfZ0AgFQi1L!iu0_0nrQs*pA|<>65zN zi^e8k48ZFxSoz^#MF2ZJtCTZA$_n7*1`HgqJ{I!El_LSBGNZ-Ozee{gi30cXHocZ9 ze1iYJ9KYP?0aS`va7H1;vHF4c z5f7`h!g=eaU;1{g)$qeHWvqTG$>X`h!`z0UYtKlqLX$#!s9hQq+~nIUZ86xa-0m*o zXY$eCHv8F*Du0iAJ<`W?&b`tkR0^uBJ_)vape%%yDfqM*y~}PA9JEwZp@3Ces4?^i zn9bRxls~1peLIL^sD#lnyQj6ZI>i|tVYJb%X@g%s*6uzCwb1=$J>IUMnQ^3)feUr7 z@zp#OOYcZ={*YJm^4?p|_P61|mB$-og6(ploNfqC=`Ps+QbwUxhqku!dDS(inyD}! zW0@~N%gWl+D6c@K>Ca5&73&rvYAVc&z)s}#Fte?DtFyq^U5A#x;u0@KQ|Zb~=se3n zuFUN2hFVk5i7erR5G6O~hJc%;Tqvqk+ zcYKQlyjnDm?iU6G<9%Uw=N24i?7Q?}y?(zaLoOgHC=*iK3KK|Q>ow<*YQzH8tb0T` zY8=y=_YPA~_-S6_04wN#oLH~4PDvN5@IwO8q=*s?Z)3U0WD^I|MH`!^T3w^F^D8Rb z-|)U-rtdRjUAQ}Wy*n>=#9B{RvBXibKrUvYLq}8Wc5|*+#TVVj&nB+E0o1Q1Z;s&* z#hrnK8DsJ5nl(7nbL8M`JiENfrO{^)8(3b(!k#q*)_}pl0{$q?y&ApXB4kL21h;IyG0KE~eTFv?+UXmY1K%B{c4g>P+wsa57Y04x#v zM$X(~Q6U8_qTgGUPF4D)o7_`$xw&yXG@xl_zN>n(-uM^X$)I^Eo*jylJ=YfMt7vSe z-U_z0DmAkycrZ~>fr=2AG-<`i!&BbGSs)#r6@s4?OhPVT!D=uc1YsnIAP3K(#Z^Fq z15?cgVcGAp16#|PI<&Kygsr@h?3G$B?>1sG(-0`hIT)Gn`j0@U_eoeci$50GOd}>X zMZQ+b^1EeB+i7mC>Uy=&VyRS=f4=gysi%Nh*uz?!&6eUHzR?HTZIh}ZIQX(%-aer_ z-C^=;aZ{xYRyPE%^9_QaoYNW-d3E`i=sNIKQAxv`X9!=16K#8jH<>zkh!oFD3x{E9 zu=Ac+)aZdRU;=Yx1E4e;t}f4xuH>eDH;j8E)MnYeBh|Ca-7Zn>P2_D%m3vUV6M{Zx z?;~d1+xfJ#_iR4CTfI={H*%hW_#ct?9lemqUEBE%-yoCX#)x5ZEc9BC&sWtzsA`zC z&xS&cn`XS0iM&a6c5y?-U!QZe)U(W_<%7-h%$%J@4%_tMe=q`Qha~5-iImL5b}kAJ zKc%VjS>(%`D?AFMJmbt5>c`N_L2U`ipai9=)iK-KosRDs-;s?oXTI{ShvW#GIUz=i)~dx!@%-rGj1aqf-o0$FBPwY8v5-{<2_$~M1s?G4RM6jVU* z>ftVW6~ICF-9IQOqIpz;rn2pEEX7o%^Dq3UwOqcS*&{_B0xGOW=t%QfK&4kpx@Lal*?bRyscMP#+gQV;o8!eDP-`oU zbd{%|H4+jS0iEQ_Tkj~#c9gwn;P9*x^r#FRX$RGX2nu+d(blNCTB2jR=xe4?y$iZ; z$lHD{)bgyaXIH1FZa=A73qICstU@^pIhLWp}0oBbJILfu0z$ zM}q*9W7x86QhV^qa-so0?Q=>~+@y+OqsHys&|z7##tCT)xYbWK=mdEbIT7&7qH}8U zMsc!tzO|0gqPF z__W=dyOCW}s}Ah-|>a47R_Fp(5Z8kU|ct zqsG_d9C{gT!|GxBt1qmtgB-tEyhfH45~g2QOvQ`~`c(X6b3lSkkvdhDyA!yu9U>|( zuXTvAb>IesS<8AYDbNmkT%Njcv;*h)R_{?*53i_qv^D;VCx|0Q~0hTps zgi0j_$nCYY02HSO=8IVSYFTFif=$*^wJzv=Jm8jXvfKuPS&_4`Os#e$0Hk80X(T+w zp%hk%yNl0lL<<4|zlOAtLP{|k%EhP-IPd|R{#Ohk6_GDf5za4s1IihNdLpIWq`Qr` zo1|WPLbPN)G$I=+ttaB0!y_RNdth-t@-Xt$6Z{mR>&q&)WfNhiM0f~Z>RhYoqq_#vUmqm7N+MyWH1CBl^}<{;TJ5GlUx9n;RP|yPE%Ym3{Eqz&;v-mycc{7 z6v<33Dk!m2XtEvfL&4ilX@D+lE?p%zIxb!ZYUfDAv_`s%&4fmzwkxrtb23yRH8nBo zXroy2_FT+bdfh~c>G`T^qjX(Jx~_W8#6~y99da#G5y6*URpUK+V=8Jd;G&kzJHMqS zGxNfU%GzC-wcX&6z4fULrjtvOp=;(0sW*c6X_OMxW!CZ_aym+jHU#Y$V@Y~B;JX3$ z*$o4(aSf&0Te>;flp&7qHqGOW!{&Bjlc5p@Q*t{hP4d#6?^1kUIGf_DUHr#GTjgN~ zIZ)@3_>*+a?OIMC+e7Gw_u`)PBa7uX2g1?)bI8i=njiYrA*BwlY~K{xb#s=D!}8J) z`D-beov7IN2JCc~c+;m*764n1dyB=%_39-pqqbzDg(YUk)IqyeprX~sl>N%cS^E62 z@y4MYX`vaJ;xN3urB}U~FP8!ar<+nWUuZX(j+)5!e^twIe0vHWmfMv62VN`}4nM@y z5-$CPSd>0j0mFlp^HQurJlUa$0`<%WtmX=Z_O{beqvS;xt(X$eDx{0w?qEc&yz5X{ z{WTZHv?15%X_Q%b(65;9o}XbD?O9%O81QnX-&#c3nvj;}{(c}bf=dkcH0xF-0=EBgm{8Xs%V z>GGfJD{eRlv^uz6tLs#mJ3Az3@1}Y^zN!?wKVFTj_CcO}n$Ic!W4QsneH{4m7R4N} zQBDuzi6m*gX7mwpzM!3)mndei13>Di+^$s=bi73s#l%It^ z#?Rgzy-%Kdzk#7}rdSh1My_X^zYa4an5wL!&NY(BzkZgDciE#1v^k|O>4LFW$nY#5 zR9?z>7*e%jrgh_>-uf_*H=+`f;qYp@3lKIPuXt|%DBElkem@!q;$dCb5Zhr!l(*uU zx-|b-3fE&#>jY54&Bwsl#=|RAp)%6-ATqqx!Ludy@ERr!<9F9=@M8IA*>gJDH(Q2; zH4El42H}C3>eb;veTUY`Dflxw2&)u=hRC3gLY{T}k1K2M(sXb`!e!yOd$qF4H~L-b zr?(5~hW%R0{LxpVhh2VQhSPcBWT~-zLovXF#3hDlO!YIhh#5EMiu65kw;8Lc4LDPs zR_oy{{bAg~*DHEB;Hp){t6tDSI0RuBERRU{h7`VB^cpF=J=y-?q@oZANzVP;er4Xq zsm-D-g|jpmL8Y9LSymJ)a1uTgBWMiKj`zmyw%!25JoUTVC95t;V#U1yB0yTiFHAUH z%^_xd*_Qq}DOw1nU{Akao^x^jmfPXnddaH=!2edr%6@i!?(Xo7MMuZEc}J4o!}rO? zDhmT6nP1DdZe1L;T*#{B#B3S8$(lr@J0L2);YEYHuLr!~M0X@Myv9(rJrh4=!BR_L z1z$!|Q^&|yaFe{YVc6`5)%f+m3?cwudo0hk^gFB3K|L;F%rlIefY}U#)Rk*^zTI(^ z_hOVDt)`v|xD=h=<9)NhFzj(?V!hB`Z=%dr8_gSF2^N9({c>&uUP!(O(=RlDH2#np zaJDM+3c`yu;C10~pCxeG&~(v0=r|k)F>8g#l{)SDV+x8j&IhAT;zJBE+u6Wi7-FG4K578Q}wxA-kO8@H?>Si9h$5f0;i?A6B_ z+^UWw!S;)$+!sv#B5~iY@8!0ZyG_l|I0e)G`0F25g2><5*v`oR4UFjMK!buXW%C#u z0FhWMHqZMte_g8i)CW|X=ZsA*q@=kEVBAG20cq~QRiN0t*JyKHp3Y@P7;w@CkBn{z-{=~CY9yx z8;)Dnvz8Apk}yOxbiBQ2a{XTJzgqt47ZES}^TNGnze_$Tx=zC+)##a*$o(n8pkQb`J|t0 z3~e&;hw>9%hB*^hlUrJefZ8z-kKVh>2BPIMELjNwynrxa6$-3l8=E0Tg;spRy#xXV zM16%n*#<(`{<*S$*6g4A?4M`zpIzgho%X--bPASKnEvkP+($Ma551uTJ6N?H`>I#1 zyDr{!rn#6fx4h`IsNbF~-7e4b4%}E(bZ=B3a4lcp)EPG0RF;V)a3Ie7jekvqIlSbx zd&#{6M>LQ7@sjjXrRa;!TYf%iIo4Oc#wMH;Mi*U*N>YMh5eS5#1wLM^! z{$w*sOg7y83@AoxlTR`G*M4U|IQW0wCH^RTdHOig8aQNc^%Ng|Be^igtxlbEyao>9 z5~}6#4@xb|TaJyjG=ctNMVCEc(x!j1sbkpC0X~M`)NNp%NzAp3g4M3AA1dpBKHNyrRSw5^|~!wafZF zXZZj`TV@q)d!7DC2I@z)muE#j?pbr_Wg){~o*v3PPX5UjrNk!bgkG>brg*XKt+J@- zWv<`Zb3upvEQSM{-rJvSZ;KDRKb~6aKLNVjPN65f%7$NMPiT#7;ekKd0u6WL|M^G% ztkFOB(SLqKE(zM+KqyD7oGOS|t^M&`ovrbI*dFSnhp~)UJQBo~0JJ5y@26%g}0<-hMFC;q(?{12MW{~Zd( zf8n1i^!{X%k_9@@ZTZ*e%^THL>{V9msx|0;xpxr{Bszif*4^dh`J(@$cKh^mx(hRW z1AU;z0GP3sQv#>RAK67$i9ddEM9-i}=%dK`m%KkGfff6uX>H<>&G^E<-stw%DF!P4 zD9^Fy6ZtLv`e}2$JyU@{|KfT(a9ZYxqH^{sJ5U46>RxqxV&F7)GMe{-C8}3=-m4`e zeQoH9=Z(9G>TJ8)Xc6>Bwcmk`_XL1T0DuMU&LFX#=CQMQchP@CFvp-i(0A^ShBS2K ze_%8-tZM)!^LHd%?D6v7Fb=Dqo_KOvotFt(`VXMzz%b45i|9-A3hVIS2#jGp(-lB| zhED^n{2}pgl;$LJmG|hZ7l>il@gFG8KbP^(YW#C=u>JF-{IhTT|9lt4hvY3`rbes0 zn;*{Ibuzx+lJ}_Wa3~=7;k;7Tru_F~4ylZFSe>bQv`^Kj*r0%J1DoSZPCEKu0~Et! zP9M-w#unn#mcXeGzQubYw1AVu0OG#L^q13kfOX0HjfEcFD?MTnYd`Rc(NuQWX#rHv zG^PdZ@O|eEGGMO?Vy`}Cq#8FiPhQEjv`~X*noNLpv>pi^*n0aCh6D?0Jwk)teKLWK z)rB6VEyplQe;cHUP6cFBuj$;s|E21@ausbaJ}ikeBxs_C*&M9aL6G{8ZWZL%HH#Z& z+Wo6<3=CITS4V=CLI3*pLDRj!gY(^oYX}0>T~F`f+vh8EK7if}FGa$72Gz^HOuTL_ z%WDUzZa3|Ho{9gvTw?!R%M>UDtQ|Aok3JQXc}A=3u~`yk4AUE^o=;g9nAfM3#VzMc ze4MXLDXmSln_l<%%V{Q633RKe-7!D^^*vxfxaEDT}Rh+2=^rmMF!he=ReMbOe{E>+Kn%58d{HX?N~cM=gC)i3&F zAP6ATl$ShTw~IexV^xKNT9YW3Q+p!m7@duqkbzdvH6}0*ipkt`;U>C{iZO39J}Gx9 zF#{OF^n9n2!gg~UBVr5krk#=Nze?rlUpX?r zabqYqL!OnbzLFANEO-y47n6JN2-))kDy-ApZrkLmL^Az#>;8pGv4+r1BxxkA76MB> zc$}-w+|BZhOx5w70AdbMkrzqf zXL(N-{Hc0<7&X^v%X(v~OfYHE9#!d4H29^1bN{kS7HTlW)dDaV8+P+@lW*-R`U5qe za%}r4%)Bd zz7Amr$mRyBmx_8bYicrd_ZKdvu`Z&HvA!#NAMU$9Ulrn^Fp5v?OA?Qo zAd{*{Z)je>yL<&aRDxSt2a(%=hcr2SE8Bo)>b+b$Y}DT3$rujWZcywp7vCSb_>;}C zl4_$I&uOQq;WfG`qB~OMekQ4J!a7sm{v$3i4(pTo%rf_127Kr)#4GyFZhtKgqjY5| z^(Bh5LVTae7ZugeL{sUvJRS8XtvDqF#Qi?~F(toPj3LwgSdGMzOW2HzjLcU%TYGTN zd#zAP)zLI^^F9ltM`X`&r_B~dLB&BuD|_Hn@vPN1QO063b1CZeA$5!ah+UO@F$V;N*F`EiIFBxO3|zDiy)dr%S=;>rbi3 zsT!tysX=wMObMk!4bs#;9+GhFPM$L^JWOJ3A9HlS`XpNr_r3Dow@ltI9?zAft8XRI zo-@&$CfKNDWqeGJ^?|nTz6U4LHH9F{+0@F1^%2-;GI>QXp1@!voUU8)2kJDd#0yEq ztIqnv`>t2U4}60>XDXq+Gf86yOMpZfq&g^HzTxm#^e^6;x41IHgES zb|{$JLocNMZNV!$x%(oX*wu%2_Cpdd0r~dI>pgwiQG){Y{D?~E#gc(&Zxg+baqStB z^ng=E_gkGgYwlW}iXFAlS+gVV6cEKz6DM3@jim)^sC#rBy)D;`wh5$Z65K)FJSQk| zG@^g!kUoEqz~};0Pe*j+r?gtC%l0wNlA60C`q=d)oqGP!X>JG*JXh7=%3W?rFXVabe0o$HsPZm;C9P75L@yT>f3Y?@5}ZTBJ}{0V zZfcY&mWpc|b?8Bbg>AmqpD%Fo9>i&n)mluJZCaOC^yhxAFS_g*bu)P->i%8K{TCqK z_4a_LU6gOOnWb``c4DEvA8X_LT;=9!1l_I{LJ$?Y-+dJgu3W2@g?N5fa7ivE{`=he zznb^|^a9||KiM+%Rakv!F7(Dx_Y>U+wpBe~4EvjTz%ef)0BBJksh|2e_4h1T!4CLU z$ADvQ2GAi|9(k}uD*Yc}2mk5g|4Rnw|En=(=iQxtvi;Q`i{8HR7*KTd5&sR2oML{9 zUS0<6Y820%et*>e9-ZO*8(?Lk<`Mo9*g%Z;l?-^#p!)R9zUO~=;4hUhHSa{EtK=#9 zPi8Uu4=&#$>sL)344l-jjGl#c)9wc(te)rVa`krK9PhIfeNiEZ&sz(6S+9yj(f2yGaz_OE%RROxN4=D4Z4sI;)* z6^bF3WSmoiX|ST{yNRj{D7fb{=B=McxI}1NcfSVdt)8aub4c74vIl{Onz`1iK|(w% z##0IVwhqu>f=)4+Xi@_n3TzjTqYN@_6<3q$MkO+GEK-m*Dl&tVaM9!z;iliT4~iUhe&gM(z924J)9mH8_7#j&?sVbcPFZ^929H?JCch>TLTD`V;w z?!I`q|A2gvvsKefFIwti)+n$1Ewx^^FMa7^*g(}a z%aH1SQp@KD%=n3QKQF3 zzRTj#?2b_a^d{^y>9N%NDZ|-oizXQ%Ga-IHdxJ_E-@Sx}s~jzl5pD6qEP_F)l#_HD z>c^vP~G7b_iG7jZ0 zI9bSyRZ|qcXiHT*PI9?GRdv#P8_~hB)>x-*P*eY`$DvTyF7H{q7-m(&8B?k=Rh!gf zgY~yJN{h0j;F)~9B*{}J&kenKXZLnjl^bfdh>gD%iM{~=Y!`~KBd|AfG2rjvc8e(C zZqM7fJrPeQL#j2`mv%yLuTF-%4iz`2_yzd=*=)0>UohRXVB=o&a4L|yCP7lq(78qv z1qv881!eZ$b+3nGyl##a-XoXFa*|7I4{MP5dBw?D42zh-AW5YG@cl`8Kr$^=5tor8 z0sdiQn+Ri#c3A^K*B&+^-a~oSh39M&-x91FJ;axJ3|DfZFVh}X(p7q#uQI+-PJ-*V=HQAGKw)_QdYGV7olnOAGwpS#@D+t9x?ck&eRh%X7 zIcJ0d60<4(NmAT);G4+0UD{`^6oRBc|HCS;l|4O8WMVq^AD6B%YVK&t1XVic7eb!D zaFceZkqKUlk5KpS#ypNR1w;XTS2VtX^Og@+L9m{V3s= zKn$}o1AzC8NpbJan7o&oa zK|#JB0+-LAR|<$1IxlP({Kk;^$<`#8jqE7Xds=mEHmBQxE$U$C_V^ic!>FfBc~ENnj(;Ml#2qGkjmM zk64F%Gx*&X)>}mwL+H0po+D=!M%2e$oscd=sQwGZ1tRU&i*T#(Aqd?f1vcJE1X5Gb z2eq1w`;Fl7>KOZ$rh`fzXeiz$51$AGDEIA!6>C_|9y|sKEMWi-S2DPY;&(jB+X)5YtLI_#Vj!~Z7PX*LmtjuZ+@hGImOm~w ze%IbkB#vuj^4T-ee8T41vn{!W9ICq)X1pI&qXo;;2;qDVM|R5>>rnm0Zp6tH?Wp;r zJ^MACB3DbbTJ6E;KCD<~VTNBRe-G~7iF<rV(_(*b)3R$;vTply$EIZ0y)0VL4b_cD_Wq z@5Qxom{9(m38|R%P3=FLW(b5o0c%CGx6~31n|w2K-sBiO3?WhWBX_c1#DV^L)iO_G zyTpqyng##V=YB06{m$|*RH(l8)*P~A$A($$_K=YWZ2UpG4#dbDQ9ujX!W1Bxos=4` zyew{P`ETsKc`)1k|M%PP)vF7tT6?WmtG3v>P)m`k?S&>Ph=hu*ttyG4mQ+gcZELCh zDp750RU$%3k=U2oFVr5?-b7J*(h@0=>wK>BoBP~n=KRiY?wK>^o^xjIf6QPeh~)i# zEzjrUxo*!)jj6d3DRq#CS1n%UdMKRd{9aeVd&=?0-{k8o5uxyE7-cXXljyy4}-HC2Jj z4$t5YF4nj1-M$5r5y8T-6vza6dUaGPIhIPCYY_fV~S9jD(D4-$}0u4y;eGo%L(5L|s{Yi-4skVFVikGYOFk znU@-$Acz5P+)CZo5M`cSjes2U$nk?Te)zs6-~Qkg%_C`E>>ZEFZa}cD;`~DzfU1P|SZONhc1`70^M=GNL1@FzN!34hTm+j%sykuIbs7}$4f!p;_Xx66|>*;Vv`@EEJ zhQ{^aBrV6nx_gToo;O6TrJPP%XJ+Mz!lb-~ki(ixj?AuRa6-$z0Ti)@I$0)+C1k%V z5JGm00Y@~WF)LOLP?7&xpan&g4rM!VhpfmL;E>h(XwWS2?D^1+X zneo$fO1AZOjM38@+J|_CB?NwbWb^v(L!w}xCN=P6FmrH2bRU8%_>$LJyR-ER zFTA7q>NssM$z7VG+!`f@sqbtIP;H4b*PC%&X_n($i8;Ti0{HgOoy~=3EJ_CnFZV7& zlZ9U8qfc2_VE3-?+3Sfo#vOmImU=9dk5SPZlsG=`tiEgZ2hlMO`s5nTRu_QDV zdcu;!XbYMUy~pkYr?G%`I|ezB75eM?$R%0IpaCyJ5j9ri>rRj$(nInuBnc zHd*1;0n&IAj{oxo+mp7TepMS+Uaq+-v@mE(_0>NlHl$h0194$LUs}~3t#imWc3sc_ zs20X*gaZDsf3Nj`Nys>Gu}5bUyv`U|ze4NJs7^tAWxkH>t%ZagxpwV8mfZid8`b~n z=lwrzVuvH=T8{H&*oF(cG4BOpE9NG2j&4;8OLV=SID^^q+lmjsTd0wXQjcDqHW>Ty zd3Jinob({?VZxMEMU6wt08;RS9U33(uKVRv7gd546!!XrP9^H%TV@Q^>R;BseBi|q zNd~ZlQrM&0VQJi;;tb~|W3{NP+^g(d+g63}jA5pN+7sJLT^`^vQz#{=-RV=wl9)uS zpnIus)hPtj79e4%`(6_=TvYy(;E#U{Tm!Lq1dv}WKFe1Lh^vP^KgbV2gHy3N;x-P5 zI>vjN{A&aG)GBlu>M66|nEG&IlWy#T&-5TCP$q+u6|}+ z-SLdMjN}JiSO{oAvp3qT=-lN1MK^e<2@cnmbHvzsDhLYIm`I8I)6|}IOlXX0*v`8p zHm*5B$-Bq4l}ezzDEN1&?P+_5c4$WB`emwRjw_-B)<8bKuZJnn?DYV&Nv+z=Vt9i@ z=GQ0%S{b;@4*y4^$7!C7TjO|<5B}nl2Sj@A+ZrWFTFUI%#>#!O<~M$Uk-IYDoh2@q z$8mk1rCU_@u5-*Iv{zkKUX*1Fo-YJypp&npg`HzSR58crv7X2QI0*^EE(?P$M1T0v zKIS?zx3w{Ev3ZL57n_fp7l|9tQsprq5yWf)M|3H-GSon zKrv#W@1f1z9d_zlvc%T@5nvcM*>)*O4>#Um<_Te@1Q}#rR#)D%Ra=VI5KveB85^aaQUt`A|`!>2X zom=RHP-7rmX%58{izbr&C9ycF_nEQh{}?%DR&^j%OEp2tz0*gAPH^h@((zsZ@u zWQhi}+s)ljt0H|P9&{D>zFW;42X*;|>%f<9GMJUQ7DA@43_;Pai7)m!6r=Fr0!MvN z4~8>?|*g8WIH2cwun^dKk)t8^?#eXMfrfE)A*k0UlghFn+8DAZuO&iOxV#( zCRGaaN=`BqWij63?Z*Og&-WAuqw$6mk8joKn6Sw7^0mE4as>KoAKDu<>++-|3mHMJ z+RZq+0C%x~nfLdePG5A$mcqaSpA+5|)n$Iq0AttdHgoQ0pQCCT^*SVBjJm!j8$)w$ zb_Ll#fAp%nKW8hYF@PBr+bqhl>JXD#jem!`(Mpsiu{^}r#9iFo)!w}O#;tT;H7xTb zPtVS~>NL?VLGRM&wYBN~pWlUoXk?>Si=JsB@OKV34A5`F$%+QMC9sR%GB&5&w^4~1 z8zYyqfX3k5vihh`XG`zy&3^Vo$JSw7d#92X+au24^NyZ4PsN5ENo6Hp)ZUSmw;pYb zAkSl(#p}YeN^k1uT2k*8&CQ&D94aF|^+MP~m6j@JUPUr8DY36Ah%b$%l2P1jC#=4|}l_TOYY zrXDQvh6p?TOLN(mEir(vY5?#wm6d1aTzJ;)d*IkctcGtcH;pJYiNRm1j|ek&iWrU*cXir-Hy$Z;1Ma$oL2JPF zm)ADG7y02I`O$x7^ZGAR$N*XMONp))t|l(^??bnCGunw!f7M(B_%#Y1f&#|X@6?}i zUM*bZ=YYH^!JD(ec35z%0V?wGgDmjfsOj)Ffg%HcjTr#&_7Bw^yf(FE&Pul}{WS2H zqB4oN^v;&bqoiUN7p?rfE{~~ZZJuXn6zKiqsr%AFw!c{;oVGb)r;U+49|k_Z--_ee zN~L=sPprm52wt^``{l!B|L$aROzVRObusYFOOJjOgJ2gwm0A%~>^rjUV?XT~rn7=) z<`qSP=7KV`A1jsVGt}1?Qt;J{sWk9|mdPJoQI(k+&mu)}j!O$j-Yj#TckW~p%O=#l z#{4mf)iQOpTbV{pZVX(om3UL#qdLZ|;W)T4bajd{`!n)^FzO5OHy%BJ+qDIHouGrg zGQU(dwrC-;fSlFtKXI5&Gyv<{pJ@A@}utRP2T+GK?Cz8l$l4mqnpnH z>`CCiDqfC1yZ{8WP6|#Qh^ zE++(Gb@YQN1Lb0dBsYjt`H$nCUI~PpTUX8*EIZ%J_=B=96Ng|rh&o=dBgNUdPi2~G^(39$6Ax`ug#rh z-!gRk6I0=~=6?RoKvl^%kU9g~6C7n)|9p|pZ|c8$5thTCEPb95h3~qV=;?iIxz8|{ zg(gyT9n*3Q;bE2zOBRt?=55SZU~i9LXl6+wU!u7b60VFC^7O7eN2dx0D156#Xd zJHB6(>Icy={XIXzTV(O4-9>=rGt}q zP5Evdk9bpc=HLlF3>DrI@SugsI11b)0~pBB~@V+;ZsM#P&h(t+DNJ zH^`13SRk&bf*kdyDPoqw`weEoUGD`WNlHK)1sx6HDW853ylVqhEJbEvwHN z_%he7SxN!#E{)P`Dlzx+nb_i$fhL*Q3@D>K|XMz&zOxN^$T9AuAiIZZ1?rN3r{@O@bnKE9Usma<}b$9zX{JuR^UqX17*XEvMsdD zbsSPaF`zNc;f@mca$$vJfvRwC_Ld2z@P58tm}#X`rh%)p*l7>Z1Zs4U`PmJmbiWuL?u@KTRqwuPw=Yl9&FkptHa#9X5!&a zmQ8>|4I|`gxb5jRd*Ax9@~om{^M9SV?AERT%9L>3sC0N3bY@bp^`a5RN>Tr-GFSOi z$(@B~JtcQRObk;47ORVWWaNB@y`we3tz6DV_rm>870o(r=+g^+c8dV@68~(+lX*!m zM8%I!msfAE8ui|uL0mKfg#@3tANaZCHXc0oC0}SP`c4J}x%jRruPIbBLWhJlaf~55 zf7?^bx?8Yfg4go9OImsScrP-~%Retj#9Henv${Ga=yB%>pXcg_?r-w6Emck@QXM1` zsV-9SW$GrMlPsLasg40p7^QGH1#W8%0HGq>b9Yx{UDad`#}xosE5&!%uW_J|3!hQ? zY+jiD_z=mG;0eo?t_Dzh<-BS)sX(t#U(iwj`o;|=6>MA|pXBZmZ% zg6Xkxl@B8Y9OZa{l#BAoZ=)DP9mPVKy@eGgHrjFYR@(f`$MNw}?G6>wbN@t7X_aGs zo;;T!>lTclSR3_JAXSbu1m;6Y*5~K7RqwC&bg1uOq_lr(EY+gNGH+TId1T%FvN#E) zV-Q;rPGCoAgMsz|}g6hT@pmc*QH5l4q5Ouk=|cKt9M<`>P6*`#?u zDk6#V8gFhBM~)|5FR`Iqe+TuKH>wZL(2ewj4rs|dhxOY36&Q$A6+wdvPh-c}MYA4@jG5Cun72?Q^DSqt{bm97%ywe0~E1EBecvs05Pn@NLbofqS)^ z@Z(&$kRMN01cRs|y~A=RWkI8fX%ygN$8~*gW7gU0h_MG5H~*Y&AMNTTG2KP97ymx= zIzgXZ+ne#0^CvvvQF?~*T|<4(nLlwZU^0%mna3PV0!vDC{G&>EMnId3 z)ibDk{G2AkR49}b$!HM|p>fn>w+~Qr+|l1Txn#7)7(D{E+J!cV@MDeZZhWpRfE}rS z4U<|LF{oU8QL?gMnIvtUZE7i(v_30`MT0)r+td!Y0{7yrF-jLzs+K+or9%-vC+gG> z$m#XSk}eJsbnSaMcbC%XP??gplb8JWp;OI3?6g4XfM%}D2lJXZmph0~(S-+p`kQL8 zQsX1_p94&5o_7%+%ZY4Yg5!zFaUHf&w=u^cqYIwv%_6UsGQp=>0}3%j#Uk27qF=al zN&T&rdrNaCzfOE8l}j+6k{z7+eyy`z6m&7Dz}ax{z5TO1N~>|X_OSB4q~ zb45!+DsMyPlHE1hnZN*RDRu0XXJ%JJHmNY85oC1l{@0A#sFTBvhMzn&t!3cEOuFk; zRFmwe?Hfnj!@)^Q0(>gz3pp;iaD}l%+%5bXRFb=;Rl&&*z-#X= z-;q;P`sF~&wWekGJ>MZ>A>2h&xx=aYN?Mqfv&{026)0M7Pf>7s}Kb&X5Da3XrQnS?T9tqSd zzMJk(A{D;ej#~$I*Xp+W&VZNW@qAAYNKY3J5?}KAq)p!bc!IWG;rYb(=tW zoNWKqnzaG7PCaFfWtIYPEU^My6uk1;_mR1=P3orVcTGpTJ~LVlWTN9<)kEQ|uZ?F- zr<%ot)}H|Rg2Iet{el-Ir1iVPpw$fZGg!yxlsE77B{fG1VtXncyY?D@Z1qSUr`Cez z3MmlF9|bm@h@es}pS&K<=&fOaieOkt ziMPm_mG4B!ujqZrQ%1Sr0&HVPnYq+J&r%PqZ$!m)fl~8@8`z>&`ITHRcO4|@`5^j7 zmsHT6amY$9kuZD0;+C7M0nYSR8E81i90G8<`_E`Fk{F<3*oAVbi*=IOu9OF|Ng-+_ zMxBoBqpfXC(jPcz_#?`vj6deh79qv;@R!?>F$Mg)yc-LtlfxlmQ(`P?3J_Pa%MIX- zmGXM^66ds(Kx|{)OQv3&k{HJ+y3y?&E@Jz$pQ)5D85if=kg#RT{wmEiM2V=i>LhMdUl8opovb0SG}*`70~tmYIQYd{<$5p6Q}yUEXE92rrq_0?f}Q zHe|k}$_a*0Un374BnnjcH#q)r9H)gVi3?`OScV(YQ=(-6yd6-f@pag{@jFdW@UPQH zIJ$53XRU7lK*)t>3vG{qd6QfxX;HSCU3+Wfy#(2Yc_FZG>~{SaJMH}p=XTN!JZC9Z z2~nmwmcG|p!aP1|bK{}5d0k?k4+4O=xOJL7%m;hHnoQi61=g*aUPcA8%H{%Q5Klih~O+Q!Z^_Ie$EI;YnuuRAyWw zmOar%yb?ol<(rT2r^^DM6OG0NP*dZ(=Ftk1-iIaWC(Lo*x= zm^nt{$s`;5d|9MjG=Wf3LN|WPJlqBL2AU(z^upG<1t*IIPBB04%$x57SEMHJL(5#g znX5FhYXHvY{7K$EQ?&4sVXU7Y^O>?J{`czI)Q(6H8<2-xAKUaEg`KS)-FRC*<-P6abG$YMa7U` zxRzjjm=W1N_wqHIEsN9$cUEE2Nh?7X&MMr>N|R^*ZMJ5dD)slF?%d434|#%PTDA}F z{v)jm%)JWe?Nn5K(kqgR(hKvmi36VbC~$W$HhzXZqge&uN~OYvuU59Yu~Xw*U?#S4SP#+p%kDm5vFX0PSkg>`jhD9Wr+%{O0ZFd$JGU0s zW)zk^IAxb?3LZn3IPfUNcU{_BtukLFaxp_TR^~VBu`5ATuXwD|xtyln&9bO61h(KE zW#gizQ=Untf!D@)(p94!6+lt()Tujw1HmgYpK?GR!d71ZZI`}|KCXu^A^m=D+x>Ku z_(9(O-9`hYy7R;~pV;-4K1##9NMKNx2#j*8TGK zEbLj~VE>SnzT8vkjdyjuu6AA}ZIR4$wmP9RJvTRiLvxzB{IpTi@F`yAOMnp0w$uGi_w3y%_bE0p+Y{;^4BSMDXnuC@?k^-6RLUr!Y<1_RR$mlXR;3X2PXrYWnU>~fG1w(s*^c9ozC0^0boUfZt za(36H_Qf$#Vt6n+&+Z@%?5#Bz8!0hi5Fo+x{8+_TV4uj91@0V!?F60h4msVl5WUJ3 z;&_cz6Ajf&dGOK$6F3bx0!({;1!Y`WH3(pP8a-^?Hg|5LQf|J6$P|N6ba z=6_Dq6H#0R_Jp8N&hOBjp(D7KulE7kRg8hyffcso&`!dkC4<4ACvb*>_L}*jgk<7Y~IObH0a z`0ErNEH<((>Wkw!HXT%dZ`^Z&5x{jju>ON4;lsinlEv?=@W zF6#$bkjYs$4{O&29k4g`-&|#}0D*)v{tWM~FS{Gr;Bw&b@f1$EJ&<^!t|Pcyq;EiGP6?$j-bD$l>(Z>e0x1E@e79JG*VJ(_AKf8mw}wc*odaTh7TU*0fkOSoA&L zmZf+4+uVf-`hLI>UYMeb6URF)K#ifVCxMX}Ug-vE|utpuv+tO?7(Fusns4wb_7X0sPS>A^vStn_RZm|p_` zWuy#T$Pj2yMo8yL??S$e;6DVeQSpbr_(2ys4Z~8OH|iC;@|GbLLwK|q_c>{{hczo^ zogff3wBl&FqW;1*>9#34!4pQ>;f7bLIT#s=$1;x_!LpKEK{g1Ah^{lkE?RP_ z3~h5g`&{9cs;q@t-97Us!PoJFM1rD8P)pNs$?n#_619#pQqsMACR*;Muvk~m291=J*j}1` zuciIWWhoPsJP-1BY|idKzqqWHW5a4JSEv4OP&nL@;hRic<$BM0!K(rF{3C2On2Op^D1k4 z0Z{5=RGbN(pvYc(IH_NA{|fcwliE7lOyr>YaYHS&nS0aqmI*-)NJ%0^){;nkoEH70 z?rwxKXgQ~#rb*`~(R9Dvqe^{!`OEB8=MhtNwmMTePJjONHO$i~x zgQJN-Nn|x*kh=FLTI62+2K-|5$W(7&wCDLc2YX28=N(18wn&*_13wS<;YQ>971aC- zoe&qDP$wXaP)oW*%!FiA?6Luh)kk=j%e!rJ+|ndT^hQvN6T3>1?sJ^U;K(qrKq z47Ghc^Y(eGxhS2^u}N-;F*O#=Owps#!E(;s?^gPJS+iD%(T$K~=zhm5)@O^dS&SSq zBR4DDys!rDb{jQ41snWTQ}F@wn9cB`#KKw2m0I%A)d3`^M&4+S1c3(Qo42#l{Tc{l z0{~Q>R+3><65rxe060rC&3=}_ge-+uEJ4GSrqgZnReI|yUlPq6Y}uh7snNNzRTKJt z-t9z&oO1h?2%1s~=(p@AZZb(rU)RE}m+B1dvHhggxWcfZ~>d*9SkY0*-AVEHwife}Sny;m# z`MNZtn=j%|v$r}u;>^#ok=Y-3DjxN`hy~?CI2G2(rzGd7Hd}V@Xcy`b`(S7Od3ok~ zjhx;7`70M^Q0N+B(A>1M%M^|Mf|aIS+J?+<4n~!E$&O{GXI2+gNLF0V_dtSb!^V8t zr$?7uSKL>jmy_)uz56yLk`_&%T)&c9nN|Rg#@LU(%0Y#cB9b*po52R&(hGVik;esX zaxG8Iwejk_OqLJWTlsi5+w%rTDY@Ww5q|48{Y<0 z*sHDy(P}eKiq`L~w0K!Q%NSFz(6${vpVbLN+(n&Ar7gK8_*2c#L3$<#;+Y^P^C31E zSR?{6z<4b943GnB$nO3)r!#)Gp@zYSFu{j;!R>ovi`DrdnG@WRRg`;P0G8`J{rO!( zLya?BKI?S>(P!@F@PrX$v}nb_U-Ij_>~k}B7>Z08se&Y>4~dnD$a1Qve>FGv?hG)0bV}JJu60a46;-1Od?v4u89X27Nw0L#O!xYArnmRClAdqJq)*T?w!>F7E{fNbG5MDfVk|HF&+ zKj=RD-$|tZue#5Uxa|K$+t=#**)IFA%wKo$_@$+*IHI?((4h*%*&r*kZq%Xn7`$_^ zWH5ax=H3H~^mRXyhX=j(k?bG4zGcqZ-+%k<=8CVl%)m#d6CcGRvO1%-Wrm{Ic`g4z zHmkx>1<&@x1QNRdyc8g)aRY#Mc*x@#uw9M(8$h4}gf#PhR?D8qH!#n39)PO=^s~uE zr@s$fi#P_D5B32JDu7wrHw5GY-}K)8Q{6`W2Sn`w$K&A0wlZIR3*hJithV(zU~vAn z8PISnJTFekKo0+Il5l7N9}2iwWdV_hF_U_5Q-A7E>;EX9;=em8 z4*%7x$H(#P+47L#oz#R#ow)XwfN5ON!Jf+rDDrhb2W&(Qyq<0@eqe9K;4cDSIX~gA zQ>GH$Ficu;^7ZtIU1GN$KRL84Wi&cA5SR_90U_TVYG}nj9Tv&9MAIErAF@W#C>ly| zNg}_CcJCU^$CE1XYO;=U@a*G)1 zu=HIrDyc#eVfObSzv~htA>_M-PbJqcm7VlHx}k(NzwJ=W%9k<0g7h9sV!C3y?MouN zXpBW+EMfx`lt#WnNN;|{V-7A00P)zf=YsxRKvmoh&2coZ<)r*61pqQkKmxKiareH$o$r2i7-hIHIdFC=1d`O>$5wY`&@8VKh_1 zTFN~K`D7HBj%XnWg#1s1vnc!a?`mWrMsP7f6Y(gk{MH}M<4U-&VA6S}p~(*Ay4EX4 zGzh8s0*<~8B=W%EIVk%)zajx6@<^peijp>GJ_&!ANsVa{UgU+EkzXY5#^0Inqs=Sf zlz-2O%n2v_fwZCcq(M9=r0V6_qWEjw7@>UCilf@1OmrVFlx)Syhqm&rc8qt6L$M3{|% zEdKI7Ci0WH&QSAjymTn9g5-i$7SYcVJB8Z}IPD`)XD83aGiOnd--a8n*Lx(;%;b?{ z`ilN6b=t5&gi^QNhq0|IlPkM2hCx<`>(u2U$H={wS>wr2x|Vz-X1xCC#r@BI5vU;T#VMOqWt;1&!>$W0&z zpp7SPdm}Bg@2~7zK;I?bU+J3h827ykKUkEn-cqv6k8Pg-@+Q2GTLKgpLr{7Gv1oO4 z=^fK(U$1q3O8v`L_g_0d*i~Jc)H$4ExU8v7{;Hhbi$--;#ILJ=nhm6zmoYtLR^-H9 ztiQ*G`z+=`7!O9p+Z_@#iCs%k=lqD@t<$GaY;IcNS0bK%*BcZJseu(D(jVoJdKqbu zIXkk9?jHxsN8pAw!K3MD#B;P}D{aEmjIsULf4CFalsd5`J}OJ-w>+Oy<*WztZ;q)B zy;C|f%r|4TNV=lMSc$X>_B>zJuvvD9o_x7=rF`bEL{bl{v|mwA1&ofSe=ro$cNz!P z>qqP*VA8E`mZT^RI|M(Dqdmd^8Fk#W$C#TXEp^W*4zr1g#q~8d))|bi?z>g+#t2fK zae6{SQj(h%cktjfy~(>POf8cWe7&xg7fixYojHPhu41IvNgQajRgO?^n{bX}m9QD| zy76NEYM9%)w!8(y0EbO1b$ll1?OJMEnp(oIFk)P$|&2g2GVJ8wev6W^SBDZRX3?>>d@QX1Dkzdn=i?hnv#uDK)DAOohfrMj;q zY)GE|^)XPmhW^);-uGA<#y-x_027>%Z(_fgMqn*Uj%7K;FPQ@fzh^Omes=|)!-5{* z<%r*jlD^H8?}$9bE^1*W>Ivc6EjY=*LN2`a9eZz{=i31!%g~#ywA+meCf`qDcBev2 z+zY(OZdPP>H+NaT>@aH)>9GpjUw85iwlt8zg{9#ZC!btLpczOv%^-l3bQkECqmM)d zOX$qWi#SOy6uL5x+l(R>jjvxqV%jIeEpQHg0|iNEuY{WG9aMuK*#2up>C`yinR8J; z(N@7{LZCL}SIEvx@yC+b)fxCbe=@5-w7oi4ww7}&ny*Ekx8zPls&HVL1>c7LB1e1^ zwmCUmyaMTjI21VA^qLY@SmGh7j--PeJ>PzxlJw_(&@28DNt)^)Oo>e^qZDdl9+shT zrKByx41^5Xcw!QmX1|!RVPqV>nZM9-Y$ft|Jum$UOHm&uKG+_xAk^sK2cD)qXGV_# zezBGsolR|viD~`a(ZKK%5(HuXF!K>Hbo(11TC@i+@@4JT{sdBs z@{aOMwz$v6%(Zw8%&lntX|`v#k`y~BjFdv5&ikSb;~Qm|9;cjF+|`DchsxqPkI58X zvJT(uEzaCl!lo2Gx{`cGz^JXQNRT+_G)c3~E!P6VrDfH@Xb?={&!RMLT3MgkYO6)@ z_gTZ^y_O?nRfjaqVyk1;QpN+rUmY^MjCpT><)@i={Ni5Y=15M*3s)%h#Ofp?h|ll+q%5FOC`+nOClP9}kG z9IF2BkZ+4=TU~pineqCH=hiyi#pCZj#KpC-*Gya*S`|XotV^+ZkZ?lDL|P#EM&|Ua zfr%++ISv@C14Wl+?T^-A{gI_K%tdLEV~#9s*%6+sgqAyKiCgkRX;B^yB(g-Utt*Ectz`G5 z7*0XF$X$m-+xn*)dUNB;r|3oyt2dpb4Ru>L`3!R>Ep=}V1EXFk@sL$ZSH`kNg=QyR zwxQ-YW7teZhmc!x*#(+Sq#VNyoJQx4!tFl+69Cigmi;q)9bnbHCj6gra{nP_|EH9l zM?m0eL=j&L&>*D7hYh@BB}d(4i^X8)Wmd!Vo`s0aOW_)jAJ~GdK+5cAUO;=;Bj1ex zRNb>l*68;1L+j8-!yRV~16)hTl7$`0GzRR8>I&c1k$fi`+%;7r(NI7QF1e^yAlmo(;8XL3gRCfY-edqP zGf5!pj`!i&kRpegg-7t0%+>jqE9&(E!26z?!E1a9< zQDr@CEoMUFlo^%$3_&Bhn42@jBUHzn;|K@q$sW)c8ctkVf)nj++A8C8puO4b%;?1% ziYAN?wS<;XI9=F%E`%6Byu<||&#kk*q-W~UNrS5%ir@eP9(EtjyUe`>L?KfT@vV8O z#dxz?9(7@xrhipYq;2m3rGS@c$EoQ!!02u0okBOCthE(JuQ2;T zHcoaNgBM)*x1{(I!nc4_+qkb-DS9*kRtq zkJ@ZIes{;+a2yZmo9Pa{iw#7Zink!cpszHUckLZ1A#d}AUuL+F*zn?fr_jDKGxzBN zBH@5^>_uS6ibY>8oZxyE4bctzlRznAD3tYwq{C(^?NbK1F{A1SBz*9``OLSW&yzS6 zZY29ycOUhIq1;X-I#ezo#WaAU-X>PqNAB^Kd~D}4#J`&EoaLGcf#F|xjE_OUIj{L#{ zDkrK{S)x&k6(xd03lwn69|b2FeDzYbIJE*VzaTcQAKSB|RrdJPZPdy@`fU?ekzzVY z%EUPa-S>IqBlG(VlwzhiL4NJ{Nu%aW?{LCf_DP3~Dd2q0%&0P3;=o9*eh;xq zH}w~xw@q$CtI&Y`0B#yeHN1m^cHR zICv~GO8!Q{tj*HTD|>~xa$^Rnqf1$T<@F@2cl|zxww~ zS~CMO$oID%^-EB0L-Lky5!Kgyb?-K>k2T1bO#Ij)N=uzOjA9Rd zoS#x&$f)@UP6DdXTwNW35KdTG!GpKE;GMXgsOVx3oW2kJus;4Om244TbEV#+TlMyJn}w) zz7foo`Da*w6XOEA<(NTcKbRTSnfEkGCdxZiW(5c=?_mclT!}AQQM$m6rBH9XV!}kb zvkEZ8<~POYy|SE~=^NTk7iPM+10yIA$hw|h7*hA4j;WV%7wOCdH8wqY%l35KBA2!t zl7{Z>9mF9LAstIt*-LtS6BD4MGt(=Y3nH+Ekh590gq%#L;7 znoss~T$qhu>E#N!Y-JG=r#se@7k?#td}$-#9ouC!k!7qST0$SHwJ~^i<>SoJ3&}e< z`m6i4_j7FT)s=kLjq*~|T@CsXbbhKKEtx7m^=&+=f$LAMEdc;Nd9&Psx zh1x6_&&j5@upt0Affw~~GzkMTAVUzeGEn4b<{NmaEHdUN=K7_icRt_1^p^EyRzM1B zetj{Y@5a7TzzM&`&U-mABed$r09U1D~N8CHsswG*;0bCcpPd7$T_?@6nieYJhF6ly`S}hVr~q_qr*dtZ&8TC zg||hX;Xz1z1Vf;?W`2czCGRZ?cmgHYausWb48%^(`hMWGLv76GE#*3Lbs4nqfMc!} zAyaFYm3vnetT_g0(Un4GurQ{Hq^Ne!;1AFn+4##kN@NytKI- z|Jnpz(PSD|W5KpcX}rj4a%tN=xgmayrFixFnBL|#+Wq#|cu&UN8c%(A$K~^|62g7v zhPZ#8p*junJ$QM!t*dzu=ZWEifWr`(7d2K!QpzU@>N%4+x!vI-Ox86}) zK&PBz7Dta|6{`=#|K8|u6e(7Z)4^M_gz$!5W%JX2+y=ya0ooNRw%nV;uIhga}Jb7}zvB&xqVfMBYrI&Gmm*lP##Cz9Vnot(A z98?KGgt)TuegqXw`MduFWY1tBb2%Jz5YTy_Zr0#mWs5z<9{peg?uxw3A;}}VT=JRjxqF6q{Sa`_Pw7ZBjbTIf_MYL z?KX^<<`^8!eE)8^i$~_hALQ{x3(=9kaj4zk=YmrJVn`1%q3`Ak95>?pGx=zf=6GtK zwuy8VY1O2Njz&k3d@o-!4Ra@<&vQe!er+3fzGX`zcnPNvMq`i~*G7qp#0(2VnC>Mg z6e(qrBDAI|R7>ttGkE`G`~#+t@Tvlnd9Gv>*6)BtYfkQ$lk|VOF7y|?#Ru}VSS@mH z{44AhU`7`sIM0i}OKLCoz=>Vs?8abI&p3apIn^3?l!|5MfE{bc-;O@0siWs!U3vIw z@=IOyYp7?gbfCH(x;MKism|BgaGliO(+8h#Ps?mTPyRsP-;57_7BWh3cwbjMOWN@C z_Z^*>cdf}-=-(`zJ^zzDB+$nzalk2!fC9VSLrNgei(`32-$OSd*}H!L_Yc-Xo`;dd zr9rS;!k1*90rC;}2BNC>?|WRNBzO#*~u zl-@_GB4rRl2t@>fARR&(KtQ@6EkQ(@BqHHal6ar*S!dmQ?!D*ye)qTTTED;MFOslQ zp7z=M+55d;Z&{S(@t`(NY;)i4^*S*hYe&;5<&i3=p-NC78D+GmXMQ;^e_}Nmg-R`* zX_Hcybqqp^r0SFSMpSOaWTp_MR7sL8m-C%6eD_l_oy@h)UFW=jdx4<{TaDwRb);c< z*;iM>#(rm>#lo)xxy!@$Jv+3|{_V4uo0|pW8jQ;4uG{_YIz$%0)Dp=O5S4D@aZ7}b zqIkYi>882%in+P6qILD@d?%lQz72EI-5AcS>B&|v6!Ju}UDleE-k`9Rqa{KlImsTC zlAW2nx?2=i^m=Klgz42OrvVg$P|9?IfF}d`2k4<|C5WZ=FnkDP)r4>See%THs)1k>;n=1@^an?5$xZRO!c_T%>bJ-d=MRNO&lM{H{PlViQny~L5t{i@kQKK#?gd!F4poNy z?@B;mvMQ=jX3nNk`#ZY4Vj_mPrSkRW>akl6vGN9fG8Ts%shG;?9u(Q;_@-fr^jD&a zaD9+1(NxQtHJR03Q!e>G7yY9% zUtg0zVj^!E%$W}I4vsO>qcth;_#aj6a!iS2Hi7J$)BLcK?&nYf<}1H3whPn9L8Bxo zae=mLw2uuU2FvFyWkwFI!;f-RhM8KH%p-+w{xYX&p29+%34+*_?wz2>vGwCU1Pl`vSH&h-s zxzRhQx{q}+bTM&kY49$eK)+a+tQGUVg=wuuYIS&NdN`k$BHv=El+Haw)wmV#-?H-3 zN0M|k**{{P(&vf#W-pi$+lLs?Zt2EoZHPF|53r!6hS3e%ve&2!GCgo%HnhaI{R(9= zPkZ8J>nYNPL2uhqVs*{OJjt#5+72~mTQa;$8hzx@q6!6sN|W6?2Jf7vG2-8*KGh6d zaW!RU*5T?eWtH$-*XWyy%Hy5JeYJ_8fxGYJ7h6Q}ZN>ZEOLQ~&Qv5Wuex#9k&_}30 zTExV3W#9rB;*V&W!r1uE3?SYt+R=g_)l1)O(?Y(g&*9wPGYslSFx<#5vC(F=uk-eQ zd-sLnYL8Bv8+E9AGH7W>bZfX=05ei36R(_`Qlfpn<2&FQIq5UeW0SIJw*R1_!aAz} zVS7%xyg&tPuz@rIlVfyc*DDB-YE-Pw@O_MrLDokaNCUg|fIL`?5-ETPbEk`bJqH9Z zJNhxlxeB<8z)%J5I1s#Im0+aI2%|0~;)R()H385Vm|>+~w8a!SI2_RMh7Fr5ZALfW z)6Gjwym4PD{mGSX#>{0+k-d9IH7~@Riohate2whA5h3fyDxXL>4#%DRqbn(W>=%i)P}yX1`a=j$a84ee>r^h|nVcX}LL^v?Lw^>n%w zb@T}E!%*C!p(FE#l_U^wlgaNBggGxx#R(rcZZRN zAEO28FuwQ%k(2v1upOy7 z$Fi^e9QyX&|GgrZw|7w;AThQ2n`aWWYho`B2NVN#`lW5{uPp2)_rw~rX+E@^wP63k z8(x>i?UXcLo=GGIIp&<_>0_+*!jmx&vv{5K1nVVVybx10JF($LAA8NTradU2b`Umum->yk8*nK^5qpr)Sf#y@3ErfC)&S=l1T{oN>i&p+3 zOL{7 zsyA=eOF`$$7AJR3P=ot(^J1MOiHWiFRq^#Z651+FE!`pI=G`ycRwof z|H)4u$$Yd8+S(vJFV)v=3?aLjb1T~^EQTlr{l%Bd4~#NMbl$C5Ca(jaD@0z|N$O~X zZwoWOE-jlrq1W)4R{>sB{CPmEaveS^jLrGM>hIilGprj6kx7?a>_1u`0D=)7v!Ya% zoVAL!rR?P1gXC^=UEA@$fyX)piry&fd_N@d0rkM@?wT27ue{qL=Xs^?L~IsF7FMTU zaM+k;sW^5B^<_Ng>P|AS75qe7p-}!NGAgTxsgHV5;agg~9+BP&A}h~XQ!8kUo&_qL3tDO zvb{mg*{(E4LZ$fX)RG9S9%?3|Kl;SnhE;1*3_?5tNv64%#`#?#U{I~nIdd$rfwom-Ya}%TbCw5UE+1bQai((h;cPlpQYtGyvQGXUyJKH zHGJ?619KlgZHe)9`wJU;abgyHDz>t=7|;Aq4$oBXPl?OOWLtL{3mX?@7|o@cPgWvD z3UnhN4sWiGI(^Zr@snwGn>pV}JG=4H*H0woZ6LPMwAKCLI{GnDN^e(lsds6@iI}v^B3qXlw*{&(9=a7aCh68 zH}j0rD}3>1-YJglMF}998kxw;M^?BzGEL-4iJcfKTXC-;`@rC2bjr^0 zn-_p26vf5EZ@52C!!Ul!=LP;92!9o=GExHtZX?mOR*>-+Ke2qjfH~2_J;E&NtA?9c zip})C1E@WVJnd88wry^3=fRdc!Zp}&3g^gD+L3W;e`@Z%J-s@7O{laOX4TlEk38|Q zU8(|Z&mt>LgmOsJp87U1_VwB4cjPr3R#&U#Es@$$sWPUhHS3yT`?(6sH)xjI==<{( z2&)40YF)td)g^J9Am^_JaO^k=@bh3q(7C(joL9Fw)$OvI%{9rbVxUMj#Yw1$Y1J)f z*a-OkZZO||jxm#@xuw=%;m?(4w8}|SCck~~xgYvOu;*s}doLByuLkugt2PS#cNCo- z2io+E_t5JrUcQ!#n7|L_cpymXUWlMdA#_c+F(PXPzMThSS`Ra_&LApK77$dLhzfCQ zqb7NYoI8@4txYeV`h+kKvRKDF=1KsSE9rB|{T-Ctw02Qn_^-Gtx?ONSoFro{7G_z4 zHFIJ%CTmibBsPt-r(yOrX3PFwJrP0$#Lr)St+52^tR zYuD~)WYspciaA_U=6=aJ8mEy&wr-KPKgslEq{G!NeEa^1^LL-h8+2u~kGYvZeBZU8 z%w!vKN@a9;vAMzr(ltHPRsHf{WD6_MSL;dq^035;S!c~0ob{-=9uTX>S%XGCVoZ1^ z2t{aDz?${C{W3wI3B8W*ay!Z3O`upMNSh_&%!)TR{b(kPZ`{*kFJZbqbkQkoj;WuF zv$Ly-oRl5prXftwj;!Js=brfEZQDku(kHPT#b@PNcWpg zz%GK(6)M)Qfx9;G?I88;$ly$mfB1p1Z_!Ddj$8@-l(Z#6-5@{$u%ntz&?oPT3>S0i z-ul`-0eif|eiGW}^lt0jht-GHRY<{@qRJ!MpEhpS%6$+v6$>q^=m_s7wOLNBX8P;u zZHg0*uwX;SWSJvtPZBkSsSXz55|nm+#Iw z{y$muWCNFt{Hzi_snQQF11e28xyv~sc}Oq`tA#)`+|X|-OoOQ1w{q)lrDVBUT(e1h ztGG3MVD1N!5b7Js+ojpav`xBI9Q3?0=BnVkoWZN|PWQE8;A z#Oit07b9)91Teaio)!kB2(V9cN(nR(5Fe^|74WCS;|Y95Iv`FojH%oCV!QpbOjC=<7~lOP)Q&Bxuh?1Yrip5x8}s=5P3OyqY-^eM@%oKdBqj!L^%9tx_8ql zSN&*viLCrRdGDc|z${sp5~qv#(1sc@()qRb8^f(fe1p)Wm{?zwOU+!LmB?NaHcu<{ zTflt2OiJ6}rK!HLA2WTR0gyWT3MWB(DXydL%aba=m~IsVb}wXpV}!-)5KdyfoR|+E zmA$trHlte!IRcH&2lmlBI4^fv==8S&h}$G=`%uH@tpMHR-!c?{xop)LB1njcpMLP9 z7ZFRIDiq2m+q^6{$g!^YZ1QywanRsH@raH!2&uSgs$KT^&Z3mv{<^UQO2;cclvA){ z@xffb{Bve~on@i%fgJlRK&62f_KI0=fE-qfz7*Fg0nYEo5CmYSXzIo*xxH4rY)c>$ zj4dKe&Gd#-VF$MX3x}z#R^7WDj6a=*%e<#^b)AB)o?GlUY?#^`dFDyfm2?lx`tZ2m zUU%Hte4jo{XPV%<{SCyIP|=d`;46JvvD#~Y&!a!&Gpd6-yy9z+CC^i{&8JCMu3fZ7 z2wl;U+(n4ozlfAwyh68%Z#Q7>^mfdJ^-bVX>8je#4~@>(zh(;f)vn{Ix#m>nOw2&*eF z(2o}$FDEzcno7hCvK}YBPk`nuJ9*i1ebs3w#n>6ui zhB4I+dPp1oI6^UwZA;fYy1A&qjH7V%nwTREbk!y{Up0_vTKf)m<)9k!TjatWOAG5r z4w9*MPyPAtjpn8WYGJLnPx>? zE@191hjlF-!Cjtdu)H=!X~Xp3PmUX%6ZEj0+`jixL&x>XP3Hx#8xsvO$t}mm{R~lb zXZdyquT$ZEGq?5@zoY_#cvW72PUbYFqRl z485-^f+&Xm(U0o2uPFM0R-&oiUg1X!mJ;nhMEbnBrgEa>ScCGQ;%Bc;O^h&Bx5gOKX-POb28EJ!GMXAU|29w%FVb zvDEhwS&3`bRmmUEwEvWO|I2n^ql=gM|B(jbUtYogsRkn4s^2QnDYvU5njnH#0q-Cn zI4`D`JwRA)BlH^BQ#IrB?lpI$vct(f;wKjz0B5Rnh_IE;NOjf1c{{sJv9`Afo|lN# zLuWgI8L|PIXb`gCxG0<=Hhw=iMYMF>|C;Rl-ubM2cPSkyoxy-OBL7q|lISoN{VBLo zRY$V>Qhrtb3!=W#*NNmTblHl8d}ROkL9nH1hr15cGD}z+Ow6|Gfgg7e-pCtI@FOIH zG;Yiz7#}@AiaRR3^T$t~-wCrG%U<#AGjvN`8X>yXm2P-P(09j;8Ip6Wz`j?yKW@9G zVd=!8PHHQt+VkGXZESmbgDM!)GhT9qelba`9%hR!RFskR|O^Xi(59L(HAzVAw{j;zPQkJ>d7>Q;*TF?66MxlgnM ziC4vmHcfQ|EG0ots<(?=CX(C;UHUppkoEQ}@tt7NGL#8pTQg5Oc~85^yI!0QkcA!} z01Z1XfWr3!VcLy%4wO4h6@9a2ViPe=l?lav_6r`XL^d91N>gQ|l_MuwY96I2+q7|B zirHG_B^lxj5f~R*I6BJ-^eNoCc=Gwa_qc77se_}Pzs@m|nV8y@g6geWptqM9XZH$rQOH$G(s$)Hq*`8MsC&(kU7xSvGz5B|zk7nTDDrj4SCjMvLJ&4kA z7U(o*l#Me|z?!$%l57MKdSR~Df*+H%Tx_N1cUh3+jykhGEruC=skTFJZO%m&CO@;s_LIPmvh=I$&1SAI|bOn4p(+F#fQjZHIN*_zGi8 zG!xZAQq4~%qs^r&Cd-)L)6QL@Ql+j2LXXjFDYv<`vUr(8$nUYpslNd&~sU=NZj(8JR@zKTESP<0!ujE*XJq_yDASqEw;!;!)!o|Z|oC2jtOT#RZ zTzn7yFd%Yp6+aFdM7287Km!t1F5q#XjZNqBg)=eB@x2|wSSywjbH9rq0W?~-1F4L~ z9V8IE4Ch_K!cAvd&6xt7izlVCA}kw7aSf`LPQn_=v;mkc@pJBLyCx(*y6MqOjfRV> z+o{UfYQ&!FJv#{W>BvhXz3}l|9m&X>GS%DV!Gi^=cE$XGF%>&}=s4B*1%4k3-s3DW zcblAxAn@$=1@!vznwu8I7o7@25Vt<9PgLuu*iQLot{xb3`LfH|h(;#pM|jk?9nw6_ zg#cDJ04pCxKkHAX*H(5pz7Ook;uYrwAR=>q7)Wb4V;{xXkDlqv#3|R-ziX#5C9X2O zVp?^};Ym9oEr#MUl`{SJ$>`8mdzJRq#DpAo)H@=4enr^=VjA492G=onCvML(2GGx0 zuS$uC3nWou21#0{+@?f%szmm<4eM2#%lTrbz&MG`%Q1n%1v=Z?l5Q!boN7dUYb!77Q&pg`-y!xV^MN?T_<|hs7RO zyP8arqI+&@sfApeoXdT)7jK83&c4if{-ejwqEPT!4KaZQ(Hm8b6hr1S^0LOS+Yub!I5O*jUi}R`>Ya#apRnL*Bi^ChL;d3iNQ%U{XKp+`Ef^C#MGH{VQlr zNvjl75Om@!l_;Xb$AC>7&_47u1?)KhpZWq)mKe`*7nrLx+s{e>XWLw=oPZUNLcuad zZhcXO85Y+7KLXqzb!FThdwxy)Pq*EV^WVyU&);+I^`tO|MEEQYx)R$7Gm(pzvCg!tcmri8!>sKogxieO3p-RgOcHcKu2y^HaJMA<2;R+>;$Xh_* zff0U~3kx{P#xaVvZI?A;ml||qTh*6S?OD>ynr&x>GumCcE?3l%FbJA9K0dBbjfcD=3eu zJ~q>bXH_va3qXTcrO1C_Z?qQ-I_ztNd@4Gk@T1cbg%D2bL|>6ahbJe1){zAF)_gy~ zGD+{AD~dJ}*l1S;{280{rAO*`?I9+Cwg6&%clfbk=yzP%4LOlordfQ5j77S*Z#%Sc zCd5Iz4Q|yA+LRPLkI0%oV|1o_qEa6jzBA)(UopkR@~xuD=<-^D`SL|oi2Vnfpp=Sf zGXI}3oa)ibL`S>s2i&tnc}uElbzIN8$Nschn_a?sz`Isv`4CLd=HROMqqwby>r1)Vsd>l%q~ZP$X{C` zK3UO(Mt{%8Xuc~v#d*%wWt6Z!a?%@iVxdPUJHKERsY|tC+kA|r-oi5s`$xs^v<0M2 zGsCuKqL!_W0rhrc-!}aqjhS96g2vQ{`}G%Fa&F)(!rY}oCrj^tpUfP_X1IW3e|wB3 zh=-57X#+y)rMlUyiBmq&Oh8BS;K~6EmSI;m_NhDnT-L{|?>05YGR}F-wuq1l&p_K# z=2s?fk=;(Lt!0xO6?Uv$7|=N_YM~edzFmKC3p>M#;bgZf;>YlOz}|rZP6=>lh&VDZ zouk6IsPVsA8GR*81X#&AqG5k+Oy>h3)~P5}XH*Qh(o?&!C~qXq$ms|;);BYTp+2fQF+D?26(A|Im^qEz5-Bzn^N{MTg1y>(V5y9Ib6|keYvUAc zfh}NYBD4Xs)xZqy@8JIi0Tkqt8RArG4u+={!~Eb3h%l!|duxICzBx#Wk?3c<3=^Kz zsz_#AFwQqC#=Ky)+whJHnLWwp%4m62yF;A&5YO5)Bk@XuEa?{{0CTh<1Jhx{Is6hR3vg^Udg z4}zTwQ^<9HFvD-d!s724sjDHcaP0p6x|$w;V4_cFpt5%FovpN*Q&cnIjV@!_1jcC| z7(57t!bE?l9;l&x?K&s0E^6547y7Auwg=Lqe2LUZSp|8UT(vA^`bu(dl&vPq^Qlc9 zQ3{o-`R)fOYbXU2K^NH;CjR-sCH#0;`%z8>{0{g-8?89cDUeu@SQVD8<9&aRa2T*y zk%-$d62>N*Gq{iM83|xpt_4VAJ-FwgCnvh{jB<&tgX5SF1TR--lH6q-uWaT~ht=)c z_bGktD$m!}h)y}Gr-vR~BpSyltoXr7&DOpV(TB0m zJ=VtXq8+Ew!>GoDQgF;beI|JkDvdpgWm}Y3coDIShC&PKp{acBqt(VQ<`=F8-&3nm zfNU`Ni!ma7@AE`bou205uEcjE^WU`BUv?+4IUz1=%vplv&^Ttk~nbE z_wo4!tTQWO5--a(Vd$pg&kr$k9bN%m;eTNpyTrkjFt6RnaU-`#bZ2FZHvKJ1%ZZuY zALGB1+pBr1db+M%woaM#l+q-TE4wjtO7iwbg>_|@UcN|0X^E-i`l^%~xw|yiAQk)R zR;AO>{-_9&l97#kDCI}wWTxa86g5=KDRo~5XuUzGm!UCvPnwU(}i zC+jIp>57JHvPCvoZXoIcgkJV@!O5S%gBw^doNtXX57XNl=$e9E2PYgp8}UxV%xkb- z{dwTI(IL*Ai-5*C-vBZhOZ3u+^_eXliyb*%gPCNUF+A>QL#fHj^}lm6kMP}!^;L$u zQz1Nzy>oj?RIbRY_fAjYm8Nc^!ft7og?rpNnH9BS%B}t@+bSXnwlcl9D(!*qTfxWZ z#rG@`^G|c|slBOQuE$ zXU_=O_GGN}b%@uT1>6-1m@-u9fMiNh3Yn?p2*ifVB?iz9wTvfArb8Aqd%gmxw26-s zt79R<7NM;MIP1{<_r^K{Sz<~wcWTpH`H-s5rlC!TWx@R(*{!3pcJT#K>ZQ+Jp%eDi z^24Dr)nW{0z-p3#RT^H5+Xu>yo_m#TO2_vXr7h|J5-TMyoN#JP z(jy}yHWc$KP^jv>t$LHfw9Sp1a2|7iliZ!VUb6A9-CwHl5iRdkemVVt)Lkf_n^^gq znwQi=iJcx~y7HHWh)b0N$wVbf{jsczxAMQMe522MJ`EH-xz_zjvf^n{5Xt7rDBs@O z7O86v5xEpaD-mQeIgSj)7pPq;ltX8&$~9|>vTrbFx;p?xuSf0Yaq{n44>7$Q!1KFw zeen$7??Wq!k&Xw~^f0CfnBI0h$0QV9V0vvs$?Ox;FBO>*-Otk{aB%KeZf_xUnv}D@ zy?C}hyyr6a9HlN9+ZWvTQ7HLGb%efKc=B6s(^irNUtHQ@-J!MESSJ@In%PkM#x^uI zpCmcm7s)?m>x(iILpot)zwQ}U8@!)?&I<+{$kxW4KpidF66yRGAHYK1=i9;3sNQR$ z-^-UtdcC)NgSZKPm;Sr4%KsRr|Cim+zdbt+75&NccRgM6;Lt|L4`V|fe`9Wi=8sQB zKaTwc`4eoqfp=+-3ssmCxQh(CM5b9M;RMWzu;T20d8hX>UUZs$j_&GthWkFrXHT-U zagyTvJRu`8G&<_-u|nr?q~jXT0HkGrsCN{+jRH_L(;w+_l!Y^LeB{I_|=u3pjbE zA~<-z+`nDGP^80qPC>gc$M>Mgc#Uzq#H`Ea944iq`buZ>(ae`vQje2=;^epF0s?}u z_@u&D%jfk@i1r6cou^A~eVj**U8TT8O1+bqN+jHy%n6I%O_77nIqIv}^v`tl4vw5C zU$-q%A&s6=S6*g*qfca#9#`_;4~%|Jy*vBxY)$=I7P`Vr&`MZdEPPOwk%Sw$#9jlI zXqrcT+Ruag?WqI%7AK)yiyb~gxCeG#i-hiS()T(bpbyC<=5SF0?2zUbjRyS%3tcAm zCoE9~{SPi|V)*eFX0h?T%`cxYFRjtCJ7PXEMcs5?HyczJ%)GD7K7MWJ;J=G?vY7QG_jli!x$?z+vuLM(g6GiG~FuLAXuZn@Ee$jo!(W8F84pp8gD7&CMe&g{&+6-Azkn zNNK9lG?^73ESAtsU$yzAz+kEpaUoR^X1S7fG}?w{l9pd$nqwY4bg|)_&Twr1yWv1v zf7Gp1Pt*4}TZmUGdgC3D-zKX_U9uZ<3a7@9=z}M~PinThm$A8 z!o3c7+_-&8%R8TMuB-bH#kgeeo=ra zR9)c)R0yaA8^E64*DQ`D+lxf^e55eGhdNGjSe`I-mz&CH2Wi|ZlC^b>+;!UVn^6rn zVd&3eYnlQm;!P{TsTy+dwI|=LF zP)~w;_^;EAIk^|j3>$h-RBfNLmuhPBkz!i+#);sBhW;nL+3p7!`HFmZe49p2rmiRr zXs;->Xn%S6Z0J!+$>VNQkki^t17Frr8bCevA>GXjv(yRs_R#QC>q_~56cGI>%Ap=kI+9^#CV_7{|(BF?48r>P3}1A26*lbv|2=o_=Mrb zp8cLTiYzqGc0Z79K}l)6Bn6$%oAor`J{RZw=pIw@{o(Cd?R9O!F#u!!j6ECjh7BQE zoYp^S?n(A$jV$M%lqxxYQeP5wi#PbU|2t2P|B2-I7HlUm8WTcC`G8#KcNC758$+E- zs=&f+o8#ZMGFZh`-us2nDUELfmtcuC6xVC}mQD9gu^LY*N|nCIHnCMjqc^N=Hpq4% zt9E?fY(jkA_|%-8d+YzkCggdWE-Kp#QnxY2O3M1l(`$(?(qMgxnn<_2dG+7S6GhUj zc!JEge>-;Uw$zQ&e_Z7~9sT#|l*h+Do*=;9{qpa`$=Js!wfuTg=MN3=1Ytk^$xH71 zqIAfc=dYvYvcYEOQ7;BW%g6psxl|)Vwl_UBmG!6D10U_u{NG>Z`HG07iDu$-V^kT|&oUnQ_`spfq2$eEO`gtf zK)~-wi^q({g3s5k>WI~~gx#MB2-B>;j0{zA zj6~FZBEuUlQ(Q&{TGUU@h9I`SB|Z873bUH}uNoZxQ9URdmkHLTI)6ZtTU}QcM7rJN z#$kRdpJ8zS6@ut`vu#063;lDevR|+}jmVTE1{@DFpnBU$G#MH=l_is0 z%^y;!_jzN2sB2fCKR5p_)^tOurN3~zF6&OW+YgeI69RNkDJy+Rp3FDQs<;MP$}L=8 zf1_%Vfl8+;D?j{?JMn)PL?_d4{NBW1wfw3yZh1MK+$-wM7nWh6pDm0Idi(abTl(jh zL;n(Zt(()B^Kjg*PJ+? z9cObM(fiEWG5Nv~t7AA-aAf9-;Fm`w#&R#LOQf?u?qZhC>QPHYTrEz&7CH*u+7y1^ z=#s5>TPVIK^ZL8>hshwJWxB1Hazgp2H~JTzkxWT_rX-mk3W4VhoXNI10@enGP?;Z(MkB;sp;~=(@KY*&cHujTwXd5S|V&* zta-<4^*{N<^7b!L!g_!5q~cEiMU2DX9F(90t#>KG=rX)l7&NX;jP9xb+zzP&B54cf zdJfx5hs9}Z)igxwwLQ0zwr+VPVV`^X{Kmrzy$^N@bPbv+C?YZLW}4)*{JEgqHz{4T zbI6OIl5o4dy}E=zJd>=XcLP+y5t07-$d?+0;=;&*H?&%8yrR1QfUFwHPviG|i8gs9 zTmdcpgDUG4w{yH*0a)oG@c6gO0%5G8TCk3F#(Pn^jk!0PqB9)WuH?*?pE5d+)v}&c zOuq5%n*+=|$}h})c{|u@xn~P#!pBj0a=?4?9@sTYAoJ0dN z+I(hBDf0G*F^C7zsZ!PylJ)SlKo}4a{(Rq4twMWaDmyPtp&)=fnc`z|c^#pBaWb5U z3g&!MAqpXp_e**Sug$iPlwpJEquXM|fFkWWn1N_^5t6~V9FH0RgKA~o0;cFuyq#|P z$+VfYRzmTn_`bmM@L6Z|s8E4n-mV{U=~0)fOV;n-4_z?$`eRyWR8S)#JQVb*RvSkf z=GFtRCIhj#aou)RMY;{ax$=lfa(Q#en)hJn8KohoSDA$xY0~55M?_R9r|e0O6(lB2 zOBj(el98%w5VA2DyjFR1-h=P@M*6l5lLoUsumMJPy9vj&mu5e(ESM5$l%a zKnYU;B7jpLXXA3@BpNM-0)()>U%NN$UQs%xbO_rPQApJtg^8OJMOGPKc9dp5cB1?3 z;#GrCB5lZ%SosF{*SmlwZp?%S)7~Kyxbf{FQcdVpgSTLr+$_fo!JQi)?df zlzi7uQ>H8OhtU68Ln)60W|5AFZLJTg?@sA!&R$`y0@w%GPUP0p@F#iL% zc{$dCHVoF#)gx&gsg%*ALM7nI%jBbz_lqZ%lWA{7)(H@brEhoVhjlkiX2gPHd)b84 zJrQJm-ZtRrJli}a5A{bxzaSwqSIgT=Xx&Y42X0@l2Z~%8r)e!Sn(EsV zHyZ06)4vw_8PC#!8|*M!@14G<*+zuflGH8>WxLqPU(UZ%PL;Si@z>O*#j{8!bMm#I zDznC(-!qFLtQ1?=(K}NXkt)9JDj|&bVLop6s~CH2L|awVOrHX#3;rU zR>(MbEFmTnG8SN==2x63u*(QyO*`+B+1FnO9CyYV_Y?|G-P|3O0pj}U+J&Oy>PMd@ zcuufAi00?5^uk_l4Tj3Yf=YyN{<28NSn8O0(+ug^GUMCVjn?+9hCZr@@yYMwpie^F zMi71I9K`DpE8`>8j+AFtW7~pA`MTv9@$|E@V;^SAro<^G{QkCeBn#I9mjfNa{%w@_ zu$uPfwv7|#xhC6YdFK$Q0G9xl%g4?5B4&Uy5FJ3b66r59r!V{h5%wndGvcY!sMmvhU-c|Zfx>@b&AqKM&>jZ!=A2M#hTqe@V9`EQg41FQ8%H`dF_ zl@zUQGfkPS% zwsEIx&h1-+#yM7xiWn~mznhFVxZVUlVPpsBqlw~{jH0Um@g+9QJlD8A`Uazq=;>bW z(&baLWL)Vu92{QeH1MLr8H2GHRjG)R!@j-lTw)=U&^=h7u7U&$JLgao#q;$1sCw(@ zl(y1h?NxK7H{WbtFvVWIzjtMooIJ#M{~le{HW=I!<(H9F!W*u@SRVmB;c40{($(+3 zDH(wU9k?V{9Z2phs%OPA2vj#|@TY3VfB=}-m+aeM#f$1V8MFl&ew3;IuMMByEb5O3RSU>ec03X|JvR%v9z&}0p91vB* zwr^mjJ%2IGD5F}8H+*_C6(7BGd_SG$Z?N%cv&YV<^vQb-Jt!;pQLv@)MOIpV@r!Po zEZpTlI?Y(N}5>s!#+Ii&KtEwA<5a#@6l8^4_*OSyYa4F%)6t$@7aMlU`2DyDUhE@CeQ-s9dP=b z61M8{&`yG2l`*#Pw~q_dX?y=kKgTAO zTJaD+nC)^0d&3@uXIJ*|>x9eNO4-KduSmV>ZFkWX9xL97vQi#?FtuufjCj@AQH@Bl z3z?R)!4C}D8rG9km4bic7=W(l7yXKkDF9+}i9{c#zX0;VH8u=@SDm`G5MB1U@IvFv zr}ioBr8>)F2E-Dd3j)@mYffSr7z2s#{0q=Uj|cC)N>y(Ss|;GuQ(o!2X`V7SBa&<& zlF0pic~vL7->1Naon2K!w!u@$^IY*-$2bYT%Rd&iHwSOkyZuQ0@-jUSufv_`kZ2X8 zL+8?|peQ7u1bD_hx=~agMT~pdY?jh`b51ed=;C_b>yi~h0*Gv=i7}c1t)f2{L8>WZ z1Ww}e2j?iKFMq%NI(SmbE7{0zoTw|0&>9Yu35-)I0Gk9UFMRyT|BBSo**v!=YBg9;1G%;YM zqhWFGb9~o&`K&Q%e$gbV()YC;!{T`3`Gp8=kAn`DbkoL^d<13Vnq(3dgq^A2YRS?Q zpjO3G1A){jIp`yk-Nmz2_RfEw57d4@A)D>no-@d*ur1#t4-9Igcu)Ffxo@i|BHgw; z^^G?JW^Gu1PI2{cCuYDrW{DbzB{(qPNgxvhe!}vF4_;{kr%x|$J!$s|Dh()FF<%yx zVhf>yK#^Hf{ba}o!@D20U)w`MNtU4@tTMV|uPrIQ%lMSuh%i>Asd@7JiV8(euIy~K zlYhT?aFpwG>9+XisnS(3g-PESMQ1TJHJ9JH^~hKivR-$U-?fzW(d>n;-f5PtN4OJF zpEB>Fk87ksw+|nfVEdOj``cHT;%h8lz=L7Y3l#_@B;XH?hi+T+vGVq#SK8xbI(V#@ zX8o5I;BjBHhfTX3zQAmt@oas+sIvJ5=k*^Krbt4bt=AQ!iF$YBEjd?Wzij@YnOds@2fgHUdfo;D&%UQbu6`|573clCK$0j&S z#T7-YDqEEhq2lBZaP0e2+L$>_%W;@8V`(Kl6)z5s9ELNEyL6ZMVDij(>YPGD88FkE zR};H8bjqxYd#S#eS(6~rW!G#ulPnGva)*Q3)yCX&sEEdfsq7UKL#08NdXkC*TS2_R z>CaE3!sLV)*O(}Pu$jT27oZ<2dFq?O2I%^5)IH~0!=TQN6a4sz@t=+(!v9gj@ z%y2eiXt(pF_s&6kvw(29B2Ok#P{wxhzJDvE!YmXY5+Hzmq)`(J6}BKYv}g~kM`u;X z#$r9DeC#u_+!MNNiTvmil+nlqQaZ_Yrr4>%ZA5zw`64(lE3e}KjSAgBVU@>YB?l1b zi`JFf6?y}@`pXrlkd1MoqCwIF@)5~kCGJ!9ab^yg$xAZ>)2u3>ga&rcAt!w5Fy{24 z=*H&MpFGr5U-7o)w2^&km~_L`mltOuwg134yDqp`c#x~N6Z5R^pK7c;!l(2V3CR$$ zer~mxWs@0`fk1yCv=rpR)KxY7q$V;mKynmFKnb+e5#9D7J*-fu))`wTRz(c|?)`s= zB%l7TsyJ6eL;XGc10y2?j}#Q-i@3TjELfguYp8B$^;g@M5E&d0y>RuUcDw1LvoBy%uj-eafG3-gC>H;kvxlDZ_Gp@>~bc@}A^a!uKE0|KuIe+4rO0!5SCCfWUqQr`y0l5c&&Hmqj<}?qG8u z01nC)>ENavA~5FI&LBlyJpDg>1(c~=2q%mk{mH{TOT%#S@oXaqmv8AOj~Pz~;Ez`a z3F-o7tcO}$9Gopw^pi&o0%VOsJ3dDffF}T?C;wl9M^1i_l#4$$qCJJLrO zjR(tS;2i$yBd|mDY-OKwA^?1#0dnEjJ`<>5TW`7GC(pMB0zY};X6WFSU7!%WJm70! zUA}YuC(qZ;pFD{-nFrhiUGRYfKe)l);4}t-f&9tyL+=Q9nfNim&wJ_wudglIrGcLY zYPi1N{^!X2b7cOxX8yTm{#nibi8KGd#+f^opdi&1jGlMr5xZ8*bkjT^ht`Y%MkYP_ zO1*6T6=?j&dXa&<`ihw;dH*jV%LI(WU_5^OJ7LTXjIWF_92l+FqW&Mly%njQ2S1Lf zGut-J>t`#~NCD^i?eWkXWz*lVAr@^SJ<|SFUnT!)f5X^M_*Nj}EFDspF0XU`Riwy8 z2mHjRSBUP{at#8JPWNR$xDCVeg6zMtS?QnVLF4xXywngcx8*_KmGi{(;5dkc!PA@f z4-13v(gJ=gyG1y;i*rYKfAQ2z>V1pR*(PNq+?>#@NF)4^pkUbIP6TFi4}0r7lwPzQ zdmxIFV7l%+;-0~RDMKD6>05VY#pmXL*oGRB`E{*Js>!alY)u-+cL~}4eq-AsTw6Dx zRO<9CbmJyld08q4txx9rn&7Ob zg1&dKVF6seRnSUa(-_PzT$y0f^VI1_0FWI3O2R(J6M#P%{b2fJ1?8T@#9_Xnz$b#? zGhpTt|DV1SPPc6h19Lv`&2(cbl>Hq7q~fIrV-`@h&Kic@4P`Jv0aPwWv<>_s!LS}U zm^gvCjvaG}n*hEq5ptZc`w>pp0X)Hocc^{lXMC zD0IvUJSrocR!ux^=p;EWx;Z%9bPGGLbIgwX`y2@wRKESC-)xvApOaX!UZ>rR&nC?x zhH8lo^^J{B{>;zPls>9*&yYu3?CCyVj5xqRY(rfHE7??)6f7s!e>ihs54*t8``(NVBU5E%1 zd+#08TKp$QR@5|G}4M55CBQb_6*vAiC!Jt#kf1JC;P=%%L6->2nQLBI$gNr^T6D3ZY@G85!DI zj4mb?Xr3LbvUgT@d@K}}xL%*~!KpoO9rkH?ku@b0uQoY)O1WEEp^dSHh6kO@f^9Q2 z)MmCm64MEuLg&AaPtF)SoF0AT-*d`Vu&o-vDPk{P(LFZ6J2gDR#smbWU~^YeGlgiD zpA4uA$=aO)6rFC%)Z9o?AOMS?G3%DLpNa-1XMrQW?uu;pP5?>~H*A7l?4AC43T*ObONx)` zZE?2j z`IkM=XQ0E&dI5w1j09erFf{##HPWHNP)m)u<}|T|Y9&%WHxDWb^wS!))cB=Ux1oj|Z1(yv>U_#^~_c@;S^yn`FS5pPS2? zv4UT}<`3SgpHlI-{=?4%!~L@9eaZ6@QT`&O$(GxkGqOJ(&YGF%tbAp$vj&e6fh3fh zN~p|*D~71WZEj`+j1yrAG&s9eyt{I%66?972J2ZvJY{P!?4u_xRSP~5H?7v3fub?# z#Ef8hu5cVs$ioX*BkeI$A@;5=P=eDRkoG^@u}iAn)*&I}S$k9ue&BK8ZbV2{4yZ29 zRohHo3*VwUibHv2m-OT3kwE@1(CIDrg(jprk`xUkDchf7Ns@wqi_V2|hSu2=m>2=O z(sl(nEK54^tScMyf@6D`l0Gb9Kuo~*r;bDGaP0oVrEj6+w|+voo=p0D-P_sqqi8YL z+afPrUVxNJ*vAOBBMU?u=4HmvEmE$dSmVJE3fTH3c{Ceu7lKt7zc-p4MKy|tS`~=a zkH!0H&B;J@R?u?L@D}ZewCr-bG~YXzw^BEk0BVg1;8%4=o%0ioKi!V+P(Fb(VM3`! z!iYOmAqpo?BI_!B5Zy%Kxxf@N%^|bI^SV_45|ML}DMO_+HZWvs=d1;1${uZhlvd> zZLm=9V|f(GAaD`RXFpz?M{x!v(CEj`%ut$erm5f&>se$u(~>ND%F6RTmnGiXlk90| zaq$nTzbm{A8OkN#GkRc%k6DMPmzLRQhX+&&hE3ixc$cnUmsOwrI$zr7RCyHRKo8gtxumCo@JOqnWRIIV9ZT~X;-I2c)KJ8 zpqh&@4czGH#P~L2+8n3Jn3e_Tp5LH(mvd)||Mh4X&$m!teSyv-zTYQR>pxh0FHor) z^am71fcF8sgI3h-f3k$4x>A|&>ioK~|yfe42wm4c349QJRH?mmO(d#ciRJ14c zo`(jH*~^pCs_BsLV2yMUAPLTw+X{;Qfg9ce8X$F`XW>`W4lrY0lZIA}KsuF{26^c) zWA`o}oKuHDw%aVQ(jpMYIvI)()sXVt+88fx4T?yo^SpJ&9BAe z%eo_eJbx@;bUC%OC7`8ZZuZbZ9*U7@6m7d4LZVhPoqe5gmfYoKdNb2m2em6riV@H> zh@|~##@?TdHpxpyS5EE&`3=#q4m#=?l544o`MWlX{R)T%Nv5yx)F2?M6w%0(lPfeo zm%vN#R*yNKrEVj!%sUfB;CGmheYHv`sc@~C zstfih1mQpGOIwydl=bK$Z(V2-aj)*QElGm)*xg#mZo=u^ev5_XFGB+OvKwQqm3e zB1=Ay-zWIw9&?G>86TqzK%jbD?<@`{8k$ngVZZ#r5J1{`EV&Fq`nKDI$`Eg^@Yymf zlBc)Zq>+EI#QYzuuUa*YZ>?OqAp87)=hw+~<;Dj6X5on`Edl+ZdiihD^;nDkAgsuS zNq!3)tg5yAYv(pKJ(2#Ct=`A3I2{d2sm0$}-^E7`_Spod$v~~m?6W~pA_{gkN9IeN z?-zKGNIK>PwIC42fW>J`ax#a>FkVBECAHsRpVnCN^qh30OsGmvGI8ytZ)-r^+b_cL zlu6=T2I1|xpirYjjke;%`plVIP(d!1J3Sok85k^m-ulkt ze)V(^;aFf3DuIzrNjY4C7HVFqaYVb!gyN6y{^<@CH2*Y|e??Bh$s}l7$ITwrfR1us z`Fh$5aReKYFRtk$D~*QIa2E5HGd);ihXSpVeKqm|+dM+E3rBL30-^|6%J=SB5-jk{ z5}HtDQ`R; zwhRuAj2Y8hSvg2=ijvk`{UrwzFX=*AlJPQuLvD@CF z2$`UQ->#46m)lz0Ed8*y1@hDyV|%{cokfL00(km2|2^ z$?vCWw2_A4qJ52yU1!R2U-Bjw-W=^qoQP_x!Uyl{N{QJH7{z6tJ#}8Y-30_83qNm;N>p z4>219E2_r{=!obIne5;c(yZ1nwKa*R0UrWMMiDYO=h#PYVbjUvlrp0 z`>4d$j2zm#_GlW6vgJ){O6ibhI?z^6)c&YI*xqJlQWvf;CCTGVb5d?j!W;T@>>ldu z+JSyxg6clkxm~knI(iH9u5`V%prbkWX>(fl(sA=hI?oq<0S2PbCwho@H(W6+IHjq6 zB@ORTVv??BWBnDJFI~KKeO1O&&S~;=r|J$@Lbm6dcx4%9G<_&9Q@YqD-M2xJRHa#@ zmFrI$xpk$}8+8RmQu2P&d+f*>RcEG5Q*>A|IlZ!$aiPA4Q`1LxMJ-RRH#q~VEsmnD zC9>f4FTRhTR7hM&wQib{ZEp967-x5Z!Qbgzv?OMUDU zmahlq{4D*RxSEaTi;aIezF?QBV&F1I*wQ25ccaZndH31~uNTgDdZM_Ims*&}LQZBf z!;;0jeFhOt6MP*_<+ALC@&OL&7%65B?c0f8)yeOSQnYu;C+B%)2N%47uVlFN&fB$I zi$X%A2N2Tvs7rJr`^l%CiQ?y^`#1Ii6>LePT|*(I-*)xPXC~~$6({oFuJU72)RS^s zC!1^>KSK1|{RQqC-?PvqSrHz-S+o6Wi-jo256=f!)U7MN$nOQrr%S`;06e9Lv~Uzt z{SSBHQ{CZz6;qdIS!5PaDvi&jv!Q-Y|7Ti~m?x+%du@M9KnnjnCT3iy)ncosC{is> z%R=S5il>j4KggBQ8y{xjHvaMM4dBbnCe$9O@=F%ZuQ(2!ad7)ZF7=<>)|J4O64}D0 z&uki7h6p+>Lor={uO!byT^(nm80?rPX#RRB2p#1*t$h(Hk^VRk3l8=j zlto=Y@8gaG{!3R4+kX*3@ISsMiIb$8`n%YSgBFWHXQ<=b0V>y~wv42GX6mgz^hp$B zixLWzCSxcOCCj;AeD$FPk87H*1w7hX7DkNb#A=#R6=68#oGjan1=*?f`N*1+rM-HR zdyU(#!4(XCcPK)!?N%BI91O8j2W<#C#UD%*(#z<~d}ARQueB+>5TdW-I1Zgck>VzZ zRUgGm$29C-G8H_o57b@CFF$OWc|R~S%>BvskMcXNJ7;q|nT9IzRwU{TT|-sTcwK-3l}M(p z*_43uJ>I%Q@}(|Ne6k;*a)etagN56m*V~q%?MREqljRmoifCW^{B^sm$N<}{kid6q zH7{Hfb^mW2+5f@r^zRP7n48!{)m|VkGtzOaau9s`g7diRNRMg4X}^nCV+ju)z!FW-Hp} zmqb6|9c34Z~s3-nE%h6YX8w7%I!fxF;=mF<7b!}cnWf~0=zup z-hhI0noli8naPHkUV0)87j2W|g@rlcRGOfe{R9pe*lCHlPby zJ)ThJzVrq9{~Dn9Zl%yJJyN4`MdWIz>BA;}nU&dE*T{&q!`D*;OCFQP>-E_{tixoL zbICiv^@~$%@HO!Bv5kAX75Q+pFo_il@`f>`w5O95I>-3QHg+Hqc!^QKePO8C zPxFID2iJg7P&?AZC7lS3im@!>i9B=EG!vt%S5 zV3_JOwWlC4!)W?+w*()9=Q(X`f6iu?lnY7;*bHpwGh~l02goj@E9EdZc<*KtpRcpB z;NfL(a=?o1)t58pxl>1U`V*+-VcpW>2hE5(*4+o_SKasup*B-bx6c38<&qKNQx=eFNuU!8ViY{ zuTOm3ezs6|NEx0npzI=~77yc84IPM|ftbYabxLk)tv8dYgXczHs5T5=MkG4Fs~)Yt zr6}}5O2^?aII6al98mWWs=toc9mDBQ$_U%w7?6O`cb&=tSplK-`R)P=+9Ke7-uxw; zXz&6DA{uQwglM{%;1JvW0KXVyWv8zDF~B3UJP-S9AmrqV;-|I%#bAirP`pBjaj0Fk zl($+7OE;!_n124?BE2`0b&bg}3t{e8vH;Z?JqgZI7-@5tOw$VFL#j-)oewYfF{7>- zs#mVhb4@0$V5WEGK8Cbnr#Q90QgM&q6 zY!k&+(|mWk0ALk|D2Jk-d0s zIAw5h=D8EBNp{cicldOv0(vIxx=!V|WJD^9ec;%g>GEDbr~8%%RnyeHm>D7|j7TLl z0D;xCjfFuX%kOTdUu_NLVn^{$#0E^Ue4c60jkrCGEeTS}nPn^FTVNC{3q zIb`C%wyb^q27ZjYB9OCl#WrYTSo%q!>)tbZj`ncS7tnJWTG_-v(vYC?al9)8{hFgFA7Y`F6#78d}_ABZ^=Zxk-27Mm41WJ%#uPy3& z?1qzwl}kf9(K)9qA^zZ2e<~zqRm4*WUwr_VF*NDMiMc%mJ??-u<%(Nri z+jHvMH(;M{EA(TH1;?`$unM~Kn)eQtL*i&v^20gPOM${|x9Gb^BdL0C`eShOJpyM+Ay3PQ|)4md%Zl;wj{)LGB{v*aHXUx1S zx5#CMwniQxQCwlq;(Y?Yo zZ(!EIyC%WD!?dlztLesFXcfm-#o|P@$|vlxBGxal z)+QzSau*Iz!l3&LGg*rcPq$;FPmG!2fu7V`f*Y1K4F9RvC7$z$j^p<_NBB5+qmj`V zQl04Ani=<9&6zeM55WhDbKdF}tg{f97=1rd^U1+R3vKshS@*V?s(>p|idm_yfn$Rr`ZHv>4A@m&81f=t7 zk5nQ}LG{ z6HWUXL?fTNxiu^&jE~vU2?+n*Kfauue@}|KB5B5RK4cT3$?1~JurFUme1V+NL#T?= z4E@D19eg6Tzc@QJJQlGlTl9J2iw1`gCSS%QdS^!LeP-S^uz)F3m!6RZ(0luXu}Zt) zYCxKoVj9y_gt{Jso}F0GP`@IacbyqQ47=R(fRmEnmDxmjEQ%17k!Y9m1*krYqsz0Vh>*QOhPnLvfos94Aq6(rua{U!+Ut&R9TzOK!Vbu5n~`-g|Y76`RF&aoZAn0I)YSj z2N7bJ4$aTejv*)4kxI043>U=p@w-KK42KhGORB*Wu|L%}BK0$q9Bln1ZfTnNE1G}M zj;d_9s`BXJxJ+xTm3Q(cHdnzTa!;>aOjT-%AZ3(q!uCDWFmgisXt1NB4H@L;rW?;bx}5FXqnTDB zSr((l>RPDZexG@UCke!dO?;S1-#6<+OR zTuCS?e=W2LFk8{%FM_`cTyMcI$wRBeZz<@b746W90le9OhnGLUm>oNyAS1Y0NN}8I zZLhS+Ed?f=1m?G;ChS(YX$sPtHzOo@Vb_P~B2a^~7}y50P0&BH%e`8F&UA^{!8Mf@ zoxEFp2l$(kL5umFjgoOEXYeO?raJguRcJ|a-&x|hjBp;N#PCkAWSAbLy6ynJZV4%- zA8x}zLSwA{>oe!*A`n<8!ivxr?f0m9=(=sM``Sd<@~!e{g$c9%G(|rjaJ@ra2M~Ty z5OtKV7dUpFbZ_An`Ju3cqD{94V;A@p`x_l{sfbAJk86XrnDZ($aT9KgZEANk6hD4| zz}o;xW?G6CL+3xvUk6;pK)H{+30QL?-9}=_^g|kPA+-W|p2o9vMR7TNw(3`y3%o-i z4L;R0?%RqO+YVHr?rIwt=x!U54%9+L4Jv_Pu?DmlYe#Fzsb`EOj+Iw%>j&{mSL7Gw zM6?{;gmwRr@aq=8L;7?vuxYp%s`uya=J`p6*8Hz}2cnf~itUf<-4nVEfeO|cLv|?t zwzD&R5E`7kaRl15Kej+5_O_~#xp=5gR)|XryhxdGmhLw4A z^IzJdKD%?hhPPENe5vg443DqxQnjgbEa#%V1S-7;RTqtv%wpCn=H%ZlC|bxli;t%U zrMYBf+vH{Tc^E#RFQ|R+V;C7Emr+BM)%a-$HNIC<@4a4>B+~9mmz(lEB}n3<9Z6YO z>jBZVHXh}uDVZMYkOgMZ@ozB`MHlK}Jc!I5iC@Mb@idSSe1V5Lg}lW$1GMRq_X>o1 z^&3dejtB?JQE5EQWy^r1pWkGIZF}i=cvCdIzIULdGr4zAD>z$eGF~(Ux$mJc7LryB zsds0G1fZRqHJ9T&4dg_?yaDPj@6xE9;z%u88`R@jf9S^tJui7x<^`YoNn(gCXKRh!*RFGDg){cI!~W>NX7JFSRzvm{tjdSwNy!@ zN+#yCHWK}pc-Z>N9TBUP8JQkz1rtaSadt>);1Pf^qfOt{w=`BW{vPc~E`MKh&on~N zCz4|5uR!y2KiykOA>NJ+D>t37#ah^{;=_G4BFfqe97HM|DV6X)cTQv(orx!QQ5iz) zaUV5K#@Szcj}*rd(B3b|_I(-^$Q!Q=pFEltz_{1;J0+;1eYMp^Zc_%t;>%Xqt$p)< z#Bf)RAB_BW0@=SS2T$dJ{t7@6>A%zI-Fd`hrvGFs&*=^CdW~we+k+k5<5?}j2A;f< z0R3dELf-nxMudP-A)!F*Ssts!3@H6Yf8?UNaV%Y+P>duUnQyNP;NdBq(};UYr+VsP`8X+H4g1u!kb&8yi<0qTpYKE*jUSU$9XDe`Aa26 zk!ah!z7rMdqoJmQQrMj|bv=8eCC9SvjP^!FFP|k}B3Z9BEF0dRe%@&U&w*akGs&|K z7|!|5Jo8*rLTjixOkpuNyw+LwM8*nyh8 z-YpNH-}?jh9QNx2W)W5LEYdj3wh@Q}zSd@|+C=@{Ga)@t+ds@9;46aw#n-*Ep z*i_X7k?#}i@Xj|#%ln=6H|Bd9bSr@&sN(oaTj%dfo$PBtt6UH2(%z{IdAns^S}A|K zK?~aVxvvWY6tJf3sgxeYRP-Z{3^88FTQ!h}eGgW+c_{d#a;*N{R;b;0n5!y(^TdAY z@hvsTa~Fb(d}&8#bH>z^f?8l{V{KjfDq7%smFm@oXJ#0h&l`tuRwad2c0cGOY{Id=4B_QbL%?9#g(p-=G5RKIxcL z3QN>mTp_W)Mx#Ldk^1Kt2M1mb;fk~OER?6!vln+ZB{+Ffi0L2Ux8Hg|Fa}(SXpY@1 zyIJtga?CYWH>$pgw0+LE(2x;GX4>ASe}#r3I#hTlZUPLWrbR^6k`qyyQTU8{>R~nZ!G%~K?mB|_fJ`cxwm-Q+go(_8_X2m;x zZOL$tBQNfX^ngckCWHb9tnkob4!1WIC*mm*AgA5y8I&P;23+u{C;egsr%=X#&&0hE zLgsraVX~WwZ;sx{F7^$Aghxq*_m5pL)%`Y zX_uCd7r(D(ODQb-D`GM$__vb0Y2tHy(nq=Ukp88FJ_Yw+E$kgmTaBuO4E+?xTtj-k z-n%*!mnISy7m}e+@JM;;W`;-F#~Vd%1AOn@Z%mercY$LaryVqa<&8c%nEyuDx?NX2 z9vY{Jf4`!w_R0I8V62S?#rSoDx?K0|$THypqm+RCgc#^@#-(WuU$FFP>ZAE>1DwFC z?-(*2FO|RCGgDe)g;^zF{t~?Ly`xmDH;}W&>APeiKKP2`l6utJ9Ya~w4)X$P$RkK7 zq^?-GHX$;3;M;N?<@n*6i4vbYI3lI&>sY2JeIVV3tQRN}nnNa1~HT-O)NLaz=*W-r}`w`?|#}{DcjR>qBrU70k!V<@NWz*_u0#E>!(E1gI7a z^IgzHL+#$3b(7!@17IT;Q3vAmY1Ug&f42K0XAzxWAJU6X$tS$zN2OgZ?k7XjICZk$0;Re(5$ z)#+{!6U@x{$%fWqLy=s7#^-QT5{iyy0M)Ic4;uiFXaWWmMS+(9#g7sV#B$>@uM8D1FMq zG!N_ialqhVUG9(p42~9Tl61@(YIln!c_M=ZzNJ5x4fNmkYWDx^)&D<^l{ai%lKyvJ zsUy5mwA!&DUCpMgt#hn_OLb~$*1Rp{q;kt-e)79qb97i!f*4|1Y7p+}Gp6BqruiRk zO`x^^8=z0A2Ar&bqxI0^cgdfl|66+M+;#6CSH;(>i7`!G7(J!}&2uMk=q_|0r zhMSJ;rw5V`^yAy7h*b33pJJR32eXFcgk&BcFjDRc6E`)u> zn{94!WbnZa`b={mUQtdfLR!5KTN*i~dZ-~^Rz-3=ZnU_$XOJo)c+O^4-43h%ln>=F z>lH{Wv9T_=XP8|AWhYs`MB)x`-i)fnRnkI&b3_|Un3)B~B6&|!D`O1vZkf}DTB_C` z$7C`RXge;jGaI6%Sc2;uBCoe;+UW9x`?#TPMf*vI*q<+N>AN^K$X!0^(BXj!gkE@O zK^BSYm+QOpSs$(Uwp>5R%p?ttS6OHw=D*070`XEV;iG@wWK`btxm@g}dk5&(3C^Wy~hW_6fCpY5h2dCTOb4 z7a*ErMak|#b*$TW?sR8vQUmZOL}{vufm%01CC)yL*U3O}sg@)2gL$ zclwG+y5bJ@&g@uFh9;n{tQ*SI_kHsgyI=ZZv6DZhue17aGu7kx?%H`Wh@ODzGCct% z9^FRlliEsLFnfCBQy0^g33PaYl*PLVNZObRri2zg@T(d0;g~5s!Ha) z|6cylA;#sxo-m}@3^XFW=!gmnWu2)dHMptb(f(;s0~XlVcy4$^5l_dN6F;U4;OSFl z<{kSW|LUw62YE*OJJ5zHT(G6Vh14*y?OzZZR8Z=PUX?Kw#0eL&_TBBHaS|vKdn&5K zq!%-tpFPLMlpJ5*A3tF!+AC0Z9)IGX`w49-xqHSRScm&9$=R%%`D6jprOd#XCwf<~ z3Wb5Z!M*1h-lMDWZC9y}!)x5+pk#U?_kDyS#y*q0iUd(aj6i3UXEl3lF_-g$_^UU# zUO|bu;J5aU59YuPLPppAwdNH>US(z-!(zG?YPnw=D>GwYNr{WSDIHqhI)wmfWT5rv zBcw64NPyYf(>dG0&xom_&dxzd=BZcAe}Cz?)lj7BxOtgc>}03?DYYDIr>W~ev2Pby zMkuRFgro)+L(^1}OV>9xmlcd}Rvl9=*hj7;>a0bm{1L_-eV?%x)coppC6DMo<@dj2m-h=J#e$>Z z&13p1=`wS6De-;hY$_#Zf!|2PN53OJnpVWKHeL8@Dw&;-bPji5VF`AP5UG%~9S;oH zlK4xtP((Im?7Iu1TemI7;uYR>J>D5Tj$fsB>eBbi z)IdoGafYD3cup*0sVnVe(Zyg{5PNztdPRW#d7@5)s>dR31%2wo+r@jvSD6T8ds{#* zg_q%<3@%!pS?|06@&hpe^#^(TlG|D;E$~<15S9B47){)*UFY)!&Xs+_ zEib3e=#F()NRMg##IlEU4km+suZ~3V!wCtWno8ZPX*y`jA=Rgae6ve~<{I0@$zqZr zie`E)IY-{zX;;`0%X_{#lRka!Semp#r~A3GJ6vR8(Pl^E9W&buiTZ+jS`La>hr{d^ zuk1`B$eSn0?Hn_dJ`3Dmmw1vN6B7!kk~~Pi(@N#4a0@rb&bl`JnaUdfH_S+@HVida zMaKddhgw3~{8}zTj~-gL|L}P^SMcCXHlw+j!rCdli~pm^KZ;=;Z`8E%{J<(L|3M3; z{KH8}w|J}pO?-7NHJkpOk2iTR_>11+fh^EjfuNjvdpUFLd%d)LOkS>b)6{j9Rb?f5 zIKtI!eN)qLx+DjdA-t#_)CEfHZ1n7T4V3*yzzzdkiXuKkDVo0+EwOh?XDKl4#P$ty zn(WM~QJH$NlzwiU!U9cJoS0S2&HF-+R&ux!UL7sxH)YNAiCb|FMC{t&3}yMaY{eY7 zsV=`QCD;{j!5;v;r+C#s&n>;Ed_2S-_}ylPh7^tIH|5AIbz0yBSC~W{OKMm*RI6)A z*ZUd6P$X;l)yEQ%*sf?pFY`=d_E%l!mVnC`NghK%h3Rf(fxkxM)>?)?4Y}zZJNX`l zu%?UX76>uhf~_#nGszlFcui{KP8dF5{}^8Nwb4GLG4#ualtUZYKAF#wuez_F-P352 z%U^8@SeWJD-RjfVYWb&Nq4^6IvH1fVJ=gL?EgnQ_h^ z`VKymtz<{B#_9RYVYS9;M{irqs2draEnIGsR4eWkV5q8pJ<-WQJn7bpG0^WG?{4Eh zFOo^k?HC%=}q{jWwO*e+E4f77R@i!tV-=Jk-O-?v`r&vQA4dAQV}fdTSuO> z$&fmqoG#K!#cx$ldI-yIz9bB8j{7_HxgFtA#?0lXADhWtsyE3&d4yNA-3+nfi~$b- zFi6O~A5-(}q;BtLE{U-viiYXgZ5-R1qKAAhADwYNeH{ae8ALU>r5>BJ#*EHQ zNHb-q3o|8Mg-Q%dN@}~`ovAj^5|@-KGb6X-HO)eX816D?+kMFuP6>~EJ_~8akS+{1 ze1*=t$OL-=$!*2Yx>|aKUr{~dmcu4w#P0=##+8FFgVn*U;jZuy83v_4K%R+UX1*_! zX1cy+MpApC5|Oai2#>tw&kmecB#-Lm&GL@(SC1;Clqu1VR4(596{}BJS4TVVQIqn~ zU`@PW=@T-2V^OE4Dh`;%+hnG|8Y8DWq6|9z2xov{~2i_RNpE71yJNWi3^aU?>et_;ZeFQR(H)kr=aoF8~gWl1n z-@vFXE7Et7vw_oYHE9%PXtZJFdl{8dOBvh%+_nVJ6x<0+=%o#TJM%j2@|;MKNc)0aq*L24@@dl8E;s)@MQxdiYakF;JUtHR;y`_H{dk* zYq<$7ZZgD(7&y%nF+n75;-Le;P%aAgIT|cbUS5;Exuw0O*J6|1V!xoKrPpakVk~eX zT$iTP5E-CBAug&ePKv&uPTdU9;Q^4TLkdAGi`4fk+11+B%*Q|394Vue2ku~DAJ##A zi7iIadbwymcdlXjDe6&Q7~x4s_j<>9;LCKzdy+Ux_`?E>PU_-3{q@95SgDV~so? z1e^hP)n>ncUWm$}S#I`e%zijds~xCXe;ySF`RL{paW(gM?AHr#OrPMJCI)YPfpHUN zcn2Ae!_3#!V_fa`8V<*L_j!bsp2!;VmSx>*lcW9VX)W90anVR*JoV+MiBOKT3s?u) z`0mzu76+W3NaH4ypIkqETaV%xCLp;c0eW85=Ux&;pNfSQ)O?GAM z$h>*~LZFIQf;n8p%4DtK)y>8BfCl}jLVuHzAVShma6=g>_+>bJ=n|g2XDzxG2eD1{ z9ZP9#4K|AWHb-&dmQ+t%e|4Cc?|FdoSgG3g)+0sG+;5cIGGI5UdrZg8yC;Q2H-~ry+ znY`H+UIvV8?C4`~XoRSDyh9gzq(oK}Eiu|6!iDD&%<0~sPhG_gGhWXsS`ikUZ!Q~p6rupb5+|c!riQG-r7c>Cq^n^nPs7V9#eY{CMzGdh# z#B@*XnFz)QVH&{FzDz`%0eW74QeaFSBd*%O3bz z<2Q@tF6{KHudJgM%TOXL(KRD6dgA7GsSoV(BBL^$J`RW+N-JXI{Tw>PNB7zTPQ$rO z$uT;kFgpxLd}GWBexuGWw>z><`tZg*ioQn2Pd2GA$@x0bW-@j#ewwM zH!mWV3~J*%#8>rt)Q8(qkzF%Ekqua(Cq%D?PIJkRT&~)4S}|LQoIi6&utuTH$YXnE zj>&V_=TAT~JRdcb;jS7+sA6A>k#47}R#6M|4UWsc7e*_D5o2gmexDKdV9;S(%LDG_#fnTLlUeD{4O z@FKP;xc*aYfnHgVv{}u>58yGXX%|Jq9xU#$Dd=H(%SeA>-*TU$KQj- z`-&;sOE%*!jtr*d5g$ZJO%Tal9V4}~_k^WiFTqf$%#G(KX4E3~W;yc4N}E$Wt%JSS zs*jGi=q4o6g@A}3pk~p#I?CF*o_5t?OUf!{iYeMruFk$eah|GoixQd>zGW6FNqkJjXPm1Bbj+pXIKa4Tkg&9lzQN9-<6bppCR zo6~1V(>I?_0mR=^L(q~WJJOx8l4pbt_M_Kcv881c31U&S3vzAdsZt)e!CNf18OQ$X z19IQR4Nhh6^hBY~BU<*J4FpfN=kNs!f}{qUb*^=fQ)~&8@EdI^$rv%79J_dS8{!s2 zJwbAgAfnJmGMv1=Bw6?|L_7dRO-u-<^S5`LkwRLUC)L!;nD|{Uf$3u15EY&})?i*- zoP99pV*7_sumXew6~^RTlz0KeR7Un}SYC`7)Q@~>B=HUPn~@wdkknaknaWb#n3rZG zQkEhY{qo+~(^ryaj+nkjT+Ti78}{vWHS|Rmd3>*6#V-n3mP60D$~Ys^^F^skAQZ+I zEdEEgBTJ8(HgJ=^Ah)=A&MAwI^jV$2+ecya?n`*zMu7KysPYa~uTf1jbzjATzgj+= z8Yobal3c;up5|(cM+oI2F7oF}p))-#87LRNv|&YP-@1L=4PnP6!E^O~W9d%E9J;dA z%ttED!=IW-a?3frxI4!{pSKIdub*SCHUoNm?rc_MH}OX0@^9!bzbO5!^UI5ThkY2E zY+LBj?Z#Y)n}}2WSOm4ge)AxrwY~9kQ8Ggc#<*yx=HTF%p*40?#k3hXbT#rOp^y81 zB>k6~<+FnZtlRpLr*X=ty>iiBKGqf>l&uDy)6b96^y)0mr z^6h%(>O4xklcM$!Fdt71jPl?`{?};z|G|Cy-)E}-i!c7qWcp+OTbh=?K9$iN`r}+w zk|)Re{;wKoRAIqo2RROta|q58@WB3$!~Ir<68o5pt^ z9!xvjbhtF{bnnaOx!J$!TwFg7s}LvK2#20cEzg#6*&UbEO_`V%?6Kd{2oD`;W(jjoG!*=8pDv?aeJY!_CcY zErsjR$w@O_g?`D>7&!Q8$HTOt-+O)+gUeH#`;@}KeUD{I+gS`-EI92NBwf)CKAtI~ zYlGtmiWB-2ceOvID+1$vkne%Yr2Sm1Q^ac1&e?cX7;(wYAqQ_w@&TZBU^u zOwJd)-)DJ(MOYa`}D^yuy*iXx5;9{KDUj?~(JgT@~v(lXF z%&?_9M2sg1$6o$7E~yyYplg>&*1cUJcLAbA&34dwtX*H8 zTzauWHe2KS%)G~U*B@&2t5U%a9JyMQ&5parm!Y-JC`|v6|JpJ(-5J8#e6;U}xfxOG z;9?+yJy;v@c`@m9rrDuq|KjZqZj(jbM-nEc=U>DcV+vhRWE7NB3HNZOJyQM zhX2uKobtzFk=HLG{Y@TNM?TpqH#-w4R(&pd_1B{PQxq&u)Xd;**4U1j>{i3fpU#{N2tQhrRLDT%T zzigG-t;nf?n7+LmxOkEc!IsyYJsz7xMESd6B+~Ww1HR8WkKjbnVvjqbz^8#NzxQ2J z9F(fNYL(`iDi3P~Y-jPTOX^M4<&7-V^#0-g%~~_8@aON@Hjv}FN83}!`zlWB&X|HQ zwfZentMRJmAD$nO4p#c<(u77kA?O%I*UVuV9l7-)#^D3@;W*Ud)@1z;Tt}&WH4|wJ z@<<|&bX&K1Zu*U2opZFVEU9_gXyOLXSGZOQl}}#JyS$wGa=B005u@z#uRrORW`>10 zq89PZXq1@0AKagaB?6e&3Zj2HX93Kit-hhUZL-09fKy=B!y~e6{@!QDGO-A!84JzX z8PyRX|27}>Uh$NB3v(SgzkOjHUD3lHi`yaxk3TIs8F~H4 zaPeJE|51^*CR;gOGWX;s8)8b5OLUiF%9WU*SP`Z8pyYOTu(ni_%H7+~iMpR<1{4}f z`)N+Tj|+2Eb1WtruIL38-<&46KM8->)Yz-n*yaL%AOu~Ny8)3-T19UTzMU*AKU{uH zIIJ66bA?44oV{lxd1=kiL?YqZNKySi8>3jK5x^`Dl5eff!`dG2{qQ_pZm~0$?~(cY zQlS;Oy88~x={XPYQhL6sZm;fx0Y)ck2H2W*ePTt}#N>?K; zd{Tav1eHQ7xBQY)7#ExIsSg)OK4Nt)o<(naKFO+W-S%$nbP1ejJSxP>r0L(w#T3it z1S$;&mYPL}9fh~;JnSz| z{ucj}Ejux07`4w}046VcwTz8ejh}2`_-~zi&62|)@WPK9hNz!xZp`SPY|i=Ftgov- z*{pR=L$ZFd{jR;M1xURtPk?#N*bA28t)ddp(~WH=s~s334^*8>%AiJ!c^0jJ|M*dR z6P7zeI0n$CoSx}Cx|h;`|F@m^cfOy0?+H2+tN&bJ?#{&I@EfM8{M#EGjUmJ)w=tU^PFK;s<2VjmA~X#DZXBzhh1L)B z8!w%WyHbB>Gj0(kcI9;MW{xOQXS&AGwY4wYU0|*|!ofz4U1spj2bJ&AnP)r(#9HtB zD^c{UiuETLKjx8TwAQuOZPQlp z6T!@Ti$%&a-@X>KK5B8jSyJ|e;&*na1W1zb5J~9aOr6D3YvU89Qic>f!y1xvmVFlg zG;zD|Yxa##zP1D#U||a>y$-ftw=Fi|tCFcFKyIX!_uaB9*VEeXbkegY$dAXBPh=~| z;&0o_$=g_lgH@WxAxCA>1^erE%H3y`DGmm9HHs~I26pro!~Af=LO&O!riXbAGiEye zm4^{)Pg(;$yo1Zm0S^{6W7cikOaJez=lBDpUk)7`Puk4;mF*F4o2TYQhx3F?P3wmr z6iNSW^4D(RmwDwCf$L@-rjD^@4Z>%LfFxm?{F7}TKsb{EyFn-Apy6FWu)x;|0NW?4FQ1V$9oyeX5I);A}BN)(svYncs=z0Hw8vZvq#DCb%gGYrlWT@D^+{swIuwTZjdHnm|-DAH9Kd5XSB`9iNgDU9#(U{#h>Efb2dU!ls zv&emO5J;Xl$w}7$3yVYGFQDC1B&&~W4#a18Whke>)3O@=>Q8AhQCL#}8!I~o{|sLE z=CXji`OYd!oc9<=^cnr*w78Q_qys%v{iAKx6a}RsdGaglq?+{xcz&%jyjjUCKvqO{ z1C|}5S5f5WAdiJ3c~HF$OP~XIeHCVU(5Upo2cUHVs-b^D(D^25!>4>Ffb1xt?HJ&K z{tmFye~anu+_OUY1fD1%fj5RaGPfQ%>;US3_kgenpsmXhC`Ctd%rpNYjUEETd0;c# z7fv%#-vCn40u%bLHwykQ_TDtA$*fx!rOHwigouD30x4Uhk*G)?NGSyf2$8-gA|e7J z(rHv6r2?T(NvlXBLJXlXAe|r*Av6La(hDddiHLNP(1cgga8~Ve_PF1-&v(w)``-KK zoKgSCAbHoj+Bw%V=X{4M#&e7%EQBk+Rvc_ui9`xHzVv5E`4NX;S$e?kad z;2a|_4bWxO8#ol$N8LCCgRYw|07Gvo$_4o^t@Ur<{P}PHFkUF;?Iu_7INTp7_Gkfa z^(>q{MfkWbOoT0h8r0a}KTtnL`vnz=LN^rj3-vD!_cd%a=qq?V9D`yEZpeS47%lLs zC4mfl9Jm`|bP?PNrHJP)%df$)ci{ewD*o^J_KmL27Z;x58*}g-$K2bE6-OfGt@gq9K=)gnHx#SpqqlLq(r=79jXU36H5|n* zKL2tjOo{J(*}>2Gx3^8RVncGwbkF1q5G;#Pu1eNzruy8de=`4Z~L} zYWKz5Z;O_2vAgpzi|H7Pd6PQ7yNT%Ibo0U=+io6=z+E(%t!3u!->#8#*Dc29mr2*W z$$pPgE_0t(Pa&k(oZ2LYl17vZvHL%fwoi876iip2h5{3iSFL$o{rbrp7A(Z$Qc;i{1QOOCDxjD*k@n!(u%B^^ zY9wei2F3J%RD^LJ*1%^Py9u09lYiAabH2??B5rm66q>K&S0LDh=UWy00!E-Gfy#cR z3Ea}NI9evvm%5?!S2-8MYKkhn0DRPhPk>n{1_^o-_ihFFYTzcB5H?k?u{T6lDCbM( z&%&V4`i1&U5ff@BoUa-w2T0{5;2gtzRq$;5T@-sPAGdl2&e4nngKDA)qU@~;uYhj~ zx+>`;$O7+w6JmiEN8AI1^>>SeDq-!Yja_UN;XbWCVZ4d(aNj-LH{UiozRe4BL!4MlczOPzo6Apzc$ix2P$wtFz#nPmRXTRGEbBen*}<`}wK=LB~JU zi)0R|lYf+|Md?J^lWoE$P2Z3mVy4D>v|bVQUe;0C498x{!ip%rw1(uo!)Cs;JZZst z!KKJW;2=vkPfcnF@7E%+qW<6bcm|)UCPZFaY+j^Nm-xb!_4a3ZR0^tZN$ITQzaUQg zFQz*DPmF0>Hi!OKcx0IJaZscPyp~F89Chu`z|k$Ff03y7|M~^CJRAiCY=@@+>M0LG ze?x(7r+ylsehOZ9|8ZVaf2!|?Oq158$c1CT1w32$L^2Do%Ku|q>gZekSKroy7||j^ zs(htW+Ttzz*Bf@wAy|g;Sw+9EBGvzM!0vw$EBlXeG)15o{l%7p5d5)*5**6&{+***?IV$rP_T3lGqn3 zrku$<#>Z79g(#g)$05GTrN-|s7B@w<^WB~i0`HVf468ey_a`nMBxFdhzy9=kisjgs z{&=jrxaxredFTtdXNMzMEZh&D7va2an_HYm?)hnle_+q?EX`95`VF^O=(#j*?XT0; zmLa4aHpS{@*}>puVl+j7&Kzrs5a5~A1s;V!;Tf2FbA#jqOggIg7aAaqjr1)QN&6X9 zOe_D06E_h1Y=gFI^bqTw(#J|qhk>M5!{zdU4E6Jk!DAgnF({@ibn);`)>nh!Xc1d6U3BL@`z_tI0V`!Tp|PvnfR*WVTUmFzJcFP~N2OEcdD%6zCTqHO zUXgrBvr7*>v#3{A)x4Ll&x?pW#?fu1Clu@{k37No86nA62bE$?{>*)bWiW|nln4^Y z$VNZ5A6z{3a6bpht)|b`Rrxu%zv@vxeX!Jj{dSzvQ&r(|szOurPA6T_d!+E8%yT!S zk8}*=j#`F*EJh>Z<6J*6j%`}3z9adJZJRUA%JSoFm+s+%gV|eBdaujcWsZmDMS^;_ z8jrN6Uj{Q+R`YrG?ZyfpzNSvNjjl!=VO}w9J0IYdNz8rmn`hDPrykbQNMBa?q@4dH z?-skGhXX0N`JQE7Lbya|Pg{<4T`fs2P`#sZ&v&8gNn*t(?&V8YF6HPXUat4Q5D}`p zC*}QsWy!W)k^fPa9XZIeiylGkK=~~F><^cUK8a=;;w0vykJhkblOVs-kK1%fF0Th` zjqJ8&4eao~QjzjJ{f66ATG_9OCx5!$^LnSzxpSXX4}B?m_oG0qc3@>`VhLr&?rWV? zlV%%|22MT;gJ{%(hA-W)3)X{Xy(I&@otRXt+TvC-F?|MX@q~P*Bw02SL zu21Q8=QA67I1ChK+~1=QigDnZ))lxe<3bYTn#a@42brQgR&Dh0-b?Y!)o8|F83 zk(Kyn`r*0pMjGN`bJf=S9sSpHuG^9O02$VSO%c8kiTk^;9zDpUWz|%hHz{7P_U!L;E`7!lX`#w#VP8BUhkgvIy z-KnRpSIh+V$+XTIZMn<9RkYbyBTk4%K0iWc*{R2{q=*K)jCkVxM}##-E` z*Kze<>tFT3Ar1V(1;F>7NF7H>QyJ36;JzMv9e@h4w{U_&5ST^q58~SmjcMG^pn_i5 z?Mc7-$$k<3a^lrjH(_tv3eV8wae4u3V04NTnj=cPRv-b7GN1@$ii65A19$r z*!qNhc=knF5A)qfCHD8ZdR&0`vz|lQV8*X>9%!iaZ0}j@wS03?F~H`-bn#cx5~!N5 zz2*LSbjf-71+snBgJb6kmkALoqybjGPx}*?$RpP$Z<$YY)Qg`fUvZ2rTCi_3HOpwG z9DT`)Iu`h_(VQ>Kv33Kuypub$QwJ_NuOhWp?se$#;n|!e|O`eFroR_6pMDU z!UTV;y&-O&Mz_A!_{q@ri_5vu5vCV|ETX(%Te6tGsy&a?N@dn_F8eO=_c(Jk2ZhR4 z;L^{v;R&DyAqkt_E>vTi<;=3@1eEs;Q=j*>=6tWI9*cU3)JgJY8SNVXEH@FR<@tMQ z+}NY`ET+e2?S&sjvbsyN{yJGr6!-SoE90N9W!%0Z*v0pz6tG@i%ZJBNYnJ&WjrUbM zI9p{;U#O0}?0>iAr88Htx^(^v)NYsNOoI6ynO>Ab+=TIEO}Mr%q1RK?dnf3Tr-7xJ z>C9^0^7_L8<84g}Rx}R0b=)~Q@>H-kqW0E}@(rEpOQWS=?tL0gPSkx(&cfe)PVS~O zbcr6K>(nRbdS=yEd$K-X!)6*MM<2CkoMR*o5UNzu4ESG4{08%>{U)8eq zjp^E&%iS+8sYOfIh`KlmABrth|o}S-E-F)M^46Is$ZK8VT?v!$*J7*4ce2_ z{MmM%B}`C8OgCi1eD!h^b*duK37y_2Yj+A`X1nV=vjAnCx|ID#;u(}ZZ8-c3%lLL~ z&G1CSwrcP?8G@u5_!g`R^GIO{AFZqi-Hl>dqiB6F+N&SIUaU9Hw~Y$ow@yEN4ZD({ z)8Nh3xg1Ug{7zUc|9!xDddHYnOl9Z4}~<$ z_+E*f1yTfirr3$6xC4?r}#!xgDW8U!*|QF;KXxc%d#+=brt4LIE}$G9+_Ygtj)J&xJk{Nl^_ zXP)k}*FPf7SA99BugBN31?S(QZ!}ZlxX3Qt6<)=Px~<@QPnL&H z5*UC#1q%eFfdLlS@;z zz4FV}mX-9Nruy9^u^{`eIq3u{u`uxSxX$PBo3nSc5rKPG%{aI%1(!bQd*LO3G>qR% zj~Tf?&i8%mzF+jfe2WIIu3{mLc&K`(?u(@4*Tk8?&^iR*VymK85n>~C*)#38>xww~ z;RT7e_tY{GV%k(N8P5+5hQsCN^S2L47K*%<@X;GZ8kLlv(Dym^lor0-^~{GC3+M9A z62%Aot-ww$u6WjH%vmHp2jZ9eI(GJ*VD6-d;~DjdjNEImpB|My5^T3mJ*bp$dC9mvl0&l~0-xB)q?W*i%lXgzgpEw_ml8q8EtA}?jt*s%%%y4V=v=<$=&`?6Y7_z!&e>d(W z9$9cQNsotyG;j+(ci41ysBBviM-M{(#v$P&x0alxOD>I`SN@e_mOz=`uru-DzF8it zD`4cE`{Wt7+2RisM6S(&yQ@?K3^nf8v>_IXkI_qA^zP3|)i z7*H_J9-hux>E02tRSv(A>YfZJqHH3Blq(~SD~cy$4ZW_1rA>dioKFGk1}h+(vZBW7 zIFIAo9-ulflF*%YWpOF&&l|q!mK;>VTJZMSpwi<_=!N;XkyC-wFYo=s#{e^N=;-w&mF<`1`PF(eeOKB$WwWMV zdQ9xD?DA+T_^UZ~mkbW`WY^3Qj{*7+y zfFV3KTgwewKAVjSoG9r||BZ!CJ!KO4Qg!-Ezn~y~nq+8_Rd$a+9l=QmKI^XhSnxwU ztxx21>-iXl>)n|{dL}aX${>nRf0XC%#0n2B%=91fq1E+wt$R2gx_WH-|cXLJseqg%$d6< zqPZ4GAbdVE`091hh%R3?jBAC7$A~rW1viQK0(M5^rNLy9e7XF;D1r!^f zXScq1Cw%sh@I$?Dz(V$YLH3I7;x^Kpe5z&oy1u_YnYS@n(C*t!nO;)=;RTFDQnMQJ zeggF1JY!R2VN$A%qba8sZ1}XSgCce(crm!lmTS#C2=}(1`+ky8zkVyI*C)T9I#r3O zYHp;;c+t*7S>~-UiAjjkVSJM||1J5pH9iHYJEa~ssC64xqq&XBtWdR^G)m?Mg@nr= z(ooCyOz?WoFKEivh+18n4jnZOI z1jc6`y{>jD57C;_{IY%zCJ?r@bPVb<)ydvEcI{GN$>6rn?c$nk4c%aB8D2urvgjuD!%KP7vv0#e!6yYV0>OK$Z@2pbSBpMTQ4(8 zS@QkMx$sck>I*rut3mWj>SDk0)VO)M)f7c+uK8fnxwJ()v)WXlwRVsf z<6>r^#kqUB>Q#TB9&L&cfNhgz)ZcAB#`gsUqz~X@>7AIGf<+=b$rMuO>3U%a%rj6m zZBi1dy5rA5xDPcSfSlc+SL{Bq0hOKQ13*N9Kxdvj3j z3LdxLip~`*c=LW24l5rU+7!7Yj!v1}52*s<5CNO}_O`Mq zQ%IGoWU$okbrfo0O<)vf8|(5o2D84P7B9xURnD%LjOVYa z;(G23RKTQ6ID0`rNr&qQRXm#oj3kr=N6q_FnQXrf%KaR43{srw?&}_J-N|n+h;ahC~0Mp8fG}m@MAdMNaTf z8t2OTSSfCS@V(P7TI`d3Ei-qFW1LZnS-Q zTwxG?wZyb-ns~LlSw-}WU8}%djdoFWc={_@K*Su1yD{xr-{UL+3pNp5xcKCCZop7l zrr+L<^|qyP>GpgDL8-A0RK|@FJi%RC8Y6MBY;Z3LzMF;F0fdd{zX0}>F|S}z?0p8B zW`uF3Lb05i&Jc_vx1ex?$}w@_sKl%-`U!V0N}v7E9hDC*L#7V~mL)$aZAI@QJA}0P zDBpSLy+5`?+g!peMeL!64hH{>$zItMxkgs}ouTUdwPdbU3MjtpFuna8@Y>%#?r^>O zZKWQ0n5OM6CAK!HC6!EZ40B9$ag%Pf4@o)1SO^#*Tb>UnZZhXq2wpbbWLLLi>Q&lN zGSTB3OD(8m+zV>Erw)x9E=Yo#uzzD?I{NOrVPYb|;xY1ZXXlgt`{LoDQwHaiPgEB$ z(J@W--Xy-dr@_9Amjm8xZi(j0$`$VQH>1<}I<=8su^K(1Sh9CzneN8DLYo2&zu58@ zGgc=_Nu@K2)>nT&YLau}OSZ3D^f=LN&qn6<>2QwT&P%8$jmt+|@3bR-FK%EsrFVAR z3TtUSk*Rj-u$xD1Kq+Y$Q%7CIaYo%a>-YHTykpQk^j1C+lI5&$EeEIpeetuzvAI4b zYBxU=8fCv~aSL9cFsza$+>_+jfOa633uwPDj$DnYZLz z*>i}RY#ni)@0Zo0u4ldpF@Z>5m#*=T$X`_NsJ_>Y;H98?x=%OIq!MQyNmn0S7B?BlSnbu*Qy82IPA_wS=G@mP16sX()1_tFt^lRV(7Ll zJo)A5@7jJ9(U$l**rFlNmN2%&AkgTt-%yfkgv5efKnco}E!qmdU$7G>Wm&Tau@t6C zo6Gm5@G_&o_Qvv?2EnmNeL%qM)WC%Ah7#`>S@BR7{ny4vt%o!xCxBWV%mA~VKEnvV{VC3fE`h#%QE-Y*GG9fcvmIPwR8leXz1*d z(rZPo?Q0*Qc~`DppvX52wJ}QuzveP*q{7JhB`pPyCdCDfLaj&@hy&b4B~JFDG`qgj z?FO0Mb5?MlFV61aUSSusEgCY-VxRNRLBqE=uofX8&_E?nDk9aN_>bd&Kwv<#Op^Pk zcAhBB`L^vq@MVs24*d6oYqiNwmYYspX>sVhtcXeKzW(7k1D_Un$Z+Yl4%7vYhqQ z3ZV?-XUT>qQLFenxG4E{j|l1AQEqU$bG ztw`0_c$G{hkMx$>q`Ergxw^q563Wb_#M0AA_LA&1PfP32hIfcX|Bm`&YTC#5#`!Ed zysnx3(5-6L>}yjY7GUrA4d`2j59``x;)1p+^an&Xr^|s&+4*J^_j3&as49328cPS# zYeW~N83lKhr3U%OI1{E2aogiyCLC1O>}o#bfi3fBe!E&GiY_994n;6C_l za_dgXbJDZGb%UrX-F@c^#h{0bgRXa+V|%OY(qi&7&r!BlSqx@0_#RGJpQYvJ=?`B2 znj54?whD8(4Uw!nQdF`{&4ScELHr1$+~KpD*fS1Biy7pa>6R3Ebi;_OZ@5Qa^Ec;pyP2_6JJj zO<$5Ht!%PJw__lXQt!`xsYO|^+;sP~?0D})j zs9H<42=kt+6WWLBYxU=WCu} z-@gVgc}zfx)Gv=7@PbhVca8g5@G;n%98~*K>H}coYi8~rVgCtvGJ_@NgvZ!JbP1$s z*p#}FbBD3<&?&Syw*Pz6;R-CKrO`Mqj8V8V|HQ-j+m{7?2H};pVPfAM*m@ogY;5D4 z;!28}!)J#yV|D4Eufb4OOIB zS=djPCPH}LX}~SMi7{0q0QmuyMBhh-*sL&vgefOC2km@(dz4vd2HVw-PPoJ&Xcm11#-` zcIBm|dI63Btsz>4SFLw$wX`GDy}Mo}XO=OMq82`*Vv}(~K2y#n&u!RS&rH?wc#Pxi zfg;Q6xxA=Aq1b7Tc)L)WlMU9WFMZ4oXn!@QOa+gPPIWGxF2(kFL7fb0%!XMZd!2O? zkF;PG?7-qotA~&$m?>s6ObKB&Bg$go)Z>yMB=W23K1YsCQd0PY2! zud(J{`VOjRj#HAc?eQtMOLq|s^vbIx^epSu@5Sxb%Nfpd>CBd1yoCxcDNSEPe01ls zz?od-4M5}f1#H|KV=2fA$M~`4iNbt9U7?6aT0*gmps0wW8jH!TVk2M*-ETfMxOdep{o-f$wKc+yPu_`t+N;6$#yy3C z&vKk;gzf&fb&@JRp1C^OK5DgZP9c^!Offp@NX}x0goN0)f6ctEr)sUT(^yU|cDqmD zQPUN)R6;v*zSVdzD~EKq(6`5}9GuD-ArG%2+A4oUp7_|lZD{Ty;9!WoPwTczv@J4+ z#l-saj4w|o=gsNc#q3|cmdCROr1N5cSdL!1p4?>1Nt|HffYIyYJ_j6mdJuy7y?P2I zc_qiDpcH$bRe!%(5^LY~AQIL8)Oxqkz7}0$ZG-pe3|>P7%6QbWW^gU(vwOt5bYf3J zA+4sQF~ch`g*H8`J}Ld2VSK2=H}ixhC*^Q_mL&e9ElJ^g#Wfebj*iM0@@VgzO`cj0 zqG$ec60FzY)S=ny+D1Z7#6;Eh}ZDJ-U ziVWtA@H%iWI%x$7^lj1m`Jo?>63?*|hI#vvjBr4Ci{KH`x%8}1hCLHg17w(VzRob~ z z3`IDQM=N50)g$h-kMUDU2q_b*BJMDA)VFhq^e9xA=OGfRToSc%$+nav27eDs$?|-% zoQ^{30ljIkiL%wS6VHv6rZ5TJwzK8rGxMQL7MudP#E1x-{EOsL6FGITLa-cPRP>^n*no$1zb! zsHtMAO!HphK=dg{3nU%3piWgN)28}ph@Ic^cQmtMgeD9F_eZpXps*>TH1c{qM>MuJ zFRXn7ks{Z=2LmfVl>&EXZ$R#5RSl!zt&_htxYw>6c5^|NKD%Xj(V3fXOcjqxLgZROw`Whh!GcUY=3Nubf5sw)?U88;Hh~#W)Cpjk6kr*5$3;^gMH~U=NFditKW^WCz&?NOtJLPJ7S-Hnt8b5 zxK@GUjzEu?1V{zlhDe^=3Ux7i1G!HG4{KmMgro31SCJO%x!^5WG*g#$%-w5LxIZv| z2m2Tyo{zYMH0K#%3*IQe+Yq4oNDL^qdq>RAMoJN(v=uvR7mte1&gW@O6e%0Kwy8?! z(<{x+m7^xIDRDJa~w{*9Wi*FtPoq6rt{-L@9Q?-56cyQ-gabGuC!hZk$21DJ`7m9 zzILqMNFj}+ek;uMV<*f_FV|@2@6yR3GX74yEA-$G&P=_LstDpML3jF-KjnzgBP%)35MV zCt2OJ^vZZvm(#-U3$(DHmpp8h2AIX^x5heEEeoLC|rvjNP4iNROIV&oU*%43TgRrP1*JwuQEd?mT4hEk>`|`|$pv4~udZ&4bhZ zhx=<)24!1qr0|Eyed6wjEO$NiCcsTbb1YOi?if`{UL(bqb5frMywm$h=`i!Pic?(!TcVE!@%2%0O{!A@E%W{w$}j$v zSpOzCqu1(mppsMj(lxD&11yD**9OKk*|-YJT%tTv)h9%KZ#*|Ga#+^l;?vUB3e^F! z+2s55Se2t)o(=l%ht6EWmC1Linve94^JTViW#27OBLpeiN+^`2*;Dmp`1H!1rK}wj}+~hKr2zO&+_j zzgIB*uT@;mzZ1MGKv|9hiwj12646Hhw;In|aa8k&-78ALU^jjTP6`*khIw3KgmB>l+-^10gQ&ZKX&yUBOgs$VR!yD=SB zVK0@KU6*{&?QzG&sFbr|?dIR_CDLy>2!2EL@r9X>%w^g0LQ{1FZ#x74SUXRzKMZgG zd-fL}0d5WgTvst4B<;UgV=M{I@FTpQ+X^|T4UoaKT2e4hKOOwTAkqZDsm36(>%3Q$ z&*h&3xoiZ!G)wrF(z+zL0#DGMx1RuU2}}TPwgf);$&e3IrY^i-Vgwf;>^$}BJTAD8 zrvgGO=E7voVarXCo2@v)j!lu`H_Lv2rtQ_60-iGV@DJaa6kYzqRUG#w_^v~7CY9fg z`$QFfF@BBfn#mUA;DLe+QH%Z@q`G}aP32R$zeWRybR+ab*;e6FDxm2Gy|M6Xr3tN_ z2{^71hXRV#tMJ);eE{1oK}TVN2Z(so#=xe?wlWL89%_S!-V{0KX#5x@gx|$*UB6&9 zzU#eVM1lkFhz}bte2?PZ17u2mJmesrH;ma>0bjRqGXeh23wqTd#5^lgg3)boH?fivLXzzK{FF6UzSMWYEAE}Tgc*1OKGz+0$4_QUJe z1@ZLlxV0z9{q7?WySWy%I*77lf;tAxNI>FLwFKIj-;whOE)DuSNcF|>6sRDaBsgoV zFgyqtN!oC6_-U;oJnG9V1eyctN}z@x(`G(;161p$gQzc04h+Cr+GIlkF9X#H7Z^u7 z+Y4=JFmQIS;@0NjD*WS{BI$+#GHxMCkVRdkGJAvV1xw)nUclX6#Lp>OY>KD^)6hIA zENI{v_Ix%A&;N7j<6~hLq8;=dmN;Z5Jc1B5n}cy2gm1%j`Lp1HOb;;L&sXBm#Y><* zc|WdBMeqmp8d~6hl2P9(oB@|SWw2Cie&^4CmX64^qIo;nV5r>QD;Ni`>HLT5?FgO$ z42=9nS@iMI->H+cK>vrj;ibB3-=z{YEY1MNeQhqK8Uw-yt-I zf#VL|qwpV7exfemrbdN@w#aCf@Ei^-^y>Oy+?thtCgtB3BM=tLzls**G$Dj_dgPxV>7qi)+EfKQM3zbp}4aYuN1%)W;iYum_KB;c@2A(r{ z2omT07TpJKeR6LQNgKu_Kv4U+S%MwhN=c#atVQgY8>Ls#<03#|RJPa2*N}D;BhSao zA{;$dU^Bcbup512&|L7`I*I~i3Eu}=D~`=(RZU_(9~Z5_y3MnkI}qcnsm?aKl<*x& z*7z4lj$z(HAqe*l*HRLd)uA1wxU2JYY4G&B)lje{oBN2J#FA?hSWoUli(v1~*n!fr z)g0EfgvFASH$V^IbMmteEufSi_k;$O74)~6y{p^UKXmtpH zZ5vzPr^0-u;tn^pABs4lm;vAXQ^u5y0}M9&ibOa&B-F0a+bBD&GEFy6cshe)V)10 zVAdX9we}AkCU;@R2ilqrGvpb>_*Ms(Yq!jrW}~@8eKULf3l2S*&bD4sh%_GsiCbtf z$dH@y5(?v;j2`vU1?P#I?e30a;ofa(qd5J6>#9tO*{!zTVX`LQTI0K zPSGW2jScIDBV}1uWh&MfTXw}{LPUkfFUR?nE&(nj<(KzueT3Gc*^-N9vW{IwIZ$j@ zcpfrgHdjN73&kpiEASDS3i;DRB}1i7t;E=N=N_=@%ADiKt$MdEkeU8=9*Oyydn2=U zYwsi549z-a>f7WP_KllxjRa4r{?>*}SPNx$^surlo4)Xvb8^XgkZ-|e(Qws^xhT0O zP>_s5z&v&5DO|EHYWEC#EkB#7s&Vbu3`c{$<2m1!@NsyJ z;zJ-k3`Kl4dwtGWQ~uL2q}BQa$)?sIdhlbT9c6oHx9v=BTqzRi_U3R*@Xu@uD*cYE z^A4o)f!eK_4kWugi|*Wj(wcAY-Bb^6Rld&!yEIxwIG8iVMkE8CQy*Hvi>S{_F_$># zR)i>DiKp8htPBOyw61JatkY1^{IFr+uXAiSz5&OJT3qq(3xl(b$e1OuYw~|}cQ2cm zNF?f1{n>HRuFWP+Q*TaYc-AiKK=SNpYoDCV7mm0@5`Lu6@??=^G0Vasxo8_H@!drDuw`wYtBr2t zX#a|ieCkzGMF+)(0Os+2TF2}3OoX49L&?zex`QB7;W%-7*Vvg8&XFZFs0f_;?>MMr zM6_2mWqH}4^RIIO9wxSbV|R?LYWg5)>l-Ry#tQ>eItJ>U60-l%)w zuw>)CY?ZiERAti&$gm?P?^S0-fU)AxYFY2?#dJ&i^~(MZGre(nYgrE2j~16g^ZZ7?xI z|3pYnab@&KXxX+x%NlZb@$T$VOf7z)H;iWrZFzGWL>GSj1SAy*>|iG8u24I2A7`x| zuhE2ghV|>r|JmBC6MetUSfcs$z#`w4aV(IwG5Ly-hx^^y$E+#adYD*%90s~{- z+Y|vOvMPtWs&?_x59YMk5a*?bnfP9_r^&N5W!@Y|yWC4eOVZ)9^sIqW8yWV;%7CfR`-O486&fhDyLC57ZI9`tj+|<4 zQ3FS}oiOgS#P#DC<5?IyZbvv3--v!gSB~yBJ~-0k%mysJh}boZJod}{xNKZQM6uVL zfn3sBisibK&iI1v_(J7;-R#r(_l0q-c17~#W}-&qg7a-7iRJl6o^L_Cu}QJq6Qg4C zaf)Pdv@w*cy_QZ9jnB86sW2#Q%B@^`k#K2{HW>O$Sk8G^9$b^+tom|<3&oF6H3}EW zDhtDM3*KaK-#;>%6m94p8O9M^wC9?Z0a%X2VQV6XmiAXs%okow} zgOh7AqCoa78=XnbM?THx19MVMykdNkqf7!3O;7*E7=vw@j8_tHtAYIwJ$w%9*_W_~ z51Ys$JCD6%#QQs?h>aQfkM?{UC{kC6Bihy8a*TBys&JnevT1UB>fy%C$h%US;XkFN z*m$1A%c(87yy}1cU4UboX_(ru|M?;@n+$I&*usxOOS{}pjTCrvC$84KoxX){43d5W zG5B$O2mKygkuS>WoGNI;h)4IT(3&L&A~4&rR&A5wBdfT0l;m^aiTP}0ts2fq3JbMc zv%)xq1>2^R5EACAi}!vJxJzYRTS;y8giFLi;NXTpp-Z@&a&?b)%m&dz!$;kL)SX*q zR$ohY9uL*LcDD8u^=@M8P)6x_3VKHuB2=3Z-{Pan^1rBZw8j2$#;iTbz?NtuskrF= z$P7uqQ>Sv_yRKbpJ`NQzsf#B>+feZ+g-wwF^q2`Zm!sT3g34Ql!jHmuQ0?jiVUkRf(ILGeSOcrUHP1T6_#+Y40q2SFbF zRJG<^&Qj~-;a89n;}KtdijXk5WAHgYY6=9LmXZ$kF9mC8N9{*iF=0FAZ;*?1ii&!1 z#KUW&rfx6DnB#dr{CzmXT9RHajW4g7x`khH!i zXBo?~SGbZ;I@_;}{-QI{SX%fYM>F##TZ?PU_FIVM92jiJiDtKPG15(2rg{W0mnse_ zHkQ(+v~7venf#vo4AT@!_r$K8-jkDi-Sb=hfd8yfr|0c~io|z;kup_~+bPyoKX+JC zOaD97sj3rY>VtqH*5eKt15*>9Fw5{St!c<*wTn4SRlW$YC&#rhN#zxmb^G>V#NDOw zxd$>>hTh%^jtoaVeSJM^FRcMciU;GLVVf^np%#5P1SY+8Nr7*MiOE@N)pwujTv!QW z@5F8lwkaQC&%{4RVmMEK0HN*Qo1i3S%6`69us_4p=|c@g!n_Phv{KhKc=q;bOyQaQ z6K5pQN{XifI!*>kz{+euIi7<;(8V&FqLg#Lm+JGjF2GNDPR*BQ&Uq`AqWtb>9k~A0 znPNLiTF6wZO}BKpecWT{Tj_a1-L=x{4-3&d`6^Hz8_)eDsEAmC%r6ZaZ;c+zF>j?L zZ|J667^-Q)ny^&vArC)`ROHP3QJZog@XN?JTpDZDD*TB7mR+2Vn6ym{V_%!u z$u|+0#b*#l|o(_WAc8o(9l4yTb2*VlQ>3e<|$Fj7R%3?dDQ-?+g$H@b!ItDgws zj@0>NT~4HMN*J8I+HRT4-44gsg5jkfC>Q`lsvFeKm#)NANf8j5$E__BO9i!JdWtjz zH*{UiiTK!#XmY&Ft42TAt5!_@cKEbqo|=Oa!e7kBJ}2AisjwYf-G*+Db|6xWRRxvd zM$TLc2mfXX#utH1fut$DdIZl{`k61!zLCP*!?{sbMq?Maa7L4V08d8Y|^YnaEPUi3`gqO>s-#uPutXGmu$kLQhyqD0M`t=Y;85Me1vkt z+%P3RtUZhuvA`@Y+|D~^_2K&?S0t_>$0{lT13zi5qW1ujkzkFDFEiTdFHG! z@XQbW^Nji+cd~_fU6VS|!kmdMG~i?&cFuRfiFVQIT41dPEE@``tYW63o`89 zbm?s40PUiXqU?3fX17bONnyYM$?uF-a^`k&lom&nYs03uq4q|<-#zEn?Z#IYJYI}A zW_21my)8M-4NC9KGTv9uzS+7Y9hiS0+twd`4kWfuVxO(k^7k1y^!Kuq+)X)G8#8p9 zWd<+gQdAP#SR9wE8T-Al9o|!9s;`2q7Rn!eqOG_gqs*aYhS^Urh|5SW$tk16 zRHhR0(Ah(HWfL2D*fLkhtFZd1%YFwweckO@Rd~t8dCHxMp;pGlA@3a>lo`=fDhQe? zH!BKz=nBT-SUIK`Ka4}_q{d)=MqQwa1_n+EL>c)>Y!uguZBxXumR#yEHgKo!otK7I zI=mFvqHQ>(PxEe*e>6sQCTc@Xs@{dzeP?6W)N>qB<0`R4*OyIB{#ia@kr9SwtD(6% zvU>fruHh8@iz8wc+A^goN$-fN1}*`+lEYKbK8sZ{c6Kcn!Ypq&N$Ia3y@C?Sc55dT z0Y<1myowwePfC^645__peajM4@1^viJQBeWh|}EM8vXN?`fGZ{Z7e*x4t$)|Iq%}8 zk*4GN$$g?j>Era{Ltk>}vhn!{>G*cX`kJ=UvUqZLdBeOzro~vGZ?B4Plw4Rg%uMg9 z(N5&P>$HBNnI=X$<>*mUs?O8LD>{K&F8m`X^EDFSeg_B%xCK!%kjg$(^V2};{clHq zy0RsOL+AeiaIeZ4P*wcv;FT@g{zGN&KPLeHr#@|t{#SxA|DXoWG*-gBCHVn(5iSKV zDXX#%qQUfBAI8}g$bG@330G@?OEgSAeO#%SSt3a43&2#^D48V=(c-#WM z6a1WY7?lqyGnc`m?nUCpWFb$0{%+wMc^z=a2Z(^91=?G96L{QR)~F!M!)jBc91c^u z$^RU@2tt;Z00Q<6FuCL(B4hpy-b1ZER0deym(jntd3^eQp=va7o&oSvnIL@s1vHkC zRu9N=D7@@Vkx%$h$`2%f^s3JTl&t32oi#c?uahr5Dr|#&1dK5@Zn77spE-0EIN@eB zAQxO$DQ8L8*$j-woX&CZJ3NXb0C_|pNd4k4M-%{?n5kiZ6dwGA0n<2}NE-A5I1pQ1 z*n>>k6!GkoS41S@zPyVD)hA<}!Vj2@a**ntbA40fb{QC&zkbq9YRJNMO){xG-P|o; z6jYSqZvMFCG{o3_Vf)e;=#ne&U8lpS>tdTCOvt9Ww{ZOFvoA!19a_h-+egOh)ETAYuO)U;`li?7r?`O zoEHJ`PZp)QHR6`!3G5I2XW`dp_<7)VvH)_3gk@0t#&^2VE(8Za=(SA|;S2Xstxb_o z{9nX8q`vMsT(_P#s4g%j2w(ev!H$D3`{8O40s=_M7H0EZE9P*ofu)-Nj1yD}OcGFE z79r5-iwJ4zYM%>xCT#RzU2*c5V$U7I4McxLPc>L)f8AVRZm3vxkIUs@pxuuAj6XwAFQ%zM26 z9-XcX5y7~h{)-gIF}OqFTQ=`AVuRznDPr|3I_V*nM`=u@DhDuQ&%L?9?B z9?ygc=sSc+gmC=2BWOHgN>@POgIm*(nhN0)u8S~Rz`e0S5t4&W*8V>Wa|m2f(t8uY z(sYgm&+XhD1=Ho{WBAfb!ZyrDFuSMOM6d|1-V{W(3C&wT90W#ya}ECDsIq+5rwI8A zi?|zLsP+SvD*vT>+Qoo<3MQoDz8jE6IC7m+&VsC4e!^E?$MxazxUNThu=T3%3yP>~ zKMC^vI|X0s1z5Z=xAgWSG(S&x0z62ZMi+Q-X1OXLLOy+z{(>OD2?4Jt$yjz2N&Te0 z!KAJ>lD_=F^NY;ETXQ8?aAEBy8(iSpiaGS@pTosrp!58gCOBAciu8>XY}_9P&b$=i zJWfqy1_=iCKz1B%VJn!TSs6fW@Zmqj2k@UF5C}6q1!4r|ZB*`-ArSFOa}$|0KfNj= zQuEWl5X<;aJ@KDA_RpVin;-r@UH?8^|2|#+K3)HhI$fh(aV;L3BEbR^;f`UR4#=4L z{=l!;BS2Xj>H(}XvG$k%gYz8grPvzOE+7Cl46N4PZ(dtL9TnvbtJ9$4uJ4ndyczQa za!{Cc{_@7SwUwjQC7t@;;A+v`D7Gyu!PuCyzCeABHTiGsy?0QP+uJsZ+pTP+iGXyn z73p0;ib%F7AVFFvp$HKXY0^Og#6+bx*-{mOs1QS_(nJu55Xu%*q&EeU2uPDeBt}v^ zYrpfo=llKUoA1ndXU=(NzM1`>l4qW>)_T@+-`9Oz*G+}oB?aJSbeNAAldbd-peaDJ zZ0JLa{As38hRM(?%ozrfsRBBF$=9+HoETqne5sHvYV;or2)SN5 z!21ly(J88o)a2$paZgc_j={)1-U~A*NA$V+pzy%h=k_(WzLg#AF*V~v8Dp1=gLS#( zPKT>0#M!9wgT^>?HscQE3dU$}K0 z0<>WWM}(o0Y<4V&r^8seqlG?7A9iDXe9Q`_-MEdVXbGHenKF%`Cy-~iGMd+O_N|+A zmZjFNo@80U?=Dk4Z#u4v>Qvs0Ofio1rMvr~NAWU;vpt-4OP&ouZV`+lVW=9JK6ATJ z-Ewz$?~6!vnA2b~BU1>;KcuOkqE}vMUK^+$1@9Yd)-~#Xw03T7x;QtM`U-LW2<_*qn#K>TIvH7)JM$})}RtR z2uhNf)(X~^@I3jCHc&rV2Zl1gPj>7xOsCR2zdWpDU0b)E&`HK`_Pr|za?z8%8Rlzm zSY9+S(4Se2KECF&<`YQMRj4ft?*p`i7@zPt8U?!oc1)`N`ba>=l2R?JVCGL1p#o z;F+G2lMa=ILVcF-{^dijMiTRu3yAFNE?0BI@&v3$ox%!Crc1Q+7h2Hm*apOAFxvq$ zVI}~y_aAL2xQ!+PN1u^HRx?d_6yVLrebV7hUC1mnDQvpZ%G*z0$u>1epr^vR`-RlGL#5j^EApbNm@9y*Yj*)u#M z9_=)WwxMr#Ys8kXJqvbNjYr<)7=QY_(sye3QKEBiSmWVoAkQD2b58%%_$q zgcqu(9EtXOs|3&XzJ6ZaAqwtR9L*5cODsQb5$BBG@yLL*^-@>vES1!HNF0$A%}lnq z`WUV-ETrn1b^Y_ZT7kAIBF+XK$9;zN$Ig@^0O1d#hvGH^d~<%-W1_>5;w8`nwhB-! zuQxqId3+rdB!_YYIcMlw88IJqy65F33rTYrtqFZLR3xz8G~&{lcR%DVCDm-YmX}*-`Fk|`Y%0N$4YnqU zG8oQdTbN^}E&i=S7&wa`lBMcRWK>gHHzpWX1G8JNum@3AG?7#w|CB+MJ9v5ao7k>! zeg~1he{VHTh!xwfus36Q$dmqwC!N|oGKWf}+Er5*js!Hi-)yyiN&i@|)be)eIc~x` zOZUA~t+s`I@>leCiJGv~+Pj2;p~GuhiaRDTdZT!onZcOOib}O8t1`bAmK6-qo;FvT zG={sSFiedsYF-k>f7TKPi|luHgGDVxKeirYZha{6k!C^OWpcm`T5cNpsk=v9rzT~1ZZPv|lw{BK{_| zeXg*hgcc+v=W1-!qB==PNxT7;r1AD-^ylrqL47XaIWb5#f7qdjImHV0r4AovrFt?3 zZg-ppvOF+=5RSff6JY8_Dx5R6wnN<%f$El^uvlKUj&H^Xou(Nsx+!DTPY0Gvv*Ov(`*VvmK0lB>*fOz`Deo$;j=zFqZA{wN_2_X`wuY+Z@zpRnCI4 zQ5cW~%X6&iQTCMjz9Av2OZ4w|uCX3Q0acg~YwVTlEPsQBr(A;))f&&38G--T(HYFfl+&N%fL)gW4eO6Y^(B@~lyD?$0h zclygTc=)I9GAkjG4=f`}K941?6sq9XzDSm63<@8tP_r|APc$<5b2;hwr?&dRi4T}K z#ui6{#splG9^7HHJ3TeMBbWg-r}Iek^>ep5dh{aNzPdbWK$j9~eyZEVFb(uc??g~G zTT{>oO45s)HV()`s4)0MbEl>J$&GlgiOI*+0nfjGDT!OVOffKV?4yOhFpC?@E#S|H z&Gu;as*bEu2WlVk>||D)nz@V8K6UJV<%lKX&UIxK@gFME=D|Xc_>v%Ef_7LC1r{0U zSg|zh>PWB)g$AqvKs&V?CxSMj+xL+~S=}5%I-3ef?~q*FI><`wEt6q{rSit8_Gx8D zpVM)P`ke^aq`s_t82LnS*YVEAk)$^&tSJSi;Pk zr+eb}x5mW8QHRA=<5P;t1B%hH9!n$5g&Tp@m-me=N%zA-C5s6e<#%w$A|v_`<(d{U zLRvQ~K3hyYA4$x@hX$wxtiCo0y7Y|s;<}}_+;)=jvlp;G@oDv2Fz8DHydv-;Cu%v{ zEY3V4_1(M%w+rZ*al&Xp24Z8W{ximSv?GG;k2c{JVw62t>Fj?U50uLvV z-+i92nT_Pg%&tnjm8BKz40-th!fIt~D&b&DTWPJ1zg@-EAI62xZJS5BQJKsgZf=Jf z<39SADV|>7Dd^$6zA^=1gBGN(Bk6^*InPR^&IRb0YS6cPaffRz`00LRRq0MlU=8pYTca(`n2Oe{GVV-rn(&#>_KRHz={FW;fIo?J1i)` zRf#tFt+M0tl*i4FrEimagRz2ze^nB^58{DM0Gu5B(w6HBiV)=}^z8f4zYMZ;1IZ`axs#WYYkc&|vw&e{U@H7m?_ZP)yNfw}iS zU=TPNW=a4PR3HG_aqfG5n2E@vNBbWA|6>C>@(;%I561Hk#`6!x^M4e^gB68Jw=B#~ zQ3xCf4dyM@X|^{WA__q3JcT*Uo`-c$T) z{l@K#Ub z$P!irHRPS?-bRPcef!jUS!AP}cVKuiEDa|F@*Vr%SU})8)%Wmatx6yG5Rxbd@@Ora zEn9t`d51r8>=<4vJ#MDx!p0$myt+R6E>@(P718ma<76z80HGh5?k_8)>FhRhg7 z9f@rY=vZf4!o&hs&a)!e)@Wz{=*ObygKU>z^IaA)a|z6Al)yvu0Fmf!dWr;Sw05!@+}C^a4N7)!6|&TFvAu{lbEXgKK% zalyjqWQ@zX6N8LPb{Oo;-BZbJ-y!0&(I^5b{7;|J9k4BZY^t z9n84Gm>oBopW4`^#_r1$tHYHxv+C11VX2`7hA_e@DI=#LUDn0t1q?T|a##_yP+4Op z%J0X;^$GG>OZbXg&BZE5ysKH?7(vG6>C?8={xdlnpd5@~k{O5$pq(gP5CYw1_#0Uk zi}fs({^Yu0T3ALk1M+HK^F780HAG(6hd#?TMH|{Dd5EO61aC>M&dW}5pj0m#s!m2| z*Rg;y;jZ^883R2xW^YtStORLHz-#3~{g9@$((mh?wEIT!x0jv5-<%A(ujad?elP2> zro!Peh?t;$3C;5mLtIHcJi6-sfQX!er(ltYPK^WMK1IVya&Q%YdvH?S4I1;-mRkL; zQwouY-yrs92dd529EDVq{OuE=615mz%rv$Ics}j-Br^=Dt!bW=*lujSQFwuKWI{QGRc2LI29>1CBCG8;iU9mm3-WC2PBBD9{)?P3>uSmzTU_0-RBi)Ou zlxJ=i&-)r%tF@y_hV&eAlq}b%OGB`dd=1T0{DP%;4Tv;KmKn`Tt>leEn^p#pi6pum zM3_gOL^iD?Vom^GJx?7;%y5^)I4M9j=LqNYBpON)x#eL|^vV|7TWnTO(PvD6%rKA> z!#JfSGe({&*2>zoOqwcct(UY_&m#}$)=LlKO&*-5=SNNqN&D)BiQTSEA-ccQPLaMG z=o37Kha2GGPQ6v}4lc?T7Iqz~?}__1fAY1cUh%u%s{HH)iYy`MK^UC4ez9S+)?yrQ z;+W)PJ>F3r`Ntf~ifw{6pCW|N#t-`DgBKtfY7k)~b#^sZ{68&+4XsQ|bT7$IrpC9NCep$QEqR@GY5~ zO(lwm6lMrQf494piatsMh>-wen7kkk9PC1C8H(iDZEzq^eMg_Mc~y48E8Yvz`(5T# zQ$&40Y#C(KU=L+8@?)-)E_vIjFwnZ&Xc;(l0lW$}}F&R?pv7{Tspz zifniBQ4e`{*sxyJnY{Fc;q+?a4GmtL7*pl=MzP4&rCfPJ%`WTfiuL;vViX%?>kJdi z;u&yDNmp`a6<_`269l!`O%>`ecvmS~qS&5B7NBkHgT6OAfmR&jU>Dmuj4Qo)qe#)3 z4tx4GB|D{4`Ei(k!V`}9`$G2lOwYS-uSW|Q!|lc90`7WDm{fP1CV1y1H8>;pTy+;| zTyp-14&6@>uCbF=aqP!w>PivK%d0oA(fJxtFyX5jFg(J=;C88Cw(8(?*F=T??omW4jlGx|CV=eb7TYLi`lL`Tg|e0!o;$CmO*@uCeu_ z+BXdm`ktU-wD0u6c8p!n=TK|kPIlmA%>yl^cBe0DYnQ3$?!I+5K?M~ksXG&rnYl!- z*9d0IrjfbniN`V}tax=JbyQZle)m*nN!8^*YJ?H8deK|qfJ?odu8T&E{c%MLqGR-` zr9)+c{iZdYpP9$4tc3;+%dKMhIOi5UfAVD`=@;MnGqLpVTA3eU46->Zdp?aMow-*-Go~`Sv8=`z`9;Y;Qj$hvi(1~964q~~@fqFu6JgQ{s zPDE@@b1K`+E4%;7$qPfphjPmf7ar5aDPm@0cWbIDw}-}ht|&?-6}YH}7D}eCIwK`D z8PjsY5dJ|LCSZzaNze#1veb(y&<<9ZVSDlZLLN3dgIci}F_rdx$_nf8AaTT6nPc4h zp&kAPUvNT2i5*dlM(FK$e==LG`C)LyRNc3xFpB+Qui=Oaw~y}Ao`HwY=|B50KW!1+ z@Fx)|*(;Uy)wvM(sNYp^iMgxXC2LfjQz}CepD=kA;i6eNn@WJW0#XW^RJ8#t1@`W>?`O{ZqiQ9Ed;LNZVSDY zn%ZM8jCm`Ql8OJwxy;f@SVU%;DY9f4VU&eT)3X$4vdL6x2Lc;gvFd?K?UEmcCb!_y zO(bt?jfVJJjl~FPs3bbPX#U19s&sk#tnI^5qbgJD!sXFdEfVRwjRU6M+Il7any1So zlnBIdeu7f4L1jK46VT!PVBItV|yV|X0mIOM|koKyT_Dln?9%@Izsg3xmUYO z^MpnFLns3I@VeO{#*K%7-6An_>9fMoiIDOa4bRo0Ry3acxIY-+`AdX?UTY z`C+M_q7Lt)xj5LK!%c;eFg$Sh_2Z#>rkXw!F&_zf}>#Zc+57ez(&DkyO4pn;HCETt$N8?3XJ zBgiVE55QOiKeiTHgPw&G)Q6@*_I0VU>b^o#ckM{}@ya-cOmNcVx8RBAjDhUv&XXRa zrstxS?W#sA@I|22c=bL{$>Qa<(t{^Hm1<%FVNKodm-A`rFD^BTM`m?3;^AUMn|;Pk z1s=9ND2J7$0az2*V8Y!oh(ebV&vJOmFD;NdUpd*DRq9C237Y+W z*EHPJ0@F^&(C>7Dohq7_8r#u|3^8qvN2b)Ht)2uj+xz+)a|y|Vm2bzGFD?zGHyGvk z+4hvEEkCe3Rwj0@Mbcwuoh$?=Xa>P5f?SEejYX8S>4{f4_by9Zt^oS2@Q*UuQFayh z!sYSxD?cXR2W1^fnpVh@VQ=?+oJ|6kcfd z!VPS(EhG-7r-3P9LO}{5;~g446@_Cl@~-5* z*t%Wsg37>r*K8S%97Bk*O5eGtC3m%=)XKH&NSBsVeTsA42laDe2Y1d?1diinVgmb1 z-78@?jqWEH^o~B7KjuCmca=V3?ZkBIzx?c5uw(78!Io2j`m=IXWAhpUEb`NxN{dP! z_@s!}fIu&SF4;@EgKJ*t$0jbwkgh)f_XrULzBBxrv6Qe>=$=h|>=E>N`arMSRaOf< zwYN-&^0eYE+s#0i2YQHjB~GQdnu8A4ruWF0L-I3T*Z{eJ`L-ujSx`QoCJqnjO0x zd!1|HwZ;?K`>Z~a;XuDxaxroxl4#NI)|v^b56y7zcXt&J@XMew?zh7ox|H4ADxFHy z*#6%7OCGguYA|F|{;hxQ`SKqinXrF9?yD-Ri1#WU$IG?(Ltgc>*_~TQZW%4e|K$5? zKNPsS0J<7n=+rIUf477EU(WO|JkbB$Zz4ZO|G~xn!NvZ;#s0y?{*S`Nj`A)3d4OM0 z(a14#o$}nictp*~e9`#TR^g<~}{+(UYUq z_x8=mIq@L(4||FqywlCh|5Pqx) zO%=^E^|IFHkf7Q|u`5}c6)U}9E;e~GKR2s%_z%dDE-6UL3QdHgDe_x=4}JvFn@Uw& zpN+7T$ z;oV~(;4BJ#bbvBXs{Z&H5QOT`2)so_kj$479f;FN?Q(GTW>uhG;BL}x-|g&BkWk4E zm0D^9C%At~%Z(^dwXsIzoRLRt9ZfLq+n`7%G?k9~KqGCfjqd7{87K)1>f@zc8pCp< zT8|&EnJpQALJ`RTbO`6Xw$3`MxP zSTTU?U5eu-p{;=GK)e1A`Og1*uP~ATj>V8Zjbh}mGx7BAyvG=mdW0(Q!ohLlvAfRp zn_-VyIi+St0W}>0nB>)2`Md;-GWIVk-T}bExr(5~@|s0@i+6~dfErT|40pywwnF83 z{5in=Cqjq!3~d^-upL8cUE$_|*#>e0OH?i%`;r13NCGBkm*aq}Atzv^~Uodk#?aM;~lWVZX*g==MnVA*c_QYaHML zvX0mRYz}yqiYAH=u)wJvAh8u#Q=l9?O&;O_>CphltN3|;74-s-9c-dY1y2sby$R}; zPt|d@IN&bys-CU+RfiiZcR9RE4cw9*3tY<|!aj>#YXVGtGIy<>8wZXjsFy7RWr#`w z09iNY!-3?Z1a&hI|C4X<0_mCvTat9uo(tfS)Pi?F_M!o3N+e#{y&rZuJjeCkdbS9| z=_j9O&9@&piwm4TbLJ9wyzCIWs_FOuNWt#bgSYD?_J=iq>A9KU z@poT`T^i;NLO^G198`n&lTVeB3M#g{(A}M!JRVRal=8+&Of%jPau>lQE$4pgi7-Zgs*bQ4S?M2iaUUO<+kQ!DD3_4N~3}j$VBJ@); zBZoIw2gZ9Uj;t~o3vT>R(wrFY74T>0RY5W&z(hrX$T~*|jN~y)6u4NP07ZqHf&xQT z>n{W`K!x!xCQysGrTpDkt_w*j_z7|=nzY`BW8r7U5g^}-7j~$S-ALk^AM|T1Yk>EcjtF)qVSq4Q1|&E6<$(LU1AMBEVAo1m zh)X~_6<)s%7SiY%E^is~HCsUi_zxOpVRxq@jJfqwZy~gBhkq4*?0Ewo{_9~nmYWrY zT{ED9rWMF9hU{8S<=zsyKPST*=tiC$DFQE=!+U<4|CFKu4aXkoV$tP9&C9PIKYas? z|1>)EJ9wM0JFmLlBg$?SL+jUiIs@65tAM(hhUsnCoA)^)m{({q%7mRg2q2r_QBwt z7hH(%Ms3v{oiF*rZB^ezmovY_&R@m&7c8qhd}h7$1$lBvB_Quw>~+n9Dg7T9B@8Dn z78%9+6TBMhTgWlUzdSad!(R6SiP9NIN1Ky+2dk=1yy;!s=lJ^gqOhHAz@8^=nU+s3 z48ibJhxPXJUidP-iHUPW&GNmGd082!>#AesJcB2KpLU+xncH1|^z;HbE%Q-9L8-5% zVWZlf5|w#%{H<$y`JVAhi^qPQ_@PzDiw}a(2Qo457ZlL)Yy*xm!>V6LZZ(V7TWdXzT1hgHG_!hVmH`{(iZ0lkA|qgQ{^|xRRaDhdTAr-!ZEt8ts`wPG1~1@ zMXgrdDB%{UMLr#kT(a%|ZnAX%eQu27Q_r!Y1G5!Q;1gT=aNiM(#CQu-oaO06=Ot4; zGZzFtd#igPU$uAcDiJa}jQ7=b3KzxJu7)xT=YJpD$yPAej$W;Lyfu;?^QGUVLEKs8 z%LJTk>y-b|Fi_n^cKNXjyWmmD;UGsG7Q}&x31=CmR189JD;ACjG1EU%xNjwM0F-Vs$+7QLfryx4F~arx2Fi z-DEZO@U#5o*Z&=2@c#@dK>us)c128BZPcN# zU(hFj%siAMLdB+s1I4^YS>1vNN18Ip@Xz2}VHR#4H&b|TY6QZv+1WyZ?%m1xnW@2# zDi8JCd}A-Iy&X%{4;-w?U#&D(-zsxal(K$lcA+Q4E=H4^uc%&#E;K0LXL2vA$}-EN z=!~`Z$epB;Z+9C*)%tZk#`7H5R-d*~HT7|6W`a{)z~%$mf+go=CHQlI2Na*;#wnJSf2^}=>=rmOp(Jg}t;*Dkv8U*Clu0-f1R#sLuubQ7Aj}RPI z4HnXH5@yPnVKZP@#Zc^qh?*uaawvp!j4*Gc%amS?+CO}Ofy~70Vfix&?RniqDw00{ zeh%+}_F?Fdb5fgFV{n%H(`m9+e|d%XDzn|Mmc!gN8TW8$eB=2EuelH1a4jRV0OiAY z@rHDy>ZZHZ>|k?vfxM=jVs2El#08TRRc%s{M8`MJ8Xgu8jJ#Q0%I&q9wyTpA$moHc zsQD1spWA2d5{`;{>*VqyC?P01x90Ji&x&sJsZG%;-emFcD{K}5yBF=wkY(O(d2f2! zho0JR(%LJaF?<9K;g)a|eHbEemZ5N_N_O;_lT_C>dLC6Ll{y{+6`EP7iDwp7qKdT) zFEBJ)tA}mX-C!?*#?Q1|a^=@gR<-Dodz!}& zR;z*1Ppx|0Au_e`wM$9-ViI>}~4+=4)Css6ycL?_Cg9 z(84|g*))!?=I(NN#mrm5Mq7$>iK+u_}k$L^)l$6;MUW3I8*{+@4dUMneV|NlMw7Q>s4{Nz3=mw7W5wd%-w?N}RG`lLiyuGh+oZ_utup4oUMN36x`U_uu zN5HH50mp^sy8h(bmH=r0h-?1b1$U@YhGYgoc+c~PF)u*``2s|M;2P7n*#bbzI%HlW z$?iJML6NvhI02C#2Hv2=I{m$$#I@{KK+;!uP!Rq>ndiFzL4VzbBU7tFUj+Z!gmU$q zcd!5CV`27TXZ-JB*I&@~CNu0g4{~}SY$LFBfs?!7fMFzL%LTjs7Roht{(fTk$LS9U zkZo??!tPYi$m_N`Jh%m07dzd_K89U~a!l)Dfm^8b5;e*}SJA*}GS7}% z0QUSKE&*f*zq5i7_FS4MPX?0z?k)(s^9a;fZpPn@$tSR)5U)9qm)Plm!@Wu1^fvL! z*e?!0`Q)(Qe*EyV|6wHv<)(r`o~l9P=)f3riwW|WhVM)dFxNl%G7UQIzjr{n2|%y@ z98wvf!YL&QK{qU@vAhQqbL3sf*K?#RVeI21aq#G78jd!s`=s3;5bnu~Xb>YR5wfsM zm+%$d%XiXddEn~#$-HSC^A~O#bjN|?0H&x+l7kB48jEe0Ksoo32cR1#yOG>rqktdP zWb9Yp1#!~GXEQM5Yqc!0*WsUh3EluA z;~JpMAUbJxnri=e8O*>CSdBNqipyk!b9^=v2PI0wNk4r_wH1o!hR(nNlogx-NN z!?De=yhqTcx8N)tKqps!2&|U^dnYw;by7c+c@~ubGNJi1X1&1s6)($l1c!hdY61qp zD?RNiMeJnkB?Pa?nS;{>^Af;-^13^G!TNZaKD>$P3i!zL7%xfPK1!Rf=?Tg@cyF(5Jb_)jBb)xCf5pszKNdw8d8_T2N& zp+I`mj|SM(RI8&5-2>j(Djav;d&}YRyzIX4={n8VEk(W`JmGGdPMMyB4R!Zw_jxYv zL+p~Z>6jL<5U0@t3Bmdd@h_!cYNldBzeCG(l0YUx+V~cOWhp9`R-+`{C-Z1i8=JIj zUuGWg636EFBXi=Prn678#Hf}@W`=;urJ$oQ6HsRQ;%3w!GaIIc3FW+QWI5# z1>S4XAdEDNz*a<`8ue%FBi!ne1Q3h`X|Wn9&>=us&$jaHct+nML-p_=ly09O-)B5!w9`=n@&7Vv# zY08T%4WLxQ3BxLza*89y@~C!)a!*}?)dj0e){TfP567nsXswL?D{5m|*NG!#%FR{n zj*fRq>aUJ2Zx4e$aPf40GH!;zJ8+H-V})5U_7!ak0U_i9g_+9OOMO$O$cP!pM%ypq z(u$yx`lQ1s{@(Agvz;SdEOfg;SvG!^C)$yYK2u>)zS=6|mz+IoI<}IM^bB=Zj_mZX zz|>yA!L1I}k91>x*z05vSc*J|EWHwZ-F41(lBpwvdWFI+@*c!K3E&wtj&5CFoa#6u+@$oh zX7)$}vsA;OYb->uqp9tdwLreOS*!avL8hebl<}C;rT5A1*B%HNAnwhdC{Y`)B5td^ zHcoIedMA3X>{yJVm5@w~lKt%R{rH$TS$L^L;fjPGfDd2vN8x`PZ)#p`8!IR`j>5n{ zFEoXbMN$^{%#=9S>0v3RXP=@ySrD*E-Fe?7!I5NClxZ?13H6+lEW--SoEaZ0%$MHO z4V;N~Mn-r?us!_LFwxIb7{TXRtYCQa#8zmGh7B=E@{p^O+8}A)pdGx>FRRxlN_`By zqj%r!O4EmS-9G2y{3@rrB6}0dC5#P3u2fkS))3(F{>_vWeD7M4vv11KGq)1CV?|9p zR1LowGarw-BgM_C&EwdYh0EnDZ6!g1DVhS;m{1(c4kJ$bbX!GWWUdi>pyUSM1yBob2?NtjD$2qQgwaaVY%mQ;lEfU*uk77&rEr3~KoX zxiT}F{mk)F_JAVvh+EjAlZ<^X&03172HW?(#=NlRD@?oxV>jL>SwSb(SYQ zhjO~mT(FqOykwcrxs3k>--UNNR2p<0M|vaodUTq+|(FggJ-`HfMFyp|&jZ z*_^qqeL+b2CCEOMCwbPs=PBq1w@vBW>JdUL?tehDMVw^?`_;2DX#KO0?t0mE#O%Z; znVY1v<8)qq1uMs)?Y)wWaQ)SSRmWr)ZtYFOp{ga3o$kmhl=wU{h2CMUsif&*<_2@X zL$0ol^r;Q%(_qOlN|L`+_m?P@ay_CSRbOU>R;(oDb8(aEw6F0}?fF;N?9`iScFyar zWIO#O&sjvpj2oNJNh7S8JUEzRs?*Jzlnc9l?HWFCNU9xTxD4FtUp*3ApxR z)>*vtqx#9Auh*UM(x-=KG%Zy`;U%)+PuJ|k$1k^CJpQENZNIwd5AXdIKM*=+T<*S8 zO|U*b8zV0xe$<#0A0!$TSunGUgEW4By}NB+kxzA_zCjCoGD3VB zguK_1uhn)=7>6u}q#$}7yySyyyt<6GZklcXuw@ytWjSJ#W*`pgmgTtDfo4kWIa$E9 z8kuEMl=8QGwb&31ns#J&tFL~~8g9Rt5&C={WvX|mvg3RsE0xk$pVq;@c)zAYy1IzI zS@__wXTW5kZs7O!I#rwK(;x`x6ns+k zgYlm2vxryp>gr^&O~H8YwHF5^^iXXcU$TEgbdQ$lOsndIc$jiBTI{guthhQmbK z7$RvcTD}(I*PN1i;()P2n(ZutH+?}tzx(s*Ae0dK~` zqRLF3pTZxfPlP-5PhjPbV7`GBqZi79eBcet^JR87vQ{}xK2@kXGVK#oqE7h*t0j`+t)Af0FzE&y#yTMN@^di;45M zWE;$o1(>r*i|AWfhq`XvHQMY0bv`cU57|S9%vFy*xIKG9x@&vy%tNT&TEY$}dvy7L zYMRbhp|oG>`g-IlXLT#0)s6YGm|SChEE!Frkp6-7y zJYe)n1qi2Y!%0uw?9?-1L{2zJ)t4nc35)BTr`C zx}j|uexoj`IRsQ=cRA#*pwh(hF|Ycf{^&@^+gtmWU&%g{_v8~S3+LPTy9#w5i=6uF zU(lTYX2JJ=sej$SRJH!GO#R0)b>|<;)PF2f|NAUc`43Q$8A!nn$$C^mPnRAmd(r*? zpm&a0g;4B;SVevFkoPTHVjpZ{g_3kp304yZvR>yriaptx398hwkUFm~D=9N)i=8ba z?0~EzsA7PR&^z+57Ij$2psqtwOmDYNl?&mEW3A?3R^Zx-PL-8vNwDj~lDule zY@&jPW3sCGq2^0V#&1~G)cV;(P&&AC5ADM82faRp6RF^9)pe9xjHI@bk;yzEx?gRuo< zHt`PWeAm%vvQ|aSVj{19JR{qvr{t*AjPYPp;Q&WMrx)2gd>ioHkQpRVP;c7GZ0GOJ zB&8_qVSMimlVLpxh<_?ukOP% z3cIQC$jzeq?LKauf%2+r_d7&1=H)FQ$ZaeLq(;dJ%jlNO7AecaYrSLg6*sG3;>4ct zq$=a!l!|JT{M-zpx>0&nUPn2sKu$ru)7o4#BC^*Qju2yeDpv83$N>+zn&}3Z-{vFbuF4W zj_W*5T=X`paCRoBsn$B(dz%W3_{|&hRnmy*uX-Z#bE|ja?+f z)|7((PLQ-2L_c-naH(8RnTDu6e^O0@b&H^;uf0Ti-RD;C=gQ*Y`Qw+?Udd()p7>rg z8N594b%=`Syj`98*hw~>U?K@GNU#w~CHL3L<2)a|bIEY;gZg)vAp#Aduu`@L40y%6}8TlJp92jebMDH{}1e)y=Tc3mN>o)C- z`;N#VRAPT@`}eFJwvE>5vqk3gvTVwnmJv76VPZ!JwyC3?S<;k&dgwy5S4XW0s{zj5 zIkQ2!v#)KGv2(SPKqhggxZun}z_{vIme_-*+G;*@m(PZFkXh&Om~z#A7rn!^1=mnj zxdk?}r6w-VqPfW?;qZJv71xSc$JAx}{;)aRz=9yhk$n#B#qi%Oppt&!yddC;R+ZijOvH-Sj&`J8*6Z5H3LEbWpx-6}OjX#zOXB4~2q`!=!YK z7`Hk`WG0xY$FQsRB?JJv#ilHg^^h`fr|TFC7AEP!u`k?GxIf_qxqZhNSDSvHz;-D0|Rh0c^OUzQVgg&me zv=iW3pSqOO(yvM(CaB*ZJd|U?l)B{YvsvX_d#lG`s3Br0U)`DLsaLLKuS#~!dFj!4 zDMY5Z(k`sHK-#$)QCuk8WNK$W$Iq-{g|L;-pv?gdnO$bg2-_53Br@6D3X&g?O>$yc zQS`{60j%g1IxwM=;~-y74RBs7+PRa&C~`T1bU}_;|AG_qsrVv zIH#jx#q#MB>hzBXBl^^hHu7}etiy1}`lytmD-5alaH6sCD0x^^{k}C#?|^!=?s$aR zaA9c4TXFS~98s!rrR`#qY5g8cv64avzmwss%MqcDNs5~hhkCkciYbrgO0S=P)-@TK zNIZaP=aG?zIr|uUDfVg23;Y}l7MIouRMovuQIrnVvXl*Xdz#0XqkPYP^nA=96Sp5V zWn(YQx+Z`+IvV4p;YnY5n~dk(tNd*8w)9J0&DyCG=P!@qOXbc4uq7?StABGT*Y;Bz zsfQa7CtpUUs0bNu1dUZoIcFV(!QoEX*4yNdwcc5s&B13!i6$`x#mPas1v1UeON9{8 zeVc*r9u^oHyJW>(^0O=IZ7i$Wy^A@kdod@$sC8~sbslQUap zzc^MSXI+a7OffbOd>hv!Go0zurmhIPLO++fz=VrEZf#sx7GB}vK2_?{k%n(B+BKJq zb$5KN`+`Iq~?uHUuF<7fGB zQZ@XUC`tbXgK6w+7=82nAJL7_>_A6O4(h*8S$+*XWeo>=M6r*3-{*+)hnULSYRJ&{ zuf`M9%=fS*s;P-e@=gAqs~*cyHL1*M_m_;&PNxtk{yEAPZ?@Nr{hWKIjcS&QmKk71`mDAN6cN5ypFk3HXw`ZyAqQuk#^$wc-hIKKZp8`T!8!GXy^Nl z4gk)q5PE+Xuz|7Mb;eG}1ZUAhVt4KkLO4TEYGX$%VGWFyyNLcgJPVIttt@>?y~|Ub z>B>;q-n)Z4j6V9Rj>wpYL9srUMlSUGyw%-CPxN>b9NJ=uhda+uXL9r%0~}&&b=91% zbmX|Cu1nD-y(Y>I|3P0~&KV!5cTP}^@j_CsO&JJ>#(g;)b61pj*nnJ4bhhhZRwL7j zA^ZrkiG1zLmB}E=lRU-U+f@$yMAddkx#WD9O(L?7a%?IqU%LV7N#UN@mw!X2fY9@rzs+OW=(S~$yFrI~6nskEMU#Ohwp~6WCMgA8NQZPdd zCyky+*z_)6A0J%(Vx^;CnjNIt`(gk^!aeuRq)4+PIex5#7FGeXrAH*~&bcn8h!C{5 zB*jHOkAQJ_&Q8dxTQ~3^S73};_gL1{bwk-c`Cn#`LBE2FTZ5AOUsZ$p(nEqOY3`EI z>+=mwYX;%-I;M%!XSIFfNtdd=R(0Fi8=&8Qu`3Kf;QvI_`h}UQwN;x>lntz2@+2Ib z$|J`k>Jrp!dg#>)X-t(NE41_H+r#;s+i9LEy9>iMg+A)U!6i$0Ux@^`T7%MdsPyg5 zO=osBr*V{`zIz5tJ6qnpj`n3(m9Pe4)als^^7Z2p&Tus-MU+IB(+K@r9mP%vacTZXgidhx5sd^SGOJ`l>3>RSr&!eM^Mcbhp-zSRn8OOy5YJno@ zxku9lQZPU<#!`3F$Mkk$IFp-kT{XwnZpDK0;Vyj&Cz zqg!u8GYT~Kb)DlPr+HGC-vTp#a}6|7X7hK0(4Eyh3j)yCC$Hr0&Dft&zz~u8IAe5z zwcDd<5c)9JHU5o%H{7Ln>3K_9O+(4X_WHnsQUic;{1|f^bH2w)+#3;^dnyAw`)0NH z4^`tMf|8kiOQlfGT`9`6p1_$nMP$Zt2ckoUtVmi-_FkP*_+-^Uxt;%H?^v#UqQ+CA zH7HVO{R;gmhfMPA#&U;Q^>$0(s|vpruSnt}-ef)j&$xKb;)J47yD3 zf7J+oph+f)2ciThP`SR5UXL?u<3QArraj#rdFWU$?9x*Uky!(5?Y3`HSH-%Q|Fu>7 z9t%l`PjXcmOm{l;t%eSB;}$qUiZe}Za+PvL#1M5#DPf$+Rv+id?7(58fw+bozpGB+ zE2b%X+*@zmd~i{AJyQf}tB9C^^S$FaGH1>RfDQG?>a>slT#o;n)%-tmr+(&6{mh;E znLG6#kURB{B5Hi+VTvzUMD6uB3q&kgO8_O^=gmUt)}laOw-IC}V(YNmv1o8+snM@PS1o(K*qIHIgXHsu{Cp)huZ6<-@E80p z?9sQ;?K_+g?mF&zygX8Sgxky26f;T|=BMoV*Jw-j!YYJ(W}0wpXdZKAq(T z-(ZWp%4VG%FZ%=6h%spc;yO;tpA51{R*dRL?Ao-77*?`6WAOFz<%HCCVxSSRgd zCgDfF9#^*-HH-gRa@3Wp&zi0+1Fm@}_7)bO%DbG0Y@NjCfB*Dk27hQ`aETB_FMBl8 z>tN&WrS(F3iH%-0J%@WqKuMUA4J=L6!rk?ERn7Ix=?zEAlKUL5?#&}+@i5I}+VYz`(CjCG7e#M;NQO9+H6nZb< zju;)#?bAHpvl$VDr#=fj_G>@G7xWEM+X#ss1AC0rU2Dhj2++}Jc-iX4^OoGYCW51= z2kBh80e9$f?w2y8y|OM+47)*+u6!{`P}zO>i2NR(e$Y5aKs-OJX#c@hkI~wL-Dlqu ze`_t(@=&`3Q2N1_400$TzfsIt7dd}TQnYr>xTnF~w-Qkxlc}7^ zlljPlv1a<5aRE?CSIi|~r2_qhekT=-*SSPkHJHf=pBL=u5RUEkP#-%3lMJQr84O&k zbr}w_gsz>lb6264$)N8#8^Y3Us*0ju#}8Z2)NdQU%vNg94xJz+rl0BY==!yNh?(*sgq135a@BA5A%(+>{YW~+>L z)BSAXeDrnyURGLGih(+1|5NUW*=Cm9YXBxV>A?Qo-&;~nPm}}FbdmH9@yzb(a>Yrk z5=2^)2YK~vJdoe!{?{z}7I0}FzRkadY0H~9S<$e2LRT`eq`o?X7#vdPQ!+a}H#>K3 z@Czg;CaUvYOkU%1COd|3@%{T8yDVa2S>l+Qb2=>VeBEKg#~K8V{GUPP9!TQ zEUcf3)#Lr$DF!Mf!RNq08zq6g=(msKH42Mf8c7mifrb`-`Y6nsxt^K0Q<2IT5db;5 zC4Gjj{aD4AX!>?$W|IZ{7z&8)=KaR<>1*e>YRWoHewd)YkYva0^#(ZQ|KRH+#giPx zVz0K#85(@6mQ%d@l>|nryj6=(P*a^GTw?f7^i1N<{AqhU>dRPd;#gkQK>D%K-J5YoNXkXl?VV1=C)%@m` zsM0mjfU7Ph$7^V}yO&=Nob6H5sk)(Q8*D|s3$!(0# zIW*LI0=(Fo(w+V#Dd#YzKq5P}7cPwn5?>HwpCN;&SbRl%IneJk_I2@>#gDm0qNCOc z6m`1w0W_|=&27TzqlXAx8DS8fbOX*F8&B?Q&eUy;FKA*mD(1 zdb{MDjK3rVQpKlXDL@3CS z9#UyOz!e99niXL*Q-xWj>NWqT@*!m5l!kI+_{wh8V*`7I;DEpKLkY@HdaF`$Jze)D zrFq)sqFq`e3<8BzhWbKl0~TF~dQ$}di9Y+%H`%II@{6oS4h$G!O}m1X7_i`7*9q2- z7B>|_*!>&=2=PzZ;bh!E$8id3Kf3z&F>MuH^a5?ByLm@4 zg*>F(H}=k-7BTMe4u}a4tUm8q<@sO#7UCA?2%8(T`O?x`8 zrd6E^(mJ`OYIciFg#?(Ay68b9r=d(B$KA^D^+D56^*LKoV#U0MkrG+jA(pacAce6% z9HgNU@WmDZaiWezsOceURwp=^v6Vvnj=uBZf7;vGA+*a&JG@a`C;+U4H?K1|n3mP(S=#ub5Nd6Eb$a>3`w!nj0vCiYt_OPT#(aY{GD0>~ccGZj zRv@O9dlRrN<(je+sUiC=fm~TEw4K5}mRK_jghb>F@d`ph>GRQ7ay8+$lQo)T*#cF? z7wL5;;~G03O4m{Bofx4n-UTE+FTPN-f>aCI%^~tLbL}U*6GK`2!P%kf2(yOV=xY5Yi9rSNXzpZ(KdxKTcury@bedpL6nthXt zZ%V{*F{c)X66Dy>-)2|bsu5BF$f_ck(SZ!m)h>? zCiWK~Zs9E*vBTTd-T{ml60D}ut~!}0CmT--Ew8ps3pC3uBDw(Uglsa;RYqM8vq>qj@nZ$_JICG5>4$@hRag>ua zf%50rkI<3Sos1&wEcOI;3MItT!{`D}Vwf@BoCHe?j(Y(W zBG~~4V*6c`i1>iw6YQD3Lq->YijQm87G~#M%RMCXu1vtR@HDg;m3Akw{IXwj3f@4X zzOlp7u$HL3CzQIDR5~`EGLdQi|P&HnpXQFOHj%i-P*e;ii6 zpJ`aTE84s4%soPxo^#G^uj z3tr|b(Xse6WD0LAb-Et7Y2=;oW_2vzenkM#x2ijurfzOR6-*!@?N3_vg%v6wa$LYE zVB#&H05*g6fGpNlAEYq=W4>2O>d?SQ@1-rxEbZm<yi6v?j$VYg%CQi<6s>Wzcf2q95RLrj z@PkU$SJjz-bA$w{eYO5XJRWIdN1UwIvu*0Mu0Hio=i$E@0Qxih_-BdlXNm7;iSIw4 z#P>gg#{S_~`hO7p{_pm5{fr&^IlBHFU4M?Q|4E~(+}kN~;A~w&W`(o0zkircf%}?4 z_Vg*nhuJrdr)aV<2H*2#qjHw~eY8TJoIV&Ye(@fBQyxU?Jaqu2&av}mKlq-k@fnRV zNIU!eAb{Z{?tAgmeLw%=|NZsw1n-OLrlZq@2EB>Wn(+F}*G)v=SV-6{#zmjD6sGmW z{HYgD9_`OpxbV7IG5p~2ylc)I z!=9X30nXD)iO{xwr@h)0lzwP09XLL7Od>JqPp?mCQFMMG>j#YE6IMKS2~l2t6ywyprljYA z`NBp~1n%9vDvSji$>bo%Va_bVpFl0$`WOZzuf@eOEw`w(j!4#r_NC0N(t|1>O9=Ko;BSPQ})Moi{f zqGQkKo8>$@T_*Q79?Z!BEAIF~hwv)Xvkz|NXPfUD=oqG(50}-V_+=taO7NS;-#pAl zKS3OA3Ut6bXq8?vwILw0=K~6#SI~`wM>z9)rLe#y&1hz3LJd|H@a-!34SNi;k3G^0 zO>EgYR!s`%6eW)&n+BONjI!uBNkVcHeW@MV-60qIhGjTE9m^0b@^vq{(Z1}@q6FcU z5x2{d$>!HWql!^^qp?k);zTglxIm^sM={uZidwUK=q}Q7@ekcrRT_(cifEl=B(YwhUwuG{h94W2bzG*SOZ9#+r`TcA!nS zm#;dC8J{^7d$qHfocpI>lb4-&d3wI#$5qOi`2lL>si()_;si@|d)n2LiGaXGlMnmp_-6 zKbM#PQOnD%1o->o5kL4mx5RL3mx-hNCci`An9E?0lOSu3DcRNez@Vdu5o6oCD5q_m zc6TMNg2gEaB(CPxbC0liu?+Yzr?m|zee%>4^d`l`TR7IoldyIU<)Kk;&Cyzhy|56K z8kCd}XhdsNix$|~JzYbfF&ED7e^8{RO^=%D8E7`jR8;9-uzB=h$(D$F)5 zfXVuPH|)`a_?uR;(x7|m5q3kR3av)@5;7kQ&FLMs`ASMGzE}^MGoin6e?Izs$CceIAy2efI9?%Kz{w{;y%Om6-< zq+)JXeDp)w^+KQU$x`XnMvC2|!XPw)os;hiQyh^znPS-c)-1&B$vCPkT zI+-)kY7`50VD1lHO>7oJdDqr|cr+VeH zMQtCc=E;CvX%C&4csWF@r8#|lvd5;(8D%DrbL&Nt385OVS(8?BjEk7SoMbE}@I+p) zRx)%zztr7NFP}QE@s6=7@Fc*u`}{{E(P@nLg;KVp$Mf_Jj96K~g?x=R(^TTRkem!5 z8Tz&M=$x)2TcQ}ffS)h35KQVt1mO?VkC#ecO)GHhndERMiHcj@IsO+~g$xVZOUPrS z8qa!eicLgRgV9vqGkgRx+<#YjbFE8nYz05!qgS}KvCWhy6ELSkF)_>s` z@bGD-=*PQ}$+a~bn9D(gg7qO7&j0P(#kD)5h>6)Nj(>NwCY2g1zH=5cH?~QTx&3@+ zdY|SSPw|P+vL{PtXEWr_H>Z^Qc@e9U^-Xp3vwig{hn@ODv)M-+J7i@uUk#D3ulu~E z0Ro>BYTJUF<%0{03Sv`3Q-ifQX_+1XH^!)Qg=~)d2h_qh5lH340@3B@^9+%6*hR3t z0O?i7#q;zN>78j5LslL&KQR-7ssLtoQPg}>!c#N)yb`?GeF1$7AY7TGAT-}3KU+P7 zi@CGb7Bip}T75_Z_6)D$6?&_y`&ry~1sf(l&yHv_Yk-m{X>=*Gq%^6koi^9ki-%%iE||a6^oJx9Pj?^S7#b z=XgGpAgMf%^J%^&&#|H)-v+O?O&pDHU7L(o`C{ZmoP3lPRZOYcbe}V+BjjL*Ao8DZ z{vru3xZVqZF57#lIh}Ffc_%A3lbtfxbxi&=+bk}kNYY}4dy#q3@M0HeG@=;X$g5r5 zX*y-7-XocBo&0rfE_zNAWjygA{c0>|?yYR2aRYHVF-H7743pgCbqJ7R_}r*HJ9 zaHE0ErLEaTdi=NE5^37H0U@CYG@4;xtinm3%DQN)B0}(b1O7w3 z2n77_^{*19L^}J$6ZrhnEvK4@>*OqaT$#bP!v1FwawjIm0?t^v_Xb2ef@?IR>^?1g zy1d@PAI?e|r_zh;msMUCnwBTrPW^IW^bSdRpSHWZ`WnaGy`?4Eoqu~Trd!(UXpNFn z&v4DEXvEPr=;5`3SPA6sHw1nkJ|t&@EsCtiQIGzg1>CR&Py?4KXuT@}c!Q~S$Ism} zkog|~*nd(@_Whg5A^%b!ldu>g@xB633icau!YgdEG?+vP(^@3JL#~yLx~Jn84w3P@ zLn{{w3XP%H21IL`8bTfZls9Sa*p;Bpo&VmTl4CPr&@|;|aQn7RR&1VnW45>6>xq!> zYNuBxUG#0UpXCQ33~Z{^bQBx}&fI#LsBM>hoiTai*{4EcGeNK-n|W-qX6*BD@~#j% z~%Nkv%feUgk(L0ZFrP!zWmpG{;%<)N>{ z$E7cmF?7CfL7p8^*Af`5@%$j^2}wO_l4vm26Xrtkb}6WQM5*8Y*atH7N<(f`-ZmBn zE$(f|cvahM!yFws#-n(b;B8fV&h2b9UB%5+2M()OwPn*o(eCyKqN)YWG(rJ+&ORWl z%Xe4APxPbAA~kR%8GhtFV51yM(c(`CAB>3{U>E1H3#q|f-5>3VZcTs^O z+fTgIVcsRW)2dasDhq52*9)9mu^hd}OkSr<3Vf3sy6ulZ7E}=RE(-~qb^6nGnIo3H zW^7&+8eg#&&y_x2R7b4fx&e#NFs>*`V6HE}rGE(8BQKoF$V?)OBz4G-!4$zu=IC0o z&mDaTb1{NW5=5IJ#+%&FgTAMh1TdWW2y3B?4uaG+U9AJZf5|?@Aod!B$0Vo=Ii%~G zo3?A*Q1sRkC<~5|FlUs>)K^>K4Vr2rWIl$FeS)r zuc`Pug+wBS$5TgjJdWka=mks?O$ZQgYoXYbieO1x1W42sHG`?cp^WTh~y6fzEcPjzTiD#sq+S5sbE}4zmyy_1^509 z7WDnk_VY$jJ2ha&R6fXSx?2tI%>Qx70HHM(_j-ZRxvN{a58Q++%R^lJai1SSXJdK2 zTeIMf+8KN}m%}>034mH%&=f@|$J&DbJNqtMok!)`fw0PTLnCiM_Xpo;1US6AC8r6zeBV_!kmjv85)0uNGCKD>&;Q(h{@i~4XKz1;aV>!u7u4HEPF)D7 zuw*^Nb$@^~uJn=s2NWyu0mcmmzJ>=)8}|}0Dq`nK!~_>LKK}X(r(`qac}$+JynEF5 zvPWHZp(|JI&yE~j{qm(oa=LOahB^zn1iE6p7G6gOf(qAdeJhGku~Kop?Q->oyV-v0 zb)!DQRMfl$^6JjW^A8cI=N2xx4<2%NxU%4)?*Xo56MUyZvhK^h*b0le0{E;5r@I*# zil7s@K`}xVE%0v0udhLIy4#GwSnl))C%$d+6x*477<(cbmm=0B^n;J9XhSaQNNptP z+J*xkDvlMYERWM~-mKp~`!%Nc`0Ed?0o0m0gqBAa-A*wrU6y8{krU{UgCEI$9c^^n ziQu?vyBh}0Rg+2q3x>R|$9w)CwBIc&x~|`DmWlB8hmtg4^D4ul?95~_P@`AP#~HU4B)D9U6Ehsb*b{Qfb*_BdS(fpidDv_jJ*eO zE8lu+t-&mMoG0$L1%TO8eBegJi|M9aRPr$+t;C+4yaYVO@_cWuK~DQdbE76DaM0mQ zu8ploLhYtvETho!@O9RBzQulxtrMg;qK*1zI$NTmLCSwvR0g8gX={h7>37?Eto{_jLC}*hl^?PfXmG+{BU3O-B9w!v=-__!}fX|3C+IER| zpZIZ_rwR13Lwexin9ET$=va5QZqLXuaOM!65Ez+9*JkQXqN0d_ zd0|cYSqj|^2FA%1=MehM{_%LJv92+Chej?$&{EMf4PST*uT7UIt6W8jcNw&);2nzns>y0@FHdd52c(adrU6ASs$vhcg}8%}sOmij4fa&DFM8Gl|ZP z9q?^CJq~&GXgzmz>CmT=SLv)h{FJjyv5i$>*-*3?I|gD)JkB^Iox8VtGefPu4(J!x}xw18H*LURO@rxdom6v=xJlw6&`SS9& z>q4qLKet82Vy4TIy52wDIB+gOEhJZ~*Gy)wpv(b5b~d$FI8~vQ)63ZJF=3kg8V_@~Kql4WJ*B8%U zE;#Z&BV{;S>v88M!OGc@<00<^yPViT=rHvuAwizE7ND%JN6_~V#NwT)M8zueAj2C1>4$|QQwEn+O!JQVK2B?+*yWyz zJNv}ylI@|5D&03vt8{H9EFRSq*fSlp;uI240^7=26K}EurgBYkCi+J6Q%jNU7;NBS zSE7Xz0=fJ?HBXypqKmM(eRKCRr*g_j3&`Xhr0=yZ)m~;efHn))y?i6S0N#a3?*0oc z(B5>jE|(OHb~~^gO>g~*0*D{j=_}o8JIGw+@()SuvY`98Jq)?v1XXgefcP16t8 zOFi)AA~e)T{Q_ZoTXBEH<+XRb#fR#id!$*1tk=gKbH>A$b<#2Lq@C=7inSkn{;+3+ z$>l>{P?4HF|A}x#Tl-`VvhC3o?jYMY9`22kU+X0Q-~*dV*V3Wp?2wH#+)^$?j*O!z z|H;~i&awp#lt8!_b`HGUJ@4MOcGMi7NfYj;-a(0yD6SH5=}gE*s^LMdNj&-VxIJY9 zqps1CROJw_6HBm$l5*|RnJOPXJLk>#=0&Sk;cPPm*KXHUD#zO6<2iw}fvS=l(l$;V z5RN9r(ghfB`nqO0T;Yh3oMJvUE+@&E&6K&@6xVS;K0+`U#zL>BL6=bR z&-DVI)Vxo*_F5y^gCJ^334OdpEN4sK@NpE)MHZyuWoEJ7uwY5yJp#-@9*>SzBbK-4 zxFox*C|PWy>8yqxpn+1s_vq{}=iQu;LIWbE3<=B7=+12T_; z6Ya@eJ6xABPYWYCxjiHa!bL(JEV8V&0c+-5=W#UT)m{D`$We5Ew-$DO@fhX~PonW9 z+mb|u^KmWMrm1D?9%op6xe=;h%^l?)qj0sMc`_JU%kH^+NDoP}q|*}^Z`L)5_iNn; z?4EC=9+Grb-x;(j+;qKA`xr?yMP{~;%%JJMYF27$MOQ4P+RpC`VXTFGqT)iER8?9( zNY6)Zgr4hewaElw@rU<2O?u3{o##k_6bfCsI0#3G3Jn_iq6V46KFV+rhHg?^?k$*qVX8fS_pQz*5OFynr&E) z%Ugdl5eES$Z>IURM(2re!&>ZtVP?usW3|dK`5<#jWu}mz77=Q<>UcGq*e4+Yg7T8B z$Vz1!IC99Q<`Og((kRK5N2zgYO93ACe-k48U0pUhF?WpAN-(37!lW zz#am}FG;3#;!aRJlTCEGS6SI!@8e8EFwrhn}Y1| ze36kZcil~y-+Y4oYqO6cDqoANx|~xvNkE3YWvJQ{tg=5WOIgfEoJ<%FbF*}mh$m6s z>U-vgE?4VzIo+%(_q+6&IGqrPXg8Q#4YRi;I8ZhvRQu!hTDTXXX}Fu2bC;<<_#jv~ zhJUXRV;fb95eRwP!YJ81MBmC3Lw|c^^lLqUqRajb#udt;Q(xhZfToIX>^nK^S*|BM z9b@=()rF8jMjh$(Bme^wPORe6XZ`_YV4_RFXtw7QBqPldV?mIFU9bPj)hXzUM$lpfZuve z2IdVU&L~?8TGjYZz0u2|LQJvK6@oNM(A^grV$cP^0^SyxJEUpR{(JFnv;7VKCw_xD3*Z8`C*Q*Y71%x>Os;2S%HUeX0kUN}gZ zE@td;t;>LY4o=}Aa_H*;y0qd<#+>F+&bnKDGDW?%S3|*gvgwG(Sjbytj@5Lh7g!*u z2GtB)XZAOy*`_{o@K}il5YHws37K4Hu!|&#ruOBN+w%j@uOfk$E;Dr$^`$w}#)OIP z1>yRRMNtZV5#jV+j;vZ+xaKk`QOKO{vR#{xs`;2yFzQ%Lo$27x|w6CS85Q#Qif|L?LKjc$%KC{*8~8vOzr5%=WwA{7&9T zr%6321p^s}WR9g8VxF7Y>OmrdaR}JX=I8CbeS`R|FYYmBQqQ0twEeEryccXNHelKF z59Z8aq{}nKmm#cI7x!iGAJ<#$)BC#ga4xg-FbM0vC37kbp0998sSEO0^z!#>@69@8 zoA>?gw#i=@t{wShafG}JmYIJ)k#b`5M240^yvPZY1-jA48?SB%a!+Jsye$iQxwx`8 zv-2^wB{y~KQ{MJY>arQ-^)lh{fM|)=mXB?#UCV-6`J-R-etYkg-n@k?!J*cl34P;{ zI^Vbd-YaYH=UACJ_~nhOJPesa4M`d=6NQ@6GKBs9E0Ev^p9?=1KEjjTnY_af6Dc6= zD}J}b?uR4zu4=tn66QNz`Kxw{3RNT2Oy=eJwr6SwB6l~sj1z9~loT`e&k z#i1VS#(0q4Jh~@igLs~UtbK7VXTR&6$YB12`K*_QQ6nz8ixV9sGuuxOHuR;4D*Ryr zhO&b{oe{?O_`t7u>*6oJAM{G<%=b=vUElvf9LaOm*V^g|iu>$%(R_vfKt{_tdFLD0 z@g8}K{higTVEWlaw#l+hDcr$tZg(vAw=y|lyV!n=03WQ9vUqVIb9ip~Ni(#dQGfP> zH}>6^mean)i>f=HAQg)<#)yA3LWLuHdSX&~&*{EqhzQjLa;;w1Ia$%o9MlgZ2VVIe zl5L>hetKNuzaxwOJJaizH^2nX1acJI-HF(OajfQk`|!v4N3Z@m$}h|8{LTle_B=;~ zaBG(;Mh^)6Ul0D5tu?>=_?M%L{^`H|8Mg3qLjE}+<9<%a{~t`qQY{|xSQYjAWE288E)^%t;?-JJ7-&rm0mt3tW_gKtoc z4^BTe-nmlI02ag~lW^aCe&^?&qB(NEPQzKx*djmpG%EQ($-FBl$t%w9=ZSlq{K2=J z!v|rd`lG%~IDw^v)E+qZ`O|&>KRF#S8W4?WY|Z$jSEtk(r}w~h^RmsP%i-ZyI9^v2+^-?EFZh2-?)@+6&<70I6hD=jjN9NRg5B}cO}~t1#C}xy#cy9a zdy#t&ym$3037y*~Cj9mt_}>ox=M?pS|JRS7kwZV#_&?KalFr5XGb?{IvT$=t=b6Vir>oUQqV(7UZn;skgDbCxdP$jrf z@`}r_DD|4cri9Dw3KocQDjX2azV5gjI^=G;N)@`tnGEQE7hXN_Hw!2{mujwR08Bq! z9X`+bR6{IZR%slE+3xK?}ow2*hfB~iC`5LY2Ghl zo$xu$w+V0g!nW2RBG5ip=~0pDu0VLJ`1&-_ZG8hCKuXWzn(F3tz<^8D0JtyXOtAAd zvLM~Z)w^4!OO8BcZMQc!DCIC?8GY=c$t-lWNy6|VsLce$SqI}gl0%n5Lvbs#h3qz~ z8+e$wVR<$oy({$eTv<3PY#_kbree9K^%-$Fo1Rj+EN|}Rj}T35QY_yan0g15T3UM_ zWcn<1XW$Ld$Ew_G;#-I<#K9@h6qz@*W{g?Szr`x~1^ZSW4n(u^I0)9(J&L(-gD*WZ zVX*<4fYtN^eU9=^VhH4K{X^DNBl#9xh!}Y!)kB&sF_IL5fmWYsDgDSh3`8)^Z*J9C z&LoH6>pP1V5nC4P((wkWm1DK3UCYUzKkvf+U`YjR7wO1HITRgas5s`(1&oHR3i8fp zGjmT(o_(?Q8v8}Kv9LeqVxOJL2dOsn-LMbgbW;g}G5O%<$s*q~=Ejefuac&?f9#R4 z8tDEmh!{_Ljb;{aoxul=vK-rmL&HY^~p8YQ^fXwa_S5;J`RN&ePu@uc8bl ze~ox1;?U)H;78K-ak8wFjv1Y31F1>@IAI;q+&)Ilw>dNDzcvWJ^vV)dE&_0iFc<>rkn!8meT_OiijxV78}Vq8}C*e zHCdm%tVFxjV>@84@yD`CC8DLNqU^Pf4*W*7eI|@JM)i8GrgNRS>7y1^cso8d-@NyW zO;mQ3ClhklqyJG^UQ=p($y+!m*!i++ih2j;C)}M*ufm*t0az{t;r1lyH^IF4iV5Bs zG~w#B58E-8d$AG)EcQ5py>7Eu?V|IRmdd@W_}#Aa}r`&6M_whuyc7k-n71X6bUjV^$QH1LctZY6TBBxK`f4A zt~F1=Y)HUM%dy06OA*F}#`62DK|8i6YwLbFpwEB`0vEfAB`D)fo8qflfrAI0>6C2V zNmw7TEz%JARHCEYogLc|TiM1jEN{9Hzit4IkRmik&?gMM zW=D5-O7Y~^zm*7fUW498Nm4v7ykvd3I;FjsY>@Gg5kkAzkYv+)k%X5IfgZ~LJ*p-KW^_C5Zu&SzE|_^*}JG# z1?iD@d){nlPwI#;ifdROOCRZhOA<1n*PgddVq&j`afd$?G$&dY4v}Xm<(MvwW9ZxZ>hmFBX5Q(Fu%mLlc|Vco@_& zn}comos|b21~iP(7#K+Bh>~8rlm8))5fUGUCYZeho)9tSlVs@O)#=<^4^dDxVp)Dk zH+D;dtL1x`Mh&Cum!+3V8@W7A`=a)s@CP3bOc`@P8%WiN`>B5(nST>7l~vAIogui- zWKJ>4Y3>FF0#Hfxh-2O==!3*7@!Fmo7b{>hV7oe!>GKbidm} z>yG^%=L@|xG?o*>&%TZ77K;lNr<;6K4 zlJuE4okdafoXJb}g;;ulh%82hU6K(j_Q!--5=MtJAGIALXg!Y6kuR$fw5T4e4B9$T zGIGeqBlC>^wm?{DIX=H5N@bur1C`L@by9-WUtIRA>_Tn|GW#0MMlEjgIqe1QuN9&| zRG>|2C_&mg^z9W*^TDWSJu^{N#qdwh3iD{di2dFRc)(D=7Ijgu*(gXJYzdpwK z&$I!t;H^E34q$deB*`N8;77lXBz7Ec9@WOlas62P6BEJ>!*(r0j83n~KvF0{L?Gbv zvCPX2?y&apYk1Xey?mPrrrb;ZG_}ra=~k_46Q1`9Y}?0#vho;`eah^*B9JWFCj5|jaYO{2 zWikqy=9a&r#JMCIw$e+C!z6rI>Z+fFa8e(LUu_O^&~a|@7OutMhvL^>Jh z=?`SD3~ncTHR$bxfUAwc%deeY+maUsor<0i=BLY6D{2&m$ab)95 z46BtQS-Flq)dnhe>7)nBy#_fWJ<%aDb8pkFyFo!AWgLiT8WA=ssCxSm*;^?+gKU1$ zG<^g~jL+1CX#vAZRL_-x)i%4^5yde1Q{l9iwa6$$!l~F>iH$i+Or&Fm7*WTR@?Bgp zzT3yCDz(Vcj+nNSIbCT|p6y@h<)bFyy?MG~=3Hp3?e_s(GEQPUB-ZHo3*aGJ-5SVa zh#W;*^8}oOB$IAx=%5Y;bciR?KO{5TWvwUVCyd$n>C~z}6S`cj4LA>B3&$tKtn0_U z9(;biom%G+;-5QC^p&2b%Uw8!|6?LT*(3LozH(VgNmDJ-KA_h$tg;}pA+5@`#I<6T zsophjCepKA-?a3p)qf&4;vl&?ukjXM#K$TaAx?W)#k2zTQ^N275n7@PP*#TKr?13O z;MH6Mz_|uHvUr%{j$aQ&8}|in34yxYnfn+|1~fUpC4DVil@EnZ;YB_%XmXp=JremeC<@h1^#zx7}_auFXo%gGKlpy^}myo5KClR4VX1213 zaKZq7P^1C`6rj>ej&&#mEFR%Xu(up1HjOkvbTG>-i9C`>L*3Kf6f|aiDq?*9KiGQ@ zu%@=HT^P4pv1cP9h)7gKx~PECV+8>r(xphYB4S`8(j?Sn0g;+b-_jx?gn)<$h$0Z_ z0s_*Tlq3}C2_=M-#hsq&SML4K`TqYu&v)&O!oO0~20 z!kWi#G55?h#w#BiNFjA)SY*na(=72Y@@T(vrAAsWQ}p*di$}o|`o*GN?C~|9Td*OA^6{?dH&ToqV--f>(R0G=-7I6?0=K!7_J^TZ~PQAsJ0g8 zs<0er7=fl2tPbj2`my+t12W2n9>#{Z*!z<;)DFdfLhO!LOS8unYI!+6Nts=7xLBO~ zlNqve^^mxm;~x4NpP-wLZbG$i3_Va>_0_>zg98t$n>L;E%(aN1h15``sV7boLvoMV zD#&)M{6_QMq?p=_%av&GxSV|SvzvSR9}~-&=VhP0?oG-SuuvlM3tL3Wv}a(=AC_aQ zw3p{%q5a#+Ohveq6Gqm`aQ*|Iy*6zpm^p3@2Jnr@-<6~;^Hoq1LwBxo?9_4dB_~M zx^q3s?BL+(t)_Q&XT8V1^Lv#NkZ0X|HY-K=ZN7DJhU-vQuIW(9cokF@K|%QnNj9rC zU2U;zjNgT=os{w0rhI$Iy3~x+!8#;TAHi|}4fZ5h&eUOH?)i`^ZNVP+2(GZ6{7c!L zVF54nX4XyIgh2@_B-eH-5f*;KxUBBY>}!*^FHJOh8)ZOk?=3v59_OuN)=qq!`8dKP z0(%2%5z|J^2>pBlx61J>O3e&2%afFpzVc$s8|kIz(L=H}ERms17%7^crO;J>+Y=Cb zhv9O4&?YOz(K;tG*3rg;qs86)p<5XtfTo*Ljtm_J^aGhJsLB^aigsxtweu(e!soma z&$aubpkq-h);dq0Uk*JMeVy3k>Oxc>Z&6iB(i)TL3`&u0jV>)D$y>81ZJ1Z5sERe(ULw6LFd!J`mhAWq2^t=3hD6{$+IssDkw+?ulr6~%XBbp0a z#x~2huP#bZtYdO@%zQEp)vtyfk3-6nbob=sOeJHsM`uR;WiGZ_){Wa^BE*GmXvN)Kl5p}olXYx+CYot)HXQ>)Hzv$b*ZK7G99beBv4)aH#DRylBxy7?GVVrv(whj2 zlp6Q=I$Ab$lu@UrS9*PVa3TsaYy#{P}Z|eyfp_agdOj9o-nO z@G{OUpif43FvcU!;ee=@{FL&F(VY|Hg+1}t4C39{)`xY%#hkLAoJ*Ng--hzLF1FjYWzHf(S;pci6rGd(lk*<|Hk`Q|2WsJdvn2t}BVtgOU0guv^Nf3rAvJ)1;_s2c+Fz~i8opee zbLnW;t~{!S*0G)TS6qx7ti1M?eux{^50`9v+NB@8f^Cd5iIf>MN-?>Tuca(JWE2T+j%Rm`BURsbzR-tA%b=C?xp%tT!!kn6+U@2gFY5Lg zDX7eh&lK5yP@kAtr3=>#3DAe{Yw7}heTqDS;XgIM6{lncs5TVPRTUyP@cWSnR^W;* zLBFl3{z$d+mrtD^CZ1u}uf})a#lBgjaV-n|6qnE6{ z(n^L*7ojLuP0Dk^B*RNqZ(6Ma>v7sgwPdT#l{>qGu5Zy-Ot=!Pv2aeo{au|{Ym7{$ zxZ}-Nt7YvIObjn+UQ;Fmw|kKpz>@JEA?%t;>>qkS*-i)^+WHPi$IB^bL9oj+Z`U<} zqGptF=iN=bvAiNn;Rj~62FHzy-+rC!^lgdrkwMRzK1$RG63@HX(vG!K^J`W+T2mBx zN%D5LllhIJNImHV8AAn0Nrc`slRe=38}oFbjp{&1}tXWJsHWra-SOjl!}>XY*39vS#tB zAEagP8XF`?Pbg?_;qL30FXkE5Q>3a5vZ#-iE5j1h?E#PS(=Eg0&H74_GIq0zrjh#5j{s)29jku{&}Dp_f7qWN#zoP} z(G=lo1>L$K!8b)6{)$x2{05fYe9e$p$IC+rLsj+XdmP(`P1|4298AOgmiO}2uaDbh z#-j!g>)DzHE=`*ymYeR*0okIL(>bp_r~I%QrD^RcnMOpdRf^PBkBdceensg_Y1Pu> z9OaawM1-S|*Dk#}r4WVpx8i8?a)}k<+(V%Cdcv@25zCU^^bxr;&+Ee<`6|iK1{SIs z76>sfEzt=ZqM3xB?U>okye$z;duZ_c>^6-T7tIH;S)CGwr%070B2sfl+YPg4l8wXe z^f;X`n!F%tWaM(p2P4o7CQ>9#ndX5lEyVclF|Ir(WCHPIh1))d*WhsOxne>Xe_Gkt`I3uu%)$<6X}M z&&oT!=+sJ7GGKR!WwbuL<1Yb{p}dj^O$E0R)XaM?GRyE8V{@!x5LDpO zHWYw?cVC)oiR2}2yC~6=9Z)zE{ai!ynS_uR%7g1Vh}{~vxDq&fai3}6auI1?ZRM*1`b4AjElLlUlaLM`<#-COTdvvW}<%EBiG}hMV+L| zETXxTzJj&6prujLJr6^t_ok&?$HPq`#*n_ssX3~g#^h3qWX$#t`t5Ju8047rc7byJ z(_cV~Pn@L<>%mAWvIahNR_c+zMh-6be8+O@ULE}$^5df9+;92CHMhLXT~2P#yyal7 z5jN*^GG4FUI5yYlWZdM7!Am+~Ehcfo_}2Xk@05iE+e0k)(`G zyo}a<|291Zn>Z8Jb8jIDReT6$K?blM^y=0??3k&g4Fi8Ia^j~zT;XZhP>NE|OUy?= zQ^K^IWq15a=o()Sag8tY0^H5s4mEU8k(@)In2`Oh0XxELNCJ|1F;EzqKEu_^ zw@&LFW^(|L+wcs`qzsSX0A|)95ugaB`K#+e_4{cXP9W5W;+3M797AO-ITeCj9GZ8O zKNhvBG6YB@dKW7dn-n#%q0H&vc@AE;4C6 zZ$Rzp2Dm4{Jqk85h1d8x5`wQPuK0Vi7!cb)doOaqsmu?&Hy;VHBO^4)lvXYTRi1%n zLjBJI{+L9TiZz;22?p2?v2T!;z`W48fNkbY>tXOWw>tjqRKltfG6=;4Hz^W^b5VWc zDX5_w@MaRkb|=j_5Ei`O<-^Sr00}v*S0B=ygyFsjXZ{Lq>Bj_>mE91_$9A4qMP>np zjLAu492H5}Xb~sO8dD^dB%NiEr4#DoDGz1LnZaW%K!zK^^|Q1 z2G!s(QqRNV00qv~%blNCgf+gSsT?^lRb3o3sFONN?|wz#)JbraQ8KAwYTwyzYvwsZ zIL9@<9Qd&IhZ4YZtc78gnh3sE8v2LYx!|LQ7=62g5t zJZk`A#q%@8V9nR~_B#6#=5Y{v1zg}a)DJ6`%=P6YMk;PD>A%Qi+Ouay9 zFawwTEuJSbPC&~GDE6&CSW2(~S_BYF`Uw(|;5Gn+W>&{R4r9juk6 zB8xuEc9)t^g;tMO?1p+BAClSKciR>%9|LCH`U)AS?~6Z}gf1J#<5Ed@#Q-id2UP-- zgy-YP1s|Lv++4B`nl`AC@@DLZb|SqJs>53`-3sdwkO;aq>P zv`rDH<={_7VLh;sGrVmpVD+75g3|pCW+s???X&E-7kZvv?d9&_;qG}A?amJ4bK=bOO-Bm$WP1tkRb1Hyzsu(xBduy*U*nT*qe8q#xOyOd z-hzaFo_`yB`rp<%iq(Q-00$e`}rq~3%)sl(64=vo#jzKaHRzp*Ts;CVN4cN#r2x2M& zAtycc;cUg!)z4S{m1pbUKFUAz+kflkDLe4L+*U`2c^+`CR$nF#ETMjxw0<1&^6Jw$ zm*DZloPGm)$mx#!SjmU`w$`okIyTsm)QxjtSHH@=G$U?%9j_j38Wf+o&v~E1On0JD zH}+e1)MSyFl&pIMMQY#oq<3vz(onOfA2Fv;kMR2>y_(Yc7`mCyhjXkG9VgQ_Ru2QI zbS6h>AEs~lQnA&5A*|ccW5h~j3}2?djn4K*y`x)(6}|J9@usBu+VBL8$!#MY6>YBR zU$H%Pe&LELrE_K{w-w9kU;SgW!P@idtO?GE<(p$S8ZX=Vje3++xzL(ma>ZIwc!sv^ z{g~_(d>8TQLWR`=C$8X~?KPRRBoDK!X$qF3A3ct@j?sIsj}+jVbZbk(4MRdKh{0c3 z`YNBMxCxZfBkJ-)Aw=0}dPq%$NNg?UO?@!r|?)get)@wn2d7Fo& zI)_qSgl*bAo4kVLUk`s17A<;%E^mmJ3cuFgt=hQ9cx%bM9WP<*4uRtefv6th*op{Y z=Y`ynK{ZFu1h4X*n)dpAkJTi_4|fH{=x&WM-K{Zq^GTCTr*Stdv>o81BEwBzh0plvkex!9(LDL@C9JwK=<)OO`OR0eA z?1dt;2AF@svkkzI2%!f5O8!bG)2gA#PlOb3tI;Z!9w>NB16Js?Rkd!j-Tv?W1X#cl zb3982=eN0Qo#?mq#N0oem|JFsp8%HZ!({mLN?7QTguSRYcQUr=y{qdHbp=(p-|)CU3r)Zs4rk&4=Z6X%@~+8S^Vwxx{@4=OtIh-f;H_=PrgRLQUsu zW10kBi7n)FC&`5Ab>w;fy*tc%mv{iQ_s|-uycDPv*pCs>mcnhP)`iep4cf{M&=(Yi zCQWe?XMjhWZ9qbWDG!R&`<=?CmIrQi(QfdTTQ#;htC%LhUASL2bVq|0OjA1}7-tz(Ew?Mt3mXk=VzDd^o{zJ)?du&k3Y zek>^eu|?&YHHhkxR2MuXH}}Uu?#@l9u)nnJ{1eT@#y@(1xJd##k|Q#{#@Fdwqr9P= z{MB{EX7vic8j#&RWlDeuNkMUq?^61+T?s!F9c^UHGwDDsC%|w$fJMm5hNko=vTy?( z0bkZ_KM&W4xbiF<`A4*+poY(MJX{yQ#<#4bh`KJv>gBQiLxki9+Kc~(_x{DPlz*xv z(DzZagj{NOS=N}nGi=cDtlP`LX8QpH%g2rV3TWdu?P&{!C)?RP31Q27r5;o2k9uud zgKykLP@W8shW_Mr1_s^`H_k+5Y*eiN6I-J4D=JNO!c_=kE_A& zDbhoIb$TTyBWDA{=Qx;FVomGn7697RM$LqAj{&Fn0V~$7SgU}%%mgH@4|rqMwp^Y zxIANK(<&2H8spw|`YPDO-j@G8Fue5KZO zvJeo{{@E+L2)r^{#4h1l)WR*)7HEbByfQ_e1ytCq3YD9KZvgLbY0UI%6KZtc4Z}-r z*#O2Ei$~-slUMbGXDY!%t;~c1w;ZUVh4;gpV2EJ}oY8ia3^+y_wQykHT!&?$X|FWG z2`)GaVgxv@P;VqX1f1O)tAxs3@!O|?!)K_p^a>0uW}*_$5ociF*H1~&3>0jr!30+; z$Ym11{(eNsyM|hc&}H2YG*wsifQ<;D!%SIvD>zfQ7XrLpfp4w~e7pkRH9kPSt1$g^ zjqm+=E^sb8CdRx_V`9faAqW!8wq3yjRE=0wBKIf4^}~J6YkUT zjOPP?rUCD70^&u_SYRGGdoZhKchw$=&iiuJ14>Y6N8NAsi(OS z1Ueex93I|`TK4B&csV&60oJRSywoL8Lm6HbX03h?T!}!_+ka0L%IBu^aQ6tp4L!5q zG$GFFhrshe0Bs?>#^<66c27eoeLq|e1DXOM!TLGWX#G+vM`%41x^`j0yNtF&QTIAMh9-{+JFPM{-qZ5k(B)Ccgp? z_oj*nXS+Er$J%X$t%HAn_VJ&{2kjONz$RdxXa=pDdY%C42k71KT*<<-T^^N$1sM?D z`Dl%SqtBl?pzz}tX~VN%YNT30m}LTp8RrU+@7OAHUhD{x`v?9-2=B@9VfM=w#7BQH zq}ZnvBrj49M9Oa21y)%y3OIDY0`(6?5&p_i@k2o4U&|i#Px1TzaE$)X$tw6;2*sR} za3iv)TH@pz?$4}ujg9(mUem?P&dbT{c)8`%!G~KuN#wbNfDr!or6fUyE@QX9|x6}uz8|>gzDCiW559tmVYO4Rq~z}03@MCCBRCvfGEk)KYZs_p&B}(e{lbc|*nU zer4gN$>Y`W? z-jm@i(DF0QLpI$kdPOm6sRN}?UE^Cl<`1+$!92Lf^wy-ONI@{(4zdHp5hVk4#k-^} z+SLJsp$2*!GNzOff*PG-_*Jb2%3dW~;3WJ*jA>}l6(G)Djj zZjgn$l1K~T{yyKE3^yp&g0UmQI!)lCig1N4n8yy}W~c#6L-OKn!F<13e2jiE4}$B5 zptn-I`{X-lo*8P7+$MM;7_1Y}t40^N6W~rqg{#n<9ds~N)-9siPV`~!A1k$r9CZYk zd3J8#@tzChp-HGGjLg2Kv}9WibTb%b-&>j@4~x4n2TMSo8h|7MZT&`pZV$q%tu^{B zX}zs%jtw}3O*D-Ttno!dcmmVPEd+l3erp@h{SD1f1wL-Uf#6ydk3vEO{iqYHqV zm;f;k!ncw7)4GLH= zosH1wJOj`B-A)`R(iJ@VmGbovKO*a z86k{i|EQ=vfPuyhDnkNWO>CGQiF!<+sL%W;^itDe!rSgT43rAz(9+ z0g1$scIl18AKR_|uZ|1oEk8HOdsYc#N)Xg}_?h+ck7vVe-o`3uRb5*Q>hr3C*ySGy z)kLu6MnA!dVB<7$f%(zp!N{q|!pwV;r)x31V^86ZTLh4~v4oq4#h~enl@rbkCHNeQ zj>dBiled$Xp9LQ22ALZmGyMd%6My)+d$8^vth)#6?!mfy@b8=!pyx4+pJ>gZ`bm{ZIXXe{DGE|9jT9b$!1c{$CIOuZREF!~Z{qn*Nix|6jEKzgFM> zQ#=F!RcPnb9~hj6`KGSQ*!$ig)9JMDsr&sAqUzJWdF3zOOtQ~A^%yt1WWLI{7j5-6 z!)?dwzO9j;3-qJova`q2rHrb2<9I*zoU2{8@ z9&W}WDu!vEf4e_&V@-uZyt3lsta^hY1MObDynjfYsXH0 zp6bly&wumKBvU8lW=g=@!J;}A7KvfF-Y(eKfRML($-xS+u#RbC@R zJ()P`@T8h^lZLvHYKL)d*xK4M1KLpy_xsf#e<3iPz9=If-Xn;YsZr-cnH6%3?g9sz z6)ZT6llvIBiDgWK5L5k%!SZEig~MjNT7^sM$eV6kN2PjGb2+X%w4YJ~O!h6;z>TjH zejT$8uQ4X#u2kQcvLvd#^8F)1`{n{$`F3l(&M&pHUd{k@8Jj*cD5K;5C09yvn<>-W zNeuw-j@<6dMBGc=2I$9fJM)atN#@7fAEt}mjT7=V5sDkhPpKi^Jc`A~X|GP9=a z4z{aYrzz;;Jgc24-%Jd~x|0B8z*)yMVWaTVU*B+*V~lHbRJnUiBZy(S!Z$^E&vxDM`kinF)={qJ>a%!>cku@@Cai%KHT!M&I7Y^ zGz!B%xll7HZ!(N~d?_=s`_gD6S2y>h-hPtox2f|IB-zIEKb_rtUnA$w*a~eA-fI-d zl3)p(gOD(^{8LZvZzxVJg3SZT83WyK;vk9A&_<`hO|QVKt~tOBqzM9|M|Bwx++sPR z3+ECd&6hzk+7HC5?LfqQ(zBJB43fLCO;u#jXv0p$#G zYkXm3I4yeBND(%kuF2;h7^Z7{HZPY}@gu|B-|+)}aE)saiv30Wm%M3z9LAc!#u75! zAa0`e4ybnp++sPX`7%BfWU{csZ^*nzOc6+2yg=@B0{M-y8>s5@3s2js2@9TFkWVhK zd5{GT@d7 z7nL3Mh6Ko?H-R96j6&aD7Kp~?PEaL54%U$ukOv-#BrdfJr1Hu2tsMDx)u2wrHt1U( zG&)Wmc}8FwkzMk6C?_-2pI|&>4z|(~Hte}S%tX< z=Wo(ziiXjrq~K;dFm@X;y(P`z1s{lxCUXR!Ux#`8Ro@qXuwZz3QE(N>k62aK)`WVe zLF!RXNz;QuWa9^>;ZGr9&@!=NOE-}8KTvEQNR&my#X0jHV0R`) z)GL3?zczx^h)RLJ!#IyZ(^ajAZ1Tu({;HZQ9>u%}rfXUI%QWE>ya~ZxRcs(U`rwo80u4=b*_k2ZHOg>;^F&4;1_AT^JfSE;4@kia0E4T_AyXS zx9$dRqB&XrZgM1k_~8$jzaOr;?aw;!fBoZM%xC`PpUZq{pX)~5 z3^?k#5r?U+*u#pX&TnGXe?*D;`!PU`=x${WVyBZYIi$>(vA;%Jz02Pq^V;pwx>)aL z(`h0VZ54-Eu~j{y4!mbcE^0IL7e{^b;!YQl7EPiynI_7=R^94Q5NkU$u-!zg*xp24 zHwoR>8S(r%#Ufr!r@?6Vdp)xRwNAXUTW@h>%AS;fgu{a7GDfj_#h*IUv5O>=T#Mb^ z#&;JvsNM_h&WAT$!3)1hp z2=hXh+C!9^IJ*Y>%E^&M=zav%B9e2)kcK#bE2%*pIQYeXgI5mgvEq}oee{w?Uh~^q zM{-+tHH|bOcl6@CKid5iQoZ+{ucx=VcqZk5k_E?MtcIQBwC|ndCjAtmeo*#}*UOVt zM$73DwJ*J#(%Lwk)Af7evdy!o-`d@uXGpiWC%JpXl8kjSFLhO$r^`q-U8Ec@tTYKJ zv{F2Pl4HTO))K$ddF(hNtJ12AZ)jS_WIJTl(Wkt#q zezS?6w#bOev#XG4w&|-Y%AqpaiFN`e8S)NR=Z+b_m<(8PBVt$Xyt&qEHK_io)-U|_ zQLi(4Tk__t)|J|6Aer z|LFp*K4iUYpvI7tphUG0h#N7%ymHKIw14EA0TtR;A)+ZVV#d* zosVIikAd=k1s_A#KNr7O!-%k^E0MdFZZVKG2oaX$#|m+SKjf3`82Bfi_#NMxd_1pO z_n#u=4PEUGPmYN?c)xZ0vvJVvvAd*3nOe0Y6H+2a+dX0;vS@pj8DT#Sv5rQuWckDV?(N|^V|l*V zJ?A%Jq1~?Vajs$k%O`JWr-lxrYXa}D;6mAHc}3ApxL0{zpgHGgw2yC$^|coX2`lbs zL94+~2~44pz@ONZGtKSdu65=}6+ZNB_NmwHxFz7*8jv8&9fBOtOR+kM7_Kb0)1$WFQe!~_CGY4ITW;1} z?OOcdfK%J!Tq@+Awrk|S0ICLsb!n}yo zoL!f3FDO&)NQPqY+p#?_mp#HP+(U=j_tp5T1N4niD-0Us0oYKY<+EKwe}>E;QUF~p z03^-#JHnrkwZptK#Zvv1DAbRODFNWAd`=>O+J*GwulEL9?+vz2LHLhT5Z0vvZY_CU>UZAjW6-o3 zYu9UN4zR7%f}M`O#C=2NW)P8pbiy%|z@`^gPbsoaPs2twv{qiUGsHs|sZTVMADo9P z3>F;6Av9w3v@O5_Is~J6SE#MiN6Q4c#ed5agY>~JdvMv`6CVLEpM5f5q{;u~3%@?# z`ydaxj-3&k0N7Czz=UB)-|1hZl&Vl1QA?opK9})=@@e)fBR>*K{3VU?kN!|2h(ez-s2I+M zDTuA7z~`CqjJ=J|nvdr1y~cM?1Pn!ulFz%O+|O6}|KcuxnI+WPKUqeneS-IP3H6h+ z_C}Vs53TiyxB|=Bm4XTvMTIXRcC&0KngkK5_~F-dVnk804e}S39#?=isKF7ZR@-!D zg>lT=O!|kf7PoVgKjqtXNj929xb2a=m!l&>G*rKHxBRloWi6fIBXLf$<`Xl#Z!LPw zXAK8+%~G;&TyF%8F1j+zJr)}L>=uu+o-pHtZk1Q{d!%^p+We(QdO0HZrzn;8Hd<7dmUU3piUO*Bw?j>kX9KMS)y@c zBJ#gwsu(HgyW!Jr7qXBSSt}4nhm}h~MG$$hnz{^KW;nj`_a?e`T~c$LoiqbeN$wk{ z-bu@3St5duadk7XD1l}227d?ocy#_C?pF5!jB)p|9eF#+5r^NC#~%i3O6rS*nUd@( zHmVRG2U%*Ss`pLTmWig=m^sm2x1=47kdn0|p7V}#P_42ykDIa#))(4?ZLb~kIw9nh zA<~%GL%eJ6R|y6iog_`xh45H3}n) z0?_l`R)BOa{G`qQy+@R%mchpr(C<~=-`PH67>nFljCmw3k%kavWodcXd?6RijE7r1 zwCO3S@-mrVvYZ5SGXJ=~$j#UnQM(9QcZ;2}sxETF)!I#3WExdqQaN^(#+_F~&j}>e zWw&mNc`@1Zilmaz)8tX6Exd)|8#g4S+$u+T9ThpK?(i8x(>Wf$NlG=q#3N+s zKrS&{6c_ctgVO6#_WRLmH=0b|r+jtwFG4#&SkdaSnGMz_In^<>`Z-i^;nhaF+NG)PLaV1LX~af9BTNut=DF z0D#k}9AG>2ZDauxGkb5j&Gj|DA!(>*3d6QWouLz8BYX&wdF?I%&c7=~pQJ#mCsEzJXeqgf6A|#=iNqzs-*sn zJCaJDr)&BG8>%T3T7E>Ky;|c-&f}3$GX!`L$yTa~m*Wj@g*Bz%*o_SYZqocu(2UPW z;#gvu^6-Lv8+huk1fEHZ#TwrtxTQOwu=<_740Kx>hUtWa-^R5Df-%Lu1Gsl^kKP9y zV4qDkoqH+FsnFo!$UI5HcEqZRofdd+rH;(gsT0T0Ctwr+y`z{HDh>k1B?GQ#o)moq zyqZPvPMEQOTjOH|Zi4z8qoL_{w2i?GH1{dK3dsTHTtB$DIRS7kA?MAZ>1Gs9&1~gl z>#D|Apkr?>-iMZ7(5j~m6=9Q$oa0b$Ip;7mjpAy)@`foGt~r2yFX0>Apas00B-9Iq z5!i!{#Lv)XVABN!4h%CT{0{+x@jgA+)rMBZ{$4wBnI{KUVe~!S983v# zjMj)357ey_j|hy?gDhuDG>;|#iz4G%qCl}hERpKX3;WIs+XV?AuML1|3?oJGaXT`& zyn*jHGzXaY#Gs|s)obWg-5n@i2$V*}P7WcDhOr*zpLa6L&9 zBOs>v-3jW$aSOvLOFm7?Yp{p8^!qiB4ws zQqk$-U>nr0c+mVf8-mdE2asBOwcCW)FcZE6eI z9@-0|LH3`Q$HJl-Hg;t#yWx-6bY(B9xxZ=~kIQr*Dks0Wvq1j7CL7ir=PbYFVQy+Y z7qoe;LG%KP-N;(#SMnfZmPqWj|!e=>L!;r_eeG>P>ZInBd!MS+(lD~10G*s>=H z%zNy7nB;P@`0o*&&JOU==1^dCS%MR)yMA{7;xso6!xfx|YcRd1*Z9uTGd;*Fzj^cg zYhc|RHBbX^G3p@g^T8fkXc(vRPc!QGfy~(kC?Ix8BHzH?OzP42kHHu{1lOy0z3P z!^G_CTa&w~C1bv6G>$w}X;Osg_oi8f1n#op?(5GB;C?@7vQyG#x}W0x^DSFPo5;I9&6*zs6R!_pbR}zk zoig`%d@@16@P@2?uFkgD>>TCErl-fuoe6rC;-@I@TA$EUQ0Nf1ef|ZW1zr}iK9QOi zusn;P>4j3`Die2)D`TZ`CWCWvkmb{davEZ?4?hR|6cNkX2J7p+jp-`6`JrSb5OU)r z-!m*vc*&ya0lViRT-QHWoQ3n&V=*W;gU(i6zZDrOp~I?cs+^lGuareqH8!4#y>K-s zcCcA&%eC&~tqH#$bj0*A2)cOD<04`f4iO__{g3%K1?5H6-jI6IxGGIKzU1&E&stw~ z5}lALqVIB1-%zLV^oqmkEJCPEtMdHc;(vmnIr1d0sTrq}(dUY%=Sc^rHTY zIIlN`(+(bh*4~!4<=mN{WB4{6N#ol{l6w79Q2z6-^4-_Y)PuIhK0;5IXZM+JxOJ;b z-rU^GAzV)!QYrEh!PijfrgzSlk!=>+)6HN}`KX%4lNAC|x8C|vCa^rTRiK~N1 zoy%K}r1z+&%EW=|KiEzGfHm^3COQ2lOFjP<8(YWJPOsNctk+Pi*HHYQQA4r#9(V)5 z1dA?VdIOuokBWp-{1q_YhQFA?7hsT&?5ziFr}m#h#cIcp=f6q<@AE0($oPmn2vtgv zM^^&HpxzS3e!#%cY(lnYgQ!eH1l5(Z@9r@C>NRl!2K4IR$ts3cQ@QwRM_u2%d6?}qnJCpkXWK88coly=TKXW+c4=-}l+ z>#;*gl-Z?biP}@@!Q4sLJ27aD^UkjF;Xi|8)UqjZU!Mhe=+)7WHwuk@q3$T$w8?V+ z)4O%f_p40eC`KwLu71v{*pDzWmsRb^B3isSJY{4YBsd;xY)BZ_Mu%oRY<2&9>!^{4 z|53L`H9biYSrnbe>Z#Umu7DnT5eNGtOyl0_y^ZEw9t6(+Rp`cTmt$GQtk^ueYZqOW z6T0mw!!`S%Vx#E7^DYxj6tXNr`%HmPd*Gbb!0?zUX6z#Jxo(tTp1#mK*Nr-HQ>KyD z!_!EF8Pja+2+^{yUdelyiD^B0mxPh~W49+H)hd%<)Xd9n?RGu8 z?@otZO=UmQJc+wL$M2S4)pRa=!7k=Buk=%5I*r6gM;Mjk~>1^yB@t52wg|$=CPQPjyl; zBXtoaU!kFUpA=9^5PB>qmmW?6q4OIFrkt7kkvyL()C^;@( zSZ{$Uqpcs~-}3bhXO}1~Vm@T?>j$;m$j^n=xNYsZ@?^ned;9adGj%eL=|`F5?};U~ zEA7nT0{=u)TGiIdoN z+_}nQP01U!>XeuCA2}vA+dEvHCdm^xF1F!gnQoo(yLsUxvrl(D3yeRSQKnQc-MDu7 z*8c4c@|QjQFOZHDh;bVu=9`@5zQWNUsC^oLQ7Pg42A9=n(d$Z8hJ zc8{CcT{~;`;YEg{#i{X%x5g-^Y1R`s;%~OlKaiOF7ZZ4_SH`X*Dc6yd>qyG~%aN4- zWvgx*Fm9B&-XpD#*Z7Jj?sIK`Wwz&=-41&B;|H{NieA84bFwJaiFVdn30B!}Bkf2o zi;=^F+!uQawN{z|&;q1st%FL&;- z_-@j}+-+|0)-e%>Cr+jB8O@h+@au(>@1R_eeUc+jEt%0Qg@mw+NXA|6DIe*NcNIf5 z)2bKq7ntJPaAm~JTxFWG%tbP*-a{c4(zwq11gpGz#r!qIIYyQ z+4#A+r z?2+y_Z%Q-Di)-j7Wc#8l4=hyN@`xGJJReUxX}h=~r1@k@RSMSruE;wz3PYJ@l6t%9 zrP=fTNb9@Vd6M&IwY+riVE&-JNpaaxuTX1VY5%&=qnetnB3g8dBCU3|sQr#$oL#bB z9-^>95FQvVZYg9~QljuV@E43geo?s%RqCN~xqcnZ?5(rkAnFQ3I}z)LRS&;hbox|> z!H%5cRtN4&Xe2W16zEEoT;XkQI!aq96Fa}uURU+ds=e;qQDtr}edGA5oow;8&Vg6$ zsj{ylM*7>0(WXn9AvI~5HIDC(iIatHE#JPOf|La5APF@<628ZM?;Y=X&$;*g&aLnFjdA{PFogA# zwVpNSn)5g3{LQ+t+VV#44XL}P0)O=W-rNvp%sc#9|2x=~HRDZJa6)EhQU zm)wEBVbv^U5{HWH8qQ_BCoG~Xn_OFH@^mSpP-(~Erio6^6NT&bY%I}TxaAGv6`S4a z%VIV+f1osr+(+^yQsWnMoWymMTC7VdlN&L(WpF*f!OF3oPv(aK!SOEzRCsO}Ct&Vi z<{^|zwHU%)Cy2ZQ&z}uT2l(>xJJCm?02UVY9mqR+!@Ochy--I?#e;jiza{Gv54sFl zFmW(jaAK=VQD3=6e(`F;KDR}|Wra(jncCH(8_}=WF7I%B)e)Y3^?e(&>@wmMuH^BY zY~1w}NS0eH7wx^4@~E-#_l20QhiN`nTICQi$W5R0@(Ie16j&v!dP+1Dy>>!FM~YU& zhcOmrEx&F^>=~5#;;P$HteLoKR92{~G01F?F5g>;?)yR=4VDF<4>;KP@AKPf-w+Yx zI1ONSReT5_1ttYZ{8eFo`}XYHi;6w)?NiBAY~HU}aJc?kM()4J`~8c)p??Q1@GnDX zPyRcalm4UF{U62d|0s6m zG8Q<{T=%nnHlc-(cxAEWX7Bg!n(HO*787|7Bwjd*U-HGQU-0rctmy(33SS_V zcLg-IDB(4(_sP^%De=;FEY@y+b96F1rT5J=3tFMB+q$FQG;5$pdXigsbZ`#ZllRm> zhcW(C+M?ALbu|Ow%jFAV4;QQl*)@W?%5ElVSk<|HbC{kf?^}V8LI-=$NBlrLVS;#f zKDh}!a6*IERs)~Vyz5JV6AXY<$1l<2Cp4X(h=vrlLpmzc7x^D-$kMdCK-~`}OgOlW03<+1m+V z5D{{Ik(6T~dlq}x%`b(0SRI$rWm&A9nNb{|b)%?E!v*egzURFqxReA|MR&raUoAJ5 z$yJdOMo(#TS|rjbYUQjs5KGn+HOxJgv%|b5)jQfQAKm(YFDdz-K3V@Gv;xe7pd?Hk zs2g$$y1XF2^`cdoaI}N7+~_GJ3ruK;S(CEV!wh+$n_>JjeQ@*?l-XpoPEVZ;%L4mBOCTD8}G75}7SiCUNj2-J=eKfrPZ2ymPizHj+hsFH?V*W1K zK>XopnzKc^nt53V;{8&%z1pjt7`Vai|(A;vVC zqh4^(R)A-s(@FnrJm3TWgFLIwVQi_p#WR!7ERS!CY5U0;3};&Cuh9dY85Zvs@ej7r z;nQIq<+%8?cs5zruT<{>z7+7SGn?G&!Ti2&xUt8`t}XO0-VRswC4i)IBejGr#@0In+m5vmjZyGB%F^;&#DXMc%5wzfq zg`8?r(LUxTTMj=W2@<2!A8+l)k_1bqR<>c%#9sFXn}AF}NhBy7G)LCw9bIb8>mI;w z4=}VeqGo5OLRzdIrwcU3LRO1yk|&UTQ={(-%^_1-pPNR9_c8igqmLJ;?Mrq(?Kg>j zQ8KR0dVzrF03k>l1?{-C}Nb8HTGH1vYgm=4B?uq8-KeO^`~O#u48` zKrt9v4aM(s)OM6N0fJxbSOL+saerG!(P0gF9yALsHuS(_0uL0j+LJXMOYCjGez$cP z^oISO#iJS{Rh0N!mZB@dBE_n29I<7doxH_gwr@NQq!o5jTCSGBQwmPw)lB)0T_rKR?O21P4=$(Wibjz&d6N|Iab|_unBGXbSyoE879xE{y=@G169mTjW$%( z9WHnr>rGKMhmPucc-a8;8EiGUHyF_dSpAnUYZzOg9k(x}Jk*2Y9qC;j0!JT9SA<+% zJLZGCZD4{ve`-qhZzhQnbExCsDvL4=W|)PF#fy2!*D|I+Q_w}&Bmd2dp=?IOJT}qR zZQR%DMwW)huhvCU4@I2P!$lfaRsoT^D{)|_up(=v9g zp-PcW&&e$yh`X+p^;4EX9iN@>mfNZY(=l;m%w6(XJxbbg@@^-PD5^6AMh#~nASf@IuOYQz_foIeAy7S@QE`buF z-T~@~8#i>fZN7^snHN>d3#^SU`Xh@dMNj5x%Mtb_*aGA-S z3$6MYpwl&E#QN(PeB-n#Dk_~(dNTBb72b6$@`)~@li>Hu8-2youuG&K zpE9>FVM@{B*zP9{PUxDawtl?HWZTux2%S=E?YAGsbAl6|9k5!df4PBLvmKb|HjmP5 zpdT|2-kKV%N#xOhe*0#yzvZ#!4>6xdkp8Pry7BhOEoEemUzzASyW%;#M;Pm(9bz}vOat@p!NVa9yCDonc`cs9a8 zH?5`{^G5BIQGy5PE5bl(3PIL8&5+w;4CG^THBn~#nrqt-=a29EiS`&k=SVZnZfM8^ z_>CE!$_m%+-w$2lNwbiK(=yi!-nS`d=Bz0WVvsS ziF%qGP)|t#8d8N`U!G6y<_WbmCH8C#YrH8QBw%}Ptl1=RJ0Eljjg?eYrQWt}`Q*RY zT&K8gd*`M#L$=>j;vsQsD!@(6{DN5R2696G`2IH-Ps?5d5QwUU>)`p6=Cks|paJoS zZkV7cL-c30`%(OC7X?4NL}Lfpw4*q<+1Gn`S55M+jg*yY6}??o31F8y6zmi3hCc_A z9Hhe-s67e_kTY#JW*t`M7&HWJ);XSG7}flI0*dVIs6!aGJq8a?li91Mb!v~}6E)T* zsOP2mUs_jGdj4F{Zpc!;Qfw24jH7JI_nRmjETDh>URgJ3uL7&PgS)6SAE)MspK<^0 z(qUGLC*?=4;l$1Hb{V-6=3=n3*w?}7H^zmN6J##0;zi91l!w!B++$@Xo@NmXpn|P7 z2J$0t1A5a9og4waJbin77czZMF83)+mH1+~&T^vdGEAp-I8}fAT+$QiIjfeY`nr*f zktI{4-S^vPQ~D)_3-N#+E#cM}$-kM$x1OhHf#g{&9CqKEw)BwQIpTHfVEA}KLr;*7 zBRM-&y~fZvh@Yik(kOn6RWM_0U3D^BOerwH7+Ic7H;eL z+*Kz^LYR>2dl)MT%U9!ftV0Hkp=zT^jtgnyUjEaDQE(2=$~~S?@@z2TdoTYD^vt@mSK|8PP%1b{NytHpuyj++ zxx=Kxl&}+DwI}0)e~^xI!;AZH(vKBKBV()%blKNNi!F-9Omf`KBE?c81h42CZMOXe zVxw-^TCgEFmN~C%{Za@MVAr{6j0u=d zLxM@sfdoH(bi2Tv4u?rRo7ALHgIc@kN^m`JxpS<5R6z5HuliW^`3SvB;2w5nEQz+s ztm=^VFj#{=o>cQf^*uYbnc7!!CnC;an8}y0l~Oow?-zqG{ydfFqps9r*yS5g6`8I< zPQCTEQ6^pqG_P5?RtmPxeL$fVmIr^1xthG;gm5hcgpt1jy=jbGLlrSc@hPj6t$Gqd z3~KE4YqvI->l2mH>?jUW`@Th!MFZ=bv|^9VQp#iMMDB-3^dJvp%3*TLD|*y)A?d;i zP0WXwt`nL{nBbl^E`x`@8nP~>x91XTXB$&N=*3KaNbMBUH}?-k*O|@LJ~eXdn-e67 z6SJw@>-#JdV4M-mTtzE<9{n|nDbO4+SOOM68~v=Ij8Lazud5G%D!)rpZoVtCW|*nr zIepsIHoJV!OB=q8Dz$Cs1c#AaAT%1cZ0WW!G8&e9VEL64LrkVa)2n%PXh`4!;(A>8 zx7@v$y5>-1$bX4WXrR zmFUH1?(^)IjZhea(qJ;N|S)^5UOI!QN$KyW5pY^RG$huEr*x{0rI_Y+sP*BYDjlGX;{kQ80~w9yYXA zdSJ3ynMDEpuCPsxn67_;O}{Y zsS|@&o-vn%4bGk)-4pmQo|H(*cRhMmaAu!#UHyxva|q)^=5=MjK;X2<{AU9J?L5da zv##g7RA)a?UT~NjCXExGr@QLB9@pr+#7+$3SGjUFe-Uh7S*;e5gTL`JiE};agLzDW zlEqt`PFbxBciQ3nXy!14a0eU>ztnaGh-dQ#4913?B4V()tXsIVsKk<*g}DpR==Lf7 z@g36Ez4RxjIuOfhvvbaYCu?oFC*zh(qIJh*OLweXj-X~-0#2w*ckQNrIRcgH2~@=f z3=V{4=jdat=V+cffCTUo;mVyx_}=t#qh7z9hMQ=u%zYuT{JL_zvvHkf2*ccNz7e-O zxur8YQ;^a8zi|{>WywQ-pRWkU$rHROOl?5*Y5UJiAd}Xg2HxQ@KX=3flLg9rXQWSP zQdUm)=f|nZmfC5%IN~MgkA8a2nxpbij($E;oo=AMD4xvp{dcnU|!SYCP_cD+%42}T`@MUm=6s`W`AA?2c$K`y;+3rY&eB!K9d0?%9qW)Mo!Z-eocFa`jW;?Ae?{aS;( z9ynPG&Wc$n%B1r6VyKTaN-2K;E`J$_Mb0p`Id^UoLNv9Xr1F9zZI$~r1-LVXgeV(LMsneJeI?;)F)0jE0=BM#O%p}}o|tD`B5zbFT=At*Spm+AuW ze5eu8)W}uJB zzFVh%ebi)R7g?m&@_zWUQtSA}tKY9Y(+>P0Bt8$jOW^F3W_WKW1@CHnfM(8IkQKAm zzwmp_69tv>ZgT*zdm3M?J)3ZqSMb@II$N9}UVD^)i2W|w{U-4Vnjf859$yfir&8o| zh5M_kgu2R@t4>+EobUeYI5+JO`(2H0bhqJ<=D*pM?tdDl8>g-|pwX+hxNNZKZ=OIt z{_DZ$7Wf$a!CPkMS#`KB&63*h>y(%v;18H6;18dRjOYwogQ;mW-Vn&hbEft6BcoFv zLVO0{CVws@ z7Z^l{PL7PE*>wC~Y>_ZsY+|J%X6&>7N7-6~MZVYvm1G9U>Gj)niV>Z^7*;orYaeFu7!1#K3jr{5NKbz_e)(ov%064*(tWue=* zNIs#t)vtO&L*t350Cb_VE5J~}Fp43Zn?9jQ96CiwF9i0Ww806@M;>pg;;nza;=?~( zL7JffeX{{mPVa7!PXWQ*1^-h57Tix*C915f?})iPd1Z5(H~x>?Q!6i`ZvQsFeS!DK z(Ak;4T1$q;faUdYpmzE#?b8F$i>2pG*deMu6+SrfTshaK&!}k0Y6s_5wAiE1U@UJQLu8@w}xy7S}$ZdFue21kV-b$X?Hx|M%-t#F-Q> za9Bx56r{`6-$(tHbnnw|oBp@3zy0wv=uDEY>*=5WYZ89k$LRgc@IyemCV9>6NkRb% zDa(Mb-vP40XZvq2`n3b_RxH#d|L}!dC}y=6pl1mDwX^@68|?s98j4@WHFUsZF)R{K zJoohGPLXK4`DAKGy?&%H*lR9hZO#=`+;;X+6C1&v9jo^^^S%=$;JP@hC&ILwbM@5- z-b)gf#EW();fR8Fa%t$>vgyj?+Nnzn-oVb6St2e>$>xeo1YG0Nu+~+0_F&tO+#95! zowGEys`cSPjd;(}r9x$;jtTiF*%~G<_6xg6`!OQ62W9t800C2 zKnBOo?P{u>{RQa7zz@%5lJEBbd6~mNlV_4^ALbAW@BFl!zpLjACqZIQXcirTGYk8R zP3tcIM+nLM*9r9pAStm|aF8QuD6o5({(KrU=?(7=HTDS_SY_Z@6<{0*&W2fufC5Jw zJ?0VsU^9V;{Ji2PH21bH!EuFBz^!p7G;I8N@L@?H=>Zoed_t3+Pe=U*u15ez|C3O| zkHPsdWaa{>nBl3f>k|squ304oWm0dO_qyMxy|?2zhN`7{`Hxw3+BO_k+9j6O9WRIT z(L3icI{J~Ntn(Ij8khIBkM@d_IvfdT#_c0$sis^8Ux4!Xy;v{1pK7hI>bIB3*iPw7 z)w9BteU_Sw`BRtv$n@^~Q;GWO zHQ@ggHRb=78)d72Vq0(H;+v$&@^~h1Y{SUMiTATBDwPF=4tnleyl79sfova7)#=*m zd;GV4K6JYt+naG$zALHEcVi^Y^>I)b-OU2}xC)WSR~~36B#4#v(pR`WbhC(X7cnq4 zzpR|*EK-u1si?)nme{U8-j4TC(tRHvQRe8w?rQebQ;|-^T=a8SV^`yj&8k9frZnJ+ zF+}5gDD2=^Fv*@&enOKmz6eK>`GGx!0D>8~a8aNbpRMKC4=@i3CIFK#S`D!C zVMp5yJa1pD3_zuH5fa}oW>RiTm*V961tb1;1y_nIoH`& zpwn(ba(A2JC*KMo<0xuxmr;AiQ$);{r=(T~g?C7cfNXTk5k=8L*PQp^cmBTf-3oT) z*+X#L4y3M!aykqUE*v`^LiH+NhU4{Ge*;wL0AM*`2W(Gh2KXlcF%4e;2x91Pb7V!p z0Y3ZzA{?Tp7N{@4fcz8Uh|(uCie~W6Oz3R@u*>lPRz3$%$2q|N*ywtI7W`2L_gugUeY^XaLoM>x!t$RQjto|`Iu8f9mQvgN7jG$y-R!m5JCo~b0QJBMWbUzrc z(`pLrzd>MSaD#xUxh%5}BFgY`ozQ6hyauRTJ5FG+=*SxRzsR8q^cJM4Yz{!C(o$tI zDa9M`SXTVn;UvF)@Zp#L+n4cm5tym;fdn=n_=dveNAS@D^l&X8R#KU|yS~O&r;1Rl zOVQnCj_A_$1Gsx*4UuWHgG*xf+d_C>U1k9GF1q?c0BP=$M*R0sT+f9im{abOolH#M zP2}$dF@>2<)qB+`Q1Ua7bRhtw_TAWB?6YMvFjp??9y`ox(E;7=pxz|(aJaly3zu$R zmh*W%9-A)qx>bYwihot3dt+J74k8XEH=}aXxK1%hE;}wTd1U}lq2h|-?k1(URS*g6`iUuu^&kvN^aae0~Ql*bJWGCh|| zy?iSCKWp0jJFKAp?xCju9?0c}f5U!%EY+lTpk4}ibK&UyYs=J!)IS$N=+&*Ow*x*e zH*MbfPg{9^JwNdEHeq*e_MT1t1y8?y7ouV8PmOEj--g?6LNgaI@zPXD5~h=nXF|;d zNfsaqVxGev1KJ;^Hlb!5it7)AjF9yLacI6Z6>xMb*rNm{TSu74MdP2;Qw|nmz8o&Z1R!-96dB*kt#_D@?)#+Q|9w@bI~VRaR%l?=S|1q( z@w5bGm?1A8a)6{0?D=%#6kg^z_Um_qFCooC9@v|2;bvg@-w?Nk)b8NbL4avDm;`5= z!_EQn>YIbGj_znY$UZVVn)rL7Q%;)lQ@51J91eWD=j$C;)3}deIYTO~3@>7W{cPOjg#5j#s`B>OJsaCSc3_$#(x|N3Ewt-G53);l7JL2RqU=qyN#0Ib>4JxGVU#JR z*0*zP|B8xU7OV!vCGvSLdpJyGt?f4w_*l*jU}XD9S|}oUd|W-2gQ=5ZkB!`?`qMMg z@!&JiXuKs>`aH~Z%>RT2gk1uR(1>r!{O0*p>2>EtAQqG^3f{bd%OZHn{ ze!pQMXjam+pw*I#m z5(nGSBwG!+VN}~qnA&idAj~m;eh@n&Jg?|>uHmuS?!AgoOro@?Ic?t|KN zis{rrhHBI81*oK=E{tpxUm5BaHTFrZ*lz62a>tB|KNK}65x-p8Ym$Rd6lv^L|6CX%5 z^7VH&&MJ2BT{fIDD~#MHHJe}K>N-|3Gu2n*lf`SK?E|NE{^p0bFj-O;@s2jh?O2^0 zGQ1VW)sK^oX|5aj%ti7hq;`h!liUIp%D?S-Uv}3xM_BGeU-mfgBLx}|ujYUAc1rDN zj{M$y^+R_8c^4r%+AD5B`GKj|K(H%qch_^dG{Pj6%@cy`U7{1Jm26DP`?<7og2vs9 zBMOK6&QJ6Vztok#Sq9C@NKFN{P3FV3ev{NCvss};H9o2052S&j8AStmq2gm#d*7g+ zAtu*$cMS=ezl`=5f8p}}wKDu`Y~(*N6d*(nppFpW;2oL%X=?GGe8@jrc=B;xvty?Q zpfZzG7HP(v6Pg>6(hU^lJGZX>dHns8(+)*%3YV;@Y;9~37w!%}k(T##vLY!H#UG5+ ze$be*ZK3Fyv=N?FdpbKhd7N&YR16Hf*>$$A_5QZLY>{ilmv?Q`SUWE^Ng~?|Sw<|| zowLtB5~Gu@pkjvX5bXs<>)uE(_0ku}9RVl3I2NUjegzSe!CcME zhX@V8?qBqTS9^EQ(NzKviVHzwi|Q`X0yqfZ!7cDh&gj zmCJA+fcpxANd15lu-pZ>wZ%YgNkjYAkusV27rbbHLeobFka0@5WbTD*CD+Y-U&phs7Y{IAntH46TS_EF=Ik{@^`g}!c=abKV4YB`gGem zQp``7BYD99yDrWFr9%5vv4d zXFr3P6x|8{a(BI10f3YZc`%47&?hG}rzr_&IE8e)S38j>vlie)0_%e_th`|mvlqi2QR*K*oKZl);=@nrYKNdC^MwBscd zC(zei9>%qsLe$2F2@pKuZ-#*uDmeyfo|xOcz`fsSP;SQ))5kAf>yu%h5xZQn40hJc z)g4_l%*YW|bmjwi)1?z5qq7MCqLATWUc>RRtd~z%hMtIGEn5O1FdVBE?m3hCf&)%N> zGgjg=s~SN4XaHYQ#U;({fkszo|GvPLW_c7v6L8M?^HCO!p|UH@PR4WQ zzyF64JbxRWxqORH-!8zH$E|au9hz!Rz%MJiow?BtNhdUAx`-|(klUb8w zL7A04z;yJOsq5VILxy5DYihS^YwkOT4rKddB4*_KM8Y;3LsCr=a6hMKa-2naRdsZA zW)#1aq}ztvEweF7S+<)se%~jA{ra6(F zEP@o|5@deiAbRhcc$dgpim~@BQtbUf*}(TbzB<9jKblImgtv71EQvaO7T?08&y#NQ zNHeZhk@T=EZf*3<4Icl;T*CR`=F zy*950G&JR2o!Jqc+3992LBNAFjrM(CW~{N6Gznhbm~XQ2d4I05n6tt~-o$MUVQjYF zTa-FtY7P%B+LCtL+7T7+v$Po9O$^Z+^Vbj58?F1mY5BfsG{6&NSzSN2i!wE6jeXlv zHRiwm6;a#vzWK#0PrU8xM=xD-3tw?rmESGo>gx7MSdfmpscYF3=|pu6vcpS$Oh5Hs9Mo_B z@Nl(Xw7vU#irw(IuxOmo(hprV^7rp$-@g+V^W{-t&;~9DKE9^;nJVO$*nl$^O>X@A zr0w63OWN0yi=Zh7z?G^_@}_8!?@&X+UjKeEeaSwNx(*kkSytl_0j8mch3uaON>z4~ zK!|OC*}nK+9v(@fx+#3Td00mSas$tK_m|Qf>rjuwJa5rW86HgUxYOXLjYjA??*lbo zLs9jxs{(qs@N%1E);i<_e=~K`9@KyAHuAE2D7&3GP3Z8ZnRkIlzh{u-pl6VB8=nUx zGuyy+>R9}{r+i34@Yp|7Wb3CilWgA$sFE~opFIYRU zDb7emyK$p_Mn02}Z&8=Ect0k!>#=0v=v;1VV<>Ut`OWO1B-(^4M(div#FXU2qPlo5 zF+H4ba%bweLS<#fjORPLLZ4tAf$t8hdN@yYlaz1M4w*O9q~S&1+}lJ58xGfowpwpf zno6N52bxvmo{uMLQHr)HoK>NwI@U~-tX>+Hez^ zltb}_ForfWSsKT!ZtSjBV5O(twdT<^d{-NP@IrSzMP8~&0L?YytK%@4(8jqH((JW6 zQ}As^$-1d)%1CcEzUEn)`BbfGM*XnjaE(om<@<37l)R*@5m4yAzQW&JJtnvQ30)!W z`PiOlH|1c{T7RIhjb!2-x=(Cev<;L-ol0PuVAkCf6P zWGfB7%;*DsYcR9!G=JXV)H3=Y>p8PJymgEwXq~e3mUa-bB~QZ_44kB;G&_^W{kQ1; zdNfO=)4jkT1#Rxy6xFJ`NpdEG{4MNeca5xP-k6)~$~;@)OAL z(LIv19Tptx9)?-s(5zVX!6>qJ0W@xQ!pYOm2S@SHpLko}Xty+r_q@5brX+ktIm*)d z+-Q3{!o^u!RXtzZb7TKm^S8a#r)!hu&1)Yl)|ZcNezBn@Ym|o;+%C7$N3CW&<4!6r zycg;zRE}6Zs>poh7I@I?d*5sG$)2eHrlu;x8{B=bAJwgTfzRr*-EENaWf9}bfXsYn zzjwj*o}z(qR{VH7QXsW8x-c$S`QCu&mWXJxmzx(J!AU3hE@jd41ApP4OC?(&PXU={ znuhC*SICR*>g(!*tfM-Prnc?*E#sypw(Z4DlV+{Cu_j3aN4-Dwc#7i|CDD?PT|50c zH@wo?`h0cm0Y2@=soxquEA`n6WVuU*^q1u3rPK=c?z+qK_Z#SY_bIixDxgHxa?+dp zTZzwoN$SVE!J)EW4>q;;C~(Lh81(uP`Qw8NG@JrCG|B>b&rf|8o_a3pp9|NT+_*mO zFKKHyQQP#RIf_Y7r>%ahLI^10Sukvi6YtXzYDm!&KejFBX&lk=2d{Tnt_Ut`FxHbM}a}10F zi*lzv>A+^ZLM~p}yLR)Y_y?2y*^ViFJu6JZ$ZoEN^3UQp;kI#k)fRnKgE|Y@*|{0p zDO-RZez<#6SX7I}z%+X?!s2~SnzM!#$}THHSp1&7*3H!B2P!(d5q3qhaWeYCLi@U# z;w|nYYiWtnkm0>Pi?(5T52lA*UVBXHXt|bkPt9@f-R#xfev__BNr4QOK%nF~(14tJP*mY^mCHMd_{xk@CS>CmQ^p-W?)oNAMJyWltY2uK zGD;an-dh!^3s@|*^*B;rDlBv32u4Ltpx!Z@o2YH^Sa;jIdM&_8plM_P>CB7vLnP`> zaaBq%jTtk1x}IViN-UITu-cJy8Wi)=!>_RCe+UrrDk$tehP?JibU7476p6fX2J z(OLO!n`V}>&MBfRx|SW0L3CI7+XenpR!_&i`0ezM)aP_2hK@A^6say zyG|*_F~7n5cnAQF#+XY+boX~EyAQK}Hf+DW7}yTP+OCK39A;B*1w*K>u(z)My*dOE zy|;Qu`4}b%xi=pJ6z5Uy+{(Z6n1cEVpD)bgBMzh@OCtSY!E-e)N#Of&kW15~+$U$s z2@jbgZ>)!Lw#B#$W8F}pXwAH7Dl1otm=#as;##;*I`dxi@k4T~O+}3*vv-ICJ96SP zf4E62!q^Dv(QNWmCee3+g8E)=@(GtdZPhedo>5;aStxcBk)_A6&^x-njgqpdcG;Bv zrP`cA+9v;8kn{g_>-o3(lN>_yoQ!Ned+g?ZwA?Ys4r2oXG4c4G9)0QiL~kqukorK1Xc4J6GuBY4%0`==~U>b!qN06+bazQCJ`&WEYDIi>c$*ZA33)D1Qvq0~0iY&g)(a?_QL#p#jJX!bs-L-?;mi^8JiF=Ig-+axG zdh7Z+ql=3bO9gZ>zRL)Ei{t~1KcqC)Vqzc!BZ}UFA{WoXhsoOe9bt`)!aQS&5r-g=SgD&*zHJE#*V7|%l&{nzGCGDS4WY>q&yJo zbmWx<^<0(er0X}*_}fzF&!OkUF|R@+4{i$+r{IB>*`9#&s)^d$K*= z!-LBob=okR^Jufsc8!m5R3Pfx6Bv6rv{=m)Npe?Jb(QCwYz*-qU`!{CE-zc@e6~ey|aAgda!LJn5Y`}Or z1@s`)7o_oj81V5h8=yh@)9m4eCm|pkX91OW=NB2$=zjh!pt0Rw+d}`ohyH(fH2h?k zI?*-(9+;yl-{K1XFmx=Bp+u>HDYH{Qv#H-sXw-YmY4%N@-3&tr{ULcPVHAyp8-*}UK)*m zeh}SM&*Ah5O^*o8vDFEUDx9=P1M&bZdjoZ{Z)rE3$Ur{PLK@Bg;YOs%{k^CiQ%Y{Y zd4Z8uJv5Y$IcDxWnku)Vaj9<1dynEVZ;(%9zTHBqMzoi#v^}mVTSensD<2bu(u)e$ zxHF?1CnRuj{#o}0`mm9rY;CmQ zq-mh;)B6^aYV7lp%y{8WJ;+5;3*O~pt_v1UOXk5HqZyzK*2Hu1MXV5FLAG1RzKugtxAdR1mk#sB(RFSYDAQ0%Ff`$euOAtuAD+$Z<0;M~R$nrYWe z*RQ;3)#9rVZP#RX)b#}lyyAflTO#0HwyQQ)M>@F|rS!-$_iL(1O4Wv5-VM82kS1Ipc_6uO9wfySYXgv6v zZPygm7D$59P$!{5aT@ZjZNg(}pt4my)|z^vcTBAw3C7@D`cVR5E6Hg%vS&MOQKOiZ zK6cwID_0n_pT7xvWm=Lcu$Vqs9KJu^#3^RxKA_a&fJ%FK_&paKm~DncJBkb*7&2v* zjXyqIt?(49yDG*@Mjcz3=AP9|@%3AZ(|z{p6WM??iGnG3Cus16cjhUO{L0AMl#(`K zQq&xtFLS|)%I$i3rvqMtZoh!qWzw^QMCP;0Cto6kcX4-DpP6MOSVpyiU4rnr+PB9x ze~cyO)_-Z7#Xzo?m|BJ+X!qR^zjeLu813jSKL!{W=t} zXnBEGTQb-x*q{Ovq~hhfh;1;*uyiO45Npkp^*2vRl9uTh|FRf~HG66H0cC?UMvJkg zi%w)CP%6I0$v!wtQkaT9_~Rw=ElQQpdrA)MHWAUI5{4l5;gSJo9)%*K}+Ai}%zf>bW|^ zE?2*fgKjh@u_kVruFF6Ds(YAtzk|2BTTPOKQWI%ucF-iI%aP+8)8}ZdgK>*4l+e(y z>Yb=lj7C6Mw53X>TRJE%EF#BORJ=6a5c;}-7`Pf>;+<(BB*&6D@gR=Xqiqi$_YKf} zmmCSu%pMI8363|M^0TfXB@=b0+Z(vAc0QXG7^Z^Uvknq01|UrO-bHrZ2Z(zaSJkYC zjeDppPWBb@7jWA)R^LoVl!Coa>E-cGmaeo59yby3~1T!TWdoREf+Rl(-i68)j zd)>wfGtb1$#+JGVdp5-E<;PUP3`jaoQhz*A(=H}?-)|L&N?P1w%onmfM#kB&%Q-!w9Xxsw8g!UCvg+l!?4~d_z?ERX)*?T6?{P+Gg#SwS(c! zBiW{4`{O+AmdA*KyipG}T=hg=YOxEbUEyg{;NupY1SG zPs{wgfKfr}#zAPVo%uaW8DzYH*`n>Vqn#tYT1s#2nuMIY`AckXJ^hukfa@rO3`Lft z%H_x$Oi(|gr2L)JRh1XL(iUP9idCv}G-fI3@|c)QFm|GMHzpQyDNF=rI=P3^#xEi0 z!uNPQ>3P5xNg=p{PH;@HOB7jl-jK=7hU1m$I_K2IL`N5m!g7eaV%~2b6bv5K$7qSC zpdgMJLnx5}lw~?jCVdeUT&FvJLvG#7Y{8j3&`-zo2lcvNs+WPVZpqqXcBEf-uF}Ff zPNW27aJkSyCSpyoU$kGmK~Y>pFGaNbh7Wm`0PcbFk)3LZ^67&~YUT@rr@uqv_fL}& z0rH}Q_<2%p$o$}y_58Z-s5T)2NHBct+W$kzLuelyF$jEw|pjV~V8zb}@L}AU(wNs9)HUS--pCeI(d*O*7lA z*}u`J?lN-nhiM6sE6hoyR-nmQXXwlO*SJD4Q60xMU(3o5`SrOvqlwDq`rY#GlxPq9 z76_(Jc?o2`CVymGci1;QPSb(7C$=^1kIda?OFA zAXI=Rp><<-`?a{7M+=&c*?;I>&m0WbJB9T@T9}FSo&CeO+akWUie!^6GTPbSmwqf| zZ{0HwRtsR%1Hv*Ywb?d_kULZ43+cw%`i0K!lk2UsD_8tM*Hj zD?I*WnkHo{i2QxFlwO2?^MEn5u zKEWkML$wbl9j9_Jl90iCg;Wo?34v3=@{`Y{>iQwK8|sGqPZ4c+&XcuXK{}&mmM`WY zq5>8^CqQ}Gt>&kGARKCU)H)({Q;UAi*u17!*0*WA->cESBwr??oxxHdpOxTo%;kr3 zN<)6vqT_xLnV*NT*uk_W zD#K*%96zLv0`>Skkr$HyHTw*7hOCYoOm05F%HY|#c`dTMBP-=d8k`O!J!>~d?@K`( z3~nK%Dr@)2Cb{0F9jG7cCO9&_QnF*Nx&KBQk)_xmlpShbG9w1q9;u~ixS+n0oNIe! zRY;V|n@T-H#{B8_hwF=#sQW8Tr31irC|5*!|(8bmSQHe(A8jR(QE_$M>Pd!P6OH)(?@# zjSJNPL!W=X1ahc^K*fGDq}q=mHjDcq879cj96y`J>imFc!bz~(cJ3Gv{~C@An-p ze6B_ip(oy=!;!gd*NHpa2bCXBx-`p*erXtMJ5d`|Nh!gk}UX$_F zlF(jXZzguXidd1~xO$0KVwcCQ;gP3kmTkJkNYN<8-O06%{qAX9aMNLX|pUg3=4lPt!j<+VouI?ZEqX!r7{OY0e zj@X}ZI|lfdTvk*mf-O09tfa>Uy*8l(c;gTkn{jsKxR5w5=2>1kPTF|N*?w$rB77Lu zh|Rj*Uc#`4wI%7;=gTgKot`>%E3@mvV4!fBY?E)X2jjs-i|VX%*W*|IdFIORw!2w` z{u1#{lQo<7M>{>HU7+Xh(|4@*6}8><^6);jTYc%D)T>;@u@zt`EHR~Z?7vLezT|#u zTWZ4b71iq`12&v0K=2W8G~F784;3Jf@}MBfWB$8Iz?n^lwkWV4D9^0kb7r6-#yh}3 zais8&4RQJ&5YZP`^GqXk)DNgf2hCzwzl?JEqNqX+t^w9f^G_$4o+lKAp=B2(`lHg)y?oDOGu{VRUNV zX197bX5}^hxKoJwIP#6Gm-vX`qaqXmS?gH!L=(kCUPXVMB36z9o7~*t`iT1nOiz?9 zgi`X+5s^si)R%$ASDtdzYM~IpSG^k0=Up#$F`fYB^&&)pnm;Y0t8W7eI`Ms10(RY5 zJD0>c^uy!}HUZ@-1Rp4YXz*ip2LP1-R61BChx_U>=80!~Rj~gCjQo!s&9-aWDl0$B zDO7d`0?w6EpQNOTI-z0#p>aCJXWLrtm0OgL+oi~9gDCPW#M{|60ZT>=skp^A^`yv| z`UE4u53L5RQS%wE+G%cnF_2(RV~QGpggnIhhS!s{CoQfk?8rG&Lx>oekY){7|6*d6 zey$BhinO@u>3OEsFkbHMh<8A-4taqZo*fid3rnlV2By(Ar6s}op_Bb|>TYln)FIN( zCOg-quzELc^QH@)&f25oqg4`U#%oJ|i7lDHEyiMP&V?98HOAs-9-C{*{A>q}_hnV< zaNV1dCDb5zdG1?rN94;67A`_-FVz`SO8LqWbnZq;U(jevWU!#MjGUvvr^GXroe!N@)?zAi7ybWAh1U`QvrUFgI^!7xKXg&4RKnR{{oRM|7n z3tS+!i(}Tu4X>Sa=9-Q=*+GRn0n~`RlUI00-62m|ufm^$mxHorY~q}f0#0`K?ksEAc*i9he}g8(RHSly*U~uCJsTo_Y%xyDtP(+oM$n#!o=P%Mth2al&Qi`C*%dqLZaHB z0=zsKf>^+qd|)y~&3uo{nuRpfqBYFNGDtWH^^ugN?$Y=|TccbEpqt01mI1!X)E};c zc7(9q`h*2Pbs-Ud3v0=2bQ<6RubMzNQ-<`5XCa~u>Cm=INL_*RA_yAnW#3k4gQAb1 zc}CQ3EV-7VjJtQ!aF=9GEu}`QmZH5S7EjGdYxHTQHF#QSjdOFtDkQD2%C|n%Xtw)r Dn>jqY diff --git a/docs/reference/ml/images/ml-data-topmetrics.jpg b/docs/reference/ml/images/ml-data-topmetrics.jpg deleted file mode 100644 index 80eab8f2037308af69a7b4068dee005881ec13e1..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 101513 zcmeFZ2UJt*+b0@CMY>3@If_c}9V0p70R)6d??gaAK%`5MKva4^N>f@y#7LK3qzDP& z2ud#k3M3Jbq9h`53n97t{O7xK?|f^%S>L*|=B_nsPO`|}?D{_Me&473e$TU6A6P4p zlNP4trVutZ2;>U*g|JAFt0t%bKM2Ii3Ze*sKu$o|QUV}H!8Py@hy?ftfv`VigRp~3 zw!gPMJ@S9=Ws7~v{$JPI{yvcP6{3IbW=Ld6#LbY<-?UWKA^OJVR!9H78uoX>$Y9o@Lc`FI;Jj%U4r2%KPRV~b^H6M!5MU}G0xV-X-w@Sexm{`L9SY2Xjr z5%#0Uj&q#gixq9{nfqYyWS#!2bL%Vu8Q^MY4Y< z7uX4GM~)t4Kg#j9Tx>_8{}x=}=&{op$4_0d;qVF*JfnI4gpl#W^2UBnSuI7#YaoCSa^;0 zR~6NMq&9)he zSEoX`{in-h<=!>l&3&gnZ2Mr4^nJTeyvUN%z@*BwKxL&sqB+6NzWV)ORe1C6j&L&r zpTxLGF4>m%d{j2Eb~3lzO*{jC-6u8E7u7lLT-~K!Dmo%sdDmjwTtyap>5VXR=lXZ|V%351 ze&1iFPWSu0s)*T=oCzbZ9;0=*t~n+dY6ln#Z44-!tq}XwwuJi?l=iM!%J*(ZiKkcg zpm)%ygOay~ShtDmOrZIaU6pM_$@=-dZY<93!U58e1$kOu#eyVAE;HdCaNB+sWL!p& z1>w9Y7UwU(f?U$iVaV4)7xvo^k3}IEVoNLtZQrOe0Xz*R$2>L8UrUC6kpmA*5>EJc zS2LS_h55me$bt;Gvh9_a9Udjasn1?vf;N|b2+zQ&KCvvw(3kxJe&#(Y7O?!?s5uq9 zXo(q9$dUhRoE!5&7mmpjf&mVp|F1@I%!WaK2!aO=r1u{-)L_P)>{*chtQ{5vTCwR2 z-)A4dkW?W1qq58gVNk%v8Q!`Vvd^`N1vGkC5Xz4@OAZ#~cNBtg?kUuTfEl-zWkI-< zIOx4~prjslEXZ-aK1O^_)pZu+3qL0dlGwt&rJu9UNx+ayt~2Lm`o1VtV^)YiS&*EG zJ!R-28wpM=xc5&-{nJtZjHrL6-oGi%|8sOy<9-7Rax@dzp=sfftxk#Yf_QSN9}7bI zNf_Z5o)0P$nQoP=`Kq{lZ33oLiz>c47dhhg#5c!!=5)sda|lNFmW}a^h3R<*7km}{ zE;^-!|0et%c6zj9AjXt~pHQNdI;^z%t5=lQj6bs~dB_itsu3~A_cE4VnL)4KVb6gtO@3uz7^c^cW-zL^za@-js>X^?B6>S z_<>+S@)D_wOrm%nv)N_-muL?3djM*L*{{ih9J;UKXc@?FmzdtiAVQF9Y`^~pi`bnv z4fB*i3{)jvWf=%@qpl_37~+6$EyIT9u#>vlm(JOeC8fc9H#Lg&z5BifnTk7*a6h54 z_*dNS5r?4NhLdQEVVGe}e+PPs?a7*cnLfFjpVL=Jv%?E#EA}zM+?%E(h~z8aOUQ?Z zC|XyVj7i!}bjC$Li?AaxQShJDN!LOw~UTb=d!=4bYHyqWBJ3KB{{dY)`>l2 zyNgTHX4}Zkuc0c5^duR%Wb*IR3~id69L-Yp9V+F_n>Ezcc6NdRj^D24iCsD3z_{N2 zOZA6s=&OcQ1c$?{1-3P^M!_^NvSRp&yHEd59W9{SBjbq)$=4%hnWPLIkx-c64P5N!KI}4&uP(JVQXcg$5 z(j2tVEW)q6(w0jlIDCE(Q6)cxQ?&@St+GY8Y!L18ywRf^9BJ>uI$*DlSvb(1B&+e> zAF#<8bQ8{~<1vzX)bl7x$v!dW`Ls`bfc~Ja15Gq#FN6z~;!dsayrmK~#us!Ko6;Ke zN5O^s1jmV;^(nQwXJs6ZBp*tD?QS$vdHq)C1HYhQlf<0zyH_zpm%?>1uKIf9lNI$R zWqd8=hN>^PC6jR(_?c40bR7kQYc-t1!m{G{fag=DoiALn-Zm@8+dQ_V>ob}qQ%7id zKKc&!G-ubza@Uf;$#9&FX|22c#;{F6mR{_W+=WkW2)bplTTN`=pK%4`rJJg0dG}8q zz+UG1+5Vn+>(^U0y&F#Uijy^nMcX=yI%v#6pu=g~{BjyRfYU;TY&Ej}%TevWduX%?irMeV7SctX|*1%yv>MM!#}Iu%5feT5Yf4A1L(T%@1qjy&dtdDQ6YbJ zRouh^Mh(`+@jXfioXnV=%+ZX~z4`ZCK1ns$L9n-{l)q3TO;(7Eg8}^pb^(43v6_-OIW7t>G0X)?)$`Q(r5y{Na zGEMzoHZi7Q*~9Kjiu!29s;^7@i!TpO#Om1Cb^nSPtD}?&6&?!{Mn7;~ecYw5j1%9h zOxdq~^j@?3{EeyuQKUeKccbS}@d9%<7h53p@gddl@Z>KLqPoWg0fVXk?t*_D37pGe zU@>p{_O~dY&U2K}YFJ}IDu);q>#5MaofOPo?O!<`7;G#^ZtDPs@}33RyZi(@?}15K z-wzaGV@>|ARITi8b;0$A-D8^#OirqHogDoSYGqc+ApD+dKTWra3fd%QVHiJt!|L0U z1|WG~O%P}x`4g0$=-gu(&}stqSLub%UwIhftvXMB{5Zd~covK?KI=_dZlT5`wC)1g z0yNR1O|^2T!p^IfTA$5%HMlx|TjqsRURX4*Z8q2Z@jn{H<9PfuBD}-Is*NT~6@MrW z^TC&vPR!zMsKf`A*w6;+>oP8I>f&@$stZcYu z!2=8h0$UACEO(2UTwttaca>I1D3u*3Qp+*oM-7kQAyMT7&y)oWx`&jk~3-W@QR3e-HR^Lam#0QAYYBKy) z=8!*Ko;lWhWtNnEdSJ~t)6*BBMJjore;09dV?6tme$if#w_K`olK!a7Y$#l1-_rjB zEZ#qPBs8XvXN{Yq$XmfX)ZossDKbw%uS-H@LYbugI}ev3!OH@v1RrT0ZTV}?F;cVa|)Q)JX>FHs&P3eEocI+R0YJ$^IcN_W0z@6(l1ptD$!3FydAEM7fZ)opoU|5c zNhbuv?IXi#`bW=Y!d=fCzGC5?)w#r$5z7FTdYvn3er+zR$_IXr+jr@he#V7I)Rc+k z2b!7ZJVi-R(dH$wcv-S$?T~Z=R=^%N(BbYWsXR!*rI!Y3HAd{$xyg(=qGDoa@xwPmR$+EcbhZGUiVS?DYzu!cm>oe)j7mR!KUP;Lmg@$(PsLwn)%ooXvBp99RnlR0X ziY1?1zWvzb`O;yp((BiMJvc1VXHIYs0lOsX@;##rKt|_1;3J*+4zi$$pY1Ybs1-6 zi-KQKxYey@SN4mzhZ{h|pWcAv27kYc5fuMXEfI)611h=nAir*{0@|vldUhC3SBcVf zEvdDHM1_?I9$m$}je+X?0B;Y&Y++<`-PagYGYlUFPb>*>#Ys%lj?`NcO7!-wjtCC4 z0P$sZc9LoNhSTBIRm0Aix=MVDpDhY!zD%1>Wop>0{nBD`#jzlAl{7WxU|%A3zp7gt z`RWivjXo1Y`wp{EV=Rc96qT_&g#4x7v2!3m&0#im8c=yLrr4aM|Dl5%{{Vz@lrf(V zn8#RGl&h(fE2`YOTY=cB7?G2R z%0SLAoXEVZrb6!URJ&D9GRb}JhA(JI6SB* zJn#8uG|~4lB_eZ3y4X)=HSL|2vZt(c7Qx(Dvmt%ZAwO`>-AimL=c4)B?*kf%b~cIj z-g!5uV`))4$5A<#XwgZnmUaAuUa^@Uhq0Gtm0fIiuFm|ZdEB6^7Ccf@{)jvEA^plD zli!XTJbQFLGUk1FcOPs-i>Fw3z-DMQrk-E0a7|xvqf5V5Mm6uHx4gYM(Y;l{+GZ`W zLZ5Lh&tL4XAzq!k(IJP31>}+)E9ZKefLy=#J&SNqaHFJLym0|F%+SlRx zW|V^OS?G_31eqYCTRE7uA3konP(14FqoA)W?4AzyQBVY%c;hajn4wJ-Qr`NZ)Tla zUu{vU2fdIEGbYEY4UjY$i^5421=p*`1THHMhci^|3Thqxpul{up64o+cO29o4sfj9 zv?wZdFq^M#H8_;L1{#2S@zaI{uUFJW5EM23L5f<+^ZH_nC6WzgxIB3GuybOS$|#W0 zpEdpSr;OUv`H2^?N?Cdi%RZs#*Yi&cZQ9#<64un0&7hpGIxzw_etd@!U7)A7GXAQj ziA3AUjcgv?HOLDy;5$aiS1uIgiInv$P~sTze6&_>{XBFgZ~|MCN{i0X(9yhlb=O^6 zO9onFTFRMAl16%ex>T01o`+6L!VK7YPfrdLpKEP2ovr+Yp>a2#hV)jp`U zs7LuZ2%^lJ7(SR28OrB^Hkb*C47NbB0hiyz^HWojgQHZqm=wF_Y3&0mL9@1w=H8Cp z&X~Om&tC`ZU6O9m8`J5j>$cnq&AGoM9qJg9NY2EnyjP+b@g-m2<^z$F6vNSG*A90) zFM`vJH?1y*I8*7ykKi@88?cFVfXV5&=E-ukR$I8oupnt4k%X8#YH~xAQ2TKAAh6+x z0$+nWZiR{;u3l}7qzSzwh0;o4SGTljVo5f$1mjXg9ltM=iLw-=5K{qIrkKoSAREuDONT=+pTCA;fnChuwKadb6O#0cS(F69u%?+)eZhT6p0dF5fg0Klupudxo*5&= z>$Xwb@}GYSn$y$0`-h$a1LVZvM5R2z z>^_E*-ojh4hrBGv5pYu-4ab69sD_i#Xekz?To1JOBrM3ueEAbjc7&OLn+ib|hBO3X zZ0qV6{3A>dd|&c!*MQ_kK`^83F$4fGU_p#;!&wl#kr@jz@f(Ivi^^a@E~|pB0Y2N8 z1sMg&oRqW7S*SjWhT3WpkIpA z2n$jIl3{%2hqYjPLI6>a;tI)v|N3OZf(TMp)(<&5{vIVfzrZ&Cai0Y_)ks!5tTeQ>*eT(@rm;7sQE^itvxjk{D@4cE|Qf)DK>7{6j zJ;%B05SrQP{gLvMDT-nzmWU34QPWz-l1$yb5#JnCI!fxcVZEZIaDk6{*W~Q({J!zsA9uVg5jSu^_))WNDD0cXwrCvFW@WzMl6E#LAFT)R&fT=@r3~ zA}@y6HcjfpE(QgwZnM)!@LLTF#VeRZCR_YCw1tD1li{f|XQafOEz<_VA`avLJ9;b- zLtU5~#IS>HU<9o!a|2T((7%pX1lZqzo1jVBZb;!z3RuNa%_iN_QQF zpLBM<#<8t7Ft$+hJCODa9WsqJp-z!=`g=m?cYEAI(&D*+;@}knh8b>g3=7v zw|Jgs7hs?x8r0h0&{@Z&NpPx2Nk}if+w!IV{r7&0+UYavN*7EFRQ>&Zqy2-R(r8IA zZck;n&#oVq!lG#hbPN?UfSf;|6R7K{FmtLTaY-C#kD&C41J?ukc=ndFj3mcf!v?7(*N4{vZ+b?sL=7%lck_e!-)vu8^7j5Gjkk(Kmd5)Gh7Zt4?2- ziSORsdgslVP=F`s`Qfu{QPI@O?O#%dZPE_qirpWl=uSr9Bk8|nq=DKc^fmMK&@l8E z>dpWN-LfEU3{TmR`97iqoUqs@g4!{8-AMIprorDVAc)JHA5nV76Ww}oG)q!pN^67A zO#}%ph$>sg?`J(ixq0|?cct$s+7w7cg+BiAYCd18x-CRepd`@7W?+cdN%|hWiEfK} z$ID|MWFLras*=!eIN zHqP^LbnyB5suikPJFSPg(YNSE+4*{GMdbozCo5in!A3RP-oa3R62QnsZkho-xP;0j za}cdWjVHghUifq(2+J$EvXjzsL9)(s^R0xHlavh7PCWaz zu)?YsY-HZcX*S@%-I|~KvHiek_mVWxUVUauPKVRLTfr222B+sJeJ!1^cB+4+@~Z+1 z5=03fj%tK&f2)M4!i5W`h>BtK+XI0$n_eqxS#DpT7BI~mawD{kA|HxliGImc1{6sIH{)htE3RM4LG6Ys(8 zGS{-Ta}1R`Sads~_Jdy1UhewG#lYZk>i6=@ntY#g*4Y7;$B@$70b>i}ReR=M@jVGR z2VzN?90x*F;t^J^u|YyC#lqaUxMs9bC`|0i+7on{tO9D-S*_L~=T*b4t6P)(J{OZV z&!r5`KHVS|SABhH*dpzDJD}R*7KUn*!xxBYn{7Q%`y#{9F&!_8f|9UA_(>F~osx4M z1%U7|GPeH;6S}!{5wIi0olz1mCPaH4r!Et$j%*Fx{p_05^CF4N4KU8X`WPf}yHUE3 zM+v|Aq|C57FSJ8S*!_--O<3vGVCD0gV#>j;_P?IZ%9NdPP;Uvc817Z#&cGcdnat?a zmT1g-x>;PS${ZqZc!iyOovE%lxKiU2s=ejVE=!m#w-8Iu6Uuw(rJ?US6XS7n02ciQ z0mzFI5m2Cl?m6457C*|A29Q)WIyr-&a-a*en*vXw$zw7%=i1_VW|-#zNmF2vq`;6Y z|H71-j#ofyjq70lZKyx{*H8!C-0$>sX79zGHhr#fYNWBsYuBXE{#{Sv`D!qn_DA_= zy@_>Ev@6g$W+}`y6cPgu3o zay5bI9s0e(O0jHGW|7y{!lstO-`smKewGDk9@>5zK~jNJ{0YTSjx8BOLp)%KpIuN( zW4MvCZja#GsGopbmsV=ugBG{|kc2iMy>`op%{d7a#ObQAUrvv`Gozz~G&;%H42?Qx z`6R+Pmtf!5Z~CAJ_qAqvvP>qo>YTMskj2S}pH5n~Hk+TZ%KpP&ZL7=8Rs3CFXJ8x4 ztvsW)!cE>!KEv5uiqL-eJ$f6YdX8xTm&w?9r#H(j*r72=K=KV8E67ww-v&acqhtLd zMoKg=6-8jXSDN7lB&_UkegrrMT!s5G7$Fqi%T83Y#9J_jIw=3f-K^GX_fMOqDtZ>` zwH{(cq4>+^6F>I0Ke>UDnyot~s%IVbDz8y&NNmbBY=BhhsGc9#@JQ-Mg>6Nt^KXwb zYt}|HD_|nXC$s%_gtiLtDSFu$4^E+j=$!u5+%!+fUyt+qBszC>AL?}24Zm?wa0QnH zzZpB0y~I~cf@ZhwQ41e1%%5lZcuvEdD6$c8{%Q}<@RdVB%V}6}HLx_iV>$hG56dfM|wu_&w9kdTzm^gie38xh18GTJ)|a~Y9O z8^+{ao+)(9;8LqZDtOKzeA4qnKE`>r8z$a#boeA^-D5tim@n>H=Xta18dlq8iJ^Lk zi$BV^ITIPjL24&(s|f@x{KUH*zngG^wM{N!OALK6 z@YO1Y*T5jWU0C>Co%PS0wlY_!|U2LM+cWAfAk>h52n4?mV5D&QYWx7(!Zf zZE8poQxY}g0c?;+&(Pje-7eJ(1qss(3je7sxWFw2ckSXA88st5>QFMnpaxa!*+HdO zMM$~k7hs=T#qf50+?2UARlB2heJ#l~O2Yp0i`DxBegVecG4sHt=OTAzQ)UrB}OuQ5R8MBMo)LK7;2AakzR40BTML2!4FYLaNe zZ$M%f%y#KL_1jO+Hd;^9W#wo?9yYe>z;h)rsGjtMV=oqo7Z| zIe|N8*sA`m<@7O-eaXB*8gZ;MJejygXHLB`{y74$f_2C9(9VSD>zm#b#uX<1W9cdaBaa-F0y+=SQi*Ua%8sCMWy( zq~IG6s}qQPud8r-hr#Yk&d3pS8RQ{T{xEysdsTfmStY{jc{NVT>PmP|;8g$nsll12 zMF;^)Zf#0x&E+zoO7ZGPIcKM%2*%Ey4yo4ppHB4pG7P*6bwzaQ6k^56jxDloj~qf| zuJocJ!n>RpLKwR2oY9#8EY}b00{1|Buht9$6HR?Kxul{tSn+!XIlrtfuDlQLVM%O4 z#lVCsfPIR?)a~sb%N(1`GmW(4wOia9MRC(k1&A<|!`_F9GLaclT7{iJa-Kxn`FusA zr5b#@uc!S@oNcS2`<{ZS7X8nhvlIF?E0vyN&%catyXvFuZxo{fn`Q^@{GRrO z297T{*YTao@uar$!v^w>xHDjfUy^Jqz=#HQ<=MQi{G#W0H@j3DAZ^x~@+BK;1QsYZ zR~Yks>v6FZPQ!am%PTqNUq#ILJ8D;e$yLYmJuT8h(_Dzjn!Q}Mo>TmY1;?`KfR|qC zKrN!Bj-eQaSNa>up&7SrGSAK_=TmH%qc$Pt3g3rE`79Ba zNw0SRd2Dto|2%@6^3{PlM88f88O-5#VORpqB(?i@<&x2MFP!ldaOQBrN#|b8#2YXy zXhy9*zLg9WAgnsD>=>GTho6QlZQD+&8md1EoOu95A4Jz=`$n`sjIkE%g=TP%hqM<1nq#e%-6j~gNZ&dGpN zLb5?ewe3memJA0qB@-4FO@gN|kAf*610Uj*(3feLEVK&_2=bg${ed_hAS&W&RhgYe zN{b2+2)awm4dU8(t35fm;eFH4pjS>Y=?bQ4>K#p|Q8*^gCwU~y*TXH#e7A1yFm9yoVf4!J4MAT`;{G_}St zemCowb;&u~S{sTZoNBY2-m5qXW2e^i_cgC47lN`leU+pk={IR+1AV8^c14RMIQ}qX zB_%PKc?Nw6D37DbUZuwEq_!@n2r`UjVUlF{iG>d+i7`Ug`?2guE3VBsL|0o+=T&k= zY5~XQCo+m z2Uj+iTj2N6F;SF;BW?_3pn-}c_hrSOVE9qb6{h*n;*-jO^?rpHfxIsPpF!k&*V%>j zM6?6RBNh)9AgvzK(<(L#DmRnyDjC33KKqR^HONs+qZ=0CL}y(8uW8d_DT_ z6(2g>^uAtc{cIiQs-y7wdj(}I3G`QY(ml1Njo8ulw3O`wF!bb%{|nxV9IOjk`T1ri z%@sqac^^WqD|op<##8%FPD(TWhq18q=uE^-j>E>-%dVa5lwMnH_w)I#Va&O!cc2B~iB$WDyRQbu%C6NYYB_hfK<^ZGeX~~ogVRT{A70?Y6%`|wOb9=mq97K+cSy@a*!ct!GL513 z{1Zx&{&buX@C-nbaVh94`vv)xjGNS)WT&AvqoW;7xt0%VcLz z#0ru@%j-HBwXc>+NiSd84^h-@=zr$Jgs z4nDS-VGTuaN4uI+`4%Xt^SXgpWm=xP*dAx+Wo|b~W7{|OI3xQSn}+oA)P4&q<7;(x zN2h(iuD{)T?|1XR#bP2PK%OeG8)U_Nq?Y%yP|5cU??S-D`nmrM{&GHk7R$p_0uHPI zX)-*)5OKfTjH32n8QQHwi)IRhq#9|=cC$T26`N0W^U?(SU6b>ARmZypXs3RfO$#yw zA4gqqksP?2-!r^1IX@hnHQ@dm$Et2ANYmlTJBv_y8Lf{Ylme1sT=DGa5Nb*q7<|-0 z-}Ld~4%->N&wwNAq>x0kz=~RijBtB``OG&adjMRZTQOd>)QM=AU)D*T-8DmqUkdY$ z2&>V}7ktC^b8P!Q1q;%k;!J%^|L-s_>h9nY4}ham2v`v#{tew;dFq|cXTVth-Jc&Y ze*8(Up08hk$z3|L70&|whAE>{dqG#~{SIJDzI7kNI#HX7c1f-gm9~nJo5uF*oWc6m zs+t+!^V|@By|lJpm-sU`lO#WSDx;-B{x#9O&d%z5(R{@&3j*RpRO>k);ove_s&D|) zoHGQSkDx~%mPR&Y%}JOs)Q}735$UE@(PV#oh$vcaycfzGgCGBa$?Ow@NmKEp?2G_P zuoJPUtx>yIbh;jUh1#B^?IAN|)z%~6_XTFnp%GQx;Yj!_R#MD~I9ZuWRPfUdovySK zPYo|T$J5MhTrX0p=&*$lbQ2JN+huRC+8VetB5quwfETb9UxR87l5d8SsNp!Ue1mM; ze{B^fm-b%6wmdb;KM9ZLOs;f=H8!NtHNM4jFvJcUqdu@8E2UU=rd<4}D3q^T4JxxR6Ewmb{v8CV; zVqa#fY)fb|i`-)tTsqmxs@(jAzSBS~zsod;`rKM=4Op`f%nY6Pbjhg6#?O}OY;PeX z6(0>uRv8P$z@yTt#A_xpvD7WP$283Az>zt!EMNp$mwtm=EXeg7F2?Cxa@5>c4)Wq_ z5JNKLfr>Do6l6-T0=2U5%_sN(64D(&{?L2=iyl*ZHWDkn4CV))RMx(zcNxiRqWUCo z|3xi*uS`P0QdDuq_Kmlu;V<;}c@ry&(p=Sp{e~7?z%@Kx1$X)6%*`2kQy#^%#6w{^ z@2-Z9P!&<*#pasOE?QSN`9`$`c1C_2RhUs~m{z@qd#qrC+l!e@k{|4vk*KjKv1ac_ zQW2O_s3|ak^g9lQ|tgwQ)O|nM`PvKa3+Pe1WviPHmc?shHzm1F4YNG zPGUhsSP)+<`v#`PD3_u(ze^f>le2&&wP(Ui_}^fUqU_vSk(vC|*47%t$tBDHak|@_ zjFo2if(d9rgZn##q~^ZYQ>{CEn{#(dY~Pbt1Qn-eRSgC?CvnEva>=J3U7t~l;I}?m z2c|`xlQ=n>q%0*ArfMGl%h?%#XfI;?LOc!;d{!QU|!H)ypfh8Kj0&fn~bEaN?|yN(dD7B18t|-`@n7^1^^% z@0lo0N=JifU0_V|Eg(Ta+^a?zCu*V3cRL0XxHkQ&<5nR3VI~R2$5#rw@x>v$(&^+T zDZ~QSY9EeEs3=&4ZZc0HgfIDZNKA(nM3SW1l?Y^w4uAc(#u)cHTa>G?Lads>Va+Bna8;+d9<6(BH^MjxwSKZeqOsD$_=P~#C4)<4Suu7s4@P_fJ*U)${8|G| z2Eqx}$pOj3Rn;8h z6-U*qw5MZMo*F_2q4T?MWG({8D2f`0SkoJTEj&T`wGVVVVo&ee^TQecW2Wp_Ao^J193ay-9a!lNjK2vZ#@b3rV1 z;JJH{HR`d#WMOl@2b7Bm=CZU#<1rEp`NQTHho#;$z0~W6Etnwa0zVPngiL}(Q-1)U z5y9|-6Qm(oMiTINy1Brg8udQH1h8vBJ5rkgU8oqaZr((TOD>Jo%VD?$JO#@3zt1{N zcd9c0i6_1~{6WVag&lXvtV_XsbJ#7D9iU0ePtYG#z^qSYhBhcw$=g+^(bMd4m&P+Y zTdQXk#aka`G?awWeUq~uIT6eZWD?^w@QS+PS0q5HSA$ch(x};S42ryiGo8+-gg+DKRrs0 z2d1H~%omhQSZU;(q>tOLKCyX3u&cDOr#n!rOY;>*Hw_&&R<%mjt*&c`f0Z)Pv9fM? zHJ{iNFh1x-8Iy=QyPEHi`O?0|!p7lcyWY;rUq82{56+1b%2uv&2n+eP^ag8iSEg_ z!8MjRk7X0TUnKdd{+R}eC|^UHm2U!neVGZo7FE6aC;l)zbEB@*fb??ob{PG3=;3>F zrK*4;Z%6NPPy1=5u?$qgh0?Q&SzHGB%C?pI@8_-bQX?m41B?RtJ!uJ~o$WZ_Jwy?V(fqtfE` zqY{6^`gW}crB_zdUsl_be@$Bd$_}vZ<;Klc*^%9IBuC4-AEKm$0|p)JSKpf3g=Vgv zkU32irSk!mE6_9Xqj^F1 zsi?1xiGeo5jZf^Wy<|`5IF`Y=(`&>AdK3}-cA~BhQaBSwtger#e)^MlzK&7U`Cdb@ z5u238dXkf5;*g^LTRrH~N|^pl3x}%F_E`nemyYk92V#fN=ghyb;$RN$hv=(-bsz8b z6!Ild!)LlN$6f0mbPXkDHlc(uD;2Vo8@)QYcFYfk_U+u%bR~>{O8O>meO;~1&9Dzy z9)gzcp^+8FSjd0|J?Q+vYK=zL@_`D(Fo@DR66|PSdpl4a;n->lB_oYs3z}t8Fp&9T- z9Fac@!^>1*PBzC+^zp+Ss5tV_Km&(4*+b3&dd$w)h-0o5nZCqM8@u1D!O)b6DJ=mS_g_sy_Ng$0J@3C&B5;qMCJ=MZwa>x06*?{oeAmo5pE^ zQ=g_qRykEV+Se#JysR+p(a|rd0`t&0cfGbq%~D{ldAXxzm2pGkh`M;Z99w3kVbCK7 z-=tDkoqz+^VAEWBiz4xq(euLxkxKsT<^!b; zh2zegUNeE!m4S~RM~umkY^Lk%HnTm6Vj;EhhZZ`OMksv)^y&e3fZ-fyRdSb94vVyV zPCHQ${ctd}XffM^ZSCEn;U9&#D+ST8EIwl{ibq* zwwmxpQH=+8z=FewQR!8RY_=?~Rp*+!O7g(e(@m4n0zHmg{Q1G_H5TpBEM&TbH2N`X?6j5PbGrgh|;lV5s|&Y~^BTy7x3xX;kEoZO>|M@|U3TB$i7I5&w(ckKU(Hh=6V z^lR%n?Zd4t)rd-8d1b%g{Iw2cr)ID&#a`OIT3e}7einxis!`Orm`9SjX;^c%@}9NO z)+BLN>?c?E(&)4P>k+U%!AtfpR^PhVz7KSKx=^Z^ufCaw1E<8$I_3luH0}QPjD7C{euBenr;IgPo6Sumr(bNYPWvxBiB|7ReDgna;q_VNNsEAjXFa>;Ra1Xl!}H=3N1 zc*{r#B}(*^qkc!t@^XE<7(#GW%_1A@L>@SHtUX^(j&WCnhn%?4$#ly^xrLJ&U6uvC zI*Jk!8n!Fb@m~H3754~^>E#B7&4af1X&mB_>ujB!Fwe+*rOEVr{aE4TCMW2okM1(S2@(*OC}5GUwD4lHG0g5=JQBcQM z*nk=cY1(vZyQ|$0_PlCHpkO83-Rh?C$r03pdRbvpVL6KeJ&ku7^ZBq|tY(tMT(+** z70ShY!`rPA`Mws~zQ=tGTSc^_izi>KI&Goh2?Hye7ANbR*zfg|G&egF*Ci)XnoxH; z=Dj?Sx2SsxHiLA@+f~?R#aN(lw%XG% z9q?={)j~c>O%feOD;(&%SNI*pPO?WHtz&Ybf1710jDddbdu1be07G^f3ci5<6WAs0 zwZjSNlP+4cRiQX--mDO z&G-k-1l&NKxGq0Kn2uWAra2ew_OA|SiO;UNIPsIHc6lNJsW@p`Q&n12)nv_fpyP1! z-6o>k23#y%9%sq4X zxB+$~UJ~_z`DwW&`s+|I^}&Th4%b|I>|p^zr#exOZ|$<}x1> zy5WjUiHcq34cPxy89_QczH_Z<{G09_-;Olfcj{b~h`KTPm3MRePrThX4UFsk6RUE4 zQND&%GxemsR^yUQ{oy(ute_)o94A;PZBg`Q?RLJHZDmQAkrhunziWvdW@vF6W@|82Janhp!{ixUNB5)JzHBxt;l$R7|e z9clbOfk_>M_SO)kp3rUH116R+*;H7R>H0Uu_p{ zzI@ciGRE+-aFmj9d zFAdF9yp{uu;^1~r))f4GjsIk(7smj+1^0DvW$Zp&JYFXLU9Zfz zGA!!A6zxQ4j~Iuj;MyD%U^ zUy|#Zt&If3qZ15k>hYRtR4l;}8%$;5{9gtXwYDdrgP*z&3K^ZF-bp+6zDu8DrWZ!r zc>pGITp(ah_R0bczfnDm`J&3E^kUp|I}$6Zj+>M;t9TyE7-VHU%~qMs+h56UWIsoMZU$9 z{f4n!v2FgN1@(-Qb-8Dm3Im#@=5Jpb3*@a)#M;nbpT~dfYnEv}P%%I7cSZk!>hq6k zpI%QPwlhU(ex-47=bw-d8X)zhIF#Y;A1TD>g{3-*WC&mu$$Nb9_gt4b9m^U?D?=hV zXfZM_Owm~hr82NCIj647ToY%ar-%O?0vi2D4d}~V&Jhb#m zr>}(NETuY9aWB>Ndl4NT`B&%Q1fXh#O&?9xCG<`?R5?-zKMv7Q%;r=ZUclRVD*i_L z)T3DNF;C2)(wA>wNz&HgKPS>hMB+1i+!rbGf%1G?cVALs!SWA%th$t zb;PTiklZ`=Yek9@2dDFQ#u;|&)cmwAH6%KU70*~t#K_PD1kms;Yo_n^`@>so-~NMt zP9$R-LGTi}NsuUDOmt$02GBJnEvt~OH$7+FrN$$rczFr`+KhrL$&AB}rnDv>X7|xl zJpAnNN~29%_ISy6b~v$hNNHnsP5|s)IoKCw9yn1TqqigPuKLBvU$7Tmw96BMQ*K8Q zNW+HVeP)xDtG0&r9gZOB_oeecS=DUILo5BvZ4)^w`VF~0GH+z8swq%oYynF7t^`Me zowky;gS9pqOG3IcqB3=3#o=jCrtC8*+fSqzr(oS;y9MRGx}PJcND;@uXf5J}NdgoW za4)F%&d4OubiRE`rQ0Mr@{(jzWzK{>-cD2*5|XbQ3VX9PCLnJBPf*eOt#a$$dYRi) zhRRgML;@tNSh7BdzZF_UwKazfI3C`@gffD+CmHrM-vMIs^@G*rCtX0lYtJlAsl(`a z1ryj$Vy{@_PiG#q3&3N5QX}1n>_~NaqXmKH>+cHqup@$-%JROhhVEjDj^)PR=&2XL| zN)W$TzQm94=}s4gIzgNk9_HMqLrD*;&rVjqUT%DgODI}3hZmIFCAfsa66NQG{abEVgQSnK>bsks-htPbh+ z4ZQl+@hg88DE18wy96A#Lo%_s$`g1ye2#~`pgI5hnCZW|rg7GKHVd7~hlmKnrL;wO zg7XA^=0}kdJjW3RwSmGvlK+JMVYvyLEKJo8Qsc==9!(^Pg!y%cn zPVEgY6n;LaIJeOcmqP;$KPeJ=(o|u91ly$Il z%s_wI)A`npE^neSdMI1 zC@m|wM3t$Se9VQ>J9-u;fzdVv)M=Ev1S2mOpdR7sOe}tbP-zqYA7K_^~8ExgUr8|x*VV1LRUU5Lx2*^!cj}lqw)%7a$3U7@Bys)&!_`j$IQ<`od5Gh z*9FT^u2;b^zQK`l`JsG4Zyx3-G!+{?tAScs+Sfq*BdTv%IXU1B`a_BWm7prNsG20p zf$b^0UCylKL>;$X4Kx4+d@dYsC)DA~4H1G~^W&T-;H@)0<_=)g&}5bh3I4L@*nnUWTAImnJp=DX75_sJEyimia2Fv5P`q5uJ;)Vp}zL! zIe6};o35%I#c@RqssJgG&2!ALuXD#zC}T1diS75??oEsC{l%a-eV$EEsb(2)puWVd zuuP}z>#9;3;O>>HGpd4xJ`*@d=|Czzew1ODsL5?DuQZBIKPM~<)&1J$zdUdJ^RO%G z5Q?#2*bHc9Vw-$-0vNvc0Lbk22t%<-M@ZYvih4AZ`!S`M>sc^#UHC7dC08ttO1+3zd~IRJq@3yJl9pm=)y`I-PTI9flsr370DfkOlnehBCrKib$Sx?>7jvBDR#6|IL6F{Oc zJJiOgALRn$`CTsbFx?`t(%R~S$rWZkMU@PAnM5!F!BnE4v)Ba3JA4Is3QeIrTJ_nuGMm9jFuD06q!??6b>_zPAdi;!Zjmuus_f6?9FwI{tRGZjp& zlb#aCqI?8rN>jJ!3;Uwpw2A)a%+Bs;3)#a0mrS$v zI0e#tN<437xXP}MTiKr4&VyFdbA$>#MY=ns{w|Jwh?y8s?eDl>43k2h06&P;`4ezV zKgba1L$<`1p2yxVJ-f)c#2|LtEXEA9+fa0ZSH?1yMYb`q0d#?+>qwzrnrFTtxsH_< zG2na%!zOuKuhWJC=;j$!>|%^OE!M_P8`nOdQ-Tx-mRJcD4KJAEYdn86xQ0d;M)LJE z74vMp_@!ccAt6!Oz(RD`z;2yr7Vb~wF=)3hH}pcOHEz8s5$(56baSIGaS6!Pj_2H; z(GbQZ6B72Tsp>r0IZCs4mVuw2dw`ukAdZgh{EWhJf@~HQdk#j+8Hxy-MI&{UVXJ1Q zPp4cG$GgInb8_-4Bcz!PuGuL4qp1#TJ-rhtmk@29eLtH6>1<(w9F2~D=evSD-?bB{ ztBC}@_47v+g*JK_56+w%)?!710U0iWn%5tiQ{RYBjQ3Ls!y~RyL?wV>nTNlHAm45L z^Lu!By2q|}+<^9gk&`dHxXNm4oCu1kpzl?OmjmL0c>&8 zZt9)^HfThX4o+{fnct@r%%*_?yqM?Y9;cO|uLVCwMD+-qMjgaok8bJCh@regKRKjH zA82G(vEe^x!4mG6+wF-R&~frCd1li_H|J3-X(P0_YOA zJdt)qIXECOKuT#kt31O&nbs(dh(xW(_b~Ff$MT9rZ5fn<-XFi6!@X2Oxn?&UL;gA8 zNBH+Qm|b^`%{Fqu#_SU(lJ?Ju#a)Wrd8{8mu)v&R%;Ylx*cy}YO&=^pDiuO-{$NfY z7ara)B@Q!OXn&7m99Oos1d_l)_Z!=R+9x9-OYqyyb)1JsXjnmw>V{-KT-mHYQhwVX znw4Ax2gZKFtN1kJW)zoQCkqh@(LVpokSRt~( z08M~(hciphchfd33Z#D$x9fjL`hKirpi?YL1k)7-{$Q-mZU@pI0&fZiW6gfgz*054 z<(?d$KEeZUyWEqKOU%rCrhWgh!qKa}9r!F2+ERrM??>_+tD+0A=lTSCc9To4ziY<+ zQTQz);Ph-n43T&4#@FkXtPa1yjxpsLpxf;AY7Lg+li{<&O9%A+)tYo_?}6U+1wuyl zKPQ6wR`5#wU5cj=!VX?%?czb7t*!^ekEj?iQp=_8Y8#w6A%YGy1DB{p4@4CWMolL5a8|~B)GMOb8yW{g*srYS zx-F}q6@OK0`aJ{-a{#EsA06XlKQdbpoQ2_zVz_J0$D^}YZ46${t;x8{TjHf>x4RVO z_}NmuC1Q+>mdvY6vw_54BhcO6XItTZ1vi>&f_6?V1_ zLE_U;O#Hc93nkTUSB-`Y7W?NxMBTV-`y;F6(AaJW=uzaMQUJDbnIj^pLkk!?6*so9 zpL#4E_vu)gvCqPNn>9Jj37)t(zO$DSVJxg0orzn)rW9P*Sj ze7~%<|LCTz%X7mrQTd+ls%+hVKC06@Dtp#S(36CA84K8}5bQK`LPzW{%yfrnQ6(C~ zwQF)7M3#yl$-PbY&2NjfFtST420y4;ox>OO7va3sh+iUp;|$~MF=nL{z-p|xO=U>2 zx){sdP9_Q%DBF;k#-g%I0m>ecoy#QNVLWHoG1mtIB59?Mkz5_88I!h?!6p|u4veef z2j|ktzNOVtYJmvPfkv9*W8Fu4u|8i(bb^|0LmQ0eMV(7h7>KN%;SZYD;hVjX42TLd z%$HJa=14|e0uk=?Z%7>?bl|UVN_Lc+5c`C>y#n1v=fPVg*b>i%Wf(zzPnvx4@3P1E z{K>v8gIh)K(ydiOx~nF8X6v2|z`~0Iu4yJ2mG;RcFE`4?>j^)iXNhoNv*eW}VpiTm z@&4F{KL-x*DJ#s%$9TU6i%1;-p)cr)&bs2x_hVPsiL@iZN|c_$!#wt5b5Wh>oL@Go z47v;=V&MgbjTw{4Bt|>huZ0m2<&{O=BN{NJA*CNb9N|qrs_o&`KQ+(S>v}lu#|>v| zb+3d($dOhZydc`;w`9i$=KHnnU#x~@wXs`(mvnXjn_gPqoOCG7@n~k_nWjD1dhwl_ zT1z}z7}hyc`)e#!pM0`Q+>}8!XFj{?_X)iRLcZ}==Zfb*$JvEEy#}Id}WKhf6rdKupf@O}R=q(J|%SLMv*eAV^P966V+Ztz}U1SDA6loe!JTzNDVj z&%BRE3d;3Lzea?sz!13z^aF*P1rr}HrFO0kZMxy?n00D;i05mpYraF|2Ac$oHceIi zRByEJP9uYNC9$pMxDVVU_VmyZh=6tkAO7i2CYj^vHp%>T!o6}5WIr1ofRHS|!Wio# z&hOJMPH}AYlZg?*0-J-eF|~e((+BK`jb__W+D1Pb^xeQ{O3cyaWc&f?uCAB8dJq29 zUmJhvoYshrBv=VUYd+9uOc7?}hUeyh_%Nc7m^94oj>f>P)$Rw_=w$lB#+lTZF z$L>rplNJ~`RMkXhQjoCc51*iZSDp=inc{+PBC)h9+4h%+%g0#bBYVkpVr zv<_5g-3H=INe%XG&$9(5MSE5*1O3saA@N_uYNH8UPm&IC2vYs@7NpYQ^Tj3MGq0;0 z6TAA+>DO{0KMSkwzNw-y9KJZyambVD1nEGt2d3PXqCf578%|j zW?wE5X0`9%danNeg1j1LV2^LP0|uam%-|#Q4%0tnZkzz1t&{(~7WTyK?Oa(y!TXtJ z_h0DW+qi6?@@?}&jjFQB(1$17f1D5fQ;zRjDdX<$oF)Y;p-0RFuBBqSgQ@dgB=89TU$fn{Kuv8af+5_Lt_XU%EwrpFBX~2!%eX)Za`!P`Fi5-Ds z?y|!(>;Sr01CxI9D8EZ=11U&4%rL$;{r$nOM{gLpYi$){k&%_`vSX>(nq$!(bXO0J zwHX5zGo!+oqPV`^{p$-wj>03Vf5(peDOhWJS>lD##1*o3wMyAfUfoxJ{;PQV60b*#x(UW>@`pq;wiF{9$_>^}8;3EA}R;Id5U%yvD1_x_p-Z2FnT` zsLVRu^tbZ^UyVVQt=w}`QNC*9=|QibLQ^-QW&=6RhfOV&g%=Tl_{ze6PP|gD!_}n= zF3h{QEV$f0o4Pb3v9pq{Bkmz>t7885dmWkTHlM3MEPWPZnhyfrrg{H3Za#M$v^%mN zhn@8_jMy=WN$4o)9}Q@FVs*bgZ{dQ5(1Kf!OJp(qnwk(-=+&FO!JSW4gK#9ybZG&U z6+02VHG2<0rD^8_^J1odPGTo`r?dm?F(<*5p;Di)eZms*ICZj=Q8p? zKAhA~TsUz`LIhi?f7E%OxOG4G@M`7JFg@~&ga^TP02dnbCi&B2_z{jqQ2<5l6 zGHm?enUAlg}nU)6)JsY4xPvkB#G)e@?iW`o@;V zwFj=_IokL94(>Fien3TsG(0gYkka-JWZm(B`#5{H{D{~7ckRdF%a`kYJkMF;A1(yG zuPcS&nqumt5D}AAQEu|>yIafNwG)=8;C7`rG?WFI^EJhMCU7DEVQ) z!{^eJYb&02QVq5#XEIy1e26n+k>9r8w(rezSw&1j3j*=^(B70k1nVObuX>wGzB5l9 zz7V)0aq(a0TbJ(1;JSkDC}ebf?Xn%$yj`4c_6a;fE}SOA_hJvWZwBJ$oWF}by#Da< z+6(IONFbloTZNN8#T`|%I*`}g@?hh?Pc{A(D3)h+J71+G0Pj8TuDlpfad0vEeW%-Q zRObZBc;Aw=p^(2IbVJ8Fk+!xeV=|;#Gv54T&G?QGZ6{X4=T2|-(+yQH^hl&RBO;nK zz8q86-g)>t^ZUk^FMg#C`DW9IVUHOEU}_Y;iJwO}{W+mq$DiADy+OD31WM9CNcWl+ zjPI-1r9ZE(*dF+GkcY8bwvoX#U-sfUha>*`Lg3VY;}pHX*gSxZ;$q8PHlr~I{jY+( zBMGQ`ON&+n6s!Gtn`Gq3UMeK0PJMTzc+eheBr<1cLtm1NqkrACLFS@sPXyQ^6CZ*INPin`wAT z$c&gu(D#m)Ctu|22GKvZl|v#w2esu&hU1_(cnd3H)3Ka;itS-ERjz=ur_xt>EVnnIy48^L?!U^RfePh{GW?pbiMlO zs54%zjh&|kXn_0i=L?cwTSi32{EwEHJ`!(KIS?Vz9*nYpVczd!j^x@2? zdcWmv?foLe7v0H5Yw?Z$SQhpF%QydD|NKub!v6|ga*~n65dqW2`p4pte_%wb;>JoY zG1e1I6wv#utfQj1bEu_@Oy9&y%)|s73MqE1jt>5?bdmg?!I?d9WIc8EJLsEU?a=mz z3*YGL2px#z4%>228Ww-){Zig#lhN-PDpCW#XJ4c=4zNw{=IHoazcU zeLZ#QZ&b#GuInU2*g@jI3{rLJilU&~Ot!SV(p00(JjAI4@m8}$_cNY%R10j_5zCZ1 z=8<3d5^&5TMU4S3BLD2FKZ-x*VLQ@ipK@+i#f^aZ#!`<}mf0=T{PeghEfj!`7vQT$ zOjN^&Rtz$BZO7|0JGjcw0sZ3l&}ACmIe$+2g;bXHGK*U8K2)9Wn;s5Fua4ss6_fg? zUHIHJ;kuO2&^l>5d_k?eUScs1XfC+U+HxjBInai0YweMb%Z0GoWldZf>(HUcQ`N_5 zyShlJuODaVQSSH$Ggs6b-k4yP5Z^iISR>GfUD&0qCpKv=#?Y60thuPZ@g|x#53d0E z3Y`uh*?9g7n#*!37&jYdfFAB|k7cFMV|^pvw$L?szJ+XwU+; zU7KEK>7O#7dF0WU{wk3j`lCNLCvY*7{VRUHt?jnu%C{=$i3u2as(aKPU;T;u$)VN=@_G&LQhhR0|Q_)^+OOSjNr#zMFM0)j!m=LcC5oO5D z>s@jYfLqVpD+Y}^;2|}IooC_cv`uIc*0P%W|J(}z=!6TtbgmuzBW6D9h{QHNya|wT zfCydI+$_tVjbh?^;^cy#0*-xF4V^3-P;!=yqr16K;?!q2R^jXO>~pMH>P}tpM}#OT zKV@RwNgv^x{EHni+c0IIN;kpfu3xCLax8ItIh_EKE?1hUdrGsgouhc@95=IknuLA41ZGbH7;&r&M)}bExEt9dzT@$TF=okjqyh@L zm>eG{lHv6MsOgnZ$CB(AnmVAdFg1u&87~Xw-g<(@3UhTmMUV0bH^0R?Ahj&K)vfrY zqw1Z*?s?Irg6I3Xr-Lv^&6*z*wikGb^@Wugg`}f;5}cuW1m_rkh;%l=<<$_#i3=2@ z8c%SrE!Vg*b}0v*pzwuHkEfNVbpb}VtIl<%UH*1x^bXT>zj9Sla5dhGhCB-fHlrzN zCV{xQb_D3Ve{k%9glVD6QO2>}@i_84a-g)aCEkP^7%ABRGrqGN#WdP-&xjONJd2}F zmkH=|Hv}Y@>(&+@W`)p8vv4Ng_{|D*JAY7*HH3aUd4QDvY2@1=I4~Z7gQ&q(_hBWd zJ&ws4k~CbX^gAv-Csb71HseB!KN!Y>cA{a$PA|DZ-mcA^cWmZqp_a6IG_R*}Z8I8U ze#xPSR{diN-5%GUAcs;K6G+Zeu*;N8#h=!^IoV{h5G1#-b>{<3)}w zLvhEDp0&*nrrN}Joc3M8N(3MqZzD#IvL0(<^g{M!FSGAQ)CC->NPs+!8fi{%)pU0t z^xl@|Tj5U5;p-B9-t&1;f9JR>yhT!qEE4g6Rh#6pIbUgJCDwq(`a++=w(Ka=lw0TV z+Z(Yy8Nq>m4zj$0paHeP@U!6_D6iJ+qy_@0?fh+GY%TpGpdHT#f8SVwtZq2X^!RObWXX&%fT+UDd3cXz^X>;e4kCXjF+LbwkTIniH!V%17 z@z8O2J4ZfKlBX;3nYXCCO4ul4x|*hHm&!t0di<1@bv?%Myr|J1WqztXZesH8pX1i_ zUq8o7OGHf2*kGK)3{?ZS)G) z1(1S^d4@v+WDau!(rWR_fdJ67{k-IFfttFI!PDsjV7HiAk0D5IvZ1ABs%XztL{qc0Bz zr1pH?DKD`*0$pJ|Ks;Ri%1F28QH#Ke&hH63Bn%MBp5`3of}k3;@{atOiB~M`dOS4p z0wLPjxWt1_|45WWjndcw&$zlCKiGWYfnFfO4GViWsiv-X<6(gD4Wqz-d(EU&sS$)5 z*d|kz_kx~O{0rvx_8TiJ_)E%zG@gs@kW3+hox{q;WX{)($?!<&i;JO83oS?buR|YZ zUFS5Al{=LHPgH1)Ev!bu%%S=!#81uW>%^pTwZTRS?{we)?TXIzWBem*(*pi7u{NAu(7K+{tRNos~3hi`+Nq4qvg zThsW|9TE8b)qM)R7CRVHLM(I61*q?^Ctg?c&`!S%7N z@)RPEGByIv6 zxMvKLEXF$h>hfYV2iPUcHogg%AX6}!IMihWy*qC{jWj3GcF#Mb_@%We8A-DRJj9YWmS{TVhev-K03JagHKT9RvKjEt`(1^#&%NQAyScmTzu2ie?py zIJ!LfRqM;`{n_2fMR?N|{A11ZYVqBrrxYLYcxi1B`Ls;KiICNM>Fx?D5YEP&NphJUy!w6lpk~NAF6Xi zrrrj{$+5xxVn%>Yzc0S`FMn*42h-_}_Z}T!LZFK8ipX;h@NLe*RH{)E z6~)!cLn3r8{d0bhVgnFV>cZJW9`@b)KlwoQBh)g!Ku9FYMtHl*$N^{D-|36G+3irZ zWv*RaZUA%|z*!y?_wUlvn5Y$S3a~*@JWaT7nWu>;hz_R0)>GrIVZ51Zr3^}siOTq{ zu1s(@Sm2j{6#EeQ38c8PSa)sjwGIDpcr0#MOn0wpg4;0*8oZZPx z>t>+SR{Ha+Mr!5&5@R|@o!)urSP^|~*|x2#-A!s>jCS?{NgZmaPpMk!Q=)Q1-WS2{ z(5n7ouv7JaBQz4G$dqFjIs2YlSwptLHA!=y{Evka|JC-5% z2ZYTZB}At5t2pHzvh8~vAr%|up_LWHvy}_^?HtXce3LVr3sW4f8je2W9%FT%0)Ap- zv%RtMO4yC5P$rwgh%%k7-5XKu;SQ|c+@IM`^95A2?8*kL=uF3!z;lI4);6@7_ZkCHOVtlRtvAZCEwO*nIvsVZ3i4DNK3ijOd5 zOl;Q0Ax|tJyW_7i|IjBnIq%03kUSV}hRufqUrJVD`z6Ni(M~UpS%y_ z(qizN+ObK!M(1*|@1$Dkt%v&aZ(qb%;#+dN+ann-U!q)2Wp?`*04M!ehYvCZm)-*d zZiu*HLGz-cVl$QS)vtsLx9pOCmp1m=LxC`~_Dji)*bWUazav*FK9;_twYNFB{?tB6 zU(e7y)GpI|_QP_*eyI?~gZXfs;vU>9Aj{DhKjvc--HhHgl{>IQUvC~O%|?&~us)Tq z+hL{ido~x?nX4o#Db+yt(75`sST(Wc(8X1wdxWPSwS+SmZ510+1NQITmm(OX`g@qo zN&Ijk5e_u10%@<7{I8w01SFbp2~^r}nO(`bx5xdWp}N}Pu9=Ktri8zqroqTtD9mrj z)&#-i;s^o9vno?QD?zt#H&OOd<2Xo}5e~3b=zi|9Y%(*ur(H4qm=`UoVGo2;&UGnm z#7Qz)xhSSRrAy9bYp?z(jnDm3srS(@oiUKO{q1Qx!5Wb3rkNGs{8qEp=cwd{yo^ks zc;0xiTG=3?#Y`}Q(M91ZA`AJtS1wN@p0m^~e|oH^4YOP;?gu zCrKSDw^x8rbq z(}F$D<}{F5NqfC9sv&wQEj*<6(990DVO}ixF4xF`Do_p5p&o&I+@(K()GG!FRycOB z`J}GQb!#ujY0%A@8;Q@o`4h@FqrbOARRtK0zMf$*3r8H` z97_Y;iYVmXO-!K{UwW(V=rFry$~D(ieB*6h-zw+RpT5Z^zeejUAVrmg(VC1pXaO`V za9%!2gcKFnR4&!>Mx*FP1)a;l-g?U@qsn|}KTzQy!n);tc4t-w*_*g{s()F=gb(wO zUB={P#WIl;O>vF@yM}2&UBfXPDFti!OjDZJYZJgJ@+VU-v(P`{{$zyluI$!Bqod}c z;|mj!sz=urdrxfSEnOz3$2M12OldhiaIo)%)@u9X)b!Rxle^dEIMpN-Jp}@A^SYe6 zUNfXiH*kNG`XqV!I#Z`y;oGe_|D5#{NYPQE>dAzGae{l1Jj||$;OY5AR0UXpS{U@^ zg^|r%IjfjS+_73grW{{oCozKPD;8;8lAH+UZ1;{m^~-+RuQ4hA_7Fy8F}2uzXx()t z!pbUENIg16PT0kwcEiZ5zYdQ=WOR-cza>>)^;~UAwc(F{i{~Ed2u-cVji@+mtu)@H zs1l*Ymim)f866$HCm@MTH!a9 ztL7lD0;#irZVDF`-OmiCivt7!Y$B2rwf*xDv;k0L*VFWDPGG#4S)5DmmyTWoJPa$~ zWY-NAL~?Z4ub5J%jOE9=ZUvd0OfDBIaALtTX5_&0*M9CIPYxj&Bx`%SDZ8F!kZvx< z+g0tK6Jg~E{j+aYqwEFkPh0T1;~Sq?tLb{AWEeUU3=BO*swVIzknE>KE~~Q*=b<&Q z`p}yDxhlHMmjvp?YKQjh-k}hHlzG0uL7Y@$C;f}AA>1!V8rTu6iJK;il6~bKgZaR| zH-N=>d!^khrowx`Ve=S3s@RBZA@F@}T=q=UjU@pFHYrYkb4zmkQ-su64d0jyr1hiq zsldzUE9Ae6QXkb+@A=f-O7wNq2r9~j5I286w0-GVx7|4{(SM=o(mB*ZIL>%6H&FrX z=!CAsISS|3MjDJ%neM;b9y@7%&iz-N%6y>9;LUksGpKEjR{+gs=M_`qkd?~>Z%-eZ zpfy)U_JAgVjS&#&vq=WFHg{>dV{{l~j>7W!h*pj@bu1pt+W~%!w(l!ZU?)fa0H+|Rp#50r>*M<@oNC|P+weHUd@3KQWVQE*Jze4k!<$Zi&ojx4bg2t zqenX}tEPH~-pUFBE@A?lr?bi)cR_Ej;H4j}TH}7zUmNg}%bDk#3Nn-dW1WY#O0LqR zol|sZZ<`*xNdCY;=W&s*H2rxA8+6}y@kb@6c_R$A-Bye9{+QivQ&(t7iY>GncklV6 z^*<{7L|Sm;;uttrcj@*u>#vIgYoD!G%fY#SI3us2u5V}F819^Rykb~I0Q*95P|GP+ zNX~ENfs%^fmsba&M&VF+lUMBhiHX%)pr7%E<+6{=L6eyJEFG(Yunp0e!C#KC0wcsH z%Zz)0$N@=BU56uAfxLgmJFK$jw zpu>!LiiMdymczJqva<*%#}aAA+hQgUol5K*(XnIbKi&B?CiaDMV@iQ3TGF<%^&eda z!fHeq@o9nT<7V>xpA+q9XDZW)(Y*#~aL4kY< zmwrrEZ&LkxM5=q-ryY_^neSOmN3VBGOfa|Drw>&zTx={eycc{bPJnGpx9B@uPFWU< z6i5`?&nrE@r+HzXEk$jipnsK~LIV_gNp|8S6-Xj+>x~JUJf-rB2j!w-9;xO381!Cm zZ^S2k)RCO>Ju1v|#3$4l1zL$F_r}^8CMp>Ct1E?hXhkYn58<4Kg%Ry#5J479r6i|* ztWbl}Q{k01qC8t?x?pa_Yn~x}t9vr$k9n^{Mc57Z3thC|Z7UEMuyiB>C7@hZ&Wqo)MP7kQyZr26_{zbV&- zI=xiNEGxHM_Kt_vT#*-4q)ontX~o)QlE|4RA{;4zzJN?{EENI;FsypyWKAxkDfH65 zS0>^ZNwygkDQjbryW)0c7g5icO~F{xZDiQ{y++8C2N-Rto4DEJmk1)ie>;m%h4|1s z)8Zykq=PH)HQ<3=GJ14fe^XQiy4kEK4o8>PP_@|6aT9wba=ajSh43PPN(@bfk|2!u zMO;av#1O=7Imb|F3 zi=cjQbLCK*fj$+-UZyVP0yHstOHXNh!QL2UY~GZDCFLgs{d2-cAX5`4GYXC~=yIY6 zv|`UPAIGYeoz(3vkkxan3}k!sZd^m2Lzjs=Fb0x2H;x*KPPJNoj^}MAQ&TA&KC+;_ zhL<3^(qK^e;A3LSC{s@hZQ^R9i#NXqvg31XeHN_Ub`Vs(cSsp+>W|5^i#)ojs z8DJnemVyM_Z1-(akj>&00~1k>>og~F1)!iIss1;8>ENX3wMLrU2{stPAZLKoY5W1* z2eE7A0dE50BW^wnJTx-^)s(<{bum9`o-4`%6PH=d_b;I4&zbj?39YqA%yMU79^RdFPua?+X9Vr z7ZBX}HdKuC279=d>bhPrJhDGj%DpP3w{NP2@xAj_|BQPntL^>^0Oyxwq6dkq6R&vHoPxq@P^LQy)z%Ya~loKGemax zB6)1fU4-?|R!@r>Y1gPkI(W$`KK)>E!6sW!KV?{X`b`C0+C)ySXyu15oaFATS*!B$ z-_vCYsbK`jIh^H0{Oi?GZ1*i}@O#a<{O&GU6LB_HzaN=aqMWmphduXz^!N>hx|?7k zGxJN`7});3d|`{%x9WNlRe-yaT(Kc`ZiX06LrMl5%Q9wivec)ySw){poBBN%4H;CG zIRmt_5SSY@mpNT5|NKa3t=qyz zob(i_4!$%}8}VMDvn|D6CHxQ)@+LceD-23<>DN{IG3ve$K^-&7Xzee+SvPMwWUNbu z7Ev8?q*Wk|Io>=PAvo*s_GzL>RexAW2Cz`TXwq7#`}n(E=;Pnt$qd`E3m9!i){YB4 z?$h$=Uj|+J#~)=i*@`PJ0;f6W*-wtIF^K)kqHz;>2mTmAK#6?v z>vUifQe=+ZXm+PV(=Vx)_S0c}Ed>tqbHpcRis|HO3KmWMyZqMs1W7qpd{Q^vJK1S0 z`bD?>2QRp$xV|{(suuetMAW}W>jf)P*`c??EKi(2djM2&Io=;mkd~@T?-|Oe|E`5a zpg#&zJXWg1JbjZXkY65lc1f?1ar0zh6JY?5VFWPJje%z-C+`Jd1=0H5%e*F%#{k4a zBG^P_jINm+2cFf%oWZn4-uC!F6UZr!e*ls$BBPn8M~ z4mKgx2~>G}1J}|+*}DYlB5>4q!!yh-*!NioIxgcAanKTD;WPp~|9gEa5+yNOTOzAt zQ4reisd@|jK!XdgE~s30Dsq?9%_)Y6D%)n`LAO`c)OpV1VLg?Kf|5bCD5cr_ab%-h zFE}0GBw(&ovkU3xUNKJHEeSMVQvcWF3Rho-Fa6AO{OcyinF!z=DYaunJ>8b;jg~pA z%NZ$Kbaism@cq~9*GxBGAjuD%mWNm1;d@uCDg5v!4o4XejEk!Wp>a#sV3%Z_dUBd_ z)}eX#iY@rSQQb$V^9e}91(WciL*Le57NZ;Xb9qd?F9&-TWJcTJ*q6<#tRPtOq59z}?0Gc=5`bnc6}QoyBBjt+YDZQ;*z)xtEm+j@sEBSqk#U^O z7q;wgbpK(QTb>NWz%~E$Fvk1}u*s&#$o6=aF1qWBJh%=^?1KUdm$L=$idQlYDira%dQA9nO?(;e5(#DR@EhO-2FD&% z9E)>q16WlZmN3&X33NR_56q24xsa2|!UgI`t4uI2+Gm#)$)>L4b(xb}>sx0gz+!01 zpA%v-muJ?FZ(=kv(jVS)`+wMb^Kdrzz3tl?TCK6DDW=sy5p&IvvRVsER8)wV6-!N3 z6cvQ5))2GRnp#DwL_`cVmm-xI7K$Q`Zop1ItlGaQef$&0?-$l6O?b}Z!2PI%tK!87Msx0+>d;{uBiV=OhUxtjRt;c| zle^>BKA}4b+*X|E&1TQ$T>Md&`W^XsA!P{4MgPfsl#5ZGz%OjO+?))pJ%5Ar1ezYy z95xfkhnw7tHSrD9BW!ExI8Qn-lpIyH%3p|Aj#HaYe}Z+WYF5Fea!EHT>c!WbrxGel zN5P-RYgCjmTYq;{iS2M*4}+W)W*VuUEv6aR+qOwq{-%&*(iVQK0uL$y>z| zuZf^K4u+w>*6BFGK|>9gc!}|FpnKs@Z)Kz-8F6;a%(&F$Q4e!I5S<~rFSbw1snQ-HKZUJua*1rA`~|Ew3)PNfZr`7w6mp zg>Kz?+}wRxbR{$DmfpH|qd>f>pyeh912+)&6vj;)oK-~?!yMw~EDs!3q++WU{>rz! z^nK=SlOImtdfTQ}2_Yndey!aZ^E9WhxM=n1KzY}ac(B@uT6DH^N9|9&X z70++N%OTnx5hzQg^6H%XOHYF>86o|<96JdQ(d4hY*XY|Falq1e$(y6l?}A*&-rpcL zH6Kp|6^kV@<_F!q;<)20z6{Pgy`WpYlMsevT2C}0Gn*88J2t*ClY3IrRh1ruKlbw z2f}3$FCNvY8Esw*&0h5b@C3J_fh*^&w?Gin;phqQ6&_vWF}+E7x4V3h0+m}D%~EJ` z|HjjAnsN;Mw9)T1@P4dmpnIdKF5Og?tIo<{7$k5t30rmn>+V_9K)Tt^k6fCQ)*s;?weB6b>HE&8g@v5&b0fAFC!u`HQp z9t#_k-#B#QUSgZ>oM71E@Dd%yhy0jZQ>Fon8J>;z(i!u2fkTIF7Un9CCd&)hiq4tT zOHzerV|(P~I0V*2A6~RWa!EbJgWhZnWRf}d%Sx1l!ywD?r3u1RZS}_T-Svt7)foXj zmx##_-)mhyDF-ZZqU{zsGVN=gG&8<2Wr*4Ktg|o~T#UJ*n3`ueqMQGNe3jS7qI;Lb z-}^fN>XvBwjW=4KtAL6a2d}jF{!Kc2HoPkGDGf<*ta@Gr+vA4!;aR_a0XvAYhi1tEL0@>mmy1~x9aOO`X#1CYwuoM?sq5QX>r zKpG#NygIS-M?Y_pm|aKy5LI8~Qd^R(|0ho`>1je%&F`6}A=}FtP6Bm;X1u?5(|d@! zh;q_&Kdhiz?H0JYz(A2#dgua3@2(NDNGtF129u1;(+qIs1#$;(wpI_pG`_$>AeZ=xK>_8vhPu)Hphg0JvquS z*qLT1%zzbUaNS+fg~I9bl1B+#o$~d?rVa)1&nZpTl+60YR-E4G{TP6CmUoa5F>hwe zvT2-KSCHJZJ~^&_{Eq0+N8c#9!iS zvXjuY?<=iJcnk_hT~%6{CxCN z^*e(IR>b7tfDr^aiJbo0`z}UK8~`;+6(Ygb(wVf`a-V`e0ysJtf(}G(HM@QXuU3RP zctKjN!03zy2hyQzBm_NZR1ts(CFA;TlCPQx-tV#B7Y8r9L-y)9_-c5rI7I4=h4grl zY9(Ubgg3r-L3ON4o$G&8(OmaVi3u$7#!P*$MC}K0f)DlM#}=EC3)qLtn*GJ9y(TJS zT!5`_DzQ~!3M4C0j1b!54`u|k&o8Wy-?=9=v+wBcnD0WZYtZ%EVs?$EzzxY_dQK_h zy|((jEASd{gcqW2^5ZP0y-qr<&V3A>;p;LpGnMq?Jmf4ZEF%yy`Tm*mLYJ$x&{_h` zw<-K|Um01?imClLMk>Y-Y9gN-2r;~_CvYY3ZVix_=zle_uj^9KeEcW2k(e4`J7)$o zI5tT>!1}CXZ!X?1+rOu9dT%k=a~6AAci0}NzZ>WXdd9(BV~-G7TT8a7Trbw)FO7hw zvqsZ}HS&;|Y3l-H^IJXT)n7H}-v|KRzW$Z5TFKCHqE}UQ#4Bl(ND9X^p#2h@v#07u zpMV{zd>p|^ycRjR1XzKWYNzr@m&3*lhm{9qkE`K|cg>N+G$$!W=$RW&i(SwG zUSy^_>_ws%xSnVp#c`>y_>nLr{_Q!QgmwJxvp%>bfnVr1` zjI7(HFED}?Tm{Y~BX;<9Kgna(y7G(5TDmU3!uAo^c-{tRc%8k_7~~rCDn0InXr}CV z)}^wQGJ(bvWtd~g!OS*ZEQxk)$a4Wjk_vb3lUyx+{#2(=2?Yl5skMQV^+m>xMY?Wp zU$ZsrAA1v`{Of!ZsncY?iz@>ICyWNQ3f6XafLqm%AC4-H&pQLxud^on$780fT+Yyb z7VF^m4m}?hu?NfBCBLub zax9rq)Tg5yC5y~M*zwyg4UQ#alm~6Lzx6nN7_geQvPmrRYye(NFZgbVtWv*e5}s8B zb&ri=3hJ1#qHtWBh!Yv19FLC<^q(JWS`ebAoIv=QN#9HYqC}?zDaQ-XmsT4TO3QQE zBUQqRA>sjb{_7Tbj;xQVW4L0F0s_5vRF_P|753b`Qkdw+_qf`+#9`7IP)Csg?U739 zWk#`zDS5CkbRk@+7X5TT{is3LHUTi-Q#Id_|2E>{v#||F1+RsHx~-l%5zY{h`@RoQ zYeG_wB>?A}r+C*zOClQZPjycfW)-t4=G?|3U3V^FLHu|bPC6ovf9I>|J3X1l?b^-x zJ}9wlD%A;k$v8#FO;3U}7Wv{`yfig3>>>3NqI_+yD(~aA`l+@X`!gYTR=|7_W&Md| zds1GZPEQ?clK=d;jdUV0<&2iKMQdRBf`A(s2?k1_l$W=>qdObRQZtoKc`F!wbR-|j z0IBv@hmxFwrRLG?R5oIWdFu&#bcq58Ho~Z*N&t6wlD-!*M`oHP#$WL7Qsq2kP|l*W z845kN@-rOm4kW`?&>*4XicgjEy2&Y4d}bt|eFdBr#_4umb2imYA3+RqiVK&7;MrnrOXQJH`prVI0*J}Nl$XS{Ru3cFIwK2 zhEBclx?fODAWYP*?vzgzon`?}OFZTrC7YZAly-cR!j3k3Q1jh`iG2-@72C2K`vbs= z3%M_{i4{yHF@Xh$q5PGneZc1w=mf&ZOm|qbCa482Eyc__0glY|YHt?0O353WVzS0F_`e0k%$ zzs>aDy$Slp^HG!q_|?O)n|e$T&&E2>GvR+9tNQn!@_z;t|Ce3%zls0-{n!3^f%d-# zGy8El=e6vk*c-ftP-I!lbtS~B?2G*A>SE5F%{c+xTJtU&6LW8#WrU!gSHrp=B^S43L44Zw;t zcPF0`VG4oebS1k!k;y^eLzV|}sl0}`m*IM{hA&S(cf%ZBz3>O|v_q2eV0qJxbh2Zy z?uYU4qMzzi6xC;M{?kKb9KpkO%zDi;`aI~*ZC4{tqjkvz>8yOkpsR$ z>3y2Xzi-gYL9$zWO;6P7EzY&>-(zLHie&@~(`Mha@8DDRwtDKb9kM-J+LG6-$LemQ z?D5wkf>nDit2xGOOBP~mZgb3SR^R=C{Ix352S1Uk2-nGOB2wS2>gNR+dxSbcaxX}t z`+*)X8Sl`U09vVSfyS^Apl04}1x;hdh-#nIN*!~NbX4;`E4N}-!^NaG{;y64v|o>K zx@pUFR)_B)T9djvo865)r5nOg8chPqfBe5%P*~yJCaMX`tv$-7e&2WmF)?ghR=nY2 z;N95;jxlhzNWIAxq$GZhlVLb?ooOEvopxvcG_6@OHS}w`a1Fe~HDh!IMsPu6<_u`| zgT%xxu$D4Uuu;M7?G3t1-geJeEeT`9%f1J-{>TeF4^vu8~_gr0C zAh{~jy$-!=di4=6YO}<9?J@;qL+SK&LwLpIj(U--ONkB_gsg#HXWJdpVF1RNJ(b7G z0mel9cG*%V@BW#iOiJLDAIwZoLB-iR?l{00a}H=kC^_FXjz7WyD0+7bfCm5vlxPz$h6%lvT&7J`Amd7M4t7M z9xO(=8$!WWu%^{G2}-l=&IO=j@3abVy63Ey_VXA=s{8lbIX^6A&z}6$-BJkI>H@K2 z(SSv}7k{?nG-K|H-Ri5fA*NE&XHW6<55%Yrj3}!5XxF*$2}WkA2yhFW!(2al0btEusp~N5v#8&>G6D}mB!WP~ z)I|x5A|Dbg^r zsJ&p)W43kXL0F%mPXMA1?&(PLYH)*t^B@iQi-S@??7|7cDklLF7mCvSiLaUH2IWy18+UPt!_1rsl~mCqBNoUh0`75)c`e2v*ln?u~uBQUWjL}k7iI&-!y;K&3p4Di4Ui;L9 z_-qq`o@^R&hfPJrZ?|NA^)dVhtZ&6*3+Wm zJnL1?Y(t_)PQ?bJ6G#yxV-QC#u}h`#>Kyi=-Td)zJU^$EIWM;Fdqm`3VX1cGf%N|2 z6`+l*fU22RBG2l?(Skjhle@i?te9^+nN9c<4`FsmBY`vRN{1>bJovn$YQ@UJm!ku% zkBs{owpz)&CeC^;M)9g=Hb?S}_1?UATo_K=)|zpnoG4~C7^F~{Dph%4Z<3lIhDv~K z*2;&6bssrC%GexI^MSQ^B|R?&Jyy$4zQBuuee7G4Dk2}cB*~gfDqEDCY})d=TUj|O z?3i1Op*lV3)1u+6{P9h~J6Hd9YM+Pj#M6Q8olcYl;T>V3e(Ul#p6}z|#w@*{M+ReQ$IuQ|aPw*@VcuW<5-GY|Ht$H!f5HBt*nSN1S;W&%Mc-#q1yG%dI~-Jo(f8K$4mp*bAuc7J<9fwpi-a9bvqA^vF*#LidU?b{O^u5 z&=LwVu7Hz8A6P9k3kw7;ikxFj;Uu&6Y)=HZRN$6&%=%7O%xR`-VpPk5Vp=+@H$3Fp zCVStv>*K4oTw2Nkh*z2<+mluXt&Mq{Uwz|wSoX^|o)-ow*w17d{kymS*}(9>M)7~< zGX7U+iT@bm|J5fR#5n{W+e=3kTpkk;<~Pbrb6|W)xkD0z4Bm|Wh&HYS(rEQ zk1Y>zwqFv)Zz;}?wxG_UqW-P*B}zNcrVh$b{H}A1 z&1;Cn4Tp$8#_F>2)(}e2qYKm|w~P|@s8Io|xR+lJ_lc8c#fT|{?I3#+T>Km)e!SRe zK_}x;9)wPEBpQAkDRv1hx=1*SIbiD?pC%v5b3G@^rjCEYTw!}OlH56xRIETpupXrk zo9S{9D-GpcLUZx_0O4t`C4tRr)66;Wt)Nc49_M=C`cMT^tp!sqnUnLyD{sWs>AsH5 zmVCBzOp;IFjfep2Q|w$I>mQ~~H9T@GjseV2+9X)U#Yr&C?RXHc301K;TZzaZdjc20 zgfe%gYR7M(c~$TlH{-r`iYw3#1a>GYMKq1)xnx0W!|$^2QrK?IE^&Dq;;G0ShUt$=y~7OU6yWUj95KRJ&GQxlospZD(IuvrpQ#j*U!**AuL3$aIo)pu>j?fPG zm@O*I4A3(IH>S3n>$CrJ8JDO$Ht_Mf#rNq%sj-^G9@KCc`bo98OQ4?eSc#AmEF3_F zYec9JVV?u8ADOc72|$L(_7pk_7?2V$Mr9noZh2AG@e_ur0LR|^MeB>V5vGFvEoU}d z9Frth+-tBouYAqWe>{nQJX2iUIv=r$Q=-!6!ozJmi-(Gb7J>&_FmdJBZhcelMWOhS zjYCzm=Ta{v$>j7L#`@{9Mjm6Ak2A#NP2SrT;v%tUTQh1pu#9Yl^#yPv`x{R(*QR`{ zSMhDLoMg`w!oM)Cvld54cNd)!ymARMk>xB63;yt#G&WQfZh#Vo7ps{oFkjKh0Jp=b z$|-d!&x+}lS_k@ANY*v9IJ@xMGAyq#dZfaK7cNRnDYnGr6f_1<(=%Q?4nXXS;OdJ( zcgxF^TijSmT(P?StDG>u>RFB}Bl0UCJf#8MVet@jqw{q91pq}ClG@?P#x9MN3Xy=C zE6H@IJ(B|3GC9XqNcl|OijA(TjSRNfl#2^qYMox9@02EAoE6`kNR}uzQnv5*u7CMn z-LMaQR$DsjBruv58xjWdFp7%O6Ai>2UzF&ho++}U^P!Z{@}}*qLy6!0?v;cRX<@)(wqk3f)lT2hespVWl(=$7Z=HnksG_q+b)wWlP zqEH5+^SU`*hSoQ$ysEs)PTBHikg};p_C)*5m+dpD-sH(=Rn++5={iI+jGa_wH{GLB zL7rS)=pdvZ^PH`!;SyQGVgo9OuOpJeFx}A4VSG8b)=a(#5*SX-E!|c&e0ivR_rL@S zOmFeWL)bIN>RX;Ndstr1(9fJC<{X88k@NL{(GR>16!>%%Qd<}JBpz~H4w$%vcxlcd zbwQYu9$W4*GaFFvfu@*E*4FB#2gQND2GwYs3ihtnT}voZ!eOdPRD4L|xn7P*Q13@i zdnQVe@LcdnSmG8;t2uzGQ&k0DjVzTlUqL;g=gVv&YNSVsqY|6OX%IRP{k20Xfx^~2 z27v&?%TJQtM+v~%a+Kqi?mg{@qG?LmLhalWEg8|kd-nOdpUIfnZl~hceeb9(1bilWZ=!t9O_67 z_|Dl)cf~g`M-X2 z|4CEQze-X6$0y1xJVboHW54zE>eX<8mQUyY6h8as*q@0RB5l$*UMM?_XN@t@VHF_i z>blh)@%GwtLxme-N-sM;{t=wvW*c1NY_6O3OcmBLCav9>%%p#(`L4tM8_(5DV8wWP z>l@E;fHKdaK^k^lyA;c~g8jxbtjMEqi*0*!Qdg6!R0lLXl|TDodqL$J&$B+DJ%OL(YEVdFyixkeq-C z?;B4cjK^e{iT^6#4{Wdi6vij3pLzd3|3^`*AOCBdvl&D9>gl2W3TUV8EUj;BYR&(3 zrO50DxB0;dw*i4m0k>Xa^QD$E?acCd-v7G_#=i(r{ry1|c=RhH%gXoR`Lv4xiuo1} zS%zcQ&_~8+DpQ^{>YS;1_UPvy>4SEOzQ9^0@%)O)KxLnHS_eVJSn+J$+?|2Iom+;z z1C_l-FdGZ)wBUi(-yXTNETpQtjwt|)$YVacLlrb_3EN{jt3ZoaV@ou%MqevPH!J+^ z*jk86h@3~ca|+nk7QXSAjej;32H59a1tHP3Tr(309co_Iy=0vU>Mdi-N(NQA-X5FW zQ%pj_vQAd3Ocgb%zzo466;qd*%D@M(+-gRkQ{W7#J&P}2LafI1orTy6$t*`Cn?^5E z>1a5+=|i?@#+6D>5L6Ex3Q0qVrqIQ91XA0xy=*7CRONHL>QO6T3BY~_3=kpd@xt+A zE=uXg>YPt@oJiL6k{d;!w1dL{Fx+PXAeU;Gnebp!s*7jJ?&qgAVz#W>glhDoZ#;m@ z*@_r21wkg(MC(e1ganvOq-VNaIi-{WOgyQ*gJ;DmU(8?>J$*eYz!kZbm4glrabMaO zU8q&iPvjv16*_{DQfFr^cr95|@Ij^_5_6_d+4w{ z)72uc=s2{qfaz3`d7;u}Q>!MQ5TrXLUEki;*xuS1GS;EQIyO-T$QOK}*LBjO-Q`u# zt!5BHVss`%m(_O5D|;gpFprL?z#ha#$?NfMSaboc9`{qe5V$7 zMeTyYL?JLbnSglVQ} z_;(633aomlAOkP`jmPWg)uHvF(y^%e`Dusak>j_A>e+Mg1N(TT9Z#R;Py5k$zw~>( zEY5DSDZNXVSu65&cR%}UBYw3_VfqL+8Q*sFB!iszb#$_`_QFhTi0{so?ADs?FKuzv zJENNRNuTQWa*{u4{PdMFY1-0IB;>c}8MJd;xA$HTYFWR(FjCOGA3u>Z8><}d!hP6T zwlVn;*YV%H*Z-@i&)ee(>^#1(toK9NU3E*g68=~Tu4g4j^ECw9i%oB@qb?@T_Gd${ zu#6e^YN|FtaQyOaI;iC7MeGbnePTWS0{wa8MDX6Ih#W5sa@-YRKCZi=2~wdat|5(L z!Mdu97tQrZ`()94RjIZYoam{^SgWhlNOeiRvzz0jlwwfiIUOsz$~<`|7{Sd(%^YI_ zD!kh>U6zCa0zt%(7%))}d=E1jbS;8p5uQ<|P#Uku4dWnLI0l@+ZtXwTo&c<2F1HE@ zfonZ)2@7=+gb?QG?HZ^-Fyl}Tw0MKgHLpUK0HOpCP8V4JDp!0=tlNyqOU4si+}-Er zsC(&myI~RaXE*ygRIP%lC0hq zi>ElHIwV1%p?~pVor(jRPa!@$D$of$fI-Itl}6g9xn}`HalxT6SLTSI3lvfJr$aBW z|I%#2=(2E$#|j>Si(<{`K_REm#Qv`z4orlChMaZ(d+o>+)|D3I0!BO@56 zf!sC?KZ|0ZdPn8lIQ)z$QYEKipT;ny;WOGxUJ*A8%&4=TNha!qy*1iXb!(@WXZTIU z(ZpXSLNGSIj3>c;M4&)pX|nlL^)tHeV#@|tC$YZ%j^NEOvSy7TJ#k}5f)pE*yAD^c z+RT(@YyxspNzu8C?yiWfzKa%BxuKU=NRrvzqjDBvKuwpFzXQ=ir^meRJ*1i7l6T4> ztk`*pT?ORwZauy7u|e-2Gsr(*F#HP}?%z*^0w{id@s!CGZVZP%mX))llnNYUG6e*b zu7CCHwPbg-G8YAa3LL&KtzwvKERFlj>=XX8`>l)`pZ+4_&XfkCq?AwG zYMPR<>1?GEG=n@(@0^3R2o;9}r_&tPR(h+f3zI>_3+Z2*wkmDKPZ%D?=q`ZV=;X^u z11q(FvFGSjp;Z}@-HRMxf_Y$aInYG*DC4icwQSho8SvsmBPYzoX}!e+P&6w$8ER9- zg|gB%p*XzfgoS>4UU>$Ohrz=nm6U>lpTKFjke({z z?yO2~?i6xyslj?LRU#G*KHiWw>${OkfIiaKl!h-Emv zIsvXDaEnTB0TmbYV&&g&#Fb?NiJu#UuM|oIR^jAb?q;Z zMtd?Zl14fM^;CT)RhZhk@4?fNI?yz9GEv!{lu~9Xsb(R`pGt=1Zh3`%_Wg5-p%PDt z@5O)gr!<-#{jr;j{hAtpWSKy|@eHi7?T?540s|6opW0VEFiwAXAF6ru=!xVp`Ni5q z8?<~>sbK9k70krH%VsFO*8Cp*v`_vVD{d#7mMxD?I2ON?e)oXF z*Wr%-t%?0cUao5bG}L2oPut?FP{+CYrv{30akrVFz4Q@nrvN$`USX_5xJu6;G07T8 z#}cJ?rm;C9vcEW7DxP`&B9VSc;=D zf#8g3rbL?Jx=cC2Oii4`0uMVoxp^yk4-4X0_qR{uZcp2fi7wL$dyn&&k!_72odM^y zRAe3~SEXMX(qWYMp*ggJ?GWZr|4!dF8R!I`nmaA!6FL*^tsOu`YgW(#R=TYLC`#5$y1nM`#sfOj>o$_sG7jd(Vo zw%;AX`mEUF=F_X!5*A!=-cuJ$naCBbCf%QB?b4qj9QF_?P7% zSWzXA0KRTv_n^o3CAVEF)j;-^afYf70U~7ynZZ1XLrHYiB-7yXaghkA81eC%9rhoC zW%tXtu*Z;Go%TQaZiY20U(6FwI(;>kLl98J%vQ4v{X1ylCLtTJj!!TDaX}XH0d9kT z6aYfNw$IaVJa%k4D757Lo>rf2t9u2HqoKI92BfSjqGI2 z59gIM{}|)gS`oCr+P8M&#iSZD3aD~femv|I3YuT}#o^&2^Fv!=N^ZhQ|;Lr7& z+Rn9RC~;R_GaZ6Q%gOS3w1A_#owtfty%g+|U0FI=ITmWJUu7ZKW)2hmGNQrA;FV_m zFxpBBC97sNm4 zT=qZ_oVUXE6ZLsXOZ{}L`euVsxhiYWNoe^(O#q}vPZSD1;H6? zM{=pAiz12&+vfiGz_}woaJM@B1Y}a{iHT?8Q0Sz#Xd+vzGyy>45L_HK#uKYvUNw4& zNOY^EpRX>PM~#-e_zw@w{~hFC+;_lXWhCsl&d&sQB_~XB}K;jV3j@ViyBr7D&E<1o4Gm25YRk?UO({ zJ!OE8jhS%IaksYNlSNOtvyxB<2 z?H4ayR0~_q*ED}13Ob-D2uu@+ge*EAe;WfjdPWKP#v^|`WZFzji5wEXUnsnz!=f!8 zWOb-iY!tqCp;Ls8g)vufj9uDdDu21H0&XeJlG&OO4_YtLdNuY*qZDwA_RE9oFOSv+ zK=?0hYlo9?Lr8yb8jpk6)HT>%9m5rmr-@Ia^PxvB-kt9Ee*oWw0qPuowz3b zwuA#y6)A%1u1^xo#0B!B<0Z%$s)%Dcb!qL?ausd1F384PUR4^N&`26h1QNBUsZGYh zDW*-R9rq}}g;~wA-7#my?=sx5AmFd!dxd|Qn9X$sRs-~r`_EilF~S|39Oir!b>YXi zJ`!E$+3J5a`h}Ep!fBwAwr8BXy?gg2^^6a<)a9CN?B6n_;>@Jeh{?@rCh~+uoxH$8 z@SWmQ9)pvc*6$qZjN8x7k@)m(TGFESp zWz=P@_9;~>ui?!!znONox3bse>-$b>eSNE8;I1s9|E8)#HxmeO`h3KPfW`gcddi>rnw|9EK?mWvGGVs?2D!yl zbG8T|#Wqe+m~tq)>BxD%r8U~HEov$E4&Fl)5*gP5-z*{I_iPA3pxA^l1vFr42_vG}$55^%yCi*iQ2~DnUFJg9yJE8Ydf~fP7XH2(2GZIk zdT5$v+Jdot z+6$6zN{y&t1d9=k(*2%A(RekC!%=?61-8$A%vi_iAhs&AfaZxSFHV`F3Dpam@;?FM=bgS~XCP?E5F68rorekX^=zm!gO2nCrJFmym%F+br154pr+cdJ zEp5tUNHC?x{wsYW+{SBtumokSVG2^%T;fC8LY=?pT7`1 z1z~kB2zFG_9*0Di=Uaoh1bB8+4x*|Oj(K=+lc3>$?9IvEpW}Q2pjdlHDe(q@-*`aK z)2xs_ioDvM%CRaYY8rFm**wSkTF2Y?Qy4E!PI&RllIwEVfV67D7nMvsf^`>7&b>RO9V;n4;3Uy|92J3{ot zou@HfR&dq%qA}QAhfT!T1bJZR{CJ?pNMN~U#NsQ2_M>;<<_4?^%*r9KDe5*kmOdUX z^xQ!b(fn@H>BkfCD(sUl4nTdcmg~f@RGDX+@x0?5iUf3=7dtzv1J=H`@Gvfvt0{zKPGqw(p$P zv0jQtX_i@XK$;{Pdkf2PTB8lt{(|RQWZ=1%u&lI9uY%CUqsF6zBgzBI8Pn$E_?b?l zPTL)q$5r|J!eWLa)B8n5wSua%c)c&is-88S+joCyh+guoJ$JL7ualCvspvixM%j&4 z=p4evcDBm{e903$tQ_Q9KyFPqmgRz3&pm5MtvLvJIX1v^#J9$!Dz@pb zt~(z+n@e5$Xz-Z3@qIzahVb+K%Z8)n1>Y~PmR&RKSW98wxm*}V*vpb1t@>!xjrjQ1|FCCLTLY+bffA>Z2?m1+}( z)2XGlVq{{{8!fbp6b5Mkk33(x6T0V5e;PKA$^*|p@5pNrAKz556~5L~HfnJra?_i& zr+rqF*Rh3H(xv9M&i)Ico?}Fq7TM;i*`{YAo)6>2>iX2{Arq1!0qO5{RS*QY1yQf@ z$$xP4hpgx(qUi9~3%o6?+BP2&XIu^zXU6=oM2UWTXOA4cXDq1wS?Z6Cdy7DS=H7G$ z8H%0W^fof7&+1(c6D7gqM~Wd%FYQ-K`A&bF%?t@aTwYBNC4+($;X=^Q0TWaJmqioa z5to(c1soE5jYz+gUN5#(D%7+2Mgp#xYJD1)CuC*rKy&C(Y4x~~{Sae%^r$&I8T$gD zh5jZifVo}|C^`jHJO||=$DveVOzI~Kew``QWtWBxwbtjC*ipjBRC;3m`uYq21^%V3JF%!0*nX)T- zk*5kyMMl&>sFW&HA1W==v*woblrJdKCkEzR>znz8^tO(_Ii(awO3Q@2y~8gK&&}Bf z*R?TA0Eg^@05p~X*i4eTz?^Wl!m$YDfr8LqK#gz7E<@m0$i}{>@SsQdz9)azd5jXv zq8%@W@3~h&gG~a?uP75m0M;75bmZy6HqD4YM_y&Ro5=aOHojUrhR6%b$9YEfuJ zBY4ke6eTBcGRi(x1)1@jv0abOSbbiTS!X}j*L-R)6B0p0AWOgAcO(A%j|64w>hH@``M1N`xsNI@}7k1F1`aT9KXnV=g@nX+w?D_1xjislGwEH{{j# zwxBz{(gWXDATq>PlzZJ(2g+;rQTHZ0MYq;oEK6I>YTvzgXUyi6;=S$K?fkliX&dL% zXPE1p5a#G9Yc?-+@eEXi>G|DN!J{|F25+ILXImy`k{4m3a&V@n=*IK`NH=tYs$#>f zi>ZH?^1(phsf@3~(Hn=&<4qxOAo9weFZIU@bE}J7ZB&$QO;}ZQPX4%Q{RqE%i7>b< z90PMitgN6g?`w=*_5?;~G&oz8x9PI?#u%c(5ZbN5KczpSba==boes~~r zKpGBQR_?XW4T6jj6dj$S?=mmgklICW5bg6DGUrudAx*@9k6p0bm1;jpWo|J06~prP zW3f#InGVZk_4A&IU*n4_HYYd}*L2XY%-hEI;$TCtiIC6JtR6^DVJBh!xYX~g{l-HN zi(k6J&Yz94B`{1xm>Rmw>#a$Ke&4A@omriz5up5_W+C$fc)T>BE{l9TmIde=2;g9n zk+2$nmC9KmM6N@}?ZcGI@t^^A+kRxalN<3n(A{2U9%zPO<yf(0^)8s{a_In6*9n)FF3|%?R9stW?PMF8dHMx6_sP5?RZlhSP z9ZUkJSpDy}g66Fq=(k$y>zfk8)qCk|GpUs#hmlaCe87lSTA16!ql^$0 z;5mQife6%gZPU%0`?Zp4U!ZnDWuSuYT}1FFsu-)jQYI?ByEOl(&7NAK$--yr3L5v8 zoMc<#Fn)|+46sbHd8j6!{SHZy?sBITOX%0}be`F~JhNc#6BYJj zsQZ9lME=KLr#>boPS^)jkYL)CQ&mL-d30ZhKkSAZ8_^i%HuTGT(qS#1#}-K%UKj)U zBe6}%(Bu7B;%p*8rgGq8%hO+1BkIy9{CkT+3mghS%UDwPp~l>4SdgV->DvHG!=y4>y!&$Dc+kPTU)o-y0-0!E>H%04!aM4*AXO zBcQCP9VVo+Pv+grFOpl9j~;xR@ulVYMqMDeoy-q`OW}Qosuc}mrRZx%i;el3o9w`Q zt9P_BSC_I^m3l#@84_E|ny;@!9ZNr@&Gu(fmZ~X@daXM@VpRMpO}Eb}C2^65#+xyt zG*jM9d7*KQ4ud+|DAUwqIgIHyo}{)VVF`4CwTTIbOhWruPU&rh2ysTlZM5`jjuLDr z?`itWCeBP|{Tb3z#>W`vWa*nfZF;RUUwmF?wy_MOVIjgrOZAc939@72In+~W@#K=o zwBTz_MZ`n~1g$b#qt;LnjMD6J=r=fgbLo6%Ai}BRx};hI6e;XGaB3v%ASA8i3Y*e< z6trZyeNg7_*Lc6S6PMDF|W+zf0sC^t<4)O(frMnSdvk zIQI!0GiKykLenT8$FdioT}5c7FUS-)y_l|PIx@Cz{+)o@GR$a{vB)D^t1t&61AJy{ zgvmD>)im5)Me-dod_2T@eYB-upNqWdTI`Vv?$flkHXurVTVFws>R4dH zOqLO`)#_mcE(dru8V-gNAQ4&|x8GF9+lzf&74wvVgkfVn3)<2`VVN(^hEsE#HuU8$ zvy}R6)4L=&B;JHeV@ez=z)Ij~I~xmQYi9PpZe?xje|1ne2{hJP?wfwU5fpNg_{`bW zma%)L4{AvW?4^`=Sw*dupU9M4uO1hdN0E z<|l{O)_m(OcyRWzi3#-vYr3b|4uM=r9SWRs02_^WU67;Aq)8+`B44YU1T&7CuXS#j zN$=qhh~Pobrp0Xu|Hf0bZS;9Q9uYV5k9p%Ccgz1GuPWqITDmBQKJPHs8YY@D4VZUQ4<#tcHRbHl zEUZtaff>7C2MzuC6Rf=jANP#Y(O(G!&_6p~Gq4F?vP?eyiR+nvE3fHKE_6VSx@^-u z*`tPS&irwsL+QnST9WyYr1LHDQ0~lExSm)H9nb{N ze5WMEOC-x5V6B%-X|^Fge#LsPw|c=O1NPb&qPy z)r6ek@w3Q0tD8za(Xu+Il-$|MvZH)9-erHvqk9S|n)XQj7Z4N4q@|10+rY z%|jt`*OUP)`x2i=SI^yWw;j}av^Eg>Fy}7msJSQoJBigzYvtz+v%?b^tu(|4IE?oq zNz+%#!bZ33lA4>BzydkLUB5v?K01j+%~O0l(iMCyTPFjhf4;(6;DJu0BKT4EmABHaLk)u90&GfaP>_kvqQVT3vAsOAl$TKo$p69FyN5%$|9#)B)hba) z_9JGMWWNhxFn(zvV;LD^(~wCLM#wIMiIx4{Du(RSn8p}mvKbv)c%qd%&@G;^Nc@8|P=z1}cVWA`%G<&4bJR{z`Y+&)(` z_z~WlA8qtV$C&xIR-o4E!w5mfxcTwD-*6`UewJ=Rvln>3Bu)Vc_|7mF-x>En%>se} zSikEJgShc7;2Ypu=7}?$5=0-(1ul>U_oLhrFY^Ocnb$*lMi&BRK$aN-r#&NEwnLQr z)K>=>^JRfI)HXL8qMv*sUT*n$#D_^Adr8M+J^q{FoKsKRZ#Qh7xAV}c%t%tcd^~ex z=gx0>JAEIV*LvziZHS(VKt-m94y#s11(Ca5v$Dfp_ip%s=X}X}@9LdBT>03o2k?&q z3lM1yBYh;N=)H>ghAhvA-x|%`&ZJ^1##9glmuT_bVV}C}+I<&RA*nRGB=IU|d^06|qk|za0~nLt_`V zy2<}H?D9X!ke&wjRwP%PIJkgy{q$;47OKQoWydFM2jAHa7N~L$_x#Y!6qtraliuiP z>8gy;_cZq{K5zQ{9B4)e_M_(u{~8fmknWuH{xnM-;>^omQG2#N)D!W3Y2kSZ32`?p zoA}!FGOGkpJ$!fdYW`D_l3mrTbE={5YoiFFl0Vb4p=~wGH?S%Uh2EUsv+%DtPPa?| zS-$E@f}p#p{s5AE2d6%9a2jhf!CI)D$+HfRJWpDBYna5@7t9!+ycBM}8-r`K3;jj1 zJ;k38)7DIrvubehb}W3?6ZbXXi+#CRDGDQIu*+Wdliyo!#W+~b!@`Mst;2$y)R*cz zwhwn+UmND^f(@cl+rHj2*pBzDcvs=XzfQuP?Ak9s5Lv*1#&vCoM4VboWHH^(?_Ua0tR+QTYvDturpsM(G7;T#fcZrYIaB?E5?+xc8qoV^_ z;&11BWD|S`W~c0ub>G)KB<{VK4tB}6m~wS^w&J82Ya1yWef?RU>xymB!8p2MD>c{h z7Wj1BoA{q2#8bPD*BV_1Um7=80CZQtwB(hNVxm1aCz)-Zc4l0l{`9@=v$;XBlIVKF zeI>^WyvS($Z;kM-TE2S!yQowzjk|@Xxkt1H!37_rxjTJV2MJc4&P4ADXgPxpWt}!Y5jcrBT39v4kH%yq6Lv7;X*9()~DCF z6(}Xgv`;CgR&;=jR3MK$ryT~MgL_Z`6cE-3PVg^t`g>(jk)MH6P%88!CxsSWWfy+2 zFcLW5U2d4rRh=NV(;M0z4PV(=##m2mxnY zbVH(ow5;(S!7E2@5+RX@>?>APNESjP6agce0zIpgEl8xY{6uk6r3upsV+S)59xho; zBFdwj+C3Z#LhqJ(crfqgSDL&e+rYIAXU+G1+b;zMMF5J64Sdykg0tNXPWmC66eovm z27>{dl9>ynLEC~8mmE*eJ*)Z?JJN%4C@&PvO%=F+K7p#yCj;(!U|J_>kLP9Dib?{I zfLoTVKH#D@Uapii7mTb%CMLrS7wXE?nTRYha}+;6F;!H3$);B7ECE%eF`bb?hc> zPmrBd0w$Bl-OR9E!fOQ!tr&Bt6sMyl{hPRg2S4g^A`{Y_N#qB*a6KU|K#lEWyIJMJ zMsZH&lf_Q~KX16jKp$2I7Eketo{fa7lx45d$nWPaOU=yaYT3@aNZp@uy-RkowMIlN zv4n@G*YeM(9)r1sW`;VPADu(CQQlO& zNijV-`abb@>uATKsgg457jXI&;QJ#b9o^=j1LTq&nvgko9Hb$Cfd{|IVLw! z3^UFl0sAowk(XbRU-h1-esM+J_}*MpPz&%|h+CMCT#tUzS%Mr%2o?_jsrh&?h>mH7yI8)Y;DzK{D4|jc;!hnr6 zT|YYP5VqJ6WRUHff6KMsVmG(4Ceu!7whlG9!Hw3;+p?-*1)54bCL^Lw)l=Pr>fNh+!F6++bvI(l zfaV@b|0U_#-q95`Q#epq5uK{-j%3_W_oWjuNos9!1JOj!qYC0~$94Qhte&ar5k;hf zJ34ewV_C`w{nUxs3g+VM(;1*zA#&b;r~C3hB3gO9G0c)0fn_2uf$tpxcntYb@a9Kr zj$CwF+_|Bk<}ZpkE!Vc&4_QE{HT7x+??&k!M^nG#&y`x*D!KJ`EeE2ArnoqN@`nSiTbf z@G?Z8#9hswT}(782{P_0)~>7fsqIW;D~h|e(5Hf|v%hEjJOjd^Yx#<=-l?XSpXV40 z-+o)mEWh0MXWvY@s!qi7Yx!!3*Wu>(RRE9P9bXM28O_6>NYjsQNgsFMnW1Fi3SoAX zy_IopD(MwKDjmKG18`J$OU(d}|x``rcl@vfEB~(69WMRd62!0p( zX1~1i*Fa0@gI@q$RS8@3K4ij+yUWNG;&)wZ9zXjc(Y|GpzArY2=tzYpY^;7+)ihaq z5!sT%{=9K3?d)swI{CR(0gUxVT!^0tB$o@2z0-p)qSAM23lF;~KqB=c^;+*iJ7to% z0O-dUFN=02%DN&yIwd`u%QjIRh&?S|dh2=cS19ndDL7vaGDrbjm5*UA z8_X;Wl7cYIU!9cN7=~R$l$JN}H28VzR{sb8^M8k({sY1N5|S0K*eM=2(kT_9fb-|4 zvno>KP8(41_?a}_UbnVY@~`{MR95z>cfm%B()1{mM?4Pf|A!)mvS+s#;}%?q-G~vL^CXllpv0DBo~8 zAd@0Br&7uk?`;-T{J0HXeoq02FY044Qu|HSRys5bJ1c$6#rWIkS*x|>4`3;Q1$!jr zM@#o&=HlVFGvJiJ4@n=$Z_FO#vEMe`cim&4WV73K>-gb)ZCdxHp=zAeEOUAGeM!&J zZ{ef>*#|3K6Gn;}1AiZKc#(eZLl)-+&R5-ZjOHwxT4<G%ksS} z0XR9;kc^wv$X#D`EBWwYxf(ufG#k=X&C-wfj_Ik@yiq^7rgibG4ur7_kdm;;X9Nhg zR_W~V^5W!c-$(wi42~DYE-xDc389)DmStba&+C?rXvUB++j=p_JgQPP)rX(Q`W^{b zH{#@{BSU`T!+7oj2i=sf`+lsbv^z~95})q1xwK^cf}ZtKYH1xkH$gA6&vrwVF9T= zp?Pe~^rvetrUk~aH8D~gy5jHJ8aXFRxkUxX6$f<#PM>JFqlC_WLz&;Rto2@>7;1PA zm8k+Cvt@3RUT88T5ANxrFOhBTdqvgK`$4yl-gA7Vv%L9;eD*6}zOBLw7S|~7hQ13| zBENwND1uaWk@pJ4u@9K7a zJCX#nVq7+UuzB#Sf0u;0I?H2i8gNLk7samwbAzMX-JPn_+Ft{3&VEO!$ELsDhpaIQ zx$pgd3=>13xJN&Rk4p^6JjW(yWXn$^!(2k8@SH}9{DOfy>ivtFyZNfJX+#@>!?^X1 zE`|Lub`0BY{(PJQ50(xXt(gqfCnBKx>TZOQR6Gi9qQDd+4J`^)j_dI<{q zu%v(8(JI(;!C+6*gwsRh(-}^0NFH)T<74eMKo`Y>!4(9pv3Dj%Qn#-i(g(|b+L2k+ z6?wvZ)waN-v|ltJv5g8*s08K! z(kLFtp7-hgA>G5M#*Cr^^{QGd-~IV%_B|v9Q-~f1TSH0@pnQ9gJeT^%k)__;w~w1y zX;jBlK7RCW7|8~a1J*4#{`1B_w){=ir1CQpnKf~`%fOQ@+k-6%i|C}K(99c}PbDu8 z!Nv9e@<}fxDET_DcjSEP$DY*8{Aky|`lRNyvs3Cu#U`*&KA*W~0NB*U_`pS^x&V(p zvP=_{$4gsocCu;vo;OFxoUQ|h{-1$=qW&;`Rwc|^zJZVgPC*U zvow}&@AgRnXrI)iD|~5*s?3~CNqvg*!{Xm-(<1W^C+yB60)Z#*mR<>o;Ptk!dVhNpLxwB5&$v{oa^ikPHbE$o2g zYasI;Z!mLIybI2jmh{A;DXt_PRK7*yDp_r!0(ipAO~LDwBA-%^xn`Zpy1Cx~Jterq zUF)fMx7&9hG6o>hUGiPGw{w687}OFIcXT{tOCU_dAuGDu?$KGaG$8E|xtkGbHgCp& zq);mPUA-vUgC(w4+iV1in)8qt824*}e4SCmn9GmmfCv4fL+hUIg=TRAZTzHlceMDyGf;P^g%9PW9kg(MU9;N1bNV899!CGJQ!YV zxH7n<3M(}f3b_cKznLYU9-aO3d-w4(wTe`@a%wvmUKGTidubpjy`P=GOT zr}X7laOT^sIg)_6B$5*a=ies^T)78?s*Sc0fPG;rYy~o+V(Q4iwDc9~??Y)Jo3-sO z1e8ahY+gXjlT6q}LPF1yiizEH|6264frR5M#2p9MJzmJt!>Y(z8Bo-{FWfnyF-ORjV(=jw_4T7q`00DbGVguWZ1j8A+-bh>FkWM2 z^0P&)wLx(3_`z|0-5QV0|HO6d^#V1u^8o^a2jeSnawe|6>=>_mGGlJd%1sA1$uutO zip~4q+;DI;b2qnG?h<=#G{2RE3i^S`4GahmH==I3wl@V>vl`pv08S*%?68T|)}E)4 zd2^@i$+_%OZ3IRpEbBvsVT7Ba-E@ji-m>18yVw&)A!4;C1Tmfbc!_Hghx`^&koBx8 zpCGGXkAFcv_>B)B)56a8tFR+|XsNHc?-qM4<>J8DGOc=xi~f1X#c*zo@?8$$|y+KWr8_1RDm?`Gw$_L9V2iSd(eNxh>}|zC9!$P`j;aG zjXmcl7Yrob(oWCSESnpY+vf~k@U+H^h4J|9@F4rcr9Ppdks7Rz{pTV8QEP)p#LCNv zjhh<-x*Pc)od@ohU11cLpS3p~Z2Wyf{PMA9>tnCR23Cr0y){1vctZ6}-fT@~-7RXY zj$4Tv0QRZ74BdByfN~>}>x?*f%f~OF6XV6qRhK*L_YgthTV9%@<|@R1Vt~>Ka@^bZ zxm_7#j-G8;b+GlDeP8gnG3H%%47N1*e8>3}5xQyX`)I{$UuXf>`ZwFAq-y4DS~tg| zY>%EQC3~}kFHJO7v#cj0%S4Zc#@P2oe4f)S=^S^&hj~#r) zdH;Rr*UJDy8ZR2ZVOMqae~<$I$D>E6LV(~0E?H(C{LB?+cTKjIhGM@A>lZDreOGeoEwV8_ zH*$B3>1)DwPB9RLOg-mPKrZVy)kz-WuG#)!GXz*bPnAdL3T`Ycy?&|U7(eh86!1AE zn9JWNc1vnksSJn%if~G#r>4H%1D((FlP)OnFD#rfE^M(22{_@Twd>|4dn$QRYurq> zPHWF}$Rrao6u!A5q2(1`3IqfO4f|kt(5YS@E}n#zMj`zX1grcS*V5JXaU#4_VI@gf z>fr<)khl}HU-@41l};UO*i<Zy?d8O27yXN<8 z8UG|9@c+E+{C}>F|B3z*Y;Z8$)ijd?tj_@M4bs&oE^$e$vCeCw*zy6eAw0cN05aO{ zrDZ}dv>m+Wfd6RPvPBdqXq$XOy{v8Pqyq;aZ_Z50=C>}%jo65J5O+ePkvw!+G?nKl zpI!U>?Uk8gk~Q*CSlJzG8#bABx5?XCGQGkOMSwYxhENgY5me|i0;(g*gmpKcmN^;M@bU`RhubW^zmVl)(GtUmZ_v=<{tE5Tz2GZ1!R*TFe~1 zO!+G!SNP*L8M}jedEcI~`&1QH8t5{hXSkM0wFyZhc}VS-%St|Weyy>3fwZLo^lvV8 zwjktC^n6Vi7T4iYnX+2*@k;S{nf&VG*IL@@?UrKZP#G7Ww4Rk0Sqmvgr3086BF`?m zL&B-YcE%m$i`v5L^8H-EkyxD_mRs-0y9u_#ZYI<;ZlSBOdOrsT^gvtY7A&5Gno@I_ z;1lS8g%hKIu0xMM$DEZREq4g@D~X~HZ$N-O&f#d*h{Og;e}djB!@Tt(dUNYr|4p^= zZttBd!)2b@S{ET%-pN@6Q?{B9#->xI15`c`8LnZHR;vRpMntMjF?veZK=s}Ul^&n4 zrPPV)dbv_KI@2XI#~`B;eOPXknn)FkiMX&)pASs7drqz}y*~{THG)f=IUQ(gejo6h z_yxs($fjj4Tm&w)i}C{3oC!9s$cO9ED+=^X3C#JiTp>2F(4$uXUW*gofS%P` zV*-nKF%N8l#|xPye$8?(I80J1a9{JQ8$ru*T)gLv(%CYj>>hi=%AyyVJ<%nstRKw~ z89u7VjjDCTlsP~&SA^7kDJ|l+IK{jL} z`X)%5~G4|$Iq1!qy{bES_QTLqt%a1nOc z5q2@`b?C0tJ8fA(j=tX%A0VgBFF%Oi%>S$F+x1jvF>S!{<$n$A{Fkre;hlKi)BTr5 zmPuV8oavrE&JrMB4Rtf(j7PVOf(`>eku3wZWjBYhlEmaxA?ii zT*Gl7zXu8ci~u(e#(^T2X_vB_3FJe_av&0c)-dzpXR;|wlY?zO${tDcArA8)Y` zKNPK*F;?k1mcOE+mec5tiO`rNp_(-JY>2i6axqH7aBZK^yeu+HRa@1jPz5tp8LbhT z=#1~0%k5i~X>v?PH;VUc9cTOgedqzsd+)b{@ruI-hjG^YdQRu8rFdYmTPl4>wHJ`j zrIQ6lkT>P3N~b1q&Ou9=BjVV&9~*0^NpoXAHkgh=dts(sTkxU8)9{eO8^E@JLsu=m z`N_t`0*{)!fJ-zUR`*sx2*Y(H(q6HE6XgYx)0hiESgkcZ7IqEpm!9{G8=?0u66B(6 zjU>T31It(B4eRwPU7eFUsrLF5R0ehHMvYZzjn%zSow(kGU%Ar{xR$)Z{o?lPTrd+Y z^HulV414B}oBW{V__W#DxHDXVmk_^|lh^}3Mc|+_1aNI?h`CeL(&EXDnEg6@UEt5l z9u*9?S;c3h`>{sF@1b8SS_Z0blH`~jr{_!tG~iPErJ;r#!-jT-XnJVw=lLtFoyRvY z!^Ih7=fmm)Iux?0uWQBBj{MbRJ9vt!+Q5##GudMf$#Csuhu>+GwuWAL;Y9X0kNsg- zwYpw=2N?P%`S&RKEQ$K@DmiOgm3zTh@m@>N_JJ0VaX8V)Sz~W^J07-X-3RxSAK?ep z@(FCAEk)Ld2-b(u2AW@-$3%TuFF(e~iqq!Wrqe&obx8(NqQg(4WKxQQv%ObsT*@6Q z!b~UoS%Kc_6ErEJY8n-P^V18KKTx$15NmDiT&eDszh*z{LN*=t-$Xpw*-XF(qfCR( zrszC7TN370>1Z$re8g(%F_D^9rCjZ4(q?|Z6moqcwQ;p@#f>pCm&^Nuvp@jOV(XE# zIWXIc$leCW4KF~sO|v7ZM<4F}tJ|WmCpJV97`*`>FO#2x47)_<`qn3?2_y|89}&7xNeZntSIM#>`D?-^ND@M9*hn29 zZ-*^ZWbo6Z`|hCsT$TO1Topz_!L4nYcT^XE|B0V|EqcOEYSW2jG>(` zaZ_?bYCO8bgN_o97O@pXZ3eWh6aoidVrk0BIzGU7_Z;4*92p&&nSXm()`YC~nm8n_ z8Lsy7Vt#GwBMi^2y3CG--Ns z@y{O-5qYrGM%a}9IQa;|5HljF@i^u}j!ue5G6vWF5P13geTd>H>Q!2)HbqQNQ5H^R zLMB(MxO!`GA4rEi4iuL;fP7}DqSrA2_&O~z!HU;#LHrihc2+#NJ&K(xiH!?U5`@e| zD7y6kE{_U@`Ss3I6F4&;&aTU7`*Bw8PZ%XJ4jRxplWdn1E!r)fDVlhBc{$QenR$cJ z>WFMDnhci@sJ4IvL|r>KS27*3`)O#IcskIzIJdbnHM=9($i@j%!68_NT$_3GJVckA z7PkLc;poTv&Tp5xRu<%KkA1qDnXKbqYaO$|I~zA{2Gjwfg|HROzYo>>F}H>IvJA*b zM!Q{2fRbdQ4QC|9kkaAd-gPk!$|?IWx<2b#c4Efu=qz)Zy?SLSxt$Uieap-z{at+I z4+Fnw_@!miPkx@4ikHH06rt>44WCBO8ChGQ1ct_Yv(l?cT$A?SS+HV?7aih+x9cyb zUa^J-*p_?&(5SFCae=IOsg)x5Vadln^>TOGE7(p*&r zq$Q#AEY_9*HRHr}+~LJ6CgZ#-1Io{FK2tGEDJ_!`K{uy&9NW{@K05~eDztj_*wAn5 zJyp>d8w0nFs=UT`MzgTbL)YrEuQ#v|5yT;QSW}KUk3sks{J$o-T7gIY`I7hxVqYBDMXuhlc zG3}r`-xPI`W4cX+FUO3sb)CjT>Nbfrbwp3=6FyXgNi9@`UFkJSR!5TI)Dyo2+O%sp zQ|)DehH1SQ)hQD;Tu5c|Z{G#OCJeXs2o1Xq1=1B;Yi*x12R&FZPyQW_l^*GA+U8 zPQI*o{^c_qbf4p9uFyL4bOq~os6rFxAkTM691ueG#K4RESl5qQm~DQ!6lL`JVX60N z6^#g%qj*w^sjXP+3XG2Hn=DXONLwu>xyfw|A(Q<~s$(IB%SYh(%BR<45zjQ}b^|V4 zv{Pa&-UMC%*7@LmM@0!1nx!`&QHr_546FGxLxwymyY^=HzQIyO($|Vypfad(i)J*< z1?rq-utF>)s<+X62{ssjTOeG^T*0-pBBBT8lGJ)bD>gBM*(tDew zeU3T+^hu5PB7yyIG1*2#+fGGh0uf{)*jrwFtIMM0RItT5D^5bSv5xG*+|G-DH@7_= zWNttOx};=u6M#l~@uTws6HZ}@K=I~BKElOsy5%i7DfOvLecUuUoN z&P0W=J_n<&l^(^p-9Fp#y8{Nl@nvT+3ZuW0g@Ea2`HbsjIAu(aWxR}O6hY!a7!l{P zQlk1~p#==DstUnt*l!`p2t?^Z`hY_f{^8>E`3L_K#rpT;{69fZjvT^KWA{Q1Uh^!) z&HcDujycd1t^E5-;u$}3Uv-`0=fVn|rYm!oW$@Lgb+<0juO5nEc;}S48~ej#$ghp5 zww$t@mhfof(&j+pcXcQ&f>(ZJ5?OwS0aou9`m1y$Jz~GhjBt!(%G77fP}Hr%_C%a} zuLBu(@}-_Souu#H&WAw=raIv92nhmRsG`8(dPyI+QtCCqFieL#YnHeD?y+$^=c?}8 zFB7b;RH$YPb4%<~cc&INxMK+{-dtFT9lXYMtz6C(scOU$dcd5zuN9&TI4e3=HHFHG z?K=5!rca&FBk3#15D}X6*YF$m-)&vbzOoioT!)-@gGhN7i>ZarSA55RL=A`ykH0ySY z8t`sSb_K=@W0O181*e}sZ|-WD>IC0^bg=7ut=%PHX^x@^@b%lj$;8y<(mY@L)CGWg zXKxd$uD`{|7*9Xnkf8;j!&$qBRu7azpIv_S9jdI`y^8zo)Ba`I$5S;8QzE9pnufea zS^^=9tl@H(f2{_f&TEN?tBwUaVX1@M5a|^Gm{HUsd0Q$Jo2BHqIjt+6L=wtgBWG16 z@Indcl$0z#(2F!v8>&01Y_*hjL>I%q&K0@>+>91qRY*1+zi!iing4)gahwUVE~hCr zL5Ou_Y|)F0&P^?oHyhhJ=s921bewcfEqq>;&Ns=4zN*=h+t+TKz4Td)EhhB_!=fzn z!%f~?@btw=)r<&xNEWa$-hoI4oUS2Fw2D+WMMV0Kmn3>lvbQBi|1u~wmbg^}zn@4# z708+la*7I+3eH#=t&cICl4I^tK|t2s#YdiKB+{7i2xIVQ`uYlv`ziU*|1rP&-yAu> z-w74ruJH#sFPG4ac^`I5UzdVEP^}mHq^@G0z`pMlE&8~`y^B8nHBdqB4*O;26x|`v zxwCO1B1qMa-Z}X#$m;X#AiSxqwb-Os#e|7`G8d^iO|nWhu>llzG^bgBmruaM@G1fJ z1rm8zvsTBG{D2ave9XvvHbe?@Q*)}6AcFLF$~8hw)eHfR-FMMaYifNLpVH1hMeudG zK!xI#=+Y{8ySodt-f6<=1ualnWmcbu7ZDSYqLRIBC)jf+&SJW`W!<}g^0S=&w1`%e z^8HGFqWwgq2bM&C@n}r@GxbAZfODz04FUbK_YxrE*D)ZuN{|s*E5b}f<_9W_3QC7! zA_FE#M)EyQ`pK9`RaEJ%?;qiYpRAnnRpHJ=cn8#f$i|bd=0Zf}Z?asouegxZtpf1C z75_3$oSWK;wni-FNIIuYcRUMq0tG~=lrUW+*M=iHN%i&x@h9E2}IjQZZ7 zpak;**{IOPcXR*J384m2*IEiZT~NU(%910}sXdPG*qSYau>Aef4Z3>s)5(B|>NT~! z=XZglxmr{3`Ve64Z#~DA8wjH$Kgk1b*2}JCvTDxRD*{*(LE2Y+?}~}K>X5DlCw{AO zPm{m<%6L(-U4>i1K=mg?Y%5`HS@_|ag0^qOLF}jgT`$EZ4mfr5tLFaMxn5A#fiYBz zGryLk)3l}Y$&st`u=ZMDvG)AutkrM#NWzm?qhm1sj}h_Yth}Tew=4@P?1Ho17}+Lw zo6^%j)Ups?bmc{K?8r|ho3J-v_IXEAeypAkTujny|3W8N~wxfYHfh)IcJ||)EBw_SM;U$h|VU$x{CM|F}PW@J!!%v%a ztrP_Bo8F|xL_SAaV^JTuv8O@TYhr)k#-NXZ3f##ib-4x1sW*PVM^ ziE6g;BIr#qs%4-}xdtN3JDuXlqMp|Bq%f0+$7ZM^NWjZA^#yPfKlVP` zGOgt>tDv<87bfd9fSdQW(128D~%2AG4`ynwmYJARqDVHT`xKNP2Ze&+Y_1^K}8rQ0ov@)EG zuDCsfiufE(5%)>Y_l5`-(FX&IU~_rSCtd`95de~T(;U?vv^4e;uaEySdZ}V=?>eVE zJ#?vJmUK1O8ffFf0lj&m*eJG^p5Ve*iT!w_rek6*3G3q=a(LdJv-)|DwK7cp#f^Fd zCSrQe@mTlD)B)x3FxHNFf#QYN8a!FE8p_MVNB0}_-6TA&cG>?L{XKw~$P?36`L?;x zaJ7HbrJ-1|cPwt$r^FGrr%G!~zW$=p&Q3<}#6%{IJw5 zAteH@BhHBblI!}(f!`NZ^(jsfa82jePsd}-wBsCSZDtstFVvZoJ8?Rgm}UZP*1!4i zfyA8uy+M{I7IXic+t<`FNEsv!!Z#I(h$FvF9SRuH#XK$e8QGwuN&toP227I8uWLl? z7@x4>eJ!wXq6CcN7PGK>&a?agwu>W%3!RWGKw^j8i5d9cqr@i3A-D0uTB3F%>XCY% zk(&0pkpTmq+l9O+p4Q)o@~roZ0F6^B4%i=0+QVo`+?fMMK>x`@hq@iq>HAG$|I5I# zNqRJb8g}cb>wFJedy-)aU84z=q9<&E{i;m|E3+c z2cYi_a{}}@jfyMrKM=>x^r1y?Lg6bfrX$O)-}WyAg(hjbmFuL=_zz8B-S_fx#2=}< zI1g@^c<;y}LLN*y`(Zz zhPc(YNc``=IbZ#C|K-dTPT{M?w-mO+T7nVyC!7Lbe9r}lcS}Omu?PpHoJOm$ z_DigfIs!WaA9X$C_vSGuu=nd#I@!gQX68Z31T1FoUoU2!bOxE(fD$71P!WxrjS?9Y zk^GL%=#(Fmq)mUVU_VCmsz(7^RU@M`fLZ60$#5b;28xMp5^%)bfT!x?Yp(&pf{4;^ zSdVX{I;-`J`aW8heY#M;e>_>Ob&o8$&T^;wyp$dVYYM!xcF;YW`-ZT8mOCt}o3<($&`x08>^Q2)(6a}OiX&Yu{fS>K%&Zc& zd%+S1h6ghS*umbq=6g=>7Wf3Wj;YT^L5_%*IqWYy9`Z$JzypW(Qfhm+#xy+*I9`{! z@)#m)`<^jc;E<7#4T`b0Iz?9hVmlw6UlKI%`?PzJcjIOvc%WvAl=vo;9$m7(kGa-- z?c67a*kR}7W6oiL6_129t9t=Z*^(ii37{?qnjY9jCc3FJcd6^aVq;Xl_#fy)=FWT8 zK+Q~WIuO3?>5@25iW`ESvA<2Zp9>qxEPA>$2w?aZ=dwoPy#gob%<8j z5MlWGq2dCXx7rQRjC z`sgD+a>c~Oy^de(hH1E?!g{9}qq!}l%f}fJ^@b`=RuRuDK1qWfgN!IPRch_7LRur- z_tu#!L`Na(an`1^V4&QTyWP9}KNiQ?T*u!b#=RAaz~?=by{H(FYF~2s zV!7swK!u2jVJpfkOMV3CiJ}h&1CNCbxBfolbC)VCz4&;XV5^W$x}-Lq?RZZlZVb_m zBw8ceOQ+;p5vyVOK$Y&r&%>VQRuD<5>NPEqQkojUk9rZ=*O)nHoTKjk=u;7SUt(K6 zF!0wuS{(lqNan~PeM0us*s`tNPp6bM8eVv(h*fzfiX97jcNz>6it3mL?C#(F^{Lcy ziC`Y5T|`Pt^pv57WYW(UL=F)?{{i)S`^4U%>FY%Nj=3hUoUaJpQU)MKU<*~n)#O7g z6|a4qPVEg1(5!zKr4(f%NL9L9gryv zrl~o!V``ibXCzV9Gxp50<~nn;s@ZiPryz@9TESoQ`z0e6pMLpxscTSju;o6sJ-l}$ z+hi~U+vcuPXrCS~m@vm4o52zjqsuQe>uwC5TE7}O1rFTYrx5IC!**g#t}~&9CvH(v z_~FW|Z|T^+M=S29>u;9n-rycTl1fVD4d*sdH_PDg5Wg+I!Ps!vOztzcf%1?44Ip{s z5b*zmzaj4~?R!I{;bFot(_bY1yd2y; zHFvs_eflAj+d9bnSes=Ztb%{wYStNSyRepMyG~nIGt-b4h!pq>@*oun=EZcaqGI~Kji!zXpv7_B9DaC(y z_+@KktAg7X2lTI3ygE?X2i_U};H-GSC#USok!9&-K#yY^`+fi=vnOV|m$qX2%k{ak zy%v{@%Hn%ta>z>~C$M0qY@w3usF5~g+ohv|fkM4s?+R$2Jp1uc7S}Xbq|gHNwifD> z(>qO5iqSk=h`>jW2#eqzQCw*k!NNxn5$mJS8e@HEs?)@gvKrNuG9Agk5( z{4(F@OFf_TP7BLJ`ZJ2_&J*2x5Im{VK5%D>{OS8GTibi~fgk|gj=!BLp1OEALVR|0 zmsiR!*)s#I(q1al?SZPV6dVj#wC-#$FQJ8>&FTP6)gJu%NfHb6W@NN$hOoAO8l=w6 zug3^}>|vD7w@(hY` z#%KZ=!k^VguE5Gd!UOxz&0T$9Ss=}65$_+L)n-|S%%)uf#}12215#h6!&UEF1A6q_nr=R-|2*}5{afXKV@<*hal3#nZX=E| zvYr$7k{sA+!l&0=Q`i;}2=nb(pMan@7-a4o@}erL0D6S;Whjd95dk}K0o%`%RRVD= z5-DZQz0GPl-$cU5rsIr>^7B+P$e+Z&2$$@_t=WOdd_BJVfNrs_glmPlGY=DV&NR#! zxq-Dt^gHqSY`Nw^@cz7E~OW}EE*#BOd+e<*a|$Fn*VdS~le{8;vT zGt(hWx5guakk2ZQR^io<<(5K)qj*zCX%B^wG^a9H#Pg!t9yvqxZy#TZRlTs^z%}VYz+pSL&y}Z^Fk^@!bcs=0Kc)|QUB38L?H(s;t zeL_&~V)B;xm7PyNq#*sNIJ^Bq3O~wa(kS|C#mSG5_PZOqJjATMhm9WPL1p$clC+kW zrB3g7NmAMAdtRJx8a9&2t?RDcJ`Dn2XiUV~kqjQfnddbgJvckYME1Bwtd@%(F7ggU zBww+4CVrZ0XnMOQV2krs(52Vo60rhse^zDKFR=j#;>v;Wemzvz>tGa3?_%htFaEp< zNMFsCz}*gqmu(qY*x|w6I4D1tmANP>@Zt2ft)*#yMboi9J+9>{B`ovV!X!h5*nOXC z^f4n%&FxboKI8@aC+nWj1*d_X-*E(K^iLY1-qF;}sXIqi<*M0%C?_6|AY^>Wg<$9M z!I^3Tv%RHO(>m;gcP{zxrt+G-)bg9w_2uU#S0)+bt4U>GUMq8|3^Gx6^m~9+XxR6| z3`>q{u4I7MY0g427a#AZ`wP&`)RkrFw~xNUJ;T~y$bM8^#$q`r|Foc7r|c|V8Z<69FD z)m&UqL`Hd_*Jrbn-xwDd^Njx__7S)6^DkeAL^}GB$cc0<_+JAaO)4@k@*wZnh^6V^bF*E9Lv(J8;iD`$xD&l8#*+7WjPLE;N*4lZ1iHMXM)9Rox0MljyXTS!ixr zi7+3<<&j1dp1t=|d(8}mpAyt%@(%4+dGvN(*eBhg#rvTTAB^uV(!o6=vt~U2JoM{< z9x#)LPhqgSdcZOQkr|+agr-z+Y&}kA66eRg{L0IcYq6HrsrnYDPluq>fZVE3KRB75 zZD!MY|I#Sh`A2E>EQOdgqIuItC&sW`7CIQV*(H(EV|AgTzD%UZRb1c2Z|Jn^MR!`D zjTFVYdxmU&D-`+3j#cR_l*g(@5&(TKyw1Lu9gZ@ppQvbb&xq`7O&i3^eX&3OfCn$b z(7mO_m}@*`ehU?Mmd~ihq4?H&s&Qj~A5sJw4`*+NGG z9@iR?M9ZteN)KKZvuoI)N&`t6lVLq-$P|gOat$%M1~Anft|bG`)v`Nq0|}R5J*ur% zk5i9TXg-vLA4>*i3$^4vEqmHleMYUngA)~9P_~~@hBoHc?`e1(^v84Tz`&vFI8e8U zUrVPC#q6*va{V|$DQ?|p@f%!PZ$);e99M=)5Lj~lEMDTrcPq)Ylrwr3WE#o?bcth4 z_V!y2bI6w+_YUws)RCE7pr0vAMSYrlS*IX0k$~vZZKlL2vCVIvZDJi=A9LZpsCB?# zF>vJbe9{Rcby%6)6@goS+H1aBKfmTr0_>UT%8R10CmXPAIfl_|J#37A8G6A5Z+>1< zT!f#I?kJt+66&1ghYHPIH~t@RLHhp=7Zjp}{tH}C^ZyGjc)bX+^wCYMkjHt&Y4k4W z1EKL%MbI3BXW1nFYFio&*eM#7GJVeF1^hVwk`h3HrgMcD7Yzo_H6MucOP&BA^_7t` zpkK9yoaA7oHN(k3e~F*a^~(jwz=2X^z=%W&SssWNmLk1H@MknelaOazPmUYD$7qC7 z;nD<{L9*((WmJ|16!T|gI!W!OnoU?cQ6$nw*l{8M+~?s~-n0F(&T|5HzC9;qhP9Uc z)&B28IhKw6eduIM1@j|pbcRf>A1fDld9mmns}HFGT_DXZQn zmm2j&i*CepG*gNExQ#hShw~nISpLq5^WG_>W9AljFyv$xRk*a?wessZ1EG`ZQ~7}d zk|~mU6u09V&ucsDZ89j>)?pPR*+k|I%5Zp&MhZxy>zs-jTTebm=wi1k8K#r(Ctm0K zRoP2w#Ri_W0RTCVlG^8%a=XQ6ms$Op&@;GOGpqX*x={6l32+lF0m!t?F`7aBy=K4{ z8fUU!W-iX@90V4603Sx^HQ>kpHih%y+Voh8t%HUZTG7Q=i2RRQHX%jJ6i8??s;J!CR@5fR-^zUK2lWK>dZD+kMnfb~M6$Zob5a@a z74x>3PfKv>@r}NHG?A>)w7zuLy%eAgA$h!r5A||caNnYvXO59z6-@+(VO-BpXLsLJRtUF!=gpD6OZRrMw^`FD zv{Se45e!-0vvx6EzOdbWdQHfxXs&F!nevV6;8{DGtXWWps++&c0}x#;I^_>n1?uIkuVYs(@&!Z#E@y ziR$AtA&6SeJ;mR3;p5rbj*gs)G{cvHxxe5vIYwEjR-9y@0V=n?VzM$7Q$lgxyH{}L zwdn*QrT;@~=ViQ8&%!tlDvs^EK-Mk?$m;R$!=y+nUbsIet>G$K|p1wf26X+#26&&ZE zA%D8dFGTR|I3=&ZV&6pLW`(IiMzPz+us*+!e%{`s)+yO%1}da)Ry|@aLGdhg zx*Bcx)E}44q^4#0%1u66aSj-&w$ejXPv=xkMsFBJj;P+=HQ`K_fizAbmsKJ3(`tRp zY-3dICkXQ9H)H@Lr|jJ3lnHbFPAoo5;XikZ|1Pig&q0EpxHikIY|OZC-DCnYT5Ev`o4;FFYlIEfPqECah_}22svS`f6_X8b@tn zre}@Q)|}cB##tQyc|zXig1Yoa;jjB2GCub{Ii4unTgJLbDZSAm`tt@p<<$M_yi%P- zuJ_GmD-0ggtdY^F%ZqJXgEtBF-b8y7aFG|n7ej|M3KrCIVKRe$2@gNd6hgH32+ozn z(l?Eh8T*sAJUjbe)%$<^7O8tOtB6_2{C~$F|8$-IoA3TJyx_m82Pp5(%^v`of-fza zr87EZ#JUW@E}0v5oJirQSAW#VlHSBTzFJw*oKF7rzRC0IM zQ8Acfp-9@Z)7tf0G&rg3+)gT!Ah_r_E=CFQR-%2vE=OT(E^ke~4|^2b2bXOZ)c3wa z*-S9;RifzrIk4cX_{^Nqc#x6BUgGGL)(Ie(x@J8q<)Y%mgN;|w-bd(5kX`Nc%7PMz znDp15xdPx|d_GUZDRZY&4u9b}&u0i@Skm=cXhfYrYdJLQKH`-Ay6jA`tsU^o_eo1u zv(Y)Hpp)iA8^qH_Kz{3vgld{$V^;M&y7J(S=Si2$hSjc3mmwSD*<{=^lVB zas7K28+$xpw(jBha+Dl%b=AKn>r9G%WCWtDUpnwUleDmisA>r39g0C6h6V?kzUXUq z3z$rg@9`-!RIRG<2^g>EyjK=^rX@j7q9eztF!clZcIc|c(0bS0T_gImB7k@Y!YcdcV6_`WCdKa2a#{< zt~p1)=#0=hOa85(e@)~6uY39bJswg1kB&!+U2C*$l$a#47oK2oK9yGeGifaInPJyeg#;KBwIhjs;+^Kd!b-7@p+dHD1mFbF1J+auSDZW$ihLiZ9^oKFVx-Rb~ z?#UG0yG;Gr^NH#eh*YRGanY1w;h$MoCi%lhN>pglX=D5p0#yk(2>d3|rzB)pKGXWc z&cy6km+oydV_6+FZD0Kyn)IBt+Z_p`RvLP^{g=2l(jNfNCHo^*FtjVD&H!#aiRCQ^ z*BrKu#CoC%9$krje7OxP2bWuo(o12Y&YA$x!86Npt>0aHI|wWwl(|_TXdDlJ2OvSE zrcZ5T3M4`u99^IwwF03%3S^9y3~*`bgpX|h_DkbH-a;#<0qJTnc1 zVcyC>=+pi?TSvxf;R+g; zsLnBuzi__gbFJUe$g>oaq}~tiS=htq(2{@!>Uevig0C+;qDD!lr{tK=~2o#pt>KkbQRFv(!)KAmSk4 zg@6>u7fR=UP`;er?SwpErh1Y{*!O_bfRxUWq5k=6gL|Ei&0)on1jDP_$u($pPezDi z#Xt&c0Oq3F;Ze)d8xWY*I(48ivQG+HT~qt0kU#3EOK!EJIzt+n`B376#mL)n$w?=| zDifD>t9JL2JjY|<;^QJ)VVC8EUW+}LJHx6!DE?*Qix=|BjY*?uf_%UQ=q%2)+2g}p zH2vaQ(g+UNriH-%cu>K{WdZ+%sbXlY`P^CB%>OZ<>XfQDdd^i~6j>8=YHEVb; z$JWA15~6F$7@G=8YUE<&ME!l(V-Z3AcGsCfBhch&6ND*O!d z$wc@o-H+h|I%~h)I|;Fr4IVvc+Rv{u<}P1~h7wjS)@2qcL))|8qcLr0{~@KJcfSKn zbiT~M0{y|uX z(TmUU%~doTT^cnJMsxCOkJU2W zsR!3h?tG2?Y3i@8$?3B>ghqEAM@bL#PV(}%-fz--iCxc2P`wHRes3*UpL-p9uU90p zd45Sfmy<%i!S^0#{YrK1by*DSdCAn8EbWSP!65yY=5*gH*2@xOdKJ~}l7OIqv0GH0 zCZgkrLA}?K>D5?uWb0`%=;Wsg@A?i?>)rP!m_Nnr+cqE?vf}_&CgJYx55kx<>d-xJO<@SPVL90HdM9!pZd#) z9`wJp%HgV|4H}M|8{N3tECVra_{{z_q{1djDzv=~aMjWI zq#ri#{>#4L45YoOJ;`OiyW8H{s$XgQ{6~{StnaPycgU-; z3Ns|Lnw2e3c`{@DC?D$GSi!$=yRF@8lg>W0B6F`~dGb~WExF!72GHo`1Fm%M^bTkX ze**BTphYoFcWAixS;xjXtt~6c6N+KHvDv8nc7sWNld)sgqkRsmo3kq z)9w(uvSV(oI!v9rxh~&7+n=g)i+;VKwQP7{yydSoldT*lmaS$#KV0ARrqQWGmZcEW zQT*op8b^i+WSs6)#kWNQZ8zs7RL}!cYK+hDU}YpULCB#dgtF)y6ziqtaeJ9)-3 z9~%CUhcH@r_>dc2TG;dVp1rug_sB3W33En&P6MaHwFIt-yDF=58uIPA!;gt3-9@8m zq|2Pr<1OFxuvhMx8@dT8sr4?OW*qMAE;25pedd3V#A~j7uFlt-$uOTvqa!*Zx>{(K z=@p1uUz-!POx0A~T0X7oS_WtE9cjkHC~0fDgu%0~nlm(ssb6!NfLNzx&;Y`=ZwzO4 zrk{l0R@@!2I0iH3bxDYdIpPgBv8{+I-#0HG*zTGk23oZW-Xv7(#7(1%778nK7X@Ea zzs!S(cdj)EJ^}56Oz1qT=tC|Aw=0+BAm#r0QpMZ1ZVsw_<&71|iyg|a=EF|$K~)=s zy4nGIG1>)M=99tV%DSS;B|Z4{J7p)SEH~#;=#=nyF}m?sk0sq^9qzf86=Qn|A*^a> z?&hq%bj~x=zeZ&2PPvfXqu1lbrvQ8vaEThd3zL83MkI9a9(ie$EeVWzW<4$ZC&DeH z1LTh3qTsncM;!`rl$EX4t%g%NJ*I_WhJLh1bWA=UNp~M4BXUAZ-hX;O|CX6!d$t7u z`KvD<5{~Q1*L;k+x0r!`m-A_^DQH#e;vn*KQ0wclX}^ED;{SV2{@{>>Dv)u?zFXd$ zodrKiG7_cJ@u9?RUU??WY+-b-$fIXh86Zr@W^xHKD{!e6p+oA2ru6NLJ!O)u!gJ3m z17_eq&FJ0amZK)=;cE~ThihShb@#rH7!({RVDS4wpV_0TD=*7o`hh5wdf(l8q zsZz7gb%+RmgudbTI*C?z?QhGD`KGp(6EqFzB=AD6jBpBHW?E@oht$8jma$Tko3ZM{ zaecL`Ep`z|cS|RCYKSPp%k1++kLPp8n$o^{PTy$l>#Zfc**||@Hb$rI#W$2$ks`!B{WyMOVwLn31^%o8e-dVMs9P=?!)0O z!g_orF#XZosR<`Yh_OP8c(}2so~3$F6tR$b@HthZx0KLZrYpQa=9r(k6t_PRGhL)( ze4K_e;eMa{icb8fyY{|d$@k2o@R>>&{LI0muXyKv&SS1%RN8c!0y8HSkS>90syyNo z3U*g)h5EDnkSf(Gt=7F8$>0j zM7i>*V3Cb+3N1bF9W>k*IZ2NkL=|XDB6CVHLW;cY220*0?&D;`{WJwv*JC`SY0<04 zn+#s$#8&kd5Jf_tsYKZUdi zotF+xCt?iArX~`NC7lmM8bZZgJpkb6D|E6y6@D0gnkdgwHHx^|FGDkZk>IsgMKsKl{NDNlBgSL;Cr_ z=_dq(^oUE-Ti#>i=$dW#EiCS$mfbYx=88~5>NnUW9fNM>s@C%x1c{pMh-Bjbt$#n=?_UPBrHOsHDl$E>@-KC&CmHi6h=u5e zmtRzrkZoQw^55$K(kW);drXPshJD1HkR!*ly!r(IF=EAPq>6`>j6@wb5FJjT>$ zQQ9A!PeF68CHR?{SDt+TjuE9Bj#$U5Mbq5@R>9wUaO;WKAK}MVz#zt(cfP&5a>5(r zg&4p&UMv{3&F$OO6p^_OsfAn6J3w*mCEmjw5=_R2`(oF+oef*-8@^BmN9+gg;mcy~g4!yRuzrm~aj{2 zz7}VdyTdR}Wce(n(Ndn3$jWM>lAE1@ngj*Iz!+(wDkbu@U1*4tymL2rdLK8pZA9R$ zKVHH-s3O-Z06AA|C0Om7Eo7Ca#z_IVu{*oRu_;(-60Z6cu4raUFEu=1J;H8pdm+%< zmu45K);|jT-WOjN039iRj4L8fhPHP%Cx6ns?qiCsOxYTgseZEvIDBCjPA6U&&~mD` zWSD^RKIATj^Lq#LF{l!)t{@L-r+kd@;~H6INUsJ|p+=wW(`Zx_nu(BI8GT#w+nSSX zz@+o)7!6SPKwS|AF3%e||VSg5dxikvY=*|4j^Dj4} z0Rdgvr$cnHM;OQ6|9P3(I1Fu)%;}m4FZD&Xk7ZTB>AqBV_t2Uza|dPazuQ4c{Q&&;%t*Te zBfOdK`OjOScQQ1hN-~Wizut{os4U+jj$U|1kaJrumou_=a26E`vN?B>I+aDm&%0j7 zoq&1J#1UiR^>QSg8IlR-%hV@2L9D?g>yJeLM4cvsfHh^6R!KAc z5ULMtKS@b={_=(1{yV9x>%#m{v1Cu5XAad><-Svyw6t|drRyx&sfnBvJBN*NQU9Om znEy0`|M$Yg!5)jYuXn}nyzD)s!wi9cqH&jOAjV{^bo(bF%`Kkw9X&#etR~v@0O3=$ zI$|-l%kD~5!OSJWZXrH|hL``m);#fXZSmPtA}_P__H)LOH(Im{6^ODkDiDTR)lZZ^ zKYY;|B>a*~_b%0J+v4{@KDlVfm3+SWJ;T))^YpgrNgUnZ2@f4uMV_n_BD@DP_y~O` zQeH_>i^D^vig2jA{8*>HI8b!z5pH&;KP7hWCBZmW1#c1+^hk6i5J+UT*p139tAfXk zPetqYuBU{T?wzvqakkDN}dk-g;7OG-Tv?a$LCSrLAv6m=}XK&b# zJ;}(H6g_ht?>u7XFGL|;E;M%O55&|bNz0JD zoti9pS-X1%;_~x7*Hz@V+dD^(XFW%gJ>H_wkEa%MTH5i$Q@tS;rzx&dSzMj(m6=w^ zO+gd)EfRmP*Jr0qJ1?eAX@5G|C4MWnisJ;dNW(%tgOg@$)?^oS!xj#%>G811^Nd2a z#qSnB0u{Vj4F?f`)J;`Y5n8azXGeR0VSk^+@82!%GSzqvvZ9}?sDQRsJ@cX1n;|@V z8wzj3)rvjFViiQm){_xy=OV_}PqhW-Q5|%t=?QQ~%3s8I-Ci_M4F&SfS*M2#Z{}a$ zrD<4Quknk71>Z0FQbO%%JtM7r8C`Zdc!W-IIA3uuL(NjL>{526vOTT`Nfd&MuwPffQA1q(|hqKlYMOdE*BanenxqjEJ`IJsGAj`I^;IM5OJ6?V~ z)|`mte9ibW|Iq;ifWBi5M5BUjI{`tgt_t3`k<04T5UQqfp2aERLHma?FWWp5D!t-2 zvT80zOT<~noOrk_THzd!tIB0xE0>vKPh?(kNdmRv;(4$&pEE8cz}-`j7lv?)Y9gCC zp;yqECYn{F{qR4G3(m0gj^TThTh#25gGEtwh6zIFrkS2^R~y3;ub`+v@8@^A4h+&` zEGNSk^Up2sV0O|i-j4yo#qCq`NB-Wkd19Q#5d;6XxuaalPT^ZL058aJ{}KzN5eJ?u zV7Ju_^mbf*Myppp=YIO1b*cZb0uqny)OKCh1B-nCQQ+t-0C6-G5ZJ}@`vaygMCpBG zY@?Rj>oWjf#DH-o6oBB1aU8l~&({m`bNfUU!CpSt6wFdGT-Szj_I#W_(diA|*y+2`&O~LGiof0$9Pza3SlZeiXq-i9V@SO`yS}7+hG%zQHM$1km{FlyX zE6wq*4X-F>8`f@O6JxXMe%7Ciy5d0L97Q>W6q=G+84{5C zrVRc_zKQ8m)rf6F(bry(UZ38q%~tYE?7rBSQF`%lHwm+q91RvB5vsP-LnmAn#-?)| zlIMT;V)5gsaE3b{&%hb2A0wR?1=WeNd)gN>p#5nrZ=`O`BjS4sp=GHzi-sRrF@9X} z6J6bJBa4-NAUtqK8=Z(SMHmCAKY8f{nu~m*4)S!Fy^d$_AZlmJtj{SEi<43QWy&+c z6K$&H*u7|`j;yAR0J~<+{Tr{|S`mA$Wz@XvvAdWgJ3S7&4{eGMz81Qr4Nio%KHZMq z9fwjaZ0sPdLj?3uQ zmh!dfYxwqvk3;frg5HS&#xgy@$9!mEmZmRmV-?&#kC6B0xw588p}OZYir*na8sow#1%1buLAW7uJ8&7lNP1oiGNc zK?^Fj{Y6U)gy-uS;3M#ZE1yM{EOwN|+CDBesZ2qnQcFDOA*vLtUJ;$DAYSgB09KKO za{nYSj}pUr;U^}_*}EEo@8UbcI&xeDkl6*jy=Xy5CHkj)sNbyhyQNWbG6?i1UAZW{ z;Z+IROhe+th$#2WfZq{=Oc;2|pd|Zv>b;&1?)UNs^FjA(8Xi$C%Tx=j-_uqTN-SD~ zMQN{~GWiw+Zva8?;q}MZwqXxNo!hv~o|mq20D0eI5fqN(4kZTUE{7*=Wn@`dVNIxI^1^r@wv}WV5Dz`dYkq+4W^RrPKoO z>un9E)z{}Ruwf0Y_&8W$vV-imWu{NCTN)%?9{+AMtQ z?>(+t-f!w#K6Fw#XO55mOLp$R>2aK-hYv&hc-NjaO?;%E+l;=y9^parxT5>X_~KtT z*IGokHV;3_{H^0@TF=OMpV=pGTc?rf+1Ktq_lz8Iva(gEygb()vdQp;f>blRxm^C{ zom&f=ccPwYzFQws`rP+@Md_Jt{@hIC?SDYt*sSscB&}^rlBO`OdtTXd^>(bBkltF1 zr+JD(KToqTY36z!6A`PvzIUR1sIxt5(PWRa)gDT}Pp=sT9)Eon+XO@@vkbGPp0Lz- zH}4hf?Iiq(?se$~^yt&OY{qc zh0-@Xjf*CGorX59T%)8=?@^Hj5PcT+@paC}8sO|qk>hg~sur#9#OJsWip=F%7JCB9 zC4f}h8BTf9hsj$oiv_S`Kqm7@lq#RqkIYi9hA@LCYo_X*N)Js^ z)sV8<=n^2klY_s%_E!^0Px`l~E>?H*aSNX_4#=PD8I>`eWFwM}W@tJ^P!6Fo_@&q?{DZ>5ISOYBCc(RSmntCN9xN@hmfa7I4hX0x$V&PHZWs5wc+_a*TtA*~tn<_*!jQ^O12+WBKsXM0ymYx@-Q7Z^a)sl#HHXMbyJ zZ-HJY(fKe#(4n{7nEG<1zTCB`=Mhy+VWI>V=3HHa4&R}Xtp=hodCMCU=)fNNaOwz< zBfh;@_r-o zr?aQ|ESaL<{3CN&Dx(GPk*qVubVSw}SqWn~c`0a)vyXi=>0%aG7O8w0@#-ehM5FJ+ zE20+C%~NODuf%qzG?RW1QW(baJ1HbG)g%~?gv6qg0f^cf^flV@e6xIS`Rtjk+MR;U z(F{=BS_J`El6OgWo)?Ergga!P7fXpyJ9Hs_A^GR9_)DRPk5n^PzWUGvVB90sJ7G9%eD6YiM`tn z^Ttk*v8u8(pJil=mJ1tQq3!Z;#YwaOjj#T0?g*{IVZ~2&N%407P*1qi1_O&i% z4*uDjp$DZOo;-b8U|XF(XRA{BfX0OdbpQH!T^?pP^jWN~wC`?rNY@EY15rms!tJKI zyp+v_UQ}f=%V)XoWPl?Xsh00Ti0@W8&DInonXa0?&gG4_BBpiaYd*7}w)EVMp-gNvD(_XZl_!=fp`6G47lP z3M&?Z0#Oe-^^Kk2#$&`8D?NN_fx3d5H5Qex$>O@(?yC!P2NBHk2j#DHeP9!X)(5$?%&z5kN;H^H< z03hC7>Z$)7t|*F&;Q;XgC7}NWyu#)W#_qM{wl#o{lgtD)U;7F${i*PuM7ftIjEN+3 z>C9V3($Yj5cB|?!&|EpijSpgs0FEH&)!X;IfK!@t?U^d}9O|;}-QaFXTjmUvflD_8Hp#3q#n(`Cf2(dsk*$pu_90yyOgErJ34kyi)|% zBsyFkheO;%9(Dq+`E0l<$pPrwW~Y`(KLb3@EmCDF4Z_&F>I?v04Fd#9(6tgRp7ss1*fUaU_OpRmKDj)9>#B;wLsFOo zNZ2F=+6*Ag_K_lXYFhy0GiB?AC`9-($#B{T@P;*0;^7xa`vIW$(v|7@b2@b6QAKg2 zpfA%Oh|tIdhPbcBLWG2jAh==xrmT^&5@lWZh>4M`&+n%=tTzO2qoylQVcSgIPxFwaEfZq<}Ro!_2(c1i*r_BMOp9fy5c z05U+aY&rJ`$R<8kRaDID zOcm<&!p($T&%yG6aq_Tz0&jv`Ab^v^)r&O++3fjjtdvoG() zkO0ku%2ja9B`oAuL2a3G>nIV5Yxg=va|c;^6-vdHCCRH4_sd`qV=sXIfbMhd=UIOE zH?8q3U++4Ct77^?P9n=s*`(yU)?hx#=m870D!rQLEA5v~zdN+T!NJ{_M{yZ18i}xQ zFG%xNZe%RT>}K#QWxKL-t%Q>IUjqB;l+IJG?qvLHJVn9h|34zbIQ3hR@< z#K{s}*&uvNQEVG$Ew*Rgd2G`|!_Nkf-+}YzK7jR4{G4<5^qC|~(U(BZs&fSfmXAX^ z^R0#XWO;v_)svBPGZ_m2?qxiKEUzewphG7)E_rz?!`=oS#><=bMia;pUH7U98$0)Z zqcQ$_!{p#7aLj*x#UcQEL~Jg2zpGi?lfD4Xn%@tiQtmb+d~%?K5whsk z@4`2DR$W0cEJ6Q9`!!5ihUwyyNJ-jC$NGH~igZHDHN?zkJrE@9Wz*xO7vCb)?mUOy z>5mY-7OrAMck7mu7wDs6ZFBH~bVB$mqZMO2&G1o)p=hCD@7&o*pNWOZ-lY_#Ppy>9 z)v;Tfl5g-*8a+TEW?ULs{hoT&*E_DzEvGd|z zWS|prGN$V3*++cZ%%hbHk8OTH8t2*-Rr~08Ux)~_dQ)UXU6|XOC%Xo%VSYnbXpu6( zVi|yGcxk7&%WpnD_JC+M+#w~d$)eXRRu8x*m)4UE$HlIk4itOlAlE8?I6Grk{vEMs zAVNhoPMU-4KnN+J)(i7m3wsu$eXo#D=c8ZsM zh-5Uul&U1Av*aC#GORKa5q1H*54Wq$#O}j+n*s!NySTl9yO2qy9SFv!8iC0mZF_anFWWVzZpgL6w$C%psdMUzmL5O0&GLswy z6fX;6-^i?OC|{={6<*xfs~CHN4)tstT0wOKzkKFT9~M*lP?b7XSFUSMGDBoFht+H( ztrMclMOruh45I8lk5djemqyy*RW7n(_fLe=wXZVXNRczM*?Rjg;r;z^w-Rg!c)Wzw z5q+W)%*#`Tw}=LU_OeBDDiyi;{Y9;^f^@Z#xdb$j>oy8vZ-wao3j>8HXo-> zHZq<11(!LGY?U96{gu@zGtSbkO#ICcX3%g%`rx0~2<}sG4pcmRy7^-pCI`z z&~K=JtC$jN)h&O|vmf-*y-bleb~)#~YGvBf0X!Iq8uUt+qlS*1%#yd6eD250Gfs}_ z)!+OP$c;)DeX2D9KbSX3Z3{Ac8ksZu)&B=zqB&IX_Z~%{M{|&~o)r6&C>?0Hz>fgE zb)btBZ*j7|f{@bQpD2zKP^VxQX`;IRUQo&tQM^wr&WT@iF!SbS%&i%eP(8Pf|0&{) ziVJa5v#zRUNGa)s)?kUg!cF_A>F}-)mjaW+X@Cy{H$jFTJ=y4X!abo9vDJHZcjpf^r&v7D&r_R56s}5quv;zcLevlS37HN5Op6G^km_O^=dREi}84hi)z z2l}YsSU?g3P$Qk>uJWO~rJwZj4 zRcoJrk@KMsQGZre4QPwtP%HpdOeE<-ZXpq21?ba!S2nkbr)(AIep@b~LXOI5Fe5D* zCpj-cNW5`9sx!yq>*n<)1w`iQ#QHTkJ=x1A_$HO<@fCkP zdYYG4w>IRvXpsUII79<$(E5H01=1;DJW`OnwvQ$b7ZmoEC&wMriCC|mPg1p}XfBl* zK5N>pH0V@yg_5;IL?9Azd|c}>?X}^wcFDg)F8}Mw%KthO`4_z2|M-FbjGX-6wlKhD zaX*5&CjGj2;ERL&?id^{a7`xd7jAPY_re_#;F4hJmh9y32k*s75_xDr#K;4o%I5q5 ze60HPX1T(phn}Kxc2uP(+d4+ue)?k&(_A%wf06gQKGiDRujkzpaau89hh4$$E=?7H zz~Va2Mp{! zTMIh!ne$_7gM{-@nCr;YCjVdasg~ zHaqr!uz$^WAug$CB!B-BJ3Hy`J<@7Ise;Cet9-zD7RsMfm^SLbUz_0FKXm;`#~VpR zgkM6w#{|{J@UIo_vMB>J8AgKpxd1@6Xqc7hc$pVQcbY6sP;@Feg$$&F+^^SI7qffP zGcoAUP7PXeEoB`=&%MU_Y(FSLK4``a0<~6RC{ui=5_P_%Gr%I`?9}a^8(q-#=aq}K z!O4vzd8VkNLaZ*3Z4{2C_=z4+&A+aJAI6T(z%_*D3c(4|N#W^ssppuNxp-aGQLoNP zo$(=?@%6>XO{J9N^|Os8alta*v6p|e*Uem^8jR>c(3IRYQEl!Kp&9fQ6o zZ_!8)EA3PQ#z5DEUHRDYHt&PNGO{Z3B`#K1K%e=D0-P8oCaT(5SLpPX9g|B_n@YBG zhy9=3wcvnTrrsCGztFdm7~9HWPvxDlR=jiasbv-2+HjDZ2XRZwaV*r%o`!4>8iZF4 z`$fX>Y*cv@#X8RqGNsSFs2Odv)12kwVO{})rZorR_gOzmZ>Nq**55WLzw5m_SFXzR zpsa!%SO7EZbTV6RS92$ik$sfd?jEcEMg(ThAMJ#T{U&#n-g@9xdGiUok~0zRZOZ`- z#EGBB^~7e8k5=0-S%0k71_)lg3m;2ZOzC!_$|Id#>;mRuw5i?vfbco~(0ZV9TTcdc`!ykQnmYaR0)E>f&9g2qy)8D6k40yH$D1 zGp{u6szqCUc76)*U%ry!#A3j%#P zZ#{_EkD&f+;JvRfN(hlpshXfa%9*z(nRsDR3NC!*s>XBT_KM9#lx+>--+Op^2~6#UF4e$Y ztt76LNALWBvw|(x<`wR7JwBY6ee54x-Nt|vo=qv!rq&H(WcS-S?1shxt%DKcN1o> z;eEEXNSXp9RhZkasw%=Q#2ki8V#LToc6)u|BSFxItmc*J_A_{BT_XkeJv-glql3VP z%9afI;;YPO&h4czDy-w%ru8Gj(xyUW?INBIrpj}5{ieBs-hz^-LI zZH3G3oB|ZkGP|-z^S7-yEuwZA zjtC=OAbds|S+8eMsdE|laQLfPX4H?Gc^@lRKZ@%0QHi^;({%0cJvS((gQ0}uAX&2A zgGhs`e@^e8Qx{NDVP4e(Aq2VPN>3HzOh_Rv9Ovm??pWijl9=Z&}dkZgo4iiK=hOkz9rdU-jildX=NY=k)GagwR& z`D(+FFOz;1zy^<_sCJl$$t8E|W4%7BJ~5wcU9xVaZew>R=gtektalnfjCw!(@Ecs0 zA}m6<{k%(>p?93*D$H<&+K4rEquEih72G(*mG1;r`ok^wxduV+*DdSxhwt8Ead#$O zt02C6;Oh3*tAj$>KGbPP&y4{WB2UrVXc;0>HkktIOSaQgSA*^+&Ix|V5;gRQ!ItAz zha1B|Dy{crI@QV4b2jo~ZibK9RHlvlC9d#J#dE?v zRkMzXavPEFLOq!Hml7S@f8lmfL}J0cq8sHgbbfms-i_t=gR9N<8pWHkYG5*>KK}I#0ngCqkTY#^<7PDiQRjV{A=d9}VjyJ*l1s*7#zY zGyet?wt;z-_U97Pv!!8b$t7P6GKELzLc;R{TRNM8;etM|*y0{|4=JdUbhi95z^u3o zMAf|azl;TmuTE})C1#1zy`6e5Bg#+U^+g-k>?L>}NkctRHc8A&hL~AVlnU09**5Xz zRglq}%wIa{HgR2b904)XI?gRKaCe+s*u?VrJo}m)OGM?Pb*^fTg_|d+a~t*MXdt5{ zLkVNo$wpJ=FB%6E!&ji!GUgfeSYy`vFUo4rz-v8IY%b2#Ep^<1pFIn40{o9~G~nbR zWLv0oYVVGMB^BIA=e;C(U;Da}yHs^xKL|bgqCmuQH_@eqOvB!&@VS#h?_^H8%I)>; z@5#v|hicIy^MfwCeFaPGKYKaY6TKITeM6|8Iv@&G=7`mYhA&MSx6HLuORtm* zS~S470SToxI=Aw61xgxI#kVdyouFlh_~iW9mDP#G!(t_+*D7W2B(WFK&6rzx@Ay7^ z1gvHCiA|Zzqe|d3=90(b1EZXb-SZ>^(WifiUp{f$KQG5M^(PsXCjcM;yjXRrbKmg7 zs&3TZd-^b&*_nh+Z4tK5<<7jkv7R(yw(y?O>I&(Hy#sy^>u52A({KAecPpq8=!2!P zF2cwx@Zq~#TSS*eA_*>+sI{$5LzQ0btL1?JZ**VSqc>ZDH;EmZmet_7%p(UhWHK{N z8L24NbiH@&0)LkuoPsH7ZS;4w zmfw|MsB5MWZ#g8F9i50hEV|75P%oI~u?n`JvXrz_GbG&@!Dah6#W_A``eB(G_9VV> z?$V7va0iBp@&Lo~3FWu>hDVaj@*2_Ot@83%aDk;P)he2HFoE?UjOMa z81WbZR9(F*W@AGe-y?709C0>^2}w#PDab1Fu;cX*MuU(&Mn`@sY{^$Zm8!b+SN@}g zfsrnX3K!O2ms;Q)ltk72Jj>mo*^c#I`mNFD$z18Xah&VkV*g~WRdT#u`1~OcVpLy$ z|A3&_zD3}~2P9}3353m^Ys(v+kd>FSfZIwo4D>BJ6A>P4mIRRqF5I@dX|9ug@dcHm zZ>&@AlBKOdK^=D~379IAQG(!Xa%4v9*Jw*%)N$!WEb+b*0FwHPx5K(n~YyIs%aIE?|s;4Oir6Kh9o%*U_8fujW)O zubyv;A~Hv+&dYT*R_J&>$yQ_m3S92iwHm>f5Vw?W)1C-z>zw!3tc4j`+%N@XL@Yfa zzVG#8WCzS*OI_ZKW(5y`WU0>5VMaBkNN4k8Mp#j)`!}!pHLYzskrI#4JO3l`4NGH+l3Qt&6VqR$J1UAbLvo+Tyl^`?aAXizI!Y_uG!w!OSUMM8fDySLkj+P6SBwh#UOuTXa zu$bosF{V1^|G20%okx}^>pL3yMM`F?d7AKL?L^GwZm1P`i@fb8s@%Hvq^(UTH95U1 zopcN4O3=&DA(vla6n*&cUgWc6uegwUVlMJ5e(e2|>Th1K&RVl?`mf-p`!F~yrz{(r z)WGrd#SiJJV?@%)Wur+VzDx4{k>X4GETr>D?3CRP&c z#_UzGW6hoRa7aS>3pLVaddrlb3U>PPJnY7N%F)^G_<@#J^4CUcD=o{Q5x3^BzYlFd zh4-3Em3vCv`=x9Lx7cG_vS>uEcOUXXW?o)aqC;6HC_@SBF>I()T?oVEK^7Smh-wH$ z&nPC**7;td=D>6d5AKiaGnWJ4)%yY+wr)?5!b1Ff0Ajh& z`a#I{&%}#8-e$#a;|%ao(#6SL&}(8G$3K}F##yuLCGfx?(H+NWokugb!k>9u`Lca* zIw~g3t!T11&Na#ZzTY49Lz++Qey~Ei)$lzVoC9Y{&exP`P0|kRFpCqYeV!vV-&+Fp zz9trOHb8lBHMJDB%cF)CGlEkD4^Tz517y7w`x;_|n&VWd&blb5d{+LNphQ&zEYT@T zjJ6T|$yuK*dc#rqPTXb}%{5kwink)0g!!M!Yeh?lxc%<9{-1>rh4B&EPg-{59iMfL zYd~WdZ`)F1f@kAOU7lrOQO7z4*H4kldSFZe1&MtR}|3XWaLvVMD>YqKY2aekmhve#|qVW7M8KpAESZV zDH2~4+H}uLGWDy)?gIIqOoEfBxt5PTu}1pG#3(DiJo#Ho;q$qp(_gU3n8GBIzVN^V z0^~vs;;g4YDakraR6Q}LpXGhBy-Y!4D)&{r<*^Y%M>eR-sIdfcJpl>uNki065F&TJgS}o4!+Y-QbStQ@ZI`}+QVx7+Xc`{R4NegFLa z@%?N6?T_t!c)lKw$MgR1AIzHi=^uB;6ks=v3CA)=+S{J5(zh)CIU~$xuV?SKyG#E( zb!Ifuj&xGE<*vysI%O+>5sjGTbz?C5OfyycCwB;l1=F(}CN5I=#G7Dq&sm3&%V+mM zRu<323~0iThE5Eyk3F>T%A858odbGPZ%}>CZXo)e3d&9+P3zo+P1Efob0r(SbC9Li|;OfzgSq* z!)+i$scZxuE%5q`BlDd%oCUX7-bx-=WuGu*JK+K##*G5~i4Z3f1RLu1sB(vTfNMGm zN$HiqK2PU+iWzXncW*aHBN1G$rIAiP6HkRk`N%F7d?dq1?`_SI*Z2A zT8~J59P(VnL7SPo3fqR_c=8&_a<*8J$5Eq6)@yzru4vj1B zTbA3n)}2Dvc3kmcqrtSzwtBh7nGluj^C3q8;;L_B%F5ZDLg1a%`}H@e=%4H+$@ncP zkiJ+1YJ1)&h;%UT_^oYt+bfJ75L1*_=`#L-^P-@uc*_^;gHz5|GWf7na8z-Zp?!<> zF|M~kd2-aHUUEK5+^{gjPNruua8YW6UYRrVdQ3i*faVHPydE~aef0Mp!V81k&`RN= zgQhJp@CG@g>MaZR;4BeQ|4a#fBVr1mx@{|7Kp zXksT=kz-Re5#T8krxDmD2P<^t#uO*)Di9Nq!zAt5!_M5I(|NX%Y~BIW6LfsvUOK={ zw1H*zpN1PI$21)nK@+bBsU_5xEwj9G@8J{|Z0qTN7*Nkg_tJduXkN3@X<5u4bX~s# zBQDM8P?xNWIwsr^X`v~)XLizVI0;jG*sdAcw)!$OgQG@wwbA=cV!Vdti09g7! zY1LBRU|CL)XJPn(YkK!pP-dsaoqrn3XBpP=b_g~? z%s4Pm2=6WPIGsZwjHa3{2n{%GhP+9wUesS=BI5D&v#OEv>zq5s3kmCrfN{g8x^zWp z4Ln(Bs|z8==>bIdwuYiS3AH{m{s{53h>=RtFZVA}mdG(?Eo447>yY+IEvk8d5Xv3^ zv?DzAL2>m|AgJgpXE=Bn6?FNg0H&o9`1<(9Y{eJHpMehh4|H;so&+GiX=e^;~*=G`vv$TSwS`pBCUZPG1 znm*YiGf{Dxxw5z>YdzByy!>W8L=~V6u4UA>{X`r@Jwjq?IW0hDrookLD^MlE4xIOC zOh%nX9%Z1ip^=0gpQ5ZC0M`Pbc$!8ED5s!{;ZbU*LE1h1;nI01n!=DJH{jIUZL99} zvb}syc!{8Y*3YY)*G6Z}L6r4w=P>-$Hj~=C!SA!%^`~r}k}kb;ie(T_+eE{%)k)VR zFy4#%;xXP2&unwBG~#-*L@x^#NIu;WCnxbBGYCas@di~lm&}MhcGKG1XzSI4Jz0p0 z9DAjCH?6~a&IBI?^$rbXJ08ZfZ+6iUquS_B=M3siKn@-L^fB=PP_PpoPQf*iZYx=t z&uMvm-Rt0^HCzgwM3LitW%T@%n;K$BwyLApj9*ngkWB4;~=^|NhxVX5d&1qhk^Qtgxn6xlc{l`)ECS0}8VY zJH|v7B7(CP+J~u!?wlBLa!UR4ua@jFW`nBcIwO@1t!unY^L0WPGEK0jO}fem%zkXL z+ev@^r(i<0r|A_j3Ka|#h72mXqdjGls272J5kI;`XG3u(&nQGr%HVEz%B^1<;rIfK z-z#J1*X7eam7?O0#`}_t1Z5IXM}aPw*W3uaF3`e?ZJDY!V^c@WRY$))F}3&=GjgUe z9jjx9KB~5SPzG)`hX)Ul@2goC?u|*3U05zO=N}e>@Ji)px?^ zcwsN=`KrO9oeHm;Y8x$d0iiJRwUMmTvQej2v?~TslvfVM8#pPrj3|javk<2&ST@Bi zlmicC`yh$J>~%aiBgCj!?y68nH?Uj%h$jfp!DDA(;*U21?wUg+Md{T`W8w*XUwp-+ z+qkRa(UOh6Tsh^`bSfe|31VaVyPH0;@PsBJuQ4m1xyJOZv?e614MYv0||Z0o*43 zfvVk|3#LkX9Sn(Q-cH%AEo-7dbfh8UaeVDI!`RvpG~+J7lvFgq1F0h}O&#o9^KfQn z!`gPhfWul}to6e)rF1(u5F0S}!z;tFsw4TzGPW+)K2uvVJBAvn=H5{494^w|IBw~X zet9V^H#c;cQ7-N-7Uzr;-!p(DTq4h2_=FVYRa%p}Oo|N?@4!aIdj}sZf5SMyC%Ph} zw;hjl{Bq;*IxJVY3{ko}81K94xpZ;MV{oZ>EjZgSr$l0$*BIGSz*>QR{ThqUAh0$B z8%68X{f&2MqLdtggr$2vw3llT9kzDq{!A#oR+;EYQt1y1`%uwGk#)09& z12%AU-=FWVKdT1VY-FhNAFy4B3Z`NtAs@i@+kRo5{Wup&dj?n}Y^U2w5m+EBwj^t5 z`i!pQmFI7bqU#Ss2f&I%9UdN(5ghdE&?=-ZSFnX|PLRm<$DZu(bh?)3x>HigzWI_r zM^UfVr$F{|ooK)EH9^WYcmgi)3Eo4ZhPl4hp(UKB?CeOnIFWR@*lNV64J6&VF7Cu1YK&;wR0lZ#F^iLp~;p=zLNb}GI*8)%~0+nb+KtNnI8J_CejuVw3bfmE_s;k&1r z1?zTx4|YO+tvB|Kd{>oo33ZNXhNvAc#aR8P04+mhybpwGj42isJXTU6WqV_z%#&Kf@87Q?OcC z<_QF-WzN{72e-5QeNC(R;a`xnuwW&qx1nhiDfnVNw7n}*lwPD!!bTP(LHd1_l)3uA z+7D>AH~uG1$jR=F>HBjDKjPKhx36bQV|&_7l2rV@3Ja;`H1Ow zG_8R^S&XpcXF;`?RFRW8BB)!!3hp+c!+DOBU6SX}~Z>M!GVVv9P^R z7!*a(C_OM#2%syN12-t$lSvEj5rr=BC@&{yEmP(azkU=Me7n+0$>;3LXSgFwIXmvL z2O$aZe&7UPdw8tfLYyYmn7&>tTxE*@)56Co&qX4~WOdJi({jIzOGTY><1%b z9YEZ47$sr_9Bg#eBKp+A7iky~yh>5%pq?h-;y{+h&KvZ{Bt;)&K3QIE z_b<=Atq|h)NcWV9c%WuhmoU86(FhEccUw)R2}OHb`KG5M8(UTQu;LWlE;Lct^(wy9 zqkHchb?Uj$bt1XBp|&a7!&OH!LEG4>lI_rdg5_!`#BBkVb20>l6fd?kq1fI(~A?QwhZfE2YpRc4=|6vU&Z_x8a|tre=t>@quJ* z;D`({mVtmyLQvr>+gFdU{+Lo6ZT36(MTZlRijUTMk6~LS6{=t>wXw;XJtwrLG%uI! zb{0QBn?AFqKs??dDE78EJuxaPeT)&;&QFTRODk&WhHQ0#mL8LJIlQ$SJtF`c8;#V^?Sa`mGf zd?qe}^4TNn1GD!=Zm1Ijnv->_6fFmomiK`yz3!D&R>9zEm1Gs!szP5R^^z?t2n99F zg2-PRj`y2R%-7@U+RxY&-#2<0rx+L$*Tx8(QW(|bH&kp}f-|buVDbtc#6+ARq^4md z0zcMp(rObuO0ujZj@ODSag#csDcxJlBImv}~kCK^oh@^iBe1Ms~4 z9P|95T7|#H>AdAJV*`Zc7x)8hy}(0UTFbj}qvu%9Eh{SIcEvD{;}AW{u(47fF0Fi8 zsfE->(V$$TPh}#R`|)Zc%yW(J@7YF`N_#M*V_qUJY0ut6s&b0>x>3K`a* zjIN=W*vD1B3(*Q#&B)1{!$~yX{{3IZWp(Vr1=DpQuiY>6ra+Q`?3oqEj;S-^2M=fN z$v8HNyR%z0q_;b;!*MDTHoh~LElzyK%VZi&9-KWU#rN*7s((qkTP}Uzzx?ozxtu_+ zIb25BD1QBAL6HXgy3OzF3{x9|2bUf1Jqsu*6&1r#UdoQSipwK(yAEwGOnZb5Ssh}i z7=5@m{;*Vbfa4$Qa%nUY>j!V`=@|)U;v@(*))^?ABuWi~=u-PD*{_>p1I+iur^yS0 z%wtz>f10=s3@nV0YCLgA<#M5|SB1^2JfX*QcoQVzLtKA(@m_|S#T=m;+d3f& zURTw&FBa|7@38UuVXW?^ej>sh6W8Mrl26*I9K}r9=-rzYnDh>{2_99CVpl(}rPr*o ztf!@(=S)an5jz=ee(q|-v#KEnrCgxBGFC6OTwHyhP4bZr&4Oj$_XrOMZC@4ciU*&#BU_~@+uGSf4302P<*>_IP%TPMY3I2qrim5`U)=+FxJjiHiwr& z7x?oH(IYepi-o)ESlGl+j~+={fS=b!-9^&O-ZYt5#qlb_cSZRJD#7i7up3zfq+zGD zVM~={CS8#5TF-8yUI40%3rQ(k2q$_vO2Fi8&FIKRbp!skU5zz+nrP7(b13eVI6e(f z3(Smh;s-Apf0&mDu5AjO_a8+=F$*KP@UodoN5wD6a*=d9bRWF{!<6Ow^&>ch(QaWS z8msMKDJ;(?l%ksJm=;W1!_XAa77i5_3^ObDIs`1!Pzc;nZ3Ok4+*t*R<^fn*ro$z- zZ)t75fA2}SSbTIw$_6y7t&Udx^lt}(Rk=lmlCdO?ew7>1)h^poJUN{X$X_+b7R-N^ zIDASku?#E=7>he27O=uX%HJ&r0`(r2zk^g~M4)S&dInt5X9?Ji?C~ya*uRDH1X)ny z{kTqEBfd(JfQ;u*3hl4E)qbQqSm~2xBP)hKB50+yNiQpgJ|)wk7*Fv+Pw$b;Q1a6` zg||i)De5xfv5w4;r2KtpZB7)=dO-f%SUtxqE?GJlC15SQz(1MAEF6TQI5ILF_b`y{=E=xB;0%X>lSAK8Pw~H z45#CXX*aRVj2bd9HxgYM zN%A>I(-6H`74@e8I8B0{%z6z=%f0m)in#1Xy#zy|I{|K5ZwO7q@!C4H+ZSi*sM$Gv z=*&N^&D{23Z(EKT#UFq7S^eCIF3G*DPXyeR0F3P0pn?s{ z<#S9qBkE%6K3dIOb)ouvn&-nR%0@Xie=%228SKn$Cbu%wnNOGS0b)jKaAe`tjB9 z9qkC=g4bDw@_^C>{)KumusVDUl_XSyX`wc$(@zv4 z0BY<3Nl2z%l!XG3#Ty?>Cn3j?PoDME1Ojn~tcJEDfiby^i`3G9+(+(ii^oiR{1o2c zQLEz-=GR7W7$N*o)@kJ~{X!faXLqK72^lo5G5WA*5f?%EI_I`p%Bh4$E|~P9D1~rj zDSh9>UCd5d>B0A1YV~6;_n#IlBL0J9^8Z%*J>CV2|8K7ke|HOZP&EkI`-!`9aKin{ z5TcT&w(qm#O_@=6SNrc7Hm6D-V)-dI{D0nOrpLRX!IRH8SDtaDsTFm{^x1wb;rN=^h5T5y(D*c4#gPnDga@iuXex5 zyd$~)y~kP|SoKNj1T_EQ8$WG;LGf?9OXG$57Ju(CqU^c%KVR`bzr}x->*AN(7|8U* zEPuTay@gsFHu3p0?P^vxGyhj)LK$;OGwtxvPtFV9>&Q=B~{Qs80A|NBbB)>Q#zd*rQ&w#d!pzDFvWBr1D9^wm$SS00=*T7bFC}Pw& zaUqAY)5e3MK^H%$7$+4qadL@?OGrwos;O&eYMGdtnOj&|IlH*JxqEne1&4%&g-1k2 zC8wmOrDtSj6_=Ejl~+_&HMg|3wRd!OO`1Gq>a^)IX3ko)c*)Xb%U7&iwQ2K~t=qQm z*tzS_;UhlG{sQ?67@iPs z0ZE9@(EJ4q1V$zn7G@T9kiQt2%0V6%WMNe_WD{}>WKS#c@69{;yl(wme1fGL-^}9gEKH;Q3n144WY~gpzi;7fyt8P|1Aa{ zW=3FQVishuXZZIe{@_3UUT@EbXBkCG(=Y3Ux?hKgxD&{}~pjx7B~( z_WRH9xP+npknw~3UrZJ0{}~vzy#HtJAng87p51Hrq7U!epTs;asmkqMS8Jr0IRE(P z#~JMZFZ}yr<97C6$4qPgIY-i)+@~=r?_;!h-ZO9K0{*l6Z+-s9<^S90-(h(XJJvsz zf0ulnTYcbH`th^&$F9f8{+KK`;n)7A8pHAjv)^+^+__b8`$O)ZMz; zS3Br(&a`t+UVVyi2zuyXck@GgkB#a_=EGLzy*2EQ`uTrcKX^}hb9dkco6v{8$^O^> z*shH@vvBdQ%ci}0k6s=7B*ekGaPBwrcTc!p9cr)@KfLCC=3Jw)8RepHcUU#lEijbt zWvKsu@!#h?p5Ncv-<|#QsDu3J{6FIF|8f5O9awjNzr=rrB!9NLb8*bEMduIill$>t zpVh~A`&S%Z@W=i~ZSImORw@4%w70- Date: Thu, 10 Jan 2019 12:34:44 -0700 Subject: [PATCH 086/186] Link ILM management and policy information in ILM API documentation (#37324) Previously these were only linked in a circuitous way rather than being available from the top level API documentation and "Put Lifecycle" API docs. This makes them slightly easier to find for a user. --- docs/reference/ilm/apis/ilm-api.asciidoc | 4 +++- docs/reference/ilm/apis/put-lifecycle.asciidoc | 3 ++- 2 files changed, 5 insertions(+), 2 deletions(-) diff --git a/docs/reference/ilm/apis/ilm-api.asciidoc b/docs/reference/ilm/apis/ilm-api.asciidoc index dcc3d1962cb64..457ca3daf5d74 100644 --- a/docs/reference/ilm/apis/ilm-api.asciidoc +++ b/docs/reference/ilm/apis/ilm-api.asciidoc @@ -3,7 +3,9 @@ beta[] -You can use the following APIs to manage policies on indices. +You can use the following APIs to manage policies on indices. See +<> for more information +about Index Lifecycle Management. [float] [[ilm-api-policy-endpoint]] diff --git a/docs/reference/ilm/apis/put-lifecycle.asciidoc b/docs/reference/ilm/apis/put-lifecycle.asciidoc index 83facc6f42b01..21d48fd8cbddf 100644 --- a/docs/reference/ilm/apis/put-lifecycle.asciidoc +++ b/docs/reference/ilm/apis/put-lifecycle.asciidoc @@ -8,7 +8,8 @@ beta[] -Creates or updates lifecycle policy. +Creates or updates lifecycle policy. See <> +for definitions of policy components. ==== Request From 83f7423cd65c661eeca46fc2f9dd874529ccff53 Mon Sep 17 00:00:00 2001 From: Costin Leau Date: Thu, 10 Jan 2019 22:18:53 +0200 Subject: [PATCH 087/186] SQL: Fix bug regarding alias fields with dots (#37279) Field of types aliases that have dots in name are returned without a hierarchy by field_caps, as oppose to the mapping api or field with concrete types, which in turn breaks IndexResolver. This commit fixes this by creating the backing hierarchy similar to the mapping api. Close #37224 --- .../xpack/sql/qa/jdbc/CsvSpecTestCase.java | 1 + .../xpack/sql/qa/jdbc/DataLoader.java | 4 + .../sql/qa/src/main/resources/alias.csv-spec | 10 +- .../qa/src/main/resources/command.csv-spec | 8 +- .../src/main/resources/field-alias.csv-spec | 129 ++++++++++++++++++ .../sql/analysis/index/IndexResolver.java | 17 ++- .../xpack/sql/plan/logical/EsRelation.java | 2 +- .../sql/plan/logical/command/ShowColumns.java | 3 +- .../elasticsearch/xpack/sql/type/EsField.java | 2 +- .../analyzer/VerifierErrorMessagesTests.java | 27 +++- 10 files changed, 190 insertions(+), 13 deletions(-) create mode 100644 x-pack/plugin/sql/qa/src/main/resources/field-alias.csv-spec diff --git a/x-pack/plugin/sql/qa/src/main/java/org/elasticsearch/xpack/sql/qa/jdbc/CsvSpecTestCase.java b/x-pack/plugin/sql/qa/src/main/java/org/elasticsearch/xpack/sql/qa/jdbc/CsvSpecTestCase.java index abf56cee9c766..d8b6375e7ca96 100644 --- a/x-pack/plugin/sql/qa/src/main/java/org/elasticsearch/xpack/sql/qa/jdbc/CsvSpecTestCase.java +++ b/x-pack/plugin/sql/qa/src/main/java/org/elasticsearch/xpack/sql/qa/jdbc/CsvSpecTestCase.java @@ -42,6 +42,7 @@ public static List readScriptSpec() throws Exception { tests.addAll(readScriptSpec("/nested.csv-spec", parser)); tests.addAll(readScriptSpec("/functions.csv-spec", parser)); tests.addAll(readScriptSpec("/math.csv-spec", parser)); + tests.addAll(readScriptSpec("/field-alias.csv-spec", parser)); return tests; } diff --git a/x-pack/plugin/sql/qa/src/main/java/org/elasticsearch/xpack/sql/qa/jdbc/DataLoader.java b/x-pack/plugin/sql/qa/src/main/java/org/elasticsearch/xpack/sql/qa/jdbc/DataLoader.java index 53669f9de0eb9..4985dda404dd7 100644 --- a/x-pack/plugin/sql/qa/src/main/java/org/elasticsearch/xpack/sql/qa/jdbc/DataLoader.java +++ b/x-pack/plugin/sql/qa/src/main/java/org/elasticsearch/xpack/sql/qa/jdbc/DataLoader.java @@ -105,6 +105,10 @@ private static void loadEmpDatasetIntoEs(RestClient client, String index, String if (extraFields) { createIndex.startObject("extra_gender").field("type", "keyword").endObject(); + createIndex.startObject("extra.info.gender") + .field("type", "alias") + .field("path", "gender") + .endObject(); } createIndex.startObject("birth_date").field("type", "date").endObject(); diff --git a/x-pack/plugin/sql/qa/src/main/resources/alias.csv-spec b/x-pack/plugin/sql/qa/src/main/resources/alias.csv-spec index e87aaecf6f332..fe8e6e5da4e63 100644 --- a/x-pack/plugin/sql/qa/src/main/resources/alias.csv-spec +++ b/x-pack/plugin/sql/qa/src/main/resources/alias.csv-spec @@ -36,6 +36,9 @@ dep.dep_name.keyword|VARCHAR |KEYWORD dep.from_date |TIMESTAMP |DATE dep.to_date |TIMESTAMP |DATE emp_no |INTEGER |INTEGER +extra |STRUCT |OBJECT +extra.info |STRUCT |OBJECT +extra.info.gender |VARCHAR |KEYWORD extra_gender |VARCHAR |KEYWORD extra_no |INTEGER |INTEGER first_name |VARCHAR |TEXT @@ -45,7 +48,7 @@ hire_date |TIMESTAMP |DATE languages |TINYINT |BYTE last_name |VARCHAR |TEXT last_name.keyword |VARCHAR |KEYWORD -salary |INTEGER |INTEGER +salary |INTEGER |INTEGER ; describePattern @@ -61,6 +64,9 @@ dep.dep_name.keyword|VARCHAR |KEYWORD dep.from_date |TIMESTAMP |DATE dep.to_date |TIMESTAMP |DATE emp_no |INTEGER |INTEGER +extra |STRUCT |OBJECT +extra.info |STRUCT |OBJECT +extra.info.gender |VARCHAR |KEYWORD extra_gender |VARCHAR |KEYWORD extra_no |INTEGER |INTEGER first_name |VARCHAR |TEXT @@ -70,7 +76,7 @@ hire_date |TIMESTAMP |DATE languages |TINYINT |BYTE last_name |VARCHAR |TEXT last_name.keyword |VARCHAR |KEYWORD -salary |INTEGER |INTEGER +salary |INTEGER |INTEGER ; showAlias diff --git a/x-pack/plugin/sql/qa/src/main/resources/command.csv-spec b/x-pack/plugin/sql/qa/src/main/resources/command.csv-spec index 7c9c98f6d0446..c52a5f807bde1 100644 --- a/x-pack/plugin/sql/qa/src/main/resources/command.csv-spec +++ b/x-pack/plugin/sql/qa/src/main/resources/command.csv-spec @@ -236,6 +236,9 @@ dep.dep_name.keyword|VARCHAR |KEYWORD dep.from_date |TIMESTAMP |DATE dep.to_date |TIMESTAMP |DATE emp_no |INTEGER |INTEGER +extra |STRUCT |OBJECT +extra.info |STRUCT |OBJECT +extra.info.gender |VARCHAR |KEYWORD extra_gender |VARCHAR |KEYWORD extra_no |INTEGER |INTEGER first_name |VARCHAR |TEXT @@ -261,6 +264,9 @@ dep.dep_name.keyword|VARCHAR |KEYWORD dep.from_date |TIMESTAMP |DATE dep.to_date |TIMESTAMP |DATE emp_no |INTEGER |INTEGER +extra |STRUCT |OBJECT +extra.info |STRUCT |OBJECT +extra.info.gender |VARCHAR |KEYWORD extra_gender |VARCHAR |KEYWORD extra_no |INTEGER |INTEGER first_name |VARCHAR |TEXT @@ -270,7 +276,7 @@ hire_date |TIMESTAMP |DATE languages |TINYINT |BYTE last_name |VARCHAR |TEXT last_name.keyword |VARCHAR |KEYWORD -salary |INTEGER |INTEGER +salary |INTEGER |INTEGER ; describeSimpleIdentifier diff --git a/x-pack/plugin/sql/qa/src/main/resources/field-alias.csv-spec b/x-pack/plugin/sql/qa/src/main/resources/field-alias.csv-spec new file mode 100644 index 0000000000000..977c0e8309567 --- /dev/null +++ b/x-pack/plugin/sql/qa/src/main/resources/field-alias.csv-spec @@ -0,0 +1,129 @@ +// +// Tests testing field alias (introduced in ES 6.4) +// + +// filtering + +filterEquals +SELECT extra.info.gender gender FROM "test_emp_copy" WHERE gender = 'M' LIMIT 5; + + gender +--------------- +M +M +M +M +M + +; + +filterNotEquals +SELECT extra.info.gender gender FROM "test_emp_copy" WHERE gender <> 'M' ORDER BY gender LIMIT 5; + + gender +--------------- +F +F +F +F +F +; + +aggWithNullFilter +SELECT COUNT(*) count FROM test_emp_copy WHERE extra.info.gender IS NULL; + + count:l +--------------- +10 +; + +functionOverAlias +SELECT BIT_LENGTH(extra.info.gender) bit FROM test_emp_copy ORDER BY extra.info.gender LIMIT 1; + + bit +--------------- +8 +; + + +singlePercentileWithoutComma +SELECT extra.info.gender AS gender, PERCENTILE(emp_no, 97) p1 FROM test_emp_copy GROUP BY extra.info.gender; + +gender:s | p1:d +null |10019.0 +F |10099.51 +M |10095.789999999999 +; + +singlePercentileWithComma +SELECT extra.info.gender AS gender, PERCENTILE(emp_no, 97.76) p1 FROM test_emp_copy GROUP BY extra.info.gender; + +gender:s | p1:d +null |10019.0 +F |10099.7608 +M |10096.2232 +; + +multiplePercentilesOneWithCommaOneWithout +SELECT extra.info.gender AS gender, PERCENTILE(emp_no, 92.45) p1, PERCENTILE(emp_no, 91) p2 FROM test_emp_copy GROUP BY extra.info.gender; + +gender:s | p1:d | p2:d +null |10018.745 |10018.599999999999 +F |10098.0085 |10096.119999999999 +M |10091.393 |10090.37 +; + +multiplePercentilesWithoutComma +SELECT extra.info.gender AS gender, PERCENTILE(emp_no, 91) p1, PERCENTILE(emp_no, 89) p2 FROM test_emp_copy GROUP BY extra.info.gender; + +gender:s | p1:d | p2:d +null |10018.599999999999 |10018.4 +F |10096.119999999999 |10093.74 +M |10090.37 |10086.92 +; + +multiplePercentilesWithComma +SELECT extra.info.gender AS gender, PERCENTILE(emp_no, 85.7) p1, PERCENTILE(emp_no, 94.3) p2 FROM test_emp_copy GROUP BY extra.info.gender; + +gender:s | p1:d | p2:d +null |10018.070000000002 |10018.929999999998 +F |10091.343 |10098.619 +M |10084.349 |10093.502 +; + +percentileRank +SELECT extra.info.gender AS gender, PERCENTILE_RANK(emp_no, 10025) rank FROM test_emp_copy GROUP BY extra.info.gender; + +gender:s | rank:d +null |100.0 +F |17.424242424242426 +M |15.350877192982457 +; + +multiplePercentileRanks +SELECT extra.info.gender AS gender, PERCENTILE_RANK(emp_no, 10030.0) rank1, PERCENTILE_RANK(emp_no, 10025) rank2 FROM test_emp_copy GROUP BY extra.info.gender; + +gender:s | rank1:d | rank2:d +null |100.0 |100.0 +F |21.445221445221442 |17.424242424242426 +M |21.929824561403507 |15.350877192982457 +; + +multiplePercentilesAndPercentileRank +SELECT extra.info.gender AS gender, PERCENTILE(emp_no, 97.76) p1, PERCENTILE(emp_no, 93.3) p2, PERCENTILE_RANK(emp_no, 10025) rank FROM test_emp_copy GROUP BY extra.info.gender; + +gender:s | p1:d | p2:d | rank:d +null |10019.0 |10018.83 |100.0 +F |10099.7608 |10098.289 |17.424242424242426 +M |10096.2232 |10092.362 |15.350877192982457 +; + +kurtosisAndSkewnessGroup +SELECT extra.info.gender AS gender, KURTOSIS(salary) k, SKEWNESS(salary) s FROM test_emp_copy GROUP BY extra.info.gender; + +gender:s | k:d | s:d + +null |2.2215791166941923 |-0.03373126000214023 +F |1.7873117044424276 |0.05504995122217512 +M |2.280646181070106 |0.44302407229580243 +; diff --git a/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/analysis/index/IndexResolver.java b/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/analysis/index/IndexResolver.java index 52e09ae31c727..b3fdb4d1170fd 100644 --- a/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/analysis/index/IndexResolver.java +++ b/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/analysis/index/IndexResolver.java @@ -26,7 +26,6 @@ import org.elasticsearch.common.Strings; import org.elasticsearch.common.collect.ImmutableOpenMap; import org.elasticsearch.index.IndexNotFoundException; -import org.elasticsearch.xpack.sql.SqlIllegalArgumentException; import org.elasticsearch.xpack.sql.type.DataType; import org.elasticsearch.xpack.sql.type.DateEsField; import org.elasticsearch.xpack.sql.type.EsField; @@ -351,12 +350,18 @@ private static EsField createField(String fieldName, Map map = globalCaps.get(parentName); + Function fieldFunction; + + // lack of parent implies the field is an alias if (map == null) { - throw new SqlIllegalArgumentException("Cannot find field {}; this is likely a bug", parentName); + // as such, create the field manually + fieldFunction = s -> createField(s, DataType.OBJECT.name(), new TreeMap<>(), false); + } else { + FieldCapabilities parentCap = map.values().iterator().next(); + fieldFunction = s -> createField(s, parentCap.getType(), new TreeMap<>(), parentCap.isAggregatable()); } - FieldCapabilities parentCap = map.values().iterator().next(); - parent = createField(parentName, globalCaps, hierarchicalMapping, flattedMapping, - s -> createField(s, parentCap.getType(), new TreeMap<>(), parentCap.isAggregatable())); + + parent = createField(parentName, globalCaps, hierarchicalMapping, flattedMapping, fieldFunction); } parentProps = parent.getProperties(); } @@ -368,7 +373,7 @@ private static EsField createField(String fieldName, Map props, boolean isAggregateable) { DataType esType = DataType.fromTypeName(typeName); switch (esType) { diff --git a/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/plan/logical/EsRelation.java b/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/plan/logical/EsRelation.java index 83f9fed31e8c7..a90fb751c5e70 100644 --- a/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/plan/logical/EsRelation.java +++ b/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/plan/logical/EsRelation.java @@ -8,8 +8,8 @@ import org.elasticsearch.xpack.sql.analysis.index.EsIndex; import org.elasticsearch.xpack.sql.expression.Attribute; import org.elasticsearch.xpack.sql.expression.FieldAttribute; -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.EsField; import java.util.ArrayList; diff --git a/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/plan/logical/command/ShowColumns.java b/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/plan/logical/command/ShowColumns.java index eea82a0d2ae9c..8325d3a01e754 100644 --- a/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/plan/logical/command/ShowColumns.java +++ b/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/plan/logical/command/ShowColumns.java @@ -12,8 +12,8 @@ import org.elasticsearch.xpack.sql.session.Rows; import org.elasticsearch.xpack.sql.session.SchemaRowSet; import org.elasticsearch.xpack.sql.session.SqlSession; -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.EsField; import org.elasticsearch.xpack.sql.type.KeywordEsField; @@ -80,6 +80,7 @@ private void fillInRows(Map mapping, String prefix, List mapping = new LinkedHashMap<>(); + + mapping.put("field", new EsField("field", DataType.OBJECT, + singletonMap("alias", new EsField("alias", DataType.KEYWORD, emptyMap(), true)), false)); + + IndexResolution resolution = IndexResolution.valid(new EsIndex("test", mapping)); + + // check the nested alias is seen + accept(resolution, "SELECT field.alias FROM test"); + // or its hierarhcy + accept(resolution, "SELECT field.* FROM test"); + + // check typos + assertEquals("1:8: Unknown column [field.alas], did you mean [field.alias]?", error(resolution, "SELECT field.alas FROM test")); + + // non-existing parents for aliases are not seen by the user + assertEquals("1:8: Cannot use field [field] type [object] only its subfields", error(resolution, "SELECT field FROM test")); + } + public void testMultipleColumnsWithWildcard1() { assertEquals("1:14: Unknown column [a]\n" + "line 1:17: Unknown column [b]\n" + From e6d3d85db44c07f097facba450f1ee73e87b8640 Mon Sep 17 00:00:00 2001 From: Jay Modi Date: Thu, 10 Jan 2019 13:27:25 -0700 Subject: [PATCH 088/186] Ensure latch is counted down in ssl reload test (#37313) This change ensures we always countdown the latch in the SSLConfigurationReloaderTests to prevent the suite from timing out in case of an exception. Additionally, we also increase the logging of the resource watcher in case an IOException occurs. See #36053 --- .../core/ssl/SSLConfigurationReloaderTests.java | 16 +++++++++------- 1 file changed, 9 insertions(+), 7 deletions(-) diff --git a/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/ssl/SSLConfigurationReloaderTests.java b/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/ssl/SSLConfigurationReloaderTests.java index 2290a34752819..cb9996ac90db5 100644 --- a/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/ssl/SSLConfigurationReloaderTests.java +++ b/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/ssl/SSLConfigurationReloaderTests.java @@ -17,6 +17,7 @@ import org.elasticsearch.test.ESTestCase; import org.elasticsearch.test.http.MockResponse; import org.elasticsearch.test.http.MockWebServer; +import org.elasticsearch.test.junit.annotations.TestLogging; import org.elasticsearch.threadpool.TestThreadPool; import org.elasticsearch.threadpool.ThreadPool; import org.elasticsearch.watcher.ResourceWatcherService; @@ -55,6 +56,7 @@ /** * Unit tests for the reloading of SSL configuration */ +@TestLogging("org.elasticsearch.watcher:TRACE") public class SSLConfigurationReloaderTests extends ESTestCase { private ThreadPool threadPool; @@ -435,20 +437,20 @@ void reloadSSLContext(SSLConfiguration configuration) { assertThat(sslService.sslContextHolder(config).sslContext(), sameInstance(context)); } - private void validateSSLConfigurationIsReloaded(Settings settings, Environment env, - Consumer preChecks, - Runnable modificationFunction, - Consumer postChecks) - throws Exception { + private void validateSSLConfigurationIsReloaded(Settings settings, Environment env, Consumer preChecks, + Runnable modificationFunction, Consumer postChecks) throws Exception { final CountDownLatch reloadLatch = new CountDownLatch(1); final SSLService sslService = new SSLService(settings, env); final SSLConfiguration config = sslService.getSSLConfiguration("xpack.ssl"); new SSLConfigurationReloader(env, sslService, resourceWatcherService) { @Override void reloadSSLContext(SSLConfiguration configuration) { - super.reloadSSLContext(configuration); - reloadLatch.countDown(); + try { + super.reloadSSLContext(configuration); + } finally { + reloadLatch.countDown(); + } } }; // Baseline checks From 04dcb13ac48c12e1f30e99de74908792615f5566 Mon Sep 17 00:00:00 2001 From: Lee Hinman Date: Thu, 10 Jan 2019 13:38:14 -0700 Subject: [PATCH 089/186] Remove "beta" modifier from ILM documentation (#37326) --- docs/reference/ilm/apis/delete-lifecycle.asciidoc | 2 -- docs/reference/ilm/apis/explain.asciidoc | 2 -- docs/reference/ilm/apis/get-lifecycle.asciidoc | 2 -- docs/reference/ilm/apis/get-status.asciidoc | 2 -- docs/reference/ilm/apis/ilm-api.asciidoc | 2 -- docs/reference/ilm/apis/move-to-step.asciidoc | 2 -- docs/reference/ilm/apis/put-lifecycle.asciidoc | 2 -- .../ilm/apis/remove-policy-from-index.asciidoc | 2 -- docs/reference/ilm/apis/retry-policy.asciidoc | 2 -- docs/reference/ilm/apis/start.asciidoc | 2 -- docs/reference/ilm/apis/stop.asciidoc | 2 -- docs/reference/ilm/error-handling.asciidoc | 2 -- docs/reference/ilm/getting-started-ilm.asciidoc | 8 -------- docs/reference/ilm/ilm-and-snapshots.asciidoc | 4 +--- docs/reference/ilm/index.asciidoc | 1 - docs/reference/ilm/policy-definitions.asciidoc | 11 ----------- docs/reference/ilm/set-up-lifecycle-policy.asciidoc | 6 ------ docs/reference/ilm/start-stop-ilm.asciidoc | 2 -- docs/reference/ilm/update-lifecycle-policy.asciidoc | 8 -------- docs/reference/ilm/using-policies-rollover.asciidoc | 4 ---- 20 files changed, 1 insertion(+), 67 deletions(-) diff --git a/docs/reference/ilm/apis/delete-lifecycle.asciidoc b/docs/reference/ilm/apis/delete-lifecycle.asciidoc index 0b9fce20d10d0..9c45542836096 100644 --- a/docs/reference/ilm/apis/delete-lifecycle.asciidoc +++ b/docs/reference/ilm/apis/delete-lifecycle.asciidoc @@ -6,8 +6,6 @@ Delete policy ++++ -beta[] - Deletes a lifecycle policy. ==== Request diff --git a/docs/reference/ilm/apis/explain.asciidoc b/docs/reference/ilm/apis/explain.asciidoc index 1c8f5c9d861f1..66762ead9eb32 100644 --- a/docs/reference/ilm/apis/explain.asciidoc +++ b/docs/reference/ilm/apis/explain.asciidoc @@ -6,8 +6,6 @@ Explain lifecycle ++++ -beta[] - Shows an index's current lifecycle status. ==== Request diff --git a/docs/reference/ilm/apis/get-lifecycle.asciidoc b/docs/reference/ilm/apis/get-lifecycle.asciidoc index 161e82b091b3e..9bdf14d970caa 100644 --- a/docs/reference/ilm/apis/get-lifecycle.asciidoc +++ b/docs/reference/ilm/apis/get-lifecycle.asciidoc @@ -6,8 +6,6 @@ Get policy ++++ -beta[] - Retrieves a lifecycle policy. ==== Request diff --git a/docs/reference/ilm/apis/get-status.asciidoc b/docs/reference/ilm/apis/get-status.asciidoc index 4dceb18a3611a..5e67e26cb160c 100644 --- a/docs/reference/ilm/apis/get-status.asciidoc +++ b/docs/reference/ilm/apis/get-status.asciidoc @@ -6,8 +6,6 @@ Get {ilm} status ++++ -beta[] - Retrieves the current {ilm} ({ilm-init}) status. ==== Request diff --git a/docs/reference/ilm/apis/ilm-api.asciidoc b/docs/reference/ilm/apis/ilm-api.asciidoc index 457ca3daf5d74..edfc96d113fc7 100644 --- a/docs/reference/ilm/apis/ilm-api.asciidoc +++ b/docs/reference/ilm/apis/ilm-api.asciidoc @@ -1,8 +1,6 @@ [[index-lifecycle-management-api]] == {ilm-cap} API -beta[] - You can use the following APIs to manage policies on indices. See <> for more information about Index Lifecycle Management. diff --git a/docs/reference/ilm/apis/move-to-step.asciidoc b/docs/reference/ilm/apis/move-to-step.asciidoc index 6d648f5270209..57ea1a226ea40 100644 --- a/docs/reference/ilm/apis/move-to-step.asciidoc +++ b/docs/reference/ilm/apis/move-to-step.asciidoc @@ -6,8 +6,6 @@ Move to step ++++ -beta[] - Triggers execution of a specific step in the lifecycle policy. ==== Request diff --git a/docs/reference/ilm/apis/put-lifecycle.asciidoc b/docs/reference/ilm/apis/put-lifecycle.asciidoc index 21d48fd8cbddf..3e07e5f0f03ee 100644 --- a/docs/reference/ilm/apis/put-lifecycle.asciidoc +++ b/docs/reference/ilm/apis/put-lifecycle.asciidoc @@ -6,8 +6,6 @@ Create policy ++++ -beta[] - Creates or updates lifecycle policy. See <> for definitions of policy components. diff --git a/docs/reference/ilm/apis/remove-policy-from-index.asciidoc b/docs/reference/ilm/apis/remove-policy-from-index.asciidoc index b8b44c3d0fe28..888d3f17eecac 100644 --- a/docs/reference/ilm/apis/remove-policy-from-index.asciidoc +++ b/docs/reference/ilm/apis/remove-policy-from-index.asciidoc @@ -6,8 +6,6 @@ Remove policy ++++ -beta[] - Removes the assigned lifecycle policy from an index. ==== Request diff --git a/docs/reference/ilm/apis/retry-policy.asciidoc b/docs/reference/ilm/apis/retry-policy.asciidoc index 2d4944a7ac40d..c67540d2003f7 100644 --- a/docs/reference/ilm/apis/retry-policy.asciidoc +++ b/docs/reference/ilm/apis/retry-policy.asciidoc @@ -6,8 +6,6 @@ Retry policy ++++ -beta[] - Retry executing the policy for an index that is in the ERROR step. ==== Request diff --git a/docs/reference/ilm/apis/start.asciidoc b/docs/reference/ilm/apis/start.asciidoc index 97b7985c65428..92ca2a9806379 100644 --- a/docs/reference/ilm/apis/start.asciidoc +++ b/docs/reference/ilm/apis/start.asciidoc @@ -6,8 +6,6 @@ Start {ilm} ++++ -beta[] - Start the {ilm} ({ilm-init}) plugin. ==== Request diff --git a/docs/reference/ilm/apis/stop.asciidoc b/docs/reference/ilm/apis/stop.asciidoc index b2a1dba411a16..dee1cf4fa3579 100644 --- a/docs/reference/ilm/apis/stop.asciidoc +++ b/docs/reference/ilm/apis/stop.asciidoc @@ -6,8 +6,6 @@ Stop {ilm} ++++ -beta[] - Stop the {ilm} ({ilm-init}) plugin. ==== Request diff --git a/docs/reference/ilm/error-handling.asciidoc b/docs/reference/ilm/error-handling.asciidoc index 4d6a8d963afb3..abe643255bf95 100644 --- a/docs/reference/ilm/error-handling.asciidoc +++ b/docs/reference/ilm/error-handling.asciidoc @@ -3,8 +3,6 @@ [[index-lifecycle-error-handling]] == Index lifecycle error handling -beta[] - During Index Lifecycle Management's execution of the policy for an index, it's possible for a step to encounter an error during its execution. When this happens, ILM will move the management state into an "error" step. This halts diff --git a/docs/reference/ilm/getting-started-ilm.asciidoc b/docs/reference/ilm/getting-started-ilm.asciidoc index 63193544bfff5..f06c95f49c067 100644 --- a/docs/reference/ilm/getting-started-ilm.asciidoc +++ b/docs/reference/ilm/getting-started-ilm.asciidoc @@ -3,8 +3,6 @@ [[getting-started-index-lifecycle-management]] == Getting started with {ilm} -beta[] - Let's jump into {ilm} ({ilm-init}) by working through a hands-on scenario. This section will leverage many new concepts unique to {ilm-init} that you may not be familiar with. The following sections will explore @@ -19,8 +17,6 @@ after 90 days. === Setting up a new policy -beta[] - There are many new features introduced by {ilm-init}, but we will only focus on a few that are needed for our example. For starters, we will use the <> API to define our first policy. Lifecycle @@ -70,8 +66,6 @@ The index will be deleted 90 days after it is rolled over. === Applying a policy to our index -beta[] - There are <> to associate a policy to an index. Since we wish specific settings to be applied to the new index created from Rollover, we will set the policy via @@ -143,8 +137,6 @@ alias to be read-only for the source index. === Checking progress -beta[] - Now that we have an index managed by our policy, how do we tell what is going on? Which phase are we in? Is something broken? This section will go over a few APIs and their responses to help us inspect our indices with respect diff --git a/docs/reference/ilm/ilm-and-snapshots.asciidoc b/docs/reference/ilm/ilm-and-snapshots.asciidoc index 45028b4f1f85c..c71fbe0471da2 100644 --- a/docs/reference/ilm/ilm-and-snapshots.asciidoc +++ b/docs/reference/ilm/ilm-and-snapshots.asciidoc @@ -3,8 +3,6 @@ [[index-lifecycle-and-snapshots]] == Restoring snapshots of managed indices -beta[] - When restoring a snapshot that contains indices managed by Index Lifecycle Management, the lifecycle will automatically continue to execute after the snapshot is restored. Notably, the `min_age` is relative to the original @@ -32,4 +30,4 @@ prevent the execution of the lifecycle policy for an index: 3. Perform whatever operations you wish before resuming lifecycle execution, or remove the lifecycle policy from the index using the <> -4. Resume execution of lifecycle policies using the <> \ No newline at end of file +4. Resume execution of lifecycle policies using the <> diff --git a/docs/reference/ilm/index.asciidoc b/docs/reference/ilm/index.asciidoc index 2e7862712a6fd..aa27ab1386b80 100644 --- a/docs/reference/ilm/index.asciidoc +++ b/docs/reference/ilm/index.asciidoc @@ -5,7 +5,6 @@ [partintro] -- -beta[] The <> enable you to automate how you want to manage your indices over time. Rather than simply diff --git a/docs/reference/ilm/policy-definitions.asciidoc b/docs/reference/ilm/policy-definitions.asciidoc index d00782472b3f4..ab2c0a039f9d0 100644 --- a/docs/reference/ilm/policy-definitions.asciidoc +++ b/docs/reference/ilm/policy-definitions.asciidoc @@ -1,11 +1,8 @@ -beta[] [role="xpack"] [testenv="basic"] [[ilm-policy-definition]] == Policy phases and actions -beta[] - There are four stages in the index lifecycle, in the order they are executed. @@ -26,8 +23,6 @@ phase and the delete phase, while another may define all four phases. === Timing -beta[] - Indices enter phases based on a phase's `min_age` parameter. The index will not enter the phase until the index's age is older than that of the `min_age`. The parameter is configured using a time @@ -76,8 +71,6 @@ and transition into the next phase. === Phase Execution -beta[] - The current phase definition, of an index's policy being executed, is stored in the index's metadata. The phase and its actions are compiled into a series of discrete steps that are executed sequentially. Since some {ilm-init} actions @@ -89,8 +82,6 @@ executing. === Actions -beta[] - The below list shows the actions which are available in each phase. * Hot @@ -582,8 +573,6 @@ PUT _ilm/policy/my_policy === Full Policy -beta[] - With all of these actions, we can support complex management strategies for our indices. This policy will define an index that will start in the hot phase, rolling over every 50 GB or 7 days. After 30 days it enters the warm phase diff --git a/docs/reference/ilm/set-up-lifecycle-policy.asciidoc b/docs/reference/ilm/set-up-lifecycle-policy.asciidoc index 1ab8d4399d5e7..7af686238f334 100644 --- a/docs/reference/ilm/set-up-lifecycle-policy.asciidoc +++ b/docs/reference/ilm/set-up-lifecycle-policy.asciidoc @@ -3,8 +3,6 @@ [[set-up-lifecycle-policy]] == Set up {ilm} policy -beta[] - In order for an index to use an {ilm} policy to manage its lifecycle we must first define a lifecycle policy for it to use. The following request creates a policy called `my_policy` in Elasticsearch which we can later use to manage our @@ -49,8 +47,6 @@ To set the policy for an index there are two options: [[applying-policy-to-template]] === Applying a policy to an index template -beta[] - The `index.lifecycle.name` setting can be set in an index template so that it is automatically applied to indexes matching the templates index pattern: @@ -95,8 +91,6 @@ create a new index and roll the alias over to use the new index automatically. === Apply a policy to a create index request -beta[] - The `index.lifecycle.name` setting can be set on an individual create index request so {ilm} immediately starts managing the index: diff --git a/docs/reference/ilm/start-stop-ilm.asciidoc b/docs/reference/ilm/start-stop-ilm.asciidoc index 4414c13ee0f5c..e5366f028a9c7 100644 --- a/docs/reference/ilm/start-stop-ilm.asciidoc +++ b/docs/reference/ilm/start-stop-ilm.asciidoc @@ -3,8 +3,6 @@ [[start-stop-ilm]] == Start and stop {ilm} -beta[] - All indices that are managed by ILM will continue to execute their policies. There may be times when this is not desired on certain indices, or maybe even all the indices in a cluster. For example, diff --git a/docs/reference/ilm/update-lifecycle-policy.asciidoc b/docs/reference/ilm/update-lifecycle-policy.asciidoc index a566c737b7c0d..3e6627fdd3a7e 100644 --- a/docs/reference/ilm/update-lifecycle-policy.asciidoc +++ b/docs/reference/ilm/update-lifecycle-policy.asciidoc @@ -6,8 +6,6 @@ Update policy ++++ -beta[] - You can update an existing lifecycle policy to fix mistakes or change strategies for newly created indices. It is possible to update policy definitions and an index's `index.lifecycle.name` settings independently. To prevent the situation @@ -22,8 +20,6 @@ their effects on policy execution on indices. === Updates to policies not managing indices -beta[] - Indices not referencing an existing policy that is updated will not be affected. If an index is assigned to the policy, it will be assigned the latest version of that policy @@ -137,8 +133,6 @@ the policy. === Updates to executing policies -beta[] - Indices preserve the phase definition from the latest policy version that existed at the time that it entered that phase. Changes to the currently-executing phase within policy updates will not be reflected during execution. This means that updates to the `hot` phase, for example, will not affect @@ -445,8 +439,6 @@ GET my_index/_ilm/explain === Switching policies for an index -beta[] - Setting `index.lifecycle.name` to a different policy behaves much like a policy update, but instead of just switching to a different version, it switches to a different policy. diff --git a/docs/reference/ilm/using-policies-rollover.asciidoc b/docs/reference/ilm/using-policies-rollover.asciidoc index dd1b49d8d302c..266346fb8629f 100644 --- a/docs/reference/ilm/using-policies-rollover.asciidoc +++ b/docs/reference/ilm/using-policies-rollover.asciidoc @@ -3,8 +3,6 @@ [[using-policies-rollover]] == Using policies to manage index rollover -beta[] - The rollover action enables you to automatically roll over to a new index based on the index size, document count, or age. When a rollover is triggered, a new index is created, the write alias is updated to point to the new index, and all @@ -127,8 +125,6 @@ the new index, enabling indexing to continue uninterrupted. === Skipping Rollover -beta[] - The `index.lifecycle.indexing_complete` setting indicates to {ilm} whether this index has already been rolled over. If it is set to `true`, that indicates that this index has already been rolled over and does not need to be rolled over From 434430506b760257f6d3defd4cc5b6be5be8cc2b Mon Sep 17 00:00:00 2001 From: markharwood Date: Thu, 10 Jan 2019 21:35:19 +0000 Subject: [PATCH 090/186] Type removal - added deprecation warnings to _bulk apis (#36549) MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Added warnings checks to existing tests Added “defaultTypeIfNull” to DocWriteRequest interface so that Bulk requests can override a null choice of document type with any global custom choice. Related to #35190 --- .../action/bulk/TransportNoopBulkAction.java | 4 +- .../client/RequestConverters.java | 4 +- .../elasticsearch/client/BulkProcessorIT.java | 145 ++++++++++++++---- .../client/BulkProcessorRetryIT.java | 2 +- .../BulkRequestWithGlobalParametersIT.java | 38 +++-- .../java/org/elasticsearch/client/CrudIT.java | 8 +- .../client/MachineLearningGetResultsIT.java | 17 +- .../client/MachineLearningIT.java | 6 +- .../client/RequestConvertersTests.java | 51 +++--- .../org/elasticsearch/client/RollupIT.java | 6 +- .../documentation/CRUDDocumentationIT.java | 4 +- .../documentation/RollupDocumentationIT.java | 4 +- .../high-level/document/bulk.asciidoc | 4 +- .../index/reindex/ManyDocumentsIT.java | 2 +- .../upgrades/FullClusterRestartIT.java | 3 + .../elasticsearch/upgrades/IndexingIT.java | 2 + .../org/elasticsearch/upgrades/XPackIT.java | 2 + .../elasticsearch/action/DocWriteRequest.java | 9 ++ .../action/bulk/BulkRequest.java | 64 +++++++- .../action/bulk/BulkRequestBuilder.java | 22 ++- .../action/bulk/TransportBulkAction.java | 3 +- .../action/delete/DeleteRequest.java | 30 +++- .../action/index/IndexRequest.java | 35 ++++- .../action/update/UpdateRequest.java | 29 +++- .../rest/action/document/RestBulkAction.java | 8 + .../action/bulk/BulkIntegrationIT.java | 11 +- .../action/bulk/BulkRequestTests.java | 55 +++++-- .../bulk/TransportBulkActionTookTests.java | 5 +- .../action/delete/DeleteRequestTests.java | 11 +- .../action/update/UpdateRequestTests.java | 11 +- .../action/document/RestBulkActionTests.java | 2 +- .../ml/integration/DatafeedJobsRestIT.java | 2 + .../output/AutodetectStateProcessorTests.java | 8 +- .../action/MonitoringBulkRequestTests.java | 8 + .../monitoring/integration/MonitoringIT.java | 2 + .../action/RestMonitoringBulkActionTests.java | 7 + .../sql/qa/multi_node/RestSqlMultinodeIT.java | 2 +- .../sql/qa/security/SqlSecurityTestCase.java | 6 +- .../xpack/sql/qa/security/UserFunctionIT.java | 2 +- .../qa/single_node/JdbcShardFailureIT.java | 2 +- .../xpack/sql/qa/cli/FetchSizeTestCase.java | 2 +- .../xpack/sql/qa/jdbc/DataLoader.java | 10 +- .../xpack/sql/qa/jdbc/FetchSizeTestCase.java | 4 +- .../xpack/sql/qa/rest/RestSqlTestCase.java | 6 +- .../elasticsearch/multi_node/RollupIT.java | 2 +- .../elasticsearch/upgrades/IndexingIT.java | 2 + 46 files changed, 498 insertions(+), 164 deletions(-) diff --git a/client/client-benchmark-noop-api-plugin/src/main/java/org/elasticsearch/plugin/noop/action/bulk/TransportNoopBulkAction.java b/client/client-benchmark-noop-api-plugin/src/main/java/org/elasticsearch/plugin/noop/action/bulk/TransportNoopBulkAction.java index 6580d213548f9..671e6b338e2df 100644 --- a/client/client-benchmark-noop-api-plugin/src/main/java/org/elasticsearch/plugin/noop/action/bulk/TransportNoopBulkAction.java +++ b/client/client-benchmark-noop-api-plugin/src/main/java/org/elasticsearch/plugin/noop/action/bulk/TransportNoopBulkAction.java @@ -32,13 +32,15 @@ import org.elasticsearch.tasks.Task; import org.elasticsearch.transport.TransportService; +import java.util.function.Supplier; + public class TransportNoopBulkAction extends HandledTransportAction { private static final BulkItemResponse ITEM_RESPONSE = new BulkItemResponse(1, DocWriteRequest.OpType.UPDATE, new UpdateResponse(new ShardId("mock", "", 1), "mock_type", "1", 1L, DocWriteResponse.Result.CREATED)); @Inject public TransportNoopBulkAction(TransportService transportService, ActionFilters actionFilters) { - super(NoopBulkAction.NAME, transportService, actionFilters, BulkRequest::new); + super(NoopBulkAction.NAME, transportService, actionFilters, (Supplier) BulkRequest::new); } @Override diff --git a/client/rest-high-level/src/main/java/org/elasticsearch/client/RequestConverters.java b/client/rest-high-level/src/main/java/org/elasticsearch/client/RequestConverters.java index d4f67ba5e9e8c..7a6a562e4fb7f 100644 --- a/client/rest-high-level/src/main/java/org/elasticsearch/client/RequestConverters.java +++ b/client/rest-high-level/src/main/java/org/elasticsearch/client/RequestConverters.java @@ -165,7 +165,9 @@ static Request bulk(BulkRequest bulkRequest) throws IOException { metadata.field("_index", action.index()); } if (Strings.hasLength(action.type())) { - metadata.field("_type", action.type()); + if (MapperService.SINGLE_MAPPING_NAME.equals(action.type()) == false) { + metadata.field("_type", action.type()); + } } if (Strings.hasLength(action.id())) { metadata.field("_id", action.id()); diff --git a/client/rest-high-level/src/test/java/org/elasticsearch/client/BulkProcessorIT.java b/client/rest-high-level/src/test/java/org/elasticsearch/client/BulkProcessorIT.java index 2b870dbc475ea..72ffcd7c5062e 100644 --- a/client/rest-high-level/src/test/java/org/elasticsearch/client/BulkProcessorIT.java +++ b/client/rest-high-level/src/test/java/org/elasticsearch/client/BulkProcessorIT.java @@ -35,6 +35,8 @@ import org.elasticsearch.common.unit.ByteSizeValue; import org.elasticsearch.common.unit.TimeValue; import org.elasticsearch.common.xcontent.XContentType; +import org.elasticsearch.index.mapper.MapperService; +import org.elasticsearch.rest.action.document.RestBulkAction; import org.elasticsearch.search.SearchHit; import org.hamcrest.Matcher; import org.hamcrest.Matchers; @@ -70,8 +72,15 @@ public class BulkProcessorIT extends ESRestHighLevelClientTestCase { private static BulkProcessor.Builder initBulkProcessorBuilder(BulkProcessor.Listener listener) { return BulkProcessor.builder( - (request, bulkListener) -> highLevelClient().bulkAsync(request, RequestOptions.DEFAULT, bulkListener), listener); + (request, bulkListener) -> highLevelClient().bulkAsync(request, RequestOptions.DEFAULT, + bulkListener), listener); } + + private static BulkProcessor.Builder initBulkProcessorBuilderUsingTypes(BulkProcessor.Listener listener) { + return BulkProcessor.builder( + (request, bulkListener) -> highLevelClient().bulkAsync(request, expectWarnings(RestBulkAction.TYPES_DEPRECATION_MESSAGE), + bulkListener), listener); + } public void testThatBulkProcessorCountIsCorrect() throws Exception { final CountDownLatch latch = new CountDownLatch(1); @@ -320,35 +329,105 @@ public void testGlobalParametersAndSingleRequest() throws Exception { public void testGlobalParametersAndBulkProcessor() throws Exception { createIndexWithMultipleShards("test"); - final CountDownLatch latch = new CountDownLatch(1); - BulkProcessorTestListener listener = new BulkProcessorTestListener(latch); createFieldAddingPipleine("pipeline_id", "fieldNameXYZ", "valueXYZ"); + final String customType = "testType"; + final String ignoredType = "ignoredType"; int numDocs = randomIntBetween(10, 10); - try (BulkProcessor processor = initBulkProcessorBuilder(listener) - //let's make sure that the bulk action limit trips, one single execution will index all the documents - .setConcurrentRequests(randomIntBetween(0, 1)).setBulkActions(numDocs) - .setFlushInterval(TimeValue.timeValueHours(24)).setBulkSize(new ByteSizeValue(1, ByteSizeUnit.GB)) - .setGlobalIndex("test") - .setGlobalType("_doc") - .setGlobalRouting("routing") - .setGlobalPipeline("pipeline_id") - .build()) { - - indexDocs(processor, numDocs, null, null, "test", "pipeline_id"); - latch.await(); - - assertThat(listener.beforeCounts.get(), equalTo(1)); - assertThat(listener.afterCounts.get(), equalTo(1)); - assertThat(listener.bulkFailures.size(), equalTo(0)); - assertResponseItems(listener.bulkItems, numDocs); - - Iterable hits = searchAll(new SearchRequest("test").routing("routing")); + { + final CountDownLatch latch = new CountDownLatch(1); + BulkProcessorTestListener listener = new BulkProcessorTestListener(latch); + //Check that untyped document additions inherit the global type + String globalType = customType; + String localType = null; + try (BulkProcessor processor = initBulkProcessorBuilderUsingTypes(listener) + //let's make sure that the bulk action limit trips, one single execution will index all the documents + .setConcurrentRequests(randomIntBetween(0, 1)).setBulkActions(numDocs) + .setFlushInterval(TimeValue.timeValueHours(24)).setBulkSize(new ByteSizeValue(1, ByteSizeUnit.GB)) + .setGlobalIndex("test") + .setGlobalType(globalType) + .setGlobalRouting("routing") + .setGlobalPipeline("pipeline_id") + .build()) { + + indexDocs(processor, numDocs, null, localType, "test", globalType, "pipeline_id"); + latch.await(); + + assertThat(listener.beforeCounts.get(), equalTo(1)); + assertThat(listener.afterCounts.get(), equalTo(1)); + assertThat(listener.bulkFailures.size(), equalTo(0)); + assertResponseItems(listener.bulkItems, numDocs, globalType); + + Iterable hits = searchAll(new SearchRequest("test").routing("routing")); + + assertThat(hits, everyItem(hasProperty(fieldFromSource("fieldNameXYZ"), equalTo("valueXYZ")))); + assertThat(hits, everyItem(Matchers.allOf(hasIndex("test"), hasType(globalType)))); + assertThat(hits, containsInAnyOrder(expectedIds(numDocs))); + } - assertThat(hits, everyItem(hasProperty(fieldFromSource("fieldNameXYZ"), equalTo("valueXYZ")))); - assertThat(hits, everyItem(Matchers.allOf(hasIndex("test"), hasType("_doc")))); - assertThat(hits, containsInAnyOrder(expectedIds(numDocs))); } + { + //Check that typed document additions don't inherit the global type + String globalType = ignoredType; + String localType = customType; + final CountDownLatch latch = new CountDownLatch(1); + BulkProcessorTestListener listener = new BulkProcessorTestListener(latch); + try (BulkProcessor processor = initBulkProcessorBuilderUsingTypes(listener) + //let's make sure that the bulk action limit trips, one single execution will index all the documents + .setConcurrentRequests(randomIntBetween(0, 1)).setBulkActions(numDocs) + .setFlushInterval(TimeValue.timeValueHours(24)).setBulkSize(new ByteSizeValue(1, ByteSizeUnit.GB)) + .setGlobalIndex("test") + .setGlobalType(globalType) + .setGlobalRouting("routing") + .setGlobalPipeline("pipeline_id") + .build()) { + indexDocs(processor, numDocs, null, localType, "test", globalType, "pipeline_id"); + latch.await(); + + assertThat(listener.beforeCounts.get(), equalTo(1)); + assertThat(listener.afterCounts.get(), equalTo(1)); + assertThat(listener.bulkFailures.size(), equalTo(0)); + assertResponseItems(listener.bulkItems, numDocs, localType); + + Iterable hits = searchAll(new SearchRequest("test").routing("routing")); + + assertThat(hits, everyItem(hasProperty(fieldFromSource("fieldNameXYZ"), equalTo("valueXYZ")))); + assertThat(hits, everyItem(Matchers.allOf(hasIndex("test"), hasType(localType)))); + assertThat(hits, containsInAnyOrder(expectedIds(numDocs))); + } + } + { + //Check that untyped document additions and untyped global inherit the established custom type + // (the custom document type introduced to the mapping by the earlier code in this test) + String globalType = null; + String localType = null; + final CountDownLatch latch = new CountDownLatch(1); + BulkProcessorTestListener listener = new BulkProcessorTestListener(latch); + try (BulkProcessor processor = initBulkProcessorBuilder(listener) + //let's make sure that the bulk action limit trips, one single execution will index all the documents + .setConcurrentRequests(randomIntBetween(0, 1)).setBulkActions(numDocs) + .setFlushInterval(TimeValue.timeValueHours(24)).setBulkSize(new ByteSizeValue(1, ByteSizeUnit.GB)) + .setGlobalIndex("test") + .setGlobalType(globalType) + .setGlobalRouting("routing") + .setGlobalPipeline("pipeline_id") + .build()) { + indexDocs(processor, numDocs, null, localType, "test", globalType, "pipeline_id"); + latch.await(); + + assertThat(listener.beforeCounts.get(), equalTo(1)); + assertThat(listener.afterCounts.get(), equalTo(1)); + assertThat(listener.bulkFailures.size(), equalTo(0)); + assertResponseItems(listener.bulkItems, numDocs, MapperService.SINGLE_MAPPING_NAME); + + Iterable hits = searchAll(new SearchRequest("test").routing("routing")); + + assertThat(hits, everyItem(hasProperty(fieldFromSource("fieldNameXYZ"), equalTo("valueXYZ")))); + assertThat(hits, everyItem(Matchers.allOf(hasIndex("test"), hasType(customType)))); + assertThat(hits, containsInAnyOrder(expectedIds(numDocs))); + } + } + assertWarnings(RestBulkAction.TYPES_DEPRECATION_MESSAGE); } @SuppressWarnings("unchecked") @@ -359,15 +438,15 @@ private Matcher[] expectedIds(int numDocs) { .>toArray(Matcher[]::new); } - private static MultiGetRequest indexDocs(BulkProcessor processor, int numDocs, String localIndex, + private static MultiGetRequest indexDocs(BulkProcessor processor, int numDocs, String localIndex, String localType, String globalIndex, String globalType, String globalPipeline) throws Exception { MultiGetRequest multiGetRequest = new MultiGetRequest(); for (int i = 1; i <= numDocs; i++) { if (randomBoolean()) { - processor.add(new IndexRequest(localIndex).id(Integer.toString(i)) + processor.add(new IndexRequest(localIndex, localType, Integer.toString(i)) .source(XContentType.JSON, "field", randomRealisticUnicodeOfLengthBetween(1, 30))); } else { - BytesArray data = bytesBulkRequest(localIndex, "_doc", i); + BytesArray data = bytesBulkRequest(localIndex, localType, i); processor.add(data, globalIndex, globalType, globalPipeline, null, XContentType.JSON); } multiGetRequest.add(localIndex, Integer.toString(i)); @@ -396,15 +475,19 @@ private static BytesArray bytesBulkRequest(String localIndex, String localType, } private static MultiGetRequest indexDocs(BulkProcessor processor, int numDocs) throws Exception { - return indexDocs(processor, numDocs, "test", null, null, null); + return indexDocs(processor, numDocs, "test", null, null, null, null); } - + private static void assertResponseItems(List bulkItemResponses, int numDocs) { + assertResponseItems(bulkItemResponses, numDocs, MapperService.SINGLE_MAPPING_NAME); + } + + private static void assertResponseItems(List bulkItemResponses, int numDocs, String expectedType) { assertThat(bulkItemResponses.size(), is(numDocs)); int i = 1; for (BulkItemResponse bulkItemResponse : bulkItemResponses) { assertThat(bulkItemResponse.getIndex(), equalTo("test")); - assertThat(bulkItemResponse.getType(), equalTo("_doc")); + assertThat(bulkItemResponse.getType(), equalTo(expectedType)); assertThat(bulkItemResponse.getId(), equalTo(Integer.toString(i++))); assertThat("item " + i + " failed with cause: " + bulkItemResponse.getFailureMessage(), bulkItemResponse.isFailed(), equalTo(false)); diff --git a/client/rest-high-level/src/test/java/org/elasticsearch/client/BulkProcessorRetryIT.java b/client/rest-high-level/src/test/java/org/elasticsearch/client/BulkProcessorRetryIT.java index 6262a5b968513..e3b280225cb7b 100644 --- a/client/rest-high-level/src/test/java/org/elasticsearch/client/BulkProcessorRetryIT.java +++ b/client/rest-high-level/src/test/java/org/elasticsearch/client/BulkProcessorRetryIT.java @@ -143,7 +143,7 @@ public void afterBulk(long executionId, BulkRequest request, Throwable failure) private static MultiGetRequest indexDocs(BulkProcessor processor, int numDocs) { MultiGetRequest multiGetRequest = new MultiGetRequest(); for (int i = 1; i <= numDocs; i++) { - processor.add(new IndexRequest(INDEX_NAME, "_doc", Integer.toString(i)) + processor.add(new IndexRequest(INDEX_NAME).id(Integer.toString(i)) .source(XContentType.JSON, "field", randomRealisticUnicodeOfCodepointLengthBetween(1, 30))); multiGetRequest.add(INDEX_NAME, Integer.toString(i)); } diff --git a/client/rest-high-level/src/test/java/org/elasticsearch/client/BulkRequestWithGlobalParametersIT.java b/client/rest-high-level/src/test/java/org/elasticsearch/client/BulkRequestWithGlobalParametersIT.java index bb9f78622c821..3020eb0329b5c 100644 --- a/client/rest-high-level/src/test/java/org/elasticsearch/client/BulkRequestWithGlobalParametersIT.java +++ b/client/rest-high-level/src/test/java/org/elasticsearch/client/BulkRequestWithGlobalParametersIT.java @@ -24,6 +24,7 @@ import org.elasticsearch.action.index.IndexRequest; import org.elasticsearch.action.search.SearchRequest; import org.elasticsearch.common.xcontent.XContentType; +import org.elasticsearch.rest.action.document.RestBulkAction; import org.elasticsearch.search.SearchHit; import java.io.IOException; @@ -140,20 +141,19 @@ public void testIndexGlobalAndPerRequest() throws IOException { } public void testGlobalType() throws IOException { - BulkRequest request = new BulkRequest(null, "_doc"); + BulkRequest request = new BulkRequest(null, "global_type"); request.add(new IndexRequest("index").id("1") .source(XContentType.JSON, "field", "bulk1")); request.add(new IndexRequest("index").id("2") .source(XContentType.JSON, "field", "bulk2")); - bulk(request); + bulkWithTypes(request); Iterable hits = searchAll("index"); - assertThat(hits, everyItem(hasType("_doc"))); + assertThat(hits, everyItem(hasType("global_type"))); } @SuppressWarnings("unchecked") - @AwaitsFix(bugUrl = "https://github.com/elastic/elasticsearch/pull/36549") public void testTypeGlobalAndPerRequest() throws IOException { BulkRequest request = new BulkRequest(null, "global_type"); request.add(new IndexRequest("index1", "local_type", "1") @@ -161,7 +161,7 @@ public void testTypeGlobalAndPerRequest() throws IOException { request.add(new IndexRequest("index2").id("2") // will take global type .source(XContentType.JSON, "field", "bulk2")); - bulk(request); + bulkWithTypes(request); Iterable hits = searchAll("index1", "index2"); assertThat(hits, containsInAnyOrder( @@ -174,7 +174,7 @@ public void testTypeGlobalAndPerRequest() throws IOException { @SuppressWarnings("unchecked") public void testGlobalRouting() throws IOException { createIndexWithMultipleShards("index"); - BulkRequest request = new BulkRequest(null, null); + BulkRequest request = new BulkRequest(null); request.add(new IndexRequest("index").id("1") .source(XContentType.JSON, "field", "bulk1")); request.add(new IndexRequest("index").id("2") @@ -191,7 +191,7 @@ public void testGlobalRouting() throws IOException { @SuppressWarnings("unchecked") public void testMixLocalAndGlobalRouting() throws IOException { - BulkRequest request = new BulkRequest(null, null); + BulkRequest request = new BulkRequest(null); request.routing("globalRouting"); request.add(new IndexRequest("index").id("1") .source(XContentType.JSON, "field", "bulk1")); @@ -204,12 +204,32 @@ public void testMixLocalAndGlobalRouting() throws IOException { Iterable hits = searchAll(new SearchRequest("index").routing("globalRouting", "localRouting")); assertThat(hits, containsInAnyOrder(hasId("1"), hasId("2"))); } + + public void testGlobalIndexNoTypes() throws IOException { + BulkRequest request = new BulkRequest("global_index"); + request.add(new IndexRequest().id("1") + .source(XContentType.JSON, "field", "bulk1")); + request.add(new IndexRequest().id("2") + .source(XContentType.JSON, "field", "bulk2")); - private BulkResponse bulk(BulkRequest request) throws IOException { - BulkResponse bulkResponse = execute(request, highLevelClient()::bulk, highLevelClient()::bulkAsync); + bulk(request); + + Iterable hits = searchAll("global_index"); + assertThat(hits, everyItem(hasIndex("global_index"))); + } + + private BulkResponse bulkWithTypes(BulkRequest request) throws IOException { + BulkResponse bulkResponse = execute(request, highLevelClient()::bulk, highLevelClient()::bulkAsync, + expectWarnings(RestBulkAction.TYPES_DEPRECATION_MESSAGE)); assertFalse(bulkResponse.hasFailures()); return bulkResponse; } + + private BulkResponse bulk(BulkRequest request) throws IOException { + BulkResponse bulkResponse = execute(request, highLevelClient()::bulk, highLevelClient()::bulkAsync, RequestOptions.DEFAULT); + assertFalse(bulkResponse.hasFailures()); + return bulkResponse; + } @SuppressWarnings("unchecked") private static Function fieldFromSource(String fieldName) { 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 bdfc3fe4383ee..19a5b03b6d6cd 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 @@ -65,6 +65,7 @@ import org.elasticsearch.index.reindex.UpdateByQueryAction; import org.elasticsearch.index.reindex.UpdateByQueryRequest; import org.elasticsearch.rest.RestStatus; +import org.elasticsearch.rest.action.document.RestBulkAction; import org.elasticsearch.rest.action.document.RestDeleteAction; import org.elasticsearch.rest.action.document.RestGetAction; import org.elasticsearch.rest.action.document.RestMultiGetAction; @@ -449,7 +450,7 @@ public void testMultiGetWithTypes() throws IOException { bulk.add(new IndexRequest("index", "type", "id2") .source("{\"field\":\"value2\"}", XContentType.JSON)); - highLevelClient().bulk(bulk, RequestOptions.DEFAULT); + highLevelClient().bulk(bulk, expectWarnings(RestBulkAction.TYPES_DEPRECATION_MESSAGE)); MultiGetRequest multiGetRequest = new MultiGetRequest(); multiGetRequest.add("index", "id1"); multiGetRequest.add("index", "type", "id2"); @@ -819,7 +820,7 @@ public void testBulk() throws IOException { } } - BulkResponse bulkResponse = execute(bulkRequest, highLevelClient()::bulk, highLevelClient()::bulkAsync); + BulkResponse bulkResponse = execute(bulkRequest, highLevelClient()::bulk, highLevelClient()::bulkAsync, RequestOptions.DEFAULT); assertEquals(RestStatus.OK, bulkResponse.status()); assertTrue(bulkResponse.getTook().getMillis() > 0); assertEquals(nbItems, bulkResponse.getItems().length); @@ -1080,7 +1081,8 @@ public void afterBulk(long executionId, BulkRequest request, Throwable failure) }; try (BulkProcessor processor = BulkProcessor.builder( - (request, bulkListener) -> highLevelClient().bulkAsync(request, RequestOptions.DEFAULT, bulkListener), listener) + (request, bulkListener) -> highLevelClient().bulkAsync(request, + RequestOptions.DEFAULT, bulkListener), listener) .setConcurrentRequests(0) .setBulkSize(new ByteSizeValue(5, ByteSizeUnit.GB)) .setBulkActions(nbItems + 1) diff --git a/client/rest-high-level/src/test/java/org/elasticsearch/client/MachineLearningGetResultsIT.java b/client/rest-high-level/src/test/java/org/elasticsearch/client/MachineLearningGetResultsIT.java index 2ee09b496fc85..9b364975c773a 100644 --- a/client/rest-high-level/src/test/java/org/elasticsearch/client/MachineLearningGetResultsIT.java +++ b/client/rest-high-level/src/test/java/org/elasticsearch/client/MachineLearningGetResultsIT.java @@ -65,7 +65,6 @@ public class MachineLearningGetResultsIT extends ESRestHighLevelClientTestCase { private static final String RESULTS_INDEX = ".ml-anomalies-shared"; - private static final String DOC = "doc"; private static final String JOB_ID = "get-results-it-job"; @@ -100,7 +99,7 @@ public void createJobAndIndexResults() throws IOException { } private void addBucketIndexRequest(long timestamp, boolean isInterim, BulkRequest bulkRequest) { - IndexRequest indexRequest = new IndexRequest(RESULTS_INDEX, DOC); + IndexRequest indexRequest = new IndexRequest(RESULTS_INDEX); double bucketScore = randomDoubleBetween(0.0, 100.0, true); bucketStats.report(bucketScore); indexRequest.source("{\"job_id\":\"" + JOB_ID + "\", \"result_type\":\"bucket\", \"timestamp\": " + timestamp + "," + @@ -122,7 +121,7 @@ private void addRecordIndexRequests(long timestamp, boolean isInterim, BulkReque } private void addRecordIndexRequest(long timestamp, boolean isInterim, BulkRequest bulkRequest) { - IndexRequest indexRequest = new IndexRequest(RESULTS_INDEX, DOC); + IndexRequest indexRequest = new IndexRequest(RESULTS_INDEX); double recordScore = randomDoubleBetween(0.0, 100.0, true); recordStats.report(recordScore); double p = randomDoubleBetween(0.0, 0.05, false); @@ -133,7 +132,7 @@ private void addRecordIndexRequest(long timestamp, boolean isInterim, BulkReques } private void addCategoryIndexRequest(long categoryId, String categoryName, BulkRequest bulkRequest) { - IndexRequest indexRequest = new IndexRequest(RESULTS_INDEX, DOC); + IndexRequest indexRequest = new IndexRequest(RESULTS_INDEX); indexRequest.source("{\"job_id\":\"" + JOB_ID + "\", \"category_id\": " + categoryId + ", \"terms\": \"" + categoryName + "\", \"regex\": \".*?" + categoryName + ".*\", \"max_matching_length\": 3, \"examples\": [\"" + categoryName + "\"]}", XContentType.JSON); @@ -151,7 +150,7 @@ private void addCategoriesIndexRequests(BulkRequest bulkRequest) { private void addModelSnapshotIndexRequests(BulkRequest bulkRequest) { { - IndexRequest indexRequest = new IndexRequest(RESULTS_INDEX, DOC); + IndexRequest indexRequest = new IndexRequest(RESULTS_INDEX); indexRequest.source("{\"job_id\":\"" + JOB_ID + "\", \"timestamp\":1541587919000, " + "\"description\":\"State persisted due to job close at 2018-11-07T10:51:59+0000\", \"snapshot_id\":\"1541587919\"," + "\"snapshot_doc_count\":1, \"model_size_stats\":{\"job_id\":\"" + JOB_ID + "\", \"result_type\":\"model_size_stats\"," + @@ -162,7 +161,7 @@ private void addModelSnapshotIndexRequests(BulkRequest bulkRequest) { bulkRequest.add(indexRequest); } { - IndexRequest indexRequest = new IndexRequest(RESULTS_INDEX, DOC); + IndexRequest indexRequest = new IndexRequest(RESULTS_INDEX); indexRequest.source("{\"job_id\":\"" + JOB_ID + "\", \"timestamp\":1541588919000, " + "\"description\":\"State persisted due to job close at 2018-11-07T11:08:39+0000\", \"snapshot_id\":\"1541588919\"," + "\"snapshot_doc_count\":1, \"model_size_stats\":{\"job_id\":\"" + JOB_ID + "\", \"result_type\":\"model_size_stats\"," + @@ -173,7 +172,7 @@ private void addModelSnapshotIndexRequests(BulkRequest bulkRequest) { bulkRequest.add(indexRequest); } { - IndexRequest indexRequest = new IndexRequest(RESULTS_INDEX, DOC); + IndexRequest indexRequest = new IndexRequest(RESULTS_INDEX); indexRequest.source("{\"job_id\":\"" + JOB_ID + "\", \"timestamp\":1541589919000, " + "\"description\":\"State persisted due to job close at 2018-11-07T11:25:19+0000\", \"snapshot_id\":\"1541589919\"," + "\"snapshot_doc_count\":1, \"model_size_stats\":{\"job_id\":\"" + JOB_ID + "\", \"result_type\":\"model_size_stats\"," + @@ -752,7 +751,7 @@ public void testGetOverallBuckets() throws IOException { BulkRequest bulkRequest = new BulkRequest(); bulkRequest.setRefreshPolicy(WriteRequest.RefreshPolicy.IMMEDIATE); for (Bucket bucket : firstBuckets) { - IndexRequest indexRequest = new IndexRequest(RESULTS_INDEX, DOC); + IndexRequest indexRequest = new IndexRequest(RESULTS_INDEX); indexRequest.source("{\"job_id\":\"" + anotherJobId + "\", \"result_type\":\"bucket\", \"timestamp\": " + bucket.getTimestamp().getTime() + "," + "\"bucket_span\": 3600,\"is_interim\": " + bucket.isInterim() + ", \"anomaly_score\": " + String.valueOf(bucket.getAnomalyScore() + 10.0) + "}", XContentType.JSON); @@ -923,7 +922,7 @@ public void testGetInfluencers() throws IOException { // Last one score is higher double score = isLast ? 90.0 : 42.0; - IndexRequest indexRequest = new IndexRequest(RESULTS_INDEX, DOC); + IndexRequest indexRequest = new IndexRequest(RESULTS_INDEX); indexRequest.source("{\"job_id\":\"" + JOB_ID + "\", \"result_type\":\"influencer\", \"timestamp\": " + timestamp + "," + "\"bucket_span\": 3600,\"is_interim\": " + isInterim + ", \"influencer_score\": " + score + ", " + "\"influencer_field_name\":\"my_influencer\", \"influencer_field_value\": \"inf_1\", \"probability\":" diff --git a/client/rest-high-level/src/test/java/org/elasticsearch/client/MachineLearningIT.java b/client/rest-high-level/src/test/java/org/elasticsearch/client/MachineLearningIT.java index 4f009e53450b1..1971ff6461a74 100644 --- a/client/rest-high-level/src/test/java/org/elasticsearch/client/MachineLearningIT.java +++ b/client/rest-high-level/src/test/java/org/elasticsearch/client/MachineLearningIT.java @@ -538,7 +538,6 @@ public void testStartDatafeed() throws Exception { while(pastCopy < now) { IndexRequest doc = new IndexRequest(); doc.index(indexName); - doc.type("_doc"); doc.id("id" + i); doc.source("{\"total\":" +randomInt(1000) + ",\"timestamp\":"+ pastCopy +"}", XContentType.JSON); bulk.add(doc); @@ -747,7 +746,6 @@ public void testPreviewDatafeed() throws Exception { Integer total = randomInt(1000); IndexRequest doc = new IndexRequest(); doc.index(indexName); - doc.type("_doc"); doc.id("id" + i); doc.source("{\"total\":" + total + ",\"timestamp\":"+ thePast +"}", XContentType.JSON); bulk.add(doc); @@ -807,7 +805,7 @@ private String createExpiredData(String jobId) throws Exception { long timestamp = nowMillis - TimeValue.timeValueHours(totalBuckets - bucket).getMillis(); int bucketRate = bucket == anomalousBucket ? anomalousRate : normalRate; for (int point = 0; point < bucketRate; point++) { - IndexRequest indexRequest = new IndexRequest(indexId, "_doc"); + IndexRequest indexRequest = new IndexRequest(indexId); indexRequest.source(XContentType.JSON, "timestamp", timestamp, "total", randomInt(1000)); bulk.add(indexRequest); } @@ -817,7 +815,7 @@ private String createExpiredData(String jobId) throws Exception { { // Index a randomly named unused state document String docId = "non_existing_job_" + randomFrom("model_state_1234567#1", "quantiles", "categorizer_state#1"); - IndexRequest indexRequest = new IndexRequest(".ml-state", "_doc", docId); + IndexRequest indexRequest = new IndexRequest(".ml-state").id(docId); indexRequest.source(Collections.emptyMap(), XContentType.JSON); indexRequest.setRefreshPolicy(WriteRequest.RefreshPolicy.IMMEDIATE); highLevelClient().index(indexRequest, RequestOptions.DEFAULT); diff --git a/client/rest-high-level/src/test/java/org/elasticsearch/client/RequestConvertersTests.java b/client/rest-high-level/src/test/java/org/elasticsearch/client/RequestConvertersTests.java index 468bcf2bf8a1d..f31c562332687 100644 --- a/client/rest-high-level/src/test/java/org/elasticsearch/client/RequestConvertersTests.java +++ b/client/rest-high-level/src/test/java/org/elasticsearch/client/RequestConvertersTests.java @@ -860,7 +860,6 @@ public void testBulk() throws IOException { int nbItems = randomIntBetween(10, 100); for (int i = 0; i < nbItems; i++) { String index = randomAlphaOfLength(5); - String type = randomAlphaOfLength(5); String id = randomAlphaOfLength(5); BytesReference source = RandomObjects.randomSource(random(), xContentType); @@ -868,16 +867,16 @@ public void testBulk() throws IOException { DocWriteRequest docWriteRequest; if (opType == DocWriteRequest.OpType.INDEX) { - IndexRequest indexRequest = new IndexRequest(index, type, id).source(source, xContentType); + IndexRequest indexRequest = new IndexRequest(index).id(id).source(source, xContentType); docWriteRequest = indexRequest; if (randomBoolean()) { indexRequest.setPipeline(randomAlphaOfLength(5)); } } else if (opType == DocWriteRequest.OpType.CREATE) { - IndexRequest createRequest = new IndexRequest(index, type, id).source(source, xContentType).create(true); + IndexRequest createRequest = new IndexRequest(index).id(id).source(source, xContentType).create(true); docWriteRequest = createRequest; } else if (opType == DocWriteRequest.OpType.UPDATE) { - final UpdateRequest updateRequest = new UpdateRequest(index, type, id).doc(new IndexRequest().source(source, xContentType)); + final UpdateRequest updateRequest = new UpdateRequest(index, id).doc(new IndexRequest().source(source, xContentType)); docWriteRequest = updateRequest; if (randomBoolean()) { updateRequest.retryOnConflict(randomIntBetween(1, 5)); @@ -886,7 +885,7 @@ public void testBulk() throws IOException { randomizeFetchSourceContextParams(updateRequest::fetchSource, new HashMap<>()); } } else if (opType == DocWriteRequest.OpType.DELETE) { - docWriteRequest = new DeleteRequest(index, type, id); + docWriteRequest = new DeleteRequest(index, id); } else { throw new UnsupportedOperationException("optype [" + opType + "] not supported"); } @@ -954,9 +953,9 @@ public void testBulk() throws IOException { public void testBulkWithDifferentContentTypes() throws IOException { { BulkRequest bulkRequest = new BulkRequest(); - bulkRequest.add(new DeleteRequest("index", "type", "0")); - bulkRequest.add(new UpdateRequest("index", "type", "1").script(mockScript("test"))); - bulkRequest.add(new DeleteRequest("index", "type", "2")); + bulkRequest.add(new DeleteRequest("index", "0")); + bulkRequest.add(new UpdateRequest("index", "1").script(mockScript("test"))); + bulkRequest.add(new DeleteRequest("index", "2")); Request request = RequestConverters.bulk(bulkRequest); assertEquals(XContentType.JSON.mediaTypeWithoutParameters(), request.getEntity().getContentType().getValue()); @@ -964,16 +963,16 @@ public void testBulkWithDifferentContentTypes() throws IOException { { XContentType xContentType = randomFrom(XContentType.JSON, XContentType.SMILE); BulkRequest bulkRequest = new BulkRequest(); - bulkRequest.add(new DeleteRequest("index", "type", "0")); - bulkRequest.add(new IndexRequest("index", "type", "0").source(singletonMap("field", "value"), xContentType)); - bulkRequest.add(new DeleteRequest("index", "type", "2")); + bulkRequest.add(new DeleteRequest("index", "0")); + bulkRequest.add(new IndexRequest("index").id("0").source(singletonMap("field", "value"), xContentType)); + bulkRequest.add(new DeleteRequest("index", "2")); Request request = RequestConverters.bulk(bulkRequest); assertEquals(xContentType.mediaTypeWithoutParameters(), request.getEntity().getContentType().getValue()); } { XContentType xContentType = randomFrom(XContentType.JSON, XContentType.SMILE); - UpdateRequest updateRequest = new UpdateRequest("index", "type", "0"); + UpdateRequest updateRequest = new UpdateRequest("index", "0"); if (randomBoolean()) { updateRequest.doc(new IndexRequest().source(singletonMap("field", "value"), xContentType)); } else { @@ -985,8 +984,8 @@ public void testBulkWithDifferentContentTypes() throws IOException { } { BulkRequest bulkRequest = new BulkRequest(); - bulkRequest.add(new IndexRequest("index", "type", "0").source(singletonMap("field", "value"), XContentType.SMILE)); - bulkRequest.add(new IndexRequest("index", "type", "1").source(singletonMap("field", "value"), XContentType.JSON)); + bulkRequest.add(new IndexRequest("index").id("0").source(singletonMap("field", "value"), XContentType.SMILE)); + bulkRequest.add(new IndexRequest("index").id("1").source(singletonMap("field", "value"), XContentType.JSON)); IllegalArgumentException exception = expectThrows(IllegalArgumentException.class, () -> RequestConverters.bulk(bulkRequest)); assertEquals( "Mismatching content-type found for request with content-type [JSON], " + "previous requests have content-type [SMILE]", @@ -994,9 +993,9 @@ public void testBulkWithDifferentContentTypes() throws IOException { } { BulkRequest bulkRequest = new BulkRequest(); - bulkRequest.add(new IndexRequest("index", "type", "0").source(singletonMap("field", "value"), XContentType.JSON)); - bulkRequest.add(new IndexRequest("index", "type", "1").source(singletonMap("field", "value"), XContentType.JSON)); - bulkRequest.add(new UpdateRequest("index", "type", "2") + bulkRequest.add(new IndexRequest("index").id("0").source(singletonMap("field", "value"), XContentType.JSON)); + bulkRequest.add(new IndexRequest("index").id("1").source(singletonMap("field", "value"), XContentType.JSON)); + bulkRequest.add(new UpdateRequest("index", "2") .doc(new IndexRequest().source(singletonMap("field", "value"), XContentType.JSON)) .upsert(new IndexRequest().source(singletonMap("field", "value"), XContentType.SMILE))); IllegalArgumentException exception = expectThrows(IllegalArgumentException.class, () -> RequestConverters.bulk(bulkRequest)); @@ -1007,12 +1006,12 @@ public void testBulkWithDifferentContentTypes() throws IOException { { XContentType xContentType = randomFrom(XContentType.CBOR, XContentType.YAML); BulkRequest bulkRequest = new BulkRequest(); - bulkRequest.add(new DeleteRequest("index", "type", "0")); - bulkRequest.add(new IndexRequest("index", "type", "1").source(singletonMap("field", "value"), XContentType.JSON)); - bulkRequest.add(new DeleteRequest("index", "type", "2")); - bulkRequest.add(new DeleteRequest("index", "type", "3")); - bulkRequest.add(new IndexRequest("index", "type", "4").source(singletonMap("field", "value"), XContentType.JSON)); - bulkRequest.add(new IndexRequest("index", "type", "1").source(singletonMap("field", "value"), xContentType)); + bulkRequest.add(new DeleteRequest("index", "0")); + bulkRequest.add(new IndexRequest("index").id("1").source(singletonMap("field", "value"), XContentType.JSON)); + bulkRequest.add(new DeleteRequest("index", "2")); + bulkRequest.add(new DeleteRequest("index", "3")); + bulkRequest.add(new IndexRequest("index").id("4").source(singletonMap("field", "value"), XContentType.JSON)); + bulkRequest.add(new IndexRequest("index").id("1").source(singletonMap("field", "value"), xContentType)); IllegalArgumentException exception = expectThrows(IllegalArgumentException.class, () -> RequestConverters.bulk(bulkRequest)); assertEquals("Unsupported content-type found for request with content-type [" + xContentType + "], only JSON and SMILE are supported", exception.getMessage()); @@ -1022,11 +1021,11 @@ public void testBulkWithDifferentContentTypes() throws IOException { public void testGlobalPipelineOnBulkRequest() throws IOException { BulkRequest bulkRequest = new BulkRequest(); bulkRequest.pipeline("xyz"); - bulkRequest.add(new IndexRequest("test", "doc", "11") + bulkRequest.add(new IndexRequest("test").id("11") .source(XContentType.JSON, "field", "bulk1")); - bulkRequest.add(new IndexRequest("test", "doc", "12") + bulkRequest.add(new IndexRequest("test").id("12") .source(XContentType.JSON, "field", "bulk2")); - bulkRequest.add(new IndexRequest("test", "doc", "13") + bulkRequest.add(new IndexRequest("test").id("13") .source(XContentType.JSON, "field", "bulk3")); Request request = RequestConverters.bulk(bulkRequest); diff --git a/client/rest-high-level/src/test/java/org/elasticsearch/client/RollupIT.java b/client/rest-high-level/src/test/java/org/elasticsearch/client/RollupIT.java index 429b36d1d10f2..e57493acdf302 100644 --- a/client/rest-high-level/src/test/java/org/elasticsearch/client/RollupIT.java +++ b/client/rest-high-level/src/test/java/org/elasticsearch/client/RollupIT.java @@ -114,7 +114,7 @@ public int indexDocs() throws Exception { for (int second = 0; second < 60; second = second + 10) { final int value = randomIntBetween(0, 100); - final IndexRequest indexRequest = new IndexRequest("docs", "doc"); + final IndexRequest indexRequest = new IndexRequest("docs"); indexRequest.source(jsonBuilder() .startObject() .field("value", value) @@ -293,7 +293,7 @@ public void testGetRollupCaps() throws Exception { for (int second = 0; second < 60; second = second + 10) { final int value = randomIntBetween(0, 100); - final IndexRequest indexRequest = new IndexRequest("docs", "doc"); + final IndexRequest indexRequest = new IndexRequest("docs"); indexRequest.source(jsonBuilder() .startObject() .field("value", value) @@ -405,7 +405,7 @@ public void testGetRollupIndexCaps() throws Exception { for (int second = 0; second < 60; second = second + 10) { final int value = randomIntBetween(0, 100); - final IndexRequest indexRequest = new IndexRequest("docs", "doc"); + final IndexRequest indexRequest = new IndexRequest("docs"); indexRequest.source(jsonBuilder() .startObject() .field("value", value) diff --git a/client/rest-high-level/src/test/java/org/elasticsearch/client/documentation/CRUDDocumentationIT.java b/client/rest-high-level/src/test/java/org/elasticsearch/client/documentation/CRUDDocumentationIT.java index f2f9070d27b3f..894b569f614f3 100644 --- a/client/rest-high-level/src/test/java/org/elasticsearch/client/documentation/CRUDDocumentationIT.java +++ b/client/rest-high-level/src/test/java/org/elasticsearch/client/documentation/CRUDDocumentationIT.java @@ -701,7 +701,7 @@ public void testBulk() throws Exception { request.add(new IndexRequest("posts").id("4") // <3> .source(XContentType.JSON,"field", "baz")); // end::bulk-request-with-mixed-operations - BulkResponse bulkResponse = client.bulk(request, RequestOptions.DEFAULT); + BulkResponse bulkResponse = client.bulk(request, RequestOptions.DEFAULT); assertSame(RestStatus.OK, bulkResponse.status()); assertFalse(bulkResponse.hasFailures()); @@ -758,7 +758,7 @@ public void testBulk() throws Exception { // end::bulk-request-routing // tag::bulk-request-index-type - BulkRequest defaulted = new BulkRequest("posts","_doc"); // <1> + BulkRequest defaulted = new BulkRequest("posts"); // <1> // end::bulk-request-index-type // tag::bulk-execute-listener diff --git a/client/rest-high-level/src/test/java/org/elasticsearch/client/documentation/RollupDocumentationIT.java b/client/rest-high-level/src/test/java/org/elasticsearch/client/documentation/RollupDocumentationIT.java index 2163b188ee0e2..a1fcdbb7bfc83 100644 --- a/client/rest-high-level/src/test/java/org/elasticsearch/client/documentation/RollupDocumentationIT.java +++ b/client/rest-high-level/src/test/java/org/elasticsearch/client/documentation/RollupDocumentationIT.java @@ -89,7 +89,7 @@ public void setUpDocs() throws IOException { final BulkRequest bulkRequest = new BulkRequest(); bulkRequest.setRefreshPolicy(WriteRequest.RefreshPolicy.IMMEDIATE); for (int i = 0; i < 50; i++) { - final IndexRequest indexRequest = new IndexRequest("docs", "doc"); + final IndexRequest indexRequest = new IndexRequest("docs"); indexRequest.source(jsonBuilder() .startObject() .field("timestamp", String.format(Locale.ROOT, "2018-01-01T00:%02d:00Z", i)) @@ -103,7 +103,7 @@ public void setUpDocs() throws IOException { .endObject()); bulkRequest.add(indexRequest); } - BulkResponse bulkResponse = highLevelClient().bulk(bulkRequest, RequestOptions.DEFAULT); + BulkResponse bulkResponse = highLevelClient().bulk(bulkRequest, RequestOptions.DEFAULT); assertEquals(RestStatus.OK, bulkResponse.status()); assertFalse(bulkResponse.hasFailures()); diff --git a/docs/java-rest/high-level/document/bulk.asciidoc b/docs/java-rest/high-level/document/bulk.asciidoc index c50a1f790583b..061516388c4bb 100644 --- a/docs/java-rest/high-level/document/bulk.asciidoc +++ b/docs/java-rest/high-level/document/bulk.asciidoc @@ -86,8 +86,8 @@ include-tagged::{doc-tests-file}[{api}-request-routing] -------------------------------------------------- include-tagged::{doc-tests-file}[{api}-request-index-type] -------------------------------------------------- -<1> A bulk request with global index and type used on all sub requests, unless overridden on a sub request. -Both parameters are @Nullable and can only be set during +{request}+ creation. +<1> A bulk request with a global index used on all sub requests, unless overridden on a sub request. +This parameter is @Nullable and can only be set during +{request}+ creation. include::../execution.asciidoc[] diff --git a/modules/reindex/src/test/java/org/elasticsearch/index/reindex/ManyDocumentsIT.java b/modules/reindex/src/test/java/org/elasticsearch/index/reindex/ManyDocumentsIT.java index 6aa1046492ccc..b86f28452cc96 100644 --- a/modules/reindex/src/test/java/org/elasticsearch/index/reindex/ManyDocumentsIT.java +++ b/modules/reindex/src/test/java/org/elasticsearch/index/reindex/ManyDocumentsIT.java @@ -44,7 +44,7 @@ public void setupTestIndex() throws IOException { bulk.append("{\"index\":{}}\n"); bulk.append("{\"test\":\"test\"}\n"); } - Request request = new Request("POST", "/test/test/_bulk"); + Request request = new Request("POST", "/test/_bulk"); request.addParameter("refresh", "true"); request.setJsonEntity(bulk.toString()); client().performRequest(request); diff --git a/qa/full-cluster-restart/src/test/java/org/elasticsearch/upgrades/FullClusterRestartIT.java b/qa/full-cluster-restart/src/test/java/org/elasticsearch/upgrades/FullClusterRestartIT.java index 7519cade0f29b..e8bd2d95697a8 100644 --- a/qa/full-cluster-restart/src/test/java/org/elasticsearch/upgrades/FullClusterRestartIT.java +++ b/qa/full-cluster-restart/src/test/java/org/elasticsearch/upgrades/FullClusterRestartIT.java @@ -33,6 +33,7 @@ import org.elasticsearch.common.xcontent.XContentBuilder; import org.elasticsearch.common.xcontent.json.JsonXContent; import org.elasticsearch.common.xcontent.support.XContentMapValues; +import org.elasticsearch.rest.action.document.RestBulkAction; import org.elasticsearch.rest.action.document.RestGetAction; import org.elasticsearch.rest.action.document.RestUpdateAction; import org.elasticsearch.rest.action.search.RestExplainAction; @@ -495,6 +496,7 @@ public void testRollover() throws IOException { Request bulkRequest = new Request("POST", "/" + index + "_write/doc/_bulk"); bulkRequest.setJsonEntity(bulk.toString()); bulkRequest.addParameter("refresh", ""); + bulkRequest.setOptions(expectWarnings(RestBulkAction.TYPES_DEPRECATION_MESSAGE)); assertThat(EntityUtils.toString(client().performRequest(bulkRequest).getEntity()), containsString("\"errors\":false")); if (isRunningAgainstOldCluster()) { @@ -1070,6 +1072,7 @@ private void checkSnapshot(String snapshotName, int count, Version tookOnVersion Request writeToRestoredRequest = new Request("POST", "/restored_" + index + "/doc/_bulk"); writeToRestoredRequest.addParameter("refresh", "true"); writeToRestoredRequest.setJsonEntity(bulk.toString()); + writeToRestoredRequest.setOptions(expectWarnings(RestBulkAction.TYPES_DEPRECATION_MESSAGE)); assertThat(EntityUtils.toString(client().performRequest(writeToRestoredRequest).getEntity()), containsString("\"errors\":false")); // And count to make sure the add worked diff --git a/qa/rolling-upgrade/src/test/java/org/elasticsearch/upgrades/IndexingIT.java b/qa/rolling-upgrade/src/test/java/org/elasticsearch/upgrades/IndexingIT.java index e4736a79d05db..c80218c50ebe9 100644 --- a/qa/rolling-upgrade/src/test/java/org/elasticsearch/upgrades/IndexingIT.java +++ b/qa/rolling-upgrade/src/test/java/org/elasticsearch/upgrades/IndexingIT.java @@ -22,6 +22,7 @@ import org.elasticsearch.common.Booleans; import org.elasticsearch.common.Strings; import org.elasticsearch.common.xcontent.XContentBuilder; +import org.elasticsearch.rest.action.document.RestBulkAction; import org.elasticsearch.Version; import org.elasticsearch.client.Request; import org.elasticsearch.client.Response; @@ -152,6 +153,7 @@ private void bulk(String index, String valueSuffix, int count) throws IOExceptio } Request bulk = new Request("POST", "/_bulk"); bulk.addParameter("refresh", "true"); + bulk.setOptions(expectWarnings(RestBulkAction.TYPES_DEPRECATION_MESSAGE)); bulk.setJsonEntity(b.toString()); client().performRequest(bulk); } diff --git a/qa/rolling-upgrade/src/test/java/org/elasticsearch/upgrades/XPackIT.java b/qa/rolling-upgrade/src/test/java/org/elasticsearch/upgrades/XPackIT.java index cecdb0a6a2d2f..1704deada1762 100644 --- a/qa/rolling-upgrade/src/test/java/org/elasticsearch/upgrades/XPackIT.java +++ b/qa/rolling-upgrade/src/test/java/org/elasticsearch/upgrades/XPackIT.java @@ -21,6 +21,7 @@ import org.apache.http.util.EntityUtils; import org.junit.Before; import org.elasticsearch.client.Request; +import org.elasticsearch.rest.action.document.RestBulkAction; import java.io.IOException; @@ -59,6 +60,7 @@ public void testBasicFeature() throws IOException { + "{\"index\":{}}\n" + "{\"f\": \"2\"}\n"); bulk.addParameter("refresh", "true"); + bulk.setOptions(expectWarnings(RestBulkAction.TYPES_DEPRECATION_MESSAGE)); client().performRequest(bulk); Request sql = new Request("POST", "/_sql"); diff --git a/server/src/main/java/org/elasticsearch/action/DocWriteRequest.java b/server/src/main/java/org/elasticsearch/action/DocWriteRequest.java index e2d01aad230bd..fdf62e951a517 100644 --- a/server/src/main/java/org/elasticsearch/action/DocWriteRequest.java +++ b/server/src/main/java/org/elasticsearch/action/DocWriteRequest.java @@ -60,6 +60,15 @@ public interface DocWriteRequest extends IndicesRequest { */ String type(); + /** + * Set the default type supplied to a bulk + * request if this individual request's type is null + * or empty + * @return the Request + */ + T defaultTypeIfNull(String defaultType); + + /** * Get the id of the document for this request * @return the id diff --git a/server/src/main/java/org/elasticsearch/action/bulk/BulkRequest.java b/server/src/main/java/org/elasticsearch/action/bulk/BulkRequest.java index 01b9c22b4d107..292d75db5cff0 100644 --- a/server/src/main/java/org/elasticsearch/action/bulk/BulkRequest.java +++ b/server/src/main/java/org/elasticsearch/action/bulk/BulkRequest.java @@ -19,6 +19,7 @@ package org.elasticsearch.action.bulk; +import org.apache.logging.log4j.LogManager; import org.elasticsearch.action.ActionRequest; import org.elasticsearch.action.ActionRequestValidationException; import org.elasticsearch.action.CompositeIndicesRequest; @@ -36,6 +37,7 @@ import org.elasticsearch.common.bytes.BytesReference; 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.uid.Versions; import org.elasticsearch.common.unit.TimeValue; import org.elasticsearch.common.xcontent.LoggingDeprecationHandler; @@ -45,6 +47,8 @@ import org.elasticsearch.common.xcontent.XContentType; import org.elasticsearch.index.VersionType; import org.elasticsearch.index.seqno.SequenceNumbers; +import org.elasticsearch.rest.action.document.RestBulkAction; +import org.elasticsearch.index.mapper.MapperService; import org.elasticsearch.search.fetch.subphase.FetchSourceContext; import java.io.IOException; @@ -68,7 +72,7 @@ public class BulkRequest extends ActionRequest implements CompositeIndicesRequest, WriteRequest { private static final int REQUEST_OVERHEAD = 50; - + private static final DeprecationLogger deprecationLogger = new DeprecationLogger(LogManager.getLogger(BulkRequest.class)); private static final ParseField INDEX = new ParseField("_index"); private static final ParseField TYPE = new ParseField("_type"); private static final ParseField ID = new ParseField("_id"); @@ -104,6 +108,14 @@ public class BulkRequest extends ActionRequest implements CompositeIndicesReques public BulkRequest() { } + public BulkRequest(@Nullable String globalIndex) { + this.globalIndex = globalIndex; + } + + /** + * @deprecated Types are in the process of being removed. Use {@link #BulkRequest(String)} instead + */ + @Deprecated public BulkRequest(@Nullable String globalIndex, @Nullable String globalType) { this.globalIndex = globalIndex; this.globalType = globalType; @@ -280,28 +292,71 @@ public BulkRequest add(byte[] data, int from, int length, XContentType xContentT /** * Adds a framed data in binary format + * @deprecated use {@link #add(byte[], int, int, String, XContentType)} instead */ + @Deprecated public BulkRequest add(byte[] data, int from, int length, @Nullable String defaultIndex, @Nullable String defaultType, XContentType xContentType) throws IOException { return add(new BytesArray(data, from, length), defaultIndex, defaultType, xContentType); } + /** * Adds a framed data in binary format */ + public BulkRequest add(byte[] data, int from, int length, @Nullable String defaultIndex, + XContentType xContentType) throws IOException { + return add(new BytesArray(data, from, length), defaultIndex, MapperService.SINGLE_MAPPING_NAME, xContentType); + } + + /** + * Adds a framed data in binary format + * @deprecated use {@link #add(BytesReference, String, XContentType)} instead + */ + @Deprecated public BulkRequest add(BytesReference data, @Nullable String defaultIndex, @Nullable String defaultType, XContentType xContentType) throws IOException { return add(data, defaultIndex, defaultType, null, null, null, null, true, xContentType); } + + /** + * Adds a framed data in binary format + */ + public BulkRequest add(BytesReference data, @Nullable String defaultIndex, + XContentType xContentType) throws IOException { + return add(data, defaultIndex, MapperService.SINGLE_MAPPING_NAME, null, null, null, null, true, xContentType); + } /** * Adds a framed data in binary format + * @deprecated use {@link #add(BytesReference, String, boolean, XContentType)} instead */ + @Deprecated public BulkRequest add(BytesReference data, @Nullable String defaultIndex, @Nullable String defaultType, boolean allowExplicitIndex, XContentType xContentType) throws IOException { return add(data, defaultIndex, defaultType, null, null, null, null, allowExplicitIndex, xContentType); } + + /** + * Adds a framed data in binary format + */ + public BulkRequest add(BytesReference data, @Nullable String defaultIndex, boolean allowExplicitIndex, + XContentType xContentType) throws IOException { + return add(data, defaultIndex, MapperService.SINGLE_MAPPING_NAME, null, null, null, null, allowExplicitIndex, xContentType); + } + + public BulkRequest add(BytesReference data, @Nullable String defaultIndex, + @Nullable String defaultRouting, @Nullable FetchSourceContext defaultFetchSourceContext, + @Nullable String defaultPipeline, @Nullable Object payload, boolean allowExplicitIndex, + XContentType xContentType) throws IOException { + return add(data, defaultIndex, MapperService.SINGLE_MAPPING_NAME, defaultRouting, defaultFetchSourceContext, + defaultPipeline, payload, allowExplicitIndex, xContentType); + } + /** + * @deprecated use {@link #add(BytesReference, String, String, FetchSourceContext, String, Object, boolean, XContentType)} instead + */ + @Deprecated public BulkRequest add(BytesReference data, @Nullable String defaultIndex, @Nullable String defaultType, @Nullable String defaultRouting, @Nullable FetchSourceContext defaultFetchSourceContext, @Nullable String defaultPipeline, @Nullable Object payload, boolean allowExplicitIndex, @@ -371,7 +426,8 @@ public BulkRequest add(BytesReference data, @Nullable String defaultIndex, @Null throw new IllegalArgumentException("explicit index in bulk is not allowed"); } index = parser.text(); - } else if (TYPE.match(currentFieldName, parser.getDeprecationHandler())) { + } else if (TYPE.match(currentFieldName, parser.getDeprecationHandler())) { + deprecationLogger.deprecatedAndMaybeLog("bulk_with_types", RestBulkAction.TYPES_DEPRECATION_MESSAGE); type = parser.text(); } else if (ID.match(currentFieldName, parser.getDeprecationHandler())) { id = parser.text(); @@ -625,7 +681,9 @@ public String getDescription() { private void applyGlobalMandatoryParameters(DocWriteRequest request) { request.index(valueOrDefault(request.index(), globalIndex)); - request.type(valueOrDefault(request.type(), globalType)); + if (Strings.isNullOrEmpty(globalType) == false && MapperService.SINGLE_MAPPING_NAME.equals(globalType) == false) { + request.defaultTypeIfNull(globalType); + } } private static String valueOrDefault(String value, String globalDefault) { diff --git a/server/src/main/java/org/elasticsearch/action/bulk/BulkRequestBuilder.java b/server/src/main/java/org/elasticsearch/action/bulk/BulkRequestBuilder.java index fc91f4f907ee2..34837d0e696db 100644 --- a/server/src/main/java/org/elasticsearch/action/bulk/BulkRequestBuilder.java +++ b/server/src/main/java/org/elasticsearch/action/bulk/BulkRequestBuilder.java @@ -33,6 +33,7 @@ import org.elasticsearch.common.Nullable; import org.elasticsearch.common.unit.TimeValue; import org.elasticsearch.common.xcontent.XContentType; +import org.elasticsearch.index.mapper.MapperService; /** * A bulk request holds an ordered {@link IndexRequest}s and {@link DeleteRequest}s and allows to executes @@ -41,10 +42,18 @@ public class BulkRequestBuilder extends ActionRequestBuilder implements WriteRequestBuilder { + /** + * @deprecated use {@link #BulkRequestBuilder(ElasticsearchClient, BulkAction, String)} instead + */ + @Deprecated public BulkRequestBuilder(ElasticsearchClient client, BulkAction action, @Nullable String globalIndex, @Nullable String globalType) { super(client, action, new BulkRequest(globalIndex, globalType)); } + public BulkRequestBuilder(ElasticsearchClient client, BulkAction action, @Nullable String globalIndex) { + super(client, action, new BulkRequest(globalIndex)); + } + public BulkRequestBuilder(ElasticsearchClient client, BulkAction action) { super(client, action, new BulkRequest()); } @@ -104,19 +113,30 @@ public BulkRequestBuilder add(UpdateRequestBuilder request) { * Adds a framed data in binary format */ public BulkRequestBuilder add(byte[] data, int from, int length, XContentType xContentType) throws Exception { - request.add(data, from, length, null, null, xContentType); + request.add(data, from, length, null, xContentType); return this; } /** * Adds a framed data in binary format + * @deprecated use {@link #add(byte[], int, int, String, XContentType)} instead */ + @Deprecated public BulkRequestBuilder add(byte[] data, int from, int length, @Nullable String defaultIndex, @Nullable String defaultType, XContentType xContentType) throws Exception { request.add(data, from, length, defaultIndex, defaultType, xContentType); return this; } + /** + * Adds a framed data in binary format + */ + public BulkRequestBuilder add(byte[] data, int from, int length, @Nullable String defaultIndex, + XContentType xContentType) throws Exception { + request.add(data, from, length, defaultIndex, MapperService.SINGLE_MAPPING_NAME, xContentType); + return this; + } + /** * Sets the number of shard copies that must be active before proceeding with the write. * See {@link ReplicationRequest#waitForActiveShards(ActiveShardCount)} for details. diff --git a/server/src/main/java/org/elasticsearch/action/bulk/TransportBulkAction.java b/server/src/main/java/org/elasticsearch/action/bulk/TransportBulkAction.java index a89d162979f5f..7979b633efebe 100644 --- a/server/src/main/java/org/elasticsearch/action/bulk/TransportBulkAction.java +++ b/server/src/main/java/org/elasticsearch/action/bulk/TransportBulkAction.java @@ -78,6 +78,7 @@ import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicInteger; import java.util.function.LongSupplier; +import java.util.function.Supplier; import java.util.stream.Collectors; import static java.util.Collections.emptyMap; @@ -113,7 +114,7 @@ public TransportBulkAction(ThreadPool threadPool, TransportService transportServ TransportShardBulkAction shardBulkAction, NodeClient client, ActionFilters actionFilters, IndexNameExpressionResolver indexNameExpressionResolver, AutoCreateIndex autoCreateIndex, LongSupplier relativeTimeProvider) { - super(BulkAction.NAME, transportService, actionFilters, BulkRequest::new); + super(BulkAction.NAME, transportService, actionFilters, (Supplier) BulkRequest::new); Objects.requireNonNull(relativeTimeProvider); this.threadPool = threadPool; this.clusterService = clusterService; diff --git a/server/src/main/java/org/elasticsearch/action/delete/DeleteRequest.java b/server/src/main/java/org/elasticsearch/action/delete/DeleteRequest.java index 7fb19bbf9eb93..8d2967fd28ba4 100644 --- a/server/src/main/java/org/elasticsearch/action/delete/DeleteRequest.java +++ b/server/src/main/java/org/elasticsearch/action/delete/DeleteRequest.java @@ -53,7 +53,8 @@ public class DeleteRequest extends ReplicatedWriteRequest implements DocWriteRequest, CompositeIndicesRequest { - private String type = MapperService.SINGLE_MAPPING_NAME; + // Set to null initially so we can know to override in bulk requests that have a default type. + private String type; private String id; @Nullable private String routing; @@ -103,7 +104,7 @@ public DeleteRequest(String index, String id) { @Override public ActionRequestValidationException validate() { ActionRequestValidationException validationException = super.validate(); - if (Strings.isEmpty(type)) { + if (Strings.isEmpty(type())) { validationException = addValidationError("type is missing", validationException); } if (Strings.isEmpty(id)) { @@ -142,6 +143,9 @@ public ActionRequestValidationException validate() { @Deprecated @Override public String type() { + if (type == null) { + return MapperService.SINGLE_MAPPING_NAME; + } return type; } @@ -156,6 +160,22 @@ public DeleteRequest type(String type) { this.type = type; return this; } + + /** + * Set the default type supplied to a bulk + * request if this individual request's type is null + * or empty + * + * @deprecated Types are in the process of being removed. + */ + @Deprecated + @Override + public DeleteRequest defaultTypeIfNull(String defaultType) { + if (Strings.isNullOrEmpty(type)) { + type = defaultType; + } + return this; + } /** * The id of the document to delete. @@ -295,7 +315,9 @@ public void readFrom(StreamInput in) throws IOException { @Override public void writeTo(StreamOutput out) throws IOException { super.writeTo(out); - out.writeString(type); + // A 7.x request allows null types but if deserialized in a 6.x node will cause nullpointer exceptions. + // So we use the type accessor method here to make the type non-null (will default it to "_doc"). + out.writeString(type()); out.writeString(id); out.writeOptionalString(routing()); if (out.getVersion().before(Version.V_7_0_0)) { @@ -316,7 +338,7 @@ public void writeTo(StreamOutput out) throws IOException { @Override public String toString() { - return "delete {[" + index + "][" + type + "][" + id + "]}"; + return "delete {[" + index + "][" + type() + "][" + id + "]}"; } /** diff --git a/server/src/main/java/org/elasticsearch/action/index/IndexRequest.java b/server/src/main/java/org/elasticsearch/action/index/IndexRequest.java index 195f6863b3d0c..a9aac3025de1e 100644 --- a/server/src/main/java/org/elasticsearch/action/index/IndexRequest.java +++ b/server/src/main/java/org/elasticsearch/action/index/IndexRequest.java @@ -31,6 +31,7 @@ import org.elasticsearch.cluster.metadata.MappingMetaData; import org.elasticsearch.cluster.metadata.MetaData; import org.elasticsearch.common.Nullable; +import org.elasticsearch.common.Strings; import org.elasticsearch.common.UUIDs; import org.elasticsearch.common.bytes.BytesArray; import org.elasticsearch.common.bytes.BytesReference; @@ -82,7 +83,8 @@ public class IndexRequest extends ReplicatedWriteRequest implement */ static final int MAX_SOURCE_LENGTH_IN_TOSTRING = 2048; - private String type = MapperService.SINGLE_MAPPING_NAME; + // Set to null initially so we can know to override in bulk requests that have a default type. + private String type; private String id; @Nullable private String routing; @@ -152,12 +154,12 @@ public IndexRequest(String index, String type, String id) { @Override public ActionRequestValidationException validate() { ActionRequestValidationException validationException = super.validate(); - if (type == null) { - validationException = addValidationError("type is missing", validationException); - } if (source == null) { validationException = addValidationError("source is missing", validationException); } + if (Strings.isEmpty(type())) { + validationException = addValidationError("type is missing", validationException); + } if (contentType == null) { validationException = addValidationError("content type is missing", validationException); } @@ -239,6 +241,9 @@ public XContentType getContentType() { @Deprecated @Override public String type() { + if (type == null) { + return MapperService.SINGLE_MAPPING_NAME; + } return type; } @@ -253,6 +258,20 @@ public IndexRequest type(String type) { return this; } + /** + * Set the default type supplied to a bulk + * request if this individual request's type is null + * or empty + * @deprecated Types are in the process of being removed. + */ + @Deprecated + @Override + public IndexRequest defaultTypeIfNull(String defaultType) { + if (Strings.isNullOrEmpty(type)) { + type = defaultType; + } + return this; + } /** * The id of the indexed document. If not set, will be automatically generated. */ @@ -563,7 +582,7 @@ public void process(Version indexCreatedVersion, @Nullable MappingMetaData mappi if (mappingMd != null) { // might as well check for routing here if (mappingMd.routing().required() && routing == null) { - throw new RoutingMissingException(concreteIndex, type, id); + throw new RoutingMissingException(concreteIndex, type(), id); } } @@ -629,7 +648,9 @@ public void readFrom(StreamInput in) throws IOException { @Override public void writeTo(StreamOutput out) throws IOException { super.writeTo(out); - out.writeOptionalString(type); + // A 7.x request allows null types but if deserialized in a 6.x node will cause nullpointer exceptions. + // So we use the type accessor method here to make the type non-null (will default it to "_doc"). + out.writeOptionalString(type()); out.writeOptionalString(id); out.writeOptionalString(routing); if (out.getVersion().before(Version.V_7_0_0)) { @@ -679,7 +700,7 @@ public String toString() { } catch (Exception e) { // ignore } - return "index {[" + index + "][" + type + "][" + id + "], source[" + sSource + "]}"; + return "index {[" + index + "][" + type() + "][" + id + "], source[" + sSource + "]}"; } diff --git a/server/src/main/java/org/elasticsearch/action/update/UpdateRequest.java b/server/src/main/java/org/elasticsearch/action/update/UpdateRequest.java index cd0b5629b7600..a7805b4cbdbad 100644 --- a/server/src/main/java/org/elasticsearch/action/update/UpdateRequest.java +++ b/server/src/main/java/org/elasticsearch/action/update/UpdateRequest.java @@ -91,7 +91,8 @@ public class UpdateRequest extends InstanceShardOperationRequest ObjectParser.ValueType.OBJECT_ARRAY_BOOLEAN_OR_STRING); } - private String type = MapperService.SINGLE_MAPPING_NAME; + // Set to null initially so we can know to override in bulk requests that have a default type. + private String type; private String id; @Nullable private String routing; @@ -146,7 +147,7 @@ public ActionRequestValidationException validate() { if(upsertRequest != null && upsertRequest.version() != Versions.MATCH_ANY) { validationException = addValidationError("can't provide version in upsert request", validationException); } - if (Strings.isEmpty(type)) { + if (Strings.isEmpty(type())) { validationException = addValidationError("type is missing", validationException); } if (Strings.isEmpty(id)) { @@ -189,6 +190,9 @@ public ActionRequestValidationException validate() { @Deprecated @Override public String type() { + if (type == null) { + return MapperService.SINGLE_MAPPING_NAME; + } return type; } @@ -203,6 +207,21 @@ public UpdateRequest type(String type) { return this; } + /** + * Set the default type supplied to a bulk + * request if this individual request's type is null + * or empty + * @deprecated Types are in the process of being removed. + */ + @Deprecated + @Override + public UpdateRequest defaultTypeIfNull(String defaultType) { + if (Strings.isNullOrEmpty(type)) { + type = defaultType; + } + return this; + } + /** * The id of the indexed document. */ @@ -800,7 +819,9 @@ public void readFrom(StreamInput in) throws IOException { public void writeTo(StreamOutput out) throws IOException { super.writeTo(out); waitForActiveShards.writeTo(out); - out.writeString(type); + // A 7.x request allows null types but if deserialized in a 6.x node will cause nullpointer exceptions. + // So we use the type accessor method here to make the type non-null (will default it to "_doc"). + out.writeString(type()); out.writeString(id); out.writeOptionalString(routing); if (out.getVersion().before(Version.V_7_0_0)) { @@ -887,7 +908,7 @@ public XContentBuilder toXContent(XContentBuilder builder, Params params) throws public String toString() { StringBuilder res = new StringBuilder() .append("update {[").append(index) - .append("][").append(type) + .append("][").append(type()) .append("][").append(id).append("]"); res.append(", doc_as_upsert[").append(docAsUpsert).append("]"); if (doc != null) { diff --git a/server/src/main/java/org/elasticsearch/rest/action/document/RestBulkAction.java b/server/src/main/java/org/elasticsearch/rest/action/document/RestBulkAction.java index 5cfbbfdb1b524..7861a4fe9d1bd 100644 --- a/server/src/main/java/org/elasticsearch/rest/action/document/RestBulkAction.java +++ b/server/src/main/java/org/elasticsearch/rest/action/document/RestBulkAction.java @@ -19,17 +19,20 @@ package org.elasticsearch.rest.action.document; +import org.apache.logging.log4j.LogManager; import org.elasticsearch.action.bulk.BulkRequest; import org.elasticsearch.action.bulk.BulkShardRequest; import org.elasticsearch.action.support.ActiveShardCount; import org.elasticsearch.client.Requests; import org.elasticsearch.client.node.NodeClient; +import org.elasticsearch.common.logging.DeprecationLogger; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.index.mapper.MapperService; import org.elasticsearch.rest.BaseRestHandler; import org.elasticsearch.rest.RestController; import org.elasticsearch.rest.RestRequest; import org.elasticsearch.rest.action.RestStatusToXContentListener; +import org.elasticsearch.rest.action.search.RestSearchAction; import org.elasticsearch.search.fetch.subphase.FetchSourceContext; import java.io.IOException; @@ -49,6 +52,9 @@ public class RestBulkAction extends BaseRestHandler { private final boolean allowExplicitIndex; + private static final DeprecationLogger deprecationLogger = new DeprecationLogger(LogManager.getLogger(RestSearchAction.class)); + public static final String TYPES_DEPRECATION_MESSAGE = "[types removal]" + + " Specifying types in bulk requests is deprecated."; public RestBulkAction(Settings settings, RestController controller) { super(settings); @@ -76,6 +82,8 @@ public RestChannelConsumer prepareRequest(final RestRequest request, final NodeC String defaultType = request.param("type"); if (defaultType == null) { defaultType = MapperService.SINGLE_MAPPING_NAME; + } else { + deprecationLogger.deprecatedAndMaybeLog("bulk_with_types", RestBulkAction.TYPES_DEPRECATION_MESSAGE); } String defaultRouting = request.param("routing"); FetchSourceContext defaultFetchSourceContext = FetchSourceContext.parseFromRestRequest(request); diff --git a/server/src/test/java/org/elasticsearch/action/bulk/BulkIntegrationIT.java b/server/src/test/java/org/elasticsearch/action/bulk/BulkIntegrationIT.java index 325894366fe64..de8a2d9d67bbe 100644 --- a/server/src/test/java/org/elasticsearch/action/bulk/BulkIntegrationIT.java +++ b/server/src/test/java/org/elasticsearch/action/bulk/BulkIntegrationIT.java @@ -57,7 +57,7 @@ protected Collection> nodePlugins() { public void testBulkIndexCreatesMapping() throws Exception { String bulkAction = copyToStringFromClasspath("/org/elasticsearch/action/bulk/bulk-log.json"); BulkRequestBuilder bulkBuilder = client().prepareBulk(); - bulkBuilder.add(bulkAction.getBytes(StandardCharsets.UTF_8), 0, bulkAction.length(), null, null, XContentType.JSON); + bulkBuilder.add(bulkAction.getBytes(StandardCharsets.UTF_8), 0, bulkAction.length(), null, XContentType.JSON); bulkBuilder.get(); assertBusy(() -> { GetMappingsResponse mappingsResponse = client().admin().indices().prepareGetMappings().get(); @@ -105,16 +105,13 @@ public void testBulkWithGlobalDefaults() throws Exception { String bulkAction = copyToStringFromClasspath("/org/elasticsearch/action/bulk/simple-bulk-missing-index-type.json"); { BulkRequestBuilder bulkBuilder = client().prepareBulk(); - bulkBuilder.add(bulkAction.getBytes(StandardCharsets.UTF_8), 0, bulkAction.length(), null, null, XContentType.JSON); + bulkBuilder.add(bulkAction.getBytes(StandardCharsets.UTF_8), 0, bulkAction.length(), null, XContentType.JSON); ActionRequestValidationException ex = expectThrows(ActionRequestValidationException.class, bulkBuilder::get); assertThat(ex.validationErrors(), containsInAnyOrder( "index is missing", "index is missing", - "index is missing", - "type is missing", - "type is missing", - "type is missing")); + "index is missing")); } { @@ -123,7 +120,7 @@ public void testBulkWithGlobalDefaults() throws Exception { .routing("routing") .pipeline("pipeline"); - bulkBuilder.add(bulkAction.getBytes(StandardCharsets.UTF_8), 0, bulkAction.length(), null, null, XContentType.JSON); + bulkBuilder.add(bulkAction.getBytes(StandardCharsets.UTF_8), 0, bulkAction.length(), null, XContentType.JSON); BulkResponse bulkItemResponses = bulkBuilder.get(); assertFalse(bulkItemResponses.hasFailures()); } diff --git a/server/src/test/java/org/elasticsearch/action/bulk/BulkRequestTests.java b/server/src/test/java/org/elasticsearch/action/bulk/BulkRequestTests.java index 65935bea96e59..75701e0685290 100644 --- a/server/src/test/java/org/elasticsearch/action/bulk/BulkRequestTests.java +++ b/server/src/test/java/org/elasticsearch/action/bulk/BulkRequestTests.java @@ -34,6 +34,7 @@ import org.elasticsearch.common.xcontent.XContentFactory; import org.elasticsearch.common.xcontent.XContentHelper; import org.elasticsearch.common.xcontent.XContentType; +import org.elasticsearch.rest.action.document.RestBulkAction; import org.elasticsearch.script.Script; import org.elasticsearch.test.ESTestCase; @@ -57,42 +58,46 @@ public class BulkRequestTests extends ESTestCase { public void testSimpleBulk1() throws Exception { String bulkAction = copyToStringFromClasspath("/org/elasticsearch/action/bulk/simple-bulk.json"); BulkRequest bulkRequest = new BulkRequest(); - bulkRequest.add(bulkAction.getBytes(StandardCharsets.UTF_8), 0, bulkAction.length(), null, null, XContentType.JSON); + bulkRequest.add(bulkAction.getBytes(StandardCharsets.UTF_8), 0, bulkAction.length(), null, XContentType.JSON); assertThat(bulkRequest.numberOfActions(), equalTo(3)); assertThat(((IndexRequest) bulkRequest.requests().get(0)).source(), equalTo(new BytesArray("{ \"field1\" : \"value1\" }"))); assertThat(bulkRequest.requests().get(1), instanceOf(DeleteRequest.class)); assertThat(((IndexRequest) bulkRequest.requests().get(2)).source(), equalTo(new BytesArray("{ \"field1\" : \"value3\" }"))); + //This test's JSON contains outdated references to types + assertWarnings(RestBulkAction.TYPES_DEPRECATION_MESSAGE); } public void testSimpleBulkWithCarriageReturn() throws Exception { String bulkAction = "{ \"index\":{\"_index\":\"test\",\"_type\":\"type1\",\"_id\":\"1\"} }\r\n{ \"field1\" : \"value1\" }\r\n"; BulkRequest bulkRequest = new BulkRequest(); - bulkRequest.add(bulkAction.getBytes(StandardCharsets.UTF_8), 0, bulkAction.length(), null, null, XContentType.JSON); + bulkRequest.add(bulkAction.getBytes(StandardCharsets.UTF_8), 0, bulkAction.length(), null, XContentType.JSON); assertThat(bulkRequest.numberOfActions(), equalTo(1)); assertThat(((IndexRequest) bulkRequest.requests().get(0)).source(), equalTo(new BytesArray("{ \"field1\" : \"value1\" }"))); Map sourceMap = XContentHelper.convertToMap(((IndexRequest) bulkRequest.requests().get(0)).source(), false, XContentType.JSON).v2(); assertEquals("value1", sourceMap.get("field1")); + //This test's JSON contains outdated references to types + assertWarnings(RestBulkAction.TYPES_DEPRECATION_MESSAGE); } public void testSimpleBulk2() throws Exception { String bulkAction = copyToStringFromClasspath("/org/elasticsearch/action/bulk/simple-bulk2.json"); BulkRequest bulkRequest = new BulkRequest(); - bulkRequest.add(bulkAction.getBytes(StandardCharsets.UTF_8), 0, bulkAction.length(), null, null, XContentType.JSON); + bulkRequest.add(bulkAction.getBytes(StandardCharsets.UTF_8), 0, bulkAction.length(), null, XContentType.JSON); assertThat(bulkRequest.numberOfActions(), equalTo(3)); } public void testSimpleBulk3() throws Exception { String bulkAction = copyToStringFromClasspath("/org/elasticsearch/action/bulk/simple-bulk3.json"); BulkRequest bulkRequest = new BulkRequest(); - bulkRequest.add(bulkAction.getBytes(StandardCharsets.UTF_8), 0, bulkAction.length(), null, null, XContentType.JSON); + bulkRequest.add(bulkAction.getBytes(StandardCharsets.UTF_8), 0, bulkAction.length(), null, XContentType.JSON); assertThat(bulkRequest.numberOfActions(), equalTo(3)); } public void testSimpleBulk4() throws Exception { String bulkAction = copyToStringFromClasspath("/org/elasticsearch/action/bulk/simple-bulk4.json"); BulkRequest bulkRequest = new BulkRequest(); - bulkRequest.add(bulkAction.getBytes(StandardCharsets.UTF_8), 0, bulkAction.length(), null, null, XContentType.JSON); + bulkRequest.add(bulkAction.getBytes(StandardCharsets.UTF_8), 0, bulkAction.length(), null, XContentType.JSON); assertThat(bulkRequest.numberOfActions(), equalTo(4)); assertThat(bulkRequest.requests().get(0).id(), equalTo("1")); assertThat(((UpdateRequest) bulkRequest.requests().get(0)).retryOnConflict(), equalTo(2)); @@ -109,6 +114,8 @@ public void testSimpleBulk4() throws Exception { assertThat(scriptParams.size(), equalTo(1)); assertThat(scriptParams.get("param1"), equalTo(1)); assertThat(((UpdateRequest) bulkRequest.requests().get(1)).upsertRequest().source().utf8ToString(), equalTo("{\"counter\":1}")); + //This test's JSON contains outdated references to types + assertWarnings(RestBulkAction.TYPES_DEPRECATION_MESSAGE); } public void testBulkAllowExplicitIndex() throws Exception { @@ -120,6 +127,8 @@ public void testBulkAllowExplicitIndex() throws Exception { String bulkAction = copyToStringFromClasspath("/org/elasticsearch/action/bulk/simple-bulk5.json"); new BulkRequest().add(new BytesArray(bulkAction.getBytes(StandardCharsets.UTF_8)), "test", null, false, XContentType.JSON); + //This test's JSON contains outdated references to types + assertWarnings(RestBulkAction.TYPES_DEPRECATION_MESSAGE); } public void testBulkAddIterable() { @@ -139,32 +148,38 @@ public void testSimpleBulk6() throws Exception { String bulkAction = copyToStringFromClasspath("/org/elasticsearch/action/bulk/simple-bulk6.json"); BulkRequest bulkRequest = new BulkRequest(); ParsingException exc = expectThrows(ParsingException.class, - () -> bulkRequest.add(bulkAction.getBytes(StandardCharsets.UTF_8), 0, bulkAction.length(), null, null, XContentType.JSON)); + () -> bulkRequest.add(bulkAction.getBytes(StandardCharsets.UTF_8), 0, bulkAction.length(), null, XContentType.JSON)); assertThat(exc.getMessage(), containsString("Unknown key for a VALUE_STRING in [hello]")); + //This test's JSON contains outdated references to types + assertWarnings(RestBulkAction.TYPES_DEPRECATION_MESSAGE); } public void testSimpleBulk7() throws Exception { String bulkAction = copyToStringFromClasspath("/org/elasticsearch/action/bulk/simple-bulk7.json"); BulkRequest bulkRequest = new BulkRequest(); IllegalArgumentException exc = expectThrows(IllegalArgumentException.class, - () -> bulkRequest.add(bulkAction.getBytes(StandardCharsets.UTF_8), 0, bulkAction.length(), null, null, XContentType.JSON)); + () -> bulkRequest.add(bulkAction.getBytes(StandardCharsets.UTF_8), 0, bulkAction.length(), null, XContentType.JSON)); assertThat(exc.getMessage(), containsString("Malformed action/metadata line [5], expected a simple value for field [_unknown] but found [START_ARRAY]")); + //This test's JSON contains outdated references to types + assertWarnings(RestBulkAction.TYPES_DEPRECATION_MESSAGE); } public void testSimpleBulk8() throws Exception { String bulkAction = copyToStringFromClasspath("/org/elasticsearch/action/bulk/simple-bulk8.json"); BulkRequest bulkRequest = new BulkRequest(); IllegalArgumentException exc = expectThrows(IllegalArgumentException.class, - () -> bulkRequest.add(bulkAction.getBytes(StandardCharsets.UTF_8), 0, bulkAction.length(), null, null, XContentType.JSON)); + () -> bulkRequest.add(bulkAction.getBytes(StandardCharsets.UTF_8), 0, bulkAction.length(), null, XContentType.JSON)); assertThat(exc.getMessage(), containsString("Action/metadata line [3] contains an unknown parameter [_foo]")); + //This test's JSON contains outdated references to types + assertWarnings(RestBulkAction.TYPES_DEPRECATION_MESSAGE); } public void testSimpleBulk9() throws Exception { String bulkAction = copyToStringFromClasspath("/org/elasticsearch/action/bulk/simple-bulk9.json"); BulkRequest bulkRequest = new BulkRequest(); IllegalArgumentException exc = expectThrows(IllegalArgumentException.class, - () -> bulkRequest.add(bulkAction.getBytes(StandardCharsets.UTF_8), 0, bulkAction.length(), null, null, XContentType.JSON)); + () -> bulkRequest.add(bulkAction.getBytes(StandardCharsets.UTF_8), 0, bulkAction.length(), null, XContentType.JSON)); assertThat(exc.getMessage(), containsString("Malformed action/metadata line [3], expected START_OBJECT or END_OBJECT but found [START_ARRAY]")); } @@ -172,8 +187,10 @@ public void testSimpleBulk9() throws Exception { public void testSimpleBulk10() throws Exception { String bulkAction = copyToStringFromClasspath("/org/elasticsearch/action/bulk/simple-bulk10.json"); BulkRequest bulkRequest = new BulkRequest(); - bulkRequest.add(bulkAction.getBytes(StandardCharsets.UTF_8), 0, bulkAction.length(), null, null, XContentType.JSON); + bulkRequest.add(bulkAction.getBytes(StandardCharsets.UTF_8), 0, bulkAction.length(), null, XContentType.JSON); assertThat(bulkRequest.numberOfActions(), equalTo(9)); + //This test's JSON contains outdated references to types + assertWarnings(RestBulkAction.TYPES_DEPRECATION_MESSAGE); } public void testBulkActionShouldNotContainArray() throws Exception { @@ -181,13 +198,13 @@ public void testBulkActionShouldNotContainArray() throws Exception { + "{ \"field1\" : \"value1\" }\r\n"; BulkRequest bulkRequest = new BulkRequest(); IllegalArgumentException exc = expectThrows(IllegalArgumentException.class, - () -> bulkRequest.add(bulkAction.getBytes(StandardCharsets.UTF_8), 0, bulkAction.length(), null, null, XContentType.JSON)); + () -> bulkRequest.add(bulkAction.getBytes(StandardCharsets.UTF_8), 0, bulkAction.length(), null, XContentType.JSON)); assertEquals(exc.getMessage(), "Malformed action/metadata line [1]" + ", expected a simple value for field [_index] but found [START_ARRAY]"); } public void testBulkEmptyObject() throws Exception { - String bulkIndexAction = "{ \"index\":{\"_index\":\"test\",\"_type\":\"type1\",\"_id\":\"1\"} }\r\n"; + String bulkIndexAction = "{ \"index\":{\"_index\":\"test\",\"_id\":\"1\"} }\r\n"; String bulkIndexSource = "{ \"field1\" : \"value1\" }\r\n"; String emptyObject = "{}\r\n"; StringBuilder bulk = new StringBuilder(); @@ -207,7 +224,7 @@ public void testBulkEmptyObject() throws Exception { String bulkAction = bulk.toString(); BulkRequest bulkRequest = new BulkRequest(); IllegalArgumentException exc = expectThrows(IllegalArgumentException.class, - () -> bulkRequest.add(bulkAction.getBytes(StandardCharsets.UTF_8), 0, bulkAction.length(), null, null, XContentType.JSON)); + () -> bulkRequest.add(bulkAction.getBytes(StandardCharsets.UTF_8), 0, bulkAction.length(), null, XContentType.JSON)); assertThat(exc.getMessage(), containsString("Malformed action/metadata line [" + emptyLine + "], expected FIELD_NAME but found [END_OBJECT]")); } @@ -218,7 +235,7 @@ public void testBulkRequestWithRefresh() throws Exception { // We force here a "id is missing" validation error bulkRequest.add(new DeleteRequest("index", "type", null).setRefreshPolicy(RefreshPolicy.IMMEDIATE)); // We force here a "type is missing" validation error - bulkRequest.add(new DeleteRequest("index", null, "id")); + bulkRequest.add(new DeleteRequest("index", "", "id")); bulkRequest.add(new DeleteRequest("index", "type", "id").setRefreshPolicy(RefreshPolicy.IMMEDIATE)); bulkRequest.add(new UpdateRequest("index", "type", "id").doc("{}", XContentType.JSON).setRefreshPolicy(RefreshPolicy.IMMEDIATE)); bulkRequest.add(new IndexRequest("index", "type", "id").source("{}", XContentType.JSON).setRefreshPolicy(RefreshPolicy.IMMEDIATE)); @@ -290,6 +307,8 @@ public void testSmileIsSupported() throws IOException { IndexRequest request = (IndexRequest) docWriteRequest; assertEquals(1, request.sourceAsMap().size()); assertEquals("value", request.sourceAsMap().get("field")); + //This test's content contains outdated references to types + assertWarnings(RestBulkAction.TYPES_DEPRECATION_MESSAGE); } public void testToValidateUpsertRequestAndVersionInBulkRequest() throws IOException { @@ -324,18 +343,22 @@ public void testToValidateUpsertRequestAndVersionInBulkRequest() throws IOExcept bulkRequest.add(data, null, null, xContentType); assertThat(bulkRequest.validate().validationErrors(), contains("can't provide both upsert request and a version", "can't provide version in upsert request")); + //This test's JSON contains outdated references to types + assertWarnings(RestBulkAction.TYPES_DEPRECATION_MESSAGE); } public void testBulkTerminatedByNewline() throws Exception { String bulkAction = copyToStringFromClasspath("/org/elasticsearch/action/bulk/simple-bulk11.json"); IllegalArgumentException expectThrows = expectThrows(IllegalArgumentException.class, () -> new BulkRequest() - .add(bulkAction.getBytes(StandardCharsets.UTF_8), 0, bulkAction.length(), null, null, XContentType.JSON)); + .add(bulkAction.getBytes(StandardCharsets.UTF_8), 0, bulkAction.length(), null, XContentType.JSON)); assertEquals("The bulk request must be terminated by a newline [\n]", expectThrows.getMessage()); String bulkActionWithNewLine = bulkAction + "\n"; BulkRequest bulkRequestWithNewLine = new BulkRequest(); - bulkRequestWithNewLine.add(bulkActionWithNewLine.getBytes(StandardCharsets.UTF_8), 0, bulkActionWithNewLine.length(), null, null, + bulkRequestWithNewLine.add(bulkActionWithNewLine.getBytes(StandardCharsets.UTF_8), 0, bulkActionWithNewLine.length(), null, XContentType.JSON); assertEquals(3, bulkRequestWithNewLine.numberOfActions()); + //This test's JSON contains outdated references to types + assertWarnings(RestBulkAction.TYPES_DEPRECATION_MESSAGE); } } diff --git a/server/src/test/java/org/elasticsearch/action/bulk/TransportBulkActionTookTests.java b/server/src/test/java/org/elasticsearch/action/bulk/TransportBulkActionTookTests.java index 593bba9f377b9..89ccbefb00e59 100644 --- a/server/src/test/java/org/elasticsearch/action/bulk/TransportBulkActionTookTests.java +++ b/server/src/test/java/org/elasticsearch/action/bulk/TransportBulkActionTookTests.java @@ -38,6 +38,7 @@ import org.elasticsearch.common.util.concurrent.AtomicArray; import org.elasticsearch.common.xcontent.XContentType; import org.elasticsearch.index.IndexNotFoundException; +import org.elasticsearch.rest.action.document.RestBulkAction; import org.elasticsearch.tasks.Task; import org.elasticsearch.test.ESTestCase; import org.elasticsearch.test.transport.CapturingTransport; @@ -178,7 +179,7 @@ private void runTestTook(boolean controlled) throws Exception { bulkAction = Strings.replace(bulkAction, "\r\n", "\n"); } BulkRequest bulkRequest = new BulkRequest(); - bulkRequest.add(bulkAction.getBytes(StandardCharsets.UTF_8), 0, bulkAction.length(), null, null, XContentType.JSON); + bulkRequest.add(bulkAction.getBytes(StandardCharsets.UTF_8), 0, bulkAction.length(), null, XContentType.JSON); AtomicLong expected = new AtomicLong(); TransportBulkAction action = createAction(controlled, expected); action.doExecute(null, bulkRequest, new ActionListener() { @@ -200,6 +201,8 @@ public void onFailure(Exception e) { } }); + //This test's JSON contains outdated references to types + assertWarnings(RestBulkAction.TYPES_DEPRECATION_MESSAGE); } static class Resolver extends IndexNameExpressionResolver { diff --git a/server/src/test/java/org/elasticsearch/action/delete/DeleteRequestTests.java b/server/src/test/java/org/elasticsearch/action/delete/DeleteRequestTests.java index 5f897d0b8349b..4fe4c3548c224 100644 --- a/server/src/test/java/org/elasticsearch/action/delete/DeleteRequestTests.java +++ b/server/src/test/java/org/elasticsearch/action/delete/DeleteRequestTests.java @@ -36,11 +36,20 @@ public void testValidation() { } { - final DeleteRequest request = new DeleteRequest("index4", randomBoolean() ? "" : null, randomBoolean() ? "" : null); + //Empty types are accepted but fail validation + final DeleteRequest request = new DeleteRequest("index4", "", randomBoolean() ? "" : null); final ActionRequestValidationException validate = request.validate(); assertThat(validate, not(nullValue())); assertThat(validate.validationErrors(), hasItems("type is missing", "id is missing")); } + { + // Null types are defaulted + final DeleteRequest request = new DeleteRequest("index4", randomBoolean() ? "" : null); + final ActionRequestValidationException validate = request.validate(); + + assertThat(validate, not(nullValue())); + assertThat(validate.validationErrors(), hasItems("id is missing")); + } } } diff --git a/server/src/test/java/org/elasticsearch/action/update/UpdateRequestTests.java b/server/src/test/java/org/elasticsearch/action/update/UpdateRequestTests.java index e33c7ca5324ab..4534fbe23e3c2 100644 --- a/server/src/test/java/org/elasticsearch/action/update/UpdateRequestTests.java +++ b/server/src/test/java/org/elasticsearch/action/update/UpdateRequestTests.java @@ -524,9 +524,18 @@ public void testValidate() { assertThat(validate, nullValue()); } + { + // Null types are defaulted to "_doc" + UpdateRequest request = new UpdateRequest("index", null, randomBoolean() ? "" : null); + request.doc("{}", XContentType.JSON); + ActionRequestValidationException validate = request.validate(); + assertThat(validate, not(nullValue())); + assertThat(validate.validationErrors(), hasItems("id is missing")); + } { - UpdateRequest request = new UpdateRequest("index", randomBoolean() ? "" : null, randomBoolean() ? "" : null); + // Non-null types are accepted but fail validation + UpdateRequest request = new UpdateRequest("index", "", randomBoolean() ? "" : null); request.doc("{}", XContentType.JSON); ActionRequestValidationException validate = request.validate(); diff --git a/server/src/test/java/org/elasticsearch/rest/action/document/RestBulkActionTests.java b/server/src/test/java/org/elasticsearch/rest/action/document/RestBulkActionTests.java index 5011946914a50..b19c8c6412abc 100644 --- a/server/src/test/java/org/elasticsearch/rest/action/document/RestBulkActionTests.java +++ b/server/src/test/java/org/elasticsearch/rest/action/document/RestBulkActionTests.java @@ -51,7 +51,7 @@ public void testBulkPipelineUpsert() throws Exception { new RestBulkAction(settings(Version.CURRENT).build(), mock(RestController.class)) .handleRequest( new FakeRestRequest.Builder( - xContentRegistry()).withPath("my_index/my_type/_bulk").withParams(params) + xContentRegistry()).withPath("my_index/_bulk").withParams(params) .withContent( new BytesArray( "{\"index\":{\"_id\":\"1\"}}\n" + diff --git a/x-pack/plugin/ml/qa/native-multi-node-tests/src/test/java/org/elasticsearch/xpack/ml/integration/DatafeedJobsRestIT.java b/x-pack/plugin/ml/qa/native-multi-node-tests/src/test/java/org/elasticsearch/xpack/ml/integration/DatafeedJobsRestIT.java index d5aa2c6b4353f..64d8f73c4a3e5 100644 --- a/x-pack/plugin/ml/qa/native-multi-node-tests/src/test/java/org/elasticsearch/xpack/ml/integration/DatafeedJobsRestIT.java +++ b/x-pack/plugin/ml/qa/native-multi-node-tests/src/test/java/org/elasticsearch/xpack/ml/integration/DatafeedJobsRestIT.java @@ -13,6 +13,7 @@ import org.elasticsearch.client.RestClient; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.util.concurrent.ThreadContext; +import org.elasticsearch.rest.action.document.RestBulkAction; import org.elasticsearch.test.SecuritySettingsSourceField; import org.elasticsearch.test.rest.ESRestTestCase; import org.elasticsearch.xpack.core.ml.integration.MlRestTestStateCleaner; @@ -1196,6 +1197,7 @@ private void bulkIndex(String bulk) throws IOException { bulkRequest.setJsonEntity(bulk); bulkRequest.addParameter("refresh", "true"); bulkRequest.addParameter("pretty", null); + bulkRequest.setOptions(expectWarnings(RestBulkAction.TYPES_DEPRECATION_MESSAGE)); String bulkResponse = EntityUtils.toString(client().performRequest(bulkRequest).getEntity()); assertThat(bulkResponse, not(containsString("\"errors\": false"))); } diff --git a/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/job/process/autodetect/output/AutodetectStateProcessorTests.java b/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/job/process/autodetect/output/AutodetectStateProcessorTests.java index e4fb5a7f07456..4f5477a75f863 100644 --- a/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/job/process/autodetect/output/AutodetectStateProcessorTests.java +++ b/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/job/process/autodetect/output/AutodetectStateProcessorTests.java @@ -39,13 +39,13 @@ public class AutodetectStateProcessorTests extends ESTestCase { private static final String STATE_SAMPLE = "" - + "{\"index\": {\"_index\": \"test\", \"_type\": \"type1\", \"_id\": \"1\"}}\n" + + "{\"index\": {\"_index\": \"test\", \"_id\": \"1\"}}\n" + "{ \"field\" : \"value1\" }\n" + "\0" - + "{\"index\": {\"_index\": \"test\", \"_type\": \"type1\", \"_id\": \"2\"}}\n" + + "{\"index\": {\"_index\": \"test\", \"_id\": \"2\"}}\n" + "{ \"field\" : \"value2\" }\n" + "\0" - + "{\"index\": {\"_index\": \"test\", \"_type\": \"type1\", \"_id\": \"3\"}}\n" + + "{\"index\": {\"_index\": \"test\", \"_id\": \"3\"}}\n" + "{ \"field\" : \"value3\" }\n" + "\0"; @@ -118,7 +118,7 @@ public void testStateReadGivenConsecutiveSpacesFollowedByZeroByte() throws IOExc public void testLargeStateRead() throws Exception { StringBuilder builder = new StringBuilder(NUM_LARGE_DOCS * (LARGE_DOC_SIZE + 10)); // 10 for header and separators for (int docNum = 1; docNum <= NUM_LARGE_DOCS; ++docNum) { - builder.append("{\"index\":{\"_index\":\"header").append(docNum).append("\",\"_type\":\"type\"}}\n"); + builder.append("{\"index\":{\"_index\":\"header").append(docNum).append("\"}}\n"); for (int count = 0; count < (LARGE_DOC_SIZE / "data".length()); ++count) { builder.append("data"); } diff --git a/x-pack/plugin/monitoring/src/test/java/org/elasticsearch/xpack/monitoring/action/MonitoringBulkRequestTests.java b/x-pack/plugin/monitoring/src/test/java/org/elasticsearch/xpack/monitoring/action/MonitoringBulkRequestTests.java index dc5cad7c94fd4..a8ab2960194f2 100644 --- a/x-pack/plugin/monitoring/src/test/java/org/elasticsearch/xpack/monitoring/action/MonitoringBulkRequestTests.java +++ b/x-pack/plugin/monitoring/src/test/java/org/elasticsearch/xpack/monitoring/action/MonitoringBulkRequestTests.java @@ -14,6 +14,7 @@ import org.elasticsearch.common.xcontent.XContentBuilder; import org.elasticsearch.common.xcontent.XContentFactory; import org.elasticsearch.common.xcontent.XContentType; +import org.elasticsearch.rest.action.document.RestBulkAction; import org.elasticsearch.test.ESTestCase; import org.elasticsearch.test.RandomObjects; import org.elasticsearch.xpack.core.monitoring.MonitoredSystem; @@ -141,6 +142,8 @@ public void testAddRequestContent() throws IOException { assertThat(bulkDoc.getXContentType(), equalTo(xContentType)); ++count; } + //This test's JSON contains outdated references to types + assertWarnings(RestBulkAction.TYPES_DEPRECATION_MESSAGE); } public void testAddRequestContentWithEmptySource() throws IOException { @@ -188,6 +191,8 @@ public void testAddRequestContentWithEmptySource() throws IOException { ); assertThat(e.getMessage(), containsString("source is missing for monitoring document [][doc][" + nbDocs + "]")); + //This test's JSON contains outdated references to types + assertWarnings(RestBulkAction.TYPES_DEPRECATION_MESSAGE); } public void testAddRequestContentWithUnrecognizedIndexName() throws IOException { @@ -225,6 +230,9 @@ public void testAddRequestContentWithUnrecognizedIndexName() throws IOException ); assertThat(e.getMessage(), containsString("unrecognized index name [" + indexName + "]")); + //This test's JSON contains outdated references to types + assertWarnings(RestBulkAction.TYPES_DEPRECATION_MESSAGE); + } public void testSerialization() throws IOException { diff --git a/x-pack/plugin/monitoring/src/test/java/org/elasticsearch/xpack/monitoring/integration/MonitoringIT.java b/x-pack/plugin/monitoring/src/test/java/org/elasticsearch/xpack/monitoring/integration/MonitoringIT.java index dd45c3e06db5d..555f2659113fd 100644 --- a/x-pack/plugin/monitoring/src/test/java/org/elasticsearch/xpack/monitoring/integration/MonitoringIT.java +++ b/x-pack/plugin/monitoring/src/test/java/org/elasticsearch/xpack/monitoring/integration/MonitoringIT.java @@ -26,6 +26,7 @@ import org.elasticsearch.license.License; import org.elasticsearch.plugins.Plugin; import org.elasticsearch.rest.RestStatus; +import org.elasticsearch.rest.action.document.RestBulkAction; import org.elasticsearch.search.SearchHit; import org.elasticsearch.search.SearchHits; import org.elasticsearch.search.collapse.CollapseBuilder; @@ -180,6 +181,7 @@ public void testMonitoringBulk() throws Exception { assertMonitoringDoc(toMap(hit), system, "test", interval); } }); + assertWarnings(RestBulkAction.TYPES_DEPRECATION_MESSAGE); } /** diff --git a/x-pack/plugin/monitoring/src/test/java/org/elasticsearch/xpack/monitoring/rest/action/RestMonitoringBulkActionTests.java b/x-pack/plugin/monitoring/src/test/java/org/elasticsearch/xpack/monitoring/rest/action/RestMonitoringBulkActionTests.java index 7a4427c9f0fdc..10fc10e3f973d 100644 --- a/x-pack/plugin/monitoring/src/test/java/org/elasticsearch/xpack/monitoring/rest/action/RestMonitoringBulkActionTests.java +++ b/x-pack/plugin/monitoring/src/test/java/org/elasticsearch/xpack/monitoring/rest/action/RestMonitoringBulkActionTests.java @@ -23,6 +23,7 @@ import org.elasticsearch.rest.RestResponse; import org.elasticsearch.rest.RestStatus; import org.elasticsearch.rest.action.RestBuilderListener; +import org.elasticsearch.rest.action.document.RestBulkAction; import org.elasticsearch.test.ESTestCase; import org.elasticsearch.test.rest.FakeRestRequest; import org.elasticsearch.xpack.core.XPackClient; @@ -121,6 +122,8 @@ public void testNoErrors() throws Exception { assertThat(restResponse.status(), is(RestStatus.OK)); assertThat(restResponse.content().utf8ToString(), is("{\"took\":" + response.getTookInMillis() + ",\"ignored\":false,\"errors\":false}")); + //This test's JSON contains outdated references to types + assertWarnings(RestBulkAction.TYPES_DEPRECATION_MESSAGE); } public void testNoErrorsButIgnored() throws Exception { @@ -131,6 +134,8 @@ public void testNoErrorsButIgnored() throws Exception { assertThat(restResponse.status(), is(RestStatus.OK)); assertThat(restResponse.content().utf8ToString(), is("{\"took\":" + response.getTookInMillis() + ",\"ignored\":true,\"errors\":false}")); + //This test's JSON contains outdated references to types + assertWarnings(RestBulkAction.TYPES_DEPRECATION_MESSAGE); } public void testWithErrors() throws Exception { @@ -150,6 +155,8 @@ public void testWithErrors() throws Exception { assertThat(restResponse.status(), is(RestStatus.INTERNAL_SERVER_ERROR)); assertThat(restResponse.content().utf8ToString(), is("{\"took\":" + response.getTookInMillis() + ",\"ignored\":false,\"errors\":true,\"error\":" + errorJson + "}")); + //This test's JSON contains outdated references to types + assertWarnings(RestBulkAction.TYPES_DEPRECATION_MESSAGE); } /** diff --git a/x-pack/plugin/sql/qa/multi-node/src/test/java/org/elasticsearch/xpack/sql/qa/multi_node/RestSqlMultinodeIT.java b/x-pack/plugin/sql/qa/multi-node/src/test/java/org/elasticsearch/xpack/sql/qa/multi_node/RestSqlMultinodeIT.java index 2a4dd79001d1c..95066306cc2c2 100644 --- a/x-pack/plugin/sql/qa/multi-node/src/test/java/org/elasticsearch/xpack/sql/qa/multi_node/RestSqlMultinodeIT.java +++ b/x-pack/plugin/sql/qa/multi-node/src/test/java/org/elasticsearch/xpack/sql/qa/multi_node/RestSqlMultinodeIT.java @@ -83,7 +83,7 @@ public void testIndexOnWrongNode() throws IOException { } private void createTestData(int documents) throws UnsupportedCharsetException, IOException { - Request request = new Request("PUT", "/test/test/_bulk"); + Request request = new Request("PUT", "/test/_bulk"); request.addParameter("refresh", "true"); StringBuilder bulk = new StringBuilder(); diff --git a/x-pack/plugin/sql/qa/security/src/test/java/org/elasticsearch/xpack/sql/qa/security/SqlSecurityTestCase.java b/x-pack/plugin/sql/qa/security/src/test/java/org/elasticsearch/xpack/sql/qa/security/SqlSecurityTestCase.java index 1dc765e242a2e..053fe0c4775dc 100644 --- a/x-pack/plugin/sql/qa/security/src/test/java/org/elasticsearch/xpack/sql/qa/security/SqlSecurityTestCase.java +++ b/x-pack/plugin/sql/qa/security/src/test/java/org/elasticsearch/xpack/sql/qa/security/SqlSecurityTestCase.java @@ -141,11 +141,11 @@ public void oneTimeSetup() throws Exception { request.addParameter("refresh", "true"); StringBuilder bulk = new StringBuilder(); - bulk.append("{\"index\":{\"_index\": \"test\", \"_type\": \"doc\", \"_id\":\"1\"}\n"); + bulk.append("{\"index\":{\"_index\": \"test\", \"_id\":\"1\"}\n"); bulk.append("{\"a\": 1, \"b\": 2, \"c\": 3}\n"); - bulk.append("{\"index\":{\"_index\": \"test\", \"_type\": \"doc\", \"_id\":\"2\"}\n"); + bulk.append("{\"index\":{\"_index\": \"test\", \"_id\":\"2\"}\n"); bulk.append("{\"a\": 4, \"b\": 5, \"c\": 6}\n"); - bulk.append("{\"index\":{\"_index\": \"bort\", \"_type\": \"doc\", \"_id\":\"1\"}\n"); + bulk.append("{\"index\":{\"_index\": \"bort\", \"_id\":\"1\"}\n"); bulk.append("{\"a\": \"test\"}\n"); request.setJsonEntity(bulk.toString()); client().performRequest(request); diff --git a/x-pack/plugin/sql/qa/security/src/test/java/org/elasticsearch/xpack/sql/qa/security/UserFunctionIT.java b/x-pack/plugin/sql/qa/security/src/test/java/org/elasticsearch/xpack/sql/qa/security/UserFunctionIT.java index 5d37e38bc4cd6..639ffd17e34f3 100644 --- a/x-pack/plugin/sql/qa/security/src/test/java/org/elasticsearch/xpack/sql/qa/security/UserFunctionIT.java +++ b/x-pack/plugin/sql/qa/security/src/test/java/org/elasticsearch/xpack/sql/qa/security/UserFunctionIT.java @@ -203,7 +203,7 @@ private static Map toMap(Response response) throws IOException { } private void index(String... docs) throws IOException { - Request request = new Request("POST", "/test/test/_bulk"); + Request request = new Request("POST", "/test/_bulk"); request.addParameter("refresh", "true"); StringBuilder bulk = new StringBuilder(); for (String doc : docs) { diff --git a/x-pack/plugin/sql/qa/single-node/src/test/java/org/elasticsearch/xpack/sql/qa/single_node/JdbcShardFailureIT.java b/x-pack/plugin/sql/qa/single-node/src/test/java/org/elasticsearch/xpack/sql/qa/single_node/JdbcShardFailureIT.java index 487dc5f717348..f7c490cc8b490 100644 --- a/x-pack/plugin/sql/qa/single-node/src/test/java/org/elasticsearch/xpack/sql/qa/single_node/JdbcShardFailureIT.java +++ b/x-pack/plugin/sql/qa/single-node/src/test/java/org/elasticsearch/xpack/sql/qa/single_node/JdbcShardFailureIT.java @@ -31,7 +31,7 @@ public void createTestIndex() throws IOException { createTest2.addParameter("timeout", "100ms"); client().performRequest(createTest2); - Request request = new Request("PUT", "/test1/doc/_bulk"); + Request request = new Request("PUT", "/test1/_bulk"); request.addParameter("refresh", "true"); StringBuilder bulk = new StringBuilder(); for (int i = 0; i < 20; i++) { diff --git a/x-pack/plugin/sql/qa/src/main/java/org/elasticsearch/xpack/sql/qa/cli/FetchSizeTestCase.java b/x-pack/plugin/sql/qa/src/main/java/org/elasticsearch/xpack/sql/qa/cli/FetchSizeTestCase.java index afc4c302995d5..84f74bcbac137 100644 --- a/x-pack/plugin/sql/qa/src/main/java/org/elasticsearch/xpack/sql/qa/cli/FetchSizeTestCase.java +++ b/x-pack/plugin/sql/qa/src/main/java/org/elasticsearch/xpack/sql/qa/cli/FetchSizeTestCase.java @@ -16,7 +16,7 @@ */ public abstract class FetchSizeTestCase extends CliIntegrationTestCase { public void testSelect() throws IOException { - Request request = new Request("PUT", "/test/doc/_bulk"); + Request request = new Request("PUT", "/test/_bulk"); request.addParameter("refresh", "true"); StringBuilder bulk = new StringBuilder(); for (int i = 0; i < 20; i++) { diff --git a/x-pack/plugin/sql/qa/src/main/java/org/elasticsearch/xpack/sql/qa/jdbc/DataLoader.java b/x-pack/plugin/sql/qa/src/main/java/org/elasticsearch/xpack/sql/qa/jdbc/DataLoader.java index 4985dda404dd7..efa39ec517eb2 100644 --- a/x-pack/plugin/sql/qa/src/main/java/org/elasticsearch/xpack/sql/qa/jdbc/DataLoader.java +++ b/x-pack/plugin/sql/qa/src/main/java/org/elasticsearch/xpack/sql/qa/jdbc/DataLoader.java @@ -83,7 +83,7 @@ private static void loadEmpDatasetIntoEs(RestClient client, String index, String createIndex.endObject(); createIndex.startObject("mappings"); { - createIndex.startObject("emp"); + createIndex.startObject("_doc"); { createIndex.startObject("properties"); { @@ -157,7 +157,7 @@ private static void loadEmpDatasetIntoEs(RestClient client, String index, String list.add(dep); }); - request = new Request("POST", "/" + index + "/emp/_bulk?refresh=wait_for"); + request = new Request("POST", "/" + index + "/_bulk?refresh=wait_for"); request.addParameter("refresh", "true"); StringBuilder bulk = new StringBuilder(); csvToLines(fileName, (titles, fields) -> { @@ -232,7 +232,7 @@ protected static void loadLogsDatasetIntoEs(RestClient client, String index, Str request.setJsonEntity(Strings.toString(createIndex)); client.performRequest(request); - request = new Request("POST", "/" + index + "/_doc/_bulk?refresh=wait_for"); + request = new Request("POST", "/" + index + "/_bulk?refresh=wait_for"); request.addParameter("refresh", "true"); StringBuilder bulk = new StringBuilder(); csvToLines(filename, (titles, fields) -> { @@ -263,7 +263,7 @@ protected static void loadLibDatasetIntoEs(RestClient client, String index) thro createIndex.endObject(); createIndex.startObject("mappings"); { - createIndex.startObject("book"); + createIndex.startObject("_doc"); { createIndex.startObject("properties"); { @@ -280,7 +280,7 @@ protected static void loadLibDatasetIntoEs(RestClient client, String index) thro request.setJsonEntity(Strings.toString(createIndex)); client.performRequest(request); - request = new Request("POST", "/" + index + "/book/_bulk?refresh=wait_for"); + request = new Request("POST", "/" + index + "/_bulk?refresh=wait_for"); request.addParameter("refresh", "true"); StringBuilder bulk = new StringBuilder(); csvToLines("library", (titles, fields) -> { diff --git a/x-pack/plugin/sql/qa/src/main/java/org/elasticsearch/xpack/sql/qa/jdbc/FetchSizeTestCase.java b/x-pack/plugin/sql/qa/src/main/java/org/elasticsearch/xpack/sql/qa/jdbc/FetchSizeTestCase.java index fee5901bc4cb2..86fa4805ba6ea 100644 --- a/x-pack/plugin/sql/qa/src/main/java/org/elasticsearch/xpack/sql/qa/jdbc/FetchSizeTestCase.java +++ b/x-pack/plugin/sql/qa/src/main/java/org/elasticsearch/xpack/sql/qa/jdbc/FetchSizeTestCase.java @@ -30,7 +30,7 @@ public void createTestIndex() throws IOException { XContentBuilder createIndex = JsonXContent.contentBuilder().startObject(); createIndex.startObject("mappings"); { - createIndex.startObject("doc"); + createIndex.startObject("_doc"); { createIndex.startObject("properties"); { @@ -48,7 +48,7 @@ public void createTestIndex() throws IOException { request.setJsonEntity(Strings.toString(createIndex)); client().performRequest(request); - request = new Request("PUT", "/test/doc/_bulk"); + request = new Request("PUT", "/test/_bulk"); request.addParameter("refresh", "true"); StringBuilder bulk = new StringBuilder(); StringBuilder bulkLine; diff --git a/x-pack/plugin/sql/qa/src/main/java/org/elasticsearch/xpack/sql/qa/rest/RestSqlTestCase.java b/x-pack/plugin/sql/qa/src/main/java/org/elasticsearch/xpack/sql/qa/rest/RestSqlTestCase.java index a1d284304b573..21faaf17c4ea1 100644 --- a/x-pack/plugin/sql/qa/src/main/java/org/elasticsearch/xpack/sql/qa/rest/RestSqlTestCase.java +++ b/x-pack/plugin/sql/qa/src/main/java/org/elasticsearch/xpack/sql/qa/rest/RestSqlTestCase.java @@ -75,7 +75,7 @@ public void testBasicQuery() throws IOException { } public void testNextPage() throws IOException { - Request request = new Request("POST", "/test/test/_bulk"); + Request request = new Request("POST", "/test/_bulk"); request.addParameter("refresh", "true"); String mode = randomMode(); StringBuilder bulk = new StringBuilder(); @@ -141,7 +141,7 @@ public void testTimeZone() throws IOException { } public void testScoreWithFieldNamedScore() throws IOException { - Request request = new Request("POST", "/test/test/_bulk"); + Request request = new Request("POST", "/test/_bulk"); request.addParameter("refresh", "true"); String mode = randomMode(); StringBuilder bulk = new StringBuilder(); @@ -719,7 +719,7 @@ public static String mode(String mode) { } protected void index(String... docs) throws IOException { - Request request = new Request("POST", "/test/_doc/_bulk"); + Request request = new Request("POST", "/test/_bulk"); request.addParameter("refresh", "true"); StringBuilder bulk = new StringBuilder(); for (String doc : docs) { diff --git a/x-pack/qa/multi-node/src/test/java/org/elasticsearch/multi_node/RollupIT.java b/x-pack/qa/multi-node/src/test/java/org/elasticsearch/multi_node/RollupIT.java index a7dd57bea89ed..1feb05b9c6468 100644 --- a/x-pack/qa/multi-node/src/test/java/org/elasticsearch/multi_node/RollupIT.java +++ b/x-pack/qa/multi-node/src/test/java/org/elasticsearch/multi_node/RollupIT.java @@ -92,7 +92,7 @@ public void testBigRollup() throws Exception { // index documents for the rollup job final StringBuilder bulk = new StringBuilder(); for (int i = 0; i < numDocs; i++) { - bulk.append("{\"index\":{\"_index\":\"rollup-docs\",\"_type\":\"_doc\"}}\n"); + bulk.append("{\"index\":{\"_index\":\"rollup-docs\"}}\n"); ZonedDateTime zdt = ZonedDateTime.ofInstant(Instant.ofEpochSecond(1531221196 + (60*i)), ZoneId.of("UTC")); String date = zdt.format(DateTimeFormatter.ISO_OFFSET_DATE_TIME); bulk.append("{\"timestamp\":\"").append(date).append("\",\"value\":").append(i).append("}\n"); diff --git a/x-pack/qa/rolling-upgrade/src/test/java/org/elasticsearch/upgrades/IndexingIT.java b/x-pack/qa/rolling-upgrade/src/test/java/org/elasticsearch/upgrades/IndexingIT.java index b080a77e0646d..8d8d999a55b8d 100644 --- a/x-pack/qa/rolling-upgrade/src/test/java/org/elasticsearch/upgrades/IndexingIT.java +++ b/x-pack/qa/rolling-upgrade/src/test/java/org/elasticsearch/upgrades/IndexingIT.java @@ -9,6 +9,7 @@ import org.elasticsearch.common.Booleans; import org.elasticsearch.common.Strings; import org.elasticsearch.common.xcontent.XContentBuilder; +import org.elasticsearch.rest.action.document.RestBulkAction; import org.elasticsearch.Version; import org.elasticsearch.client.Request; import org.elasticsearch.client.Response; @@ -137,6 +138,7 @@ private void bulk(String index, String valueSuffix, int count) throws IOExceptio } Request bulk = new Request("POST", "/_bulk"); bulk.addParameter("refresh", "true"); + bulk.setOptions(expectWarnings(RestBulkAction.TYPES_DEPRECATION_MESSAGE)); bulk.setJsonEntity(b.toString()); client().performRequest(bulk); } From b5b93a27464ab2fae2f2bf817bf64cd8102eba9e Mon Sep 17 00:00:00 2001 From: Jack Conradson Date: Thu, 10 Jan 2019 13:58:00 -0800 Subject: [PATCH 091/186] Rename ParameterMap to DeprecationMap (#37317) Mechanical change to rename ParameterMap to DeprecationMap as this seems more appropriate for an extended Map to issue deprecation warnings. --- .../script/AbstractSortScript.java | 2 +- .../script/AggregationScript.java | 2 +- ...{ParameterMap.java => DeprecationMap.java} | 34 +++++++++---------- .../org/elasticsearch/script/FieldScript.java | 2 +- .../org/elasticsearch/script/ScoreScript.java | 2 +- .../script/ScriptedMetricAggContexts.java | 2 +- .../script/TermsSetQueryScript.java | 2 +- 7 files changed, 23 insertions(+), 23 deletions(-) rename server/src/main/java/org/elasticsearch/script/{ParameterMap.java => DeprecationMap.java} (74%) diff --git a/server/src/main/java/org/elasticsearch/script/AbstractSortScript.java b/server/src/main/java/org/elasticsearch/script/AbstractSortScript.java index 1d8de9f95f40a..949ca5bdb7f0f 100644 --- a/server/src/main/java/org/elasticsearch/script/AbstractSortScript.java +++ b/server/src/main/java/org/elasticsearch/script/AbstractSortScript.java @@ -66,7 +66,7 @@ abstract class AbstractSortScript implements ScorerAware { this.leafLookup = lookup.getLeafSearchLookup(leafContext); Map parameters = new HashMap<>(params); parameters.putAll(leafLookup.asMap()); - this.params = new ParameterMap(parameters, DEPRECATIONS); + this.params = new DeprecationMap(parameters, DEPRECATIONS); } protected AbstractSortScript() { diff --git a/server/src/main/java/org/elasticsearch/script/AggregationScript.java b/server/src/main/java/org/elasticsearch/script/AggregationScript.java index 8e1b485816618..5d36b91785f8a 100644 --- a/server/src/main/java/org/elasticsearch/script/AggregationScript.java +++ b/server/src/main/java/org/elasticsearch/script/AggregationScript.java @@ -71,7 +71,7 @@ public abstract class AggregationScript implements ScorerAware { private Object value; public AggregationScript(Map params, SearchLookup lookup, LeafReaderContext leafContext) { - this.params = new ParameterMap(new HashMap<>(params), DEPRECATIONS); + this.params = new DeprecationMap(new HashMap<>(params), DEPRECATIONS); this.leafLookup = lookup.getLeafSearchLookup(leafContext); this.params.putAll(leafLookup.asMap()); } diff --git a/server/src/main/java/org/elasticsearch/script/ParameterMap.java b/server/src/main/java/org/elasticsearch/script/DeprecationMap.java similarity index 74% rename from server/src/main/java/org/elasticsearch/script/ParameterMap.java rename to server/src/main/java/org/elasticsearch/script/DeprecationMap.java index b40c0f9b401d3..5b14e2e3b1168 100644 --- a/server/src/main/java/org/elasticsearch/script/ParameterMap.java +++ b/server/src/main/java/org/elasticsearch/script/DeprecationMap.java @@ -26,38 +26,38 @@ import java.util.Map; import java.util.Set; -public final class ParameterMap implements Map { +public final class DeprecationMap implements Map { private static final DeprecationLogger deprecationLogger = - new DeprecationLogger(LogManager.getLogger(ParameterMap.class)); + new DeprecationLogger(LogManager.getLogger(DeprecationMap.class)); - private final Map params; + private final Map delegate; private final Map deprecations; - public ParameterMap(Map params, Map deprecations) { - this.params = params; + public DeprecationMap(Map delegate, Map deprecations) { + this.delegate = delegate; this.deprecations = deprecations; } @Override public int size() { - return params.size(); + return delegate.size(); } @Override public boolean isEmpty() { - return params.isEmpty(); + return delegate.isEmpty(); } @Override public boolean containsKey(final Object key) { - return params.containsKey(key); + return delegate.containsKey(key); } @Override public boolean containsValue(final Object value) { - return params.containsValue(value); + return delegate.containsValue(value); } @Override @@ -66,41 +66,41 @@ public Object get(final Object key) { if (deprecationMessage != null) { deprecationLogger.deprecated(deprecationMessage); } - return params.get(key); + return delegate.get(key); } @Override public Object put(final String key, final Object value) { - return params.put(key, value); + return delegate.put(key, value); } @Override public Object remove(final Object key) { - return params.remove(key); + return delegate.remove(key); } @Override public void putAll(final Map m) { - params.putAll(m); + delegate.putAll(m); } @Override public void clear() { - params.clear(); + delegate.clear(); } @Override public Set keySet() { - return params.keySet(); + return delegate.keySet(); } @Override public Collection values() { - return params.values(); + return delegate.values(); } @Override public Set> entrySet() { - return params.entrySet(); + return delegate.entrySet(); } } diff --git a/server/src/main/java/org/elasticsearch/script/FieldScript.java b/server/src/main/java/org/elasticsearch/script/FieldScript.java index 29684a6447776..4f0e3c7229037 100644 --- a/server/src/main/java/org/elasticsearch/script/FieldScript.java +++ b/server/src/main/java/org/elasticsearch/script/FieldScript.java @@ -63,7 +63,7 @@ public FieldScript(Map params, SearchLookup lookup, LeafReaderCo this.leafLookup = lookup.getLeafSearchLookup(leafContext); params = new HashMap<>(params); params.putAll(leafLookup.asMap()); - this.params = new ParameterMap(params, DEPRECATIONS); + this.params = new DeprecationMap(params, DEPRECATIONS); } // for expression engine diff --git a/server/src/main/java/org/elasticsearch/script/ScoreScript.java b/server/src/main/java/org/elasticsearch/script/ScoreScript.java index c88c68fd407a2..78d206888e998 100644 --- a/server/src/main/java/org/elasticsearch/script/ScoreScript.java +++ b/server/src/main/java/org/elasticsearch/script/ScoreScript.java @@ -73,7 +73,7 @@ public ScoreScript(Map params, SearchLookup lookup, LeafReaderCo this.leafLookup = lookup.getLeafSearchLookup(leafContext); params = new HashMap<>(params); params.putAll(leafLookup.asMap()); - this.params = new ParameterMap(params, DEPRECATIONS); + this.params = new DeprecationMap(params, DEPRECATIONS); } } diff --git a/server/src/main/java/org/elasticsearch/script/ScriptedMetricAggContexts.java b/server/src/main/java/org/elasticsearch/script/ScriptedMetricAggContexts.java index 4c51b9fed69ec..a17503a026f24 100644 --- a/server/src/main/java/org/elasticsearch/script/ScriptedMetricAggContexts.java +++ b/server/src/main/java/org/elasticsearch/script/ScriptedMetricAggContexts.java @@ -95,7 +95,7 @@ public MapScript(Map params, Map state, SearchLo if (leafLookup != null) { params = new HashMap<>(params); // copy params so we aren't modifying input params.putAll(leafLookup.asMap()); // add lookup vars - params = new ParameterMap(params, DEPRECATIONS); // wrap with deprecations + params = new DeprecationMap(params, DEPRECATIONS); // wrap with deprecations } this.params = params; } diff --git a/server/src/main/java/org/elasticsearch/script/TermsSetQueryScript.java b/server/src/main/java/org/elasticsearch/script/TermsSetQueryScript.java index 122e3defe7586..9462b6a8e811a 100644 --- a/server/src/main/java/org/elasticsearch/script/TermsSetQueryScript.java +++ b/server/src/main/java/org/elasticsearch/script/TermsSetQueryScript.java @@ -64,7 +64,7 @@ public TermsSetQueryScript(Map params, SearchLookup lookup, Leaf Map parameters = new HashMap<>(params); this.leafLookup = lookup.getLeafSearchLookup(leafContext); parameters.putAll(leafLookup.asMap()); - this.params = new ParameterMap(parameters, DEPRECATIONS); + this.params = new DeprecationMap(parameters, DEPRECATIONS); } protected TermsSetQueryScript() { From 9fd4d6ba4be313e722f475b6d72e7d2c3ceb2cbd Mon Sep 17 00:00:00 2001 From: Jake Landis Date: Thu, 10 Jan 2019 17:06:44 -0600 Subject: [PATCH 092/186] remove `+` references for xpack.monitoring.collection.indices (#37282) `+` for index name inclusions is no longer supported for 6.x+. This commit removes references of the `+` from the documenation. System indices additional example is also included. fixes #37237 --- .../monitoring/collecting-monitoring-data.asciidoc | 6 ++++-- docs/reference/settings/monitoring-settings.asciidoc | 7 ++++--- 2 files changed, 8 insertions(+), 5 deletions(-) diff --git a/docs/reference/monitoring/collecting-monitoring-data.asciidoc b/docs/reference/monitoring/collecting-monitoring-data.asciidoc index 432b5f4d01f4c..8ab0443edb167 100644 --- a/docs/reference/monitoring/collecting-monitoring-data.asciidoc +++ b/docs/reference/monitoring/collecting-monitoring-data.asciidoc @@ -88,9 +88,11 @@ example: xpack.monitoring.collection.indices: logstash-*, index1, test2 ---------------------------------- -You can prepend `+` or `-` to explicitly include or exclude index names or +You can prepend `-` to explicitly exclude index names or patterns. For example, to include all indices that start with `test` except -`test3`, you could specify `+test*,-test3`. +`test3`, you could specify `test*,-test3`. To include system indices such as +.security and .kibana, add `.*` to the list of included names. +For example `.*,test*,-test3` -- .. Optional: Specify how often to collect monitoring data. The default value for diff --git a/docs/reference/settings/monitoring-settings.asciidoc b/docs/reference/settings/monitoring-settings.asciidoc index 56bfaf5d4f863..59a766d4dd0ca 100644 --- a/docs/reference/settings/monitoring-settings.asciidoc +++ b/docs/reference/settings/monitoring-settings.asciidoc @@ -76,9 +76,10 @@ Sets the timeout for collecting the cluster statistics. Defaults to `10s`. Controls which indices Monitoring collects data from. Defaults to all indices. Specify the index names as a comma-separated list, for example `test1,test2,test3`. Names can include wildcards, for -example `test*`. You can explicitly include or exclude indices by prepending -`+` to include the index, or `-` to exclude the index. For example, to include all indices that -start with `test` except `test3`, you could specify `+test*,-test3`. +example `test*`. You can explicitly exclude indices by prepending `-`. For example `test*,-test3` will +monitor all indexes that start with `test` except for `test3`. System indices like .security* or .kibana* +always start with a `.`, and generally should be monitored. Consider adding `.*` to the list of indices +ensure monitoring of system indices. For example `.*,test*,-test3` `xpack.monitoring.collection.index.stats.timeout`:: From edc95c8a8e1cb0ab890b81894c378ac3f2fd2ab9 Mon Sep 17 00:00:00 2001 From: Jason Tedor Date: Thu, 10 Jan 2019 18:13:05 -0800 Subject: [PATCH 093/186] Add validation for retention lease construction (#37312) This commit adds some simple validation that the values input to the retention lease constructor our valid values. We will later rely on these values being within the validated range. --- .../index/seqno/RetentionLease.java | 6 +++ .../index/seqno/RetentionLeaseTests.java | 50 +++++++++++++++++++ 2 files changed, 56 insertions(+) create mode 100644 server/src/test/java/org/elasticsearch/index/seqno/RetentionLeaseTests.java diff --git a/server/src/main/java/org/elasticsearch/index/seqno/RetentionLease.java b/server/src/main/java/org/elasticsearch/index/seqno/RetentionLease.java index 2881aac73c2c3..076b707a5df42 100644 --- a/server/src/main/java/org/elasticsearch/index/seqno/RetentionLease.java +++ b/server/src/main/java/org/elasticsearch/index/seqno/RetentionLease.java @@ -81,6 +81,12 @@ public String source() { * @param source the source of the retention lease */ public RetentionLease(final String id, final long retainingSequenceNumber, final long timestamp, final String source) { + if (retainingSequenceNumber < SequenceNumbers.UNASSIGNED_SEQ_NO) { + throw new IllegalArgumentException("retention lease retaining sequence number [" + retainingSequenceNumber + "] out of range"); + } + if (timestamp < 0) { + throw new IllegalArgumentException("retention lease timestamp [" + timestamp + "] out of range"); + } this.id = id; this.retainingSequenceNumber = retainingSequenceNumber; this.timestamp = timestamp; diff --git a/server/src/test/java/org/elasticsearch/index/seqno/RetentionLeaseTests.java b/server/src/test/java/org/elasticsearch/index/seqno/RetentionLeaseTests.java new file mode 100644 index 0000000000000..a5e4af5d0e6a3 --- /dev/null +++ b/server/src/test/java/org/elasticsearch/index/seqno/RetentionLeaseTests.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.index.seqno; + +import org.elasticsearch.test.ESTestCase; + +import static org.elasticsearch.index.seqno.SequenceNumbers.UNASSIGNED_SEQ_NO; +import static org.hamcrest.Matchers.containsString; +import static org.hamcrest.Matchers.hasToString; + +public class RetentionLeaseTests extends ESTestCase { + + public void testRetainingSequenceNumberOutOfRange() { + final long retainingSequenceNumber = randomLongBetween(Long.MIN_VALUE, UNASSIGNED_SEQ_NO - 1); + final IllegalArgumentException e = expectThrows( + IllegalArgumentException.class, + () -> new RetentionLease("id", retainingSequenceNumber, randomNonNegativeLong(), "source")); + assertThat( + e, + hasToString(containsString("retention lease retaining sequence number [" + retainingSequenceNumber + "] out of range"))); + } + + public void testTimestampOutOfRange() { + final long timestamp = randomLongBetween(Long.MIN_VALUE, -1); + final IllegalArgumentException e = expectThrows( + IllegalArgumentException.class, + () -> new RetentionLease("id", randomLongBetween(SequenceNumbers.NO_OPS_PERFORMED, Long.MAX_VALUE), timestamp, "source")); + assertThat( + e, + hasToString(containsString("retention lease timestamp [" + timestamp + "] out of range"))); + } + +} From 822626dadf512e7ff6f31ded00cafeb555cf0729 Mon Sep 17 00:00:00 2001 From: Jason Tedor Date: Thu, 10 Jan 2019 18:34:55 -0800 Subject: [PATCH 094/186] Make consistent empty retention lease supplier This commit makes the use of empty retention lease suppliers to always be an empty list as opposed to in some cases an empty set. This commit is solely for consistency reasons, there is no functional change here. --- .../org/elasticsearch/index/engine/InternalEngineTests.java | 2 +- .../org/elasticsearch/index/shard/RefreshListenersTests.java | 2 +- .../java/org/elasticsearch/index/engine/EngineTestCase.java | 2 +- 3 files changed, 3 insertions(+), 3 deletions(-) diff --git a/server/src/test/java/org/elasticsearch/index/engine/InternalEngineTests.java b/server/src/test/java/org/elasticsearch/index/engine/InternalEngineTests.java index 82e9869a1cd78..f88aaedd6991f 100644 --- a/server/src/test/java/org/elasticsearch/index/engine/InternalEngineTests.java +++ b/server/src/test/java/org/elasticsearch/index/engine/InternalEngineTests.java @@ -3011,7 +3011,7 @@ public void testRecoverFromForeignTranslog() throws IOException { new CodecService(null, logger), config.getEventListener(), IndexSearcher.getDefaultQueryCache(), IndexSearcher.getDefaultQueryCachingPolicy(), translogConfig, TimeValue.timeValueMinutes(5), config.getExternalRefreshListener(), config.getInternalRefreshListener(), null, - new NoneCircuitBreakerService(), () -> UNASSIGNED_SEQ_NO, Collections::emptySet, primaryTerm::get, + new NoneCircuitBreakerService(), () -> UNASSIGNED_SEQ_NO, Collections::emptyList, primaryTerm::get, tombstoneDocSupplier()); expectThrows(EngineCreationFailureException.class, () -> new InternalEngine(brokenConfig)); 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 ab383f6ef49d6..fe1f4355c9838 100644 --- a/server/src/test/java/org/elasticsearch/index/shard/RefreshListenersTests.java +++ b/server/src/test/java/org/elasticsearch/index/shard/RefreshListenersTests.java @@ -125,7 +125,7 @@ public void onFailedEngine(String reason, @Nullable Exception e) { indexSettings, null, store, newMergePolicy(), iwc.getAnalyzer(), iwc.getSimilarity(), new CodecService(null, logger), eventListener, IndexSearcher.getDefaultQueryCache(), IndexSearcher.getDefaultQueryCachingPolicy(), translogConfig, TimeValue.timeValueMinutes(5), Collections.singletonList(listeners), Collections.emptyList(), null, - new NoneCircuitBreakerService(), () -> SequenceNumbers.NO_OPS_PERFORMED, Collections::emptySet, + new NoneCircuitBreakerService(), () -> SequenceNumbers.NO_OPS_PERFORMED, Collections::emptyList, () -> primaryTerm, EngineTestCase.tombstoneDocSupplier()); engine = new InternalEngine(config); engine.initializeMaxSeqNoOfUpdatesOrDeletes(); diff --git a/test/framework/src/main/java/org/elasticsearch/index/engine/EngineTestCase.java b/test/framework/src/main/java/org/elasticsearch/index/engine/EngineTestCase.java index 043cb85a091ca..58059cd30e382 100644 --- a/test/framework/src/main/java/org/elasticsearch/index/engine/EngineTestCase.java +++ b/test/framework/src/main/java/org/elasticsearch/index/engine/EngineTestCase.java @@ -608,7 +608,7 @@ public void onFailedEngine(String reason, @Nullable Exception e) { retentionLeasesSupplier = replicationTracker::getRetentionLeases; } else { globalCheckpointSupplier = maybeGlobalCheckpointSupplier; - retentionLeasesSupplier = Collections::emptySet; + retentionLeasesSupplier = Collections::emptyList; } EngineConfig config = new EngineConfig(shardId, allocationId.getId(), threadPool, indexSettings, null, store, mergePolicy, iwc.getAnalyzer(), iwc.getSimilarity(), new CodecService(null, logger), listener, From 860a8a7b235a91d6927ac8e2171f039c23b8de3e Mon Sep 17 00:00:00 2001 From: Armin Braun Date: Fri, 11 Jan 2019 08:07:55 +0100 Subject: [PATCH 095/186] Improve Precision for scaled_float (#37169) * Use `toString` and `Bigdecimal` parsing to get intuitive behaviour for `scaled_float` as discussed in #32570 * Closes #32570 --- .../index/mapper/ScaledFloatFieldMapper.java | 24 ++++++++++++++----- .../mapper/ScaledFloatFieldTypeTests.java | 2 ++ 2 files changed, 20 insertions(+), 6 deletions(-) diff --git a/modules/mapper-extras/src/main/java/org/elasticsearch/index/mapper/ScaledFloatFieldMapper.java b/modules/mapper-extras/src/main/java/org/elasticsearch/index/mapper/ScaledFloatFieldMapper.java index 07ee5b5dc6243..d3719ec884fa1 100644 --- a/modules/mapper-extras/src/main/java/org/elasticsearch/index/mapper/ScaledFloatFieldMapper.java +++ b/modules/mapper-extras/src/main/java/org/elasticsearch/index/mapper/ScaledFloatFieldMapper.java @@ -62,6 +62,7 @@ import org.joda.time.DateTimeZone; import java.io.IOException; +import java.math.BigDecimal; import java.util.ArrayList; import java.util.Collections; import java.util.Iterator; @@ -227,8 +228,7 @@ public Query existsQuery(QueryShardContext context) { @Override public Query termQuery(Object value, QueryShardContext context) { failIfNotIndexed(); - double queryValue = parse(value); - long scaledValue = Math.round(queryValue * scalingFactor); + long scaledValue = Math.round(scale(value)); Query query = NumberFieldMapper.NumberType.LONG.termQuery(name(), scaledValue); if (boost() != 1f) { query = new BoostQuery(query, boost()); @@ -241,8 +241,7 @@ public Query termsQuery(List values, QueryShardContext context) { failIfNotIndexed(); List scaledValues = new ArrayList<>(values.size()); for (Object value : values) { - double queryValue = parse(value); - long scaledValue = Math.round(queryValue * scalingFactor); + long scaledValue = Math.round(scale(value)); scaledValues.add(scaledValue); } Query query = NumberFieldMapper.NumberType.LONG.termsQuery(name(), Collections.unmodifiableList(scaledValues)); @@ -257,7 +256,7 @@ public Query rangeQuery(Object lowerTerm, Object upperTerm, boolean includeLower failIfNotIndexed(); Long lo = null; if (lowerTerm != null) { - double dValue = parse(lowerTerm) * scalingFactor; + double dValue = scale(lowerTerm); if (includeLower == false) { dValue = Math.nextUp(dValue); } @@ -265,7 +264,7 @@ public Query rangeQuery(Object lowerTerm, Object upperTerm, boolean includeLower } Long hi = null; if (upperTerm != null) { - double dValue = parse(upperTerm) * scalingFactor; + double dValue = scale(upperTerm); if (includeUpper == false) { dValue = Math.nextDown(dValue); } @@ -326,6 +325,19 @@ public boolean equals(Object o) { public int hashCode() { return 31 * super.hashCode() + Double.hashCode(scalingFactor); } + + /** + * Parses input value and multiplies it with the scaling factor. + * Uses the round-trip of creating a {@link BigDecimal} from the stringified {@code double} + * input to ensure intuitively exact floating point operations. + * (e.g. for a scaling factor of 100, JVM behaviour results in {@code 79.99D * 100 ==> 7998.99..} compared to + * {@code scale(79.99) ==> 7999}) + * @param input Input value to parse floating point num from + * @return Scaled value + */ + private double scale(Object input) { + return new BigDecimal(Double.toString(parse(input))).multiply(BigDecimal.valueOf(scalingFactor)).doubleValue(); + } } private Explicit ignoreMalformed; diff --git a/modules/mapper-extras/src/test/java/org/elasticsearch/index/mapper/ScaledFloatFieldTypeTests.java b/modules/mapper-extras/src/test/java/org/elasticsearch/index/mapper/ScaledFloatFieldTypeTests.java index 1d88022b3e0e0..4389e809bfb2f 100644 --- a/modules/mapper-extras/src/test/java/org/elasticsearch/index/mapper/ScaledFloatFieldTypeTests.java +++ b/modules/mapper-extras/src/test/java/org/elasticsearch/index/mapper/ScaledFloatFieldTypeTests.java @@ -140,6 +140,8 @@ public void testRoundsUpperBoundCorrectly() { assertEquals("scaled_float:[-9223372036854775808 TO 10]", scaledFloatQ.toString()); scaledFloatQ = ft.rangeQuery(null, 0.105, true, true, null); assertEquals("scaled_float:[-9223372036854775808 TO 10]", scaledFloatQ.toString()); + scaledFloatQ = ft.rangeQuery(null, 79.99, true, true, null); + assertEquals("scaled_float:[-9223372036854775808 TO 7999]", scaledFloatQ.toString()); } public void testRoundsLowerBoundCorrectly() { From 80084138dd64cbdf922838978148a139c64d85df Mon Sep 17 00:00:00 2001 From: Ioannis Kakavas Date: Fri, 11 Jan 2019 09:22:40 +0200 Subject: [PATCH 096/186] [DOCS] Fix link to role mapping doc --- .../en/security/authentication/configuring-saml-realm.asciidoc | 2 +- x-pack/docs/en/security/authentication/saml-guide.asciidoc | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/x-pack/docs/en/security/authentication/configuring-saml-realm.asciidoc b/x-pack/docs/en/security/authentication/configuring-saml-realm.asciidoc index de8adde156c2e..b0bdd67d2deef 100644 --- a/x-pack/docs/en/security/authentication/configuring-saml-realm.asciidoc +++ b/x-pack/docs/en/security/authentication/configuring-saml-realm.asciidoc @@ -222,7 +222,7 @@ Your SAML users cannot do anything until they are assigned roles. This can be do through either the {stack-ov}/saml-role-mapping.html[role mapping API], or with {stack-ov}/realm-chains.html#authorization_realms[authorization realms]. -NOTE: You cannot use {stack-ov}/defining-roles.html#roles-management-file[role mapping files] +NOTE: You cannot use {stack-ov}/mapping-roles.html#mapping-roles-file[role mapping files] to grant roles to users authenticating via SAML. -- diff --git a/x-pack/docs/en/security/authentication/saml-guide.asciidoc b/x-pack/docs/en/security/authentication/saml-guide.asciidoc index 36119afdcddd7..9201a5520f76a 100644 --- a/x-pack/docs/en/security/authentication/saml-guide.asciidoc +++ b/x-pack/docs/en/security/authentication/saml-guide.asciidoc @@ -625,7 +625,7 @@ through either the {ref}/security-api-put-role-mapping.html[add role mapping API], or with <>. -NOTE: You cannot use {stack-ov}/defining-roles.html#roles-management-file[role mapping files] +NOTE: You cannot use {stack-ov}/mapping-roles.html#mapping-roles-file[role mapping files] to grant roles to users authenticating via SAML. This is an example of a simple role mapping that grants the `kibana_user` role From 37493c204da0945b7b2cd59f4eec7d4a6e8e1974 Mon Sep 17 00:00:00 2001 From: Martijn van Groningen Date: Fri, 11 Jan 2019 09:02:46 +0100 Subject: [PATCH 097/186] Unmuted test now that #37239 has been merged and backported. Relates to #37231 --- .../test/java/org/elasticsearch/upgrades/CCRIT.java | 12 ++++++------ 1 file changed, 6 insertions(+), 6 deletions(-) diff --git a/x-pack/qa/rolling-upgrade/src/test/java/org/elasticsearch/upgrades/CCRIT.java b/x-pack/qa/rolling-upgrade/src/test/java/org/elasticsearch/upgrades/CCRIT.java index 443fddce52d4a..f2914c3514d90 100644 --- a/x-pack/qa/rolling-upgrade/src/test/java/org/elasticsearch/upgrades/CCRIT.java +++ b/x-pack/qa/rolling-upgrade/src/test/java/org/elasticsearch/upgrades/CCRIT.java @@ -8,7 +8,6 @@ import org.apache.http.util.EntityUtils; import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; -import org.apache.lucene.util.LuceneTestCase; import org.elasticsearch.Version; import org.elasticsearch.client.Request; import org.elasticsearch.client.Response; @@ -22,7 +21,6 @@ import static org.hamcrest.Matchers.equalTo; -@LuceneTestCase.AwaitsFix(bugUrl = "https://github.com/elastic/elasticsearch/issues/37231") public class CCRIT extends AbstractUpgradeTestCase { private static final Logger LOGGER = LogManager.getLogger(CCRIT.class); @@ -38,8 +36,8 @@ protected boolean preserveClusterSettings() { } public void testIndexFollowing() throws Exception { - assumeTrue("CCR became available in 6.5 and test relies on a fix that was shipped with 6.5.4", - UPGRADE_FROM_VERSION.onOrAfter(Version.V_6_5_4)); + assumeTrue("CCR became available in 6.5, but test relies on a fix that was shipped with 6.6.0", + UPGRADE_FROM_VERSION.onOrAfter(Version.V_6_6_0)); setupRemoteCluster(); final String leaderIndex = "my-leader-index"; @@ -92,8 +90,8 @@ public void testIndexFollowing() throws Exception { } public void testAutoFollowing() throws Exception { - assumeTrue("CCR became available in 6.5 and test relies on a fix that was shipped with 6.5.4", - UPGRADE_FROM_VERSION.onOrAfter(Version.V_6_5_4)); + assumeTrue("CCR became available in 6.5, but test relies on a fix that was shipped with 6.6.0", + UPGRADE_FROM_VERSION.onOrAfter(Version.V_6_6_0)); setupRemoteCluster(); final Settings indexSettings = Settings.builder() @@ -268,6 +266,8 @@ private int getNumberOfSuccessfulFollowedIndices() throws IOException { private void assertFollowerGlobalCheckpoint(String followerIndex, int expectedFollowerCheckpoint) throws IOException { Request statsRequest = new Request("GET", "/" + followerIndex + "/_stats"); statsRequest.addParameter("level", "shards"); + // Just docs metric is sufficient here: + statsRequest.addParameter("metric", "docs"); Map response = toMap(client().performRequest(statsRequest)); LOGGER.info("INDEX STATS={}", response); assertThat(((Map) response.get("indices")).size(), equalTo(1)); From bbd093059f5b18416d46a315b873c473b11648f3 Mon Sep 17 00:00:00 2001 From: Alexander Reelsen Date: Fri, 11 Jan 2019 09:22:47 +0100 Subject: [PATCH 098/186] Add whitelist to watcher HttpClient (#36817) This adds a configurable whitelist to the HTTP client in watcher. By default every URL is allowed to retain BWC. A dynamically configurable setting named "xpack.http.whitelist" was added that allows to configure an array of URLs, which can also contain simple regexes. Closes #29937 --- .../settings/notification-settings.asciidoc | 8 + .../elasticsearch/xpack/watcher/Watcher.java | 2 +- .../xpack/watcher/common/http/HttpClient.java | 84 ++++++++- .../watcher/common/http/HttpRequest.java | 29 ++-- .../watcher/common/http/HttpSettings.java | 5 + .../actions/webhook/WebhookActionTests.java | 4 +- .../watcher/common/http/HttpClientTests.java | 162 ++++++++++++++++-- .../http/HttpConnectionTimeoutTests.java | 11 +- .../common/http/HttpReadTimeoutTests.java | 10 +- .../watcher/common/http/HttpRequestTests.java | 6 + 10 files changed, 286 insertions(+), 35 deletions(-) diff --git a/docs/reference/settings/notification-settings.asciidoc b/docs/reference/settings/notification-settings.asciidoc index 25cecfc63627e..e098f22716876 100644 --- a/docs/reference/settings/notification-settings.asciidoc +++ b/docs/reference/settings/notification-settings.asciidoc @@ -64,6 +64,14 @@ request is aborted. Specifies the maximum size an HTTP response is allowed to have, defaults to `10mb`, the maximum configurable value is `50mb`. +`xpack.http.whitelist`:: +A list of URLs, that the internal HTTP client is allowed to connect to. This +client is used in the HTTP input, the webhook, the slack, pagerduty, hipchat +and jira actions. This setting can be updated dynamically. It defaults to `*` +allowing everything. Note: If you configure this setting and you are using one +of the slack/pagerduty/hipchat actions, you have to ensure that the +corresponding endpoints are whitelisted as well. + [[ssl-notification-settings]] :ssl-prefix: xpack.http :component: {watcher} diff --git a/x-pack/plugin/watcher/src/main/java/org/elasticsearch/xpack/watcher/Watcher.java b/x-pack/plugin/watcher/src/main/java/org/elasticsearch/xpack/watcher/Watcher.java index 3ea99e5787fe0..7b17d7f9973d1 100644 --- a/x-pack/plugin/watcher/src/main/java/org/elasticsearch/xpack/watcher/Watcher.java +++ b/x-pack/plugin/watcher/src/main/java/org/elasticsearch/xpack/watcher/Watcher.java @@ -273,7 +273,7 @@ public Collection createComponents(Client client, ClusterService cluster new WatcherIndexTemplateRegistry(clusterService, threadPool, client); // http client - httpClient = new HttpClient(settings, getSslService(), cryptoService); + httpClient = new HttpClient(settings, getSslService(), cryptoService, clusterService); // notification EmailService emailService = new EmailService(settings, cryptoService, clusterService.getClusterSettings()); diff --git a/x-pack/plugin/watcher/src/main/java/org/elasticsearch/xpack/watcher/common/http/HttpClient.java b/x-pack/plugin/watcher/src/main/java/org/elasticsearch/xpack/watcher/common/http/HttpClient.java index a2714e02c6229..10fb8889fae33 100644 --- a/x-pack/plugin/watcher/src/main/java/org/elasticsearch/xpack/watcher/common/http/HttpClient.java +++ b/x-pack/plugin/watcher/src/main/java/org/elasticsearch/xpack/watcher/common/http/HttpClient.java @@ -8,7 +8,9 @@ import org.apache.http.Header; import org.apache.http.HttpHeaders; import org.apache.http.HttpHost; +import org.apache.http.HttpRequestInterceptor; import org.apache.http.NameValuePair; +import org.apache.http.ProtocolException; import org.apache.http.auth.AuthScope; import org.apache.http.auth.Credentials; import org.apache.http.auth.UsernamePasswordCredentials; @@ -19,6 +21,7 @@ import org.apache.http.client.methods.HttpEntityEnclosingRequestBase; import org.apache.http.client.methods.HttpHead; import org.apache.http.client.methods.HttpRequestBase; +import org.apache.http.client.methods.HttpRequestWrapper; import org.apache.http.client.protocol.HttpClientContext; import org.apache.http.client.utils.URIUtils; import org.apache.http.client.utils.URLEncodedUtils; @@ -31,11 +34,20 @@ import org.apache.http.impl.client.BasicAuthCache; import org.apache.http.impl.client.BasicCredentialsProvider; import org.apache.http.impl.client.CloseableHttpClient; +import org.apache.http.impl.client.DefaultRedirectStrategy; import org.apache.http.impl.client.HttpClientBuilder; import org.apache.http.message.BasicNameValuePair; +import org.apache.http.protocol.HttpContext; import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; +import org.apache.lucene.util.automaton.Automaton; +import org.apache.lucene.util.automaton.CharacterRunAutomaton; +import org.apache.lucene.util.automaton.MinimizationOperations; +import org.apache.lucene.util.automaton.Operations; +import org.elasticsearch.ElasticsearchException; +import org.elasticsearch.cluster.service.ClusterService; import org.elasticsearch.common.Strings; +import org.elasticsearch.common.regex.Regex; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.unit.ByteSizeValue; import org.elasticsearch.common.unit.TimeValue; @@ -59,6 +71,7 @@ import java.util.HashMap; import java.util.List; import java.util.Map; +import java.util.concurrent.atomic.AtomicReference; public class HttpClient implements Closeable { @@ -69,20 +82,29 @@ public class HttpClient implements Closeable { private static final int MAX_CONNECTIONS = 500; private static final Logger logger = LogManager.getLogger(HttpClient.class); + private final AtomicReference whitelistAutomaton = new AtomicReference<>(); private final CloseableHttpClient client; private final HttpProxy settingsProxy; private final TimeValue defaultConnectionTimeout; private final TimeValue defaultReadTimeout; private final ByteSizeValue maxResponseSize; private final CryptoService cryptoService; + private final SSLService sslService; - public HttpClient(Settings settings, SSLService sslService, CryptoService cryptoService) { + public HttpClient(Settings settings, SSLService sslService, CryptoService cryptoService, ClusterService clusterService) { this.defaultConnectionTimeout = HttpSettings.CONNECTION_TIMEOUT.get(settings); this.defaultReadTimeout = HttpSettings.READ_TIMEOUT.get(settings); this.maxResponseSize = HttpSettings.MAX_HTTP_RESPONSE_SIZE.get(settings); this.settingsProxy = getProxyFromSettings(settings); this.cryptoService = cryptoService; + this.sslService = sslService; + setWhitelistAutomaton(HttpSettings.HOSTS_WHITELIST.get(settings)); + clusterService.getClusterSettings().addSettingsUpdateConsumer(HttpSettings.HOSTS_WHITELIST, this::setWhitelistAutomaton); + this.client = createHttpClient(); + } + + private CloseableHttpClient createHttpClient() { HttpClientBuilder clientBuilder = HttpClientBuilder.create(); // ssl setup @@ -95,8 +117,48 @@ public HttpClient(Settings settings, SSLService sslService, CryptoService crypto clientBuilder.evictExpiredConnections(); clientBuilder.setMaxConnPerRoute(MAX_CONNECTIONS); clientBuilder.setMaxConnTotal(MAX_CONNECTIONS); + clientBuilder.setRedirectStrategy(new DefaultRedirectStrategy() { + @Override + public boolean isRedirected(org.apache.http.HttpRequest request, org.apache.http.HttpResponse response, + HttpContext context) throws ProtocolException { + boolean isRedirected = super.isRedirected(request, response, context); + if (isRedirected) { + String host = response.getHeaders("Location")[0].getValue(); + if (isWhitelisted(host) == false) { + throw new ElasticsearchException("host [" + host + "] is not whitelisted in setting [" + + HttpSettings.HOSTS_WHITELIST.getKey() + "], will not redirect"); + } + } + + return isRedirected; + } + }); + + clientBuilder.addInterceptorFirst((HttpRequestInterceptor) (request, context) -> { + if (request instanceof HttpRequestWrapper == false) { + throw new ElasticsearchException("unable to check request [{}/{}] for white listing", request, + request.getClass().getName()); + } + + HttpRequestWrapper wrapper = ((HttpRequestWrapper) request); + final String host; + if (wrapper.getTarget() != null) { + host = wrapper.getTarget().toURI(); + } else { + host = wrapper.getOriginal().getRequestLine().getUri(); + } - client = clientBuilder.build(); + if (isWhitelisted(host) == false) { + throw new ElasticsearchException("host [" + host + "] is not whitelisted in setting [" + + HttpSettings.HOSTS_WHITELIST.getKey() + "], will not connect"); + } + }); + + return clientBuilder.build(); + } + + private void setWhitelistAutomaton(List whiteListedHosts) { + whitelistAutomaton.set(createAutomaton(whiteListedHosts)); } public HttpResponse execute(HttpRequest request) throws IOException { @@ -285,6 +347,24 @@ final class HttpMethodWithEntity extends HttpEntityEnclosingRequestBase { public String getMethod() { return methodName; } + } + private boolean isWhitelisted(String host) { + return whitelistAutomaton.get().run(host); + } + + private static final CharacterRunAutomaton MATCH_ALL_AUTOMATON = new CharacterRunAutomaton(Regex.simpleMatchToAutomaton("*")); + // visible for testing + static CharacterRunAutomaton createAutomaton(List whiteListedHosts) { + if (whiteListedHosts.isEmpty()) { + // the default is to accept everything, this should change in the next major version, being 8.0 + // we could emit depreciation warning here, if the whitelist is empty + return MATCH_ALL_AUTOMATON; + } + + Automaton whiteListAutomaton = Regex.simpleMatchToAutomaton(whiteListedHosts.toArray(Strings.EMPTY_ARRAY)); + whiteListAutomaton = MinimizationOperations.minimize(whiteListAutomaton, Operations.DEFAULT_MAX_DETERMINIZED_STATES); + return new CharacterRunAutomaton(whiteListAutomaton); + } } diff --git a/x-pack/plugin/watcher/src/main/java/org/elasticsearch/xpack/watcher/common/http/HttpRequest.java b/x-pack/plugin/watcher/src/main/java/org/elasticsearch/xpack/watcher/common/http/HttpRequest.java index 6bcfc9e5e7d5f..3cda915b7f336 100644 --- a/x-pack/plugin/watcher/src/main/java/org/elasticsearch/xpack/watcher/common/http/HttpRequest.java +++ b/x-pack/plugin/watcher/src/main/java/org/elasticsearch/xpack/watcher/common/http/HttpRequest.java @@ -35,6 +35,7 @@ import java.util.HashMap; import java.util.Map; import java.util.Objects; +import java.util.stream.Collectors; import static java.util.Collections.emptyMap; import static java.util.Collections.unmodifiableMap; @@ -154,10 +155,8 @@ public XContentBuilder toXContent(XContentBuilder builder, ToXContent.Params toX builder.field(Field.PARAMS.getPreferredName(), this.params); } if (headers.isEmpty() == false) { - if (WatcherParams.hideSecrets(toXContentParams) && headers.containsKey("Authorization")) { - Map sanitizedHeaders = new HashMap<>(headers); - sanitizedHeaders.put("Authorization", WatcherXContentParser.REDACTED_PASSWORD); - builder.field(Field.HEADERS.getPreferredName(), sanitizedHeaders); + if (WatcherParams.hideSecrets(toXContentParams)) { + builder.field(Field.HEADERS.getPreferredName(), sanitizeHeaders(headers)); } else { builder.field(Field.HEADERS.getPreferredName(), headers); } @@ -184,6 +183,15 @@ public XContentBuilder toXContent(XContentBuilder builder, ToXContent.Params toX return builder.endObject(); } + private Map sanitizeHeaders(Map headers) { + if (headers.containsKey("Authorization") == false) { + return headers; + } + Map sanitizedHeaders = new HashMap<>(headers); + sanitizedHeaders.put("Authorization", WatcherXContentParser.REDACTED_PASSWORD); + return sanitizedHeaders; + } + @Override public boolean equals(Object o) { if (this == o) return true; @@ -220,16 +228,9 @@ public String toString() { sb.append("port=[").append(port).append("], "); sb.append("path=[").append(path).append("], "); if (!headers.isEmpty()) { - sb.append(", headers=["); - boolean first = true; - for (Map.Entry header : headers.entrySet()) { - if (!first) { - sb.append(", "); - } - sb.append("[").append(header.getKey()).append(": ").append(header.getValue()).append("]"); - first = false; - } - sb.append("], "); + sb.append(sanitizeHeaders(headers).entrySet().stream() + .map(header -> header.getKey() + ": " + header.getValue()) + .collect(Collectors.joining(", ", "headers=[", "], "))); } if (auth != null) { sb.append("auth=[").append(BasicAuth.TYPE).append("], "); diff --git a/x-pack/plugin/watcher/src/main/java/org/elasticsearch/xpack/watcher/common/http/HttpSettings.java b/x-pack/plugin/watcher/src/main/java/org/elasticsearch/xpack/watcher/common/http/HttpSettings.java index af4a20d596cd0..2894d77a28807 100644 --- a/x-pack/plugin/watcher/src/main/java/org/elasticsearch/xpack/watcher/common/http/HttpSettings.java +++ b/x-pack/plugin/watcher/src/main/java/org/elasticsearch/xpack/watcher/common/http/HttpSettings.java @@ -13,7 +13,9 @@ import org.elasticsearch.xpack.core.ssl.SSLConfigurationSettings; import java.util.ArrayList; +import java.util.Collections; import java.util.List; +import java.util.function.Function; /** * Handles the configuration and parsing of settings for the xpack.http. prefix @@ -36,6 +38,8 @@ public class HttpSettings { static final Setting PROXY_HOST = Setting.simpleString(PROXY_HOST_KEY, Property.NodeScope); static final Setting PROXY_SCHEME = Setting.simpleString(PROXY_SCHEME_KEY, Scheme::parse, Property.NodeScope); static final Setting PROXY_PORT = Setting.intSetting(PROXY_PORT_KEY, 0, 0, 0xFFFF, Property.NodeScope); + static final Setting> HOSTS_WHITELIST = Setting.listSetting("xpack.http.whitelist", Collections.singletonList("*"), + Function.identity(), Property.NodeScope, Property.Dynamic); static final Setting MAX_HTTP_RESPONSE_SIZE = Setting.byteSizeSetting("xpack.http.max_response_size", new ByteSizeValue(10, ByteSizeUnit.MB), // default @@ -54,6 +58,7 @@ public static List> getSettings() { settings.add(PROXY_PORT); settings.add(PROXY_SCHEME); settings.add(MAX_HTTP_RESPONSE_SIZE); + settings.add(HOSTS_WHITELIST); return settings; } diff --git a/x-pack/plugin/watcher/src/test/java/org/elasticsearch/xpack/watcher/actions/webhook/WebhookActionTests.java b/x-pack/plugin/watcher/src/test/java/org/elasticsearch/xpack/watcher/actions/webhook/WebhookActionTests.java index 511fcd7698e35..e8b59ca9e6310 100644 --- a/x-pack/plugin/watcher/src/test/java/org/elasticsearch/xpack/watcher/actions/webhook/WebhookActionTests.java +++ b/x-pack/plugin/watcher/src/test/java/org/elasticsearch/xpack/watcher/actions/webhook/WebhookActionTests.java @@ -47,6 +47,7 @@ import static org.elasticsearch.common.unit.TimeValue.timeValueSeconds; import static org.elasticsearch.common.xcontent.XContentFactory.jsonBuilder; +import static org.elasticsearch.xpack.watcher.common.http.HttpClientTests.mockClusterService; import static org.hamcrest.CoreMatchers.instanceOf; import static org.hamcrest.CoreMatchers.notNullValue; import static org.hamcrest.Matchers.containsString; @@ -214,7 +215,8 @@ private WebhookActionFactory webhookFactory(HttpClient client) { public void testThatSelectingProxyWorks() throws Exception { Environment environment = TestEnvironment.newEnvironment(Settings.builder().put("path.home", createTempDir()).build()); - try (HttpClient httpClient = new HttpClient(Settings.EMPTY, new SSLService(environment.settings(), environment), null); + try (HttpClient httpClient = new HttpClient(Settings.EMPTY, new SSLService(environment.settings(), environment), null, + mockClusterService()); MockWebServer proxyServer = new MockWebServer()) { proxyServer.start(); proxyServer.enqueue(new MockResponse().setResponseCode(200).setBody("fullProxiedContent")); diff --git a/x-pack/plugin/watcher/src/test/java/org/elasticsearch/xpack/watcher/common/http/HttpClientTests.java b/x-pack/plugin/watcher/src/test/java/org/elasticsearch/xpack/watcher/common/http/HttpClientTests.java index 519dbbeee8685..88225efba466a 100644 --- a/x-pack/plugin/watcher/src/test/java/org/elasticsearch/xpack/watcher/common/http/HttpClientTests.java +++ b/x-pack/plugin/watcher/src/test/java/org/elasticsearch/xpack/watcher/common/http/HttpClientTests.java @@ -11,6 +11,10 @@ import org.apache.http.client.config.RequestConfig; import org.apache.logging.log4j.message.ParameterizedMessage; import org.apache.logging.log4j.util.Supplier; +import org.apache.lucene.util.automaton.CharacterRunAutomaton; +import org.elasticsearch.ElasticsearchException; +import org.elasticsearch.cluster.service.ClusterService; +import org.elasticsearch.common.settings.ClusterSettings; import org.elasticsearch.common.settings.MockSecureSettings; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.unit.ByteSizeUnit; @@ -40,6 +44,9 @@ import java.net.SocketTimeoutException; import java.nio.charset.StandardCharsets; import java.nio.file.Path; +import java.util.Arrays; +import java.util.Collections; +import java.util.HashSet; import java.util.Locale; import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; @@ -55,6 +62,8 @@ import static org.hamcrest.Matchers.nullValue; import static org.hamcrest.Matchers.startsWith; import static org.hamcrest.core.Is.is; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; public class HttpClientTests extends ESTestCase { @@ -65,7 +74,10 @@ public class HttpClientTests extends ESTestCase { @Before public void init() throws Exception { webServer.start(); - httpClient = new HttpClient(Settings.EMPTY, new SSLService(environment.settings(), environment), null); + ClusterService clusterService = mock(ClusterService.class); + ClusterSettings clusterSettings = new ClusterSettings(Settings.EMPTY, new HashSet<>(HttpSettings.getSettings())); + when(clusterService.getClusterSettings()).thenReturn(clusterSettings); + httpClient = new HttpClient(Settings.EMPTY, new SSLService(environment.settings(), environment), null, clusterService); } @After @@ -179,7 +191,7 @@ public void testHttps() throws Exception { .setSecureSettings(secureSettings) .build(); } - try (HttpClient client = new HttpClient(settings, new SSLService(settings, environment), null)) { + try (HttpClient client = new HttpClient(settings, new SSLService(settings, environment), null, mockClusterService())) { secureSettings = new MockSecureSettings(); // We can't use the client created above for the server since it is only a truststore secureSettings.setString("xpack.ssl.secure_key_passphrase", "testnode"); @@ -220,7 +232,7 @@ public void testHttpsDisableHostnameVerification() throws Exception { } settings = builder.build(); } - try (HttpClient client = new HttpClient(settings, new SSLService(settings, environment), null)) { + try (HttpClient client = new HttpClient(settings, new SSLService(settings, environment), null, mockClusterService())) { MockSecureSettings secureSettings = new MockSecureSettings(); // We can't use the client created above for the server since it only defines a truststore secureSettings.setString("xpack.ssl.secure_key_passphrase", "testnode-no-subjaltname"); @@ -247,7 +259,7 @@ public void testHttpsClientAuth() throws Exception { .build(); TestsSSLService sslService = new TestsSSLService(settings, environment); - try (HttpClient client = new HttpClient(settings, sslService, null)) { + try (HttpClient client = new HttpClient(settings, sslService, null, mockClusterService())) { testSslMockWebserver(client, sslService.sslContext(), true); } } @@ -295,7 +307,7 @@ public void testHttpResponseWithAnyStatusCodeCanReturnBody() throws Exception { @Network public void testHttpsWithoutTruststore() throws Exception { - try (HttpClient client = new HttpClient(Settings.EMPTY, new SSLService(Settings.EMPTY, environment), null)) { + try (HttpClient client = new HttpClient(Settings.EMPTY, new SSLService(Settings.EMPTY, environment), null, mockClusterService())) { // Known server with a valid cert from a commercial CA HttpRequest.Builder request = HttpRequest.builder("www.elastic.co", 443).scheme(Scheme.HTTPS); HttpResponse response = client.execute(request.build()); @@ -319,7 +331,7 @@ public void testThatProxyCanBeConfigured() throws Exception { .method(HttpMethod.GET) .path("/"); - try (HttpClient client = new HttpClient(settings, new SSLService(settings, environment), null)) { + try (HttpClient client = new HttpClient(settings, new SSLService(settings, environment), null, mockClusterService())) { HttpResponse response = client.execute(requestBuilder.build()); assertThat(response.status(), equalTo(200)); assertThat(response.body().utf8ToString(), equalTo("fullProxiedContent")); @@ -400,7 +412,7 @@ public void testProxyCanHaveDifferentSchemeThanRequest() throws Exception { .scheme(Scheme.HTTP) .path("/"); - try (HttpClient client = new HttpClient(settings, new SSLService(settings, environment), null)) { + try (HttpClient client = new HttpClient(settings, new SSLService(settings, environment), null, mockClusterService())) { HttpResponse response = client.execute(requestBuilder.build()); assertThat(response.status(), equalTo(200)); assertThat(response.body().utf8ToString(), equalTo("fullProxiedContent")); @@ -428,7 +440,7 @@ public void testThatProxyCanBeOverriddenByRequest() throws Exception { .proxy(new HttpProxy("localhost", proxyServer.getPort(), Scheme.HTTP)) .path("/"); - try (HttpClient client = new HttpClient(settings, new SSLService(settings, environment), null)) { + try (HttpClient client = new HttpClient(settings, new SSLService(settings, environment), null, mockClusterService())) { HttpResponse response = client.execute(requestBuilder.build()); assertThat(response.status(), equalTo(200)); assertThat(response.body().utf8ToString(), equalTo("fullProxiedContent")); @@ -449,7 +461,7 @@ public void testThatProxyConfigurationRequiresHostAndPort() { } IllegalArgumentException e = expectThrows(IllegalArgumentException.class, - () -> new HttpClient(settings.build(), new SSLService(settings.build(), environment), null)); + () -> new HttpClient(settings.build(), new SSLService(settings.build(), environment), null, mockClusterService())); assertThat(e.getMessage(), containsString("HTTP proxy requires both settings: [xpack.http.proxy.host] and [xpack.http.proxy.port]")); } @@ -548,7 +560,8 @@ public void testMaxHttpResponseSize() throws Exception { HttpRequest.Builder requestBuilder = HttpRequest.builder("localhost", webServer.getPort()).method(HttpMethod.GET).path("/"); - try (HttpClient client = new HttpClient(settings, new SSLService(environment.settings(), environment), null)) { + try (HttpClient client = new HttpClient(settings, new SSLService(environment.settings(), environment), null, + mockClusterService())) { IOException e = expectThrows(IOException.class, () -> client.execute(requestBuilder.build())); assertThat(e.getMessage(), startsWith("Maximum limit of")); } @@ -617,4 +630,133 @@ public void testThatUrlDoesNotContainQuestionMarkAtTheEnd() throws Exception { assertThat(webServer.requests(), hasSize(1)); assertThat(webServer.requests().get(0).getUri().getRawPath(), is("/foo")); } + + public void testThatWhiteListingWorks() throws Exception { + webServer.enqueue(new MockResponse().setResponseCode(200).setBody("whatever")); + Settings settings = Settings.builder().put(HttpSettings.HOSTS_WHITELIST.getKey(), getWebserverUri()).build(); + + try (HttpClient client = new HttpClient(settings, new SSLService(environment.settings(), environment), null, + mockClusterService())) { + HttpRequest request = HttpRequest.builder(webServer.getHostName(), webServer.getPort()).path("foo").build(); + client.execute(request); + } + } + + public void testThatWhiteListBlocksRequests() throws Exception { + Settings settings = Settings.builder() + .put(HttpSettings.HOSTS_WHITELIST.getKey(), getWebserverUri()) + .build(); + + try (HttpClient client = new HttpClient(settings, new SSLService(environment.settings(), environment), null, + mockClusterService())) { + HttpRequest request = HttpRequest.builder("blocked.domain.org", webServer.getPort()) + .path("foo") + .build(); + ElasticsearchException e = expectThrows(ElasticsearchException.class, () -> client.execute(request)); + assertThat(e.getMessage(), is("host [http://blocked.domain.org:" + webServer.getPort() + + "] is not whitelisted in setting [xpack.http.whitelist], will not connect")); + } + } + + public void testThatWhiteListBlocksRedirects() throws Exception { + String redirectUrl = "http://blocked.domain.org:" + webServer.getPort() + "/foo"; + webServer.enqueue(new MockResponse().setResponseCode(302).addHeader("Location", redirectUrl)); + HttpMethod method = randomFrom(HttpMethod.GET, HttpMethod.HEAD); + + if (method == HttpMethod.GET) { + webServer.enqueue(new MockResponse().setResponseCode(200).setBody("shouldBeRead")); + } else if (method == HttpMethod.HEAD) { + webServer.enqueue(new MockResponse().setResponseCode(200)); + } + + Settings settings = Settings.builder().put(HttpSettings.HOSTS_WHITELIST.getKey(), getWebserverUri()).build(); + + try (HttpClient client = new HttpClient(settings, new SSLService(environment.settings(), environment), null, + mockClusterService())) { + HttpRequest request = HttpRequest.builder(webServer.getHostName(), webServer.getPort()).path("/") + .method(method) + .build(); + ElasticsearchException e = expectThrows(ElasticsearchException.class, () -> client.execute(request)); + assertThat(e.getMessage(), is("host [" + redirectUrl + "] is not whitelisted in setting [xpack.http.whitelist], " + + "will not redirect")); + } + } + + public void testThatWhiteListingWorksForRedirects() throws Exception { + int numberOfRedirects = randomIntBetween(1, 10); + for (int i = 0; i < numberOfRedirects; i++) { + String redirectUrl = "http://" + webServer.getHostName() + ":" + webServer.getPort() + "/redirect" + i; + webServer.enqueue(new MockResponse().setResponseCode(302).addHeader("Location", redirectUrl)); + } + webServer.enqueue(new MockResponse().setResponseCode(200).setBody("shouldBeRead")); + + Settings settings = Settings.builder().put(HttpSettings.HOSTS_WHITELIST.getKey(), getWebserverUri() + "*").build(); + + try (HttpClient client = new HttpClient(settings, new SSLService(environment.settings(), environment), null, + mockClusterService())) { + HttpRequest request = HttpRequest.builder(webServer.getHostName(), webServer.getPort()).path("/") + .method(HttpMethod.GET) + .build(); + HttpResponse response = client.execute(request); + + assertThat(webServer.requests(), hasSize(numberOfRedirects + 1)); + assertThat(response.body().utf8ToString(), is("shouldBeRead")); + } + } + + public void testThatWhiteListReloadingWorks() throws Exception { + webServer.enqueue(new MockResponse().setResponseCode(200).setBody("whatever")); + Settings settings = Settings.builder().put(HttpSettings.HOSTS_WHITELIST.getKey(), "example.org").build(); + ClusterService clusterService = mock(ClusterService.class); + ClusterSettings clusterSettings = new ClusterSettings(settings, new HashSet<>(HttpSettings.getSettings())); + when(clusterService.getClusterSettings()).thenReturn(clusterSettings); + + try (HttpClient client = + new HttpClient(settings, new SSLService(environment.settings(), environment), null, clusterService)) { + + // blacklisted + HttpRequest request = HttpRequest.builder(webServer.getHostName(), webServer.getPort()).path("/") + .method(HttpMethod.GET) + .build(); + ElasticsearchException e = expectThrows(ElasticsearchException.class, () -> client.execute(request)); + assertThat(e.getMessage(), containsString("is not whitelisted")); + + Settings newSettings = Settings.builder().put(HttpSettings.HOSTS_WHITELIST.getKey(), getWebserverUri()).build(); + clusterSettings.applySettings(newSettings); + + HttpResponse response = client.execute(request); + assertThat(response.status(), is(200)); + } + } + + public void testAutomatonWhitelisting() { + CharacterRunAutomaton automaton = HttpClient.createAutomaton(Arrays.asList("https://example*", "https://bar.com/foo", + "htt*://www.test.org")); + assertThat(automaton.run("https://example.org"), is(true)); + assertThat(automaton.run("https://example.com"), is(true)); + assertThat(automaton.run("https://examples.com"), is(true)); + assertThat(automaton.run("https://example-website.com"), is(true)); + assertThat(automaton.run("https://noexample.com"), is(false)); + assertThat(automaton.run("https://bar.com/foo"), is(true)); + assertThat(automaton.run("https://bar.com/foo2"), is(false)); + assertThat(automaton.run("https://bar.com"), is(false)); + assertThat(automaton.run("https://www.test.org"), is(true)); + assertThat(automaton.run("http://www.test.org"), is(true)); + } + + public void testWhitelistEverythingByDefault() { + CharacterRunAutomaton automaton = HttpClient.createAutomaton(Collections.emptyList()); + assertThat(automaton.run(randomAlphaOfLength(10)), is(true)); + } + + public static ClusterService mockClusterService() { + ClusterService clusterService = mock(ClusterService.class); + ClusterSettings clusterSettings = new ClusterSettings(Settings.EMPTY, new HashSet<>(HttpSettings.getSettings())); + when(clusterService.getClusterSettings()).thenReturn(clusterSettings); + return clusterService; + } + + private String getWebserverUri() { + return String.format(Locale.ROOT, "http://%s:%s", webServer.getHostName(), webServer.getPort()); + } } diff --git a/x-pack/plugin/watcher/src/test/java/org/elasticsearch/xpack/watcher/common/http/HttpConnectionTimeoutTests.java b/x-pack/plugin/watcher/src/test/java/org/elasticsearch/xpack/watcher/common/http/HttpConnectionTimeoutTests.java index 21efe5b2b94ea..3451c771e3e60 100644 --- a/x-pack/plugin/watcher/src/test/java/org/elasticsearch/xpack/watcher/common/http/HttpConnectionTimeoutTests.java +++ b/x-pack/plugin/watcher/src/test/java/org/elasticsearch/xpack/watcher/common/http/HttpConnectionTimeoutTests.java @@ -14,6 +14,7 @@ import org.elasticsearch.test.junit.annotations.Network; import org.elasticsearch.xpack.core.ssl.SSLService; +import static org.elasticsearch.xpack.watcher.common.http.HttpClientTests.mockClusterService; import static org.hamcrest.Matchers.greaterThan; import static org.hamcrest.Matchers.lessThan; @@ -24,7 +25,8 @@ public class HttpConnectionTimeoutTests extends ESTestCase { @Network public void testDefaultTimeout() throws Exception { Environment environment = TestEnvironment.newEnvironment(Settings.builder().put("path.home", createTempDir()).build()); - HttpClient httpClient = new HttpClient(Settings.EMPTY, new SSLService(environment.settings(), environment), null); + HttpClient httpClient = new HttpClient(Settings.EMPTY, new SSLService(environment.settings(), environment), null, + mockClusterService()); HttpRequest request = HttpRequest.builder(UNROUTABLE_IP, 12345) .method(HttpMethod.POST) @@ -49,7 +51,8 @@ public void testDefaultTimeout() throws Exception { public void testDefaultTimeoutCustom() throws Exception { Environment environment = TestEnvironment.newEnvironment(Settings.builder().put("path.home", createTempDir()).build()); HttpClient httpClient = new HttpClient(Settings.builder() - .put("xpack.http.default_connection_timeout", "5s").build(), new SSLService(environment.settings(), environment), null); + .put("xpack.http.default_connection_timeout", "5s").build(), new SSLService(environment.settings(), environment), null, + mockClusterService()); HttpRequest request = HttpRequest.builder(UNROUTABLE_IP, 12345) .method(HttpMethod.POST) @@ -74,7 +77,8 @@ public void testDefaultTimeoutCustom() throws Exception { public void testTimeoutCustomPerRequest() throws Exception { Environment environment = TestEnvironment.newEnvironment(Settings.builder().put("path.home", createTempDir()).build()); HttpClient httpClient = new HttpClient(Settings.builder() - .put("xpack.http.default_connection_timeout", "10s").build(), new SSLService(environment.settings(), environment), null); + .put("xpack.http.default_connection_timeout", "10s").build(), new SSLService(environment.settings(), environment), null, + mockClusterService()); HttpRequest request = HttpRequest.builder(UNROUTABLE_IP, 12345) .connectionTimeout(TimeValue.timeValueSeconds(5)) @@ -95,5 +99,4 @@ public void testTimeoutCustomPerRequest() throws Exception { // expected } } - } diff --git a/x-pack/plugin/watcher/src/test/java/org/elasticsearch/xpack/watcher/common/http/HttpReadTimeoutTests.java b/x-pack/plugin/watcher/src/test/java/org/elasticsearch/xpack/watcher/common/http/HttpReadTimeoutTests.java index bc328dc586e9e..e534a2a90757e 100644 --- a/x-pack/plugin/watcher/src/test/java/org/elasticsearch/xpack/watcher/common/http/HttpReadTimeoutTests.java +++ b/x-pack/plugin/watcher/src/test/java/org/elasticsearch/xpack/watcher/common/http/HttpReadTimeoutTests.java @@ -18,6 +18,7 @@ import java.net.SocketTimeoutException; +import static org.elasticsearch.xpack.watcher.common.http.HttpClientTests.mockClusterService; import static org.hamcrest.Matchers.greaterThan; import static org.hamcrest.Matchers.lessThan; @@ -43,7 +44,8 @@ public void testDefaultTimeout() throws Exception { .path("/") .build(); - try (HttpClient httpClient = new HttpClient(Settings.EMPTY, new SSLService(environment.settings(), environment), null)) { + try (HttpClient httpClient = new HttpClient(Settings.EMPTY, new SSLService(environment.settings(), environment), + null, mockClusterService())) { long start = System.nanoTime(); expectThrows(SocketTimeoutException.class, () -> httpClient.execute(request)); @@ -65,7 +67,8 @@ public void testDefaultTimeoutCustom() throws Exception { .build(); try (HttpClient httpClient = new HttpClient(Settings.builder() - .put("xpack.http.default_read_timeout", "3s").build(), new SSLService(environment.settings(), environment), null)) { + .put("xpack.http.default_read_timeout", "3s").build(), new SSLService(environment.settings(), environment), + null, mockClusterService())) { long start = System.nanoTime(); expectThrows(SocketTimeoutException.class, () -> httpClient.execute(request)); @@ -88,7 +91,8 @@ public void testTimeoutCustomPerRequest() throws Exception { .build(); try (HttpClient httpClient = new HttpClient(Settings.builder() - .put("xpack.http.default_read_timeout", "10s").build(), new SSLService(environment.settings(), environment), null)) { + .put("xpack.http.default_read_timeout", "10s").build(), new SSLService(environment.settings(), environment), + null, mockClusterService())) { long start = System.nanoTime(); expectThrows(SocketTimeoutException.class, () -> httpClient.execute(request)); diff --git a/x-pack/plugin/watcher/src/test/java/org/elasticsearch/xpack/watcher/common/http/HttpRequestTests.java b/x-pack/plugin/watcher/src/test/java/org/elasticsearch/xpack/watcher/common/http/HttpRequestTests.java index 0d1541577a58b..dea6db9aaf4de 100644 --- a/x-pack/plugin/watcher/src/test/java/org/elasticsearch/xpack/watcher/common/http/HttpRequestTests.java +++ b/x-pack/plugin/watcher/src/test/java/org/elasticsearch/xpack/watcher/common/http/HttpRequestTests.java @@ -149,6 +149,12 @@ public void testXContentRemovesAuthorization() throws Exception { } } + public void testToStringDoesNotContainAuthorizationheader() { + HttpRequest request = HttpRequest.builder("localhost", 443).setHeader("Authorization", "Bearer Foo").build(); + assertThat(request.toString(), not(containsString("Bearer Foo"))); + assertThat(request.toString(), containsString("Authorization: " + WatcherXContentParser.REDACTED_PASSWORD)); + } + private void assertThatManualBuilderEqualsParsingFromUrl(String url, HttpRequest.Builder builder) throws Exception { XContentBuilder urlContentBuilder = jsonBuilder().startObject().field("url", url).endObject(); XContentParser urlContentParser = createParser(urlContentBuilder); From 9f3da013d899f458ff56ded1a57fab4818f90e47 Mon Sep 17 00:00:00 2001 From: Alexander Reelsen Date: Fri, 11 Jan 2019 09:25:05 +0100 Subject: [PATCH 099/186] Date/Time parsing: Use java time API instead of exception handling (#37222) * Add benchmark * Use java time API instead of exception handling when several formatters are used, the existing way of parsing those is to throw an exception catch it, and try the next one. This is is considerably slower than the approach taken in joda time, so that indexing is reduced when a date format like `x||y` is used and y is the date format being used. This commit now uses the java API to parse the date by appending the date time formatters to each other and does not rely on exception handling. * fix benchmark * fix tests by changing formatter, also expose printer * restore optional printing logic to fix tests * fix tests * incorporate review comments --- .../time/DateFormatterBenchmark.java | 58 +++++++ .../common/time/DateFormatter.java | 2 +- .../common/time/DateFormatters.java | 158 ++++-------------- .../common/time/JavaDateFormatter.java | 70 +++----- .../joda/JavaJodaTimeDuellingTests.java | 2 - .../common/time/DateFormattersTests.java | 47 ++---- 6 files changed, 129 insertions(+), 208 deletions(-) create mode 100644 benchmarks/src/main/java/org/elasticsearch/benchmark/time/DateFormatterBenchmark.java diff --git a/benchmarks/src/main/java/org/elasticsearch/benchmark/time/DateFormatterBenchmark.java b/benchmarks/src/main/java/org/elasticsearch/benchmark/time/DateFormatterBenchmark.java new file mode 100644 index 0000000000000..b30b3ada0ab64 --- /dev/null +++ b/benchmarks/src/main/java/org/elasticsearch/benchmark/time/DateFormatterBenchmark.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.benchmark.time; + +import org.elasticsearch.common.joda.Joda; +import org.elasticsearch.common.time.DateFormatter; +import org.openjdk.jmh.annotations.Benchmark; +import org.openjdk.jmh.annotations.BenchmarkMode; +import org.openjdk.jmh.annotations.Fork; +import org.openjdk.jmh.annotations.Measurement; +import org.openjdk.jmh.annotations.Mode; +import org.openjdk.jmh.annotations.OutputTimeUnit; +import org.openjdk.jmh.annotations.Scope; +import org.openjdk.jmh.annotations.State; +import org.openjdk.jmh.annotations.Warmup; + +import java.time.temporal.TemporalAccessor; +import java.util.concurrent.TimeUnit; + +@Fork(3) +@Warmup(iterations = 10) +@Measurement(iterations = 10) +@BenchmarkMode(Mode.AverageTime) +@OutputTimeUnit(TimeUnit.NANOSECONDS) +@State(Scope.Benchmark) +@SuppressWarnings("unused") //invoked by benchmarking framework +public class DateFormatterBenchmark { + + private final DateFormatter javaFormatter = DateFormatter.forPattern("8year_month_day||ordinal_date||epoch_millis"); + private final DateFormatter jodaFormatter = Joda.forPattern("year_month_day||ordinal_date||epoch_millis"); + + @Benchmark + public TemporalAccessor parseJavaDate() { + return javaFormatter.parse("1234567890"); + } + + @Benchmark + public TemporalAccessor parseJodaDate() { + return jodaFormatter.parse("1234567890"); + } +} + diff --git a/server/src/main/java/org/elasticsearch/common/time/DateFormatter.java b/server/src/main/java/org/elasticsearch/common/time/DateFormatter.java index 49c5e7626072b..e89317ad288c0 100644 --- a/server/src/main/java/org/elasticsearch/common/time/DateFormatter.java +++ b/server/src/main/java/org/elasticsearch/common/time/DateFormatter.java @@ -145,7 +145,7 @@ static DateFormatter forPattern(String input) { if (formatters.size() == 1) { return formatters.get(0); } - return new DateFormatters.MergedDateFormatter(input, formatters); + return DateFormatters.merge(input, formatters); } } diff --git a/server/src/main/java/org/elasticsearch/common/time/DateFormatters.java b/server/src/main/java/org/elasticsearch/common/time/DateFormatters.java index d5a8f4b7a5112..d3bf5eb2a641c 100644 --- a/server/src/main/java/org/elasticsearch/common/time/DateFormatters.java +++ b/server/src/main/java/org/elasticsearch/common/time/DateFormatters.java @@ -19,7 +19,6 @@ package org.elasticsearch.common.time; -import org.elasticsearch.ElasticsearchParseException; import org.elasticsearch.common.Strings; import java.time.DateTimeException; @@ -31,7 +30,6 @@ import java.time.ZonedDateTime; import java.time.format.DateTimeFormatter; import java.time.format.DateTimeFormatterBuilder; -import java.time.format.DateTimeParseException; import java.time.format.ResolverStyle; import java.time.format.SignStyle; import java.time.temporal.ChronoField; @@ -40,10 +38,9 @@ import java.time.temporal.TemporalAdjusters; import java.time.temporal.TemporalQueries; import java.time.temporal.WeekFields; -import java.util.Collections; +import java.util.ArrayList; import java.util.List; import java.util.Locale; -import java.util.stream.Collectors; import static java.time.temporal.ChronoField.DAY_OF_MONTH; import static java.time.temporal.ChronoField.DAY_OF_WEEK; @@ -77,21 +74,17 @@ public class DateFormatters { .appendValue(SECOND_OF_MINUTE, 2, 2, SignStyle.NOT_NEGATIVE) .toFormatter(Locale.ROOT); - private static final DateTimeFormatter STRICT_DATE_OPTIONAL_TIME_FORMATTER_1 = new DateTimeFormatterBuilder() + private static final DateTimeFormatter STRICT_DATE_OPTIONAL_TIME_PRINTER = new DateTimeFormatterBuilder() .append(STRICT_YEAR_MONTH_DAY_FORMATTER) - .optionalStart() .appendLiteral('T') .append(STRICT_HOUR_MINUTE_SECOND_FORMATTER) - .optionalStart() - .appendFraction(MILLI_OF_SECOND, 3, 3, true) - .optionalEnd() + .appendFraction(NANO_OF_SECOND, 3, 9, true) .optionalStart() .appendZoneOrOffsetId() .optionalEnd() - .optionalEnd() .toFormatter(Locale.ROOT); - private static final DateTimeFormatter STRICT_DATE_OPTIONAL_TIME_FORMATTER_2 = new DateTimeFormatterBuilder() + private static final DateTimeFormatter STRICT_DATE_OPTIONAL_TIME_FORMATTER = new DateTimeFormatterBuilder() .append(STRICT_YEAR_MONTH_DAY_FORMATTER) .optionalStart() .appendLiteral('T') @@ -100,7 +93,10 @@ public class DateFormatters { .appendFraction(MILLI_OF_SECOND, 3, 3, true) .optionalEnd() .optionalStart() - .appendOffset("+HHmm", "Z") + .appendZoneOrOffsetId() + .optionalEnd() + .optionalStart() + .append(TIME_ZONE_FORMATTER_NO_COLON) .optionalEnd() .optionalEnd() .toFormatter(Locale.ROOT); @@ -109,10 +105,9 @@ public class DateFormatters { * Returns a generic ISO datetime parser where the date is mandatory and the time is optional. */ private static final DateFormatter STRICT_DATE_OPTIONAL_TIME = - new JavaDateFormatter("strict_date_optional_time", STRICT_DATE_OPTIONAL_TIME_FORMATTER_1, - STRICT_DATE_OPTIONAL_TIME_FORMATTER_1, STRICT_DATE_OPTIONAL_TIME_FORMATTER_2); + new JavaDateFormatter("strict_date_optional_time", STRICT_DATE_OPTIONAL_TIME_PRINTER, STRICT_DATE_OPTIONAL_TIME_FORMATTER); - private static final DateTimeFormatter STRICT_DATE_OPTIONAL_TIME_FORMATTER_WITH_NANOS_1 = new DateTimeFormatterBuilder() + private static final DateTimeFormatter STRICT_DATE_OPTIONAL_TIME_FORMATTER_WITH_NANOS = new DateTimeFormatterBuilder() .append(STRICT_YEAR_MONTH_DAY_FORMATTER) .optionalStart() .appendLiteral('T') @@ -124,26 +119,17 @@ public class DateFormatters { .appendZoneOrOffsetId() .optionalEnd() .optionalStart() - .appendOffset("+HHmm", "Z") + .append(TIME_ZONE_FORMATTER_NO_COLON) .optionalEnd() .optionalEnd() .toFormatter(Locale.ROOT); - private static final DateTimeFormatter STRICT_DATE_OPTIONAL_TIME_PRINTER = new DateTimeFormatterBuilder() - .append(STRICT_YEAR_MONTH_DAY_FORMATTER) - .appendLiteral('T') - .append(STRICT_HOUR_MINUTE_SECOND_FORMATTER) - .appendFraction(NANO_OF_SECOND, 3, 9, true) - .optionalStart() - .appendZoneOrOffsetId() - .optionalEnd() - .toFormatter(Locale.ROOT); - /** * Returns a generic ISO datetime parser where the date is mandatory and the time is optional with nanosecond resolution. */ private static final DateFormatter STRICT_DATE_OPTIONAL_TIME_NANOS = new JavaDateFormatter("strict_date_optional_time_nanos", - STRICT_DATE_OPTIONAL_TIME_PRINTER, STRICT_DATE_OPTIONAL_TIME_FORMATTER_WITH_NANOS_1); + STRICT_DATE_OPTIONAL_TIME_PRINTER, STRICT_DATE_OPTIONAL_TIME_FORMATTER_WITH_NANOS); + ///////////////////////////////////////// // // BEGIN basic time formatters @@ -818,7 +804,7 @@ public class DateFormatters { * yyyy-MM-dd'T'HH:mm:ss.SSSZ */ private static final DateFormatter DATE_OPTIONAL_TIME = new JavaDateFormatter("date_optional_time", - STRICT_DATE_OPTIONAL_TIME_FORMATTER_1, + STRICT_DATE_OPTIONAL_TIME_PRINTER, new DateTimeFormatterBuilder() .append(DATE_FORMATTER) .optionalStart() @@ -836,26 +822,6 @@ public class DateFormatters { .appendFraction(MILLI_OF_SECOND, 1, 3, true) .optionalEnd() .optionalStart().appendZoneOrOffsetId().optionalEnd() - .optionalEnd() - .optionalEnd() - .optionalEnd() - .toFormatter(Locale.ROOT), - new DateTimeFormatterBuilder() - .append(DATE_FORMATTER) - .optionalStart() - .appendLiteral('T') - .optionalStart() - .appendValue(HOUR_OF_DAY, 1, 2, SignStyle.NOT_NEGATIVE) - .optionalStart() - .appendLiteral(':') - .appendValue(MINUTE_OF_HOUR, 1, 2, SignStyle.NOT_NEGATIVE) - .optionalStart() - .appendLiteral(':') - .appendValue(SECOND_OF_MINUTE, 1, 2, SignStyle.NOT_NEGATIVE) - .optionalEnd() - .optionalStart() - .appendFraction(MILLI_OF_SECOND, 1, 3, true) - .optionalEnd() .optionalStart().appendOffset("+HHmm", "Z").optionalEnd() .optionalEnd() .optionalEnd() @@ -1006,7 +972,7 @@ public class DateFormatters { * (yyyy-MM-dd'T'HH:mm:ss.SSSZZ). */ private static final DateFormatter DATE_TIME = new JavaDateFormatter("date_time", - STRICT_DATE_OPTIONAL_TIME_FORMATTER_1, + STRICT_DATE_OPTIONAL_TIME_PRINTER, new DateTimeFormatterBuilder().append(DATE_TIME_FORMATTER).appendZoneOrOffsetId().toFormatter(Locale.ROOT), new DateTimeFormatterBuilder().append(DATE_TIME_FORMATTER).append(TIME_ZONE_FORMATTER_NO_COLON).toFormatter(Locale.ROOT) ); @@ -1483,90 +1449,22 @@ public static DateFormatter forPattern(String input) { } } - static class MergedDateFormatter implements DateFormatter { - - private final String pattern; - // package private for tests - final List formatters; - private final List dateMathParsers; - - MergedDateFormatter(String pattern, List formatters) { - assert formatters.size() > 0; - this.pattern = pattern; - this.formatters = Collections.unmodifiableList(formatters); - this.dateMathParsers = formatters.stream().map(DateFormatter::toDateMathParser).collect(Collectors.toList()); - } - - @Override - public TemporalAccessor parse(String input) { - IllegalArgumentException failure = null; - for (DateFormatter formatter : formatters) { - try { - return formatter.parse(input); - // TODO: remove DateTimeParseException when JavaDateFormatter throws IAE - } catch (IllegalArgumentException | DateTimeParseException e) { - if (failure == null) { - // wrap so the entire multi format is in the message - failure = new IllegalArgumentException("failed to parse date field [" + input + "] with format [" + pattern + "]", - e); - } else { - failure.addSuppressed(e); - } - } + static JavaDateFormatter merge(String pattern, List formatters) { + assert formatters.size() > 0; + + List dateTimeFormatters = new ArrayList<>(formatters.size()); + DateTimeFormatter printer = null; + for (DateFormatter formatter : formatters) { + assert formatter instanceof JavaDateFormatter; + JavaDateFormatter javaDateFormatter = (JavaDateFormatter) formatter; + DateTimeFormatter dateTimeFormatter = javaDateFormatter.getParser(); + if (printer == null) { + printer = javaDateFormatter.getPrinter(); } - throw failure; - } - - @Override - public DateFormatter withZone(ZoneId zoneId) { - return new MergedDateFormatter(pattern, formatters.stream().map(f -> f.withZone(zoneId)).collect(Collectors.toList())); - } - - @Override - public DateFormatter withLocale(Locale locale) { - return new MergedDateFormatter(pattern, formatters.stream().map(f -> f.withLocale(locale)).collect(Collectors.toList())); - } - - @Override - public String format(TemporalAccessor accessor) { - return formatters.get(0).format(accessor); + dateTimeFormatters.add(dateTimeFormatter); } - @Override - public String pattern() { - return pattern; - } - - @Override - public Locale locale() { - return formatters.get(0).locale(); - } - - @Override - public ZoneId zone() { - return formatters.get(0).zone(); - } - - @Override - public DateMathParser toDateMathParser() { - return (text, now, roundUp, tz) -> { - ElasticsearchParseException failure = null; - for (DateMathParser parser : dateMathParsers) { - try { - return parser.parse(text, now, roundUp, tz); - } catch (ElasticsearchParseException e) { - if (failure == null) { - // wrap so the entire multi format is in the message - failure = new ElasticsearchParseException("failed to parse date field [" + text + "] with format [" - + pattern + "]", e); - } else { - failure.addSuppressed(e); - } - } - } - throw failure; - }; - } + return new JavaDateFormatter(pattern, printer, dateTimeFormatters.toArray(new DateTimeFormatter[0])); } private static final ZonedDateTime EPOCH_ZONED_DATE_TIME = Instant.EPOCH.atZone(ZoneOffset.UTC); diff --git a/server/src/main/java/org/elasticsearch/common/time/JavaDateFormatter.java b/server/src/main/java/org/elasticsearch/common/time/JavaDateFormatter.java index 68e2cfd4fe317..0fce14b764ef1 100644 --- a/server/src/main/java/org/elasticsearch/common/time/JavaDateFormatter.java +++ b/server/src/main/java/org/elasticsearch/common/time/JavaDateFormatter.java @@ -19,10 +19,11 @@ package org.elasticsearch.common.time; +import org.elasticsearch.common.Strings; + import java.time.ZoneId; import java.time.format.DateTimeFormatter; import java.time.format.DateTimeFormatterBuilder; -import java.time.format.DateTimeParseException; import java.time.temporal.ChronoField; import java.time.temporal.TemporalAccessor; import java.time.temporal.TemporalField; @@ -47,7 +48,7 @@ class JavaDateFormatter implements DateFormatter { private final String format; private final DateTimeFormatter printer; - private final DateTimeFormatter[] parsers; + private final DateTimeFormatter parser; JavaDateFormatter(String format, DateTimeFormatter printer, DateTimeFormatter... parsers) { if (printer == null) { @@ -62,61 +63,54 @@ class JavaDateFormatter implements DateFormatter { throw new IllegalArgumentException("formatters must have the same locale"); } if (parsers.length == 0) { - this.parsers = new DateTimeFormatter[]{printer}; + this.parser = printer; + } else if (parsers.length == 1) { + this.parser = parsers[0]; } else { - this.parsers = parsers; + DateTimeFormatterBuilder builder = new DateTimeFormatterBuilder(); + for (DateTimeFormatter parser : parsers) { + builder.appendOptional(parser); + } + this.parser = builder.toFormatter(Locale.ROOT); } this.format = format; this.printer = printer; } + DateTimeFormatter getParser() { + return parser; + } + + DateTimeFormatter getPrinter() { + return printer; + } + @Override public TemporalAccessor parse(String input) { - DateTimeParseException failure = null; - for (int i = 0; i < parsers.length; i++) { - try { - return parsers[i].parse(input); - } catch (DateTimeParseException e) { - if (failure == null) { - failure = e; - } else { - failure.addSuppressed(e); - } - } + if (Strings.isNullOrEmpty(input)) { + throw new IllegalArgumentException("cannot parse empty date"); } - - // ensure that all parsers exceptions are returned instead of only the last one - throw failure; + return parser.parse(input); } @Override public DateFormatter withZone(ZoneId zoneId) { // shortcurt to not create new objects unnecessarily - if (zoneId.equals(parsers[0].getZone())) { + if (zoneId.equals(parser.getZone())) { return this; } - final DateTimeFormatter[] parsersWithZone = new DateTimeFormatter[parsers.length]; - for (int i = 0; i < parsers.length; i++) { - parsersWithZone[i] = parsers[i].withZone(zoneId); - } - - return new JavaDateFormatter(format, printer.withZone(zoneId), parsersWithZone); + return new JavaDateFormatter(format, printer.withZone(zoneId), parser.withZone(zoneId)); } @Override public DateFormatter withLocale(Locale locale) { // shortcurt to not create new objects unnecessarily - if (locale.equals(parsers[0].getLocale())) { + if (locale.equals(parser.getLocale())) { return this; } - final DateTimeFormatter[] parsersWithZone = new DateTimeFormatter[parsers.length]; - for (int i = 0; i < parsers.length; i++) { - parsersWithZone[i] = parsers[i].withLocale(locale); - } - - return new JavaDateFormatter(format, printer.withLocale(locale), parsersWithZone); + return new JavaDateFormatter(format, printer.withLocale(locale), parser.withLocale(locale)); } @Override @@ -132,17 +126,7 @@ public String pattern() { JavaDateFormatter parseDefaulting(Map fields) { final DateTimeFormatterBuilder parseDefaultingBuilder = new DateTimeFormatterBuilder().append(printer); fields.forEach(parseDefaultingBuilder::parseDefaulting); - if (parsers.length == 1 && parsers[0].equals(printer)) { - return new JavaDateFormatter(format, parseDefaultingBuilder.toFormatter(Locale.ROOT)); - } else { - final DateTimeFormatter[] parsersWithDefaulting = new DateTimeFormatter[parsers.length]; - for (int i = 0; i < parsers.length; i++) { - DateTimeFormatterBuilder builder = new DateTimeFormatterBuilder().append(parsers[i]); - fields.forEach(builder::parseDefaulting); - parsersWithDefaulting[i] = builder.toFormatter(Locale.ROOT); - } - return new JavaDateFormatter(format, parseDefaultingBuilder.toFormatter(Locale.ROOT), parsersWithDefaulting); - } + return new JavaDateFormatter(format, parseDefaultingBuilder.toFormatter(Locale.ROOT)); } @Override diff --git a/server/src/test/java/org/elasticsearch/common/joda/JavaJodaTimeDuellingTests.java b/server/src/test/java/org/elasticsearch/common/joda/JavaJodaTimeDuellingTests.java index a96fee5b6333e..b2370dadb604c 100644 --- a/server/src/test/java/org/elasticsearch/common/joda/JavaJodaTimeDuellingTests.java +++ b/server/src/test/java/org/elasticsearch/common/joda/JavaJodaTimeDuellingTests.java @@ -72,8 +72,6 @@ public void testCustomTimeFormats() { public void testDuellingFormatsValidParsing() { assertSameDate("1522332219", "epoch_second"); - assertSameDate("1522332219.", "epoch_second"); - assertSameDate("1522332219.0", "epoch_second"); assertSameDate("0", "epoch_second"); assertSameDate("1", "epoch_second"); assertSameDate("1522332219321", "epoch_millis"); diff --git a/server/src/test/java/org/elasticsearch/common/time/DateFormattersTests.java b/server/src/test/java/org/elasticsearch/common/time/DateFormattersTests.java index 8d79f9d3600e7..a2858284593d1 100644 --- a/server/src/test/java/org/elasticsearch/common/time/DateFormattersTests.java +++ b/server/src/test/java/org/elasticsearch/common/time/DateFormattersTests.java @@ -23,6 +23,7 @@ import java.time.Instant; import java.time.ZoneId; +import java.time.ZonedDateTime; import java.time.format.DateTimeParseException; import java.time.temporal.TemporalAccessor; import java.util.Locale; @@ -42,21 +43,11 @@ public class DateFormattersTests extends ESTestCase { // as this feature is supported it also makes sense to make it exact public void testEpochMillisParser() { DateFormatter formatter = DateFormatters.forPattern("epoch_millis"); - { - Instant instant = Instant.from(formatter.parse("12345.6789")); - assertThat(instant.getEpochSecond(), is(12L)); - assertThat(instant.getNano(), is(345_678_900)); - } { Instant instant = Instant.from(formatter.parse("12345")); assertThat(instant.getEpochSecond(), is(12L)); assertThat(instant.getNano(), is(345_000_000)); } - { - Instant instant = Instant.from(formatter.parse("12345.")); - assertThat(instant.getEpochSecond(), is(12L)); - assertThat(instant.getNano(), is(345_000_000)); - } { Instant instant = Instant.from(formatter.parse("0")); assertThat(instant.getEpochSecond(), is(0L)); @@ -79,25 +70,12 @@ public void testEpochMilliParser() { public void testEpochSecondParser() { DateFormatter formatter = DateFormatters.forPattern("epoch_second"); - assertThat(Instant.from(formatter.parse("1234.567")).toEpochMilli(), is(1234567L)); - assertThat(Instant.from(formatter.parse("1234.")).getNano(), is(0)); - assertThat(Instant.from(formatter.parse("1234.")).getEpochSecond(), is(1234L)); - assertThat(Instant.from(formatter.parse("1234.1")).getNano(), is(100_000_000)); - assertThat(Instant.from(formatter.parse("1234.12")).getNano(), is(120_000_000)); - assertThat(Instant.from(formatter.parse("1234.123")).getNano(), is(123_000_000)); - assertThat(Instant.from(formatter.parse("1234.1234")).getNano(), is(123_400_000)); - assertThat(Instant.from(formatter.parse("1234.12345")).getNano(), is(123_450_000)); - assertThat(Instant.from(formatter.parse("1234.123456")).getNano(), is(123_456_000)); - assertThat(Instant.from(formatter.parse("1234.1234567")).getNano(), is(123_456_700)); - assertThat(Instant.from(formatter.parse("1234.12345678")).getNano(), is(123_456_780)); - assertThat(Instant.from(formatter.parse("1234.123456789")).getNano(), is(123_456_789)); - - DateTimeParseException e = expectThrows(DateTimeParseException.class, () -> formatter.parse("1234.1234567890")); - assertThat(e.getMessage(), is("Text '1234.1234567890' could not be parsed, unparsed text found at index 4")); - e = expectThrows(DateTimeParseException.class, () -> formatter.parse("1234.123456789013221")); - assertThat(e.getMessage(), is("Text '1234.123456789013221' could not be parsed, unparsed text found at index 4")); + DateTimeParseException e = expectThrows(DateTimeParseException.class, () -> formatter.parse("1234.1")); + assertThat(e.getMessage(), is("Text '1234.1' could not be parsed, unparsed text found at index 4")); + e = expectThrows(DateTimeParseException.class, () -> formatter.parse("1234.")); + assertThat(e.getMessage(), is("Text '1234.' could not be parsed, unparsed text found at index 4")); e = expectThrows(DateTimeParseException.class, () -> formatter.parse("abc")); - assertThat(e.getMessage(), is("Text 'abc' could not be parsed at index 0")); + assertThat(e.getMessage(), is("Text 'abc' could not be parsed, unparsed text found at index 0")); e = expectThrows(DateTimeParseException.class, () -> formatter.parse("1234.abc")); assertThat(e.getMessage(), is("Text '1234.abc' could not be parsed, unparsed text found at index 4")); } @@ -109,6 +87,14 @@ public void testEpochMilliParsersWithDifferentFormatters() { assertThat(formatter.pattern(), is("strict_date_optional_time||epoch_millis")); } + public void testParsersWithMultipleInternalFormats() throws Exception { + ZonedDateTime first = DateFormatters.toZonedDateTime( + DateFormatters.forPattern("strict_date_optional_time_nanos").parse("2018-05-15T17:14:56+0100")); + ZonedDateTime second = DateFormatters.toZonedDateTime( + DateFormatters.forPattern("strict_date_optional_time_nanos").parse("2018-05-15T17:14:56+01:00")); + assertThat(first, is(second)); + } + public void testLocales() { assertThat(DateFormatters.forPattern("strict_date_optional_time").locale(), is(Locale.ROOT)); Locale locale = randomLocale(random()); @@ -157,10 +143,7 @@ public void testForceJava8() { assertThat(DateFormatter.forPattern("8date_optional_time"), instanceOf(JavaDateFormatter.class)); // named formats too DateFormatter formatter = DateFormatter.forPattern("8date_optional_time||ww-MM-dd"); - assertThat(formatter, instanceOf(DateFormatters.MergedDateFormatter.class)); - DateFormatters.MergedDateFormatter mergedFormatter = (DateFormatters.MergedDateFormatter) formatter; - assertThat(mergedFormatter.formatters.get(0), instanceOf(JavaDateFormatter.class)); - assertThat(mergedFormatter.formatters.get(1), instanceOf(JavaDateFormatter.class)); + assertThat(formatter, instanceOf(JavaDateFormatter.class)); } public void testParsingStrictNanoDates() { From 360c430ad7893f083049dbefccb3df9a9000cb81 Mon Sep 17 00:00:00 2001 From: Nhat Nguyen Date: Fri, 11 Jan 2019 03:33:06 -0500 Subject: [PATCH 100/186] Add runAfter and notifyOnce wrapper to ActionListener (#37331) Relates #37291 --- .../elasticsearch/action/ActionListener.java | 44 +++++++++++++++ .../action/ActionListenerTests.java | 53 +++++++++++++++++++ 2 files changed, 97 insertions(+) diff --git a/server/src/main/java/org/elasticsearch/action/ActionListener.java b/server/src/main/java/org/elasticsearch/action/ActionListener.java index f639f139b55fc..07bc519346f7c 100644 --- a/server/src/main/java/org/elasticsearch/action/ActionListener.java +++ b/server/src/main/java/org/elasticsearch/action/ActionListener.java @@ -136,4 +136,48 @@ static void onFailure(Iterable> listeners, E } ExceptionsHelper.maybeThrowRuntimeAndSuppress(exceptionList); } + + /** + * Wraps a given listener and returns a new listener which executes the provided {@code runAfter} + * callback when the listener is notified via either {@code #onResponse} or {@code #onFailure}. + */ + static ActionListener runAfter(ActionListener delegate, Runnable runAfter) { + return new ActionListener() { + @Override + public void onResponse(Response response) { + try { + delegate.onResponse(response); + } finally { + runAfter.run(); + } + } + + @Override + public void onFailure(Exception e) { + try { + delegate.onFailure(e); + } finally { + runAfter.run(); + } + } + }; + } + + /** + * Wraps a given listener and returns a new listener which makes sure {@link #onResponse(Object)} + * and {@link #onFailure(Exception)} of the provided listener will be called at most once. + */ + static ActionListener notifyOnce(ActionListener delegate) { + return new NotifyOnceListener() { + @Override + protected void innerOnResponse(Response response) { + delegate.onResponse(response); + } + + @Override + protected void innerOnFailure(Exception e) { + delegate.onFailure(e); + } + }; + } } diff --git a/server/src/test/java/org/elasticsearch/action/ActionListenerTests.java b/server/src/test/java/org/elasticsearch/action/ActionListenerTests.java index 6414c81058bec..1d4d83457b20e 100644 --- a/server/src/test/java/org/elasticsearch/action/ActionListenerTests.java +++ b/server/src/test/java/org/elasticsearch/action/ActionListenerTests.java @@ -23,9 +23,12 @@ import java.util.ArrayList; import java.util.List; +import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicReference; +import static org.hamcrest.Matchers.equalTo; + public class ActionListenerTests extends ESTestCase { public void testWrap() { @@ -148,4 +151,54 @@ public void testOnFailure() { assertEquals("listener index " + i, "booom", excList.get(i).get().getMessage()); } } + + public void testRunAfter() { + { + AtomicBoolean afterSuccess = new AtomicBoolean(); + ActionListener listener = ActionListener.runAfter(ActionListener.wrap(r -> {}, e -> {}), () -> afterSuccess.set(true)); + listener.onResponse(null); + assertThat(afterSuccess.get(), equalTo(true)); + } + { + AtomicBoolean afterFailure = new AtomicBoolean(); + ActionListener listener = ActionListener.runAfter(ActionListener.wrap(r -> {}, e -> {}), () -> afterFailure.set(true)); + listener.onFailure(null); + assertThat(afterFailure.get(), equalTo(true)); + } + } + + public void testNotifyOnce() { + AtomicInteger onResponseTimes = new AtomicInteger(); + AtomicInteger onFailureTimes = new AtomicInteger(); + ActionListener listener = ActionListener.notifyOnce(new ActionListener() { + @Override + public void onResponse(Object o) { + onResponseTimes.getAndIncrement(); + } + @Override + public void onFailure(Exception e) { + onFailureTimes.getAndIncrement(); + } + }); + boolean success = randomBoolean(); + if (success) { + listener.onResponse(null); + } else { + listener.onFailure(new RuntimeException("test")); + } + for (int iters = between(0, 10), i = 0; i < iters; i++) { + if (randomBoolean()) { + listener.onResponse(null); + } else { + listener.onFailure(new RuntimeException("test")); + } + } + if (success) { + assertThat(onResponseTimes.get(), equalTo(1)); + assertThat(onFailureTimes.get(), equalTo(0)); + } else { + assertThat(onResponseTimes.get(), equalTo(0)); + assertThat(onFailureTimes.get(), equalTo(1)); + } + } } From 0a50821bb29538431ac80916817012927677c9f7 Mon Sep 17 00:00:00 2001 From: Ignacio Vera Date: Fri, 11 Jan 2019 09:37:18 +0100 Subject: [PATCH 101/186] Geo: Do not normalize the longitude with value -180 for Lucene shapes (#37299) Lucene based shapes should not normalize the longitude value -180 to 180. --- .../common/geo/builders/PolygonBuilder.java | 6 +- .../common/geo/GeoJsonShapeParserTests.java | 2 +- .../search/geo/GeoShapeIntegrationIT.java | 83 +++++++++++++++++++ 3 files changed, 89 insertions(+), 2 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/common/geo/builders/PolygonBuilder.java b/server/src/main/java/org/elasticsearch/common/geo/builders/PolygonBuilder.java index 8f7876d2ba9f2..ac19642949c86 100644 --- a/server/src/main/java/org/elasticsearch/common/geo/builders/PolygonBuilder.java +++ b/server/src/main/java/org/elasticsearch/common/geo/builders/PolygonBuilder.java @@ -342,6 +342,8 @@ protected static org.apache.lucene.geo.Polygon polygonLucene(Coordinate[][] poly holes = new org.apache.lucene.geo.Polygon[polygon.length - 1]; for (int i = 0; i < holes.length; ++i) { Coordinate[] coords = polygon[i+1]; + //We do not have holes on the dateline as they get eliminated + //when breaking the polygon around it. double[] x = new double[coords.length]; double[] y = new double[coords.length]; for (int c = 0; c < coords.length; ++c) { @@ -357,7 +359,9 @@ protected static org.apache.lucene.geo.Polygon polygonLucene(Coordinate[][] poly double[] x = new double[shell.length]; double[] y = new double[shell.length]; for (int i = 0; i < shell.length; ++i) { - x[i] = normalizeLon(shell[i].x); + //Lucene Tessellator treats different +180 and -180 and we should keep the sign. + //normalizeLon method excludes -180. + x[i] = Math.abs(shell[i].x) > 180 ? normalizeLon(shell[i].x) : shell[i].x; y[i] = normalizeLat(shell[i].y); } diff --git a/server/src/test/java/org/elasticsearch/common/geo/GeoJsonShapeParserTests.java b/server/src/test/java/org/elasticsearch/common/geo/GeoJsonShapeParserTests.java index 2acabee8797f4..7cfa4e7d9c5dc 100644 --- a/server/src/test/java/org/elasticsearch/common/geo/GeoJsonShapeParserTests.java +++ b/server/src/test/java/org/elasticsearch/common/geo/GeoJsonShapeParserTests.java @@ -1126,7 +1126,7 @@ public void testParseGeometryCollection() throws IOException { ), new org.apache.lucene.geo.Polygon( new double[] {12.142857142857142d, -12.142857142857142d, -10d, 10d, 12.142857142857142d}, - new double[] {180d, 180d, -177d, -177d, 180d} + new double[] {-180d, -180d, -177d, -177d, -180d} ) }; assertGeometryEquals(luceneExpected, geometryCollectionGeoJson, false); diff --git a/server/src/test/java/org/elasticsearch/search/geo/GeoShapeIntegrationIT.java b/server/src/test/java/org/elasticsearch/search/geo/GeoShapeIntegrationIT.java index b120b54687607..a7faa04017258 100644 --- a/server/src/test/java/org/elasticsearch/search/geo/GeoShapeIntegrationIT.java +++ b/server/src/test/java/org/elasticsearch/search/geo/GeoShapeIntegrationIT.java @@ -22,6 +22,7 @@ import org.elasticsearch.cluster.ClusterState; import org.elasticsearch.cluster.routing.IndexShardRoutingTable; import org.elasticsearch.common.Strings; +import org.elasticsearch.common.geo.builders.PointBuilder; import org.elasticsearch.common.geo.builders.ShapeBuilder; import org.elasticsearch.common.xcontent.XContentFactory; import org.elasticsearch.common.xcontent.XContentType; @@ -158,6 +159,88 @@ public void testIndexShapeRouting() throws Exception { assertThat(searchResponse.getHits().getTotalHits().value, equalTo(1L)); } + public void testIndexPolygonDateLine() throws Exception { + String mappingVector = "{\n" + + " \"properties\": {\n" + + " \"shape\": {\n" + + " \"type\": \"geo_shape\"\n" + + " }\n" + + " }\n" + + " }"; + + String mappingQuad = "{\n" + + " \"properties\": {\n" + + " \"shape\": {\n" + + " \"type\": \"geo_shape\",\n" + + " \"tree\": \"quadtree\"\n" + + " }\n" + + " }\n" + + " }"; + + + // create index + assertAcked(client().admin().indices().prepareCreate("vector").addMapping("doc", mappingVector, XContentType.JSON).get()); + ensureGreen(); + + assertAcked(client().admin().indices().prepareCreate("quad").addMapping("doc", mappingQuad, XContentType.JSON).get()); + ensureGreen(); + + String source = "{\n" + + " \"shape\" : \"POLYGON((179 0, -179 0, -179 2, 179 2, 179 0))\""+ + "}"; + + indexRandom(true, client().prepareIndex("quad", "doc", "0").setSource(source, XContentType.JSON)); + indexRandom(true, client().prepareIndex("vector", "doc", "0").setSource(source, XContentType.JSON)); + + SearchResponse searchResponse = client().prepareSearch("quad").setQuery( + geoShapeQuery("shape", new PointBuilder(-179.75, 1)) + ).get(); + + + assertThat(searchResponse.getHits().getTotalHits().value, equalTo(1L)); + + searchResponse = client().prepareSearch("quad").setQuery( + geoShapeQuery("shape", new PointBuilder(90, 1)) + ).get(); + + assertThat(searchResponse.getHits().getTotalHits().value, equalTo(0L)); + + searchResponse = client().prepareSearch("quad").setQuery( + geoShapeQuery("shape", new PointBuilder(-180, 1)) + ).get(); + + assertThat(searchResponse.getHits().getTotalHits().value, equalTo(1L)); + searchResponse = client().prepareSearch("quad").setQuery( + geoShapeQuery("shape", new PointBuilder(180, 1)) + ).get(); + + assertThat(searchResponse.getHits().getTotalHits().value, equalTo(1L)); + + searchResponse = client().prepareSearch("vector").setQuery( + geoShapeQuery("shape", new PointBuilder(90, 1)) + ).get(); + + assertThat(searchResponse.getHits().getTotalHits().value, equalTo(0L)); + + searchResponse = client().prepareSearch("vector").setQuery( + geoShapeQuery("shape", new PointBuilder(-179.75, 1)) + ).get(); + + assertThat(searchResponse.getHits().getTotalHits().value, equalTo(1L)); + + searchResponse = client().prepareSearch("vector").setQuery( + geoShapeQuery("shape", new PointBuilder(-180, 1)) + ).get(); + + assertThat(searchResponse.getHits().getTotalHits().value, equalTo(1L)); + + searchResponse = client().prepareSearch("vector").setQuery( + geoShapeQuery("shape", new PointBuilder(180, 1)) + ).get(); + + assertThat(searchResponse.getHits().getTotalHits().value, equalTo(1L)); + } + private String findNodeName(String index) { ClusterState state = client().admin().cluster().prepareState().get().getState(); IndexShardRoutingTable shard = state.getRoutingTable().index(index).shard(0); From 3e73911cbe81c05d71439fe9230bf4eeacf2e9b6 Mon Sep 17 00:00:00 2001 From: Alpar Torok Date: Fri, 11 Jan 2019 10:45:40 +0200 Subject: [PATCH 102/186] Mute PrimaryAllocationIT.testForceStaleReplicaToBePromotedToPrimaryOnWrongNode Tracking issue: #37345 --- .../org/elasticsearch/cluster/routing/PrimaryAllocationIT.java | 1 + 1 file changed, 1 insertion(+) diff --git a/server/src/test/java/org/elasticsearch/cluster/routing/PrimaryAllocationIT.java b/server/src/test/java/org/elasticsearch/cluster/routing/PrimaryAllocationIT.java index 213d8175fcd18..3f826c587e683 100644 --- a/server/src/test/java/org/elasticsearch/cluster/routing/PrimaryAllocationIT.java +++ b/server/src/test/java/org/elasticsearch/cluster/routing/PrimaryAllocationIT.java @@ -265,6 +265,7 @@ public void testForceStaleReplicaToBePromotedToPrimary() throws Exception { assertThat(newHistoryUUIds, hasSize(1)); } + @AwaitsFix(bugUrl = "https://github.com/elastic/elasticsearch/issues/37345") public void testForceStaleReplicaToBePromotedToPrimaryOnWrongNode() throws Exception { String master = internalCluster().startMasterOnlyNode(Settings.EMPTY); internalCluster().startDataOnlyNodes(2); From b24dc2c5417af62a007ef0104d8bb6d62488a7b7 Mon Sep 17 00:00:00 2001 From: Jim Ferenczi Date: Fri, 11 Jan 2019 09:47:46 +0100 Subject: [PATCH 103/186] [TEST] Awaits tasks termination in the RestHighLevelClient tests (#37302) This change ensures that TasksIT#testGetValidTask and ReindexIT#testReindexTask don't leave a non-completed task on the cluster when they finish. Closes #35644 --- .../org/elasticsearch/client/ReindexIT.java | 18 ++++++------------ .../java/org/elasticsearch/client/TasksIT.java | 7 +++++-- 2 files changed, 11 insertions(+), 14 deletions(-) diff --git a/client/rest-high-level/src/test/java/org/elasticsearch/client/ReindexIT.java b/client/rest-high-level/src/test/java/org/elasticsearch/client/ReindexIT.java index 73594d8a878a8..cfdd29cdfbfbf 100644 --- a/client/rest-high-level/src/test/java/org/elasticsearch/client/ReindexIT.java +++ b/client/rest-high-level/src/test/java/org/elasticsearch/client/ReindexIT.java @@ -23,6 +23,7 @@ import org.elasticsearch.action.index.IndexRequest; import org.elasticsearch.action.support.WriteRequest; import org.elasticsearch.client.tasks.TaskSubmissionResponse; +import org.elasticsearch.common.CheckedRunnable; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.xcontent.XContentType; import org.elasticsearch.index.query.IdsQueryBuilder; @@ -32,7 +33,6 @@ import java.io.IOException; import java.util.Collections; -import java.util.function.BooleanSupplier; public class ReindexIT extends ESRestHighLevelClientTestCase { @@ -82,7 +82,7 @@ public void testReindex() throws IOException { } } - public void testReindexTask() throws IOException, InterruptedException { + public void testReindexTask() throws Exception { final String sourceIndex = "source123"; final String destinationIndex = "dest2"; { @@ -118,20 +118,14 @@ public void testReindexTask() throws IOException, InterruptedException { String taskId = reindexSubmission.getTask(); // <3> // end::submit-reindex-task - BooleanSupplier hasUpgradeCompleted = checkCompletionStatus(taskId); - awaitBusy(hasUpgradeCompleted); + assertBusy(checkCompletionStatus(client(), taskId)); } } - private BooleanSupplier checkCompletionStatus(String taskId) { + static CheckedRunnable checkCompletionStatus(RestClient client, String taskId) { return () -> { - try { - Response response = client().performRequest(new Request("GET", "/_tasks/" + taskId)); - return (boolean) entityAsMap(response).get("completed"); - } catch (IOException e) { - fail(e.getMessage()); - return false; - } + Response response = client.performRequest(new Request("GET", "/_tasks/" + taskId)); + assertTrue((boolean) entityAsMap(response).get("completed")); }; } } diff --git a/client/rest-high-level/src/test/java/org/elasticsearch/client/TasksIT.java b/client/rest-high-level/src/test/java/org/elasticsearch/client/TasksIT.java index 28b909df5d420..5dc168eadfe5e 100644 --- a/client/rest-high-level/src/test/java/org/elasticsearch/client/TasksIT.java +++ b/client/rest-high-level/src/test/java/org/elasticsearch/client/TasksIT.java @@ -72,7 +72,7 @@ public void testListTasks() throws IOException { assertTrue("List tasks were not found", listTasksFound); } - public void testGetValidTask() throws IOException { + public void testGetValidTask() throws Exception { // Run a Reindex to create a task @@ -112,7 +112,10 @@ public void testGetValidTask() throws IOException { TaskInfo info = taskResponse.getTaskInfo(); assertTrue(info.isCancellable()); assertEquals("reindex from [source1] to [dest][_doc]", info.getDescription()); - assertEquals("indices:data/write/reindex", info.getAction()); + assertEquals("indices:data/write/reindex", info.getAction()); + if (taskResponse.isCompleted() == false) { + assertBusy(ReindexIT.checkCompletionStatus(client(), taskId.toString())); + } } public void testGetInvalidTask() throws IOException { From 82ca2d62decbb496047d1eab4160c98099c19171 Mon Sep 17 00:00:00 2001 From: Alpar Torok Date: Fri, 11 Jan 2019 10:54:53 +0200 Subject: [PATCH 104/186] Mute CloseWhileRelocatingShardsIT.testCloseWhileRelocatingShards Tracked by #37274 --- .../indices/state/CloseWhileRelocatingShardsIT.java | 1 + 1 file changed, 1 insertion(+) 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 02484eb0de14d..165360c35972d 100644 --- a/server/src/test/java/org/elasticsearch/indices/state/CloseWhileRelocatingShardsIT.java +++ b/server/src/test/java/org/elasticsearch/indices/state/CloseWhileRelocatingShardsIT.java @@ -64,6 +64,7 @@ protected int numberOfReplicas() { return 1; } + @AwaitsFix(bugUrl = "https://github.com/elastic/elasticsearch/issues/37274") public void testCloseWhileRelocatingShards() throws Exception { final String[] indices = new String[randomIntBetween(1, 3)]; final Map docsPerIndex = new HashMap<>(); From 3a929c7aeac203fdcacbc2470bf6574ca116c97c Mon Sep 17 00:00:00 2001 From: Yannick Welsch Date: Fri, 11 Jan 2019 10:15:39 +0100 Subject: [PATCH 105/186] Increase assertBusy timeouts for RefreshListenersTests --- .../elasticsearch/index/shard/RefreshListenersTests.java | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) 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 fe1f4355c9838..53c3e86ee01fb 100644 --- a/server/src/test/java/org/elasticsearch/index/shard/RefreshListenersTests.java +++ b/server/src/test/java/org/elasticsearch/index/shard/RefreshListenersTests.java @@ -72,6 +72,7 @@ import java.util.List; import java.util.Locale; import java.util.concurrent.CountDownLatch; +import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicReference; import java.util.function.Consumer; @@ -264,7 +265,6 @@ 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(() -> { @@ -281,7 +281,7 @@ public void testConcurrentRefresh() throws Exception { if (immediate) { assertNotNull(listener.forcedRefresh.get()); } else { - assertBusy(() -> assertNotNull(listener.forcedRefresh.get())); + assertBusy(() -> assertNotNull(listener.forcedRefresh.get()), 1, TimeUnit.MINUTES); } assertFalse(listener.forcedRefresh.get()); listener.assertNoError(); @@ -316,7 +316,7 @@ public void testLotsOfThreads() throws Exception { DummyRefreshListener listener = new DummyRefreshListener(); listeners.addOrNotify(index.getTranslogLocation(), listener); - assertBusy(() -> assertNotNull("listener never called", listener.forcedRefresh.get())); + assertBusy(() -> assertNotNull("listener never called", listener.forcedRefresh.get()), 1, TimeUnit.MINUTES); if (threadCount < maxListeners) { assertFalse(listener.forcedRefresh.get()); } From 100ef2cc0ef140cc02bb5b3006e13f7110463f6a Mon Sep 17 00:00:00 2001 From: Alpar Torok Date: Fri, 11 Jan 2019 11:23:54 +0200 Subject: [PATCH 106/186] Fix artifactId in plugin poms (#37315) --- .../org/elasticsearch/gradle/plugin/PluginBuildPlugin.groovy | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/buildSrc/src/main/groovy/org/elasticsearch/gradle/plugin/PluginBuildPlugin.groovy b/buildSrc/src/main/groovy/org/elasticsearch/gradle/plugin/PluginBuildPlugin.groovy index d4f7325093218..306ac4a05e87a 100644 --- a/buildSrc/src/main/groovy/org/elasticsearch/gradle/plugin/PluginBuildPlugin.groovy +++ b/buildSrc/src/main/groovy/org/elasticsearch/gradle/plugin/PluginBuildPlugin.groovy @@ -114,7 +114,7 @@ public class PluginBuildPlugin extends BuildPlugin { generatePOMTask.ext.pomFileName = "${project.archivesBaseName}-client-${project.versions.elasticsearch}.pom" } } else { - project.plugins.withType(MavenPublishPlugin).whenPluginAdded { + if (project.plugins.hasPlugin(MavenPublishPlugin)) { project.publishing.publications.nebula(MavenPublication).artifactId( project.pluginProperties.extension.name ) From ed2b5e80ebec290d8da7fdaa9d3420e64cb1b3c7 Mon Sep 17 00:00:00 2001 From: Andrei Stefan Date: Fri, 11 Jan 2019 11:24:31 +0200 Subject: [PATCH 107/186] Update discovery-ec2.asciidoc (#28925) * Be more clear about the need to defining tags in EC2 to be able to filter ES instances. --- docs/plugins/discovery-ec2.asciidoc | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/docs/plugins/discovery-ec2.asciidoc b/docs/plugins/discovery-ec2.asciidoc index 2710cf46bff4c..57b5b8468fafb 100644 --- a/docs/plugins/discovery-ec2.asciidoc +++ b/docs/plugins/discovery-ec2.asciidoc @@ -185,9 +185,9 @@ Management Console. It should look similar to this. ===== Filtering by Tags The ec2 discovery can also filter machines to include in the cluster based on tags (and not just groups). The settings -to use include the `discovery.ec2.tag.` prefix. For example, setting `discovery.ec2.tag.stage` to `dev` will only -filter instances with a tag key set to `stage`, and a value of `dev`. Several tags set will require all of those tags -to be set for the instance to be included. +to use include the `discovery.ec2.tag.` prefix. For example, if you defined a tag `stage` in EC2 and set it to `dev`, +setting `discovery.ec2.tag.stage` to `dev` will only filter instances with a tag key set to `stage`, and a value +of `dev`. Adding multiple `discovery.ec2.tag` settings will require all of those tags to be set for the instance to be included. One practical use for tag filtering is when an ec2 cluster contains many nodes that are not running Elasticsearch. In this case (particularly with high `discovery.zen.ping_timeout` values) there is a risk that a new node's discovery phase From 4d3928d4449e0202f48eb5fb15179e4a9e7847e4 Mon Sep 17 00:00:00 2001 From: Yannick Welsch Date: Fri, 11 Jan 2019 11:25:20 +0100 Subject: [PATCH 108/186] Increase timeouts in UnicastZenPingTests Relates to #37268 --- .../discovery/zen/UnicastZenPingTests.java | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/server/src/test/java/org/elasticsearch/discovery/zen/UnicastZenPingTests.java b/server/src/test/java/org/elasticsearch/discovery/zen/UnicastZenPingTests.java index fedbf02a8e86f..f06ef3e72808a 100644 --- a/server/src/test/java/org/elasticsearch/discovery/zen/UnicastZenPingTests.java +++ b/server/src/test/java/org/elasticsearch/discovery/zen/UnicastZenPingTests.java @@ -401,7 +401,7 @@ public BoundTransportAddress boundAddress() { Collections.singletonList("127.0.0.1"), limitPortCounts, transportService, - TimeValue.timeValueSeconds(1)); + TimeValue.timeValueSeconds(30)); assertThat(transportAddresses, hasSize(limitPortCounts)); final Set ports = new HashSet<>(); for (final TransportAddress address : transportAddresses) { @@ -445,7 +445,7 @@ public BoundTransportAddress boundAddress() { Collections.singletonList(NetworkAddress.format(loopbackAddress)), 10, transportService, - TimeValue.timeValueSeconds(1)); + TimeValue.timeValueSeconds(30)); assertThat(transportAddresses, hasSize(7)); final Set ports = new HashSet<>(); for (final TransportAddress address : transportAddresses) { @@ -496,7 +496,7 @@ public TransportAddress[] addressesFromString(String address, int perAddressLimi Arrays.asList(hostname), 1, transportService, - TimeValue.timeValueSeconds(1) + TimeValue.timeValueSeconds(30) ); assertThat(transportAddresses, empty()); @@ -547,7 +547,7 @@ public TransportAddress[] addressesFromString(String address, int perAddressLimi new TransportService(Settings.EMPTY, transport, threadPool, TransportService.NOOP_TRANSPORT_INTERCEPTOR, x -> null, null, Collections.emptySet()); closeables.push(transportService); - final TimeValue resolveTimeout = TimeValue.timeValueSeconds(randomIntBetween(1, 3)); + final TimeValue resolveTimeout = TimeValue.timeValueSeconds(randomIntBetween(3, 5)); try { final List transportAddresses = UnicastZenPing.resolveHostsLists( executorService, @@ -722,7 +722,7 @@ public BoundTransportAddress boundAddress() { Arrays.asList("127.0.0.1:9300:9300", "127.0.0.1:9301"), 1, transportService, - TimeValue.timeValueSeconds(1)); + TimeValue.timeValueSeconds(30)); assertThat(transportAddresses, hasSize(1)); // only one of the two is valid and will be used assertThat(transportAddresses.get(0).getAddress(), equalTo("127.0.0.1")); assertThat(transportAddresses.get(0).getPort(), equalTo(9301)); From 1da59db3fbe7099eacce8a022c694ad8b9005aa8 Mon Sep 17 00:00:00 2001 From: David Roberts Date: Fri, 11 Jan 2019 13:22:35 +0000 Subject: [PATCH 109/186] [ML] Wait for autodetect to be ready in the datafeed (#37349) This is a reinforcement of #37227. It turns out that persistent tasks are not made stale if the node they were running on is restarted and the master node does not notice this. The main scenario where this happens is when minimum master nodes is the same as the number of nodes in the cluster, so the cluster cannot elect a master node when any node is restarted. When an ML node restarts we need the datafeeds for any jobs that were running on that node to not just wait until the jobs are allocated, but to wait for the autodetect process of the job to start up. In the case of reassignment of the job persistent task this was dealt with by the stale status test. But in the case where a node restarts but its persistent tasks are not reassigned we need a deeper test. Fixes #36810 --- .../xpack/ml/MachineLearning.java | 4 +- .../xpack/ml/datafeed/DatafeedManager.java | 29 ++++++++-- .../autodetect/AutodetectProcessManager.java | 27 +++++++-- .../ml/datafeed/DatafeedManagerTests.java | 55 +++++++++++++++++-- 4 files changed, 99 insertions(+), 16 deletions(-) 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 418add2757fbc..cc259f51c1e34 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 @@ -432,7 +432,7 @@ public Collection createComponents(Client client, ClusterService cluster DatafeedJobBuilder datafeedJobBuilder = new DatafeedJobBuilder(client, settings, xContentRegistry, auditor, System::currentTimeMillis); DatafeedManager datafeedManager = new DatafeedManager(threadPool, client, clusterService, datafeedJobBuilder, - System::currentTimeMillis, auditor); + System::currentTimeMillis, auditor, autodetectProcessManager); this.datafeedManager.set(datafeedManager); MlLifeCycleService mlLifeCycleService = new MlLifeCycleService(environment, clusterService, datafeedManager, autodetectProcessManager); @@ -473,7 +473,7 @@ public List> getPersistentTasksExecutor(ClusterServic return Arrays.asList( new TransportOpenJobAction.OpenJobPersistentTasksExecutor(settings, clusterService, autodetectProcessManager.get(), memoryTracker.get(), client), - new TransportStartDatafeedAction.StartDatafeedPersistentTasksExecutor( datafeedManager.get()) + new TransportStartDatafeedAction.StartDatafeedPersistentTasksExecutor(datafeedManager.get()) ); } 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 6367a13100ed0..e004a718b13fa 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 @@ -27,9 +27,11 @@ import org.elasticsearch.xpack.core.ml.action.StartDatafeedAction; import org.elasticsearch.xpack.core.ml.datafeed.DatafeedState; import org.elasticsearch.xpack.core.ml.job.config.JobState; +import org.elasticsearch.xpack.core.ml.job.config.JobTaskState; import org.elasticsearch.xpack.core.ml.job.messages.Messages; import org.elasticsearch.xpack.ml.MachineLearning; import org.elasticsearch.xpack.ml.action.TransportStartDatafeedAction; +import org.elasticsearch.xpack.ml.job.process.autodetect.AutodetectProcessManager; import org.elasticsearch.xpack.ml.notifications.Auditor; import java.util.ArrayList; @@ -62,16 +64,18 @@ public class DatafeedManager { private final ConcurrentMap runningDatafeedsOnThisNode = new ConcurrentHashMap<>(); private final DatafeedJobBuilder datafeedJobBuilder; private final TaskRunner taskRunner = new TaskRunner(); + private final AutodetectProcessManager autodetectProcessManager; private volatile boolean isolated; public DatafeedManager(ThreadPool threadPool, Client client, ClusterService clusterService, DatafeedJobBuilder datafeedJobBuilder, - Supplier currentTimeSupplier, Auditor auditor) { + Supplier currentTimeSupplier, Auditor auditor, AutodetectProcessManager autodetectProcessManager) { this.client = Objects.requireNonNull(client); this.clusterService = Objects.requireNonNull(clusterService); this.threadPool = threadPool; this.currentTimeSupplier = Objects.requireNonNull(currentTimeSupplier); this.auditor = Objects.requireNonNull(auditor); this.datafeedJobBuilder = Objects.requireNonNull(datafeedJobBuilder); + this.autodetectProcessManager = autodetectProcessManager; clusterService.addListener(taskRunner); } @@ -256,6 +260,21 @@ private JobState getJobState(PersistentTasksCustomMetaData tasks, TransportStart return MlTasks.getJobStateModifiedForReassignments(getJobId(datafeedTask), tasks); } + private boolean jobHasOpenAutodetectCommunicator(PersistentTasksCustomMetaData tasks, + TransportStartDatafeedAction.DatafeedTask datafeedTask) { + PersistentTasksCustomMetaData.PersistentTask jobTask = MlTasks.getJobTask(getJobId(datafeedTask), tasks); + if (jobTask == null) { + return false; + } + + JobTaskState state = (JobTaskState) jobTask.getState(); + if (state == null || state.isStatusStale(jobTask)) { + return false; + } + + return autodetectProcessManager.hasOpenAutodetectCommunicator(jobTask.getAllocationId()); + } + private TimeValue computeNextDelay(long next) { return new TimeValue(Math.max(1, next - currentTimeSupplier.get())); } @@ -446,7 +465,7 @@ private class TaskRunner implements ClusterStateListener { private void runWhenJobIsOpened(TransportStartDatafeedAction.DatafeedTask datafeedTask) { ClusterState clusterState = clusterService.state(); PersistentTasksCustomMetaData tasks = clusterState.getMetaData().custom(PersistentTasksCustomMetaData.TYPE); - if (getJobState(tasks, datafeedTask) == JobState.OPENED) { + if (getJobState(tasks, datafeedTask) == JobState.OPENED && jobHasOpenAutodetectCommunicator(tasks, datafeedTask)) { runTask(datafeedTask); } else { logger.info("Datafeed [{}] is waiting for job [{}] to be opened", @@ -485,10 +504,10 @@ public void clusterChanged(ClusterChangedEvent event) { continue; } JobState jobState = getJobState(currentTasks, datafeedTask); - if (jobState == JobState.OPENED) { - runTask(datafeedTask); - } else if (jobState == JobState.OPENING) { + if (jobState == JobState.OPENING || jobHasOpenAutodetectCommunicator(currentTasks, datafeedTask) == false) { remainingTasks.add(datafeedTask); + } else if (jobState == JobState.OPENED) { + runTask(datafeedTask); } else { logger.warn("Datafeed [{}] is stopping because job [{}] state is [{}]", datafeedTask.getDatafeedId(), getJobId(datafeedTask), jobState); diff --git a/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/job/process/autodetect/AutodetectProcessManager.java b/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/job/process/autodetect/AutodetectProcessManager.java index 251a2a5224ae9..32507df53cef7 100644 --- a/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/job/process/autodetect/AutodetectProcessManager.java +++ b/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/job/process/autodetect/AutodetectProcessManager.java @@ -212,6 +212,13 @@ public void killAllProcessesOnThisNode() { */ public void persistJob(JobTask jobTask, Consumer handler) { AutodetectCommunicator communicator = getOpenAutodetectCommunicator(jobTask); + if (communicator == null) { + String message = String.format(Locale.ROOT, "Cannot persist because job [%s] does not have a corresponding autodetect process", + jobTask.getJobId()); + logger.debug(message); + handler.accept(ExceptionsHelper.conflictStatusException(message)); + return; + } communicator.persistJob((aVoid, e) -> handler.accept(e)); } @@ -239,7 +246,8 @@ public void processData(JobTask jobTask, AnalysisRegistry analysisRegistry, Inpu XContentType xContentType, DataLoadParams params, BiConsumer handler) { AutodetectCommunicator communicator = getOpenAutodetectCommunicator(jobTask); if (communicator == null) { - throw ExceptionsHelper.conflictStatusException("Cannot process data because job [" + jobTask.getJobId() + "] is not open"); + throw ExceptionsHelper.conflictStatusException("Cannot process data because job [" + jobTask.getJobId() + + "] does not have a corresponding autodetect process"); } communicator.writeToJob(input, analysisRegistry, xContentType, params, handler); } @@ -257,7 +265,8 @@ public void flushJob(JobTask jobTask, FlushJobParams params, ActionListener handler) { AutodetectCommunicator communicator = getOpenAutodetectCommunicator(jobTask); if (communicator == null) { - String message = "Cannot process update model debug config because job [" + jobTask.getJobId() + "] is not open"; + String message = "Cannot process update model debug config because job [" + jobTask.getJobId() + + "] does not have a corresponding autodetect process"; logger.debug(message); handler.accept(ExceptionsHelper.conflictStatusException(message)); return; @@ -663,6 +674,14 @@ private AutodetectCommunicator getOpenAutodetectCommunicator(JobTask jobTask) { return null; } + public boolean hasOpenAutodetectCommunicator(long jobAllocationId) { + ProcessContext processContext = processByAllocation.get(jobAllocationId); + if (processContext != null && processContext.getState() == ProcessContext.ProcessStateName.RUNNING) { + return processContext.getAutodetectCommunicator() != null; + } + return false; + } + public Optional jobOpenTime(JobTask jobTask) { AutodetectCommunicator communicator = getAutodetectCommunicator(jobTask); if (communicator == null) { 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 9bf883232c623..858f7e0f7d1c9 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 @@ -39,6 +39,7 @@ import org.elasticsearch.xpack.ml.action.TransportStartDatafeedAction.DatafeedTask; import org.elasticsearch.xpack.ml.action.TransportStartDatafeedActionTests; import org.elasticsearch.xpack.ml.job.persistence.MockClientBuilder; +import org.elasticsearch.xpack.ml.job.process.autodetect.AutodetectProcessManager; import org.elasticsearch.xpack.ml.notifications.Auditor; import org.junit.Before; import org.mockito.ArgumentCaptor; @@ -48,6 +49,7 @@ import java.util.Date; import java.util.concurrent.ExecutorService; import java.util.concurrent.ScheduledFuture; +import java.util.concurrent.atomic.AtomicBoolean; import java.util.function.Consumer; import static org.elasticsearch.xpack.ml.action.TransportOpenJobActionTests.addJobTask; @@ -74,13 +76,14 @@ public class DatafeedManagerTests extends ESTestCase { private long currentTime = 120000; private Auditor auditor; private ArgumentCaptor capturedClusterStateListener = ArgumentCaptor.forClass(ClusterStateListener.class); + private AtomicBoolean hasOpenAutodetectCommunicator; @Before @SuppressWarnings("unchecked") public void setUpTests() { Job.Builder job = createDatafeedJob().setCreateTime(new Date()); - PersistentTasksCustomMetaData.Builder tasksBuilder = PersistentTasksCustomMetaData.builder(); + PersistentTasksCustomMetaData.Builder tasksBuilder = PersistentTasksCustomMetaData.builder(); addJobTask(job.getId(), "node_id", JobState.OPENED, tasksBuilder); PersistentTasksCustomMetaData tasks = tasksBuilder.build(); DiscoveryNodes nodes = DiscoveryNodes.builder() @@ -128,7 +131,12 @@ public void setUpTests() { return null; }).when(datafeedJobBuilder).build(any(), any()); - datafeedManager = new DatafeedManager(threadPool, client, clusterService, datafeedJobBuilder, () -> currentTime, auditor); + hasOpenAutodetectCommunicator = new AtomicBoolean(true); + AutodetectProcessManager autodetectProcessManager = mock(AutodetectProcessManager.class); + doAnswer(invocation -> hasOpenAutodetectCommunicator.get()).when(autodetectProcessManager).hasOpenAutodetectCommunicator(anyLong()); + + datafeedManager = new DatafeedManager(threadPool, client, clusterService, datafeedJobBuilder, () -> currentTime, auditor, + autodetectProcessManager); verify(clusterService).addListener(capturedClusterStateListener.capture()); } @@ -259,7 +267,7 @@ public void testDatafeedTaskWaitsUntilJobIsOpened() { // 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.OPENING, tasksBuilder); addJobTask("another_job", "node_id", JobState.OPENED, tasksBuilder); ClusterState.Builder anotherJobCs = ClusterState.builder(clusterService.state()) @@ -270,7 +278,7 @@ public void testDatafeedTaskWaitsUntilJobIsOpened() { // Still no run verify(threadPool, never()).executor(MachineLearning.DATAFEED_THREAD_POOL_NAME); - tasksBuilder = PersistentTasksCustomMetaData.builder(); + 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())); @@ -278,7 +286,44 @@ public void testDatafeedTaskWaitsUntilJobIsOpened() { capturedClusterStateListener.getValue().clusterChanged( new ClusterChangedEvent("_source", jobOpenedCs.build(), anotherJobCs.build())); - // Now it should run as the job state chanded to OPENED + // Now it should run as the job state changed to OPENED + verify(threadPool, times(1)).executor(MachineLearning.DATAFEED_THREAD_POOL_NAME); + } + + public void testDatafeedTaskWaitsUntilAutodetectCommunicatorIsOpen() { + + hasOpenAutodetectCommunicator.set(false); + + PersistentTasksCustomMetaData.Builder tasksBuilder = PersistentTasksCustomMetaData.builder(); + addJobTask("job_id", "node_id", JobState.OPENED, tasksBuilder); + 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 doesn't have an open autodetect communicator + verify(threadPool, never()).executor(MachineLearning.DATAFEED_THREAD_POOL_NAME); + + tasksBuilder = PersistentTasksCustomMetaData.builder(); + addJobTask("job_id", "node_id", JobState.OPENED, tasksBuilder); + 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); + + hasOpenAutodetectCommunicator.set(true); + + capturedClusterStateListener.getValue().clusterChanged( + new ClusterChangedEvent("_source", cs.build(), anotherJobCs.build())); + + // Now it should run as the autodetect communicator is open verify(threadPool, times(1)).executor(MachineLearning.DATAFEED_THREAD_POOL_NAME); } From 19a7e0f4eb4e094ddc7f6e1195c685ef22537bb8 Mon Sep 17 00:00:00 2001 From: Benjamin Trent Date: Fri, 11 Jan 2019 08:03:41 -0600 Subject: [PATCH 110/186] ML: update .ml-state actions to support > 1 index (#37307) * ML: Updating .ml-state calls to be able to support > 1 index * Matching bulk delete behavior with dbq * Adjusting state name * refreshing indices before search * fixing line length * adjusting index expansion options --- .../persistence/AnomalyDetectorsIndex.java | 10 ++- .../AnomalyDetectorsIndexFields.java | 2 +- .../ml/integration/DeleteExpiredDataIT.java | 3 +- .../xpack/ml/MachineLearning.java | 2 +- .../ml/action/TransportDeleteJobAction.java | 9 +- .../TransportDeleteModelSnapshotAction.java | 6 +- .../ml/action/TransportOpenJobAction.java | 8 +- .../ml/job/persistence/JobDataDeleter.java | 45 ++++++---- .../job/persistence/JobResultsPersister.java | 2 +- .../job/persistence/JobResultsProvider.java | 10 +-- .../ml/job/persistence/StateStreamer.java | 23 +++-- .../ml/job/retention/UnusedStateRemover.java | 53 ++++++----- .../ml/integration/JobResultsProviderIT.java | 2 +- .../ml/integration/MlConfigMigratorIT.java | 17 ++-- .../ml/integration/NetworkDisruptionIT.java | 6 +- .../ml/job/persistence/MockClientBuilder.java | 5 ++ .../job/persistence/StateStreamerTests.java | 87 +++++++++++++------ 17 files changed, 184 insertions(+), 106 deletions(-) diff --git a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/ml/job/persistence/AnomalyDetectorsIndex.java b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/ml/job/persistence/AnomalyDetectorsIndex.java index 673e796ef7e1f..b9f887d2d49fc 100644 --- a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/ml/job/persistence/AnomalyDetectorsIndex.java +++ b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/ml/job/persistence/AnomalyDetectorsIndex.java @@ -44,7 +44,15 @@ public static String resultsWriteAlias(String jobId) { * @return The index name */ public static String jobStateIndexName() { - return AnomalyDetectorsIndexFields.STATE_INDEX_NAME; + return AnomalyDetectorsIndexFields.STATE_INDEX_PREFIX; + } + + /** + * The name pattern to capture all .ml-state prefixed indices + * @return The .ml-state index pattern + */ + public static String jobStateIndexPattern() { + return AnomalyDetectorsIndexFields.STATE_INDEX_PREFIX + "*"; } /** diff --git a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/ml/job/persistence/AnomalyDetectorsIndexFields.java b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/ml/job/persistence/AnomalyDetectorsIndexFields.java index 527ba5dc1458b..96f21876223ce 100644 --- a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/ml/job/persistence/AnomalyDetectorsIndexFields.java +++ b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/ml/job/persistence/AnomalyDetectorsIndexFields.java @@ -9,7 +9,7 @@ public final class AnomalyDetectorsIndexFields { public static final String CONFIG_INDEX = ".ml-config"; public static final String RESULTS_INDEX_PREFIX = ".ml-anomalies-"; - public static final String STATE_INDEX_NAME = ".ml-state"; + public static final String STATE_INDEX_PREFIX = ".ml-state"; public static final String RESULTS_INDEX_DEFAULT = "shared"; private AnomalyDetectorsIndexFields() {} diff --git a/x-pack/plugin/ml/qa/native-multi-node-tests/src/test/java/org/elasticsearch/xpack/ml/integration/DeleteExpiredDataIT.java b/x-pack/plugin/ml/qa/native-multi-node-tests/src/test/java/org/elasticsearch/xpack/ml/integration/DeleteExpiredDataIT.java index 2c5e7326ad5c7..f2ca43bf53c26 100644 --- a/x-pack/plugin/ml/qa/native-multi-node-tests/src/test/java/org/elasticsearch/xpack/ml/integration/DeleteExpiredDataIT.java +++ b/x-pack/plugin/ml/qa/native-multi-node-tests/src/test/java/org/elasticsearch/xpack/ml/integration/DeleteExpiredDataIT.java @@ -239,8 +239,9 @@ public void testDeleteExpiredData() throws Exception { } // Verify .ml-state doesn't contain unused state documents - SearchResponse stateDocsResponse = client().prepareSearch(AnomalyDetectorsIndex.jobStateIndexName()) + SearchResponse stateDocsResponse = client().prepareSearch(AnomalyDetectorsIndex.jobStateIndexPattern()) .setFetchSource(false) + .setTrackTotalHits(true) .setSize(10000) .get(); 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 cc259f51c1e34..11d302470c708 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 @@ -702,7 +702,7 @@ public UnaryOperator> getIndexTemplateMetaDat try (XContentBuilder stateMapping = ElasticsearchMappings.stateMapping()) { IndexTemplateMetaData stateTemplate = IndexTemplateMetaData.builder(AnomalyDetectorsIndex.jobStateIndexName()) - .patterns(Collections.singletonList(AnomalyDetectorsIndex.jobStateIndexName())) + .patterns(Collections.singletonList(AnomalyDetectorsIndex.jobStateIndexPattern())) // TODO review these settings .settings(Settings.builder() .put(IndexMetaData.SETTING_AUTO_EXPAND_REPLICAS, "0-1") diff --git a/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/action/TransportDeleteJobAction.java b/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/action/TransportDeleteJobAction.java index b186ea2184845..876f2cd1aaccd 100644 --- a/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/action/TransportDeleteJobAction.java +++ b/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/action/TransportDeleteJobAction.java @@ -14,7 +14,6 @@ import org.elasticsearch.action.admin.indices.alias.get.GetAliasesResponse; import org.elasticsearch.action.admin.indices.delete.DeleteIndexRequest; import org.elasticsearch.action.bulk.BulkItemResponse; -import org.elasticsearch.action.bulk.BulkResponse; import org.elasticsearch.action.search.SearchAction; import org.elasticsearch.action.search.SearchRequest; import org.elasticsearch.action.search.SearchResponse; @@ -387,7 +386,7 @@ private void deleteJobDocuments(ParentTaskAssigningClient parentTaskClient, Stri failureHandler); // Step 2. Delete state done, delete the quantiles - ActionListener deleteStateHandler = ActionListener.wrap( + ActionListener deleteStateHandler = ActionListener.wrap( bulkResponse -> deleteQuantiles(parentTaskClient, jobId, deleteQuantilesHandler), failureHandler); @@ -397,7 +396,7 @@ private void deleteJobDocuments(ParentTaskAssigningClient parentTaskClient, Stri private void deleteQuantiles(ParentTaskAssigningClient parentTaskClient, String jobId, ActionListener finishedHandler) { // The quantiles type and doc ID changed in v5.5 so delete both the old and new format - DeleteByQueryRequest request = new DeleteByQueryRequest(AnomalyDetectorsIndex.jobStateIndexName()); + DeleteByQueryRequest request = new DeleteByQueryRequest(AnomalyDetectorsIndex.jobStateIndexPattern()); // Just use ID here, not type, as trying to delete different types spams the logs with an exception stack trace IdsQueryBuilder query = new IdsQueryBuilder().addIds(Quantiles.documentId(jobId)); request.setQuery(query); @@ -417,7 +416,7 @@ private void deleteQuantiles(ParentTaskAssigningClient parentTaskClient, String })); } - private void deleteModelState(ParentTaskAssigningClient parentTaskClient, String jobId, ActionListener listener) { + private void deleteModelState(ParentTaskAssigningClient parentTaskClient, String jobId, ActionListener listener) { GetModelSnapshotsAction.Request request = new GetModelSnapshotsAction.Request(jobId, null); request.setPageParams(new PageParams(0, MAX_SNAPSHOTS_TO_DELETE)); executeAsyncWithOrigin(parentTaskClient, ML_ORIGIN, GetModelSnapshotsAction.INSTANCE, request, ActionListener.wrap( @@ -432,7 +431,7 @@ private void deleteModelState(ParentTaskAssigningClient parentTaskClient, String private void deleteCategorizerState(ParentTaskAssigningClient parentTaskClient, String jobId, int docNum, ActionListener finishedHandler) { // The categorizer state type and doc ID changed in v5.5 so delete both the old and new format - DeleteByQueryRequest request = new DeleteByQueryRequest(AnomalyDetectorsIndex.jobStateIndexName()); + DeleteByQueryRequest request = new DeleteByQueryRequest(AnomalyDetectorsIndex.jobStateIndexPattern()); // Just use ID here, not type, as trying to delete different types spams the logs with an exception stack trace IdsQueryBuilder query = new IdsQueryBuilder().addIds(CategorizerState.documentId(jobId, docNum)); request.setQuery(query); diff --git a/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/action/TransportDeleteModelSnapshotAction.java b/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/action/TransportDeleteModelSnapshotAction.java index 61592e4ddfa77..9904ecad0999f 100644 --- a/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/action/TransportDeleteModelSnapshotAction.java +++ b/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/action/TransportDeleteModelSnapshotAction.java @@ -7,12 +7,12 @@ import org.elasticsearch.ResourceNotFoundException; import org.elasticsearch.action.ActionListener; -import org.elasticsearch.action.bulk.BulkResponse; import org.elasticsearch.action.support.ActionFilters; import org.elasticsearch.action.support.HandledTransportAction; import org.elasticsearch.action.support.master.AcknowledgedResponse; import org.elasticsearch.client.Client; import org.elasticsearch.common.inject.Inject; +import org.elasticsearch.index.reindex.BulkByScrollResponse; import org.elasticsearch.tasks.Task; import org.elasticsearch.transport.TransportService; import org.elasticsearch.xpack.core.ml.action.DeleteModelSnapshotAction; @@ -79,9 +79,9 @@ protected void doExecute(Task task, DeleteModelSnapshotAction.Request request, // Delete the snapshot and any associated state files JobDataDeleter deleter = new JobDataDeleter(client, request.getJobId()); deleter.deleteModelSnapshots(Collections.singletonList(deleteCandidate), - new ActionListener() { + new ActionListener() { @Override - public void onResponse(BulkResponse bulkResponse) { + public void onResponse(BulkByScrollResponse bulkResponse) { String msg = Messages.getMessage(Messages.JOB_AUDIT_SNAPSHOT_DELETED, deleteCandidate.getSnapshotId(), deleteCandidate.getDescription()); 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 c81a539fb0ea4..b7b4fb3aad4c9 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 @@ -16,6 +16,7 @@ import org.elasticsearch.action.admin.indices.mapping.put.PutMappingAction; import org.elasticsearch.action.admin.indices.mapping.put.PutMappingRequest; import org.elasticsearch.action.support.ActionFilters; +import org.elasticsearch.action.support.IndicesOptions; import org.elasticsearch.action.support.master.AcknowledgedResponse; import org.elasticsearch.action.support.master.TransportMasterNodeAction; import org.elasticsearch.client.Client; @@ -368,13 +369,14 @@ static String nodeNameAndMlAttributes(DiscoveryNode node) { static String[] indicesOfInterest(String resultsIndex) { if (resultsIndex == null) { - return new String[]{AnomalyDetectorsIndex.jobStateIndexName(), MlMetaIndex.INDEX_NAME}; + return new String[]{AnomalyDetectorsIndex.jobStateIndexPattern(), MlMetaIndex.INDEX_NAME}; } - return new String[]{AnomalyDetectorsIndex.jobStateIndexName(), resultsIndex, MlMetaIndex.INDEX_NAME}; + return new String[]{AnomalyDetectorsIndex.jobStateIndexPattern(), resultsIndex, MlMetaIndex.INDEX_NAME}; } static List verifyIndicesPrimaryShardsAreActive(String resultsIndex, ClusterState clusterState) { - String[] indices = indicesOfInterest(resultsIndex); + IndexNameExpressionResolver resolver = new IndexNameExpressionResolver(); + String[] indices = resolver.concreteIndexNames(clusterState, IndicesOptions.lenientExpandOpen(), indicesOfInterest(resultsIndex)); List unavailableIndices = new ArrayList<>(indices.length); for (String index : indices) { // Indices are created on demand from templates. diff --git a/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/job/persistence/JobDataDeleter.java b/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/job/persistence/JobDataDeleter.java index b65feb68da056..c96388213c8c0 100644 --- a/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/job/persistence/JobDataDeleter.java +++ b/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/job/persistence/JobDataDeleter.java @@ -8,26 +8,28 @@ import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; import org.elasticsearch.action.ActionListener; -import org.elasticsearch.action.bulk.BulkAction; -import org.elasticsearch.action.bulk.BulkItemResponse; -import org.elasticsearch.action.bulk.BulkRequestBuilder; -import org.elasticsearch.action.bulk.BulkResponse; import org.elasticsearch.action.support.IndicesOptions; -import org.elasticsearch.action.support.WriteRequest; import org.elasticsearch.client.Client; +import org.elasticsearch.common.unit.TimeValue; import org.elasticsearch.common.util.concurrent.ThreadContext; import org.elasticsearch.index.query.ConstantScoreQueryBuilder; +import org.elasticsearch.index.query.IdsQueryBuilder; import org.elasticsearch.index.query.QueryBuilder; import org.elasticsearch.index.query.QueryBuilders; +import org.elasticsearch.index.reindex.BulkByScrollResponse; +import org.elasticsearch.index.reindex.BulkByScrollTask; import org.elasticsearch.index.reindex.DeleteByQueryAction; import org.elasticsearch.index.reindex.DeleteByQueryRequest; import org.elasticsearch.xpack.core.ml.job.persistence.AnomalyDetectorsIndex; -import org.elasticsearch.xpack.core.ml.job.persistence.ElasticsearchMappings; import org.elasticsearch.xpack.core.ml.job.process.autodetect.state.ModelSnapshot; import org.elasticsearch.xpack.core.ml.job.results.Result; +import java.util.ArrayList; +import java.util.Collections; +import java.util.HashSet; import java.util.List; import java.util.Objects; +import java.util.Set; import static org.elasticsearch.xpack.core.ClientHelper.ML_ORIGIN; import static org.elasticsearch.xpack.core.ClientHelper.executeAsyncWithOrigin; @@ -50,27 +52,34 @@ public JobDataDeleter(Client client, String jobId) { * * @param modelSnapshots the model snapshots to delete */ - public void deleteModelSnapshots(List modelSnapshots, ActionListener listener) { + public void deleteModelSnapshots(List modelSnapshots, ActionListener listener) { if (modelSnapshots.isEmpty()) { - listener.onResponse(new BulkResponse(new BulkItemResponse[0], 0L)); + listener.onResponse(new BulkByScrollResponse(TimeValue.ZERO, + new BulkByScrollTask.Status(Collections.emptyList(), null), + Collections.emptyList(), + Collections.emptyList(), + false)); return; } - String stateIndexName = AnomalyDetectorsIndex.jobStateIndexName(); + String stateIndexName = AnomalyDetectorsIndex.jobStateIndexPattern(); - BulkRequestBuilder bulkRequestBuilder = client.prepareBulk(); + List idsToDelete = new ArrayList<>(); + Set indices = new HashSet<>(); + indices.add(stateIndexName); for (ModelSnapshot modelSnapshot : modelSnapshots) { - for (String stateDocId : modelSnapshot.stateDocumentIds()) { - bulkRequestBuilder.add(client.prepareDelete(stateIndexName, ElasticsearchMappings.DOC_TYPE, stateDocId)); - } - - bulkRequestBuilder.add(client.prepareDelete(AnomalyDetectorsIndex.jobResultsAliasedName(modelSnapshot.getJobId()), - ElasticsearchMappings.DOC_TYPE, ModelSnapshot.documentId(modelSnapshot))); + idsToDelete.addAll(modelSnapshot.stateDocumentIds()); + idsToDelete.add(ModelSnapshot.documentId(modelSnapshot)); + indices.add(AnomalyDetectorsIndex.jobResultsAliasedName(modelSnapshot.getJobId())); } - bulkRequestBuilder.setRefreshPolicy(WriteRequest.RefreshPolicy.IMMEDIATE); + DeleteByQueryRequest deleteByQueryRequest = new DeleteByQueryRequest(indices.toArray(new String[0])) + .setRefresh(true) + .setIndicesOptions(IndicesOptions.lenientExpandOpen()) + .setQuery(new IdsQueryBuilder().addIds(idsToDelete.toArray(new String[0]))); + try { - executeAsyncWithOrigin(client, ML_ORIGIN, BulkAction.INSTANCE, bulkRequestBuilder.request(), listener); + executeAsyncWithOrigin(client, ML_ORIGIN, DeleteByQueryAction.INSTANCE, deleteByQueryRequest, listener); } catch (Exception e) { listener.onFailure(e); } diff --git a/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/job/persistence/JobResultsPersister.java b/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/job/persistence/JobResultsPersister.java index 32a52410f2d25..e57d85aefa72c 100644 --- a/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/job/persistence/JobResultsPersister.java +++ b/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/job/persistence/JobResultsPersister.java @@ -305,7 +305,7 @@ public void commitResultWrites(String jobId) { * @param jobId The job Id * */ public void commitStateWrites(String jobId) { - String indexName = AnomalyDetectorsIndex.jobStateIndexName(); + String indexName = AnomalyDetectorsIndex.jobStateIndexPattern(); // Refresh should wait for Lucene to make the data searchable logger.trace("[{}] ES API CALL: refresh index {}", jobId, indexName); RefreshRequest refreshRequest = new RefreshRequest(indexName); diff --git a/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/job/persistence/JobResultsProvider.java b/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/job/persistence/JobResultsProvider.java index 17d173bf22fc6..b942c49c14e73 100644 --- a/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/job/persistence/JobResultsProvider.java +++ b/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/job/persistence/JobResultsProvider.java @@ -157,14 +157,14 @@ public JobResultsProvider(Client client, Settings settings) { */ public void checkForLeftOverDocuments(Job job, ActionListener listener) { - SearchRequestBuilder stateDocSearch = client.prepareSearch(AnomalyDetectorsIndex.jobStateIndexName()) + SearchRequestBuilder stateDocSearch = client.prepareSearch(AnomalyDetectorsIndex.jobStateIndexPattern()) .setQuery(QueryBuilders.idsQuery().addIds(CategorizerState.documentId(job.getId(), 1), CategorizerState.v54DocumentId(job.getId(), 1))) - .setIndicesOptions(IndicesOptions.lenientExpandOpen()); + .setIndicesOptions(IndicesOptions.strictExpand()); - SearchRequestBuilder quantilesDocSearch = client.prepareSearch(AnomalyDetectorsIndex.jobStateIndexName()) + SearchRequestBuilder quantilesDocSearch = client.prepareSearch(AnomalyDetectorsIndex.jobStateIndexPattern()) .setQuery(QueryBuilders.idsQuery().addIds(Quantiles.documentId(job.getId()), Quantiles.v54DocumentId(job.getId()))) - .setIndicesOptions(IndicesOptions.lenientExpandOpen()); + .setIndicesOptions(IndicesOptions.strictExpand()); String resultsIndexName = job.getResultsIndexName(); SearchRequestBuilder resultDocSearch = client.prepareSearch(resultsIndexName) @@ -396,7 +396,7 @@ public void getAutodetectParams(Job job, Consumer consumer, Co AutodetectParams.Builder paramsBuilder = new AutodetectParams.Builder(job.getId()); String resultsIndex = AnomalyDetectorsIndex.jobResultsAliasedName(jobId); - String stateIndex = AnomalyDetectorsIndex.jobStateIndexName(); + String stateIndex = AnomalyDetectorsIndex.jobStateIndexPattern(); MultiSearchRequestBuilder msearch = client.prepareMultiSearch() .add(createLatestDataCountsSearch(resultsIndex, jobId)) diff --git a/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/job/persistence/StateStreamer.java b/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/job/persistence/StateStreamer.java index 9a2c6a4938b2f..3ed91412042c7 100644 --- a/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/job/persistence/StateStreamer.java +++ b/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/job/persistence/StateStreamer.java @@ -9,10 +9,11 @@ import org.apache.logging.log4j.Logger; import org.apache.lucene.util.BytesRef; import org.apache.lucene.util.BytesRefIterator; -import org.elasticsearch.action.get.GetResponse; +import org.elasticsearch.action.search.SearchResponse; import org.elasticsearch.client.Client; import org.elasticsearch.common.bytes.BytesReference; import org.elasticsearch.common.util.concurrent.ThreadContext; +import org.elasticsearch.index.query.QueryBuilders; import org.elasticsearch.xpack.core.ml.job.persistence.AnomalyDetectorsIndex; import org.elasticsearch.xpack.core.ml.job.persistence.ElasticsearchMappings; import org.elasticsearch.xpack.core.ml.job.process.autodetect.state.CategorizerState; @@ -62,7 +63,7 @@ public void cancel() { * @param restoreStream the stream to write the state to */ public void restoreStateToStream(String jobId, ModelSnapshot modelSnapshot, OutputStream restoreStream) throws IOException { - String indexName = AnomalyDetectorsIndex.jobStateIndexName(); + String indexName = AnomalyDetectorsIndex.jobStateIndexPattern(); // First try to restore model state. for (String stateDocId : modelSnapshot.stateDocumentIds()) { @@ -73,13 +74,16 @@ public void restoreStateToStream(String jobId, ModelSnapshot modelSnapshot, Outp LOGGER.trace("ES API CALL: get ID {} from index {}", stateDocId, indexName); try (ThreadContext.StoredContext ignore = stashWithOrigin(client.threadPool().getThreadContext(), ML_ORIGIN)) { - GetResponse stateResponse = client.prepareGet(indexName, ElasticsearchMappings.DOC_TYPE, stateDocId).get(); - if (!stateResponse.isExists()) { + SearchResponse stateResponse = client.prepareSearch(indexName) + .setTypes(ElasticsearchMappings.DOC_TYPE) + .setSize(1) + .setQuery(QueryBuilders.idsQuery().addIds(stateDocId)).get(); + if (stateResponse.getHits().getHits().length == 0) { LOGGER.error("Expected {} documents for model state for {} snapshot {} but failed to find {}", modelSnapshot.getSnapshotDocCount(), jobId, modelSnapshot.getSnapshotId(), stateDocId); break; } - writeStateToStream(stateResponse.getSourceAsBytesRef(), restoreStream); + writeStateToStream(stateResponse.getHits().getAt(0).getSourceRef(), restoreStream); } } @@ -97,11 +101,14 @@ public void restoreStateToStream(String jobId, ModelSnapshot modelSnapshot, Outp LOGGER.trace("ES API CALL: get ID {} from index {}", docId, indexName); try (ThreadContext.StoredContext ignore = stashWithOrigin(client.threadPool().getThreadContext(), ML_ORIGIN)) { - GetResponse stateResponse = client.prepareGet(indexName, ElasticsearchMappings.DOC_TYPE, docId).get(); - if (!stateResponse.isExists()) { + SearchResponse stateResponse = client.prepareSearch(indexName) + .setTypes(ElasticsearchMappings.DOC_TYPE) + .setSize(1) + .setQuery(QueryBuilders.idsQuery().addIds(docId)).get(); + if (stateResponse.getHits().getHits().length == 0) { break; } - writeStateToStream(stateResponse.getSourceAsBytesRef(), restoreStream); + writeStateToStream(stateResponse.getHits().getAt(0).getSourceRef(), restoreStream); } } diff --git a/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/job/retention/UnusedStateRemover.java b/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/job/retention/UnusedStateRemover.java index 66030c56823c9..249d3761b5842 100644 --- a/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/job/retention/UnusedStateRemover.java +++ b/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/job/retention/UnusedStateRemover.java @@ -8,11 +8,13 @@ import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; import org.elasticsearch.action.ActionListener; -import org.elasticsearch.action.bulk.BulkRequestBuilder; -import org.elasticsearch.action.bulk.BulkResponse; -import org.elasticsearch.action.delete.DeleteRequest; +import org.elasticsearch.action.support.IndicesOptions; import org.elasticsearch.client.Client; import org.elasticsearch.cluster.service.ClusterService; +import org.elasticsearch.common.Strings; +import org.elasticsearch.index.query.QueryBuilders; +import org.elasticsearch.index.reindex.DeleteByQueryAction; +import org.elasticsearch.index.reindex.DeleteByQueryRequest; import org.elasticsearch.xpack.core.ml.MlMetadata; import org.elasticsearch.xpack.core.ml.job.config.Job; import org.elasticsearch.xpack.core.ml.job.persistence.AnomalyDetectorsIndex; @@ -23,6 +25,7 @@ import org.elasticsearch.xpack.ml.job.persistence.BatchedJobsIterator; import org.elasticsearch.xpack.ml.job.persistence.BatchedStateDocIdsIterator; +import java.util.ArrayList; import java.util.Arrays; import java.util.Deque; import java.util.HashSet; @@ -51,9 +54,9 @@ public UnusedStateRemover(Client client, ClusterService clusterService) { @Override public void remove(ActionListener listener) { try { - BulkRequestBuilder deleteUnusedStateRequestBuilder = findUnusedStateDocs(); - if (deleteUnusedStateRequestBuilder.numberOfActions() > 0) { - executeDeleteUnusedStateDocs(deleteUnusedStateRequestBuilder, listener); + List unusedStateDocIds = findUnusedStateDocIds(); + if (unusedStateDocIds.size() > 0) { + executeDeleteUnusedStateDocs(unusedStateDocIds, listener); } else { listener.onResponse(true); } @@ -62,10 +65,11 @@ public void remove(ActionListener listener) { } } - private BulkRequestBuilder findUnusedStateDocs() { + private List findUnusedStateDocIds() { Set jobIds = getJobIds(); - BulkRequestBuilder deleteUnusedStateRequestBuilder = client.prepareBulk(); - BatchedStateDocIdsIterator stateDocIdsIterator = new BatchedStateDocIdsIterator(client, AnomalyDetectorsIndex.jobStateIndexName()); + List stateDocIdsToDelete = new ArrayList<>(); + BatchedStateDocIdsIterator stateDocIdsIterator = new BatchedStateDocIdsIterator(client, + AnomalyDetectorsIndex.jobStateIndexPattern()); while (stateDocIdsIterator.hasNext()) { Deque stateDocIds = stateDocIdsIterator.next(); for (String stateDocId : stateDocIds) { @@ -75,12 +79,11 @@ private BulkRequestBuilder findUnusedStateDocs() { continue; } if (jobIds.contains(jobId) == false) { - deleteUnusedStateRequestBuilder.add(new DeleteRequest( - AnomalyDetectorsIndex.jobStateIndexName(), ElasticsearchMappings.DOC_TYPE, stateDocId)); + stateDocIdsToDelete.add(stateDocId); } } } - return deleteUnusedStateRequestBuilder; + return stateDocIdsToDelete; } private Set getJobIds() { @@ -98,27 +101,29 @@ private Set getJobIds() { return jobIds; } - private void executeDeleteUnusedStateDocs(BulkRequestBuilder deleteUnusedStateRequestBuilder, ActionListener listener) { + private void executeDeleteUnusedStateDocs(List unusedDocIds, ActionListener listener) { LOGGER.info("Found [{}] unused state documents; attempting to delete", - deleteUnusedStateRequestBuilder.numberOfActions()); - deleteUnusedStateRequestBuilder.execute(new ActionListener() { - @Override - public void onResponse(BulkResponse bulkItemResponses) { - if (bulkItemResponses.hasFailures()) { + unusedDocIds.size()); + DeleteByQueryRequest deleteByQueryRequest = new DeleteByQueryRequest(AnomalyDetectorsIndex.jobStateIndexPattern()) + .types(ElasticsearchMappings.DOC_TYPE) + .setIndicesOptions(IndicesOptions.lenientExpandOpen()) + .setQuery(QueryBuilders.idsQuery().addIds(unusedDocIds.toArray(new String[0]))); + client.execute(DeleteByQueryAction.INSTANCE, deleteByQueryRequest, ActionListener.wrap( + response -> { + if (response.getBulkFailures().size() > 0 || response.getSearchFailures().size() > 0) { LOGGER.error("Some unused state documents could not be deleted due to failures: {}", - bulkItemResponses.buildFailureMessage()); + Strings.collectionToCommaDelimitedString(response.getBulkFailures()) + + "," + Strings.collectionToCommaDelimitedString(response.getSearchFailures())); } else { LOGGER.info("Successfully deleted all unused state documents"); } listener.onResponse(true); - } - - @Override - public void onFailure(Exception e) { + }, + e -> { LOGGER.error("Error deleting unused model state documents: ", e); listener.onFailure(e); } - }); + )); } private static class JobIdExtractor { diff --git a/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/integration/JobResultsProviderIT.java b/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/integration/JobResultsProviderIT.java index 3843181a0bc3c..02cc738477cfb 100644 --- a/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/integration/JobResultsProviderIT.java +++ b/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/integration/JobResultsProviderIT.java @@ -379,7 +379,7 @@ public void testGetAutodetectParams() throws Exception { Quantiles quantiles = new Quantiles(jobId, new Date(), "quantile-state"); indexQuantiles(quantiles); - client().admin().indices().prepareRefresh(MlMetaIndex.INDEX_NAME, AnomalyDetectorsIndex.jobStateIndexName(), + client().admin().indices().prepareRefresh(MlMetaIndex.INDEX_NAME, AnomalyDetectorsIndex.jobStateIndexPattern(), AnomalyDetectorsIndex.jobResultsAliasedName(jobId)).get(); diff --git a/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/integration/MlConfigMigratorIT.java b/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/integration/MlConfigMigratorIT.java index 87c0e4ac824ce..33b2484766669 100644 --- a/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/integration/MlConfigMigratorIT.java +++ b/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/integration/MlConfigMigratorIT.java @@ -6,8 +6,8 @@ package org.elasticsearch.xpack.ml.integration; import org.elasticsearch.Version; -import org.elasticsearch.action.get.GetResponse; import org.elasticsearch.action.index.IndexResponse; +import org.elasticsearch.action.search.SearchResponse; import org.elasticsearch.cluster.ClusterName; import org.elasticsearch.cluster.ClusterState; import org.elasticsearch.cluster.ClusterStateUpdateTask; @@ -28,6 +28,7 @@ import org.elasticsearch.common.xcontent.XContentParser; import org.elasticsearch.common.xcontent.XContentType; import org.elasticsearch.index.Index; +import org.elasticsearch.index.query.QueryBuilders; import org.elasticsearch.index.shard.ShardId; import org.elasticsearch.xpack.core.ml.MlMetadata; import org.elasticsearch.xpack.core.ml.datafeed.DatafeedConfig; @@ -52,6 +53,7 @@ import static org.elasticsearch.xpack.core.ml.job.config.JobTests.buildJobBuilder; import static org.hamcrest.Matchers.empty; +import static org.hamcrest.Matchers.greaterThan; import static org.hamcrest.Matchers.hasSize; import static org.hamcrest.Matchers.is; import static org.mockito.Matchers.eq; @@ -308,12 +310,17 @@ public void testMigrateConfigsWithoutTasks_GivenMigrationIsDisabled() throws Int } public void assertSnapshot(MlMetadata expectedMlMetadata) throws IOException { - GetResponse getResponse = client() - .prepareGet(AnomalyDetectorsIndex.jobStateIndexName(), ElasticsearchMappings.DOC_TYPE, "ml-config").get(); + client().admin().indices().prepareRefresh(AnomalyDetectorsIndex.jobStateIndexPattern()).execute(); + SearchResponse searchResponse = client() + .prepareSearch(AnomalyDetectorsIndex.jobStateIndexPattern()) + .setTypes(ElasticsearchMappings.DOC_TYPE) + .setSize(1) + .setQuery(QueryBuilders.idsQuery().addIds("ml-config")) + .get(); - assertTrue(getResponse.isExists()); + assertThat(searchResponse.getHits().getHits().length, greaterThan(0)); - try (InputStream stream = getResponse.getSourceAsBytesRef().streamInput(); + try (InputStream stream = searchResponse.getHits().getAt(0).getSourceRef().streamInput(); XContentParser parser = XContentFactory.xContent(XContentType.JSON) .createParser(NamedXContentRegistry.EMPTY, LoggingDeprecationHandler.INSTANCE, stream)) { MlMetadata recoveredMeta = MlMetadata.LENIENT_PARSER.apply(parser, null).build(); diff --git a/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/integration/NetworkDisruptionIT.java b/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/integration/NetworkDisruptionIT.java index 4ab369120e019..3304963ae3570 100644 --- a/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/integration/NetworkDisruptionIT.java +++ b/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/integration/NetworkDisruptionIT.java @@ -93,8 +93,9 @@ public void testJobRelocation() throws Exception { assertEquals(newJobNode, finalJobNode); // The job running on the original node should have been killed, and hence should not have persisted quantiles - SearchResponse searchResponse = client().prepareSearch(AnomalyDetectorsIndex.jobStateIndexName()) + SearchResponse searchResponse = client().prepareSearch(AnomalyDetectorsIndex.jobStateIndexPattern()) .setQuery(QueryBuilders.idsQuery().addIds(Quantiles.documentId(job.getId()))) + .setTrackTotalHits(true) .setIndicesOptions(IndicesOptions.lenientExpandOpen()).execute().actionGet(); assertEquals(0L, searchResponse.getHits().getTotalHits().value); @@ -103,8 +104,9 @@ public void testJobRelocation() throws Exception { assertTrue(closeJobResponse.isClosed()); // The relocated job was closed rather than killed, and hence should have persisted quantiles - searchResponse = client().prepareSearch(AnomalyDetectorsIndex.jobStateIndexName()) + searchResponse = client().prepareSearch(AnomalyDetectorsIndex.jobStateIndexPattern()) .setQuery(QueryBuilders.idsQuery().addIds(Quantiles.documentId(job.getId()))) + .setTrackTotalHits(true) .setIndicesOptions(IndicesOptions.lenientExpandOpen()).execute().actionGet(); assertEquals(1L, searchResponse.getHits().getTotalHits().value); } diff --git a/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/job/persistence/MockClientBuilder.java b/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/job/persistence/MockClientBuilder.java index 627465f1d4f21..4a4284e2d1456 100644 --- a/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/job/persistence/MockClientBuilder.java +++ b/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/job/persistence/MockClientBuilder.java @@ -271,6 +271,11 @@ public MockClientBuilder prepareSearch(String index, String type, int from, int return this; } + public MockClientBuilder prepareSearches(String index, SearchRequestBuilder first, SearchRequestBuilder... searches) { + when(client.prepareSearch(eq(index))).thenReturn(first, searches); + return this; + } + /** * Creates a {@link SearchResponse} with a {@link SearchHit} for each element of {@code docs} * @param indexName Index being searched diff --git a/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/job/persistence/StateStreamerTests.java b/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/job/persistence/StateStreamerTests.java index beae3959308e9..1629a8bcdbad5 100644 --- a/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/job/persistence/StateStreamerTests.java +++ b/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/job/persistence/StateStreamerTests.java @@ -5,14 +5,20 @@ */ package org.elasticsearch.xpack.ml.job.persistence; -import org.elasticsearch.action.get.GetResponse; +import org.elasticsearch.action.search.SearchRequestBuilder; +import org.elasticsearch.action.search.SearchResponse; import org.elasticsearch.client.Client; import org.elasticsearch.common.bytes.BytesReference; import org.elasticsearch.common.xcontent.XContentFactory; +import org.elasticsearch.index.query.QueryBuilder; +import org.elasticsearch.index.query.QueryBuilders; import org.elasticsearch.mock.orig.Mockito; +import org.elasticsearch.search.SearchHit; +import org.elasticsearch.search.SearchHits; +import org.elasticsearch.search.sort.SortBuilder; +import org.elasticsearch.search.sort.SortOrder; import org.elasticsearch.test.ESTestCase; import org.elasticsearch.xpack.core.ml.job.persistence.AnomalyDetectorsIndex; -import org.elasticsearch.xpack.core.ml.job.persistence.ElasticsearchMappings; import org.elasticsearch.xpack.core.ml.job.process.autodetect.state.CategorizerState; import org.elasticsearch.xpack.core.ml.job.process.autodetect.state.ModelSnapshot; import org.elasticsearch.xpack.core.ml.job.process.autodetect.state.ModelState; @@ -21,9 +27,14 @@ import java.io.IOException; import java.io.OutputStream; import java.nio.charset.StandardCharsets; +import java.util.Collections; import java.util.HashMap; +import java.util.List; import java.util.Map; +import static org.mockito.Matchers.any; +import static org.mockito.Matchers.anyInt; +import static org.mockito.Matchers.eq; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.when; @@ -36,25 +47,25 @@ public void testRestoreStateToStream() throws Exception { String snapshotId = "123"; Map categorizerState = new HashMap<>(); categorizerState.put("catName", "catVal"); - GetResponse categorizerStateGetResponse1 = createGetResponse(true, categorizerState); - GetResponse categorizerStateGetResponse2 = createGetResponse(false, null); - Map modelState = new HashMap<>(); - modelState.put("modName", "modVal1"); - GetResponse modelStateGetResponse1 = createGetResponse(true, modelState); - modelState.put("modName", "modVal2"); - GetResponse modelStateGetResponse2 = createGetResponse(true, modelState); - - MockClientBuilder clientBuilder = new MockClientBuilder(CLUSTER_NAME).addClusterStatusYellowResponse() - .prepareGet(AnomalyDetectorsIndex.jobStateIndexName(), ElasticsearchMappings.DOC_TYPE, - CategorizerState.documentId(JOB_ID, 1), categorizerStateGetResponse1) - .prepareGet(AnomalyDetectorsIndex.jobStateIndexName(), ElasticsearchMappings.DOC_TYPE, - CategorizerState.documentId(JOB_ID, 2), categorizerStateGetResponse2) - .prepareGet(AnomalyDetectorsIndex.jobStateIndexName(), ElasticsearchMappings.DOC_TYPE, - ModelState.documentId(JOB_ID, snapshotId, 1), modelStateGetResponse1) - .prepareGet(AnomalyDetectorsIndex.jobStateIndexName(), ElasticsearchMappings.DOC_TYPE, - ModelState.documentId(JOB_ID, snapshotId, 2), modelStateGetResponse2); + Map modelState1 = new HashMap<>(); + modelState1.put("modName1", "modVal1"); + Map modelState2 = new HashMap<>(); + modelState2.put("modName2", "modVal2"); + SearchRequestBuilder builder1 = prepareSearchBuilder(createSearchResponse(Collections.singletonList(modelState1)), + QueryBuilders.idsQuery().addIds(ModelState.documentId(JOB_ID, snapshotId, 1))); + SearchRequestBuilder builder2 = prepareSearchBuilder(createSearchResponse(Collections.singletonList(modelState2)), + QueryBuilders.idsQuery().addIds(ModelState.documentId(JOB_ID, snapshotId, 2))); + SearchRequestBuilder builder3 = prepareSearchBuilder(createSearchResponse(Collections.singletonList(categorizerState)), + QueryBuilders.idsQuery().addIds(CategorizerState.documentId(JOB_ID, 1))); + SearchRequestBuilder builder4 = prepareSearchBuilder(createSearchResponse(Collections.emptyList()), + QueryBuilders.idsQuery().addIds(CategorizerState.documentId(JOB_ID, 2))); + + MockClientBuilder clientBuilder = new MockClientBuilder(CLUSTER_NAME) + .addClusterStatusYellowResponse() + .prepareSearches(AnomalyDetectorsIndex.jobStateIndexPattern(), builder1, builder2, builder3, builder4); + ModelSnapshot modelSnapshot = new ModelSnapshot.Builder(JOB_ID).setSnapshotId(snapshotId).setSnapshotDocCount(2).build(); ByteArrayOutputStream stream = new ByteArrayOutputStream(); @@ -64,8 +75,8 @@ public void testRestoreStateToStream() throws Exception { String[] restoreData = stream.toString(StandardCharsets.UTF_8.name()).split("\0"); assertEquals(3, restoreData.length); - assertEquals("{\"modName\":\"modVal1\"}", restoreData[0]); - assertEquals("{\"modName\":\"modVal2\"}", restoreData[1]); + assertEquals("{\"modName1\":\"modVal1\"}", restoreData[0]); + assertEquals("{\"modName2\":\"modVal2\"}", restoreData[1]); assertEquals("{\"catName\":\"catVal\"}", restoreData[2]); } @@ -80,10 +91,32 @@ public void testCancelBeforeRestoreWasCalled() throws IOException { Mockito.verifyNoMoreInteractions(outputStream); } - private static GetResponse createGetResponse(boolean exists, Map source) throws IOException { - GetResponse getResponse = mock(GetResponse.class); - when(getResponse.isExists()).thenReturn(exists); - when(getResponse.getSourceAsBytesRef()).thenReturn(BytesReference.bytes(XContentFactory.jsonBuilder().map(source))); - return getResponse; + private static SearchResponse createSearchResponse(List> source) throws IOException { + SearchResponse searchResponse = mock(SearchResponse.class); + SearchHit[] hits = new SearchHit[source.size()]; + int i = 0; + for (Map s : source) { + SearchHit hit = new SearchHit(1).sourceRef(BytesReference.bytes(XContentFactory.jsonBuilder().map(s))); + hits[i++] = hit; + } + SearchHits searchHits = new SearchHits(hits, null, (float)0.0); + when(searchResponse.getHits()).thenReturn(searchHits); + return searchResponse; + } + + private static SearchRequestBuilder prepareSearchBuilder(SearchResponse response, QueryBuilder queryBuilder) { + SearchRequestBuilder builder = mock(SearchRequestBuilder.class); + when(builder.setTypes(any())).thenReturn(builder); + when(builder.addSort(any(SortBuilder.class))).thenReturn(builder); + when(builder.setQuery(queryBuilder)).thenReturn(builder); + when(builder.setPostFilter(any())).thenReturn(builder); + when(builder.setFrom(anyInt())).thenReturn(builder); + when(builder.setSize(anyInt())).thenReturn(builder); + when(builder.setFetchSource(eq(true))).thenReturn(builder); + when(builder.addDocValueField(any(String.class))).thenReturn(builder); + when(builder.addDocValueField(any(String.class), any(String.class))).thenReturn(builder); + when(builder.addSort(any(String.class), any(SortOrder.class))).thenReturn(builder); + when(builder.get()).thenReturn(response); + return builder; } -} \ No newline at end of file +} From d21df2a17a2ed96b0f307505ba0c6c66696e05dd Mon Sep 17 00:00:00 2001 From: Boaz Leskes Date: Fri, 11 Jan 2019 06:23:55 -0800 Subject: [PATCH 111/186] Use Sequence number powered OCC for processing updates (#37308) Updates perform realtime get, perform the requested update and then index the document again using optimistic concurrency control. This PR changes the logic to use sequence numbers instead of versioning. Note that the current versioning logic isn't suffering from the same problem as external OCC requests because the get and indexing is always done on the same primary. Relates #36148 Relates #10708 --- .../action/update/UpdateHelper.java | 24 +++++-------------- .../action/update/UpdateRequestTests.java | 19 --------------- 2 files changed, 6 insertions(+), 37 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/action/update/UpdateHelper.java b/server/src/main/java/org/elasticsearch/action/update/UpdateHelper.java index 255161c8f3200..a8a5fb8f72f30 100644 --- a/server/src/main/java/org/elasticsearch/action/update/UpdateHelper.java +++ b/server/src/main/java/org/elasticsearch/action/update/UpdateHelper.java @@ -163,19 +163,6 @@ Result prepareUpsert(ShardId shardId, UpdateRequest request, final GetResult get return new Result(indexRequest, DocWriteResponse.Result.CREATED, null, null); } - /** - * Calculate the version to use for the update request, using either the existing version if internal versioning is used, or the get - * result document's version if the version type is "FORCE". - */ - static long calculateUpdateVersion(UpdateRequest request, GetResult getResult) { - if (request.versionType() != VersionType.INTERNAL) { - assert request.versionType() == VersionType.FORCE; - return request.version(); // remember, match_any is excluded by the conflict test - } else { - return getResult.getVersion(); - } - } - /** * Calculate a routing value to be used, either the included index request's routing, or retrieved document's routing when defined. */ @@ -195,7 +182,6 @@ static String calculateRouting(GetResult getResult, @Nullable IndexRequest updat * containing a new {@code IndexRequest} to be executed on the primary and replicas. */ Result prepareUpdateIndexRequest(ShardId shardId, UpdateRequest request, GetResult getResult, boolean detectNoop) { - final long updateVersion = calculateUpdateVersion(request, getResult); final IndexRequest currentRequest = request.doc(); final String routing = calculateRouting(getResult, currentRequest); final Tuple> sourceAndContent = XContentHelper.convertToMap(getResult.internalSourceRef(), true); @@ -215,7 +201,8 @@ Result prepareUpdateIndexRequest(ShardId shardId, UpdateRequest request, GetResu } else { final IndexRequest finalIndexRequest = Requests.indexRequest(request.index()) .type(request.type()).id(request.id()).routing(routing) - .source(updatedSourceAsMap, updateSourceContentType).version(updateVersion).versionType(request.versionType()) + .source(updatedSourceAsMap, updateSourceContentType) + .setIfSeqNo(getResult.getSeqNo()).setIfPrimaryTerm(getResult.getPrimaryTerm()) .waitForActiveShards(request.waitForActiveShards()).timeout(request.timeout()) .setRefreshPolicy(request.getRefreshPolicy()); return new Result(finalIndexRequest, DocWriteResponse.Result.UPDATED, updatedSourceAsMap, updateSourceContentType); @@ -228,7 +215,6 @@ Result prepareUpdateIndexRequest(ShardId shardId, UpdateRequest request, GetResu * primary and replicas. */ Result prepareUpdateScriptRequest(ShardId shardId, UpdateRequest request, GetResult getResult, LongSupplier nowInMillis) { - final long updateVersion = calculateUpdateVersion(request, getResult); final IndexRequest currentRequest = request.doc(); final String routing = calculateRouting(getResult, currentRequest); final Tuple> sourceAndContent = XContentHelper.convertToMap(getResult.internalSourceRef(), true); @@ -256,14 +242,16 @@ Result prepareUpdateScriptRequest(ShardId shardId, UpdateRequest request, GetRes case INDEX: final IndexRequest indexRequest = Requests.indexRequest(request.index()) .type(request.type()).id(request.id()).routing(routing) - .source(updatedSourceAsMap, updateSourceContentType).version(updateVersion).versionType(request.versionType()) + .source(updatedSourceAsMap, updateSourceContentType) + .setIfSeqNo(getResult.getSeqNo()).setIfPrimaryTerm(getResult.getPrimaryTerm()) .waitForActiveShards(request.waitForActiveShards()).timeout(request.timeout()) .setRefreshPolicy(request.getRefreshPolicy()); return new Result(indexRequest, DocWriteResponse.Result.UPDATED, updatedSourceAsMap, updateSourceContentType); case DELETE: DeleteRequest deleteRequest = Requests.deleteRequest(request.index()) .type(request.type()).id(request.id()).routing(routing) - .version(updateVersion).versionType(request.versionType()).waitForActiveShards(request.waitForActiveShards()) + .setIfSeqNo(getResult.getSeqNo()).setIfPrimaryTerm(getResult.getPrimaryTerm()) + .waitForActiveShards(request.waitForActiveShards()) .timeout(request.timeout()).setRefreshPolicy(request.getRefreshPolicy()); return new Result(deleteRequest, DocWriteResponse.Result.DELETED, updatedSourceAsMap, updateSourceContentType); default: diff --git a/server/src/test/java/org/elasticsearch/action/update/UpdateRequestTests.java b/server/src/test/java/org/elasticsearch/action/update/UpdateRequestTests.java index 4534fbe23e3c2..5a734352eafb2 100644 --- a/server/src/test/java/org/elasticsearch/action/update/UpdateRequestTests.java +++ b/server/src/test/java/org/elasticsearch/action/update/UpdateRequestTests.java @@ -38,7 +38,6 @@ import org.elasticsearch.common.xcontent.XContentType; import org.elasticsearch.common.xcontent.json.JsonXContent; import org.elasticsearch.env.Environment; -import org.elasticsearch.index.VersionType; import org.elasticsearch.index.get.GetResult; import org.elasticsearch.index.shard.ShardId; import org.elasticsearch.script.MockScriptEngine; @@ -570,24 +569,6 @@ public void testRoutingExtraction() throws Exception { assertThat(UpdateHelper.calculateRouting(getResult, indexRequest), equalTo("routing1")); } - @SuppressWarnings("deprecated") // VersionType.FORCE is deprecated - public void testCalculateUpdateVersion() throws Exception { - long randomVersion = randomIntBetween(0, 100); - GetResult getResult = new GetResult("test", "type", "1", 0, 1, randomVersion, true, new BytesArray("{}"), null); - - UpdateRequest request = new UpdateRequest("test", "type1", "1"); - long version = UpdateHelper.calculateUpdateVersion(request, getResult); - - // Use the get result's version - assertThat(version, equalTo(randomVersion)); - - request = new UpdateRequest("test", "type1", "1").versionType(VersionType.FORCE).version(1337); - version = UpdateHelper.calculateUpdateVersion(request, getResult); - - // Use the forced update request version - assertThat(version, equalTo(1337L)); - } - public void testNoopDetection() throws Exception { ShardId shardId = new ShardId("test", "", 0); GetResult getResult = new GetResult("test", "type", "1", 0, 1, 0, true, From fad63298082cccaba6c681aec896e7d636998cfe Mon Sep 17 00:00:00 2001 From: Jake Landis Date: Fri, 11 Jan 2019 08:58:41 -0600 Subject: [PATCH 112/186] add link to freeze from cold phase (#37332) --- docs/reference/ilm/policy-definitions.asciidoc | 1 + 1 file changed, 1 insertion(+) diff --git a/docs/reference/ilm/policy-definitions.asciidoc b/docs/reference/ilm/policy-definitions.asciidoc index ab2c0a039f9d0..1253c87b3943e 100644 --- a/docs/reference/ilm/policy-definitions.asciidoc +++ b/docs/reference/ilm/policy-definitions.asciidoc @@ -93,6 +93,7 @@ The below list shows the actions which are available in each phase. - <> * Cold - <> + - <> * Delete - <> From f4abf9628aa35138f892db93be5a7a3fb3639a5e Mon Sep 17 00:00:00 2001 From: Yannick Welsch Date: Fri, 11 Jan 2019 16:06:48 +0100 Subject: [PATCH 113/186] Mock connections more accurately in DisruptableMockTransport (#37296) This commit moves DisruptableMockTransport to use a more accurate representation of connection management, which allows to use the full connection manager and does not require mocking out any behavior. With this, we can implement restarting nodes in CoordinatorTests. --- .../coordination/CoordinationState.java | 2 +- .../cluster/coordination/Coordinator.java | 2 +- .../cluster/coordination/Join.java | 4 + .../coordination/CoordinatorTests.java | 197 ++++++++++++------ .../snapshots/SnapshotsServiceTests.java | 47 ++--- .../disruption/DisruptableMockTransport.java | 132 ++++++++---- .../DisruptableMockTransportTests.java | 55 ++--- 7 files changed, 267 insertions(+), 172 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/cluster/coordination/CoordinationState.java b/server/src/main/java/org/elasticsearch/cluster/coordination/CoordinationState.java index fc511870b332e..4d542566ccd70 100644 --- a/server/src/main/java/org/elasticsearch/cluster/coordination/CoordinationState.java +++ b/server/src/main/java/org/elasticsearch/cluster/coordination/CoordinationState.java @@ -209,7 +209,7 @@ public Join handleStartJoin(StartJoinRequest startJoinRequest) { * @throws CoordinationStateRejectedException if the arguments were incompatible with the current state of this object. */ public boolean handleJoin(Join join) { - assert join.getTargetNode().equals(localNode) : "handling join " + join + " for the wrong node " + localNode; + assert join.targetMatches(localNode) : "handling join " + join + " for the wrong node " + localNode; if (join.getTerm() != getCurrentTerm()) { logger.debug("handleJoin: ignored join due to term mismatch (expected: [{}], actual: [{}])", 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 233423a391c19..1d7ed0bdc7c2b 100644 --- a/server/src/main/java/org/elasticsearch/cluster/coordination/Coordinator.java +++ b/server/src/main/java/org/elasticsearch/cluster/coordination/Coordinator.java @@ -310,7 +310,7 @@ PublishWithJoinResponse handlePublishRequest(PublishRequest publishRequest) { private static Optional joinWithDestination(Optional lastJoin, DiscoveryNode leader, long term) { if (lastJoin.isPresent() - && lastJoin.get().getTargetNode().getId().equals(leader.getId()) + && lastJoin.get().targetMatches(leader) && lastJoin.get().getTerm() == term) { return lastJoin; } diff --git a/server/src/main/java/org/elasticsearch/cluster/coordination/Join.java b/server/src/main/java/org/elasticsearch/cluster/coordination/Join.java index 1c69c4a62b80a..1b1c3ff7aed11 100644 --- a/server/src/main/java/org/elasticsearch/cluster/coordination/Join.java +++ b/server/src/main/java/org/elasticsearch/cluster/coordination/Join.java @@ -78,6 +78,10 @@ public DiscoveryNode getTargetNode() { return targetNode; } + public boolean targetMatches(DiscoveryNode matchingNode) { + return targetNode.getId().equals(matchingNode.getId()); + } + public long getLastAcceptedVersion() { return lastAcceptedVersion; } 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 36879df5ac06d..b03d05fa36238 100644 --- a/server/src/test/java/org/elasticsearch/cluster/coordination/CoordinatorTests.java +++ b/server/src/test/java/org/elasticsearch/cluster/coordination/CoordinatorTests.java @@ -26,6 +26,7 @@ import org.elasticsearch.ElasticsearchException; import org.elasticsearch.Version; import org.elasticsearch.action.ActionListener; +import org.elasticsearch.cluster.ClusterModule; import org.elasticsearch.cluster.ClusterName; import org.elasticsearch.cluster.ClusterState; import org.elasticsearch.cluster.ClusterStateUpdateTask; @@ -42,6 +43,10 @@ import org.elasticsearch.cluster.service.ClusterApplier; import org.elasticsearch.common.Randomness; import org.elasticsearch.common.UUIDs; +import org.elasticsearch.common.io.stream.BytesStreamOutput; +import org.elasticsearch.common.io.stream.NamedWriteableAwareStreamInput; +import org.elasticsearch.common.io.stream.NamedWriteableRegistry; +import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.lease.Releasable; import org.elasticsearch.common.settings.ClusterSettings; import org.elasticsearch.common.settings.Setting; @@ -77,7 +82,6 @@ import java.util.function.BiConsumer; import java.util.function.Consumer; import java.util.function.Function; -import java.util.function.Predicate; import java.util.function.Supplier; import java.util.function.UnaryOperator; import java.util.stream.Collectors; @@ -107,7 +111,6 @@ import static org.elasticsearch.discovery.DiscoverySettings.NO_MASTER_BLOCK_WRITES; import static org.elasticsearch.discovery.PeerFinder.DISCOVERY_FIND_PEERS_INTERVAL_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.containsString; import static org.hamcrest.Matchers.empty; @@ -930,7 +933,7 @@ public void testStayCandidateAfterReceivingFollowerCheckFromKnownMaster() { final ClusterNode leader = cluster.getAnyLeader(); final ClusterNode nonLeader = cluster.getAnyNodeExcept(leader); - onNode(nonLeader.getLocalNode(), () -> { + nonLeader.onNode(() -> { logger.debug("forcing {} to become candidate", nonLeader.getId()); synchronized (nonLeader.coordinator.mutex) { nonLeader.coordinator.becomeCandidate("forced"); @@ -1161,6 +1164,11 @@ void runRandomly() { assertThat("may reconnect disconnected nodes, probably unexpected", disconnectedNodes, empty()); assertThat("may reconnect blackholed nodes, probably unexpected", blackholedNodes, empty()); + final List cleanupActions = new ArrayList<>(); + cleanupActions.add(disconnectedNodes::clear); + cleanupActions.add(blackholedNodes::clear); + cleanupActions.add(() -> disruptStorage = false); + final int randomSteps = scaledRandomIntBetween(10, 10000); logger.info("--> start of safety phase of at least [{}] steps", randomSteps); @@ -1183,7 +1191,7 @@ void runRandomly() { if (rarely()) { final ClusterNode clusterNode = getAnyNodePreferringLeaders(); final int newValue = randomInt(); - onNode(clusterNode.getLocalNode(), () -> { + clusterNode.onNode(() -> { logger.debug("----> [runRandomly {}] proposing new value [{}] to [{}]", thisStep, newValue, clusterNode.getId()); clusterNode.submitValue(newValue); @@ -1191,15 +1199,34 @@ void runRandomly() { } else if (rarely()) { final ClusterNode clusterNode = getAnyNodePreferringLeaders(); final boolean autoShrinkVotingConfiguration = randomBoolean(); - onNode(clusterNode.getLocalNode(), + clusterNode.onNode( () -> { logger.debug("----> [runRandomly {}] setting auto-shrink configuration to {} on {}", thisStep, autoShrinkVotingConfiguration, clusterNode.getId()); clusterNode.submitSetAutoShrinkVotingConfiguration(autoShrinkVotingConfiguration); }).run(); + } else if (rarely()) { + // reboot random node + final ClusterNode clusterNode = getAnyNode(); + logger.debug("----> [runRandomly {}] rebooting [{}]", thisStep, clusterNode.getId()); + clusterNode.close(); + clusterNodes.forEach( + cn -> deterministicTaskQueue.scheduleNow(cn.onNode( + new Runnable() { + @Override + public void run() { + cn.transportService.disconnectFromNode(clusterNode.getLocalNode()); + } + + @Override + public String toString() { + return "disconnect from " + clusterNode.getLocalNode() + " after shutdown"; + } + }))); + clusterNodes.replaceAll(cn -> cn == clusterNode ? cn.restartedNode() : cn); } else if (rarely()) { final ClusterNode clusterNode = getAnyNode(); - onNode(clusterNode.getLocalNode(), () -> { + clusterNode.onNode(() -> { logger.debug("----> [runRandomly {}] forcing {} to become candidate", thisStep, clusterNode.getId()); synchronized (clusterNode.coordinator.mutex) { clusterNode.coordinator.becomeCandidate("runRandomly"); @@ -1227,7 +1254,7 @@ void runRandomly() { } } else if (rarely()) { final ClusterNode clusterNode = getAnyNode(); - onNode(clusterNode.getLocalNode(), + clusterNode.onNode( () -> { logger.debug("----> [runRandomly {}] applying initial configuration {} to {}", thisStep, initialConfiguration, clusterNode.getId()); @@ -1252,9 +1279,9 @@ void runRandomly() { assertConsistentStates(); } - disconnectedNodes.clear(); - blackholedNodes.clear(); - disruptStorage = false; + logger.debug("running {} cleanup actions", cleanupActions.size()); + cleanupActions.forEach(Runnable::run); + logger.debug("finished running cleanup actions"); } private void assertConsistentStates() { @@ -1406,18 +1433,28 @@ ClusterNode getAnyLeader() { return randomFrom(allLeaders); } + private final ConnectionStatus preferredUnknownNodeConnectionStatus = + randomFrom(ConnectionStatus.DISCONNECTED, ConnectionStatus.BLACK_HOLE); + private ConnectionStatus getConnectionStatus(DiscoveryNode sender, DiscoveryNode destination) { ConnectionStatus connectionStatus; if (blackholedNodes.contains(sender.getId()) || blackholedNodes.contains(destination.getId())) { connectionStatus = ConnectionStatus.BLACK_HOLE; } else if (disconnectedNodes.contains(sender.getId()) || disconnectedNodes.contains(destination.getId())) { connectionStatus = ConnectionStatus.DISCONNECTED; - } else { + } else if (nodeExists(sender) && nodeExists(destination)) { connectionStatus = ConnectionStatus.CONNECTED; + } else { + connectionStatus = usually() ? preferredUnknownNodeConnectionStatus : + randomFrom(ConnectionStatus.DISCONNECTED, ConnectionStatus.BLACK_HOLE); } return connectionStatus; } + boolean nodeExists(DiscoveryNode node) { + return clusterNodes.stream().anyMatch(cn -> cn.getLocalNode().equals(node)); + } + ClusterNode getAnyMasterEligibleNode() { return randomFrom(clusterNodes.stream().filter(n -> n.getLocalNode().isMasterNode()).collect(Collectors.toList())); } @@ -1486,7 +1523,7 @@ class ClusterNode { private final int nodeIndex; private Coordinator coordinator; - private DiscoveryNode localNode; + private final DiscoveryNode localNode; private final PersistedState persistedState; private FakeClusterApplier clusterApplier; private AckedFakeThreadPoolMasterService masterService; @@ -1496,63 +1533,34 @@ class ClusterNode { private ClusterStateApplyResponse clusterStateApplyResponse = ClusterStateApplyResponse.SUCCEED; ClusterNode(int nodeIndex, boolean masterEligible) { - this.nodeIndex = nodeIndex; - localNode = createDiscoveryNode(masterEligible); - persistedState = new MockPersistedState(0L, - clusterState(0L, 0L, localNode, VotingConfiguration.EMPTY_CONFIG, VotingConfiguration.EMPTY_CONFIG, 0L)); - onNode(localNode, this::setUp).run(); + this(nodeIndex, createDiscoveryNode(nodeIndex, masterEligible), + localNode -> new MockPersistedState(0L, + clusterState(0L, 0L, localNode, VotingConfiguration.EMPTY_CONFIG, VotingConfiguration.EMPTY_CONFIG, 0L))); } - private DiscoveryNode createDiscoveryNode(boolean masterEligible) { - final TransportAddress address = buildNewFakeTransportAddress(); - return new DiscoveryNode("", "node" + nodeIndex, - UUIDs.randomBase64UUID(random()), // generated deterministically for repeatable tests - address.address().getHostString(), address.getAddress(), address, Collections.emptyMap(), - masterEligible ? EnumSet.allOf(Role.class) : emptySet(), Version.CURRENT); + ClusterNode(int nodeIndex, DiscoveryNode localNode, Function persistedStateSupplier) { + this.nodeIndex = nodeIndex; + this.localNode = localNode; + persistedState = persistedStateSupplier.apply(localNode); + onNodeLog(localNode, this::setUp).run(); } private void setUp() { - mockTransport = new DisruptableMockTransport(logger) { - @Override - protected DiscoveryNode getLocalNode() { - return localNode; - } - + mockTransport = new DisruptableMockTransport(localNode, logger) { @Override - protected ConnectionStatus getConnectionStatus(DiscoveryNode sender, DiscoveryNode destination) { - return Cluster.this.getConnectionStatus(sender, destination); + protected void execute(Runnable runnable) { + deterministicTaskQueue.scheduleNow(onNode(runnable)); } @Override - protected Optional getDisruptedCapturingTransport(DiscoveryNode node, String action) { - final Predicate matchesDestination; - if (action.equals(HANDSHAKE_ACTION_NAME)) { - matchesDestination = n -> n.getLocalNode().getAddress().equals(node.getAddress()); - } else { - matchesDestination = n -> n.getLocalNode().equals(node); - } - return clusterNodes.stream().filter(matchesDestination).findAny().map(cn -> cn.mockTransport); + protected ConnectionStatus getConnectionStatus(DiscoveryNode destination) { + return Cluster.this.getConnectionStatus(getLocalNode(), destination); } @Override - protected void handle(DiscoveryNode sender, DiscoveryNode destination, String action, Runnable doDelivery) { - // handshake needs to run inline as the caller blockingly waits on the result - if (action.equals(HANDSHAKE_ACTION_NAME)) { - onNode(destination, doDelivery).run(); - } else { - deterministicTaskQueue.scheduleNow(onNode(destination, doDelivery)); - } - } - - @Override - protected void onBlackholedDuringSend(long requestId, String action, DiscoveryNode destination) { - if (action.equals(HANDSHAKE_ACTION_NAME)) { - logger.trace("ignoring blackhole and delivering {}", getRequestDescription(requestId, action, destination)); - // handshakes always have a timeout, and are sent in a blocking fashion, so we must respond with an exception. - sendFromTo(destination, getLocalNode(), action, getDisconnectException(requestId, action, destination)); - } else { - super.onBlackholedDuringSend(requestId, action, destination); - } + protected Optional getDisruptableMockTransport(TransportAddress address) { + return clusterNodes.stream().map(cn -> cn.mockTransport) + .filter(transport -> transport.getLocalNode().getAddress().equals(address)).findAny(); } }; @@ -1563,9 +1571,9 @@ protected void onBlackholedDuringSend(long requestId, String action, DiscoveryNo final ClusterSettings clusterSettings = new ClusterSettings(settings, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS); clusterApplier = new FakeClusterApplier(settings, clusterSettings); masterService = new AckedFakeThreadPoolMasterService("test_node", "test", - runnable -> deterministicTaskQueue.scheduleNow(onNode(localNode, runnable))); + runnable -> deterministicTaskQueue.scheduleNow(onNode(runnable))); transportService = mockTransport.createTransportService( - settings, deterministicTaskQueue.getThreadPool(runnable -> onNode(localNode, runnable)), NOOP_TRANSPORT_INTERCEPTOR, + settings, deterministicTaskQueue.getThreadPool(this::onNode), NOOP_TRANSPORT_INTERCEPTOR, a -> localNode, null, emptySet()); final Collection> onJoinValidators = Collections.singletonList((dn, cs) -> extraJoinValidators.forEach(validator -> validator.accept(dn, cs))); @@ -1574,6 +1582,7 @@ protected void onBlackholedDuringSend(long requestId, String action, DiscoveryNo Cluster.this::provideUnicastHosts, clusterApplier, onJoinValidators, Randomness.get()); masterService.setClusterStatePublisher(coordinator); + logger.trace("starting up [{}]", localNode); transportService.start(); transportService.acceptIncomingRequests(); masterService.start(); @@ -1581,6 +1590,37 @@ protected void onBlackholedDuringSend(long requestId, String action, DiscoveryNo coordinator.startInitialJoin(); } + void close() { + logger.trace("taking down [{}]", localNode); + //transportService.stop(); // does blocking stuff :/ + masterService.stop(); + coordinator.stop(); + //transportService.close(); // does blocking stuff :/ + masterService.close(); + coordinator.close(); + } + + ClusterNode restartedNode() { + final TransportAddress address = randomBoolean() ? buildNewFakeTransportAddress() : localNode.getAddress(); + final DiscoveryNode newLocalNode = new DiscoveryNode(localNode.getName(), localNode.getId(), + UUIDs.randomBase64UUID(random()), // generated deterministically for repeatable tests + address.address().getHostString(), address.getAddress(), address, Collections.emptyMap(), + localNode.isMasterNode() ? EnumSet.allOf(Role.class) : emptySet(), Version.CURRENT); + final PersistedState newPersistedState; + try { + BytesStreamOutput outStream = new BytesStreamOutput(); + outStream.setVersion(Version.CURRENT); + persistedState.getLastAcceptedState().writeTo(outStream); + StreamInput inStream = new NamedWriteableAwareStreamInput(outStream.bytes().streamInput(), + new NamedWriteableRegistry(ClusterModule.getNamedWriteables())); + newPersistedState = new MockPersistedState(persistedState.getCurrentTerm(), + ClusterState.readFrom(inStream, newLocalNode)); // adapts it to new localNode instance + } catch (IOException e) { + throw new UncheckedIOException(e); + } + return new ClusterNode(nodeIndex, newLocalNode, node -> newPersistedState); + } + private PersistedState getPersistedState() { return persistedState; } @@ -1615,6 +1655,25 @@ ClusterStateApplyResponse getClusterStateApplyResponse() { return clusterStateApplyResponse; } + Runnable onNode(Runnable runnable) { + final Runnable wrapped = onNodeLog(localNode, runnable); + return new Runnable() { + @Override + public void run() { + if (clusterNodes.contains(ClusterNode.this) == false) { + logger.trace("ignoring runnable {} from node {} as node has been removed from cluster", runnable, localNode); + return; + } + wrapped.run(); + } + + @Override + public String toString() { + return wrapped.toString(); + } + }; + } + void submitSetAutoShrinkVotingConfiguration(final boolean autoShrinkVotingConfiguration) { submitUpdateTask("set master nodes failure tolerance [" + autoShrinkVotingConfiguration + "]", cs -> ClusterState.builder(cs).metaData( @@ -1633,7 +1692,7 @@ AckCollector submitValue(final long value) { AckCollector submitUpdateTask(String source, UnaryOperator clusterStateUpdate) { final AckCollector ackCollector = new AckCollector(); - onNode(localNode, () -> { + onNode(() -> { logger.trace("[{}] submitUpdateTask: enqueueing [{}]", localNode.getId(), source); final long submittedTerm = coordinator.getCurrentTerm(); masterService.submitStateUpdateTask(source, @@ -1698,7 +1757,7 @@ ClusterState getLastAppliedClusterState() { } void applyInitialConfiguration() { - onNode(localNode, () -> { + onNode(() -> { try { coordinator.setInitialConfiguration(initialConfiguration); logger.info("successfully set initial configuration to {}", initialConfiguration); @@ -1734,7 +1793,7 @@ public void setInitialState(ClusterState initialState) { public void onNewClusterState(String source, Supplier clusterStateSupplier, ClusterApplyListener listener) { switch (clusterStateApplyResponse) { case SUCCEED: - deterministicTaskQueue.scheduleNow(onNode(localNode, new Runnable() { + deterministicTaskQueue.scheduleNow(onNode(new Runnable() { @Override public void run() { final ClusterState oldClusterState = clusterApplier.lastAppliedClusterState; @@ -1754,7 +1813,7 @@ public String toString() { })); break; case FAIL: - deterministicTaskQueue.scheduleNow(onNode(localNode, new Runnable() { + deterministicTaskQueue.scheduleNow(onNode(new Runnable() { @Override public void run() { listener.onFailure(source, new ElasticsearchException("cluster state application failed")); @@ -1768,7 +1827,7 @@ public String toString() { break; case HANG: if (randomBoolean()) { - deterministicTaskQueue.scheduleNow(onNode(localNode, new Runnable() { + deterministicTaskQueue.scheduleNow(onNode(new Runnable() { @Override public void run() { final ClusterState oldClusterState = clusterApplier.lastAppliedClusterState; @@ -1796,7 +1855,7 @@ private List provideUnicastHosts(HostsResolver ignored) { } } - public static Runnable onNode(DiscoveryNode node, Runnable runnable) { + public static Runnable onNodeLog(DiscoveryNode node, Runnable runnable) { final String nodeId = "{" + node.getId() + "}{" + node.getEphemeralId() + "}"; return new Runnable() { @Override @@ -1880,6 +1939,14 @@ public void onNodeAck(DiscoveryNode node, Exception e) { } } + private static DiscoveryNode createDiscoveryNode(int nodeIndex, boolean masterEligible) { + final TransportAddress address = buildNewFakeTransportAddress(); + return new DiscoveryNode("", "node" + nodeIndex, + UUIDs.randomBase64UUID(random()), // generated deterministically for repeatable tests + address.address().getHostString(), address.getAddress(), address, Collections.emptyMap(), + masterEligible ? EnumSet.allOf(Role.class) : emptySet(), Version.CURRENT); + } + /** * How to behave with a new cluster state */ diff --git a/server/src/test/java/org/elasticsearch/snapshots/SnapshotsServiceTests.java b/server/src/test/java/org/elasticsearch/snapshots/SnapshotsServiceTests.java index 34160e901006f..b4dbbef65cce2 100644 --- a/server/src/test/java/org/elasticsearch/snapshots/SnapshotsServiceTests.java +++ b/server/src/test/java/org/elasticsearch/snapshots/SnapshotsServiceTests.java @@ -68,6 +68,7 @@ import org.elasticsearch.common.settings.ClusterSettings; import org.elasticsearch.common.settings.IndexScopedSettings; import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.common.transport.TransportAddress; import org.elasticsearch.common.util.BigArrays; import org.elasticsearch.common.util.PageCacheRecycler; import org.elasticsearch.common.util.concurrent.PrioritizedEsThreadPoolExecutor; @@ -114,7 +115,6 @@ import java.util.Optional; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; -import java.util.function.Predicate; import java.util.function.Supplier; import java.util.stream.Collectors; @@ -122,7 +122,6 @@ 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; @@ -388,41 +387,26 @@ protected PrioritizedEsThreadPoolExecutor createThreadPoolExecutor() { return new MockSinglePrioritizingExecutor(node.getName(), deterministicTaskQueue); } }); - mockTransport = new DisruptableMockTransport(logger) { + mockTransport = new DisruptableMockTransport(node, logger) { @Override - protected DiscoveryNode getLocalNode() { - return node; - } - - @Override - protected ConnectionStatus getConnectionStatus(DiscoveryNode sender, DiscoveryNode destination) { + protected ConnectionStatus getConnectionStatus(DiscoveryNode destination) { return ConnectionStatus.CONNECTED; } @Override - protected Optional getDisruptedCapturingTransport(DiscoveryNode node, String action) { - 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); + protected Optional getDisruptableMockTransport(TransportAddress address) { + return testClusterNodes.nodes.values().stream().map(cn -> cn.mockTransport) + .filter(transport -> transport.getLocalNode().getAddress().equals(address)) + .findAny(); } @Override - protected void handle(DiscoveryNode sender, DiscoveryNode destination, String action, Runnable 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); - } + protected void execute(Runnable runnable) { + deterministicTaskQueue.scheduleNow(CoordinatorTests.onNodeLog(getLocalNode(), runnable)); } }; transportService = mockTransport.createTransportService( - settings, deterministicTaskQueue.getThreadPool(runnable -> CoordinatorTests.onNode(node, runnable)), + settings, deterministicTaskQueue.getThreadPool(runnable -> CoordinatorTests.onNodeLog(node, runnable)), NOOP_TRANSPORT_INTERCEPTOR, a -> node, null, emptySet() ); @@ -544,7 +528,16 @@ public void start(ClusterState initialState) { coordinator.start(); masterService.start(); clusterService.getClusterApplierService().setNodeConnectionsService( - new NodeConnectionsService(clusterService.getSettings(), threadPool, transportService)); + new NodeConnectionsService(clusterService.getSettings(), threadPool, transportService) { + @Override + public void connectToNodes(DiscoveryNodes discoveryNodes) { + // override this method as it does blocking calls + for (final DiscoveryNode node : discoveryNodes) { + transportService.connectToNode(node); + } + super.connectToNodes(discoveryNodes); + } + }); clusterService.getClusterApplierService().start(); indicesService.start(); indicesClusterStateService.start(); diff --git a/test/framework/src/main/java/org/elasticsearch/test/disruption/DisruptableMockTransport.java b/test/framework/src/main/java/org/elasticsearch/test/disruption/DisruptableMockTransport.java index 2743c11a2b6e9..2a1101c6d7986 100644 --- a/test/framework/src/main/java/org/elasticsearch/test/disruption/DisruptableMockTransport.java +++ b/test/framework/src/main/java/org/elasticsearch/test/disruption/DisruptableMockTransport.java @@ -20,80 +20,123 @@ import org.apache.logging.log4j.Logger; import org.apache.logging.log4j.message.ParameterizedMessage; +import org.elasticsearch.action.ActionListener; import org.elasticsearch.cluster.ClusterModule; import org.elasticsearch.cluster.node.DiscoveryNode; +import org.elasticsearch.common.Nullable; import org.elasticsearch.common.io.stream.NamedWriteableRegistry; +import org.elasticsearch.common.lease.Releasable; +import org.elasticsearch.common.settings.ClusterSettings; +import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.common.transport.BoundTransportAddress; +import org.elasticsearch.common.transport.TransportAddress; import org.elasticsearch.test.transport.MockTransport; +import org.elasticsearch.threadpool.ThreadPool; +import org.elasticsearch.transport.CloseableConnection; import org.elasticsearch.transport.ConnectTransportException; +import org.elasticsearch.transport.ConnectionProfile; import org.elasticsearch.transport.RequestHandlerRegistry; import org.elasticsearch.transport.TransportChannel; +import org.elasticsearch.transport.TransportException; +import org.elasticsearch.transport.TransportInterceptor; import org.elasticsearch.transport.TransportRequest; +import org.elasticsearch.transport.TransportRequestOptions; import org.elasticsearch.transport.TransportResponse; +import org.elasticsearch.transport.TransportService; import java.io.IOException; import java.util.Optional; +import java.util.Set; +import java.util.function.Function; import static org.elasticsearch.test.ESTestCase.copyWriteable; +import static org.elasticsearch.transport.TransportService.HANDSHAKE_ACTION_NAME; public abstract class DisruptableMockTransport extends MockTransport { + private final DiscoveryNode localNode; private final Logger logger; - public DisruptableMockTransport(Logger logger) { + public DisruptableMockTransport(DiscoveryNode localNode, Logger logger) { + this.localNode = localNode; this.logger = logger; } - protected abstract DiscoveryNode getLocalNode(); + protected abstract ConnectionStatus getConnectionStatus(DiscoveryNode destination); - protected abstract ConnectionStatus getConnectionStatus(DiscoveryNode sender, DiscoveryNode destination); + protected abstract Optional getDisruptableMockTransport(TransportAddress address); - protected abstract Optional getDisruptedCapturingTransport(DiscoveryNode node, String action); + protected abstract void execute(Runnable runnable); - protected abstract void handle(DiscoveryNode sender, DiscoveryNode destination, String action, Runnable doDelivery); + protected final void execute(String action, Runnable runnable) { + // handshake needs to run inline as the caller blockingly waits on the result + if (action.equals(HANDSHAKE_ACTION_NAME)) { + runnable.run(); + } else { - protected final void sendFromTo(DiscoveryNode sender, DiscoveryNode destination, String action, Runnable doDelivery) { - handle(sender, destination, action, new Runnable() { - @Override - public void run() { - if (getDisruptedCapturingTransport(destination, action).isPresent()) { - doDelivery.run(); - } else { - logger.trace("unknown destination in {}", this); - } - } + execute(runnable); + } + } - @Override - public String toString() { - return doDelivery.toString(); - } - }); + public DiscoveryNode getLocalNode() { + return localNode; + } + + @Override + public TransportService createTransportService(Settings settings, ThreadPool threadPool, TransportInterceptor interceptor, + Function localNodeFactory, + @Nullable ClusterSettings clusterSettings, Set taskHeaders) { + return new TransportService(settings, this, threadPool, interceptor, localNodeFactory, clusterSettings, taskHeaders); } @Override - protected void onSendRequest(long requestId, String action, TransportRequest request, DiscoveryNode destination) { + public Releasable openConnection(DiscoveryNode node, ConnectionProfile profile, ActionListener listener) { + final Optional matchingTransport = getDisruptableMockTransport(node.getAddress()); + if (matchingTransport.isPresent()) { + listener.onResponse(new CloseableConnection() { + @Override + public DiscoveryNode getNode() { + return node; + } - assert destination.equals(getLocalNode()) == false : "non-local message from " + getLocalNode() + " to itself"; + @Override + public void sendRequest(long requestId, String action, TransportRequest request, TransportRequestOptions options) + throws TransportException { + onSendRequest(requestId, action, request, matchingTransport.get()); + } + }); + return () -> {}; + } else { + throw new ConnectTransportException(node, "node " + node + " does not exist"); + } + } - sendFromTo(getLocalNode(), destination, action, new Runnable() { + protected void onSendRequest(long requestId, String action, TransportRequest request, + DisruptableMockTransport destinationTransport) { + + assert destinationTransport.getLocalNode().equals(getLocalNode()) == false : + "non-local message from " + getLocalNode() + " to itself"; + + execute(action, new Runnable() { @Override public void run() { - switch (getConnectionStatus(getLocalNode(), destination)) { + switch (getConnectionStatus(destinationTransport.getLocalNode())) { case BLACK_HOLE: - onBlackholedDuringSend(requestId, action, destination); + onBlackholedDuringSend(requestId, action, destinationTransport); break; case DISCONNECTED: - onDisconnectedDuringSend(requestId, action, destination); + onDisconnectedDuringSend(requestId, action, destinationTransport); break; case CONNECTED: - onConnectedDuringSend(requestId, action, request, destination); + onConnectedDuringSend(requestId, action, request, destinationTransport); break; } } @Override public String toString() { - return getRequestDescription(requestId, action, destination); + return getRequestDescription(requestId, action, destinationTransport.getLocalNode()); } }); } @@ -117,20 +160,27 @@ protected String getRequestDescription(long requestId, String action, DiscoveryN requestId, action, getLocalNode(), destination).getFormattedMessage(); } - protected void onBlackholedDuringSend(long requestId, String action, DiscoveryNode destination) { - logger.trace("dropping {}", getRequestDescription(requestId, action, destination)); + protected void onBlackholedDuringSend(long requestId, String action, DisruptableMockTransport destinationTransport) { + if (action.equals(HANDSHAKE_ACTION_NAME)) { + logger.trace("ignoring blackhole and delivering {}", + getRequestDescription(requestId, action, destinationTransport.getLocalNode())); + // handshakes always have a timeout, and are sent in a blocking fashion, so we must respond with an exception. + destinationTransport.execute(action, getDisconnectException(requestId, action, destinationTransport.getLocalNode())); + } else { + logger.trace("dropping {}", getRequestDescription(requestId, action, destinationTransport.getLocalNode())); + } } - protected void onDisconnectedDuringSend(long requestId, String action, DiscoveryNode destination) { - sendFromTo(destination, getLocalNode(), action, getDisconnectException(requestId, action, destination)); + protected void onDisconnectedDuringSend(long requestId, String action, DisruptableMockTransport destinationTransport) { + destinationTransport.execute(action, getDisconnectException(requestId, action, destinationTransport.getLocalNode())); } - protected void onConnectedDuringSend(long requestId, String action, TransportRequest request, DiscoveryNode destination) { - Optional destinationTransport = getDisruptedCapturingTransport(destination, action); - assert destinationTransport.isPresent(); - + protected void onConnectedDuringSend(long requestId, String action, TransportRequest request, + DisruptableMockTransport destinationTransport) { final RequestHandlerRegistry requestHandler = - destinationTransport.get().getRequestHandler(action); + destinationTransport.getRequestHandler(action); + + final DiscoveryNode destination = destinationTransport.getLocalNode(); final String requestDescription = getRequestDescription(requestId, action, destination); @@ -147,10 +197,10 @@ public String getChannelType() { @Override public void sendResponse(final TransportResponse response) { - sendFromTo(destination, getLocalNode(), action, new Runnable() { + execute(action, new Runnable() { @Override public void run() { - if (getConnectionStatus(destination, getLocalNode()) != ConnectionStatus.CONNECTED) { + if (destinationTransport.getConnectionStatus(getLocalNode()) != ConnectionStatus.CONNECTED) { logger.trace("dropping response to {}: channel is not CONNECTED", requestDescription); } else { @@ -167,10 +217,10 @@ public String toString() { @Override public void sendResponse(Exception exception) { - sendFromTo(destination, getLocalNode(), action, new Runnable() { + execute(action, new Runnable() { @Override public void run() { - if (getConnectionStatus(destination, getLocalNode()) != ConnectionStatus.CONNECTED) { + if (destinationTransport.getConnectionStatus(getLocalNode()) != ConnectionStatus.CONNECTED) { logger.trace("dropping response to {}: channel is not CONNECTED", requestDescription); } else { diff --git a/test/framework/src/test/java/org/elasticsearch/test/disruption/DisruptableMockTransportTests.java b/test/framework/src/test/java/org/elasticsearch/test/disruption/DisruptableMockTransportTests.java index 3e98d8245432d..14aa79e87956e 100644 --- a/test/framework/src/test/java/org/elasticsearch/test/disruption/DisruptableMockTransportTests.java +++ b/test/framework/src/test/java/org/elasticsearch/test/disruption/DisruptableMockTransportTests.java @@ -25,6 +25,7 @@ import org.elasticsearch.common.collect.Tuple; import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.common.transport.TransportAddress; import org.elasticsearch.node.Node; import org.elasticsearch.test.ESTestCase; import org.elasticsearch.test.disruption.DisruptableMockTransport.ConnectionStatus; @@ -85,9 +86,6 @@ ConnectionStatus getConnectionStatus(DiscoveryNode sender, DiscoveryNode destina public void initTransports() { node1 = new DiscoveryNode("node1", buildNewFakeTransportAddress(), Version.CURRENT); node2 = new DiscoveryNode("node2", buildNewFakeTransportAddress(), Version.CURRENT); - List discoNodes = new ArrayList<>(); - discoNodes.add(node1); - discoNodes.add(node2); disconnectedLinks = new HashSet<>(); blackholedLinks = new HashSet<>(); @@ -97,57 +95,37 @@ public void initTransports() { deterministicTaskQueue = new DeterministicTaskQueue( Settings.builder().put(Node.NODE_NAME_SETTING.getKey(), "dummy").build(), random()); - transport1 = new DisruptableMockTransport(logger) { + transport1 = new DisruptableMockTransport(node1, logger) { @Override - protected DiscoveryNode getLocalNode() { - return node1; + protected ConnectionStatus getConnectionStatus(DiscoveryNode destination) { + return DisruptableMockTransportTests.this.getConnectionStatus(getLocalNode(), destination); } @Override - protected ConnectionStatus getConnectionStatus(DiscoveryNode sender, DiscoveryNode destination) { - return DisruptableMockTransportTests.this.getConnectionStatus(sender, destination); + protected Optional getDisruptableMockTransport(TransportAddress address) { + return transports.stream().filter(t -> t.getLocalNode().getAddress().equals(address)).findAny(); } @Override - protected Optional getDisruptedCapturingTransport(DiscoveryNode destination, String action) { - int index = discoNodes.indexOf(destination); - if (index == -1) { - return Optional.empty(); - } else { - return Optional.of(transports.get(index)); - } - } - - @Override - protected void handle(DiscoveryNode sender, DiscoveryNode destination, String action, Runnable doDelivery) { - deterministicTaskQueue.scheduleNow(doDelivery); + protected void execute(Runnable runnable) { + deterministicTaskQueue.scheduleNow(runnable); } }; - transport2 = new DisruptableMockTransport(logger) { + transport2 = new DisruptableMockTransport(node2, logger) { @Override - protected DiscoveryNode getLocalNode() { - return node2; + protected ConnectionStatus getConnectionStatus(DiscoveryNode destination) { + return DisruptableMockTransportTests.this.getConnectionStatus(getLocalNode(), destination); } @Override - protected ConnectionStatus getConnectionStatus(DiscoveryNode sender, DiscoveryNode destination) { - return DisruptableMockTransportTests.this.getConnectionStatus(sender, destination); + protected Optional getDisruptableMockTransport(TransportAddress address) { + return transports.stream().filter(t -> t.getLocalNode().getAddress().equals(address)).findAny(); } @Override - protected Optional getDisruptedCapturingTransport(DiscoveryNode destination, String action) { - int index = discoNodes.indexOf(destination); - if (index == -1) { - return Optional.empty(); - } else { - return Optional.of(transports.get(index)); - } - } - - @Override - protected void handle(DiscoveryNode sender, DiscoveryNode destination, String action, Runnable doDelivery) { - deterministicTaskQueue.scheduleNow(doDelivery); + protected void execute(Runnable runnable) { + deterministicTaskQueue.scheduleNow(runnable); } }; @@ -161,6 +139,9 @@ protected void handle(DiscoveryNode sender, DiscoveryNode destination, String ac service1.start(); service2.start(); + + service1.connectToNode(node2); + service2.connectToNode(node1); } From 1eba1d1df91aa1797a7372bd7ceffb7010f869ad Mon Sep 17 00:00:00 2001 From: Armin Braun Date: Fri, 11 Jan 2019 16:09:26 +0100 Subject: [PATCH 114/186] Fix SnapshotDisruptionIT Race Condition (#37358) * Due to a race between retrying the snapshot creation and the failed snapshot create trying to delete the snapshot there is no guarantee that the snapshot is eventually created by retries * Adjusted the assertion accordingly * Closes #36779 --- .../discovery/SnapshotDisruptionIT.java | 39 +++++++++---------- 1 file changed, 18 insertions(+), 21 deletions(-) diff --git a/server/src/test/java/org/elasticsearch/discovery/SnapshotDisruptionIT.java b/server/src/test/java/org/elasticsearch/discovery/SnapshotDisruptionIT.java index f6a95a3ed5b35..db3c71c398da7 100644 --- a/server/src/test/java/org/elasticsearch/discovery/SnapshotDisruptionIT.java +++ b/server/src/test/java/org/elasticsearch/discovery/SnapshotDisruptionIT.java @@ -133,21 +133,7 @@ public void clusterChanged(ClusterChangedEvent event) { logger.info("--> waiting for disruption to start"); assertTrue(disruptionStarted.await(1, TimeUnit.MINUTES)); - logger.info("--> wait until the snapshot is done"); - assertBusy(() -> { - ClusterState state = dataNodeClient().admin().cluster().prepareState().get().getState(); - SnapshotsInProgress snapshots = state.custom(SnapshotsInProgress.TYPE); - SnapshotDeletionsInProgress snapshotDeletionsInProgress = state.custom(SnapshotDeletionsInProgress.TYPE); - if (snapshots != null && snapshots.entries().size() > 0) { - logger.info("Current snapshot state [{}]", snapshots.entries().get(0).state()); - fail("Snapshot is still running"); - } else if (snapshotDeletionsInProgress != null && snapshotDeletionsInProgress.hasDeletionsInProgress()) { - logger.info("Current snapshot deletion state [{}]", snapshotDeletionsInProgress); - fail("Snapshot deletion is still running"); - } else { - logger.info("Snapshot is no longer in the cluster state"); - } - }, 1, TimeUnit.MINUTES); + assertAllSnapshotsCompleted(); logger.info("--> verify that snapshot was successful or no longer exist"); assertBusy(() -> { @@ -177,14 +163,25 @@ public void clusterChanged(ClusterChangedEvent event) { } } - logger.info("--> verify that snapshot eventually will be created due to retries"); + assertAllSnapshotsCompleted(); + } + + private void assertAllSnapshotsCompleted() throws Exception { + logger.info("--> wait until the snapshot is done"); assertBusy(() -> { - try { - assertSnapshotExists("test-repo", "test-snap-2"); - } catch (SnapshotMissingException ex) { - throw new AssertionError(ex); + ClusterState state = dataNodeClient().admin().cluster().prepareState().get().getState(); + SnapshotsInProgress snapshots = state.custom(SnapshotsInProgress.TYPE); + SnapshotDeletionsInProgress snapshotDeletionsInProgress = state.custom(SnapshotDeletionsInProgress.TYPE); + if (snapshots != null && snapshots.entries().isEmpty() == false) { + logger.info("Current snapshot state [{}]", snapshots.entries().get(0).state()); + fail("Snapshot is still running"); + } else if (snapshotDeletionsInProgress != null && snapshotDeletionsInProgress.hasDeletionsInProgress()) { + logger.info("Current snapshot deletion state [{}]", snapshotDeletionsInProgress); + fail("Snapshot deletion is still running"); + } else { + logger.info("Snapshot is no longer in the cluster state"); } - }, 1, TimeUnit.MINUTES); + }, 1L, TimeUnit.MINUTES); } private void assertSnapshotExists(String repository, String snapshot) { From cae71cddfef98ca9768b4798f081ae89e502e02c Mon Sep 17 00:00:00 2001 From: Boaz Leskes Date: Fri, 11 Jan 2019 07:59:15 -0800 Subject: [PATCH 115/186] Document Seq No powered optimistic concurrency control (#37284) Add documentation to describe the new sequence number powered optimistic concurrency control Relates #36148 Relates #10708 --- docs/reference/docs.asciidoc | 2 + docs/reference/docs/bulk.asciidoc | 11 ++ .../docs/concurrency-control.asciidoc | 114 ++++++++++++ docs/reference/docs/delete.asciidoc | 10 + docs/reference/docs/index_.asciidoc | 173 +++++++++--------- 5 files changed, 227 insertions(+), 83 deletions(-) create mode 100644 docs/reference/docs/concurrency-control.asciidoc diff --git a/docs/reference/docs.asciidoc b/docs/reference/docs.asciidoc index a8ab282853c8f..5c4c471b0a131 100644 --- a/docs/reference/docs.asciidoc +++ b/docs/reference/docs.asciidoc @@ -50,3 +50,5 @@ include::docs/termvectors.asciidoc[] include::docs/multi-termvectors.asciidoc[] include::docs/refresh.asciidoc[] + +include::docs/concurrency-control.asciidoc[] diff --git a/docs/reference/docs/bulk.asciidoc b/docs/reference/docs/bulk.asciidoc index 7ee634ccef649..0aae2365d965e 100644 --- a/docs/reference/docs/bulk.asciidoc +++ b/docs/reference/docs/bulk.asciidoc @@ -197,6 +197,17 @@ size for your particular workload. If using the HTTP API, make sure that the client does not send HTTP chunks, as this will slow things down. +[float] +[[bulk-optimistic-concurrency-control]] +=== Optimistic Concurrency Control + +Each `index` and `delete` action within a bulk API call may include the +`if_seq_no` and `if_primary_term` parameters in their respective action +and meta data lines. The `if_seq_no` and `if_primary_term` parameters control +how operations are executed, based on the last modification to existing +documents. See <> for more details. + + [float] [[bulk-versioning]] === Versioning diff --git a/docs/reference/docs/concurrency-control.asciidoc b/docs/reference/docs/concurrency-control.asciidoc new file mode 100644 index 0000000000000..d457b14068e26 --- /dev/null +++ b/docs/reference/docs/concurrency-control.asciidoc @@ -0,0 +1,114 @@ +[[optimistic-concurrency-control]] +== Optimistic concurrency control + +Elasticsearch is distributed. When documents are created, updated, or deleted, +the new version of the document has to be replicated to other nodes in the cluster. +Elasticsearch is also asynchronous and concurrent, meaning that these replication +requests are sent in parallel, and may arrive at their destination out of sequence. +Elasticsearch needs a way of ensuring that an older version of a document never +overwrites a newer version. + + +To ensure an older version of a document doesn't overwrite a newer version, every +operation performed to a document is assigned a sequence number by the primary +shard that coordinates that change. The sequence number is increased with each +operation and thus newer operations are guaranteed to have a higher sequence +number than older operations. Elasticsearch can then use the sequence number of +operations to make sure they never override a newer document version is never +overridden by a change that has a smaller sequence number assigned to it. + +For example, the following indexing command will create a document and assign it +an initial sequence number and primary term: + +[source,js] +-------------------------------------------------- +PUT products/_doc/1567 +{ + "product" : "r2d2", + "details" : "A resourceful astromech droid" +} +-------------------------------------------------- +// CONSOLE + +You can see the assigned sequence number and primary term in the +the `_seq_no` and `_primary_term` fields of the response: + +[source,js] +-------------------------------------------------- +{ + "_shards" : { + "total" : 2, + "failed" : 0, + "successful" : 1 + }, + "_index" : "products", + "_type" : "_doc", + "_id" : "1567", + "_version" : 1, + "_seq_no" : 362, + "_primary_term" : 2, + "result" : "created" +} +-------------------------------------------------- +// TESTRESPONSE[s/"_seq_no" : \d+/"_seq_no" : $body._seq_no/ s/"_primary_term" : 2/"_primary_term" : $body._primary_term/] + + +Elasticsearch keeps tracks of the sequence number and primary of the last +operation to have changed each of the document it stores. The sequence number +and primary term are returned in the `_seq_no` and `_primary_term` fields in +the response of the <>: + +[source,js] +-------------------------------------------------- +GET products/_doc/1567 +-------------------------------------------------- +// CONSOLE +// TEST[continued] + +returns: + +[source,js] +-------------------------------------------------- +{ + "_index" : "products", + "_type" : "_doc", + "_id" : "1567", + "_version" : 1, + "_seq_no" : 362, + "_primary_term" : 2, + "found": true, + "_source" : { + "product" : "r2d2", + "details" : "A resourceful astromech droid" + } +} +-------------------------------------------------- +// TESTRESPONSE[s/"_seq_no" : \d+/"_seq_no" : $body._seq_no/ s/"_primary_term" : 2/"_primary_term" : $body._primary_term/] + + +Note: The <> can return the `_seq_no` and `_primary_term` +for each search hit by requesting the `_seq_no` and `_primary_term` <>. + +The sequence number and the primary term uniquely identify a change. By noting down +the sequence number and primary term returned, you can make sure to only change the +document if no other change was made to it since you retrieved it. This +is done by setting the `if_seq_no` and `if_primary_term` parameters of either the +<> or the <>. + +For example, the following indexing call will make sure to add a tag to the +document without losing any potential change to the description or an addition +of another tag by another API: + +[source,js] +-------------------------------------------------- +PUT products/_doc/1567?if_seq_no=362&if_primary_term=2 +{ + "product" : "r2d2", + "details" : "A resourceful astromech droid", + "tags": ["droid"] +} +-------------------------------------------------- +// CONSOLE +// TEST[continued] +// TEST[catch: conflict] + diff --git a/docs/reference/docs/delete.asciidoc b/docs/reference/docs/delete.asciidoc index 146c822a7bf27..bc6f7b840048d 100644 --- a/docs/reference/docs/delete.asciidoc +++ b/docs/reference/docs/delete.asciidoc @@ -35,6 +35,16 @@ The result of the above delete operation is: // TESTRESPONSE[s/"_primary_term" : 1/"_primary_term" : $body._primary_term/] // TESTRESPONSE[s/"_seq_no" : 5/"_seq_no" : $body._seq_no/] +[float] +[[optimistic-concurrency-control-delete]] +=== Optimistic concurrency control + +Delete operations can be made optional and only be performed if the last +modification to the document was assigned the sequence number and primary +term specified by the `if_seq_no` and `if_primary_term` parameters. If a +mismatch is detected, the operation will result in a `VersionConflictException` +and a status code of 409. See <> for more details. + [float] [[delete-versioning]] === Versioning diff --git a/docs/reference/docs/index_.asciidoc b/docs/reference/docs/index_.asciidoc index 70769b5b67237..572186487f44f 100644 --- a/docs/reference/docs/index_.asciidoc +++ b/docs/reference/docs/index_.asciidoc @@ -79,89 +79,6 @@ Automatic index creation can include a pattern based white/black list, for example, set `action.auto_create_index` to `+aaa*,-bbb*,+ccc*,-*` (+ meaning allowed, and - meaning disallowed). -[float] -[[index-versioning]] -=== Versioning - -Each indexed document is given a version number. The associated -`version` number is returned as part of the response to the index API -request. The index API optionally allows for -http://en.wikipedia.org/wiki/Optimistic_concurrency_control[optimistic -concurrency control] when the `version` parameter is specified. This -will control the version of the document the operation is intended to be -executed against. A good example of a use case for versioning is -performing a transactional read-then-update. Specifying a `version` from -the document initially read ensures no changes have happened in the -meantime. For example: - -[source,js] --------------------------------------------------- -PUT twitter/_doc/1?version=2 -{ - "message" : "elasticsearch now has versioning support, double cool!" -} --------------------------------------------------- -// CONSOLE -// TEST[continued] -// TEST[catch: conflict] - -*NOTE:* versioning is completely real time, and is not affected by the -near real time aspects of search operations. If no version is provided, -then the operation is executed without any version checks. - -By default, internal versioning is used that starts at 1 and increments -with each update, deletes included. Optionally, the version number can be -supplemented with an external value (for example, if maintained in a -database). To enable this functionality, `version_type` should be set to -`external`. The value provided must be a numeric, long value greater or equal to 0, -and less than around 9.2e+18. When using the external version type, instead -of checking for a matching version number, the system checks to see if -the version number passed to the index request is greater than the -version of the currently stored document. If true, the document will be -indexed and the new version number used. If the value provided is less -than or equal to the stored document's version number, a version -conflict will occur and the index operation will fail. - -WARNING: External versioning supports the value 0 as a valid version number. -This allows the version to be in sync with an external versioning system -where version numbers start from zero instead of one. It has the side effect -that documents with version number equal to zero cannot neither be updated -using the <> nor be deleted -using the <> as long as their -version number is equal to zero. - -A nice side effect is that there is no need to maintain strict ordering -of async indexing operations executed as a result of changes to a source -database, as long as version numbers from the source database are used. -Even the simple case of updating the Elasticsearch index using data from -a database is simplified if external versioning is used, as only the -latest version will be used if the index operations are out of order for -whatever reason. - -[float] -==== Version types - -Next to the `internal` & `external` version types explained above, Elasticsearch -also supports other types for specific use cases. Here is an overview of -the different version types and their semantics. - -`internal`:: only index the document if the given version is identical to the version -of the stored document. - -`external` or `external_gt`:: only index the document if the given version is strictly higher -than the version of the stored document *or* if there is no existing document. The given -version will be used as the new version and will be stored with the new document. The supplied -version must be a non-negative long number. - -`external_gte`:: only index the document if the given version is *equal* or higher -than the version of the stored document. If there is no existing document -the operation will succeed as well. The given version will be used as the new version -and will be stored with the new document. The supplied version must be a non-negative long number. - -*NOTE*: The `external_gte` version type is meant for special use cases and -should be used with care. If used incorrectly, it can result in loss of data. -There is another option, `force`, which is deprecated because it can cause -primary and replica shards to diverge. [float] [[operation-type]] @@ -238,6 +155,16 @@ The result of the above index operation is: -------------------------------------------------- // TESTRESPONSE[s/W0tpsmIBdwcYyG50zbta/$body._id/ s/"successful" : 2/"successful" : 1/] +[float] +[[optimistic-concurrency-control-index]] +=== Optimistic concurrency control + +Index operations can be made optional and only be performed if the last +modification to the document was assigned the sequence number and primary +term specified by the `if_seq_no` and `if_primary_term` parameters. If a +mismatch is detected, the operation will result in a `VersionConflictException` +and a status code of 409. See <> for more details. + [float] [[index-routing]] === Routing @@ -380,3 +307,83 @@ PUT twitter/_doc/1?timeout=5m } -------------------------------------------------- // CONSOLE + +[float] +[[index-versioning]] +=== Versioning + +Each indexed document is given a version number. By default, +internal versioning is used that starts at 1 and increments +with each update, deletes included. Optionally, the version number can be +set to an external value (for example, if maintained in a +database). To enable this functionality, `version_type` should be set to +`external`. The value provided must be a numeric, long value greater or equal to 0, +and less than around 9.2e+18. + +When using the external version type, the system checks to see if +the version number passed to the index request is greater than the +version of the currently stored document. If true, the document will be +indexed and the new version number used. If the value provided is less +than or equal to the stored document's version number, a version +conflict will occur and the index operation will fail. For example: + +[source,js] +-------------------------------------------------- +PUT twitter/_doc/1?version=2&version_type=external +{ + "message" : "elasticsearch now has versioning support, double cool!" +} +-------------------------------------------------- +// CONSOLE +// TEST[continued] + +*NOTE:* versioning is completely real time, and is not affected by the +near real time aspects of search operations. If no version is provided, +then the operation is executed without any version checks. + +The above will succeed since the the supplied version of 2 is higher than +the current document version of 1. If the document was already updated +and it's version was set to 2 or higher, the indexing command will fail +and result in a conflict (409 http status code). + +WARNING: External versioning supports the value 0 as a valid version number. +This allows the version to be in sync with an external versioning system +where version numbers start from zero instead of one. It has the side effect +that documents with version number equal to zero cannot neither be updated +using the <> nor be deleted +using the <> as long as their +version number is equal to zero. + +A nice side effect is that there is no need to maintain strict ordering +of async indexing operations executed as a result of changes to a source +database, as long as version numbers from the source database are used. +Even the simple case of updating the Elasticsearch index using data from +a database is simplified if external versioning is used, as only the +latest version will be used if the index operations are out of order for +whatever reason. + +[float] +==== Version types + +Next to the `external` version type explained above, Elasticsearch +also supports other types for specific use cases. Here is an overview of +the different version types and their semantics. + +`internal`:: only index the document if the given version is identical to the version +of the stored document. + +`external` or `external_gt`:: only index the document if the given version is strictly higher +than the version of the stored document *or* if there is no existing document. The given +version will be used as the new version and will be stored with the new document. The supplied +version must be a non-negative long number. + +`external_gte`:: only index the document if the given version is *equal* or higher +than the version of the stored document. If there is no existing document +the operation will succeed as well. The given version will be used as the new version +and will be stored with the new document. The supplied version must be a non-negative long number. + +*NOTE*: The `external_gte` version type is meant for special use cases and +should be used with care. If used incorrectly, it can result in loss of data. +There is another option, `force`, which is deprecated because it can cause +primary and replica shards to diverge. + From 093d4a371cca29d6567f6c2b166eff78468beac9 Mon Sep 17 00:00:00 2001 From: Boaz Leskes Date: Fri, 11 Jan 2019 08:07:35 -0800 Subject: [PATCH 116/186] followup to #37284 with additional feedback --- docs/reference/docs/concurrency-control.asciidoc | 4 ++-- docs/reference/docs/index_.asciidoc | 6 +++--- 2 files changed, 5 insertions(+), 5 deletions(-) diff --git a/docs/reference/docs/concurrency-control.asciidoc b/docs/reference/docs/concurrency-control.asciidoc index d457b14068e26..e695e6b5127c9 100644 --- a/docs/reference/docs/concurrency-control.asciidoc +++ b/docs/reference/docs/concurrency-control.asciidoc @@ -14,8 +14,8 @@ operation performed to a document is assigned a sequence number by the primary shard that coordinates that change. The sequence number is increased with each operation and thus newer operations are guaranteed to have a higher sequence number than older operations. Elasticsearch can then use the sequence number of -operations to make sure they never override a newer document version is never -overridden by a change that has a smaller sequence number assigned to it. +operations to make sure a newer document version is never overridden by +a change that has a smaller sequence number assigned to it. For example, the following indexing command will create a document and assign it an initial sequence number and primary term: diff --git a/docs/reference/docs/index_.asciidoc b/docs/reference/docs/index_.asciidoc index 572186487f44f..f706aaca3e034 100644 --- a/docs/reference/docs/index_.asciidoc +++ b/docs/reference/docs/index_.asciidoc @@ -343,13 +343,13 @@ then the operation is executed without any version checks. The above will succeed since the the supplied version of 2 is higher than the current document version of 1. If the document was already updated -and it's version was set to 2 or higher, the indexing command will fail +and its version was set to 2 or higher, the indexing command will fail and result in a conflict (409 http status code). WARNING: External versioning supports the value 0 as a valid version number. This allows the version to be in sync with an external versioning system where version numbers start from zero instead of one. It has the side effect -that documents with version number equal to zero cannot neither be updated +that documents with version number equal to zero can neither be updated using the <> nor be deleted using the <> as long as their version number is equal to zero. @@ -359,7 +359,7 @@ of async indexing operations executed as a result of changes to a source database, as long as version numbers from the source database are used. Even the simple case of updating the Elasticsearch index using data from a database is simplified if external versioning is used, as only the -latest version will be used if the index operations are out of order for +latest version will be used if the index operations arrive out of order for whatever reason. [float] From bb6d8784e71857fd5cea7808e219c606c343b6f7 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Christoph=20B=C3=BCscher?= Date: Fri, 11 Jan 2019 17:24:12 +0100 Subject: [PATCH 117/186] Switch indices.get rest after backport of `include_type_name` (#37351) With the `include_type_name` available now for indices.get on 6.x after the backport, the corresponsing yaml test can include anything from 6.7 on. Also changing the RestGetIndicesActionTests base test class. --- .../rest-api-spec/test/indices.get/10_basic.yml | 10 ++++++++-- .../admin/indices/RestGetIndicesActionTests.java | 4 ++-- 2 files changed, 10 insertions(+), 4 deletions(-) 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 ef028dcdf67e8..e7c0af2ca1422 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 @@ -55,8 +55,8 @@ setup: --- "Test include_type_name": - skip: - version: " - 6.99.99" - reason: the include_type_name parameter is not backported to pre 7.0 versions yet + version: " - 6.6.99" + reason: the include_type_name parameter is not supported before 6.7 - do: indices.get: @@ -74,6 +74,12 @@ setup: - is_true: test_index.mappings - is_false: test_index.mappings.type_1 +--- +"Test include_type_name dafaults to false": + - skip: + version: " - 6.99.99" + reason: the include_type_name parameter default is different on 6.x and 7.0, so only test this on 7.0 clusters + - do: indices.get: index: test_index 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 index 5f157cd298d4d..e4eb0edbb8582 100644 --- 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 @@ -23,8 +23,8 @@ 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 org.elasticsearch.test.rest.RestActionTestCase; import java.io.IOException; import java.util.HashMap; @@ -33,7 +33,7 @@ import static org.elasticsearch.rest.BaseRestHandler.INCLUDE_TYPE_NAME_PARAMETER; import static org.mockito.Mockito.mock; -public class RestGetIndicesActionTests extends ESTestCase { +public class RestGetIndicesActionTests extends RestActionTestCase { /** * Test that setting the "include_type_name" parameter raises a warning From e4391afd98c99e05ce1ca2163d1a096a0405d72b Mon Sep 17 00:00:00 2001 From: Martijn van Groningen Date: Fri, 11 Jan 2019 17:26:01 +0100 Subject: [PATCH 118/186] Test fix, wait for auto follower to have stopped in the background Relates to #36761 --- .../test/java/org/elasticsearch/xpack/ccr/AutoFollowIT.java | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/AutoFollowIT.java b/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/AutoFollowIT.java index 1db42dba44e77..286e5badee133 100644 --- a/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/AutoFollowIT.java +++ b/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/AutoFollowIT.java @@ -139,7 +139,13 @@ public void testAutoFollowManyIndices() throws Exception { assertThat(autoFollowStats.getNumberOfSuccessfulFollowIndices(), equalTo((long) expectedVal1)); }); + // Delete auto follow pattern and make sure that in the background the auto follower has stopped + // then the leader index created after that should never be auto followed: deleteAutoFollowPatternSetting(); + assertBusy(() -> { + AutoFollowStats autoFollowStats = getAutoFollowStats(); + assertThat(autoFollowStats.getAutoFollowedClusters().size(), equalTo(0)); + }); createLeaderIndex("logs-does-not-count", leaderIndexSettings); putAutoFollowPatterns("my-pattern", new String[] {"logs-*"}); From 953fb9352f439baa162ee5773e82f253f1e70a0e Mon Sep 17 00:00:00 2001 From: David Roberts Date: Fri, 11 Jan 2019 16:31:55 +0000 Subject: [PATCH 119/186] [ML] Update error message for process update (#37363) When this message was first added the model debug config was the only thing that could be updated, but now more aspects of the config can be updated so the message needs to be more general. --- .../ml/job/process/autodetect/AutodetectProcessManager.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/job/process/autodetect/AutodetectProcessManager.java b/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/job/process/autodetect/AutodetectProcessManager.java index 32507df53cef7..ef03b4f9e7160 100644 --- a/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/job/process/autodetect/AutodetectProcessManager.java +++ b/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/job/process/autodetect/AutodetectProcessManager.java @@ -337,7 +337,7 @@ public void forecastJob(JobTask jobTask, ForecastParams params, Consumer handler) { AutodetectCommunicator communicator = getOpenAutodetectCommunicator(jobTask); if (communicator == null) { - String message = "Cannot process update model debug config because job [" + jobTask.getJobId() + + String message = "Cannot update the job config because job [" + jobTask.getJobId() + "] does not have a corresponding autodetect process"; logger.debug(message); handler.accept(ExceptionsHelper.conflictStatusException(message)); From 955d3aea194b4709a37f88057b8f2dd9a9af760c Mon Sep 17 00:00:00 2001 From: Gordon Brown Date: Fri, 11 Jan 2019 09:35:38 -0700 Subject: [PATCH 120/186] Mute testRoundRobinWithFailures (#32190) --- .../authc/ldap/support/SessionFactoryLoadBalancingTests.java | 1 + 1 file changed, 1 insertion(+) diff --git a/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/authc/ldap/support/SessionFactoryLoadBalancingTests.java b/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/authc/ldap/support/SessionFactoryLoadBalancingTests.java index 87cc73d9a137e..1483b2f474bf0 100644 --- a/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/authc/ldap/support/SessionFactoryLoadBalancingTests.java +++ b/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/authc/ldap/support/SessionFactoryLoadBalancingTests.java @@ -75,6 +75,7 @@ public void testRoundRobin() throws Exception { } } + @AwaitsFix(bugUrl = "https://github.com/elastic/elasticsearch/issues/32190") public void testRoundRobinWithFailures() throws Exception { assumeTrue("at least one ldap server should be present for this test", ldapServers.length > 1); logger.debug("using [{}] ldap servers, urls {}", ldapServers.length, ldapUrls()); From 70cee18e56b638525c83054fe795ca1e8ca4f26a Mon Sep 17 00:00:00 2001 From: Nhat Nguyen Date: Fri, 11 Jan 2019 13:06:17 -0500 Subject: [PATCH 121/186] Introduce StepListener (#37327) This commit introduces StepListener which provides a simple way to write a flow consisting of multiple asynchronous steps without having nested callbacks. Relates #37291 --- .../elasticsearch/action/StepListener.java | 89 ++++++++++++++ .../util/concurrent/ListenableFuture.java | 8 +- .../action/StepListenerTests.java | 113 ++++++++++++++++++ 3 files changed, 209 insertions(+), 1 deletion(-) create mode 100644 server/src/main/java/org/elasticsearch/action/StepListener.java create mode 100644 server/src/test/java/org/elasticsearch/action/StepListenerTests.java diff --git a/server/src/main/java/org/elasticsearch/action/StepListener.java b/server/src/main/java/org/elasticsearch/action/StepListener.java new file mode 100644 index 0000000000000..efbf8c755d57c --- /dev/null +++ b/server/src/main/java/org/elasticsearch/action/StepListener.java @@ -0,0 +1,89 @@ +/* + * 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; + +import org.elasticsearch.common.CheckedConsumer; +import org.elasticsearch.common.util.concurrent.EsExecutors; +import org.elasticsearch.common.util.concurrent.FutureUtils; +import org.elasticsearch.common.util.concurrent.ListenableFuture; + +import java.util.function.Consumer; + +/** + * A {@link StepListener} provides a simple way to write a flow consisting of + * multiple asynchronous steps without having nested callbacks. For example: + * + *
{@code
+ *  void asyncFlowMethod(... ActionListener flowListener) {
+ *    StepListener step1 = new StepListener<>();
+ *    asyncStep1(..., step1);
+
+ *    StepListener step2 = new StepListener<>();
+ *    step1.whenComplete(r1 -> {
+ *      asyncStep2(r1, ..., step2);
+ *    }, flowListener::onFailure);
+ *
+ *    step2.whenComplete(r2 -> {
+ *      R1 r1 = step1.result();
+ *      R r = combine(r1, r2);
+ *     flowListener.onResponse(r);
+ *    }, flowListener::onFailure);
+ *  }
+ * }
+ */ + +public final class StepListener implements ActionListener { + private final ListenableFuture delegate; + + public StepListener() { + this.delegate = new ListenableFuture<>(); + } + + @Override + public void onResponse(Response response) { + delegate.onResponse(response); + } + + @Override + public void onFailure(Exception e) { + delegate.onFailure(e); + } + + /** + * Registers the given actions which are called when this step is completed. If this step is completed successfully, + * the {@code onResponse} is called with the result; otherwise the {@code onFailure} is called with the failure. + * + * @param onResponse is called when this step is completed successfully + * @param onFailure is called when this step is completed with a failure + */ + public void whenComplete(CheckedConsumer onResponse, Consumer onFailure) { + delegate.addListener(ActionListener.wrap(onResponse, onFailure), EsExecutors.newDirectExecutorService(), null); + } + + /** + * Gets the result of this step. This method will throw {@link IllegalStateException} if this step is not completed yet. + */ + public Response result() { + if (delegate.isDone() == false) { + throw new IllegalStateException("step is not completed yet"); + } + return FutureUtils.get(delegate); + } +} diff --git a/server/src/main/java/org/elasticsearch/common/util/concurrent/ListenableFuture.java b/server/src/main/java/org/elasticsearch/common/util/concurrent/ListenableFuture.java index 725069c5937ed..d4ba603aca458 100644 --- a/server/src/main/java/org/elasticsearch/common/util/concurrent/ListenableFuture.java +++ b/server/src/main/java/org/elasticsearch/common/util/concurrent/ListenableFuture.java @@ -60,7 +60,13 @@ public void addListener(ActionListener listener, ExecutorService executor, Th if (done) { run = true; } else { - listeners.add(new Tuple<>(ContextPreservingActionListener.wrapPreservingContext(listener, threadContext), executor)); + final ActionListener wrappedListener; + if (threadContext == null) { + wrappedListener = listener; + } else { + wrappedListener = ContextPreservingActionListener.wrapPreservingContext(listener, threadContext); + } + listeners.add(new Tuple<>(wrappedListener, executor)); run = false; } } diff --git a/server/src/test/java/org/elasticsearch/action/StepListenerTests.java b/server/src/test/java/org/elasticsearch/action/StepListenerTests.java new file mode 100644 index 0000000000000..15e88830e47e9 --- /dev/null +++ b/server/src/test/java/org/elasticsearch/action/StepListenerTests.java @@ -0,0 +1,113 @@ +/* + * 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; + +import org.elasticsearch.test.ESTestCase; +import org.elasticsearch.threadpool.TestThreadPool; +import org.elasticsearch.threadpool.ThreadPool; +import org.junit.After; +import org.junit.Before; + +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.function.Consumer; + +import static org.hamcrest.Matchers.equalTo; + +public class StepListenerTests extends ESTestCase { + private ThreadPool threadPool; + + @Before + public void setUpThreadPool() { + threadPool = new TestThreadPool(getTestName()); + } + + @After + public void tearDownThreadPool() { + terminate(threadPool); + } + + public void testSimpleSteps() throws Exception { + CountDownLatch latch = new CountDownLatch(1); + Consumer onFailure = e -> { + latch.countDown(); + fail("test a happy path"); + }; + + StepListener step1 = new StepListener<>(); //[a]sync provide a string + executeAction(() -> step1.onResponse("hello")); + StepListener step2 = new StepListener<>(); //[a]sync calculate the length of the string + step1.whenComplete(str -> executeAction(() -> step2.onResponse(str.length())), onFailure); + step2.whenComplete(length -> executeAction(latch::countDown), onFailure); + latch.await(); + assertThat(step1.result(), equalTo("hello")); + assertThat(step2.result(), equalTo(5)); + } + + public void testAbortOnFailure() throws Exception { + CountDownLatch latch = new CountDownLatch(1); + int failedStep = randomBoolean() ? 1 : 2; + AtomicInteger failureNotified = new AtomicInteger(); + Consumer onFailure = e -> { + failureNotified.getAndIncrement(); + latch.countDown(); + assertThat(e.getMessage(), equalTo("failed at step " + failedStep)); + }; + + StepListener step1 = new StepListener<>(); //[a]sync provide a string + if (failedStep == 1) { + executeAction(() -> step1.onFailure(new RuntimeException("failed at step 1"))); + } else { + executeAction(() -> step1.onResponse("hello")); + } + + StepListener step2 = new StepListener<>(); //[a]sync calculate the length of the string + step1.whenComplete(str -> { + if (failedStep == 2) { + executeAction(() -> step2.onFailure(new RuntimeException("failed at step 2"))); + } else { + executeAction(() -> step2.onResponse(str.length())); + } + }, onFailure); + + step2.whenComplete(length -> latch.countDown(), onFailure); + latch.await(); + assertThat(failureNotified.get(), equalTo(1)); + + if (failedStep == 1) { + assertThat(expectThrows(RuntimeException.class, step1::result).getMessage(), + equalTo("failed at step 1")); + assertThat(expectThrows(RuntimeException.class, step2::result).getMessage(), + equalTo("step is not completed yet")); + } else { + assertThat(step1.result(), equalTo("hello")); + assertThat(expectThrows(RuntimeException.class, step2::result).getMessage(), + equalTo("failed at step 2")); + } + } + + private void executeAction(Runnable runnable) { + if (randomBoolean()) { + threadPool.generic().execute(runnable); + } else { + runnable.run(); + } + } +} From 827ece73c840aab82ef1ab677a83c7c0901ac27e Mon Sep 17 00:00:00 2001 From: Gordon Brown Date: Fri, 11 Jan 2019 11:11:27 -0700 Subject: [PATCH 122/186] Mute MlConfigMigratorIT.testMigrateConfigs (#37374) --- .../elasticsearch/xpack/ml/integration/MlConfigMigratorIT.java | 1 + 1 file changed, 1 insertion(+) diff --git a/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/integration/MlConfigMigratorIT.java b/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/integration/MlConfigMigratorIT.java index 33b2484766669..023286af6b9e9 100644 --- a/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/integration/MlConfigMigratorIT.java +++ b/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/integration/MlConfigMigratorIT.java @@ -120,6 +120,7 @@ public void testWriteConfigToIndex() throws InterruptedException { assertNull(alreadyMigratedJob.getCustomSettings()); } + @AwaitsFix(bugUrl = "https://github.com/elastic/elasticsearch/issues/37374") public void testMigrateConfigs() throws InterruptedException, IOException { // and jobs and datafeeds clusterstate MlMetadata.Builder mlMetadata = new MlMetadata.Builder(); From de52ba1f78edced3060804df7301c305b384009c Mon Sep 17 00:00:00 2001 From: Zachary Tong Date: Fri, 11 Jan 2019 13:16:05 -0500 Subject: [PATCH 123/186] Fix RollupDocumentation test to wait for job to stop Also adds some extra state debug information to various log messages --- .../client/documentation/RollupDocumentationIT.java | 11 ++++++++++- .../xpack/core/indexing/AsyncTwoPhaseIndexer.java | 6 +++--- .../rollup/action/TransportStopRollupAction.java | 7 ++++++- 3 files changed, 19 insertions(+), 5 deletions(-) diff --git a/client/rest-high-level/src/test/java/org/elasticsearch/client/documentation/RollupDocumentationIT.java b/client/rest-high-level/src/test/java/org/elasticsearch/client/documentation/RollupDocumentationIT.java index a1fcdbb7bfc83..8125c2f41f4c9 100644 --- a/client/rest-high-level/src/test/java/org/elasticsearch/client/documentation/RollupDocumentationIT.java +++ b/client/rest-high-level/src/test/java/org/elasticsearch/client/documentation/RollupDocumentationIT.java @@ -261,6 +261,14 @@ public void testStartRollupJob() throws Exception { } catch (Exception e) { // Swallow any exception, this test does not test actually cancelling. } + // stop job to prevent spamming exceptions on next start request + StopRollupJobRequest stopRequest = new StopRollupJobRequest(id); + stopRequest.waitForCompletion(); + stopRequest.timeout(TimeValue.timeValueSeconds(10)); + + StopRollupJobResponse response = client.rollup().stopRollupJob(stopRequest, RequestOptions.DEFAULT); + assertTrue(response.isAcknowledged()); + // tag::rollup-start-job-execute-listener ActionListener listener = new ActionListener() { @Override @@ -282,7 +290,8 @@ public void onFailure(Exception e) { assertTrue(latch.await(30L, TimeUnit.SECONDS)); // stop job so it can correctly be deleted by the test teardown - rc.stopRollupJob(new StopRollupJobRequest(id), RequestOptions.DEFAULT); + response = rc.stopRollupJob(stopRequest, RequestOptions.DEFAULT); + assertTrue(response.isAcknowledged()); } @SuppressWarnings("unused") diff --git a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/indexing/AsyncTwoPhaseIndexer.java b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/indexing/AsyncTwoPhaseIndexer.java index 899226ad48519..df8eeb71e61b1 100644 --- a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/indexing/AsyncTwoPhaseIndexer.java +++ b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/indexing/AsyncTwoPhaseIndexer.java @@ -137,7 +137,8 @@ public synchronized boolean maybeTriggerAsyncJob(long now) { case INDEXING: case STOPPING: case ABORTING: - logger.warn("Schedule was triggered for job [" + getJobId() + "], but prior indexer is still running."); + logger.warn("Schedule was triggered for job [" + getJobId() + "], but prior indexer is still running " + + "(with state [" + currentState + "]"); return false; case STOPPED: @@ -381,8 +382,7 @@ private boolean checkState(IndexerState currentState) { case STOPPING: logger.info("Indexer job encountered [" + IndexerState.STOPPING + "] state, halting indexer."); - doSaveState(finishAndSetState(), getPosition(), () -> { - }); + doSaveState(finishAndSetState(), getPosition(), () -> {}); return false; case STOPPED: diff --git a/x-pack/plugin/rollup/src/main/java/org/elasticsearch/xpack/rollup/action/TransportStopRollupAction.java b/x-pack/plugin/rollup/src/main/java/org/elasticsearch/xpack/rollup/action/TransportStopRollupAction.java index 2715dd65e844d..09bfa57a39765 100644 --- a/x-pack/plugin/rollup/src/main/java/org/elasticsearch/xpack/rollup/action/TransportStopRollupAction.java +++ b/x-pack/plugin/rollup/src/main/java/org/elasticsearch/xpack/rollup/action/TransportStopRollupAction.java @@ -81,10 +81,15 @@ private static ActionListener maybeWrapWithBlockin listener.onResponse(response); } else { listener.onFailure(new ElasticsearchTimeoutException("Timed out after [" + request.timeout().getStringRep() - + "] while waiting for rollup job [" + request.getId() + "] to stop")); + + "] while waiting for rollup job [" + request.getId() + "] to stop. State was [" + + ((RollupJobStatus) jobTask.getStatus()).getIndexerState() + "]")); } } catch (InterruptedException e) { listener.onFailure(e); + } catch (Exception e) { + listener.onFailure(new ElasticsearchTimeoutException("Encountered unexpected error while waiting for " + + "rollup job [" + request.getId() + "] to stop. State was [" + + ((RollupJobStatus) jobTask.getStatus()).getIndexerState() + "].", e)); } }); From 5101e518918ffeb05e679b10504a8bb5c5bd2632 Mon Sep 17 00:00:00 2001 From: Benjamin Trent Date: Fri, 11 Jan 2019 13:29:30 -0600 Subject: [PATCH 124/186] ML: Fix testMigrateConfigs (#37373) * ML: :s/execute/get * Fixing other broken tests * unmuting test --- .../xpack/ml/action/TransportOpenJobActionTests.java | 7 +++++-- .../xpack/ml/integration/BasicDistributedJobsIT.java | 6 ++++-- .../xpack/ml/integration/MlConfigMigratorIT.java | 3 +-- 3 files changed, 10 insertions(+), 6 deletions(-) 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 04dfa5f27502d..7d72ef7f633e1 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 @@ -8,9 +8,11 @@ import org.elasticsearch.ElasticsearchStatusException; import org.elasticsearch.ResourceNotFoundException; import org.elasticsearch.Version; +import org.elasticsearch.action.support.IndicesOptions; import org.elasticsearch.cluster.ClusterName; import org.elasticsearch.cluster.ClusterState; import org.elasticsearch.cluster.metadata.IndexMetaData; +import org.elasticsearch.cluster.metadata.IndexNameExpressionResolver; import org.elasticsearch.cluster.metadata.MappingMetaData; import org.elasticsearch.cluster.metadata.MetaData; import org.elasticsearch.cluster.node.DiscoveryNode; @@ -455,8 +457,9 @@ public void testVerifyIndicesPrimaryShardsAreActive() { metaData = new MetaData.Builder(cs.metaData()); routingTable = new RoutingTable.Builder(cs.routingTable()); - - String indexToRemove = randomFrom(TransportOpenJobAction.indicesOfInterest(".ml-anomalies-shared")); + IndexNameExpressionResolver indexNameExpressionResolver = new IndexNameExpressionResolver(); + String indexToRemove = randomFrom(indexNameExpressionResolver.concreteIndexNames(cs, IndicesOptions.lenientExpandOpen(), + TransportOpenJobAction.indicesOfInterest(".ml-anomalies-shared"))); if (randomBoolean()) { routingTable.remove(indexToRemove); } else { diff --git a/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/integration/BasicDistributedJobsIT.java b/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/integration/BasicDistributedJobsIT.java index 77041f40e0f4b..7779f4e13d0ea 100644 --- a/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/integration/BasicDistributedJobsIT.java +++ b/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/integration/BasicDistributedJobsIT.java @@ -51,6 +51,7 @@ import java.util.concurrent.TimeUnit; import static org.elasticsearch.persistent.PersistentTasksClusterService.needsReassignment; +import static org.hamcrest.Matchers.containsString; import static org.hamcrest.Matchers.hasEntry; public class BasicDistributedJobsIT extends BaseMlIntegTestCase { @@ -399,8 +400,9 @@ public void testMlStateAndResultsIndicesNotAvailable() throws Exception { String detailedMessage = detail.getMessage(); assertTrue(detailedMessage, detailedMessage.startsWith("Could not open job because no suitable nodes were found, allocation explanation")); - assertTrue(detailedMessage, detailedMessage.endsWith("because not all primary shards are active for the following indices " + - "[.ml-state,.ml-anomalies-shared]]")); + assertThat(detailedMessage, containsString("because not all primary shards are active for the following indices")); + assertThat(detailedMessage, containsString(".ml-state")); + assertThat(detailedMessage, containsString(".ml-anomalies-shared")); logger.info("Start data node"); String nonMlNode = internalCluster().startNode(Settings.builder() diff --git a/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/integration/MlConfigMigratorIT.java b/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/integration/MlConfigMigratorIT.java index 023286af6b9e9..4993da215afbc 100644 --- a/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/integration/MlConfigMigratorIT.java +++ b/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/integration/MlConfigMigratorIT.java @@ -120,7 +120,6 @@ public void testWriteConfigToIndex() throws InterruptedException { assertNull(alreadyMigratedJob.getCustomSettings()); } - @AwaitsFix(bugUrl = "https://github.com/elastic/elasticsearch/issues/37374") public void testMigrateConfigs() throws InterruptedException, IOException { // and jobs and datafeeds clusterstate MlMetadata.Builder mlMetadata = new MlMetadata.Builder(); @@ -311,7 +310,7 @@ public void testMigrateConfigsWithoutTasks_GivenMigrationIsDisabled() throws Int } public void assertSnapshot(MlMetadata expectedMlMetadata) throws IOException { - client().admin().indices().prepareRefresh(AnomalyDetectorsIndex.jobStateIndexPattern()).execute(); + client().admin().indices().prepareRefresh(AnomalyDetectorsIndex.jobStateIndexPattern()).get(); SearchResponse searchResponse = client() .prepareSearch(AnomalyDetectorsIndex.jobStateIndexPattern()) .setTypes(ElasticsearchMappings.DOC_TYPE) From 85531f0285839ebc2838129cefe8c2b5dc7ea999 Mon Sep 17 00:00:00 2001 From: Marios Trivyzas Date: Fri, 11 Jan 2019 22:43:17 +0200 Subject: [PATCH 125/186] SQL: [Tests] Fix and enable internalClusterTests (#37300) SqlPlugin cannot have more than one public constructor, so for the testing purposes the `getLicenseState()` should be overriden. Fixes: #37191 Co-authored-by: Michael Basnight --- x-pack/plugin/sql/build.gradle | 7 ++- .../xpack/sql/plugin/SqlPlugin.java | 58 +++++++++---------- .../sql/action/AbstractSqlIntegTestCase.java | 30 +++------- .../sql/action/LocalStateSQLXPackPlugin.java | 27 +++++++++ .../xpack/sql/action/SqlActionIT.java | 7 +-- .../sql/action/SqlClearCursorActionIT.java | 12 ++-- .../xpack/sql/action/SqlDisabledIT.java | 13 ++++- .../xpack/sql/action/SqlLicenseIT.java | 6 +- .../sql/action/SqlTranslateActionIT.java | 11 ++-- .../xpack/sql/plugin/SqlPluginTests.java | 12 ++-- 10 files changed, 100 insertions(+), 83 deletions(-) create mode 100644 x-pack/plugin/sql/src/test/java/org/elasticsearch/xpack/sql/action/LocalStateSQLXPackPlugin.java diff --git a/x-pack/plugin/sql/build.gradle b/x-pack/plugin/sql/build.gradle index f5dc3175d4162..cd2f82ee7b308 100644 --- a/x-pack/plugin/sql/build.gradle +++ b/x-pack/plugin/sql/build.gradle @@ -21,10 +21,13 @@ archivesBaseName = 'x-pack-sql' integTest.enabled = false task internalClusterTest(type: RandomizedTestingTask, - group: JavaBasePlugin.VERIFICATION_GROUP -) { + group: JavaBasePlugin.VERIFICATION_GROUP, + dependsOn: unitTest.dependsOn) { include '**/*IT.class' + systemProperty 'es.set.netty.runtime.available.processors', 'false' } +check.dependsOn internalClusterTest +internalClusterTest.mustRunAfter test dependencies { // "org.elasticsearch.plugin:x-pack-core:${version}" doesn't work with idea because the testArtifacts are also here diff --git a/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/plugin/SqlPlugin.java b/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/plugin/SqlPlugin.java index 8c759a7d87f8b..9f569206438d2 100644 --- a/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/plugin/SqlPlugin.java +++ b/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/plugin/SqlPlugin.java @@ -50,40 +50,38 @@ public class SqlPlugin extends Plugin implements ActionPlugin { private final boolean enabled; - private final SqlLicenseChecker sqlLicenseChecker; - - SqlPlugin(boolean enabled, SqlLicenseChecker sqlLicenseChecker) { - this.enabled = enabled; - this.sqlLicenseChecker = sqlLicenseChecker; - } + private final SqlLicenseChecker sqlLicenseChecker = new SqlLicenseChecker( + (mode) -> { + XPackLicenseState licenseState = getLicenseState(); + switch (mode) { + case JDBC: + if (licenseState.isJdbcAllowed() == false) { + throw LicenseUtils.newComplianceException("jdbc"); + } + break; + case ODBC: + if (licenseState.isOdbcAllowed() == false) { + throw LicenseUtils.newComplianceException("odbc"); + } + break; + case PLAIN: + if (licenseState.isSqlAllowed() == false) { + throw LicenseUtils.newComplianceException(XPackField.SQL); + } + break; + default: + throw new IllegalArgumentException("Unknown SQL mode " + mode); + } + } + ); public SqlPlugin(Settings settings) { - this(XPackSettings.SQL_ENABLED.get(settings), new SqlLicenseChecker( - (mode) -> { - XPackLicenseState licenseState = XPackPlugin.getSharedLicenseState(); - switch (mode) { - case JDBC: - if (licenseState.isJdbcAllowed() == false) { - throw LicenseUtils.newComplianceException("jdbc"); - } - break; - case ODBC: - if (licenseState.isOdbcAllowed() == false) { - throw LicenseUtils.newComplianceException("odbc"); - } - break; - case PLAIN: - if (licenseState.isSqlAllowed() == false) { - throw LicenseUtils.newComplianceException(XPackField.SQL); - } - break; - default: - throw new IllegalArgumentException("Unknown SQL mode " + mode); - } - } - )); + this.enabled = XPackSettings.SQL_ENABLED.get(settings); } + // overridable by tests + protected XPackLicenseState getLicenseState() { return XPackPlugin.getSharedLicenseState(); } + @Override public Collection createComponents(Client client, ClusterService clusterService, ThreadPool threadPool, ResourceWatcherService resourceWatcherService, ScriptService scriptService, diff --git a/x-pack/plugin/sql/src/test/java/org/elasticsearch/xpack/sql/action/AbstractSqlIntegTestCase.java b/x-pack/plugin/sql/src/test/java/org/elasticsearch/xpack/sql/action/AbstractSqlIntegTestCase.java index 0e7d2888f9e16..c741667ba9ebf 100644 --- a/x-pack/plugin/sql/src/test/java/org/elasticsearch/xpack/sql/action/AbstractSqlIntegTestCase.java +++ b/x-pack/plugin/sql/src/test/java/org/elasticsearch/xpack/sql/action/AbstractSqlIntegTestCase.java @@ -5,24 +5,19 @@ */ package org.elasticsearch.xpack.sql.action; -import org.elasticsearch.analysis.common.CommonAnalysisPlugin; import org.elasticsearch.common.settings.Settings; -import org.elasticsearch.index.reindex.ReindexPlugin; +import org.elasticsearch.license.LicenseService; import org.elasticsearch.plugins.Plugin; import org.elasticsearch.test.ESIntegTestCase; -import org.elasticsearch.test.discovery.TestZenDiscovery; -import org.elasticsearch.xpack.core.XPackPlugin; import org.elasticsearch.xpack.core.XPackSettings; -import java.util.Arrays; import java.util.Collection; +import java.util.Collections; -public abstract class AbstractSqlIntegTestCase extends ESIntegTestCase { +import static org.elasticsearch.test.ESIntegTestCase.Scope.SUITE; - @Override - protected boolean ignoreExternalCluster() { - return true; - } +@ESIntegTestCase.ClusterScope(scope = SUITE, numDataNodes = 0, numClientNodes = 0, maxNumDataNodes = 0, transportClientRatio = 0) +public abstract class AbstractSqlIntegTestCase extends ESIntegTestCase { @Override protected Settings nodeSettings(int nodeOrdinal) { @@ -32,29 +27,18 @@ protected Settings nodeSettings(int nodeOrdinal) { settings.put(XPackSettings.WATCHER_ENABLED.getKey(), false); settings.put(XPackSettings.GRAPH_ENABLED.getKey(), false); settings.put(XPackSettings.MACHINE_LEARNING_ENABLED.getKey(), false); - settings.put("xpack.ml.autodetect_process", false); + settings.put(LicenseService.SELF_GENERATED_LICENSE_TYPE.getKey(), "trial"); return settings.build(); } @Override protected Collection> nodePlugins() { - return Arrays.asList(XPackPlugin.class, CommonAnalysisPlugin.class, ReindexPlugin.class); + return Collections.singletonList(LocalStateSQLXPackPlugin.class); } @Override protected Collection> transportClientPlugins() { return nodePlugins(); } - - @Override - protected Settings transportClientSettings() { - // Plugin should be loaded on the transport client as well - return nodeSettings(0); - } - - @Override - protected Collection> getMockPlugins() { - return Arrays.asList(TestZenDiscovery.TestPlugin.class, TestSeedPlugin.class); - } } diff --git a/x-pack/plugin/sql/src/test/java/org/elasticsearch/xpack/sql/action/LocalStateSQLXPackPlugin.java b/x-pack/plugin/sql/src/test/java/org/elasticsearch/xpack/sql/action/LocalStateSQLXPackPlugin.java new file mode 100644 index 0000000000000..1203bfae6776e --- /dev/null +++ b/x-pack/plugin/sql/src/test/java/org/elasticsearch/xpack/sql/action/LocalStateSQLXPackPlugin.java @@ -0,0 +1,27 @@ +/* + * 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.sql.action; + +import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.license.XPackLicenseState; +import org.elasticsearch.xpack.core.LocalStateCompositeXPackPlugin; +import org.elasticsearch.xpack.sql.plugin.SqlPlugin; + +import java.nio.file.Path; + +public class LocalStateSQLXPackPlugin extends LocalStateCompositeXPackPlugin { + + public LocalStateSQLXPackPlugin(final Settings settings, final Path configPath) throws Exception { + super(settings, configPath); + LocalStateSQLXPackPlugin thisVar = this; + plugins.add(new SqlPlugin(settings) { + @Override + protected XPackLicenseState getLicenseState() { + return thisVar.getLicenseState(); + } + }); + } +} diff --git a/x-pack/plugin/sql/src/test/java/org/elasticsearch/xpack/sql/action/SqlActionIT.java b/x-pack/plugin/sql/src/test/java/org/elasticsearch/xpack/sql/action/SqlActionIT.java index c71d7c274947f..43ea7fe92ebee 100644 --- a/x-pack/plugin/sql/src/test/java/org/elasticsearch/xpack/sql/action/SqlActionIT.java +++ b/x-pack/plugin/sql/src/test/java/org/elasticsearch/xpack/sql/action/SqlActionIT.java @@ -18,12 +18,11 @@ public class SqlActionIT extends AbstractSqlIntegTestCase { - @AwaitsFix(bugUrl = "https://github.com/elastic/elasticsearch/issues/37191") - public void testSqlAction() throws Exception { + public void testSqlAction() { assertAcked(client().admin().indices().prepareCreate("test").get()); client().prepareBulk() - .add(new IndexRequest("test", "doc", "1").source("data", "bar", "count", 42)) - .add(new IndexRequest("test", "doc", "2").source("data", "baz", "count", 43)) + .add(new IndexRequest("test").id("1").source("data", "bar", "count", 42)) + .add(new IndexRequest("test").id("2").source("data", "baz", "count", 43)) .setRefreshPolicy(WriteRequest.RefreshPolicy.IMMEDIATE) .get(); ensureYellow("test"); diff --git a/x-pack/plugin/sql/src/test/java/org/elasticsearch/xpack/sql/action/SqlClearCursorActionIT.java b/x-pack/plugin/sql/src/test/java/org/elasticsearch/xpack/sql/action/SqlClearCursorActionIT.java index 952104b49eebb..d509c1325fed2 100644 --- a/x-pack/plugin/sql/src/test/java/org/elasticsearch/xpack/sql/action/SqlClearCursorActionIT.java +++ b/x-pack/plugin/sql/src/test/java/org/elasticsearch/xpack/sql/action/SqlClearCursorActionIT.java @@ -5,7 +5,6 @@ */ package org.elasticsearch.xpack.sql.action; -import org.apache.lucene.util.LuceneTestCase.AwaitsFix; import org.elasticsearch.action.bulk.BulkRequestBuilder; import org.elasticsearch.action.index.IndexRequest; import org.elasticsearch.action.support.WriteRequest; @@ -17,16 +16,15 @@ import static org.hamcrest.Matchers.not; import static org.hamcrest.Matchers.notNullValue; -@AwaitsFix(bugUrl = "https://github.com/elastic/elasticsearch/issues/37191") public class SqlClearCursorActionIT extends AbstractSqlIntegTestCase { - public void testSqlClearCursorAction() throws Exception { + public void testSqlClearCursorAction() { assertAcked(client().admin().indices().prepareCreate("test").get()); BulkRequestBuilder bulkRequestBuilder = client().prepareBulk(); int indexSize = randomIntBetween(100, 300); logger.info("Indexing {} records", indexSize); for (int i = 0; i < indexSize; i++) { - bulkRequestBuilder.add(new IndexRequest("test", "doc", "id" + i).source("data", "bar", "count", i)); + bulkRequestBuilder.add(new IndexRequest("test").id("id" + i).source("data", "bar", "count", i)); } bulkRequestBuilder.setRefreshPolicy(WriteRequest.RefreshPolicy.IMMEDIATE).get(); ensureYellow("test"); @@ -50,13 +48,13 @@ public void testSqlClearCursorAction() throws Exception { assertEquals(0, getNumberOfSearchContexts()); } - public void testAutoCursorCleanup() throws Exception { + public void testAutoCursorCleanup() { assertAcked(client().admin().indices().prepareCreate("test").get()); BulkRequestBuilder bulkRequestBuilder = client().prepareBulk(); int indexSize = randomIntBetween(100, 300); logger.info("Indexing {} records", indexSize); for (int i = 0; i < indexSize; i++) { - bulkRequestBuilder.add(new IndexRequest("test", "doc", "id" + i).source("data", "bar", "count", i)); + bulkRequestBuilder.add(new IndexRequest("test").id("id" + i).source("data", "bar", "count", i)); } bulkRequestBuilder.setRefreshPolicy(WriteRequest.RefreshPolicy.IMMEDIATE).get(); ensureYellow("test"); @@ -77,7 +75,7 @@ public void testAutoCursorCleanup() throws Exception { do { sqlQueryResponse = new SqlQueryRequestBuilder(client(), SqlQueryAction.INSTANCE).cursor(sqlQueryResponse.cursor()).get(); fetched += sqlQueryResponse.size(); - } while (sqlQueryResponse.cursor().equals("") == false); + } while (sqlQueryResponse.cursor().isEmpty() == false); assertEquals(indexSize, fetched); SqlClearCursorResponse cleanCursorResponse = new SqlClearCursorRequestBuilder(client(), SqlClearCursorAction.INSTANCE) diff --git a/x-pack/plugin/sql/src/test/java/org/elasticsearch/xpack/sql/action/SqlDisabledIT.java b/x-pack/plugin/sql/src/test/java/org/elasticsearch/xpack/sql/action/SqlDisabledIT.java index 0a56e804a00c6..51be147005173 100644 --- a/x-pack/plugin/sql/src/test/java/org/elasticsearch/xpack/sql/action/SqlDisabledIT.java +++ b/x-pack/plugin/sql/src/test/java/org/elasticsearch/xpack/sql/action/SqlDisabledIT.java @@ -6,13 +6,23 @@ package org.elasticsearch.xpack.sql.action; import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.plugins.Plugin; +import org.elasticsearch.xpack.core.XPackPlugin; import org.elasticsearch.xpack.core.XPackSettings; +import java.util.Collection; +import java.util.Collections; + import static org.hamcrest.CoreMatchers.either; import static org.hamcrest.CoreMatchers.startsWith; public class SqlDisabledIT extends AbstractSqlIntegTestCase { + @Override + protected Collection> nodePlugins() { + return Collections.singletonList(XPackPlugin.class); + } + @Override protected Settings nodeSettings(int nodeOrdinal) { return Settings.builder() @@ -29,8 +39,7 @@ protected Settings transportClientSettings() { .build(); } - @AwaitsFix(bugUrl = "https://github.com/elastic/elasticsearch/issues/37191") - public void testSqlAction() throws Exception { + public void testSqlAction() { Throwable throwable = expectThrows(Throwable.class, () -> new SqlQueryRequestBuilder(client(), SqlQueryAction.INSTANCE).query("SHOW tables").get()); assertThat(throwable.getMessage(), diff --git a/x-pack/plugin/sql/src/test/java/org/elasticsearch/xpack/sql/action/SqlLicenseIT.java b/x-pack/plugin/sql/src/test/java/org/elasticsearch/xpack/sql/action/SqlLicenseIT.java index 62f7b42c69944..50dda656ab4d1 100644 --- a/x-pack/plugin/sql/src/test/java/org/elasticsearch/xpack/sql/action/SqlLicenseIT.java +++ b/x-pack/plugin/sql/src/test/java/org/elasticsearch/xpack/sql/action/SqlLicenseIT.java @@ -35,7 +35,7 @@ import static org.elasticsearch.license.XPackLicenseStateTests.randomTrialOrPlatinumMode; import static org.hamcrest.Matchers.equalTo; -@LuceneTestCase.AwaitsFix(bugUrl = "https://github.com/elastic/elasticsearch/issues/37191") +@LuceneTestCase.AwaitsFix(bugUrl = "https://github.com/elastic/elasticsearch/issues/37320") public class SqlLicenseIT extends AbstractLicensesIntegrationTestCase { @Override protected boolean ignoreExternalCluster() { @@ -164,8 +164,8 @@ public void testSqlTranslateActionLicense() throws Exception { private void setupTestIndex() { ElasticsearchAssertions.assertAcked(client().admin().indices().prepareCreate("test").get()); client().prepareBulk() - .add(new IndexRequest("test", "doc", "1").source("data", "bar", "count", 42)) - .add(new IndexRequest("test", "doc", "2").source("data", "baz", "count", 43)) + .add(new IndexRequest("test").id("1").source("data", "bar", "count", 42)) + .add(new IndexRequest("test").id("2").source("data", "baz", "count", 43)) .setRefreshPolicy(WriteRequest.RefreshPolicy.IMMEDIATE) .get(); } diff --git a/x-pack/plugin/sql/src/test/java/org/elasticsearch/xpack/sql/action/SqlTranslateActionIT.java b/x-pack/plugin/sql/src/test/java/org/elasticsearch/xpack/sql/action/SqlTranslateActionIT.java index d86245dcbfae7..3dc41ad9dd362 100644 --- a/x-pack/plugin/sql/src/test/java/org/elasticsearch/xpack/sql/action/SqlTranslateActionIT.java +++ b/x-pack/plugin/sql/src/test/java/org/elasticsearch/xpack/sql/action/SqlTranslateActionIT.java @@ -17,12 +17,11 @@ public class SqlTranslateActionIT extends AbstractSqlIntegTestCase { - @AwaitsFix(bugUrl = "https://github.com/elastic/elasticsearch/issues/37191") - public void testSqlTranslateAction() throws Exception { + public void testSqlTranslateAction() { assertAcked(client().admin().indices().prepareCreate("test").get()); client().prepareBulk() - .add(new IndexRequest("test", "doc", "1").source("data", "bar", "count", 42)) - .add(new IndexRequest("test", "doc", "2").source("data", "baz", "count", 43)) + .add(new IndexRequest("test").id("1").source("data", "bar", "count", 42)) + .add(new IndexRequest("test").id("2").source("data", "baz", "count", 43)) .setRefreshPolicy(WriteRequest.RefreshPolicy.IMMEDIATE) .get(); ensureYellow("test"); @@ -33,11 +32,11 @@ public void testSqlTranslateAction() throws Exception { .query("SELECT " + columns + " FROM test ORDER BY count").get(); SearchSourceBuilder source = response.source(); FetchSourceContext fetch = source.fetchSource(); - assertEquals(true, fetch.fetchSource()); + assertTrue(fetch.fetchSource()); assertArrayEquals(new String[] { "data" }, fetch.includes()); assertEquals( singletonList(new DocValueFieldsContext.FieldAndFormat("count", DocValueFieldsContext.USE_DEFAULT_FORMAT)), source.docValueFields()); - assertEquals(singletonList(SortBuilders.fieldSort("count")), source.sorts()); + assertEquals(singletonList(SortBuilders.fieldSort("count").missing("_last").unmappedType("long")), source.sorts()); } } diff --git a/x-pack/plugin/sql/src/test/java/org/elasticsearch/xpack/sql/plugin/SqlPluginTests.java b/x-pack/plugin/sql/src/test/java/org/elasticsearch/xpack/sql/plugin/SqlPluginTests.java index 363254f414c49..be8ac7b9968dc 100644 --- a/x-pack/plugin/sql/src/test/java/org/elasticsearch/xpack/sql/plugin/SqlPluginTests.java +++ b/x-pack/plugin/sql/src/test/java/org/elasticsearch/xpack/sql/plugin/SqlPluginTests.java @@ -25,14 +25,14 @@ public class SqlPluginTests extends ESTestCase { public void testSqlDisabled() { - SqlPlugin plugin = new SqlPlugin(false, new SqlLicenseChecker((mode) -> {})); + Settings settings = Settings.builder().put("xpack.sql.enabled", false).build(); + SqlPlugin plugin = new SqlPlugin(settings); assertThat(plugin.createComponents(mock(Client.class), "cluster", new NamedWriteableRegistry(Cursors.getNamedWriteables())), - empty()); + empty()); assertThat(plugin.getActions(), empty()); assertThat(plugin.getRestHandlers(Settings.EMPTY, mock(RestController.class), - new ClusterSettings(Settings.EMPTY, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS), - IndexScopedSettings.DEFAULT_SCOPED_SETTINGS, new SettingsFilter(Collections.emptyList()), - mock(IndexNameExpressionResolver.class), () -> mock(DiscoveryNodes.class)), empty()); + new ClusterSettings(Settings.EMPTY, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS), + IndexScopedSettings.DEFAULT_SCOPED_SETTINGS, new SettingsFilter(Collections.emptyList()), + mock(IndexNameExpressionResolver.class), () -> mock(DiscoveryNodes.class)), empty()); } - } From 359222c55c50bf9c15062ca16c71420c41d60100 Mon Sep 17 00:00:00 2001 From: Marios Trivyzas Date: Fri, 11 Jan 2019 23:08:00 +0200 Subject: [PATCH 126/186] SQL: Make `FULL` non-reserved keyword in the grammar (#37377) Since `full` can be common as a field name or part of a field name (e.g.: `full.name` or `name.full`), it's nice if it's not a reserved keyword of the grammar so a user can use it without resorting to quotes. Fixes: #37376 --- x-pack/plugin/sql/src/main/antlr/SqlBase.g4 | 2 +- .../xpack/sql/parser/SqlBaseParser.java | 24 +++++++++++++------ .../xpack/sql/parser/SqlParserTests.java | 22 +++++++++++++++++ 3 files changed, 40 insertions(+), 8 deletions(-) diff --git a/x-pack/plugin/sql/src/main/antlr/SqlBase.g4 b/x-pack/plugin/sql/src/main/antlr/SqlBase.g4 index 3bed074b03a2f..6435d80d04073 100644 --- a/x-pack/plugin/sql/src/main/antlr/SqlBase.g4 +++ b/x-pack/plugin/sql/src/main/antlr/SqlBase.g4 @@ -342,7 +342,7 @@ nonReserved | CATALOGS | COLUMNS | CURRENT | DAY | DEBUG | EXECUTABLE | EXPLAIN - | FIRST | FORMAT | FUNCTIONS + | FIRST | FORMAT | FULL | FUNCTIONS | GRAPHVIZ | HOUR | INTERVAL diff --git a/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/parser/SqlBaseParser.java b/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/parser/SqlBaseParser.java index 323aeea30eaf2..8ba886404536b 100644 --- a/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/parser/SqlBaseParser.java +++ b/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/parser/SqlBaseParser.java @@ -811,6 +811,7 @@ public final StatementContext statement() throws RecognitionException { case EXPLAIN: case FIRST: case FORMAT: + case FULL: case FUNCTIONS: case GRAPHVIZ: case HOUR: @@ -887,6 +888,7 @@ public final StatementContext statement() throws RecognitionException { case EXPLAIN: case FIRST: case FORMAT: + case FULL: case FUNCTIONS: case GRAPHVIZ: case HOUR: @@ -957,6 +959,7 @@ public final StatementContext statement() throws RecognitionException { case EXPLAIN: case FIRST: case FORMAT: + case FULL: case FUNCTIONS: case GRAPHVIZ: case HOUR: @@ -1140,6 +1143,7 @@ public final StatementContext statement() throws RecognitionException { case EXPLAIN: case FIRST: case FORMAT: + case FULL: case FUNCTIONS: case GRAPHVIZ: case HOUR: @@ -2056,7 +2060,7 @@ public final GroupingExpressionsContext groupingExpressions() throws Recognition match(T__0); setState(332); _la = _input.LA(1); - if ((((_la) & ~0x3f) == 0 && ((1L << _la) & ((1L << T__0) | (1L << ANALYZE) | (1L << ANALYZED) | (1L << CAST) | (1L << CATALOGS) | (1L << COLUMNS) | (1L << CONVERT) | (1L << CURRENT) | (1L << CURRENT_TIMESTAMP) | (1L << DAY) | (1L << DEBUG) | (1L << EXECUTABLE) | (1L << EXISTS) | (1L << EXPLAIN) | (1L << EXTRACT) | (1L << FALSE) | (1L << FIRST) | (1L << FORMAT) | (1L << FUNCTIONS) | (1L << GRAPHVIZ) | (1L << HOUR) | (1L << INTERVAL) | (1L << LAST) | (1L << LEFT) | (1L << LIMIT) | (1L << MAPPED) | (1L << MATCH) | (1L << MINUTE) | (1L << MONTH) | (1L << NOT) | (1L << NULL) | (1L << OPTIMIZED))) != 0) || ((((_la - 67)) & ~0x3f) == 0 && ((1L << (_la - 67)) & ((1L << (PARSED - 67)) | (1L << (PHYSICAL - 67)) | (1L << (PLAN - 67)) | (1L << (RIGHT - 67)) | (1L << (RLIKE - 67)) | (1L << (QUERY - 67)) | (1L << (SCHEMAS - 67)) | (1L << (SECOND - 67)) | (1L << (SHOW - 67)) | (1L << (SYS - 67)) | (1L << (TABLES - 67)) | (1L << (TEXT - 67)) | (1L << (TRUE - 67)) | (1L << (TYPE - 67)) | (1L << (TYPES - 67)) | (1L << (VERIFY - 67)) | (1L << (YEAR - 67)) | (1L << (FUNCTION_ESC - 67)) | (1L << (DATE_ESC - 67)) | (1L << (TIME_ESC - 67)) | (1L << (TIMESTAMP_ESC - 67)) | (1L << (GUID_ESC - 67)) | (1L << (PLUS - 67)) | (1L << (MINUS - 67)) | (1L << (ASTERISK - 67)) | (1L << (PARAM - 67)) | (1L << (STRING - 67)) | (1L << (INTEGER_VALUE - 67)) | (1L << (DECIMAL_VALUE - 67)) | (1L << (IDENTIFIER - 67)) | (1L << (DIGIT_IDENTIFIER - 67)) | (1L << (QUOTED_IDENTIFIER - 67)) | (1L << (BACKQUOTED_IDENTIFIER - 67)))) != 0)) { + if ((((_la) & ~0x3f) == 0 && ((1L << _la) & ((1L << T__0) | (1L << ANALYZE) | (1L << ANALYZED) | (1L << CAST) | (1L << CATALOGS) | (1L << COLUMNS) | (1L << CONVERT) | (1L << CURRENT) | (1L << CURRENT_TIMESTAMP) | (1L << DAY) | (1L << DEBUG) | (1L << EXECUTABLE) | (1L << EXISTS) | (1L << EXPLAIN) | (1L << EXTRACT) | (1L << FALSE) | (1L << FIRST) | (1L << FORMAT) | (1L << FULL) | (1L << FUNCTIONS) | (1L << GRAPHVIZ) | (1L << HOUR) | (1L << INTERVAL) | (1L << LAST) | (1L << LEFT) | (1L << LIMIT) | (1L << MAPPED) | (1L << MATCH) | (1L << MINUTE) | (1L << MONTH) | (1L << NOT) | (1L << NULL) | (1L << OPTIMIZED))) != 0) || ((((_la - 67)) & ~0x3f) == 0 && ((1L << (_la - 67)) & ((1L << (PARSED - 67)) | (1L << (PHYSICAL - 67)) | (1L << (PLAN - 67)) | (1L << (RIGHT - 67)) | (1L << (RLIKE - 67)) | (1L << (QUERY - 67)) | (1L << (SCHEMAS - 67)) | (1L << (SECOND - 67)) | (1L << (SHOW - 67)) | (1L << (SYS - 67)) | (1L << (TABLES - 67)) | (1L << (TEXT - 67)) | (1L << (TRUE - 67)) | (1L << (TYPE - 67)) | (1L << (TYPES - 67)) | (1L << (VERIFY - 67)) | (1L << (YEAR - 67)) | (1L << (FUNCTION_ESC - 67)) | (1L << (DATE_ESC - 67)) | (1L << (TIME_ESC - 67)) | (1L << (TIMESTAMP_ESC - 67)) | (1L << (GUID_ESC - 67)) | (1L << (PLUS - 67)) | (1L << (MINUS - 67)) | (1L << (ASTERISK - 67)) | (1L << (PARAM - 67)) | (1L << (STRING - 67)) | (1L << (INTEGER_VALUE - 67)) | (1L << (DECIMAL_VALUE - 67)) | (1L << (IDENTIFIER - 67)) | (1L << (DIGIT_IDENTIFIER - 67)) | (1L << (QUOTED_IDENTIFIER - 67)) | (1L << (BACKQUOTED_IDENTIFIER - 67)))) != 0)) { { setState(324); expression(); @@ -3870,6 +3874,7 @@ private ValueExpressionContext valueExpression(int _p) throws RecognitionExcepti case FALSE: case FIRST: case FORMAT: + case FULL: case FUNCTIONS: case GRAPHVIZ: case HOUR: @@ -4257,7 +4262,7 @@ public final PrimaryExpressionContext primaryExpression() throws RecognitionExce { setState(581); _la = _input.LA(1); - if ((((_la) & ~0x3f) == 0 && ((1L << _la) & ((1L << ANALYZE) | (1L << ANALYZED) | (1L << CATALOGS) | (1L << COLUMNS) | (1L << CURRENT) | (1L << DAY) | (1L << DEBUG) | (1L << EXECUTABLE) | (1L << EXPLAIN) | (1L << FIRST) | (1L << FORMAT) | (1L << FUNCTIONS) | (1L << GRAPHVIZ) | (1L << HOUR) | (1L << INTERVAL) | (1L << LAST) | (1L << LIMIT) | (1L << MAPPED) | (1L << MINUTE) | (1L << MONTH) | (1L << OPTIMIZED))) != 0) || ((((_la - 67)) & ~0x3f) == 0 && ((1L << (_la - 67)) & ((1L << (PARSED - 67)) | (1L << (PHYSICAL - 67)) | (1L << (PLAN - 67)) | (1L << (RLIKE - 67)) | (1L << (QUERY - 67)) | (1L << (SCHEMAS - 67)) | (1L << (SECOND - 67)) | (1L << (SHOW - 67)) | (1L << (SYS - 67)) | (1L << (TABLES - 67)) | (1L << (TEXT - 67)) | (1L << (TYPE - 67)) | (1L << (TYPES - 67)) | (1L << (VERIFY - 67)) | (1L << (YEAR - 67)) | (1L << (IDENTIFIER - 67)) | (1L << (DIGIT_IDENTIFIER - 67)) | (1L << (QUOTED_IDENTIFIER - 67)) | (1L << (BACKQUOTED_IDENTIFIER - 67)))) != 0)) { + if ((((_la) & ~0x3f) == 0 && ((1L << _la) & ((1L << ANALYZE) | (1L << ANALYZED) | (1L << CATALOGS) | (1L << COLUMNS) | (1L << CURRENT) | (1L << DAY) | (1L << DEBUG) | (1L << EXECUTABLE) | (1L << EXPLAIN) | (1L << FIRST) | (1L << FORMAT) | (1L << FULL) | (1L << FUNCTIONS) | (1L << GRAPHVIZ) | (1L << HOUR) | (1L << INTERVAL) | (1L << LAST) | (1L << LIMIT) | (1L << MAPPED) | (1L << MINUTE) | (1L << MONTH) | (1L << OPTIMIZED))) != 0) || ((((_la - 67)) & ~0x3f) == 0 && ((1L << (_la - 67)) & ((1L << (PARSED - 67)) | (1L << (PHYSICAL - 67)) | (1L << (PLAN - 67)) | (1L << (RLIKE - 67)) | (1L << (QUERY - 67)) | (1L << (SCHEMAS - 67)) | (1L << (SECOND - 67)) | (1L << (SHOW - 67)) | (1L << (SYS - 67)) | (1L << (TABLES - 67)) | (1L << (TEXT - 67)) | (1L << (TYPE - 67)) | (1L << (TYPES - 67)) | (1L << (VERIFY - 67)) | (1L << (YEAR - 67)) | (1L << (IDENTIFIER - 67)) | (1L << (DIGIT_IDENTIFIER - 67)) | (1L << (QUOTED_IDENTIFIER - 67)) | (1L << (BACKQUOTED_IDENTIFIER - 67)))) != 0)) { { setState(578); qualifiedName(); @@ -4756,6 +4761,7 @@ public final FunctionExpressionContext functionExpression() throws RecognitionEx case EXPLAIN: case FIRST: case FORMAT: + case FULL: case FUNCTIONS: case GRAPHVIZ: case HOUR: @@ -4864,7 +4870,7 @@ public final FunctionTemplateContext functionTemplate() throws RecognitionExcept match(T__0); setState(664); _la = _input.LA(1); - if ((((_la) & ~0x3f) == 0 && ((1L << _la) & ((1L << T__0) | (1L << ALL) | (1L << ANALYZE) | (1L << ANALYZED) | (1L << CAST) | (1L << CATALOGS) | (1L << COLUMNS) | (1L << CONVERT) | (1L << CURRENT) | (1L << CURRENT_TIMESTAMP) | (1L << DAY) | (1L << DEBUG) | (1L << DISTINCT) | (1L << EXECUTABLE) | (1L << EXISTS) | (1L << EXPLAIN) | (1L << EXTRACT) | (1L << FALSE) | (1L << FIRST) | (1L << FORMAT) | (1L << FUNCTIONS) | (1L << GRAPHVIZ) | (1L << HOUR) | (1L << INTERVAL) | (1L << LAST) | (1L << LEFT) | (1L << LIMIT) | (1L << MAPPED) | (1L << MATCH) | (1L << MINUTE) | (1L << MONTH) | (1L << NOT) | (1L << NULL) | (1L << OPTIMIZED))) != 0) || ((((_la - 67)) & ~0x3f) == 0 && ((1L << (_la - 67)) & ((1L << (PARSED - 67)) | (1L << (PHYSICAL - 67)) | (1L << (PLAN - 67)) | (1L << (RIGHT - 67)) | (1L << (RLIKE - 67)) | (1L << (QUERY - 67)) | (1L << (SCHEMAS - 67)) | (1L << (SECOND - 67)) | (1L << (SHOW - 67)) | (1L << (SYS - 67)) | (1L << (TABLES - 67)) | (1L << (TEXT - 67)) | (1L << (TRUE - 67)) | (1L << (TYPE - 67)) | (1L << (TYPES - 67)) | (1L << (VERIFY - 67)) | (1L << (YEAR - 67)) | (1L << (FUNCTION_ESC - 67)) | (1L << (DATE_ESC - 67)) | (1L << (TIME_ESC - 67)) | (1L << (TIMESTAMP_ESC - 67)) | (1L << (GUID_ESC - 67)) | (1L << (PLUS - 67)) | (1L << (MINUS - 67)) | (1L << (ASTERISK - 67)) | (1L << (PARAM - 67)) | (1L << (STRING - 67)) | (1L << (INTEGER_VALUE - 67)) | (1L << (DECIMAL_VALUE - 67)) | (1L << (IDENTIFIER - 67)) | (1L << (DIGIT_IDENTIFIER - 67)) | (1L << (QUOTED_IDENTIFIER - 67)) | (1L << (BACKQUOTED_IDENTIFIER - 67)))) != 0)) { + if ((((_la) & ~0x3f) == 0 && ((1L << _la) & ((1L << T__0) | (1L << ALL) | (1L << ANALYZE) | (1L << ANALYZED) | (1L << CAST) | (1L << CATALOGS) | (1L << COLUMNS) | (1L << CONVERT) | (1L << CURRENT) | (1L << CURRENT_TIMESTAMP) | (1L << DAY) | (1L << DEBUG) | (1L << DISTINCT) | (1L << EXECUTABLE) | (1L << EXISTS) | (1L << EXPLAIN) | (1L << EXTRACT) | (1L << FALSE) | (1L << FIRST) | (1L << FORMAT) | (1L << FULL) | (1L << FUNCTIONS) | (1L << GRAPHVIZ) | (1L << HOUR) | (1L << INTERVAL) | (1L << LAST) | (1L << LEFT) | (1L << LIMIT) | (1L << MAPPED) | (1L << MATCH) | (1L << MINUTE) | (1L << MONTH) | (1L << NOT) | (1L << NULL) | (1L << OPTIMIZED))) != 0) || ((((_la - 67)) & ~0x3f) == 0 && ((1L << (_la - 67)) & ((1L << (PARSED - 67)) | (1L << (PHYSICAL - 67)) | (1L << (PLAN - 67)) | (1L << (RIGHT - 67)) | (1L << (RLIKE - 67)) | (1L << (QUERY - 67)) | (1L << (SCHEMAS - 67)) | (1L << (SECOND - 67)) | (1L << (SHOW - 67)) | (1L << (SYS - 67)) | (1L << (TABLES - 67)) | (1L << (TEXT - 67)) | (1L << (TRUE - 67)) | (1L << (TYPE - 67)) | (1L << (TYPES - 67)) | (1L << (VERIFY - 67)) | (1L << (YEAR - 67)) | (1L << (FUNCTION_ESC - 67)) | (1L << (DATE_ESC - 67)) | (1L << (TIME_ESC - 67)) | (1L << (TIMESTAMP_ESC - 67)) | (1L << (GUID_ESC - 67)) | (1L << (PLUS - 67)) | (1L << (MINUS - 67)) | (1L << (ASTERISK - 67)) | (1L << (PARAM - 67)) | (1L << (STRING - 67)) | (1L << (INTEGER_VALUE - 67)) | (1L << (DECIMAL_VALUE - 67)) | (1L << (IDENTIFIER - 67)) | (1L << (DIGIT_IDENTIFIER - 67)) | (1L << (QUOTED_IDENTIFIER - 67)) | (1L << (BACKQUOTED_IDENTIFIER - 67)))) != 0)) { { setState(654); _la = _input.LA(1); @@ -4967,6 +4973,7 @@ public final FunctionNameContext functionName() throws RecognitionException { case EXPLAIN: case FIRST: case FORMAT: + case FULL: case FUNCTIONS: case GRAPHVIZ: case HOUR: @@ -5809,6 +5816,7 @@ public final IdentifierContext identifier() throws RecognitionException { case EXPLAIN: case FIRST: case FORMAT: + case FULL: case FUNCTIONS: case GRAPHVIZ: case HOUR: @@ -5899,7 +5907,7 @@ public final TableIdentifierContext tableIdentifier() throws RecognitionExceptio { setState(739); _la = _input.LA(1); - if ((((_la) & ~0x3f) == 0 && ((1L << _la) & ((1L << ANALYZE) | (1L << ANALYZED) | (1L << CATALOGS) | (1L << COLUMNS) | (1L << CURRENT) | (1L << DAY) | (1L << DEBUG) | (1L << EXECUTABLE) | (1L << EXPLAIN) | (1L << FIRST) | (1L << FORMAT) | (1L << FUNCTIONS) | (1L << GRAPHVIZ) | (1L << HOUR) | (1L << INTERVAL) | (1L << LAST) | (1L << LIMIT) | (1L << MAPPED) | (1L << MINUTE) | (1L << MONTH) | (1L << OPTIMIZED))) != 0) || ((((_la - 67)) & ~0x3f) == 0 && ((1L << (_la - 67)) & ((1L << (PARSED - 67)) | (1L << (PHYSICAL - 67)) | (1L << (PLAN - 67)) | (1L << (RLIKE - 67)) | (1L << (QUERY - 67)) | (1L << (SCHEMAS - 67)) | (1L << (SECOND - 67)) | (1L << (SHOW - 67)) | (1L << (SYS - 67)) | (1L << (TABLES - 67)) | (1L << (TEXT - 67)) | (1L << (TYPE - 67)) | (1L << (TYPES - 67)) | (1L << (VERIFY - 67)) | (1L << (YEAR - 67)) | (1L << (IDENTIFIER - 67)) | (1L << (DIGIT_IDENTIFIER - 67)) | (1L << (QUOTED_IDENTIFIER - 67)) | (1L << (BACKQUOTED_IDENTIFIER - 67)))) != 0)) { + if ((((_la) & ~0x3f) == 0 && ((1L << _la) & ((1L << ANALYZE) | (1L << ANALYZED) | (1L << CATALOGS) | (1L << COLUMNS) | (1L << CURRENT) | (1L << DAY) | (1L << DEBUG) | (1L << EXECUTABLE) | (1L << EXPLAIN) | (1L << FIRST) | (1L << FORMAT) | (1L << FULL) | (1L << FUNCTIONS) | (1L << GRAPHVIZ) | (1L << HOUR) | (1L << INTERVAL) | (1L << LAST) | (1L << LIMIT) | (1L << MAPPED) | (1L << MINUTE) | (1L << MONTH) | (1L << OPTIMIZED))) != 0) || ((((_la - 67)) & ~0x3f) == 0 && ((1L << (_la - 67)) & ((1L << (PARSED - 67)) | (1L << (PHYSICAL - 67)) | (1L << (PLAN - 67)) | (1L << (RLIKE - 67)) | (1L << (QUERY - 67)) | (1L << (SCHEMAS - 67)) | (1L << (SECOND - 67)) | (1L << (SHOW - 67)) | (1L << (SYS - 67)) | (1L << (TABLES - 67)) | (1L << (TEXT - 67)) | (1L << (TYPE - 67)) | (1L << (TYPES - 67)) | (1L << (VERIFY - 67)) | (1L << (YEAR - 67)) | (1L << (IDENTIFIER - 67)) | (1L << (DIGIT_IDENTIFIER - 67)) | (1L << (QUOTED_IDENTIFIER - 67)) | (1L << (BACKQUOTED_IDENTIFIER - 67)))) != 0)) { { setState(736); ((TableIdentifierContext)_localctx).catalog = identifier(); @@ -6101,6 +6109,7 @@ public final UnquoteIdentifierContext unquoteIdentifier() throws RecognitionExce case EXPLAIN: case FIRST: case FORMAT: + case FULL: case FUNCTIONS: case GRAPHVIZ: case HOUR: @@ -6300,6 +6309,7 @@ public static class NonReservedContext extends ParserRuleContext { public TerminalNode EXPLAIN() { return getToken(SqlBaseParser.EXPLAIN, 0); } public TerminalNode FIRST() { return getToken(SqlBaseParser.FIRST, 0); } public TerminalNode FORMAT() { return getToken(SqlBaseParser.FORMAT, 0); } + public TerminalNode FULL() { return getToken(SqlBaseParser.FULL, 0); } public TerminalNode FUNCTIONS() { return getToken(SqlBaseParser.FUNCTIONS, 0); } public TerminalNode GRAPHVIZ() { return getToken(SqlBaseParser.GRAPHVIZ, 0); } public TerminalNode HOUR() { return getToken(SqlBaseParser.HOUR, 0); } @@ -6353,7 +6363,7 @@ public final NonReservedContext nonReserved() throws RecognitionException { { setState(765); _la = _input.LA(1); - if ( !((((_la) & ~0x3f) == 0 && ((1L << _la) & ((1L << ANALYZE) | (1L << ANALYZED) | (1L << CATALOGS) | (1L << COLUMNS) | (1L << CURRENT) | (1L << DAY) | (1L << DEBUG) | (1L << EXECUTABLE) | (1L << EXPLAIN) | (1L << FIRST) | (1L << FORMAT) | (1L << FUNCTIONS) | (1L << GRAPHVIZ) | (1L << HOUR) | (1L << INTERVAL) | (1L << LAST) | (1L << LIMIT) | (1L << MAPPED) | (1L << MINUTE) | (1L << MONTH) | (1L << OPTIMIZED))) != 0) || ((((_la - 67)) & ~0x3f) == 0 && ((1L << (_la - 67)) & ((1L << (PARSED - 67)) | (1L << (PHYSICAL - 67)) | (1L << (PLAN - 67)) | (1L << (RLIKE - 67)) | (1L << (QUERY - 67)) | (1L << (SCHEMAS - 67)) | (1L << (SECOND - 67)) | (1L << (SHOW - 67)) | (1L << (SYS - 67)) | (1L << (TABLES - 67)) | (1L << (TEXT - 67)) | (1L << (TYPE - 67)) | (1L << (TYPES - 67)) | (1L << (VERIFY - 67)) | (1L << (YEAR - 67)))) != 0)) ) { + if ( !((((_la) & ~0x3f) == 0 && ((1L << _la) & ((1L << ANALYZE) | (1L << ANALYZED) | (1L << CATALOGS) | (1L << COLUMNS) | (1L << CURRENT) | (1L << DAY) | (1L << DEBUG) | (1L << EXECUTABLE) | (1L << EXPLAIN) | (1L << FIRST) | (1L << FORMAT) | (1L << FULL) | (1L << FUNCTIONS) | (1L << GRAPHVIZ) | (1L << HOUR) | (1L << INTERVAL) | (1L << LAST) | (1L << LIMIT) | (1L << MAPPED) | (1L << MINUTE) | (1L << MONTH) | (1L << OPTIMIZED))) != 0) || ((((_la - 67)) & ~0x3f) == 0 && ((1L << (_la - 67)) & ((1L << (PARSED - 67)) | (1L << (PHYSICAL - 67)) | (1L << (PLAN - 67)) | (1L << (RLIKE - 67)) | (1L << (QUERY - 67)) | (1L << (SCHEMAS - 67)) | (1L << (SECOND - 67)) | (1L << (SHOW - 67)) | (1L << (SYS - 67)) | (1L << (TABLES - 67)) | (1L << (TEXT - 67)) | (1L << (TYPE - 67)) | (1L << (TYPES - 67)) | (1L << (VERIFY - 67)) | (1L << (YEAR - 67)))) != 0)) ) { _errHandler.recoverInline(this); } else { consume(); @@ -6464,8 +6474,8 @@ private boolean valueExpression_sempred(ValueExpressionContext _localctx, int pr "\64\668:<>@BDFHJLNPRTVXZ\\^`bdfhjl\2\22\b\2\7\7\t\t\36\36\66\66AAEE\4"+ "\2((SS\4\2\t\tAA\4\2%%--\3\2\32\33\3\2mn\4\2\7\7vv\4\2\r\r\32\32\4\2#"+ "#\62\62\4\2\7\7\34\34\3\2oq\3\2fl\4\2\"\"TT\7\2\27\30+,8;LM\\]\3\2tu\31"+ - "\2\b\t\22\23\25\25\27\27\31\31\36\36 #$\'(++//\62\62\65\6688::AAEGIL"+ - "OPRSVWYY\\\\\u035f\2n\3\2\2\2\4q\3\2\2\2\6\u00de\3\2\2\2\b\u00e9\3\2\2"+ + "\2\b\t\22\23\25\25\27\27\31\31\36\36 #$&(++//\62\62\65\6688::AAEGILO"+ + "PRSVWYY\\\\\u035f\2n\3\2\2\2\4q\3\2\2\2\6\u00de\3\2\2\2\b\u00e9\3\2\2"+ "\2\n\u00ed\3\2\2\2\f\u0102\3\2\2\2\16\u0109\3\2\2\2\20\u010b\3\2\2\2\22"+ "\u0113\3\2\2\2\24\u012f\3\2\2\2\26\u0139\3\2\2\2\30\u0143\3\2\2\2\32\u0152"+ "\3\2\2\2\34\u0154\3\2\2\2\36\u015a\3\2\2\2 \u015c\3\2\2\2\"\u0163\3\2"+ diff --git a/x-pack/plugin/sql/src/test/java/org/elasticsearch/xpack/sql/parser/SqlParserTests.java b/x-pack/plugin/sql/src/test/java/org/elasticsearch/xpack/sql/parser/SqlParserTests.java index 8e0074798a503..199b4e119d81b 100644 --- a/x-pack/plugin/sql/src/test/java/org/elasticsearch/xpack/sql/parser/SqlParserTests.java +++ b/x-pack/plugin/sql/src/test/java/org/elasticsearch/xpack/sql/parser/SqlParserTests.java @@ -31,6 +31,7 @@ import java.util.ArrayList; import java.util.List; +import java.util.StringJoiner; import static java.util.Collections.nCopies; import static java.util.stream.Collectors.toList; @@ -67,6 +68,27 @@ public void testSelectRightFunction() { assertEquals("RIGHT", f.functionName()); } + public void testsSelectNonReservedKeywords() { + String[] reserved = new String[] { + "ANALYZE", "ANALYZED", "CATALOGS", "COLUMNS", "CURRENT", "DAY", "DEBUG", "EXECUTABLE", "EXPLAIN", + "FIRST", "FORMAT", "FULL", "FUNCTIONS", "GRAPHVIZ", "HOUR", "INTERVAL", "LAST", "LIMIT", + "MAPPED", "MINUTE", "MONTH", "OPTIMIZED", "PARSED", "PHYSICAL", "PLAN", "QUERY", "RLIKE", + "SCHEMAS", "SECOND", "SHOW", "SYS", "TABLES", "TEXT", "TYPE", "TYPES", "VERIFY", "YEAR"}; + StringJoiner sj = new StringJoiner(","); + for (String s : reserved) { + sj.add(s); + } + + Project project = project(parseStatement("SELECT " + sj.toString() + " FROM foo")); + assertEquals(reserved.length, project.projections().size()); + + for (int i = 0; i < project.projections().size(); i++) { + NamedExpression ne = project.projections().get(i); + assertEquals(UnresolvedAttribute.class, ne.getClass()); + assertEquals(reserved[i], ne.name()); + } + } + public void testOrderByField() { Order.OrderDirection dir = randomFrom(Order.OrderDirection.values()); OrderBy ob = orderBy(parseStatement("SELECT * FROM foo ORDER BY bar" + stringForDirection(dir))); From 63fe3c6ed604d2aee3ae5266b35e87f35ff09bee Mon Sep 17 00:00:00 2001 From: Armin Braun Date: Fri, 11 Jan 2019 23:26:04 +0100 Subject: [PATCH 127/186] Fix PrimaryAllocationIT Race Condition (#37355) * Fix PrimaryAllocationIT Race Condition * Forcing a stale primary allocation on a green index was tripping the assertion that was removed * Added a test that this case still errors out correctly * Made the ability to wipe stopped datanode's data public on the internal test cluster and used it to ensure correct behaviour on the fixed test * Previously it simply passed because the test finished before the index went green and would NPE when the index was green at the time of the shard store status request, that would then come up empty * Closes #37345 --- .../TransportClusterRerouteAction.java | 6 ++++- .../cluster/routing/PrimaryAllocationIT.java | 25 +++++++++++++++++-- .../test/InternalTestCluster.java | 3 +-- 3 files changed, 29 insertions(+), 5 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/action/admin/cluster/reroute/TransportClusterRerouteAction.java b/server/src/main/java/org/elasticsearch/action/admin/cluster/reroute/TransportClusterRerouteAction.java index d5cc35b2205ac..7f29f0bb6db8b 100644 --- a/server/src/main/java/org/elasticsearch/action/admin/cluster/reroute/TransportClusterRerouteAction.java +++ b/server/src/main/java/org/elasticsearch/action/admin/cluster/reroute/TransportClusterRerouteAction.java @@ -113,7 +113,11 @@ private void verifyThenSubmitUpdate(ClusterRerouteRequest request, ActionListene for (Map.Entry> entry : stalePrimaryAllocations.entrySet()) { final String index = entry.getKey(); final ImmutableOpenIntMap> indexStatus = status.get(index); - assert indexStatus != null; + if (indexStatus == null) { + // The index in the stale primary allocation request was green and hence filtered out by the store status + // request. We ignore it here since the relevant exception will be thrown by the reroute action later on. + continue; + } for (AbstractAllocateAllocationCommand command : entry.getValue()) { final List shardStatus = indexStatus.get(command.shardId()); diff --git a/server/src/test/java/org/elasticsearch/cluster/routing/PrimaryAllocationIT.java b/server/src/test/java/org/elasticsearch/cluster/routing/PrimaryAllocationIT.java index 3f826c587e683..b106944e97065 100644 --- a/server/src/test/java/org/elasticsearch/cluster/routing/PrimaryAllocationIT.java +++ b/server/src/test/java/org/elasticsearch/cluster/routing/PrimaryAllocationIT.java @@ -265,7 +265,6 @@ public void testForceStaleReplicaToBePromotedToPrimary() throws Exception { assertThat(newHistoryUUIds, hasSize(1)); } - @AwaitsFix(bugUrl = "https://github.com/elastic/elasticsearch/issues/37345") public void testForceStaleReplicaToBePromotedToPrimaryOnWrongNode() throws Exception { String master = internalCluster().startMasterOnlyNode(Settings.EMPTY); internalCluster().startDataOnlyNodes(2); @@ -275,7 +274,10 @@ public void testForceStaleReplicaToBePromotedToPrimaryOnWrongNode() throws Excep .put("index.number_of_replicas", 1)).get()); ensureGreen(); createStaleReplicaScenario(master); - internalCluster().startDataOnlyNodes(2); + // Ensure the stopped primary's data is deleted so that it doesn't get picked up by the next datanode we start + internalCluster().wipePendingDataDirectories(); + internalCluster().startDataOnlyNodes(1); + ensureStableCluster(3, master); final int shardId = 0; final List nodeNames = new ArrayList<>(Arrays.asList(internalCluster().getNodeNames())); nodeNames.remove(master); @@ -292,6 +294,25 @@ public void testForceStaleReplicaToBePromotedToPrimaryOnWrongNode() throws Excep equalTo("No data for shard [" + shardId + "] of index [" + idxName + "] found on node [" + nodeWithoutData + ']')); } + public void testForceStaleReplicaToBePromotedForGreenIndex() { + internalCluster().startMasterOnlyNode(Settings.EMPTY); + final List dataNodes = internalCluster().startDataOnlyNodes(2); + final String idxName = "test"; + assertAcked(client().admin().indices().prepareCreate(idxName) + .setSettings(Settings.builder().put("index.number_of_shards", 1) + .put("index.number_of_replicas", 1)).get()); + ensureGreen(); + final String nodeWithoutData = randomFrom(dataNodes); + final int shardId = 0; + IllegalArgumentException iae = expectThrows( + IllegalArgumentException.class, + () -> client().admin().cluster().prepareReroute() + .add(new AllocateStalePrimaryAllocationCommand(idxName, shardId, nodeWithoutData, true)).get()); + assertThat( + iae.getMessage(), + equalTo("[allocate_stale_primary] primary [" + idxName+ "][" + shardId + "] is already assigned")); + } + public void testForceStaleReplicaToBePromotedForMissingIndex() { internalCluster().startMasterOnlyNode(Settings.EMPTY); final String dataNode = internalCluster().startDataOnlyNode(); diff --git a/test/framework/src/main/java/org/elasticsearch/test/InternalTestCluster.java b/test/framework/src/main/java/org/elasticsearch/test/InternalTestCluster.java index e6e11dacb749f..e4a11ad414ffe 100644 --- a/test/framework/src/main/java/org/elasticsearch/test/InternalTestCluster.java +++ b/test/framework/src/main/java/org/elasticsearch/test/InternalTestCluster.java @@ -1398,8 +1398,7 @@ private void randomlyResetClients() { } } - private void wipePendingDataDirectories() { - assert Thread.holdsLock(this); + public synchronized void wipePendingDataDirectories() { if (!dataDirToClean.isEmpty()) { try { for (Path path : dataDirToClean) { From 44a1071018b1563a15548fda14f9abbf11985606 Mon Sep 17 00:00:00 2001 From: Nhat Nguyen Date: Sat, 12 Jan 2019 12:49:48 -0500 Subject: [PATCH 128/186] Make recovery source partially non-blocking (#37291) Today a peer-recovery may run into a deadlock if the value of node_concurrent_recoveries is too high. This happens because the peer-recovery is executed in a blocking fashion. This commit attempts to make the recovery source partially non-blocking. I will make three follow-ups to make it fully non-blocking: (1) send translog operations, (2) primary relocation, (3) send commit files. Relates #36195 --- .../common/util/CancellableThreads.java | 81 ++++++++++++------- .../recovery/PeerRecoverySourceService.java | 13 ++- .../indices/recovery/RecoveryResponse.java | 2 +- .../recovery/RecoverySourceHandler.java | 79 ++++++++++-------- .../common/util/CancellableThreadsTests.java | 37 ++++++++- .../recovery/RecoverySourceHandlerTests.java | 7 +- .../index/shard/IndexShardTestCase.java | 7 +- 7 files changed, 149 insertions(+), 77 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/common/util/CancellableThreads.java b/server/src/main/java/org/elasticsearch/common/util/CancellableThreads.java index 3037c7a73c0d5..ec3664fd5b469 100644 --- a/server/src/main/java/org/elasticsearch/common/util/CancellableThreads.java +++ b/server/src/main/java/org/elasticsearch/common/util/CancellableThreads.java @@ -18,6 +18,7 @@ */ package org.elasticsearch.common.util; +import org.apache.lucene.util.SetOnce; import org.apache.lucene.util.ThreadInterruptedException; import org.elasticsearch.ElasticsearchException; import org.elasticsearch.common.Nullable; @@ -38,34 +39,35 @@ public class CancellableThreads { private final Set threads = new HashSet<>(); // needs to be volatile as it is also read outside of synchronized blocks. private volatile boolean cancelled = false; + private final SetOnce onCancel = new SetOnce<>(); private String reason; public synchronized boolean isCancelled() { return cancelled; } - - /** call this will throw an exception if operation was cancelled. - * Override {@link #onCancel(String, Exception)} for custom failure logic */ - public synchronized void checkForCancel() { - if (isCancelled()) { - onCancel(reason, null); - } + public void checkForCancel() { + checkForCancel(null); } - /** - * called if {@link #checkForCancel()} was invoked after the operation was cancelled. - * the default implementation always throws an {@link ExecutionCancelledException}, suppressing - * any other exception that occurred before cancellation - * @param reason reason for failure supplied by the caller of {@link #cancel} - * @param suppressedException any error that was encountered during the execution before the operation was cancelled. - */ - protected void onCancel(String reason, @Nullable Exception suppressedException) { - RuntimeException e = new ExecutionCancelledException("operation was cancelled reason [" + reason + "]"); - if (suppressedException != null) { - e.addSuppressed(suppressedException); + private void checkForCancel(Exception beforeCancelException) { + if (isCancelled()) { + final String reason; + final OnCancel onCancel; + synchronized (this) { + reason = this.reason; + onCancel = this.onCancel.get(); + } + if (onCancel != null) { + onCancel.onCancel(reason, beforeCancelException); + } + // fallback to the default exception + final RuntimeException cancelExp = new ExecutionCancelledException("operation was cancelled reason [" + reason + "]"); + if (beforeCancelException != null) { + cancelExp.addSuppressed(beforeCancelException); + } + throw cancelExp; } - throw e; } private synchronized boolean add() { @@ -125,17 +127,14 @@ public void executeIO(IOInterruptible interruptible) throws IOException { // clear the flag interrupted flag as we are checking for failure.. Thread.interrupted(); } - synchronized (this) { - if (isCancelled()) { - onCancel(reason, ioException != null ? ioException : runtimeException); - } else if (ioException != null) { - // if we're not canceling, we throw the original exception - throw ioException; - } - if (runtimeException != null) { - // if we're not canceling, we throw the original exception - throw runtimeException; - } + checkForCancel(ioException != null ? ioException : runtimeException); + if (ioException != null) { + // if we're not canceling, we throw the original exception + throw ioException; + } + if (runtimeException != null) { + // if we're not canceling, we throw the original exception + throw runtimeException; } if (cancelledByExternalInterrupt) { // restore interrupt flag to at least adhere to expected behavior @@ -185,4 +184,26 @@ public ExecutionCancelledException(StreamInput in) throws IOException { super(in); } } + + /** + * Registers a callback that will be invoked when some running operations are cancelled or {@link #checkForCancel()} is called. + */ + public synchronized void setOnCancel(OnCancel onCancel) { + this.onCancel.set(onCancel); + } + + @FunctionalInterface + public interface OnCancel { + /** + * Called when some running operations are cancelled or {@link #checkForCancel()} is explicitly called. + * If this method throws an exception, cancelling tasks will fail with that exception; otherwise they + * will fail with the default exception {@link ExecutionCancelledException}. + * + * @param reason the reason of the cancellation + * @param beforeCancelException any error that was encountered during the execution before the operations were cancelled. + * @see #checkForCancel() + * @see #setOnCancel(OnCancel) + */ + void onCancel(String reason, @Nullable Exception beforeCancelException); + } } diff --git a/server/src/main/java/org/elasticsearch/indices/recovery/PeerRecoverySourceService.java b/server/src/main/java/org/elasticsearch/indices/recovery/PeerRecoverySourceService.java index 69af8841f4b40..9c30ab156c08a 100644 --- a/server/src/main/java/org/elasticsearch/indices/recovery/PeerRecoverySourceService.java +++ b/server/src/main/java/org/elasticsearch/indices/recovery/PeerRecoverySourceService.java @@ -22,6 +22,8 @@ import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; import org.elasticsearch.ExceptionsHelper; +import org.elasticsearch.action.ActionListener; +import org.elasticsearch.action.support.HandledTransportAction; import org.elasticsearch.cluster.routing.ShardRouting; import org.elasticsearch.common.Nullable; import org.elasticsearch.common.inject.Inject; @@ -81,7 +83,7 @@ public void beforeIndexShardClosed(ShardId shardId, @Nullable IndexShard indexSh } } - private RecoveryResponse recover(final StartRecoveryRequest request) throws IOException { + private void recover(StartRecoveryRequest request, ActionListener listener) throws IOException { final IndexService indexService = indicesService.indexServiceSafe(request.shardId().getIndex()); final IndexShard shard = indexService.getShard(request.shardId().id()); @@ -101,18 +103,13 @@ private RecoveryResponse recover(final StartRecoveryRequest request) throws IOEx RecoverySourceHandler handler = ongoingRecoveries.addNewRecovery(request, shard); logger.trace("[{}][{}] starting recovery to {}", request.shardId().getIndex().getName(), request.shardId().id(), request.targetNode()); - try { - return handler.recoverToTarget(); - } finally { - ongoingRecoveries.remove(shard, handler); - } + handler.recoverToTarget(ActionListener.runAfter(listener, () -> ongoingRecoveries.remove(shard, handler))); } class StartRecoveryTransportRequestHandler implements TransportRequestHandler { @Override public void messageReceived(final StartRecoveryRequest request, final TransportChannel channel, Task task) throws Exception { - RecoveryResponse response = recover(request); - channel.sendResponse(response); + recover(request, new HandledTransportAction.ChannelActionListener<>(channel, Actions.START_RECOVERY, request)); } } diff --git a/server/src/main/java/org/elasticsearch/indices/recovery/RecoveryResponse.java b/server/src/main/java/org/elasticsearch/indices/recovery/RecoveryResponse.java index 02d4ff5dbc13b..605d202cbaf9b 100644 --- a/server/src/main/java/org/elasticsearch/indices/recovery/RecoveryResponse.java +++ b/server/src/main/java/org/elasticsearch/indices/recovery/RecoveryResponse.java @@ -26,7 +26,7 @@ import java.io.IOException; import java.util.List; -final class RecoveryResponse extends TransportResponse { +public final class RecoveryResponse extends TransportResponse { final List phase1FileNames; final List phase1FileSizes; diff --git a/server/src/main/java/org/elasticsearch/indices/recovery/RecoverySourceHandler.java b/server/src/main/java/org/elasticsearch/indices/recovery/RecoverySourceHandler.java index 315af6b4ae084..639ab47799366 100644 --- a/server/src/main/java/org/elasticsearch/indices/recovery/RecoverySourceHandler.java +++ b/server/src/main/java/org/elasticsearch/indices/recovery/RecoverySourceHandler.java @@ -34,7 +34,6 @@ import org.elasticsearch.action.ActionListener; import org.elasticsearch.cluster.routing.IndexShardRoutingTable; import org.elasticsearch.cluster.routing.ShardRouting; -import org.elasticsearch.common.Nullable; import org.elasticsearch.common.StopWatch; import org.elasticsearch.common.bytes.BytesArray; import org.elasticsearch.common.lease.Releasable; @@ -70,6 +69,7 @@ import java.util.List; import java.util.Locale; import java.util.concurrent.CompletableFuture; +import java.util.concurrent.CopyOnWriteArrayList; import java.util.concurrent.atomic.AtomicLong; import java.util.function.Function; import java.util.function.Supplier; @@ -96,22 +96,7 @@ public class RecoverySourceHandler { private final StartRecoveryRequest request; private final int chunkSizeInBytes; private final RecoveryTargetHandler recoveryTarget; - - private final CancellableThreads cancellableThreads = new CancellableThreads() { - @Override - protected void onCancel(String reason, @Nullable Exception suppressedException) { - RuntimeException e; - if (shard.state() == IndexShardState.CLOSED) { // check if the shard got closed on us - e = new IndexShardClosedException(shard.shardId(), "shard is closed and recovery was canceled reason [" + reason + "]"); - } else { - e = new ExecutionCancelledException("recovery was canceled reason [" + reason + "]"); - } - if (suppressedException != null) { - e.addSuppressed(suppressedException); - } - throw e; - } - }; + private final CancellableThreads cancellableThreads = new CancellableThreads(); public RecoverySourceHandler(final IndexShard shard, RecoveryTargetHandler recoveryTarget, final StartRecoveryRequest request, @@ -131,19 +116,37 @@ public StartRecoveryRequest getRequest() { /** * performs the recovery from the local engine to the target */ - public RecoveryResponse recoverToTarget() throws IOException { - runUnderPrimaryPermit(() -> { - final IndexShardRoutingTable routingTable = shard.getReplicationGroup().getRoutingTable(); - ShardRouting targetShardRouting = routingTable.getByAllocationId(request.targetAllocationId()); - if (targetShardRouting == null) { - logger.debug("delaying recovery of {} as it is not listed as assigned to target node {}", request.shardId(), - request.targetNode()); - throw new DelayRecoveryException("source node does not have the shard listed in its state as allocated on the node"); - } - assert targetShardRouting.initializing() : "expected recovery target to be initializing but was " + targetShardRouting; - }, shardId + " validating recovery target ["+ request.targetAllocationId() + "] registered ", shard, cancellableThreads, logger); - - try (Closeable ignored = shard.acquireRetentionLockForPeerRecovery()) { + public void recoverToTarget(ActionListener listener) { + final List resources = new CopyOnWriteArrayList<>(); + final Closeable releaseResources = () -> IOUtils.close(resources); + final ActionListener wrappedListener = ActionListener.notifyOnce(listener); + try { + cancellableThreads.setOnCancel((reason, beforeCancelEx) -> { + final RuntimeException e; + if (shard.state() == IndexShardState.CLOSED) { // check if the shard got closed on us + e = new IndexShardClosedException(shard.shardId(), "shard is closed and recovery was canceled reason [" + reason + "]"); + } else { + e = new CancellableThreads.ExecutionCancelledException("recovery was canceled reason [" + reason + "]"); + } + if (beforeCancelEx != null) { + e.addSuppressed(beforeCancelEx); + } + IOUtils.closeWhileHandlingException(releaseResources, () -> wrappedListener.onFailure(e)); + throw e; + }); + runUnderPrimaryPermit(() -> { + final IndexShardRoutingTable routingTable = shard.getReplicationGroup().getRoutingTable(); + ShardRouting targetShardRouting = routingTable.getByAllocationId(request.targetAllocationId()); + if (targetShardRouting == null) { + logger.debug("delaying recovery of {} as it is not listed as assigned to target node {}", request.shardId(), + request.targetNode()); + throw new DelayRecoveryException("source node does not have the shard listed in its state as allocated on the node"); + } + assert targetShardRouting.initializing() : "expected recovery target to be initializing but was " + targetShardRouting; + }, shardId + " validating recovery target ["+ request.targetAllocationId() + "] registered ", + shard, cancellableThreads, logger); + final Closeable retentionLock = shard.acquireRetentionLockForPeerRecovery(); + resources.add(retentionLock); final long startingSeqNo; final long requiredSeqNoRangeStart; final boolean isSequenceNumberBasedRecovery = request.startingSeqNo() != SequenceNumbers.UNASSIGNED_SEQ_NO && @@ -217,6 +220,8 @@ public RecoveryResponse recoverToTarget() throws IOException { } final SendSnapshotResult sendSnapshotResult; try (Translog.Snapshot snapshot = shard.getHistoryOperations("peer-recovery", startingSeqNo)) { + // we can release the retention lock here because the snapshot itself will retain the required operations. + IOUtils.close(retentionLock, () -> resources.remove(retentionLock)); // we have to capture the max_seen_auto_id_timestamp and the max_seq_no_of_updates to make sure that these values // are at least as high as the corresponding values on the primary when any of these operations were executed on it. final long maxSeenAutoIdTimestamp = shard.getMaxSeenAutoIdTimestamp(); @@ -229,10 +234,16 @@ public RecoveryResponse recoverToTarget() throws IOException { finalizeRecovery(sendSnapshotResult.targetLocalCheckpoint); final long phase1ThrottlingWaitTime = 0L; // TODO: return the actual throttle time - return new RecoveryResponse(sendFileResult.phase1FileNames, sendFileResult.phase1FileSizes, - sendFileResult.phase1ExistingFileNames, sendFileResult.phase1ExistingFileSizes, sendFileResult.totalSize, - sendFileResult.existingTotalSize, sendFileResult.took.millis(), phase1ThrottlingWaitTime, prepareEngineTime.millis(), - sendSnapshotResult.totalOperations, sendSnapshotResult.tookTime.millis()); + assert resources.isEmpty() : "not every resource is released [" + resources + "]"; + IOUtils.close(resources); + wrappedListener.onResponse( + new RecoveryResponse(sendFileResult.phase1FileNames, sendFileResult.phase1FileSizes, + sendFileResult.phase1ExistingFileNames, sendFileResult.phase1ExistingFileSizes, sendFileResult.totalSize, + sendFileResult.existingTotalSize, sendFileResult.took.millis(), phase1ThrottlingWaitTime, prepareEngineTime.millis(), + sendSnapshotResult.totalOperations, sendSnapshotResult.tookTime.millis()) + ); + } catch (Exception e) { + IOUtils.closeWhileHandlingException(releaseResources, () -> wrappedListener.onFailure(e)); } } diff --git a/server/src/test/java/org/elasticsearch/common/util/CancellableThreadsTests.java b/server/src/test/java/org/elasticsearch/common/util/CancellableThreadsTests.java index 2b937730e4750..8b8d17cbf3bf2 100644 --- a/server/src/test/java/org/elasticsearch/common/util/CancellableThreadsTests.java +++ b/server/src/test/java/org/elasticsearch/common/util/CancellableThreadsTests.java @@ -24,7 +24,12 @@ import org.hamcrest.Matchers; import java.io.IOException; +import java.util.Arrays; import java.util.concurrent.CountDownLatch; +import java.util.concurrent.atomic.AtomicInteger; + +import static org.elasticsearch.common.util.CancellableThreads.ExecutionCancelledException; +import static org.hamcrest.Matchers.equalTo; public class CancellableThreadsTests extends ESTestCase { public static class CustomException extends RuntimeException { @@ -39,6 +44,8 @@ public IOCustomException(String msg) { } } + static class ThrowOnCancelException extends RuntimeException { + } private class TestPlan { public final int id; @@ -167,6 +174,19 @@ public void testCancellableThreads() throws InterruptedException { } readyForCancel.await(); + final boolean throwInOnCancel = randomBoolean(); + final AtomicInteger invokeTimes = new AtomicInteger(); + cancellableThreads.setOnCancel((reason, beforeCancelException) -> { + invokeTimes.getAndIncrement(); + if (throwInOnCancel) { + ThrowOnCancelException e = new ThrowOnCancelException(); + if (beforeCancelException != null) { + e.addSuppressed(beforeCancelException); + } + throw e; + } + }); + cancellableThreads.cancel("test"); for (Thread thread : threads) { thread.join(20000); @@ -181,7 +201,11 @@ public void testCancellableThreads() throws InterruptedException { assertNull(exceptions[i]); } else { // in all other cases, we expect a cancellation exception. - assertThat(exceptions[i], Matchers.instanceOf(CancellableThreads.ExecutionCancelledException.class)); + if (throwInOnCancel) { + assertThat(exceptions[i], Matchers.instanceOf(ThrowOnCancelException.class)); + } else { + assertThat(exceptions[i], Matchers.instanceOf(ExecutionCancelledException.class)); + } if (plan.exceptAfterCancel) { assertThat(exceptions[i].getSuppressed(), Matchers.arrayContaining( @@ -191,8 +215,17 @@ public void testCancellableThreads() throws InterruptedException { assertThat(exceptions[i].getSuppressed(), Matchers.emptyArray()); } } - assertThat(interrupted[plan.id], Matchers.equalTo(plan.presetInterrupt)); + assertThat(interrupted[plan.id], equalTo(plan.presetInterrupt)); + } + assertThat(invokeTimes.longValue(), + equalTo(Arrays.stream(plans).filter(p -> p.exceptBeforeCancel == false && p.exitBeforeCancel == false).count())); + if (throwInOnCancel) { + expectThrows(ThrowOnCancelException.class, cancellableThreads::checkForCancel); + } else { + expectThrows(ExecutionCancelledException.class, cancellableThreads::checkForCancel); } + assertThat(invokeTimes.longValue(), + equalTo(Arrays.stream(plans).filter(p -> p.exceptBeforeCancel == false && p.exitBeforeCancel == false).count() + 1)); } } diff --git a/server/src/test/java/org/elasticsearch/indices/recovery/RecoverySourceHandlerTests.java b/server/src/test/java/org/elasticsearch/indices/recovery/RecoverySourceHandlerTests.java index 3f6a8072d86d5..17e88d2864d05 100644 --- a/server/src/test/java/org/elasticsearch/indices/recovery/RecoverySourceHandlerTests.java +++ b/server/src/test/java/org/elasticsearch/indices/recovery/RecoverySourceHandlerTests.java @@ -35,6 +35,7 @@ import org.elasticsearch.ExceptionsHelper; import org.elasticsearch.Version; import org.elasticsearch.action.ActionListener; +import org.elasticsearch.action.support.PlainActionFuture; import org.elasticsearch.cluster.metadata.IndexMetaData; import org.elasticsearch.cluster.node.DiscoveryNode; import org.elasticsearch.common.UUIDs; @@ -433,7 +434,11 @@ SendSnapshotResult phase2(long startingSeqNo, long requiredSeqNoRangeStart, long } }; - expectThrows(IndexShardRelocatedException.class, handler::recoverToTarget); + PlainActionFuture future = new PlainActionFuture<>(); + expectThrows(IndexShardRelocatedException.class, () -> { + handler.recoverToTarget(future); + future.actionGet(); + }); assertFalse(phase1Called.get()); assertFalse(prepareTargetForTranslogCalled.get()); assertFalse(phase2Called.get()); diff --git a/test/framework/src/main/java/org/elasticsearch/index/shard/IndexShardTestCase.java b/test/framework/src/main/java/org/elasticsearch/index/shard/IndexShardTestCase.java index 2e13cd6e66543..367e0ded60775 100644 --- a/test/framework/src/main/java/org/elasticsearch/index/shard/IndexShardTestCase.java +++ b/test/framework/src/main/java/org/elasticsearch/index/shard/IndexShardTestCase.java @@ -23,6 +23,7 @@ import org.elasticsearch.Version; import org.elasticsearch.action.admin.indices.flush.FlushRequest; import org.elasticsearch.action.index.IndexRequest; +import org.elasticsearch.action.support.PlainActionFuture; import org.elasticsearch.action.support.replication.TransportReplicationAction; import org.elasticsearch.cluster.metadata.IndexMetaData; import org.elasticsearch.cluster.node.DiscoveryNode; @@ -67,6 +68,7 @@ import org.elasticsearch.indices.breaker.HierarchyCircuitBreakerService; import org.elasticsearch.indices.recovery.PeerRecoveryTargetService; import org.elasticsearch.indices.recovery.RecoveryFailedException; +import org.elasticsearch.indices.recovery.RecoveryResponse; import org.elasticsearch.indices.recovery.RecoverySourceHandler; import org.elasticsearch.indices.recovery.RecoveryState; import org.elasticsearch.indices.recovery.RecoveryTarget; @@ -604,7 +606,10 @@ protected final void recoverUnstartedReplica(final IndexShard replica, (int) ByteSizeUnit.MB.toBytes(1)); primary.updateShardState(primary.routingEntry(), primary.getPendingPrimaryTerm(), null, currentClusterStateVersion.incrementAndGet(), inSyncIds, routingTable, Collections.emptySet()); - recovery.recoverToTarget(); + + PlainActionFuture future = new PlainActionFuture<>(); + recovery.recoverToTarget(future); + future.actionGet(); recoveryTarget.markAsDone(); } From 03be4dbaca182a360e0a6cf3aa547739f44f1308 Mon Sep 17 00:00:00 2001 From: Jason Tedor Date: Sat, 12 Jan 2019 14:43:19 -0800 Subject: [PATCH 129/186] Introduce retention lease persistence (#37375) This commit introduces the persistence of retention leases by persisting them in index commits and recovering them when recovering a shard from store. --- docs/reference/indices/flush.asciidoc | 4 +- .../elasticsearch/index/engine/Engine.java | 1 + .../index/engine/InternalEngine.java | 10 +- .../index/engine/SoftDeletesPolicy.java | 15 ++- .../index/seqno/ReplicationTracker.java | 11 ++ .../index/seqno/RetentionLease.java | 107 +++++++++++++++++ .../elasticsearch/index/shard/IndexShard.java | 11 ++ .../index/engine/InternalEngineTests.java | 27 ++++- .../index/seqno/RetentionLeaseTests.java | 70 +++++++++++- .../shard/IndexShardRetentionLeaseTests.java | 53 +++++++++ .../index/engine/EngineTestCase.java | 108 ++++++++++++++---- 11 files changed, 386 insertions(+), 31 deletions(-) diff --git a/docs/reference/indices/flush.asciidoc b/docs/reference/indices/flush.asciidoc index c6bf60182fd76..a0027756ab57a 100644 --- a/docs/reference/indices/flush.asciidoc +++ b/docs/reference/indices/flush.asciidoc @@ -102,7 +102,8 @@ which returns something similar to: "max_seq_no" : "-1", "sync_id" : "AVvFY-071siAOuFGEO9P", <1> "max_unsafe_auto_id_timestamp" : "-1", - "min_retained_seq_no": "0" + "min_retained_seq_no" : "0", + "retention_leases" : "id:replica-0;retaining_seq_no:0;timestamp:1547235588;source:replica" }, "num_docs" : 0 } @@ -117,6 +118,7 @@ which returns something similar to: // TESTRESPONSE[s/"translog_uuid" : "hnOG3xFcTDeoI_kvvvOdNA"/"translog_uuid": $body.indices.twitter.shards.0.0.commit.user_data.translog_uuid/] // TESTRESPONSE[s/"history_uuid" : "XP7KDJGiS1a2fHYiFL5TXQ"/"history_uuid": $body.indices.twitter.shards.0.0.commit.user_data.history_uuid/] // TESTRESPONSE[s/"sync_id" : "AVvFY-071siAOuFGEO9P"/"sync_id": $body.indices.twitter.shards.0.0.commit.user_data.sync_id/] +// TESTRESPONSE[s/"retention_leases" : "id:replica-0;retaining_seq_no:0;timestamp:1547235588;source:replica"/"retention_leases": $body.indices.twitter.shards.0.0.commit.user_data.retention_leases/] <1> the `sync id` marker [float] diff --git a/server/src/main/java/org/elasticsearch/index/engine/Engine.java b/server/src/main/java/org/elasticsearch/index/engine/Engine.java index 008b85331030d..e1df104d338df 100644 --- a/server/src/main/java/org/elasticsearch/index/engine/Engine.java +++ b/server/src/main/java/org/elasticsearch/index/engine/Engine.java @@ -113,6 +113,7 @@ public abstract class Engine implements Closeable { public static final String SYNC_COMMIT_ID = "sync_id"; public static final String HISTORY_UUID_KEY = "history_uuid"; public static final String MIN_RETAINED_SEQNO = "min_retained_seq_no"; + public static final String RETENTION_LEASES = "retention_leases"; public static final String MAX_UNSAFE_AUTO_ID_TIMESTAMP_COMMIT_ID = "max_unsafe_auto_id_timestamp"; protected final ShardId shardId; diff --git a/server/src/main/java/org/elasticsearch/index/engine/InternalEngine.java b/server/src/main/java/org/elasticsearch/index/engine/InternalEngine.java index d0e55fc13eeda..df52c6bc0213f 100644 --- a/server/src/main/java/org/elasticsearch/index/engine/InternalEngine.java +++ b/server/src/main/java/org/elasticsearch/index/engine/InternalEngine.java @@ -51,6 +51,7 @@ import org.elasticsearch.action.index.IndexRequest; import org.elasticsearch.common.Nullable; import org.elasticsearch.common.SuppressForbidden; +import org.elasticsearch.common.collect.Tuple; import org.elasticsearch.common.lease.Releasable; import org.elasticsearch.common.lucene.LoggerInfoStream; import org.elasticsearch.common.lucene.Lucene; @@ -74,6 +75,7 @@ import org.elasticsearch.index.merge.MergeStats; import org.elasticsearch.index.merge.OnGoingMerge; import org.elasticsearch.index.seqno.LocalCheckpointTracker; +import org.elasticsearch.index.seqno.RetentionLease; import org.elasticsearch.index.seqno.SeqNoStats; import org.elasticsearch.index.seqno.SequenceNumbers; import org.elasticsearch.index.shard.ElasticsearchMergePolicy; @@ -2336,7 +2338,13 @@ protected void commitIndexWriter(final IndexWriter writer, final Translog transl commitData.put(MAX_UNSAFE_AUTO_ID_TIMESTAMP_COMMIT_ID, Long.toString(maxUnsafeAutoIdTimestamp.get())); commitData.put(HISTORY_UUID_KEY, historyUUID); if (softDeleteEnabled) { - commitData.put(Engine.MIN_RETAINED_SEQNO, Long.toString(softDeletesPolicy.getMinRetainedSeqNo())); + /* + * We sample these from the policy (which occurs under a lock) to ensure that we have a consistent view of the minimum + * retained sequence number, and the retention leases. + */ + final Tuple> retentionPolicy = softDeletesPolicy.getRetentionPolicy(); + commitData.put(Engine.MIN_RETAINED_SEQNO, Long.toString(retentionPolicy.v1())); + commitData.put(Engine.RETENTION_LEASES, RetentionLease.encodeRetentionLeases(retentionPolicy.v2())); } logger.trace("committing writer with commit data [{}]", commitData); return commitData.entrySet().iterator(); diff --git a/server/src/main/java/org/elasticsearch/index/engine/SoftDeletesPolicy.java b/server/src/main/java/org/elasticsearch/index/engine/SoftDeletesPolicy.java index c957902d8df77..a2452d4b53eb9 100644 --- a/server/src/main/java/org/elasticsearch/index/engine/SoftDeletesPolicy.java +++ b/server/src/main/java/org/elasticsearch/index/engine/SoftDeletesPolicy.java @@ -21,6 +21,7 @@ import org.apache.lucene.document.LongPoint; import org.apache.lucene.search.Query; +import org.elasticsearch.common.collect.Tuple; import org.elasticsearch.common.lease.Releasable; import org.elasticsearch.index.mapper.SeqNoFieldMapper; import org.elasticsearch.index.seqno.RetentionLease; @@ -45,6 +46,7 @@ final class SoftDeletesPolicy { private long retentionOperations; // The min seq_no value that is retained - ops after this seq# should exist in the Lucene index. private long minRetainedSeqNo; + private Collection retentionLeases; // provides the retention leases used to calculate the minimum sequence number to retain private final Supplier> retentionLeasesSupplier; @@ -57,6 +59,7 @@ final class SoftDeletesPolicy { this.retentionOperations = retentionOperations; this.minRetainedSeqNo = minRetainedSeqNo; this.retentionLeasesSupplier = Objects.requireNonNull(retentionLeasesSupplier); + retentionLeases = retentionLeasesSupplier.get(); this.localCheckpointOfSafeCommit = SequenceNumbers.NO_OPS_PERFORMED; this.retentionLockCount = 0; } @@ -106,7 +109,11 @@ private synchronized void releaseRetentionLock() { * Operations whose seq# is least this value should exist in the Lucene index. */ synchronized long getMinRetainedSeqNo() { - // Do not advance if the retention lock is held + return getRetentionPolicy().v1(); + } + + public synchronized Tuple> getRetentionPolicy() { + // do not advance if the retention lock is held if (retentionLockCount == 0) { /* * This policy retains operations for two purposes: peer-recovery and querying changes history. @@ -119,8 +126,8 @@ synchronized long getMinRetainedSeqNo() { */ // calculate the minimum sequence number to retain based on retention leases - final long minimumRetainingSequenceNumber = retentionLeasesSupplier - .get() + retentionLeases = retentionLeasesSupplier.get(); + final long minimumRetainingSequenceNumber = retentionLeases .stream() .mapToLong(RetentionLease::retainingSequenceNumber) .min() @@ -139,7 +146,7 @@ synchronized long getMinRetainedSeqNo() { */ minRetainedSeqNo = Math.max(minRetainedSeqNo, minSeqNoToRetain); } - return minRetainedSeqNo; + return Tuple.tuple(minRetainedSeqNo, retentionLeases); } /** diff --git a/server/src/main/java/org/elasticsearch/index/seqno/ReplicationTracker.java b/server/src/main/java/org/elasticsearch/index/seqno/ReplicationTracker.java index 4298e5712bfc6..f309512ec98b6 100644 --- a/server/src/main/java/org/elasticsearch/index/seqno/ReplicationTracker.java +++ b/server/src/main/java/org/elasticsearch/index/seqno/ReplicationTracker.java @@ -185,6 +185,17 @@ public synchronized void addOrUpdateRetentionLease(final String id, final long r retentionLeases.put(id, new RetentionLease(id, retainingSequenceNumber, currentTimeMillisSupplier.getAsLong(), source)); } + /** + * Updates retention leases on a replica. + * + * @param retentionLeases the retention leases + */ + public synchronized void updateRetentionLeasesOnReplica(final Collection retentionLeases) { + assert primaryMode == false; + this.retentionLeases.clear(); + this.retentionLeases.putAll(retentionLeases.stream().collect(Collectors.toMap(RetentionLease::id, Function.identity()))); + } + public static class CheckpointState implements Writeable { /** diff --git a/server/src/main/java/org/elasticsearch/index/seqno/RetentionLease.java b/server/src/main/java/org/elasticsearch/index/seqno/RetentionLease.java index 076b707a5df42..f763759261385 100644 --- a/server/src/main/java/org/elasticsearch/index/seqno/RetentionLease.java +++ b/server/src/main/java/org/elasticsearch/index/seqno/RetentionLease.java @@ -19,6 +19,13 @@ package org.elasticsearch.index.seqno; +import java.util.Arrays; +import java.util.Collection; +import java.util.Collections; +import java.util.Locale; +import java.util.Objects; +import java.util.stream.Collectors; + /** * A "shard history retention lease" (or "retention lease" for short) is conceptually a marker containing a retaining sequence number such * that all operations with sequence number at least that retaining sequence number will be retained during merge operations (which could @@ -81,18 +88,118 @@ public String source() { * @param source the source of the retention lease */ public RetentionLease(final String id, final long retainingSequenceNumber, final long timestamp, final String source) { + Objects.requireNonNull(id); + if (id.isEmpty()) { + throw new IllegalArgumentException("retention lease ID can not be empty"); + } + if (id.contains(":") || id.contains(";") || id.contains(",")) { + // retention lease IDs can not contain these characters because they are used in encoding retention leases + throw new IllegalArgumentException("retention lease ID can not contain any of [:;,] but was [" + id + "]"); + } if (retainingSequenceNumber < SequenceNumbers.UNASSIGNED_SEQ_NO) { throw new IllegalArgumentException("retention lease retaining sequence number [" + retainingSequenceNumber + "] out of range"); } if (timestamp < 0) { throw new IllegalArgumentException("retention lease timestamp [" + timestamp + "] out of range"); } + Objects.requireNonNull(source); + if (source.isEmpty()) { + throw new IllegalArgumentException("retention lease source can not be empty"); + } + if (source.contains(":") || source.contains(";") || source.contains(",")) { + // retention lease sources can not contain these characters because they are used in encoding retention leases + throw new IllegalArgumentException("retention lease source can not contain any of [:;,] but was [" + source + "]"); + } this.id = id; this.retainingSequenceNumber = retainingSequenceNumber; this.timestamp = timestamp; this.source = source; } + /** + * Encodes a retention lease as a string. This encoding can be decoded by {@link #decodeRetentionLease(String)}. The retention lease is + * encoded in the format id:{id};retaining_seq_no:{retainingSequenecNumber};timestamp:{timestamp};source:{source}. + * + * @param retentionLease the retention lease + * @return the encoding of the retention lease + */ + static String encodeRetentionLease(final RetentionLease retentionLease) { + Objects.requireNonNull(retentionLease); + return String.format( + Locale.ROOT, + "id:%s;retaining_seq_no:%d;timestamp:%d;source:%s", + retentionLease.id(), + retentionLease.retainingSequenceNumber(), + retentionLease.timestamp(), + retentionLease.source()); + } + + /** + * Encodes a collection of retention leases as a string. This encoding can be decoed by {@link #decodeRetentionLeases(String)}. The + * encoding is a comma-separated encoding of each retention lease as encoded by {@link #encodeRetentionLease(RetentionLease)}. + * + * @param retentionLeases the retention leases + * @return the encoding of the retention leases + */ + public static String encodeRetentionLeases(final Collection retentionLeases) { + Objects.requireNonNull(retentionLeases); + return retentionLeases.stream().map(RetentionLease::encodeRetentionLease).collect(Collectors.joining(",")); + } + + /** + * Decodes a retention lease encoded by {@link #encodeRetentionLease(RetentionLease)}. + * + * @param encodedRetentionLease an encoded retention lease + * @return the decoded retention lease + */ + static RetentionLease decodeRetentionLease(final String encodedRetentionLease) { + Objects.requireNonNull(encodedRetentionLease); + final String[] fields = encodedRetentionLease.split(";"); + assert fields.length == 4 : Arrays.toString(fields); + assert fields[0].matches("id:[^:;,]+") : fields[0]; + final String id = fields[0].substring("id:".length()); + assert fields[1].matches("retaining_seq_no:\\d+") : fields[1]; + final long retainingSequenceNumber = Long.parseLong(fields[1].substring("retaining_seq_no:".length())); + assert fields[2].matches("timestamp:\\d+") : fields[2]; + final long timestamp = Long.parseLong(fields[2].substring("timestamp:".length())); + assert fields[3].matches("source:[^:;,]+") : fields[3]; + final String source = fields[3].substring("source:".length()); + return new RetentionLease(id, retainingSequenceNumber, timestamp, source); + } + + /** + * Decodes retention leases encoded by {@link #encodeRetentionLeases(Collection)}. + * + * @param encodedRetentionLeases an encoded collection of retention leases + * @return the decoded retention leases + */ + public static Collection decodeRetentionLeases(final String encodedRetentionLeases) { + Objects.requireNonNull(encodedRetentionLeases); + if (encodedRetentionLeases.isEmpty()) { + return Collections.emptyList(); + } + assert Arrays.stream(encodedRetentionLeases.split(",")) + .allMatch(s -> s.matches("id:[^:;,]+;retaining_seq_no:\\d+;timestamp:\\d+;source:[^:;,]+")) + : encodedRetentionLeases; + return Arrays.stream(encodedRetentionLeases.split(",")).map(RetentionLease::decodeRetentionLease).collect(Collectors.toList()); + } + + @Override + public boolean equals(final Object o) { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + final RetentionLease that = (RetentionLease) o; + return Objects.equals(id, that.id) && + retainingSequenceNumber == that.retainingSequenceNumber && + timestamp == that.timestamp && + Objects.equals(source, that.source); + } + + @Override + public int hashCode() { + return Objects.hash(id, retainingSequenceNumber, timestamp, source); + } + @Override public String toString() { return "RetentionLease{" + diff --git a/server/src/main/java/org/elasticsearch/index/shard/IndexShard.java b/server/src/main/java/org/elasticsearch/index/shard/IndexShard.java index 66b8e607b5c76..270ce20819102 100644 --- a/server/src/main/java/org/elasticsearch/index/shard/IndexShard.java +++ b/server/src/main/java/org/elasticsearch/index/shard/IndexShard.java @@ -106,6 +106,7 @@ import org.elasticsearch.index.search.stats.SearchStats; import org.elasticsearch.index.search.stats.ShardSearchStats; import org.elasticsearch.index.seqno.ReplicationTracker; +import org.elasticsearch.index.seqno.RetentionLease; import org.elasticsearch.index.seqno.SeqNoStats; import org.elasticsearch.index.seqno.SequenceNumbers; import org.elasticsearch.index.shard.PrimaryReplicaSyncer.ResyncTask; @@ -140,6 +141,7 @@ import java.nio.channels.ClosedByInterruptException; import java.nio.charset.StandardCharsets; import java.util.ArrayList; +import java.util.Collection; import java.util.Collections; import java.util.EnumSet; import java.util.List; @@ -1416,6 +1418,7 @@ private void innerOpenEngineAndTranslog() throws IOException { final String translogUUID = store.readLastCommittedSegmentsInfo().getUserData().get(Translog.TRANSLOG_UUID_KEY); final long globalCheckpoint = Translog.readGlobalCheckpoint(translogConfig.getTranslogPath(), translogUUID); replicationTracker.updateGlobalCheckpointOnReplica(globalCheckpoint, "read from translog checkpoint"); + replicationTracker.updateRetentionLeasesOnReplica(getRetentionLeases(store.readLastCommittedSegmentsInfo())); trimUnsafeCommits(); synchronized (mutex) { verifyNotClosed(); @@ -1435,6 +1438,14 @@ private void innerOpenEngineAndTranslog() throws IOException { assert recoveryState.getStage() == RecoveryState.Stage.TRANSLOG : "TRANSLOG stage expected but was: " + recoveryState.getStage(); } + static Collection getRetentionLeases(final SegmentInfos segmentInfos) { + final String committedRetentionLeases = segmentInfos.getUserData().get(Engine.RETENTION_LEASES); + if (committedRetentionLeases == null) { + return Collections.emptyList(); + } + return RetentionLease.decodeRetentionLeases(committedRetentionLeases); + } + private void trimUnsafeCommits() throws IOException { assert currentEngineReference.get() == null || currentEngineReference.get() instanceof ReadOnlyEngine : "a write engine is running"; final String translogUUID = store.readLastCommittedSegmentsInfo().getUserData().get(Translog.TRANSLOG_UUID_KEY); diff --git a/server/src/test/java/org/elasticsearch/index/engine/InternalEngineTests.java b/server/src/test/java/org/elasticsearch/index/engine/InternalEngineTests.java index f88aaedd6991f..796d7eb0c60ec 100644 --- a/server/src/test/java/org/elasticsearch/index/engine/InternalEngineTests.java +++ b/server/src/test/java/org/elasticsearch/index/engine/InternalEngineTests.java @@ -116,6 +116,7 @@ import org.elasticsearch.index.mapper.SourceFieldMapper; import org.elasticsearch.index.seqno.LocalCheckpointTracker; import org.elasticsearch.index.seqno.ReplicationTracker; +import org.elasticsearch.index.seqno.RetentionLease; import org.elasticsearch.index.seqno.SeqNoStats; import org.elasticsearch.index.seqno.SequenceNumbers; import org.elasticsearch.index.shard.IndexSearcherWrapper; @@ -140,6 +141,7 @@ import java.util.ArrayList; import java.util.Arrays; import java.util.Base64; +import java.util.Collection; import java.util.Collections; import java.util.Comparator; import java.util.HashMap; @@ -5241,13 +5243,14 @@ public void testKeepMinRetainedSeqNoByMergePolicy() throws IOException { final IndexMetaData indexMetaData = IndexMetaData.builder(defaultSettings.getIndexMetaData()).settings(settings).build(); final IndexSettings indexSettings = IndexSettingsModule.newIndexSettings(indexMetaData); final AtomicLong globalCheckpoint = new AtomicLong(SequenceNumbers.NO_OPS_PERFORMED); + final AtomicReference> leasesHolder = new AtomicReference<>(Collections.emptyList()); final List operations = generateSingleDocHistory(true, randomFrom(VersionType.INTERNAL, VersionType.EXTERNAL), 2, 10, 300, "2"); Randomness.shuffle(operations); Set existingSeqNos = new HashSet<>(); store = createStore(); - engine = createEngine(config(indexSettings, store, createTempDir(), newMergePolicy(), null, null, - globalCheckpoint::get)); + engine = createEngine( + config(indexSettings, store, createTempDir(), newMergePolicy(), null, null, globalCheckpoint::get, leasesHolder::get)); assertThat(engine.getMinRetainedSeqNo(), equalTo(0L)); long lastMinRetainedSeqNo = engine.getMinRetainedSeqNo(); for (Engine.Operation op : operations) { @@ -5261,6 +5264,18 @@ public void testKeepMinRetainedSeqNoByMergePolicy() throws IOException { if (randomBoolean()) { globalCheckpoint.set(randomLongBetween(globalCheckpoint.get(), engine.getLocalCheckpointTracker().getCheckpoint())); } + if (randomBoolean()) { + final int length = randomIntBetween(0, 8); + final List leases = new ArrayList<>(length); + for (int i = 0; i < length; i++) { + final String id = randomAlphaOfLength(8); + final long retainingSequenceNumber = randomLongBetween(0L, Math.max(0L, globalCheckpoint.get())); + final long timestamp = randomLongBetween(0L, Long.MAX_VALUE); + final String source = randomAlphaOfLength(8); + leases.add(new RetentionLease(id, retainingSequenceNumber, timestamp, source)); + } + leasesHolder.set(leases); + } if (rarely()) { settings.put(IndexSettings.INDEX_SOFT_DELETES_RETENTION_OPERATIONS_SETTING.getKey(), randomLongBetween(0, 10)); indexSettings.updateIndexMetaData(IndexMetaData.builder(defaultSettings.getIndexMetaData()).settings(settings).build()); @@ -5273,6 +5288,14 @@ public void testKeepMinRetainedSeqNoByMergePolicy() throws IOException { engine.flush(true, true); assertThat(Long.parseLong(engine.getLastCommittedSegmentInfos().userData.get(Engine.MIN_RETAINED_SEQNO)), equalTo(engine.getMinRetainedSeqNo())); + final Collection leases = leasesHolder.get(); + if (leases.isEmpty()) { + assertThat(engine.getLastCommittedSegmentInfos().getUserData().get(Engine.RETENTION_LEASES), equalTo("")); + } else { + assertThat( + engine.getLastCommittedSegmentInfos().getUserData().get(Engine.RETENTION_LEASES), + equalTo(RetentionLease.encodeRetentionLeases(leases))); + } } if (rarely()) { engine.forceMerge(randomBoolean()); diff --git a/server/src/test/java/org/elasticsearch/index/seqno/RetentionLeaseTests.java b/server/src/test/java/org/elasticsearch/index/seqno/RetentionLeaseTests.java index a5e4af5d0e6a3..c4340a381ce25 100644 --- a/server/src/test/java/org/elasticsearch/index/seqno/RetentionLeaseTests.java +++ b/server/src/test/java/org/elasticsearch/index/seqno/RetentionLeaseTests.java @@ -21,12 +21,34 @@ import org.elasticsearch.test.ESTestCase; +import java.util.ArrayList; +import java.util.Collection; +import java.util.List; + import static org.elasticsearch.index.seqno.SequenceNumbers.UNASSIGNED_SEQ_NO; +import static org.hamcrest.Matchers.contains; import static org.hamcrest.Matchers.containsString; +import static org.hamcrest.Matchers.empty; +import static org.hamcrest.Matchers.equalTo; import static org.hamcrest.Matchers.hasToString; public class RetentionLeaseTests extends ESTestCase { + public void testInvalidId() { + final String id = "id" + randomFrom(":", ";", ","); + final IllegalArgumentException e = expectThrows( + IllegalArgumentException.class, + () -> new RetentionLease(id, randomNonNegativeLong(), randomNonNegativeLong(), "source")); + assertThat(e, hasToString(containsString("retention lease ID can not contain any of [:;,] but was [" + id + "]"))); + } + + public void testEmptyId() { + final IllegalArgumentException e = expectThrows( + IllegalArgumentException.class, + () -> new RetentionLease("", randomNonNegativeLong(), randomNonNegativeLong(), "source")); + assertThat(e, hasToString(containsString("retention lease ID can not be empty"))); + } + public void testRetainingSequenceNumberOutOfRange() { final long retainingSequenceNumber = randomLongBetween(Long.MIN_VALUE, UNASSIGNED_SEQ_NO - 1); final IllegalArgumentException e = expectThrows( @@ -42,9 +64,51 @@ public void testTimestampOutOfRange() { final IllegalArgumentException e = expectThrows( IllegalArgumentException.class, () -> new RetentionLease("id", randomLongBetween(SequenceNumbers.NO_OPS_PERFORMED, Long.MAX_VALUE), timestamp, "source")); - assertThat( - e, - hasToString(containsString("retention lease timestamp [" + timestamp + "] out of range"))); + assertThat(e, hasToString(containsString("retention lease timestamp [" + timestamp + "] out of range"))); + } + + public void testInvalidSource() { + final String source = "source" + randomFrom(":", ";", ","); + final IllegalArgumentException e = expectThrows( + IllegalArgumentException.class, + () -> new RetentionLease("id", randomNonNegativeLong(), randomNonNegativeLong(), source)); + assertThat(e, hasToString(containsString("retention lease source can not contain any of [:;,] but was [" + source + "]"))); + } + + public void testEmptySource() { + final IllegalArgumentException e = expectThrows( + IllegalArgumentException.class, + () -> new RetentionLease("id", randomNonNegativeLong(), randomNonNegativeLong(), "")); + assertThat(e, hasToString(containsString("retention lease source can not be empty"))); + } + + public void testRetentionLeaseEncoding() { + final String id = randomAlphaOfLength(8); + final long retainingSequenceNumber = randomNonNegativeLong(); + final long timestamp = randomNonNegativeLong(); + final String source = randomAlphaOfLength(8); + final RetentionLease retentionLease = new RetentionLease(id, retainingSequenceNumber, timestamp, source); + assertThat(RetentionLease.decodeRetentionLease(RetentionLease.encodeRetentionLease(retentionLease)), equalTo(retentionLease)); + } + + public void testRetentionLeasesEncoding() { + final int length = randomIntBetween(0, 8); + final List retentionLeases = new ArrayList<>(length); + for (int i = 0; i < length; i++) { + final String id = randomAlphaOfLength(8); + final long retainingSequenceNumber = randomNonNegativeLong(); + final long timestamp = randomNonNegativeLong(); + final String source = randomAlphaOfLength(8); + final RetentionLease retentionLease = new RetentionLease(id, retainingSequenceNumber, timestamp, source); + retentionLeases.add(retentionLease); + } + final Collection decodedRetentionLeases = + RetentionLease.decodeRetentionLeases(RetentionLease.encodeRetentionLeases(retentionLeases)); + if (length == 0) { + assertThat(decodedRetentionLeases, empty()); + } else { + assertThat(decodedRetentionLeases, contains(retentionLeases.toArray(new RetentionLease[0]))); + } } } diff --git a/server/src/test/java/org/elasticsearch/index/shard/IndexShardRetentionLeaseTests.java b/server/src/test/java/org/elasticsearch/index/shard/IndexShardRetentionLeaseTests.java index bd2a33617eecf..e95b52280ca2e 100644 --- a/server/src/test/java/org/elasticsearch/index/shard/IndexShardRetentionLeaseTests.java +++ b/server/src/test/java/org/elasticsearch/index/shard/IndexShardRetentionLeaseTests.java @@ -19,9 +19,14 @@ package org.elasticsearch.index.shard; +import org.apache.lucene.index.SegmentInfos; +import org.elasticsearch.action.admin.indices.flush.FlushRequest; +import org.elasticsearch.cluster.routing.RecoverySource; +import org.elasticsearch.cluster.routing.ShardRoutingHelper; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.unit.TimeValue; import org.elasticsearch.index.IndexSettings; +import org.elasticsearch.index.engine.Engine; import org.elasticsearch.index.engine.InternalEngineFactory; import org.elasticsearch.index.seqno.RetentionLease; import org.elasticsearch.index.seqno.SequenceNumbers; @@ -33,9 +38,11 @@ import java.util.Map; import java.util.concurrent.ExecutorService; import java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicLong; import java.util.function.LongSupplier; +import static org.hamcrest.Matchers.contains; import static org.hamcrest.Matchers.equalTo; import static org.hamcrest.Matchers.hasItem; import static org.hamcrest.Matchers.hasSize; @@ -127,6 +134,52 @@ public void testExpiration() throws IOException { } } + public void testCommit() throws IOException { + final Settings settings = Settings.builder() + .put(IndexSettings.INDEX_SOFT_DELETES_SETTING.getKey(), true) + .put(IndexSettings.INDEX_SOFT_DELETES_RETENTION_LEASE_SETTING.getKey(), Long.MAX_VALUE, TimeUnit.NANOSECONDS) + .build(); + final IndexShard indexShard = newStartedShard( + true, + settings, + new InternalEngineFactory()); + try { + final int length = randomIntBetween(0, 8); + final long[] minimumRetainingSequenceNumbers = new long[length]; + for (int i = 0; i < length; i++) { + minimumRetainingSequenceNumbers[i] = randomLongBetween(SequenceNumbers.NO_OPS_PERFORMED, Long.MAX_VALUE); + currentTimeMillis.set(TimeUnit.NANOSECONDS.toMillis(randomNonNegativeLong())); + indexShard.addOrUpdateRetentionLease(Integer.toString(i), minimumRetainingSequenceNumbers[i], "test-" + i); + } + + currentTimeMillis.set(TimeUnit.NANOSECONDS.toMillis(Long.MAX_VALUE)); + + // force a commit + indexShard.flush(new FlushRequest().force(true)); + + // the committed retention leases should equal our current retention leases + final SegmentInfos segmentCommitInfos = indexShard.store().readLastCommittedSegmentsInfo(); + assertTrue(segmentCommitInfos.getUserData().containsKey(Engine.RETENTION_LEASES)); + final Collection retentionLeases = indexShard.getEngine().config().retentionLeasesSupplier().get(); + assertThat(IndexShard.getRetentionLeases(segmentCommitInfos), contains(retentionLeases.toArray(new RetentionLease[0]))); + + // when we recover, we should recover the retention leases + final IndexShard recoveredShard = reinitShard( + indexShard, + ShardRoutingHelper.initWithSameId(indexShard.routingEntry(), RecoverySource.ExistingStoreRecoverySource.INSTANCE)); + try { + recoverShardFromStore(recoveredShard); + assertThat( + recoveredShard.getEngine().config().retentionLeasesSupplier().get(), + contains(retentionLeases.toArray(new RetentionLease[0]))); + } finally { + closeShards(recoveredShard); + } + } finally { + closeShards(indexShard); + } + } + private void assertRetentionLeases( final IndexShard indexShard, final int size, diff --git a/test/framework/src/main/java/org/elasticsearch/index/engine/EngineTestCase.java b/test/framework/src/main/java/org/elasticsearch/index/engine/EngineTestCase.java index 58059cd30e382..8b463f33b9081 100644 --- a/test/framework/src/main/java/org/elasticsearch/index/engine/EngineTestCase.java +++ b/test/framework/src/main/java/org/elasticsearch/index/engine/EngineTestCase.java @@ -578,44 +578,112 @@ public EngineConfig config(IndexSettings indexSettings, Store store, Path transl public EngineConfig config(IndexSettings indexSettings, Store store, Path translogPath, MergePolicy mergePolicy, ReferenceManager.RefreshListener refreshListener, Sort indexSort, LongSupplier globalCheckpointSupplier) { - return config(indexSettings, store, translogPath, mergePolicy, refreshListener, null, indexSort, globalCheckpointSupplier, - new NoneCircuitBreakerService()); + return config( + indexSettings, + store, + translogPath, + mergePolicy, + refreshListener, + indexSort, + globalCheckpointSupplier, + globalCheckpointSupplier == null ? null : Collections::emptyList); + } + + public EngineConfig config( + final IndexSettings indexSettings, + final Store store, + final Path translogPath, + final MergePolicy mergePolicy, + final ReferenceManager.RefreshListener refreshListener, + final Sort indexSort, + final LongSupplier globalCheckpointSupplier, + final Supplier> retentionLeasesSupplier) { + return config( + indexSettings, + store, + translogPath, + mergePolicy, + refreshListener, + null, + indexSort, + globalCheckpointSupplier, + retentionLeasesSupplier, + new NoneCircuitBreakerService()); } public EngineConfig config(IndexSettings indexSettings, Store store, Path translogPath, MergePolicy mergePolicy, ReferenceManager.RefreshListener externalRefreshListener, ReferenceManager.RefreshListener internalRefreshListener, - Sort indexSort, @Nullable final LongSupplier maybeGlobalCheckpointSupplier, + Sort indexSort, @Nullable LongSupplier maybeGlobalCheckpointSupplier, CircuitBreakerService breakerService) { - IndexWriterConfig iwc = newIndexWriterConfig(); - TranslogConfig translogConfig = new TranslogConfig(shardId, translogPath, indexSettings, BigArrays.NON_RECYCLING_INSTANCE); - Engine.EventListener listener = new Engine.EventListener() { - @Override - public void onFailedEngine(String reason, @Nullable Exception e) { - // we don't need to notify anybody in this test - } - }; + return config( + indexSettings, + store, + translogPath, + mergePolicy, + externalRefreshListener, + internalRefreshListener, + indexSort, + maybeGlobalCheckpointSupplier, + maybeGlobalCheckpointSupplier == null ? null : Collections::emptyList, + breakerService); + } + + public EngineConfig config( + final IndexSettings indexSettings, + final Store store, + final Path translogPath, + final MergePolicy mergePolicy, + final ReferenceManager.RefreshListener externalRefreshListener, + final ReferenceManager.RefreshListener internalRefreshListener, + final Sort indexSort, + final @Nullable LongSupplier maybeGlobalCheckpointSupplier, + final @Nullable Supplier> maybeRetentionLeasesSupplier, + final CircuitBreakerService breakerService) { + final IndexWriterConfig iwc = newIndexWriterConfig(); + final TranslogConfig translogConfig = new TranslogConfig(shardId, translogPath, indexSettings, BigArrays.NON_RECYCLING_INSTANCE); + final Engine.EventListener listener = new Engine.EventListener() {}; // we don't need to notify anybody in this test final List extRefreshListenerList = - externalRefreshListener == null ? emptyList() : Collections.singletonList(externalRefreshListener); + externalRefreshListener == null ? emptyList() : Collections.singletonList(externalRefreshListener); final List intRefreshListenerList = - internalRefreshListener == null ? emptyList() : Collections.singletonList(internalRefreshListener); + internalRefreshListener == null ? emptyList() : Collections.singletonList(internalRefreshListener); final LongSupplier globalCheckpointSupplier; final Supplier> retentionLeasesSupplier; if (maybeGlobalCheckpointSupplier == null) { + assert maybeRetentionLeasesSupplier == null; final ReplicationTracker replicationTracker = new ReplicationTracker( shardId, allocationId.getId(), indexSettings, SequenceNumbers.NO_OPS_PERFORMED, update -> {}, () -> 0L); globalCheckpointSupplier = replicationTracker; retentionLeasesSupplier = replicationTracker::getRetentionLeases; } else { + assert maybeRetentionLeasesSupplier != null; globalCheckpointSupplier = maybeGlobalCheckpointSupplier; - retentionLeasesSupplier = Collections::emptyList; + retentionLeasesSupplier = maybeRetentionLeasesSupplier; } - EngineConfig config = new EngineConfig(shardId, allocationId.getId(), threadPool, indexSettings, null, store, - mergePolicy, iwc.getAnalyzer(), iwc.getSimilarity(), new CodecService(null, logger), listener, - IndexSearcher.getDefaultQueryCache(), IndexSearcher.getDefaultQueryCachingPolicy(), translogConfig, - TimeValue.timeValueMinutes(5), extRefreshListenerList, intRefreshListenerList, indexSort, - breakerService, globalCheckpointSupplier, retentionLeasesSupplier, primaryTerm::get, tombstoneDocSupplier()); - return config; + return new EngineConfig( + shardId, + allocationId.getId(), + threadPool, + indexSettings, + null, + store, + mergePolicy, + iwc.getAnalyzer(), + iwc.getSimilarity(), + new CodecService(null, logger), + listener, + IndexSearcher.getDefaultQueryCache(), + IndexSearcher.getDefaultQueryCachingPolicy(), + translogConfig, + TimeValue.timeValueMinutes(5), + extRefreshListenerList, + intRefreshListenerList, + indexSort, + breakerService, + globalCheckpointSupplier, + retentionLeasesSupplier, + primaryTerm::get, + tombstoneDocSupplier()); } protected static final BytesReference B_1 = new BytesArray(new byte[]{1}); From a4339ec7e9b6e6cc2ca4511048cae5915ad17b58 Mon Sep 17 00:00:00 2001 From: Costin Leau Date: Sun, 13 Jan 2019 01:40:22 +0200 Subject: [PATCH 130/186] SQL: Use declared source for error messages (#37161) Improve error messages by returning the original SQL statement declaration instead of trying to reproduce it as the casing and whitespaces are not preserved accurately leading to small differences. Close #37161 --- .../sql/qa/multi_node/RestSqlMultinodeIT.java | 2 +- .../xpack/sql/qa/security/UserFunctionIT.java | 12 +- .../sql/qa/single_node/CliExplainIT.java | 12 +- .../xpack/sql/qa/jdbc/DebugCsvSpec.java | 2 +- .../sql/qa/src/main/resources/agg.csv-spec | 6 +- .../sql/qa/src/main/resources/docs.csv-spec | 104 ++++++------- .../qa/src/main/resources/functions.csv-spec | 38 ++--- .../sql/qa/src/main/resources/math.csv-spec | 86 +++++------ .../sql/qa/src/main/resources/select.csv-spec | 142 +++++++++--------- .../xpack/sql/analysis/analyzer/Analyzer.java | 9 +- .../xpack/sql/expression/Literal.java | 2 +- .../sql/expression/function/Function.java | 16 +- .../function/UnresolvedFunction.java | 2 +- .../expression/function/aggregate/Max.java | 6 +- .../expression/function/aggregate/Min.java | 4 +- .../function/aggregate/NumericAggregate.java | 2 +- .../function/aggregate/Percentile.java | 4 +- .../function/aggregate/PercentileRank.java | 4 +- .../sql/expression/function/scalar/Cast.java | 12 -- .../scalar/ConfigurationFunction.java | 6 - .../scalar/datetime/BaseDateTimeFunction.java | 16 +- .../scalar/math/BinaryNumericFunction.java | 4 +- .../expression/function/scalar/math/E.java | 8 +- .../expression/function/scalar/math/Pi.java | 8 +- .../scalar/string/BinaryStringFunction.java | 4 +- .../string/BinaryStringNumericFunction.java | 2 +- .../string/BinaryStringStringFunction.java | 2 +- .../function/scalar/string/Concat.java | 6 +- .../function/scalar/string/Insert.java | 8 +- .../function/scalar/string/Locate.java | 6 +- .../function/scalar/string/Replace.java | 8 +- .../function/scalar/string/Substring.java | 8 +- .../expression/predicate/BinaryPredicate.java | 28 ---- .../xpack/sql/expression/predicate/Range.java | 14 +- .../predicate/logical/BinaryLogic.java | 2 +- .../sql/expression/predicate/logical/Not.java | 4 +- .../predicate/operator/arithmetic/Neg.java | 10 +- .../operator/comparison/BinaryComparison.java | 13 +- .../predicate/operator/comparison/In.java | 8 - .../xpack/sql/parser/AbstractBuilder.java | 10 ++ .../xpack/sql/parser/ExpressionBuilder.java | 65 +++++--- .../analyzer/VerifierErrorMessagesTests.java | 44 +++--- .../function/NamedExpressionTests.java | 41 +++-- .../scalar/datetime/DayOfYearTests.java | 3 +- .../xpack/sql/optimizer/OptimizerTests.java | 2 +- .../sql/parser/EscapedFunctionsTests.java | 36 ++--- .../xpack/sql/parser/ExpressionTests.java | 52 ++++--- .../xpack/sql/parser/SqlParserTests.java | 17 ++- .../xpack/sql/planner/QueryFolderTests.java | 32 ++-- .../sql/planner/QueryTranslatorTests.java | 18 ++- .../sql/type/DataTypeConversionTests.java | 8 +- 51 files changed, 458 insertions(+), 500 deletions(-) diff --git a/x-pack/plugin/sql/qa/multi-node/src/test/java/org/elasticsearch/xpack/sql/qa/multi_node/RestSqlMultinodeIT.java b/x-pack/plugin/sql/qa/multi-node/src/test/java/org/elasticsearch/xpack/sql/qa/multi_node/RestSqlMultinodeIT.java index 95066306cc2c2..16ed3053ab46f 100644 --- a/x-pack/plugin/sql/qa/multi-node/src/test/java/org/elasticsearch/xpack/sql/qa/multi_node/RestSqlMultinodeIT.java +++ b/x-pack/plugin/sql/qa/multi-node/src/test/java/org/elasticsearch/xpack/sql/qa/multi_node/RestSqlMultinodeIT.java @@ -108,7 +108,7 @@ private Map responseToMap(Response response) throws IOException private void assertCount(RestClient client, int count) throws IOException { Map expected = new HashMap<>(); String mode = randomMode(); - expected.put("columns", singletonList(columnInfo(mode, "COUNT(1)", "long", JDBCType.BIGINT, 20))); + expected.put("columns", singletonList(columnInfo(mode, "COUNT(*)", "long", JDBCType.BIGINT, 20))); expected.put("rows", singletonList(singletonList(count))); Request request = new Request("POST", "/_sql"); diff --git a/x-pack/plugin/sql/qa/security/src/test/java/org/elasticsearch/xpack/sql/qa/security/UserFunctionIT.java b/x-pack/plugin/sql/qa/security/src/test/java/org/elasticsearch/xpack/sql/qa/security/UserFunctionIT.java index 639ffd17e34f3..66492bab8f56f 100644 --- a/x-pack/plugin/sql/qa/security/src/test/java/org/elasticsearch/xpack/sql/qa/security/UserFunctionIT.java +++ b/x-pack/plugin/sql/qa/security/src/test/java/org/elasticsearch/xpack/sql/qa/security/UserFunctionIT.java @@ -60,7 +60,7 @@ protected String getProtocol() { @Before private void setUpUsers() throws IOException { int usersCount = name.getMethodName().startsWith("testSingle") ? 1 : randomIntBetween(5, 15); - users = new ArrayList(usersCount); + users = new ArrayList<>(usersCount); users.addAll(randomUnique(() -> randomAlphaOfLengthBetween(1, 15), usersCount)); for (String user : users) { createUser(user, MINIMAL_ACCESS_ROLE); @@ -80,7 +80,7 @@ public void testSingleRandomUser() throws IOException { Map expected = new HashMap<>(); expected.put("columns", Arrays.asList( - columnInfo(mode, "USER", "keyword", JDBCType.VARCHAR, 0))); + columnInfo(mode, "USER()", "keyword", JDBCType.VARCHAR, 0))); expected.put("rows", Arrays.asList(Arrays.asList(randomUserName))); Map actual = runSql(randomUserName, mode, SQL); @@ -96,7 +96,7 @@ public void testSingleRandomUserWithWhereEvaluatingTrue() throws IOException { Map expected = new HashMap<>(); expected.put("columns", Arrays.asList( - columnInfo(mode, "USER", "keyword", JDBCType.VARCHAR, 0))); + columnInfo(mode, "USER()", "keyword", JDBCType.VARCHAR, 0))); expected.put("rows", Arrays.asList(Arrays.asList(randomUserName), Arrays.asList(randomUserName), Arrays.asList(randomUserName))); @@ -114,7 +114,7 @@ public void testSingleRandomUserWithWhereEvaluatingFalse() throws IOException { Map expected = new HashMap<>(); expected.put("columns", Arrays.asList( - columnInfo(mode, "USER", "keyword", JDBCType.VARCHAR, 0))); + columnInfo(mode, "USER()", "keyword", JDBCType.VARCHAR, 0))); expected.put("rows", Collections.>emptyList()); String anotherRandomUserName = randomValueOtherThan(randomUserName, () -> randomAlphaOfLengthBetween(1, 15)); Map actual = runSql(randomUserName, mode, SQL + " FROM test WHERE USER()='" + anotherRandomUserName + "' LIMIT 3"); @@ -129,7 +129,7 @@ public void testMultipleRandomUsersAccess() throws IOException { Map expected = new HashMap<>(); expected.put("columns", Arrays.asList( - columnInfo(mode, "USER", "keyword", JDBCType.VARCHAR, 0))); + columnInfo(mode, "USER()", "keyword", JDBCType.VARCHAR, 0))); expected.put("rows", Arrays.asList(Arrays.asList(randomlyPickedUsername))); Map actual = runSql(randomlyPickedUsername, mode, SQL); @@ -147,7 +147,7 @@ public void testSingleUserSelectFromIndex() throws IOException { Map expected = new HashMap<>(); expected.put("columns", Arrays.asList( - columnInfo(mode, "USER", "keyword", JDBCType.VARCHAR, 0))); + columnInfo(mode, "USER()", "keyword", JDBCType.VARCHAR, 0))); expected.put("rows", Arrays.asList(Arrays.asList(randomUserName), Arrays.asList(randomUserName), Arrays.asList(randomUserName))); diff --git a/x-pack/plugin/sql/qa/single-node/src/test/java/org/elasticsearch/xpack/sql/qa/single_node/CliExplainIT.java b/x-pack/plugin/sql/qa/single-node/src/test/java/org/elasticsearch/xpack/sql/qa/single_node/CliExplainIT.java index c2027ccbfcc6b..58c5ae5c78e91 100644 --- a/x-pack/plugin/sql/qa/single-node/src/test/java/org/elasticsearch/xpack/sql/qa/single_node/CliExplainIT.java +++ b/x-pack/plugin/sql/qa/single-node/src/test/java/org/elasticsearch/xpack/sql/qa/single_node/CliExplainIT.java @@ -64,7 +64,7 @@ public void testExplainWithWhere() throws IOException { assertThat(readLine(), startsWith("----------")); assertThat(readLine(), startsWith("With[{}]")); assertThat(readLine(), startsWith("\\_Project[[?*]]")); - assertThat(readLine(), startsWith(" \\_Filter[?i == 2]")); + assertThat(readLine(), startsWith(" \\_Filter[i = 2#")); assertThat(readLine(), startsWith(" \\_UnresolvedRelation[[][index=test],null,Unknown index [test]]")); assertEquals("", readLine()); @@ -72,14 +72,14 @@ public void testExplainWithWhere() throws IOException { containsString("plan")); assertThat(readLine(), startsWith("----------")); assertThat(readLine(), startsWith("Project[[i{f}#")); - assertThat(readLine(), startsWith("\\_Filter[i{f}#")); + assertThat(readLine(), startsWith("\\_Filter[i = 2#")); assertThat(readLine(), startsWith(" \\_EsRelation[test][i{f}#")); assertEquals("", readLine()); assertThat(command("EXPLAIN (PLAN OPTIMIZED) SELECT * FROM test WHERE i = 2"), containsString("plan")); assertThat(readLine(), startsWith("----------")); assertThat(readLine(), startsWith("Project[[i{f}#")); - assertThat(readLine(), startsWith("\\_Filter[i{f}#")); + assertThat(readLine(), startsWith("\\_Filter[i = 2#")); assertThat(readLine(), startsWith(" \\_EsRelation[test][i{f}#")); assertEquals("", readLine()); @@ -124,20 +124,20 @@ public void testExplainWithCount() throws IOException { assertThat(command("EXPLAIN (PLAN PARSED) SELECT COUNT(*) FROM test"), containsString("plan")); assertThat(readLine(), startsWith("----------")); assertThat(readLine(), startsWith("With[{}]")); - assertThat(readLine(), startsWith("\\_Project[[?COUNT(?*)]]")); + assertThat(readLine(), startsWith("\\_Project[[?COUNT(*)]]")); assertThat(readLine(), startsWith(" \\_UnresolvedRelation[[][index=test],null,Unknown index [test]]")); assertEquals("", readLine()); assertThat(command("EXPLAIN " + (randomBoolean() ? "" : "(PLAN ANALYZED) ") + "SELECT COUNT(*) FROM test"), containsString("plan")); assertThat(readLine(), startsWith("----------")); - assertThat(readLine(), startsWith("Aggregate[[],[COUNT(1)#")); + assertThat(readLine(), startsWith("Aggregate[[],[COUNT(*)#")); assertThat(readLine(), startsWith("\\_EsRelation[test][i{f}#")); assertEquals("", readLine()); assertThat(command("EXPLAIN (PLAN OPTIMIZED) SELECT COUNT(*) FROM test"), containsString("plan")); assertThat(readLine(), startsWith("----------")); - assertThat(readLine(), startsWith("Aggregate[[],[COUNT(1)#")); + assertThat(readLine(), startsWith("Aggregate[[],[COUNT(*)#")); assertThat(readLine(), startsWith("\\_EsRelation[test][i{f}#")); assertEquals("", readLine()); diff --git a/x-pack/plugin/sql/qa/src/main/java/org/elasticsearch/xpack/sql/qa/jdbc/DebugCsvSpec.java b/x-pack/plugin/sql/qa/src/main/java/org/elasticsearch/xpack/sql/qa/jdbc/DebugCsvSpec.java index 8f07fd879b73d..d5a633e5ea388 100644 --- a/x-pack/plugin/sql/qa/src/main/java/org/elasticsearch/xpack/sql/qa/jdbc/DebugCsvSpec.java +++ b/x-pack/plugin/sql/qa/src/main/java/org/elasticsearch/xpack/sql/qa/jdbc/DebugCsvSpec.java @@ -20,7 +20,7 @@ import static org.elasticsearch.xpack.sql.qa.jdbc.CsvTestUtils.executeCsvQuery; import static org.elasticsearch.xpack.sql.qa.jdbc.CsvTestUtils.specParser; -@TestLogging(JdbcTestUtils.SQL_TRACE) +@TestLogging("org.elasticsearch.xpack.sql:TRACE") public abstract class DebugCsvSpec extends SpecBaseIntegrationTestCase { private final CsvTestCase testCase; diff --git a/x-pack/plugin/sql/qa/src/main/resources/agg.csv-spec b/x-pack/plugin/sql/qa/src/main/resources/agg.csv-spec index bdb94321b76d5..3717afefab3bd 100644 --- a/x-pack/plugin/sql/qa/src/main/resources/agg.csv-spec +++ b/x-pack/plugin/sql/qa/src/main/resources/agg.csv-spec @@ -169,7 +169,7 @@ FROM "test_emp" GROUP BY "TEMP" ORDER BY "TEMP" LIMIT 20; aggAndOrderByCastedValue SELECT CHAR_LENGTH(SPACE(CAST(languages AS SMALLINT))), COUNT(*) FROM test_emp GROUP BY 1 ORDER BY 1 DESC; -CHAR_LENGTH(SPACE(CAST(languages AS SMALLINT))):i| COUNT(1):l +CHAR_LENGTH(SPACE(CAST(languages AS SMALLINT))):i| COUNT(*):l -------------------------------------------------+--------------- 5 |21 4 |18 @@ -180,9 +180,9 @@ null |10 ; aggAndOrderByCastedFunctionValue -SELECT ROUND(SQRT(CAST(EXP(languages) AS SMALLINT)), 2), COUNT(*) FROM test_emp GROUP BY 1 ORDER BY 1 DESC; +SELECT ROUND(SQRT(CAST(EXP(languages) AS SMALLINT)),2), COUNT(*) FROM test_emp GROUP BY 1 ORDER BY 1 DESC; -ROUND(SQRT(CAST(EXP(languages) AS SMALLINT)),2):d| COUNT(1):l +ROUND(SQRT(CAST(EXP(languages) AS SMALLINT)),2):d| COUNT(*):l -------------------------------------------------+--------------- 12.17 |21 7.42 |18 diff --git a/x-pack/plugin/sql/qa/src/main/resources/docs.csv-spec b/x-pack/plugin/sql/qa/src/main/resources/docs.csv-spec index 2903292b1adff..e6bde4795a85f 100644 --- a/x-pack/plugin/sql/qa/src/main/resources/docs.csv-spec +++ b/x-pack/plugin/sql/qa/src/main/resources/docs.csv-spec @@ -1124,8 +1124,8 @@ stringAscii // tag::stringAscii SELECT ASCII('Elastic'); -ASCII(Elastic) ---------------- +ASCII('Elastic') +---------------- 69 // end::stringAscii ; @@ -1134,8 +1134,8 @@ stringBitLength // tag::stringBitLength SELECT BIT_LENGTH('Elastic'); -BIT_LENGTH(Elastic) -------------------- +BIT_LENGTH('Elastic') +--------------------- 56 // end::stringBitLength ; @@ -1154,8 +1154,8 @@ stringCharLength // tag::stringCharLength SELECT CHAR_LENGTH('Elastic'); -CHAR_LENGTH(Elastic) --------------------- +CHAR_LENGTH('Elastic') +---------------------- 7 // end::stringCharLength ; @@ -1164,8 +1164,8 @@ stringConcat // tag::stringConcat SELECT CONCAT('Elasticsearch', ' SQL'); -CONCAT(Elasticsearch, SQL) --------------------------- +CONCAT('Elasticsearch', ' SQL') +------------------------------- Elasticsearch SQL // end::stringConcat ; @@ -1174,8 +1174,8 @@ stringInsert // tag::stringInsert SELECT INSERT('Elastic ', 8, 1, 'search'); -INSERT(Elastic ,8,1,search) ---------------------------- +INSERT('Elastic ', 8, 1, 'search') +---------------------------------- Elasticsearch // end::stringInsert ; @@ -1184,8 +1184,8 @@ stringLCase // tag::stringLCase SELECT LCASE('Elastic'); -LCASE(Elastic) ---------------- +LCASE('Elastic') +---------------- elastic // end::stringLCase ; @@ -1194,8 +1194,8 @@ stringLeft // tag::stringLeft SELECT LEFT('Elastic',3); -LEFT(Elastic,3) ---------------- +LEFT('Elastic',3) +----------------- Ela // end::stringLeft ; @@ -1204,8 +1204,8 @@ stringLength // tag::stringLength SELECT LENGTH('Elastic '); -LENGTH(Elastic ) ------------------- +LENGTH('Elastic ') +-------------------- 7 // end::stringLength ; @@ -1214,8 +1214,8 @@ stringLocateWoStart // tag::stringLocateWoStart SELECT LOCATE('a', 'Elasticsearch'); -LOCATE(a,Elasticsearch) ------------------------ +LOCATE('a', 'Elasticsearch') +---------------------------- 3 // end::stringLocateWoStart ; @@ -1224,8 +1224,8 @@ stringLocateWithStart // tag::stringLocateWithStart SELECT LOCATE('a', 'Elasticsearch', 5); -LOCATE(a,Elasticsearch,5) -------------------------- +LOCATE('a', 'Elasticsearch', 5) +------------------------------- 10 // end::stringLocateWithStart ; @@ -1234,8 +1234,8 @@ stringLTrim // tag::stringLTrim SELECT LTRIM(' Elastic'); -LTRIM( Elastic) ------------------ +LTRIM(' Elastic') +------------------- Elastic // end::stringLTrim ; @@ -1244,8 +1244,8 @@ stringOctetLength // tag::stringOctetLength SELECT OCTET_LENGTH('Elastic'); -OCTET_LENGTH(Elastic) -------------------- +OCTET_LENGTH('Elastic') +----------------------- 7 // end::stringOctetLength ; @@ -1254,8 +1254,8 @@ stringPosition // tag::stringPosition SELECT POSITION('Elastic', 'Elasticsearch'); -POSITION(Elastic,Elasticsearch) -------------------------------- +POSITION('Elastic', 'Elasticsearch') +------------------------------------ 1 // end::stringPosition ; @@ -1264,8 +1264,8 @@ stringRepeat // tag::stringRepeat SELECT REPEAT('La', 3); - REPEAT(La,3) ---------------- + REPEAT('La', 3) +---------------- LaLaLa // end::stringRepeat ; @@ -1274,8 +1274,8 @@ stringReplace // tag::stringReplace SELECT REPLACE('Elastic','El','Fant'); - REPLACE(Elastic,El,Fant) ------------------------------ +REPLACE('Elastic','El','Fant') +------------------------------ Fantastic // end::stringReplace ; @@ -1284,8 +1284,8 @@ stringRight // tag::stringRight SELECT RIGHT('Elastic',3); -RIGHT(Elastic,3) ----------------- +RIGHT('Elastic',3) +------------------ tic // end::stringRight ; @@ -1294,8 +1294,8 @@ stringRTrim // tag::stringRTrim SELECT RTRIM('Elastic '); -RTRIM(Elastic ) ------------------ +RTRIM('Elastic ') +------------------- Elastic // end::stringRTrim ; @@ -1316,8 +1316,8 @@ stringSubString // tag::stringSubString SELECT SUBSTRING('Elasticsearch', 0, 7); -SUBSTRING(Elasticsearch,0,7) ----------------------------- +SUBSTRING('Elasticsearch', 0, 7) +-------------------------------- Elastic // end::stringSubString ; @@ -1326,8 +1326,8 @@ stringUCase // tag::stringUCase SELECT UCASE('Elastic'); -UCASE(Elastic) ---------------- +UCASE('Elastic') +---------------- ELASTIC // end::stringUCase ; @@ -1417,7 +1417,7 @@ mathInlineAcos // tag::mathInlineAcos SELECT ACOS(COS(PI())), PI(); - ACOS(COS(PI)) | PI + ACOS(COS(PI())) | PI() -----------------+----------------- 3.141592653589793|3.141592653589793 // end::mathInlineAcos @@ -1467,8 +1467,8 @@ mathInlineCeiling // tag::mathInlineCeiling SELECT CEIL(125.01), CEILING(-125.99); - CEIL(125.01) | CEIL(-125.99) ----------------+--------------- + CEIL(125.01) |CEILING(-125.99) +---------------+---------------- 126 |-125 // end::mathInlineCeiling ; @@ -1507,7 +1507,7 @@ mathInlineDegrees // tag::mathInlineDegrees SELECT DEGREES(PI() * 2), DEGREES(PI()); -DEGREES((PI) * 2)| DEGREES(PI) +DEGREES(PI() * 2)| DEGREES(PI()) -----------------+--------------- 360.0 |180.0 // end::mathInlineDegrees @@ -1517,7 +1517,7 @@ mathEulersNumber // tag::mathEulersNumber SELECT E(), CEIL(E()); - E | CEIL(E) + E() | CEIL(E()) -----------------+--------------- 2.718281828459045|3 // end::mathEulersNumber @@ -1527,7 +1527,7 @@ mathExpInline // tag::mathExpInline SELECT EXP(1), E(), EXP(2), E() * E(); - EXP(1) | E | EXP(2) | (E) * (E) + EXP(1) | E() | EXP(2) | E() * E() -----------------+-----------------+----------------+------------------ 2.718281828459045|2.718281828459045|7.38905609893065|7.3890560989306495 // end::mathExpInline @@ -1537,7 +1537,7 @@ mathExpm1Inline // tag::mathExpm1Inline SELECT E(), EXP(2), EXPM1(2); - E | EXP(2) | EXPM1(2) + E() | EXP(2) | EXPM1(2) -----------------+----------------+---------------- 2.718281828459045|7.38905609893065|6.38905609893065 // end::mathExpm1Inline @@ -1567,8 +1567,8 @@ mathInlineLog10 // tag::mathInlineLog10 SELECT LOG10(5), LOG(5)/LOG(10); - LOG10(5) |(LOG(5)) / (LOG(10)) -------------------+-------------------- + LOG10(5) | LOG(5)/LOG(10) +------------------+----------------------- 0.6989700043360189|0.6989700043360187 // end::mathInlineLog10 ; @@ -1577,7 +1577,7 @@ mathPINumber // tag::mathPINumber SELECT PI(); - PI + PI() ----------------- 3.141592653589793 // end::mathPINumber @@ -1587,7 +1587,7 @@ mathInlinePowerPositive // tag::mathInlinePowerPositive SELECT POWER(3, 2), POWER(3, 3); - POWER(3,2) | POWER(3,3) + POWER(3, 2) | POWER(3, 3) ---------------+--------------- 9.0 |27.0 // end::mathInlinePowerPositive @@ -1597,7 +1597,7 @@ mathInlinePowerNegative // tag::mathInlinePowerNegative SELECT POWER(5, -1), POWER(5, -2); - POWER(5,-1) | POWER(5,-2) + POWER(5, -1) | POWER(5, -2) ---------------+--------------- 0.2 |0.04 // end::mathInlinePowerNegative @@ -1607,7 +1607,7 @@ mathInlineRadians // tag::mathInlineRadians SELECT RADIANS(90), PI()/2; - RADIANS(90) | (PI) / 2 + RADIANS(90) | PI()/2 ------------------+------------------ 1.5707963267948966|1.5707963267948966 // end::mathInlineRadians @@ -1677,7 +1677,7 @@ mathInlineSqrt // tag::mathInlineSqrt SELECT SQRT(EXP(2)), E(), SQRT(25); - SQRT(EXP(2)) | E | SQRT(25) + SQRT(EXP(2)) | E() | SQRT(25) -----------------+-----------------+--------------- 2.718281828459045|2.718281828459045|5.0 // end::mathInlineSqrt diff --git a/x-pack/plugin/sql/qa/src/main/resources/functions.csv-spec b/x-pack/plugin/sql/qa/src/main/resources/functions.csv-spec index 6fec225df0c50..a89013cb32bca 100644 --- a/x-pack/plugin/sql/qa/src/main/resources/functions.csv-spec +++ b/x-pack/plugin/sql/qa/src/main/resources/functions.csv-spec @@ -263,18 +263,18 @@ SELECT POSITION('x',LCASE("first_name")) pos, "first_name" FROM "test_emp" WHERE ; selectPositionWithLcaseAndConditionWithGroupByAndOrderBy -SELECT POSITION('m',LCASE("first_name")), COUNT(*) pos FROM "test_emp" +SELECT POSITION('m',LCASE("first_name")) AS pos, COUNT(*) c FROM "test_emp" WHERE POSITION('m',LCASE("first_name")) != 0 GROUP BY POSITION('m',LCASE("first_name")) ORDER BY POSITION('m',LCASE("first_name")) DESC; -POSITION(m,LCASE(first_name)):i| pos:l --------------------------------+--------------- -9 |1 -7 |1 -4 |2 -3 |6 -2 |1 -1 |9 +pos:i| c:l +-----+----- +9 |1 +7 |1 +4 |2 +3 |6 +2 |1 +1 |9 ; selectInsertWithPositionAndCondition @@ -299,8 +299,8 @@ ReMzi |3 selectLocateAndInsertWithLocateWithConditionAndThreeParameters SELECT LOCATE('a',"first_name",7) pos, INSERT("first_name",LOCATE('a',"first_name",7),1,'AAA') FROM "test_emp" WHERE LOCATE('a',"first_name",7) > 0; - pos:i |INSERT(first_name,LOCATE(a,first_name,7),1,AAA):s ----------------+----------------------------------------------- + pos:i |INSERT("first_name",LOCATE('a',"first_name",7),1,'AAA'):s +---------------+--------------------------------------------------------- 8 |ChirstiAAAn 7 |DuangkAAAew 8 |PrasadrAAAm @@ -317,8 +317,8 @@ SELECT LOCATE('a',"first_name",7) pos, INSERT("first_name",LOCATE('a',"first_nam selectLocateAndInsertWithLocateWithConditionAndTwoParameters SELECT LOCATE('a',"first_name") pos, INSERT("first_name",LOCATE('a',"first_name"),1,'AAA') FROM "test_emp" WHERE LOCATE('a',"first_name") > 0 ORDER BY "first_name" LIMIT 10; - pos:i |INSERT(first_name,LOCATE(a,first_name),1,AAA):s ----------------+--------------------------------------------- + pos:i |INSERT("first_name",LOCATE('a',"first_name"),1,'AAA'):s +---------------+------------------------------------------------------- 5 |AlejAAAndro 3 |AmAAAbile 7 |ArumugAAAm @@ -455,23 +455,23 @@ AlejandRo |1 checkColumnNameWithNestedArithmeticFunctionCallsOnTableColumn -SELECT CHAR(emp_no % 10000) FROM "test_emp" WHERE emp_no > 10064 ORDER BY emp_no LIMIT 1; +SELECT CHAR(emp_no % 10000) AS c FROM "test_emp" WHERE emp_no > 10064 ORDER BY emp_no LIMIT 1; -CHAR((emp_no) % 10000):s +c:s A ; checkColumnNameWithComplexNestedArithmeticFunctionCallsOnTableColumn1 -SELECT CHAR(emp_no % (7000 + 3000)) FROM "test_emp" WHERE emp_no > 10065 ORDER BY emp_no LIMIT 1; +SELECT CHAR(emp_no % (7000 + 3000)) AS c FROM "test_emp" WHERE emp_no > 10065 ORDER BY emp_no LIMIT 1; -CHAR((emp_no) % (7000 + 3000)):s +c:s B ; checkColumnNameWithComplexNestedArithmeticFunctionCallsOnTableColumn2 -SELECT CHAR((emp_no % (emp_no - 1 + 1)) + 67) FROM "test_emp" WHERE emp_no > 10066 ORDER BY emp_no LIMIT 1; +SELECT CHAR((emp_no % (emp_no - 1 + 1)) + 67) AS c FROM "test_emp" WHERE emp_no > 10066 ORDER BY emp_no LIMIT 1; -CHAR(((emp_no) % (((emp_no) - 1) + 1)) + 67):s +c:s C ; diff --git a/x-pack/plugin/sql/qa/src/main/resources/math.csv-spec b/x-pack/plugin/sql/qa/src/main/resources/math.csv-spec index 2df93b3795443..19eda7b92f9c0 100644 --- a/x-pack/plugin/sql/qa/src/main/resources/math.csv-spec +++ b/x-pack/plugin/sql/qa/src/main/resources/math.csv-spec @@ -2,27 +2,27 @@ truncateWithAsciiHavingAndOrderBy SELECT TRUNCATE(ASCII(LEFT(first_name, 1)), 1), COUNT(*) count FROM test_emp GROUP BY ASCII(LEFT(first_name, 1)) HAVING COUNT(*) > 5 ORDER BY TRUNCATE(ASCII(LEFT(first_name, 1)), 1) DESC; -TRUNCATE(ASCII(LEFT(first_name,1)),1):i| count:l ----------------------------------------+--------------- -null |10 -66 |7 -72 |6 -75 |7 -77 |9 -83 |11 +TRUNCATE(ASCII(LEFT(first_name, 1)), 1):i| count:l +-----------------------------------------+--------------- +null |10 +66 |7 +72 |6 +75 |7 +77 |9 +83 |11 ; truncateWithNoSecondParameterWithAsciiHavingAndOrderBy SELECT TRUNCATE(ASCII(LEFT(first_name, 1))), COUNT(*) count FROM test_emp GROUP BY ASCII(LEFT(first_name, 1)) HAVING COUNT(*) > 5 ORDER BY TRUNCATE(ASCII(LEFT(first_name, 1))) DESC; -TRUNCATE(ASCII(LEFT(first_name,1)),0):i| count:l ----------------------------------------+--------------- -null |10 -66 |7 -72 |6 -75 |7 -77 |9 -83 |11 +TRUNCATE(ASCII(LEFT(first_name, 1))):i| count:l +--------------------------------------+--------------- +null |10 +66 |7 +72 |6 +75 |7 +77 |9 +83 |11 ; roundWithGroupByAndOrderBy @@ -97,21 +97,21 @@ SELECT MIN(salary) mi, MAX(salary) ma, COUNT(*) c, TRUNCATE(AVG(salary)) tr FROM minMaxTruncateAndRoundOfAverageWithHavingRoundAndTruncate SELECT MIN(salary) mi, MAX(salary) ma, YEAR(hire_date) year, ROUND(AVG(languages), 1), TRUNCATE(AVG(languages), 1), COUNT(*) FROM test_emp GROUP BY YEAR(hire_date) HAVING ROUND(AVG(languages), 1) > 2.5 AND TRUNCATE(AVG(languages), 1) <= 3.0 ORDER BY YEAR(hire_date); - mi:i | ma:i | year:i |ROUND(AVG(languages),1):d|TRUNCATE(AVG(languages),1):d| COUNT(1):l ----------------+---------------+---------------+-------------------------+----------------------------+--------------- -25324 |70011 |1986 |3.0 |3.0 |15 -25945 |73578 |1987 |2.9 |2.8 |9 -25976 |74970 |1988 |3.0 |3.0 |13 -31120 |71165 |1989 |3.1 |3.0 |12 -30404 |58715 |1992 |3.0 |3.0 |3 -35742 |67492 |1993 |2.8 |2.7 |4 -45656 |45656 |1995 |3.0 |3.0 |1 + mi:i | ma:i | year:i |ROUND(AVG(languages), 1):d|TRUNCATE(AVG(languages), 1):d| COUNT(*):l +---------------+---------------+---------------+--------------------------+-----------------------------+--------------- +25324 |70011 |1986 |3.0 |3.0 |15 +25945 |73578 |1987 |2.9 |2.8 |9 +25976 |74970 |1988 |3.0 |3.0 |13 +31120 |71165 |1989 |3.1 |3.0 |12 +30404 |58715 |1992 |3.0 |3.0 |3 +35742 |67492 |1993 |2.8 |2.7 |4 +45656 |45656 |1995 |3.0 |3.0 |1 ; minMaxRoundWithHavingRound -SELECT MIN(salary) mi, MAX(salary) ma, YEAR(hire_date) year, ROUND(AVG(languages), 1), COUNT(*) FROM test_emp GROUP BY YEAR(hire_date) HAVING ROUND(AVG(languages), 1) > 2.5 ORDER BY YEAR(hire_date); +SELECT MIN(salary) mi, MAX(salary) ma, YEAR(hire_date) year, ROUND(AVG(languages),1), COUNT(*) FROM test_emp GROUP BY YEAR(hire_date) HAVING ROUND(AVG(languages), 1) > 2.5 ORDER BY YEAR(hire_date); - mi:i | ma:i | year:i |ROUND(AVG(languages),1):d| COUNT(1):l + mi:i | ma:i | year:i |ROUND(AVG(languages),1):d| COUNT(*):l ---------------+---------------+---------------+-------------------------+--------------- 26436 |74999 |1984 |3.1 |11 31897 |61805 |1985 |3.5 |11 @@ -127,9 +127,9 @@ SELECT MIN(salary) mi, MAX(salary) ma, YEAR(hire_date) year, ROUND(AVG(languages ; groupByAndOrderByTruncateWithPositiveParameter -SELECT TRUNCATE(AVG(salary), 2), AVG(salary), COUNT(*) FROM test_emp GROUP BY TRUNCATE(salary, 2) ORDER BY TRUNCATE(salary, 2) DESC LIMIT 10; +SELECT TRUNCATE(AVG(salary),2), AVG(salary), COUNT(*) FROM test_emp GROUP BY TRUNCATE(salary, 2) ORDER BY TRUNCATE(salary, 2) DESC LIMIT 10; -TRUNCATE(AVG(salary),2):d| AVG(salary):d | COUNT(1):l +TRUNCATE(AVG(salary),2):d| AVG(salary):d | COUNT(*):l -------------------------+---------------+--------------- 74999.0 |74999.0 |1 74970.0 |74970.0 |1 @@ -144,9 +144,9 @@ TRUNCATE(AVG(salary),2):d| AVG(salary):d | COUNT(1):l ; groupByAndOrderByRoundWithPositiveParameter -SELECT ROUND(AVG(salary), 2), AVG(salary), COUNT(*) FROM test_emp GROUP BY ROUND(salary, 2) ORDER BY ROUND(salary, 2) DESC LIMIT 10; +SELECT ROUND(AVG(salary),2), AVG(salary), COUNT(*) FROM test_emp GROUP BY ROUND(salary, 2) ORDER BY ROUND(salary, 2) DESC LIMIT 10; -ROUND(AVG(salary),2):d| AVG(salary):d | COUNT(1):l +ROUND(AVG(salary),2):d| AVG(salary):d | COUNT(*):l ----------------------+---------------+--------------- 74999.0 |74999.0 |1 74970.0 |74970.0 |1 @@ -163,16 +163,16 @@ ROUND(AVG(salary),2):d| AVG(salary):d | COUNT(1):l groupByAndOrderByRoundWithNoSecondParameter SELECT ROUND(AVG(salary)), ROUND(salary) rounded, AVG(salary), COUNT(*) FROM test_emp GROUP BY rounded ORDER BY rounded DESC LIMIT 10; -ROUND(AVG(salary),0):d| rounded:i | AVG(salary):d | COUNT(1):l -----------------------+---------------+---------------+--------------- -74999.0 |74999 |74999.0 |1 -74970.0 |74970 |74970.0 |1 -74572.0 |74572 |74572.0 |1 -73851.0 |73851 |73851.0 |1 -73717.0 |73717 |73717.0 |1 -73578.0 |73578 |73578.0 |1 -71165.0 |71165 |71165.0 |1 -70011.0 |70011 |70011.0 |1 -69904.0 |69904 |69904.0 |1 -68547.0 |68547 |68547.0 |1 +ROUND(AVG(salary)):d| rounded:i | AVG(salary):d | COUNT(*):l +--------------------+---------------+---------------+--------------- +74999.0 |74999 |74999.0 |1 +74970.0 |74970 |74970.0 |1 +74572.0 |74572 |74572.0 |1 +73851.0 |73851 |73851.0 |1 +73717.0 |73717 |73717.0 |1 +73578.0 |73578 |73578.0 |1 +71165.0 |71165 |71165.0 |1 +70011.0 |70011 |70011.0 |1 +69904.0 |69904 |69904.0 |1 +68547.0 |68547 |68547.0 |1 ; diff --git a/x-pack/plugin/sql/qa/src/main/resources/select.csv-spec b/x-pack/plugin/sql/qa/src/main/resources/select.csv-spec index 2aa7d9bdc7b51..684a451e33173 100644 --- a/x-pack/plugin/sql/qa/src/main/resources/select.csv-spec +++ b/x-pack/plugin/sql/qa/src/main/resources/select.csv-spec @@ -5,39 +5,39 @@ equalsSelectClause SELECT CAST(4 = 4 AS STRING), CAST(NOT 4 = 4 AS STRING), CAST(3 = 4 AS STRING), CAST(NOT 3 = 4 AS STRING), CAST(1 = null AS STRING), CAST(NOT null = 1 AS STRING); - CAST(4 == 4 AS VARCHAR):s | CAST(NOT(4 == 4) AS VARCHAR):s | CAST(3 == 4 AS VARCHAR):s | CAST(NOT(3 == 4) AS VARCHAR):s | CAST(1 == null AS VARCHAR):s | CAST(NOT(null == 1) AS VARCHAR):s -----------------------------+---------------------------------+----------------------------+---------------------------------+-------------------------------+----------------------------------- -true |false |false |true |null |null +CAST(4 = 4 AS STRING):s|CAST(NOT 4 = 4 AS STRING):s|CAST(3 = 4 AS STRING):s|CAST(NOT 3 = 4 AS STRING):s|CAST(1 = null AS STRING):s|CAST(NOT null = 1 AS STRING):s +-----------------------+---------------------------+-----------------------+---------------------------+--------------------------+---------------------------- +true |false |false |true |null |null ; notEqualsSelectClause SELECT CAST(4 != 4 AS STRING), CAST(NOT 4 != 4 AS STRING), CAST(3 != 4 AS STRING), CAST(NOT 3 != 4 AS STRING), CAST(1 != null AS STRING), CAST(NOT 1 != null AS STRING); - CAST(4 != 4 AS VARCHAR):s | CAST(NOT(4 != 4) AS VARCHAR):s | CAST(3 != 4 AS VARCHAR):s | CAST(NOT(3 != 4) AS VARCHAR):s | CAST(1 != null AS VARCHAR):s | CAST(NOT(1 != null) AS VARCHAR):s -----------------------------+---------------------------------+----------------------------+---------------------------------+-------------------------------+----------------------------------- -false |true |true |false |null |null +CAST(4 != 4 AS STRING):s|CAST(NOT 4 != 4 AS STRING):s|CAST(3 != 4 AS STRING):s|CAST(NOT 3 != 4 AS STRING):s|CAST(1 != null AS STRING):s|CAST(NOT 1 != null AS STRING):s +------------------------+----------------------------+------------------------+----------------------------+---------------------------+------------------------------- +false |true |true |false |null |null ; equalSelectClauseWithTableColumns SELECT CAST(languages = 2 AS STRING), CAST(NOT languages = 2 AS STRING), CAST(languages = null AS STRING), CAST(NOT languages = null AS STRING) FROM "test_emp" WHERE emp_no IN(10018, 10019, 10020) ORDER BY emp_no; - CAST((languages) == 2 AS VARCHAR):s | CAST(NOT((languages) == 2) AS VARCHAR):s | CAST((languages) == null AS VARCHAR):s | CAST(NOT((languages) == null) AS VARCHAR):s ---------------------------------------+-------------------------------------------+-----------------------------------------+--------------------------------------------- -true |false |null |null -false |true |null |null -null |null |null |null +CAST(languages = 2 AS STRING):s|CAST(NOT languages = 2 AS STRING):s|CAST(languages = null AS STRING):s|CAST(NOT languages = null AS STRING):s +-------------------------------+-----------------------------------+----------------------------------+-------------------------------------- +true |false |null |null +false |true |null |null +null |null |null |null ; notEqualsAndNotEqualsSelectClauseWithTableColumns SELECT CAST(languages != 2 AS STRING), CAST(NOT languages != 2 AS STRING), CAST(languages != null AS STRING), CAST(NOT languages != null AS STRING) FROM "test_emp" WHERE emp_no IN(10018, 10019, 10020) ORDER BY emp_no; - CAST((languages) != 2 AS VARCHAR):s | CAST(NOT((languages) != 2) AS VARCHAR):s | CAST((languages) != null AS VARCHAR):s | CAST(NOT((languages) != null) AS VARCHAR):s ---------------------------------------+-------------------------------------------+-----------------------------------------+--------------------------------------------- -false |true |null |null -true |false |null |null -null |null |null |null +CAST(languages != 2 AS STRING):s|CAST(NOT languages != 2 AS STRING):s|CAST(languages != null AS STRING):s|CAST(NOT languages != null AS STRING):s +--------------------------------+------------------------------------+-----------------------------------+--------------------------------------- +false |true |null |null +true |false |null |null +null |null |null |null ; @@ -48,27 +48,27 @@ null |null selectWithOrAndNullHandling SELECT CAST(true OR null AS STRING), CAST(null OR true AS STRING), CAST(false OR null AS STRING), CAST(null OR false AS STRING), CAST(null OR null AS STRING); - CAST(true OR null AS VARCHAR):s | CAST(null OR true AS VARCHAR):s | CAST(false OR null AS VARCHAR):s | CAST(null OR false AS VARCHAR):s | CAST(null OR null AS VARCHAR):s -----------------------------------+----------------------------------+-----------------------------------+-----------------------------------+--------------------------------- -true |true |null |null |null +CAST(true OR null AS STRING):s|CAST(null OR true AS STRING):s|CAST(false OR null AS STRING):s|CAST(null OR false AS STRING):s|CAST(null OR null AS STRING):s +------------------------------+------------------------------+-------------------------------+-------------------------------+---------------------------- +true |true |null |null |null ; selectWithAndAndNullHandling SELECT CAST(true AND null AS STRING), CAST(null AND true AS STRING), CAST(false AND null AS STRING), CAST(null AND false AS STRING), CAST(null AND null AS STRING); - CAST(true AND null AS VARCHAR):s | CAST(null AND true AS VARCHAR):s | CAST(false AND null AS VARCHAR):s | CAST(null AND false AS VARCHAR):s | CAST(null AND null AS VARCHAR):s ------------------------------------+-----------------------------------+------------------------------------+------------------------------------+---------------------------------- -null |null |false |false |null +CAST(true AND null AS STRING):s|CAST(null AND true AS STRING):s|CAST(false AND null AS STRING):s|CAST(null AND false AS STRING):s|CAST(null AND null AS STRING):s +-------------------------------+-------------------------------+--------------------------------+--------------------------------+----------------------------- +null |null |false |false |null ; selectWithOrAndAndAndNullHandling_WithTableColumns SELECT CAST(languages = 2 OR null AS STRING), CAST(languages = 2 AND null AS STRING) FROM test_emp WHERE emp_no BETWEEN 10018 AND 10020 ORDER BY emp_no; - CAST(((languages) == 2) OR null AS VARCHAR):s | CAST(((languages) == 2) AND null AS VARCHAR):s ------------------------------------------------+------------------------------------------------ -true |null -null |false -null |null +CAST(languages = 2 OR null AS STRING):s|CAST(languages = 2 AND null AS STRING):s +---------------------------------------+---------------------------------------- +true |null +null |false +null |null ; @@ -76,45 +76,45 @@ null |null // SELECT with IN // inWithLiterals -SELECT 1 IN (1, 2, 3), 1 IN (2, 3); +SELECT 1 IN (1,2, 3), 1 IN (2, 3); - 1 IN (1, 2, 3):b | 1 IN (2, 3):b --------------------+------------- -true |false +1 IN (1,2, 3) | 1 IN (2, 3) +---------------+--------------- +true |false ; inWithLiteralsAndFunctions SELECT 1 IN (2 - 1, 2, 3), abs(-1) IN (2, 3, abs(4 - 5)); - 1 IN (2 - 1, 2, 3) | ABS(-1) IN (2, 3, ABS(4 - 5)) ----------------------+------------------------------ -true |true +1 IN (2 - 1, 2, 3)|abs(-1) IN (2, 3, abs(4 - 5)) +------------------+----------------------------- +true |true ; inWithLiteralsAndNegation SELECT 1 NOT IN (1, 1 + 1, 3), 1 NOT IN (2, 3); - NOT(1 IN (1, 1 + 1, 3)) | NOT(1 IN (2, 3)) ---------------------------+----------------- -false |true +1 NOT IN (1, 1 + 1, 3)|1 NOT IN (2, 3) +----------------------+--------------- +false |true ; // Need to CAST as STRING since for boolean types the jdbc CSV translates null -> false inWithNullHandling SELECT CAST(2 IN (1, null, 3) AS STRING), CAST(3 IN (1, null, 3) AS STRING), CAST(null IN (1, null, 3) AS STRING), CAST(null IN (1, 2, 3) AS STRING); - CAST(2 IN (1, null, 3) AS VARCHAR):s | CAST(3 IN (1, null, 3) AS VARCHAR):s | CAST(null IN (1, null, 3) AS VARCHAR):s | CAST(null IN (1, 2, 3) AS VARCHAR):s ----------------------------------------+--------------------------------------+------------------------------------------+-------------------------------------- -null |true |null |null +CAST(2 IN (1, null, 3) AS STRING):s|CAST(3 IN (1, null, 3) AS STRING):s|CAST(null IN (1, null, 3) AS STRING):s|CAST(null IN (1, 2, 3) AS STRING):s +-----------------------------------+-----------------------------------+--------------------------------------+----------------------------------- +null |true |null |null ; inWithNullHandlingAndNegation SELECT CAST(NOT 2 IN (1, null, 3) AS STRING), CAST(3 NOT IN (1, null, 3) AS STRING), CAST(NOT null IN (1, null, 3) AS STRING), CAST(null NOT IN (1, 2, 3) AS STRING); - CAST(NOT(2 IN (1, null, 3)) AS VARCHAR):s | CAST(NOT(3 IN (1, null, 3)) AS VARCHAR):s | CAST(NOT(null IN (1, null, 3)) AS VARCHAR):s | CAST(NOT(null IN (1, 2, 3)) AS VARCHAR):s ---------------------------------------------+--------------------------------------------+-----------------------------------------------+------------------------------------------- -null |false |null |null +CAST(NOT 2 IN (1, null, 3) AS STRING):s|CAST(3 NOT IN (1, null, 3) AS STRING):s|CAST(NOT null IN (1, null, 3) AS STRING):s|CAST(null NOT IN (1, 2, 3) AS STRING):s +---------------------------------------+---------------------------------------+------------------------------------------+--------------------------------------- +null |false |null |null ; // @@ -134,34 +134,34 @@ false inWithTableColumnAndFunction SELECT emp_no IN (10000, 10000 + 1, abs(-10000 - 2)) FROM test_emp WHERE emp_no BETWEEN 10001 AND 10004 ORDER BY emp_no; - emp_no IN (10000, 10000 + 1, ABS(-10000 - 2)):b ------------------------------------------------- -true -true -false -false +emp_no IN (10000, 10000 + 1, abs(-10000 - 2)) +--------------------------------------------- +true +true +false +false ; inWithTableColumnAndNegation SELECT emp_no NOT IN (10000, 10000 + 1, 10002) FROM test_emp WHERE emp_no BETWEEN 10001 AND 10004 ORDER BY emp_no; - NOT(emp_no IN (10000, 10000 + 1, 10002)):b -------------------------------------------- -false -false -true -true +emp_no NOT IN (10000, 10000 + 1, 10002) +--------------------------------------- +false +false +true +true ; inWithTableColumnAndComplexFunctions SELECT emp_no IN (1, abs(1 - 10002), 3) OR emp_no NOT IN (10000, 10000 + 2, 10003) FROM test_emp WHERE emp_no BETWEEN 10001 AND 10004 ORDER BY emp_no; -(emp_no IN (1, ABS(1 - 10002), 3)) OR (NOT(emp_no IN (10000, 10000 + 2, 10003))):b ----------------------------------------------------------------------------------- -true -false -false -true +emp_no IN (1, abs(1 - 10002), 3) OR emp_no NOT IN (10000, 10000 + 2, 10003) +--------------------------------------------------------------------------- +true +false +false +true ; @@ -169,19 +169,19 @@ true inWithTableColumnAndNullHandling SELECT emp_no, CAST(languages IN (2, 3) AS STRING), CAST(languages IN (2, null, 3) AS STRING) FROM test_emp WHERE emp_no BETWEEN 10018 AND 10020 ORDER BY emp_no; - emp_no:i | CAST(languages IN (2, 3) AS VARCHAR):s | CAST(languages IN (2, null, 3) AS VARCHAR):s -----------+-----------------------------------------+---------------------------------------------- -10018 |true |true -10019 |false |null -10020 |null |null + emp_no:i |CAST(languages IN (2, 3) AS STRING):s|CAST(languages IN (2, null, 3) AS STRING):s +---------------+-------------------------------------+------------------------------------------- +10018 |true |true +10019 |false |null +10020 |null |null ; inWithTableColumnAndNullHandlingAndNegation SELECT emp_no, CAST(languages NOT IN (2, 3) AS STRING), CAST(NOT languages IN (2, null, 3) AS STRING) FROM test_emp WHERE emp_no BETWEEN 10018 AND 10020 ORDER BY emp_no; - emp_no:i | CAST(NOT(languages IN (2, 3)) AS VARCHAR):s | CAST(NOT(languages IN (2, null, 3)) AS VARCHAR):s -----------+----------------------------------------------+--------------------------------------------------- -10018 |false |false -10019 |true |null -10020 |null |null + emp_no:i |CAST(languages NOT IN (2, 3) AS STRING):s|CAST(NOT languages IN (2, null, 3) AS STRING):s +---------------+-----------------------------------------+----------------------------------------------- +10018 |false |false +10019 |true |null +10020 |null |null ; diff --git a/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/analysis/analyzer/Analyzer.java b/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/analysis/analyzer/Analyzer.java index 090fea80b14c4..097758976da84 100644 --- a/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/analysis/analyzer/Analyzer.java +++ b/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/analysis/analyzer/Analyzer.java @@ -772,7 +772,7 @@ private Expression collectResolvedAndReplace(Expression e, Map> seen for (Function seenFunction : list) { if (uf.arguments().equals(seenFunction.arguments())) { // Special check for COUNT: an already seen COUNT function will be returned only if its DISTINCT property - // matches the one from the unresolved function to be checked. + // matches the one from the unresolved function to be checked. if (seenFunction instanceof Count) { if (uf.sameAs((Count) seenFunction)) { return seenFunction; @@ -899,8 +899,7 @@ private List assignAliases(List expr return new Alias(c.source(), ((NamedExpression) c.field()).name(), c); } } - //TODO: maybe add something closer to SQL - return new Alias(child.source(), child.toString(), child); + return new Alias(child.source(), child.sourceText(), child); }, UnresolvedAlias.class); newExpr.add(expr.equals(transformed) ? expr : transformed); } @@ -1031,7 +1030,7 @@ private Expression rule(Expression e, List seen) { } private boolean functionsEquals(Function f, Function seenFunction) { - return f.name().equals(seenFunction.name()) && f.arguments().equals(seenFunction.arguments()); + return f.sourceText().equals(seenFunction.sourceText()) && f.arguments().equals(seenFunction.arguments()); } } diff --git a/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/expression/Literal.java b/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/expression/Literal.java index 1d029f3133039..579fd934b48ea 100644 --- a/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/expression/Literal.java +++ b/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/expression/Literal.java @@ -36,7 +36,7 @@ public Literal(Source source, Object value, DataType dataType) { } public Literal(Source source, String name, Object value, DataType dataType) { - super(source, name == null ? String.valueOf(value) : name, emptyList(), null); + super(source, name == null ? source.text() : name, emptyList(), null); this.dataType = dataType; this.value = DataTypeConversion.convert(value, dataType); } diff --git a/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/expression/function/Function.java b/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/expression/function/Function.java index f63145f6a25b5..06171d43cc036 100644 --- a/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/expression/function/Function.java +++ b/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/expression/function/Function.java @@ -14,7 +14,6 @@ import org.elasticsearch.xpack.sql.util.StringUtils; import java.util.List; -import java.util.StringJoiner; /** * Any SQL expression with parentheses, like {@code MAX()}, or {@code ABS()}. A @@ -33,7 +32,7 @@ protected Function(Source source, List children, ExpressionId id, bo // cannot detect name yet so override the name super(source, null, children, id, synthetic); functionName = StringUtils.camelCaseToUnderscore(getClass().getSimpleName()); - name = functionName() + functionArgs(); + name = source.text(); } public final List arguments() { @@ -52,7 +51,7 @@ public Nullability nullable() { @Override public String toString() { - return name() + "#" + id(); + return sourceText() + "#" + id(); } public String functionName() { @@ -64,16 +63,7 @@ public String functionId() { return id().toString(); } - protected String functionArgs() { - StringJoiner sj = new StringJoiner(",", "(", ")"); - for (Expression child : children()) { - String val = child instanceof NamedExpression && child.resolved() ? Expressions.name(child) : child.toString(); - sj.add(val); - } - return sj.toString(); - } - public boolean functionEquals(Function f) { return f != null && getClass() == f.getClass() && arguments().equals(f.arguments()); } -} +} \ No newline at end of file diff --git a/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/expression/function/UnresolvedFunction.java b/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/expression/function/UnresolvedFunction.java index 82b4d95a9b47b..52bbed17346b3 100644 --- a/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/expression/function/UnresolvedFunction.java +++ b/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/expression/function/UnresolvedFunction.java @@ -166,7 +166,7 @@ public String unresolvedMessage() { @Override public String toString() { - return UNRESOLVED_PREFIX + functionName() + functionArgs(); + return UNRESOLVED_PREFIX + sourceText(); } @Override diff --git a/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/expression/function/aggregate/Max.java b/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/expression/function/aggregate/Max.java index 5df73793677ed..e66dfdebc6b36 100644 --- a/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/expression/function/aggregate/Max.java +++ b/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/expression/function/aggregate/Max.java @@ -8,8 +8,8 @@ import org.elasticsearch.xpack.sql.expression.Expression; import org.elasticsearch.xpack.sql.expression.Expressions; import org.elasticsearch.xpack.sql.expression.Expressions.ParamOrdinal; -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 java.util.List; @@ -45,6 +45,6 @@ public String innerName() { @Override protected TypeResolution resolveType() { - return Expressions.typeMustBeNumericOrDate(field(), functionName(), ParamOrdinal.DEFAULT); + return Expressions.typeMustBeNumericOrDate(field(), sourceText(), ParamOrdinal.DEFAULT); } -} +} \ No newline at end of file diff --git a/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/expression/function/aggregate/Min.java b/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/expression/function/aggregate/Min.java index 09ee16619cd80..bc3c5952f3896 100644 --- a/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/expression/function/aggregate/Min.java +++ b/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/expression/function/aggregate/Min.java @@ -8,8 +8,8 @@ import org.elasticsearch.xpack.sql.expression.Expression; import org.elasticsearch.xpack.sql.expression.Expressions; import org.elasticsearch.xpack.sql.expression.Expressions.ParamOrdinal; -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 java.util.List; @@ -48,6 +48,6 @@ public String innerName() { @Override protected TypeResolution resolveType() { - return Expressions.typeMustBeNumericOrDate(field(), functionName(), ParamOrdinal.DEFAULT); + return Expressions.typeMustBeNumericOrDate(field(), sourceText(), ParamOrdinal.DEFAULT); } } diff --git a/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/expression/function/aggregate/NumericAggregate.java b/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/expression/function/aggregate/NumericAggregate.java index 7a6fe313544e7..bfe0d2ded7e34 100644 --- a/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/expression/function/aggregate/NumericAggregate.java +++ b/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/expression/function/aggregate/NumericAggregate.java @@ -25,7 +25,7 @@ abstract class NumericAggregate extends AggregateFunction { @Override protected TypeResolution resolveType() { - return Expressions.typeMustBeNumeric(field(), functionName(), ParamOrdinal.DEFAULT); + return Expressions.typeMustBeNumeric(field(), sourceText(), ParamOrdinal.DEFAULT); } @Override diff --git a/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/expression/function/aggregate/Percentile.java b/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/expression/function/aggregate/Percentile.java index ee5304b915c48..593466f4c4773 100644 --- a/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/expression/function/aggregate/Percentile.java +++ b/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/expression/function/aggregate/Percentile.java @@ -10,8 +10,8 @@ import org.elasticsearch.xpack.sql.expression.Expressions; import org.elasticsearch.xpack.sql.expression.Expressions.ParamOrdinal; import org.elasticsearch.xpack.sql.expression.Foldables; -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 java.util.List; @@ -52,7 +52,7 @@ protected TypeResolution resolveType() { return resolution; } - return Expressions.typeMustBeNumeric(percent, functionName(), ParamOrdinal.DEFAULT); + return Expressions.typeMustBeNumeric(percent, sourceText(), ParamOrdinal.DEFAULT); } public Expression percent() { diff --git a/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/expression/function/aggregate/PercentileRank.java b/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/expression/function/aggregate/PercentileRank.java index c8f51476af3d3..72614f8265f48 100644 --- a/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/expression/function/aggregate/PercentileRank.java +++ b/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/expression/function/aggregate/PercentileRank.java @@ -10,8 +10,8 @@ import org.elasticsearch.xpack.sql.expression.Expressions; import org.elasticsearch.xpack.sql.expression.Expressions.ParamOrdinal; import org.elasticsearch.xpack.sql.expression.Foldables; -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 java.util.List; @@ -52,7 +52,7 @@ protected TypeResolution resolveType() { return resolution; } - return Expressions.typeMustBeNumeric(value, functionName(), ParamOrdinal.DEFAULT); + return Expressions.typeMustBeNumeric(value, sourceText(), ParamOrdinal.DEFAULT); } public Expression value() { 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 04f9b13ff3329..fd82d2bb4db23 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 @@ -111,16 +111,4 @@ public boolean equals(Object obj) { return Objects.equals(dataType, other.dataType()) && Objects.equals(field(), other.field()); } - - @Override - public String toString() { - return functionName() + "(" + field().toString() + " AS " + to().sqlName() + ")#" + id(); - } - - @Override - public String name() { - StringBuilder sb = new StringBuilder(super.name()); - sb.insert(sb.length() - 1, " AS " + to().sqlName()); - return sb.toString(); - } } diff --git a/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/expression/function/scalar/ConfigurationFunction.java b/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/expression/function/scalar/ConfigurationFunction.java index 39ee00d9b7253..a24fba1b13569 100644 --- a/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/expression/function/scalar/ConfigurationFunction.java +++ b/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/expression/function/scalar/ConfigurationFunction.java @@ -12,7 +12,6 @@ import org.elasticsearch.xpack.sql.session.Configuration; import org.elasticsearch.xpack.sql.tree.Source; import org.elasticsearch.xpack.sql.type.DataType; -import org.elasticsearch.xpack.sql.util.StringUtils; import java.util.List; import java.util.Objects; @@ -55,11 +54,6 @@ public boolean foldable() { @Override public abstract Object fold(); - @Override - protected String functionArgs() { - return StringUtils.EMPTY; - } - @Override public ScriptTemplate asScript() { return asScript(this); diff --git a/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/expression/function/scalar/datetime/BaseDateTimeFunction.java b/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/expression/function/scalar/datetime/BaseDateTimeFunction.java index cf4fab276c12b..345498afd00f1 100644 --- a/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/expression/function/scalar/datetime/BaseDateTimeFunction.java +++ b/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/expression/function/scalar/datetime/BaseDateTimeFunction.java @@ -10,8 +10,8 @@ import org.elasticsearch.xpack.sql.expression.Expressions; import org.elasticsearch.xpack.sql.expression.Expressions.ParamOrdinal; import org.elasticsearch.xpack.sql.expression.function.scalar.UnaryScalarFunction; -import org.elasticsearch.xpack.sql.tree.Source; import org.elasticsearch.xpack.sql.tree.NodeInfo; +import org.elasticsearch.xpack.sql.tree.Source; import java.time.ZoneId; import java.time.ZonedDateTime; @@ -20,17 +20,10 @@ abstract class BaseDateTimeFunction extends UnaryScalarFunction { private final ZoneId zoneId; - private final String name; BaseDateTimeFunction(Source source, Expression field, ZoneId zoneId) { super(source, field); this.zoneId = zoneId; - - StringBuilder sb = new StringBuilder(super.name()); - // add timezone as last argument - sb.insert(sb.length() - 1, " [" + zoneId.getId() + "]"); - - this.name = sb.toString(); } @Override @@ -42,17 +35,12 @@ protected final NodeInfo info() { @Override protected TypeResolution resolveType() { - return Expressions.typeMustBeDate(field(), functionName(), ParamOrdinal.DEFAULT); + return Expressions.typeMustBeDate(field(), sourceText(), ParamOrdinal.DEFAULT); } public ZoneId zoneId() { return zoneId; } - - @Override - public String name() { - return name; - } @Override public boolean foldable() { diff --git a/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/expression/function/scalar/math/BinaryNumericFunction.java b/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/expression/function/scalar/math/BinaryNumericFunction.java index 5f04edec65051..f3369bf14a457 100644 --- a/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/expression/function/scalar/math/BinaryNumericFunction.java +++ b/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/expression/function/scalar/math/BinaryNumericFunction.java @@ -36,12 +36,12 @@ protected TypeResolution resolveType() { return new TypeResolution("Unresolved children"); } - TypeResolution resolution = Expressions.typeMustBeNumeric(left(), functionName(), ParamOrdinal.FIRST); + TypeResolution resolution = Expressions.typeMustBeNumeric(left(), sourceText(), ParamOrdinal.FIRST); if (resolution.unresolved()) { return resolution; } - return Expressions.typeMustBeNumeric(right(), functionName(), ParamOrdinal.SECOND); + return Expressions.typeMustBeNumeric(right(), sourceText(), ParamOrdinal.SECOND); } @Override diff --git a/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/expression/function/scalar/math/E.java b/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/expression/function/scalar/math/E.java index 843ecccb265af..265b96984b581 100644 --- a/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/expression/function/scalar/math/E.java +++ b/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/expression/function/scalar/math/E.java @@ -11,10 +11,9 @@ import org.elasticsearch.xpack.sql.expression.function.scalar.math.MathProcessor.MathOperation; import org.elasticsearch.xpack.sql.expression.gen.script.Params; 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.StringUtils; public class E extends MathFunction { @@ -39,11 +38,6 @@ public Object fold() { return Math.E; } - @Override - protected String functionArgs() { - return StringUtils.EMPTY; - } - @Override public ScriptTemplate asScript() { return TEMPLATE; diff --git a/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/expression/function/scalar/math/Pi.java b/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/expression/function/scalar/math/Pi.java index f6dabb1f211dd..7fb966c3201a3 100644 --- a/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/expression/function/scalar/math/Pi.java +++ b/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/expression/function/scalar/math/Pi.java @@ -11,10 +11,9 @@ import org.elasticsearch.xpack.sql.expression.function.scalar.math.MathProcessor.MathOperation; import org.elasticsearch.xpack.sql.expression.gen.script.Params; 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.StringUtils; public class Pi extends MathFunction { @@ -39,11 +38,6 @@ public Object fold() { return Math.PI; } - @Override - protected String functionArgs() { - return StringUtils.EMPTY; - } - @Override public ScriptTemplate asScript() { return TEMPLATE; diff --git a/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/expression/function/scalar/string/BinaryStringFunction.java b/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/expression/function/scalar/string/BinaryStringFunction.java index eb0d9423e9d3d..fd294564b642c 100644 --- a/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/expression/function/scalar/string/BinaryStringFunction.java +++ b/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/expression/function/scalar/string/BinaryStringFunction.java @@ -6,6 +6,7 @@ package org.elasticsearch.xpack.sql.expression.function.scalar.string; import org.elasticsearch.xpack.sql.expression.Expression; +import org.elasticsearch.xpack.sql.expression.Expressions.ParamOrdinal; import org.elasticsearch.xpack.sql.expression.FieldAttribute; import org.elasticsearch.xpack.sql.expression.function.scalar.BinaryScalarFunction; import org.elasticsearch.xpack.sql.expression.gen.script.ScriptTemplate; @@ -15,7 +16,6 @@ import java.util.Objects; import java.util.function.BiFunction; -import static org.elasticsearch.xpack.sql.expression.Expressions.ParamOrdinal; import static org.elasticsearch.xpack.sql.expression.Expressions.typeMustBeString; import static org.elasticsearch.xpack.sql.expression.gen.script.ParamsBuilder.paramsBuilder; @@ -42,7 +42,7 @@ protected TypeResolution resolveType() { return new TypeResolution("Unresolved children"); } - TypeResolution resolution = typeMustBeString(left(), functionName(), ParamOrdinal.FIRST); + TypeResolution resolution = typeMustBeString(left(), sourceText(), ParamOrdinal.FIRST); if (resolution.unresolved()) { return resolution; } diff --git a/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/expression/function/scalar/string/BinaryStringNumericFunction.java b/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/expression/function/scalar/string/BinaryStringNumericFunction.java index 1c15e3ec5d73f..d9f767d1ce81a 100644 --- a/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/expression/function/scalar/string/BinaryStringNumericFunction.java +++ b/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/expression/function/scalar/string/BinaryStringNumericFunction.java @@ -26,7 +26,7 @@ public BinaryStringNumericFunction(Source source, Expression left, Expression ri @Override protected TypeResolution resolveSecondParameterInputType(Expression e) { - return Expressions.typeMustBeNumeric(e,functionName(), Expressions.ParamOrdinal.SECOND); + return Expressions.typeMustBeNumeric(e, sourceText(), Expressions.ParamOrdinal.SECOND); } @Override diff --git a/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/expression/function/scalar/string/BinaryStringStringFunction.java b/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/expression/function/scalar/string/BinaryStringStringFunction.java index 9a937d4647f50..51189f6a4efff 100644 --- a/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/expression/function/scalar/string/BinaryStringStringFunction.java +++ b/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/expression/function/scalar/string/BinaryStringStringFunction.java @@ -21,7 +21,7 @@ public BinaryStringStringFunction(Source source, Expression left, Expression rig @Override protected TypeResolution resolveSecondParameterInputType(Expression e) { - return Expressions.typeMustBeString(e, functionName(), Expressions.ParamOrdinal.SECOND); + return Expressions.typeMustBeString(e, sourceText(), Expressions.ParamOrdinal.SECOND); } @Override diff --git a/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/expression/function/scalar/string/Concat.java b/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/expression/function/scalar/string/Concat.java index 9a132c012a817..1f9833133a98a 100644 --- a/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/expression/function/scalar/string/Concat.java +++ b/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/expression/function/scalar/string/Concat.java @@ -13,8 +13,8 @@ import org.elasticsearch.xpack.sql.expression.function.scalar.BinaryScalarFunction; 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 static org.elasticsearch.xpack.sql.expression.function.scalar.string.ConcatFunctionProcessor.process; @@ -37,12 +37,12 @@ protected TypeResolution resolveType() { return new TypeResolution("Unresolved children"); } - TypeResolution sourceResolution = Expressions.typeMustBeString(left(), functionName(), ParamOrdinal.FIRST); + TypeResolution sourceResolution = Expressions.typeMustBeString(left(), sourceText(), ParamOrdinal.FIRST); if (sourceResolution.unresolved()) { return sourceResolution; } - return Expressions.typeMustBeString(right(), functionName(), ParamOrdinal.SECOND); + return Expressions.typeMustBeString(right(), sourceText(), ParamOrdinal.SECOND); } @Override diff --git a/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/expression/function/scalar/string/Insert.java b/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/expression/function/scalar/string/Insert.java index 7f15e9ce6117f..8e6fc2052928e 100644 --- a/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/expression/function/scalar/string/Insert.java +++ b/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/expression/function/scalar/string/Insert.java @@ -46,22 +46,22 @@ protected TypeResolution resolveType() { return new TypeResolution("Unresolved children"); } - TypeResolution sourceResolution = Expressions.typeMustBeString(source, functionName(), ParamOrdinal.FIRST); + TypeResolution sourceResolution = Expressions.typeMustBeString(source, sourceText(), ParamOrdinal.FIRST); if (sourceResolution.unresolved()) { return sourceResolution; } - TypeResolution startResolution = Expressions.typeMustBeNumeric(start, functionName(), ParamOrdinal.SECOND); + TypeResolution startResolution = Expressions.typeMustBeNumeric(start, sourceText(), ParamOrdinal.SECOND); if (startResolution.unresolved()) { return startResolution; } - TypeResolution lengthResolution = Expressions.typeMustBeNumeric(length, functionName(), ParamOrdinal.THIRD); + TypeResolution lengthResolution = Expressions.typeMustBeNumeric(length, sourceText(), ParamOrdinal.THIRD); if (lengthResolution.unresolved()) { return lengthResolution; } - return Expressions.typeMustBeString(replacement, functionName(), ParamOrdinal.FOURTH); + return Expressions.typeMustBeString(replacement, sourceText(), ParamOrdinal.FOURTH); } @Override diff --git a/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/expression/function/scalar/string/Locate.java b/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/expression/function/scalar/string/Locate.java index cf3d7ed6e6a12..042ec1a736373 100644 --- a/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/expression/function/scalar/string/Locate.java +++ b/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/expression/function/scalar/string/Locate.java @@ -48,19 +48,19 @@ protected TypeResolution resolveType() { return new TypeResolution("Unresolved children"); } - TypeResolution patternResolution = Expressions.typeMustBeString(pattern, functionName(), ParamOrdinal.FIRST); + TypeResolution patternResolution = Expressions.typeMustBeString(pattern, sourceText(), ParamOrdinal.FIRST); if (patternResolution.unresolved()) { return patternResolution; } - TypeResolution sourceResolution = Expressions.typeMustBeString(source, functionName(), ParamOrdinal.SECOND); + TypeResolution sourceResolution = Expressions.typeMustBeString(source, sourceText(), ParamOrdinal.SECOND); if (sourceResolution.unresolved()) { return sourceResolution; } return start == null ? TypeResolution.TYPE_RESOLVED : - Expressions.typeMustBeNumeric(start, functionName(), ParamOrdinal.THIRD); + Expressions.typeMustBeNumeric(start, sourceText(), ParamOrdinal.THIRD); } @Override diff --git a/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/expression/function/scalar/string/Replace.java b/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/expression/function/scalar/string/Replace.java index 82994fae9f000..cd960f1f3b3ea 100644 --- a/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/expression/function/scalar/string/Replace.java +++ b/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/expression/function/scalar/string/Replace.java @@ -44,17 +44,17 @@ protected TypeResolution resolveType() { return new TypeResolution("Unresolved children"); } - TypeResolution sourceResolution = Expressions.typeMustBeString(source, functionName(), ParamOrdinal.FIRST); + TypeResolution sourceResolution = Expressions.typeMustBeString(source, sourceText(), ParamOrdinal.FIRST); if (sourceResolution.unresolved()) { return sourceResolution; } - TypeResolution patternResolution = Expressions.typeMustBeString(pattern, functionName(), ParamOrdinal.SECOND); + TypeResolution patternResolution = Expressions.typeMustBeString(pattern, sourceText(), ParamOrdinal.SECOND); if (patternResolution.unresolved()) { return patternResolution; } - return Expressions.typeMustBeString(replacement, functionName(), ParamOrdinal.THIRD); + return Expressions.typeMustBeString(replacement, sourceText(), ParamOrdinal.THIRD); } @Override @@ -124,4 +124,4 @@ public Expression replaceChildren(List newChildren) { return new Replace(source(), newChildren.get(0), newChildren.get(1), newChildren.get(2)); } -} +} \ No newline at end of file diff --git a/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/expression/function/scalar/string/Substring.java b/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/expression/function/scalar/string/Substring.java index c049551f90ef9..75178e73fce46 100644 --- a/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/expression/function/scalar/string/Substring.java +++ b/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/expression/function/scalar/string/Substring.java @@ -45,17 +45,17 @@ protected TypeResolution resolveType() { return new TypeResolution("Unresolved children"); } - TypeResolution sourceResolution = Expressions.typeMustBeString(source, functionName(), ParamOrdinal.FIRST); + TypeResolution sourceResolution = Expressions.typeMustBeString(source, sourceText(), ParamOrdinal.FIRST); if (sourceResolution.unresolved()) { return sourceResolution; } - TypeResolution startResolution = Expressions.typeMustBeNumeric(start, functionName(), ParamOrdinal.SECOND); + TypeResolution startResolution = Expressions.typeMustBeNumeric(start, sourceText(), ParamOrdinal.SECOND); if (startResolution.unresolved()) { return startResolution; } - return Expressions.typeMustBeNumeric(length, functionName(), ParamOrdinal.THIRD); + return Expressions.typeMustBeNumeric(length, sourceText(), ParamOrdinal.THIRD); } @Override @@ -124,4 +124,4 @@ public Expression replaceChildren(List newChildren) { return new Substring(source(), newChildren.get(0), newChildren.get(1), newChildren.get(2)); } -} +} \ No newline at end of file diff --git a/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/expression/predicate/BinaryPredicate.java b/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/expression/predicate/BinaryPredicate.java index 277fa7e4998e5..eb7265dc29bc8 100644 --- a/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/expression/predicate/BinaryPredicate.java +++ b/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/expression/predicate/BinaryPredicate.java @@ -6,8 +6,6 @@ package org.elasticsearch.xpack.sql.expression.predicate; import org.elasticsearch.xpack.sql.expression.Expression; -import org.elasticsearch.xpack.sql.expression.Expressions; -import org.elasticsearch.xpack.sql.expression.Literal; import org.elasticsearch.xpack.sql.expression.function.scalar.BinaryScalarFunction; import org.elasticsearch.xpack.sql.tree.Source; @@ -21,12 +19,10 @@ */ public abstract class BinaryPredicate> extends BinaryScalarFunction { - private final String name; private final F function; protected BinaryPredicate(Source source, Expression left, Expression right, F function) { super(source, left, right); - this.name = name(left, right, function.symbol()); this.function = function; } @@ -65,11 +61,6 @@ public boolean equals(Object obj) { && Objects.equals(right(), other.right()); } - @Override - public String name() { - return name; - } - public String symbol() { return function.symbol(); } @@ -77,23 +68,4 @@ public String symbol() { public F function() { return function; } - - private static String name(Expression left, Expression right, String symbol) { - StringBuilder sb = new StringBuilder(); - sb.append(Expressions.name(left)); - if (!(left instanceof Literal)) { - sb.insert(0, "("); - sb.append(")"); - } - sb.append(" "); - sb.append(symbol); - sb.append(" "); - int pos = sb.length(); - sb.append(Expressions.name(right)); - if (!(right instanceof Literal)) { - sb.insert(pos, "("); - sb.append(")"); - } - return sb.toString(); - } } \ No newline at end of file diff --git a/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/expression/predicate/Range.java b/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/expression/predicate/Range.java index a189b7fda0eeb..aa1a3a32e5541 100644 --- a/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/expression/predicate/Range.java +++ b/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/expression/predicate/Range.java @@ -18,8 +18,8 @@ import org.elasticsearch.xpack.sql.expression.predicate.operator.comparison.BinaryComparison; import org.elasticsearch.xpack.sql.expression.predicate.operator.comparison.BinaryComparisonPipe; import org.elasticsearch.xpack.sql.expression.predicate.operator.comparison.BinaryComparisonProcessor.BinaryComparisonOperation; -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 java.util.List; @@ -33,7 +33,6 @@ // BETWEEN or range - is a mix of gt(e) AND lt(e) public class Range extends ScalarFunction { - private final String name; private final Expression value, lower, upper; private final boolean includeLower, includeUpper; @@ -45,12 +44,6 @@ public Range(Source source, Expression value, Expression lower, boolean includeL this.upper = upper; this.includeLower = includeLower; this.includeUpper = includeUpper; - this.name = name(value, lower, upper, includeLower, includeUpper); - } - - @Override - public String name() { - return name; } @Override @@ -212,9 +205,4 @@ private static String name(Expression value, Expression lower, Expression upper, return sb.toString(); } - - @Override - public String toString() { - return name(); - } } diff --git a/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/expression/predicate/logical/BinaryLogic.java b/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/expression/predicate/logical/BinaryLogic.java index d4bce9feab3ec..930636657fc48 100644 --- a/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/expression/predicate/logical/BinaryLogic.java +++ b/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/expression/predicate/logical/BinaryLogic.java @@ -27,7 +27,7 @@ public DataType dataType() { @Override protected TypeResolution resolveInputType(Expression e, Expressions.ParamOrdinal paramOrdinal) { - return Expressions.typeMustBeBoolean(e, functionName(), paramOrdinal); + return Expressions.typeMustBeBoolean(e, sourceText(), paramOrdinal); } @Override diff --git a/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/expression/predicate/logical/Not.java b/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/expression/predicate/logical/Not.java index 8d326c570292b..bbadaa6ef6430 100644 --- a/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/expression/predicate/logical/Not.java +++ b/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/expression/predicate/logical/Not.java @@ -12,8 +12,8 @@ import org.elasticsearch.xpack.sql.expression.gen.processor.Processor; import org.elasticsearch.xpack.sql.expression.gen.script.Scripts; import org.elasticsearch.xpack.sql.expression.predicate.Negatable; -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; public class Not extends UnaryScalarFunction { @@ -37,7 +37,7 @@ protected TypeResolution resolveType() { if (DataType.BOOLEAN == field().dataType()) { return TypeResolution.TYPE_RESOLVED; } - return Expressions.typeMustBeBoolean(field(), functionName(), ParamOrdinal.DEFAULT); + return Expressions.typeMustBeBoolean(field(), sourceText(), ParamOrdinal.DEFAULT); } @Override diff --git a/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/expression/predicate/operator/arithmetic/Neg.java b/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/expression/predicate/operator/arithmetic/Neg.java index 265a1eb993c94..3ada9a523a13c 100644 --- a/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/expression/predicate/operator/arithmetic/Neg.java +++ b/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/expression/predicate/operator/arithmetic/Neg.java @@ -8,13 +8,12 @@ import org.elasticsearch.xpack.sql.expression.Expression; import org.elasticsearch.xpack.sql.expression.Expressions; import org.elasticsearch.xpack.sql.expression.Expressions.ParamOrdinal; -import org.elasticsearch.xpack.sql.expression.NamedExpression; import org.elasticsearch.xpack.sql.expression.function.scalar.UnaryScalarFunction; import org.elasticsearch.xpack.sql.expression.gen.processor.Processor; import org.elasticsearch.xpack.sql.expression.gen.script.Scripts; import org.elasticsearch.xpack.sql.expression.predicate.operator.arithmetic.UnaryArithmeticProcessor.UnaryArithmeticOperation; -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; /** @@ -38,7 +37,7 @@ protected Neg replaceChild(Expression newChild) { @Override protected TypeResolution resolveType() { - return Expressions.typeMustBeNumeric(field(), functionName(), ParamOrdinal.DEFAULT); + return Expressions.typeMustBeNumeric(field(), sourceText(), ParamOrdinal.DEFAULT); } @Override @@ -51,11 +50,6 @@ public DataType dataType() { return field().dataType(); } - @Override - public String name() { - return "-" + (field() instanceof NamedExpression && field().resolved() ? Expressions.name(field()) : field().toString()); - } - @Override public String processScript(String script) { return Scripts.formatTemplate(Scripts.SQL_SCRIPTS + ".neg(" + script + ")"); diff --git a/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/expression/predicate/operator/comparison/BinaryComparison.java b/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/expression/predicate/operator/comparison/BinaryComparison.java index 815fa10b12fd1..4e7473907c890 100644 --- a/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/expression/predicate/operator/comparison/BinaryComparison.java +++ b/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/expression/predicate/operator/comparison/BinaryComparison.java @@ -40,18 +40,7 @@ protected Pipe makePipe() { return new BinaryComparisonPipe(source(), this, Expressions.pipe(left()), Expressions.pipe(right()), function()); } - @Override - public String toString() { - StringBuilder sb = new StringBuilder(); - sb.append(left()); - sb.append(" "); - sb.append(symbol()); - sb.append(" "); - sb.append(right()); - return sb.toString(); - } - public static Integer compare(Object left, Object right) { return Comparisons.compare(left, right); } -} +} \ No newline at end of file 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 f9e6b72fb5958..f8f0bb35b504e 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 @@ -21,7 +21,6 @@ import java.util.LinkedHashSet; import java.util.List; import java.util.Objects; -import java.util.StringJoiner; import java.util.stream.Collectors; import static org.elasticsearch.common.logging.LoggerMessageFormat.format; @@ -85,13 +84,6 @@ public Boolean fold() { return InProcessor.apply(value.fold(), Foldables.valuesOf(list, value.dataType())); } - @Override - public String name() { - StringJoiner sj = new StringJoiner(", ", " IN (", ")"); - list.forEach(e -> sj.add(Expressions.name(e))); - return Expressions.name(value) + sj.toString(); - } - @Override public ScriptTemplate asScript() { ScriptTemplate leftScript = asScript(value); diff --git a/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/parser/AbstractBuilder.java b/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/parser/AbstractBuilder.java index 1d6c264321787..81c2e7578ccb9 100644 --- a/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/parser/AbstractBuilder.java +++ b/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/parser/AbstractBuilder.java @@ -92,6 +92,16 @@ static Source source(Token token) { return new Source(new Location(token.getLine(), token.getCharPositionInLine()), text); } + Source source(ParserRuleContext begin, ParserRuleContext end) { + Check.notNull(begin, "begin is null"); + Check.notNull(end, "end is null"); + Token start = begin.start; + Token stop = end.stop != null ? end.stop : begin.stop; + Interval interval = new Interval(start.getStartIndex(), stop.getStopIndex()); + String text = start.getInputStream().getText(interval); + return new Source(new Location(start.getLine(), start.getCharPositionInLine()), text); + } + /** * Retrieves the raw text of the node (without interpreting it as a string literal). */ diff --git a/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/parser/ExpressionBuilder.java b/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/parser/ExpressionBuilder.java index e103976760fb3..132c390337628 100644 --- a/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/parser/ExpressionBuilder.java +++ b/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/parser/ExpressionBuilder.java @@ -11,6 +11,7 @@ import org.antlr.v4.runtime.tree.TerminalNode; import org.elasticsearch.common.Booleans; import org.elasticsearch.common.Strings; +import org.elasticsearch.common.collect.Tuple; import org.elasticsearch.xpack.sql.SqlIllegalArgumentException; import org.elasticsearch.xpack.sql.expression.Alias; import org.elasticsearch.xpack.sql.expression.Exists; @@ -215,7 +216,7 @@ public Expression visitPredicated(PredicatedContext ctx) { } PredicateContext pCtx = ctx.predicate(); - Source source = source(pCtx); + Source source = source(ctx.valueExpression(), ctx); Expression e = null; switch (pCtx.kind.getType()) { @@ -320,7 +321,7 @@ public Object visitArithmeticUnary(ArithmeticUnaryContext ctx) { if (value instanceof Literal) { // Minus already processed together with literal number return value; } - return new Neg(source(ctx.operator), value); + return new Neg(source(ctx), value); default: throw new ParsingException(source, "Unknown arithmetic {}", source.text()); } @@ -331,7 +332,7 @@ public Object visitArithmeticBinary(ArithmeticBinaryContext ctx) { Expression left = expression(ctx.left); Expression right = expression(ctx.right); - Source source = source(ctx.operator); + Source source = source(ctx); switch (ctx.operator.getType()) { case SqlBaseParser.ASTERISK: @@ -611,7 +612,7 @@ public Literal visitInterval(IntervalContext interval) { Interval timeInterval = value instanceof Period ? new IntervalYearMonth((Period) value, intervalType) : new IntervalDayTime((Duration) value, intervalType); - return new Literal(source(interval), text(interval), timeInterval, timeInterval.dataType()); + return new Literal(source(interval), timeInterval, timeInterval.dataType()); } private TemporalAmount of(NumberContext valueNumeric, TimeUnit unit) { @@ -689,23 +690,24 @@ public Expression visitStringLiteral(StringLiteralContext ctx) { @Override public Literal visitDecimalLiteral(DecimalLiteralContext ctx) { - String string = (hasMinusFromParent(ctx) ? "-" : "") + ctx.getText(); + Tuple tuple = withMinus(ctx); + try { - return new Literal(source(ctx), Double.valueOf(StringUtils.parseDouble(string)), DataType.DOUBLE); + return new Literal(tuple.v1(), Double.valueOf(StringUtils.parseDouble(tuple.v2())), DataType.DOUBLE); } catch (SqlIllegalArgumentException siae) { - throw new ParsingException(source(ctx), siae.getMessage()); + throw new ParsingException(tuple.v1(), siae.getMessage()); } } @Override public Literal visitIntegerLiteral(IntegerLiteralContext ctx) { - String string = (hasMinusFromParent(ctx) ? "-" : "") + ctx.getText(); + Tuple tuple = withMinus(ctx); long value; try { - value = Long.valueOf(StringUtils.parseLong(string)); + value = Long.valueOf(StringUtils.parseLong(tuple.v2())); } catch (SqlIllegalArgumentException siae) { - throw new ParsingException(source(ctx), siae.getMessage()); + throw new ParsingException(tuple.v1(), siae.getMessage()); } Object val = Long.valueOf(value); @@ -715,7 +717,7 @@ public Literal visitIntegerLiteral(IntegerLiteralContext ctx) { type = DataType.INTEGER; val = Integer.valueOf((int) value); } - return new Literal(source(ctx), val, type); + return new Literal(tuple.v1(), val, type); } @Override @@ -876,7 +878,30 @@ public Literal visitGuidEscapedLiteral(GuidEscapedLiteralContext ctx) { return new Literal(source(ctx), string, DataType.KEYWORD); } - private boolean hasMinusFromParent(SqlBaseParser.NumberContext ctx) { + /** + * Return the source and the value of the given number, + * taking into account MINUS (-) if needed. + */ + private static Tuple withMinus(NumberContext ctx) { + String string = ctx.getText(); + Source source = minusAwareSource(ctx); + + if (source != null) { + string = "-" + string; + } else { + source = source(ctx); + } + + return new Tuple<>(source, string); + } + + /** + * Checks the presence of MINUS (-) in the parent and if found, + * returns the parent source or null otherwise. + * Parsing of the value should not depend on the returned source + * as it might contain extra spaces. + */ + private static Source minusAwareSource(SqlBaseParser.NumberContext ctx) { ParserRuleContext parentCtx = ctx.getParent(); if (parentCtx != null) { if (parentCtx instanceof SqlBaseParser.NumericLiteralContext) { @@ -886,17 +911,23 @@ private boolean hasMinusFromParent(SqlBaseParser.NumberContext ctx) { if (parentCtx != null && parentCtx instanceof SqlBaseParser.ValueExpressionDefaultContext) { parentCtx = parentCtx.getParent(); if (parentCtx != null && parentCtx instanceof SqlBaseParser.ArithmeticUnaryContext) { - return ((ArithmeticUnaryContext) parentCtx).MINUS() != null; + if (((ArithmeticUnaryContext) parentCtx).MINUS() != null) { + return source(parentCtx); + } } } } } else if (parentCtx instanceof SqlBaseParser.IntervalContext) { IntervalContext ic = (IntervalContext) parentCtx; - return ic.sign != null && ic.sign.getType() == SqlBaseParser.MINUS; + if (ic.sign != null && ic.sign.getType() == SqlBaseParser.MINUS) { + return source(ic); + } } else if (parentCtx instanceof SqlBaseParser.SysTypesContext) { - return ((SysTypesContext) parentCtx).MINUS() != null; + if (((SysTypesContext) parentCtx).MINUS() != null) { + return source(parentCtx); + } } } - return false; + return null; } -} +} \ No newline at end of file diff --git a/x-pack/plugin/sql/src/test/java/org/elasticsearch/xpack/sql/analysis/analyzer/VerifierErrorMessagesTests.java b/x-pack/plugin/sql/src/test/java/org/elasticsearch/xpack/sql/analysis/analyzer/VerifierErrorMessagesTests.java index 6366ba85503b0..2d20d92e17826 100644 --- a/x-pack/plugin/sql/src/test/java/org/elasticsearch/xpack/sql/analysis/analyzer/VerifierErrorMessagesTests.java +++ b/x-pack/plugin/sql/src/test/java/org/elasticsearch/xpack/sql/analysis/analyzer/VerifierErrorMessagesTests.java @@ -239,7 +239,7 @@ public void testGroupByOrderByAliasedInSelectAllowed() { } public void testGroupByOrderByScalarOverNonGrouped() { - assertEquals("1:50: Cannot order by non-grouped column [YEAR(date [Z])], expected [text]", + assertEquals("1:50: Cannot order by non-grouped column [YEAR(date)], expected [text]", error("SELECT MAX(int) FROM test GROUP BY text ORDER BY YEAR(date)")); } @@ -249,7 +249,7 @@ public void testGroupByOrderByFieldFromGroupByFunction() { } public void testGroupByOrderByScalarOverNonGrouped_WithHaving() { - assertEquals("1:71: Cannot order by non-grouped column [YEAR(date [Z])], expected [text]", + assertEquals("1:71: Cannot order by non-grouped column [YEAR(date)], expected [text]", error("SELECT MAX(int) FROM test GROUP BY text HAVING MAX(int) > 10 ORDER BY YEAR(date)")); } @@ -373,12 +373,12 @@ public void testInNestedWithDifferentDataTypesFromLeftValue_WhereClause() { } public void testNotSupportedAggregateOnDate() { - assertEquals("1:8: [AVG] argument must be [numeric], found value [date] type [date]", + assertEquals("1:8: [AVG(date)] argument must be [numeric], found value [date] type [date]", error("SELECT AVG(date) FROM test")); } public void testNotSupportedAggregateOnString() { - assertEquals("1:8: [MAX] argument must be [numeric or date], found value [keyword] type [keyword]", + assertEquals("1:8: [MAX(keyword)] argument must be [numeric or date], found value [keyword] type [keyword]", error("SELECT MAX(keyword) FROM test")); } @@ -388,53 +388,53 @@ public void testInvalidTypeForStringFunction_WithOneArg() { } public void testInvalidTypeForNumericFunction_WithOneArg() { - assertEquals("1:8: [COS] argument must be [numeric], found value [foo] type [keyword]", + assertEquals("1:8: [COS] argument must be [numeric], found value ['foo'] type [keyword]", error("SELECT COS('foo')")); } public void testInvalidTypeForBooleanFunction_WithOneArg() { - assertEquals("1:8: [NOT] argument must be [boolean], found value [foo] type [keyword]", + assertEquals("1:8: [NOT 'foo'] argument must be [boolean], found value ['foo'] type [keyword]", error("SELECT NOT 'foo'")); } public void testInvalidTypeForStringFunction_WithTwoArgs() { - assertEquals("1:8: [CONCAT] first argument must be [string], found value [1] type [integer]", + assertEquals("1:8: [CONCAT(1, 'bar')] first argument must be [string], found value [1] type [integer]", error("SELECT CONCAT(1, 'bar')")); - assertEquals("1:8: [CONCAT] second argument must be [string], found value [2] type [integer]", + assertEquals("1:8: [CONCAT('foo', 2)] second argument must be [string], found value [2] type [integer]", error("SELECT CONCAT('foo', 2)")); } public void testInvalidTypeForNumericFunction_WithTwoArgs() { - assertEquals("1:8: [TRUNCATE] first argument must be [numeric], found value [foo] type [keyword]", + assertEquals("1:8: [TRUNCATE('foo', 2)] first argument must be [numeric], found value ['foo'] type [keyword]", error("SELECT TRUNCATE('foo', 2)")); - assertEquals("1:8: [TRUNCATE] second argument must be [numeric], found value [bar] type [keyword]", + assertEquals("1:8: [TRUNCATE(1.2, 'bar')] second argument must be [numeric], found value ['bar'] type [keyword]", error("SELECT TRUNCATE(1.2, 'bar')")); } public void testInvalidTypeForBooleanFuntion_WithTwoArgs() { - assertEquals("1:8: [OR] first argument must be [boolean], found value [1] type [integer]", + assertEquals("1:8: [1 OR true] first argument must be [boolean], found value [1] type [integer]", error("SELECT 1 OR true")); - assertEquals("1:8: [OR] second argument must be [boolean], found value [2] type [integer]", + assertEquals("1:8: [true OR 2] second argument must be [boolean], found value [2] type [integer]", error("SELECT true OR 2")); } public void testInvalidTypeForFunction_WithThreeArgs() { - assertEquals("1:8: [REPLACE] first argument must be [string], found value [1] type [integer]", + assertEquals("1:8: [REPLACE(1, 'foo', 'bar')] first argument must be [string], found value [1] type [integer]", error("SELECT REPLACE(1, 'foo', 'bar')")); - assertEquals("1:8: [REPLACE] second argument must be [string], found value [2] type [integer]", + assertEquals("1:8: [REPLACE('text', 2, 'bar')] second argument must be [string], found value [2] type [integer]", error("SELECT REPLACE('text', 2, 'bar')")); - assertEquals("1:8: [REPLACE] third argument must be [string], found value [3] type [integer]", + assertEquals("1:8: [REPLACE('text', 'foo', 3)] third argument must be [string], found value [3] type [integer]", error("SELECT REPLACE('text', 'foo', 3)")); } public void testInvalidTypeForFunction_WithFourArgs() { - assertEquals("1:8: [INSERT] first argument must be [string], found value [1] type [integer]", + assertEquals("1:8: [INSERT(1, 1, 2, 'new')] first argument must be [string], found value [1] type [integer]", error("SELECT INSERT(1, 1, 2, 'new')")); - assertEquals("1:8: [INSERT] second argument must be [numeric], found value [foo] type [keyword]", + assertEquals("1:8: [INSERT('text', 'foo', 2, 'new')] second argument must be [numeric], found value ['foo'] type [keyword]", error("SELECT INSERT('text', 'foo', 2, 'new')")); - assertEquals("1:8: [INSERT] third argument must be [numeric], found value [bar] type [keyword]", + assertEquals("1:8: [INSERT('text', 1, 'bar', 'new')] third argument must be [numeric], found value ['bar'] type [keyword]", error("SELECT INSERT('text', 1, 'bar', 'new')")); - assertEquals("1:8: [INSERT] fourth argument must be [string], found value [3] type [integer]", + assertEquals("1:8: [INSERT('text', 1, 2, 3)] fourth argument must be [string], found value [3] type [integer]", error("SELECT INSERT('text', 1, 2, 3)")); } @@ -508,7 +508,7 @@ public void testAggsInWhere() { } public void testHistogramInFilter() { - assertEquals("1:63: Cannot filter on grouping function [HISTOGRAM(date)], use its argument instead", + assertEquals("1:63: Cannot filter on grouping function [HISTOGRAM(date, INTERVAL 1 MONTH)], use its argument instead", error("SELECT HISTOGRAM(date, INTERVAL 1 MONTH) AS h FROM test WHERE " + "HISTOGRAM(date, INTERVAL 1 MONTH) > CAST('2000-01-01' AS DATE) GROUP BY h")); } @@ -522,8 +522,8 @@ public void testHistogramInHaving() { public void testGroupByScalarOnTopOfGrouping() { assertEquals( - "1:14: Cannot combine [HISTOGRAM(date)] grouping function inside GROUP BY, " - + "found [MONTH_OF_YEAR(HISTOGRAM(date) [Z])]; consider moving the expression inside the histogram", + "1:14: Cannot combine [HISTOGRAM(date, INTERVAL 1 MONTH)] grouping function inside " + + "GROUP BY, found [MONTH(HISTOGRAM(date, INTERVAL 1 MONTH))]; consider moving the expression inside the histogram", error("SELECT MONTH(HISTOGRAM(date, INTERVAL 1 MONTH)) AS h FROM test GROUP BY h")); } diff --git a/x-pack/plugin/sql/src/test/java/org/elasticsearch/xpack/sql/expression/function/NamedExpressionTests.java b/x-pack/plugin/sql/src/test/java/org/elasticsearch/xpack/sql/expression/function/NamedExpressionTests.java index d723e79274ed9..4ac4450620626 100644 --- a/x-pack/plugin/sql/src/test/java/org/elasticsearch/xpack/sql/expression/function/NamedExpressionTests.java +++ b/x-pack/plugin/sql/src/test/java/org/elasticsearch/xpack/sql/expression/function/NamedExpressionTests.java @@ -14,6 +14,8 @@ import org.elasticsearch.xpack.sql.expression.predicate.operator.arithmetic.Mul; import org.elasticsearch.xpack.sql.expression.predicate.operator.arithmetic.Neg; import org.elasticsearch.xpack.sql.expression.predicate.operator.arithmetic.Sub; +import org.elasticsearch.xpack.sql.tree.Location; +import org.elasticsearch.xpack.sql.tree.Source; import org.elasticsearch.xpack.sql.type.DataType; import org.elasticsearch.xpack.sql.type.EsField; @@ -23,29 +25,40 @@ public class NamedExpressionTests extends ESTestCase { public void testArithmeticFunctionName() { - Add add = new Add(EMPTY, l(5), l(2)); - assertEquals("5 + 2", add.name()); + String e = "5 + 2"; + Add add = new Add(s(e), l(5), l(2)); + assertEquals(e, add.sourceText()); - Div div = new Div(EMPTY, l(5), l(2)); - assertEquals("5 / 2", div.name()); + e = "5 / 2"; + Div div = new Div(s(e), l(5), l(2)); + assertEquals(e, div.sourceText()); - Mod mod = new Mod(EMPTY, l(5), l(2)); - assertEquals("5 % 2", mod.name()); + e = "5%2"; + Mod mod = new Mod(s(e), l(5), l(2)); + assertEquals(e, mod.sourceText()); - Mul mul = new Mul(EMPTY, l(5), l(2)); - assertEquals("5 * 2", mul.name()); + e = "5 * 2"; + Mul mul = new Mul(s(e), l(5), l(2)); + assertEquals(e, mul.sourceText()); - Sub sub = new Sub(EMPTY, l(5), l(2)); - assertEquals("5 - 2", sub.name()); + e = "5 -2"; + Sub sub = new Sub(s(e), l(5), l(2)); + assertEquals(e, sub.sourceText()); - Neg neg = new Neg(EMPTY, l(5)); - assertEquals("-5", neg.name()); + e = " - 5"; + Neg neg = new Neg(s(e), l(5)); + assertEquals(e, neg.sourceText()); } public void testNameForArithmeticFunctionAppliedOnTableColumn() { FieldAttribute fa = new FieldAttribute(EMPTY, "myField", new EsField("myESField", DataType.INTEGER, emptyMap(), true)); - Add add = new Add(EMPTY, fa, l(10)); - assertEquals("(myField) + 10", add.name()); + String e = "myField + 10"; + Add add = new Add(s(e), fa, l(10)); + assertEquals(e, add.sourceText()); + } + + private static Source s(String text) { + return new Source(Location.EMPTY, text); } private static Literal l(Object value) { diff --git a/x-pack/plugin/sql/src/test/java/org/elasticsearch/xpack/sql/expression/function/scalar/datetime/DayOfYearTests.java b/x-pack/plugin/sql/src/test/java/org/elasticsearch/xpack/sql/expression/function/scalar/datetime/DayOfYearTests.java index 6bd4a8fe1baca..2a7af2916373e 100644 --- a/x-pack/plugin/sql/src/test/java/org/elasticsearch/xpack/sql/expression/function/scalar/datetime/DayOfYearTests.java +++ b/x-pack/plugin/sql/src/test/java/org/elasticsearch/xpack/sql/expression/function/scalar/datetime/DayOfYearTests.java @@ -7,6 +7,7 @@ import org.elasticsearch.test.ESTestCase; import org.elasticsearch.xpack.sql.expression.Literal; +import org.elasticsearch.xpack.sql.tree.Source; import org.elasticsearch.xpack.sql.type.DataType; import java.time.ZoneId; @@ -27,6 +28,6 @@ private Object extract(Object value, ZoneId zoneId) { } private DayOfYear build(Object value, ZoneId zoneId) { - return new DayOfYear(null, new Literal(null, value, DataType.DATE), zoneId); + return new DayOfYear(Source.EMPTY, new Literal(Source.EMPTY, value, DataType.DATE), zoneId); } } diff --git a/x-pack/plugin/sql/src/test/java/org/elasticsearch/xpack/sql/optimizer/OptimizerTests.java b/x-pack/plugin/sql/src/test/java/org/elasticsearch/xpack/sql/optimizer/OptimizerTests.java index bc5bc0985cb29..700097c46163a 100644 --- a/x-pack/plugin/sql/src/test/java/org/elasticsearch/xpack/sql/optimizer/OptimizerTests.java +++ b/x-pack/plugin/sql/src/test/java/org/elasticsearch/xpack/sql/optimizer/OptimizerTests.java @@ -585,7 +585,7 @@ public void testNullEqualsWithNullLiteralBecomesIsNull() { BooleanLiteralsOnTheRight swapLiteralsToRight = new BooleanLiteralsOnTheRight(); BinaryComparisonSimplification bcSimpl = new BinaryComparisonSimplification(); FieldAttribute fa = getFieldAttribute(); - Source source = new Source(1, 10, StringUtils.EMPTY); + Source source = new Source(1, 10, "IS_NULL(a)"); Expression e = bcSimpl.rule(swapLiteralsToRight.rule(new NullEquals(source, fa, NULL))); assertEquals(IsNull.class, e.getClass()); diff --git a/x-pack/plugin/sql/src/test/java/org/elasticsearch/xpack/sql/parser/EscapedFunctionsTests.java b/x-pack/plugin/sql/src/test/java/org/elasticsearch/xpack/sql/parser/EscapedFunctionsTests.java index 9492d63b7d323..5969f8e5ed2cd 100644 --- a/x-pack/plugin/sql/src/test/java/org/elasticsearch/xpack/sql/parser/EscapedFunctionsTests.java +++ b/x-pack/plugin/sql/src/test/java/org/elasticsearch/xpack/sql/parser/EscapedFunctionsTests.java @@ -80,12 +80,12 @@ private Function function(String name) { public void testFunctionNoArg() { Function f = function("SCORE()"); - assertEquals("SCORE", f.functionName()); + assertEquals("{fn SCORE()}", f.sourceText()); } public void testFunctionOneArg() { Function f = function("ABS(foo)"); - assertEquals("ABS", f.functionName()); + assertEquals("{fn ABS(foo)}", f.sourceText()); assertEquals(1, f.arguments().size()); Expression arg = f.arguments().get(0); assertThat(arg, instanceOf(UnresolvedAttribute.class)); @@ -95,75 +95,77 @@ public void testFunctionOneArg() { public void testFunctionOneArgFunction() { Function f = function("ABS({fn SCORE()})"); - assertEquals("ABS", f.functionName()); + assertEquals("{fn ABS({fn SCORE()})}", f.sourceText()); assertEquals(1, f.arguments().size()); Expression arg = f.arguments().get(0); assertThat(arg, instanceOf(UnresolvedFunction.class)); UnresolvedFunction uf = (UnresolvedFunction) arg; - assertThat(uf.name(), is("SCORE")); + assertThat(uf.sourceText(), is("{fn SCORE()}")); } public void testFunctionFloorWithExtract() { - Function f = function("CAST({fn FLOOR({fn EXTRACT(YEAR FROM \"foo\")})} AS int)"); - assertEquals("CAST", f.functionName()); + String e = "CAST({fn FLOOR({fn EXTRACT(YEAR FROM \"foo\")})} AS int)"; + Function f = function(e); + assertEquals(e, f.sourceText()); assertEquals(1, f.arguments().size()); Expression arg = f.arguments().get(0); assertThat(arg, instanceOf(UnresolvedFunction.class)); f = (Function) arg; - assertEquals("FLOOR", f.functionName()); + assertEquals("{fn FLOOR({fn EXTRACT(YEAR FROM \"foo\")})}", f.sourceText()); assertEquals(1, f.arguments().size()); arg = f.arguments().get(0); assertThat(arg, instanceOf(UnresolvedFunction.class)); UnresolvedFunction uf = (UnresolvedFunction) arg; - assertThat(uf.name(), is("YEAR")); + assertThat(uf.sourceText(), is("EXTRACT(YEAR FROM \"foo\")")); } public void testFunctionWithFunctionWithArg() { Function f = function("POWER(foo, {fn POWER({fn SCORE()}, {fN SCORE()})})"); - assertEquals("POWER", f.functionName()); + assertEquals("{fn POWER(foo, {fn POWER({fn SCORE()}, {fN SCORE()})})}", f.sourceText()); assertEquals(2, f.arguments().size()); Expression arg = f.arguments().get(1); assertThat(arg, instanceOf(UnresolvedFunction.class)); UnresolvedFunction uf = (UnresolvedFunction) arg; - assertThat(uf.name(), is("POWER")); + assertThat(uf.sourceText(), is("{fn POWER({fn SCORE()}, {fN SCORE()})}")); assertEquals(2, uf.arguments().size()); List args = uf.arguments(); arg = args.get(0); assertThat(arg, instanceOf(UnresolvedFunction.class)); uf = (UnresolvedFunction) arg; - assertThat(uf.name(), is("SCORE")); + assertThat(uf.sourceText(), is("{fn SCORE()}")); arg = args.get(1); assertThat(arg, instanceOf(UnresolvedFunction.class)); uf = (UnresolvedFunction) arg; - assertThat(uf.name(), is("SCORE")); + assertThat(uf.sourceText(), is("{fN SCORE()}")); } public void testFunctionWithFunctionWithArgAndParams() { - Function f = (Function) parser.createExpression("POWER(?, {fn POWER({fn ABS(?)}, {fN ABS(?)})})", + String e = "POWER(?, {fn POWER({fn ABS(?)}, {fN ABS(?)})})"; + Function f = (Function) parser.createExpression(e, asList(new SqlTypedParamValue(DataType.LONG.esType, 1), new SqlTypedParamValue(DataType.LONG.esType, 1), new SqlTypedParamValue(DataType.LONG.esType, 1))); - assertEquals("POWER", f.functionName()); + assertEquals(e, f.sourceText()); assertEquals(2, f.arguments().size()); Expression arg = f.arguments().get(1); assertThat(arg, instanceOf(UnresolvedFunction.class)); UnresolvedFunction uf = (UnresolvedFunction) arg; - assertThat(uf.name(), is("POWER")); + assertEquals("{fn POWER({fn ABS(?)}, {fN ABS(?)})}", uf.sourceText()); assertEquals(2, uf.arguments().size()); List args = uf.arguments(); arg = args.get(0); assertThat(arg, instanceOf(UnresolvedFunction.class)); uf = (UnresolvedFunction) arg; - assertThat(uf.name(), is("ABS")); + assertThat(uf.sourceText(), is("{fn ABS(?)}")); arg = args.get(1); assertThat(arg, instanceOf(UnresolvedFunction.class)); uf = (UnresolvedFunction) arg; - assertThat(uf.name(), is("ABS")); + assertThat(uf.sourceText(), is("{fN ABS(?)}")); } public void testDateLiteral() { diff --git a/x-pack/plugin/sql/src/test/java/org/elasticsearch/xpack/sql/parser/ExpressionTests.java b/x-pack/plugin/sql/src/test/java/org/elasticsearch/xpack/sql/parser/ExpressionTests.java index e56bd4ecd11ac..2e34e947944d4 100644 --- a/x-pack/plugin/sql/src/test/java/org/elasticsearch/xpack/sql/parser/ExpressionTests.java +++ b/x-pack/plugin/sql/src/test/java/org/elasticsearch/xpack/sql/parser/ExpressionTests.java @@ -26,7 +26,6 @@ import java.util.Locale; import static java.lang.String.format; -import static org.hamcrest.core.StringStartsWith.startsWith; public class ExpressionTests extends ESTestCase { @@ -36,7 +35,7 @@ public void testTokenFunctionName() { Expression lt = parser.createExpression("LEFT()"); assertEquals(UnresolvedFunction.class, lt.getClass()); UnresolvedFunction uf = (UnresolvedFunction) lt; - assertEquals("LEFT", uf.functionName()); + assertEquals("LEFT()", uf.sourceText()); } public void testLiteralBoolean() { @@ -182,10 +181,10 @@ private TemporalAmount intervalOf(String query) { } public void testLiteralTimesLiteral() { - Expression expr = parser.createExpression("10*2"); + Expression expr = parser.createExpression("10 *2"); assertEquals(Mul.class, expr.getClass()); Mul mul = (Mul) expr; - assertEquals("10 * 2", mul.name()); + assertEquals("10 *2", mul.sourceText()); assertEquals(DataType.INTEGER, mul.dataType()); } @@ -193,39 +192,46 @@ public void testFunctionTimesLiteral() { Expression expr = parser.createExpression("PI()*2"); assertEquals(Mul.class, expr.getClass()); Mul mul = (Mul) expr; - assertEquals("(PI) * 2", mul.name()); + assertEquals("PI()*2", mul.sourceText()); + } + + public void testNegativeLiteral() { + Expression expr = parser.createExpression("- 6"); + assertEquals(Literal.class, expr.getClass()); + assertEquals("- 6", expr.sourceText()); } public void testComplexArithmetic() { - Expression expr = parser.createExpression("-(((a-2)-(-3))+b)"); + String sql = "-(((a-2)-(-3))+b)"; + Expression expr = parser.createExpression(sql); assertEquals(Neg.class, expr.getClass()); Neg neg = (Neg) expr; - assertThat(neg.name(), startsWith("-(((a) - 2) - -3) + (b)#")); + assertEquals(sql, neg.sourceText()); assertEquals(1, neg.children().size()); assertEquals(Add.class, neg.children().get(0).getClass()); Add add = (Add) neg.children().get(0); - assertEquals("(((a) - 2) - -3) + (b)", add.name()); + assertEquals("((a-2)-(-3))+b", add.sourceText()); assertEquals(2, add.children().size()); assertEquals("?b", add.children().get(1).toString()); assertEquals(Sub.class, add.children().get(0).getClass()); Sub sub1 = (Sub) add.children().get(0); - assertEquals("((a) - 2) - -3", sub1.name()); + assertEquals("(a-2)-(-3)", sub1.sourceText()); assertEquals(2, sub1.children().size()); assertEquals(Literal.class, sub1.children().get(1).getClass()); - assertEquals("-3", ((Literal) sub1.children().get(1)).name()); + assertEquals("-3", ((Literal) sub1.children().get(1)).sourceText()); assertEquals(Sub.class, sub1.children().get(0).getClass()); Sub sub2 = (Sub) sub1.children().get(0); assertEquals(2, sub2.children().size()); assertEquals("?a", sub2.children().get(0).toString()); assertEquals(Literal.class, sub2.children().get(1).getClass()); - assertEquals("2", ((Literal) sub2.children().get(1)).name()); + assertEquals("2", ((Literal) sub2.children().get(1)).sourceText()); } public void testEquals() { Expression expr = parser.createExpression("a = 10"); assertEquals(Equals.class, expr.getClass()); Equals eq = (Equals) expr; - assertEquals("(a) == 10", eq.name()); + assertEquals("a = 10", eq.sourceText()); assertEquals(2, eq.children().size()); } @@ -233,7 +239,7 @@ public void testNullEquals() { Expression expr = parser.createExpression("a <=> 10"); assertEquals(NullEquals.class, expr.getClass()); NullEquals nullEquals = (NullEquals) expr; - assertEquals("(a) <=> 10", nullEquals.name()); + assertEquals("a <=> 10", nullEquals.sourceText()); assertEquals(2, nullEquals.children().size()); } @@ -241,12 +247,12 @@ public void testNotEquals() { Expression expr = parser.createExpression("a != 10"); assertEquals(NotEquals.class, expr.getClass()); NotEquals neq = (NotEquals) expr; - assertEquals("(a) != 10", neq.name()); + assertEquals("a != 10", neq.sourceText()); assertEquals(2, neq.children().size()); } public void testCastWithUnquotedDataType() { - Expression expr = parser.createExpression("CAST(10*2 AS long)"); + Expression expr = parser.createExpression("CAST(10* 2 AS long)"); assertEquals(Cast.class, expr.getClass()); Cast cast = (Cast) expr; assertEquals(DataType.INTEGER, cast.from()); @@ -254,7 +260,7 @@ public void testCastWithUnquotedDataType() { assertEquals(DataType.LONG, cast.dataType()); assertEquals(Mul.class, cast.field().getClass()); Mul mul = (Mul) cast.field(); - assertEquals("10 * 2", mul.name()); + assertEquals("10* 2", mul.sourceText()); assertEquals(DataType.INTEGER, mul.dataType()); } @@ -267,7 +273,7 @@ public void testCastWithQuotedDataType() { assertEquals(DataType.LONG, cast.dataType()); assertEquals(Mul.class, cast.field().getClass()); Mul mul = (Mul) cast.field(); - assertEquals("10 * 2", mul.name()); + assertEquals("10*2", mul.sourceText()); assertEquals(DataType.INTEGER, mul.dataType()); } @@ -285,20 +291,22 @@ public void testConvertWithUnquotedDataType() { assertEquals(DataType.LONG, cast.dataType()); assertEquals(Mul.class, cast.field().getClass()); Mul mul = (Mul) cast.field(); - assertEquals("10 * 2", mul.name()); + assertEquals("10*2", mul.sourceText()); assertEquals(DataType.INTEGER, mul.dataType()); } public void testConvertWithQuotedDataType() { - Expression expr = parser.createExpression("CONVERT(10*2, \"LonG\")"); + String e = "CONVERT(10*2, \"LonG\")"; + Expression expr = parser.createExpression(e); assertEquals(Cast.class, expr.getClass()); Cast cast = (Cast) expr; + assertEquals(e, cast.sourceText()); assertEquals(DataType.INTEGER, cast.from()); assertEquals(DataType.LONG, cast.to()); assertEquals(DataType.LONG, cast.dataType()); assertEquals(Mul.class, cast.field().getClass()); Mul mul = (Mul) cast.field(); - assertEquals("10 * 2", mul.name()); + assertEquals("10*2", mul.sourceText()); assertEquals(DataType.INTEGER, mul.dataType()); } @@ -334,7 +342,7 @@ public void testCurrentTimestamp() { Expression expr = parser.createExpression("CURRENT_TIMESTAMP"); assertEquals(UnresolvedFunction.class, expr.getClass()); UnresolvedFunction ur = (UnresolvedFunction) expr; - assertEquals("CURRENT_TIMESTAMP", ur.name()); + assertEquals("CURRENT_TIMESTAMP", ur.sourceText()); assertEquals(0, ur.children().size()); } @@ -342,7 +350,7 @@ public void testCurrentTimestampPrecision() { Expression expr = parser.createExpression("CURRENT_TIMESTAMP(4)"); assertEquals(UnresolvedFunction.class, expr.getClass()); UnresolvedFunction ur = (UnresolvedFunction) expr; - assertEquals("CURRENT_TIMESTAMP", ur.name()); + assertEquals("CURRENT_TIMESTAMP(4)", ur.sourceText()); assertEquals(1, ur.children().size()); Expression child = ur.children().get(0); assertEquals(Literal.class, child.getClass()); diff --git a/x-pack/plugin/sql/src/test/java/org/elasticsearch/xpack/sql/parser/SqlParserTests.java b/x-pack/plugin/sql/src/test/java/org/elasticsearch/xpack/sql/parser/SqlParserTests.java index 199b4e119d81b..a29feff881637 100644 --- a/x-pack/plugin/sql/src/test/java/org/elasticsearch/xpack/sql/parser/SqlParserTests.java +++ b/x-pack/plugin/sql/src/test/java/org/elasticsearch/xpack/sql/parser/SqlParserTests.java @@ -6,6 +6,7 @@ package org.elasticsearch.xpack.sql.parser; import com.google.common.base.Joiner; + import org.elasticsearch.test.ESTestCase; import org.elasticsearch.xpack.sql.expression.NamedExpression; import org.elasticsearch.xpack.sql.expression.Order; @@ -15,6 +16,7 @@ import org.elasticsearch.xpack.sql.expression.predicate.fulltext.MatchQueryPredicate; import org.elasticsearch.xpack.sql.expression.predicate.fulltext.MultiMatchQueryPredicate; import org.elasticsearch.xpack.sql.expression.predicate.fulltext.StringQueryPredicate; +import org.elasticsearch.xpack.sql.expression.predicate.operator.arithmetic.Add; import org.elasticsearch.xpack.sql.expression.predicate.operator.comparison.In; import org.elasticsearch.xpack.sql.parser.SqlBaseParser.BooleanExpressionContext; import org.elasticsearch.xpack.sql.parser.SqlBaseParser.QueryPrimaryDefaultContext; @@ -60,12 +62,17 @@ public void testSelectField() { public void testSelectScore() { UnresolvedFunction f = singleProjection(project(parseStatement("SELECT SCORE() FROM foo")), UnresolvedFunction.class); - assertEquals("SCORE", f.functionName()); + assertEquals("SCORE()", f.sourceText()); + } + + public void testSelectAddWithParanthesis() { + Add f = singleProjection(project(parseStatement("SELECT (1 + 2)")), Add.class); + assertEquals("1 + 2", f.sourceText()); } public void testSelectRightFunction() { UnresolvedFunction f = singleProjection(project(parseStatement("SELECT RIGHT()")), UnresolvedFunction.class); - assertEquals("RIGHT", f.functionName()); + assertEquals("RIGHT()", f.sourceText()); } public void testsSelectNonReservedKeywords() { @@ -102,13 +109,13 @@ public void testOrderByField() { public void testOrderByScore() { Order.OrderDirection dir = randomFrom(Order.OrderDirection.values()); - OrderBy ob = orderBy(parseStatement("SELECT * FROM foo ORDER BY SCORE()" + stringForDirection(dir))); + OrderBy ob = orderBy(parseStatement("SELECT * FROM foo ORDER BY SCORE( )" + stringForDirection(dir))); assertThat(ob.order(), hasSize(1)); Order o = ob.order().get(0); assertEquals(dir, o.direction()); assertThat(o.child(), instanceOf(UnresolvedFunction.class)); UnresolvedFunction f = (UnresolvedFunction) o.child(); - assertEquals("SCORE", f.functionName()); + assertEquals("SCORE( )", f.sourceText()); } public void testOrderByTwo() { @@ -299,7 +306,7 @@ public void testLimitStackOverflowForInAndLiteralsIsNotApplied() { In in = (In) filter.condition(); assertEquals("?a", in.value().toString()); assertEquals(noChildren, in.list().size()); - assertThat(in.list().get(0).toString(), startsWith("(a) + (b)#")); + assertThat(in.list().get(0).toString(), startsWith("a + b#")); } public void testDecrementOfDepthCounter() { diff --git a/x-pack/plugin/sql/src/test/java/org/elasticsearch/xpack/sql/planner/QueryFolderTests.java b/x-pack/plugin/sql/src/test/java/org/elasticsearch/xpack/sql/planner/QueryFolderTests.java index c20f4e9d632af..214d935251994 100644 --- a/x-pack/plugin/sql/src/test/java/org/elasticsearch/xpack/sql/planner/QueryFolderTests.java +++ b/x-pack/plugin/sql/src/test/java/org/elasticsearch/xpack/sql/planner/QueryFolderTests.java @@ -76,7 +76,7 @@ public void testLocalExecWithPrunedFilterWithFunction() { assertEquals(EmptyExecutable.class, le.executable().getClass()); EmptyExecutable ee = (EmptyExecutable) le.executable(); assertEquals(1, ee.output().size()); - assertThat(ee.output().get(0).toString(), startsWith("E{c}#")); + assertThat(ee.output().get(0).toString(), startsWith("E(){c}#")); } public void testLocalExecWithPrunedFilterWithFunctionAndAggregation() { @@ -86,7 +86,7 @@ public void testLocalExecWithPrunedFilterWithFunctionAndAggregation() { assertEquals(EmptyExecutable.class, le.executable().getClass()); EmptyExecutable ee = (EmptyExecutable) le.executable(); assertEquals(1, ee.output().size()); - assertThat(ee.output().get(0).toString(), startsWith("E{c}#")); + assertThat(ee.output().get(0).toString(), startsWith("E(){c}#")); } public void testLocalExecWithoutFromClause() { @@ -96,9 +96,9 @@ public void testLocalExecWithoutFromClause() { assertEquals(SingletonExecutable.class, le.executable().getClass()); SingletonExecutable ee = (SingletonExecutable) le.executable(); assertEquals(3, ee.output().size()); - assertThat(ee.output().get(0).toString(), startsWith("E{c}#")); - assertThat(ee.output().get(1).toString(), startsWith("foo{c}#")); - assertThat(ee.output().get(2).toString(), startsWith("ABS(10){c}#")); + assertThat(ee.output().get(0).toString(), startsWith("E(){c}#")); + assertThat(ee.output().get(1).toString(), startsWith("'foo'{c}#")); + assertThat(ee.output().get(2).toString(), startsWith("abs(10){c}#")); } public void testLocalExecWithoutFromClauseWithPrunedFilter() { @@ -108,7 +108,7 @@ public void testLocalExecWithoutFromClauseWithPrunedFilter() { assertEquals(EmptyExecutable.class, le.executable().getClass()); EmptyExecutable ee = (EmptyExecutable) le.executable(); assertEquals(1, ee.output().size()); - assertThat(ee.output().get(0).toString(), startsWith("E{c}#")); + assertThat(ee.output().get(0).toString(), startsWith("E(){c}#")); } public void testFoldingOfIsNull() { @@ -137,7 +137,7 @@ public void testFoldingToLocalExecBooleanAndNull_HavingClause() { EmptyExecutable ee = (EmptyExecutable) le.executable(); assertEquals(2, ee.output().size()); assertThat(ee.output().get(0).toString(), startsWith("keyword{f}#")); - assertThat(ee.output().get(1).toString(), startsWith("MAX(int){a->")); + assertThat(ee.output().get(1).toString(), startsWith("max(int){a->")); } public void testFoldingBooleanOrNull_WhereClause() { @@ -159,7 +159,7 @@ public void testFoldingBooleanOrNull_HavingClause() { "\"lang\":\"painless\",\"params\":{\"v0\":10}},")); assertEquals(2, ee.output().size()); assertThat(ee.output().get(0).toString(), startsWith("keyword{f}#")); - assertThat(ee.output().get(1).toString(), startsWith("MAX(int){a->")); + assertThat(ee.output().get(1).toString(), startsWith("max(int){a->")); } public void testFoldingOfIsNotNull() { @@ -208,7 +208,7 @@ public void testFoldingToLocalExecWithProjectWithGroupBy_WithOrderAndLimit() { EmptyExecutable ee = (EmptyExecutable) le.executable(); assertEquals(2, ee.output().size()); assertThat(ee.output().get(0).toString(), startsWith("keyword{f}#")); - assertThat(ee.output().get(1).toString(), startsWith("MAX(int){a->")); + assertThat(ee.output().get(1).toString(), startsWith("max(int){a->")); } public void testFoldingToLocalExecWithProjectWithGroupBy_WithHaving_WithOrderAndLimit() { @@ -219,7 +219,7 @@ public void testFoldingToLocalExecWithProjectWithGroupBy_WithHaving_WithOrderAnd EmptyExecutable ee = (EmptyExecutable) le.executable(); assertEquals(2, ee.output().size()); assertThat(ee.output().get(0).toString(), startsWith("keyword{f}#")); - assertThat(ee.output().get(1).toString(), startsWith("MAX(int){a->")); + assertThat(ee.output().get(1).toString(), startsWith("max(int){a->")); } public void testGroupKeyTypes_Boolean() { @@ -232,7 +232,7 @@ public void testGroupKeyTypes_Boolean() { "\"lang\":\"painless\",\"params\":{\"v0\":\"int\",\"v1\":10}},\"missing_bucket\":true," + "\"value_type\":\"boolean\",\"order\":\"asc\"}}}]}}}")); assertEquals(2, ee.output().size()); - assertThat(ee.output().get(0).toString(), startsWith("COUNT(1){a->")); + assertThat(ee.output().get(0).toString(), startsWith("count(*){a->")); assertThat(ee.output().get(1).toString(), startsWith("a{s->")); } @@ -246,7 +246,7 @@ public void testGroupKeyTypes_Integer() { "\"lang\":\"painless\",\"params\":{\"v0\":\"int\",\"v1\":10}},\"missing_bucket\":true," + "\"value_type\":\"long\",\"order\":\"asc\"}}}]}}}")); assertEquals(2, ee.output().size()); - assertThat(ee.output().get(0).toString(), startsWith("COUNT(1){a->")); + assertThat(ee.output().get(0).toString(), startsWith("count(*){a->")); assertThat(ee.output().get(1).toString(), startsWith("a{s->")); } @@ -260,7 +260,7 @@ public void testGroupKeyTypes_Rational() { "\"lang\":\"painless\",\"params\":{\"v0\":\"int\"}},\"missing_bucket\":true," + "\"value_type\":\"double\",\"order\":\"asc\"}}}]}}}")); assertEquals(2, ee.output().size()); - assertThat(ee.output().get(0).toString(), startsWith("COUNT(1){a->")); + assertThat(ee.output().get(0).toString(), startsWith("count(*){a->")); assertThat(ee.output().get(1).toString(), startsWith("a{s->")); } @@ -274,7 +274,7 @@ public void testGroupKeyTypes_String() { "\"lang\":\"painless\",\"params\":{\"v0\":\"keyword\"}},\"missing_bucket\":true," + "\"value_type\":\"string\",\"order\":\"asc\"}}}]}}}")); assertEquals(2, ee.output().size()); - assertThat(ee.output().get(0).toString(), startsWith("COUNT(1){a->")); + assertThat(ee.output().get(0).toString(), startsWith("count(*){a->")); assertThat(ee.output().get(1).toString(), startsWith("a{s->")); } @@ -288,7 +288,7 @@ public void testGroupKeyTypes_IP() { "\"lang\":\"painless\",\"params\":{\"v0\":\"keyword\",\"v1\":\"IP\"}}," + "\"missing_bucket\":true,\"value_type\":\"ip\",\"order\":\"asc\"}}}]}}}")); assertEquals(2, ee.output().size()); - assertThat(ee.output().get(0).toString(), startsWith("COUNT(1){a->")); + assertThat(ee.output().get(0).toString(), startsWith("count(*){a->")); assertThat(ee.output().get(1).toString(), startsWith("a{s->")); } @@ -303,7 +303,7 @@ public void testGroupKeyTypes_Date() { "\"v0\":\"date\",\"v1\":\"P1Y2M\",\"v2\":\"INTERVAL_YEAR_TO_MONTH\"}},\"missing_bucket\":true," + "\"value_type\":\"date\",\"order\":\"asc\"}}}]}}}")); assertEquals(2, ee.output().size()); - assertThat(ee.output().get(0).toString(), startsWith("COUNT(1){a->")); + assertThat(ee.output().get(0).toString(), startsWith("count(*){a->")); assertThat(ee.output().get(1).toString(), startsWith("a{s->")); } diff --git a/x-pack/plugin/sql/src/test/java/org/elasticsearch/xpack/sql/planner/QueryTranslatorTests.java b/x-pack/plugin/sql/src/test/java/org/elasticsearch/xpack/sql/planner/QueryTranslatorTests.java index 8ca1d64bd12fd..23beae2fd586e 100644 --- a/x-pack/plugin/sql/src/test/java/org/elasticsearch/xpack/sql/planner/QueryTranslatorTests.java +++ b/x-pack/plugin/sql/src/test/java/org/elasticsearch/xpack/sql/planner/QueryTranslatorTests.java @@ -277,7 +277,7 @@ public void testTranslateIsNullExpression_HavingClause_Painless() { AggFilter aggFilter = translation.aggFilter; assertEquals("InternalSqlScriptUtils.nullSafeFilter(InternalSqlScriptUtils.isNull(params.a0))", aggFilter.scriptTemplate().toString()); - assertThat(aggFilter.scriptTemplate().params().toString(), startsWith("[{a=MAX(int){a->")); + assertThat(aggFilter.scriptTemplate().params().toString(), startsWith("[{a=max(int){a->")); } public void testTranslateIsNotNullExpression_HavingClause_Painless() { @@ -290,7 +290,7 @@ public void testTranslateIsNotNullExpression_HavingClause_Painless() { AggFilter aggFilter = translation.aggFilter; assertEquals("InternalSqlScriptUtils.nullSafeFilter(InternalSqlScriptUtils.isNotNull(params.a0))", aggFilter.scriptTemplate().toString()); - assertThat(aggFilter.scriptTemplate().params().toString(), startsWith("[{a=MAX(int){a->")); + assertThat(aggFilter.scriptTemplate().params().toString(), startsWith("[{a=max(int){a->")); } public void testTranslateInExpression_WhereClause() { @@ -328,8 +328,10 @@ public void testTranslateInExpressionInvalidValues_WhereClause() { Expression condition = ((Filter) p.children().get(0)).condition(); assertFalse(condition.foldable()); SqlIllegalArgumentException ex = expectThrows(SqlIllegalArgumentException.class, () -> QueryTranslator.toQuery(condition, false)); - assertEquals("Line 1:52: Comparisons against variables are not (currently) supported; " + - "offender [keyword] in [keyword IN (foo, bar, keyword)]", ex.getMessage()); + assertEquals( + "Line 1:52: Comparisons against variables are not (currently) supported; " + + "offender [keyword] in [keyword IN ('foo', 'bar', keyword)]", + ex.getMessage()); } public void testTranslateInExpression_WhereClause_Painless() { @@ -358,7 +360,7 @@ public void testTranslateInExpression_HavingClause_Painless() { AggFilter aggFilter = translation.aggFilter; assertEquals("InternalSqlScriptUtils.nullSafeFilter(InternalSqlScriptUtils.in(params.a0, params.v0))", aggFilter.scriptTemplate().toString()); - assertThat(aggFilter.scriptTemplate().params().toString(), startsWith("[{a=MAX(int){a->")); + assertThat(aggFilter.scriptTemplate().params().toString(), startsWith("[{a=max(int){a->")); assertThat(aggFilter.scriptTemplate().params().toString(), endsWith(", {v=[10, 20]}]")); } @@ -372,7 +374,7 @@ public void testTranslateInExpression_HavingClause_PainlessOneArg() { AggFilter aggFilter = translation.aggFilter; assertEquals("InternalSqlScriptUtils.nullSafeFilter(InternalSqlScriptUtils.in(params.a0, params.v0))", aggFilter.scriptTemplate().toString()); - assertThat(aggFilter.scriptTemplate().params().toString(), startsWith("[{a=MAX(int){a->")); + assertThat(aggFilter.scriptTemplate().params().toString(), startsWith("[{a=max(int){a->")); assertThat(aggFilter.scriptTemplate().params().toString(), endsWith(", {v=[10]}]")); } @@ -387,7 +389,7 @@ public void testTranslateInExpression_HavingClause_PainlessAndNullHandling() { AggFilter aggFilter = translation.aggFilter; assertEquals("InternalSqlScriptUtils.nullSafeFilter(InternalSqlScriptUtils.in(params.a0, params.v0))", aggFilter.scriptTemplate().toString()); - assertThat(aggFilter.scriptTemplate().params().toString(), startsWith("[{a=MAX(int){a->")); + assertThat(aggFilter.scriptTemplate().params().toString(), startsWith("[{a=max(int){a->")); assertThat(aggFilter.scriptTemplate().params().toString(), endsWith(", {v=[10, null, 20, 30]}]")); } @@ -406,7 +408,7 @@ public void testTranslateMathFunction_HavingClause_Painless() { assertEquals("InternalSqlScriptUtils.nullSafeFilter(InternalSqlScriptUtils.gt(InternalSqlScriptUtils." + operation.name().toLowerCase(Locale.ROOT) + "(params.a0),params.v0))", aggFilter.scriptTemplate().toString()); - assertThat(aggFilter.scriptTemplate().params().toString(), startsWith("[{a=MAX(int){a->")); + assertThat(aggFilter.scriptTemplate().params().toString(), startsWith("[{a=max(int){a->")); assertThat(aggFilter.scriptTemplate().params().toString(), endsWith(", {v=10}]")); } diff --git a/x-pack/plugin/sql/src/test/java/org/elasticsearch/xpack/sql/type/DataTypeConversionTests.java b/x-pack/plugin/sql/src/test/java/org/elasticsearch/xpack/sql/type/DataTypeConversionTests.java index 4aa3cba3d7f18..a44ce44d0f904 100644 --- a/x-pack/plugin/sql/src/test/java/org/elasticsearch/xpack/sql/type/DataTypeConversionTests.java +++ b/x-pack/plugin/sql/src/test/java/org/elasticsearch/xpack/sql/type/DataTypeConversionTests.java @@ -9,12 +9,13 @@ import org.elasticsearch.xpack.sql.SqlIllegalArgumentException; import org.elasticsearch.xpack.sql.TestUtils; import org.elasticsearch.xpack.sql.expression.Literal; +import org.elasticsearch.xpack.sql.tree.Location; +import org.elasticsearch.xpack.sql.tree.Source; import org.elasticsearch.xpack.sql.type.DataTypeConversion.Conversion; import java.time.ZonedDateTime; import static org.elasticsearch.xpack.sql.expression.function.scalar.datetime.DateTimeTestUtils.dateTime; -import static org.elasticsearch.xpack.sql.tree.Source.EMPTY; import static org.elasticsearch.xpack.sql.type.DataType.BOOLEAN; import static org.elasticsearch.xpack.sql.type.DataType.BYTE; import static org.elasticsearch.xpack.sql.type.DataType.DATE; @@ -291,9 +292,10 @@ public void testStringToIp() { } public void testIpToString() { + Source s = new Source(Location.EMPTY, "10.0.0.1"); Conversion ipToString = conversionFor(IP, KEYWORD); - assertEquals("10.0.0.1", ipToString.convert(new Literal(EMPTY, "10.0.0.1", IP))); + assertEquals("10.0.0.1", ipToString.convert(new Literal(s, "10.0.0.1", IP))); Conversion stringToIp = conversionFor(KEYWORD, IP); - assertEquals("10.0.0.1", ipToString.convert(stringToIp.convert(Literal.of(EMPTY, "10.0.0.1")))); + assertEquals("10.0.0.1", ipToString.convert(stringToIp.convert(Literal.of(s, "10.0.0.1")))); } } From 96cfa000a50a6bcee33298e2d2afd9e560684a8a Mon Sep 17 00:00:00 2001 From: Peter Dyson Date: Sun, 13 Jan 2019 16:24:34 +1000 Subject: [PATCH 131/186] [DOCS] copy_to only works one level deep, not recursively (#37249) --- docs/reference/mapping/params/copy-to.asciidoc | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/docs/reference/mapping/params/copy-to.asciidoc b/docs/reference/mapping/params/copy-to.asciidoc index 447d7baf274f4..6a1aef62bc391 100644 --- a/docs/reference/mapping/params/copy-to.asciidoc +++ b/docs/reference/mapping/params/copy-to.asciidoc @@ -61,3 +61,7 @@ Some important points: * It is the field _value_ which is copied, not the terms (which result from the analysis process). * The original <> field will not be modified to show the copied values. * The same value can be copied to multiple fields, with `"copy_to": [ "field_1", "field_2" ]` +* You cannot copy recursively via intermediary fields such as a `copy_to` on +`field_1` to `field_2` and `copy_to` on `field_2` to `field_3` expecting +indexing into `field_1` will eventuate in `field_3`, instead use copy_to +directly to multiple fields from the originating field. \ No newline at end of file From 6fd57d90dad0df06aece223e5e8e672b8d0bc908 Mon Sep 17 00:00:00 2001 From: Albert Zaharovits Date: Sun, 13 Jan 2019 15:26:23 +0200 Subject: [PATCH 132/186] Security Audit includes HTTP method for requests (#37322) Adds another field, named "request.method", to the structured logfile audit. This field is present for all events associated with a REST request (not a transport request) and the value is one of GET, POST, PUT, DELETE, OPTIONS, HEAD, PATCH, TRACE and CONNECT. --- .../en/security/auditing/event-types.asciidoc | 3 +++ .../security/auditing/output-logfile.asciidoc | 3 +-- .../core/src/main/config/log4j2.properties | 2 ++ .../audit/logfile/LoggingAuditTrail.java | 18 ++++++++++-------- .../audit/logfile/LoggingAuditTrailTests.java | 7 +++++++ 5 files changed, 23 insertions(+), 10 deletions(-) diff --git a/x-pack/docs/en/security/auditing/event-types.asciidoc b/x-pack/docs/en/security/auditing/event-types.asciidoc index 442877f31b5d3..417b26cbd09b3 100644 --- a/x-pack/docs/en/security/auditing/event-types.asciidoc +++ b/x-pack/docs/en/security/auditing/event-types.asciidoc @@ -118,6 +118,9 @@ common ones): This is URL encoded. `url.query` :: The query part of the URL (after "?", if present) of the REST request associated with this event. This is URL encoded. +`request.method` :: The HTTP method of the REST request associated with this + event. It is one of GET, POST, PUT, DELETE, OPTIONS, + HEAD, PATCH, TRACE and CONNECT. `request.body` :: The full content of the REST request associated with this event, if enabled. This contains the query body. The body is escaped according to the JSON RFC 4627. diff --git a/x-pack/docs/en/security/auditing/output-logfile.asciidoc b/x-pack/docs/en/security/auditing/output-logfile.asciidoc index d9e7eb81c3f3a..382307e893f6d 100644 --- a/x-pack/docs/en/security/auditing/output-logfile.asciidoc +++ b/x-pack/docs/en/security/auditing/output-logfile.asciidoc @@ -18,8 +18,7 @@ For more information, see {ref}/logging.html#configuring-logging-levels[configur [[audit-log-entry-format]] === Log entry format -The log entries in the `_audit.log` file -have the following format: +The log entries in the `_audit.log` file have the following format: - Each log entry is a one line JSON document and each one is printed on a separate line. - The fields of a log entry are ordered. However, if a field does not have a value it diff --git a/x-pack/plugin/core/src/main/config/log4j2.properties b/x-pack/plugin/core/src/main/config/log4j2.properties index 21b0732fed418..52b6ce3950690 100644 --- a/x-pack/plugin/core/src/main/config/log4j2.properties +++ b/x-pack/plugin/core/src/main/config/log4j2.properties @@ -22,6 +22,7 @@ appender.audit_rolling.layout.pattern = {\ %varsNotEmpty{, "realm":"%enc{%map{realm}}{JSON}"}\ %varsNotEmpty{, "url.path":"%enc{%map{url.path}}{JSON}"}\ %varsNotEmpty{, "url.query":"%enc{%map{url.query}}{JSON}"}\ + %varsNotEmpty{, "request.method":"%enc{%map{request.method}}{JSON}"}\ %varsNotEmpty{, "request.body":"%enc{%map{request.body}}{JSON}"}\ %varsNotEmpty{, "request.id":"%enc{%map{request.id}}{JSON}"}\ %varsNotEmpty{, "action":"%enc{%map{action}}{JSON}"}\ @@ -51,6 +52,7 @@ appender.audit_rolling.layout.pattern = {\ # "realm" name of a realm that has generated an "authentication_failed" or an "authentication_successful"; the subject is not yet authenticated # "url.path" the URI component between the port and the query string; it is percent (URL) encoded # "url.query" the URI component after the path and before the fragment; it is percent (URL) encoded +# "request.method" the method of the HTTP request, i.e. one of GET, POST, PUT, DELETE, OPTIONS, HEAD, PATCH, TRACE, CONNECT # "request.body" the content of the request body entity, JSON escaped # "request.id" a synthentic identifier for the incoming request, this is unique per incoming request, and consistent across all audit events generated by that request # "action" an action is the most granular operation that is authorized and this identifies it in a namespaced way (internal) diff --git a/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/audit/logfile/LoggingAuditTrail.java b/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/audit/logfile/LoggingAuditTrail.java index cdeee882c1bda..045140e331f28 100644 --- a/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/audit/logfile/LoggingAuditTrail.java +++ b/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/audit/logfile/LoggingAuditTrail.java @@ -100,6 +100,7 @@ public class LoggingAuditTrail implements AuditTrail, ClusterStateListener { public static final String REALM_FIELD_NAME = "realm"; public static final String URL_PATH_FIELD_NAME = "url.path"; public static final String URL_QUERY_FIELD_NAME = "url.query"; + public static final String REQUEST_METHOD_FIELD_NAME = "request.method"; public static final String REQUEST_BODY_FIELD_NAME = "request.body"; public static final String REQUEST_ID_FIELD_NAME = "request.id"; public static final String ACTION_FIELD_NAME = "action"; @@ -211,7 +212,7 @@ public void authenticationSuccess(String requestId, String realm, User user, Res .with(EVENT_TYPE_FIELD_NAME, REST_ORIGIN_FIELD_VALUE) .with(EVENT_ACTION_FIELD_NAME, "authentication_success") .with(REALM_FIELD_NAME, realm) - .withRestUri(request) + .withRestUriAndMethod(request) .withRequestId(requestId) .withPrincipal(user) .withRestOrigin(request) @@ -276,7 +277,7 @@ public void anonymousAccessDenied(String requestId, RestRequest request) { final StringMapMessage logEntry = new LogEntryBuilder() .with(EVENT_TYPE_FIELD_NAME, REST_ORIGIN_FIELD_VALUE) .with(EVENT_ACTION_FIELD_NAME, "anonymous_access_denied") - .withRestUri(request) + .withRestUriAndMethod(request) .withRestOrigin(request) .withRequestBody(request) .withRequestId(requestId) @@ -316,7 +317,7 @@ public void authenticationFailed(String requestId, RestRequest request) { final StringMapMessage logEntry = new LogEntryBuilder() .with(EVENT_TYPE_FIELD_NAME, REST_ORIGIN_FIELD_VALUE) .with(EVENT_ACTION_FIELD_NAME, "authentication_failed") - .withRestUri(request) + .withRestUriAndMethod(request) .withRestOrigin(request) .withRequestBody(request) .withRequestId(requestId) @@ -357,7 +358,7 @@ public void authenticationFailed(String requestId, AuthenticationToken token, Re .with(EVENT_TYPE_FIELD_NAME, REST_ORIGIN_FIELD_VALUE) .with(EVENT_ACTION_FIELD_NAME, "authentication_failed") .with(PRINCIPAL_FIELD_NAME, token.principal()) - .withRestUri(request) + .withRestUriAndMethod(request) .withRestOrigin(request) .withRequestBody(request) .withRequestId(requestId) @@ -401,7 +402,7 @@ public void authenticationFailed(String requestId, String realm, AuthenticationT .with(EVENT_ACTION_FIELD_NAME, "realm_authentication_failed") .with(REALM_FIELD_NAME, realm) .with(PRINCIPAL_FIELD_NAME, token.principal()) - .withRestUri(request) + .withRestUriAndMethod(request) .withRestOrigin(request) .withRequestBody(request) .withRequestId(requestId) @@ -468,7 +469,7 @@ public void tamperedRequest(String requestId, RestRequest request) { final StringMapMessage logEntry = new LogEntryBuilder() .with(EVENT_TYPE_FIELD_NAME, REST_ORIGIN_FIELD_VALUE) .with(EVENT_ACTION_FIELD_NAME, "tampered_request") - .withRestUri(request) + .withRestUriAndMethod(request) .withRestOrigin(request) .withRequestBody(request) .withRequestId(requestId) @@ -617,7 +618,7 @@ public void runAsDenied(String requestId, Authentication authentication, RestReq .with(EVENT_TYPE_FIELD_NAME, REST_ORIGIN_FIELD_VALUE) .with(EVENT_ACTION_FIELD_NAME, "run_as_denied") .with(PRINCIPAL_ROLES_FIELD_NAME, roleNames) - .withRestUri(request) + .withRestUriAndMethod(request) .withRunAsSubject(authentication) .withRestOrigin(request) .withRequestBody(request) @@ -637,7 +638,7 @@ private class LogEntryBuilder { logEntry = new StringMapMessage(LoggingAuditTrail.this.entryCommonFields.commonFields); } - LogEntryBuilder withRestUri(RestRequest request) { + LogEntryBuilder withRestUriAndMethod(RestRequest request) { final int queryStringIndex = request.uri().indexOf('?'); int queryStringLength = request.uri().indexOf('#'); if (queryStringLength < 0) { @@ -651,6 +652,7 @@ LogEntryBuilder withRestUri(RestRequest request) { if (queryStringIndex > -1) { logEntry.with(URL_QUERY_FIELD_NAME, request.uri().substring(queryStringIndex + 1, queryStringLength)); } + logEntry.with(REQUEST_METHOD_FIELD_NAME, request.method().toString()); return this; } diff --git a/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/audit/logfile/LoggingAuditTrailTests.java b/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/audit/logfile/LoggingAuditTrailTests.java index da4823a3f3a69..817ed2a2358d0 100644 --- a/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/audit/logfile/LoggingAuditTrailTests.java +++ b/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/audit/logfile/LoggingAuditTrailTests.java @@ -245,6 +245,7 @@ public void testAnonymousAccessDeniedRest() throws Exception { .put(LoggingAuditTrail.EVENT_ACTION_FIELD_NAME, "anonymous_access_denied") .put(LoggingAuditTrail.ORIGIN_TYPE_FIELD_NAME, LoggingAuditTrail.REST_ORIGIN_FIELD_VALUE) .put(LoggingAuditTrail.ORIGIN_ADDRESS_FIELD_NAME, NetworkAddress.format(address)) + .put(LoggingAuditTrail.REQUEST_METHOD_FIELD_NAME, request.method().toString()) .put(LoggingAuditTrail.REQUEST_BODY_FIELD_NAME, includeRequestBody && Strings.hasLength(expectedMessage) ? expectedMessage : null) .put(LoggingAuditTrail.REQUEST_ID_FIELD_NAME, requestId) @@ -346,6 +347,7 @@ public void testAuthenticationFailedRest() throws Exception { .put(LoggingAuditTrail.PRINCIPAL_FIELD_NAME, mockToken.principal()) .put(LoggingAuditTrail.ORIGIN_TYPE_FIELD_NAME, LoggingAuditTrail.REST_ORIGIN_FIELD_VALUE) .put(LoggingAuditTrail.ORIGIN_ADDRESS_FIELD_NAME, NetworkAddress.format(address)) + .put(LoggingAuditTrail.REQUEST_METHOD_FIELD_NAME, request.method().toString()) .put(LoggingAuditTrail.REQUEST_BODY_FIELD_NAME, includeRequestBody && Strings.hasLength(expectedMessage) ? expectedMessage : null) .put(LoggingAuditTrail.REQUEST_ID_FIELD_NAME, requestId) @@ -386,6 +388,7 @@ public void testAuthenticationFailedRestNoToken() throws Exception { .put(LoggingAuditTrail.PRINCIPAL_FIELD_NAME, null) .put(LoggingAuditTrail.ORIGIN_TYPE_FIELD_NAME, LoggingAuditTrail.REST_ORIGIN_FIELD_VALUE) .put(LoggingAuditTrail.ORIGIN_ADDRESS_FIELD_NAME, NetworkAddress.format(address)) + .put(LoggingAuditTrail.REQUEST_METHOD_FIELD_NAME, request.method().toString()) .put(LoggingAuditTrail.REQUEST_BODY_FIELD_NAME, includeRequestBody && Strings.hasLength(expectedMessage) ? expectedMessage : null) .put(LoggingAuditTrail.REQUEST_ID_FIELD_NAME, requestId) @@ -468,6 +471,7 @@ public void testAuthenticationFailedRealmRest() throws Exception { .put(LoggingAuditTrail.ORIGIN_ADDRESS_FIELD_NAME, NetworkAddress.format(address)) .put(LoggingAuditTrail.PRINCIPAL_FIELD_NAME, mockToken.principal()) .put(LoggingAuditTrail.ACTION_FIELD_NAME, null) + .put(LoggingAuditTrail.REQUEST_METHOD_FIELD_NAME, request.method().toString()) .put(LoggingAuditTrail.REQUEST_BODY_FIELD_NAME, includeRequestBody && Strings.hasLength(expectedMessage) ? expectedMessage : null) .put(LoggingAuditTrail.REQUEST_ID_FIELD_NAME, requestId) @@ -627,6 +631,7 @@ public void testTamperedRequestRest() throws Exception { .put(LoggingAuditTrail.EVENT_ACTION_FIELD_NAME, "tampered_request") .put(LoggingAuditTrail.ORIGIN_TYPE_FIELD_NAME, LoggingAuditTrail.REST_ORIGIN_FIELD_VALUE) .put(LoggingAuditTrail.ORIGIN_ADDRESS_FIELD_NAME, NetworkAddress.format(address)) + .put(LoggingAuditTrail.REQUEST_METHOD_FIELD_NAME, request.method().toString()) .put(LoggingAuditTrail.REQUEST_BODY_FIELD_NAME, includeRequestBody && Strings.hasLength(expectedMessage) ? expectedMessage : null) .put(LoggingAuditTrail.REQUEST_ID_FIELD_NAME, requestId) @@ -891,6 +896,7 @@ public void testAuthenticationSuccessRest() throws Exception { .put(LoggingAuditTrail.REALM_FIELD_NAME, realm) .put(LoggingAuditTrail.ORIGIN_TYPE_FIELD_NAME, LoggingAuditTrail.REST_ORIGIN_FIELD_VALUE) .put(LoggingAuditTrail.ORIGIN_ADDRESS_FIELD_NAME, NetworkAddress.format(address)) + .put(LoggingAuditTrail.REQUEST_METHOD_FIELD_NAME, request.method().toString()) .put(LoggingAuditTrail.REQUEST_BODY_FIELD_NAME, includeRequestBody && Strings.hasLength(expectedMessage) ? expectedMessage : null) .put(LoggingAuditTrail.REQUEST_ID_FIELD_NAME, requestId) @@ -1080,6 +1086,7 @@ private Tuple prepareRestContent(String uri, InetSocke } builder.withRemoteAddress(remoteAddress); builder.withParams(params); + builder.withMethod(randomFrom(RestRequest.Method.values())); return new Tuple<>(content, builder.build()); } From a566bacbc8e263a95e0975ff2639e095d4a74af8 Mon Sep 17 00:00:00 2001 From: Alpar Torok Date: Sun, 13 Jan 2019 19:33:39 +0200 Subject: [PATCH 133/186] Upgrade ASM for java 12 compatability (#37385) Closes #37371 --- x-pack/test/feature-aware/build.gradle | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/x-pack/test/feature-aware/build.gradle b/x-pack/test/feature-aware/build.gradle index 11b0e67183c8f..7e7ed3ee3a862 100644 --- a/x-pack/test/feature-aware/build.gradle +++ b/x-pack/test/feature-aware/build.gradle @@ -1,7 +1,7 @@ apply plugin: 'elasticsearch.build' dependencies { - compile 'org.ow2.asm:asm:6.2' + compile 'org.ow2.asm:asm:7.0' compile "org.elasticsearch:elasticsearch:${version}" compile "org.elasticsearch.plugin:x-pack-core:${version}" testCompile "org.elasticsearch.test:framework:${version}" From d54f88f62c1ab9f0d25f90c49b8184a1e0eb4a65 Mon Sep 17 00:00:00 2001 From: Luca Cavanna Date: Mon, 14 Jan 2019 08:32:45 +0100 Subject: [PATCH 134/186] Remove unused empty constructors from suggestions classes (#37295) We recently migrated suggestions to `Writeable`. That allows us to also clean up empty constructors and methods that called them as they are no longer needed. They are replaced by constructors that accept a `StreamInput` instance. --- .../customsuggester/CustomSuggestion.java | 12 ------- .../elasticsearch/search/suggest/Suggest.java | 33 +++---------------- .../completion/CompletionSuggestion.java | 26 +++------------ .../suggest/phrase/PhraseSuggestion.java | 20 +---------- .../search/suggest/term/TermSuggestion.java | 22 +------------ .../search/SearchModuleTests.java | 7 +--- 6 files changed, 12 insertions(+), 108 deletions(-) diff --git a/plugins/examples/custom-suggester/src/main/java/org/elasticsearch/example/customsuggester/CustomSuggestion.java b/plugins/examples/custom-suggester/src/main/java/org/elasticsearch/example/customsuggester/CustomSuggestion.java index f7ec27b7af002..05c7880e9ec24 100644 --- a/plugins/examples/custom-suggester/src/main/java/org/elasticsearch/example/customsuggester/CustomSuggestion.java +++ b/plugins/examples/custom-suggester/src/main/java/org/elasticsearch/example/customsuggester/CustomSuggestion.java @@ -77,11 +77,6 @@ public String getDummy() { return dummy; } - @Override - protected Entry newEntry() { - return new Entry(); - } - @Override protected Entry newEntry(StreamInput in) throws IOException { return new Entry(in); @@ -123,11 +118,6 @@ public void writeTo(StreamOutput out) throws IOException { out.writeString(dummy); } - @Override - protected Option newOption() { - return new Option(); - } - @Override protected Option newOption(StreamInput in) throws IOException { return new Option(in); @@ -178,8 +168,6 @@ public static class Option extends Suggest.Suggestion.Entry.Option { private String dummy; - public Option() {} - public Option(Text text, float score, String dummy) { super(text, score); this.dummy = dummy; diff --git a/server/src/main/java/org/elasticsearch/search/suggest/Suggest.java b/server/src/main/java/org/elasticsearch/search/suggest/Suggest.java index b2f2a336d3684..8d78116e15a15 100644 --- a/server/src/main/java/org/elasticsearch/search/suggest/Suggest.java +++ b/server/src/main/java/org/elasticsearch/search/suggest/Suggest.java @@ -33,7 +33,6 @@ import org.elasticsearch.common.xcontent.ObjectParser; import org.elasticsearch.common.xcontent.ToXContentFragment; import org.elasticsearch.common.xcontent.XContentBuilder; -import org.elasticsearch.common.xcontent.XContentFactory; import org.elasticsearch.common.xcontent.XContentParser; import org.elasticsearch.common.xcontent.XContentParserUtils; import org.elasticsearch.rest.action.search.RestSearchAction; @@ -46,7 +45,6 @@ import java.io.IOException; import java.util.ArrayList; -import java.util.Collections; import java.util.Comparator; import java.util.HashMap; import java.util.Iterator; @@ -78,10 +76,6 @@ public class Suggest implements Iterable>> suggestMap; - private Suggest() { - this(Collections.emptyList()); - } - public Suggest(List>> suggestions) { // we sort suggestions by their names to ensure iteration over suggestions are consistent // this is needed as we need to fill in suggestion docs in SearchPhaseController#sortDocs @@ -259,16 +253,11 @@ public int hashCode() { */ public abstract static class Suggestion implements Iterable, NamedWriteable, ToXContentFragment { - private static final String NAME = "suggestion"; - public static final int TYPE = 0; - protected String name; - protected int size; + protected final String name; + protected final int size; protected final List entries = new ArrayList<>(5); - protected Suggestion() { - } - public Suggestion(String name, int size) { this.name = name; this.size = size; // The suggested term size specified in request, only used for merging shard responses @@ -380,7 +369,6 @@ public void trim() { } } - protected abstract T newEntry(); protected abstract T newEntry(StreamInput in) throws IOException; @Override @@ -608,7 +596,6 @@ public int hashCode() { return Objects.hash(text, offset, length, options); } - protected abstract O newOption(); protected abstract O newOption(StreamInput in) throws IOException; @Override @@ -653,8 +640,8 @@ public abstract static class Option implements Writeable, ToXContentFragment { public static final ParseField SCORE = new ParseField("score"); public static final ParseField COLLATE_MATCH = new ParseField("collate_match"); - private Text text; - private Text highlighted; + private final Text text; + private final Text highlighted; private float score; private Boolean collateMatch; @@ -673,8 +660,6 @@ public Option(Text text, float score) { this(text, null, score); } - public Option() {} - public Option(StreamInput in) throws IOException { text = in.readText(); score = in.readFloat(); @@ -776,14 +761,6 @@ public int hashCode() { @Override public String toString() { - try { - XContentBuilder builder = XContentFactory.jsonBuilder().prettyPrint(); - builder.startObject(); - toXContent(builder, EMPTY_PARAMS); - builder.endObject(); - return Strings.toString(builder); - } catch (IOException e) { - return "{ \"error\" : \"" + e.getMessage() + "\"}"; - } + return Strings.toString(this, true, true); } } diff --git a/server/src/main/java/org/elasticsearch/search/suggest/completion/CompletionSuggestion.java b/server/src/main/java/org/elasticsearch/search/suggest/completion/CompletionSuggestion.java index 5dea2d8c1a0ef..0fb7e4b3f43f2 100644 --- a/server/src/main/java/org/elasticsearch/search/suggest/completion/CompletionSuggestion.java +++ b/server/src/main/java/org/elasticsearch/search/suggest/completion/CompletionSuggestion.java @@ -71,11 +71,8 @@ public final class CompletionSuggestion extends Suggest.Suggestion> contexts = Collections.emptyMap(); - private ScoreDoc doc; + private final Map> contexts; + private final ScoreDoc doc; private SearchHit hit; public static final ParseField CONTEXTS = new ParseField("contexts"); @@ -292,10 +279,6 @@ public Option(int docID, Text text, float score, Map> contex this.contexts = Objects.requireNonNull(contexts, "context map cannot be null"); } - protected Option() { - super(); - } - public Option(StreamInput in) throws IOException { super(in); this.doc = Lucene.readScoreDoc(in); @@ -455,5 +438,4 @@ public String toString() { } } } - } diff --git a/server/src/main/java/org/elasticsearch/search/suggest/phrase/PhraseSuggestion.java b/server/src/main/java/org/elasticsearch/search/suggest/phrase/PhraseSuggestion.java index 39932049dfd8f..a29c6f08a7d8b 100644 --- a/server/src/main/java/org/elasticsearch/search/suggest/phrase/PhraseSuggestion.java +++ b/server/src/main/java/org/elasticsearch/search/suggest/phrase/PhraseSuggestion.java @@ -24,7 +24,6 @@ import org.elasticsearch.common.io.stream.StreamOutput; import org.elasticsearch.common.text.Text; import org.elasticsearch.common.xcontent.ConstructingObjectParser; -import org.elasticsearch.common.xcontent.ContextParser; import org.elasticsearch.common.xcontent.ObjectParser; import org.elasticsearch.common.xcontent.XContentParser; import org.elasticsearch.search.suggest.Suggest; @@ -44,8 +43,6 @@ public class PhraseSuggestion extends Suggest.Suggestion @Deprecated public static final int TYPE = 3; - public PhraseSuggestion() {} - public PhraseSuggestion(String name, int size) { super(name, size); } @@ -64,11 +61,6 @@ public int getWriteableType() { return TYPE; } - @Override - protected Entry newEntry() { - return new Entry(); - } - @Override protected Entry newEntry(StreamInput in) throws IOException { return new Entry(in); @@ -132,19 +124,13 @@ public void addOption(Option option) { static { declareCommonFields(PARSER); - PARSER.declareObjectArray(Entry::addOptions, (ContextParser) (p, c) -> Option.fromXContent(p), - new ParseField(OPTIONS)); + PARSER.declareObjectArray(Entry::addOptions, (p, c) -> Option.fromXContent(p), new ParseField(OPTIONS)); } public static Entry fromXContent(XContentParser parser) { return PARSER.apply(parser, null); } - @Override - protected Option newOption() { - return new Option(); - } - @Override protected Option newOption(StreamInput in) throws IOException { return new Option(in); @@ -169,10 +155,6 @@ public int hashCode() { public static class Option extends Suggestion.Entry.Option { - public Option() { - super(); - } - public Option(Text text, Text highlighted, float score, Boolean collateMatch) { super(text, highlighted, score, collateMatch); } diff --git a/server/src/main/java/org/elasticsearch/search/suggest/term/TermSuggestion.java b/server/src/main/java/org/elasticsearch/search/suggest/term/TermSuggestion.java index b8e334f92835c..a1ffdd5670940 100644 --- a/server/src/main/java/org/elasticsearch/search/suggest/term/TermSuggestion.java +++ b/server/src/main/java/org/elasticsearch/search/suggest/term/TermSuggestion.java @@ -52,8 +52,6 @@ public class TermSuggestion extends Suggestion { private SortBy sort; - public TermSuggestion() {} - public TermSuggestion(String name, int size, SortBy sort) { super(name, size); this.sort = sort; @@ -61,7 +59,6 @@ public TermSuggestion(String name, int size, SortBy sort) { public TermSuggestion(StreamInput in) throws IOException { super(in); - if (in.getVersion().onOrAfter(Version.V_7_0_0)) { sort = SortBy.readFromStream(in); } @@ -80,7 +77,6 @@ public int compare(Suggestion.Entry.Option first, Suggestion.Entry.Option second } return FREQUENCY.compare(first, second); } - } // Same behaviour as comparators in suggest module, but for SuggestedWord @@ -105,7 +101,6 @@ public int compare(Suggestion.Entry.Option first, Suggestion.Entry.Option second // third criteria: term text return first.getText().compareTo(second.getText()); } - } @Override @@ -154,11 +149,6 @@ public static TermSuggestion fromXContent(XContentParser parser, String name) th return suggestion; } - @Override - protected Entry newEntry() { - return new Entry(); - } - @Override protected Entry newEntry(StreamInput in) throws IOException { return new Entry(in); @@ -184,17 +174,12 @@ public Entry(Text text, int offset, int length) { super(text, offset, length); } - public Entry() {} + private Entry() {} public Entry(StreamInput in) throws IOException { super(in); } - @Override - protected Option newOption() { - return new Option(); - } - @Override protected Option newOption(StreamInput in) throws IOException { return new Option(in); @@ -236,10 +221,6 @@ protected void mergeInto(Suggestion.Entry.Option otherOption) { freq += ((Option) otherOption).freq; } - protected Option() { - super(); - } - public void setFreq(int freq) { this.freq = freq; } @@ -283,6 +264,5 @@ public static Option fromXContent(XContentParser parser) { return PARSER.apply(parser, null); } } - } } diff --git a/server/src/test/java/org/elasticsearch/search/SearchModuleTests.java b/server/src/test/java/org/elasticsearch/search/SearchModuleTests.java index 321d50278b8be..f2d250fa1f80f 100644 --- a/server/src/test/java/org/elasticsearch/search/SearchModuleTests.java +++ b/server/src/test/java/org/elasticsearch/search/SearchModuleTests.java @@ -44,11 +44,11 @@ import org.elasticsearch.search.aggregations.bucket.significant.heuristics.SignificanceHeuristicParser; import org.elasticsearch.search.aggregations.bucket.terms.TermsAggregationBuilder; import org.elasticsearch.search.aggregations.pipeline.AbstractPipelineAggregationBuilder; -import org.elasticsearch.search.aggregations.pipeline.PipelineAggregator; import org.elasticsearch.search.aggregations.pipeline.DerivativePipelineAggregationBuilder; import org.elasticsearch.search.aggregations.pipeline.DerivativePipelineAggregator; import org.elasticsearch.search.aggregations.pipeline.InternalDerivative; import org.elasticsearch.search.aggregations.pipeline.MovAvgModel; +import org.elasticsearch.search.aggregations.pipeline.PipelineAggregator; import org.elasticsearch.search.aggregations.pipeline.SimpleModel; import org.elasticsearch.search.aggregations.support.ValuesSource; import org.elasticsearch.search.aggregations.support.ValuesSourceAggregationBuilder; @@ -579,11 +579,6 @@ private static class TestSuggestion extends Suggestion { super(in); } - @Override - protected Entry newEntry() { - return null; - } - @Override protected Entry newEntry(StreamInput in) throws IOException { return null; From 374e24c7fd09dd80437138e6ca25803abd8469f3 Mon Sep 17 00:00:00 2001 From: Ioannis Kakavas Date: Mon, 14 Jan 2019 10:50:24 +0200 Subject: [PATCH 135/186] Mute SslNullCipherTests on JDK12 JDK12 doesn't support NULL cipher for TLS by default. This commit mutes these tests on JDK12 until we decide whether we need to keep or remove them --- .../xpack/security/transport/ssl/SslNullCipherTests.java | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/transport/ssl/SslNullCipherTests.java b/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/transport/ssl/SslNullCipherTests.java index 7427c5a67e92d..f57be31632868 100644 --- a/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/transport/ssl/SslNullCipherTests.java +++ b/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/transport/ssl/SslNullCipherTests.java @@ -7,6 +7,7 @@ import org.elasticsearch.action.DocWriteResponse.Result; import org.elasticsearch.action.index.IndexResponse; +import org.elasticsearch.bootstrap.JavaVersion; import org.elasticsearch.client.Client; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.test.SecurityIntegTestCase; @@ -22,6 +23,11 @@ public static void muteInFips() { assumeFalse("Can't run in a FIPS JVM", inFipsJvm()); } + @BeforeClass + public static void muteInJDK12() { + assumeFalse("https://github.com/elastic/elasticsearch/issues/37403", JavaVersion.current().equals(JavaVersion.parse("12"))); + } + @Override public boolean transportSSLEnabled() { return true; From d44a6f9fbcc4f71116a5b6eb0573a786d67a58ae Mon Sep 17 00:00:00 2001 From: Nhat Nguyen Date: Mon, 14 Jan 2019 03:54:34 -0500 Subject: [PATCH 136/186] Simplify SyncedFlushService flow with StepListener (#37383) Today the SyncedFlushService flow is written with multiple nested callbacks which are hard to read. This commit replaces them with sequential step listeners. --- .../elasticsearch/action/StepListener.java | 3 +- .../indices/flush/SyncedFlushService.java | 81 ++++++++----------- 2 files changed, 35 insertions(+), 49 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/action/StepListener.java b/server/src/main/java/org/elasticsearch/action/StepListener.java index efbf8c755d57c..160ba23da246f 100644 --- a/server/src/main/java/org/elasticsearch/action/StepListener.java +++ b/server/src/main/java/org/elasticsearch/action/StepListener.java @@ -24,6 +24,7 @@ import org.elasticsearch.common.util.concurrent.FutureUtils; import org.elasticsearch.common.util.concurrent.ListenableFuture; +import java.util.concurrent.TimeUnit; import java.util.function.Consumer; /** @@ -84,6 +85,6 @@ public Response result() { if (delegate.isDone() == false) { throw new IllegalStateException("step is not completed yet"); } - return FutureUtils.get(delegate); + return FutureUtils.get(delegate, 0L, TimeUnit.NANOSECONDS); // this future is done already - use a non-blocking method. } } diff --git a/server/src/main/java/org/elasticsearch/indices/flush/SyncedFlushService.java b/server/src/main/java/org/elasticsearch/indices/flush/SyncedFlushService.java index e1cd85faaef5d..9bc4e4ead1269 100644 --- a/server/src/main/java/org/elasticsearch/indices/flush/SyncedFlushService.java +++ b/server/src/main/java/org/elasticsearch/indices/flush/SyncedFlushService.java @@ -24,6 +24,7 @@ import org.elasticsearch.ElasticsearchException; import org.elasticsearch.Version; import org.elasticsearch.action.ActionListener; +import org.elasticsearch.action.StepListener; import org.elasticsearch.action.admin.indices.flush.FlushRequest; import org.elasticsearch.action.admin.indices.flush.SyncedFlushResponse; import org.elasticsearch.action.support.IndicesOptions; @@ -219,56 +220,40 @@ private void innerAttemptSyncedFlush(final ShardId shardId, return; } - final ActionListener> presyncListener = - new ActionListener>() { - @Override - public void onResponse(final Map presyncResponses) { - if (presyncResponses.isEmpty()) { - actionListener.onResponse(new ShardsSyncedFlushResult(shardId, totalShards, - "all shards failed to commit on pre-sync")); - return; - } - final ActionListener inflightOpsListener = new ActionListener() { - @Override - public void onResponse(InFlightOpsResponse response) { - final int inflight = response.opCount(); - assert inflight >= 0; - if (inflight != 0) { - actionListener.onResponse(new ShardsSyncedFlushResult(shardId, totalShards, "[" + inflight + - "] ongoing operations on primary")); - } else { - // 3. now send the sync request to all the shards; - final String sharedSyncId = sharedExistingSyncId(presyncResponses); - if (sharedSyncId != null) { - assert presyncResponses.values().stream().allMatch(r -> r.existingSyncId.equals(sharedSyncId)) : - "Not all shards have the same existing sync id [" + sharedSyncId + "], responses [" + - presyncResponses + "]"; - reportSuccessWithExistingSyncId(shardId, sharedSyncId, activeShards, totalShards, - presyncResponses, actionListener); - }else { - String syncId = UUIDs.randomBase64UUID(); - sendSyncRequests(syncId, activeShards, state, presyncResponses, shardId, totalShards, actionListener); - } - } - } - - @Override - public void onFailure(Exception e) { - actionListener.onFailure(e); - } - }; - // 2. fetch in flight operations - getInflightOpsCount(shardId, state, shardRoutingTable, inflightOpsListener); + // 1. send pre-sync flushes to all replicas + final StepListener> presyncStep = new StepListener<>(); + sendPreSyncRequests(activeShards, state, shardId, presyncStep); + + // 2. fetch in flight operations + final StepListener inflightOpsStep = new StepListener<>(); + presyncStep.whenComplete(presyncResponses -> { + if (presyncResponses.isEmpty()) { + actionListener.onResponse(new ShardsSyncedFlushResult(shardId, totalShards, "all shards failed to commit on pre-sync")); + } else { + getInflightOpsCount(shardId, state, shardRoutingTable, inflightOpsStep); } - - @Override - public void onFailure(Exception e) { - actionListener.onFailure(e); + }, actionListener::onFailure); + + // 3. now send the sync request to all the shards + inflightOpsStep.whenComplete(inFlightOpsResponse -> { + final Map presyncResponses = presyncStep.result(); + final int inflight = inFlightOpsResponse.opCount(); + assert inflight >= 0; + if (inflight != 0) { + actionListener.onResponse( + new ShardsSyncedFlushResult(shardId, totalShards, "[" + inflight + "] ongoing operations on primary")); + } else { + final String sharedSyncId = sharedExistingSyncId(presyncResponses); + if (sharedSyncId != null) { + assert presyncResponses.values().stream().allMatch(r -> r.existingSyncId.equals(sharedSyncId)) : + "Not all shards have the same existing sync id [" + sharedSyncId + "], responses [" + presyncResponses + "]"; + reportSuccessWithExistingSyncId(shardId, sharedSyncId, activeShards, totalShards, presyncResponses, actionListener); + }else { + String syncId = UUIDs.randomBase64UUID(); + sendSyncRequests(syncId, activeShards, state, presyncResponses, shardId, totalShards, actionListener); + } } - }; - - // 1. send pre-sync flushes to all replicas - sendPreSyncRequests(activeShards, state, shardId, presyncListener); + }, actionListener::onFailure); } catch (Exception e) { actionListener.onFailure(e); } From 89b45f1fc6ed7bb46df5071f7ad34d80e6e87761 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Christoph=20B=C3=BCscher?= Date: Mon, 14 Jan 2019 11:14:38 +0100 Subject: [PATCH 137/186] Remove deprecated pipeline request contructors (#37366) The constructors in PutPipelineRequest and SimulatePipelineRequest that guess the xContent type from the provided source are deprecated since 6.0 and each have a counterpart that takes the xContent type as an explicit argument. Removing these ctors together with the builders and methods in ClusterAdminClient that don't have the xContent type as argument. --- .../action/ingest/PutPipelineRequest.java | 10 ---------- .../action/ingest/PutPipelineRequestBuilder.java | 5 ----- .../action/ingest/SimulatePipelineRequest.java | 13 +------------ .../ingest/SimulatePipelineRequestBuilder.java | 10 ---------- .../elasticsearch/client/ClusterAdminClient.java | 13 ------------- .../client/support/AbstractClient.java | 10 ---------- .../elasticsearch/common/io/UTF8StreamWriter.java | 8 -------- 7 files changed, 1 insertion(+), 68 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/action/ingest/PutPipelineRequest.java b/server/src/main/java/org/elasticsearch/action/ingest/PutPipelineRequest.java index abff28bcf553c..46a34717c89c0 100644 --- a/server/src/main/java/org/elasticsearch/action/ingest/PutPipelineRequest.java +++ b/server/src/main/java/org/elasticsearch/action/ingest/PutPipelineRequest.java @@ -26,7 +26,6 @@ import org.elasticsearch.common.io.stream.StreamOutput; import org.elasticsearch.common.xcontent.ToXContentObject; import org.elasticsearch.common.xcontent.XContentBuilder; -import org.elasticsearch.common.xcontent.XContentHelper; import org.elasticsearch.common.xcontent.XContentType; import java.io.IOException; @@ -38,15 +37,6 @@ public class PutPipelineRequest extends AcknowledgedRequest private BytesReference source; private XContentType xContentType; - /** - * Create a new pipeline request - * @deprecated use {@link #PutPipelineRequest(String, BytesReference, XContentType)} to avoid content type auto-detection - */ - @Deprecated - public PutPipelineRequest(String id, BytesReference source) { - this(id, source, XContentHelper.xContentType(source)); - } - /** * Create a new pipeline request with the id and source along with the content type of the source */ diff --git a/server/src/main/java/org/elasticsearch/action/ingest/PutPipelineRequestBuilder.java b/server/src/main/java/org/elasticsearch/action/ingest/PutPipelineRequestBuilder.java index 1919d98c7e1f9..aa2aff8bb9915 100644 --- a/server/src/main/java/org/elasticsearch/action/ingest/PutPipelineRequestBuilder.java +++ b/server/src/main/java/org/elasticsearch/action/ingest/PutPipelineRequestBuilder.java @@ -31,11 +31,6 @@ public PutPipelineRequestBuilder(ElasticsearchClient client, PutPipelineAction a super(client, action, new PutPipelineRequest()); } - @Deprecated - public PutPipelineRequestBuilder(ElasticsearchClient client, PutPipelineAction action, String id, BytesReference source) { - super(client, action, new PutPipelineRequest(id, source)); - } - public PutPipelineRequestBuilder(ElasticsearchClient client, PutPipelineAction action, String id, BytesReference source, XContentType xContentType) { super(client, action, new PutPipelineRequest(id, source, xContentType)); diff --git a/server/src/main/java/org/elasticsearch/action/ingest/SimulatePipelineRequest.java b/server/src/main/java/org/elasticsearch/action/ingest/SimulatePipelineRequest.java index 7514a41f5756b..4c2736e3d86de 100644 --- a/server/src/main/java/org/elasticsearch/action/ingest/SimulatePipelineRequest.java +++ b/server/src/main/java/org/elasticsearch/action/ingest/SimulatePipelineRequest.java @@ -26,11 +26,11 @@ import org.elasticsearch.common.io.stream.StreamOutput; import org.elasticsearch.common.xcontent.ToXContentObject; import org.elasticsearch.common.xcontent.XContentBuilder; -import org.elasticsearch.common.xcontent.XContentHelper; import org.elasticsearch.common.xcontent.XContentType; import org.elasticsearch.index.VersionType; import org.elasticsearch.ingest.ConfigurationUtils; import org.elasticsearch.ingest.IngestDocument; +import org.elasticsearch.ingest.IngestDocument.MetaData; import org.elasticsearch.ingest.IngestService; import org.elasticsearch.ingest.Pipeline; @@ -41,8 +41,6 @@ import java.util.Map; import java.util.Objects; -import static org.elasticsearch.ingest.IngestDocument.MetaData; - public class SimulatePipelineRequest extends ActionRequest implements ToXContentObject { private String id; @@ -50,15 +48,6 @@ public class SimulatePipelineRequest extends ActionRequest implements ToXContent private BytesReference source; private XContentType xContentType; - /** - * Create a new request - * @deprecated use {@link #SimulatePipelineRequest(BytesReference, XContentType)} that does not attempt content autodetection - */ - @Deprecated - public SimulatePipelineRequest(BytesReference source) { - this(source, XContentHelper.xContentType(source)); - } - /** * Creates a new request with the given source and its content type */ diff --git a/server/src/main/java/org/elasticsearch/action/ingest/SimulatePipelineRequestBuilder.java b/server/src/main/java/org/elasticsearch/action/ingest/SimulatePipelineRequestBuilder.java index 7b7d31603aa6b..1a5e863bd55ee 100644 --- a/server/src/main/java/org/elasticsearch/action/ingest/SimulatePipelineRequestBuilder.java +++ b/server/src/main/java/org/elasticsearch/action/ingest/SimulatePipelineRequestBuilder.java @@ -33,16 +33,6 @@ public SimulatePipelineRequestBuilder(ElasticsearchClient client, SimulatePipeli super(client, action, new SimulatePipelineRequest()); } - /** - * Create a new builder for {@link SimulatePipelineRequest}s - * @deprecated use {@link #SimulatePipelineRequestBuilder(ElasticsearchClient, SimulatePipelineAction, BytesReference, XContentType)} to - * avoid content type auto-detection on the source bytes - */ - @Deprecated - public SimulatePipelineRequestBuilder(ElasticsearchClient client, SimulatePipelineAction action, BytesReference source) { - super(client, action, new SimulatePipelineRequest(source)); - } - /** * Create a new builder for {@link SimulatePipelineRequest}s */ diff --git a/server/src/main/java/org/elasticsearch/client/ClusterAdminClient.java b/server/src/main/java/org/elasticsearch/client/ClusterAdminClient.java index 624c79996371a..42aaed10d6172 100644 --- a/server/src/main/java/org/elasticsearch/client/ClusterAdminClient.java +++ b/server/src/main/java/org/elasticsearch/client/ClusterAdminClient.java @@ -576,13 +576,6 @@ public interface ClusterAdminClient extends ElasticsearchClient { */ ActionFuture putPipeline(PutPipelineRequest request); - /** - * Stores an ingest pipeline - * @deprecated use {@link #preparePutPipeline(String, BytesReference, XContentType)} - */ - @Deprecated - PutPipelineRequestBuilder preparePutPipeline(String id, BytesReference source); - /** * Stores an ingest pipeline */ @@ -633,12 +626,6 @@ public interface ClusterAdminClient extends ElasticsearchClient { */ ActionFuture simulatePipeline(SimulatePipelineRequest request); - /** - * Simulates an ingest pipeline - */ - @Deprecated - SimulatePipelineRequestBuilder prepareSimulatePipeline(BytesReference source); - /** * Simulates an ingest pipeline */ diff --git a/server/src/main/java/org/elasticsearch/client/support/AbstractClient.java b/server/src/main/java/org/elasticsearch/client/support/AbstractClient.java index d642101e1c3e9..3fc931a85c0f7 100644 --- a/server/src/main/java/org/elasticsearch/client/support/AbstractClient.java +++ b/server/src/main/java/org/elasticsearch/client/support/AbstractClient.java @@ -1066,11 +1066,6 @@ public ActionFuture putPipeline(PutPipelineRequest request return execute(PutPipelineAction.INSTANCE, request); } - @Override - public PutPipelineRequestBuilder preparePutPipeline(String id, BytesReference source) { - return new PutPipelineRequestBuilder(this, PutPipelineAction.INSTANCE, id, source); - } - @Override public PutPipelineRequestBuilder preparePutPipeline(String id, BytesReference source, XContentType xContentType) { return new PutPipelineRequestBuilder(this, PutPipelineAction.INSTANCE, id, source, xContentType); @@ -1121,11 +1116,6 @@ public ActionFuture simulatePipeline(SimulatePipelineR return execute(SimulatePipelineAction.INSTANCE, request); } - @Override - public SimulatePipelineRequestBuilder prepareSimulatePipeline(BytesReference source) { - return new SimulatePipelineRequestBuilder(this, SimulatePipelineAction.INSTANCE, source); - } - @Override public SimulatePipelineRequestBuilder prepareSimulatePipeline(BytesReference source, XContentType xContentType) { return new SimulatePipelineRequestBuilder(this, SimulatePipelineAction.INSTANCE, source, xContentType); diff --git a/server/src/main/java/org/elasticsearch/common/io/UTF8StreamWriter.java b/server/src/main/java/org/elasticsearch/common/io/UTF8StreamWriter.java index 0ed5fd498de91..cb038d12fc2ec 100644 --- a/server/src/main/java/org/elasticsearch/common/io/UTF8StreamWriter.java +++ b/server/src/main/java/org/elasticsearch/common/io/UTF8StreamWriter.java @@ -326,12 +326,4 @@ public void reset() { _index = 0; _outputStream = null; } - - /** - * @deprecated Replaced by {@link #setOutput(OutputStream)} - */ - @Deprecated - public UTF8StreamWriter setOutputStream(OutputStream out) { - return this.setOutput(out); - } } \ No newline at end of file From 4b1368183d741bb1f5818d24b638eb38e0954d24 Mon Sep 17 00:00:00 2001 From: Jiyu-Zhang-Zendesk <36034626+Jiyu-Zhang-Zendesk@users.noreply.github.com> Date: Mon, 14 Jan 2019 11:20:25 +0100 Subject: [PATCH 138/186] Update analysis.asciidoc (#37404) STConvert plugin is made by Medcl to convert between Simplified Chinese and Traditional Chinese. It's widely used by the Search Community for Chinese --- docs/plugins/analysis.asciidoc | 1 + 1 file changed, 1 insertion(+) diff --git a/docs/plugins/analysis.asciidoc b/docs/plugins/analysis.asciidoc index 875c87124ef45..0a0bbf090495f 100644 --- a/docs/plugins/analysis.asciidoc +++ b/docs/plugins/analysis.asciidoc @@ -58,6 +58,7 @@ A number of analysis plugins have been contributed by our community: * https://github.com/ofir123/elasticsearch-network-analysis[Network Addresses Analysis Plugin] (by Ofir123) * https://github.com/medcl/elasticsearch-analysis-string2int[String2Integer Analysis Plugin] (by Medcl) * https://github.com/ZarHenry96/elasticsearch-dandelion-plugin[Dandelion Analysis Plugin] (by ZarHenry96) +* https://github.com/medcl/elasticsearch-analysis-stconvert[STConvert Analysis Plugin] (by Medcl) include::analysis-icu.asciidoc[] From 87f91485805d0f3eb54dd6eacaac5994f6faa090 Mon Sep 17 00:00:00 2001 From: Georgi Ivanov Date: Mon, 14 Jan 2019 13:03:00 +0100 Subject: [PATCH 139/186] Update the scroll example in the docs (#37394) Update the scroll example ascii and Java docs, so it is more clear when to consume the scroll documents. Before this change the user could loose the first results if one uses copy & paste. --- .../client/documentation/SearchDocumentationIT.java | 4 ++-- docs/java-rest/high-level/search/scroll.asciidoc | 4 ++-- 2 files changed, 4 insertions(+), 4 deletions(-) diff --git a/client/rest-high-level/src/test/java/org/elasticsearch/client/documentation/SearchDocumentationIT.java b/client/rest-high-level/src/test/java/org/elasticsearch/client/documentation/SearchDocumentationIT.java index dc85e85e9b29d..3a49a13479e87 100644 --- a/client/rest-high-level/src/test/java/org/elasticsearch/client/documentation/SearchDocumentationIT.java +++ b/client/rest-high-level/src/test/java/org/elasticsearch/client/documentation/SearchDocumentationIT.java @@ -715,12 +715,12 @@ public void onFailure(Exception e) { SearchHit[] searchHits = searchResponse.getHits().getHits(); while (searchHits != null && searchHits.length > 0) { // <2> - SearchScrollRequest scrollRequest = new SearchScrollRequest(scrollId); // <3> + // <3> + SearchScrollRequest scrollRequest = new SearchScrollRequest(scrollId); // <4> scrollRequest.scroll(scroll); searchResponse = client.scroll(scrollRequest, RequestOptions.DEFAULT); scrollId = searchResponse.getScrollId(); searchHits = searchResponse.getHits().getHits(); - // <4> } ClearScrollRequest clearScrollRequest = new ClearScrollRequest(); // <5> diff --git a/docs/java-rest/high-level/search/scroll.asciidoc b/docs/java-rest/high-level/search/scroll.asciidoc index 8a6d9830f88d6..8285243103abe 100644 --- a/docs/java-rest/high-level/search/scroll.asciidoc +++ b/docs/java-rest/high-level/search/scroll.asciidoc @@ -128,9 +128,9 @@ include-tagged::{doc-tests}/SearchDocumentationIT.java[search-scroll-example] <1> Initialize the search context by sending the initial `SearchRequest` <2> Retrieve all the search hits by calling the Search Scroll api in a loop until no documents are returned -<3> Create a new `SearchScrollRequest` holding the last returned scroll +<3> Process the returned search results +<4> Create a new `SearchScrollRequest` holding the last returned scroll identifier and the scroll interval -<4> Process the returned search results <5> Clear the scroll context once the scroll is completed [[java-rest-high-clear-scroll]] From 6ca076bf7467f13d7807b089f549ac429b45e8fc Mon Sep 17 00:00:00 2001 From: Tanguy Leroux Date: Mon, 14 Jan 2019 13:13:15 +0100 Subject: [PATCH 140/186] Fix ClusterBlock serialization and Close Index API logic after backport to 6.x (#37360) This commit changes the versions in the serialization logic of ClusterBlock after the backport to 6.x of the Close Index API refactoring (#37359). --- .../java/org/elasticsearch/cluster/block/ClusterBlock.java | 4 ++-- .../cluster/metadata/MetaDataIndexStateService.java | 2 +- .../org/elasticsearch/cluster/block/ClusterBlockTests.java | 6 +++--- 3 files changed, 6 insertions(+), 6 deletions(-) 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 5713462b9212f..497e296d9eeb6 100644 --- a/server/src/main/java/org/elasticsearch/cluster/block/ClusterBlock.java +++ b/server/src/main/java/org/elasticsearch/cluster/block/ClusterBlock.java @@ -138,7 +138,7 @@ 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)) { + if (in.getVersion().onOrAfter(Version.V_6_7_0)) { uuid = in.readOptionalString(); } else { uuid = null; @@ -159,7 +159,7 @@ 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)) { + if (out.getVersion().onOrAfter(Version.V_6_7_0)) { out.writeOptionalString(uuid); } out.writeString(description); 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 aa4434a0a74c6..0781cab1fe757 100644 --- a/server/src/main/java/org/elasticsearch/cluster/metadata/MetaDataIndexStateService.java +++ b/server/src/main/java/org/elasticsearch/cluster/metadata/MetaDataIndexStateService.java @@ -225,7 +225,7 @@ static ClusterState addIndexClosedBlocks(final Index[] indices, final Map 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))); From 07dc8c7eee8f329c81e514c61c5b6edaaa1ad890 Mon Sep 17 00:00:00 2001 From: Tanguy Leroux Date: Mon, 14 Jan 2019 13:14:36 +0100 Subject: [PATCH 141/186] Improve CloseWhileRelocatingShardsIT (#37348) --- .../state/CloseWhileRelocatingShardsIT.java | 159 +++++++++++------- 1 file changed, 100 insertions(+), 59 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 165360c35972d..555bf24335413 100644 --- a/server/src/test/java/org/elasticsearch/indices/state/CloseWhileRelocatingShardsIT.java +++ b/server/src/test/java/org/elasticsearch/indices/state/CloseWhileRelocatingShardsIT.java @@ -18,29 +18,40 @@ */ package org.elasticsearch.indices.state; -import org.elasticsearch.action.index.IndexResponse; +import org.elasticsearch.action.admin.cluster.reroute.ClusterRerouteRequest; import org.elasticsearch.action.support.master.AcknowledgedResponse; -import org.elasticsearch.cluster.ClusterState; +import org.elasticsearch.cluster.node.DiscoveryNode; import org.elasticsearch.cluster.routing.IndexRoutingTable; import org.elasticsearch.cluster.routing.ShardRouting; +import org.elasticsearch.cluster.routing.allocation.command.AllocationCommands; import org.elasticsearch.cluster.routing.allocation.command.MoveAllocationCommand; +import org.elasticsearch.cluster.routing.allocation.decider.ConcurrentRebalanceAllocationDecider; import org.elasticsearch.cluster.routing.allocation.decider.EnableAllocationDecider; +import org.elasticsearch.cluster.routing.allocation.decider.EnableAllocationDecider.Rebalance; +import org.elasticsearch.cluster.routing.allocation.decider.ThrottlingAllocationDecider; +import org.elasticsearch.cluster.service.ClusterService; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.util.concurrent.ConcurrentCollections; -import org.elasticsearch.rest.RestStatus; +import org.elasticsearch.indices.recovery.PeerRecoverySourceService; +import org.elasticsearch.indices.recovery.StartRecoveryRequest; +import org.elasticsearch.plugins.Plugin; import org.elasticsearch.test.BackgroundIndexer; import org.elasticsearch.test.ESIntegTestCase; +import org.elasticsearch.test.junit.annotations.TestLogging; +import org.elasticsearch.test.transport.MockTransportService; +import org.elasticsearch.transport.TransportService; import java.util.ArrayList; +import java.util.Collection; import java.util.HashMap; import java.util.List; import java.util.Map; import java.util.Set; import java.util.concurrent.CountDownLatch; +import java.util.stream.Collectors; +import java.util.stream.IntStream; -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 java.util.Collections.singletonList; import static org.elasticsearch.indices.state.CloseIndexIT.assertException; import static org.elasticsearch.indices.state.CloseIndexIT.assertIndexIsClosed; import static org.elasticsearch.indices.state.CloseIndexIT.assertIndexIsOpened; @@ -50,36 +61,52 @@ @ESIntegTestCase.ClusterScope(minNumDataNodes = 2) public class CloseWhileRelocatingShardsIT extends ESIntegTestCase { + @Override + protected Collection> nodePlugins() { + return singletonList(MockTransportService.TestPlugin.class); + } + @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) + .put(ThrottlingAllocationDecider.CLUSTER_ROUTING_ALLOCATION_NODE_CONCURRENT_RECOVERIES_SETTING.getKey(), Integer.MAX_VALUE) + .put(ConcurrentRebalanceAllocationDecider.CLUSTER_ROUTING_ALLOCATION_CLUSTER_CONCURRENT_REBALANCE_SETTING.getKey(), -1) .build(); } @Override - protected int numberOfReplicas() { - return 1; + protected int maximumNumberOfShards() { + return 3; } - @AwaitsFix(bugUrl = "https://github.com/elastic/elasticsearch/issues/37274") + @TestLogging("org.elasticsearch.cluster.metadata.MetaDataIndexStateService:DEBUG,org.elasticsearch.action.admin.indices.close:DEBUG") public void testCloseWhileRelocatingShards() throws Exception { - final String[] indices = new String[randomIntBetween(1, 3)]; + final String[] indices = new String[randomIntBetween(3, 5)]; final Map docsPerIndex = new HashMap<>(); + final Map indexers = new HashMap<>(); for (int i = 0; i < indices.length; i++) { - final String indexName = "index-" + i; - createIndex(indexName); - + final String indexName = "index-" + i; 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()); - } + switch (i) { + case 0: + logger.debug("creating empty index {}", indexName); + createIndex(indexName); + break; + case 1: + nbDocs = scaledRandomIntBetween(1, 100); + logger.debug("creating index {} with {} documents", indexName, nbDocs); + createIndex(indexName); + indexRandom(randomBoolean(), IntStream.range(0, nbDocs) + .mapToObj(n -> client().prepareIndex(indexName, "_doc").setSource("num", n)) + .collect(Collectors.toList())); + break; + default: + logger.debug("creating index {} with background indexing", indexName); + final BackgroundIndexer indexer = new BackgroundIndexer(indexName, "_doc", client(), -1, 1); + indexers.put(indexName, indexer); + waitForDocs(1, indexer); } docsPerIndex.put(indexName, (long) nbDocs); indices[i] = indexName; @@ -88,60 +115,72 @@ public void testCloseWhileRelocatingShards() throws Exception { ensureGreen(indices); assertAcked(client().admin().cluster().prepareUpdateSettings() .setTransientSettings(Settings.builder() - .put(CLUSTER_ROUTING_REBALANCE_ENABLE_SETTING.getKey(), EnableAllocationDecider.Rebalance.NONE.toString()))); + .put(EnableAllocationDecider.CLUSTER_ROUTING_REBALANCE_ENABLE_SETTING.getKey(), 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(), -1, scaledRandomIntBetween(1, 3)); - waitForDocs(1, indexer); - indexers.put(index, indexer); - } - } + final String targetNode = internalCluster().startDataOnlyNode(); + ensureClusterSizeConsistency(); // wait for the master to finish processing join. 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(); + final ClusterService clusterService = internalCluster().getInstance(ClusterService.class, internalCluster().getMasterName()); + final CountDownLatch latch = new CountDownLatch(indices.length); + final CountDownLatch release = new CountDownLatch(1); + + // relocate one shard for every index to be closed + final AllocationCommands commands = new AllocationCommands(); + for (final String index : indices) { + final NumShards numShards = getNumShards(index); + final int shardId = numShards.numPrimaries == 1 ? 0 : randomIntBetween(0, numShards.numPrimaries - 1); + final IndexRoutingTable indexRoutingTable = clusterService.state().routingTable().index(index); + + final ShardRouting primary = indexRoutingTable.shard(shardId).primaryShard(); + assertTrue(primary.started()); + + String currentNodeId = primary.currentNodeId(); + if (numShards.numReplicas > 0) { + final ShardRouting replica = indexRoutingTable.shard(shardId).replicaShards().iterator().next(); assertTrue(replica.started()); + if (randomBoolean()) { + currentNodeId = replica.currentNodeId(); + } + } - 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); + final DiscoveryNode sourceNode = clusterService.state().nodes().resolveNode(primary.currentNodeId()); + ((MockTransportService) internalCluster().getInstance(TransportService.class, targetNode)) + .addSendBehavior(internalCluster().getInstance(TransportService.class, sourceNode.getName()), + (connection, requestId, action, request, options) -> { + if (PeerRecoverySourceService.Actions.START_RECOVERY.equals(action)) { + logger.debug("blocking recovery of shard {}", ((StartRecoveryRequest) request).shardId()); + latch.countDown(); + try { + release.await(); + logger.debug("releasing recovery of shard {}", ((StartRecoveryRequest) request).shardId()); + } catch (InterruptedException e) { + throw new AssertionError(e); + } + } + connection.sendRequest(requestId, action, request, options); } - assertAcked(client().admin().cluster().prepareReroute() - .add(new MoveAllocationCommand(indexToRelocate, shardId, currentNodeId, newNode))); - }); - threads.add(thread); - thread.start(); - } + ); + commands.add(new MoveAllocationCommand(index, shardId, currentNodeId, targetNode)); } + assertAcked(client().admin().cluster().reroute(new ClusterRerouteRequest().commands(commands)).get()); + // start index closing threads + final List threads = new ArrayList<>(); for (final String indexToClose : indices) { final Thread thread = new Thread(() -> { try { latch.await(); } catch (InterruptedException e) { throw new AssertionError(e); + } finally { + release.countDown(); } + // Closing is not always acknowledged when shards are relocating: this is the case when the target shard is initializing + // or is catching up operations. In these cases the TransportVerifyShardBeforeCloseAction will detect that the global + // and max sequence number don't match and will not ack the close. AcknowledgedResponse closeResponse = client().admin().indices().prepareClose(indexToClose).get(); if (closeResponse.isAcknowledged()) { assertTrue("Index closing should not be acknowledged twice", acknowledgedCloses.add(indexToClose)); @@ -155,6 +194,7 @@ public void testCloseWhileRelocatingShards() throws Exception { for (Thread thread : threads) { thread.join(); } + for (Map.Entry entry : indexers.entrySet()) { final BackgroundIndexer indexer = entry.getValue(); indexer.setAssertNoFailuresOnStop(false); @@ -172,7 +212,8 @@ public void testCloseWhileRelocatingShards() throws Exception { } } finally { assertAcked(client().admin().cluster().prepareUpdateSettings() - .setTransientSettings(Settings.builder().putNull(CLUSTER_ROUTING_REBALANCE_ENABLE_SETTING.getKey()))); + .setTransientSettings(Settings.builder() + .putNull(EnableAllocationDecider.CLUSTER_ROUTING_REBALANCE_ENABLE_SETTING.getKey()))); } for (String index : indices) { From abe35fb99bc6d3adf7c68278044437216d647315 Mon Sep 17 00:00:00 2001 From: Daniel Mitterdorfer Date: Mon, 14 Jan 2019 13:44:32 +0100 Subject: [PATCH 142/186] Remove unused index store in directory service With this commit we remove the unused field `indexStore` from all implementations of `FsDirectoryService`. Relates #37097 --- .../store/smbmmapfs/SmbMmapFsDirectoryService.java | 5 ++--- .../index/store/smbmmapfs/SmbMmapFsIndexStore.java | 2 +- .../smbsimplefs/SmbSimpleFsDirectoryService.java | 5 ++--- .../store/smbsimplefs/SmbSimpleFsIndexStore.java | 2 +- .../index/store/FsDirectoryService.java | 4 +--- .../org/elasticsearch/index/store/IndexStore.java | 2 +- .../index/store/FsDirectoryServiceTests.java | 3 +-- .../elasticsearch/index/store/IndexStoreTests.java | 3 +-- .../test/store/MockFSDirectoryService.java | 12 +++++------- .../elasticsearch/test/store/MockFSIndexStore.java | 2 +- 10 files changed, 16 insertions(+), 24 deletions(-) diff --git a/plugins/store-smb/src/main/java/org/elasticsearch/index/store/smbmmapfs/SmbMmapFsDirectoryService.java b/plugins/store-smb/src/main/java/org/elasticsearch/index/store/smbmmapfs/SmbMmapFsDirectoryService.java index 5813d5412a6d2..1264464cf0071 100644 --- a/plugins/store-smb/src/main/java/org/elasticsearch/index/store/smbmmapfs/SmbMmapFsDirectoryService.java +++ b/plugins/store-smb/src/main/java/org/elasticsearch/index/store/smbmmapfs/SmbMmapFsDirectoryService.java @@ -25,7 +25,6 @@ import org.elasticsearch.index.IndexSettings; import org.elasticsearch.index.shard.ShardPath; import org.elasticsearch.index.store.FsDirectoryService; -import org.elasticsearch.index.store.IndexStore; import org.elasticsearch.index.store.SmbDirectoryWrapper; import java.io.IOException; @@ -33,8 +32,8 @@ public class SmbMmapFsDirectoryService extends FsDirectoryService { - public SmbMmapFsDirectoryService(IndexSettings indexSettings, IndexStore indexStore, ShardPath path) { - super(indexSettings, indexStore, path); + public SmbMmapFsDirectoryService(IndexSettings indexSettings, ShardPath path) { + super(indexSettings, path); } @Override diff --git a/plugins/store-smb/src/main/java/org/elasticsearch/index/store/smbmmapfs/SmbMmapFsIndexStore.java b/plugins/store-smb/src/main/java/org/elasticsearch/index/store/smbmmapfs/SmbMmapFsIndexStore.java index b53d22454f10d..0399348966361 100644 --- a/plugins/store-smb/src/main/java/org/elasticsearch/index/store/smbmmapfs/SmbMmapFsIndexStore.java +++ b/plugins/store-smb/src/main/java/org/elasticsearch/index/store/smbmmapfs/SmbMmapFsIndexStore.java @@ -32,6 +32,6 @@ public SmbMmapFsIndexStore(IndexSettings indexSettings) { @Override public DirectoryService newDirectoryService(ShardPath path) { - return new SmbMmapFsDirectoryService(indexSettings, this, path); + return new SmbMmapFsDirectoryService(indexSettings, path); } } diff --git a/plugins/store-smb/src/main/java/org/elasticsearch/index/store/smbsimplefs/SmbSimpleFsDirectoryService.java b/plugins/store-smb/src/main/java/org/elasticsearch/index/store/smbsimplefs/SmbSimpleFsDirectoryService.java index 66b73476d7a8b..87e45a02cf6cb 100644 --- a/plugins/store-smb/src/main/java/org/elasticsearch/index/store/smbsimplefs/SmbSimpleFsDirectoryService.java +++ b/plugins/store-smb/src/main/java/org/elasticsearch/index/store/smbsimplefs/SmbSimpleFsDirectoryService.java @@ -25,7 +25,6 @@ import org.elasticsearch.index.IndexSettings; import org.elasticsearch.index.shard.ShardPath; import org.elasticsearch.index.store.FsDirectoryService; -import org.elasticsearch.index.store.IndexStore; import org.elasticsearch.index.store.SmbDirectoryWrapper; import java.io.IOException; @@ -33,8 +32,8 @@ public class SmbSimpleFsDirectoryService extends FsDirectoryService { - public SmbSimpleFsDirectoryService(IndexSettings indexSettings, IndexStore indexStore, ShardPath path) { - super(indexSettings, indexStore, path); + public SmbSimpleFsDirectoryService(IndexSettings indexSettings, ShardPath path) { + super(indexSettings, path); } @Override diff --git a/plugins/store-smb/src/main/java/org/elasticsearch/index/store/smbsimplefs/SmbSimpleFsIndexStore.java b/plugins/store-smb/src/main/java/org/elasticsearch/index/store/smbsimplefs/SmbSimpleFsIndexStore.java index ea4cc183885d0..3b6b3c3c8990f 100644 --- a/plugins/store-smb/src/main/java/org/elasticsearch/index/store/smbsimplefs/SmbSimpleFsIndexStore.java +++ b/plugins/store-smb/src/main/java/org/elasticsearch/index/store/smbsimplefs/SmbSimpleFsIndexStore.java @@ -32,7 +32,7 @@ public SmbSimpleFsIndexStore(IndexSettings indexSettings) { @Override public DirectoryService newDirectoryService(ShardPath path) { - return new SmbSimpleFsDirectoryService(indexSettings, this, path); + return new SmbSimpleFsDirectoryService(indexSettings, path); } } diff --git a/server/src/main/java/org/elasticsearch/index/store/FsDirectoryService.java b/server/src/main/java/org/elasticsearch/index/store/FsDirectoryService.java index ab93d1759391a..badf981803603 100644 --- a/server/src/main/java/org/elasticsearch/index/store/FsDirectoryService.java +++ b/server/src/main/java/org/elasticsearch/index/store/FsDirectoryService.java @@ -45,7 +45,6 @@ import java.util.Set; public class FsDirectoryService extends DirectoryService { - protected final IndexStore indexStore; public static final Setting INDEX_LOCK_FACTOR_SETTING = new Setting<>("index.store.fs.fs_lock", "native", (s) -> { switch (s) { case "native": @@ -60,10 +59,9 @@ public class FsDirectoryService extends DirectoryService { private final ShardPath path; @Inject - public FsDirectoryService(IndexSettings indexSettings, IndexStore indexStore, ShardPath path) { + public FsDirectoryService(IndexSettings indexSettings, ShardPath path) { super(path.getShardId(), indexSettings); this.path = path; - this.indexStore = indexStore; } @Override diff --git a/server/src/main/java/org/elasticsearch/index/store/IndexStore.java b/server/src/main/java/org/elasticsearch/index/store/IndexStore.java index 5005291282557..0d41b1ac95d18 100644 --- a/server/src/main/java/org/elasticsearch/index/store/IndexStore.java +++ b/server/src/main/java/org/elasticsearch/index/store/IndexStore.java @@ -33,7 +33,7 @@ public IndexStore(IndexSettings indexSettings) { * The shard store class that should be used for each shard. */ public DirectoryService newDirectoryService(ShardPath path) { - return new FsDirectoryService(indexSettings, this, path); + return new FsDirectoryService(indexSettings, path); } } diff --git a/server/src/test/java/org/elasticsearch/index/store/FsDirectoryServiceTests.java b/server/src/test/java/org/elasticsearch/index/store/FsDirectoryServiceTests.java index 24ce9b487cc24..e84ff3f32841b 100644 --- a/server/src/test/java/org/elasticsearch/index/store/FsDirectoryServiceTests.java +++ b/server/src/test/java/org/elasticsearch/index/store/FsDirectoryServiceTests.java @@ -49,11 +49,10 @@ private void doTestPreload(String...preload) throws IOException { .putList(IndexModule.INDEX_STORE_PRE_LOAD_SETTING.getKey(), preload) .build(); IndexSettings settings = IndexSettingsModule.newIndexSettings("foo", build); - IndexStore store = new IndexStore(settings); Path tempDir = createTempDir().resolve(settings.getUUID()).resolve("0"); Files.createDirectories(tempDir); ShardPath path = new ShardPath(false, tempDir, tempDir, new ShardId(settings.getIndex(), 0)); - FsDirectoryService fsDirectoryService = new FsDirectoryService(settings, store, path); + FsDirectoryService fsDirectoryService = new FsDirectoryService(settings, path); Directory directory = fsDirectoryService.newDirectory(); assertFalse(directory instanceof SleepingLockWrapper); if (preload.length == 0) { diff --git a/server/src/test/java/org/elasticsearch/index/store/IndexStoreTests.java b/server/src/test/java/org/elasticsearch/index/store/IndexStoreTests.java index 4e22aad8d15de..21c8d1c1d78a4 100644 --- a/server/src/test/java/org/elasticsearch/index/store/IndexStoreTests.java +++ b/server/src/test/java/org/elasticsearch/index/store/IndexStoreTests.java @@ -60,8 +60,7 @@ private void doTestStoreDirectory(Index index, Path tempDir, String typeSettingV } Settings settings = settingsBuilder.build(); IndexSettings indexSettings = IndexSettingsModule.newIndexSettings("foo", settings); - FsDirectoryService service = new FsDirectoryService(indexSettings, null, - new ShardPath(false, tempDir, tempDir, new ShardId(index, 0))); + FsDirectoryService service = new FsDirectoryService(indexSettings, new ShardPath(false, tempDir, tempDir, new ShardId(index, 0))); try (Directory directory = service.newFSDirectory(tempDir, NoLockFactory.INSTANCE)) { switch (type) { case HYBRIDFS: diff --git a/test/framework/src/main/java/org/elasticsearch/test/store/MockFSDirectoryService.java b/test/framework/src/main/java/org/elasticsearch/test/store/MockFSDirectoryService.java index cdc33b38b8676..070343bf68785 100644 --- a/test/framework/src/main/java/org/elasticsearch/test/store/MockFSDirectoryService.java +++ b/test/framework/src/main/java/org/elasticsearch/test/store/MockFSDirectoryService.java @@ -43,7 +43,6 @@ import org.elasticsearch.index.shard.ShardId; import org.elasticsearch.index.shard.ShardPath; import org.elasticsearch.index.store.FsDirectoryService; -import org.elasticsearch.index.store.IndexStore; import org.elasticsearch.index.store.Store; import org.elasticsearch.test.ESIntegTestCase; import org.elasticsearch.test.ESTestCase; @@ -74,8 +73,8 @@ public class MockFSDirectoryService extends FsDirectoryService { private final boolean crashIndex; @Inject - public MockFSDirectoryService(IndexSettings idxSettings, IndexStore indexStore, final ShardPath path) { - super(idxSettings, indexStore, path); + public MockFSDirectoryService(IndexSettings idxSettings, final ShardPath path) { + super(idxSettings, path); Settings indexSettings = idxSettings.getSettings(); final long seed = idxSettings.getValue(ESIntegTestCase.INDEX_TEST_SEED_SETTING); this.random = new Random(seed); @@ -90,7 +89,7 @@ public MockFSDirectoryService(IndexSettings idxSettings, IndexStore indexStore, logger.debug("Using MockDirWrapper with seed [{}] throttle: [{}] crashIndex: [{}]", SeedUtils.formatSeed(seed), throttle, crashIndex); } - delegateService = randomDirectoryService(indexStore, path); + delegateService = randomDirectoryService(idxSettings, path); } @@ -152,8 +151,7 @@ private Directory wrap(Directory dir) { return w; } - private FsDirectoryService randomDirectoryService(IndexStore indexStore, ShardPath path) { - final IndexSettings indexSettings = indexStore.getIndexSettings(); + private FsDirectoryService randomDirectoryService(IndexSettings indexSettings, ShardPath path) { final IndexMetaData build = IndexMetaData.builder(indexSettings.getIndexMetaData()) .settings(Settings.builder() // don't use the settings from indexSettings#getSettings() they are merged with node settings and might contain @@ -163,7 +161,7 @@ private FsDirectoryService randomDirectoryService(IndexStore indexStore, ShardPa RandomPicks.randomFrom(random, IndexModule.Type.values()).getSettingsKey())) .build(); final IndexSettings newIndexSettings = new IndexSettings(build, indexSettings.getNodeSettings()); - return new FsDirectoryService(newIndexSettings, indexStore, path); + return new FsDirectoryService(newIndexSettings, path); } public static final class ElasticsearchMockDirectoryWrapper extends MockDirectoryWrapper { diff --git a/test/framework/src/main/java/org/elasticsearch/test/store/MockFSIndexStore.java b/test/framework/src/main/java/org/elasticsearch/test/store/MockFSIndexStore.java index 82ab9fc412144..1ec5087605539 100644 --- a/test/framework/src/main/java/org/elasticsearch/test/store/MockFSIndexStore.java +++ b/test/framework/src/main/java/org/elasticsearch/test/store/MockFSIndexStore.java @@ -86,7 +86,7 @@ public void onIndexModule(IndexModule indexModule) { @Override public DirectoryService newDirectoryService(ShardPath path) { - return new MockFSDirectoryService(indexSettings, this, path); + return new MockFSDirectoryService(indexSettings, path); } private static final EnumSet validCheckIndexStates = EnumSet.of( From de852765d6bd7d06497cb15136c8b61bfa93744c Mon Sep 17 00:00:00 2001 From: Martijn van Groningen Date: Mon, 14 Jan 2019 14:27:26 +0100 Subject: [PATCH 143/186] unmuted test Relates to #37014 --- .../java/org/elasticsearch/xpack/CcrIntegTestCase.java | 4 +++- .../org/elasticsearch/xpack/CcrSingleNodeTestCase.java | 6 ++++++ .../elasticsearch/xpack/ccr/LocalIndexFollowingIT.java | 8 +++++--- 3 files changed, 14 insertions(+), 4 deletions(-) diff --git a/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/CcrIntegTestCase.java b/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/CcrIntegTestCase.java index 05b20050ee3c9..7af3d690e3a99 100644 --- a/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/CcrIntegTestCase.java +++ b/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/CcrIntegTestCase.java @@ -563,9 +563,11 @@ static void removeCCRRelatedMetadataFromClusterState(ClusterService clusterServi clusterService.submitStateUpdateTask("remove-ccr-related-metadata", new ClusterStateUpdateTask() { @Override public ClusterState execute(ClusterState currentState) throws Exception { + AutoFollowMetadata empty = + new AutoFollowMetadata(Collections.emptyMap(), Collections.emptyMap(), Collections.emptyMap()); ClusterState.Builder newState = ClusterState.builder(currentState); newState.metaData(MetaData.builder(currentState.getMetaData()) - .removeCustom(AutoFollowMetadata.TYPE) + .putCustom(AutoFollowMetadata.TYPE, empty) .removeCustom(PersistentTasksCustomMetaData.TYPE) .build()); return newState.build(); diff --git a/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/CcrSingleNodeTestCase.java b/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/CcrSingleNodeTestCase.java index 2fb1f868dd7fb..e77a672f1fddb 100644 --- a/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/CcrSingleNodeTestCase.java +++ b/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/CcrSingleNodeTestCase.java @@ -18,7 +18,9 @@ import org.elasticsearch.xpack.ccr.CcrSettings; import org.elasticsearch.xpack.ccr.LocalStateCcr; import org.elasticsearch.xpack.core.XPackSettings; +import org.elasticsearch.xpack.core.ccr.AutoFollowStats; import org.elasticsearch.xpack.core.ccr.ShardFollowNodeTaskStatus; +import org.elasticsearch.xpack.core.ccr.action.CcrStatsAction; import org.elasticsearch.xpack.core.ccr.action.FollowStatsAction; import org.elasticsearch.xpack.core.ccr.action.PutFollowAction; import org.elasticsearch.xpack.core.ccr.action.ResumeFollowAction; @@ -79,6 +81,10 @@ public void removeLocalRemote() { assertAcked(client().admin().cluster().updateSettings(updateSettingsRequest).actionGet()); } + protected AutoFollowStats getAutoFollowStats() { + return client().execute(CcrStatsAction.INSTANCE, new CcrStatsAction.Request()).actionGet().getAutoFollowStats(); + } + protected ResumeFollowAction.Request getResumeFollowRequest(String followerIndex) { ResumeFollowAction.Request request = new ResumeFollowAction.Request(); request.setFollowerIndex(followerIndex); diff --git a/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/LocalIndexFollowingIT.java b/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/LocalIndexFollowingIT.java index f80fc0c07d9f4..2c50411971a1b 100644 --- a/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/LocalIndexFollowingIT.java +++ b/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/LocalIndexFollowingIT.java @@ -92,7 +92,6 @@ public void testDoNotCreateFollowerIfLeaderDoesNotHaveSoftDeletes() throws Excep assertThat(client().admin().indices().prepareExists("follower-index").get().isExists(), equalTo(false)); } - @AwaitsFix(bugUrl = "https://github.com/elastic/elasticsearch/issues/37014") public void testRemoveRemoteConnection() throws Exception { PutAutoFollowPatternAction.Request request = new PutAutoFollowPatternAction.Request(); request.setName("my_pattern"); @@ -101,6 +100,7 @@ public void testRemoveRemoteConnection() throws Exception { request.setFollowIndexNamePattern("copy-{{leader_index}}"); request.setReadPollTimeout(TimeValue.timeValueMillis(10)); assertTrue(client().execute(PutAutoFollowPatternAction.INSTANCE, request).actionGet().isAcknowledged()); + long previousNumberOfSuccessfulFollowedIndices = getAutoFollowStats().getNumberOfSuccessfulFollowIndices(); Settings leaderIndexSettings = Settings.builder() .put(IndexSettings.INDEX_SOFT_DELETES_SETTING.getKey(), true) @@ -111,7 +111,8 @@ public void testRemoveRemoteConnection() throws Exception { client().prepareIndex("logs-20200101", "doc").setSource("{}", XContentType.JSON).get(); assertBusy(() -> { CcrStatsAction.Response response = client().execute(CcrStatsAction.INSTANCE, new CcrStatsAction.Request()).actionGet(); - assertThat(response.getAutoFollowStats().getNumberOfSuccessfulFollowIndices(), equalTo(1L)); + assertThat(response.getAutoFollowStats().getNumberOfSuccessfulFollowIndices(), + equalTo(previousNumberOfSuccessfulFollowedIndices + 1)); assertThat(response.getFollowStats().getStatsResponses().size(), equalTo(1)); assertThat(response.getFollowStats().getStatsResponses().get(0).status().followerGlobalCheckpoint(), equalTo(0L)); }); @@ -127,7 +128,8 @@ public void testRemoveRemoteConnection() throws Exception { client().prepareIndex("logs-20200101", "doc").setSource("{}", XContentType.JSON).get(); assertBusy(() -> { CcrStatsAction.Response response = client().execute(CcrStatsAction.INSTANCE, new CcrStatsAction.Request()).actionGet(); - assertThat(response.getAutoFollowStats().getNumberOfSuccessfulFollowIndices(), equalTo(2L)); + assertThat(response.getAutoFollowStats().getNumberOfSuccessfulFollowIndices(), + equalTo(previousNumberOfSuccessfulFollowedIndices + 2)); FollowStatsAction.StatsRequest statsRequest = new FollowStatsAction.StatsRequest(); statsRequest.setIndices(new String[]{"copy-logs-20200101"}); From 1abe5df09c7635175e0b6216bb58563d6c879eac Mon Sep 17 00:00:00 2001 From: David Kyle Date: Mon, 14 Jan 2019 14:17:11 +0000 Subject: [PATCH 144/186] Mute IndexShardRetentionLeaseTests.testCommit #37420 --- .../elasticsearch/index/shard/IndexShardRetentionLeaseTests.java | 1 + 1 file changed, 1 insertion(+) diff --git a/server/src/test/java/org/elasticsearch/index/shard/IndexShardRetentionLeaseTests.java b/server/src/test/java/org/elasticsearch/index/shard/IndexShardRetentionLeaseTests.java index e95b52280ca2e..e15f6c45ae640 100644 --- a/server/src/test/java/org/elasticsearch/index/shard/IndexShardRetentionLeaseTests.java +++ b/server/src/test/java/org/elasticsearch/index/shard/IndexShardRetentionLeaseTests.java @@ -134,6 +134,7 @@ public void testExpiration() throws IOException { } } + @AwaitsFix(bugUrl = "https://github.com/elastic/elasticsearch/issues/37420") public void testCommit() throws IOException { final Settings settings = Settings.builder() .put(IndexSettings.INDEX_SOFT_DELETES_SETTING.getKey(), true) From 033e67fa593ab831ee008824c03f6c2f161842b6 Mon Sep 17 00:00:00 2001 From: Armin Braun Date: Mon, 14 Jan 2019 16:22:44 +0100 Subject: [PATCH 145/186] Cleanup Deadcode in Rest Tests (#37418) * Either dead code outright or redundant overrides removed --- .../test/rest/ESRestTestCase.java | 6 ++-- .../test/rest/FakeRestChannel.java | 35 ------------------- .../rest/yaml/ESClientYamlSuiteTestCase.java | 20 ----------- .../test/rest/yaml/Features.java | 25 ++++++------- 4 files changed, 13 insertions(+), 73 deletions(-) diff --git a/test/framework/src/main/java/org/elasticsearch/test/rest/ESRestTestCase.java b/test/framework/src/main/java/org/elasticsearch/test/rest/ESRestTestCase.java index 02918ea78714e..c711019bf753c 100644 --- a/test/framework/src/main/java/org/elasticsearch/test/rest/ESRestTestCase.java +++ b/test/framework/src/main/java/org/elasticsearch/test/rest/ESRestTestCase.java @@ -112,7 +112,7 @@ public static Map entityAsMap(Response response) throws IOExcept /** * Does any node in the cluster being tested have x-pack installed? */ - public static boolean hasXPack() throws IOException { + public static boolean hasXPack() { if (hasXPack == null) { throw new IllegalStateException("must be called inside of a rest test case test"); } @@ -554,7 +554,7 @@ private void wipeClusterSettings() throws IOException { } } - private void wipeRollupJobs() throws IOException, InterruptedException { + private void wipeRollupJobs() throws IOException { Response response = adminClient().performRequest(new Request("GET", "/_rollup/job/_all")); Map jobs = entityAsMap(response); @SuppressWarnings("unchecked") @@ -617,7 +617,7 @@ private static void deleteAllPolicies() throws IOException { * Logs a message if there are still running tasks. The reasoning is that any tasks still running are state the is trying to bleed into * other tests. */ - private void logIfThereAreRunningTasks() throws InterruptedException, IOException { + private void logIfThereAreRunningTasks() throws IOException { Set runningTasks = runningTasks(adminClient().performRequest(new Request("GET", "/_tasks"))); // Ignore the task list API - it doesn't count against us runningTasks.remove(ListTasksAction.NAME); diff --git a/test/framework/src/main/java/org/elasticsearch/test/rest/FakeRestChannel.java b/test/framework/src/main/java/org/elasticsearch/test/rest/FakeRestChannel.java index 2dcadce2ed17b..ff6b99bdc4a51 100644 --- a/test/framework/src/main/java/org/elasticsearch/test/rest/FakeRestChannel.java +++ b/test/framework/src/main/java/org/elasticsearch/test/rest/FakeRestChannel.java @@ -18,18 +18,12 @@ */ package org.elasticsearch.test.rest; -import org.elasticsearch.common.Nullable; -import org.elasticsearch.common.io.stream.BytesStreamOutput; -import org.elasticsearch.common.xcontent.XContentBuilder; -import org.elasticsearch.common.xcontent.XContentType; import org.elasticsearch.rest.AbstractRestChannel; import org.elasticsearch.rest.RestRequest; import org.elasticsearch.rest.RestResponse; import org.elasticsearch.rest.RestStatus; -import java.io.IOException; import java.util.concurrent.CountDownLatch; -import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicInteger; public final class FakeRestChannel extends AbstractRestChannel { @@ -43,31 +37,6 @@ public FakeRestChannel(RestRequest request, boolean detailedErrorsEnabled, int r this.latch = new CountDownLatch(responseCount); } - @Override - public XContentBuilder newBuilder() throws IOException { - return super.newBuilder(); - } - - @Override - public XContentBuilder newErrorBuilder() throws IOException { - return super.newErrorBuilder(); - } - - @Override - public XContentBuilder newBuilder(@Nullable XContentType requestContentType, boolean useFiltering) throws IOException { - return super.newBuilder(requestContentType, useFiltering); - } - - @Override - protected BytesStreamOutput newBytesOutput() { - return super.newBytesOutput(); - } - - @Override - public RestRequest request() { - return super.request(); - } - @Override public void sendResponse(RestResponse response) { this.capturedRestResponse = response; @@ -83,10 +52,6 @@ public RestResponse capturedResponse() { return capturedRestResponse; } - public boolean await() throws InterruptedException { - return latch.await(10, TimeUnit.SECONDS); - } - public AtomicInteger responses() { return responses; } diff --git a/test/framework/src/main/java/org/elasticsearch/test/rest/yaml/ESClientYamlSuiteTestCase.java b/test/framework/src/main/java/org/elasticsearch/test/rest/yaml/ESClientYamlSuiteTestCase.java index f04bead4fbfa1..ab155889ac687 100644 --- a/test/framework/src/main/java/org/elasticsearch/test/rest/yaml/ESClientYamlSuiteTestCase.java +++ b/test/framework/src/main/java/org/elasticsearch/test/rest/yaml/ESClientYamlSuiteTestCase.java @@ -319,26 +319,6 @@ private static Tuple readVersionsFromCatNodes(RestClient restC return new Tuple<>(version, masterVersion); } - private static Version readVersionsFromInfo(RestClient restClient, int numHosts) throws IOException { - Version version = null; - for (int i = 0; i < numHosts; i++) { - //we don't really use the urls here, we rely on the client doing round-robin to touch all the nodes in the cluster - Response response = restClient.performRequest(new Request("GET", "/")); - ClientYamlTestResponse restTestResponse = new ClientYamlTestResponse(response); - Object latestVersion = restTestResponse.evaluate("version.number"); - if (latestVersion == null) { - throw new RuntimeException("elasticsearch version not found in the response"); - } - final Version currentVersion = Version.fromString(latestVersion.toString()); - if (version == null) { - version = currentVersion; - } else if (version.onOrAfter(currentVersion)) { - version = currentVersion; - } - } - return version; - } - public void test() throws IOException { //skip test if it matches one of the blacklist globs for (BlacklistedPathPatternMatcher blacklistedPathMatcher : blacklistPathMatchers) { diff --git a/test/framework/src/main/java/org/elasticsearch/test/rest/yaml/Features.java b/test/framework/src/main/java/org/elasticsearch/test/rest/yaml/Features.java index d3fb500ac051a..bb5354e4fedd3 100644 --- a/test/framework/src/main/java/org/elasticsearch/test/rest/yaml/Features.java +++ b/test/framework/src/main/java/org/elasticsearch/test/rest/yaml/Features.java @@ -19,7 +19,6 @@ package org.elasticsearch.test.rest.yaml; -import java.io.IOException; import java.util.Arrays; import java.util.List; @@ -58,23 +57,19 @@ private Features() { * Tells whether all the features provided as argument are supported */ public static boolean areAllSupported(List features) { - try { - for (String feature : features) { - if (feature.equals("xpack")) { - if (false == ESRestTestCase.hasXPack()) { - return false; - } - } else if (feature.equals("no_xpack")) { - if (ESRestTestCase.hasXPack()) { - return false; - } - } else if (false == SUPPORTED.contains(feature)) { + for (String feature : features) { + if (feature.equals("xpack")) { + if (false == ESRestTestCase.hasXPack()) { return false; } + } else if (feature.equals("no_xpack")) { + if (ESRestTestCase.hasXPack()) { + return false; + } + } else if (false == SUPPORTED.contains(feature)) { + return false; } - return true; - } catch (IOException e) { - throw new RuntimeException("error checking if xpack is available", e); } + return true; } } From 2ee55a50bf83749b02b0913fa989f0d8aafaa74f Mon Sep 17 00:00:00 2001 From: David Kyle Date: Mon, 14 Jan 2019 16:39:47 +0000 Subject: [PATCH 146/186] [ML] Use String rep of Version in map for serialisation (#37416) --- .../org/elasticsearch/xpack/ml/MlConfigMigrator.java | 3 ++- .../elasticsearch/xpack/ml/MlConfigMigratorTests.java | 11 ++++++++++- 2 files changed, 12 insertions(+), 2 deletions(-) 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 bb3735f8aa3f1..fbf8c3c804eef 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 @@ -402,7 +402,8 @@ private void createConfigIndex(ActionListener listener) { public static Job updateJobForMigration(Job job) { Job.Builder builder = new Job.Builder(job); Map custom = job.getCustomSettings() == null ? new HashMap<>() : new HashMap<>(job.getCustomSettings()); - custom.put(MIGRATED_FROM_VERSION, job.getJobVersion()); + String version = job.getJobVersion() != null ? job.getJobVersion().toString() : null; + custom.put(MIGRATED_FROM_VERSION, version); builder.setCustomSettings(custom); // Increase the model memory limit for 6.1 - 6.3 jobs Version jobVersion = job.getJobVersion(); diff --git a/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/MlConfigMigratorTests.java b/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/MlConfigMigratorTests.java index d9ea035e58234..cff299b9fa1aa 100644 --- a/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/MlConfigMigratorTests.java +++ b/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/MlConfigMigratorTests.java @@ -11,6 +11,7 @@ import org.elasticsearch.cluster.ClusterName; import org.elasticsearch.cluster.ClusterState; import org.elasticsearch.cluster.metadata.MetaData; +import org.elasticsearch.common.io.stream.NamedWriteableRegistry; import org.elasticsearch.persistent.PersistentTasksCustomMetaData; import org.elasticsearch.test.ESTestCase; import org.elasticsearch.xpack.core.ml.MlMetadata; @@ -21,9 +22,11 @@ import org.elasticsearch.xpack.core.ml.job.config.Job; import org.elasticsearch.xpack.core.ml.job.config.JobTests; +import java.io.IOException; import java.util.ArrayList; import java.util.Arrays; import java.util.Collections; +import java.util.Date; import java.util.HashMap; import java.util.List; import java.util.Map; @@ -125,7 +128,7 @@ public void testUpdateJobForMigration() { Job migratedJob = MlConfigMigrator.updateJobForMigration(oldJob.build()); assertEquals(Version.CURRENT, migratedJob.getJobVersion()); assertTrue(migratedJob.getCustomSettings().containsKey(MlConfigMigrator.MIGRATED_FROM_VERSION)); - assertEquals(oldVersion, migratedJob.getCustomSettings().get(MlConfigMigrator.MIGRATED_FROM_VERSION)); + assertEquals(oldVersion.toString(), migratedJob.getCustomSettings().get(MlConfigMigrator.MIGRATED_FROM_VERSION)); } public void testUpdateJobForMigration_GivenV54Job() { @@ -138,6 +141,12 @@ public void testUpdateJobForMigration_GivenV54Job() { assertTrue(migratedJob.getCustomSettings().containsKey(MlConfigMigrator.MIGRATED_FROM_VERSION)); } + public void testSerialisationOfUpdatedJob() throws IOException { + Job migratedJob = MlConfigMigrator.updateJobForMigration(JobTests.buildJobBuilder("pre-migration").build(new Date())); + Job copy = copyWriteable(migratedJob, new NamedWriteableRegistry(Collections.emptyList()), Job::new, Version.CURRENT); + assertEquals(migratedJob, copy); + } + public void testFilterFailedJobConfigWrites() { List jobs = new ArrayList<>(); jobs.add(JobTests.buildJobBuilder("foo").build()); From 92c6c98e8d8d4131f1c90b13056bc07c83d184b8 Mon Sep 17 00:00:00 2001 From: markharwood Date: Mon, 14 Jan 2019 17:51:49 +0000 Subject: [PATCH 147/186] Performance fix. Reduce deprecation calls for the same bulk request (#37415) DeprecationLogger has warning de-duplication logic but it is expensive to run as it involves parsing existing warning headers. This PR changes the upstream bulk indexing code to do its own "event thinning" rather than relying on DeprecationLogger's trimming. Closes #37411 --- .../java/org/elasticsearch/action/bulk/BulkRequest.java | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) diff --git a/server/src/main/java/org/elasticsearch/action/bulk/BulkRequest.java b/server/src/main/java/org/elasticsearch/action/bulk/BulkRequest.java index 292d75db5cff0..e58b0dfbffbd3 100644 --- a/server/src/main/java/org/elasticsearch/action/bulk/BulkRequest.java +++ b/server/src/main/java/org/elasticsearch/action/bulk/BulkRequest.java @@ -366,6 +366,7 @@ public BulkRequest add(BytesReference data, @Nullable String defaultIndex, @Null int from = 0; int length = data.length(); byte marker = xContent.streamSeparator(); + boolean typesDeprecationLogged = false; while (true) { int nextMarker = findNextMarker(marker, from, data, length); if (nextMarker == -1) { @@ -427,7 +428,10 @@ public BulkRequest add(BytesReference data, @Nullable String defaultIndex, @Null } index = parser.text(); } else if (TYPE.match(currentFieldName, parser.getDeprecationHandler())) { - deprecationLogger.deprecatedAndMaybeLog("bulk_with_types", RestBulkAction.TYPES_DEPRECATION_MESSAGE); + if (typesDeprecationLogged == false) { + deprecationLogger.deprecatedAndMaybeLog("bulk_with_types", RestBulkAction.TYPES_DEPRECATION_MESSAGE); + typesDeprecationLogged = true; + } type = parser.text(); } else if (ID.match(currentFieldName, parser.getDeprecationHandler())) { id = parser.text(); From c801b89072a8b9d35c3a40d247ede51834f99926 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Christoph=20B=C3=BCscher?= Date: Mon, 14 Jan 2019 20:39:04 +0100 Subject: [PATCH 148/186] Fix Eclipse specific compilation issue (#37419) Without pulling out the supplier function to the enclosing class, Eclipse 4.8 complains with the following error "No enclosing instance of type CoordinatorTests.Cluster is available due to some intermediate constructor invocation" --- .../cluster/coordination/CoordinatorTests.java | 10 ++++++---- 1 file changed, 6 insertions(+), 4 deletions(-) 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 b03d05fa36238..98e21581d6f87 100644 --- a/server/src/test/java/org/elasticsearch/cluster/coordination/CoordinatorTests.java +++ b/server/src/test/java/org/elasticsearch/cluster/coordination/CoordinatorTests.java @@ -19,6 +19,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; @@ -90,10 +91,10 @@ import static org.elasticsearch.cluster.coordination.CoordinationStateTests.clusterState; import static org.elasticsearch.cluster.coordination.CoordinationStateTests.setValue; import static org.elasticsearch.cluster.coordination.CoordinationStateTests.value; +import static org.elasticsearch.cluster.coordination.Coordinator.PUBLISH_TIMEOUT_SETTING; import static org.elasticsearch.cluster.coordination.Coordinator.Mode.CANDIDATE; import static org.elasticsearch.cluster.coordination.Coordinator.Mode.FOLLOWER; import static org.elasticsearch.cluster.coordination.Coordinator.Mode.LEADER; -import static org.elasticsearch.cluster.coordination.Coordinator.PUBLISH_TIMEOUT_SETTING; import static org.elasticsearch.cluster.coordination.CoordinatorTests.Cluster.DEFAULT_DELAY_VARIABILITY; import static org.elasticsearch.cluster.coordination.ElectionSchedulerFactory.ELECTION_BACK_OFF_TIME_SETTING; import static org.elasticsearch.cluster.coordination.ElectionSchedulerFactory.ELECTION_DURATION_SETTING; @@ -1101,6 +1102,9 @@ class Cluster { private final Set blackholedNodes = new HashSet<>(); private final Map committedStatesByVersion = new HashMap<>(); + private final Function defaultPersistedStateSupplier = localNode -> new MockPersistedState(0L, + clusterState(0L, 0L, localNode, VotingConfiguration.EMPTY_CONFIG, VotingConfiguration.EMPTY_CONFIG, 0L)); + Cluster(int initialNodeCount) { this(initialNodeCount, true); } @@ -1533,9 +1537,7 @@ class ClusterNode { private ClusterStateApplyResponse clusterStateApplyResponse = ClusterStateApplyResponse.SUCCEED; ClusterNode(int nodeIndex, boolean masterEligible) { - this(nodeIndex, createDiscoveryNode(nodeIndex, masterEligible), - localNode -> new MockPersistedState(0L, - clusterState(0L, 0L, localNode, VotingConfiguration.EMPTY_CONFIG, VotingConfiguration.EMPTY_CONFIG, 0L))); + this(nodeIndex, createDiscoveryNode(nodeIndex, masterEligible), defaultPersistedStateSupplier); } ClusterNode(int nodeIndex, DiscoveryNode localNode, Function persistedStateSupplier) { From 5c68338a1c4bd3a41d566c845c2c3ee980c07d55 Mon Sep 17 00:00:00 2001 From: Tim Brooks Date: Mon, 14 Jan 2019 13:07:55 -0700 Subject: [PATCH 149/186] Implement ccr file restore (#37130) This is related to #35975. It implements a file based restore in the CcrRepository. The restore transfers files from the leader cluster to the follower cluster. It does not implement any advanced resiliency features at the moment. Any request failure will end the restore. --- .../common/bytes/PagedBytesReference.java | 13 +- .../bytes/ReleasablePagedBytesReference.java | 6 +- .../common/io/stream/BytesStreamOutput.java | 2 +- .../stream/ReleasableBytesStreamOutput.java | 2 +- .../blobstore/BlobStoreRepository.java | 240 ++------------ .../blobstore/FileRestoreContext.java | 304 ++++++++++++++++++ .../bytes/PagedBytesReferenceTests.java | 4 +- .../bytes/AbstractBytesReferenceTestCase.java | 2 +- .../index/shard/IndexShardTestCase.java | 10 + .../java/org/elasticsearch/xpack/ccr/Ccr.java | 3 + .../GetCcrRestoreFileChunkAction.java | 129 ++++++++ .../GetCcrRestoreFileChunkRequest.java | 76 +++++ .../PutCcrRestoreSessionAction.java | 16 +- .../PutCcrRestoreSessionRequest.java | 15 +- .../xpack/ccr/repository/CcrRepository.java | 141 +++++++- .../repository/CcrRestoreSourceService.java | 148 +++++++-- .../xpack/ccr/CcrRepositoryIT.java | 68 +++- .../CcrRestoreSourceServiceTests.java | 65 ++++ 18 files changed, 934 insertions(+), 310 deletions(-) create mode 100644 server/src/main/java/org/elasticsearch/repositories/blobstore/FileRestoreContext.java create mode 100644 x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/repositories/GetCcrRestoreFileChunkAction.java create mode 100644 x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/repositories/GetCcrRestoreFileChunkRequest.java diff --git a/server/src/main/java/org/elasticsearch/common/bytes/PagedBytesReference.java b/server/src/main/java/org/elasticsearch/common/bytes/PagedBytesReference.java index c754d6b8eae9e..f15b3b9cf3295 100644 --- a/server/src/main/java/org/elasticsearch/common/bytes/PagedBytesReference.java +++ b/server/src/main/java/org/elasticsearch/common/bytes/PagedBytesReference.java @@ -21,7 +21,6 @@ import org.apache.lucene.util.BytesRef; import org.apache.lucene.util.BytesRefIterator; -import org.elasticsearch.common.util.BigArrays; import org.elasticsearch.common.util.ByteArray; import org.elasticsearch.common.util.PageCacheRecycler; @@ -35,17 +34,15 @@ public class PagedBytesReference extends BytesReference { private static final int PAGE_SIZE = PageCacheRecycler.BYTE_PAGE_SIZE; - private final BigArrays bigarrays; - protected final ByteArray byteArray; + private final ByteArray byteArray; private final int offset; private final int length; - public PagedBytesReference(BigArrays bigarrays, ByteArray byteArray, int length) { - this(bigarrays, byteArray, 0, length); + public PagedBytesReference(ByteArray byteArray, int length) { + this(byteArray, 0, length); } - public PagedBytesReference(BigArrays bigarrays, ByteArray byteArray, int from, int length) { - this.bigarrays = bigarrays; + private PagedBytesReference(ByteArray byteArray, int from, int length) { this.byteArray = byteArray; this.offset = from; this.length = length; @@ -67,7 +64,7 @@ public BytesReference slice(int from, int length) { throw new IllegalArgumentException("can't slice a buffer with length [" + length() + "], with slice parameters from [" + from + "], length [" + length + "]"); } - return new PagedBytesReference(bigarrays, byteArray, offset + from, length); + return new PagedBytesReference(byteArray, offset + from, length); } @Override diff --git a/server/src/main/java/org/elasticsearch/common/bytes/ReleasablePagedBytesReference.java b/server/src/main/java/org/elasticsearch/common/bytes/ReleasablePagedBytesReference.java index ac90e546f7eb5..209a6edc5696a 100644 --- a/server/src/main/java/org/elasticsearch/common/bytes/ReleasablePagedBytesReference.java +++ b/server/src/main/java/org/elasticsearch/common/bytes/ReleasablePagedBytesReference.java @@ -21,7 +21,6 @@ import org.elasticsearch.common.lease.Releasable; import org.elasticsearch.common.lease.Releasables; -import org.elasticsearch.common.util.BigArrays; import org.elasticsearch.common.util.ByteArray; /** @@ -32,9 +31,8 @@ public final class ReleasablePagedBytesReference extends PagedBytesReference imp private final Releasable releasable; - public ReleasablePagedBytesReference(BigArrays bigarrays, ByteArray byteArray, int length, - Releasable releasable) { - super(bigarrays, byteArray, length); + public ReleasablePagedBytesReference(ByteArray byteArray, int length, Releasable releasable) { + super(byteArray, length); this.releasable = releasable; } diff --git a/server/src/main/java/org/elasticsearch/common/io/stream/BytesStreamOutput.java b/server/src/main/java/org/elasticsearch/common/io/stream/BytesStreamOutput.java index 768dbd1d17775..ad9cde3abbc48 100644 --- a/server/src/main/java/org/elasticsearch/common/io/stream/BytesStreamOutput.java +++ b/server/src/main/java/org/elasticsearch/common/io/stream/BytesStreamOutput.java @@ -140,7 +140,7 @@ public int size() { @Override public BytesReference bytes() { - return new PagedBytesReference(bigArrays, bytes, count); + return new PagedBytesReference(bytes, count); } /** diff --git a/server/src/main/java/org/elasticsearch/common/io/stream/ReleasableBytesStreamOutput.java b/server/src/main/java/org/elasticsearch/common/io/stream/ReleasableBytesStreamOutput.java index 5616da2db9636..725ecd1c3cc4f 100644 --- a/server/src/main/java/org/elasticsearch/common/io/stream/ReleasableBytesStreamOutput.java +++ b/server/src/main/java/org/elasticsearch/common/io/stream/ReleasableBytesStreamOutput.java @@ -56,7 +56,7 @@ public ReleasableBytesStreamOutput(int expectedSize, BigArrays bigArrays) { */ @Override public ReleasablePagedBytesReference bytes() { - return new ReleasablePagedBytesReference(bigArrays, bytes, count, releasable); + return new ReleasablePagedBytesReference(bytes, count, releasable); } @Override diff --git a/server/src/main/java/org/elasticsearch/repositories/blobstore/BlobStoreRepository.java b/server/src/main/java/org/elasticsearch/repositories/blobstore/BlobStoreRepository.java index 99cb1db3e3652..9f2297b48775b 100644 --- a/server/src/main/java/org/elasticsearch/repositories/blobstore/BlobStoreRepository.java +++ b/server/src/main/java/org/elasticsearch/repositories/blobstore/BlobStoreRepository.java @@ -22,15 +22,9 @@ import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; import org.apache.logging.log4j.message.ParameterizedMessage; -import org.apache.lucene.index.CorruptIndexException; import org.apache.lucene.index.IndexCommit; -import org.apache.lucene.index.IndexFormatTooNewException; -import org.apache.lucene.index.IndexFormatTooOldException; -import org.apache.lucene.index.IndexNotFoundException; -import org.apache.lucene.index.SegmentInfos; import org.apache.lucene.store.IOContext; import org.apache.lucene.store.IndexInput; -import org.apache.lucene.store.IndexOutput; import org.apache.lucene.store.RateLimiter; import org.apache.lucene.util.BytesRef; import org.apache.lucene.util.BytesRefBuilder; @@ -65,7 +59,6 @@ import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.unit.ByteSizeUnit; import org.elasticsearch.common.unit.ByteSizeValue; -import org.elasticsearch.common.util.iterable.Iterables; import org.elasticsearch.common.util.set.Sets; import org.elasticsearch.common.xcontent.LoggingDeprecationHandler; import org.elasticsearch.common.xcontent.NamedXContentRegistry; @@ -111,17 +104,12 @@ import java.nio.file.FileAlreadyExistsException; import java.nio.file.NoSuchFileException; 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.Set; import java.util.stream.Collectors; -import static java.util.Collections.emptyMap; -import static java.util.Collections.unmodifiableMap; import static org.elasticsearch.index.snapshots.blobstore.BlobStoreIndexShardSnapshot.FileInfo.canonicalName; /** @@ -864,9 +852,14 @@ public void snapshotShard(IndexShard shard, Store store, SnapshotId snapshotId, @Override public void restoreShard(IndexShard shard, SnapshotId snapshotId, Version version, IndexId indexId, ShardId snapshotShardId, RecoveryState recoveryState) { - final RestoreContext snapshotContext = new RestoreContext(shard, snapshotId, indexId, snapshotShardId, recoveryState); + final Context context = new Context(snapshotId, indexId, shard.shardId(), snapshotShardId); + BlobPath path = basePath().add("indices").add(indexId.getId()).add(Integer.toString(snapshotShardId.getId())); + BlobContainer blobContainer = blobStore().blobContainer(path); + final RestoreContext snapshotContext = new RestoreContext(shard, snapshotId, recoveryState, blobContainer); try { - snapshotContext.restore(); + BlobStoreIndexShardSnapshot snapshot = context.loadSnapshot(); + SnapshotFiles snapshotFiles = new SnapshotFiles(snapshot.snapshot(), snapshot.indexFiles()); + snapshotContext.restore(snapshotFiles); } catch (Exception e) { throw new IndexShardRestoreFailedException(shard.shardId(), "failed to restore snapshot [" + snapshotId + "]", e); } @@ -1459,216 +1452,29 @@ protected InputStream openSlice(long slice) throws IOException { /** * Context for restore operations */ - private class RestoreContext extends Context { + private class RestoreContext extends FileRestoreContext { - private final IndexShard targetShard; - - private final RecoveryState recoveryState; + private final BlobContainer blobContainer; /** * Constructs new restore context - * - * @param shard shard to restore into - * @param snapshotId snapshot id - * @param indexId id of the index being restored - * @param snapshotShardId shard in the snapshot that data should be restored from - * @param recoveryState recovery state to report progress + * @param indexShard shard to restore into + * @param snapshotId snapshot id + * @param recoveryState recovery state to report progress + * @param blobContainer the blob container to read the files from */ - RestoreContext(IndexShard shard, SnapshotId snapshotId, IndexId indexId, ShardId snapshotShardId, RecoveryState recoveryState) { - super(snapshotId, indexId, shard.shardId(), snapshotShardId); - this.recoveryState = recoveryState; - this.targetShard = shard; - } - - /** - * Performs restore operation - */ - public void restore() throws IOException { - final Store store = targetShard.store(); - store.incRef(); - try { - logger.debug("[{}] [{}] restoring to [{}] ...", snapshotId, metadata.name(), shardId); - BlobStoreIndexShardSnapshot snapshot = loadSnapshot(); - - if (snapshot.indexFiles().size() == 1 - && snapshot.indexFiles().get(0).physicalName().startsWith("segments_") - && snapshot.indexFiles().get(0).hasUnknownChecksum()) { - // If the shard has no documents, it will only contain a single segments_N file for the - // shard's snapshot. If we are restoring a snapshot created by a previous supported version, - // it is still possible that in that version, an empty shard has a segments_N file with an unsupported - // version (and no checksum), because we don't know the Lucene version to assign segments_N until we - // have written some data. Since the segments_N for an empty shard could have an incompatible Lucene - // version number and no checksum, even though the index itself is perfectly fine to restore, this - // empty shard would cause exceptions to be thrown. Since there is no data to restore from an empty - // shard anyway, we just create the empty shard here and then exit. - store.createEmpty(targetShard.indexSettings().getIndexMetaData().getCreationVersion().luceneVersion); - return; - } - - SnapshotFiles snapshotFiles = new SnapshotFiles(snapshot.snapshot(), snapshot.indexFiles()); - Store.MetadataSnapshot recoveryTargetMetadata; - try { - // this will throw an IOException if the store has no segments infos file. The - // store can still have existing files but they will be deleted just before being - // restored. - recoveryTargetMetadata = targetShard.snapshotStoreMetadata(); - } catch (IndexNotFoundException e) { - // happens when restore to an empty shard, not a big deal - logger.trace("[{}] [{}] restoring from to an empty shard", shardId, snapshotId); - recoveryTargetMetadata = Store.MetadataSnapshot.EMPTY; - } catch (IOException e) { - logger.warn(() -> new ParameterizedMessage("{} Can't read metadata from store, will not reuse any " + - "local file while restoring", shardId), e); - recoveryTargetMetadata = Store.MetadataSnapshot.EMPTY; - } - - final List filesToRecover = new ArrayList<>(); - final Map snapshotMetaData = new HashMap<>(); - final Map fileInfos = new HashMap<>(); - for (final BlobStoreIndexShardSnapshot.FileInfo fileInfo : snapshot.indexFiles()) { - try { - // in 1.3.3 we added additional hashes for .si / segments_N files - // to ensure we don't double the space in the repo since old snapshots - // don't have this hash we try to read that hash from the blob store - // in a bwc compatible way. - maybeRecalculateMetadataHash(blobContainer, fileInfo, recoveryTargetMetadata); - } catch (Exception e) { - // if the index is broken we might not be able to read it - logger.warn(() -> new ParameterizedMessage("{} Can't calculate hash from blog for file [{}] [{}]", - shardId, fileInfo.physicalName(), fileInfo.metadata()), e); - } - snapshotMetaData.put(fileInfo.metadata().name(), fileInfo.metadata()); - fileInfos.put(fileInfo.metadata().name(), fileInfo); - } - - final Store.MetadataSnapshot sourceMetaData = new Store.MetadataSnapshot(unmodifiableMap(snapshotMetaData), emptyMap(), 0); - - final StoreFileMetaData restoredSegmentsFile = sourceMetaData.getSegmentsFile(); - if (restoredSegmentsFile == null) { - throw new IndexShardRestoreFailedException(shardId, "Snapshot has no segments file"); - } - - final Store.RecoveryDiff diff = sourceMetaData.recoveryDiff(recoveryTargetMetadata); - for (StoreFileMetaData md : diff.identical) { - BlobStoreIndexShardSnapshot.FileInfo fileInfo = fileInfos.get(md.name()); - recoveryState.getIndex().addFileDetail(fileInfo.name(), fileInfo.length(), true); - if (logger.isTraceEnabled()) { - logger.trace("[{}] [{}] not_recovering [{}] from [{}], exists in local store and is same", - shardId, snapshotId, fileInfo.physicalName(), fileInfo.name()); - } - } - - for (StoreFileMetaData md : Iterables.concat(diff.different, diff.missing)) { - BlobStoreIndexShardSnapshot.FileInfo fileInfo = fileInfos.get(md.name()); - filesToRecover.add(fileInfo); - recoveryState.getIndex().addFileDetail(fileInfo.name(), fileInfo.length(), false); - if (logger.isTraceEnabled()) { - logger.trace("[{}] [{}] recovering [{}] from [{}], exists in local store but is different", shardId, snapshotId, - fileInfo.physicalName(), fileInfo.name()); - } - } - - if (filesToRecover.isEmpty()) { - logger.trace("no files to recover, all exists within the local store"); - } - - try { - // list of all existing store files - final List deleteIfExistFiles = Arrays.asList(store.directory().listAll()); - - // restore the files from the snapshot to the Lucene store - for (final BlobStoreIndexShardSnapshot.FileInfo fileToRecover : filesToRecover) { - // if a file with a same physical name already exist in the store we need to delete it - // before restoring it from the snapshot. We could be lenient and try to reuse the existing - // store files (and compare their names/length/checksum again with the snapshot files) but to - // avoid extra complexity we simply delete them and restore them again like StoreRecovery - // does with dangling indices. Any existing store file that is not restored from the snapshot - // will be clean up by RecoveryTarget.cleanFiles(). - final String physicalName = fileToRecover.physicalName(); - if (deleteIfExistFiles.contains(physicalName)) { - logger.trace("[{}] [{}] deleting pre-existing file [{}]", shardId, snapshotId, physicalName); - store.directory().deleteFile(physicalName); - } - - logger.trace("[{}] [{}] restoring file [{}]", shardId, snapshotId, fileToRecover.name()); - restoreFile(fileToRecover, store); - } - } catch (IOException ex) { - throw new IndexShardRestoreFailedException(shardId, "Failed to recover index", ex); - } - - // read the snapshot data persisted - final SegmentInfos segmentCommitInfos; - try { - segmentCommitInfos = Lucene.pruneUnreferencedFiles(restoredSegmentsFile.name(), store.directory()); - } catch (IOException e) { - throw new IndexShardRestoreFailedException(shardId, "Failed to fetch index version after copying it over", e); - } - recoveryState.getIndex().updateVersion(segmentCommitInfos.getVersion()); - - /// now, go over and clean files that are in the store, but were not in the snapshot - try { - for (String storeFile : store.directory().listAll()) { - if (Store.isAutogenerated(storeFile) || snapshotFiles.containPhysicalIndexFile(storeFile)) { - continue; //skip write.lock, checksum files and files that exist in the snapshot - } - try { - store.deleteQuiet("restore", storeFile); - store.directory().deleteFile(storeFile); - } catch (IOException e) { - logger.warn("[{}] failed to delete file [{}] during snapshot cleanup", snapshotId, storeFile); - } - } - } catch (IOException e) { - logger.warn("[{}] failed to list directory - some of files might not be deleted", snapshotId); - } - } finally { - store.decRef(); - } + RestoreContext(IndexShard indexShard, SnapshotId snapshotId, RecoveryState recoveryState, BlobContainer blobContainer) { + super(metadata.name(), indexShard, snapshotId, recoveryState, BUFFER_SIZE); + this.blobContainer = blobContainer; } - /** - * Restores a file - * This is asynchronous method. Upon completion of the operation latch is getting counted down and any failures are - * added to the {@code failures} list - * - * @param fileInfo file to be restored - */ - private void restoreFile(final BlobStoreIndexShardSnapshot.FileInfo fileInfo, final Store store) throws IOException { - boolean success = false; - - try (InputStream partSliceStream = new PartSliceStream(blobContainer, fileInfo)) { - final InputStream stream; - if (restoreRateLimiter == null) { - stream = partSliceStream; - } else { - stream = new RateLimitingInputStream(partSliceStream, restoreRateLimiter, restoreRateLimitingTimeInNanos::inc); - } - - try (IndexOutput indexOutput = store.createVerifyingOutput(fileInfo.physicalName(), - fileInfo.metadata(), IOContext.DEFAULT)) { - final byte[] buffer = new byte[BUFFER_SIZE]; - int length; - while ((length = stream.read(buffer)) > 0) { - indexOutput.writeBytes(buffer, 0, length); - recoveryState.getIndex().addRecoveredBytesToFile(fileInfo.name(), length); - } - Store.verify(indexOutput); - indexOutput.close(); - store.directory().sync(Collections.singleton(fileInfo.physicalName())); - success = true; - } catch (CorruptIndexException | IndexFormatTooOldException | IndexFormatTooNewException ex) { - try { - store.markStoreCorrupted(ex); - } catch (IOException e) { - logger.warn("store cannot be marked as corrupted", e); - } - throw ex; - } finally { - if (success == false) { - store.deleteQuiet(fileInfo.physicalName()); - } - } + @Override + protected InputStream fileInputStream(BlobStoreIndexShardSnapshot.FileInfo fileInfo) { + if (restoreRateLimiter == null) { + return new PartSliceStream(blobContainer, fileInfo); + } else { + RateLimitingInputStream.Listener listener = restoreRateLimitingTimeInNanos::inc; + return new RateLimitingInputStream(new PartSliceStream(blobContainer, fileInfo), restoreRateLimiter, listener); } } } diff --git a/server/src/main/java/org/elasticsearch/repositories/blobstore/FileRestoreContext.java b/server/src/main/java/org/elasticsearch/repositories/blobstore/FileRestoreContext.java new file mode 100644 index 0000000000000..2f837812ae2e2 --- /dev/null +++ b/server/src/main/java/org/elasticsearch/repositories/blobstore/FileRestoreContext.java @@ -0,0 +1,304 @@ +/* + * 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.repositories.blobstore; + +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; +import org.apache.logging.log4j.message.ParameterizedMessage; +import org.apache.lucene.index.CorruptIndexException; +import org.apache.lucene.index.IndexFormatTooNewException; +import org.apache.lucene.index.IndexFormatTooOldException; +import org.apache.lucene.index.SegmentInfos; +import org.apache.lucene.store.IOContext; +import org.apache.lucene.store.IndexOutput; +import org.apache.lucene.util.BytesRef; +import org.apache.lucene.util.BytesRefBuilder; +import org.elasticsearch.common.lucene.Lucene; +import org.elasticsearch.common.util.iterable.Iterables; +import org.elasticsearch.index.IndexNotFoundException; +import org.elasticsearch.index.shard.IndexShard; +import org.elasticsearch.index.shard.ShardId; +import org.elasticsearch.index.snapshots.IndexShardRestoreFailedException; +import org.elasticsearch.index.snapshots.blobstore.BlobStoreIndexShardSnapshot; +import org.elasticsearch.index.snapshots.blobstore.SnapshotFiles; +import org.elasticsearch.index.store.Store; +import org.elasticsearch.index.store.StoreFileMetaData; +import org.elasticsearch.indices.recovery.RecoveryState; +import org.elasticsearch.snapshots.SnapshotId; + +import java.io.IOException; +import java.io.InputStream; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +import static java.util.Collections.emptyMap; +import static java.util.Collections.unmodifiableMap; + +/** + * This context will execute a file restore of the lucene files. It is primarily designed to be used to + * restore from some form of a snapshot. It will setup a new store, identify files that need to be copied + * for the source, and perform the copies. Implementers must implement the functionality of opening the + * underlying file streams for snapshotted lucene file. + */ +public abstract class FileRestoreContext { + + private static final Logger logger = LogManager.getLogger(FileRestoreContext.class); + + private final String repositoryName; + private final IndexShard indexShard; + private final RecoveryState recoveryState; + private final SnapshotId snapshotId; + private final ShardId shardId; + private final int bufferSize; + + /** + * Constructs new restore context + * + * @param indexShard shard to restore into + * @param snapshotId snapshot id + * @param recoveryState recovery state to report progress + * @param bufferSize buffer size for restore + */ + protected FileRestoreContext(String repositoryName, IndexShard indexShard, SnapshotId snapshotId, RecoveryState recoveryState, + int bufferSize) { + this.repositoryName = repositoryName; + this.recoveryState = recoveryState; + this.indexShard = indexShard; + this.snapshotId = snapshotId; + this.shardId = indexShard.shardId(); + this.bufferSize = bufferSize; + } + + /** + * Performs restore operation + */ + public void restore(SnapshotFiles snapshotFiles) throws IOException { + final Store store = indexShard.store(); + store.incRef(); + try { + logger.debug("[{}] [{}] restoring to [{}] ...", snapshotId, repositoryName, shardId); + + if (snapshotFiles.indexFiles().size() == 1 + && snapshotFiles.indexFiles().get(0).physicalName().startsWith("segments_") + && snapshotFiles.indexFiles().get(0).hasUnknownChecksum()) { + // If the shard has no documents, it will only contain a single segments_N file for the + // shard's snapshot. If we are restoring a snapshot created by a previous supported version, + // it is still possible that in that version, an empty shard has a segments_N file with an unsupported + // version (and no checksum), because we don't know the Lucene version to assign segments_N until we + // have written some data. Since the segments_N for an empty shard could have an incompatible Lucene + // version number and no checksum, even though the index itself is perfectly fine to restore, this + // empty shard would cause exceptions to be thrown. Since there is no data to restore from an empty + // shard anyway, we just create the empty shard here and then exit. + store.createEmpty(indexShard.indexSettings().getIndexMetaData().getCreationVersion().luceneVersion); + return; + } + + Store.MetadataSnapshot recoveryTargetMetadata; + try { + // this will throw an IOException if the store has no segments infos file. The + // store can still have existing files but they will be deleted just before being + // restored. + recoveryTargetMetadata = indexShard.snapshotStoreMetadata(); + } catch (IndexNotFoundException e) { + // happens when restore to an empty shard, not a big deal + logger.trace("[{}] [{}] restoring from to an empty shard", shardId, snapshotId); + recoveryTargetMetadata = Store.MetadataSnapshot.EMPTY; + } catch (IOException e) { + logger.warn(new ParameterizedMessage("[{}] [{}] Can't read metadata from store, will not reuse local files during restore", + shardId, snapshotId), e); + recoveryTargetMetadata = Store.MetadataSnapshot.EMPTY; + } + + final List filesToRecover = new ArrayList<>(); + final Map snapshotMetaData = new HashMap<>(); + final Map fileInfos = new HashMap<>(); + for (final BlobStoreIndexShardSnapshot.FileInfo fileInfo : snapshotFiles.indexFiles()) { + try { + // in 1.3.3 we added additional hashes for .si / segments_N files + // to ensure we don't double the space in the repo since old snapshots + // don't have this hash we try to read that hash from the blob store + // in a bwc compatible way. + maybeRecalculateMetadataHash(fileInfo, recoveryTargetMetadata); + } catch (Exception e) { + // if the index is broken we might not be able to read it + logger.warn(new ParameterizedMessage("[{}] Can't calculate hash from blog for file [{}] [{}]", shardId, + fileInfo.physicalName(), fileInfo.metadata()), e); + } + snapshotMetaData.put(fileInfo.metadata().name(), fileInfo.metadata()); + fileInfos.put(fileInfo.metadata().name(), fileInfo); + } + + final Store.MetadataSnapshot sourceMetaData = new Store.MetadataSnapshot(unmodifiableMap(snapshotMetaData), emptyMap(), 0); + + final StoreFileMetaData restoredSegmentsFile = sourceMetaData.getSegmentsFile(); + if (restoredSegmentsFile == null) { + throw new IndexShardRestoreFailedException(shardId, "Snapshot has no segments file"); + } + + final Store.RecoveryDiff diff = sourceMetaData.recoveryDiff(recoveryTargetMetadata); + for (StoreFileMetaData md : diff.identical) { + BlobStoreIndexShardSnapshot.FileInfo fileInfo = fileInfos.get(md.name()); + recoveryState.getIndex().addFileDetail(fileInfo.name(), fileInfo.length(), true); + if (logger.isTraceEnabled()) { + logger.trace("[{}] [{}] not_recovering file [{}] from [{}], exists in local store and is same", shardId, snapshotId, + fileInfo.physicalName(), fileInfo.name()); + } + } + + for (StoreFileMetaData md : concat(diff)) { + BlobStoreIndexShardSnapshot.FileInfo fileInfo = fileInfos.get(md.name()); + filesToRecover.add(fileInfo); + recoveryState.getIndex().addFileDetail(fileInfo.name(), fileInfo.length(), false); + if (logger.isTraceEnabled()) { + logger.trace("[{}] [{}] recovering [{}] from [{}]", shardId, snapshotId, + fileInfo.physicalName(), fileInfo.name()); + } + } + + if (filesToRecover.isEmpty()) { + logger.trace("[{}] [{}] no files to recover, all exist within the local store", shardId, snapshotId); + } + + try { + // list of all existing store files + final List deleteIfExistFiles = Arrays.asList(store.directory().listAll()); + + // restore the files from the snapshot to the Lucene store + for (final BlobStoreIndexShardSnapshot.FileInfo fileToRecover : filesToRecover) { + // if a file with a same physical name already exist in the store we need to delete it + // before restoring it from the snapshot. We could be lenient and try to reuse the existing + // store files (and compare their names/length/checksum again with the snapshot files) but to + // avoid extra complexity we simply delete them and restore them again like StoreRecovery + // does with dangling indices. Any existing store file that is not restored from the snapshot + // will be clean up by RecoveryTarget.cleanFiles(). + final String physicalName = fileToRecover.physicalName(); + if (deleteIfExistFiles.contains(physicalName)) { + logger.trace("[{}] [{}] deleting pre-existing file [{}]", shardId, snapshotId, physicalName); + store.directory().deleteFile(physicalName); + } + + logger.trace("[{}] [{}] restoring file [{}]", shardId, snapshotId, fileToRecover.name()); + restoreFile(fileToRecover, store); + } + } catch (IOException ex) { + throw new IndexShardRestoreFailedException(shardId, "Failed to recover index", ex); + } + + // read the snapshot data persisted + final SegmentInfos segmentCommitInfos; + try { + segmentCommitInfos = Lucene.pruneUnreferencedFiles(restoredSegmentsFile.name(), store.directory()); + } catch (IOException e) { + throw new IndexShardRestoreFailedException(shardId, "Failed to fetch index version after copying it over", e); + } + recoveryState.getIndex().updateVersion(segmentCommitInfos.getVersion()); + + /// now, go over and clean files that are in the store, but were not in the snapshot + try { + for (String storeFile : store.directory().listAll()) { + if (Store.isAutogenerated(storeFile) || snapshotFiles.containPhysicalIndexFile(storeFile)) { + continue; //skip write.lock, checksum files and files that exist in the snapshot + } + try { + store.deleteQuiet("restore", storeFile); + store.directory().deleteFile(storeFile); + } catch (IOException e) { + logger.warn("[{}] [{}] failed to delete file [{}] during snapshot cleanup", shardId, snapshotId, storeFile); + } + } + } catch (IOException e) { + logger.warn("[{}] [{}] failed to list directory - some of files might not be deleted", shardId, snapshotId); + } + } finally { + store.decRef(); + } + } + + protected abstract InputStream fileInputStream(BlobStoreIndexShardSnapshot.FileInfo fileInfo); + + @SuppressWarnings("unchecked") + private Iterable concat(Store.RecoveryDiff diff) { + return Iterables.concat(diff.different, diff.missing); + } + + /** + * Restores a file + * + * @param fileInfo file to be restored + */ + private void restoreFile(final BlobStoreIndexShardSnapshot.FileInfo fileInfo, final Store store) throws IOException { + boolean success = false; + + try (InputStream stream = fileInputStream(fileInfo)) { + try (IndexOutput indexOutput = store.createVerifyingOutput(fileInfo.physicalName(), fileInfo.metadata(), IOContext.DEFAULT)) { + final byte[] buffer = new byte[bufferSize]; + int length; + while ((length = stream.read(buffer)) > 0) { + indexOutput.writeBytes(buffer, 0, length); + recoveryState.getIndex().addRecoveredBytesToFile(fileInfo.name(), length); + } + Store.verify(indexOutput); + indexOutput.close(); + store.directory().sync(Collections.singleton(fileInfo.physicalName())); + success = true; + } catch (CorruptIndexException | IndexFormatTooOldException | IndexFormatTooNewException ex) { + try { + store.markStoreCorrupted(ex); + } catch (IOException e) { + logger.warn("store cannot be marked as corrupted", e); + } + throw ex; + } finally { + if (success == false) { + store.deleteQuiet(fileInfo.physicalName()); + } + } + } + } + + /** + * This is a BWC layer to ensure we update the snapshots metadata with the corresponding hashes before we compare them. + * The new logic for StoreFileMetaData reads the entire {@code .si} and {@code segments.n} files to strengthen the + * comparison of the files on a per-segment / per-commit level. + */ + private void maybeRecalculateMetadataHash(final BlobStoreIndexShardSnapshot.FileInfo fileInfo, Store.MetadataSnapshot snapshot) + throws IOException { + final StoreFileMetaData metadata; + if (fileInfo != null && (metadata = snapshot.get(fileInfo.physicalName())) != null) { + if (metadata.hash().length > 0 && fileInfo.metadata().hash().length == 0) { + // we have a hash - check if our repo has a hash too otherwise we have + // to calculate it. + // we might have multiple parts even though the file is small... make sure we read all of it. + try (InputStream stream = fileInputStream(fileInfo)) { + BytesRefBuilder builder = new BytesRefBuilder(); + Store.MetadataSnapshot.hashFile(builder, stream, fileInfo.length()); + BytesRef hash = fileInfo.metadata().hash(); // reset the file infos metadata hash + assert hash.length == 0; + hash.bytes = builder.bytes(); + hash.offset = 0; + hash.length = builder.length(); + } + } + } + } +} diff --git a/server/src/test/java/org/elasticsearch/common/bytes/PagedBytesReferenceTests.java b/server/src/test/java/org/elasticsearch/common/bytes/PagedBytesReferenceTests.java index 0938b70906f52..40e02c560a9c0 100644 --- a/server/src/test/java/org/elasticsearch/common/bytes/PagedBytesReferenceTests.java +++ b/server/src/test/java/org/elasticsearch/common/bytes/PagedBytesReferenceTests.java @@ -120,8 +120,8 @@ public void testEquals() { } // get refs & compare - BytesReference pbr = new PagedBytesReference(bigarrays, ba1, length); - BytesReference pbr2 = new PagedBytesReference(bigarrays, ba2, length); + BytesReference pbr = new PagedBytesReference(ba1, length); + BytesReference pbr2 = new PagedBytesReference(ba2, length); assertEquals(pbr, pbr2); int offsetToFlip = randomIntBetween(0, length - 1); int value = ~Byte.toUnsignedInt(ba1.get(offsetToFlip)); diff --git a/test/framework/src/main/java/org/elasticsearch/common/bytes/AbstractBytesReferenceTestCase.java b/test/framework/src/main/java/org/elasticsearch/common/bytes/AbstractBytesReferenceTestCase.java index 1a00698325974..1a907feabe24a 100644 --- a/test/framework/src/main/java/org/elasticsearch/common/bytes/AbstractBytesReferenceTestCase.java +++ b/test/framework/src/main/java/org/elasticsearch/common/bytes/AbstractBytesReferenceTestCase.java @@ -528,7 +528,7 @@ public void testEquals() throws IOException { public void testSliceEquals() { int length = randomIntBetween(100, PAGE_SIZE * randomIntBetween(2, 5)); ByteArray ba1 = bigarrays.newByteArray(length, false); - BytesReference pbr = new PagedBytesReference(bigarrays, ba1, length); + BytesReference pbr = new PagedBytesReference(ba1, length); // test equality of slices int sliceFrom = randomIntBetween(0, pbr.length()); diff --git a/test/framework/src/main/java/org/elasticsearch/index/shard/IndexShardTestCase.java b/test/framework/src/main/java/org/elasticsearch/index/shard/IndexShardTestCase.java index 367e0ded60775..0ffc9f05ff265 100644 --- a/test/framework/src/main/java/org/elasticsearch/index/shard/IndexShardTestCase.java +++ b/test/framework/src/main/java/org/elasticsearch/index/shard/IndexShardTestCase.java @@ -448,6 +448,16 @@ protected IndexShard newStartedShard(final boolean primary) throws IOException { return newStartedShard(primary, Settings.EMPTY, new InternalEngineFactory()); } + /** + * Creates a new empty shard and starts it. + * + * @param primary controls whether the shard will be a primary or a replica. + * @param settings the settings to use for this shard + */ + protected IndexShard newStartedShard(final boolean primary, Settings settings) throws IOException { + return newStartedShard(primary, settings, new InternalEngineFactory()); + } + /** * Creates a new empty shard with the specified settings and engine factory and starts it. * diff --git a/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/Ccr.java b/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/Ccr.java index b3d2d05048211..370d017a4bde7 100644 --- a/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/Ccr.java +++ b/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/Ccr.java @@ -61,6 +61,7 @@ import org.elasticsearch.xpack.ccr.action.bulk.TransportBulkShardOperationsAction; import org.elasticsearch.xpack.ccr.action.repositories.ClearCcrRestoreSessionAction; import org.elasticsearch.xpack.ccr.action.repositories.DeleteInternalCcrRepositoryAction; +import org.elasticsearch.xpack.ccr.action.repositories.GetCcrRestoreFileChunkAction; import org.elasticsearch.xpack.ccr.action.repositories.PutCcrRestoreSessionAction; import org.elasticsearch.xpack.ccr.action.repositories.PutInternalCcrRepositoryAction; import org.elasticsearch.xpack.ccr.index.engine.FollowingEngineFactory; @@ -193,6 +194,8 @@ public List> getPersistentTasksExecutor(ClusterServic PutCcrRestoreSessionAction.TransportPutCcrRestoreSessionAction.class), new ActionHandler<>(ClearCcrRestoreSessionAction.INSTANCE, ClearCcrRestoreSessionAction.TransportDeleteCcrRestoreSessionAction.class), + new ActionHandler<>(GetCcrRestoreFileChunkAction.INSTANCE, + GetCcrRestoreFileChunkAction.TransportGetCcrRestoreFileChunkAction.class), // stats action new ActionHandler<>(FollowStatsAction.INSTANCE, TransportFollowStatsAction.class), new ActionHandler<>(CcrStatsAction.INSTANCE, TransportCcrStatsAction.class), diff --git a/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/repositories/GetCcrRestoreFileChunkAction.java b/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/repositories/GetCcrRestoreFileChunkAction.java new file mode 100644 index 0000000000000..3f473f25c2411 --- /dev/null +++ b/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/repositories/GetCcrRestoreFileChunkAction.java @@ -0,0 +1,129 @@ +/* + * 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.ccr.action.repositories; + +import org.elasticsearch.action.Action; +import org.elasticsearch.action.ActionListener; +import org.elasticsearch.action.ActionResponse; +import org.elasticsearch.action.support.ActionFilters; +import org.elasticsearch.action.support.HandledTransportAction; +import org.elasticsearch.common.bytes.BytesReference; +import org.elasticsearch.common.bytes.ReleasablePagedBytesReference; +import org.elasticsearch.common.inject.Inject; +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.util.BigArrays; +import org.elasticsearch.common.util.ByteArray; +import org.elasticsearch.common.util.concurrent.AbstractRunnable; +import org.elasticsearch.tasks.Task; +import org.elasticsearch.threadpool.ThreadPool; +import org.elasticsearch.transport.TransportActionProxy; +import org.elasticsearch.transport.TransportService; +import org.elasticsearch.xpack.ccr.repository.CcrRestoreSourceService; + +import java.io.IOException; + +public class GetCcrRestoreFileChunkAction extends Action { + + public static final GetCcrRestoreFileChunkAction INSTANCE = new GetCcrRestoreFileChunkAction(); + public static final String NAME = "internal:admin/ccr/restore/file_chunk/get"; + + private GetCcrRestoreFileChunkAction() { + super(NAME); + } + + @Override + public GetCcrRestoreFileChunkResponse newResponse() { + throw new UnsupportedOperationException(); + } + + @Override + public Writeable.Reader getResponseReader() { + return GetCcrRestoreFileChunkResponse::new; + } + + + public static class TransportGetCcrRestoreFileChunkAction + extends HandledTransportAction { + + private final CcrRestoreSourceService restoreSourceService; + private final ThreadPool threadPool; + private final BigArrays bigArrays; + + @Inject + public TransportGetCcrRestoreFileChunkAction(BigArrays bigArrays, TransportService transportService, ActionFilters actionFilters, + CcrRestoreSourceService restoreSourceService) { + super(NAME, transportService, actionFilters, GetCcrRestoreFileChunkRequest::new); + TransportActionProxy.registerProxyAction(transportService, NAME, GetCcrRestoreFileChunkResponse::new); + this.threadPool = transportService.getThreadPool(); + this.restoreSourceService = restoreSourceService; + this.bigArrays = bigArrays; + } + + @Override + protected void doExecute(Task task, GetCcrRestoreFileChunkRequest request, + ActionListener listener) { + threadPool.generic().execute(new AbstractRunnable() { + @Override + public void onFailure(Exception e) { + listener.onFailure(e); + } + + @Override + protected void doRun() throws Exception { + int bytesRequested = request.getSize(); + ByteArray array = bigArrays.newByteArray(bytesRequested, false); + String fileName = request.getFileName(); + String sessionUUID = request.getSessionUUID(); + // This is currently safe to do because calling `onResponse` will serialize the bytes to the network layer data + // structure on the same thread. So the bytes will be copied before the reference is released. + try (ReleasablePagedBytesReference reference = new ReleasablePagedBytesReference(array, bytesRequested, array)) { + try (CcrRestoreSourceService.SessionReader sessionReader = restoreSourceService.getSessionReader(sessionUUID)) { + long offsetAfterRead = sessionReader.readFileBytes(fileName, reference); + long offsetBeforeRead = offsetAfterRead - reference.length(); + listener.onResponse(new GetCcrRestoreFileChunkResponse(offsetBeforeRead, reference)); + } + } + } + }); + } + } + + public static class GetCcrRestoreFileChunkResponse extends ActionResponse { + + private final long offset; + private final BytesReference chunk; + + GetCcrRestoreFileChunkResponse(StreamInput streamInput) throws IOException { + super(streamInput); + offset = streamInput.readVLong(); + chunk = streamInput.readBytesReference(); + } + + GetCcrRestoreFileChunkResponse(long offset, BytesReference chunk) { + this.offset = offset; + this.chunk = chunk; + } + + public long getOffset() { + return offset; + } + + public BytesReference getChunk() { + return chunk; + } + + @Override + public void writeTo(StreamOutput out) throws IOException { + super.writeTo(out); + out.writeVLong(offset); + out.writeBytesReference(chunk); + } + + } +} diff --git a/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/repositories/GetCcrRestoreFileChunkRequest.java b/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/repositories/GetCcrRestoreFileChunkRequest.java new file mode 100644 index 0000000000000..5da0efcb372f5 --- /dev/null +++ b/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/repositories/GetCcrRestoreFileChunkRequest.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.ccr.action.repositories; + +import org.elasticsearch.action.ActionRequest; +import org.elasticsearch.action.ActionRequestValidationException; +import org.elasticsearch.cluster.node.DiscoveryNode; +import org.elasticsearch.common.io.stream.StreamInput; +import org.elasticsearch.common.io.stream.StreamOutput; +import org.elasticsearch.transport.RemoteClusterAwareRequest; + +import java.io.IOException; + +public class GetCcrRestoreFileChunkRequest extends ActionRequest implements RemoteClusterAwareRequest { + + private final DiscoveryNode node; + private final String sessionUUID; + private final String fileName; + private final int size; + + @Override + public ActionRequestValidationException validate() { + return null; + } + + public GetCcrRestoreFileChunkRequest(DiscoveryNode node, String sessionUUID, String fileName, int size) { + this.node = node; + this.sessionUUID = sessionUUID; + this.fileName = fileName; + this.size = size; + assert size > -1 : "The file chunk request size must be positive. Found: [" + size + "]."; + } + + GetCcrRestoreFileChunkRequest(StreamInput in) throws IOException { + super(in); + node = null; + sessionUUID = in.readString(); + fileName = in.readString(); + size = in.readVInt(); + } + + @Override + public void writeTo(StreamOutput out) throws IOException { + super.writeTo(out); + out.writeString(sessionUUID); + out.writeString(fileName); + out.writeVInt(size); + } + + @Override + public void readFrom(StreamInput in) throws IOException { + throw new UnsupportedOperationException(); + } + + String getSessionUUID() { + return sessionUUID; + } + + String getFileName() { + return fileName; + } + + int getSize() { + return size; + } + + @Override + public DiscoveryNode getPreferredTargetNode() { + assert node != null : "Target node is null"; + return node; + } +} diff --git a/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/repositories/PutCcrRestoreSessionAction.java b/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/repositories/PutCcrRestoreSessionAction.java index 2a1b354f5d8ea..07ee076135a1b 100644 --- a/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/repositories/PutCcrRestoreSessionAction.java +++ b/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/repositories/PutCcrRestoreSessionAction.java @@ -22,6 +22,7 @@ import org.elasticsearch.index.shard.IndexShard; import org.elasticsearch.index.shard.ShardId; import org.elasticsearch.index.shard.ShardNotFoundException; +import org.elasticsearch.index.store.Store; import org.elasticsearch.indices.IndicesService; import org.elasticsearch.threadpool.ThreadPool; import org.elasticsearch.transport.TransportService; @@ -70,8 +71,8 @@ protected PutCcrRestoreSessionResponse shardOperation(PutCcrRestoreSessionReques if (indexShard == null) { throw new ShardNotFoundException(shardId); } - ccrRestoreService.openSession(request.getSessionUUID(), indexShard); - return new PutCcrRestoreSessionResponse(clusterService.localNode()); + Store.MetadataSnapshot storeFileMetaData = ccrRestoreService.openSession(request.getSessionUUID(), indexShard); + return new PutCcrRestoreSessionResponse(clusterService.localNode(), storeFileMetaData); } @Override @@ -95,33 +96,42 @@ protected ShardsIterator shards(ClusterState state, InternalRequest request) { public static class PutCcrRestoreSessionResponse extends ActionResponse { private DiscoveryNode node; + private Store.MetadataSnapshot storeFileMetaData; PutCcrRestoreSessionResponse() { } - PutCcrRestoreSessionResponse(DiscoveryNode node) { + PutCcrRestoreSessionResponse(DiscoveryNode node, Store.MetadataSnapshot storeFileMetaData) { this.node = node; + this.storeFileMetaData = storeFileMetaData; } PutCcrRestoreSessionResponse(StreamInput in) throws IOException { super(in); node = new DiscoveryNode(in); + storeFileMetaData = new Store.MetadataSnapshot(in); } @Override public void readFrom(StreamInput in) throws IOException { super.readFrom(in); node = new DiscoveryNode(in); + storeFileMetaData = new Store.MetadataSnapshot(in); } @Override public void writeTo(StreamOutput out) throws IOException { super.writeTo(out); node.writeTo(out); + storeFileMetaData.writeTo(out); } public DiscoveryNode getNode() { return node; } + + public Store.MetadataSnapshot getStoreFileMetaData() { + return storeFileMetaData; + } } } diff --git a/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/repositories/PutCcrRestoreSessionRequest.java b/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/repositories/PutCcrRestoreSessionRequest.java index 2b94193f674af..da0c43116ee76 100644 --- a/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/repositories/PutCcrRestoreSessionRequest.java +++ b/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/repositories/PutCcrRestoreSessionRequest.java @@ -11,7 +11,6 @@ import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.io.stream.StreamOutput; import org.elasticsearch.index.shard.ShardId; -import org.elasticsearch.index.store.Store; import java.io.IOException; @@ -19,16 +18,14 @@ public class PutCcrRestoreSessionRequest extends SingleShardRequest ccrMetaData = indexShard.indexSettings().getIndexMetaData().getCustomData(Ccr.CCR_CUSTOM_METADATA_KEY); String leaderUUID = ccrMetaData.get(Ccr.CCR_CUSTOM_METADATA_LEADER_INDEX_UUID_KEY); Index leaderIndex = new Index(shardId.getIndexName(), leaderUUID); ShardId leaderShardId = new ShardId(leaderIndex, shardId.getId()); Client remoteClient = client.getRemoteClusterClient(remoteClusterAlias); - String sessionUUID = UUIDs.randomBase64UUID(); - PutCcrRestoreSessionAction.PutCcrRestoreSessionResponse response = remoteClient.execute(PutCcrRestoreSessionAction.INSTANCE, - new PutCcrRestoreSessionRequest(sessionUUID, leaderShardId, recoveryMetadata)).actionGet(); - DiscoveryNode node = response.getNode(); - // TODO: Implement file restore - closeSession(remoteClient, node, sessionUUID); + // TODO: There should be some local timeout. And if the remote cluster returns an unknown session + // response, we should be able to retry by creating a new session. + String name = metadata.name(); + try (RestoreSession restoreSession = RestoreSession.openSession(name, remoteClient, leaderShardId, indexShard, recoveryState)) { + restoreSession.restoreFiles(); + } catch (Exception e) { + throw new IndexShardRestoreFailedException(indexShard.shardId(), "failed to restore snapshot [" + snapshotId + "]", e); + } + maybeUpdateMappings(client, remoteClient, leaderIndex, indexShard.indexSettings()); } @@ -278,9 +286,108 @@ private void maybeUpdateMappings(Client localClient, Client remoteClient, Index } } - private void closeSession(Client remoteClient, DiscoveryNode node, String sessionUUID) { - ClearCcrRestoreSessionRequest clearRequest = new ClearCcrRestoreSessionRequest(sessionUUID, node); - ClearCcrRestoreSessionAction.ClearCcrRestoreSessionResponse response = - remoteClient.execute(ClearCcrRestoreSessionAction.INSTANCE, clearRequest).actionGet(); + private static class RestoreSession extends FileRestoreContext implements Closeable { + + private static final int BUFFER_SIZE = 1 << 16; + + private final Client remoteClient; + private final String sessionUUID; + private final DiscoveryNode node; + private final Store.MetadataSnapshot sourceMetaData; + + RestoreSession(String repositoryName, Client remoteClient, String sessionUUID, DiscoveryNode node, IndexShard indexShard, + RecoveryState recoveryState, Store.MetadataSnapshot sourceMetaData) { + super(repositoryName, indexShard, SNAPSHOT_ID, recoveryState, BUFFER_SIZE); + this.remoteClient = remoteClient; + this.sessionUUID = sessionUUID; + this.node = node; + this.sourceMetaData = sourceMetaData; + } + + static RestoreSession openSession(String repositoryName, Client remoteClient, ShardId leaderShardId, IndexShard indexShard, + RecoveryState recoveryState) { + String sessionUUID = UUIDs.randomBase64UUID(); + PutCcrRestoreSessionAction.PutCcrRestoreSessionResponse response = remoteClient.execute(PutCcrRestoreSessionAction.INSTANCE, + new PutCcrRestoreSessionRequest(sessionUUID, leaderShardId)).actionGet(); + return new RestoreSession(repositoryName, remoteClient, sessionUUID, response.getNode(), indexShard, recoveryState, + response.getStoreFileMetaData()); + } + + void restoreFiles() throws IOException { + ArrayList fileInfos = new ArrayList<>(); + for (StoreFileMetaData fileMetaData : sourceMetaData) { + ByteSizeValue fileSize = new ByteSizeValue(fileMetaData.length()); + fileInfos.add(new BlobStoreIndexShardSnapshot.FileInfo(fileMetaData.name(), fileMetaData, fileSize)); + } + SnapshotFiles snapshotFiles = new SnapshotFiles(LATEST, fileInfos); + restore(snapshotFiles); + } + + @Override + protected InputStream fileInputStream(BlobStoreIndexShardSnapshot.FileInfo fileInfo) { + return new RestoreFileInputStream(remoteClient, sessionUUID, node, fileInfo.metadata()); + } + + @Override + public void close() { + ClearCcrRestoreSessionRequest clearRequest = new ClearCcrRestoreSessionRequest(sessionUUID, node); + ClearCcrRestoreSessionAction.ClearCcrRestoreSessionResponse response = + remoteClient.execute(ClearCcrRestoreSessionAction.INSTANCE, clearRequest).actionGet(); + } + } + + private static class RestoreFileInputStream extends InputStream { + + private final Client remoteClient; + private final String sessionUUID; + private final DiscoveryNode node; + private final StoreFileMetaData fileToRecover; + + private long pos = 0; + + private RestoreFileInputStream(Client remoteClient, String sessionUUID, DiscoveryNode node, StoreFileMetaData fileToRecover) { + this.remoteClient = remoteClient; + this.sessionUUID = sessionUUID; + this.node = node; + this.fileToRecover = fileToRecover; + } + + + @Override + public int read() throws IOException { + throw new UnsupportedOperationException(); + } + + @Override + public int read(byte[] bytes, int off, int len) throws IOException { + long remainingBytes = fileToRecover.length() - pos; + if (remainingBytes <= 0) { + return 0; + } + + int bytesRequested = (int) Math.min(remainingBytes, len); + String fileName = fileToRecover.name(); + GetCcrRestoreFileChunkRequest request = new GetCcrRestoreFileChunkRequest(node, sessionUUID, fileName, bytesRequested); + GetCcrRestoreFileChunkAction.GetCcrRestoreFileChunkResponse response = + remoteClient.execute(GetCcrRestoreFileChunkAction.INSTANCE, request).actionGet(); + BytesReference fileChunk = response.getChunk(); + + int bytesReceived = fileChunk.length(); + if (bytesReceived > bytesRequested) { + throw new IOException("More bytes [" + bytesReceived + "] received than requested [" + bytesRequested + "]"); + } + + long leaderOffset = response.getOffset(); + assert pos == leaderOffset : "Position [" + pos + "] should be equal to the leader file offset [" + leaderOffset + "]."; + + try (StreamInput streamInput = fileChunk.streamInput()) { + int bytesRead = streamInput.read(bytes, 0, bytesReceived); + assert bytesRead == bytesReceived : "Did not read the correct number of bytes"; + } + + pos += bytesReceived; + + return bytesReceived; + } } } diff --git a/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/repository/CcrRestoreSourceService.java b/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/repository/CcrRestoreSourceService.java index 642036168ad7b..197d5ddbf38ff 100644 --- a/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/repository/CcrRestoreSourceService.java +++ b/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/repository/CcrRestoreSourceService.java @@ -8,10 +8,18 @@ import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; +import org.apache.lucene.store.IOContext; +import org.apache.lucene.store.IndexInput; +import org.apache.lucene.util.BytesRef; +import org.apache.lucene.util.BytesRefIterator; import org.elasticsearch.common.Nullable; +import org.elasticsearch.common.bytes.BytesReference; import org.elasticsearch.common.component.AbstractLifecycleComponent; +import org.elasticsearch.common.lease.Releasable; import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.common.util.concurrent.AbstractRefCounted; import org.elasticsearch.common.util.concurrent.ConcurrentCollections; +import org.elasticsearch.common.util.concurrent.KeyedLock; import org.elasticsearch.core.internal.io.IOUtils; import org.elasticsearch.index.engine.Engine; import org.elasticsearch.index.shard.IndexEventListener; @@ -23,9 +31,11 @@ import java.io.Closeable; import java.io.IOException; +import java.io.UncheckedIOException; import java.util.HashMap; import java.util.HashSet; import java.util.Map; +import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.CopyOnWriteArrayList; import java.util.function.Consumer; @@ -33,7 +43,7 @@ public class CcrRestoreSourceService extends AbstractLifecycleComponent implemen private static final Logger logger = LogManager.getLogger(CcrRestoreSourceService.class); - private final Map onGoingRestores = ConcurrentCollections.newConcurrentMap(); + private final Map onGoingRestores = ConcurrentCollections.newConcurrentMap(); private final Map> sessionsForShard = new HashMap<>(); private final CopyOnWriteArrayList> openSessionListeners = new CopyOnWriteArrayList<>(); private final CopyOnWriteArrayList> closeSessionListeners = new CopyOnWriteArrayList<>(); @@ -48,8 +58,9 @@ public synchronized void afterIndexShardClosed(ShardId shardId, @Nullable IndexS HashSet sessions = sessionsForShard.remove(indexShard); if (sessions != null) { for (String sessionUUID : sessions) { - RestoreContext restore = onGoingRestores.remove(sessionUUID); - IOUtils.closeWhileHandlingException(restore); + RestoreSession restore = onGoingRestores.remove(sessionUUID); + assert restore != null : "Session UUID [" + sessionUUID + "] registered for shard but not found in ongoing restores"; + restore.decRef(); } } } @@ -68,7 +79,7 @@ protected void doStop() { @Override protected synchronized void doClose() throws IOException { sessionsForShard.clear(); - IOUtils.closeWhileHandlingException(onGoingRestores.values()); + onGoingRestores.values().forEach(AbstractRefCounted::decRef); onGoingRestores.clear(); } @@ -88,7 +99,7 @@ synchronized HashSet getSessionsForShard(IndexShard indexShard) { } // default visibility for testing - synchronized RestoreContext getOngoingRestore(String sessionUUID) { + synchronized RestoreSession getOngoingRestore(String sessionUUID) { return onGoingRestores.get(sessionUUID); } @@ -96,7 +107,7 @@ synchronized RestoreContext getOngoingRestore(String sessionUUID) { // complete. Or it could be for session to have been touched. public synchronized Store.MetadataSnapshot openSession(String sessionUUID, IndexShard indexShard) throws IOException { boolean success = false; - RestoreContext restore = null; + RestoreSession restore = null; try { if (onGoingRestores.containsKey(sessionUUID)) { logger.debug("not opening new session [{}] as it already exists", sessionUUID); @@ -106,46 +117,72 @@ public synchronized Store.MetadataSnapshot openSession(String sessionUUID, Index if (indexShard.state() == IndexShardState.CLOSED) { throw new IndexShardClosedException(indexShard.shardId(), "cannot open ccr restore session if shard closed"); } - restore = new RestoreContext(sessionUUID, indexShard, indexShard.acquireSafeIndexCommit()); + restore = new RestoreSession(sessionUUID, indexShard, indexShard.acquireSafeIndexCommit()); onGoingRestores.put(sessionUUID, restore); openSessionListeners.forEach(c -> c.accept(sessionUUID)); - HashSet sessions = sessionsForShard.computeIfAbsent(indexShard, (s) -> new HashSet<>()); + HashSet sessions = sessionsForShard.computeIfAbsent(indexShard, (s) -> new HashSet<>()); sessions.add(sessionUUID); } Store.MetadataSnapshot metaData = restore.getMetaData(); success = true; return metaData; } finally { - if (success == false) { + if (success == false) { onGoingRestores.remove(sessionUUID); - IOUtils.closeWhileHandlingException(restore); + if (restore != null) { + restore.decRef(); + } + } + } + } + + public void closeSession(String sessionUUID) { + final RestoreSession restore; + synchronized (this) { + closeSessionListeners.forEach(c -> c.accept(sessionUUID)); + restore = onGoingRestores.remove(sessionUUID); + if (restore == null) { + logger.debug("could not close session [{}] because session not found", sessionUUID); + throw new IllegalArgumentException("session [" + sessionUUID + "] not found"); + } + HashSet sessions = sessionsForShard.get(restore.indexShard); + assert sessions != null : "No session UUIDs for shard even though one [" + sessionUUID + "] is active in ongoing restores"; + if (sessions != null) { + boolean removed = sessions.remove(sessionUUID); + assert removed : "No session found for UUID [" + sessionUUID +"]"; + if (sessions.isEmpty()) { + sessionsForShard.remove(restore.indexShard); + } } } + restore.decRef(); } - public synchronized void closeSession(String sessionUUID) { - closeSessionListeners.forEach(c -> c.accept(sessionUUID)); - RestoreContext restore = onGoingRestores.remove(sessionUUID); + public synchronized SessionReader getSessionReader(String sessionUUID) { + RestoreSession restore = onGoingRestores.get(sessionUUID); if (restore == null) { - logger.info("could not close session [{}] because session not found", sessionUUID); + logger.debug("could not get session [{}] because session not found", sessionUUID); throw new IllegalArgumentException("session [" + sessionUUID + "] not found"); } - IOUtils.closeWhileHandlingException(restore); + return new SessionReader(restore); } - private class RestoreContext implements Closeable { + private static class RestoreSession extends AbstractRefCounted { private final String sessionUUID; private final IndexShard indexShard; private final Engine.IndexCommitRef commitRef; + private final KeyedLock keyedLock = new KeyedLock<>(); + private final Map cachedInputs = new ConcurrentHashMap<>(); - private RestoreContext(String sessionUUID, IndexShard indexShard, Engine.IndexCommitRef commitRef) { + private RestoreSession(String sessionUUID, IndexShard indexShard, Engine.IndexCommitRef commitRef) { + super("restore-session"); this.sessionUUID = sessionUUID; this.indexShard = indexShard; this.commitRef = commitRef; } - Store.MetadataSnapshot getMetaData() throws IOException { + private Store.MetadataSnapshot getMetaData() throws IOException { indexShard.store().incRef(); try { return indexShard.store().getMetadata(commitRef.getIndexCommit()); @@ -154,22 +191,71 @@ Store.MetadataSnapshot getMetaData() throws IOException { } } - @Override - public void close() { - assert Thread.holdsLock(CcrRestoreSourceService.this); - removeSessionForShard(sessionUUID, indexShard); - IOUtils.closeWhileHandlingException(commitRef); - } + private long readFileBytes(String fileName, BytesReference reference) throws IOException { + Releasable lock = keyedLock.tryAcquire(fileName); + if (lock == null) { + throw new IllegalStateException("can't read from the same file on the same session concurrently"); + } + try (Releasable releasable = lock) { + final IndexInput indexInput = cachedInputs.computeIfAbsent(fileName, f -> { + try { + return commitRef.getIndexCommit().getDirectory().openInput(fileName, IOContext.READONCE); + } catch (IOException e) { + throw new UncheckedIOException(e); + } + }); - private void removeSessionForShard(String sessionUUID, IndexShard indexShard) { - logger.debug("closing session [{}] for shard [{}]", sessionUUID, indexShard.shardId()); - HashSet sessions = sessionsForShard.get(indexShard); - if (sessions != null) { - sessions.remove(sessionUUID); - if (sessions.isEmpty()) { - sessionsForShard.remove(indexShard); + BytesRefIterator refIterator = reference.iterator(); + BytesRef ref; + while ((ref = refIterator.next()) != null) { + byte[] refBytes = ref.bytes; + indexInput.readBytes(refBytes, 0, refBytes.length); } + + long offsetAfterRead = indexInput.getFilePointer(); + + if (offsetAfterRead == indexInput.length()) { + cachedInputs.remove(fileName); + IOUtils.close(indexInput); + } + + return offsetAfterRead; } } + + @Override + protected void closeInternal() { + logger.debug("closing session [{}] for shard [{}]", sessionUUID, indexShard.shardId()); + assert keyedLock.hasLockedKeys() == false : "Should not hold any file locks when closing"; + IOUtils.closeWhileHandlingException(cachedInputs.values()); + } + } + + public static class SessionReader implements Closeable { + + private final RestoreSession restoreSession; + + private SessionReader(RestoreSession restoreSession) { + this.restoreSession = restoreSession; + restoreSession.incRef(); + } + + /** + * Read bytes into the reference from the file. This method will return the offset in the file where + * the read completed. + * + * @param fileName to read + * @param reference to read bytes into + * @return the offset of the file after the read is complete + * @throws IOException if the read fails + */ + public long readFileBytes(String fileName, BytesReference reference) throws IOException { + return restoreSession.readFileBytes(fileName, reference); + } + + @Override + public void close() { + restoreSession.decRef(); + } } } diff --git a/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/CcrRepositoryIT.java b/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/CcrRepositoryIT.java index 2d3ca857ff848..36e1027dc5f87 100644 --- a/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/CcrRepositoryIT.java +++ b/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/CcrRepositoryIT.java @@ -10,6 +10,7 @@ import org.elasticsearch.action.admin.cluster.settings.ClusterUpdateSettingsRequest; import org.elasticsearch.action.admin.cluster.state.ClusterStateRequest; import org.elasticsearch.action.admin.cluster.state.ClusterStateResponse; +import org.elasticsearch.action.get.GetResponse; import org.elasticsearch.action.support.IndicesOptions; import org.elasticsearch.action.support.PlainActionFuture; import org.elasticsearch.cluster.ClusterChangedEvent; @@ -21,7 +22,6 @@ import org.elasticsearch.common.collect.ImmutableOpenMap; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.unit.TimeValue; -import org.elasticsearch.common.util.concurrent.ConcurrentCollections; import org.elasticsearch.common.xcontent.XContentType; import org.elasticsearch.common.xcontent.support.XContentMapValues; import org.elasticsearch.index.IndexSettings; @@ -40,8 +40,8 @@ import java.io.IOException; import java.util.Locale; import java.util.Map; -import java.util.Set; import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicBoolean; import static java.util.Collections.singletonMap; import static org.elasticsearch.snapshots.RestoreService.restoreInProgress; @@ -159,7 +159,7 @@ public void testThatRepositoryRecoversEmptyIndexBasedOnLeaderSettings() throws I assertNotEquals(leaderMetadata.getIndexUUID(), followerMetadata.getIndexUUID()); } - public void testThatSessionIsRegisteredWithPrimaryShard() throws IOException { + public void testDocsAreRecovered() throws Exception { String leaderClusterRepoName = CcrRepository.NAME_PREFIX + "leader_cluster"; String leaderIndex = "index1"; String followerIndex = "index2"; @@ -173,6 +173,45 @@ public void testThatSessionIsRegisteredWithPrimaryShard() throws IOException { final RestoreService restoreService = getFollowerCluster().getCurrentMasterNodeInstance(RestoreService.class); final ClusterService clusterService = getFollowerCluster().getCurrentMasterNodeInstance(ClusterService.class); + final int firstBatchNumDocs = randomIntBetween(1, 64); + logger.info("Indexing [{}] docs as first batch", firstBatchNumDocs); + for (int i = 0; i < firstBatchNumDocs; i++) { + final String source = String.format(Locale.ROOT, "{\"f\":%d}", i); + leaderClient().prepareIndex("index1", "doc", Integer.toString(i)).setSource(source, XContentType.JSON).get(); + } + + leaderClient().admin().indices().prepareFlush(leaderIndex).setForce(true).setWaitIfOngoing(true).get(); + + AtomicBoolean isRunning = new AtomicBoolean(true); + + // Concurrently index new docs with mapping changes + Thread thread = new Thread(() -> { + char[] chars = "abcdeghijklmnopqrstuvwxyz".toCharArray(); + for (char c : chars) { + if (isRunning.get() == false) { + break; + } + final String source; + long l = randomLongBetween(0, 50000); + if (randomBoolean()) { + source = String.format(Locale.ROOT, "{\"%c\":%d}", c, l); + } else { + source = String.format(Locale.ROOT, "{\"%c\":\"%d\"}", c, l); + } + for (int i = 64; i < 150; i++) { + if (isRunning.get() == false) { + break; + } + leaderClient().prepareIndex("index1", "doc", Long.toString(i)).setSource(source, XContentType.JSON).get(); + if (rarely()) { + leaderClient().admin().indices().prepareFlush(leaderIndex).setForce(true).get(); + } + } + leaderClient().admin().indices().prepareFlush(leaderIndex).setForce(true).setWaitIfOngoing(true).get(); + } + }); + thread.start(); + Settings.Builder settingsBuilder = Settings.builder() .put(IndexMetaData.SETTING_INDEX_PROVIDED_NAME, followerIndex) .put(CcrSettings.CCR_FOLLOWING_INDEX_SETTING.getKey(), true); @@ -182,22 +221,18 @@ public void testThatSessionIsRegisteredWithPrimaryShard() throws IOException { false, true, settingsBuilder.build(), new String[0], "restore_snapshot[" + leaderClusterRepoName + ":" + leaderIndex + "]"); - Set sessionsOpened = ConcurrentCollections.newConcurrentSet(); - Set sessionsClosed = ConcurrentCollections.newConcurrentSet(); - for (CcrRestoreSourceService restoreSourceService : getLeaderCluster().getDataNodeInstances(CcrRestoreSourceService.class)) { - restoreSourceService.addOpenSessionListener(sessionsOpened::add); - restoreSourceService.addCloseSessionListener(sessionsClosed::add); - } - PlainActionFuture future = PlainActionFuture.newFuture(); restoreService.restoreSnapshot(restoreRequest, waitForRestore(clusterService, future)); RestoreInfo restoreInfo = future.actionGet(); - assertEquals(numberOfPrimaryShards, sessionsOpened.size()); - assertEquals(numberOfPrimaryShards, sessionsClosed.size()); - assertEquals(restoreInfo.totalShards(), restoreInfo.successfulShards()); assertEquals(0, restoreInfo.failedShards()); + for (int i = 0; i < firstBatchNumDocs; ++i) { + assertExpectedDocument(followerIndex, i); + } + + isRunning.set(false); + thread.join(); } public void testFollowerMappingIsUpdated() throws IOException { @@ -254,6 +289,13 @@ public void testFollowerMappingIsUpdated() throws IOException { assertThat(XContentMapValues.extractValue("properties.k.type", mappingMetaData.sourceAsMap()), equalTo("long")); } + private void assertExpectedDocument(String followerIndex, final int value) { + final GetResponse getResponse = followerClient().prepareGet(followerIndex, "doc", Integer.toString(value)).get(); + assertTrue("Doc with id [" + value + "] is missing", getResponse.isExists()); + assertTrue((getResponse.getSource().containsKey("f"))); + assertThat(getResponse.getSource().get("f"), equalTo(value)); + } + private ActionListener waitForRestore(ClusterService clusterService, ActionListener listener) { return new ActionListener() { diff --git a/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/repository/CcrRestoreSourceServiceTests.java b/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/repository/CcrRestoreSourceServiceTests.java index dfa7e5ef12660..efcd93e90fd07 100644 --- a/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/repository/CcrRestoreSourceServiceTests.java +++ b/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/repository/CcrRestoreSourceServiceTests.java @@ -6,14 +6,20 @@ package org.elasticsearch.xpack.ccr.repository; +import org.apache.lucene.store.IOContext; +import org.apache.lucene.store.IndexInput; import org.elasticsearch.common.UUIDs; +import org.elasticsearch.common.bytes.BytesArray; import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.index.engine.Engine; import org.elasticsearch.index.shard.IllegalIndexShardStateException; import org.elasticsearch.index.shard.IndexShard; import org.elasticsearch.index.shard.IndexShardTestCase; +import org.elasticsearch.index.store.StoreFileMetaData; import org.junit.Before; import java.io.IOException; +import java.util.ArrayList; import java.util.HashSet; public class CcrRestoreSourceServiceTests extends IndexShardTestCase { @@ -122,4 +128,63 @@ public void testCloseShardListenerFunctionality() throws IOException { restoreSourceService.closeSession(sessionUUID3); closeShards(indexShard1, indexShard2); } + + public void testGetSessionReader() throws IOException { + IndexShard indexShard1 = newStartedShard(true); + final String sessionUUID1 = UUIDs.randomBase64UUID(); + + restoreSourceService.openSession(sessionUUID1, indexShard1); + + ArrayList files = new ArrayList<>(); + indexShard1.snapshotStoreMetadata().forEach(files::add); + + StoreFileMetaData fileMetaData = files.get(0); + String fileName = fileMetaData.name(); + + byte[] expectedBytes = new byte[(int) fileMetaData.length()]; + byte[] actualBytes = new byte[(int) fileMetaData.length()]; + Engine.IndexCommitRef indexCommitRef = indexShard1.acquireSafeIndexCommit(); + try (IndexInput indexInput = indexCommitRef.getIndexCommit().getDirectory().openInput(fileName, IOContext.READONCE)) { + indexInput.seek(0); + indexInput.readBytes(expectedBytes, 0, (int) fileMetaData.length()); + } + + BytesArray byteArray = new BytesArray(actualBytes); + try (CcrRestoreSourceService.SessionReader sessionReader = restoreSourceService.getSessionReader(sessionUUID1)) { + long offset = sessionReader.readFileBytes(fileName, byteArray); + assertEquals(offset, fileMetaData.length()); + } + + assertArrayEquals(expectedBytes, actualBytes); + restoreSourceService.closeSession(sessionUUID1); + closeShards(indexShard1); + } + + public void testGetSessionDoesNotLeakFileIfClosed() throws IOException { + Settings settings = Settings.builder().put("index.merge.enabled", false).build(); + IndexShard indexShard = newStartedShard(true, settings); + for (int i = 0; i < 5; i++) { + indexDoc(indexShard, "_doc", Integer.toString(i)); + flushShard(indexShard, true); + } + final String sessionUUID1 = UUIDs.randomBase64UUID(); + + restoreSourceService.openSession(sessionUUID1, indexShard); + + ArrayList files = new ArrayList<>(); + indexShard.snapshotStoreMetadata().forEach(files::add); + + try (CcrRestoreSourceService.SessionReader sessionReader = restoreSourceService.getSessionReader(sessionUUID1)) { + sessionReader.readFileBytes(files.get(0).name(), new BytesArray(new byte[10])); + } + + // Request a second file to ensure that original file is not leaked + try (CcrRestoreSourceService.SessionReader sessionReader = restoreSourceService.getSessionReader(sessionUUID1)) { + sessionReader.readFileBytes(files.get(1).name(), new BytesArray(new byte[10])); + } + + restoreSourceService.closeSession(sessionUUID1); + closeShards(indexShard); + // Exception will be thrown if file is not closed. + } } From 15aa3764a47bf6c9fd99dc5e0713acb12105d55c Mon Sep 17 00:00:00 2001 From: Nhat Nguyen Date: Mon, 14 Jan 2019 15:14:46 -0500 Subject: [PATCH 150/186] Reduce recovery time with compress or secure transport (#36981) Today file-chunks are sent sequentially one by one in peer-recovery. This is a correct choice since the implementation is straightforward and recovery is network bound in most of the time. However, if the connection is encrypted, we might not be able to saturate the network pipe because encrypting/decrypting are cpu bound rather than network-bound. With this commit, a source node can send multiple (default to 2) file-chunks without waiting for the acknowledgments from the target. Below are the benchmark results for PMC and NYC_taxis. - PMC (20.2 GB) | Transport | Baseline | chunks=1 | chunks=2 | chunks=3 | chunks=4 | | ----------| ---------| -------- | -------- | -------- | -------- | | Plain | 184s | 137s | 106s | 105s | 106s | | TLS | 346s | 294s | 176s | 153s | 117s | | Compress | 1556s | 1407s | 1193s | 1183s | 1211s | - NYC_Taxis (38.6GB) | Transport | Baseline | chunks=1 | chunks=2 | chunks=3 | chunks=4 | | ----------| ---------| ---------| ---------| ---------| -------- | | Plain | 321s | 249s | 191s | * | * | | TLS | 618s | 539s | 323s | 290s | 213s | | Compress | 2622s | 2421s | 2018s | 2029s | n/a | Relates #33844 --- .../modules/indices/recovery.asciidoc | 9 + .../common/settings/ClusterSettings.java | 1 + .../recovery/PeerRecoverySourceService.java | 3 +- .../recovery/PeerRecoveryTargetService.java | 15 +- .../indices/recovery/RecoverySettings.java | 17 + .../recovery/RecoverySourceHandler.java | 143 ++++---- .../indices/recovery/RecoveryTarget.java | 70 +++- .../recovery/RecoveryTargetHandler.java | 4 +- .../recovery/RemoteRecoveryTargetHandler.java | 10 +- .../PeerRecoveryTargetServiceTests.java | 86 +++++ .../recovery/RecoverySourceHandlerTests.java | 336 +++++++++++++++--- .../elasticsearch/recovery/RelocationIT.java | 2 +- .../index/shard/IndexShardTestCase.java | 5 +- .../test/InternalTestCluster.java | 2 + 14 files changed, 559 insertions(+), 144 deletions(-) diff --git a/docs/reference/modules/indices/recovery.asciidoc b/docs/reference/modules/indices/recovery.asciidoc index d9e034941f80e..d9e85c27105c8 100644 --- a/docs/reference/modules/indices/recovery.asciidoc +++ b/docs/reference/modules/indices/recovery.asciidoc @@ -20,5 +20,14 @@ peer recoveries: consume an excess of bandwidth (or other resources) which could destabilize the cluster. Defaults to `40mb`. +`indices.recovery.max_concurrent_file_chunks`:: + Controls the number of file chunk requests that can be sent in parallel per recovery. + As multiple recoveries are already running in parallel (controlled by + cluster.routing.allocation.node_concurrent_recoveries), increasing this expert-level + setting might only help in situations where peer recovery of a single shard is not + reaching the total inbound and outbound peer recovery traffic as configured by + indices.recovery.max_bytes_per_sec, but is CPU-bound instead, typically when using + transport-level security or compression. Defaults to `2`. + This setting can be dynamically updated on a live cluster with the <> API. diff --git a/server/src/main/java/org/elasticsearch/common/settings/ClusterSettings.java b/server/src/main/java/org/elasticsearch/common/settings/ClusterSettings.java index 9b2388aa4b73c..f0331ad2e30b2 100644 --- a/server/src/main/java/org/elasticsearch/common/settings/ClusterSettings.java +++ b/server/src/main/java/org/elasticsearch/common/settings/ClusterSettings.java @@ -214,6 +214,7 @@ public void apply(Settings value, Settings current, Settings previous) { RecoverySettings.INDICES_RECOVERY_ACTIVITY_TIMEOUT_SETTING, RecoverySettings.INDICES_RECOVERY_INTERNAL_ACTION_TIMEOUT_SETTING, RecoverySettings.INDICES_RECOVERY_INTERNAL_LONG_ACTION_TIMEOUT_SETTING, + RecoverySettings.INDICES_RECOVERY_MAX_CONCURRENT_FILE_CHUNKS_SETTING, ThrottlingAllocationDecider.CLUSTER_ROUTING_ALLOCATION_NODE_INITIAL_PRIMARIES_RECOVERIES_SETTING, ThrottlingAllocationDecider.CLUSTER_ROUTING_ALLOCATION_NODE_CONCURRENT_INCOMING_RECOVERIES_SETTING, ThrottlingAllocationDecider.CLUSTER_ROUTING_ALLOCATION_NODE_CONCURRENT_OUTGOING_RECOVERIES_SETTING, diff --git a/server/src/main/java/org/elasticsearch/indices/recovery/PeerRecoverySourceService.java b/server/src/main/java/org/elasticsearch/indices/recovery/PeerRecoverySourceService.java index 9c30ab156c08a..556df71ca2cb9 100644 --- a/server/src/main/java/org/elasticsearch/indices/recovery/PeerRecoverySourceService.java +++ b/server/src/main/java/org/elasticsearch/indices/recovery/PeerRecoverySourceService.java @@ -176,7 +176,8 @@ private RecoverySourceHandler createRecoverySourceHandler(StartRecoveryRequest r final RemoteRecoveryTargetHandler recoveryTarget = new RemoteRecoveryTargetHandler(request.recoveryId(), request.shardId(), transportService, request.targetNode(), recoverySettings, throttleTime -> shard.recoveryStats().addThrottleTime(throttleTime)); - handler = new RecoverySourceHandler(shard, recoveryTarget, request, recoverySettings.getChunkSize().bytesAsInt()); + handler = new RecoverySourceHandler(shard, recoveryTarget, request, + Math.toIntExact(recoverySettings.getChunkSize().getBytes()), recoverySettings.getMaxConcurrentFileChunks()); return handler; } } diff --git a/server/src/main/java/org/elasticsearch/indices/recovery/PeerRecoveryTargetService.java b/server/src/main/java/org/elasticsearch/indices/recovery/PeerRecoveryTargetService.java index 8e52a05e2ac30..8edaf0ef093ab 100644 --- a/server/src/main/java/org/elasticsearch/indices/recovery/PeerRecoveryTargetService.java +++ b/server/src/main/java/org/elasticsearch/indices/recovery/PeerRecoveryTargetService.java @@ -29,6 +29,8 @@ import org.elasticsearch.ElasticsearchException; import org.elasticsearch.ElasticsearchTimeoutException; import org.elasticsearch.ExceptionsHelper; +import org.elasticsearch.action.ActionListener; +import org.elasticsearch.action.support.HandledTransportAction; import org.elasticsearch.action.support.PlainActionFuture; import org.elasticsearch.cluster.ClusterState; import org.elasticsearch.cluster.ClusterStateObserver; @@ -602,8 +604,7 @@ class FileChunkTransportRequestHandler implements TransportRequestHandler listener = + new HandledTransportAction.ChannelActionListener<>(channel, Actions.FILE_CHUNK, request); + recoveryTarget.writeFileChunk(request.metadata(), request.position(), request.content(), request.lastChunk(), + request.totalTranslogOps(), + ActionListener.wrap(nullVal -> listener.onResponse(TransportResponse.Empty.INSTANCE), listener::onFailure)); } - channel.sendResponse(TransportResponse.Empty.INSTANCE); } } diff --git a/server/src/main/java/org/elasticsearch/indices/recovery/RecoverySettings.java b/server/src/main/java/org/elasticsearch/indices/recovery/RecoverySettings.java index 31ecd4455b165..3db04dec1d69f 100644 --- a/server/src/main/java/org/elasticsearch/indices/recovery/RecoverySettings.java +++ b/server/src/main/java/org/elasticsearch/indices/recovery/RecoverySettings.java @@ -39,6 +39,12 @@ public class RecoverySettings { Setting.byteSizeSetting("indices.recovery.max_bytes_per_sec", new ByteSizeValue(40, ByteSizeUnit.MB), Property.Dynamic, Property.NodeScope); + /** + * Controls the maximum number of file chunk requests that can be sent concurrently from the source node to the target node. + */ + public static final Setting INDICES_RECOVERY_MAX_CONCURRENT_FILE_CHUNKS_SETTING = + Setting.intSetting("indices.recovery.max_concurrent_file_chunks", 2, 1, 5, Property.Dynamic, Property.NodeScope); + /** * how long to wait before retrying after issues cause by cluster state syncing between nodes * i.e., local node is not yet known on remote node, remote shard not yet started etc. @@ -78,6 +84,7 @@ public class RecoverySettings { public static final ByteSizeValue DEFAULT_CHUNK_SIZE = new ByteSizeValue(512, ByteSizeUnit.KB); private volatile ByteSizeValue maxBytesPerSec; + private volatile int maxConcurrentFileChunks; private volatile SimpleRateLimiter rateLimiter; private volatile TimeValue retryDelayStateSync; private volatile TimeValue retryDelayNetwork; @@ -89,6 +96,7 @@ public class RecoverySettings { public RecoverySettings(Settings settings, ClusterSettings clusterSettings) { this.retryDelayStateSync = INDICES_RECOVERY_RETRY_DELAY_STATE_SYNC_SETTING.get(settings); + this.maxConcurrentFileChunks = INDICES_RECOVERY_MAX_CONCURRENT_FILE_CHUNKS_SETTING.get(settings); // doesn't have to be fast as nodes are reconnected every 10s by default (see InternalClusterService.ReconnectToNodes) // and we want to give the master time to remove a faulty node this.retryDelayNetwork = INDICES_RECOVERY_RETRY_DELAY_NETWORK_SETTING.get(settings); @@ -108,6 +116,7 @@ public RecoverySettings(Settings settings, ClusterSettings clusterSettings) { logger.debug("using max_bytes_per_sec[{}]", maxBytesPerSec); clusterSettings.addSettingsUpdateConsumer(INDICES_RECOVERY_MAX_BYTES_PER_SEC_SETTING, this::setMaxBytesPerSec); + clusterSettings.addSettingsUpdateConsumer(INDICES_RECOVERY_MAX_CONCURRENT_FILE_CHUNKS_SETTING, this::setMaxConcurrentFileChunks); clusterSettings.addSettingsUpdateConsumer(INDICES_RECOVERY_RETRY_DELAY_STATE_SYNC_SETTING, this::setRetryDelayStateSync); clusterSettings.addSettingsUpdateConsumer(INDICES_RECOVERY_RETRY_DELAY_NETWORK_SETTING, this::setRetryDelayNetwork); clusterSettings.addSettingsUpdateConsumer(INDICES_RECOVERY_INTERNAL_ACTION_TIMEOUT_SETTING, this::setInternalActionTimeout); @@ -180,4 +189,12 @@ private void setMaxBytesPerSec(ByteSizeValue maxBytesPerSec) { rateLimiter = new SimpleRateLimiter(maxBytesPerSec.getMbFrac()); } } + + public int getMaxConcurrentFileChunks() { + return maxConcurrentFileChunks; + } + + private void setMaxConcurrentFileChunks(int maxConcurrentFileChunks) { + this.maxConcurrentFileChunks = maxConcurrentFileChunks; + } } diff --git a/server/src/main/java/org/elasticsearch/indices/recovery/RecoverySourceHandler.java b/server/src/main/java/org/elasticsearch/indices/recovery/RecoverySourceHandler.java index 639ab47799366..bdda5e8d8d4f8 100644 --- a/server/src/main/java/org/elasticsearch/indices/recovery/RecoverySourceHandler.java +++ b/server/src/main/java/org/elasticsearch/indices/recovery/RecoverySourceHandler.java @@ -36,6 +36,7 @@ import org.elasticsearch.cluster.routing.ShardRouting; import org.elasticsearch.common.StopWatch; import org.elasticsearch.common.bytes.BytesArray; +import org.elasticsearch.common.collect.Tuple; import org.elasticsearch.common.lease.Releasable; import org.elasticsearch.common.logging.Loggers; import org.elasticsearch.common.lucene.store.InputStreamIndexInput; @@ -44,7 +45,6 @@ import org.elasticsearch.common.util.CancellableThreads; import org.elasticsearch.common.util.concurrent.FutureUtils; import org.elasticsearch.core.internal.io.IOUtils; -import org.elasticsearch.core.internal.io.Streams; import org.elasticsearch.index.engine.Engine; import org.elasticsearch.index.engine.RecoveryEngineException; import org.elasticsearch.index.seqno.LocalCheckpointTracker; @@ -59,10 +59,9 @@ import org.elasticsearch.threadpool.ThreadPool; import org.elasticsearch.transport.RemoteTransportException; -import java.io.BufferedOutputStream; import java.io.Closeable; import java.io.IOException; -import java.io.OutputStream; +import java.io.InputStream; import java.util.ArrayList; import java.util.Collections; import java.util.Comparator; @@ -71,10 +70,12 @@ import java.util.concurrent.CompletableFuture; import java.util.concurrent.CopyOnWriteArrayList; import java.util.concurrent.atomic.AtomicLong; -import java.util.function.Function; +import java.util.concurrent.atomic.AtomicReference; import java.util.function.Supplier; import java.util.stream.StreamSupport; +import static org.elasticsearch.index.seqno.SequenceNumbers.NO_OPS_PERFORMED; + /** * RecoverySourceHandler handles the three phases of shard recovery, which is * everything relating to copying the segment files as well as sending translog @@ -96,17 +97,19 @@ public class RecoverySourceHandler { private final StartRecoveryRequest request; private final int chunkSizeInBytes; private final RecoveryTargetHandler recoveryTarget; + private final int maxConcurrentFileChunks; private final CancellableThreads cancellableThreads = new CancellableThreads(); - public RecoverySourceHandler(final IndexShard shard, RecoveryTargetHandler recoveryTarget, - final StartRecoveryRequest request, - final int fileChunkSizeInBytes) { + public RecoverySourceHandler(final IndexShard shard, RecoveryTargetHandler recoveryTarget, final StartRecoveryRequest request, + final int fileChunkSizeInBytes, final int maxConcurrentFileChunks) { this.shard = shard; this.recoveryTarget = recoveryTarget; this.request = request; this.shardId = this.request.shardId().id(); this.logger = Loggers.getLogger(getClass(), request.shardId(), "recover to " + request.targetNode().getName()); this.chunkSizeInBytes = fileChunkSizeInBytes; + // if the target is on an old version, it won't be able to handle out-of-order file chunks. + this.maxConcurrentFileChunks = request.targetNode().getVersion().onOrAfter(Version.V_7_0_0) ? maxConcurrentFileChunks : 1; } public StartRecoveryRequest getRequest() { @@ -407,10 +410,7 @@ public SendFileResult phase1(final IndexCommit snapshot, final Supplier phase1ExistingFileNames.size(), new ByteSizeValue(existingTotalSize)); cancellableThreads.execute(() -> recoveryTarget.receiveFileInfo( phase1FileNames, phase1FileSizes, phase1ExistingFileNames, phase1ExistingFileSizes, translogOps.get())); - // How many bytes we've copied since we last called RateLimiter.pause - final Function outputStreamFactories = - md -> new BufferedOutputStream(new RecoveryOutputStream(md, translogOps), chunkSizeInBytes); - sendFiles(store, phase1Files.toArray(new StoreFileMetaData[phase1Files.size()]), outputStreamFactories); + sendFiles(store, phase1Files.toArray(new StoreFileMetaData[0]), translogOps); // Send the CLEAN_FILES request, which takes all of the files that // were transferred and renames them from their temporary file // names to the actual file names. It also writes checksums for @@ -649,73 +649,72 @@ public String toString() { '}'; } - - final class RecoveryOutputStream extends OutputStream { - private final StoreFileMetaData md; - private final Supplier translogOps; - private long position = 0; - - RecoveryOutputStream(StoreFileMetaData md, Supplier translogOps) { - this.md = md; - this.translogOps = translogOps; - } - - @Override - public void write(int b) throws IOException { - throw new UnsupportedOperationException("we can't send single bytes over the wire"); + void sendFiles(Store store, StoreFileMetaData[] files, Supplier translogOps) throws Exception { + ArrayUtil.timSort(files, Comparator.comparingLong(StoreFileMetaData::length)); // send smallest first + final LocalCheckpointTracker requestSeqIdTracker = new LocalCheckpointTracker(NO_OPS_PERFORMED, NO_OPS_PERFORMED); + final AtomicReference> error = new AtomicReference<>(); + final byte[] buffer = new byte[chunkSizeInBytes]; + for (final StoreFileMetaData md : files) { + if (error.get() != null) { + break; + } + try (IndexInput indexInput = store.directory().openInput(md.name(), IOContext.READONCE); + InputStream in = new InputStreamIndexInput(indexInput, md.length())) { + long position = 0; + int bytesRead; + while ((bytesRead = in.read(buffer, 0, buffer.length)) != -1) { + final BytesArray content = new BytesArray(buffer, 0, bytesRead); + final boolean lastChunk = position + content.length() == md.length(); + final long requestSeqId = requestSeqIdTracker.generateSeqNo(); + cancellableThreads.execute(() -> requestSeqIdTracker.waitForOpsToComplete(requestSeqId - maxConcurrentFileChunks)); + cancellableThreads.checkForCancel(); + if (error.get() != null) { + break; + } + final long requestFilePosition = position; + cancellableThreads.executeIO(() -> + recoveryTarget.writeFileChunk(md, requestFilePosition, content, lastChunk, translogOps.get(), + ActionListener.wrap( + r -> requestSeqIdTracker.markSeqNoAsCompleted(requestSeqId), + e -> { + error.compareAndSet(null, Tuple.tuple(md, e)); + requestSeqIdTracker.markSeqNoAsCompleted(requestSeqId); + } + ))); + position += content.length(); + } + } catch (Exception e) { + error.compareAndSet(null, Tuple.tuple(md, e)); + break; + } } - - @Override - public void write(byte[] b, int offset, int length) throws IOException { - sendNextChunk(position, new BytesArray(b, offset, length), md.length() == position + length); - position += length; - assert md.length() >= position : "length: " + md.length() + " but positions was: " + position; + // When we terminate exceptionally, we don't wait for the outstanding requests as we don't use their results anyway. + // This allows us to end quickly and eliminate the complexity of handling requestSeqIds in case of error. + if (error.get() == null) { + cancellableThreads.execute(() -> requestSeqIdTracker.waitForOpsToComplete(requestSeqIdTracker.getMaxSeqNo())); } - - private void sendNextChunk(long position, BytesArray content, boolean lastChunk) throws IOException { - // Actually send the file chunk to the target node, waiting for it to complete - cancellableThreads.executeIO(() -> - recoveryTarget.writeFileChunk(md, position, content, lastChunk, translogOps.get()) - ); - if (shard.state() == IndexShardState.CLOSED) { // check if the shard got closed on us - throw new IndexShardClosedException(request.shardId()); - } + if (error.get() != null) { + handleErrorOnSendFiles(store, error.get().v1(), error.get().v2()); } } - void sendFiles(Store store, StoreFileMetaData[] files, Function outputStreamFactory) throws Exception { - store.incRef(); - try { - ArrayUtil.timSort(files, Comparator.comparingLong(StoreFileMetaData::length)); // send smallest first - for (int i = 0; i < files.length; i++) { - final StoreFileMetaData md = files[i]; - try (IndexInput indexInput = store.directory().openInput(md.name(), IOContext.READONCE)) { - // it's fine that we are only having the indexInput in the try/with block. The copy methods handles - // exceptions during close correctly and doesn't hide the original exception. - Streams.copy(new InputStreamIndexInput(indexInput, md.length()), outputStreamFactory.apply(md)); - } catch (Exception e) { - final IOException corruptIndexException; - if ((corruptIndexException = ExceptionsHelper.unwrapCorruption(e)) != null) { - if (store.checkIntegrityNoException(md) == false) { // we are corrupted on the primary -- fail! - logger.warn("{} Corrupted file detected {} checksum mismatch", shardId, md); - failEngine(corruptIndexException); - throw corruptIndexException; - } else { // corruption has happened on the way to replica - RemoteTransportException exception = new RemoteTransportException("File corruption occurred on recovery but " + - "checksums are ok", null); - exception.addSuppressed(e); - logger.warn(() -> new ParameterizedMessage( - "{} Remote file corruption on node {}, recovering {}. local checksum OK", - shardId, request.targetNode(), md), corruptIndexException); - throw exception; - } - } else { - throw e; - } - } + private void handleErrorOnSendFiles(Store store, StoreFileMetaData md, Exception e) throws Exception { + final IOException corruptIndexException; + if ((corruptIndexException = ExceptionsHelper.unwrapCorruption(e)) != null) { + if (store.checkIntegrityNoException(md) == false) { // we are corrupted on the primary -- fail! + logger.warn("{} Corrupted file detected {} checksum mismatch", shardId, md); + failEngine(corruptIndexException); + throw corruptIndexException; + } else { // corruption has happened on the way to replica + RemoteTransportException exception = new RemoteTransportException( + "File corruption occurred on recovery but checksums are ok", null); + exception.addSuppressed(e); + logger.warn(() -> new ParameterizedMessage("{} Remote file corruption on node {}, recovering {}. local checksum OK", + shardId, request.targetNode(), md), corruptIndexException); + throw exception; } - } finally { - store.decRef(); + } else { + throw e; } } diff --git a/server/src/main/java/org/elasticsearch/indices/recovery/RecoveryTarget.java b/server/src/main/java/org/elasticsearch/indices/recovery/RecoveryTarget.java index 3a3a78941b1b7..54a42bcdc928a 100644 --- a/server/src/main/java/org/elasticsearch/indices/recovery/RecoveryTarget.java +++ b/server/src/main/java/org/elasticsearch/indices/recovery/RecoveryTarget.java @@ -31,6 +31,7 @@ import org.elasticsearch.ElasticsearchException; import org.elasticsearch.ExceptionsHelper; import org.elasticsearch.Version; +import org.elasticsearch.action.ActionListener; import org.elasticsearch.cluster.node.DiscoveryNode; import org.elasticsearch.common.UUIDs; import org.elasticsearch.common.bytes.BytesReference; @@ -55,10 +56,12 @@ import java.nio.file.Path; import java.util.Arrays; import java.util.Collections; +import java.util.Comparator; import java.util.Iterator; import java.util.List; import java.util.Map; import java.util.Map.Entry; +import java.util.PriorityQueue; import java.util.concurrent.ConcurrentMap; import java.util.concurrent.CountDownLatch; import java.util.concurrent.atomic.AtomicBoolean; @@ -89,6 +92,7 @@ public class RecoveryTarget extends AbstractRefCounted implements RecoveryTarget private final AtomicBoolean finished = new AtomicBoolean(); private final ConcurrentMap openIndexOutputs = ConcurrentCollections.newConcurrentMap(); + private final ConcurrentMap fileChunkWriters = ConcurrentCollections.newConcurrentMap(); private final CancellableThreads cancellableThreads; // last time this status was accessed @@ -340,6 +344,7 @@ protected void closeInternal() { } } finally { // free store. increment happens in constructor + fileChunkWriters.clear(); store.decRef(); indexShard.recoveryStats().decCurrentAsTarget(); closedLatch.countDown(); @@ -487,12 +492,10 @@ public void cleanFiles(int totalTranslogOps, Store.MetadataSnapshot sourceMetaDa } } - @Override - public void writeFileChunk(StoreFileMetaData fileMetaData, long position, BytesReference content, - boolean lastChunk, int totalTranslogOps) throws IOException { + private void innerWriteFileChunk(StoreFileMetaData fileMetaData, long position, + BytesReference content, boolean lastChunk) throws IOException { final Store store = store(); final String name = fileMetaData.name(); - state().getTranslog().totalOperations(totalTranslogOps); final RecoveryState.Index indexState = state().getIndex(); IndexOutput indexOutput; if (position == 0) { @@ -500,6 +503,7 @@ public void writeFileChunk(StoreFileMetaData fileMetaData, long position, BytesR } else { indexOutput = getOpenIndexOutput(name); } + assert indexOutput.getFilePointer() == position : "file-pointer " + indexOutput.getFilePointer() + " != " + position; BytesRefIterator iterator = content.iterator(); BytesRef scratch; while((scratch = iterator.next()) != null) { // we iterate over all pages - this is a 0-copy for all core impls @@ -522,6 +526,64 @@ public void writeFileChunk(StoreFileMetaData fileMetaData, long position, BytesR } } + @Override + public void writeFileChunk(StoreFileMetaData fileMetaData, long position, BytesReference content, + boolean lastChunk, int totalTranslogOps, ActionListener listener) { + try { + state().getTranslog().totalOperations(totalTranslogOps); + final FileChunkWriter writer = fileChunkWriters.computeIfAbsent(fileMetaData.name(), name -> new FileChunkWriter()); + writer.writeChunk(new FileChunk(fileMetaData, content, position, lastChunk)); + listener.onResponse(null); + } catch (Exception e) { + listener.onFailure(e); + } + } + + private static final class FileChunk { + final StoreFileMetaData md; + final BytesReference content; + final long position; + final boolean lastChunk; + FileChunk(StoreFileMetaData md, BytesReference content, long position, boolean lastChunk) { + this.md = md; + this.content = content; + this.position = position; + this.lastChunk = lastChunk; + } + } + + private final class FileChunkWriter { + // chunks can be delivered out of order, we need to buffer chunks if there's a gap between them. + final PriorityQueue pendingChunks = new PriorityQueue<>(Comparator.comparing(fc -> fc.position)); + long lastPosition = 0; + + void writeChunk(FileChunk newChunk) throws IOException { + synchronized (this) { + pendingChunks.add(newChunk); + } + while (true) { + final FileChunk chunk; + synchronized (this) { + chunk = pendingChunks.peek(); + if (chunk == null || chunk.position != lastPosition) { + return; + } + pendingChunks.remove(); + } + innerWriteFileChunk(chunk.md, chunk.position, chunk.content, chunk.lastChunk); + synchronized (this) { + assert lastPosition == chunk.position : "last_position " + lastPosition + " != chunk_position " + chunk.position; + lastPosition += chunk.content.length(); + if (chunk.lastChunk) { + assert pendingChunks.isEmpty() == true : "still have pending chunks [" + pendingChunks + "]"; + fileChunkWriters.remove(chunk.md.name()); + assert fileChunkWriters.containsValue(this) == false : "chunk writer [" + newChunk.md + "] was not removed"; + } + } + } + } + } + Path translogLocation() { return indexShard().shardPath().resolveTranslog(); } diff --git a/server/src/main/java/org/elasticsearch/indices/recovery/RecoveryTargetHandler.java b/server/src/main/java/org/elasticsearch/indices/recovery/RecoveryTargetHandler.java index b7c3de97b4e9c..c958665b04497 100644 --- a/server/src/main/java/org/elasticsearch/indices/recovery/RecoveryTargetHandler.java +++ b/server/src/main/java/org/elasticsearch/indices/recovery/RecoveryTargetHandler.java @@ -18,6 +18,7 @@ */ package org.elasticsearch.indices.recovery; +import org.elasticsearch.action.ActionListener; import org.elasticsearch.common.bytes.BytesReference; import org.elasticsearch.index.seqno.ReplicationTracker; import org.elasticsearch.index.store.Store; @@ -27,7 +28,6 @@ import java.io.IOException; import java.util.List; - public interface RecoveryTargetHandler { /** @@ -90,6 +90,6 @@ void receiveFileInfo(List phase1FileNames, /** writes a partial file chunk to the target store */ void writeFileChunk(StoreFileMetaData fileMetaData, long position, BytesReference content, - boolean lastChunk, int totalTranslogOps) throws IOException; + boolean lastChunk, int totalTranslogOps, ActionListener listener); } diff --git a/server/src/main/java/org/elasticsearch/indices/recovery/RemoteRecoveryTargetHandler.java b/server/src/main/java/org/elasticsearch/indices/recovery/RemoteRecoveryTargetHandler.java index 3ad7f65db38d6..560d679bbe7fe 100644 --- a/server/src/main/java/org/elasticsearch/indices/recovery/RemoteRecoveryTargetHandler.java +++ b/server/src/main/java/org/elasticsearch/indices/recovery/RemoteRecoveryTargetHandler.java @@ -21,6 +21,8 @@ import org.apache.lucene.store.RateLimiter; import org.elasticsearch.ElasticsearchException; +import org.elasticsearch.action.ActionListener; +import org.elasticsearch.action.ActionListenerResponseHandler; import org.elasticsearch.cluster.node.DiscoveryNode; import org.elasticsearch.common.bytes.BytesReference; import org.elasticsearch.index.seqno.ReplicationTracker; @@ -31,6 +33,7 @@ import org.elasticsearch.transport.EmptyTransportResponseHandler; import org.elasticsearch.transport.TransportFuture; import org.elasticsearch.transport.TransportRequestOptions; +import org.elasticsearch.transport.TransportResponse; import org.elasticsearch.transport.TransportService; import java.io.IOException; @@ -142,8 +145,8 @@ public void cleanFiles(int totalTranslogOps, Store.MetadataSnapshot sourceMetaDa } @Override - public void writeFileChunk(StoreFileMetaData fileMetaData, long position, BytesReference content, boolean - lastChunk, int totalTranslogOps) throws IOException { + public void writeFileChunk(StoreFileMetaData fileMetaData, long position, BytesReference content, + boolean lastChunk, int totalTranslogOps, ActionListener listener) { // Pause using the rate limiter, if desired, to throttle the recovery final long throttleTimeInNanos; // always fetch the ratelimiter - it might be updated in real-time on the recovery settings @@ -173,7 +176,8 @@ public void writeFileChunk(StoreFileMetaData fileMetaData, long position, BytesR * see how many translog ops we accumulate while copying files across the network. A future optimization * would be in to restart file copy again (new deltas) if we have too many translog ops are piling up. */ - throttleTimeInNanos), fileChunkRequestOptions, EmptyTransportResponseHandler.INSTANCE_SAME).txGet(); + throttleTimeInNanos), fileChunkRequestOptions, new ActionListenerResponseHandler<>( + ActionListener.wrap(r -> listener.onResponse(null), listener::onFailure), in -> TransportResponse.Empty.INSTANCE)); } } diff --git a/server/src/test/java/org/elasticsearch/indices/recovery/PeerRecoveryTargetServiceTests.java b/server/src/test/java/org/elasticsearch/indices/recovery/PeerRecoveryTargetServiceTests.java index b6f5a7b645169..a2ec88cf7b58c 100644 --- a/server/src/test/java/org/elasticsearch/indices/recovery/PeerRecoveryTargetServiceTests.java +++ b/server/src/test/java/org/elasticsearch/indices/recovery/PeerRecoveryTargetServiceTests.java @@ -24,17 +24,33 @@ import org.apache.lucene.index.IndexWriter; import org.apache.lucene.index.IndexWriterConfig; import org.apache.lucene.index.NoMergePolicy; +import org.apache.lucene.store.IOContext; +import org.apache.lucene.store.IndexInput; +import org.elasticsearch.action.ActionListener; +import org.elasticsearch.action.admin.indices.flush.FlushRequest; +import org.elasticsearch.cluster.node.DiscoveryNode; +import org.elasticsearch.common.Randomness; import org.elasticsearch.common.UUIDs; +import org.elasticsearch.common.bytes.BytesArray; import org.elasticsearch.common.lucene.Lucene; import org.elasticsearch.index.seqno.SequenceNumbers; import org.elasticsearch.index.shard.IndexShard; import org.elasticsearch.index.shard.IndexShardTestCase; +import org.elasticsearch.index.store.Store; +import org.elasticsearch.index.store.StoreFileMetaData; import org.elasticsearch.index.translog.Translog; +import java.util.ArrayList; +import java.util.Collections; import java.util.HashMap; import java.util.List; import java.util.Map; +import java.util.concurrent.ArrayBlockingQueue; +import java.util.concurrent.BlockingQueue; +import java.util.concurrent.CyclicBarrier; +import java.util.stream.Collectors; +import static org.hamcrest.Matchers.empty; import static org.hamcrest.Matchers.equalTo; public class PeerRecoveryTargetServiceTests extends IndexShardTestCase { @@ -110,4 +126,74 @@ public void testGetStartingSeqNo() throws Exception { closeShards(replica); } } + + public void testWriteFileChunksConcurrently() throws Exception { + IndexShard sourceShard = newStartedShard(true); + int numDocs = between(20, 100); + for (int i = 0; i < numDocs; i++) { + indexDoc(sourceShard, "_doc", Integer.toString(i)); + } + sourceShard.flush(new FlushRequest()); + Store.MetadataSnapshot sourceSnapshot = sourceShard.store().getMetadata(null); + List mdFiles = new ArrayList<>(); + for (StoreFileMetaData md : sourceSnapshot) { + mdFiles.add(md); + } + final IndexShard targetShard = newShard(false); + final DiscoveryNode pNode = getFakeDiscoNode(sourceShard.routingEntry().currentNodeId()); + final DiscoveryNode rNode = getFakeDiscoNode(targetShard.routingEntry().currentNodeId()); + targetShard.markAsRecovering("test-peer-recovery", new RecoveryState(targetShard.routingEntry(), rNode, pNode)); + final RecoveryTarget recoveryTarget = new RecoveryTarget(targetShard, null, null, null); + recoveryTarget.receiveFileInfo( + mdFiles.stream().map(StoreFileMetaData::name).collect(Collectors.toList()), + mdFiles.stream().map(StoreFileMetaData::length).collect(Collectors.toList()), + Collections.emptyList(), Collections.emptyList(), 0 + ); + List requests = new ArrayList<>(); + for (StoreFileMetaData md : mdFiles) { + try (IndexInput in = sourceShard.store().directory().openInput(md.name(), IOContext.READONCE)) { + int pos = 0; + while (pos < md.length()) { + int length = between(1, Math.toIntExact(md.length() - pos)); + byte[] buffer = new byte[length]; + in.readBytes(buffer, 0, length); + requests.add(new RecoveryFileChunkRequest(0, sourceShard.shardId(), md, pos, new BytesArray(buffer), + pos + length == md.length(), 1, 1)); + pos += length; + } + } + } + Randomness.shuffle(requests); + BlockingQueue queue = new ArrayBlockingQueue<>(requests.size()); + queue.addAll(requests); + Thread[] senders = new Thread[between(1, 4)]; + CyclicBarrier barrier = new CyclicBarrier(senders.length); + for (int i = 0; i < senders.length; i++) { + senders[i] = new Thread(() -> { + try { + barrier.await(); + RecoveryFileChunkRequest r; + while ((r = queue.poll()) != null) { + recoveryTarget.writeFileChunk(r.metadata(), r.position(), r.content(), r.lastChunk(), r.totalTranslogOps(), + ActionListener.wrap(ignored -> {}, + e -> { + throw new AssertionError(e); + })); + } + } catch (Exception e) { + throw new AssertionError(e); + } + }); + senders[i].start(); + } + for (Thread sender : senders) { + sender.join(); + } + recoveryTarget.renameAllTempFiles(); + recoveryTarget.decRef(); + Store.MetadataSnapshot targetSnapshot = targetShard.snapshotStoreMetadata(); + Store.RecoveryDiff diff = sourceSnapshot.recoveryDiff(targetSnapshot); + assertThat(diff.different, empty()); + closeShards(sourceShard, targetShard); + } } diff --git a/server/src/test/java/org/elasticsearch/indices/recovery/RecoverySourceHandlerTests.java b/server/src/test/java/org/elasticsearch/indices/recovery/RecoverySourceHandlerTests.java index 17e88d2864d05..7b9e8fe05dad7 100644 --- a/server/src/test/java/org/elasticsearch/indices/recovery/RecoverySourceHandlerTests.java +++ b/server/src/test/java/org/elasticsearch/indices/recovery/RecoverySourceHandlerTests.java @@ -32,12 +32,15 @@ import org.apache.lucene.store.BaseDirectoryWrapper; import org.apache.lucene.store.Directory; import org.apache.lucene.store.IOContext; +import org.apache.lucene.util.BytesRef; +import org.apache.lucene.util.BytesRefIterator; import org.elasticsearch.ExceptionsHelper; import org.elasticsearch.Version; import org.elasticsearch.action.ActionListener; import org.elasticsearch.action.support.PlainActionFuture; import org.elasticsearch.cluster.metadata.IndexMetaData; import org.elasticsearch.cluster.node.DiscoveryNode; +import org.elasticsearch.common.Numbers; import org.elasticsearch.common.UUIDs; import org.elasticsearch.common.bytes.BytesArray; import org.elasticsearch.common.bytes.BytesReference; @@ -59,6 +62,7 @@ import org.elasticsearch.index.mapper.ParsedDocument; import org.elasticsearch.index.mapper.SeqNoFieldMapper; import org.elasticsearch.index.mapper.Uid; +import org.elasticsearch.index.seqno.ReplicationTracker; import org.elasticsearch.index.seqno.SeqNoStats; import org.elasticsearch.index.seqno.SequenceNumbers; import org.elasticsearch.index.shard.IndexShard; @@ -75,19 +79,29 @@ import org.mockito.ArgumentCaptor; import java.io.IOException; +import java.io.OutputStream; import java.nio.file.Path; import java.util.ArrayList; import java.util.Arrays; import java.util.Collections; import java.util.Comparator; import java.util.List; +import java.util.concurrent.CopyOnWriteArrayList; import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.concurrent.atomic.AtomicLong; +import java.util.concurrent.atomic.AtomicReference; +import java.util.function.IntSupplier; import java.util.function.Supplier; import java.util.stream.Collectors; +import java.util.zip.CRC32; import static java.util.Collections.emptyMap; import static java.util.Collections.emptySet; +import static org.hamcrest.Matchers.containsString; import static org.hamcrest.Matchers.equalTo; +import static org.hamcrest.Matchers.hasSize; +import static org.hamcrest.core.IsNull.notNullValue; import static org.mockito.Matchers.any; import static org.mockito.Matchers.anyBoolean; import static org.mockito.Matchers.anyObject; @@ -109,8 +123,6 @@ public void testSendFiles() throws Throwable { final RecoverySettings recoverySettings = new RecoverySettings(settings, service); final StartRecoveryRequest request = getStartRecoveryRequest(); Store store = newStore(createTempDir()); - RecoverySourceHandler handler = new RecoverySourceHandler(null, null, request, - recoverySettings.getChunkSize().bytesAsInt()); Directory dir = store.directory(); RandomIndexWriter writer = new RandomIndexWriter(random(), dir, newIndexWriterConfig()); int numDocs = randomIntBetween(10, 100); @@ -129,19 +141,38 @@ public void testSendFiles() throws Throwable { metas.add(md); } Store targetStore = newStore(createTempDir()); - handler.sendFiles(store, metas.toArray(new StoreFileMetaData[0]), (md) -> { - try { - return new IndexOutputOutputStream(targetStore.createVerifyingOutput(md.name(), md, IOContext.DEFAULT)) { - @Override - public void close() throws IOException { - super.close(); - targetStore.directory().sync(Collections.singleton(md.name())); // sync otherwise MDW will mess with it + RecoveryTargetHandler target = new TestRecoveryTargetHandler() { + IndexOutputOutputStream out; + @Override + public void writeFileChunk(StoreFileMetaData md, long position, BytesReference content, boolean lastChunk, + int totalTranslogOps, ActionListener listener) { + try { + if (position == 0) { + out = new IndexOutputOutputStream(targetStore.createVerifyingOutput(md.name(), md, IOContext.DEFAULT)) { + @Override + public void close() throws IOException { + super.close(); + targetStore.directory().sync(Collections.singleton(md.name())); // sync otherwise MDW will mess with it + } + }; + } + final BytesRefIterator iterator = content.iterator(); + BytesRef scratch; + while ((scratch = iterator.next()) != null) { + out.write(scratch.bytes, scratch.offset, scratch.length); } - }; - } catch (IOException e) { - throw new RuntimeException(e); + if (lastChunk) { + out.close(); + } + listener.onResponse(null); + } catch (Exception e) { + listener.onFailure(e); + } } - }); + }; + RecoverySourceHandler handler = new RecoverySourceHandler(null, target, request, + Math.toIntExact(recoverySettings.getChunkSize().getBytes()), between(1, 5)); + handler.sendFiles(store, metas.toArray(new StoreFileMetaData[0]), () -> 0); Store.MetadataSnapshot targetStoreMetadata = targetStore.getMetadata(null); Store.RecoveryDiff recoveryDiff = targetStoreMetadata.recoveryDiff(metadata); assertEquals(metas.size(), recoveryDiff.identical.size()); @@ -176,7 +207,7 @@ public void testSendSnapshotSendsOps() throws IOException { when(shard.state()).thenReturn(IndexShardState.STARTED); final RecoveryTargetHandler recoveryTarget = mock(RecoveryTargetHandler.class); final RecoverySourceHandler handler = - new RecoverySourceHandler(shard, recoveryTarget, request, fileChunkSizeInBytes); + new RecoverySourceHandler(shard, recoveryTarget, request, fileChunkSizeInBytes, between(1, 10)); final List operations = new ArrayList<>(); final int initialNumberOfDocs = randomIntBetween(16, 64); for (int i = 0; i < initialNumberOfDocs; i++) { @@ -283,14 +314,6 @@ public void testHandleCorruptedIndexOnSendSendFiles() throws Throwable { Path tempDir = createTempDir(); Store store = newStore(tempDir, false); AtomicBoolean failedEngine = new AtomicBoolean(false); - RecoverySourceHandler handler = new RecoverySourceHandler(null, null, request, - recoverySettings.getChunkSize().bytesAsInt()) { - @Override - protected void failEngine(IOException cause) { - assertFalse(failedEngine.get()); - failedEngine.set(true); - } - }; Directory dir = store.directory(); RandomIndexWriter writer = new RandomIndexWriter(random(), dir, newIndexWriterConfig()); int numDocs = randomIntBetween(10, 100); @@ -313,20 +336,46 @@ protected void failEngine(IOException cause) { (p.getFileName().toString().equals("write.lock") || p.getFileName().toString().startsWith("extra")) == false)); Store targetStore = newStore(createTempDir(), false); - try { - handler.sendFiles(store, metas.toArray(new StoreFileMetaData[0]), (md) -> { + RecoveryTargetHandler target = new TestRecoveryTargetHandler() { + IndexOutputOutputStream out; + @Override + public void writeFileChunk(StoreFileMetaData md, long position, BytesReference content, boolean lastChunk, + int totalTranslogOps, ActionListener listener) { try { - return new IndexOutputOutputStream(targetStore.createVerifyingOutput(md.name(), md, IOContext.DEFAULT)) { - @Override - public void close() throws IOException { - super.close(); - store.directory().sync(Collections.singleton(md.name())); // sync otherwise MDW will mess with it - } - }; - } catch (IOException e) { - throw new RuntimeException(e); + if (position == 0) { + out = new IndexOutputOutputStream(targetStore.createVerifyingOutput(md.name(), md, IOContext.DEFAULT)) { + @Override + public void close() throws IOException { + super.close(); + targetStore.directory().sync(Collections.singleton(md.name())); // sync otherwise MDW will mess with it + } + }; + } + final BytesRefIterator iterator = content.iterator(); + BytesRef scratch; + while ((scratch = iterator.next()) != null) { + out.write(scratch.bytes, scratch.offset, scratch.length); + } + if (lastChunk) { + out.close(); + } + listener.onResponse(null); + } catch (Exception e) { + IOUtils.closeWhileHandlingException(out, () -> listener.onFailure(e)); } - }); + } + }; + RecoverySourceHandler handler = new RecoverySourceHandler(null, target, request, + Math.toIntExact(recoverySettings.getChunkSize().getBytes()), between(1, 8)) { + @Override + protected void failEngine(IOException cause) { + assertFalse(failedEngine.get()); + failedEngine.set(true); + } + }; + + try { + handler.sendFiles(store, metas.toArray(new StoreFileMetaData[0]), () -> 0); fail("corrupted index"); } catch (IOException ex) { assertNotNull(ExceptionsHelper.unwrapCorruption(ex)); @@ -342,14 +391,6 @@ public void testHandleExceptionOnSendFiles() throws Throwable { Path tempDir = createTempDir(); Store store = newStore(tempDir, false); AtomicBoolean failedEngine = new AtomicBoolean(false); - RecoverySourceHandler handler = new RecoverySourceHandler(null, null, request, - recoverySettings.getChunkSize().bytesAsInt()) { - @Override - protected void failEngine(IOException cause) { - assertFalse(failedEngine.get()); - failedEngine.set(true); - } - }; Directory dir = store.directory(); RandomIndexWriter writer = new RandomIndexWriter(random(), dir, newIndexWriterConfig()); int numDocs = randomIntBetween(10, 100); @@ -368,15 +409,27 @@ protected void failEngine(IOException cause) { metas.add(md); } final boolean throwCorruptedIndexException = randomBoolean(); - Store targetStore = newStore(createTempDir(), false); - try { - handler.sendFiles(store, metas.toArray(new StoreFileMetaData[0]), (md) -> { + RecoveryTargetHandler target = new TestRecoveryTargetHandler() { + @Override + public void writeFileChunk(StoreFileMetaData md, long position, BytesReference content, boolean lastChunk, + int totalTranslogOps, ActionListener listener) { if (throwCorruptedIndexException) { - throw new RuntimeException(new CorruptIndexException("foo", "bar")); + listener.onFailure(new RuntimeException(new CorruptIndexException("foo", "bar"))); } else { - throw new RuntimeException("boom"); + listener.onFailure(new RuntimeException("boom")); } - }); + } + }; + RecoverySourceHandler handler = new RecoverySourceHandler(null, target, request, + Math.toIntExact(recoverySettings.getChunkSize().getBytes()), between(1, 10)) { + @Override + protected void failEngine(IOException cause) { + assertFalse(failedEngine.get()); + failedEngine.set(true); + } + }; + try { + handler.sendFiles(store, metas.toArray(new StoreFileMetaData[0]), () -> 0); fail("exception index"); } catch (RuntimeException ex) { assertNull(ExceptionsHelper.unwrapCorruption(ex)); @@ -389,7 +442,7 @@ protected void failEngine(IOException cause) { fail("not expected here"); } assertFalse(failedEngine.get()); - IOUtils.close(store, targetStore); + IOUtils.close(store); } public void testThrowExceptionOnPrimaryRelocatedBeforePhase1Started() throws IOException { @@ -411,7 +464,8 @@ public void testThrowExceptionOnPrimaryRelocatedBeforePhase1Started() throws IOE shard, mock(RecoveryTargetHandler.class), request, - recoverySettings.getChunkSize().bytesAsInt()) { + Math.toIntExact(recoverySettings.getChunkSize().getBytes()), + between(1, 8)) { @Override public SendFileResult phase1(final IndexCommit snapshot, final Supplier translogOps) { @@ -468,9 +522,128 @@ public void testCancellationsDoesNotLeakPrimaryPermits() throws Exception { assertBusy(() -> assertTrue(freed.get())); } + public void testSendFileChunksConcurrently() throws Exception { + final IndexShard shard = mock(IndexShard.class); + when(shard.state()).thenReturn(IndexShardState.STARTED); + final List unrepliedChunks = new CopyOnWriteArrayList<>(); + final AtomicInteger sentChunks = new AtomicInteger(); + final TestRecoveryTargetHandler recoveryTarget = new TestRecoveryTargetHandler() { + final AtomicLong chunkNumberGenerator = new AtomicLong(); + @Override + public void writeFileChunk(StoreFileMetaData md, long position, BytesReference content, boolean lastChunk, + int totalTranslogOps, ActionListener listener) { + final long chunkNumber = chunkNumberGenerator.getAndIncrement(); + logger.info("--> write chunk name={} seq={}, position={}", md.name(), chunkNumber, position); + unrepliedChunks.add(new FileChunkResponse(chunkNumber, listener)); + sentChunks.incrementAndGet(); + } + }; + final int maxConcurrentChunks = between(1, 8); + final int chunkSize = between(1, 32); + final RecoverySourceHandler handler = new RecoverySourceHandler(shard, recoveryTarget, getStartRecoveryRequest(), + chunkSize, maxConcurrentChunks); + Store store = newStore(createTempDir(), false); + List files = generateFiles(store, between(1, 10), () -> between(1, chunkSize * 20)); + int totalChunks = files.stream().mapToInt(md -> ((int) md.length() + chunkSize - 1) / chunkSize).sum(); + Thread sender = new Thread(() -> { + try { + handler.sendFiles(store, files.toArray(new StoreFileMetaData[0]), () -> 0); + } catch (Exception ex) { + throw new AssertionError(ex); + } + }); + sender.start(); + assertBusy(() -> { + assertThat(sentChunks.get(), equalTo(Math.min(totalChunks, maxConcurrentChunks))); + assertThat(unrepliedChunks, hasSize(sentChunks.get())); + }); + + List ackedChunks = new ArrayList<>(); + while (sentChunks.get() < totalChunks || unrepliedChunks.isEmpty() == false) { + List chunksToAck = randomSubsetOf(between(1, unrepliedChunks.size()), unrepliedChunks); + unrepliedChunks.removeAll(chunksToAck); + ackedChunks.addAll(chunksToAck); + ackedChunks.sort(Comparator.comparing(c -> c.chunkNumber)); + int checkpoint = -1; + for (int i = 0; i < ackedChunks.size(); i++) { + if (i != ackedChunks.get(i).chunkNumber) { + break; + } else { + checkpoint = i; + } + } + int chunksToSend = Math.min( + totalChunks - sentChunks.get(), // limited by the remaining chunks + maxConcurrentChunks - (sentChunks.get() - 1 - checkpoint)); // limited by the buffering chunks + + int expectedSentChunks = sentChunks.get() + chunksToSend; + int expectedUnrepliedChunks = unrepliedChunks.size() + chunksToSend; + chunksToAck.forEach(c -> c.listener.onResponse(null)); + assertBusy(() -> { + assertThat(sentChunks.get(), equalTo(expectedSentChunks)); + assertThat(unrepliedChunks, hasSize(expectedUnrepliedChunks)); + }); + } + sender.join(); + store.close(); + } + + public void testSendFileChunksStopOnError() throws Exception { + final IndexShard shard = mock(IndexShard.class); + when(shard.state()).thenReturn(IndexShardState.STARTED); + final List unrepliedChunks = new CopyOnWriteArrayList<>(); + final AtomicInteger sentChunks = new AtomicInteger(); + final TestRecoveryTargetHandler recoveryTarget = new TestRecoveryTargetHandler() { + final AtomicLong chunkNumberGenerator = new AtomicLong(); + @Override + public void writeFileChunk(StoreFileMetaData md, long position, BytesReference content, boolean lastChunk, + int totalTranslogOps, ActionListener listener) { + final long chunkNumber = chunkNumberGenerator.getAndIncrement(); + logger.info("--> write chunk name={} seq={}, position={}", md.name(), chunkNumber, position); + unrepliedChunks.add(new FileChunkResponse(chunkNumber, listener)); + sentChunks.incrementAndGet(); + } + }; + final int maxConcurrentChunks = between(1, 4); + final int chunkSize = between(1, 16); + final RecoverySourceHandler handler = new RecoverySourceHandler(shard, recoveryTarget, getStartRecoveryRequest(), + chunkSize, maxConcurrentChunks); + Store store = newStore(createTempDir(), false); + List files = generateFiles(store, between(1, 10), () -> between(1, chunkSize * 20)); + int totalChunks = files.stream().mapToInt(md -> ((int) md.length() + chunkSize - 1) / chunkSize).sum(); + AtomicReference error = new AtomicReference<>(); + Thread sender = new Thread(() -> { + try { + handler.sendFiles(store, files.toArray(new StoreFileMetaData[0]), () -> 0); + } catch (Exception ex) { + error.set(ex); + } + }); + sender.start(); + assertBusy(() -> assertThat(sentChunks.get(), equalTo(Math.min(totalChunks, maxConcurrentChunks)))); + List failedChunks = randomSubsetOf(between(1, unrepliedChunks.size()), unrepliedChunks); + failedChunks.forEach(c -> c.listener.onFailure(new RuntimeException("test chunk exception"))); + unrepliedChunks.removeAll(failedChunks); + unrepliedChunks.forEach(c -> { + if (randomBoolean()) { + c.listener.onFailure(new RuntimeException("test")); + } else { + c.listener.onResponse(null); + } + }); + assertBusy(() -> { + assertThat(error.get(), notNullValue()); + assertThat(error.get().getMessage(), containsString("test chunk exception")); + }); + assertThat("no more chunks should be sent", sentChunks.get(), equalTo(Math.min(totalChunks, maxConcurrentChunks))); + sender.join(); + store.close(); + } + private Store newStore(Path path) throws IOException { return newStore(path, true); } + private Store newStore(Path path, boolean checkIndex) throws IOException { BaseDirectoryWrapper baseDirectoryWrapper = RecoverySourceHandlerTests.newFSDirectory(path); if (checkIndex == false) { @@ -479,5 +652,68 @@ private Store newStore(Path path, boolean checkIndex) throws IOException { return new Store(shardId, INDEX_SETTINGS, baseDirectoryWrapper, new DummyShardLock(shardId)); } + static final class FileChunkResponse { + final long chunkNumber; + final ActionListener listener; + + FileChunkResponse(long chunkNumber, ActionListener listener) { + this.chunkNumber = chunkNumber; + this.listener = listener; + } + } + + private List generateFiles(Store store, int numFiles, IntSupplier fileSizeSupplier) throws IOException { + List files = new ArrayList<>(); + for (int i = 0; i < numFiles; i++) { + byte[] buffer = randomByteArrayOfLength(fileSizeSupplier.getAsInt()); + CRC32 digest = new CRC32(); + digest.update(buffer, 0, buffer.length); + StoreFileMetaData md = new StoreFileMetaData("test-" + i, buffer.length + 8, + Store.digestToString(digest.getValue()), org.apache.lucene.util.Version.LATEST); + try (OutputStream out = new IndexOutputOutputStream(store.createVerifyingOutput(md.name(), md, IOContext.DEFAULT))) { + out.write(buffer); + out.write(Numbers.longToBytes(digest.getValue())); + } + store.directory().sync(Collections.singleton(md.name())); + files.add(md); + } + return files; + } + + class TestRecoveryTargetHandler implements RecoveryTargetHandler { + @Override + public void prepareForTranslogOperations(boolean fileBasedRecovery, int totalTranslogOps) { + } + + @Override + public void finalizeRecovery(long globalCheckpoint) { + } + + @Override + public void ensureClusterStateVersion(long clusterStateVersion) { + } + + @Override + public void handoffPrimaryContext(ReplicationTracker.PrimaryContext primaryContext) { + } + + @Override + public long indexTranslogOperations(List operations, int totalTranslogOps, long timestamp, long msu) { + return 0; + } + @Override + public void receiveFileInfo(List phase1FileNames, List phase1FileSizes, List phase1ExistingFileNames, + List phase1ExistingFileSizes, int totalTranslogOps) { + } + + @Override + public void cleanFiles(int totalTranslogOps, Store.MetadataSnapshot sourceMetaData) { + } + + @Override + public void writeFileChunk(StoreFileMetaData fileMetaData, long position, BytesReference content, boolean lastChunk, + int totalTranslogOps, ActionListener listener) { + } + } } diff --git a/server/src/test/java/org/elasticsearch/recovery/RelocationIT.java b/server/src/test/java/org/elasticsearch/recovery/RelocationIT.java index 04fb8c08a9778..45f0fce3b8143 100644 --- a/server/src/test/java/org/elasticsearch/recovery/RelocationIT.java +++ b/server/src/test/java/org/elasticsearch/recovery/RelocationIT.java @@ -385,7 +385,7 @@ public void testCancellationCleansTempFiles() throws Exception { assertFalse(client().admin().cluster().prepareHealth().setWaitForNodes("3").setWaitForGreenStatus().get().isTimedOut()); flush(); - int allowedFailures = randomIntBetween(3, 10); + int allowedFailures = randomIntBetween(3, 5); // the default of the `index.allocation.max_retries` is 5. logger.info("--> blocking recoveries from primary (allowed failures: [{}])", allowedFailures); CountDownLatch corruptionCount = new CountDownLatch(allowedFailures); ClusterService clusterService = internalCluster().getInstance(ClusterService.class, p_node); diff --git a/test/framework/src/main/java/org/elasticsearch/index/shard/IndexShardTestCase.java b/test/framework/src/main/java/org/elasticsearch/index/shard/IndexShardTestCase.java index 0ffc9f05ff265..273967509f4a1 100644 --- a/test/framework/src/main/java/org/elasticsearch/index/shard/IndexShardTestCase.java +++ b/test/framework/src/main/java/org/elasticsearch/index/shard/IndexShardTestCase.java @@ -610,10 +610,7 @@ protected final void recoverUnstartedReplica(final IndexShard replica, final StartRecoveryRequest request = new StartRecoveryRequest(replica.shardId(), targetAllocationId, pNode, rNode, snapshot, replica.routingEntry().primary(), 0, startingSeqNo); final RecoverySourceHandler recovery = new RecoverySourceHandler( - primary, - recoveryTarget, - request, - (int) ByteSizeUnit.MB.toBytes(1)); + primary, recoveryTarget, request, Math.toIntExact(ByteSizeUnit.MB.toBytes(1)), between(1, 8)); primary.updateShardState(primary.routingEntry(), primary.getPendingPrimaryTerm(), null, currentClusterStateVersion.incrementAndGet(), inSyncIds, routingTable, Collections.emptySet()); diff --git a/test/framework/src/main/java/org/elasticsearch/test/InternalTestCluster.java b/test/framework/src/main/java/org/elasticsearch/test/InternalTestCluster.java index e4a11ad414ffe..c1d61125399bb 100644 --- a/test/framework/src/main/java/org/elasticsearch/test/InternalTestCluster.java +++ b/test/framework/src/main/java/org/elasticsearch/test/InternalTestCluster.java @@ -390,6 +390,8 @@ public InternalTestCluster( // always reduce this - it can make tests really slow builder.put(RecoverySettings.INDICES_RECOVERY_RETRY_DELAY_STATE_SYNC_SETTING.getKey(), TimeValue.timeValueMillis( RandomNumbers.randomIntBetween(random, 20, 50))); + builder.put(RecoverySettings.INDICES_RECOVERY_MAX_CONCURRENT_FILE_CHUNKS_SETTING.getKey(), + RandomNumbers.randomIntBetween(random, 1, 5)); defaultSettings = builder.build(); executor = EsExecutors.newScaling("internal_test_cluster_executor", 0, Integer.MAX_VALUE, 0, TimeUnit.SECONDS, EsExecutors.daemonThreadFactory("test_" + clusterName), new ThreadContext(Settings.EMPTY)); From 1e3702da0b54e32b0e4aaa8ea1955cbbe4215afa Mon Sep 17 00:00:00 2001 From: Nhat Nguyen Date: Mon, 14 Jan 2019 11:40:11 -0500 Subject: [PATCH 151/186] Relax assertSameDocIdsOnShards assertion If the checking node no longer holds the shard copy, the assertion assertSameDocIdsOnShards might fail. This is too harsh since the assertion is to ensure the consistency between active copies. --- .../test/InternalTestCluster.java | 41 ++++++++----------- 1 file changed, 17 insertions(+), 24 deletions(-) diff --git a/test/framework/src/main/java/org/elasticsearch/test/InternalTestCluster.java b/test/framework/src/main/java/org/elasticsearch/test/InternalTestCluster.java index c1d61125399bb..70d3c95eb7f7d 100644 --- a/test/framework/src/main/java/org/elasticsearch/test/InternalTestCluster.java +++ b/test/framework/src/main/java/org/elasticsearch/test/InternalTestCluster.java @@ -139,7 +139,6 @@ import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicInteger; -import java.util.function.BiFunction; import java.util.function.Function; import java.util.function.Predicate; import java.util.stream.Collectors; @@ -1293,26 +1292,24 @@ public void assertConsistentHistoryBetweenTranslogAndLuceneIndex() throws IOExce } } + private IndexShard getShardOrNull(ClusterState clusterState, ShardRouting shardRouting) { + if (shardRouting == null || shardRouting.assignedToNode() == false) { + return null; + } + final DiscoveryNode assignedNode = clusterState.nodes().get(shardRouting.currentNodeId()); + if (assignedNode == null) { + return null; + } + return getInstance(IndicesService.class, assignedNode.getName()).getShardOrNull(shardRouting.shardId()); + } + public void assertSeqNos() throws Exception { - final BiFunction getInstanceShardInstance = (clusterState, shardRouting) -> { - if (shardRouting.assignedToNode() == false) { - return null; - } - final DiscoveryNode assignedNode = clusterState.nodes().get(shardRouting.currentNodeId()); - if (assignedNode == null) { - return null; - } - return getInstance(IndicesService.class, assignedNode.getName()).getShardOrNull(shardRouting.shardId()); - }; assertBusy(() -> { final ClusterState state = clusterService().state(); for (ObjectObjectCursor indexRoutingTable : state.routingTable().indicesRouting()) { for (IntObjectCursor indexShardRoutingTable : indexRoutingTable.value.shards()) { ShardRouting primaryShardRouting = indexShardRoutingTable.value.primaryShard(); - if (primaryShardRouting == null) { - continue; - } - final IndexShard primaryShard = getInstanceShardInstance.apply(state, primaryShardRouting); + final IndexShard primaryShard = getShardOrNull(state, primaryShardRouting); if (primaryShard == null) { continue; //just ignore - shard movement } @@ -1327,7 +1324,7 @@ public void assertSeqNos() throws Exception { assertThat(primaryShardRouting + " should have set the global checkpoint", primarySeqNoStats.getGlobalCheckpoint(), not(equalTo(SequenceNumbers.UNASSIGNED_SEQ_NO))); for (ShardRouting replicaShardRouting : indexShardRoutingTable.value.replicaShards()) { - final IndexShard replicaShard = getInstanceShardInstance.apply(state, replicaShardRouting); + final IndexShard replicaShard = getShardOrNull(state, replicaShardRouting); if (replicaShard == null) { continue; //just ignore - shard movement } @@ -1356,12 +1353,10 @@ public void assertSameDocIdsOnShards() throws Exception { for (ObjectObjectCursor indexRoutingTable : state.routingTable().indicesRouting()) { for (IntObjectCursor indexShardRoutingTable : indexRoutingTable.value.shards()) { ShardRouting primaryShardRouting = indexShardRoutingTable.value.primaryShard(); - if (primaryShardRouting == null || primaryShardRouting.assignedToNode() == false) { + IndexShard primaryShard = getShardOrNull(state, primaryShardRouting); + if (primaryShard == null) { continue; } - DiscoveryNode primaryNode = state.nodes().get(primaryShardRouting.currentNodeId()); - IndexShard primaryShard = getInstance(IndicesService.class, primaryNode.getName()) - .indexServiceSafe(primaryShardRouting.index()).getShard(primaryShardRouting.id()); final List docsOnPrimary; try { docsOnPrimary = IndexShardTestCase.getDocIdAndSeqNos(primaryShard); @@ -1369,12 +1364,10 @@ public void assertSameDocIdsOnShards() throws Exception { continue; } for (ShardRouting replicaShardRouting : indexShardRoutingTable.value.replicaShards()) { - if (replicaShardRouting.assignedToNode() == false) { + IndexShard replicaShard = getShardOrNull(state, replicaShardRouting); + if (replicaShard == null) { continue; } - DiscoveryNode replicaNode = state.nodes().get(replicaShardRouting.currentNodeId()); - IndexShard replicaShard = getInstance(IndicesService.class, replicaNode.getName()) - .indexServiceSafe(replicaShardRouting.index()).getShard(replicaShardRouting.id()); final List docsOnReplica; try { docsOnReplica = IndexShardTestCase.getDocIdAndSeqNos(replicaShard); From b86621c157eb46083f985ae830b56dfadbea6ea5 Mon Sep 17 00:00:00 2001 From: Shaunak Kashyap Date: Mon, 14 Jan 2019 12:41:10 -0800 Subject: [PATCH 152/186] Adding mapping for hostname field (#37288) This new `hostname` field is meant to be a replacement for its sibling `name` field. See https://github.com/elastic/beats/pull/9943, particularly https://github.com/elastic/beats/pull/9943#discussion_r245932581. This PR simply adds the new field (`hostname`) to the mapping without removing the old one (`name`), because a user might be running an older-version Beat (without this field rename in it) with a newer-version Monitoring ES cluster (with this PR's change in it). AFAICT the Monitoring UI isn't currently using the `name` field so no changes are necessary there yet. If it decides to start using the `name` field, it will also want to look at the value of the `hostname` field. --- x-pack/plugin/core/src/main/resources/monitoring-beats.json | 3 +++ 1 file changed, 3 insertions(+) diff --git a/x-pack/plugin/core/src/main/resources/monitoring-beats.json b/x-pack/plugin/core/src/main/resources/monitoring-beats.json index 235e516001d0b..43ccfedd923dd 100644 --- a/x-pack/plugin/core/src/main/resources/monitoring-beats.json +++ b/x-pack/plugin/core/src/main/resources/monitoring-beats.json @@ -52,6 +52,9 @@ "name": { "type": "keyword" }, + "hostname": { + "type": "keyword" + }, "os": { "properties": { "build": { From f3edbe291146d92888087c6458a08d2e4cbb602d Mon Sep 17 00:00:00 2001 From: Jay Modi Date: Mon, 14 Jan 2019 14:06:22 -0700 Subject: [PATCH 153/186] Security: remove SSL settings fallback (#36846) This commit removes the fallback for SSL settings. While this may be seen as a non user friendly change, the intention behind this change is to simplify the reasoning needed to understand what is actually being used for a given SSL configuration. Each configuration now needs to be explicitly specified as there is no global configuration or fallback to some other configuration. Closes #29797 --- client/rest-high-level/build.gradle | 2 +- .../migration/migrate_7_0/settings.asciidoc | 10 + .../configuring-tls-docker.asciidoc | 26 +- .../tls-transport.asciidoc | 4 +- .../settings/security-settings.asciidoc | 189 +++----------- docs/reference/settings/ssl-settings.asciidoc | 23 +- x-pack/docs/en/rest-api/security/ssl.asciidoc | 10 +- .../auditing/forwarding-logs.asciidoc | 16 +- .../configuring-pki-realm.asciidoc | 2 +- .../ccs-clients-integrations/java.asciidoc | 18 +- .../en/security/fips-140-compliance.asciidoc | 6 +- .../xpack/core/XPackSettings.java | 5 - .../netty4/SecurityNetty4Transport.java | 15 +- .../xpack/core/ssl/DefaultJDKTrustConfig.java | 8 +- .../xpack/core/ssl/SSLConfiguration.java | 67 +---- .../xpack/core/ssl/SSLService.java | 45 +--- .../ssl/SSLConfigurationReloaderTests.java | 47 ++-- .../ssl/SSLConfigurationSettingsTests.java | 2 +- .../xpack/core/ssl/SSLConfigurationTests.java | 80 ++---- .../xpack/core/ssl/SSLServiceTests.java | 243 ++++++++---------- .../xpack/core/ssl/TestsSSLService.java | 5 - .../exporter/http/HttpExporterSslIT.java | 8 +- .../security/PkiRealmBootstrapCheck.java | 5 +- .../esnative/ESNativeRealmMigrateTool.java | 3 +- .../transport/nio/SecurityNioTransport.java | 1 - .../test/SecuritySettingsSource.java | 112 ++++---- .../test/SettingsFilterTests.java | 16 +- ...FIPS140JKSKeystoreBootstrapCheckTests.java | 16 -- .../security/PkiRealmBootstrapCheckTests.java | 11 +- .../audit/index/IndexAuditTrailTests.java | 2 +- .../RemoteIndexAuditTrailStartingTests.java | 2 +- .../esnative/ESNativeMigrateToolTests.java | 34 ++- .../tool/CommandLineHttpClientTests.java | 31 +-- .../security/authc/ldap/LdapTestUtils.java | 32 +-- .../LdapUserSearchSessionFactoryTests.java | 2 +- .../authc/pki/PkiAuthenticationTests.java | 22 +- .../authc/pki/PkiOptionalClientAuthTests.java | 12 +- .../security/authc/saml/SamlRealmTests.java | 11 +- ...stractSimpleSecurityTransportTestCase.java | 20 +- ...ServerTransportFilterIntegrationTests.java | 76 +++--- .../netty4/IPHostnameVerificationTests.java | 26 +- ...ecurityNetty4HttpServerTransportTests.java | 16 +- .../SecurityNetty4ServerTransportTests.java | 46 +--- .../netty4/SslHostnameVerificationTests.java | 31 +-- .../SecurityNioHttpServerTransportTests.java | 14 +- .../transport/ssl/EllipticCurveSSLTests.java | 24 +- .../transport/ssl/SslIntegrationTests.java | 13 +- .../transport/ssl/SslMultiPortTests.java | 35 +-- .../xpack/ssl/SSLClientAuthTests.java | 63 ++++- .../xpack/ssl/SSLReloadIntegTests.java | 26 +- .../xpack/ssl/SSLTrustRestrictionsTests.java | 22 +- .../sql/qa/security/with-ssl/build.gradle | 6 +- .../watcher/common/http/HttpClientTests.java | 78 ++---- x-pack/qa/full-cluster-restart/build.gradle | 8 +- .../org/elasticsearch/test/OpenLdapTests.java | 41 +-- ...OpenLdapUserSearchSessionFactoryTests.java | 4 +- x-pack/qa/rolling-upgrade/build.gradle | 8 +- x-pack/qa/smoke-test-plugins-ssl/build.gradle | 3 +- .../ADLdapUserSearchSessionFactoryTests.java | 11 +- .../ldap/AbstractActiveDirectoryTestCase.java | 41 ++- .../ldap/AbstractAdLdapRealmTestCase.java | 42 +-- .../ActiveDirectorySessionFactoryTests.java | 43 ++-- .../resources/packaging/tests/certgen.bash | 18 +- 63 files changed, 783 insertions(+), 1075 deletions(-) diff --git a/client/rest-high-level/build.gradle b/client/rest-high-level/build.gradle index ed9b4451db350..b71ca82c7d094 100644 --- a/client/rest-high-level/build.gradle +++ b/client/rest-high-level/build.gradle @@ -105,7 +105,7 @@ integTestCluster { setting 'xpack.security.enabled', 'true' setting 'xpack.security.authc.token.enabled', 'true' // Truststore settings are not used since TLS is not enabled. Included for testing the get certificates API - setting 'xpack.ssl.certificate_authorities', 'testnode.crt' + setting 'xpack.security.http.ssl.certificate_authorities', 'testnode.crt' setting 'xpack.security.transport.ssl.truststore.path', 'testnode.jks' keystoreSetting 'xpack.security.transport.ssl.truststore.secure_password', 'testnode' setupCommand 'setupDummyUser', diff --git a/docs/reference/migration/migrate_7_0/settings.asciidoc b/docs/reference/migration/migrate_7_0/settings.asciidoc index 6144888fb545d..9a271c65271a3 100644 --- a/docs/reference/migration/migrate_7_0/settings.asciidoc +++ b/docs/reference/migration/migrate_7_0/settings.asciidoc @@ -121,3 +121,13 @@ xpack.security.authc.realms: Any realm specific secure settings that have been stored in the elasticsearch keystore (such as ldap bind passwords, or passwords for ssl keys) must be updated in a similar way. + +[float] +[[tls-setting-fallback]] +==== TLS/SSL settings + +The default TLS/SSL settings, which were prefixed by `xpack.ssl`, have been removed. +The removal of these default settings also removes the ability for a component to +fallback to a default configuration when using TLS. Each component (realm, transport, http, +http client, etc) must now be configured with their own settings for TLS if it is being +used. diff --git a/docs/reference/security/securing-communications/configuring-tls-docker.asciidoc b/docs/reference/security/securing-communications/configuring-tls-docker.asciidoc index c9d645e4fa234..50c63de4b4fef 100644 --- a/docs/reference/security/securing-communications/configuring-tls-docker.asciidoc +++ b/docs/reference/security/securing-communications/configuring-tls-docker.asciidoc @@ -114,11 +114,14 @@ services: - xpack.license.self_generated.type=trial <2> - xpack.security.enabled=true - xpack.security.http.ssl.enabled=true + - xpack.security.http.ssl.key=$CERTS_DIR/es01/es01.key + - xpack.security.http.ssl.certificate_authorities=$CERTS_DIR/ca/ca.crt + - xpack.security.http.ssl.certificate=$CERTS_DIR/es01/es01.crt - xpack.security.transport.ssl.enabled=true - xpack.security.transport.ssl.verification_mode=certificate <3> - - xpack.ssl.certificate_authorities=$CERTS_DIR/ca/ca.crt - - xpack.ssl.certificate=$CERTS_DIR/es01/es01.crt - - xpack.ssl.key=$CERTS_DIR/es01/es01.key + - xpack.security.transport.ssl.certificate_authorities=$CERTS_DIR/ca/ca.crt + - xpack.security.transport.ssl.certificate=$CERTS_DIR/es01/es01.crt + - xpack.security.transport.ssl.key=$CERTS_DIR/es01/es01.key volumes: ['esdata_01:/usr/share/elasticsearch/data', './certs:$CERTS_DIR'] ports: - 9200:9200 @@ -140,11 +143,14 @@ services: - xpack.license.self_generated.type=trial - xpack.security.enabled=true - xpack.security.http.ssl.enabled=true + - xpack.security.http.ssl.key=$CERTS_DIR/es02/es02.key + - xpack.security.http.ssl.certificate_authorities=$CERTS_DIR/ca/ca.crt + - xpack.security.http.ssl.certificate=$CERTS_DIR/es02/es02.crt - xpack.security.transport.ssl.enabled=true - - xpack.security.transport.ssl.verification_mode=certificate - - xpack.ssl.certificate_authorities=$CERTS_DIR/ca/ca.crt - - xpack.ssl.certificate=$CERTS_DIR/es02/es02.crt - - xpack.ssl.key=$CERTS_DIR/es02/es02.key + - xpack.security.transport.ssl.verification_mode=certificate <3> + - xpack.security.transport.ssl.certificate_authorities=$CERTS_DIR/ca/ca.crt + - xpack.security.transport.ssl.certificate=$CERTS_DIR/es02/es02.crt + - xpack.security.transport.ssl.key=$CERTS_DIR/es02/es02.key volumes: ['esdata_02:/usr/share/elasticsearch/data', './certs:$CERTS_DIR'] wait_until_ready: @@ -199,9 +205,9 @@ WARNING: Windows users not running PowerShell will need to remove `\` and join l ---- docker exec es01 /bin/bash -c "bin/elasticsearch-setup-passwords \ auto --batch \ --Expack.ssl.certificate=certificates/es01/es01.crt \ --Expack.ssl.certificate_authorities=certificates/ca/ca.crt \ --Expack.ssl.key=certificates/es01/es01.key \ +-Expack.security.http.ssl.certificate=certificates/es01/es01.crt \ +-Expack.security.http.ssl.certificate_authorities=certificates/ca/ca.crt \ +-Expack.security.http.ssl.key=certificates/es01/es01.key \ --url https://localhost:9200" ---- -- diff --git a/docs/reference/security/securing-communications/tls-transport.asciidoc b/docs/reference/security/securing-communications/tls-transport.asciidoc index fee775078d6a2..e32d123140a65 100644 --- a/docs/reference/security/securing-communications/tls-transport.asciidoc +++ b/docs/reference/security/securing-communications/tls-transport.asciidoc @@ -25,7 +25,7 @@ xpack.security.transport.ssl.truststore.path: certs/elastic-certificates.p12 <3> <1> If you used the `--dns` or `--ip` options with the `elasticsearch-certutil cert` command and you want to enable strict hostname checking, set the verification mode to `full`. -See <> for a description of these values. +See <> for a description of these values. <2> If you created a separate certificate for each node, then you might need to customize this path on each node. If the filename matches the node name, you can @@ -54,7 +54,7 @@ xpack.security.transport.ssl.certificate_authorities: [ "/home/es/config/ca.crt" <1> If you used the `--dns` or `--ip` options with the `elasticsearch-certutil cert` command and you want to enable strict hostname checking, set the verification mode to `full`. -See <> for a description of these values. +See <> for a description of these values. <2> The full path to the node key file. This must be a location within the {es} configuration directory. <3> The full path to the node certificate. This must be a location within the diff --git a/docs/reference/settings/security-settings.asciidoc b/docs/reference/settings/security-settings.asciidoc index 28c30bf665cf2..0a88a19f6f050 100644 --- a/docs/reference/settings/security-settings.asciidoc +++ b/docs/reference/settings/security-settings.asciidoc @@ -475,20 +475,18 @@ The default is `jks`. `ssl.verification_mode`:: Indicates the type of verification when using `ldaps` to protect against man in the middle attacks and certificate forgery. Values are `none`, `certificate`, -and `full`. Defaults to the value of `xpack.ssl.verification_mode`. +and `full`. Defaults to `full`. + -See <> for an explanation of -these values. +See <> for an explanation of these values. `ssl.supported_protocols`:: -Supported protocols for TLS/SSL (with versions). Defaults to the value of -`xpack.ssl.supported_protocols`. +Supported protocols for TLS/SSL (with versions). Defaults to `TLSv1.2,TLSv1.1,TLSv1`. `ssl.cipher_suites`:: Specifies the cipher suites that should be supported when communicating with the LDAP server. Supported cipher suites can be found in Oracle's http://docs.oracle.com/javase/8/docs/technotes/guides/security/SunProviders.html[ -Java Cryptography Architecture documentation]. Defaults to the value of -`xpack.ssl.cipher_suites`. +Java Cryptography Architecture documentation]. See <> +for the default value. `cache.ttl`:: Specifies the time-to-live for cached user entries. A user and a hash of its @@ -721,20 +719,18 @@ The default is `jks`. `ssl.verification_mode`:: Indicates the type of verification when using `ldaps` to protect against man in the middle attacks and certificate forgery. Values are `none`, `certificate`, -and `full`. Defaults to the value of `xpack.ssl.verification_mode`. +and `full`. Defaults to `full`. + -See <> for an explanation of -these values. +See <> for an explanation of these values. `ssl.supported_protocols`:: -Supported protocols for TLS/SSL (with versions). Defaults to the value of -`xpack.ssl.supported_protocols`. +Supported protocols for TLS/SSL (with versions). Defaults to `TLSv1.2, TLSv1.1, TLSv1`. `ssl.cipher_suites`:: Specifies the cipher suites that should be supported when communicating with the Active Directory server. Supported cipher suites can be found in Oracle's http://docs.oracle.com/javase/8/docs/technotes/guides/security/SunProviders.html[ -Java Cryptography Architecture documentation]. Defaults to the value of -`xpack.ssl.cipher_suites`. +Java Cryptography Architecture documentation]. See <> for +the default values. `cache.ttl`:: Specifies the time-to-live for cached user entries. A user and a hash of its @@ -1133,8 +1129,7 @@ One of `full` certificate path, but not the hostname) or `none` (perform no verification). Defaults to `full`. + -See <> for a more detailed -explanation of these values. +See <> for a more detailed explanation of these values. `ssl.supported_protocols`:: Specifies the supported protocols for TLS/SSL. @@ -1204,14 +1199,12 @@ through the list of URLs will continue until a successful connection is made. [float] [[ssl-tls-settings]] -==== Default TLS/SSL settings -You can configure the following TLS/SSL settings in -`elasticsearch.yml`. For more information, see -{stack-ov}/encrypting-communications.html[Encrypting communications]. These -settings are used unless they have been overridden by more specific -settings such as those for HTTP or Transport. - -`xpack.ssl.supported_protocols`:: +==== Default values for TLS/SSL settings +In general, the values below represent the default values for the various TLS +settings. For more information, see +{stack-ov}/encrypting-communications.html[Encrypting communications]. + +`ssl.supported_protocols`:: Supported protocols with versions. Valid protocols: `SSLv2Hello`, `SSLv3`, `TLSv1`, `TLSv1.1`, `TLSv1.2`. Defaults to `TLSv1.2`, `TLSv1.1`, `TLSv1`. @@ -1221,15 +1214,15 @@ NOTE: If `xpack.security.fips_mode.enabled` is `true`, you cannot use `SSLv2Hell or `SSLv3`. See <>. -- -`xpack.ssl.client_authentication`:: +`ssl.client_authentication`:: Controls the server's behavior in regard to requesting a certificate from client connections. Valid values are `required`, `optional`, and `none`. `required` forces a client to present a certificate, while `optional` requests a client certificate but the client is not required to present one. -Defaults to `required`. This global setting is not applicable for HTTP, see +Defaults to `required`, except for HTTP, which defaults to `none`. See <>. -`xpack.ssl.verification_mode`:: +`ssl.verification_mode`:: Controls the verification of certificates. Valid values are: - `full`, which verifies that the provided certificate is signed by a trusted authority (CA) and also verifies that the server's hostname (or IP @@ -1244,7 +1237,7 @@ Controls the verification of certificates. Valid values are: + The default value is `full`. -`xpack.ssl.cipher_suites`:: +`ssl.cipher_suites`:: Supported cipher suites can be found in Oracle's http://docs.oracle.com/javase/8/docs/technotes/guides/security/SunProviders.html[ Java Cryptography Architecture documentation]. Defaults to `TLS_ECDHE_RSA_WITH_AES_128_CBC_SHA256`, `TLS_ECDHE_ECDSA_WITH_AES_128_CBC_SHA256`, `TLS_ECDHE_ECDSA_WITH_AES_128_CBC_SHA`, `TLS_ECDHE_RSA_WITH_AES_128_CBC_SHA`, @@ -1255,112 +1248,15 @@ Jurisdiction Policy Files_ has been installed, the default value also includes ` [float] [[tls-ssl-key-settings]] -===== Default TLS/SSL key and trusted certificate settings +===== TLS/SSL key and trusted certificate settings The following settings are used to specify a private key, certificate, and the -trusted certificates that should be used when communicating over an SSL/TLS connection. -If none of the settings below are specified, the -<> are used. If no trusted certificates are configured, the default certificates that are trusted by the JVM will be -trusted along with the certificate(s) from the <>. The key and certificate must be in place -for connections that require client authentication or when acting as a SSL enabled server. - -[float] -===== PEM encoded files - -When using PEM encoded files, use the following settings: - -`xpack.ssl.key`:: -Path to the PEM encoded file containing the private key. - -`xpack.ssl.key_passphrase`:: -The passphrase that is used to decrypt the private key. This value is -optional as the key might not be encrypted. - -`xpack.ssl.secure_key_passphrase` (<>):: -The passphrase that is used to decrypt the private key. This value is -optional as the key might not be encrypted. - -`xpack.ssl.certificate`:: -Path to a PEM encoded file containing the certificate (or certificate chain) -that will be presented to clients when they connect. - -`xpack.ssl.certificate_authorities`:: -List of paths to the PEM encoded certificate files that should be trusted. - -[float] -===== Java keystore files - -When using Java keystore files (JKS), which contain the private key, certificate -and certificates that should be trusted, use the following settings: - -`xpack.ssl.keystore.path`:: -Path to the keystore that holds the private key and certificate. - -`xpack.ssl.keystore.password`:: -Password to the keystore. - -`xpack.ssl.keystore.secure_password` (<>):: -Password to the keystore. - -`xpack.ssl.keystore.key_password`:: -Password for the private key in the keystore. Defaults to the -same value as `xpack.ssl.keystore.password`. - -`xpack.ssl.keystore.secure_key_password` (<>):: -Password for the private key in the keystore. - -`xpack.ssl.truststore.path`:: -Path to the truststore file. - -`xpack.ssl.truststore.password`:: -Password to the truststore. - -`xpack.ssl.truststore.secure_password` (<>):: -Password to the truststore. - -WARNING: If `xpack.security.fips_mode.enabled` is `true`, you cannot use Java -keystore files. See <>. - -[float] -===== PKCS#12 files - -When using PKCS#12 container files (`.p12` or `.pfx`), which contain the -private key, certificate, and certificates that should be trusted, use -the following settings: - -`xpack.ssl.keystore.path`:: -Path to the PKCS#12 file that holds the private key and certificate. - -`xpack.ssl.keystore.type`:: -Set this to `PKCS12`. - -`xpack.ssl.keystore.password`:: -Password to the PKCS#12 file. - -`xpack.ssl.keystore.secure_password` (<>):: -Password to the PKCS#12 file. - -`xpack.ssl.keystore.key_password`:: -Password for the private key in the PKCS12 file. -Defaults to the same value as `xpack.ssl.keystore.password`. - -`xpack.ssl.keystore.secure_key_password` (<>):: -Password for the private key in the PKCS12 file. - -`xpack.ssl.truststore.path`:: -Path to the truststore file. - -`xpack.ssl.truststore.type`:: -Set this to `PKCS12`. - -`xpack.ssl.truststore.password`:: -Password to the truststore. - -`xpack.ssl.truststore.secure_password` (<>):: -Password to the truststore. - -WARNING: If `xpack.security.fips_mode.enabled` is `true`, you cannot use PKCS#12 -keystore files. See <>. +trusted certificates that should be used when communicating over an SSL/TLS +connection. If no trusted certificates are configured, the default certificates +that are trusted by the JVM will be trusted along with the certificate(s) +associated with a key in the same context. The key and certificate must be in +place for connections that require client authentication or when acting as a +SSL enabled server. [[pkcs12-truststore-note]] [NOTE] @@ -1375,33 +1271,6 @@ a PKCS#12 container includes trusted certificate ("anchor") entries look for `openssl pkcs12 -info` output, or `trustedCertEntry` in the `keytool -list` output. -[float] -===== PKCS#11 tokens - -When using a PKCS#11 cryptographic token, which contains the -private key, certificate, and certificates that should be trusted, use -the following settings: - -`xpack.ssl.keystore.type`:: -Set this to `PKCS11`. - -`xpack.ssl.truststore.type`:: -Set this to `PKCS11`. - - -[[pkcs11-truststore-note]] -[NOTE] -When configuring the PKCS#11 token that your JVM is configured to use as -a keystore or a truststore for Elasticsearch, the PIN for the token can be -configured by setting the appropriate value to `xpack.ssl.truststore.password` -or `xpack.ssl.truststore.secure_password`. In the absence of the above, {es} will -fallback to use he appropriate JVM setting (`-Djavax.net.ssl.trustStorePassword`) -if that is set. -Since there can only be one PKCS#11 token configured, only one keystore and -truststore will be usable for configuration in {es}. This in turn means -that only one certificate can be used for TLS both in the transport and the -http layer. - [[http-tls-ssl-settings]] :ssl-prefix: xpack.security.http :component: HTTP @@ -1435,7 +1304,7 @@ append the portion of the setting after `xpack.security.transport.`. For the key setting, this would be `transport.profiles.$PROFILE.xpack.security.ssl.key`. [[auditing-tls-ssl-settings]] -:ssl-prefix: xpack.security.audit.index.client.xpack +:ssl-prefix: xpack.security.audit.index.client.xpack.security.transport :component: Auditing :client-auth-default!: :server!: diff --git a/docs/reference/settings/ssl-settings.asciidoc b/docs/reference/settings/ssl-settings.asciidoc index 1757cc481c3fe..1ff9ebc03ae8d 100644 --- a/docs/reference/settings/ssl-settings.asciidoc +++ b/docs/reference/settings/ssl-settings.asciidoc @@ -12,7 +12,7 @@ endif::server[] +{ssl-prefix}.ssl.supported_protocols+:: Supported protocols with versions. Valid protocols: `SSLv2Hello`, `SSLv3`, `TLSv1`, `TLSv1.1`, `TLSv1.2`. Defaults to `TLSv1.2`, `TLSv1.1`, -`TLSv1`. Defaults to the value of `xpack.ssl.supported_protocols`. +`TLSv1`. ifdef::server[] +{ssl-prefix}.ssl.client_authentication+:: @@ -21,7 +21,7 @@ from client connections. Valid values are `required`, `optional`, and `none`. `required` forces a client to present a certificate, while `optional` requests a client certificate but the client is not required to present one. ifndef::client-auth-default[] -Defaults to the value of `xpack.ssl.client_authentication`. +Defaults to `none``. endif::client-auth-default[] ifdef::client-auth-default[] Defaults to +{client-auth-default}+. @@ -31,15 +31,12 @@ endif::server[] ifdef::verifies[] +{ssl-prefix}.ssl.verification_mode+:: Controls the verification of certificates. Valid values are `none`, -`certificate`, and `full`. -See <> for a description of these values. -Defaults to the value of `xpack.ssl.verification_mode`. +`certificate`, and `full`. Defaults to `full`. endif::verifies[] +{ssl-prefix}.ssl.cipher_suites+:: Supported cipher suites can be found in Oracle's http://docs.oracle.com/javase/8/docs/technotes/guides/security/SunProviders.html[ -Java Cryptography Architecture documentation]. Defaults to the value of -`xpack.ssl.cipher_suites`. +Java Cryptography Architecture documentation]. Defaults to ``. ===== {component} TLS/SSL Key and Trusted Certificate Settings @@ -158,4 +155,14 @@ via the following settings: Set this to `PKCS11` to indicate that the PKCS#11 token should be used as a keystore. +{ssl-prefix}.truststore.type+:: -Set this to `PKCS11` to indicate that the PKCS#11 token should be used as a truststore. \ No newline at end of file +Set this to `PKCS11` to indicate that the PKCS#11 token should be used as a truststore. + +[NOTE] +When configuring the PKCS#11 token that your JVM is configured to use as +a keystore or a truststore for Elasticsearch, the PIN for the token can be +configured by setting the appropriate value to `ssl.truststore.password` +or `ssl.truststore.secure_password` in the context that you are configuring. +Since there can only be one PKCS#11 token configured, only one keystore and +truststore will be usable for configuration in {es}. This in turn means +that only one certificate can be used for TLS both in the transport and the +http layer. diff --git a/x-pack/docs/en/rest-api/security/ssl.asciidoc b/x-pack/docs/en/rest-api/security/ssl.asciidoc index de73407355b17..d3480ac8bc006 100644 --- a/x-pack/docs/en/rest-api/security/ssl.asciidoc +++ b/x-pack/docs/en/rest-api/security/ssl.asciidoc @@ -22,16 +22,16 @@ Transport Layer Security (TLS), see The API returns a list that includes certificates from all TLS contexts including: -* Default {es} TLS settings * Settings for transport and HTTP interfaces * TLS settings that are used within authentication realms * TLS settings for remote monitoring exporters The list includes certificates that are used for configuring trust, such as -those configured in the `xpack.ssl.truststore` and -`xpack.ssl.certificate_authorities` settings. It also includes certificates that -that are used for configuring server identity, such as `xpack.ssl.keystore` and -`xpack.ssl.certificate` settings. +those configured in the `xpack.security.transport.ssl.truststore` and +`xpack.security.transport.ssl.certificate_authorities` settings. It also +includes certificates that are used for configuring server identity, such as +`xpack.security.http.ssl.keystore` and +`xpack.security.http.ssl.certificate` settings. The list does not include certificates that are sourced from the default SSL context of the Java Runtime Environment (JRE), even if those certificates are in diff --git a/x-pack/docs/en/security/auditing/forwarding-logs.asciidoc b/x-pack/docs/en/security/auditing/forwarding-logs.asciidoc index f08945a6f8a0d..5bdc25238ad2e 100644 --- a/x-pack/docs/en/security/auditing/forwarding-logs.asciidoc +++ b/x-pack/docs/en/security/auditing/forwarding-logs.asciidoc @@ -46,8 +46,8 @@ to the `elasticsearch.yml` file: [source,yaml] ----------------------------------------------------------- xpack.security.audit.index.client.xpack.security.transport.ssl.enabled: true -xpack.security.audit.index.client.xpack.ssl.keystore.path: certs/remote-elastic-certificates.p12 -xpack.security.audit.index.client.xpack.ssl.truststore.path: certs/remote-elastic-certificates.p12 +xpack.security.audit.index.client.xpack.security.transport.ssl.keystore.path: certs/remote-elastic-certificates.p12 +xpack.security.audit.index.client.xpack.security.transport.ssl.truststore.path: certs/remote-elastic-certificates.p12 ----------------------------------------------------------- For more information about these settings, see @@ -61,9 +61,9 @@ For more information about these settings, see [source, yaml] -------------------------------------------------- xpack.security.audit.index.client.xpack.security.transport.ssl.enabled: true -xpack.security.audit.index.client.xpack.ssl.key: /home/es/config/audit-client.key -xpack.security.audit.index.client.xpack.ssl.certificate: /home/es/config/audit-client.crt -xpack.security.audit.index.client.xpack.ssl.certificate_authorities: [ "/home/es/config/remote-ca.crt" ] +xpack.security.audit.index.client.xpack.security.transport.ssl.key: /home/es/config/audit-client.key +xpack.security.audit.index.client.xpack.security.transport.ssl.certificate: /home/es/config/audit-client.crt +xpack.security.audit.index.client.xpack.security.transport.ssl.certificate_authorities: [ "/home/es/config/remote-ca.crt" ] -------------------------------------------------- For more information about these settings, see @@ -78,9 +78,9 @@ your {es} keystore: -- [source,shell] ----------------------------------------------------------- -bin/elasticsearch-keystore add xpack.security.audit.index.client.xpack.ssl.keystore.secure_password +bin/elasticsearch-keystore add xpack.security.audit.index.client.xpack.security.transport.ssl.keystore.secure_password -bin/elasticsearch-keystore add xpack.security.audit.index.client.xpack.ssl.truststore.secure_password +bin/elasticsearch-keystore add xpack.security.audit.index.client.xpack.security.transport.ssl.truststore.secure_password ----------------------------------------------------------- -- @@ -89,7 +89,7 @@ bin/elasticsearch-keystore add xpack.security.audit.index.client.xpack.ssl.trust -- [source,shell] ----------------------------------------------------------- -bin/elasticsearch-keystore add xpack.security.audit.index.client.xpack.ssl.secure_key_passphrase +bin/elasticsearch-keystore add xpack.security.audit.index.client.xpack.security.transport.ssl.secure_key_passphrase ----------------------------------------------------------- -- diff --git a/x-pack/docs/en/security/authentication/configuring-pki-realm.asciidoc b/x-pack/docs/en/security/authentication/configuring-pki-realm.asciidoc index 0554be2f87286..58144d0b23c1a 100644 --- a/x-pack/docs/en/security/authentication/configuring-pki-realm.asciidoc +++ b/x-pack/docs/en/security/authentication/configuring-pki-realm.asciidoc @@ -90,7 +90,7 @@ In particular this means: * The interface must _trust_ the certificate that is presented by the client by configuring either the `truststore` or `certificate_authorities` paths, or by setting `verification_mode` to `none`. See - <> for an explanation of this + <> for an explanation of this setting. * The _protocols_ supported by the interface must be compatible with those used by the client. diff --git a/x-pack/docs/en/security/ccs-clients-integrations/java.asciidoc b/x-pack/docs/en/security/ccs-clients-integrations/java.asciidoc index 8166f5cff9bcb..a19532bdb67c5 100644 --- a/x-pack/docs/en/security/ccs-clients-integrations/java.asciidoc +++ b/x-pack/docs/en/security/ccs-clients-integrations/java.asciidoc @@ -105,9 +105,10 @@ import org.elasticsearch.xpack.client.PreBuiltXPackTransportClient; TransportClient client = new PreBuiltXPackTransportClient(Settings.builder() .put("cluster.name", "myClusterName") .put("xpack.security.user", "transport_client_user:x-pack-test-password") - .put("xpack.ssl.key", "/path/to/client.key") - .put("xpack.ssl.certificate", "/path/to/client.crt") - .put("xpack.ssl.certificate_authorities", "/path/to/ca.crt") + .put("xpack.security.transport.ssl.enabled", true) + .put("xpack.security.transport.ssl.key", "/path/to/client.key") + .put("xpack.security.transport.ssl.certificate", "/path/to/client.crt") + .put("xpack.security.transport.ssl.certificate_authorities", "/path/to/ca.crt") ... .build()); -------------------------------------------------------------------------------------------------- @@ -125,9 +126,10 @@ import org.elasticsearch.xpack.client.PreBuiltXPackTransportClient; TransportClient client = new PreBuiltXPackTransportClient(Settings.builder() .put("cluster.name", "myClusterName") .put("xpack.security.user", "transport_client_user:x-pack-test-password") - .put("xpack.ssl.key", "/path/to/client.key") - .put("xpack.ssl.certificate", "/path/to/client.crt") - .put("xpack.ssl.certificate_authorities", "/path/to/ca.crt") + .put("xpack.security.transport.ssl.enabled", true) + .put("xpack.security.transport.ssl.key", "/path/to/client.key") + .put("xpack.security.transport.ssl.certificate", "/path/to/client.crt") + .put("xpack.security.transport.ssl.certificate_authorities", "/path/to/ca.crt") .put("xpack.security.transport.ssl.enabled", "true") ... .build()) @@ -155,7 +157,7 @@ import org.elasticsearch.xpack.client.PreBuiltXPackTransportClient; TransportClient client = new PreBuiltXPackTransportClient(Settings.builder() .put("cluster.name", "myClusterName") .put("xpack.security.user", "test_user:x-pack-test-password") - .put("xpack.ssl.certificate_authorities", "/path/to/ca.crt") + .put("xpack.security.transport.ssl.certificate_authorities", "/path/to/ca.crt") .put("xpack.security.transport.ssl.enabled", "true") ... .build()) @@ -164,7 +166,7 @@ TransportClient client = new PreBuiltXPackTransportClient(Settings.builder() ------------------------------------------------------------------------------------------------------ NOTE: If you are using a public CA that is already trusted by the Java runtime, - you do not need to set the `xpack.ssl.certificate_authorities`. + you do not need to set the `xpack.security.transport.ssl.certificate_authorities`. [float] [[connecting-anonymously]] diff --git a/x-pack/docs/en/security/fips-140-compliance.asciidoc b/x-pack/docs/en/security/fips-140-compliance.asciidoc index 0216e61784cdb..6bc9be512db4e 100644 --- a/x-pack/docs/en/security/fips-140-compliance.asciidoc +++ b/x-pack/docs/en/security/fips-140-compliance.asciidoc @@ -50,12 +50,12 @@ and able to run {es} successfully in a FIPS 140-2 enabled JVM. ==== TLS SSLv2 and SSLv3 are not allowed by FIPS 140-2, so `SSLv2Hello` and `SSLv3` cannot -be used for <> +be used for <> NOTE: The use of TLS ciphers is mainly governed by the relevant crypto module (the FIPS Approved Security Provider that your JVM uses). All the ciphers that are configured by default in {es} are FIPS 140-2 compliant and as such can be -used in a FIPS 140-2 JVM. (see <>) +used in a FIPS 140-2 JVM. (see <>) [float] ==== TLS Keystores and keys @@ -71,7 +71,7 @@ options, and for trust material you can use `*.certificate_authorities`. FIPS 140-2 compliance dictates that the length of the public keys used for TLS must correspond to the strength of the symmetric key algorithm in use in TLS. -Depending on the value of <> that +Depending on the value of <> that you select to use, the TLS keys must have corresponding length according to the following table: diff --git a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/XPackSettings.java b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/XPackSettings.java index 13cc4c121daf6..22bc6f4b29482 100644 --- a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/XPackSettings.java +++ b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/XPackSettings.java @@ -159,10 +159,6 @@ private XPackSettings() { public static final SSLClientAuth HTTP_CLIENT_AUTH_DEFAULT = SSLClientAuth.NONE; public static final VerificationMode VERIFICATION_MODE_DEFAULT = VerificationMode.FULL; - // global settings that apply to everything! - public static final String GLOBAL_SSL_PREFIX = "xpack.ssl."; - private static final SSLConfigurationSettings GLOBAL_SSL = SSLConfigurationSettings.withPrefix(GLOBAL_SSL_PREFIX); - // http specific settings public static final String HTTP_SSL_PREFIX = SecurityField.setting("http.ssl."); private static final SSLConfigurationSettings HTTP_SSL = SSLConfigurationSettings.withPrefix(HTTP_SSL_PREFIX); @@ -174,7 +170,6 @@ private XPackSettings() { /** Returns all settings created in {@link XPackSettings}. */ public static List> getAllSettings() { ArrayList> settings = new ArrayList<>(); - settings.addAll(GLOBAL_SSL.getAllSettings()); settings.addAll(HTTP_SSL.getAllSettings()); settings.addAll(TRANSPORT_SSL.getAllSettings()); settings.add(SECURITY_ENABLED); diff --git a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/security/transport/netty4/SecurityNetty4Transport.java b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/security/transport/netty4/SecurityNetty4Transport.java index 4ed4246597bb5..9c32dd1e80b34 100644 --- a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/security/transport/netty4/SecurityNetty4Transport.java +++ b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/security/transport/netty4/SecurityNetty4Transport.java @@ -83,13 +83,22 @@ public static Map getTransportProfileConfigurations(Se Set profileNames = settings.getGroups("transport.profiles.", true).keySet(); Map profileConfiguration = new HashMap<>(profileNames.size() + 1); for (String profileName : profileNames) { + if (profileName.equals(TransportSettings.DEFAULT_PROFILE)) { + // don't attempt to parse ssl settings from the profile; + // profiles need to be killed with fire + if (settings.getByPrefix("transport.profiles.default.xpack.security.ssl.").isEmpty()) { + continue; + } else { + throw new IllegalArgumentException("SSL settings should not be configured for the default profile. " + + "Use the [xpack.security.transport.ssl] settings instead."); + } + } SSLConfiguration configuration = sslService.getSSLConfiguration("transport.profiles." + profileName + "." + setting("ssl")); profileConfiguration.put(profileName, configuration); } - if (profileConfiguration.containsKey(TransportSettings.DEFAULT_PROFILE) == false) { - profileConfiguration.put(TransportSettings.DEFAULT_PROFILE, defaultConfiguration); - } + assert profileConfiguration.containsKey(TransportSettings.DEFAULT_PROFILE) == false; + profileConfiguration.put(TransportSettings.DEFAULT_PROFILE, defaultConfiguration); return profileConfiguration; } diff --git a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/ssl/DefaultJDKTrustConfig.java b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/ssl/DefaultJDKTrustConfig.java index 4b5055a9e86fd..25076937f5a96 100644 --- a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/ssl/DefaultJDKTrustConfig.java +++ b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/ssl/DefaultJDKTrustConfig.java @@ -25,6 +25,7 @@ import java.util.Collection; import java.util.Collections; import java.util.List; +import java.util.Objects; /** * This class represents a trust configuration that corresponds to the default trusted certificates of the JDK @@ -71,12 +72,15 @@ public String toString() { @Override public boolean equals(Object o) { - return o == this; + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + DefaultJDKTrustConfig that = (DefaultJDKTrustConfig) o; + return Objects.equals(trustStorePassword, that.trustStorePassword); } @Override public int hashCode() { - return System.identityHashCode(this); + return Objects.hash(trustStorePassword); } /** diff --git a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/ssl/SSLConfiguration.java b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/ssl/SSLConfiguration.java index 0862cb929ef98..5857f9a6edd11 100644 --- a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/ssl/SSLConfiguration.java +++ b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/ssl/SSLConfiguration.java @@ -13,16 +13,11 @@ import org.elasticsearch.xpack.core.XPackSettings; import org.elasticsearch.xpack.core.ssl.cert.CertificateInfo; -import javax.net.ssl.KeyManagerFactory; -import javax.net.ssl.TrustManagerFactory; - import java.io.IOException; import java.nio.file.Path; import java.security.GeneralSecurityException; -import java.security.KeyStore; import java.util.ArrayList; import java.util.List; -import java.util.Objects; import static org.elasticsearch.xpack.core.ssl.SSLConfigurationSettings.getKeyStoreType; @@ -51,32 +46,14 @@ public final class SSLConfiguration { * @param settings the SSL specific settings; only the settings under a *.ssl. prefix */ SSLConfiguration(Settings settings) { - this.keyConfig = createKeyConfig(settings, (SSLConfiguration) null); - this.trustConfig = createTrustConfig(settings, keyConfig, null); + this.keyConfig = createKeyConfig(settings); + this.trustConfig = createTrustConfig(settings, keyConfig); this.ciphers = getListOrDefault(SETTINGS_PARSER.ciphers, settings, XPackSettings.DEFAULT_CIPHERS); this.supportedProtocols = getListOrDefault(SETTINGS_PARSER.supportedProtocols, settings, XPackSettings.DEFAULT_SUPPORTED_PROTOCOLS); this.sslClientAuth = SETTINGS_PARSER.clientAuth.get(settings).orElse(XPackSettings.CLIENT_AUTH_DEFAULT); this.verificationMode = SETTINGS_PARSER.verificationMode.get(settings).orElse(XPackSettings.VERIFICATION_MODE_DEFAULT); } - /** - * Creates a new SSLConfiguration from the given settings and global/default SSLConfiguration. If the settings do not contain a value - * for a given aspect, the value from the global configuration will be used. - * - * @param settings the SSL specific settings; only the settings under a *.ssl. prefix - * @param globalSSLConfiguration the default configuration that is used as a fallback - */ - SSLConfiguration(Settings settings, SSLConfiguration globalSSLConfiguration) { - Objects.requireNonNull(globalSSLConfiguration); - this.keyConfig = createKeyConfig(settings, globalSSLConfiguration); - this.trustConfig = createTrustConfig(settings, keyConfig, globalSSLConfiguration); - this.ciphers = getListOrDefault(SETTINGS_PARSER.ciphers, settings, globalSSLConfiguration.cipherSuites()); - this.supportedProtocols = getListOrDefault(SETTINGS_PARSER.supportedProtocols, settings, - globalSSLConfiguration.supportedProtocols()); - this.sslClientAuth = SETTINGS_PARSER.clientAuth.get(settings).orElse(globalSSLConfiguration.sslClientAuth()); - this.verificationMode = SETTINGS_PARSER.verificationMode.get(settings).orElse(globalSSLConfiguration.verificationMode()); - } - /** * The configuration for the key, if any, that will be used as part of this ssl configuration */ @@ -183,46 +160,28 @@ public int hashCode() { return result; } - private static KeyConfig createKeyConfig(Settings settings, SSLConfiguration global) { + private static KeyConfig createKeyConfig(Settings settings) { final String trustStoreAlgorithm = SETTINGS_PARSER.truststoreAlgorithm.get(settings); final KeyConfig config = CertParsingUtils.createKeyConfig(SETTINGS_PARSER.x509KeyPair, settings, trustStoreAlgorithm); - if (config != null) { - return config; - } - if (global != null) { - return global.keyConfig(); - } - if (System.getProperty("javax.net.ssl.keyStore") != null && System.getProperty("javax.net.ssl.keyStore").equals("NONE") == false) { - // TODO: we should not support loading a keystore from sysprops... - try (SecureString keystorePassword = new SecureString(System.getProperty("javax.net.ssl.keyStorePassword", ""))) { - return new StoreKeyConfig(System.getProperty("javax.net.ssl.keyStore"), KeyStore.getDefaultType(), keystorePassword, - keystorePassword, System.getProperty("ssl.KeyManagerFactory.algorithm", KeyManagerFactory.getDefaultAlgorithm()), - System.getProperty("ssl.TrustManagerFactory.algorithm", TrustManagerFactory.getDefaultAlgorithm())); - } - } - return KeyConfig.NONE; + return config == null ? KeyConfig.NONE : config; } - private static TrustConfig createTrustConfig(Settings settings, KeyConfig keyConfig, SSLConfiguration global) { - final TrustConfig trustConfig = createCertChainTrustConfig(settings, keyConfig, global); + private static TrustConfig createTrustConfig(Settings settings, KeyConfig keyConfig) { + final TrustConfig trustConfig = createCertChainTrustConfig(settings, keyConfig); return SETTINGS_PARSER.trustRestrictionsPath.get(settings) .map(path -> (TrustConfig) new RestrictedTrustConfig(path, trustConfig)) .orElse(trustConfig); } - private static TrustConfig createCertChainTrustConfig(Settings settings, KeyConfig keyConfig, SSLConfiguration global) { + private static TrustConfig createCertChainTrustConfig(Settings settings, KeyConfig keyConfig) { String trustStorePath = SETTINGS_PARSER.truststorePath.get(settings).orElse(null); String trustStoreType = getKeyStoreType(SETTINGS_PARSER.truststoreType, settings, trustStorePath); List caPaths = getListOrNull(SETTINGS_PARSER.caPaths, settings); if (trustStorePath != null && caPaths != null) { throw new IllegalArgumentException("you cannot specify a truststore and ca files"); } - VerificationMode verificationMode = SETTINGS_PARSER.verificationMode.get(settings).orElseGet(() -> { - if (global != null) { - return global.verificationMode(); - } - return XPackSettings.VERIFICATION_MODE_DEFAULT; - }); + + VerificationMode verificationMode = SETTINGS_PARSER.verificationMode.get(settings).orElse(XPackSettings.VERIFICATION_MODE_DEFAULT); if (verificationMode.isCertificateVerificationEnabled() == false) { return TrustAllConfig.INSTANCE; } else if (caPaths != null) { @@ -231,14 +190,6 @@ private static TrustConfig createCertChainTrustConfig(Settings settings, KeyConf String trustStoreAlgorithm = SETTINGS_PARSER.truststoreAlgorithm.get(settings); SecureString trustStorePassword = SETTINGS_PARSER.truststorePassword.get(settings); return new StoreTrustConfig(trustStorePath, trustStoreType, trustStorePassword, trustStoreAlgorithm); - } else if (global == null && System.getProperty("javax.net.ssl.trustStore") != null - && System.getProperty("javax.net.ssl.trustStore").equals("NONE") == false) { - try (SecureString truststorePassword = new SecureString(System.getProperty("javax.net.ssl.trustStorePassword", ""))) { - return new StoreTrustConfig(System.getProperty("javax.net.ssl.trustStore"), KeyStore.getDefaultType(), truststorePassword, - System.getProperty("ssl.TrustManagerFactory.algorithm", TrustManagerFactory.getDefaultAlgorithm())); - } - } else if (global != null && keyConfig == global.keyConfig()) { - return global.trustConfig(); } else if (keyConfig != KeyConfig.NONE) { return DefaultJDKTrustConfig.merge(keyConfig, getDefaultTrustStorePassword(settings)); } else { diff --git a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/ssl/SSLService.java b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/ssl/SSLService.java index 900d9468e2aa8..e832de629359a 100644 --- a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/ssl/SSLService.java +++ b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/ssl/SSLService.java @@ -31,13 +31,14 @@ import javax.net.ssl.TrustManagerFactory; import javax.net.ssl.X509ExtendedKeyManager; import javax.net.ssl.X509ExtendedTrustManager; + import java.io.IOException; import java.net.InetAddress; import java.net.Socket; import java.security.GeneralSecurityException; import java.security.KeyManagementException; -import java.security.KeyStore; import java.security.NoSuchAlgorithmException; +import java.security.KeyStore; import java.util.ArrayList; import java.util.Arrays; import java.util.Collection; @@ -81,8 +82,6 @@ public class SSLService { * always maps to the same {@link SSLContextHolder}, even if it is being used within a different context-name. */ private final Map sslContexts; - - private final SSLConfiguration globalSSLConfiguration; private final SetOnce transportSSLConfiguration = new SetOnce<>(); private final Environment env; @@ -93,16 +92,14 @@ public class SSLService { public SSLService(Settings settings, Environment environment) { this.settings = settings; this.env = environment; - this.globalSSLConfiguration = new SSLConfiguration(settings.getByPrefix(XPackSettings.GLOBAL_SSL_PREFIX)); this.sslConfigurations = new HashMap<>(); this.sslContexts = loadSSLConfigurations(); } - private SSLService(Settings settings, Environment environment, SSLConfiguration globalSSLConfiguration, - Map sslConfigurations, Map sslContexts) { + private SSLService(Settings settings, Environment environment, Map sslConfigurations, + Map sslContexts) { this.settings = settings; this.env = environment; - this.globalSSLConfiguration = globalSSLConfiguration; this.sslConfigurations = sslConfigurations; this.sslContexts = sslContexts; } @@ -113,7 +110,7 @@ private SSLService(Settings settings, Environment environment, SSLConfiguration * have been created during initialization */ public SSLService createDynamicSSLService() { - return new SSLService(settings, env, globalSSLConfiguration, sslConfigurations, sslContexts) { + return new SSLService(settings, env, sslConfigurations, sslContexts) { @Override Map loadSSLConfigurations() { @@ -265,13 +262,6 @@ public boolean isSSLClientAuthEnabled(SSLConfiguration sslConfiguration) { return sslConfiguration.sslClientAuth().enabled(); } - /** - * Returns the {@link SSLContext} for the global configuration. Mainly used for testing - */ - public SSLContext sslContext() { - return sslContextHolder(globalSSLConfiguration).sslContext(); - } - /** * Returns the {@link SSLContext} for the configuration. Mainly used for testing */ @@ -297,13 +287,10 @@ SSLContextHolder sslContextHolder(SSLConfiguration sslConfiguration) { * Returns the existing {@link SSLConfiguration} for the given settings * * @param settings the settings for the ssl configuration - * @return the ssl configuration for the provided settings. If the settings are empty, the global configuration is returned + * @return the ssl configuration for the provided settings */ - SSLConfiguration sslConfiguration(Settings settings) { - if (settings.isEmpty()) { - return globalSSLConfiguration; - } - return new SSLConfiguration(settings, globalSSLConfiguration); + public SSLConfiguration sslConfiguration(Settings settings) { + return new SSLConfiguration(settings); } public Set getTransportProfileContextNames() { @@ -403,8 +390,6 @@ private SSLContextHolder createSslContext(X509ExtendedKeyManager keyManager, X50 */ Map loadSSLConfigurations() { Map sslContextHolders = new HashMap<>(); - sslContextHolders.put(globalSSLConfiguration, createSslContext(globalSSLConfiguration)); - this.sslConfigurations.put("xpack.ssl", globalSSLConfiguration); Map sslSettingsMap = new HashMap<>(); sslSettingsMap.put(XPackSettings.HTTP_SSL_PREFIX, getHttpTransportSSLSettings(settings)); @@ -413,23 +398,19 @@ Map loadSSLConfigurations() { sslSettingsMap.putAll(getMonitoringExporterSettings(settings)); sslSettingsMap.forEach((key, sslSettings) -> { - if (sslSettings.isEmpty()) { - storeSslConfiguration(key, globalSSLConfiguration); - } else { - final SSLConfiguration configuration = new SSLConfiguration(sslSettings, globalSSLConfiguration); - storeSslConfiguration(key, configuration); - sslContextHolders.computeIfAbsent(configuration, this::createSslContext); - } + final SSLConfiguration configuration = new SSLConfiguration(sslSettings); + storeSslConfiguration(key, configuration); + sslContextHolders.computeIfAbsent(configuration, this::createSslContext); }); final Settings transportSSLSettings = settings.getByPrefix(XPackSettings.TRANSPORT_SSL_PREFIX); - final SSLConfiguration transportSSLConfiguration = new SSLConfiguration(transportSSLSettings, globalSSLConfiguration); + final SSLConfiguration transportSSLConfiguration = new SSLConfiguration(transportSSLSettings); this.transportSSLConfiguration.set(transportSSLConfiguration); storeSslConfiguration(XPackSettings.TRANSPORT_SSL_PREFIX, transportSSLConfiguration); Map profileSettings = getTransportProfileSSLSettings(settings); sslContextHolders.computeIfAbsent(transportSSLConfiguration, this::createSslContext); profileSettings.forEach((key, profileSetting) -> { - final SSLConfiguration configuration = new SSLConfiguration(profileSetting, transportSSLConfiguration); + final SSLConfiguration configuration = new SSLConfiguration(profileSetting); storeSslConfiguration(key, configuration); sslContextHolders.computeIfAbsent(configuration, this::createSslContext); }); diff --git a/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/ssl/SSLConfigurationReloaderTests.java b/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/ssl/SSLConfigurationReloaderTests.java index cb9996ac90db5..318d8e4150a1d 100644 --- a/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/ssl/SSLConfigurationReloaderTests.java +++ b/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/ssl/SSLConfigurationReloaderTests.java @@ -71,7 +71,7 @@ public void setup() { } @After - public void cleanup() throws Exception { + public void cleanup() { if (threadPool != null) { terminate(threadPool); } @@ -88,10 +88,10 @@ public void testReloadingKeyStore() throws Exception { Files.copy(getDataPath("/org/elasticsearch/xpack/security/transport/ssl/certs/simple/testnode.jks"), keystorePath); Files.copy(getDataPath("/org/elasticsearch/xpack/security/transport/ssl/certs/simple/testnode_updated.jks"), updatedKeystorePath); MockSecureSettings secureSettings = new MockSecureSettings(); - secureSettings.setString("xpack.ssl.keystore.secure_password", "testnode"); + secureSettings.setString("xpack.security.transport.ssl.keystore.secure_password", "testnode"); final Settings settings = Settings.builder() .put("path.home", createTempDir()) - .put("xpack.ssl.keystore.path", keystorePath) + .put("xpack.security.transport.ssl.keystore.path", keystorePath) .setSecureSettings(secureSettings) .build(); final Environment env = randomBoolean() ? null : TestEnvironment.newEnvironment(settings); @@ -144,11 +144,12 @@ public void testPEMKeyConfigReloading() throws Exception { Files.copy(getDataPath("/org/elasticsearch/xpack/security/transport/ssl/certs/simple/testnode_updated.crt"), updatedCertPath); Files.copy(getDataPath("/org/elasticsearch/xpack/security/transport/ssl/certs/simple/testnode.crt"), certPath); MockSecureSettings secureSettings = new MockSecureSettings(); - secureSettings.setString("xpack.ssl.secure_key_passphrase", "testnode"); + secureSettings.setString("xpack.security.transport.ssl.secure_key_passphrase", "testnode"); final Settings settings = Settings.builder() .put("path.home", createTempDir()) - .put("xpack.ssl.key", keyPath) - .put("xpack.ssl.certificate", certPath) + .put("xpack.security.transport.ssl.key", keyPath) + .put("xpack.security.transport.ssl.certificate", certPath) + .putList("xpack.security.transport.ssl.certificate_authorities", certPath.toString()) .setSecureSettings(secureSettings) .build(); final Environment env = randomBoolean() ? null : @@ -202,9 +203,9 @@ public void testReloadingTrustStore() throws Exception { Files.copy(getDataPath("/org/elasticsearch/xpack/security/transport/ssl/certs/simple/testnode_updated.jks"), updatedTruststorePath); MockSecureSettings secureSettings = new MockSecureSettings(); - secureSettings.setString("xpack.ssl.truststore.secure_password", "testnode"); + secureSettings.setString("xpack.security.transport.ssl.truststore.secure_password", "testnode"); Settings settings = Settings.builder() - .put("xpack.ssl.truststore.path", trustStorePath) + .put("xpack.security.transport.ssl.truststore.path", trustStorePath) .put("path.home", createTempDir()) .setSecureSettings(secureSettings) .build(); @@ -254,7 +255,7 @@ public void testReloadingPEMTrustConfig() throws Exception { Files.copy(getDataPath("/org/elasticsearch/xpack/security/transport/ssl/certs/simple/testnode.pem"), serverKeyPath); Files.copy(getDataPath("/org/elasticsearch/xpack/security/transport/ssl/certs/simple/testnode_updated.crt"), updatedCert); Settings settings = Settings.builder() - .put("xpack.ssl.certificate_authorities", serverCertPath) + .putList("xpack.security.transport.ssl.certificate_authorities", serverCertPath.toString()) .put("path.home", createTempDir()) .build(); Environment env = randomBoolean() ? null : TestEnvironment.newEnvironment(settings); @@ -300,15 +301,15 @@ public void testReloadingKeyStoreException() throws Exception { Path keystorePath = tempDir.resolve("testnode.jks"); Files.copy(getDataPath("/org/elasticsearch/xpack/security/transport/ssl/certs/simple/testnode.jks"), keystorePath); MockSecureSettings secureSettings = new MockSecureSettings(); - secureSettings.setString("xpack.ssl.keystore.secure_password", "testnode"); + secureSettings.setString("xpack.security.transport.ssl.keystore.secure_password", "testnode"); Settings settings = Settings.builder() - .put("xpack.ssl.keystore.path", keystorePath) + .put("xpack.security.transport.ssl.keystore.path", keystorePath) .setSecureSettings(secureSettings) .put("path.home", createTempDir()) .build(); Environment env = randomBoolean() ? null : TestEnvironment.newEnvironment(settings); final SSLService sslService = new SSLService(settings, env); - final SSLConfiguration config = sslService.getSSLConfiguration("xpack.ssl"); + final SSLConfiguration config = sslService.getSSLConfiguration("xpack.security.transport.ssl."); new SSLConfigurationReloader(env, sslService, resourceWatcherService) { @Override void reloadSSLContext(SSLConfiguration configuration) { @@ -339,17 +340,17 @@ public void testReloadingPEMKeyConfigException() throws Exception { Files.copy(getDataPath("/org/elasticsearch/xpack/security/transport/ssl/certs/simple/testnode.crt"), certPath); Files.copy(getDataPath("/org/elasticsearch/xpack/security/transport/ssl/certs/simple/testclient.crt"), clientCertPath); MockSecureSettings secureSettings = new MockSecureSettings(); - secureSettings.setString("xpack.ssl.secure_key_passphrase", "testnode"); + secureSettings.setString("xpack.security.transport.ssl.secure_key_passphrase", "testnode"); Settings settings = Settings.builder() - .put("xpack.ssl.key", keyPath) - .put("xpack.ssl.certificate", certPath) - .putList("xpack.ssl.certificate_authorities", certPath.toString(), clientCertPath.toString()) + .put("xpack.security.transport.ssl.key", keyPath) + .put("xpack.security.transport.ssl.certificate", certPath) + .putList("xpack.security.transport.ssl.certificate_authorities", certPath.toString(), clientCertPath.toString()) .put("path.home", createTempDir()) .setSecureSettings(secureSettings) .build(); Environment env = randomBoolean() ? null : TestEnvironment.newEnvironment(settings); final SSLService sslService = new SSLService(settings, env); - final SSLConfiguration config = sslService.getSSLConfiguration("xpack.ssl"); + final SSLConfiguration config = sslService.getSSLConfiguration("xpack.security.transport.ssl."); new SSLConfigurationReloader(env, sslService, resourceWatcherService) { @Override void reloadSSLContext(SSLConfiguration configuration) { @@ -376,15 +377,15 @@ public void testTrustStoreReloadException() throws Exception { Path trustStorePath = tempDir.resolve("testnode.jks"); Files.copy(getDataPath("/org/elasticsearch/xpack/security/transport/ssl/certs/simple/testnode.jks"), trustStorePath); MockSecureSettings secureSettings = new MockSecureSettings(); - secureSettings.setString("xpack.ssl.truststore.secure_password", "testnode"); + secureSettings.setString("xpack.security.transport.ssl.truststore.secure_password", "testnode"); Settings settings = Settings.builder() - .put("xpack.ssl.truststore.path", trustStorePath) + .put("xpack.security.transport.ssl.truststore.path", trustStorePath) .put("path.home", createTempDir()) .setSecureSettings(secureSettings) .build(); Environment env = randomBoolean() ? null : TestEnvironment.newEnvironment(settings); final SSLService sslService = new SSLService(settings, env); - final SSLConfiguration config = sslService.getSSLConfiguration("xpack.ssl"); + final SSLConfiguration config = sslService.getSSLConfiguration("xpack.security.transport.ssl."); new SSLConfigurationReloader(env, sslService, resourceWatcherService) { @Override void reloadSSLContext(SSLConfiguration configuration) { @@ -411,12 +412,12 @@ public void testPEMTrustReloadException() throws Exception { Path clientCertPath = tempDir.resolve("testclient.crt"); Files.copy(getDataPath("/org/elasticsearch/xpack/security/transport/ssl/certs/simple/testclient.crt"), clientCertPath); Settings settings = Settings.builder() - .putList("xpack.ssl.certificate_authorities", clientCertPath.toString()) + .putList("xpack.security.transport.ssl.certificate_authorities", clientCertPath.toString()) .put("path.home", createTempDir()) .build(); Environment env = randomBoolean() ? null : TestEnvironment.newEnvironment(settings); final SSLService sslService = new SSLService(settings, env); - final SSLConfiguration config = sslService.getSSLConfiguration("xpack.ssl"); + final SSLConfiguration config = sslService.sslConfiguration(settings.getByPrefix("xpack.security.transport.ssl.")); new SSLConfigurationReloader(env, sslService, resourceWatcherService) { @Override void reloadSSLContext(SSLConfiguration configuration) { @@ -442,7 +443,7 @@ private void validateSSLConfigurationIsReloaded(Settings settings, Environment e Runnable modificationFunction, Consumer postChecks) throws Exception { final CountDownLatch reloadLatch = new CountDownLatch(1); final SSLService sslService = new SSLService(settings, env); - final SSLConfiguration config = sslService.getSSLConfiguration("xpack.ssl"); + final SSLConfiguration config = sslService.getSSLConfiguration("xpack.security.transport.ssl"); new SSLConfigurationReloader(env, sslService, resourceWatcherService) { @Override void reloadSSLContext(SSLConfiguration configuration) { diff --git a/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/ssl/SSLConfigurationSettingsTests.java b/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/ssl/SSLConfigurationSettingsTests.java index 4b045951d6658..072f7d0d57da7 100644 --- a/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/ssl/SSLConfigurationSettingsTests.java +++ b/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/ssl/SSLConfigurationSettingsTests.java @@ -21,7 +21,7 @@ public void testParseCipherSettingsWithoutPrefix() { final SSLConfigurationSettings ssl = SSLConfigurationSettings.withoutPrefix(); assertThat(ssl.ciphers.match("cipher_suites"), is(true)); assertThat(ssl.ciphers.match("ssl.cipher_suites"), is(false)); - assertThat(ssl.ciphers.match("xpack.ssl.cipher_suites"), is(false)); + assertThat(ssl.ciphers.match("xpack.transport.security.ssl.cipher_suites"), is(false)); final Settings settings = Settings.builder() .put("cipher_suites.0", "TLS_ECDHE_RSA_WITH_AES_128_CBC_SHA256") diff --git a/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/ssl/SSLConfigurationTests.java b/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/ssl/SSLConfigurationTests.java index e0b70c09add0b..74ae2ae55c126 100644 --- a/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/ssl/SSLConfigurationTests.java +++ b/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/ssl/SSLConfigurationTests.java @@ -35,10 +35,6 @@ public void testThatSSLConfigurationHasCorrectDefaults() { assertThat(globalConfig.keyConfig(), sameInstance(KeyConfig.NONE)); assertThat(globalConfig.trustConfig(), is(not((globalConfig.keyConfig())))); assertThat(globalConfig.trustConfig(), instanceOf(DefaultJDKTrustConfig.class)); - - SSLConfiguration scopedConfig = new SSLConfiguration(Settings.EMPTY, globalConfig); - assertThat(scopedConfig.keyConfig(), sameInstance(globalConfig.keyConfig())); - assertThat(scopedConfig.trustConfig(), sameInstance(globalConfig.trustConfig())); } public void testThatOnlyKeystoreInSettingsSetsTruststoreSettings() { @@ -50,22 +46,17 @@ public void testThatOnlyKeystoreInSettingsSetsTruststoreSettings() { .setSecureSettings(secureSettings) .build(); // Pass settings in as component settings - SSLConfiguration globalSettings = new SSLConfiguration(settings); - SSLConfiguration scopedSettings = new SSLConfiguration(settings, globalSettings); - SSLConfiguration scopedEmptyGlobalSettings = - new SSLConfiguration(settings, new SSLConfiguration(Settings.EMPTY)); - for (SSLConfiguration sslConfiguration : Arrays.asList(globalSettings, scopedSettings, scopedEmptyGlobalSettings)) { - assertThat(sslConfiguration.keyConfig(), instanceOf(StoreKeyConfig.class)); - StoreKeyConfig ksKeyInfo = (StoreKeyConfig) sslConfiguration.keyConfig(); + SSLConfiguration sslConfiguration = new SSLConfiguration(settings); + assertThat(sslConfiguration.keyConfig(), instanceOf(StoreKeyConfig.class)); + StoreKeyConfig ksKeyInfo = (StoreKeyConfig) sslConfiguration.keyConfig(); - assertThat(ksKeyInfo.keyStorePath, is(equalTo(path))); - assertThat(ksKeyInfo.keyStorePassword, is(equalTo("testnode"))); - assertThat(ksKeyInfo.keyStoreType, is(equalTo("jks"))); - assertThat(ksKeyInfo.keyPassword, is(equalTo(ksKeyInfo.keyStorePassword))); - assertThat(ksKeyInfo.keyStoreAlgorithm, is(KeyManagerFactory.getDefaultAlgorithm())); - assertThat(sslConfiguration.trustConfig(), is(instanceOf(CombiningTrustConfig.class))); - assertCombiningTrustConfigContainsCorrectIssuers(sslConfiguration); - } + assertThat(ksKeyInfo.keyStorePath, is(equalTo(path))); + assertThat(ksKeyInfo.keyStorePassword, is(equalTo("testnode"))); + assertThat(ksKeyInfo.keyStoreType, is(equalTo("jks"))); + assertThat(ksKeyInfo.keyPassword, is(equalTo(ksKeyInfo.keyStorePassword))); + assertThat(ksKeyInfo.keyStoreAlgorithm, is(KeyManagerFactory.getDefaultAlgorithm())); + assertThat(sslConfiguration.trustConfig(), is(instanceOf(CombiningTrustConfig.class))); + assertCombiningTrustConfigContainsCorrectIssuers(sslConfiguration); } public void testKeystorePassword() { @@ -188,44 +179,13 @@ public void testExplicitKeystoreType() { assertThat(ksKeyInfo.keyStoreType, is(equalTo(type))); } - public void testThatProfileSettingsOverrideServiceSettings() { - MockSecureSettings profileSecureSettings = new MockSecureSettings(); - profileSecureSettings.setString("keystore.secure_password", "password"); - profileSecureSettings.setString("keystore.secure_key_password", "key"); - profileSecureSettings.setString("truststore.secure_password", "password for trust"); - Settings profileSettings = Settings.builder() - .put("keystore.path", "path") - .put("keystore.algorithm", "algo") - .put("truststore.path", "trust path") - .put("truststore.algorithm", "trusted") - .setSecureSettings(profileSecureSettings) - .build(); - - MockSecureSettings serviceSecureSettings = new MockSecureSettings(); - serviceSecureSettings.setString("xpack.ssl.keystore.secure_password", "comp password"); - serviceSecureSettings.setString("xpack.ssl.keystore.secure_key_password", "comp key"); - serviceSecureSettings.setString("xpack.ssl.truststore.secure_password", "comp password for trust"); - Settings serviceSettings = Settings.builder() - .put("xpack.ssl.keystore.path", "comp path") - .put("xpack.ssl.keystore.algorithm", "comp algo") - .put("xpack.ssl.truststore.path", "comp trust path") - .put("xpack.ssl.truststore.algorithm", "comp trusted") - .setSecureSettings(serviceSecureSettings) - .build(); - - SSLConfiguration globalSettings = new SSLConfiguration(serviceSettings); - SSLConfiguration sslConfiguration = new SSLConfiguration(profileSettings, globalSettings); - assertThat(sslConfiguration.keyConfig(), instanceOf(StoreKeyConfig.class)); - StoreKeyConfig ksKeyInfo = (StoreKeyConfig) sslConfiguration.keyConfig(); - assertThat(ksKeyInfo.keyStorePath, is(equalTo("path"))); - assertThat(ksKeyInfo.keyStorePassword, is(equalTo("password"))); - assertThat(ksKeyInfo.keyPassword, is(equalTo("key"))); - assertThat(ksKeyInfo.keyStoreAlgorithm, is(equalTo("algo"))); - assertThat(sslConfiguration.trustConfig(), instanceOf(StoreTrustConfig.class)); - StoreTrustConfig ksTrustInfo = (StoreTrustConfig) sslConfiguration.trustConfig(); - assertThat(ksTrustInfo.trustStorePath, is(equalTo("trust path"))); - assertThat(ksTrustInfo.trustStorePassword, is(equalTo("password for trust"))); - assertThat(ksTrustInfo.trustStoreAlgorithm, is(equalTo("trusted"))); + public void testThatEmptySettingsAreEqual() { + SSLConfiguration sslConfiguration = new SSLConfiguration(Settings.EMPTY); + SSLConfiguration sslConfiguration1 = new SSLConfiguration(Settings.EMPTY); + assertThat(sslConfiguration.equals(sslConfiguration1), is(equalTo(true))); + assertThat(sslConfiguration1.equals(sslConfiguration), is(equalTo(true))); + assertThat(sslConfiguration.equals(sslConfiguration), is(equalTo(true))); + assertThat(sslConfiguration1.equals(sslConfiguration1), is(equalTo(true))); } public void testThatSettingsWithDifferentKeystoresAreNotEqual() { @@ -254,6 +214,12 @@ public void testThatSettingsWithDifferentTruststoresAreNotEqual() { assertThat(sslConfiguration1.equals(sslConfiguration1), is(equalTo(true))); } + public void testThatEmptySettingsHaveSameHashCode() { + SSLConfiguration sslConfiguration = new SSLConfiguration(Settings.EMPTY); + SSLConfiguration sslConfiguration1 = new SSLConfiguration(Settings.EMPTY); + assertThat(sslConfiguration.hashCode(), is(equalTo(sslConfiguration1.hashCode()))); + } + public void testThatSettingsWithDifferentKeystoresHaveDifferentHashCode() { SSLConfiguration sslConfiguration = new SSLConfiguration(Settings.builder() .put("keystore.path", "path") diff --git a/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/ssl/SSLServiceTests.java b/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/ssl/SSLServiceTests.java index 545a8f91574a7..9b697e9f08f01 100644 --- a/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/ssl/SSLServiceTests.java +++ b/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/ssl/SSLServiceTests.java @@ -109,11 +109,11 @@ public void testThatCustomTruststoreCanBeSpecified() throws Exception { assumeFalse("Can't run in a FIPS JVM", inFipsJvm()); Path testClientStore = getDataPath("/org/elasticsearch/xpack/security/transport/ssl/certs/simple/testclient.jks"); MockSecureSettings secureSettings = new MockSecureSettings(); - secureSettings.setString("xpack.ssl.truststore.secure_password", "testnode"); + secureSettings.setString("xpack.security.transport.ssl.truststore.secure_password", "testnode"); secureSettings.setString("transport.profiles.foo.xpack.security.ssl.truststore.secure_password", "testclient"); Settings settings = Settings.builder() - .put("xpack.ssl.truststore.path", testnodeStore) - .put("xpack.ssl.truststore.type", testnodeStoreType) + .put("xpack.security.transport.ssl.truststore.path", testnodeStore) + .put("xpack.security.transport.ssl.truststore.type", testnodeStoreType) .setSecureSettings(secureSettings) .put("transport.profiles.foo.xpack.security.ssl.truststore.path", testClientStore) .build(); @@ -126,12 +126,12 @@ public void testThatCustomTruststoreCanBeSpecified() throws Exception { .setSecureSettings(secureCustomSettings) .build(); - SSLConfiguration configuration = new SSLConfiguration(customTruststoreSettings, globalConfiguration(sslService)); + SSLConfiguration configuration = new SSLConfiguration(customTruststoreSettings); SSLEngine sslEngineWithTruststore = sslService.createSSLEngine(configuration, null, -1); assertThat(sslEngineWithTruststore, is(not(nullValue()))); - SSLConfiguration globalConfig = globalConfiguration(sslService); - SSLEngine sslEngine = sslService.createSSLEngine(globalConfig, null, -1); + SSLConfiguration defaultConfig = sslService.getSSLConfiguration("xpack.security.transport.ssl"); + SSLEngine sslEngine = sslService.createSSLEngine(defaultConfig, null, -1); assertThat(sslEngineWithTruststore, is(not(sameInstance(sslEngine)))); final SSLConfiguration profileConfiguration = sslService.getSSLConfiguration("transport.profiles.foo.xpack.security.ssl"); @@ -142,20 +142,21 @@ public void testThatCustomTruststoreCanBeSpecified() throws Exception { public void testThatSslContextCachingWorks() throws Exception { MockSecureSettings secureSettings = new MockSecureSettings(); - secureSettings.setString("xpack.ssl.secure_key_passphrase", "testnode"); + secureSettings.setString("xpack.security.transport.ssl.secure_key_passphrase", "testnode"); Settings settings = Settings.builder() - .put("xpack.ssl.certificate", testnodeCert) - .put("xpack.ssl.key", testnodeKey) + .put("xpack.security.transport.ssl.certificate", testnodeCert) + .put("xpack.security.transport.ssl.key", testnodeKey) .setSecureSettings(secureSettings) .build(); SSLService sslService = new SSLService(settings, env); - SSLContext sslContext = sslService.sslContext(); - SSLContext cachedSslContext = sslService.sslContext(); + final Settings transportSSLSettings = settings.getByPrefix("xpack.security.transport.ssl."); + SSLContext sslContext = sslService.sslContext(sslService.sslConfiguration(transportSSLSettings)); + SSLContext cachedSslContext = sslService.sslContext(sslService.sslConfiguration(transportSSLSettings)); assertThat(sslContext, is(sameInstance(cachedSslContext))); - final SSLConfiguration configuration = sslService.getSSLConfiguration("xpack.ssl"); + final SSLConfiguration configuration = sslService.getSSLConfiguration("xpack.security.transport.ssl"); final SSLContext configContext = sslService.sslContext(configuration); assertThat(configContext, is(sameInstance(sslContext))); } @@ -165,14 +166,15 @@ public void testThatKeyStoreAndKeyCanHaveDifferentPasswords() throws Exception { Path differentPasswordsStore = getDataPath("/org/elasticsearch/xpack/security/transport/ssl/certs/simple/testnode-different-passwords.jks"); MockSecureSettings secureSettings = new MockSecureSettings(); - secureSettings.setString("xpack.ssl.keystore.secure_password", "testnode"); - secureSettings.setString("xpack.ssl.keystore.secure_key_password", "testnode1"); + secureSettings.setString("xpack.security.transport.ssl.keystore.secure_password", "testnode"); + secureSettings.setString("xpack.security.transport.ssl.keystore.secure_key_password", "testnode1"); Settings settings = Settings.builder() - .put("xpack.ssl.keystore.path", differentPasswordsStore) - .setSecureSettings(secureSettings) - .build(); + .put("xpack.security.transport.ssl.keystore.path", differentPasswordsStore) + .setSecureSettings(secureSettings) + .build(); + final SSLService sslService = new SSLService(settings, env); - SSLConfiguration configuration = globalConfiguration(sslService); + SSLConfiguration configuration = sslService.getSSLConfiguration("xpack.security.transport.ssl"); sslService.createSSLEngine(configuration, null, -1); } @@ -182,13 +184,13 @@ public void testIncorrectKeyPasswordThrowsException() throws Exception { getDataPath("/org/elasticsearch/xpack/security/transport/ssl/certs/simple/testnode-different-passwords.jks"); try { MockSecureSettings secureSettings = new MockSecureSettings(); - secureSettings.setString("xpack.ssl.keystore.secure_password", "testnode"); + secureSettings.setString("xpack.security.transport.ssl.keystore.secure_password", "testnode"); Settings settings = Settings.builder() - .put("xpack.ssl.keystore.path", differentPasswordsStore) - .setSecureSettings(secureSettings) - .build(); + .put("xpack.security.transport.ssl.keystore.path", differentPasswordsStore) + .setSecureSettings(secureSettings) + .build(); final SSLService sslService = new SSLService(settings, env); - SSLConfiguration configuration = globalConfiguration(sslService); + SSLConfiguration configuration = sslService.getSSLConfiguration("xpack.security.transport.ssl"); sslService.createSSLEngine(configuration, null, -1); fail("expected an exception"); } catch (ElasticsearchException e) { @@ -198,34 +200,34 @@ public void testIncorrectKeyPasswordThrowsException() throws Exception { public void testThatSSLv3IsNotEnabled() throws Exception { MockSecureSettings secureSettings = new MockSecureSettings(); - secureSettings.setString("xpack.ssl.secure_key_passphrase", "testnode"); + secureSettings.setString("xpack.security.transport.ssl.secure_key_passphrase", "testnode"); Settings settings = Settings.builder() - .put("xpack.ssl.certificate", testnodeCert) - .put("xpack.ssl.key", testnodeKey) + .put("xpack.security.transport.ssl.certificate", testnodeCert) + .put("xpack.security.transport.ssl.key", testnodeKey) .setSecureSettings(secureSettings) .build(); SSLService sslService = new SSLService(settings, env); - SSLConfiguration configuration = globalConfiguration(sslService); + SSLConfiguration configuration = sslService.getSSLConfiguration("xpack.security.transport.ssl"); SSLEngine engine = sslService.createSSLEngine(configuration, null, -1); assertThat(Arrays.asList(engine.getEnabledProtocols()), not(hasItem("SSLv3"))); } public void testThatCreateClientSSLEngineWithoutAnySettingsWorks() throws Exception { SSLService sslService = new SSLService(Settings.EMPTY, env); - SSLConfiguration configuration = globalConfiguration(sslService); + SSLConfiguration configuration = sslService.getSSLConfiguration("xpack.security.transport.ssl"); SSLEngine sslEngine = sslService.createSSLEngine(configuration, null, -1); assertThat(sslEngine, notNullValue()); } public void testThatCreateSSLEngineWithOnlyTruststoreWorks() throws Exception { MockSecureSettings secureSettings = new MockSecureSettings(); - secureSettings.setString("xpack.ssl.truststore.secure_password", "testclient"); + secureSettings.setString("xpack.security.transport.ssl.truststore.secure_password", "testclient"); Settings settings = Settings.builder() - .put("xpack.ssl.truststore.path", testclientStore) - .setSecureSettings(secureSettings) - .build(); + .put("xpack.security.transport.ssl.truststore.path", testclientStore) + .setSecureSettings(secureSettings) + .build(); SSLService sslService = new SSLService(settings, env); - SSLConfiguration configuration = globalConfiguration(sslService); + SSLConfiguration configuration = sslService.getSSLConfiguration("xpack.security.transport.ssl"); SSLEngine sslEngine = sslService.createSSLEngine(configuration, null, -1); assertThat(sslEngine, notNullValue()); } @@ -234,54 +236,52 @@ public void testThatCreateSSLEngineWithOnlyTruststoreWorks() throws Exception { public void testCreateWithKeystoreIsValidForServer() throws Exception { assumeFalse("Can't run in a FIPS JVM, JKS keystores can't be used", inFipsJvm()); MockSecureSettings secureSettings = new MockSecureSettings(); - secureSettings.setString("xpack.ssl.keystore.secure_password", "testnode"); + secureSettings.setString("xpack.security.transport.ssl.keystore.secure_password", "testnode"); Settings settings = Settings.builder() - .put("xpack.ssl.keystore.path", testnodeStore) - .put("xpack.ssl.keystore.type", testnodeStoreType) - .setSecureSettings(secureSettings) - .build(); + .put("xpack.security.transport.ssl.keystore.path", testnodeStore) + .put("xpack.security.transport.ssl.keystore.type", testnodeStoreType) + .setSecureSettings(secureSettings) + .build(); SSLService sslService = new SSLService(settings, env); - assertTrue(sslService.isConfigurationValidForServerUsage(globalConfiguration(sslService))); + assertTrue(sslService.isConfigurationValidForServerUsage(sslService.getSSLConfiguration("xpack.security.transport.ssl"))); } - public void testValidForServerWithFallback() throws Exception { + public void testValidForServer() throws Exception { assumeFalse("Can't run in a FIPS JVM, JKS keystores can't be used", inFipsJvm()); MockSecureSettings secureSettings = new MockSecureSettings(); - secureSettings.setString("xpack.ssl.truststore.secure_password", "testnode"); + secureSettings.setString("xpack.security.transport.ssl.truststore.secure_password", "testnode"); Settings settings = Settings.builder() - .put("xpack.ssl.truststore.path", testnodeStore) - .put("xpack.ssl.truststore.type", testnodeStoreType) - .setSecureSettings(secureSettings) - .build(); + .put("xpack.security.transport.ssl.truststore.path", testnodeStore) + .put("xpack.security.transport.ssl.truststore.type", testnodeStoreType) + .setSecureSettings(secureSettings) + .build(); SSLService sslService = new SSLService(settings, env); - assertFalse(sslService.isConfigurationValidForServerUsage(globalConfiguration(sslService))); + assertFalse(sslService.isConfigurationValidForServerUsage(sslService.getSSLConfiguration("xpack.security.transport.ssl"))); secureSettings.setString("xpack.security.transport.ssl.keystore.secure_password", "testnode"); settings = Settings.builder() - .put("xpack.ssl.truststore.path", testnodeStore) - .put("xpack.ssl.truststore.type", testnodeStoreType) - .setSecureSettings(secureSettings) - .put("xpack.security.transport.ssl.keystore.path", testnodeStore) - .put("xpack.security.transport.ssl.keystore.type", testnodeStoreType) - .build(); + .put("xpack.security.transport.ssl.truststore.path", testnodeStore) + .put("xpack.security.transport.ssl.truststore.type", testnodeStoreType) + .setSecureSettings(secureSettings) + .put("xpack.security.transport.ssl.keystore.path", testnodeStore) + .put("xpack.security.transport.ssl.keystore.type", testnodeStoreType) + .build(); sslService = new SSLService(settings, env); - assertFalse(sslService.isConfigurationValidForServerUsage(globalConfiguration(sslService))); assertTrue(sslService.isConfigurationValidForServerUsage(sslService.getSSLConfiguration("xpack.security.transport.ssl"))); } public void testGetVerificationMode() throws Exception { assumeFalse("Can't run in a FIPS JVM, TrustAllConfig is not a SunJSSE TrustManagers", inFipsJvm()); SSLService sslService = new SSLService(Settings.EMPTY, env); - assertThat(globalConfiguration(sslService).verificationMode(), is(XPackSettings.VERIFICATION_MODE_DEFAULT)); + assertThat(sslService.getSSLConfiguration("xpack.security.transport.ssl").verificationMode(), + is(XPackSettings.VERIFICATION_MODE_DEFAULT)); Settings settings = Settings.builder() - .put("xpack.ssl.verification_mode", "none") - .put("xpack.security.transport.ssl.verification_mode", "certificate") - .put("transport.profiles.foo.xpack.security.ssl.verification_mode", "full") - .build(); + .put("xpack.security.transport.ssl.verification_mode", "certificate") + .put("transport.profiles.foo.xpack.security.ssl.verification_mode", "full") + .build(); sslService = new SSLService(settings, env); - assertThat(globalConfiguration(sslService).verificationMode(), is(VerificationMode.NONE)); assertThat(sslService.getSSLConfiguration("xpack.security.transport.ssl.").verificationMode(), is(VerificationMode.CERTIFICATE)); assertThat(sslService.getSSLConfiguration("transport.profiles.foo.xpack.security.ssl.").verificationMode(), is(VerificationMode.FULL)); @@ -289,27 +289,25 @@ public void testGetVerificationMode() throws Exception { public void testIsSSLClientAuthEnabled() throws Exception { SSLService sslService = new SSLService(Settings.EMPTY, env); - assertTrue(globalConfiguration(sslService).sslClientAuth().enabled()); + assertTrue(sslService.getSSLConfiguration("xpack.security.transport.ssl").sslClientAuth().enabled()); Settings settings = Settings.builder() - .put("xpack.ssl.client_authentication", "none") - .put("xpack.security.transport.ssl.client_authentication", "optional") + .put("xpack.security.transport.ssl.client_authentication", "optional") .put("transport.profiles.foo.port", "9400-9410") - .build(); + .build(); sslService = new SSLService(settings, env); - assertFalse(sslService.isSSLClientAuthEnabled(globalConfiguration(sslService))); assertTrue(sslService.isSSLClientAuthEnabled(sslService.getSSLConfiguration("xpack.security.transport.ssl"))); assertTrue(sslService.isSSLClientAuthEnabled(sslService.getSSLConfiguration("transport.profiles.foo.xpack.security.ssl"))); } public void testThatHttpClientAuthDefaultsToNone() throws Exception { final Settings globalSettings = Settings.builder() - .put("xpack.security.http.ssl.enabled", true) - .put("xpack.ssl.client_authentication", SSLClientAuth.OPTIONAL.name()) - .build(); + .put("xpack.security.http.ssl.enabled", true) + .put("xpack.security.transport.ssl.client_authentication", SSLClientAuth.OPTIONAL.name()) + .build(); final SSLService sslService = new SSLService(globalSettings, env); - final SSLConfiguration globalConfig = globalConfiguration(sslService); + final SSLConfiguration globalConfig = sslService.getSSLConfiguration("xpack.security.transport.ssl"); assertThat(globalConfig.sslClientAuth(), is(SSLClientAuth.OPTIONAL)); final SSLConfiguration httpConfig = sslService.getHttpTransportSSLConfiguration(); @@ -318,14 +316,14 @@ public void testThatHttpClientAuthDefaultsToNone() throws Exception { public void testThatTruststorePasswordIsRequired() throws Exception { MockSecureSettings secureSettings = new MockSecureSettings(); - secureSettings.setString("xpack.ssl.keystore.secure_password", "testnode"); + secureSettings.setString("xpack.security.transport.ssl.keystore.secure_password", "testnode"); Settings settings = Settings.builder() - .put("xpack.ssl.keystore.path", testnodeStore) - .put("xpack.ssl.keystore.type", testnodeStoreType) - .setSecureSettings(secureSettings) - .put("xpack.ssl.truststore.path", testnodeStore) - .put("xpack.ssl.truststore.type", testnodeStoreType) - .build(); + .put("xpack.security.transport.ssl.keystore.path", testnodeStore) + .put("xpack.security.transport.ssl.keystore.type", testnodeStoreType) + .setSecureSettings(secureSettings) + .put("xpack.security.transport.ssl.truststore.path", testnodeStore) + .put("xpack.security.transport.ssl.truststore.type", testnodeStoreType) + .build(); ElasticsearchException e = expectThrows(ElasticsearchException.class, () -> new SSLService(settings, env)); assertThat(e.getMessage(), is("failed to initialize a TrustManagerFactory")); @@ -333,9 +331,9 @@ public void testThatTruststorePasswordIsRequired() throws Exception { public void testThatKeystorePasswordIsRequired() throws Exception { Settings settings = Settings.builder() - .put("xpack.ssl.keystore.path", testnodeStore) - .put("xpack.ssl.keystore.type", testnodeStoreType) - .build(); + .put("xpack.security.transport.ssl.keystore.path", testnodeStore) + .put("xpack.security.transport.ssl.keystore.type", testnodeStoreType) + .build(); ElasticsearchException e = expectThrows(ElasticsearchException.class, () -> new SSLService(settings, env)); assertThat(e.getMessage(), is("failed to create trust manager")); @@ -346,14 +344,15 @@ public void testCiphersAndInvalidCiphersWork() throws Exception { ciphers.add("foo"); ciphers.add("bar"); MockSecureSettings secureSettings = new MockSecureSettings(); - secureSettings.setString("xpack.ssl.secure_key_passphrase", "testnode"); + secureSettings.setString("xpack.security.transport.ssl.secure_key_passphrase", "testnode"); Settings settings = Settings.builder() - .put("xpack.ssl.certificate", testnodeCert) - .put("xpack.ssl.key", testnodeKey) + .put("xpack.security.transport.ssl.certificate", testnodeCert) + .put("xpack.security.transport.ssl.key", testnodeKey) .setSecureSettings(secureSettings) + .putList("xpack.security.transport.ssl.ciphers", ciphers.toArray(new String[ciphers.size()])) .build(); SSLService sslService = new SSLService(settings, env); - SSLConfiguration configuration = globalConfiguration(sslService); + SSLConfiguration configuration = sslService.getSSLConfiguration("xpack.security.transport.ssl"); SSLEngine engine = sslService.createSSLEngine(configuration, null, -1); assertThat(engine, is(notNullValue())); String[] enabledCiphers = engine.getEnabledCipherSuites(); @@ -362,14 +361,14 @@ public void testCiphersAndInvalidCiphersWork() throws Exception { public void testInvalidCiphersOnlyThrowsException() throws Exception { MockSecureSettings secureSettings = new MockSecureSettings(); - secureSettings.setString("xpack.ssl.secure_key_passphrase", "testnode"); + secureSettings.setString("xpack.security.transport.ssl.secure_key_passphrase", "testnode"); + Settings settings = Settings.builder() - .put("xpack.ssl.certificate", testnodeCert) - .put("xpack.ssl.key", testnodeKey) - .putList("xpack.ssl.cipher_suites", new String[]{"foo", "bar"}) + .put("xpack.security.transport.ssl.certificate", testnodeCert) + .put("xpack.security.transport.ssl.key", testnodeKey) .setSecureSettings(secureSettings) + .putList("xpack.security.transport.ssl.cipher_suites", new String[] { "foo", "bar" }) .build(); - IllegalArgumentException e = expectThrows(IllegalArgumentException.class, () -> new SSLService(settings, env)); assertThat(e.getMessage(), is("none of the ciphers [foo, bar] are supported by this JVM")); @@ -377,14 +376,14 @@ public void testInvalidCiphersOnlyThrowsException() throws Exception { public void testThatSSLEngineHasCipherSuitesOrderSet() throws Exception { MockSecureSettings secureSettings = new MockSecureSettings(); - secureSettings.setString("xpack.ssl.secure_key_passphrase", "testnode"); + secureSettings.setString("xpack.security.transport.ssl.secure_key_passphrase", "testnode"); Settings settings = Settings.builder() - .put("xpack.ssl.certificate", testnodeCert) - .put("xpack.ssl.key", testnodeKey) + .put("xpack.security.transport.ssl.certificate", testnodeCert) + .put("xpack.security.transport.ssl.key", testnodeKey) .setSecureSettings(secureSettings) .build(); SSLService sslService = new SSLService(settings, env); - SSLConfiguration configuration = globalConfiguration(sslService); + SSLConfiguration configuration = sslService.getSSLConfiguration("xpack.security.transport.ssl"); SSLEngine engine = sslService.createSSLEngine(configuration, null, -1); assertThat(engine, is(notNullValue())); assertTrue(engine.getSSLParameters().getUseCipherSuitesOrder()); @@ -392,14 +391,14 @@ public void testThatSSLEngineHasCipherSuitesOrderSet() throws Exception { public void testThatSSLSocketFactoryHasProperCiphersAndProtocols() throws Exception { MockSecureSettings secureSettings = new MockSecureSettings(); - secureSettings.setString("xpack.ssl.secure_key_passphrase", "testnode"); + secureSettings.setString("xpack.security.transport.ssl.secure_key_passphrase", "testnode"); Settings settings = Settings.builder() - .put("xpack.ssl.certificate", testnodeCert) - .put("xpack.ssl.key", testnodeKey) + .put("xpack.security.transport.ssl.certificate", testnodeCert) + .put("xpack.security.transport.ssl.key", testnodeKey) .setSecureSettings(secureSettings) .build(); SSLService sslService = new SSLService(settings, env); - SSLConfiguration config = globalConfiguration(sslService); + SSLConfiguration config = sslService.getSSLConfiguration("xpack.security.transport.ssl"); final SSLSocketFactory factory = sslService.sslSocketFactory(config); final String[] ciphers = sslService.supportedCiphers(factory.getSupportedCipherSuites(), config.cipherSuites(), false); assertThat(factory.getDefaultCipherSuites(), is(ciphers)); @@ -417,14 +416,14 @@ public void testThatSSLSocketFactoryHasProperCiphersAndProtocols() throws Except public void testThatSSLEngineHasProperCiphersAndProtocols() throws Exception { MockSecureSettings secureSettings = new MockSecureSettings(); - secureSettings.setString("xpack.ssl.secure_key_passphrase", "testnode"); + secureSettings.setString("xpack.security.transport.ssl.secure_key_passphrase", "testnode"); Settings settings = Settings.builder() - .put("xpack.ssl.certificate", testnodeCert) - .put("xpack.ssl.key", testnodeKey) + .put("xpack.security.transport.ssl.certificate", testnodeCert) + .put("xpack.security.transport.ssl.key", testnodeKey) .setSecureSettings(secureSettings) .build(); SSLService sslService = new SSLService(settings, env); - SSLConfiguration configuration = globalConfiguration(sslService); + SSLConfiguration configuration = sslService.getSSLConfiguration("xpack.security.transport.ssl"); SSLEngine engine = sslService.createSSLEngine(configuration, null, -1); final String[] ciphers = sslService.supportedCiphers(engine.getSupportedCipherSuites(), configuration.cipherSuites(), false); final String[] supportedProtocols = configuration.supportedProtocols().toArray(Strings.EMPTY_ARRAY); @@ -475,9 +474,9 @@ public void testSSLStrategy() { } public void testEmptyTrustManager() throws Exception { - Settings settings = Settings.builder().build(); + Settings settings = Settings.EMPTY; final SSLService sslService = new SSLService(settings, env); - X509ExtendedTrustManager trustManager = sslService.sslContextHolder(sslService.getSSLConfiguration("xpack.ssl")) + X509ExtendedTrustManager trustManager = sslService.sslContextHolder(sslService.getSSLConfiguration("xpack.security.transport.ssl")) .getEmptyTrustManager(); assertThat(trustManager.getAcceptedIssuers(), emptyArray()); } @@ -489,7 +488,6 @@ public void testGetConfigurationByContextName() throws Exception { final String[] cipherSuites = sslContext.getSupportedSSLParameters().getCipherSuites(); final String[] contextNames = { - "xpack.ssl", "xpack.http.ssl", "xpack.security.http.ssl", "xpack.security.transport.ssl", @@ -517,10 +515,6 @@ public void testGetConfigurationByContextName() throws Exception { } final Settings settings = builder - // Add a realm without SSL settings. This context name should be mapped to the global configuration - .put("xpack.security.authc.realms.file.realm3.order", 4) - // Add an exporter without SSL settings. This context name should be mapped to the global configuration - .put("xpack.monitoring.exporters.mon3.type", "http") .setSecureSettings(secureSettings) .build(); SSLService sslService = new SSLService(settings, env); @@ -535,14 +529,6 @@ public void testGetConfigurationByContextName() throws Exception { assertThat("Cipher for " + name, configuration.cipherSuites(), contains(cipherSuites[i])); assertThat("Configuration for " + name + ".", sslService.getSSLConfiguration(name + "."), sameInstance(configuration)); } - - // These contexts have no SSL settings, but for convenience we want those components to be able to access their context - // by name, and get back the global configuration - final SSLConfiguration realm3Config = sslService.getSSLConfiguration("xpack.security.authc.realms.file.realm3.ssl"); - final SSLConfiguration mon3Config = sslService.getSSLConfiguration("xpack.monitoring.exporters.mon3.ssl."); - final SSLConfiguration global = globalConfiguration(sslService); - assertThat(realm3Config, sameInstance(global)); - assertThat(mon3Config, sameInstance(global)); } public void testReadCertificateInformation() throws Exception { @@ -552,13 +538,13 @@ public void testReadCertificateInformation() throws Exception { final Path pemPath = getDataPath("/org/elasticsearch/xpack/security/transport/ssl/certs/simple/active-directory-ca.crt"); final MockSecureSettings secureSettings = new MockSecureSettings(); - secureSettings.setString("xpack.ssl.keystore.secure_password", "testnode"); - secureSettings.setString("xpack.ssl.truststore.secure_password", "testnode"); + secureSettings.setString("xpack.security.transport.ssl.keystore.secure_password", "testnode"); + secureSettings.setString("xpack.security.transport.ssl.truststore.secure_password", "testnode"); secureSettings.setString("xpack.http.ssl.keystore.secure_password", "testnode"); final Settings settings = Settings.builder() - .put("xpack.ssl.keystore.path", jksPath) - .put("xpack.ssl.truststore.path", jksPath) + .put("xpack.security.transport.ssl.keystore.path", jksPath) + .put("xpack.security.transport.ssl.truststore.path", jksPath) .put("xpack.http.ssl.keystore.path", p12Path) .put("xpack.security.authc.realms.active_directory.ad.ssl.certificate_authorities", pemPath) .setSecureSettings(secureSettings) @@ -718,7 +704,7 @@ public int getSessionCacheSize() { @Network public void testThatSSLContextWithoutSettingsWorks() throws Exception { SSLService sslService = new SSLService(Settings.EMPTY, env); - SSLContext sslContext = sslService.sslContext(); + SSLContext sslContext = sslService.sslContext(sslService.sslConfiguration(Settings.EMPTY)); try (CloseableHttpClient client = HttpClients.custom().setSSLContext(sslContext).build()) { // Execute a GET on a site known to have a valid certificate signed by a trusted public CA // This will result in a SSLHandshakeException if the SSLContext does not trust the CA, but the default @@ -730,12 +716,13 @@ public void testThatSSLContextWithoutSettingsWorks() throws Exception { @Network public void testThatSSLContextTrustsJDKTrustedCAs() throws Exception { MockSecureSettings secureSettings = new MockSecureSettings(); - secureSettings.setString("xpack.ssl.keystore.secure_password", "testclient"); + secureSettings.setString("xpack.security.transport.ssl.keystore.secure_password", "testclient"); Settings settings = Settings.builder() - .put("xpack.ssl.keystore.path", testclientStore) + .put("xpack.security.transport.ssl.keystore.path", testclientStore) .setSecureSettings(secureSettings) .build(); - SSLContext sslContext = new SSLService(settings, env).sslContext(); + SSLService sslService = new SSLService(settings, env); + SSLContext sslContext = sslService.sslContext(sslService.sslConfiguration(settings.getByPrefix("xpack.security.transport.ssl."))); try (CloseableHttpClient client = HttpClients.custom().setSSLContext(sslContext).build()) { // Execute a GET on a site known to have a valid certificate signed by a trusted public CA which will succeed because the JDK // certs are trusted by default @@ -746,7 +733,7 @@ public void testThatSSLContextTrustsJDKTrustedCAs() throws Exception { @Network public void testThatSSLIOSessionStrategyWithoutSettingsWorks() throws Exception { SSLService sslService = new SSLService(Settings.EMPTY, env); - SSLConfiguration sslConfiguration = globalConfiguration(sslService); + SSLConfiguration sslConfiguration = sslService.getSSLConfiguration("xpack.security.transport.ssl"); logger.info("SSL Configuration: {}", sslConfiguration); SSLIOSessionStrategy sslStrategy = sslService.sslIOSessionStrategy(sslConfiguration); try (CloseableHttpAsyncClient client = getAsyncHttpClient(sslStrategy)) { @@ -762,13 +749,13 @@ public void testThatSSLIOSessionStrategyWithoutSettingsWorks() throws Exception @Network public void testThatSSLIOSessionStrategyTrustsJDKTrustedCAs() throws Exception { MockSecureSettings secureSettings = new MockSecureSettings(); - secureSettings.setString("xpack.ssl.keystore.secure_password", "testclient"); + secureSettings.setString("xpack.security.transport.ssl.keystore.secure_password", "testclient"); Settings settings = Settings.builder() - .put("xpack.ssl.keystore.path", testclientStore) + .put("xpack.security.transport.ssl.keystore.path", testclientStore) .setSecureSettings(secureSettings) .build(); final SSLService sslService = new SSLService(settings, env); - SSLIOSessionStrategy sslStrategy = sslService.sslIOSessionStrategy(globalConfiguration(sslService)); + SSLIOSessionStrategy sslStrategy = sslService.sslIOSessionStrategy(sslService.getSSLConfiguration("xpack.security.transport.ssl")); try (CloseableHttpAsyncClient client = getAsyncHttpClient(sslStrategy)) { client.start(); @@ -778,10 +765,6 @@ public void testThatSSLIOSessionStrategyTrustsJDKTrustedCAs() throws Exception { } } - private static SSLConfiguration globalConfiguration(SSLService sslService) { - return sslService.getSSLConfiguration("xpack.ssl"); - } - class AssertionCallback implements FutureCallback { @Override diff --git a/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/ssl/TestsSSLService.java b/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/ssl/TestsSSLService.java index 25dc017261819..e8766225a7a92 100644 --- a/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/ssl/TestsSSLService.java +++ b/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/ssl/TestsSSLService.java @@ -19,11 +19,6 @@ public TestsSSLService(Settings settings, Environment environment) { super(settings, environment); } - @Override - public SSLContext sslContext() { - return super.sslContext(); - } - /** * Allows to get alternative ssl context, like for the http client */ diff --git a/x-pack/plugin/monitoring/src/test/java/org/elasticsearch/xpack/monitoring/exporter/http/HttpExporterSslIT.java b/x-pack/plugin/monitoring/src/test/java/org/elasticsearch/xpack/monitoring/exporter/http/HttpExporterSslIT.java index 9b108d4f8c69f..ae363cd3c8258 100644 --- a/x-pack/plugin/monitoring/src/test/java/org/elasticsearch/xpack/monitoring/exporter/http/HttpExporterSslIT.java +++ b/x-pack/plugin/monitoring/src/test/java/org/elasticsearch/xpack/monitoring/exporter/http/HttpExporterSslIT.java @@ -95,14 +95,14 @@ private MockWebServer buildWebServer() throws IOException { final Path key = getDataPath("/org/elasticsearch/xpack/monitoring/exporter/http/testnode.pem"); final Settings sslSettings = Settings.builder() - .put("xpack.ssl.certificate", cert) - .put("xpack.ssl.key", key) - .put("xpack.ssl.key_passphrase", "testnode") + .put("xpack.transport.security.ssl.certificate", cert) + .put("xpack.transport.security.ssl.key", key) + .put("xpack.transport.security.ssl.key_passphrase", "testnode") .put(globalSettings) .build(); TestsSSLService sslService = new TestsSSLService(sslSettings, environment); - final SSLContext sslContext = sslService.sslContext(Settings.EMPTY); + final SSLContext sslContext = sslService.sslContext("xpack.security.transport.ssl"); MockWebServer server = new MockWebServer(sslContext, false); server.start(); return server; diff --git a/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/PkiRealmBootstrapCheck.java b/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/PkiRealmBootstrapCheck.java index d4b05b1772e27..8f5012e1ecaf1 100644 --- a/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/PkiRealmBootstrapCheck.java +++ b/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/PkiRealmBootstrapCheck.java @@ -9,7 +9,7 @@ import org.elasticsearch.bootstrap.BootstrapContext; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.xpack.core.XPackSettings; -import org.elasticsearch.xpack.core.security.authc.RealmConfig; +import org.elasticsearch.xpack.core.security.authc.RealmConfig.RealmIdentifier; import org.elasticsearch.xpack.core.security.authc.RealmSettings; import org.elasticsearch.xpack.core.security.authc.pki.PkiRealmSettings; import org.elasticsearch.xpack.core.ssl.SSLConfiguration; @@ -37,7 +37,7 @@ class PkiRealmBootstrapCheck implements BootstrapCheck { @Override public BootstrapCheckResult check(BootstrapContext context) { final Settings settings = context.settings(); - final Map realms = RealmSettings.getRealmSettings(settings); + final Map realms = RealmSettings.getRealmSettings(settings); final boolean pkiRealmEnabled = realms.entrySet().stream() .filter(e -> PkiRealmSettings.TYPE.equals(e.getKey().getType())) .map(Map.Entry::getValue) @@ -70,6 +70,7 @@ private List getSslContextNames(Settings settings) { return list; } + // FIXME this is an antipattern move this out of a bootstrap check! @Override public boolean alwaysEnforce() { return true; diff --git a/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/authc/esnative/ESNativeRealmMigrateTool.java b/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/authc/esnative/ESNativeRealmMigrateTool.java index 3cc6c997f282f..6368f4a7510c9 100644 --- a/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/authc/esnative/ESNativeRealmMigrateTool.java +++ b/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/authc/esnative/ESNativeRealmMigrateTool.java @@ -67,7 +67,6 @@ import java.util.Set; import static org.elasticsearch.common.xcontent.XContentFactory.jsonBuilder; -import static org.elasticsearch.xpack.core.security.SecurityField.setting; /** * This is the command-line tool used for migrating users and roles from the file-based realm into the new native realm using the API for @@ -149,7 +148,7 @@ private String postURL(Settings settings, Environment env, String method, String // If using SSL, need a custom service because it's likely a self-signed certificate if ("https".equalsIgnoreCase(uri.getScheme())) { final SSLService sslService = new SSLService(settings, env); - final SSLConfiguration sslConfiguration = sslService.getSSLConfiguration(setting("http.ssl")); + final SSLConfiguration sslConfiguration = sslService.getSSLConfiguration("xpack.security.http.ssl"); final HttpsURLConnection httpsConn = (HttpsURLConnection) url.openConnection(); AccessController.doPrivileged((PrivilegedAction) () -> { // Requires permission java.lang.RuntimePermission "setFactory"; diff --git a/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/transport/nio/SecurityNioTransport.java b/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/transport/nio/SecurityNioTransport.java index b536644ad6121..7fdf946675ffd 100644 --- a/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/transport/nio/SecurityNioTransport.java +++ b/x-pack/plugin/security/src/main/java/org/elasticsearch/xpack/security/transport/nio/SecurityNioTransport.java @@ -86,7 +86,6 @@ public SecurityNioTransport(Settings settings, Version version, ThreadPool threa Map profileConfiguration = SecurityNetty4Transport.getTransportProfileConfigurations(settings, sslService, transportConfiguration); this.profileConfiguration = Collections.unmodifiableMap(profileConfiguration); - } else { profileConfiguration = Collections.emptyMap(); } diff --git a/x-pack/plugin/security/src/test/java/org/elasticsearch/test/SecuritySettingsSource.java b/x-pack/plugin/security/src/test/java/org/elasticsearch/test/SecuritySettingsSource.java index fa1cfbcba9993..adeb4a7f86569 100644 --- a/x-pack/plugin/security/src/test/java/org/elasticsearch/test/SecuritySettingsSource.java +++ b/x-pack/plugin/security/src/test/java/org/elasticsearch/test/SecuritySettingsSource.java @@ -149,7 +149,7 @@ public Path nodeConfigPath(int nodeOrdinal) { @Override public Settings transportClientSettings() { Settings.Builder builder = Settings.builder(); - addClientSSLSettings(builder, ""); + addClientSSLSettings(builder, "xpack.security.transport."); addDefaultSecurityTransportType(builder, Settings.EMPTY); if (randomBoolean()) { @@ -208,22 +208,27 @@ protected SecureString transportClientPassword() { return new SecureString(SecuritySettingsSourceField.TEST_PASSWORD.toCharArray()); } + public static void addSSLSettingsForNodePEMFiles(Settings.Builder builder, String prefix, boolean hostnameVerificationEnabled) { + addSSLSettingsForPEMFiles(builder, prefix, + "/org/elasticsearch/xpack/security/transport/ssl/certs/simple/testnode.pem", "testnode", + "/org/elasticsearch/xpack/security/transport/ssl/certs/simple/testnode.crt", + Arrays.asList("/org/elasticsearch/xpack/security/transport/ssl/certs/simple/testnode-client-profile.crt", + "/org/elasticsearch/xpack/security/transport/ssl/certs/simple/active-directory-ca.crt", + "/org/elasticsearch/xpack/security/transport/ssl/certs/simple/testclient.crt", + "/org/elasticsearch/xpack/security/transport/ssl/certs/simple/openldap.crt", + "/org/elasticsearch/xpack/security/transport/ssl/certs/simple/testnode.crt"), + hostnameVerificationEnabled, false); + } + private void addNodeSSLSettings(Settings.Builder builder) { if (sslEnabled) { + builder.put("xpack.security.transport.ssl.enabled", true); if (usePEM) { - addSSLSettingsForPEMFiles(builder, "", - "/org/elasticsearch/xpack/security/transport/ssl/certs/simple/testnode.pem", "testnode", - "/org/elasticsearch/xpack/security/transport/ssl/certs/simple/testnode.crt", - Arrays.asList("/org/elasticsearch/xpack/security/transport/ssl/certs/simple/testnode-client-profile.crt", - "/org/elasticsearch/xpack/security/transport/ssl/certs/simple/active-directory-ca.crt", - "/org/elasticsearch/xpack/security/transport/ssl/certs/simple/testclient.crt", - "/org/elasticsearch/xpack/security/transport/ssl/certs/simple/openldap.crt", - "/org/elasticsearch/xpack/security/transport/ssl/certs/simple/testnode.crt"), - sslEnabled, hostnameVerificationEnabled, false); - + addSSLSettingsForNodePEMFiles(builder, "xpack.security.transport.", hostnameVerificationEnabled); } else { - addSSLSettingsForStore(builder, "", "/org/elasticsearch/xpack/security/transport/ssl/certs/simple/testnode.jks", - "testnode", sslEnabled, hostnameVerificationEnabled, false); + addSSLSettingsForStore(builder, "xpack.security.transport.", + "/org/elasticsearch/xpack/security/transport/ssl/certs/simple/testnode.jks", "testnode", + hostnameVerificationEnabled, false); } } else if (randomBoolean()) { builder.put(XPackSettings.TRANSPORT_SSL_ENABLED.getKey(), false); @@ -231,47 +236,53 @@ private void addNodeSSLSettings(Settings.Builder builder) { } public void addClientSSLSettings(Settings.Builder builder, String prefix) { + builder.put("xpack.security.transport.ssl.enabled", sslEnabled); if (usePEM) { addSSLSettingsForPEMFiles(builder, prefix, "/org/elasticsearch/xpack/security/transport/ssl/certs/simple/testclient.pem", "testclient", "/org/elasticsearch/xpack/security/transport/ssl/certs/simple/testclient.crt", Arrays.asList("/org/elasticsearch/xpack/security/transport/ssl/certs/simple/testnode.crt", "/org/elasticsearch/xpack/security/transport/ssl/certs/simple/testclient.crt"), - sslEnabled, hostnameVerificationEnabled, true); + hostnameVerificationEnabled, true); } else { addSSLSettingsForStore(builder, prefix, "/org/elasticsearch/xpack/security/transport/ssl/certs/simple/testclient.jks", - "testclient", sslEnabled, hostnameVerificationEnabled, true); + "testclient", hostnameVerificationEnabled, true); } } + /** + * Returns the configuration settings given the location of a certificate and its password + * + * @param resourcePathToStore the location of the keystore or truststore + * @param password the password + */ + public static void addSSLSettingsForStore(Settings.Builder builder, String resourcePathToStore, String password, String prefix) { + addSSLSettingsForStore(builder, prefix, resourcePathToStore, password, true, true); + } + private static void addSSLSettingsForStore(Settings.Builder builder, String prefix, String resourcePathToStore, String password, - boolean sslEnabled, boolean hostnameVerificationEnabled, - boolean transportClient) { + boolean hostnameVerificationEnabled, boolean transportClient) { Path store = resolveResourcePath(resourcePathToStore); - - if (transportClient == false) { - builder.put(prefix + "xpack.security.http.ssl.enabled", false); - } - builder.put(XPackSettings.TRANSPORT_SSL_ENABLED.getKey(), sslEnabled); - - builder.put(prefix + "xpack.ssl.verification_mode", hostnameVerificationEnabled ? "full" : "certificate"); - builder.put(prefix + "xpack.ssl.keystore.path", store); + builder.put(prefix + "ssl.verification_mode", hostnameVerificationEnabled ? "full" : "certificate"); + builder.put(prefix + "ssl.keystore.path", store); if (transportClient) { // continue using insecure settings for clients until we figure out what to do there... - builder.put(prefix + "xpack.ssl.keystore.password", password); + builder.put(prefix + "ssl.keystore.password", password); } else { + final String finalPrefix = prefix; addSecureSettings(builder, secureSettings -> - secureSettings.setString(prefix + "xpack.ssl.keystore.secure_password", password)); + secureSettings.setString(finalPrefix + "ssl.keystore.secure_password", password)); } if (randomBoolean()) { - builder.put(prefix + "xpack.ssl.truststore.path", store); + builder.put(prefix + "ssl.truststore.path", store); if (transportClient) { // continue using insecure settings for clients until we figure out what to do there... - builder.put(prefix + "xpack.ssl.truststore.password", password); + builder.put(prefix + "ssl.truststore.password", password); } else { + final String finalPrefix = prefix; addSecureSettings(builder, secureSettings -> - secureSettings.setString(prefix + "xpack.ssl.truststore.secure_password", password)); + secureSettings.setString(finalPrefix + "ssl.truststore.secure_password", password)); } } } @@ -289,31 +300,46 @@ private static void addSSLSettingsForStore(Settings.Builder builder, String pref */ public static void addSSLSettingsForPEMFiles(Settings.Builder builder, String keyPath, String password, String certificatePath, List trustedCertificates) { - addSSLSettingsForPEMFiles(builder, "", keyPath, password, certificatePath, trustedCertificates, true, true, true); + addSSLSettingsForPEMFiles(builder, "", keyPath, password, certificatePath, trustedCertificates, true, true); + } + + /** + * Returns the SSL related configuration settings given the location of a key and certificate and the location + * of the PEM certificates to be trusted + * + * @param keyPath The path to the Private key to be used for SSL + * @param password The password with which the private key is protected + * @param certificatePath The path to the PEM formatted Certificate encapsulating the public key that corresponds + * to the Private Key specified in {@code keyPath}. Will be presented to incoming + * SSL connections. + * @param prefix The settings prefix to use before ssl setting names + * @param trustedCertificates A list of PEM formatted certificates that will be trusted. + */ + public static void addSSLSettingsForPEMFiles(Settings.Builder builder, String keyPath, String password, + String certificatePath, String prefix, List trustedCertificates) { + addSSLSettingsForPEMFiles(builder, prefix, keyPath, password, certificatePath, trustedCertificates, true, true); } private static void addSSLSettingsForPEMFiles(Settings.Builder builder, String prefix, String keyPath, String password, - String certificatePath, List trustedCertificates, boolean sslEnabled, + String certificatePath, List trustedCertificates, boolean hostnameVerificationEnabled, boolean transportClient) { - - if (transportClient == false) { - builder.put(prefix + "xpack.security.http.ssl.enabled", false); + if (prefix.equals("")) { + prefix = "xpack.security.transport."; } - builder.put(XPackSettings.TRANSPORT_SSL_ENABLED.getKey(), sslEnabled); - - builder.put(prefix + "xpack.ssl.verification_mode", hostnameVerificationEnabled ? "full" : "certificate"); - builder.put(prefix + "xpack.ssl.key", resolveResourcePath(keyPath)) - .put(prefix + "xpack.ssl.certificate", resolveResourcePath(certificatePath)); + builder.put(prefix + "ssl.verification_mode", hostnameVerificationEnabled ? "full" : "certificate"); + builder.put(prefix + "ssl.key", resolveResourcePath(keyPath)) + .put(prefix + "ssl.certificate", resolveResourcePath(certificatePath)); if (transportClient) { // continue using insecure settings for clients until we figure out what to do there... - builder.put(prefix + "xpack.ssl.key_passphrase", password); + builder.put(prefix + "ssl.key_passphrase", password); } else { + final String finalPrefix = prefix; addSecureSettings(builder, secureSettings -> - secureSettings.setString(prefix + "xpack.ssl.secure_key_passphrase", password)); + secureSettings.setString(finalPrefix + "ssl.secure_key_passphrase", password)); } if (trustedCertificates.isEmpty() == false) { - builder.put(prefix + "xpack.ssl.certificate_authorities", + builder.put(prefix + "ssl.certificate_authorities", Strings.arrayToCommaDelimitedString(resolvePathsToString(trustedCertificates))); } } diff --git a/x-pack/plugin/security/src/test/java/org/elasticsearch/test/SettingsFilterTests.java b/x-pack/plugin/security/src/test/java/org/elasticsearch/test/SettingsFilterTests.java index b4584200a4a33..014ebc72a982b 100644 --- a/x-pack/plugin/security/src/test/java/org/elasticsearch/test/SettingsFilterTests.java +++ b/x-pack/plugin/security/src/test/java/org/elasticsearch/test/SettingsFilterTests.java @@ -64,21 +64,21 @@ public void testFiltering() throws Exception { if (inFipsJvm() == false) { configureFilteredSetting("xpack.security.authc.realms.pki.pki1.truststore.path", getDataPath("/org/elasticsearch/xpack/security/transport/ssl/certs/simple/truststore-testnode-only.jks").toString()); - configureFilteredSetting("xpack.ssl.keystore.path", + configureFilteredSetting("xpack.security.transport.ssl.keystore.path", getDataPath("/org/elasticsearch/xpack/security/transport/ssl/certs/simple/testnode.jks").toString()); } configureSecureSetting("xpack.security.authc.realms.pki.pki1.truststore.secure_password", "truststore-testnode-only"); configureFilteredSetting("xpack.security.authc.realms.pki.pki1.truststore.algorithm", "SunX509"); - configureFilteredSetting("xpack.ssl.cipher_suites", + configureFilteredSetting("xpack.security.transport.ssl.cipher_suites", Strings.arrayToCommaDelimitedString(XPackSettings.DEFAULT_CIPHERS.toArray())); - configureFilteredSetting("xpack.ssl.supported_protocols", randomFrom("TLSv1", "TLSv1.1", "TLSv1.2")); - configureSecureSetting("xpack.ssl.keystore.secure_password", "testnode"); - configureFilteredSetting("xpack.ssl.keystore.algorithm", KeyManagerFactory.getDefaultAlgorithm()); - configureSecureSetting("xpack.ssl.keystore.secure_key_password", "testnode"); - configureSecureSetting("xpack.ssl.truststore.secure_password", randomAlphaOfLength(5)); - configureFilteredSetting("xpack.ssl.truststore.algorithm", TrustManagerFactory.getDefaultAlgorithm()); + configureFilteredSetting("xpack.security.transport.ssl.supported_protocols", randomFrom("TLSv1", "TLSv1.1", "TLSv1.2")); + configureSecureSetting("xpack.security.transport.ssl.keystore.secure_password", "testnode"); + configureFilteredSetting("xpack.security.transport.ssl.keystore.algorithm", KeyManagerFactory.getDefaultAlgorithm()); + configureSecureSetting("xpack.security.transport.ssl.keystore.secure_key_password", "testnode"); + configureSecureSetting("xpack.security.transport.ssl.truststore.secure_password", randomAlphaOfLength(5)); + configureFilteredSetting("xpack.security.transport.ssl.truststore.algorithm", TrustManagerFactory.getDefaultAlgorithm()); // client profile configureUnfilteredSetting("transport.profiles.client.port", "9500-9600"); diff --git a/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/FIPS140JKSKeystoreBootstrapCheckTests.java b/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/FIPS140JKSKeystoreBootstrapCheckTests.java index 53554c9fad09f..b35b8009f12ee 100644 --- a/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/FIPS140JKSKeystoreBootstrapCheckTests.java +++ b/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/FIPS140JKSKeystoreBootstrapCheckTests.java @@ -16,22 +16,6 @@ public void testNoKeystoreIsAllowed() { assertFalse(new FIPS140JKSKeystoreBootstrapCheck().check(createTestContext(settings.build(), null)).isFailure()); } - public void testSSLKeystoreTypeIsNotAllowed() { - final Settings.Builder settings = Settings.builder() - .put("xpack.security.fips_mode.enabled", "true") - .put("xpack.ssl.keystore.path", "/this/is/the/path") - .put("xpack.ssl.keystore.type", "JKS"); - assertTrue(new FIPS140JKSKeystoreBootstrapCheck().check(createTestContext(settings.build(), null)).isFailure()); - } - - public void testSSLImplicitKeystoreTypeIsNotAllowed() { - final Settings.Builder settings = Settings.builder() - .put("xpack.security.fips_mode.enabled", "true") - .put("xpack.ssl.keystore.path", "/this/is/the/path") - .put("xpack.ssl.keystore.type", "JKS"); - assertTrue(new FIPS140JKSKeystoreBootstrapCheck().check(createTestContext(settings.build(), null)).isFailure()); - } - public void testTransportSSLKeystoreTypeIsNotAllowed() { final Settings.Builder settings = Settings.builder() .put("xpack.security.fips_mode.enabled", "true") diff --git a/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/PkiRealmBootstrapCheckTests.java b/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/PkiRealmBootstrapCheckTests.java index 2bfa560ff13e8..2c62ce71d45a9 100644 --- a/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/PkiRealmBootstrapCheckTests.java +++ b/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/PkiRealmBootstrapCheckTests.java @@ -36,15 +36,9 @@ public void testBootstrapCheckWithPkiRealm() throws Exception { .build(); assertFalse(runCheck(settings, env).isFailure()); - // disable client auth default - settings = Settings.builder().put(settings) - .put("xpack.ssl.client_authentication", "none") - .build(); - env = TestEnvironment.newEnvironment(settings); - assertTrue(runCheck(settings, env).isFailure()); - // enable ssl for http settings = Settings.builder().put(settings) + .put("xpack.security.transport.ssl.enabled", false) .put("xpack.security.http.ssl.enabled", true) .build(); env = TestEnvironment.newEnvironment(settings); @@ -73,6 +67,7 @@ public void testBootstrapCheckWithPkiRealm() throws Exception { // test with transport profile settings = Settings.builder().put(settings) + .put("xpack.security.transport.ssl.enabled", true) .put("xpack.security.transport.client_authentication", "none") .put("transport.profiles.foo.xpack.security.ssl.client_authentication", randomFrom("required", "optional")) .build(); @@ -87,7 +82,7 @@ private BootstrapCheck.BootstrapCheckResult runCheck(Settings settings, Environm public void testBootstrapCheckWithDisabledRealm() throws Exception { Settings settings = Settings.builder() .put("xpack.security.authc.realms.pki.test_pki.enabled", false) - .put("xpack.ssl.client_authentication", "none") + .put("xpack.security.transport.ssl.client_authentication", "none") .put("path.home", createTempDir()) .build(); Environment env = TestEnvironment.newEnvironment(settings); diff --git a/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/audit/index/IndexAuditTrailTests.java b/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/audit/index/IndexAuditTrailTests.java index 9fe510435c5ab..2f910658b3569 100644 --- a/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/audit/index/IndexAuditTrailTests.java +++ b/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/audit/index/IndexAuditTrailTests.java @@ -246,7 +246,7 @@ protected void addDefaultSecurityTransportType(Settings.Builder builder, Setting SecuritySettingsSourceField.TEST_PASSWORD); if (remoteUseSSL) { - cluster2SettingsSource.addClientSSLSettings(builder, "xpack.security.audit.index.client."); + cluster2SettingsSource.addClientSSLSettings(builder, "xpack.security.audit.index.client.xpack.security.transport."); builder.put("xpack.security.audit.index.client.xpack.security.transport.ssl.enabled", true); } if (useSecurity == false && builder.get(NetworkModule.TRANSPORT_TYPE_KEY) == null) { diff --git a/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/audit/index/RemoteIndexAuditTrailStartingTests.java b/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/audit/index/RemoteIndexAuditTrailStartingTests.java index ba62e5b52c40e..bc893538642d0 100644 --- a/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/audit/index/RemoteIndexAuditTrailStartingTests.java +++ b/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/audit/index/RemoteIndexAuditTrailStartingTests.java @@ -111,7 +111,7 @@ public Settings nodeSettings(int nodeOrdinal) { .put("xpack.security.audit.index.settings.index.number_of_shards", 1) .put("xpack.security.audit.index.settings.index.number_of_replicas", 0); - addClientSSLSettings(builder, "xpack.security.audit.index.client."); + addClientSSLSettings(builder, "xpack.security.audit.index.client.xpack.security.transport."); builder.put("xpack.security.audit.index.client.xpack.security.transport.ssl.enabled", sslEnabled); return builder.build(); } diff --git a/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/authc/esnative/ESNativeMigrateToolTests.java b/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/authc/esnative/ESNativeMigrateToolTests.java index 7f3e2cfce9854..66bff81e5dd56 100644 --- a/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/authc/esnative/ESNativeMigrateToolTests.java +++ b/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/authc/esnative/ESNativeMigrateToolTests.java @@ -13,7 +13,6 @@ import org.elasticsearch.common.settings.Settings; import org.elasticsearch.env.Environment; import org.elasticsearch.test.NativeRealmIntegTestCase; -import org.elasticsearch.test.SecuritySettingsSource; import org.elasticsearch.common.CharArrays; import org.elasticsearch.xpack.core.security.client.SecurityClient; import org.elasticsearch.xpack.security.support.SecurityIndexManager; @@ -21,10 +20,12 @@ import java.nio.charset.StandardCharsets; import java.nio.file.Path; -import java.util.Arrays; +import java.util.Collections; import java.util.HashSet; import java.util.Set; +import static org.elasticsearch.test.SecuritySettingsSource.addSSLSettingsForNodePEMFiles; +import static org.elasticsearch.test.SecuritySettingsSource.addSSLSettingsForPEMFiles; import static org.hamcrest.Matchers.containsString; import static org.hamcrest.Matchers.is; @@ -49,11 +50,11 @@ protected boolean addMockHttpTransport() { @Override public Settings nodeSettings(int nodeOrdinal) { logger.info("--> use SSL? {}", useSSL); - Settings s = Settings.builder() - .put(super.nodeSettings(nodeOrdinal)) - .put("xpack.security.http.ssl.enabled", useSSL) - .build(); - return s; + Settings.Builder builder = Settings.builder() + .put(super.nodeSettings(nodeOrdinal)); + addSSLSettingsForNodePEMFiles(builder, "xpack.security.http.", true); + builder.put("xpack.security.http.ssl.enabled", useSSL); + return builder.build(); } @Override @@ -77,7 +78,7 @@ public void testRetrieveUsers() throws Exception { SecurityClient c = new SecurityClient(client()); logger.error("--> creating users"); int numToAdd = randomIntBetween(1,10); - Set addedUsers = new HashSet(numToAdd); + Set addedUsers = new HashSet<>(numToAdd); for (int i = 0; i < numToAdd; i++) { String uname = randomAlphaOfLength(5); c.preparePutUser(uname, "s3kirt".toCharArray(), getFastStoredHashAlgoForTests(), "role1", "user").get(); @@ -94,13 +95,15 @@ public void testRetrieveUsers() throws Exception { Settings.Builder builder = Settings.builder() .put("path.home", home) - .put("path.conf", conf.toString()); - SecuritySettingsSource.addSSLSettingsForPEMFiles( + .put("path.conf", conf.toString()) + .put("xpack.security.http.ssl.client_authentication", "none"); + addSSLSettingsForPEMFiles( builder, "/org/elasticsearch/xpack/security/transport/ssl/certs/simple/testnode.pem", "testnode", "/org/elasticsearch/xpack/security/transport/ssl/certs/simple/testnode.crt", - Arrays.asList("/org/elasticsearch/xpack/security/transport/ssl/certs/simple/testnode.crt")); + "xpack.security.http.", + Collections.singletonList("/org/elasticsearch/xpack/security/transport/ssl/certs/simple/testnode.crt")); Settings settings = builder.build(); logger.error("--> retrieving users using URL: {}, home: {}", url, home); @@ -140,12 +143,15 @@ public void testRetrieveRoles() throws Exception { String password = new String(CharArrays.toUtf8Bytes(nodeClientPassword().getChars()), StandardCharsets.UTF_8); String url = getHttpURL(); ESNativeRealmMigrateTool.MigrateUserOrRoles muor = new ESNativeRealmMigrateTool.MigrateUserOrRoles(); - Settings.Builder builder = Settings.builder().put("path.home", home); - SecuritySettingsSource.addSSLSettingsForPEMFiles(builder, + Settings.Builder builder = Settings.builder() + .put("path.home", home) + .put("xpack.security.http.ssl.client_authentication", "none"); + addSSLSettingsForPEMFiles(builder, "/org/elasticsearch/xpack/security/transport/ssl/certs/simple/testclient.pem", "testclient", "/org/elasticsearch/xpack/security/transport/ssl/certs/simple/testclient.crt", - Arrays.asList("/org/elasticsearch/xpack/security/transport/ssl/certs/simple/testnode.crt")); + "xpack.security.http.", + Collections.singletonList("/org/elasticsearch/xpack/security/transport/ssl/certs/simple/testnode.crt")); Settings settings = builder.build(); logger.error("--> retrieving roles using URL: {}, home: {}", url, home); diff --git a/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/authc/esnative/tool/CommandLineHttpClientTests.java b/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/authc/esnative/tool/CommandLineHttpClientTests.java index 9b8c3878a038d..dd4b747c5b19b 100644 --- a/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/authc/esnative/tool/CommandLineHttpClientTests.java +++ b/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/authc/esnative/tool/CommandLineHttpClientTests.java @@ -44,29 +44,16 @@ public void setup() throws Exception { } @After - public void shutdown() throws Exception { + public void shutdown() { webServer.close(); } public void testCommandLineHttpClientCanExecuteAndReturnCorrectResultUsingSSLSettings() throws Exception { Path certPath = getDataPath("/org/elasticsearch/xpack/security/transport/ssl/certs/simple/testnode.crt"); - MockSecureSettings secureSettings = new MockSecureSettings(); - Settings settings; - if (randomBoolean()) { - // with http ssl settings - secureSettings.setString("xpack.security.http.ssl.truststore.secure_password", "testnode"); - settings = Settings.builder().put("xpack.security.http.ssl.certificate_authorities", certPath.toString()) - .put("xpack.security.http.ssl.verification_mode", VerificationMode.CERTIFICATE).setSecureSettings(secureSettings) - .build(); - } else { - // with global settings - secureSettings.setString("xpack.ssl.truststore.secure_password", "testnode"); - settings = Settings.builder() - .put("xpack.ssl.certificate_authorities", certPath.toString()) - .put("xpack.ssl.verification_mode", VerificationMode.CERTIFICATE) - .setSecureSettings(secureSettings) - .build(); - } + Settings settings = Settings.builder() + .put("xpack.security.http.ssl.certificate_authorities", certPath.toString()) + .put("xpack.security.http.ssl.verification_mode", VerificationMode.CERTIFICATE) + .build(); CommandLineHttpClient client = new CommandLineHttpClient(settings, environment); HttpResponse httpResponse = client.execute("GET", new URL("https://localhost:" + webServer.getPort() + "/test"), "u1", new SecureString(new char[]{'p'}), () -> null, is -> responseBuilder(is)); @@ -80,14 +67,14 @@ private MockWebServer createMockWebServer() { Path certPath = getDataPath("/org/elasticsearch/xpack/security/transport/ssl/certs/simple/testnode.crt"); Path keyPath = getDataPath("/org/elasticsearch/xpack/security/transport/ssl/certs/simple/testnode.pem"); MockSecureSettings secureSettings = new MockSecureSettings(); - secureSettings.setString("xpack.ssl.secure_key_passphrase", "testnode"); + secureSettings.setString("xpack.security.http.ssl.secure_key_passphrase", "testnode"); Settings settings = Settings.builder() - .put("xpack.ssl.key", keyPath.toString()) - .put("xpack.ssl.certificate", certPath.toString()) + .put("xpack.security.http.ssl.key", keyPath.toString()) + .put("xpack.security.http.ssl.certificate", certPath.toString()) .setSecureSettings(secureSettings) .build(); TestsSSLService sslService = new TestsSSLService(settings, environment); - return new MockWebServer(sslService.sslContext(), false); + return new MockWebServer(sslService.sslContext("xpack.security.http.ssl."), false); } private HttpResponseBuilder responseBuilder(final InputStream is) throws IOException { diff --git a/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/authc/ldap/LdapTestUtils.java b/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/authc/ldap/LdapTestUtils.java index c91f634c1a786..65eb36aeba73b 100644 --- a/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/authc/ldap/LdapTestUtils.java +++ b/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/authc/ldap/LdapTestUtils.java @@ -13,7 +13,6 @@ import org.elasticsearch.common.settings.Settings; import org.elasticsearch.env.Environment; import org.elasticsearch.env.TestEnvironment; -import org.elasticsearch.test.ESTestCase; import org.elasticsearch.xpack.core.security.authc.ldap.support.SessionFactorySettings; import org.elasticsearch.xpack.core.ssl.SSLConfiguration; import org.elasticsearch.xpack.core.ssl.SSLService; @@ -29,26 +28,16 @@ private LdapTestUtils() { } public static LDAPConnection openConnection(String url, String bindDN, String bindPassword, Path truststore) throws Exception { - boolean useGlobalSSL = ESTestCase.randomBoolean(); Settings.Builder builder = Settings.builder().put("path.home", LuceneTestCase.createTempDir()); MockSecureSettings secureSettings = new MockSecureSettings(); builder.setSecureSettings(secureSettings); - if (useGlobalSSL) { - builder.put("xpack.ssl.truststore.path", truststore); - // fake realm to load config with certificate verification mode - builder.put("xpack.security.authc.realms.ldap.bar.ssl.truststore.path", truststore); - builder.put("xpack.security.authc.realms.ldap.bar.ssl.verification_mode", VerificationMode.CERTIFICATE); - secureSettings.setString("xpack.ssl.truststore.secure_password", "changeit"); - secureSettings.setString("xpack.security.authc.realms.ldap.bar.ssl.truststore.secure_password", "changeit"); - } else { - // fake realms so ssl will get loaded - builder.put("xpack.security.authc.realms.ldap.foo.ssl.truststore.path", truststore); - builder.put("xpack.security.authc.realms.ldap.foo.ssl.verification_mode", VerificationMode.FULL); - builder.put("xpack.security.authc.realms.ldap.bar.ssl.truststore.path", truststore); - builder.put("xpack.security.authc.realms.ldap.bar.ssl.verification_mode", VerificationMode.CERTIFICATE); - secureSettings.setString("xpack.security.authc.realms.ldap.foo.ssl.truststore.secure_password", "changeit"); - secureSettings.setString("xpack.security.authc.realms.ldap.bar.ssl.truststore.secure_password", "changeit"); - } + // fake realms so ssl will get loaded + builder.put("xpack.security.authc.realms.ldap.foo.ssl.truststore.path", truststore); + builder.put("xpack.security.authc.realms.ldap.foo.ssl.verification_mode", VerificationMode.FULL); + builder.put("xpack.security.authc.realms.ldap.bar.ssl.truststore.path", truststore); + builder.put("xpack.security.authc.realms.ldap.bar.ssl.verification_mode", VerificationMode.CERTIFICATE); + secureSettings.setString("xpack.security.authc.realms.ldap.foo.ssl.truststore.secure_password", "changeit"); + secureSettings.setString("xpack.security.authc.realms.ldap.bar.ssl.truststore.secure_password", "changeit"); Settings settings = builder.build(); Environment env = TestEnvironment.newEnvironment(settings); SSLService sslService = new SSLService(settings, env); @@ -60,12 +49,7 @@ public static LDAPConnection openConnection(String url, String bindDN, String bi options.setConnectTimeoutMillis(Math.toIntExact(SessionFactorySettings.TIMEOUT_DEFAULT.millis())); options.setResponseTimeoutMillis(SessionFactorySettings.TIMEOUT_DEFAULT.millis()); - final SSLConfiguration sslConfiguration; - if (useGlobalSSL) { - sslConfiguration = sslService.getSSLConfiguration("xpack.ssl"); - } else { - sslConfiguration = sslService.getSSLConfiguration("xpack.security.authc.realms.ldap.foo.ssl"); - } + final SSLConfiguration sslConfiguration = sslService.getSSLConfiguration("xpack.security.authc.realms.ldap.foo.ssl"); return LdapUtils.privilegedConnect(() -> new LDAPConnection(sslService.sslSocketFactory(sslConfiguration), options, ldapurl.getHost(), ldapurl.getPort(), bindDN, bindPassword)); } diff --git a/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/authc/ldap/LdapUserSearchSessionFactoryTests.java b/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/authc/ldap/LdapUserSearchSessionFactoryTests.java index 29a96f11060e3..2598b9da5507f 100644 --- a/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/authc/ldap/LdapUserSearchSessionFactoryTests.java +++ b/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/authc/ldap/LdapUserSearchSessionFactoryTests.java @@ -65,7 +65,7 @@ public void init() throws Exception { globalSettings = Settings.builder() .put("path.home", createTempDir()) - .put("xpack.ssl.certificate_authorities", certPath) + .put("xpack.security.transport.ssl.certificate_authorities", certPath) .build(); sslService = new SSLService(globalSettings, env); threadPool = new TestThreadPool("LdapUserSearchSessionFactoryTests"); diff --git a/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/authc/pki/PkiAuthenticationTests.java b/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/authc/pki/PkiAuthenticationTests.java index 7cd6833260ac4..52c87c75a13a7 100644 --- a/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/authc/pki/PkiAuthenticationTests.java +++ b/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/authc/pki/PkiAuthenticationTests.java @@ -18,7 +18,6 @@ import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.transport.TransportAddress; import org.elasticsearch.http.HttpServerTransport; -import org.elasticsearch.test.SecuritySettingsSource; import org.elasticsearch.test.SecuritySingleNodeTestCase; import org.elasticsearch.transport.Transport; import org.elasticsearch.xpack.core.TestXPackTransportClient; @@ -41,6 +40,7 @@ import java.util.Locale; import java.util.stream.Collectors; +import static org.elasticsearch.test.SecuritySettingsSource.addSSLSettingsForNodePEMFiles; import static org.elasticsearch.test.SecuritySettingsSource.addSSLSettingsForPEMFiles; import static org.hamcrest.Matchers.containsString; import static org.hamcrest.Matchers.is; @@ -60,17 +60,15 @@ protected Settings nodeSettings() { SSLClientAuth sslClientAuth = randomBoolean() ? SSLClientAuth.REQUIRED : SSLClientAuth.OPTIONAL; Settings.Builder builder = Settings.builder() - .put(super.nodeSettings()) - .put("xpack.security.http.ssl.enabled", true) + .put(super.nodeSettings()); + addSSLSettingsForNodePEMFiles(builder, "xpack.security.http.", true); + builder.put("xpack.security.http.ssl.enabled", true) .put("xpack.security.http.ssl.client_authentication", sslClientAuth) .put("xpack.security.authc.realms.file.file.order", "0") .put("xpack.security.authc.realms.pki.pki1.order", "1") .put("xpack.security.authc.realms.pki.pki1.certificate_authorities", getDataPath("/org/elasticsearch/xpack/security/transport/ssl/certs/simple/testnode.crt")) .put("xpack.security.authc.realms.pki.pki1.files.role_mapping", getDataPath("role_mapping.yml")); - - SecuritySettingsSource.addSecureSettings(builder, secureSettings -> - secureSettings.setString("xpack.security.authc.realms.pki.pki1.truststore.secure_password", "truststore-testnode-only")); return builder.build(); } @@ -158,13 +156,15 @@ private SSLContext getRestSSLContext(String keyPath, String password, String cer private TransportClient createTransportClient(Settings additionalSettings) { Settings clientSettings = transportClientSettings(); - if (additionalSettings.getByPrefix("xpack.ssl.").isEmpty() == false) { - clientSettings = clientSettings.filter(k -> k.startsWith("xpack.ssl.") == false); + if (additionalSettings.getByPrefix("xpack.security.transport.ssl.").isEmpty() == false) { + clientSettings = clientSettings.filter(k -> k.startsWith("xpack.security.transport.ssl.") == false); } - Settings.Builder builder = Settings.builder().put(clientSettings, false) - .put(additionalSettings) - .put("cluster.name", node().settings().get("cluster.name")); + Settings.Builder builder = Settings.builder() + .put("xpack.security.transport.ssl.enabled", true) + .put(clientSettings, false) + .put(additionalSettings) + .put("cluster.name", node().settings().get("cluster.name")); builder.remove(SecurityField.USER_SETTING.getKey()); builder.remove("request.headers.Authorization"); return new TestXPackTransportClient(builder.build(), LocalStateSecurity.class); diff --git a/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/authc/pki/PkiOptionalClientAuthTests.java b/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/authc/pki/PkiOptionalClientAuthTests.java index 6e1a2480d2bcb..f2a6212107307 100644 --- a/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/authc/pki/PkiOptionalClientAuthTests.java +++ b/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/authc/pki/PkiOptionalClientAuthTests.java @@ -52,6 +52,9 @@ protected Settings nodeSettings() { .put(super.nodeSettings()) .put("xpack.security.http.ssl.enabled", true) .put("xpack.security.http.ssl.client_authentication", SSLClientAuth.OPTIONAL) + .put("xpack.security.http.ssl.keystore.path", + getDataPath("/org/elasticsearch/xpack/security/transport/ssl/certs/simple/testnode.jks")) + .put("xpack.security.http.ssl.keystore.password", "testnode") .put("xpack.security.authc.realms.file.file.order", "0") .put("xpack.security.authc.realms.pki.pki1.order", "1") .put("xpack.security.authc.realms.pki.pki1.truststore.path", @@ -59,10 +62,15 @@ protected Settings nodeSettings() { .put("xpack.security.authc.realms.pki.pki1.files.role_mapping", getDataPath("role_mapping.yml")) .put("transport.profiles.want_client_auth.port", randomClientPortRange) .put("transport.profiles.want_client_auth.bind_host", "localhost") + .put("transport.profiles.want_client_auth.xpack.security.ssl.keystore.path", + getDataPath("/org/elasticsearch/xpack/security/transport/ssl/certs/simple/testnode.jks")) + .put("transport.profiles.want_client_auth.xpack.security.ssl.keystore.password", "testnode") .put("transport.profiles.want_client_auth.xpack.security.ssl.client_authentication", SSLClientAuth.OPTIONAL); - SecuritySettingsSource.addSecureSettings(builder, secureSettings -> - secureSettings.setString("xpack.security.authc.realms.pki.pki1.truststore.secure_password", "truststore-testnode-only")); + SecuritySettingsSource.addSecureSettings(builder, secureSettings -> { + secureSettings.setString("xpack.security.authc.realms.pki.pki1.truststore.secure_password", "truststore-testnode-only"); + secureSettings.setString("xpack.security.http.ssl.keystore.secure_password", "testnode"); + }); return builder.build(); } diff --git a/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/authc/saml/SamlRealmTests.java b/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/authc/saml/SamlRealmTests.java index b528008273779..d139d99bf9ce2 100644 --- a/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/authc/saml/SamlRealmTests.java +++ b/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/authc/saml/SamlRealmTests.java @@ -123,19 +123,20 @@ public void testReadIdpMetadataFromHttps() throws Exception { final Path path = getDataPath("idp1.xml"); final String body = new String(Files.readAllBytes(path), StandardCharsets.UTF_8); final MockSecureSettings mockSecureSettings = new MockSecureSettings(); - mockSecureSettings.setString("xpack.ssl.secure_key_passphrase", "testnode"); + mockSecureSettings.setString("xpack.security.http.ssl.secure_key_passphrase", "testnode"); final Settings settings = Settings.builder() - .put("xpack.ssl.key", + .put("xpack.security.http.ssl.key", getDataPath("/org/elasticsearch/xpack/security/transport/ssl/certs/simple/testnode.pem")) - .put("xpack.ssl.certificate", + .put("xpack.security.http.ssl.certificate", getDataPath("/org/elasticsearch/xpack/security/transport/ssl/certs/simple/testnode.crt")) - .put("xpack.ssl.certificate_authorities", + .put("xpack.security.http.ssl.certificate_authorities", getDataPath("/org/elasticsearch/xpack/security/transport/ssl/certs/simple/testnode.crt")) .put("path.home", createTempDir()) .setSecureSettings(mockSecureSettings) .build(); TestsSSLService sslService = new TestsSSLService(settings, TestEnvironment.newEnvironment(settings)); - try (MockWebServer proxyServer = new MockWebServer(sslService.sslContext(Settings.EMPTY), false)) { + try (MockWebServer proxyServer = + new MockWebServer(sslService.sslContext("xpack.security.http.ssl"), false)) { proxyServer.start(); proxyServer.enqueue(new MockResponse().setResponseCode(200).setBody(body).addHeader("Content-Type", "application/xml")); proxyServer.enqueue(new MockResponse().setResponseCode(200).setBody(body).addHeader("Content-Type", "application/xml")); diff --git a/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/transport/AbstractSimpleSecurityTransportTestCase.java b/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/transport/AbstractSimpleSecurityTransportTestCase.java index e01fecf97e1b3..bd7030f22e50b 100644 --- a/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/transport/AbstractSimpleSecurityTransportTestCase.java +++ b/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/transport/AbstractSimpleSecurityTransportTestCase.java @@ -82,11 +82,11 @@ protected SSLService createSSLService(Settings settings) { Path testnodeCert = getDataPath("/org/elasticsearch/xpack/security/transport/ssl/certs/simple/testnode.crt"); Path testnodeKey = getDataPath("/org/elasticsearch/xpack/security/transport/ssl/certs/simple/testnode.pem"); MockSecureSettings secureSettings = new MockSecureSettings(); - secureSettings.setString("xpack.ssl.secure_key_passphrase", "testnode"); + secureSettings.setString("xpack.security.transport.ssl.secure_key_passphrase", "testnode"); Settings settings1 = Settings.builder() .put("xpack.security.transport.ssl.enabled", true) - .put("xpack.ssl.key", testnodeKey) - .put("xpack.ssl.certificate", testnodeCert) + .put("xpack.security.transport.ssl.key", testnodeKey) + .put("xpack.security.transport.ssl.certificate", testnodeCert) .put("path.home", createTempDir()) .put(settings) .setSecureSettings(secureSettings) @@ -153,7 +153,7 @@ public void testTcpHandshake() { @SuppressForbidden(reason = "Need to open socket connection") public void testRenegotiation() throws Exception { SSLService sslService = createSSLService(); - final SSLConfiguration sslConfiguration = sslService.getSSLConfiguration("xpack.ssl"); + final SSLConfiguration sslConfiguration = sslService.getSSLConfiguration("xpack.security.transport.ssl"); SocketFactory factory = sslService.sslSocketFactory(sslConfiguration); try (SSLSocket socket = (SSLSocket) factory.createSocket()) { SocketAccess.doPrivileged(() -> socket.connect(serviceA.boundAddress().publishAddress().address())); @@ -205,7 +205,7 @@ public void testSNIServerNameIsPropagated() throws Exception { assumeFalse("Can't run in a FIPS JVM, TrustAllConfig is not a SunJSSE TrustManagers", inFipsJvm()); SSLService sslService = createSSLService(); - final SSLConfiguration sslConfiguration = sslService.getSSLConfiguration("xpack.ssl"); + final SSLConfiguration sslConfiguration = sslService.getSSLConfiguration("xpack.security.transport.ssl"); SSLContext sslContext = sslService.sslContext(sslConfiguration); final SSLServerSocketFactory serverSocketFactory = sslContext.getServerSocketFactory(); final String sniIp = "sni-hostname"; @@ -245,7 +245,9 @@ public boolean matches(SNIServerName sniServerName) { InetSocketAddress serverAddress = (InetSocketAddress) SocketAccess.doPrivileged(sslServerSocket::getLocalSocketAddress); - Settings settings = Settings.builder().put("name", "TS_TEST").put("xpack.ssl.verification_mode", "none").build(); + Settings settings = Settings.builder().put("name", "TS_TEST") + .put("xpack.security.transport.ssl.verification_mode", "none") + .build(); try (MockTransportService serviceC = build(settings, version0, null, true)) { serviceC.acceptIncomingRequests(); @@ -271,7 +273,7 @@ public void testInvalidSNIServerName() throws Exception { assumeFalse("Can't run in a FIPS JVM, TrustAllConfig is not a SunJSSE TrustManagers", inFipsJvm()); SSLService sslService = createSSLService(); - final SSLConfiguration sslConfiguration = sslService.getSSLConfiguration("xpack.ssl"); + final SSLConfiguration sslConfiguration = sslService.getSSLConfiguration("xpack.security.transport.ssl"); SSLContext sslContext = sslService.sslContext(sslConfiguration); final SSLServerSocketFactory serverSocketFactory = sslContext.getServerSocketFactory(); final String sniIp = "invalid_hostname"; @@ -290,7 +292,9 @@ public void testInvalidSNIServerName() throws Exception { InetSocketAddress serverAddress = (InetSocketAddress) SocketAccess.doPrivileged(sslServerSocket::getLocalSocketAddress); - Settings settings = Settings.builder().put("name", "TS_TEST").put("xpack.ssl.verification_mode", "none").build(); + Settings settings = Settings.builder().put("name", "TS_TEST") + .put("xpack.security.transport.ssl.verification_mode", "none") + .build(); try (MockTransportService serviceC = build(settings, version0, null, true)) { serviceC.acceptIncomingRequests(); diff --git a/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/transport/ServerTransportFilterIntegrationTests.java b/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/transport/ServerTransportFilterIntegrationTests.java index 6e95fd6aed170..2383f3b3ac739 100644 --- a/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/transport/ServerTransportFilterIntegrationTests.java +++ b/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/transport/ServerTransportFilterIntegrationTests.java @@ -32,7 +32,6 @@ import org.elasticsearch.transport.TransportService; import org.elasticsearch.xpack.core.XPackSettings; import org.elasticsearch.xpack.core.security.SecurityField; -import org.elasticsearch.xpack.core.ssl.SSLClientAuth; import org.elasticsearch.xpack.security.LocalStateSecurity; import org.junit.BeforeClass; @@ -41,8 +40,10 @@ import java.nio.file.Path; import java.util.Arrays; import java.util.Collection; +import java.util.Collections; import java.util.concurrent.CountDownLatch; +import static org.elasticsearch.test.SecuritySettingsSource.addSSLSettingsForNodePEMFiles; import static org.elasticsearch.test.SecuritySettingsSource.addSSLSettingsForPEMFiles; import static org.elasticsearch.xpack.security.test.SecurityTestUtils.writeFile; import static org.hamcrest.CoreMatchers.equalTo; @@ -63,13 +64,12 @@ public boolean transportSSLEnabled() { @Override protected Settings nodeSettings(int nodeOrdinal) { - Settings.Builder settingsBuilder = Settings.builder(); + Settings.Builder settingsBuilder = Settings.builder().put(super.nodeSettings(nodeOrdinal)); String randomClientPortRange = randomClientPort + "-" + (randomClientPort+100); + addSSLSettingsForNodePEMFiles(settingsBuilder, "transport.profiles.client.xpack.security.", true); Path certPath = getDataPath("/org/elasticsearch/xpack/security/transport/ssl/certs/simple/testnode.crt"); - settingsBuilder.put(super.nodeSettings(nodeOrdinal)) - .putList("transport.profiles.client.xpack.security.ssl.certificate_authorities", - Arrays.asList(certPath.toString())) // settings for client truststore - .put("xpack.ssl.client_authentication", SSLClientAuth.REQUIRED) + settingsBuilder.putList("transport.profiles.client.xpack.security.ssl.certificate_authorities", + Collections.singletonList(certPath.toString())) // settings for client truststore .put("transport.profiles.client.xpack.security.type", "client") .put("transport.profiles.client.port", randomClientPortRange) // make sure this is "localhost", no matter if ipv4 or ipv6, but be consistent @@ -82,7 +82,7 @@ protected Settings nodeSettings(int nodeOrdinal) { } SecuritySettingsSource.addSecureSettings(settingsBuilder, secureSettings -> - secureSettings.setString("transport.profiles.client.xpack.security.ssl.truststore.secure_password", "testnode")); + secureSettings.setString("transport.profiles.client.xpack.security.ssl.keystore.secure_password", "testnode")); return settingsBuilder.build(); } @@ -97,20 +97,20 @@ public void testThatConnectionToServerTypeConnectionWorks() throws IOException, // test that starting up a node works Settings.Builder nodeSettings = Settings.builder() - .put("node.name", "my-test-node") - .put("network.host", "localhost") - .put("cluster.name", internalCluster().getClusterName()) - .put("discovery.zen.ping.unicast.hosts", unicastHost) - .put("discovery.zen.minimum_master_nodes", - internalCluster().getInstance(Settings.class).get("discovery.zen.minimum_master_nodes")) - .put("xpack.security.enabled", true) - .put("xpack.security.audit.enabled", false) - .put(XPackSettings.WATCHER_ENABLED.getKey(), false) - .put("path.home", home) - .put(Node.NODE_MASTER_SETTING.getKey(), false) - .put(TestZenDiscovery.USE_ZEN2.getKey(), getUseZen2()) - .put(TestZenDiscovery.USE_MOCK_PINGS.getKey(), false); - //.put("xpack.ml.autodetect_process", false); + .put("node.name", "my-test-node") + .put("network.host", "localhost") + .put("cluster.name", internalCluster().getClusterName()) + .put("discovery.zen.ping.unicast.hosts", unicastHost) + .put("discovery.zen.minimum_master_nodes", + internalCluster().getInstance(Settings.class).get("discovery.zen.minimum_master_nodes")) + .put("xpack.security.enabled", true) + .put("xpack.security.audit.enabled", false) + .put("xpack.security.transport.ssl.enabled", true) + .put(XPackSettings.WATCHER_ENABLED.getKey(), false) + .put("path.home", home) + .put(Node.NODE_MASTER_SETTING.getKey(), false) + .put(TestZenDiscovery.USE_ZEN2.getKey(), getUseZen2()) + .put(TestZenDiscovery.USE_MOCK_PINGS.getKey(), false); Collection> mockPlugins = Arrays.asList( LocalStateSecurity.class, TestZenDiscovery.TestPlugin.class, MockHttpTransport.TestPlugin.class); addSSLSettingsForPEMFiles( @@ -139,22 +139,22 @@ public void testThatConnectionToClientTypeConnectionIsRejected() throws IOExcept // test that starting up a node works Settings.Builder nodeSettings = Settings.builder() - .put("xpack.security.authc.realms.file.file.order", 0) - .put("node.name", "my-test-node") - .put(SecurityField.USER_SETTING.getKey(), "test_user:" + SecuritySettingsSourceField.TEST_PASSWORD) - .put("cluster.name", internalCluster().getClusterName()) - .put("discovery.zen.ping.unicast.hosts", unicastHost) - .put("discovery.zen.minimum_master_nodes", - internalCluster().getInstance(Settings.class).get("discovery.zen.minimum_master_nodes")) - .put("xpack.security.enabled", true) - .put("xpack.security.audit.enabled", false) - .put(XPackSettings.WATCHER_ENABLED.getKey(), false) - .put("discovery.initial_state_timeout", "0s") - .put("path.home", home) - .put(Node.NODE_MASTER_SETTING.getKey(), false) - .put(TestZenDiscovery.USE_ZEN2.getKey(), getUseZen2()) - .put(TestZenDiscovery.USE_MOCK_PINGS.getKey(), false); - //.put("xpack.ml.autodetect_process", false); + .put("xpack.security.authc.realms.file.file.order", 0) + .put("node.name", "my-test-node") + .put(SecurityField.USER_SETTING.getKey(), "test_user:" + SecuritySettingsSourceField.TEST_PASSWORD) + .put("cluster.name", internalCluster().getClusterName()) + .put("discovery.zen.ping.unicast.hosts", unicastHost) + .put("discovery.zen.minimum_master_nodes", + internalCluster().getInstance(Settings.class).get("discovery.zen.minimum_master_nodes")) + .put("xpack.security.enabled", true) + .put("xpack.security.audit.enabled", false) + .put("xpack.security.transport.ssl.enabled", true) + .put(XPackSettings.WATCHER_ENABLED.getKey(), false) + .put("discovery.initial_state_timeout", "0s") + .put("path.home", home) + .put(Node.NODE_MASTER_SETTING.getKey(), false) + .put(TestZenDiscovery.USE_ZEN2.getKey(), getUseZen2()) + .put(TestZenDiscovery.USE_MOCK_PINGS.getKey(), false); Collection> mockPlugins = Arrays.asList( LocalStateSecurity.class, TestZenDiscovery.TestPlugin.class, MockHttpTransport.TestPlugin.class); addSSLSettingsForPEMFiles( @@ -162,7 +162,7 @@ public void testThatConnectionToClientTypeConnectionIsRejected() throws IOExcept "/org/elasticsearch/xpack/security/transport/ssl/certs/simple/testnode.pem", "testnode", "/org/elasticsearch/xpack/security/transport/ssl/certs/simple/testnode.crt", - Arrays.asList("/org/elasticsearch/xpack/security/transport/ssl/certs/simple/testnode.crt")); + Collections.singletonList("/org/elasticsearch/xpack/security/transport/ssl/certs/simple/testnode.crt")); try (Node node = new MockNode(nodeSettings.build(), mockPlugins)) { node.start(); TransportService instance = node.injector().getInstance(TransportService.class); diff --git a/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/transport/netty4/IPHostnameVerificationTests.java b/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/transport/netty4/IPHostnameVerificationTests.java index 130fa22603940..b1ecad0e4b4a5 100644 --- a/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/transport/netty4/IPHostnameVerificationTests.java +++ b/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/transport/netty4/IPHostnameVerificationTests.java @@ -33,7 +33,7 @@ protected boolean transportSSLEnabled() { protected Settings nodeSettings(int nodeOrdinal) { Settings settings = super.nodeSettings(nodeOrdinal); Settings.Builder builder = Settings.builder() - .put(settings.filter((s) -> s.startsWith("xpack.ssl.") == false), false); + .put(settings.filter((s) -> s.startsWith("xpack.security.transport.ssl.") == false), false); settings = builder.build(); // The default Unicast test behavior is to use 'localhost' with the port number. For this test we need to use IP @@ -56,27 +56,27 @@ protected Settings nodeSettings(int nodeOrdinal) { } SecuritySettingsSource.addSecureSettings(settingsBuilder, secureSettings -> { - secureSettings.setString("xpack.ssl.secure_key_passphrase", "testnode-ip-only"); + secureSettings.setString("xpack.security.transport.ssl.secure_key_passphrase", "testnode-ip-only"); }); - return settingsBuilder.put("xpack.ssl.key", keyPath.toAbsolutePath()) - .put("xpack.ssl.certificate", certPath.toAbsolutePath()) - .put("xpack.ssl.certificate_authorities", certPath.toAbsolutePath()) + return settingsBuilder.put("xpack.security.transport.ssl.key", keyPath.toAbsolutePath()) + .put("xpack.security.transport.ssl.certificate", certPath.toAbsolutePath()) + .put("xpack.security.transport.ssl.certificate_authorities", certPath.toAbsolutePath()) .put(TransportSettings.BIND_HOST.getKey(), "127.0.0.1") .put("network.host", "127.0.0.1") - .put("xpack.ssl.client_authentication", SSLClientAuth.NONE) - .put("xpack.ssl.verification_mode", "full") + .put("xpack.security.transport.ssl.client_authentication", SSLClientAuth.NONE) + .put("xpack.security.transport.ssl.verification_mode", "full") .build(); } @Override protected Settings transportClientSettings() { Settings clientSettings = super.transportClientSettings(); - return Settings.builder().put(clientSettings.filter(k -> k.startsWith("xpack.ssl.") == false)) - .put("xpack.ssl.verification_mode", "certificate") - .put("xpack.ssl.key", keyPath.toAbsolutePath()) - .put("xpack.ssl.certificate", certPath.toAbsolutePath()) - .put("xpack.ssl.key_passphrase", "testnode-ip-only") - .put("xpack.ssl.certificate_authorities", certPath) + return Settings.builder().put(clientSettings.filter(k -> k.startsWith("xpack.security.transport.ssl.") == false)) + .put("xpack.security.transport.ssl.verification_mode", "certificate") + .put("xpack.security.transport.ssl.key", keyPath.toAbsolutePath()) + .put("xpack.security.transport.ssl.certificate", certPath.toAbsolutePath()) + .put("xpack.security.transport.ssl.key_passphrase", "testnode-ip-only") + .put("xpack.security.transport.ssl.certificate_authorities", certPath) .build(); } diff --git a/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/transport/netty4/SecurityNetty4HttpServerTransportTests.java b/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/transport/netty4/SecurityNetty4HttpServerTransportTests.java index 23ca3c1fe9fe4..20ceee5d52e92 100644 --- a/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/transport/netty4/SecurityNetty4HttpServerTransportTests.java +++ b/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/transport/netty4/SecurityNetty4HttpServerTransportTests.java @@ -42,15 +42,15 @@ public class SecurityNetty4HttpServerTransportTests extends ESTestCase { private Path testnodeCert; private Path testnodeKey; @Before - public void createSSLService() throws Exception { + public void createSSLService() { testnodeCert = getDataPath("/org/elasticsearch/xpack/security/transport/ssl/certs/simple/testnode.crt"); testnodeKey = getDataPath("/org/elasticsearch/xpack/security/transport/ssl/certs/simple/testnode.pem"); MockSecureSettings secureSettings = new MockSecureSettings(); - secureSettings.setString("xpack.ssl.secure_key_passphrase", "testnode"); + secureSettings.setString("xpack.security.http.ssl.secure_key_passphrase", "testnode"); Settings settings = Settings.builder() - .put("xpack.ssl.key", testnodeKey) - .put("xpack.ssl.certificate", testnodeCert) + .put("xpack.security.http.ssl.key", testnodeKey) + .put("xpack.security.http.ssl.certificate", testnodeCert) .put("path.home", createTempDir()) .setSecureSettings(secureSettings) .build(); @@ -149,7 +149,7 @@ public void testCustomSSLConfiguration() throws Exception { public void testThatExceptionIsThrownWhenConfiguredWithoutSslKey() throws Exception { Settings settings = Settings.builder() - .put("xpack.ssl.certificate_authorities", testnodeCert) + .put("xpack.security.http.ssl.certificate_authorities", testnodeCert) .put(XPackSettings.HTTP_SSL_ENABLED.getKey(), true) .put("path.home", createTempDir()) .build(); @@ -163,10 +163,10 @@ public void testThatExceptionIsThrownWhenConfiguredWithoutSslKey() throws Except public void testNoExceptionWhenConfiguredWithoutSslKeySSLDisabled() throws Exception { MockSecureSettings secureSettings = new MockSecureSettings(); - secureSettings.setString("xpack.ssl.secure_key_passphrase", "testnode"); + secureSettings.setString("xpack.security.http.ssl.secure_key_passphrase", "testnode"); Settings settings = Settings.builder() - .put("xpack.ssl.key", testnodeKey) - .put("xpack.ssl.certificate", testnodeCert) + .put("xpack.security.http.ssl.key", testnodeKey) + .put("xpack.security.http.ssl.certificate", testnodeCert) .setSecureSettings(secureSettings) .put("path.home", createTempDir()) .build(); diff --git a/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/transport/netty4/SecurityNetty4ServerTransportTests.java b/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/transport/netty4/SecurityNetty4ServerTransportTests.java index 32f9828011849..f552c586409e9 100644 --- a/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/transport/netty4/SecurityNetty4ServerTransportTests.java +++ b/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/transport/netty4/SecurityNetty4ServerTransportTests.java @@ -21,11 +21,9 @@ import org.elasticsearch.threadpool.ThreadPool; import org.elasticsearch.xpack.core.security.transport.netty4.SecurityNetty4Transport; import org.elasticsearch.xpack.core.ssl.SSLClientAuth; -import org.elasticsearch.xpack.core.ssl.SSLConfiguration; import org.elasticsearch.xpack.core.ssl.SSLService; import org.junit.Before; -import javax.net.ssl.SSLEngine; import java.nio.file.Path; import java.util.Collections; import java.util.Locale; @@ -44,11 +42,11 @@ public void createSSLService() throws Exception { Path testnodeCert = getDataPath("/org/elasticsearch/xpack/security/transport/ssl/certs/simple/testnode.crt"); Path testnodeKey = getDataPath("/org/elasticsearch/xpack/security/transport/ssl/certs/simple/testnode.pem"); MockSecureSettings secureSettings = new MockSecureSettings(); - secureSettings.setString("xpack.ssl.secure_key_passphrase", "testnode"); + secureSettings.setString("xpack.security.transport.ssl.secure_key_passphrase", "testnode"); Settings settings = Settings.builder() .put("xpack.security.transport.ssl.enabled", true) - .put("xpack.ssl.key", testnodeKey) - .put("xpack.ssl.certificate", testnodeCert) + .put("xpack.security.transport.ssl.key", testnodeKey) + .put("xpack.security.transport.ssl.certificate", testnodeCert) .setSecureSettings(secureSettings) .put("path.home", createTempDir()) .build(); @@ -97,7 +95,7 @@ public void testRequiredClientAuth() throws Exception { String value = randomFrom(SSLClientAuth.REQUIRED.name(), SSLClientAuth.REQUIRED.name().toLowerCase(Locale.ROOT)); Settings settings = Settings.builder() .put(env.settings()) - .put("xpack.ssl.client_authentication", value) + .put("xpack.security.transport.ssl.client_authentication", value) .build(); sslService = new SSLService(settings, env); SecurityNetty4Transport transport = createTransport(settings); @@ -111,7 +109,7 @@ public void testNoClientAuth() throws Exception { String value = randomFrom(SSLClientAuth.NONE.name(), SSLClientAuth.NONE.name().toLowerCase(Locale.ROOT)); Settings settings = Settings.builder() .put(env.settings()) - .put("xpack.ssl.client_authentication", value) + .put("xpack.security.transport.ssl.client_authentication", value) .build(); sslService = new SSLService(settings, env); SecurityNetty4Transport transport = createTransport(settings); @@ -125,7 +123,7 @@ public void testOptionalClientAuth() throws Exception { String value = randomFrom(SSLClientAuth.OPTIONAL.name(), SSLClientAuth.OPTIONAL.name().toLowerCase(Locale.ROOT)); Settings settings = Settings.builder() .put(env.settings()) - .put("xpack.ssl.client_authentication", value) + .put("xpack.security.transport.ssl.client_authentication", value) .build(); sslService = new SSLService(settings, env); SecurityNetty4Transport transport = createTransport(settings); @@ -179,36 +177,4 @@ public void testProfileOptionalClientAuth() throws Exception { assertThat(ch.pipeline().get(SslHandler.class).engine().getNeedClientAuth(), is(false)); assertThat(ch.pipeline().get(SslHandler.class).engine().getWantClientAuth(), is(true)); } - - public void testTransportSSLOverridesGlobalSSL() throws Exception { - MockSecureSettings secureSettings = new MockSecureSettings(); - secureSettings.setString("xpack.security.transport.ssl.secure_key_passphrase", "testnode"); - Settings.Builder builder = Settings.builder() - .put("xpack.security.transport.ssl.enabled", true) - .put("xpack.security.transport.ssl.key", - getDataPath("/org/elasticsearch/xpack/security/transport/ssl/certs/simple/testnode.pem")) - .put("xpack.security.transport.ssl.certificate", - getDataPath("/org/elasticsearch/xpack/security/transport/ssl/certs/simple/testnode.crt")) - .put("xpack.security.transport.ssl.client_authentication", "none") - .put("xpack.ssl.certificate_authorities", - getDataPath("/org/elasticsearch/xpack/security/transport/ssl/certs/simple/testnode.crt")) - .setSecureSettings(secureSettings) - .put("path.home", createTempDir()); - Settings settings = builder.build(); - env = TestEnvironment.newEnvironment(settings); - sslService = new SSLService(settings, env); - SecurityNetty4Transport transport = createTransport(settings); - final ChannelHandler handler = transport.getServerChannelInitializer("default"); - final EmbeddedChannel ch = new EmbeddedChannel(handler); - final SSLEngine engine = ch.pipeline().get(SslHandler.class).engine(); - assertFalse(engine.getNeedClientAuth()); - assertFalse(engine.getWantClientAuth()); - - // get the global and verify that it is different in that it requires client auth - SSLConfiguration configuration = sslService.getSSLConfiguration("xpack.ssl"); - assertNotNull(configuration); - final SSLEngine globalEngine = sslService.createSSLEngine(configuration, null, -1); - assertTrue(globalEngine.getNeedClientAuth()); - assertFalse(globalEngine.getWantClientAuth()); - } } diff --git a/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/transport/netty4/SslHostnameVerificationTests.java b/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/transport/netty4/SslHostnameVerificationTests.java index c61b5782f75c4..30208a1158075 100644 --- a/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/transport/netty4/SslHostnameVerificationTests.java +++ b/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/transport/netty4/SslHostnameVerificationTests.java @@ -35,7 +35,7 @@ protected boolean transportSSLEnabled() { protected Settings nodeSettings(int nodeOrdinal) { Settings settings = super.nodeSettings(nodeOrdinal); Settings.Builder settingsBuilder = Settings.builder(); - settingsBuilder.put(settings.filter(k -> k.startsWith("xpack.ssl.") == false), false); + settingsBuilder.put(settings.filter(k -> k.startsWith("xpack.security.transport.ssl.") == false), false); Path keyPath; Path certPath; Path nodeCertPath; @@ -58,14 +58,15 @@ protected Settings nodeSettings(int nodeOrdinal) { } SecuritySettingsSource.addSecureSettings(settingsBuilder, secureSettings -> { - secureSettings.setString("xpack.ssl.secure_key_passphrase", "testnode-no-subjaltname"); + secureSettings.setString("xpack.security.transport.ssl.secure_key_passphrase", "testnode-no-subjaltname"); }); - return settingsBuilder.put("xpack.ssl.key", keyPath.toAbsolutePath()) - .put("xpack.ssl.certificate", certPath.toAbsolutePath()) - .putList("xpack.ssl.certificate_authorities", Arrays.asList(certPath.toString(), nodeCertPath.toString())) - // disable hostname verification as this test uses certs without a valid SAN or DNS in the CN - .put("xpack.ssl.verification_mode", "certificate") - .build(); + return settingsBuilder.put("xpack.security.transport.ssl.key", keyPath.toAbsolutePath()) + .put("xpack.security.transport.ssl.certificate", certPath.toAbsolutePath()) + .putList("xpack.security.transport.ssl.certificate_authorities", + Arrays.asList(certPath.toString(), nodeCertPath.toString())) + // disable hostname verification as this test uses certs without a valid SAN or DNS in the CN + .put("xpack.security.transport.ssl.verification_mode", "certificate") + .build(); } @Override @@ -89,13 +90,13 @@ protected Settings transportClientSettings() { Settings settings = super.transportClientSettings(); // remove all ssl settings Settings.Builder builder = Settings.builder(); - builder.put(settings.filter( k -> k.startsWith("xpack.ssl.") == false), false); + builder.put(settings.filter(k -> k.startsWith("xpack.security.transport.ssl.") == false), false); - builder.put("xpack.ssl.verification_mode", "certificate") - .put("xpack.ssl.key", keyPath.toAbsolutePath()) - .put("xpack.ssl.key_passphrase", "testnode-no-subjaltname") - .put("xpack.ssl.certificate", certPath.toAbsolutePath()) - .putList("xpack.ssl.certificate_authorities", Arrays.asList(certPath.toString(), nodeCertPath.toString())); + builder.put("xpack.security.transport.ssl.verification_mode", "certificate") + .put("xpack.security.transport.ssl.key", keyPath.toAbsolutePath()) + .put("xpack.security.transport.ssl.key_passphrase", "testnode-no-subjaltname") + .put("xpack.security.transport.ssl.certificate", certPath.toAbsolutePath()) + .putList("xpack.security.transport.ssl.certificate_authorities", Arrays.asList(certPath.toString(), nodeCertPath.toString())); return builder.build(); } @@ -105,7 +106,7 @@ public void testThatHostnameMismatchDeniesTransportClientConnection() throws Exc InetSocketAddress inetSocketAddress = transportAddress.address(); Settings settings = Settings.builder().put(transportClientSettings()) - .put("xpack.ssl.verification_mode", "full") + .put("xpack.security.transport.ssl.verification_mode", "full") .build(); try (TransportClient client = new TestXPackTransportClient(settings, LocalStateSecurity.class)) { diff --git a/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/transport/nio/SecurityNioHttpServerTransportTests.java b/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/transport/nio/SecurityNioHttpServerTransportTests.java index 9c490176eccdb..2f26456112a67 100644 --- a/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/transport/nio/SecurityNioHttpServerTransportTests.java +++ b/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/transport/nio/SecurityNioHttpServerTransportTests.java @@ -52,10 +52,10 @@ public void createSSLService() { Path testNodeKey = getDataPath("/org/elasticsearch/xpack/security/transport/ssl/certs/simple/testnode.pem"); Path testNodeCert = getDataPath("/org/elasticsearch/xpack/security/transport/ssl/certs/simple/testnode.crt"); MockSecureSettings secureSettings = new MockSecureSettings(); - secureSettings.setString("xpack.ssl.secure_key_passphrase", "testnode"); + secureSettings.setString("xpack.security.http.ssl.secure_key_passphrase", "testnode"); Settings settings = Settings.builder() - .put("xpack.ssl.key", testNodeKey) - .put("xpack.ssl.certificate", testNodeCert) + .put("xpack.security.http.ssl.key", testNodeKey) + .put("xpack.security.http.ssl.certificate", testNodeCert) .put("path.home", createTempDir()) .setSecureSettings(secureSettings) .build(); @@ -173,9 +173,9 @@ public void testCustomSSLConfiguration() throws IOException { public void testThatExceptionIsThrownWhenConfiguredWithoutSslKey() { MockSecureSettings secureSettings = new MockSecureSettings(); - secureSettings.setString("xpack.ssl.truststore.secure_password", "testnode"); + secureSettings.setString("xpack.security.http.ssl.truststore.secure_password", "testnode"); Settings settings = Settings.builder() - .put("xpack.ssl.truststore.path", + .put("xpack.security.http.ssl.truststore.path", getDataPath("/org/elasticsearch/xpack/security/transport/ssl/certs/simple/testnode.jks")) .setSecureSettings(secureSettings) .put(XPackSettings.HTTP_SSL_ENABLED.getKey(), true) @@ -193,9 +193,9 @@ public void testThatExceptionIsThrownWhenConfiguredWithoutSslKey() { public void testNoExceptionWhenConfiguredWithoutSslKeySSLDisabled() { MockSecureSettings secureSettings = new MockSecureSettings(); - secureSettings.setString("xpack.ssl.truststore.secure_password", "testnode"); + secureSettings.setString("xpack.security.http.ssl.truststore.secure_password", "testnode"); Settings settings = Settings.builder() - .put("xpack.ssl.truststore.path", + .put("xpack.security.http.ssl.truststore.path", getDataPath("/org/elasticsearch/xpack/security/transport/ssl/certs/simple/testnode.jks")) .setSecureSettings(secureSettings) .put("path.home", createTempDir()) diff --git a/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/transport/ssl/EllipticCurveSSLTests.java b/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/transport/ssl/EllipticCurveSSLTests.java index df49103a25999..5f0f3c94e36e8 100644 --- a/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/transport/ssl/EllipticCurveSSLTests.java +++ b/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/transport/ssl/EllipticCurveSSLTests.java @@ -42,11 +42,13 @@ protected Settings nodeSettings(int nodeOrdinal) { final Path keyPath = getDataPath("/org/elasticsearch/xpack/security/transport/ssl/certs/simple/prime256v1-key.pem"); final Path certPath = getDataPath("/org/elasticsearch/xpack/security/transport/ssl/certs/simple/prime256v1-cert.pem"); return Settings.builder() - .put(super.nodeSettings(nodeOrdinal).filter(s -> s.startsWith("xpack.ssl") == false)) - .put("xpack.ssl.key", keyPath) - .put("xpack.ssl.certificate", certPath) - .put("xpack.ssl.certificate_authorities", certPath) - .put("xpack.ssl.verification_mode", "certificate") // disable hostname verificate since these certs aren't setup for that + .put(super.nodeSettings(nodeOrdinal).filter(s -> s.startsWith("xpack.security.transport.ssl") == false)) + .put("xpack.security.transport.ssl.enabled", true) + .put("xpack.security.transport.ssl.key", keyPath) + .put("xpack.security.transport.ssl.certificate", certPath) + .put("xpack.security.transport.ssl.certificate_authorities", certPath) + // disable hostname verificate since these certs aren't setup for that + .put("xpack.security.transport.ssl.verification_mode", "certificate") .build(); } @@ -55,11 +57,13 @@ protected Settings transportClientSettings() { final Path keyPath = getDataPath("/org/elasticsearch/xpack/security/transport/ssl/certs/simple/prime256v1-key.pem"); final Path certPath = getDataPath("/org/elasticsearch/xpack/security/transport/ssl/certs/simple/prime256v1-cert.pem"); return Settings.builder() - .put(super.transportClientSettings().filter(s -> s.startsWith("xpack.ssl") == false)) - .put("xpack.ssl.key", keyPath) - .put("xpack.ssl.certificate", certPath) - .put("xpack.ssl.certificate_authorities", certPath) - .put("xpack.ssl.verification_mode", "certificate") // disable hostname verification since these certs aren't setup for that + .put(super.transportClientSettings().filter(s -> s.startsWith("xpack.security.transport.ssl") == false)) + .put("xpack.security.transport.ssl.enabled", true) + .put("xpack.security.transport.ssl.key", keyPath) + .put("xpack.security.transport.ssl.certificate", certPath) + .put("xpack.security.transport.ssl.certificate_authorities", certPath) + // disable hostname verificate since these certs aren't setup for that + .put("xpack.security.transport.ssl.verification_mode", "certificate") .build(); } diff --git a/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/transport/ssl/SslIntegrationTests.java b/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/transport/ssl/SslIntegrationTests.java index c147c660b74a4..5f25213beefa1 100644 --- a/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/transport/ssl/SslIntegrationTests.java +++ b/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/transport/ssl/SslIntegrationTests.java @@ -48,6 +48,7 @@ import javax.net.ssl.SSLHandshakeException; import javax.net.ssl.TrustManagerFactory; +import static org.elasticsearch.test.SecuritySettingsSource.addSSLSettingsForNodePEMFiles; import static org.elasticsearch.test.SecuritySettingsSource.addSSLSettingsForPEMFiles; import static org.hamcrest.CoreMatchers.is; import static org.hamcrest.Matchers.containsString; @@ -61,8 +62,9 @@ protected boolean addMockHttpTransport() { @Override protected Settings nodeSettings(int nodeOrdinal) { - return Settings.builder().put(super.nodeSettings(nodeOrdinal)) - .put("xpack.security.http.ssl.enabled", true).build(); + final Settings.Builder builder = Settings.builder().put(super.nodeSettings(nodeOrdinal)); + addSSLSettingsForNodePEMFiles(builder, "xpack.security.http.", randomBoolean()); + return builder.put("xpack.security.http.ssl.enabled", true).build(); } @Override @@ -82,7 +84,7 @@ public void testThatUnconfiguredCiphersAreRejected() throws Exception { .put(transportClientSettings()) .put("node.name", "programmatic_transport_client") .put("cluster.name", internalCluster().getClusterName()) - .putList("xpack.ssl.cipher_suites", unconfiguredCiphers) + .putList("xpack.security.transport.ssl.cipher_suites", unconfiguredCiphers) .build(), LocalStateSecurity.class)) { TransportAddress transportAddress = randomFrom(internalCluster().getInstance(Transport.class).boundAddress().boundAddresses()); @@ -101,7 +103,7 @@ public void testThatTransportClientUsingSSLv3ProtocolIsRejected() { .put(transportClientSettings()) .put("node.name", "programmatic_transport_client") .put("cluster.name", internalCluster().getClusterName()) - .putList("xpack.ssl.supported_protocols", new String[]{"SSLv3"}) + .putList("xpack.security.transport.ssl.supported_protocols", new String[]{"SSLv3"}) .build(), LocalStateSecurity.class)) { TransportAddress transportAddress = randomFrom(internalCluster().getInstance(Transport.class).boundAddress().boundAddresses()); @@ -120,13 +122,14 @@ public void testThatConnectionToHTTPWorks() throws Exception { builder, "/org/elasticsearch/xpack/security/transport/ssl/certs/simple/testclient.pem", "testclient", "/org/elasticsearch/xpack/security/transport/ssl/certs/simple/testclient.crt", + "xpack.security.http.", Arrays.asList("/org/elasticsearch/xpack/security/transport/ssl/certs/simple/testnode.crt")); SSLService service = new SSLService(builder.build(), null); CredentialsProvider provider = new BasicCredentialsProvider(); provider.setCredentials(AuthScope.ANY, new UsernamePasswordCredentials(nodeClientUsername(), new String(nodeClientPassword().getChars()))); - SSLConfiguration sslConfiguration = service.getSSLConfiguration("xpack.ssl"); + SSLConfiguration sslConfiguration = service.getSSLConfiguration("xpack.security.http.ssl"); try (CloseableHttpClient client = HttpClients.custom() .setSSLSocketFactory(new SSLConnectionSocketFactory(service.sslSocketFactory(sslConfiguration), SSLConnectionSocketFactory.getDefaultHostnameVerifier())) diff --git a/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/transport/ssl/SslMultiPortTests.java b/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/transport/ssl/SslMultiPortTests.java index d3ab5d092ab5b..a948fafb77a52 100644 --- a/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/transport/ssl/SslMultiPortTests.java +++ b/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/transport/ssl/SslMultiPortTests.java @@ -24,6 +24,7 @@ import java.util.Collections; import static org.elasticsearch.test.SecuritySettingsSource.TEST_USER_NAME; +import static org.elasticsearch.test.SecuritySettingsSource.addSSLSettingsForNodePEMFiles; import static org.elasticsearch.test.SecuritySettingsSource.addSSLSettingsForPEMFiles; import static org.elasticsearch.test.SecuritySettingsSourceField.TEST_PASSWORD; import static org.hamcrest.CoreMatchers.is; @@ -61,17 +62,17 @@ protected Settings nodeSettings(int nodeOrdinal) { throw new RuntimeException(e); } - Settings settings = Settings.builder() - .put(super.nodeSettings(nodeOrdinal)) - // client set up here - .put("transport.profiles.client.port", randomClientPortRange) + Settings.Builder builder = Settings.builder().put(super.nodeSettings(nodeOrdinal)); + addSSLSettingsForNodePEMFiles(builder, "transport.profiles.client.xpack.security.", true); + builder.put("transport.profiles.client.port", randomClientPortRange) // make sure this is "localhost", no matter if ipv4 or ipv6, but be consistent .put("transport.profiles.client.bind_host", "localhost") - .put("transport.profiles.client.xpack.security.ssl.certificate_authorities", trustCert.toAbsolutePath()) - .put("transport.profiles.no_client_auth.port", randomNoClientAuthPortRange) + .put("transport.profiles.client.xpack.security.ssl.certificate_authorities", trustCert.toAbsolutePath()); + addSSLSettingsForNodePEMFiles(builder, "transport.profiles.no_client_auth.xpack.security.", true); + builder.put("transport.profiles.no_client_auth.port", randomNoClientAuthPortRange) .put("transport.profiles.no_client_auth.bind_host", "localhost") - .put("transport.profiles.no_client_auth.xpack.security.ssl.client_authentication", SSLClientAuth.NONE) - .build(); + .put("transport.profiles.no_client_auth.xpack.security.ssl.client_authentication", SSLClientAuth.NONE); + final Settings settings = builder.build(); logger.info("node {} settings:\n{}", nodeOrdinal, settings); return settings; } @@ -83,7 +84,7 @@ protected boolean transportSSLEnabled() { private TransportClient createTransportClient(Settings additionalSettings) { Settings settings = Settings.builder() - .put(transportClientSettings().filter(s -> s.startsWith("xpack.ssl") == false)) + .put(transportClientSettings().filter(s -> s.startsWith("xpack.security.transport.ssl") == false)) .put("node.name", "programmatic_transport_client") .put("cluster.name", internalCluster().getClusterName()) .put("xpack.security.transport.ssl.enabled", true) @@ -270,7 +271,7 @@ public void testThatTransportClientWithOnlyTruststoreCanConnectToNoClientAuthPro .put(SecurityField.USER_SETTING.getKey(), TEST_USER_NAME + ":" + TEST_PASSWORD) .put("cluster.name", internalCluster().getClusterName()) .put("xpack.security.transport.ssl.enabled", true) - .put("xpack.ssl.certificate_authorities", + .put("xpack.security.transport.ssl.certificate_authorities", getDataPath("/org/elasticsearch/xpack/security/transport/ssl/certs/simple/testnode.crt")) .build(); try (TransportClient transportClient = new TestXPackTransportClient(settings, @@ -290,8 +291,8 @@ public void testThatTransportClientWithOnlyTruststoreCannotConnectToClientProfil .put(SecurityField.USER_SETTING.getKey(), TEST_USER_NAME + ":" + TEST_PASSWORD) .put("cluster.name", internalCluster().getClusterName()) .put("xpack.security.transport.ssl.enabled", true) - .put("xpack.ssl.client_authentication", SSLClientAuth.REQUIRED) - .put("xpack.ssl.certificate_authorities", + .put("xpack.security.transport.ssl.client_authentication", SSLClientAuth.REQUIRED) + .put("xpack.security.transport.ssl.certificate_authorities", getDataPath("/org/elasticsearch/xpack/security/transport/ssl/certs/simple/testnode.crt")) .build(); try (TransportClient transportClient = new TestXPackTransportClient(settings, @@ -314,8 +315,8 @@ public void testThatTransportClientWithOnlyTruststoreCannotConnectToDefaultProfi .put(SecurityField.USER_SETTING.getKey(), TEST_USER_NAME + ":" + TEST_PASSWORD) .put("cluster.name", internalCluster().getClusterName()) .put("xpack.security.transport.ssl.enabled", true) - .put("xpack.ssl.client_authentication", SSLClientAuth.REQUIRED) - .put("xpack.ssl.certificate_authorities", + .put("xpack.security.transport.ssl.client_authentication", SSLClientAuth.REQUIRED) + .put("xpack.security.transport.ssl.certificate_authorities", getDataPath("/org/elasticsearch/xpack/security/transport/ssl/certs/simple/testnode.crt")) .build(); try (TransportClient transportClient = new TestXPackTransportClient(settings, @@ -337,7 +338,7 @@ public void testThatSSLTransportClientWithNoTruststoreCannotConnectToDefaultProf Settings settings = Settings.builder() .put(SecurityField.USER_SETTING.getKey(), TEST_USER_NAME + ":" + TEST_PASSWORD) .put("cluster.name", internalCluster().getClusterName()) - .put("xpack.ssl.client_authentication", SSLClientAuth.REQUIRED) + .put("xpack.security.transport.ssl.client_authentication", SSLClientAuth.REQUIRED) .put("xpack.security.transport.ssl.enabled", true) .build(); try (TransportClient transportClient = new TestXPackTransportClient(settings, @@ -359,7 +360,7 @@ public void testThatSSLTransportClientWithNoTruststoreCannotConnectToClientProfi Settings settings = Settings.builder() .put(SecurityField.USER_SETTING.getKey(), TEST_USER_NAME + ":" + TEST_PASSWORD) .put("cluster.name", internalCluster().getClusterName()) - .put("xpack.ssl.client_authentication", SSLClientAuth.REQUIRED) + .put("xpack.security.transport.ssl.client_authentication", SSLClientAuth.REQUIRED) .put("xpack.security.transport.ssl.enabled", true) .build(); try (TransportClient transportClient = new TestXPackTransportClient(settings, @@ -381,7 +382,7 @@ public void testThatSSLTransportClientWithNoTruststoreCannotConnectToNoClientAut Settings settings = Settings.builder() .put(SecurityField.USER_SETTING.getKey(), TEST_USER_NAME + ":" + TEST_PASSWORD) .put("cluster.name", internalCluster().getClusterName()) - .put("xpack.ssl.client_authentication", SSLClientAuth.REQUIRED) + .put("xpack.security.transport.ssl.client_authentication", SSLClientAuth.REQUIRED) .put("xpack.security.transport.ssl.enabled", true) .build(); try (TransportClient transportClient = new TestXPackTransportClient(settings, diff --git a/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/ssl/SSLClientAuthTests.java b/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/ssl/SSLClientAuthTests.java index 21da604374f7a..7075a677a26ce 100644 --- a/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/ssl/SSLClientAuthTests.java +++ b/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/ssl/SSLClientAuthTests.java @@ -17,6 +17,7 @@ import org.elasticsearch.client.RestClient; import org.elasticsearch.client.transport.TransportClient; import org.elasticsearch.common.settings.MockSecureSettings; +import org.elasticsearch.common.settings.SecureString; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.transport.TransportAddress; import org.elasticsearch.test.SecurityIntegTestCase; @@ -31,14 +32,17 @@ import javax.net.ssl.KeyManager; import javax.net.ssl.SSLContext; import javax.net.ssl.TrustManager; - +import java.io.ByteArrayOutputStream; import java.io.IOException; +import java.io.InputStream; +import java.io.UncheckedIOException; import java.nio.file.Files; import java.nio.file.Path; import java.security.SecureRandom; import java.security.cert.CertPathBuilderException; import java.util.Arrays; import java.util.Collections; +import java.util.HashSet; import static org.elasticsearch.xpack.core.security.authc.support.UsernamePasswordToken.basicAuthHeaderValue; import static org.hamcrest.Matchers.containsString; @@ -54,13 +58,41 @@ protected boolean addMockHttpTransport() { @Override protected Settings nodeSettings(int nodeOrdinal) { - return Settings.builder() - .put(super.nodeSettings(nodeOrdinal)) + Settings baseSettings = super.nodeSettings(nodeOrdinal); + + Settings.Builder builder = Settings.builder().put(baseSettings); + baseSettings.getByPrefix("xpack.security.transport.ssl.") + .keySet() + .forEach(k -> { + String httpKey = "xpack.security.http.ssl." + k; + String value = baseSettings.get("xpack.security.transport.ssl." + k); + if (value != null) { + builder.put(httpKey, baseSettings.get("xpack.security.transport.ssl." + k)); + } + }); + + MockSecureSettings secureSettings = (MockSecureSettings) builder.getSecureSettings(); + for (String key : new HashSet<>(secureSettings.getSettingNames())) { + SecureString value = secureSettings.getString(key); + if (value == null) { + try { + if (key.startsWith("xpack.security.transport.ssl.")) { + byte[] file = toByteArray(secureSettings.getFile(key)); + secureSettings.setFile(key.replace("xpack.security.transport.ssl.", "xpack.security.http.ssl."), file); + } + } catch (IOException e) { + throw new UncheckedIOException(e); + } + } else if (key.startsWith("xpack.security.transport.ssl.")) { + secureSettings.setString(key.replace("xpack.security.transport.ssl.", "xpack.security.http.ssl."), value.toString()); + } + } + + return builder // invert the require auth settings - .put("xpack.ssl.client_authentication", SSLClientAuth.REQUIRED) + .put("xpack.security.transport.ssl.client_authentication", SSLClientAuth.NONE) .put("xpack.security.http.ssl.enabled", true) .put("xpack.security.http.ssl.client_authentication", SSLClientAuth.REQUIRED) - .put("transport.profiles.default.xpack.security.ssl.client_authentication", SSLClientAuth.NONE) .build(); } @@ -109,13 +141,13 @@ public void testThatTransportWorksWithoutSslClientAuth() throws IOException { } MockSecureSettings secureSettings = new MockSecureSettings(); - secureSettings.setString("xpack.ssl.secure_key_passphrase", "testclient-client-profile"); + secureSettings.setString("xpack.security.transport.ssl.secure_key_passphrase", "testclient-client-profile"); Settings settings = Settings.builder() .put("xpack.security.transport.ssl.enabled", true) - .put("xpack.ssl.client_authentication", SSLClientAuth.NONE) - .put("xpack.ssl.key", keyPath) - .put("xpack.ssl.certificate", certPath) - .put("xpack.ssl.certificate_authorities", nodeCertPath) + .put("xpack.security.transport.ssl.client_authentication", SSLClientAuth.NONE) + .put("xpack.security.transport.ssl.key", keyPath) + .put("xpack.security.transport.ssl.certificate", certPath) + .put("xpack.security.transport.ssl.certificate_authorities", nodeCertPath) .setSecureSettings(secureSettings) .put("cluster.name", internalCluster().getClusterName()) .put(SecurityField.USER_SETTING.getKey(), transportClientUsername() + ":" + new String(transportClientPassword().getChars())) @@ -145,4 +177,15 @@ private SSLContext getSSLContext() { throw new ElasticsearchException("failed to initialize SSLContext", e); } } + + private byte[] toByteArray(InputStream is) throws IOException { + ByteArrayOutputStream baos = new ByteArrayOutputStream(); + byte[] internalBuffer = new byte[1024]; + int read = is.read(internalBuffer); + while (read != -1) { + baos.write(internalBuffer, 0, read); + read = is.read(internalBuffer); + } + return baos.toByteArray(); + } } diff --git a/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/ssl/SSLReloadIntegTests.java b/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/ssl/SSLReloadIntegTests.java index 2928353269823..dd6985889d7ee 100644 --- a/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/ssl/SSLReloadIntegTests.java +++ b/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/ssl/SSLReloadIntegTests.java @@ -72,16 +72,16 @@ public Settings nodeSettings(int nodeOrdinal) { Settings settings = super.nodeSettings(nodeOrdinal); Settings.Builder builder = Settings.builder() - .put(settings.filter((s) -> s.startsWith("xpack.ssl.") == false)); - + .put(settings.filter((s) -> s.startsWith("xpack.security.transport.ssl.") == false)); builder.put("path.home", createTempDir()) - .put("xpack.ssl.key", nodeKeyPath) - .put("xpack.ssl.key_passphrase", "testnode") - .put("xpack.ssl.certificate", nodeCertPath) - .putList("xpack.ssl.certificate_authorities", Arrays.asList(nodeCertPath.toString(), clientCertPath.toString(), - updateableCertPath.toString())) + .put("xpack.security.transport.ssl.key", nodeKeyPath) + .put("xpack.security.transport.ssl.key_passphrase", "testnode") + .put("xpack.security.transport.ssl.certificate", nodeCertPath) + .putList("xpack.security.transport.ssl.certificate_authorities", + Arrays.asList(nodeCertPath.toString(), clientCertPath.toString(), updateableCertPath.toString())) .put("resource.reload.interval.high", "1s"); + builder.put("xpack.security.transport.ssl.enabled", true); return builder.build(); } @@ -96,18 +96,18 @@ public void testThatSSLConfigurationReloadsOnModification() throws Exception { Files.copy(getDataPath("/org/elasticsearch/xpack/security/transport/ssl/certs/simple/testnode_updated.pem"), keyPath); Files.copy(getDataPath("/org/elasticsearch/xpack/security/transport/ssl/certs/simple/testnode_updated.crt"), certPath); MockSecureSettings secureSettings = new MockSecureSettings(); - secureSettings.setString("xpack.ssl.secure_key_passphrase", "testnode"); + secureSettings.setString("xpack.security.transport.ssl.secure_key_passphrase", "testnode"); Settings settings = Settings.builder() .put("path.home", createTempDir()) - .put("xpack.ssl.key", keyPath) - .put("xpack.ssl.certificate", certPath) - .putList("xpack.ssl.certificate_authorities", Arrays.asList(nodeCertPath.toString(), clientCertPath.toString(), - updateableCertPath.toString())) + .put("xpack.security.transport.ssl.key", keyPath) + .put("xpack.security.transport.ssl.certificate", certPath) + .putList("xpack.security.transport.ssl.certificate_authorities", + Arrays.asList(nodeCertPath.toString(), clientCertPath.toString(), updateableCertPath.toString())) .setSecureSettings(secureSettings) .build(); String node = randomFrom(internalCluster().getNodeNames()); SSLService sslService = new SSLService(settings, TestEnvironment.newEnvironment(settings)); - SSLConfiguration sslConfiguration = sslService.getSSLConfiguration("xpack.ssl"); + SSLConfiguration sslConfiguration = sslService.getSSLConfiguration("xpack.security.transport.ssl"); SSLSocketFactory sslSocketFactory = sslService.sslSocketFactory(sslConfiguration); TransportAddress address = internalCluster() .getInstance(Transport.class, node).boundAddress().publishAddress(); diff --git a/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/ssl/SSLTrustRestrictionsTests.java b/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/ssl/SSLTrustRestrictionsTests.java index f513d70e881ae..a89b8fcdd6981 100644 --- a/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/ssl/SSLTrustRestrictionsTests.java +++ b/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/ssl/SSLTrustRestrictionsTests.java @@ -102,9 +102,9 @@ public static void setupCertificates() throws Exception { nodeSSL = Settings.builder() .put("xpack.security.transport.ssl.enabled", true) .put("xpack.security.transport.ssl.verification_mode", "certificate") - .putList("xpack.ssl.certificate_authorities", ca.getCertPath().toString()) - .put("xpack.ssl.key", trustedCert.getKeyPath()) - .put("xpack.ssl.certificate", trustedCert.getCertPath()) + .putList("xpack.security.transport.ssl.certificate_authorities", ca.getCertPath().toString()) + .put("xpack.security.transport.ssl.key", trustedCert.getKeyPath()) + .put("xpack.security.transport.ssl.certificate", trustedCert.getCertPath()) .build(); } @@ -122,14 +122,14 @@ public Settings nodeSettings(int nodeOrdinal) { Settings parentSettings = super.nodeSettings(nodeOrdinal); Settings.Builder builder = Settings.builder() - .put(parentSettings.filter((s) -> s.startsWith("xpack.ssl.") == false)) + .put(parentSettings.filter((s) -> s.startsWith("xpack.security.transport.ssl.") == false)) .put(nodeSSL); restrictionsPath = configPath.resolve("trust_restrictions.yml"); restrictionsTmpPath = configPath.resolve("trust_restrictions.tmp"); writeRestrictions("*.trusted"); - builder.put("xpack.ssl.trust_restrictions.path", restrictionsPath); + builder.put("xpack.security.transport.ssl.trust_restrictions.path", restrictionsPath); return builder.build(); } @@ -152,7 +152,7 @@ private void writeRestrictions(String trustedPattern) { protected Settings transportClientSettings() { Settings parentSettings = super.transportClientSettings(); Settings.Builder builder = Settings.builder() - .put(parentSettings.filter((s) -> s.startsWith("xpack.ssl.") == false)) + .put(parentSettings.filter((s) -> s.startsWith("xpack.security.transport.ssl.") == false)) .put(nodeSSL); return builder.build(); } @@ -224,15 +224,15 @@ private void runResourceWatcher() { private void tryConnect(CertificateInfo certificate) throws Exception { Settings settings = Settings.builder() .put("path.home", createTempDir()) - .put("xpack.ssl.key", certificate.getKeyPath()) - .put("xpack.ssl.certificate", certificate.getCertPath()) - .putList("xpack.ssl.certificate_authorities", ca.getCertPath().toString()) - .put("xpack.ssl.verification_mode", "certificate") + .put("xpack.security.transport.ssl.key", certificate.getKeyPath()) + .put("xpack.security.transport.ssl.certificate", certificate.getCertPath()) + .putList("xpack.security.transport.ssl.certificate_authorities", ca.getCertPath().toString()) + .put("xpack.security.transport.ssl.verification_mode", "certificate") .build(); String node = randomFrom(internalCluster().getNodeNames()); SSLService sslService = new SSLService(settings, TestEnvironment.newEnvironment(settings)); - SSLConfiguration sslConfiguration = sslService.getSSLConfiguration("xpack.ssl"); + SSLConfiguration sslConfiguration = sslService.getSSLConfiguration("xpack.security.transport.ssl"); SSLSocketFactory sslSocketFactory = sslService.sslSocketFactory(sslConfiguration); TransportAddress address = internalCluster().getInstance(Transport.class, node).boundAddress().publishAddress(); try (SSLSocket socket = (SSLSocket) sslSocketFactory.createSocket(address.getAddress(), address.getPort())) { diff --git a/x-pack/plugin/sql/qa/security/with-ssl/build.gradle b/x-pack/plugin/sql/qa/security/with-ssl/build.gradle index cfc04f97188a4..483ba513b5fa1 100644 --- a/x-pack/plugin/sql/qa/security/with-ssl/build.gradle +++ b/x-pack/plugin/sql/qa/security/with-ssl/build.gradle @@ -151,8 +151,10 @@ integTestCluster { setting 'xpack.security.transport.ssl.enabled', 'true' // ceremony to set up ssl - setting 'xpack.ssl.keystore.path', 'test-node.jks' - keystoreSetting 'xpack.ssl.keystore.secure_password', 'keypass' + setting 'xpack.security.transport.ssl.keystore.path', 'test-node.jks' + keystoreSetting 'xpack.security.transport.ssl.keystore.secure_password', 'keypass' + setting 'xpack.security.http.ssl.keystore.path', 'test-node.jks' + keystoreSetting 'xpack.security.http.ssl.keystore.secure_password', 'keypass' setting 'xpack.license.self_generated.type', 'trial' diff --git a/x-pack/plugin/watcher/src/test/java/org/elasticsearch/xpack/watcher/common/http/HttpClientTests.java b/x-pack/plugin/watcher/src/test/java/org/elasticsearch/xpack/watcher/common/http/HttpClientTests.java index 88225efba466a..6bb607d6805a3 100644 --- a/x-pack/plugin/watcher/src/test/java/org/elasticsearch/xpack/watcher/common/http/HttpClientTests.java +++ b/x-pack/plugin/watcher/src/test/java/org/elasticsearch/xpack/watcher/common/http/HttpClientTests.java @@ -81,7 +81,7 @@ public void init() throws Exception { } @After - public void shutdown() throws Exception { + public void shutdown() throws IOException { webServer.close(); httpClient.close(); } @@ -179,30 +179,22 @@ public void testHttps() throws Exception { Path certPath = getDataPath("/org/elasticsearch/xpack/security/keystore/testnode.crt"); Path keyPath = getDataPath("/org/elasticsearch/xpack/security/keystore/testnode.pem"); MockSecureSettings secureSettings = new MockSecureSettings(); - Settings settings; - if (randomBoolean()) { - settings = Settings.builder() - .put("xpack.http.ssl.certificate_authorities", trustedCertPath) - .setSecureSettings(secureSettings) - .build(); - } else { - settings = Settings.builder() - .put("xpack.ssl.certificate_authorities", trustedCertPath) - .setSecureSettings(secureSettings) - .build(); - } + Settings settings = Settings.builder() + .put("xpack.http.ssl.certificate_authorities", trustedCertPath) + .setSecureSettings(secureSettings) + .build(); try (HttpClient client = new HttpClient(settings, new SSLService(settings, environment), null, mockClusterService())) { secureSettings = new MockSecureSettings(); // We can't use the client created above for the server since it is only a truststore - secureSettings.setString("xpack.ssl.secure_key_passphrase", "testnode"); + secureSettings.setString("xpack.security.http.ssl.secure_key_passphrase", "testnode"); Settings settings2 = Settings.builder() - .put("xpack.ssl.key", keyPath) - .put("xpack.ssl.certificate", certPath) + .put("xpack.security.http.ssl.key", keyPath) + .put("xpack.security.http.ssl.certificate", certPath) .setSecureSettings(secureSettings) .build(); TestsSSLService sslService = new TestsSSLService(settings2, environment); - testSslMockWebserver(client, sslService.sslContext(), false); + testSslMockWebserver(client, sslService.sslContext("xpack.security.http.ssl"), false); } } @@ -210,40 +202,27 @@ public void testHttpsDisableHostnameVerification() throws Exception { Path certPath = getDataPath("/org/elasticsearch/xpack/security/transport/ssl/certs/simple/testnode-no-subjaltname.crt"); Path keyPath = getDataPath("/org/elasticsearch/xpack/security/transport/ssl/certs/simple/testnode-no-subjaltname.pem"); Settings settings; - if (randomBoolean()) { - MockSecureSettings secureSettings = new MockSecureSettings(); - Settings.Builder builder = Settings.builder() - .put("xpack.http.ssl.certificate_authorities", certPath); - if (inFipsJvm()) { - //Can't use TrustAllConfig in FIPS mode - builder.put("xpack.http.ssl.verification_mode", VerificationMode.CERTIFICATE); - } else { - builder.put("xpack.http.ssl.verification_mode", randomFrom(VerificationMode.NONE, VerificationMode.CERTIFICATE)); - } - settings = builder.build(); + Settings.Builder builder = Settings.builder() + .put("xpack.http.ssl.certificate_authorities", certPath); + if (inFipsJvm()) { + //Can't use TrustAllConfig in FIPS mode + builder.put("xpack.http.ssl.verification_mode", VerificationMode.CERTIFICATE); } else { - Settings.Builder builder = Settings.builder() - .put("xpack.ssl.certificate_authorities", certPath); - if (inFipsJvm()) { - //Can't use TrustAllConfig in FIPS mode - builder.put("xpack.ssl.verification_mode", VerificationMode.CERTIFICATE); - } else { - builder.put("xpack.ssl.verification_mode", randomFrom(VerificationMode.NONE, VerificationMode.CERTIFICATE)); - } - settings = builder.build(); + builder.put("xpack.http.ssl.verification_mode", randomFrom(VerificationMode.NONE, VerificationMode.CERTIFICATE)); } + settings = builder.build(); try (HttpClient client = new HttpClient(settings, new SSLService(settings, environment), null, mockClusterService())) { MockSecureSettings secureSettings = new MockSecureSettings(); // We can't use the client created above for the server since it only defines a truststore - secureSettings.setString("xpack.ssl.secure_key_passphrase", "testnode-no-subjaltname"); + secureSettings.setString("xpack.security.http.ssl.secure_key_passphrase", "testnode-no-subjaltname"); Settings settings2 = Settings.builder() - .put("xpack.ssl.key", keyPath) - .put("xpack.ssl.certificate", certPath) + .put("xpack.security.http.ssl.key", keyPath) + .put("xpack.security.http.ssl.certificate", certPath) .setSecureSettings(secureSettings) .build(); TestsSSLService sslService = new TestsSSLService(settings2, environment); - testSslMockWebserver(client, sslService.sslContext(), false); + testSslMockWebserver(client, sslService.sslContext("xpack.security.http.ssl"), false); } } @@ -251,16 +230,16 @@ public void testHttpsClientAuth() throws Exception { Path certPath = getDataPath("/org/elasticsearch/xpack/security/keystore/testnode.crt"); Path keyPath = getDataPath("/org/elasticsearch/xpack/security/keystore/testnode.pem"); MockSecureSettings secureSettings = new MockSecureSettings(); - secureSettings.setString("xpack.ssl.secure_key_passphrase", "testnode"); + secureSettings.setString("xpack.http.ssl.secure_key_passphrase", "testnode"); Settings settings = Settings.builder() - .put("xpack.ssl.key", keyPath) - .put("xpack.ssl.certificate", certPath) + .put("xpack.http.ssl.key", keyPath) + .put("xpack.http.ssl.certificate", certPath) .setSecureSettings(secureSettings) .build(); TestsSSLService sslService = new TestsSSLService(settings, environment); try (HttpClient client = new HttpClient(settings, sslService, null, mockClusterService())) { - testSslMockWebserver(client, sslService.sslContext(), true); + testSslMockWebserver(client, sslService.sslContext("xpack.http.ssl"), true); } } @@ -387,19 +366,18 @@ public void testProxyCanHaveDifferentSchemeThanRequest() throws Exception { // on top of that the proxy request is HTTPS but the real request is HTTP only MockSecureSettings serverSecureSettings = new MockSecureSettings(); // We can't use the client created above for the server since it is only a truststore - serverSecureSettings.setString("xpack.ssl.secure_key_passphrase", "testnode"); + serverSecureSettings.setString("xpack.http.ssl.secure_key_passphrase", "testnode"); Settings serverSettings = Settings.builder() - .put("xpack.ssl.key", keyPath) - .put("xpack.ssl.certificate", certPath) + .put("xpack.http.ssl.key", keyPath) + .put("xpack.http.ssl.certificate", certPath) .setSecureSettings(serverSecureSettings) .build(); TestsSSLService sslService = new TestsSSLService(serverSettings, environment); - try (MockWebServer proxyServer = new MockWebServer(sslService.sslContext(), false)) { + try (MockWebServer proxyServer = new MockWebServer(sslService.sslContext(serverSettings.getByPrefix("xpack.http.ssl.")), false)) { proxyServer.enqueue(new MockResponse().setResponseCode(200).setBody("fullProxiedContent")); proxyServer.start(); - MockSecureSettings secureSettings = new MockSecureSettings(); Settings settings = Settings.builder() .put(HttpSettings.PROXY_HOST.getKey(), "localhost") .put(HttpSettings.PROXY_PORT.getKey(), proxyServer.getPort()) diff --git a/x-pack/qa/full-cluster-restart/build.gradle b/x-pack/qa/full-cluster-restart/build.gradle index 716289359faa2..fe7d4e9974cb7 100644 --- a/x-pack/qa/full-cluster-restart/build.gradle +++ b/x-pack/qa/full-cluster-restart/build.gradle @@ -172,8 +172,8 @@ subprojects { setting 'xpack.security.enabled', 'true' setting 'xpack.security.transport.ssl.enabled', 'true' - setting 'xpack.ssl.keystore.path', 'testnode.jks' - setting 'xpack.ssl.keystore.password', 'testnode' + setting 'xpack.security.transport.ssl.keystore.path', 'testnode.jks' + setting 'xpack.security.transport.ssl.keystore.password', 'testnode' setting 'xpack.license.self_generated.type', 'trial' dependsOn copyTestNodeKeystore extraConfigFile 'testnode.jks', new File(outputDir + '/testnode.jks') @@ -217,8 +217,8 @@ subprojects { // some tests rely on the translog not being flushed setting 'indices.memory.shard_inactive_time', '20m' setting 'xpack.security.enabled', 'true' - setting 'xpack.ssl.keystore.path', 'testnode.jks' - keystoreSetting 'xpack.ssl.keystore.secure_password', 'testnode' + setting 'xpack.security.transport.ssl.keystore.path', 'testnode.jks' + keystoreSetting 'xpack.security.transport.ssl.keystore.secure_password', 'testnode' setting 'xpack.license.self_generated.type', 'trial' dependsOn copyTestNodeKeystore extraConfigFile 'testnode.jks', new File(outputDir + '/testnode.jks') diff --git a/x-pack/qa/openldap-tests/src/test/java/org/elasticsearch/test/OpenLdapTests.java b/x-pack/qa/openldap-tests/src/test/java/org/elasticsearch/test/OpenLdapTests.java index 9abac404cea14..bb88103048a40 100644 --- a/x-pack/qa/openldap-tests/src/test/java/org/elasticsearch/test/OpenLdapTests.java +++ b/x-pack/qa/openldap-tests/src/test/java/org/elasticsearch/test/OpenLdapTests.java @@ -61,13 +61,12 @@ public class OpenLdapTests extends ESTestCase { private static final SecureString PASSWORD_SECURE_STRING = new SecureString(PASSWORD.toCharArray()); public static final String REALM_NAME = "oldap-test"; - private boolean useGlobalSSL; private SSLService sslService; private ThreadPool threadPool; private Settings globalSettings; @Before - public void init() throws Exception { + public void init() { threadPool = new TestThreadPool("OpenLdapTests thread pool"); } @@ -89,32 +88,19 @@ public void initializeSslSocketFactory() throws Exception { * If we re-use a SSLContext, previously connected sessions can get re-established which breaks hostname * verification tests since a re-established connection does not perform hostname verification. */ - useGlobalSSL = randomBoolean(); MockSecureSettings mockSecureSettings = new MockSecureSettings(); Settings.Builder builder = Settings.builder().put("path.home", createTempDir()); - if (useGlobalSSL) { - builder.put("xpack.ssl.truststore.path", truststore); - mockSecureSettings.setString("xpack.ssl.truststore.secure_password", "changeit"); - - // configure realm to load config with certificate verification mode - builder.put("xpack.security.authc.realms.ldap." + REALM_NAME + ".ssl.truststore.path", truststore); - mockSecureSettings.setString("xpack.security.authc.realms.ldap." + REALM_NAME + ".ssl.truststore.secure_password", "changeit"); - builder.put("xpack.security.authc.realms.ldap." + REALM_NAME + ".ssl.verification_mode", VerificationMode.CERTIFICATE); - } else { - // fake realms so ssl will get loaded - builder.put("xpack.security.authc.realms.ldap.foo.ssl.truststore.path", truststore); - mockSecureSettings.setString("xpack.security.authc.realms.ldap.foo.ssl.truststore.secure_password", "changeit"); - builder.put("xpack.security.authc.realms.ldap.foo.ssl.verification_mode", VerificationMode.FULL); - builder.put("xpack.security.authc.realms.ldap." + REALM_NAME + ".ssl.truststore.path", truststore); - mockSecureSettings.setString("xpack.security.authc.realms.ldap." + REALM_NAME + ".ssl.truststore.secure_password", "changeit"); - builder.put("xpack.security.authc.realms.ldap." + REALM_NAME + ".ssl.verification_mode", VerificationMode.CERTIFICATE); - - // If not using global ssl, need to set the truststore for the "full verification" realm - builder.put("xpack.security.authc.realms.ldap.vmode_full.ssl.truststore.path", truststore); - mockSecureSettings.setString("xpack.security.authc.realms.ldap.vmode_full.ssl.truststore.secure_password", "changeit"); - } + // fake realms so ssl will get loaded + builder.put("xpack.security.authc.realms.ldap.foo.ssl.truststore.path", truststore); + mockSecureSettings.setString("xpack.security.authc.realms.ldap.foo.ssl.truststore.secure_password", "changeit"); + builder.put("xpack.security.authc.realms.ldap.foo.ssl.verification_mode", VerificationMode.FULL); + builder.put("xpack.security.authc.realms.ldap." + REALM_NAME + ".ssl.truststore.path", truststore); + mockSecureSettings.setString("xpack.security.authc.realms.ldap." + REALM_NAME + ".ssl.truststore.secure_password", "changeit"); + builder.put("xpack.security.authc.realms.ldap." + REALM_NAME + ".ssl.verification_mode", VerificationMode.CERTIFICATE); + + builder.put("xpack.security.authc.realms.ldap.vmode_full.ssl.truststore.path", truststore); + mockSecureSettings.setString("xpack.security.authc.realms.ldap.vmode_full.ssl.truststore.secure_password", "changeit"); builder.put("xpack.security.authc.realms.ldap.vmode_full.ssl.verification_mode", VerificationMode.FULL); - globalSettings = builder.setSecureSettings(mockSecureSettings).build(); Environment environment = TestEnvironment.newEnvironment(globalSettings); sslService = new SSLService(globalSettings, environment); @@ -290,11 +276,8 @@ private Settings buildLdapSettings(RealmConfig.RealmIdentifier realmId, String l final String[] urls = {ldapUrl}; final String[] templates = {userTemplate}; Settings.Builder builder = Settings.builder() - .put(LdapTestCase.buildLdapSettings(realmId, urls, templates, groupSearchBase, scope, null, false)); + .put(LdapTestCase.buildLdapSettings(realmId, urls, templates, groupSearchBase, scope, null, false)); builder.put(getFullSettingKey(realmId.getName(), SearchGroupsResolverSettings.USER_ATTRIBUTE), "uid"); - if (useGlobalSSL) { - return builder.build(); - } return builder .put(getFullSettingKey(realmId, SSLConfigurationSettings.TRUST_STORE_PATH_REALM), getDataPath(LDAPTRUST_PATH)) .put(getFullSettingKey(realmId, SSLConfigurationSettings.LEGACY_TRUST_STORE_PASSWORD_REALM), "changeit") diff --git a/x-pack/qa/openldap-tests/src/test/java/org/elasticsearch/xpack/security/authc/ldap/OpenLdapUserSearchSessionFactoryTests.java b/x-pack/qa/openldap-tests/src/test/java/org/elasticsearch/xpack/security/authc/ldap/OpenLdapUserSearchSessionFactoryTests.java index 5030fdecadf64..ae73c140d9eef 100644 --- a/x-pack/qa/openldap-tests/src/test/java/org/elasticsearch/xpack/security/authc/ldap/OpenLdapUserSearchSessionFactoryTests.java +++ b/x-pack/qa/openldap-tests/src/test/java/org/elasticsearch/xpack/security/authc/ldap/OpenLdapUserSearchSessionFactoryTests.java @@ -60,13 +60,13 @@ public void init() throws Exception { */ globalSettings = Settings.builder() .put("path.home", createTempDir()) - .put("xpack.ssl.certificate_authorities", caPath) + .put("xpack.security.authc.realms.ldap.ssl.certificate_authorities", caPath) .build(); threadPool = new TestThreadPool("LdapUserSearchSessionFactoryTests"); } @After - public void shutdown() throws InterruptedException { + public void shutdown() { terminate(threadPool); } diff --git a/x-pack/qa/rolling-upgrade/build.gradle b/x-pack/qa/rolling-upgrade/build.gradle index 0636f943c6d32..d154a1e248633 100644 --- a/x-pack/qa/rolling-upgrade/build.gradle +++ b/x-pack/qa/rolling-upgrade/build.gradle @@ -157,8 +157,8 @@ subprojects { setting 'xpack.security.authc.token.enabled', 'true' setting 'xpack.security.audit.enabled', 'true' setting 'xpack.security.audit.outputs', 'index' - setting 'xpack.ssl.keystore.path', 'testnode.jks' - setting 'xpack.ssl.keystore.password', 'testnode' + setting 'xpack.security.transport.ssl.keystore.path', 'testnode.jks' + setting 'xpack.security.transport.ssl.keystore.password', 'testnode' dependsOn copyTestNodeKeystore extraConfigFile 'testnode.jks', new File(outputDir + '/testnode.jks') if (version.onOrAfter('7.0.0')) { @@ -226,8 +226,8 @@ subprojects { setting 'xpack.license.self_generated.type', 'trial' setting 'xpack.security.enabled', 'true' setting 'xpack.security.transport.ssl.enabled', 'true' - setting 'xpack.ssl.keystore.path', 'testnode.jks' - keystoreSetting 'xpack.ssl.keystore.secure_password', 'testnode' + setting 'xpack.security.transport.ssl.keystore.path', 'testnode.jks' + keystoreSetting 'xpack.security.transport.ssl.keystore.secure_password', 'testnode' setting 'node.attr.upgraded', 'true' setting 'xpack.security.authc.token.enabled', 'true' setting 'xpack.security.audit.enabled', 'true' diff --git a/x-pack/qa/smoke-test-plugins-ssl/build.gradle b/x-pack/qa/smoke-test-plugins-ssl/build.gradle index 5d1bccd10a6fe..5721815f07856 100644 --- a/x-pack/qa/smoke-test-plugins-ssl/build.gradle +++ b/x-pack/qa/smoke-test-plugins-ssl/build.gradle @@ -57,16 +57,17 @@ integTestCluster { setting 'xpack.monitoring.collection.interval', '1s' setting 'xpack.monitoring.exporters._http.type', 'http' setting 'xpack.monitoring.exporters._http.enabled', 'false' - setting 'xpack.ssl.certificate_authorities', 'testnode.crt' setting 'xpack.monitoring.exporters._http.auth.username', 'monitoring_agent' setting 'xpack.monitoring.exporters._http.auth.password', 'x-pack-test-password' setting 'xpack.monitoring.exporters._http.ssl.verification_mode', 'full' + setting 'xpack.monitoring.exporters._http.ssl.certificate_authorities', 'testnode.crt' setting 'xpack.license.self_generated.type', 'trial' setting 'xpack.security.enabled', 'true' setting 'xpack.security.http.ssl.enabled', 'true' setting 'xpack.security.http.ssl.key', 'testnode.pem' setting 'xpack.security.http.ssl.certificate', 'testnode.crt' + setting 'xpack.security.http.ssl.certificate_authorities', 'testnode.crt' keystoreSetting 'xpack.security.http.ssl.secure_key_passphrase', 'testnode' setting 'xpack.ilm.enabled', 'false' diff --git a/x-pack/qa/third-party/active-directory/src/test/java/org/elasticsearch/xpack/security/authc/ldap/ADLdapUserSearchSessionFactoryTests.java b/x-pack/qa/third-party/active-directory/src/test/java/org/elasticsearch/xpack/security/authc/ldap/ADLdapUserSearchSessionFactoryTests.java index 63a0acb1b03e0..85d4955cc7f87 100644 --- a/x-pack/qa/third-party/active-directory/src/test/java/org/elasticsearch/xpack/security/authc/ldap/ADLdapUserSearchSessionFactoryTests.java +++ b/x-pack/qa/third-party/active-directory/src/test/java/org/elasticsearch/xpack/security/authc/ldap/ADLdapUserSearchSessionFactoryTests.java @@ -7,7 +7,6 @@ import org.elasticsearch.action.support.PlainActionFuture; import org.elasticsearch.common.Strings; -import org.elasticsearch.common.settings.MockSecureSettings; import org.elasticsearch.common.settings.SecureString; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.util.concurrent.ThreadContext; @@ -50,23 +49,17 @@ public void init() throws Exception { globalSettings = Settings.builder() .put("path.home", createTempDir()) - .put("xpack.ssl.certificate_authorities", certPath) + .put("xpack.security.authc.realms.active_directory.ad.ssl.certificate_authorities", certPath) .build(); sslService = new SSLService(globalSettings, env); threadPool = new TestThreadPool("ADLdapUserSearchSessionFactoryTests"); } @After - public void shutdown() throws InterruptedException { + public void shutdown() { terminate(threadPool); } - private MockSecureSettings newSecureSettings(String key, String value) { - MockSecureSettings secureSettings = new MockSecureSettings(); - secureSettings.setString(key, value); - return secureSettings; - } - @AwaitsFix(bugUrl = "https://github.com/elastic/elasticsearch/issues/35738") public void testUserSearchWithActiveDirectory() throws Exception { String groupSearchBase = "DC=ad,DC=test,DC=elasticsearch,DC=com"; diff --git a/x-pack/qa/third-party/active-directory/src/test/java/org/elasticsearch/xpack/security/authc/ldap/AbstractActiveDirectoryTestCase.java b/x-pack/qa/third-party/active-directory/src/test/java/org/elasticsearch/xpack/security/authc/ldap/AbstractActiveDirectoryTestCase.java index 5c4df3eedb812..faf225668e198 100644 --- a/x-pack/qa/third-party/active-directory/src/test/java/org/elasticsearch/xpack/security/authc/ldap/AbstractActiveDirectoryTestCase.java +++ b/x-pack/qa/third-party/active-directory/src/test/java/org/elasticsearch/xpack/security/authc/ldap/AbstractActiveDirectoryTestCase.java @@ -58,12 +58,10 @@ public abstract class AbstractActiveDirectoryTestCase extends ESTestCase { protected SSLService sslService; protected Settings globalSettings; - protected boolean useGlobalSSL; protected List certificatePaths; @Before public void initializeSslSocketFactory() throws Exception { - useGlobalSSL = randomBoolean(); // We use certificates in PEM format and `ssl.certificate_authorities` instead of ssl.trustore // so that these tests can also run in a FIPS JVM where JKS keystores can't be used. certificatePaths = new ArrayList<>(); @@ -84,19 +82,12 @@ public FileVisitResult visitFile(Path file, BasicFileAttributes attrs) throws IO * verification tests since a re-established connection does not perform hostname verification. */ Settings.Builder builder = Settings.builder().put("path.home", createTempDir()); - if (useGlobalSSL) { - builder.putList("xpack.ssl.certificate_authorities", certificatePaths); - // fake realm to load config with certificate verification mode - builder.putList("xpack.security.authc.realms.bar.ssl.certificate_authorities", certificatePaths); - builder.put("xpack.security.authc.realms.bar.ssl.verification_mode", VerificationMode.CERTIFICATE); - } else { - // fake realms so ssl will get loaded - builder.putList("xpack.security.authc.realms.foo.ssl.certificate_authorities", certificatePaths); - builder.put("xpack.security.authc.realms.foo.ssl.verification_mode", VerificationMode.FULL); - builder.putList("xpack.security.authc.realms.bar.ssl.certificate_authorities", certificatePaths); - builder.put("xpack.security.authc.realms.bar.ssl.verification_mode", VerificationMode.CERTIFICATE); - } + // fake realms so ssl will get loaded + builder.putList("xpack.security.authc.realms.foo.ssl.certificate_authorities", certificatePaths); + builder.put("xpack.security.authc.realms.foo.ssl.verification_mode", VerificationMode.FULL); + builder.putList("xpack.security.authc.realms.bar.ssl.certificate_authorities", certificatePaths); + builder.put("xpack.security.authc.realms.bar.ssl.verification_mode", VerificationMode.CERTIFICATE); globalSettings = builder.build(); Environment environment = TestEnvironment.newEnvironment(globalSettings); sslService = new SSLService(globalSettings, environment); @@ -106,24 +97,22 @@ Settings buildAdSettings(RealmConfig.RealmIdentifier realmId, String ldapUrl, St LdapSearchScope scope, boolean hostnameVerification) { final String realmName = realmId.getName(); Settings.Builder builder = Settings.builder() - .putList(getFullSettingKey(realmId, SessionFactorySettings.URLS_SETTING), ldapUrl) - .put(getFullSettingKey(realmName, ActiveDirectorySessionFactorySettings.AD_DOMAIN_NAME_SETTING), adDomainName) - .put(getFullSettingKey(realmName, ActiveDirectorySessionFactorySettings.AD_USER_SEARCH_BASEDN_SETTING), userSearchDN) - .put(getFullSettingKey(realmName, ActiveDirectorySessionFactorySettings.AD_USER_SEARCH_SCOPE_SETTING), scope) - .put(getFullSettingKey(realmName, ActiveDirectorySessionFactorySettings.AD_LDAP_PORT_SETTING), AD_LDAP_PORT) - .put(getFullSettingKey(realmName, ActiveDirectorySessionFactorySettings.AD_LDAPS_PORT_SETTING), AD_LDAPS_PORT) - .put(getFullSettingKey(realmName, ActiveDirectorySessionFactorySettings.AD_GC_LDAP_PORT_SETTING), AD_GC_LDAP_PORT) - .put(getFullSettingKey(realmName, ActiveDirectorySessionFactorySettings.AD_GC_LDAPS_PORT_SETTING), AD_GC_LDAPS_PORT) - .put(getFullSettingKey(realmId, SessionFactorySettings.FOLLOW_REFERRALS_SETTING), FOLLOW_REFERRALS); + .putList(getFullSettingKey(realmId, SessionFactorySettings.URLS_SETTING), ldapUrl) + .put(getFullSettingKey(realmName, ActiveDirectorySessionFactorySettings.AD_DOMAIN_NAME_SETTING), adDomainName) + .put(getFullSettingKey(realmName, ActiveDirectorySessionFactorySettings.AD_USER_SEARCH_BASEDN_SETTING), userSearchDN) + .put(getFullSettingKey(realmName, ActiveDirectorySessionFactorySettings.AD_USER_SEARCH_SCOPE_SETTING), scope) + .put(getFullSettingKey(realmName, ActiveDirectorySessionFactorySettings.AD_LDAP_PORT_SETTING), AD_LDAP_PORT) + .put(getFullSettingKey(realmName, ActiveDirectorySessionFactorySettings.AD_LDAPS_PORT_SETTING), AD_LDAPS_PORT) + .put(getFullSettingKey(realmName, ActiveDirectorySessionFactorySettings.AD_GC_LDAP_PORT_SETTING), AD_GC_LDAP_PORT) + .put(getFullSettingKey(realmName, ActiveDirectorySessionFactorySettings.AD_GC_LDAPS_PORT_SETTING), AD_GC_LDAPS_PORT) + .put(getFullSettingKey(realmId, SessionFactorySettings.FOLLOW_REFERRALS_SETTING), FOLLOW_REFERRALS) + .putList(getFullSettingKey(realmId, SSLConfigurationSettings.CAPATH_SETTING_REALM), certificatePaths); if (randomBoolean()) { builder.put(getFullSettingKey(realmId, SSLConfigurationSettings.VERIFICATION_MODE_SETTING_REALM), hostnameVerification ? VerificationMode.FULL : VerificationMode.CERTIFICATE); } else { builder.put(getFullSettingKey(realmId, SessionFactorySettings.HOSTNAME_VERIFICATION_SETTING), hostnameVerification); } - if (useGlobalSSL == false) { - builder.putList(getFullSettingKey(realmId, SSLConfigurationSettings.CAPATH_SETTING_REALM), certificatePaths); - } return builder.build(); } diff --git a/x-pack/qa/third-party/active-directory/src/test/java/org/elasticsearch/xpack/security/authc/ldap/AbstractAdLdapRealmTestCase.java b/x-pack/qa/third-party/active-directory/src/test/java/org/elasticsearch/xpack/security/authc/ldap/AbstractAdLdapRealmTestCase.java index 135ae14b9c85b..bfec6d100a984 100644 --- a/x-pack/qa/third-party/active-directory/src/test/java/org/elasticsearch/xpack/security/authc/ldap/AbstractAdLdapRealmTestCase.java +++ b/x-pack/qa/third-party/active-directory/src/test/java/org/elasticsearch/xpack/security/authc/ldap/AbstractAdLdapRealmTestCase.java @@ -103,13 +103,11 @@ public abstract class AbstractAdLdapRealmTestCase extends SecurityIntegTestCase protected static final String TESTNODE_CERT = "/org/elasticsearch/xpack/security/transport/ssl/certs/simple/testnode.crt"; protected static RealmConfig realmConfig; protected static List roleMappings; - protected static boolean useGlobalSSL; @BeforeClass public static void setupRealm() { realmConfig = randomFrom(RealmConfig.values()); roleMappings = realmConfig.selectRoleMappings(ESTestCase::randomBoolean); - useGlobalSSL = randomBoolean(); LogManager.getLogger(AbstractAdLdapRealmTestCase.class).info( "running test with realm configuration [{}], with direct group to role mapping [{}]. Settings [{}]", realmConfig, realmConfig.mapGroupsAsRoles, realmConfig.settings); @@ -128,13 +126,13 @@ protected Settings nodeSettings(int nodeOrdinal) { Settings.Builder builder = Settings.builder(); // don't use filter since it returns a prefixed secure setting instead of mock! Settings settingsToAdd = super.nodeSettings(nodeOrdinal); - builder.put(settingsToAdd.filter(k -> k.startsWith("xpack.ssl.") == false), false); + builder.put(settingsToAdd.filter(k -> k.startsWith("xpack.transport.security.ssl.") == false), false); MockSecureSettings mockSecureSettings = (MockSecureSettings) Settings.builder().put(settingsToAdd).getSecureSettings(); if (mockSecureSettings != null) { MockSecureSettings filteredSecureSettings = new MockSecureSettings(); builder.setSecureSettings(filteredSecureSettings); for (String secureSetting : mockSecureSettings.getSettingNames()) { - if (secureSetting.startsWith("xpack.ssl.") == false) { + if (secureSetting.startsWith("xpack.transport.security.ssl.") == false) { SecureString secureString = mockSecureSettings.getString(secureSetting); if (secureString == null) { final byte[] fileBytes; @@ -216,25 +214,6 @@ private List getRoleMappingContent(Function co .collect(Collectors.toList()); } - @Override - protected Settings transportClientSettings() { - if (useGlobalSSL) { - Path key = getDataPath(TESTNODE_KEY); - Path cert = getDataPath(TESTNODE_CERT); - Settings.Builder builder = Settings.builder() - .put(super.transportClientSettings().filter((s) -> s.startsWith("xpack.ssl.") == false)); - addSslSettingsForKeyPair(builder, key, "testnode", cert, getNodeTrustedCertificates()); - return builder.build(); - } else { - return super.transportClientSettings(); - } - } - - @Override - protected boolean transportSSLEnabled() { - return useGlobalSSL; - } - protected final void configureFileRoleMappings(Settings.Builder builder, List mappings) { String content = getRoleMappingContent(RoleMappingEntry::getFileContent, mappings).stream().collect(Collectors.joining("\n")); Path nodeFiles = createTempDir(); @@ -310,11 +289,11 @@ protected static String userHeader(String username, String password) { private void addSslSettingsForKeyPair(Settings.Builder builder, Path key, String keyPassphrase, Path cert, List certificateAuthorities) { - builder.put("xpack.ssl.key", key) - .put("xpack.ssl.key_passphrase", keyPassphrase) - .put("xpack.ssl.verification_mode", "certificate") - .put("xpack.ssl.certificate", cert) - .putList("xpack.ssl.certificate_authorities", certificateAuthorities); + builder.put("xpack.transport.security.ssl.key", key) + .put("xpack.transport.security.ssl.key_passphrase", keyPassphrase) + .put("xpack.transport.security.ssl.verification_mode", "certificate") + .put("xpack.transport.security.ssl.certificate", cert) + .putList("xpack.transport.security.ssl.certificate_authorities", certificateAuthorities); } /** @@ -462,11 +441,8 @@ protected Settings buildSettings(List certificateAuthorities, int order) .put(XPACK_SECURITY_AUTHC_REALMS_EXTERNAL + ".order", order) .put(XPACK_SECURITY_AUTHC_REALMS_EXTERNAL + ".hostname_verification", false) .put(XPACK_SECURITY_AUTHC_REALMS_EXTERNAL + ".unmapped_groups_as_roles", mapGroupsAsRoles) - .put(this.settings); - if (useGlobalSSL == false) { - builder.putList(XPACK_SECURITY_AUTHC_REALMS_EXTERNAL + ".ssl.certificate_authorities", certificateAuthorities); - } - + .put(this.settings) + .putList(XPACK_SECURITY_AUTHC_REALMS_EXTERNAL + ".ssl.certificate_authorities", certificateAuthorities); return builder.build(); } diff --git a/x-pack/qa/third-party/active-directory/src/test/java/org/elasticsearch/xpack/security/authc/ldap/ActiveDirectorySessionFactoryTests.java b/x-pack/qa/third-party/active-directory/src/test/java/org/elasticsearch/xpack/security/authc/ldap/ActiveDirectorySessionFactoryTests.java index aecee76f81831..73e1df5dd08bd 100644 --- a/x-pack/qa/third-party/active-directory/src/test/java/org/elasticsearch/xpack/security/authc/ldap/ActiveDirectorySessionFactoryTests.java +++ b/x-pack/qa/third-party/active-directory/src/test/java/org/elasticsearch/xpack/security/authc/ldap/ActiveDirectorySessionFactoryTests.java @@ -280,21 +280,16 @@ public void testStandardLdapConnection() throws Exception { String groupSearchBase = "DC=ad,DC=test,DC=elasticsearch,DC=com"; String userTemplate = "CN={0},CN=Users,DC=ad,DC=test,DC=elasticsearch,DC=com"; Settings settings = Settings.builder() - .put(LdapTestCase.buildLdapSettings( - new String[]{AD_LDAP_URL}, - new String[]{userTemplate}, - groupSearchBase, - LdapSearchScope.SUB_TREE, - null, - true)) - .put("follow_referrals", FOLLOW_REFERRALS) - .build(); - if (useGlobalSSL == false) { - settings = Settings.builder() - .put(settings) - .putList("ssl.certificate_authorities", certificatePaths) - .build(); - } + .put(LdapTestCase.buildLdapSettings( + new String[]{AD_LDAP_URL}, + new String[]{userTemplate}, + groupSearchBase, + LdapSearchScope.SUB_TREE, + null, + true)) + .put("follow_referrals", FOLLOW_REFERRALS) + .putList("ssl.certificate_authorities", certificatePaths) + .build(); RealmConfig config = configureRealm("ad-as-ldap-test", settings); LdapSessionFactory sessionFactory = new LdapSessionFactory(config, sslService, threadPool); @@ -325,9 +320,7 @@ public void testHandlingLdapReferralErrors() throws Exception { null, ignoreReferralErrors); final Settings.Builder builder = Settings.builder().put(settings).put(globalSettings); - if (useGlobalSSL == false) { - builder.putList(RealmSettings.realmSslPrefix(realmId) + "certificate_authorities", certificatePaths); - } + builder.putList(RealmSettings.realmSslPrefix(realmId) + "certificate_authorities", certificatePaths); settings = builder.build(); RealmConfig config = new RealmConfig(realmId, settings, TestEnvironment.newEnvironment(globalSettings), @@ -352,12 +345,10 @@ public void testHandlingLdapReferralErrors() throws Exception { public void testStandardLdapWithAttributeGroups() throws Exception { String userTemplate = "CN={0},CN=Users,DC=ad,DC=test,DC=elasticsearch,DC=com"; Settings settings = LdapTestCase.buildLdapSettings(new String[]{AD_LDAP_URL}, userTemplate, false); - if (useGlobalSSL == false) { - settings = Settings.builder() - .put(settings) - .putList("ssl.certificate_authorities", certificatePaths) - .build(); - } + settings = Settings.builder() + .put(settings) + .putList("ssl.certificate_authorities", certificatePaths) + .build(); RealmConfig config = configureRealm("ad-as-ldap-test", settings); LdapSessionFactory sessionFactory = new LdapSessionFactory(config, sslService, threadPool); @@ -412,9 +403,7 @@ private Settings buildAdSettings(String ldapUrl, String adDomainName, boolean ho builder.put(getFullSettingKey(REALM_ID, SessionFactorySettings.HOSTNAME_VERIFICATION_SETTING), hostnameVerification); } - if (useGlobalSSL == false) { - builder.putList(getFullSettingKey(REALM_ID, SSLConfigurationSettings.CAPATH_SETTING_REALM), certificatePaths); - } + builder.putList(getFullSettingKey(REALM_ID, SSLConfigurationSettings.CAPATH_SETTING_REALM), certificatePaths); if (useBindUser) { final String user = randomFrom("cap", "hawkeye", "hulk", "ironman", "thor", "blackwidow", "cap@ad.test.elasticsearch.com", diff --git a/x-pack/qa/vagrant/src/test/resources/packaging/tests/certgen.bash b/x-pack/qa/vagrant/src/test/resources/packaging/tests/certgen.bash index 1f81bfcbd490a..dd41b93ea6b28 100644 --- a/x-pack/qa/vagrant/src/test/resources/packaging/tests/certgen.bash +++ b/x-pack/qa/vagrant/src/test/resources/packaging/tests/certgen.bash @@ -251,9 +251,12 @@ node.data: false discovery.zen.ping.unicast.hosts: ["127.0.0.1:9301"] cluster.initial_master_nodes: ["node-master"] -xpack.ssl.key: $ESCONFIG/certs/node-master/node-master.key -xpack.ssl.certificate: $ESCONFIG/certs/node-master/node-master.crt -xpack.ssl.certificate_authorities: ["$ESCONFIG/certs/ca/ca.crt"] +xpack.security.transport.ssl.key: $ESCONFIG/certs/node-master/node-master.key +xpack.security.transport.ssl.certificate: $ESCONFIG/certs/node-master/node-master.crt +xpack.security.transport.ssl.certificate_authorities: ["$ESCONFIG/certs/ca/ca.crt"] +xpack.security.http.ssl.key: $ESCONFIG/certs/node-master/node-master.key +xpack.security.http.ssl.certificate: $ESCONFIG/certs/node-master/node-master.crt +xpack.security.http.ssl.certificate_authorities: ["$ESCONFIG/certs/ca/ca.crt"] xpack.security.transport.ssl.enabled: true transport.tcp.port: 9300 @@ -334,9 +337,12 @@ node.master: false node.data: true discovery.zen.ping.unicast.hosts: ["127.0.0.1:9300"] -xpack.ssl.key: $ESCONFIG/certs/node-data/node-data.key -xpack.ssl.certificate: $ESCONFIG/certs/node-data/node-data.crt -xpack.ssl.certificate_authorities: ["$ESCONFIG/certs/ca/ca.crt"] +xpack.security.transport.ssl.key: $ESCONFIG/certs/node-data/node-data.key +xpack.security.transport.ssl.certificate: $ESCONFIG/certs/node-data/node-data.crt +xpack.security.transport.ssl.certificate_authorities: ["$ESCONFIG/certs/ca/ca.crt"] +xpack.security.http.ssl.key: $ESCONFIG/certs/node-data/node-data.key +xpack.security.http.ssl.certificate: $ESCONFIG/certs//node-data/node-data.crt +xpack.security.http.ssl.certificate_authorities: ["$ESCONFIG/certs/ca/ca.crt"] xpack.security.transport.ssl.enabled: true transport.tcp.port: 9301 From 36a3b84fc968f9aec5c3482dee69a1e53a7ac94a Mon Sep 17 00:00:00 2001 From: Julie Tibshirani Date: Mon, 14 Jan 2019 13:08:01 -0800 Subject: [PATCH 154/186] Update the default for include_type_name to false. (#37285) * Default include_type_name to false for get and put mappings. * Default include_type_name to false for get field mappings. * Add a constant for the default include_type_name value. * Default include_type_name to false for get and put index templates. * Default include_type_name to false for create index. * Update create index calls in REST documentation to use include_type_name=true. * Some minor clean-ups around the get index API. * In REST tests, use include_type_name=true by default for index creation. * Make sure to use 'expression == false'. * Clarify the different IndexTemplateMetaData toXContent methods. * Fix FullClusterRestartIT#testSnapshotRestore. * Fix the ml_anomalies_default_mappings test. * Fix GetFieldMappingsResponseTests and GetIndexTemplateResponseTests. We make sure to specify include_type_name=true during xContent parsing, so we continue to test the legacy typed responses. XContent generation for the typeless responses is currently only covered by REST tests, but we will be adding unit test coverage for these as we implement each typeless API in the Java HLRC. This commit also refactors GetMappingsResponse to follow the same appraoch as the other mappings-related responses, where we read include_type_name out of the xContent params, instead of creating a second toXContent method. This gives better consistency in the response parsing code. * Fix more REST tests. * Improve some wording in the create index documentation. * Add a note about types removal in the create index docs. * Fix SmokeTestMonitoringWithSecurityIT#testHTTPExporterWithSSL. * Make sure to mention include_type_name in the REST docs for affected APIs. * Make sure to use 'expression == false' in FullClusterRestartIT. * Mention include_type_name in the REST templates docs. --- .../client/IndicesRequestConverters.java | 10 ++ .../java/org/elasticsearch/client/CrudIT.java | 8 +- .../elasticsearch/client/IndicesClientIT.java | 8 +- .../client/IndicesRequestConvertersTests.java | 9 ++ .../org/elasticsearch/client/SearchIT.java | 36 ++--- .../documentation/CRUDDocumentationIT.java | 68 +++++---- .../test/rest/RequestsWithoutContentIT.java | 2 +- docs/build.gradle | 1 + .../painless-context-examples.asciidoc | 2 +- .../painless/painless-execute-script.asciidoc | 4 +- docs/plugins/analysis-icu.asciidoc | 16 +-- docs/plugins/analysis-kuromoji.asciidoc | 14 +- docs/plugins/analysis-nori.asciidoc | 8 +- docs/plugins/analysis-phonetic.asciidoc | 2 +- docs/plugins/mapper-annotated-text.asciidoc | 4 +- docs/plugins/mapper-murmur3.asciidoc | 2 +- docs/plugins/mapper-size.asciidoc | 2 +- docs/plugins/store-smb.asciidoc | 2 +- .../bucket/children-aggregation.asciidoc | 2 +- .../bucket/composite-aggregation.asciidoc | 2 +- .../bucket/geodistance-aggregation.asciidoc | 2 +- .../bucket/geohashgrid-aggregation.asciidoc | 2 +- .../bucket/nested-aggregation.asciidoc | 2 +- .../bucket/parent-aggregation.asciidoc | 2 +- .../reverse-nested-aggregation.asciidoc | 2 +- .../significantterms-aggregation.asciidoc | 2 +- .../bucket/terms-aggregation.asciidoc | 2 +- .../metrics/geobounds-aggregation.asciidoc | 2 +- .../metrics/geocentroid-aggregation.asciidoc | 2 +- .../metrics/tophits-aggregation.asciidoc | 2 +- docs/reference/analysis.asciidoc | 2 +- .../analysis/analyzers/configuring.asciidoc | 2 +- .../analyzers/custom-analyzer.asciidoc | 4 +- .../analyzers/fingerprint-analyzer.asciidoc | 4 +- .../analyzers/keyword-analyzer.asciidoc | 2 +- .../analysis/analyzers/lang-analyzer.asciidoc | 68 ++++----- .../analyzers/pattern-analyzer.asciidoc | 6 +- .../analyzers/simple-analyzer.asciidoc | 2 +- .../analyzers/standard-analyzer.asciidoc | 4 +- .../analysis/analyzers/stop-analyzer.asciidoc | 4 +- .../analyzers/whitespace-analyzer.asciidoc | 2 +- .../charfilters/htmlstrip-charfilter.asciidoc | 2 +- .../charfilters/mapping-charfilter.asciidoc | 4 +- .../pattern-replace-charfilter.asciidoc | 4 +- docs/reference/analysis/normalizers.asciidoc | 2 +- docs/reference/analysis/testing.asciidoc | 2 +- .../asciifolding-tokenfilter.asciidoc | 4 +- .../cjk-bigram-tokenfilter.asciidoc | 2 +- .../common-grams-tokenfilter.asciidoc | 2 +- .../compound-word-tokenfilter.asciidoc | 2 +- .../condition-tokenfilter.asciidoc | 2 +- .../tokenfilters/elision-tokenfilter.asciidoc | 2 +- .../hunspell-tokenfilter.asciidoc | 2 +- .../keep-types-tokenfilter.asciidoc | 4 +- .../keep-words-tokenfilter.asciidoc | 2 +- .../keyword-marker-tokenfilter.asciidoc | 2 +- .../keyword-repeat-tokenfilter.asciidoc | 2 +- .../limit-token-count-tokenfilter.asciidoc | 2 +- .../lowercase-tokenfilter.asciidoc | 2 +- .../multiplexer-tokenfilter.asciidoc | 2 +- .../pattern-capture-tokenfilter.asciidoc | 4 +- .../predicate-tokenfilter.asciidoc | 2 +- .../snowball-tokenfilter.asciidoc | 2 +- .../stemmer-override-tokenfilter.asciidoc | 4 +- .../tokenfilters/stemmer-tokenfilter.asciidoc | 2 +- .../tokenfilters/stop-tokenfilter.asciidoc | 4 +- .../synonym-graph-tokenfilter.asciidoc | 8 +- .../tokenfilters/synonym-tokenfilter.asciidoc | 8 +- .../tokenizers/classic-tokenizer.asciidoc | 2 +- .../tokenizers/edgengram-tokenizer.asciidoc | 4 +- .../tokenizers/ngram-tokenizer.asciidoc | 2 +- .../pathhierarchy-tokenizer.asciidoc | 2 +- .../tokenizers/pattern-tokenizer.asciidoc | 4 +- .../simplepattern-tokenizer.asciidoc | 2 +- .../simplepatternsplit-tokenizer.asciidoc | 2 +- .../tokenizers/standard-tokenizer.asciidoc | 2 +- .../tokenizers/uaxurlemail-tokenizer.asciidoc | 2 +- docs/reference/cat/alias.asciidoc | 2 +- docs/reference/cat/fielddata.asciidoc | 2 +- docs/reference/ccr/getting-started.asciidoc | 2 +- docs/reference/docs/get.asciidoc | 2 +- docs/reference/docs/termvectors.asciidoc | 2 +- docs/reference/docs/update-by-query.asciidoc | 4 +- docs/reference/how-to/disk-usage.asciidoc | 10 +- .../how-to/recipes/stemming.asciidoc | 2 +- docs/reference/how-to/search-speed.asciidoc | 6 +- docs/reference/ilm/apis/explain.asciidoc | 2 +- docs/reference/ilm/apis/move-to-step.asciidoc | 2 +- .../apis/remove-policy-from-index.asciidoc | 2 +- docs/reference/ilm/error-handling.asciidoc | 2 +- .../ilm/getting-started-ilm.asciidoc | 2 +- .../reference/ilm/policy-definitions.asciidoc | 2 +- .../ilm/set-up-lifecycle-policy.asciidoc | 4 +- docs/reference/ilm/start-stop-ilm.asciidoc | 2 +- .../ilm/update-lifecycle-policy.asciidoc | 4 +- .../allocation/prioritization.asciidoc | 4 +- .../index-modules/index-sorting.asciidoc | 6 +- .../index-modules/similarity.asciidoc | 10 +- docs/reference/index-modules/store.asciidoc | 4 +- docs/reference/indices/aliases.asciidoc | 6 +- docs/reference/indices/analyze.asciidoc | 2 +- docs/reference/indices/create-index.asciidoc | 37 +---- .../indices/get-field-mapping.asciidoc | 116 ++++++++-------- docs/reference/indices/get-index.asciidoc | 62 +-------- docs/reference/indices/get-mapping.asciidoc | 69 ++-------- docs/reference/indices/put-mapping.asciidoc | 70 ++-------- docs/reference/indices/recovery.asciidoc | 2 +- .../reference/indices/rollover-index.asciidoc | 6 +- docs/reference/indices/split-index.asciidoc | 2 +- docs/reference/indices/templates.asciidoc | 32 ++--- .../ingest/processors/geoip.asciidoc | 2 +- docs/reference/mapping.asciidoc | 2 +- .../mapping/dynamic/field-mapping.asciidoc | 6 +- .../mapping/dynamic/templates.asciidoc | 16 +-- .../mapping/fields/field-names-field.asciidoc | 2 +- .../mapping/fields/meta-field.asciidoc | 4 +- .../mapping/fields/routing-field.asciidoc | 2 +- .../mapping/fields/source-field.asciidoc | 4 +- .../mapping/params/analyzer.asciidoc | 4 +- docs/reference/mapping/params/boost.asciidoc | 2 +- docs/reference/mapping/params/coerce.asciidoc | 4 +- .../reference/mapping/params/copy-to.asciidoc | 2 +- .../mapping/params/doc-values.asciidoc | 2 +- .../reference/mapping/params/dynamic.asciidoc | 2 +- .../params/eager-global-ordinals.asciidoc | 4 +- .../reference/mapping/params/enabled.asciidoc | 4 +- .../mapping/params/fielddata.asciidoc | 6 +- docs/reference/mapping/params/format.asciidoc | 2 +- .../mapping/params/ignore-above.asciidoc | 2 +- .../mapping/params/ignore-malformed.asciidoc | 4 +- .../mapping/params/index-options.asciidoc | 2 +- .../mapping/params/index-prefixes.asciidoc | 4 +- .../mapping/params/multi-fields.asciidoc | 4 +- .../mapping/params/normalizer.asciidoc | 2 +- docs/reference/mapping/params/norms.asciidoc | 2 +- .../mapping/params/null-value.asciidoc | 2 +- .../params/position-increment-gap.asciidoc | 2 +- .../mapping/params/properties.asciidoc | 2 +- .../mapping/params/search-analyzer.asciidoc | 2 +- .../mapping/params/similarity.asciidoc | 2 +- docs/reference/mapping/params/store.asciidoc | 2 +- .../mapping/params/term-vector.asciidoc | 2 +- .../mapping/removal_of_types.asciidoc | 10 +- docs/reference/mapping/types/alias.asciidoc | 2 +- docs/reference/mapping/types/binary.asciidoc | 2 +- docs/reference/mapping/types/boolean.asciidoc | 2 +- docs/reference/mapping/types/date.asciidoc | 4 +- .../mapping/types/dense-vector.asciidoc | 2 +- .../mapping/types/feature-vector.asciidoc | 2 +- docs/reference/mapping/types/feature.asciidoc | 2 +- .../mapping/types/geo-point.asciidoc | 2 +- .../mapping/types/geo-shape.asciidoc | 2 +- docs/reference/mapping/types/ip.asciidoc | 2 +- docs/reference/mapping/types/keyword.asciidoc | 2 +- docs/reference/mapping/types/nested.asciidoc | 2 +- docs/reference/mapping/types/numeric.asciidoc | 2 +- docs/reference/mapping/types/object.asciidoc | 2 +- .../mapping/types/parent-join.asciidoc | 8 +- .../mapping/types/percolator.asciidoc | 12 +- docs/reference/mapping/types/range.asciidoc | 4 +- .../mapping/types/sparse-vector.asciidoc | 2 +- docs/reference/mapping/types/text.asciidoc | 2 +- .../mapping/types/token-count.asciidoc | 2 +- docs/reference/ml/transforms.asciidoc | 2 +- .../modules/indices/request_cache.asciidoc | 2 +- .../modules/scripting/fields.asciidoc | 2 +- .../reference/query-dsl/exists-query.asciidoc | 2 +- .../query-dsl/feature-query.asciidoc | 2 +- .../query-dsl/geo-bounding-box-query.asciidoc | 2 +- .../query-dsl/geo-distance-query.asciidoc | 2 +- .../query-dsl/geo-shape-query.asciidoc | 4 +- docs/reference/query-dsl/mlt-query.asciidoc | 2 +- .../reference/query-dsl/nested-query.asciidoc | 2 +- .../query-dsl/parent-id-query.asciidoc | 2 +- .../query-dsl/percolate-query.asciidoc | 2 +- docs/reference/query-dsl/term-query.asciidoc | 2 +- .../query-dsl/terms-set-query.asciidoc | 2 +- .../request/highlighters-internal.asciidoc | 2 +- .../search/request/highlighting.asciidoc | 4 +- .../search/request/inner-hits.asciidoc | 8 +- .../search/request/post-filter.asciidoc | 2 +- docs/reference/search/request/sort.asciidoc | 2 +- .../suggesters/completion-suggest.asciidoc | 2 +- .../suggesters/context-suggest.asciidoc | 4 +- .../search/suggesters/phrase-suggest.asciidoc | 2 +- .../test/lang_mustache/60_typed_keys.yml | 30 ++-- .../test/update_by_query/30_new_fields.yml | 2 +- .../rest/Netty4HeadBodyIsEmptyIT.java | 28 ++-- .../upgrades/FullClusterRestartIT.java | 18 ++- .../test/indices.create/10_basic.yml | 11 +- .../indices.create/11_basic_with_types.yml | 9 ++ .../indices.get_field_mapping/10_basic.yml | 6 +- .../11_basic_with_types.yml | 6 + .../20_missing_field.yml | 2 +- .../21_missing_field_with_types.yml | 1 + .../30_missing_type.yml | 1 + .../40_missing_index.yml | 1 + .../50_field_wildcards.yml | 9 +- .../51_field_wildcards_with_types.yml | 7 + .../60_mix_typeless_typeful.yml | 4 - .../test/indices.get_mapping/10_basic.yml | 11 +- .../11_basic_with_types.yml | 14 +- .../indices.get_mapping/20_missing_type.yml | 5 + .../indices.get_mapping/30_missing_index.yml | 6 +- .../test/indices.get_mapping/40_aliases.yml | 5 +- .../50_wildcard_expansion.yml | 29 ++-- .../test/indices.get_mapping/60_empty.yml | 4 +- .../61_empty_with_types.yml | 20 +++ .../test/indices.get_template/10_basic.yml | 11 +- .../11_basic_with_types.yml | 3 + .../indices.get_template/20_get_missing.yml | 1 + .../test/indices.put_mapping/10_basic.yml | 14 +- .../11_basic_with_types.yml | 5 + .../20_mix_typeless_typeful.yml | 6 +- .../indices.put_mapping/all_path_options.yml | 28 +--- .../all_path_options_with_types.yml | 33 +++-- .../test/indices.put_template/10_basic.yml | 12 +- .../11_basic_with_types.yml | 6 + .../test/indices.shrink/20_source_mapping.yml | 8 +- .../test/indices.split/20_source_mapping.yml | 8 +- .../test/search/110_field_collapsing.yml | 7 +- .../rest-api-spec/test/suggest/30_context.yml | 6 +- .../admin/indices/get/GetIndexResponse.java | 4 +- .../mapping/get/GetFieldMappingsResponse.java | 4 +- .../mapping/get/GetMappingsResponse.java | 8 +- .../get/GetIndexTemplatesResponse.java | 12 +- .../metadata/IndexTemplateMetaData.java | 43 ++++-- .../cluster/metadata/MetaData.java | 2 +- .../metadata/TemplateUpgradeService.java | 2 +- .../elasticsearch/rest/BaseRestHandler.java | 1 + .../admin/indices/RestCreateIndexAction.java | 3 +- .../indices/RestGetFieldMappingAction.java | 6 +- .../admin/indices/RestGetMappingAction.java | 14 +- .../indices/RestPutIndexTemplateAction.java | 2 +- .../admin/indices/RestPutMappingAction.java | 6 +- .../get/GetFieldMappingsResponseTests.java | 11 ++ .../mapping/get/GetMappingsResponseTests.java | 13 ++ .../get/GetIndexTemplatesResponseTests.java | 13 ++ .../metadata/IndexTemplateMetaDataTests.java | 4 +- .../indices/RestGetMappingActionTests.java | 27 +++- .../test/rest/ESRestTestCase.java | 4 +- .../yaml/ClientYamlTestExecutionContext.java | 10 ++ .../authorization/alias-privileges.asciidoc | 2 +- .../xpack/ccr/FollowIndexIT.java | 2 +- .../elasticsearch/xpack/ccr/AutoFollowIT.java | 6 +- .../org/elasticsearch/xpack/ccr/ChainIT.java | 10 +- .../xpack/ccr/FollowIndexIT.java | 8 +- .../xpack/ccr/FollowIndexSecurityIT.java | 2 +- .../ml/integration/MlBasicMultiNodeIT.java | 10 +- .../ml/integration/DatafeedJobsRestIT.java | 130 ++++++++---------- .../ml/transforms/PainlessDomainSplitIT.java | 4 +- .../exporter/http/TemplateHttpResource.java | 16 ++- ...stractPublishableHttpResourceTestCase.java | 6 + .../exporter/http/HttpExporterIT.java | 22 ++- .../integration/IndexPrivilegeTests.java | 4 +- .../qa/single_node/JdbcShardFailureIT.java | 4 +- .../xpack/sql/qa/jdbc/ResultSetTestCase.java | 10 +- .../test/ml/ml_anomalies_default_mappings.yml | 10 +- 258 files changed, 984 insertions(+), 964 deletions(-) create mode 100644 rest-api-spec/src/main/resources/rest-api-spec/test/indices.get_mapping/61_empty_with_types.yml diff --git a/client/rest-high-level/src/main/java/org/elasticsearch/client/IndicesRequestConverters.java b/client/rest-high-level/src/main/java/org/elasticsearch/client/IndicesRequestConverters.java index bef0aa6a8f474..79b0646770845 100644 --- a/client/rest-high-level/src/main/java/org/elasticsearch/client/IndicesRequestConverters.java +++ b/client/rest-high-level/src/main/java/org/elasticsearch/client/IndicesRequestConverters.java @@ -57,6 +57,8 @@ import java.io.IOException; import java.util.Locale; +import static org.elasticsearch.rest.BaseRestHandler.INCLUDE_TYPE_NAME_PARAMETER; + final class IndicesRequestConverters { private IndicesRequestConverters() {} @@ -103,6 +105,7 @@ static Request createIndex(CreateIndexRequest createIndexRequest) throws IOExcep parameters.withTimeout(createIndexRequest.timeout()); parameters.withMasterTimeout(createIndexRequest.masterNodeTimeout()); parameters.withWaitForActiveShards(createIndexRequest.waitForActiveShards()); + parameters.putParam(INCLUDE_TYPE_NAME_PARAMETER, "true"); request.setEntity(RequestConverters.createEntity(createIndexRequest, RequestConverters.REQUEST_BODY_CONTENT_TYPE)); return request; @@ -131,6 +134,7 @@ static Request putMapping(PutMappingRequest putMappingRequest) throws IOExceptio RequestConverters.Params parameters = new RequestConverters.Params(request); parameters.withTimeout(putMappingRequest.timeout()); parameters.withMasterTimeout(putMappingRequest.masterNodeTimeout()); + parameters.putParam(INCLUDE_TYPE_NAME_PARAMETER, "true"); request.setEntity(RequestConverters.createEntity(putMappingRequest, RequestConverters.REQUEST_BODY_CONTENT_TYPE)); return request; @@ -146,6 +150,8 @@ static Request getMappings(GetMappingsRequest getMappingsRequest) throws IOExcep parameters.withMasterTimeout(getMappingsRequest.masterNodeTimeout()); parameters.withIndicesOptions(getMappingsRequest.indicesOptions()); parameters.withLocal(getMappingsRequest.local()); + parameters.putParam(INCLUDE_TYPE_NAME_PARAMETER, "true"); + return request; } @@ -165,6 +171,8 @@ static Request getFieldMapping(GetFieldMappingsRequest getFieldMappingsRequest) parameters.withIndicesOptions(getFieldMappingsRequest.indicesOptions()); parameters.withIncludeDefaults(getFieldMappingsRequest.includeDefaults()); parameters.withLocal(getFieldMappingsRequest.local()); + parameters.putParam(INCLUDE_TYPE_NAME_PARAMETER, "true"); + return request; } @@ -357,6 +365,7 @@ static Request putTemplate(PutIndexTemplateRequest putIndexTemplateRequest) thro if (Strings.hasText(putIndexTemplateRequest.cause())) { params.putParam("cause", putIndexTemplateRequest.cause()); } + params.putParam(INCLUDE_TYPE_NAME_PARAMETER, "true"); request.setEntity(RequestConverters.createEntity(putIndexTemplateRequest, RequestConverters.REQUEST_BODY_CONTENT_TYPE)); return request; } @@ -395,6 +404,7 @@ static Request getTemplates(GetIndexTemplatesRequest getIndexTemplatesRequest) { final RequestConverters.Params params = new RequestConverters.Params(request); params.withLocal(getIndexTemplatesRequest.isLocal()); params.withMasterTimeout(getIndexTemplatesRequest.getMasterNodeTimeout()); + params.putParam(INCLUDE_TYPE_NAME_PARAMETER, "true"); return request; } 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 19a5b03b6d6cd..eddb6b5758452 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 @@ -255,9 +255,7 @@ public void testSourceDoesNotExist() throws IOException { .put("number_of_shards", 1) .put("number_of_replicas", 0) .build(); - String mapping = "\"_doc\": { \"_source\": {\n" + - " \"enabled\": false\n" + - " } }"; + String mapping = "\"_source\": {\"enabled\": false}"; createIndex(noSourceIndex, settings, mapping); assertEquals( RestStatus.OK, @@ -1242,7 +1240,7 @@ public void testTermvectors() throws IOException { .put("number_of_shards", 1) .put("number_of_replicas", 0) .build(); - String mappings = "\"_doc\":{\"properties\":{\"field\":{\"type\":\"text\"}}}"; + String mappings = "\"properties\":{\"field\":{\"type\":\"text\"}}"; createIndex(sourceIndex, settings, mappings); assertEquals( RestStatus.OK, @@ -1318,7 +1316,7 @@ public void testMultiTermvectors() throws IOException { .put("number_of_shards", 1) .put("number_of_replicas", 0) .build(); - String mappings = "\"_doc\":{\"properties\":{\"field\":{\"type\":\"text\"}}}"; + String mappings = "\"properties\":{\"field\":{\"type\":\"text\"}}"; createIndex(sourceIndex, settings, mappings); assertEquals( RestStatus.OK, 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 92d7e94394594..f3a2fd2baaa3a 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 @@ -341,7 +341,7 @@ public void testGetIndex() throws IOException { .put(SETTING_NUMBER_OF_SHARDS, 1) .put(SETTING_NUMBER_OF_REPLICAS, 0) .build(); - String mappings = "\"_doc\":{\"properties\":{\"field-1\":{\"type\":\"integer\"}}}"; + String mappings = "\"properties\":{\"field-1\":{\"type\":\"integer\"}}"; createIndex(indexName, basicSettings, mappings); GetIndexRequest getIndexRequest = new GetIndexRequest() @@ -371,7 +371,7 @@ public void testGetIndexWithDefaults() throws IOException { .put(SETTING_NUMBER_OF_SHARDS, 1) .put(SETTING_NUMBER_OF_REPLICAS, 0) .build(); - String mappings = "\"_doc\":{\"properties\":{\"field-1\":{\"type\":\"integer\"}}}"; + String mappings = "\"properties\":{\"field-1\":{\"type\":\"integer\"}}"; createIndex(indexName, basicSettings, mappings); GetIndexRequest getIndexRequest = new GetIndexRequest() @@ -1251,8 +1251,8 @@ public void testPutTemplate() throws Exception { assertThat(extractRawValues("my-template.index_patterns", templates), contains("pattern-1", "name-*")); assertThat(extractValue("my-template.settings.index.number_of_shards", templates), equalTo("3")); assertThat(extractValue("my-template.settings.index.number_of_replicas", templates), equalTo("0")); - assertThat(extractValue("my-template.mappings.doc.properties.host_name.type", templates), equalTo("keyword")); - assertThat(extractValue("my-template.mappings.doc.properties.description.type", templates), equalTo("text")); + assertThat(extractValue("my-template.mappings.properties.host_name.type", templates), equalTo("keyword")); + assertThat(extractValue("my-template.mappings.properties.description.type", templates), equalTo("text")); assertThat((Map) extractValue("my-template.aliases.alias-1", templates), hasEntry("index_routing", "abc")); assertThat((Map) extractValue("my-template.aliases.{index}-write", templates), hasEntry("search_routing", "xyz")); } diff --git a/client/rest-high-level/src/test/java/org/elasticsearch/client/IndicesRequestConvertersTests.java b/client/rest-high-level/src/test/java/org/elasticsearch/client/IndicesRequestConvertersTests.java index 010cf04b73766..663c40b17a8b2 100644 --- a/client/rest-high-level/src/test/java/org/elasticsearch/client/IndicesRequestConvertersTests.java +++ b/client/rest-high-level/src/test/java/org/elasticsearch/client/IndicesRequestConvertersTests.java @@ -78,6 +78,7 @@ import static org.elasticsearch.index.RandomCreateIndexGenerator.randomCreateIndexRequest; import static org.elasticsearch.index.RandomCreateIndexGenerator.randomIndexSettings; import static org.elasticsearch.index.alias.RandomAliasActionsGenerator.randomAliasAction; +import static org.elasticsearch.rest.BaseRestHandler.INCLUDE_TYPE_NAME_PARAMETER; import static org.hamcrest.CoreMatchers.equalTo; import static org.hamcrest.Matchers.nullValue; @@ -132,6 +133,7 @@ public void testCreateIndex() throws IOException { RequestConvertersTests.setRandomTimeout(createIndexRequest::timeout, AcknowledgedRequest.DEFAULT_ACK_TIMEOUT, expectedParams); RequestConvertersTests.setRandomMasterTimeout(createIndexRequest, expectedParams); RequestConvertersTests.setRandomWaitForActiveShards(createIndexRequest::waitForActiveShards, expectedParams); + expectedParams.put(INCLUDE_TYPE_NAME_PARAMETER, "true"); Request request = IndicesRequestConverters.createIndex(createIndexRequest); Assert.assertEquals("/" + createIndexRequest.index(), request.getEndpoint()); @@ -173,6 +175,7 @@ public void testPutMapping() throws IOException { RequestConvertersTests.setRandomTimeout(putMappingRequest::timeout, AcknowledgedRequest.DEFAULT_ACK_TIMEOUT, expectedParams); RequestConvertersTests.setRandomMasterTimeout(putMappingRequest, expectedParams); + expectedParams.put(INCLUDE_TYPE_NAME_PARAMETER, "true"); Request request = IndicesRequestConverters.putMapping(putMappingRequest); StringJoiner endpoint = new StringJoiner("/", "/", ""); @@ -214,6 +217,7 @@ public void testGetMapping() throws IOException { getMappingRequest::indicesOptions, expectedParams); RequestConvertersTests.setRandomMasterTimeout(getMappingRequest, expectedParams); RequestConvertersTests.setRandomLocal(getMappingRequest, expectedParams); + expectedParams.put(INCLUDE_TYPE_NAME_PARAMETER, "true"); Request request = IndicesRequestConverters.getMappings(getMappingRequest); StringJoiner endpoint = new StringJoiner("/", "/", ""); @@ -266,6 +270,7 @@ public void testGetFieldMapping() throws IOException { RequestConvertersTests.setRandomIndicesOptions(getFieldMappingsRequest::indicesOptions, getFieldMappingsRequest::indicesOptions, expectedParams); RequestConvertersTests.setRandomLocal(getFieldMappingsRequest::local, expectedParams); + expectedParams.put(INCLUDE_TYPE_NAME_PARAMETER, "true"); Request request = IndicesRequestConverters.getFieldMapping(getFieldMappingsRequest); StringJoiner endpoint = new StringJoiner("/", "/", ""); @@ -835,6 +840,8 @@ public void testPutTemplateRequest() throws Exception { expectedParams.put("cause", cause); } RequestConvertersTests.setRandomMasterTimeout(putTemplateRequest, expectedParams); + expectedParams.put(INCLUDE_TYPE_NAME_PARAMETER, "true"); + Request request = IndicesRequestConverters.putTemplate(putTemplateRequest); Assert.assertThat(request.getEndpoint(), equalTo("/_template/" + names.get(putTemplateRequest.name()))); Assert.assertThat(request.getParameters(), equalTo(expectedParams)); @@ -888,6 +895,8 @@ public void testGetTemplateRequest() throws Exception { Map expectedParams = new HashMap<>(); RequestConvertersTests.setRandomMasterTimeout(getTemplatesRequest::setMasterNodeTimeout, expectedParams); RequestConvertersTests.setRandomLocal(getTemplatesRequest::setLocal, expectedParams); + expectedParams.put(INCLUDE_TYPE_NAME_PARAMETER, "true"); + Request request = IndicesRequestConverters.getTemplates(getTemplatesRequest); Assert.assertThat(request.getEndpoint(), equalTo("/_template/" + names.stream().map(encodes::get).collect(Collectors.joining(",")))); diff --git a/client/rest-high-level/src/test/java/org/elasticsearch/client/SearchIT.java b/client/rest-high-level/src/test/java/org/elasticsearch/client/SearchIT.java index fad42d3c44c45..bf16ce93c147d 100644 --- a/client/rest-high-level/src/test/java/org/elasticsearch/client/SearchIT.java +++ b/client/rest-high-level/src/test/java/org/elasticsearch/client/SearchIT.java @@ -140,11 +140,9 @@ public void indexDocuments() throws IOException { create.setJsonEntity( "{" + " \"mappings\": {" + - " \"_doc\": {" + - " \"properties\": {" + - " \"rating\": {" + - " \"type\": \"keyword\"" + - " }" + + " \"properties\": {" + + " \"rating\": {" + + " \"type\": \"keyword\"" + " }" + " }" + " }" + @@ -172,16 +170,14 @@ public void indexDocuments() throws IOException { create.setJsonEntity( "{" + " \"mappings\": {" + - " \"_doc\": {" + - " \"properties\": {" + - " \"field1\": {" + - " \"type\": \"keyword\"," + - " \"store\": true" + - " }," + - " \"field2\": {" + - " \"type\": \"keyword\"," + - " \"store\": true" + - " }" + + " \"properties\": {" + + " \"field1\": {" + + " \"type\": \"keyword\"," + + " \"store\": true" + + " }," + + " \"field2\": {" + + " \"type\": \"keyword\"," + + " \"store\": true" + " }" + " }" + " }" + @@ -445,12 +441,10 @@ public void testSearchWithParentJoin() throws IOException { createIndex.setJsonEntity( "{\n" + " \"mappings\": {\n" + - " \"_doc\" : {\n" + - " \"properties\" : {\n" + - " \"qa_join_field\" : {\n" + - " \"type\" : \"join\",\n" + - " \"relations\" : { \"question\" : \"answer\" }\n" + - " }\n" + + " \"properties\" : {\n" + + " \"qa_join_field\" : {\n" + + " \"type\" : \"join\",\n" + + " \"relations\" : { \"question\" : \"answer\" }\n" + " }\n" + " }\n" + " }" + diff --git a/client/rest-high-level/src/test/java/org/elasticsearch/client/documentation/CRUDDocumentationIT.java b/client/rest-high-level/src/test/java/org/elasticsearch/client/documentation/CRUDDocumentationIT.java index 894b569f614f3..5033d271261df 100644 --- a/client/rest-high-level/src/test/java/org/elasticsearch/client/documentation/CRUDDocumentationIT.java +++ b/client/rest-high-level/src/test/java/org/elasticsearch/client/documentation/CRUDDocumentationIT.java @@ -792,17 +792,15 @@ public void testReindex() throws Exception { RestHighLevelClient client = highLevelClient(); { String mapping = - "\"_doc\": {\n" + - " \"properties\": {\n" + - " \"user\": {\n" + - " \"type\": \"text\"\n" + - " },\n" + - " \"field1\": {\n" + - " \"type\": \"integer\"\n" + - " },\n" + - " \"field2\": {\n" + - " \"type\": \"integer\"\n" + - " }\n" + + " \"properties\": {\n" + + " \"user\": {\n" + + " \"type\": \"text\"\n" + + " },\n" + + " \"field1\": {\n" + + " \"type\": \"integer\"\n" + + " },\n" + + " \"field2\": {\n" + + " \"type\": \"integer\"\n" + " }\n" + " }"; createIndex("source1", Settings.EMPTY, mapping); @@ -1000,19 +998,17 @@ public void testUpdateByQuery() throws Exception { RestHighLevelClient client = highLevelClient(); { String mapping = - "\"_doc\": {\n" + - " \"properties\": {\n" + - " \"user\": {\n" + - " \"type\": \"text\"\n" + - " },\n" + - " \"field1\": {\n" + - " \"type\": \"integer\"\n" + - " },\n" + - " \"field2\": {\n" + - " \"type\": \"integer\"\n" + - " }\n" + - " }\n" + - " }"; + " \"properties\": {\n" + + " \"user\": {\n" + + " \"type\": \"text\"\n" + + " },\n" + + " \"field1\": {\n" + + " \"type\": \"integer\"\n" + + " },\n" + + " \"field2\": {\n" + + " \"type\": \"integer\"\n" + + " }\n" + + " }"; createIndex("source1", Settings.EMPTY, mapping); createIndex("source2", Settings.EMPTY, mapping); createPipeline("my_pipeline"); @@ -1125,19 +1121,17 @@ public void testDeleteByQuery() throws Exception { RestHighLevelClient client = highLevelClient(); { String mapping = - "\"_doc\": {\n" + - " \"properties\": {\n" + - " \"user\": {\n" + - " \"type\": \"text\"\n" + - " },\n" + - " \"field1\": {\n" + - " \"type\": \"integer\"\n" + - " },\n" + - " \"field2\": {\n" + - " \"type\": \"integer\"\n" + - " }\n" + - " }\n" + - " }"; + " \"properties\": {\n" + + " \"user\": {\n" + + " \"type\": \"text\"\n" + + " },\n" + + " \"field1\": {\n" + + " \"type\": \"integer\"\n" + + " },\n" + + " \"field2\": {\n" + + " \"type\": \"integer\"\n" + + " }\n" + + " }"; createIndex("source1", Settings.EMPTY, mapping); createIndex("source2", Settings.EMPTY, mapping); } diff --git a/distribution/archives/integ-test-zip/src/test/java/org/elasticsearch/test/rest/RequestsWithoutContentIT.java b/distribution/archives/integ-test-zip/src/test/java/org/elasticsearch/test/rest/RequestsWithoutContentIT.java index a6fc7b9cce18e..4d27f156b3826 100644 --- a/distribution/archives/integ-test-zip/src/test/java/org/elasticsearch/test/rest/RequestsWithoutContentIT.java +++ b/distribution/archives/integ-test-zip/src/test/java/org/elasticsearch/test/rest/RequestsWithoutContentIT.java @@ -48,7 +48,7 @@ public void testPutSettingsMissingBody() throws IOException { public void testPutMappingsMissingBody() throws IOException { ResponseException responseException = expectThrows(ResponseException.class, () -> - client().performRequest(new Request(randomBoolean() ? "POST" : "PUT", "/test_index/test_type/_mapping"))); + client().performRequest(new Request(randomBoolean() ? "POST" : "PUT", "/test_index/_mapping"))); assertResponseException(responseException, "request body is required"); } diff --git a/docs/build.gradle b/docs/build.gradle index 035667ee84309..91086ce1f346d 100644 --- a/docs/build.gradle +++ b/docs/build.gradle @@ -525,6 +525,7 @@ for (int i = 0; i < 5; i++) { buildRestTests.setups['library'] = ''' - do: indices.create: + include_type_name: true index: library body: settings: diff --git a/docs/painless/painless-contexts/painless-context-examples.asciidoc b/docs/painless/painless-contexts/painless-context-examples.asciidoc index 469f425d1d89f..79fe9056aba95 100644 --- a/docs/painless/painless-contexts/painless-context-examples.asciidoc +++ b/docs/painless/painless-contexts/painless-context-examples.asciidoc @@ -43,7 +43,7 @@ the request URL. + [source,js] ---- -PUT /seats +PUT /seats?include_type_name=true { "mappings": { "seat": { diff --git a/docs/painless/painless-execute-script.asciidoc b/docs/painless/painless-execute-script.asciidoc index 200dd3f63d56f..3b1b03ca3b698 100644 --- a/docs/painless/painless-execute-script.asciidoc +++ b/docs/painless/painless-execute-script.asciidoc @@ -71,7 +71,7 @@ index:: The name of an index containing a mapping that is compatible with the do [source,js] ---------------------------------------------------------------- -PUT /my-index +PUT /my-index?include_type_name=true { "mappings": { "_doc": { @@ -129,7 +129,7 @@ query:: If `_score` is used in the script then a query can specified that will b [source,js] ---------------------------------------------------------------- -PUT /my-index +PUT /my-index?include_type_name=true { "mappings": { "_doc": { diff --git a/docs/plugins/analysis-icu.asciidoc b/docs/plugins/analysis-icu.asciidoc index 80939496931af..a29acf7f2b0ee 100644 --- a/docs/plugins/analysis-icu.asciidoc +++ b/docs/plugins/analysis-icu.asciidoc @@ -64,7 +64,7 @@ Here are two examples, the default usage and a customised character filter: [source,js] -------------------------------------------------- -PUT icu_sample +PUT icu_sample?include_type_name=true { "settings": { "index": { @@ -112,7 +112,7 @@ using custom rules to break Myanmar and Khmer text into syllables. [source,js] -------------------------------------------------- -PUT icu_sample +PUT icu_sample?include_type_name=true { "settings": { "index": { @@ -153,7 +153,7 @@ Then create an analyzer to use this rule file as follows: [source,js] -------------------------------------------------- -PUT icu_sample +PUT icu_sample?include_type_name=true { "settings": { "index":{ @@ -221,7 +221,7 @@ Here are two examples, the default usage and a customised token filter: [source,js] -------------------------------------------------- -PUT icu_sample +PUT icu_sample?include_type_name=true { "settings": { "index": { @@ -267,7 +267,7 @@ available to all indices: [source,js] -------------------------------------------------- -PUT icu_sample +PUT icu_sample?include_type_name=true { "settings": { "index": { @@ -301,7 +301,7 @@ these filtered character are not lowercased which is why we add the [source,js] -------------------------------------------------- -PUT icu_sample +PUT icu_sample?include_type_name=true { "settings": { "index": { @@ -354,7 +354,7 @@ Below is an example of how to set up a field for sorting German names in [source,js] -------------------------- -PUT my_index +PUT my_index?include_type_name=true { "mappings": { "_doc": { @@ -503,7 +503,7 @@ For example: [source,js] -------------------------------------------------- -PUT icu_sample +PUT icu_sample?include_type_name=true { "settings": { "index": { diff --git a/docs/plugins/analysis-kuromoji.asciidoc b/docs/plugins/analysis-kuromoji.asciidoc index 383df5afb485b..fe7ed1cc317b9 100644 --- a/docs/plugins/analysis-kuromoji.asciidoc +++ b/docs/plugins/analysis-kuromoji.asciidoc @@ -124,7 +124,7 @@ Then create an analyzer as follows: [source,js] -------------------------------------------------- -PUT kuromoji_sample +PUT kuromoji_sample?include_type_name=true { "settings": { "index": { @@ -186,7 +186,7 @@ BaseFormAttribute. This acts as a lemmatizer for verbs and adjectives. Example: [source,js] -------------------------------------------------- -PUT kuromoji_sample +PUT kuromoji_sample?include_type_name=true { "settings": { "index": { @@ -243,7 +243,7 @@ For example: [source,js] -------------------------------------------------- -PUT kuromoji_sample +PUT kuromoji_sample?include_type_name=true { "settings": { "index": { @@ -317,7 +317,7 @@ katakana reading form: [source,js] -------------------------------------------------- -PUT kuromoji_sample +PUT kuromoji_sample?include_type_name=true { "settings": { "index":{ @@ -381,7 +381,7 @@ This token filter accepts the following setting: [source,js] -------------------------------------------------- -PUT kuromoji_sample +PUT kuromoji_sample?include_type_name=true { "settings": { "index": { @@ -434,7 +434,7 @@ predefined list, then use the [source,js] -------------------------------------------------- -PUT kuromoji_sample +PUT kuromoji_sample?include_type_name=true { "settings": { "index": { @@ -493,7 +493,7 @@ to regular Arabic decimal numbers in half-width characters. For example: [source,js] -------------------------------------------------- -PUT kuromoji_sample +PUT kuromoji_sample?include_type_name=true { "settings": { "index": { diff --git a/docs/plugins/analysis-nori.asciidoc b/docs/plugins/analysis-nori.asciidoc index 68ec943533aa9..1fe21fa13da97 100644 --- a/docs/plugins/analysis-nori.asciidoc +++ b/docs/plugins/analysis-nori.asciidoc @@ -90,7 +90,7 @@ Then create an analyzer as follows: [source,js] -------------------------------------------------- -PUT nori_sample +PUT nori_sample?include_type_name=true { "settings": { "index": { @@ -164,7 +164,7 @@ the `user_dictionary_rules` option: [source,js] -------------------------------------------------- -PUT nori_sample +PUT nori_sample?include_type_name=true { "settings": { "index": { @@ -332,7 +332,7 @@ For example: [source,js] -------------------------------------------------- -PUT nori_sample +PUT nori_sample?include_type_name=true { "settings": { "index": { @@ -398,7 +398,7 @@ The `nori_readingform` token filter rewrites tokens written in Hanja to their Ha [source,js] -------------------------------------------------- -PUT nori_sample +PUT nori_sample?include_type_name=true { "settings": { "index":{ diff --git a/docs/plugins/analysis-phonetic.asciidoc b/docs/plugins/analysis-phonetic.asciidoc index e22f819e1eb3e..7996edb6afba3 100644 --- a/docs/plugins/analysis-phonetic.asciidoc +++ b/docs/plugins/analysis-phonetic.asciidoc @@ -29,7 +29,7 @@ The `phonetic` token filter takes the following settings: [source,js] -------------------------------------------------- -PUT phonetic_sample +PUT phonetic_sample?include_type_name=true { "settings": { "index": { diff --git a/docs/plugins/mapper-annotated-text.asciidoc b/docs/plugins/mapper-annotated-text.asciidoc index 4528168a4d643..d026b8a98b2f9 100644 --- a/docs/plugins/mapper-annotated-text.asciidoc +++ b/docs/plugins/mapper-annotated-text.asciidoc @@ -24,7 +24,7 @@ the search index: [source,js] -------------------------- -PUT my_index +PUT my_index?include_type_name=true { "mappings": { "_doc": { @@ -168,7 +168,7 @@ sense to include them in dedicated structured fields to support discovery via ag [source,js] -------------------------- -PUT my_index +PUT my_index?include_type_name=true { "mappings": { "_doc": { diff --git a/docs/plugins/mapper-murmur3.asciidoc b/docs/plugins/mapper-murmur3.asciidoc index 49dbaa77d58bb..dc6d055d40096 100644 --- a/docs/plugins/mapper-murmur3.asciidoc +++ b/docs/plugins/mapper-murmur3.asciidoc @@ -16,7 +16,7 @@ value and its hash are stored in the index: [source,js] -------------------------- -PUT my_index +PUT my_index?include_type_name=true { "mappings": { "_doc": { diff --git a/docs/plugins/mapper-size.asciidoc b/docs/plugins/mapper-size.asciidoc index b60856a848923..141cf382568bf 100644 --- a/docs/plugins/mapper-size.asciidoc +++ b/docs/plugins/mapper-size.asciidoc @@ -15,7 +15,7 @@ In order to enable the `_size` field, set the mapping as follows: [source,js] -------------------------- -PUT my_index +PUT my_index?include_type_name=true { "mappings": { "_doc": { diff --git a/docs/plugins/store-smb.asciidoc b/docs/plugins/store-smb.asciidoc index 4f713568655cb..e0649873f8794 100644 --- a/docs/plugins/store-smb.asciidoc +++ b/docs/plugins/store-smb.asciidoc @@ -46,7 +46,7 @@ It can also be set on a per-index basis at index creation time: [source,js] ---- -PUT my_index +PUT my_index?include_type_name=true { "settings": { "index.store.type": "smb_mmap_fs" diff --git a/docs/reference/aggregations/bucket/children-aggregation.asciidoc b/docs/reference/aggregations/bucket/children-aggregation.asciidoc index dc17157e9102a..7a06d218b357f 100644 --- a/docs/reference/aggregations/bucket/children-aggregation.asciidoc +++ b/docs/reference/aggregations/bucket/children-aggregation.asciidoc @@ -11,7 +11,7 @@ For example, let's say we have an index of questions and answers. The answer typ [source,js] -------------------------------------------------- -PUT child_example +PUT child_example?include_type_name=true { "mappings": { "_doc": { diff --git a/docs/reference/aggregations/bucket/composite-aggregation.asciidoc b/docs/reference/aggregations/bucket/composite-aggregation.asciidoc index 1a63f29467d4b..cc4d778bff032 100644 --- a/docs/reference/aggregations/bucket/composite-aggregation.asciidoc +++ b/docs/reference/aggregations/bucket/composite-aggregation.asciidoc @@ -16,7 +16,7 @@ a composite bucket. [source,js] -------------------------------------------------- -PUT /sales +PUT /sales?include_type_name=true { "mappings": { "_doc": { diff --git a/docs/reference/aggregations/bucket/geodistance-aggregation.asciidoc b/docs/reference/aggregations/bucket/geodistance-aggregation.asciidoc index 9be9b73e38d49..aafcc808530a3 100644 --- a/docs/reference/aggregations/bucket/geodistance-aggregation.asciidoc +++ b/docs/reference/aggregations/bucket/geodistance-aggregation.asciidoc @@ -5,7 +5,7 @@ A multi-bucket aggregation that works on `geo_point` fields and conceptually wor [source,js] -------------------------------------------------- -PUT /museums +PUT /museums?include_type_name=true { "mappings": { "_doc": { diff --git a/docs/reference/aggregations/bucket/geohashgrid-aggregation.asciidoc b/docs/reference/aggregations/bucket/geohashgrid-aggregation.asciidoc index 91c6688b10bd5..bfaec5ee8254f 100644 --- a/docs/reference/aggregations/bucket/geohashgrid-aggregation.asciidoc +++ b/docs/reference/aggregations/bucket/geohashgrid-aggregation.asciidoc @@ -19,7 +19,7 @@ The specified field must be of type `geo_point` (which can only be set explicitl [source,js] -------------------------------------------------- -PUT /museums +PUT /museums?include_type_name=true { "mappings": { "_doc": { diff --git a/docs/reference/aggregations/bucket/nested-aggregation.asciidoc b/docs/reference/aggregations/bucket/nested-aggregation.asciidoc index 5d2d904f0eb7f..2acf760fff3d1 100644 --- a/docs/reference/aggregations/bucket/nested-aggregation.asciidoc +++ b/docs/reference/aggregations/bucket/nested-aggregation.asciidoc @@ -8,7 +8,7 @@ price for the product. The mapping could look like: [source,js] -------------------------------------------------- -PUT /index +PUT /index?include_type_name=true { "mappings": { "product" : { diff --git a/docs/reference/aggregations/bucket/parent-aggregation.asciidoc b/docs/reference/aggregations/bucket/parent-aggregation.asciidoc index 572357c01c1b5..3e0a1606ce277 100644 --- a/docs/reference/aggregations/bucket/parent-aggregation.asciidoc +++ b/docs/reference/aggregations/bucket/parent-aggregation.asciidoc @@ -11,7 +11,7 @@ For example, let's say we have an index of questions and answers. The answer typ [source,js] -------------------------------------------------- -PUT parent_example +PUT parent_example?include_type_name=true { "mappings": { "_doc": { diff --git a/docs/reference/aggregations/bucket/reverse-nested-aggregation.asciidoc b/docs/reference/aggregations/bucket/reverse-nested-aggregation.asciidoc index f45629b14e746..493326651ef99 100644 --- a/docs/reference/aggregations/bucket/reverse-nested-aggregation.asciidoc +++ b/docs/reference/aggregations/bucket/reverse-nested-aggregation.asciidoc @@ -17,7 +17,7 @@ the issue documents as nested documents. The mapping could look like: [source,js] -------------------------------------------------- -PUT /issues +PUT /issues?include_type_name=true { "mappings": { "issue" : { diff --git a/docs/reference/aggregations/bucket/significantterms-aggregation.asciidoc b/docs/reference/aggregations/bucket/significantterms-aggregation.asciidoc index bfaeecc1f82d3..0b2b769adfcbd 100644 --- a/docs/reference/aggregations/bucket/significantterms-aggregation.asciidoc +++ b/docs/reference/aggregations/bucket/significantterms-aggregation.asciidoc @@ -19,7 +19,7 @@ that is significant and probably very relevant to their search. 5/10,000,000 vs [source,js] -------------------------------------------------- -PUT /reports +PUT /reports?include_type_name=true { "mappings": { "report": { diff --git a/docs/reference/aggregations/bucket/terms-aggregation.asciidoc b/docs/reference/aggregations/bucket/terms-aggregation.asciidoc index 188b2ed3774c0..3b104c90332ae 100644 --- a/docs/reference/aggregations/bucket/terms-aggregation.asciidoc +++ b/docs/reference/aggregations/bucket/terms-aggregation.asciidoc @@ -7,7 +7,7 @@ A multi-bucket value source based aggregation where buckets are dynamically buil [source,js] -------------------------------------------------- -PUT /products +PUT /products?include_type_name=true { "mappings": { "product": { diff --git a/docs/reference/aggregations/metrics/geobounds-aggregation.asciidoc b/docs/reference/aggregations/metrics/geobounds-aggregation.asciidoc index 6a2be84d6e54f..fc9765a330711 100644 --- a/docs/reference/aggregations/metrics/geobounds-aggregation.asciidoc +++ b/docs/reference/aggregations/metrics/geobounds-aggregation.asciidoc @@ -8,7 +8,7 @@ Example: [source,js] -------------------------------------------------- -PUT /museums +PUT /museums?include_type_name=true { "mappings": { "_doc": { diff --git a/docs/reference/aggregations/metrics/geocentroid-aggregation.asciidoc b/docs/reference/aggregations/metrics/geocentroid-aggregation.asciidoc index 102c9f21b262f..0ccd58e0dc63c 100644 --- a/docs/reference/aggregations/metrics/geocentroid-aggregation.asciidoc +++ b/docs/reference/aggregations/metrics/geocentroid-aggregation.asciidoc @@ -7,7 +7,7 @@ Example: [source,js] -------------------------------------------------- -PUT /museums +PUT /museums?include_type_name=true { "mappings": { "_doc": { diff --git a/docs/reference/aggregations/metrics/tophits-aggregation.asciidoc b/docs/reference/aggregations/metrics/tophits-aggregation.asciidoc index 1d225c91e26d8..485e900d9628c 100644 --- a/docs/reference/aggregations/metrics/tophits-aggregation.asciidoc +++ b/docs/reference/aggregations/metrics/tophits-aggregation.asciidoc @@ -240,7 +240,7 @@ Let's see how it works with a real sample. Considering the following mapping: [source,js] -------------------------------------------------- -PUT /sales +PUT /sales?include_type_name=true { "mappings": { "_doc" : { diff --git a/docs/reference/analysis.asciidoc b/docs/reference/analysis.asciidoc index c5fcce3ad5fa9..11a8527bbc61a 100644 --- a/docs/reference/analysis.asciidoc +++ b/docs/reference/analysis.asciidoc @@ -39,7 +39,7 @@ Each <> field in a mapping can specify its own [source,js] ------------------------- -PUT my_index +PUT my_index?include_type_name=true { "mappings": { "_doc": { diff --git a/docs/reference/analysis/analyzers/configuring.asciidoc b/docs/reference/analysis/analyzers/configuring.asciidoc index 8ed4c09b4d32a..f010f2ad6e9b1 100644 --- a/docs/reference/analysis/analyzers/configuring.asciidoc +++ b/docs/reference/analysis/analyzers/configuring.asciidoc @@ -8,7 +8,7 @@ to support a list of stop words: [source,js] -------------------------------- -PUT my_index +PUT my_index?include_type_name=true { "settings": { "analysis": { diff --git a/docs/reference/analysis/analyzers/custom-analyzer.asciidoc b/docs/reference/analysis/analyzers/custom-analyzer.asciidoc index 92133822fa51f..153f0fe539e1a 100644 --- a/docs/reference/analysis/analyzers/custom-analyzer.asciidoc +++ b/docs/reference/analysis/analyzers/custom-analyzer.asciidoc @@ -53,7 +53,7 @@ Token Filters:: [source,js] -------------------------------- -PUT my_index +PUT my_index?include_type_name=true { "settings": { "analysis": { @@ -157,7 +157,7 @@ Here is an example: [source,js] -------------------------------------------------- -PUT my_index +PUT my_index?include_type_name=true { "settings": { "analysis": { diff --git a/docs/reference/analysis/analyzers/fingerprint-analyzer.asciidoc b/docs/reference/analysis/analyzers/fingerprint-analyzer.asciidoc index cc82d2eb8179f..28df6d2d3bcf4 100644 --- a/docs/reference/analysis/analyzers/fingerprint-analyzer.asciidoc +++ b/docs/reference/analysis/analyzers/fingerprint-analyzer.asciidoc @@ -86,7 +86,7 @@ pre-defined list of English stop words: [source,js] ---------------------------- -PUT my_index +PUT my_index?include_type_name=true { "settings": { "analysis": { @@ -158,7 +158,7 @@ customization: [source,js] ---------------------------------------------------- -PUT /fingerprint_example +PUT /fingerprint_example?include_type_name=true { "settings": { "analysis": { diff --git a/docs/reference/analysis/analyzers/keyword-analyzer.asciidoc b/docs/reference/analysis/analyzers/keyword-analyzer.asciidoc index 954b514ced605..571ff953c95c3 100644 --- a/docs/reference/analysis/analyzers/keyword-analyzer.asciidoc +++ b/docs/reference/analysis/analyzers/keyword-analyzer.asciidoc @@ -68,7 +68,7 @@ for further customization: [source,js] ---------------------------------------------------- -PUT /keyword_example +PUT /keyword_example?include_type_name=true { "settings": { "analysis": { diff --git a/docs/reference/analysis/analyzers/lang-analyzer.asciidoc b/docs/reference/analysis/analyzers/lang-analyzer.asciidoc index 9a4dcbe8aaac7..959504dbbd608 100644 --- a/docs/reference/analysis/analyzers/lang-analyzer.asciidoc +++ b/docs/reference/analysis/analyzers/lang-analyzer.asciidoc @@ -78,7 +78,7 @@ The `arabic` analyzer could be reimplemented as a `custom` analyzer as follows: [source,js] ---------------------------------------------------- -PUT /arabic_example +PUT /arabic_example?include_type_name=true { "settings": { "analysis": { @@ -128,7 +128,7 @@ The `armenian` analyzer could be reimplemented as a `custom` analyzer as follows [source,js] ---------------------------------------------------- -PUT /armenian_example +PUT /armenian_example?include_type_name=true { "settings": { "analysis": { @@ -176,7 +176,7 @@ The `basque` analyzer could be reimplemented as a `custom` analyzer as follows: [source,js] ---------------------------------------------------- -PUT /basque_example +PUT /basque_example?include_type_name=true { "settings": { "analysis": { @@ -224,7 +224,7 @@ The `bengali` analyzer could be reimplemented as a `custom` analyzer as follows: [source,js] ---------------------------------------------------- -PUT /bengali_example +PUT /bengali_example?include_type_name=true { "settings": { "analysis": { @@ -275,7 +275,7 @@ The `brazilian` analyzer could be reimplemented as a `custom` analyzer as follow [source,js] ---------------------------------------------------- -PUT /brazilian_example +PUT /brazilian_example?include_type_name=true { "settings": { "analysis": { @@ -323,7 +323,7 @@ The `bulgarian` analyzer could be reimplemented as a `custom` analyzer as follow [source,js] ---------------------------------------------------- -PUT /bulgarian_example +PUT /bulgarian_example?include_type_name=true { "settings": { "analysis": { @@ -371,7 +371,7 @@ The `catalan` analyzer could be reimplemented as a `custom` analyzer as follows: [source,js] ---------------------------------------------------- -PUT /catalan_example +PUT /catalan_example?include_type_name=true { "settings": { "analysis": { @@ -428,7 +428,7 @@ The `cjk` analyzer could be reimplemented as a `custom` analyzer as follows: [source,js] ---------------------------------------------------- -PUT /cjk_example +PUT /cjk_example?include_type_name=true { "settings": { "analysis": { @@ -474,7 +474,7 @@ The `czech` analyzer could be reimplemented as a `custom` analyzer as follows: [source,js] ---------------------------------------------------- -PUT /czech_example +PUT /czech_example?include_type_name=true { "settings": { "analysis": { @@ -522,7 +522,7 @@ The `danish` analyzer could be reimplemented as a `custom` analyzer as follows: [source,js] ---------------------------------------------------- -PUT /danish_example +PUT /danish_example?include_type_name=true { "settings": { "analysis": { @@ -570,7 +570,7 @@ The `dutch` analyzer could be reimplemented as a `custom` analyzer as follows: [source,js] ---------------------------------------------------- -PUT /dutch_example +PUT /dutch_example?include_type_name=true { "settings": { "analysis": { @@ -628,7 +628,7 @@ The `english` analyzer could be reimplemented as a `custom` analyzer as follows: [source,js] ---------------------------------------------------- -PUT /english_example +PUT /english_example?include_type_name=true { "settings": { "analysis": { @@ -681,7 +681,7 @@ The `finnish` analyzer could be reimplemented as a `custom` analyzer as follows: [source,js] ---------------------------------------------------- -PUT /finnish_example +PUT /finnish_example?include_type_name=true { "settings": { "analysis": { @@ -729,7 +729,7 @@ The `french` analyzer could be reimplemented as a `custom` analyzer as follows: [source,js] ---------------------------------------------------- -PUT /french_example +PUT /french_example?include_type_name=true { "settings": { "analysis": { @@ -787,7 +787,7 @@ The `galician` analyzer could be reimplemented as a `custom` analyzer as follows [source,js] ---------------------------------------------------- -PUT /galician_example +PUT /galician_example?include_type_name=true { "settings": { "analysis": { @@ -835,7 +835,7 @@ The `german` analyzer could be reimplemented as a `custom` analyzer as follows: [source,js] ---------------------------------------------------- -PUT /german_example +PUT /german_example?include_type_name=true { "settings": { "analysis": { @@ -884,7 +884,7 @@ The `greek` analyzer could be reimplemented as a `custom` analyzer as follows: [source,js] ---------------------------------------------------- -PUT /greek_example +PUT /greek_example?include_type_name=true { "settings": { "analysis": { @@ -936,7 +936,7 @@ The `hindi` analyzer could be reimplemented as a `custom` analyzer as follows: [source,js] ---------------------------------------------------- -PUT /hindi_example +PUT /hindi_example?include_type_name=true { "settings": { "analysis": { @@ -987,7 +987,7 @@ The `hungarian` analyzer could be reimplemented as a `custom` analyzer as follow [source,js] ---------------------------------------------------- -PUT /hungarian_example +PUT /hungarian_example?include_type_name=true { "settings": { "analysis": { @@ -1036,7 +1036,7 @@ The `indonesian` analyzer could be reimplemented as a `custom` analyzer as follo [source,js] ---------------------------------------------------- -PUT /indonesian_example +PUT /indonesian_example?include_type_name=true { "settings": { "analysis": { @@ -1084,7 +1084,7 @@ The `irish` analyzer could be reimplemented as a `custom` analyzer as follows: [source,js] ---------------------------------------------------- -PUT /irish_example +PUT /irish_example?include_type_name=true { "settings": { "analysis": { @@ -1148,7 +1148,7 @@ The `italian` analyzer could be reimplemented as a `custom` analyzer as follows: [source,js] ---------------------------------------------------- -PUT /italian_example +PUT /italian_example?include_type_name=true { "settings": { "analysis": { @@ -1207,7 +1207,7 @@ The `latvian` analyzer could be reimplemented as a `custom` analyzer as follows: [source,js] ---------------------------------------------------- -PUT /latvian_example +PUT /latvian_example?include_type_name=true { "settings": { "analysis": { @@ -1255,7 +1255,7 @@ The `lithuanian` analyzer could be reimplemented as a `custom` analyzer as follo [source,js] ---------------------------------------------------- -PUT /lithuanian_example +PUT /lithuanian_example?include_type_name=true { "settings": { "analysis": { @@ -1303,7 +1303,7 @@ The `norwegian` analyzer could be reimplemented as a `custom` analyzer as follow [source,js] ---------------------------------------------------- -PUT /norwegian_example +PUT /norwegian_example?include_type_name=true { "settings": { "analysis": { @@ -1351,7 +1351,7 @@ The `persian` analyzer could be reimplemented as a `custom` analyzer as follows: [source,js] ---------------------------------------------------- -PUT /persian_example +PUT /persian_example?include_type_name=true { "settings": { "analysis": { @@ -1397,7 +1397,7 @@ The `portuguese` analyzer could be reimplemented as a `custom` analyzer as follo [source,js] ---------------------------------------------------- -PUT /portuguese_example +PUT /portuguese_example?include_type_name=true { "settings": { "analysis": { @@ -1445,7 +1445,7 @@ The `romanian` analyzer could be reimplemented as a `custom` analyzer as follows [source,js] ---------------------------------------------------- -PUT /romanian_example +PUT /romanian_example?include_type_name=true { "settings": { "analysis": { @@ -1494,7 +1494,7 @@ The `russian` analyzer could be reimplemented as a `custom` analyzer as follows: [source,js] ---------------------------------------------------- -PUT /russian_example +PUT /russian_example?include_type_name=true { "settings": { "analysis": { @@ -1542,7 +1542,7 @@ The `sorani` analyzer could be reimplemented as a `custom` analyzer as follows: [source,js] ---------------------------------------------------- -PUT /sorani_example +PUT /sorani_example?include_type_name=true { "settings": { "analysis": { @@ -1592,7 +1592,7 @@ The `spanish` analyzer could be reimplemented as a `custom` analyzer as follows: [source,js] ---------------------------------------------------- -PUT /spanish_example +PUT /spanish_example?include_type_name=true { "settings": { "analysis": { @@ -1640,7 +1640,7 @@ The `swedish` analyzer could be reimplemented as a `custom` analyzer as follows: [source,js] ---------------------------------------------------- -PUT /swedish_example +PUT /swedish_example?include_type_name=true { "settings": { "analysis": { @@ -1688,7 +1688,7 @@ The `turkish` analyzer could be reimplemented as a `custom` analyzer as follows: [source,js] ---------------------------------------------------- -PUT /turkish_example +PUT /turkish_example?include_type_name=true { "settings": { "analysis": { @@ -1741,7 +1741,7 @@ The `thai` analyzer could be reimplemented as a `custom` analyzer as follows: [source,js] ---------------------------------------------------- -PUT /thai_example +PUT /thai_example?include_type_name=true { "settings": { "analysis": { diff --git a/docs/reference/analysis/analyzers/pattern-analyzer.asciidoc b/docs/reference/analysis/analyzers/pattern-analyzer.asciidoc index 027f37280a67d..759c781616926 100644 --- a/docs/reference/analysis/analyzers/pattern-analyzer.asciidoc +++ b/docs/reference/analysis/analyzers/pattern-analyzer.asciidoc @@ -177,7 +177,7 @@ on non-word characters or on underscores (`\W|_`), and to lower-case the result: [source,js] ---------------------------- -PUT my_index +PUT my_index?include_type_name=true { "settings": { "analysis": { @@ -266,7 +266,7 @@ The following more complicated example splits CamelCase text into tokens: [source,js] -------------------------------------------------- -PUT my_index +PUT my_index?include_type_name=true { "settings": { "analysis": { @@ -386,7 +386,7 @@ customization: [source,js] ---------------------------------------------------- -PUT /pattern_example +PUT /pattern_example?include_type_name=true { "settings": { "analysis": { diff --git a/docs/reference/analysis/analyzers/simple-analyzer.asciidoc b/docs/reference/analysis/analyzers/simple-analyzer.asciidoc index d82655d9bd8e1..23130a4fd58ab 100644 --- a/docs/reference/analysis/analyzers/simple-analyzer.asciidoc +++ b/docs/reference/analysis/analyzers/simple-analyzer.asciidoc @@ -135,7 +135,7 @@ a starting point for further customization: [source,js] ---------------------------------------------------- -PUT /simple_example +PUT /simple_example?include_type_name=true { "settings": { "analysis": { diff --git a/docs/reference/analysis/analyzers/standard-analyzer.asciidoc b/docs/reference/analysis/analyzers/standard-analyzer.asciidoc index 3097ece21db23..959e493d9d5b0 100644 --- a/docs/reference/analysis/analyzers/standard-analyzer.asciidoc +++ b/docs/reference/analysis/analyzers/standard-analyzer.asciidoc @@ -151,7 +151,7 @@ pre-defined list of English stop words: [source,js] ---------------------------- -PUT my_index +PUT my_index?include_type_name=true { "settings": { "analysis": { @@ -283,7 +283,7 @@ it, usually by adding token filters. This would recreate the built-in [source,js] ---------------------------------------------------- -PUT /standard_example +PUT /standard_example?include_type_name=true { "settings": { "analysis": { diff --git a/docs/reference/analysis/analyzers/stop-analyzer.asciidoc b/docs/reference/analysis/analyzers/stop-analyzer.asciidoc index 1b84797d94761..2586d79c8443b 100644 --- a/docs/reference/analysis/analyzers/stop-analyzer.asciidoc +++ b/docs/reference/analysis/analyzers/stop-analyzer.asciidoc @@ -130,7 +130,7 @@ words as stop words: [source,js] ---------------------------- -PUT my_index +PUT my_index?include_type_name=true { "settings": { "analysis": { @@ -248,7 +248,7 @@ customization: [source,js] ---------------------------------------------------- -PUT /stop_example +PUT /stop_example?include_type_name=true { "settings": { "analysis": { diff --git a/docs/reference/analysis/analyzers/whitespace-analyzer.asciidoc b/docs/reference/analysis/analyzers/whitespace-analyzer.asciidoc index 31ba8d9ce8f24..3ebc665abdd87 100644 --- a/docs/reference/analysis/analyzers/whitespace-analyzer.asciidoc +++ b/docs/reference/analysis/analyzers/whitespace-analyzer.asciidoc @@ -128,7 +128,7 @@ and you can use it as a starting point for further customization: [source,js] ---------------------------------------------------- -PUT /whitespace_example +PUT /whitespace_example?include_type_name=true { "settings": { "analysis": { diff --git a/docs/reference/analysis/charfilters/htmlstrip-charfilter.asciidoc b/docs/reference/analysis/charfilters/htmlstrip-charfilter.asciidoc index 6c1a1875d67ca..a9e11cf778d3a 100644 --- a/docs/reference/analysis/charfilters/htmlstrip-charfilter.asciidoc +++ b/docs/reference/analysis/charfilters/htmlstrip-charfilter.asciidoc @@ -73,7 +73,7 @@ tags in place: [source,js] ---------------------------- -PUT my_index +PUT my_index?include_type_name=true { "settings": { "analysis": { diff --git a/docs/reference/analysis/charfilters/mapping-charfilter.asciidoc b/docs/reference/analysis/charfilters/mapping-charfilter.asciidoc index 30e565d443a3b..0cbc6de782d79 100644 --- a/docs/reference/analysis/charfilters/mapping-charfilter.asciidoc +++ b/docs/reference/analysis/charfilters/mapping-charfilter.asciidoc @@ -33,7 +33,7 @@ numerals with their Latin equivalents: [source,js] ---------------------------- -PUT my_index +PUT my_index?include_type_name=true { "settings": { "analysis": { @@ -107,7 +107,7 @@ example replaces the `:)` and `:(` emoticons with a text equivalent: [source,js] ---------------------------- -PUT my_index +PUT my_index?include_type_name=true { "settings": { "analysis": { diff --git a/docs/reference/analysis/charfilters/pattern-replace-charfilter.asciidoc b/docs/reference/analysis/charfilters/pattern-replace-charfilter.asciidoc index 046f6441c07b9..adfde27138af8 100644 --- a/docs/reference/analysis/charfilters/pattern-replace-charfilter.asciidoc +++ b/docs/reference/analysis/charfilters/pattern-replace-charfilter.asciidoc @@ -49,7 +49,7 @@ replace any embedded dashes in numbers with underscores, i.e `123-456-789` -> [source,js] ---------------------------- -PUT my_index +PUT my_index?include_type_name=true { "settings": { "analysis": { @@ -100,7 +100,7 @@ camelCase words to be queried individually: [source,js] ---------------------------- -PUT my_index +PUT my_index?include_type_name=true { "settings": { "analysis": { diff --git a/docs/reference/analysis/normalizers.asciidoc b/docs/reference/analysis/normalizers.asciidoc index 80977d1d771df..64e69c3f56bf4 100644 --- a/docs/reference/analysis/normalizers.asciidoc +++ b/docs/reference/analysis/normalizers.asciidoc @@ -23,7 +23,7 @@ to get one is by building a custom one. Custom normalizers take a list of char [source,js] -------------------------------- -PUT index +PUT index?include_type_name=true { "settings": { "analysis": { diff --git a/docs/reference/analysis/testing.asciidoc b/docs/reference/analysis/testing.asciidoc index c8c7c3fa2b463..8102978635865 100644 --- a/docs/reference/analysis/testing.asciidoc +++ b/docs/reference/analysis/testing.asciidoc @@ -41,7 +41,7 @@ referred to when running the `analyze` API on a specific index: [source,js] ------------------------------------- -PUT my_index +PUT my_index?include_type_name=true { "settings": { "analysis": { diff --git a/docs/reference/analysis/tokenfilters/asciifolding-tokenfilter.asciidoc b/docs/reference/analysis/tokenfilters/asciifolding-tokenfilter.asciidoc index bd22b013334a9..77eb1bb05826d 100644 --- a/docs/reference/analysis/tokenfilters/asciifolding-tokenfilter.asciidoc +++ b/docs/reference/analysis/tokenfilters/asciifolding-tokenfilter.asciidoc @@ -8,7 +8,7 @@ equivalents, if one exists. Example: [source,js] -------------------------------------------------- -PUT /asciifold_example +PUT /asciifold_example?include_type_name=true { "settings" : { "analysis" : { @@ -30,7 +30,7 @@ example: [source,js] -------------------------------------------------- -PUT /asciifold_example +PUT /asciifold_example?include_type_name=true { "settings" : { "analysis" : { diff --git a/docs/reference/analysis/tokenfilters/cjk-bigram-tokenfilter.asciidoc b/docs/reference/analysis/tokenfilters/cjk-bigram-tokenfilter.asciidoc index cc26d025f04f9..a7ec4d9248546 100644 --- a/docs/reference/analysis/tokenfilters/cjk-bigram-tokenfilter.asciidoc +++ b/docs/reference/analysis/tokenfilters/cjk-bigram-tokenfilter.asciidoc @@ -16,7 +16,7 @@ Bigrams are generated for characters in `han`, `hiragana`, `katakana` and [source,js] -------------------------------------------------- -PUT /cjk_bigram_example +PUT /cjk_bigram_example?include_type_name=true { "settings" : { "analysis" : { diff --git a/docs/reference/analysis/tokenfilters/common-grams-tokenfilter.asciidoc b/docs/reference/analysis/tokenfilters/common-grams-tokenfilter.asciidoc index 361160ac6acc8..538e28a8cf0a1 100644 --- a/docs/reference/analysis/tokenfilters/common-grams-tokenfilter.asciidoc +++ b/docs/reference/analysis/tokenfilters/common-grams-tokenfilter.asciidoc @@ -41,7 +41,7 @@ Here is an example: [source,js] -------------------------------------------------- -PUT /common_grams_example +PUT /common_grams_example?include_type_name=true { "settings": { "analysis": { diff --git a/docs/reference/analysis/tokenfilters/compound-word-tokenfilter.asciidoc b/docs/reference/analysis/tokenfilters/compound-word-tokenfilter.asciidoc index d200c0b988bc4..b345e0b7b4c08 100644 --- a/docs/reference/analysis/tokenfilters/compound-word-tokenfilter.asciidoc +++ b/docs/reference/analysis/tokenfilters/compound-word-tokenfilter.asciidoc @@ -84,7 +84,7 @@ Here is an example: [source,js] -------------------------------------------------- -PUT /compound_word_example +PUT /compound_word_example?include_type_name=true { "settings": { "index": { diff --git a/docs/reference/analysis/tokenfilters/condition-tokenfilter.asciidoc b/docs/reference/analysis/tokenfilters/condition-tokenfilter.asciidoc index cff05559ab9e6..e241bddb12b1d 100644 --- a/docs/reference/analysis/tokenfilters/condition-tokenfilter.asciidoc +++ b/docs/reference/analysis/tokenfilters/condition-tokenfilter.asciidoc @@ -20,7 +20,7 @@ You can set it up like: [source,js] -------------------------------------------------- -PUT /condition_example +PUT /condition_example?include_type_name=true { "settings" : { "analysis" : { diff --git a/docs/reference/analysis/tokenfilters/elision-tokenfilter.asciidoc b/docs/reference/analysis/tokenfilters/elision-tokenfilter.asciidoc index 924903b9f65a8..7a28760194a71 100644 --- a/docs/reference/analysis/tokenfilters/elision-tokenfilter.asciidoc +++ b/docs/reference/analysis/tokenfilters/elision-tokenfilter.asciidoc @@ -9,7 +9,7 @@ example: [source,js] -------------------------------------------------- -PUT /elision_example +PUT /elision_example?include_type_name=true { "settings" : { "analysis" : { diff --git a/docs/reference/analysis/tokenfilters/hunspell-tokenfilter.asciidoc b/docs/reference/analysis/tokenfilters/hunspell-tokenfilter.asciidoc index cef687f761905..e68b6685a78e7 100644 --- a/docs/reference/analysis/tokenfilters/hunspell-tokenfilter.asciidoc +++ b/docs/reference/analysis/tokenfilters/hunspell-tokenfilter.asciidoc @@ -42,7 +42,7 @@ settings: [source,js] -------------------------------------------------- -PUT /hunspell_example +PUT /hunspell_example?include_type_name=true { "settings": { "analysis" : { diff --git a/docs/reference/analysis/tokenfilters/keep-types-tokenfilter.asciidoc b/docs/reference/analysis/tokenfilters/keep-types-tokenfilter.asciidoc index 33a927c4b98bf..ac5b3f368b53e 100644 --- a/docs/reference/analysis/tokenfilters/keep-types-tokenfilter.asciidoc +++ b/docs/reference/analysis/tokenfilters/keep-types-tokenfilter.asciidoc @@ -19,7 +19,7 @@ You can set it up like: [source,js] -------------------------------------------------- -PUT /keep_types_example +PUT /keep_types_example?include_type_name=true { "settings" : { "analysis" : { @@ -80,7 +80,7 @@ If the `mode` parameter is set to `exclude` like in the following example: [source,js] -------------------------------------------------- -PUT /keep_types_exclude_example +PUT /keep_types_exclude_example?include_type_name=true { "settings" : { "analysis" : { diff --git a/docs/reference/analysis/tokenfilters/keep-words-tokenfilter.asciidoc b/docs/reference/analysis/tokenfilters/keep-words-tokenfilter.asciidoc index b7385379be94b..1f1d49cfe89dc 100644 --- a/docs/reference/analysis/tokenfilters/keep-words-tokenfilter.asciidoc +++ b/docs/reference/analysis/tokenfilters/keep-words-tokenfilter.asciidoc @@ -20,7 +20,7 @@ keep_words_case:: a boolean indicating whether to lower case the words (defaults [source,js] -------------------------------------------------- -PUT /keep_words_example +PUT /keep_words_example?include_type_name=true { "settings" : { "analysis" : { diff --git a/docs/reference/analysis/tokenfilters/keyword-marker-tokenfilter.asciidoc b/docs/reference/analysis/tokenfilters/keyword-marker-tokenfilter.asciidoc index 1f1e4e655c55e..8a12b0d4757af 100644 --- a/docs/reference/analysis/tokenfilters/keyword-marker-tokenfilter.asciidoc +++ b/docs/reference/analysis/tokenfilters/keyword-marker-tokenfilter.asciidoc @@ -23,7 +23,7 @@ You can configure it like: [source,js] -------------------------------------------------- -PUT /keyword_marker_example +PUT /keyword_marker_example?include_type_name=true { "settings": { "analysis": { diff --git a/docs/reference/analysis/tokenfilters/keyword-repeat-tokenfilter.asciidoc b/docs/reference/analysis/tokenfilters/keyword-repeat-tokenfilter.asciidoc index 044e8c1476951..e0a2e4c73c96d 100644 --- a/docs/reference/analysis/tokenfilters/keyword-repeat-tokenfilter.asciidoc +++ b/docs/reference/analysis/tokenfilters/keyword-repeat-tokenfilter.asciidoc @@ -14,7 +14,7 @@ preserve both the stemmed and unstemmed version of tokens: [source,js] -------------------------------------------------- -PUT /keyword_repeat_example +PUT /keyword_repeat_example?include_type_name=true { "settings": { "analysis": { diff --git a/docs/reference/analysis/tokenfilters/limit-token-count-tokenfilter.asciidoc b/docs/reference/analysis/tokenfilters/limit-token-count-tokenfilter.asciidoc index ba2018c107626..deb13843a1ced 100644 --- a/docs/reference/analysis/tokenfilters/limit-token-count-tokenfilter.asciidoc +++ b/docs/reference/analysis/tokenfilters/limit-token-count-tokenfilter.asciidoc @@ -18,7 +18,7 @@ Here is an example: [source,js] -------------------------------------------------- -PUT /limit_example +PUT /limit_example?include_type_name=true { "settings": { "analysis": { diff --git a/docs/reference/analysis/tokenfilters/lowercase-tokenfilter.asciidoc b/docs/reference/analysis/tokenfilters/lowercase-tokenfilter.asciidoc index 519fd77ba2afd..c0c35e8a60c9d 100644 --- a/docs/reference/analysis/tokenfilters/lowercase-tokenfilter.asciidoc +++ b/docs/reference/analysis/tokenfilters/lowercase-tokenfilter.asciidoc @@ -10,7 +10,7 @@ custom analyzer [source,js] -------------------------------------------------- -PUT /lowercase_example +PUT /lowercase_example?include_type_name=true { "settings": { "analysis": { diff --git a/docs/reference/analysis/tokenfilters/multiplexer-tokenfilter.asciidoc b/docs/reference/analysis/tokenfilters/multiplexer-tokenfilter.asciidoc index e2427071fbb69..8ad3fab8f36e7 100644 --- a/docs/reference/analysis/tokenfilters/multiplexer-tokenfilter.asciidoc +++ b/docs/reference/analysis/tokenfilters/multiplexer-tokenfilter.asciidoc @@ -31,7 +31,7 @@ You can set it up like: [source,js] -------------------------------------------------- -PUT /multiplexer_example +PUT /multiplexer_example?include_type_name=true { "settings" : { "analysis" : { diff --git a/docs/reference/analysis/tokenfilters/pattern-capture-tokenfilter.asciidoc b/docs/reference/analysis/tokenfilters/pattern-capture-tokenfilter.asciidoc index 5b935d31f1289..a028abef7a1f2 100644 --- a/docs/reference/analysis/tokenfilters/pattern-capture-tokenfilter.asciidoc +++ b/docs/reference/analysis/tokenfilters/pattern-capture-tokenfilter.asciidoc @@ -46,7 +46,7 @@ This is particularly useful for indexing text like camel-case code, eg [source,js] -------------------------------------------------- -PUT test +PUT test?include_type_name=true { "settings" : { "analysis" : { @@ -87,7 +87,7 @@ Another example is analyzing email addresses: [source,js] -------------------------------------------------- -PUT test +PUT test?include_type_name=true { "settings" : { "analysis" : { diff --git a/docs/reference/analysis/tokenfilters/predicate-tokenfilter.asciidoc b/docs/reference/analysis/tokenfilters/predicate-tokenfilter.asciidoc index bebf7bd80f250..2e7f7c11631fe 100644 --- a/docs/reference/analysis/tokenfilters/predicate-tokenfilter.asciidoc +++ b/docs/reference/analysis/tokenfilters/predicate-tokenfilter.asciidoc @@ -17,7 +17,7 @@ You can set it up like: [source,js] -------------------------------------------------- -PUT /condition_example +PUT /condition_example?include_type_name=true { "settings" : { "analysis" : { diff --git a/docs/reference/analysis/tokenfilters/snowball-tokenfilter.asciidoc b/docs/reference/analysis/tokenfilters/snowball-tokenfilter.asciidoc index 99ed03649ff93..c9f8eff813626 100644 --- a/docs/reference/analysis/tokenfilters/snowball-tokenfilter.asciidoc +++ b/docs/reference/analysis/tokenfilters/snowball-tokenfilter.asciidoc @@ -12,7 +12,7 @@ For example: [source,js] -------------------------------------------------- -PUT /my_index +PUT /my_index?include_type_name=true { "settings": { "analysis" : { diff --git a/docs/reference/analysis/tokenfilters/stemmer-override-tokenfilter.asciidoc b/docs/reference/analysis/tokenfilters/stemmer-override-tokenfilter.asciidoc index e178181d1474a..b80bd517b1981 100644 --- a/docs/reference/analysis/tokenfilters/stemmer-override-tokenfilter.asciidoc +++ b/docs/reference/analysis/tokenfilters/stemmer-override-tokenfilter.asciidoc @@ -20,7 +20,7 @@ Here is an example: [source,js] -------------------------------------------------- -PUT /my_index +PUT /my_index?include_type_name=true { "settings": { "analysis" : { @@ -53,7 +53,7 @@ You can also define the overrides rules inline: [source,js] -------------------------------------------------- -PUT /my_index +PUT /my_index?include_type_name=true { "settings": { "analysis" : { diff --git a/docs/reference/analysis/tokenfilters/stemmer-tokenfilter.asciidoc b/docs/reference/analysis/tokenfilters/stemmer-tokenfilter.asciidoc index f59e2f3f2cf88..f6db7206b04f6 100644 --- a/docs/reference/analysis/tokenfilters/stemmer-tokenfilter.asciidoc +++ b/docs/reference/analysis/tokenfilters/stemmer-tokenfilter.asciidoc @@ -6,7 +6,7 @@ filters through a single unified interface. For example: [source,js] -------------------------------------------------- -PUT /my_index +PUT /my_index?include_type_name=true { "settings": { "analysis" : { diff --git a/docs/reference/analysis/tokenfilters/stop-tokenfilter.asciidoc b/docs/reference/analysis/tokenfilters/stop-tokenfilter.asciidoc index 3167a4342ac2d..579d2202c00ed 100644 --- a/docs/reference/analysis/tokenfilters/stop-tokenfilter.asciidoc +++ b/docs/reference/analysis/tokenfilters/stop-tokenfilter.asciidoc @@ -33,7 +33,7 @@ The `stopwords` parameter accepts either an array of stopwords: [source,js] ------------------------------------ -PUT /my_index +PUT /my_index?include_type_name=true { "settings": { "analysis": { @@ -53,7 +53,7 @@ or a predefined language-specific list: [source,js] ------------------------------------ -PUT /my_index +PUT /my_index?include_type_name=true { "settings": { "analysis": { diff --git a/docs/reference/analysis/tokenfilters/synonym-graph-tokenfilter.asciidoc b/docs/reference/analysis/tokenfilters/synonym-graph-tokenfilter.asciidoc index 2a555d7d044da..d614a6bcf4d82 100644 --- a/docs/reference/analysis/tokenfilters/synonym-graph-tokenfilter.asciidoc +++ b/docs/reference/analysis/tokenfilters/synonym-graph-tokenfilter.asciidoc @@ -23,7 +23,7 @@ Here is an example: [source,js] -------------------------------------------------- -PUT /test_index +PUT /test_index?include_type_name=true { "settings": { "index" : { @@ -59,7 +59,7 @@ to note that only those synonym rules which cannot get parsed are ignored. For i [source,js] -------------------------------------------------- -PUT /test_index +PUT /test_index?include_type_name=true { "settings": { "index" : { @@ -118,7 +118,7 @@ configuration file (note use of `synonyms` instead of `synonyms_path`): [source,js] -------------------------------------------------- -PUT /test_index +PUT /test_index?include_type_name=true { "settings": { "index" : { @@ -150,7 +150,7 @@ declared using `format`: [source,js] -------------------------------------------------- -PUT /test_index +PUT /test_index?include_type_name=true { "settings": { "index" : { diff --git a/docs/reference/analysis/tokenfilters/synonym-tokenfilter.asciidoc b/docs/reference/analysis/tokenfilters/synonym-tokenfilter.asciidoc index d0659f3425d3e..6bb42354c3556 100644 --- a/docs/reference/analysis/tokenfilters/synonym-tokenfilter.asciidoc +++ b/docs/reference/analysis/tokenfilters/synonym-tokenfilter.asciidoc @@ -7,7 +7,7 @@ Here is an example: [source,js] -------------------------------------------------- -PUT /test_index +PUT /test_index?include_type_name=true { "settings": { "index" : { @@ -46,7 +46,7 @@ to note that only those synonym rules which cannot get parsed are ignored. For i [source,js] -------------------------------------------------- -PUT /test_index +PUT /test_index?include_type_name=true { "settings": { "index" : { @@ -106,7 +106,7 @@ configuration file (note use of `synonyms` instead of `synonyms_path`): [source,js] -------------------------------------------------- -PUT /test_index +PUT /test_index?include_type_name=true { "settings": { "index" : { @@ -138,7 +138,7 @@ declared using `format`: [source,js] -------------------------------------------------- -PUT /test_index +PUT /test_index?include_type_name=true { "settings": { "index" : { diff --git a/docs/reference/analysis/tokenizers/classic-tokenizer.asciidoc b/docs/reference/analysis/tokenizers/classic-tokenizer.asciidoc index 52bdcbd773221..ca827e73ec6a8 100644 --- a/docs/reference/analysis/tokenizers/classic-tokenizer.asciidoc +++ b/docs/reference/analysis/tokenizers/classic-tokenizer.asciidoc @@ -145,7 +145,7 @@ In this example, we configure the `classic` tokenizer to have a [source,js] ---------------------------- -PUT my_index +PUT my_index?include_type_name=true { "settings": { "analysis": { diff --git a/docs/reference/analysis/tokenizers/edgengram-tokenizer.asciidoc b/docs/reference/analysis/tokenizers/edgengram-tokenizer.asciidoc index a34f5c801939e..14f94ff21d3dc 100644 --- a/docs/reference/analysis/tokenizers/edgengram-tokenizer.asciidoc +++ b/docs/reference/analysis/tokenizers/edgengram-tokenizer.asciidoc @@ -104,7 +104,7 @@ length `10`: [source,js] ---------------------------- -PUT my_index +PUT my_index?include_type_name=true { "settings": { "analysis": { @@ -222,7 +222,7 @@ Below is an example of how to set up a field for _search-as-you-type_: [source,js] ----------------------------------- -PUT my_index +PUT my_index?include_type_name=true { "settings": { "analysis": { diff --git a/docs/reference/analysis/tokenizers/ngram-tokenizer.asciidoc b/docs/reference/analysis/tokenizers/ngram-tokenizer.asciidoc index c182ffacd1cfe..c558a293927e0 100644 --- a/docs/reference/analysis/tokenizers/ngram-tokenizer.asciidoc +++ b/docs/reference/analysis/tokenizers/ngram-tokenizer.asciidoc @@ -209,7 +209,7 @@ digits as tokens, and to produce tri-grams (grams of length `3`): [source,js] ---------------------------- -PUT my_index +PUT my_index?include_type_name=true { "settings": { "analysis": { diff --git a/docs/reference/analysis/tokenizers/pathhierarchy-tokenizer.asciidoc b/docs/reference/analysis/tokenizers/pathhierarchy-tokenizer.asciidoc index 55aa7d66da343..b27c1fb7cefa3 100644 --- a/docs/reference/analysis/tokenizers/pathhierarchy-tokenizer.asciidoc +++ b/docs/reference/analysis/tokenizers/pathhierarchy-tokenizer.asciidoc @@ -93,7 +93,7 @@ characters, and to replace them with `/`. The first two tokens are skipped: [source,js] ---------------------------- -PUT my_index +PUT my_index?include_type_name=true { "settings": { "analysis": { diff --git a/docs/reference/analysis/tokenizers/pattern-tokenizer.asciidoc b/docs/reference/analysis/tokenizers/pattern-tokenizer.asciidoc index 5c19fcf59cc92..de52ea31372a4 100644 --- a/docs/reference/analysis/tokenizers/pattern-tokenizer.asciidoc +++ b/docs/reference/analysis/tokenizers/pattern-tokenizer.asciidoc @@ -125,7 +125,7 @@ tokens when it encounters commas: [source,js] ---------------------------- -PUT my_index +PUT my_index?include_type_name=true { "settings": { "analysis": { @@ -215,7 +215,7 @@ escaped, so the pattern ends up looking like: [source,js] ---------------------------- -PUT my_index +PUT my_index?include_type_name=true { "settings": { "analysis": { diff --git a/docs/reference/analysis/tokenizers/simplepattern-tokenizer.asciidoc b/docs/reference/analysis/tokenizers/simplepattern-tokenizer.asciidoc index adc5fc05deeb9..2f68a0b8937c0 100644 --- a/docs/reference/analysis/tokenizers/simplepattern-tokenizer.asciidoc +++ b/docs/reference/analysis/tokenizers/simplepattern-tokenizer.asciidoc @@ -36,7 +36,7 @@ three-digit numbers [source,js] ---------------------------- -PUT my_index +PUT my_index?include_type_name=true { "settings": { "analysis": { diff --git a/docs/reference/analysis/tokenizers/simplepatternsplit-tokenizer.asciidoc b/docs/reference/analysis/tokenizers/simplepatternsplit-tokenizer.asciidoc index fc2e186f97267..d74f8823ff350 100644 --- a/docs/reference/analysis/tokenizers/simplepatternsplit-tokenizer.asciidoc +++ b/docs/reference/analysis/tokenizers/simplepatternsplit-tokenizer.asciidoc @@ -37,7 +37,7 @@ text on underscores. [source,js] ---------------------------- -PUT my_index +PUT my_index?include_type_name=true { "settings": { "analysis": { diff --git a/docs/reference/analysis/tokenizers/standard-tokenizer.asciidoc b/docs/reference/analysis/tokenizers/standard-tokenizer.asciidoc index 9f77a0e13dc88..b19f31188002f 100644 --- a/docs/reference/analysis/tokenizers/standard-tokenizer.asciidoc +++ b/docs/reference/analysis/tokenizers/standard-tokenizer.asciidoc @@ -136,7 +136,7 @@ In this example, we configure the `standard` tokenizer to have a [source,js] ---------------------------- -PUT my_index +PUT my_index?include_type_name=true { "settings": { "analysis": { diff --git a/docs/reference/analysis/tokenizers/uaxurlemail-tokenizer.asciidoc b/docs/reference/analysis/tokenizers/uaxurlemail-tokenizer.asciidoc index 7fea0f1e8d8aa..8df2bc507528e 100644 --- a/docs/reference/analysis/tokenizers/uaxurlemail-tokenizer.asciidoc +++ b/docs/reference/analysis/tokenizers/uaxurlemail-tokenizer.asciidoc @@ -92,7 +92,7 @@ In this example, we configure the `uax_url_email` tokenizer to have a [source,js] ---------------------------- -PUT my_index +PUT my_index?include_type_name=true { "settings": { "analysis": { diff --git a/docs/reference/cat/alias.asciidoc b/docs/reference/cat/alias.asciidoc index 394231e448dc0..9cf831249dda9 100644 --- a/docs/reference/cat/alias.asciidoc +++ b/docs/reference/cat/alias.asciidoc @@ -8,7 +8,7 @@ including filter and routing infos. Hidden setup for example: [source,js] -------------------------------------------------- -PUT test1 +PUT test1?include_type_name=true { "aliases": { "alias1": {}, diff --git a/docs/reference/cat/fielddata.asciidoc b/docs/reference/cat/fielddata.asciidoc index e308e9d91f790..45615090dd074 100644 --- a/docs/reference/cat/fielddata.asciidoc +++ b/docs/reference/cat/fielddata.asciidoc @@ -9,7 +9,7 @@ on every data node in the cluster. Hidden setup snippet to build an index with fielddata so our results are real: [source,js] -------------------------------------------------- -PUT test +PUT test?include_type_name=true { "mappings": { "_doc": { diff --git a/docs/reference/ccr/getting-started.asciidoc b/docs/reference/ccr/getting-started.asciidoc index 4ee189793de9c..a517587534ca8 100644 --- a/docs/reference/ccr/getting-started.asciidoc +++ b/docs/reference/ccr/getting-started.asciidoc @@ -167,7 +167,7 @@ In the following example, we will create a leader index in the remote cluster: [source,js] -------------------------------------------------- -PUT /server-metrics +PUT /server-metrics?include_type_name=true { "settings" : { "index" : { diff --git a/docs/reference/docs/get.asciidoc b/docs/reference/docs/get.asciidoc index 5271b976f9677..47fee04609501 100644 --- a/docs/reference/docs/get.asciidoc +++ b/docs/reference/docs/get.asciidoc @@ -107,7 +107,7 @@ Consider for instance the following mapping: [source,js] -------------------------------------------------- -PUT twitter +PUT twitter?include_type_name=true { "mappings": { "_doc": { diff --git a/docs/reference/docs/termvectors.asciidoc b/docs/reference/docs/termvectors.asciidoc index 9ef08a7e774c2..d6227b3c36202 100644 --- a/docs/reference/docs/termvectors.asciidoc +++ b/docs/reference/docs/termvectors.asciidoc @@ -125,7 +125,7 @@ First, we create an index that stores term vectors, payloads etc. : [source,js] -------------------------------------------------- -PUT /twitter/ +PUT /twitter?include_type_name=true { "mappings": { "_doc": { "properties": { diff --git a/docs/reference/docs/update-by-query.asciidoc b/docs/reference/docs/update-by-query.asciidoc index 0221b68ce6935..deef09dbbd792 100644 --- a/docs/reference/docs/update-by-query.asciidoc +++ b/docs/reference/docs/update-by-query.asciidoc @@ -637,7 +637,7 @@ added a mapping value to pick up more fields from the data: [source,js] -------------------------------------------------- -PUT test +PUT test?include_type_name=true { "mappings": { "_doc": { @@ -659,7 +659,7 @@ POST test/_doc?refresh "text": "words words", "flag": "foo" } -PUT test/_mapping/_doc <2> +PUT test/_mapping <2> { "properties": { "text": {"type": "text"}, diff --git a/docs/reference/how-to/disk-usage.asciidoc b/docs/reference/how-to/disk-usage.asciidoc index 999a1c27e86b3..d58233a7e01cb 100644 --- a/docs/reference/how-to/disk-usage.asciidoc +++ b/docs/reference/how-to/disk-usage.asciidoc @@ -12,7 +12,7 @@ filter on, you can safely disable indexing on this field in your [source,js] -------------------------------------------------- -PUT index +PUT index?include_type_name=true { "mappings": { "_doc": { @@ -35,7 +35,7 @@ to not write norms to the index: [source,js] -------------------------------------------------- -PUT index +PUT index?include_type_name=true { "mappings": { "_doc": { @@ -58,7 +58,7 @@ Elasticsearch to not index positions: [source,js] -------------------------------------------------- -PUT index +PUT index?include_type_name=true { "mappings": { "_doc": { @@ -81,7 +81,7 @@ and scoring will assume that terms appear only once in every document. [source,js] -------------------------------------------------- -PUT index +PUT index?include_type_name=true { "mappings": { "_doc": { @@ -115,7 +115,7 @@ fields as `keyword`: [source,js] -------------------------------------------------- -PUT index +PUT index?include_type_name=true { "mappings": { "_doc": { diff --git a/docs/reference/how-to/recipes/stemming.asciidoc b/docs/reference/how-to/recipes/stemming.asciidoc index 83f1379cd32a0..c0ec3f2a04c09 100644 --- a/docs/reference/how-to/recipes/stemming.asciidoc +++ b/docs/reference/how-to/recipes/stemming.asciidoc @@ -9,7 +9,7 @@ content indexed in two different ways: [source,js] -------------------------------------------------- -PUT index +PUT index?include_type_name=true { "settings": { "analysis": { diff --git a/docs/reference/how-to/search-speed.asciidoc b/docs/reference/how-to/search-speed.asciidoc index f89eb4ed3cd39..4ab408be935ac 100644 --- a/docs/reference/how-to/search-speed.asciidoc +++ b/docs/reference/how-to/search-speed.asciidoc @@ -50,7 +50,7 @@ field. [source,js] -------------------------------------------------- -PUT movies +PUT movies?include_type_name=true { "mappings": { "_doc": { @@ -123,7 +123,7 @@ should be mapped as a <>: [source,js] -------------------------------------------------- -PUT index +PUT index?include_type_name=true { "mappings": { "_doc": { @@ -322,7 +322,7 @@ eagerly at refresh-time by configuring mappings as described below: [source,js] -------------------------------------------------- -PUT index +PUT index?include_type_name=true { "mappings": { "_doc": { diff --git a/docs/reference/ilm/apis/explain.asciidoc b/docs/reference/ilm/apis/explain.asciidoc index 66762ead9eb32..d23faf9f75d54 100644 --- a/docs/reference/ilm/apis/explain.asciidoc +++ b/docs/reference/ilm/apis/explain.asciidoc @@ -64,7 +64,7 @@ PUT _ilm/policy/my_policy } } -PUT my_index +PUT my_index?include_type_name=true { "settings": { "index.lifecycle.name": "my_policy", diff --git a/docs/reference/ilm/apis/move-to-step.asciidoc b/docs/reference/ilm/apis/move-to-step.asciidoc index 57ea1a226ea40..207d5139298b7 100644 --- a/docs/reference/ilm/apis/move-to-step.asciidoc +++ b/docs/reference/ilm/apis/move-to-step.asciidoc @@ -72,7 +72,7 @@ PUT _ilm/policy/my_policy } } -PUT my_index +PUT my_index?include_type_name=true { "settings": { "index.lifecycle.name": "my_policy" diff --git a/docs/reference/ilm/apis/remove-policy-from-index.asciidoc b/docs/reference/ilm/apis/remove-policy-from-index.asciidoc index 888d3f17eecac..81646cc135000 100644 --- a/docs/reference/ilm/apis/remove-policy-from-index.asciidoc +++ b/docs/reference/ilm/apis/remove-policy-from-index.asciidoc @@ -62,7 +62,7 @@ PUT _ilm/policy/my_policy } } -PUT my_index +PUT my_index?include_type_name=true { "settings": { "index.lifecycle.name": "my_policy" diff --git a/docs/reference/ilm/error-handling.asciidoc b/docs/reference/ilm/error-handling.asciidoc index abe643255bf95..a671e33f2b1db 100644 --- a/docs/reference/ilm/error-handling.asciidoc +++ b/docs/reference/ilm/error-handling.asciidoc @@ -41,7 +41,7 @@ telling it to use the policy they have created: [source,js] -------------------------------------------------- -PUT /myindex +PUT /myindex?include_type_name=true { "settings": { "index.number_of_shards": 2, diff --git a/docs/reference/ilm/getting-started-ilm.asciidoc b/docs/reference/ilm/getting-started-ilm.asciidoc index f06c95f49c067..5c5e188a9d474 100644 --- a/docs/reference/ilm/getting-started-ilm.asciidoc +++ b/docs/reference/ilm/getting-started-ilm.asciidoc @@ -107,7 +107,7 @@ To begin, we will want to bootstrap our first index to write to. [source,js] ----------------------- -PUT datastream-000001 +PUT datastream-000001?include_type_name=true { "aliases": { "datastream": { diff --git a/docs/reference/ilm/policy-definitions.asciidoc b/docs/reference/ilm/policy-definitions.asciidoc index 1253c87b3943e..c4125496b38fb 100644 --- a/docs/reference/ilm/policy-definitions.asciidoc +++ b/docs/reference/ilm/policy-definitions.asciidoc @@ -353,7 +353,7 @@ index "my_index" must be the write index for the alias. For more information, re [source,js] -------------------------------------------------- -PUT my_index +PUT my_index?include_type_name=true { "settings": { "index.lifecycle.name": "my_policy", diff --git a/docs/reference/ilm/set-up-lifecycle-policy.asciidoc b/docs/reference/ilm/set-up-lifecycle-policy.asciidoc index 7af686238f334..417b4bf9ef875 100644 --- a/docs/reference/ilm/set-up-lifecycle-policy.asciidoc +++ b/docs/reference/ilm/set-up-lifecycle-policy.asciidoc @@ -73,7 +73,7 @@ initial index which will be managed by our policy: [source,js] ----------------------- -PUT test-000001 +PUT test-000001?include_type_name=true { "aliases": { "test-alias":{ @@ -96,7 +96,7 @@ request so {ilm} immediately starts managing the index: [source,js] ----------------------- -PUT test-index +PUT test-index?include_type_name=true { "settings": { "number_of_shards": 1, diff --git a/docs/reference/ilm/start-stop-ilm.asciidoc b/docs/reference/ilm/start-stop-ilm.asciidoc index e5366f028a9c7..1b5666f141096 100644 --- a/docs/reference/ilm/start-stop-ilm.asciidoc +++ b/docs/reference/ilm/start-stop-ilm.asciidoc @@ -39,7 +39,7 @@ PUT _ilm/policy/my_policy } } -PUT my_index +PUT my_index?include_type_name=true { "settings": { "index.lifecycle.name": "my_policy" diff --git a/docs/reference/ilm/update-lifecycle-policy.asciidoc b/docs/reference/ilm/update-lifecycle-policy.asciidoc index 3e6627fdd3a7e..da3983d053c59 100644 --- a/docs/reference/ilm/update-lifecycle-policy.asciidoc +++ b/docs/reference/ilm/update-lifecycle-policy.asciidoc @@ -168,7 +168,7 @@ PUT _ilm/policy/my_executing_policy //// [source,js] ------------------------ -PUT my_index +PUT my_index?include_type_name=true { "settings": { "index.lifecycle.name": "my_executing_policy" @@ -486,7 +486,7 @@ PUT _ilm/policy/my_other_policy } } -PUT my_index +PUT my_index?include_type_name=true { "settings": { "index.lifecycle.name": "my_policy" diff --git a/docs/reference/index-modules/allocation/prioritization.asciidoc b/docs/reference/index-modules/allocation/prioritization.asciidoc index 6693e6adb755e..f702a2f20f67c 100644 --- a/docs/reference/index-modules/allocation/prioritization.asciidoc +++ b/docs/reference/index-modules/allocation/prioritization.asciidoc @@ -19,14 +19,14 @@ PUT index_1 PUT index_2 -PUT index_3 +PUT index_3?include_type_name=true { "settings": { "index.priority": 10 } } -PUT index_4 +PUT index_4?include_type_name=true { "settings": { "index.priority": 5 diff --git a/docs/reference/index-modules/index-sorting.asciidoc b/docs/reference/index-modules/index-sorting.asciidoc index 30ea9f43c6d7c..1a0f8c65dc98b 100644 --- a/docs/reference/index-modules/index-sorting.asciidoc +++ b/docs/reference/index-modules/index-sorting.asciidoc @@ -14,7 +14,7 @@ For instance the following example shows how to define a sort on a single field: [source,js] -------------------------------------------------- -PUT twitter +PUT twitter?include_type_name=true { "settings" : { "index" : { @@ -42,7 +42,7 @@ It is also possible to sort the index by more than one field: [source,js] -------------------------------------------------- -PUT twitter +PUT twitter?include_type_name=true { "settings" : { "index" : { @@ -118,7 +118,7 @@ For example, let's say we have an index that contains events sorted by a timesta [source,js] -------------------------------------------------- -PUT events +PUT events?include_type_name=true { "settings" : { "index" : { diff --git a/docs/reference/index-modules/similarity.asciidoc b/docs/reference/index-modules/similarity.asciidoc index 06abaadd245ec..4d2404c3b52b0 100644 --- a/docs/reference/index-modules/similarity.asciidoc +++ b/docs/reference/index-modules/similarity.asciidoc @@ -20,7 +20,7 @@ settings. [source,js] -------------------------------------------------- -PUT /index +PUT /index?include_type_name=true { "settings" : { "index" : { @@ -44,7 +44,7 @@ Here we configure the DFRSimilarity so it can be referenced as [source,js] -------------------------------------------------- -PUT /index/_mapping/_doc +PUT /index/_mapping { "properties" : { "title" : { "type" : "text", "similarity" : "my_similarity" } @@ -200,7 +200,7 @@ TF-IDF: [source,js] -------------------------------------------------- -PUT /index +PUT /index?include_type_name=true { "settings": { "number_of_shards": 1, @@ -369,7 +369,7 @@ more efficient: [source,js] -------------------------------------------------- -PUT /index +PUT /index?include_type_name=true { "settings": { "number_of_shards": 1, @@ -537,7 +537,7 @@ it is <>: [source,js] -------------------------------------------------- -PUT /index +PUT /index?include_type_name=true { "settings": { "index": { diff --git a/docs/reference/index-modules/store.asciidoc b/docs/reference/index-modules/store.asciidoc index 8c1b99a42f2a6..1483a04868e81 100644 --- a/docs/reference/index-modules/store.asciidoc +++ b/docs/reference/index-modules/store.asciidoc @@ -24,7 +24,7 @@ creation time: [source,js] --------------------------------- -PUT /my_index +PUT /my_index?include_type_name=true { "settings": { "index.store.type": "niofs" @@ -114,7 +114,7 @@ or in the index settings at index creation time: [source,js] --------------------------------- -PUT /my_index +PUT /my_index?include_type_name=true { "settings": { "index.store.preload": ["nvd", "dvd"] diff --git a/docs/reference/indices/aliases.asciidoc b/docs/reference/indices/aliases.asciidoc index f4d4917528aa2..41cd8ce325e0d 100644 --- a/docs/reference/indices/aliases.asciidoc +++ b/docs/reference/indices/aliases.asciidoc @@ -142,7 +142,7 @@ exist in the mapping: [source,js] -------------------------------------------------- -PUT /test1 +PUT /test1?include_type_name=true { "mappings": { "_doc": { @@ -376,7 +376,7 @@ First create the index and add a mapping for the `user_id` field: [source,js] -------------------------------------------------- -PUT /users +PUT /users?include_type_name=true { "mappings" : { "_doc" : { @@ -416,7 +416,7 @@ Aliases can also be specified during <>: [source,js] -------------------------------------------------- -PUT /logs_20162801 +PUT /logs_20162801?include_type_name=true { "mappings" : { "_doc" : { diff --git a/docs/reference/indices/analyze.asciidoc b/docs/reference/indices/analyze.asciidoc index 0398e35962519..8570176282ef6 100644 --- a/docs/reference/indices/analyze.asciidoc +++ b/docs/reference/indices/analyze.asciidoc @@ -224,7 +224,7 @@ The following setting allows to limit the number of tokens that can be produced: [source,js] -------------------------------------------------- -PUT analyze_sample +PUT analyze_sample?include_type_name=true { "settings" : { "index.analyze.max_token_count" : 20000 diff --git a/docs/reference/indices/create-index.asciidoc b/docs/reference/indices/create-index.asciidoc index 7eac18c716a8e..54efcd9005ac5 100644 --- a/docs/reference/indices/create-index.asciidoc +++ b/docs/reference/indices/create-index.asciidoc @@ -78,7 +78,7 @@ that can be set when creating an index, please check the [[mappings]] === Mappings -The create index API allows to provide a type mapping: +The create index API allows for providing a mapping definition: [source,js] -------------------------------------------------- @@ -88,16 +88,18 @@ PUT test "number_of_shards" : 1 }, "mappings" : { - "_doc" : { - "properties" : { - "field1" : { "type" : "text" } - } + "properties" : { + "field1" : { "type" : "text" } } } } -------------------------------------------------- // CONSOLE +NOTE: Before 7.0.0, the 'mappings' definition used to include a type name. Although specifying +types in requests is now deprecated, a type can still be provided if the request parameter +include_type_name is set. For more details, please see <>. + [float] [[create-index-aliases]] === Aliases @@ -177,28 +179,3 @@ PUT test?wait_for_active_shards=2 A detailed explanation of `wait_for_active_shards` and its possible values can be found <>. - -[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` and putting mappings directly under `mappings` -in the index creation call. - -Here is an example: - -[source,js] --------------------------------------------------- -PUT test?include_type_name=false -{ - "mappings": { - "properties": { - "foo": { - "type": "keyword" - } - } - } -} --------------------------------------------------- -// CONSOLE diff --git a/docs/reference/indices/get-field-mapping.asciidoc b/docs/reference/indices/get-field-mapping.asciidoc index 644ffea458843..e857c6994a529 100644 --- a/docs/reference/indices/get-field-mapping.asciidoc +++ b/docs/reference/indices/get-field-mapping.asciidoc @@ -9,7 +9,7 @@ For example, consider the following mapping: [source,js] -------------------------------------------------- -PUT publications +PUT publications?include_type_name=true { "mappings": { "_doc": { @@ -35,7 +35,7 @@ The following returns the mapping of the field `title` only: [source,js] -------------------------------------------------- -GET publications/_mapping/_doc/field/title +GET publications/_mapping/field/title -------------------------------------------------- // CONSOLE @@ -46,29 +46,27 @@ For which the response is: { "publications": { "mappings": { - "_doc": { - "title": { - "full_name": "title", - "mapping": { - "title": { - "type": "text" - } - } - } - } - } + "title": { + "full_name": "title", + "mapping": { + "title": { + "type": "text" + } + } + } + } } } -------------------------------------------------- // TESTRESPONSE [float] -=== Multiple Indices, Types and Fields +=== Multiple Indices and Fields -The get field mapping API can be used to get the mapping of multiple fields from more than one index or type +The get field mapping API can be used to get the mapping of multiple fields from more than one index with a single call. General usage of the API follows the -following syntax: `host:port/{index}/{type}/_mapping/field/{field}` where -`{index}`, `{type}` and `{field}` can stand for comma-separated list of names or wild cards. To +following syntax: `host:port/{index}/_mapping/field/{field}` where +`{index}` and `{field}` can stand for comma-separated list of names or wild cards. To get mappings for all indices you can use `_all` for `{index}`. The following are some examples: @@ -76,9 +74,9 @@ following are some examples: -------------------------------------------------- GET /twitter,kimchy/_mapping/field/message -GET /_all/_mapping/_doc/field/message,user.id +GET /_all/_mapping/field/message,user.id -GET /_all/_mapping/_doc/field/*.id +GET /_all/_mapping/field/*.id -------------------------------------------------- // CONSOLE // TEST[setup:twitter] @@ -93,7 +91,7 @@ For instance to select the `id` of the `author` field, you must use its full nam [source,js] -------------------------------------------------- -GET publications/_mapping/_doc/field/author.id,abstract,name +GET publications/_mapping/field/author.id,abstract,name -------------------------------------------------- // CONSOLE @@ -104,25 +102,23 @@ returns: { "publications": { "mappings": { - "_doc": { - "author.id": { - "full_name": "author.id", - "mapping": { - "id": { - "type": "text" - } - } - }, - "abstract": { - "full_name": "abstract", - "mapping": { - "abstract": { - "type": "text" - } - } - } - } - } + "author.id": { + "full_name": "author.id", + "mapping": { + "id": { + "type": "text" + } + } + }, + "abstract": { + "full_name": "abstract", + "mapping": { + "abstract": { + "type": "text" + } + } + } + } } } -------------------------------------------------- @@ -132,7 +128,7 @@ The get field mapping API also supports wildcard notation. [source,js] -------------------------------------------------- -GET publications/_mapping/_doc/field/a* +GET publications/_mapping/field/a* -------------------------------------------------- // CONSOLE @@ -143,29 +139,27 @@ returns: { "publications": { "mappings": { - "_doc": { - "author.name": { - "full_name": "author.name", - "mapping": { - "name": { - "type": "text" - } + "author.name": { + "full_name": "author.name", + "mapping": { + "name": { + "type": "text" } - }, - "abstract": { - "full_name": "abstract", - "mapping": { - "abstract": { - "type": "text" - } + } + }, + "abstract": { + "full_name": "abstract", + "mapping": { + "abstract": { + "type": "text" } - }, - "author.id": { - "full_name": "author.id", - "mapping": { - "id": { - "type": "text" - } + } + }, + "author.id": { + "full_name": "author.id", + "mapping": { + "id": { + "type": "text" } } } diff --git a/docs/reference/indices/get-index.asciidoc b/docs/reference/indices/get-index.asciidoc index 8e254a576c11d..24abc33a6929a 100644 --- a/docs/reference/indices/get-index.asciidoc +++ b/docs/reference/indices/get-index.asciidoc @@ -16,62 +16,6 @@ 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/] +NOTE: Before 7.0.0, the 'mappings' definition used to include a type name. Although mappings +in responses no longer contain a type name by default, you can still request the old format +through the parameter include_type_name. For more details, please see <>. diff --git a/docs/reference/indices/get-mapping.asciidoc b/docs/reference/indices/get-mapping.asciidoc index 4bca1a9d09d35..fbd4a860b31e8 100644 --- a/docs/reference/indices/get-mapping.asciidoc +++ b/docs/reference/indices/get-mapping.asciidoc @@ -6,29 +6,31 @@ index/type. [source,js] -------------------------------------------------- -GET /twitter/_mapping/_doc +GET /twitter/_mapping -------------------------------------------------- // CONSOLE // TEST[setup:twitter] +NOTE: Before 7.0.0, the 'mappings' definition used to include a type name. Although mappings +in responses no longer contain a type name by default, you can still request the old format +through the parameter include_type_name. For more details, please see <>. + [float] -=== Multiple Indices and Types +=== Multiple Indices -The get mapping API can be used to get more than one index or type -mapping with a single call. General usage of the API follows the -following syntax: `host:port/{index}/_mapping/{type}` where both -`{index}` and `{type}` can accept a comma-separated list of names. To -get mappings for all indices you can use `_all` for `{index}`. The -following are some examples: +The get mapping API can be used to get more than one index with a +single call. General usage of the API follows the following syntax: +`host:port/{index}/_mapping` where `{index}` can accept a comma-separated +list of names. To get mappings for all indices you can use `_all` for `{index}`. +The following are some examples: [source,js] -------------------------------------------------- -GET /_mapping/_doc - -GET /_all/_mapping/_doc +GET /twitter,kimchy/_mapping -------------------------------------------------- // CONSOLE // TEST[setup:twitter] +// TEST[s/^/PUT kimchy\nPUT book\n/] If you want to get mappings of all indices and types then the following two examples are equivalent: @@ -41,48 +43,3 @@ GET /_mapping -------------------------------------------------- // CONSOLE // TEST[setup:twitter] - -[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/_mappings?include_type_name=false --------------------------------------------------- -// CONSOLE - -which returns - -[source,js] --------------------------------------------------- -{ - "test": { - "mappings": { - "properties": { - "foo": { - "type": "keyword" - } - } - } - } -} --------------------------------------------------- -// TESTRESPONSE diff --git a/docs/reference/indices/put-mapping.asciidoc b/docs/reference/indices/put-mapping.asciidoc index 80dc52cd47d06..543497b8191dc 100644 --- a/docs/reference/indices/put-mapping.asciidoc +++ b/docs/reference/indices/put-mapping.asciidoc @@ -8,7 +8,7 @@ The PUT mapping API allows you to add fields to an existing index or to change s PUT twitter <1> {} -PUT twitter/_mapping/_doc <2> +PUT twitter/_mapping <2> { "properties": { "email": { @@ -18,11 +18,14 @@ PUT twitter/_mapping/_doc <2> } -------------------------------------------------- // CONSOLE -<1> <> called `twitter` without any type mapping. -<2> Uses the PUT mapping API to add a new field called `email` to the `_doc` mapping type. +<1> <> called `twitter` without any mapping. +<2> Uses the PUT mapping API to add a new field called `email`. -More information on how to define type mappings can be found in the -<> section. +More information on how to define mappings can be found in the <> section. + +NOTE: Before 7.0.0, the 'mappings' definition used to include a type name. Although specifying +types in requests is now deprecated, a type can still be provided if the request parameter +include_type_name is set. For more details, please see <>. [float] === Multi-index @@ -37,7 +40,7 @@ PUT twitter-1 PUT twitter-2 # Update both mappings -PUT /twitter-1,twitter-2/_mapping/_doc <1> +PUT /twitter-1,twitter-2/_mapping <1> { "properties": { "user_name": { @@ -65,7 +68,7 @@ For example: [source,js] ----------------------------------- -PUT my_index <1> +PUT my_index?include_type_name=true <1> { "mappings": { "_doc": { @@ -85,7 +88,7 @@ PUT my_index <1> } } -PUT my_index/_mapping/_doc +PUT my_index/_mapping { "properties": { "name": { @@ -109,54 +112,3 @@ PUT my_index/_mapping/_doc Each <> specifies whether or not its setting can be updated on an existing field. - -[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`. - -NOTE: This should only be done on indices that have been created with -`include_type_name=false` or that used `_doc` as a type name. - -The Console script from the above section is equivalent to the below invocation: - -[source,js] ------------------------------------ -PUT my_index?include_type_name=false <1> -{ - "mappings": { - "properties": { - "name": { - "properties": { - "first": { - "type": "text" - } - } - }, - "user_id": { - "type": "keyword" - } - } - } -} - -PUT my_index/_mapping?include_type_name=false -{ - "properties": { - "name": { - "properties": { - "last": { <2> - "type": "text" - } - } - }, - "user_id": { - "type": "keyword", - "ignore_above": 100 <3> - } - } -} ------------------------------------ -// CONSOLE diff --git a/docs/reference/indices/recovery.asciidoc b/docs/reference/indices/recovery.asciidoc index 0929b36e7742d..975beacc02cb2 100644 --- a/docs/reference/indices/recovery.asciidoc +++ b/docs/reference/indices/recovery.asciidoc @@ -24,7 +24,7 @@ indices recovery result. [source,js] -------------------------------------------------- # create the index -PUT index1 +PUT index1?include_type_name=true {"settings": {"index.number_of_shards": 1}} # create the repository diff --git a/docs/reference/indices/rollover-index.asciidoc b/docs/reference/indices/rollover-index.asciidoc index 1730c95e0dd24..d53fd46551983 100644 --- a/docs/reference/indices/rollover-index.asciidoc +++ b/docs/reference/indices/rollover-index.asciidoc @@ -187,7 +187,7 @@ override any values set in matching index templates. For example, the following [source,js] -------------------------------------------------- -PUT /logs-000001 +PUT /logs-000001?include_type_name=true { "aliases": { "logs_write": {} @@ -216,7 +216,7 @@ checked without performing the actual rollover: [source,js] -------------------------------------------------- -PUT /logs-000001 +PUT /logs-000001?include_type_name=true { "aliases": { "logs_write": {} @@ -258,7 +258,7 @@ Look at the behavior of the aliases in the following example where `is_write_ind [source,js] -------------------------------------------------- -PUT my_logs_index-000001 +PUT my_logs_index-000001?include_type_name=true { "aliases": { "logs": { "is_write_index": true } <1> diff --git a/docs/reference/indices/split-index.asciidoc b/docs/reference/indices/split-index.asciidoc index ade0a8075d582..465d7603a603d 100644 --- a/docs/reference/indices/split-index.asciidoc +++ b/docs/reference/indices/split-index.asciidoc @@ -86,7 +86,7 @@ Create a new index: [source,js] -------------------------------------------------- -PUT my_source_index +PUT my_source_index?include_type_name=true { "settings": { "index.number_of_shards" : 1 diff --git a/docs/reference/indices/templates.asciidoc b/docs/reference/indices/templates.asciidoc index 02c0b7a4cbe82..c9ac3e61c29b5 100644 --- a/docs/reference/indices/templates.asciidoc +++ b/docs/reference/indices/templates.asciidoc @@ -23,18 +23,16 @@ PUT _template/template_1 "number_of_shards": 1 }, "mappings": { - "_doc": { - "_source": { - "enabled": false + "_source": { + "enabled": false + }, + "properties": { + "host_name": { + "type": "keyword" }, - "properties": { - "host_name": { - "type": "keyword" - }, - "created_at": { - "type": "date", - "format": "EEE MMM dd HH:mm:ss Z yyyy" - } + "created_at": { + "type": "date", + "format": "EEE MMM dd HH:mm:ss Z yyyy" } } } @@ -137,6 +135,10 @@ The HTTP status code indicates if the template with the given name exists or not. Status code `200` means it exists and `404` means it does not. +NOTE: Before 7.0.0, the 'mappings' definition used to include a type name. Although mappings +no longer contain a type name by default, you can still use the old format by setting +the parameter include_type_name. For more details, please see <>. + [float] [[multiple-templates]] === Multiple Templates Matching @@ -157,9 +159,7 @@ PUT /_template/template_1 "number_of_shards" : 1 }, "mappings" : { - "_doc" : { - "_source" : { "enabled" : false } - } + "_source" : { "enabled" : false } } } @@ -171,9 +171,7 @@ PUT /_template/template_2 "number_of_shards" : 1 }, "mappings" : { - "_doc" : { - "_source" : { "enabled" : true } - } + "_source" : { "enabled" : true } } } -------------------------------------------------- diff --git a/docs/reference/ingest/processors/geoip.asciidoc b/docs/reference/ingest/processors/geoip.asciidoc index 7b80422c3aa2f..f9f902395c4b2 100644 --- a/docs/reference/ingest/processors/geoip.asciidoc +++ b/docs/reference/ingest/processors/geoip.asciidoc @@ -196,7 +196,7 @@ You can use the following mapping for the example index above: [source,js] -------------------------------------------------- -PUT my_ip_locations +PUT my_ip_locations?include_type_name=true { "mappings": { "_doc": { diff --git a/docs/reference/mapping.asciidoc b/docs/reference/mapping.asciidoc index 926d2c2ecb5dc..d7a8864431710 100644 --- a/docs/reference/mapping.asciidoc +++ b/docs/reference/mapping.asciidoc @@ -137,7 +137,7 @@ A mapping could be specified when creating an index, as follows: [source,js] --------------------------------------- -PUT my_index <1> +PUT my_index?include_type_name=true <1> { "mappings": { "_doc": { <2> diff --git a/docs/reference/mapping/dynamic/field-mapping.asciidoc b/docs/reference/mapping/dynamic/field-mapping.asciidoc index 5e0cc7e0bd6d0..c7a96a33473e5 100644 --- a/docs/reference/mapping/dynamic/field-mapping.asciidoc +++ b/docs/reference/mapping/dynamic/field-mapping.asciidoc @@ -66,7 +66,7 @@ Dynamic date detection can be disabled by setting `date_detection` to `false`: [source,js] -------------------------------------------------- -PUT my_index +PUT my_index?include_type_name=true { "mappings": { "_doc": { @@ -91,7 +91,7 @@ own <>: [source,js] -------------------------------------------------- -PUT my_index +PUT my_index?include_type_name=true { "mappings": { "_doc": { @@ -119,7 +119,7 @@ correct solution is to map these fields explicitly, but numeric detection [source,js] -------------------------------------------------- -PUT my_index +PUT my_index?include_type_name=true { "mappings": { "_doc": { diff --git a/docs/reference/mapping/dynamic/templates.asciidoc b/docs/reference/mapping/dynamic/templates.asciidoc index 3ad7da6e17744..f10a9d3475b66 100644 --- a/docs/reference/mapping/dynamic/templates.asciidoc +++ b/docs/reference/mapping/dynamic/templates.asciidoc @@ -69,7 +69,7 @@ could use the following template: [source,js] -------------------------------------------------- -PUT my_index +PUT my_index?include_type_name=true { "mappings": { "_doc": { @@ -125,7 +125,7 @@ fields: [source,js] -------------------------------------------------- -PUT my_index +PUT my_index?include_type_name=true { "mappings": { "_doc": { @@ -181,7 +181,7 @@ top-level `full_name` field, except for the `middle` field: [source,js] -------------------------------------------------- -PUT my_index +PUT my_index?include_type_name=true { "mappings": { "_doc": { @@ -222,7 +222,7 @@ field, and disables <> for all non-string fields: [source,js] -------------------------------------------------- -PUT my_index +PUT my_index?include_type_name=true { "mappings": { "_doc": { @@ -276,7 +276,7 @@ you will have to search on the exact same value that was indexed. [source,js] -------------------------------------------------- -PUT my_index +PUT my_index?include_type_name=true { "mappings": { "_doc": { @@ -306,7 +306,7 @@ before 5.0): [source,js] -------------------------------------------------- -PUT my_index +PUT my_index?include_type_name=true { "mappings": { "_doc": { @@ -334,7 +334,7 @@ disable the storage of these scoring factors in the index and save some space. [source,js] -------------------------------------------------- -PUT my_index +PUT my_index?include_type_name=true { "mappings": { "_doc": { @@ -375,7 +375,7 @@ maybe gain some indexing speed: [source,js] -------------------------------------------------- -PUT my_index +PUT my_index?include_type_name=true { "mappings": { "_doc": { diff --git a/docs/reference/mapping/fields/field-names-field.asciidoc b/docs/reference/mapping/fields/field-names-field.asciidoc index 6cba81b54f1ba..acc401cf0ec78 100644 --- a/docs/reference/mapping/fields/field-names-field.asciidoc +++ b/docs/reference/mapping/fields/field-names-field.asciidoc @@ -21,7 +21,7 @@ execute `exists` queries using those fields you might want to disable [source,js] -------------------------------------------------- -PUT tweets +PUT tweets?include_type_name=true { "mappings": { "_doc": { diff --git a/docs/reference/mapping/fields/meta-field.asciidoc b/docs/reference/mapping/fields/meta-field.asciidoc index 2e6c148b18f9d..bdfa7d94d2c0c 100644 --- a/docs/reference/mapping/fields/meta-field.asciidoc +++ b/docs/reference/mapping/fields/meta-field.asciidoc @@ -7,7 +7,7 @@ metadata, such as the class that a document belongs to: [source,js] -------------------------------------------------- -PUT my_index +PUT my_index?include_type_name=true { "mappings": { "_doc": { @@ -31,7 +31,7 @@ The `_meta` field can be updated on an existing type using the [source,js] -------------------------------------------------- -PUT my_index/_mapping/_doc +PUT my_index/_mapping { "_meta": { "class": "MyApp2::User3", diff --git a/docs/reference/mapping/fields/routing-field.asciidoc b/docs/reference/mapping/fields/routing-field.asciidoc index 07fbb80bcb664..ca823b2430b12 100644 --- a/docs/reference/mapping/fields/routing-field.asciidoc +++ b/docs/reference/mapping/fields/routing-field.asciidoc @@ -79,7 +79,7 @@ custom `routing` value required for all CRUD operations: [source,js] ------------------------------ -PUT my_index2 +PUT my_index2?include_type_name=true { "mappings": { "_doc": { diff --git a/docs/reference/mapping/fields/source-field.asciidoc b/docs/reference/mapping/fields/source-field.asciidoc index e872d31e56326..8f7943ebc0ff6 100644 --- a/docs/reference/mapping/fields/source-field.asciidoc +++ b/docs/reference/mapping/fields/source-field.asciidoc @@ -13,7 +13,7 @@ within the index. For this reason, it can be disabled as follows: [source,js] -------------------------------------------------- -PUT tweets +PUT tweets?include_type_name=true { "mappings": { "_doc": { @@ -85,7 +85,7 @@ as follows: [source,js] -------------------------------------------------- -PUT logs +PUT logs?include_type_name=true { "mappings": { "_doc": { diff --git a/docs/reference/mapping/params/analyzer.asciidoc b/docs/reference/mapping/params/analyzer.asciidoc index c885b99ab6313..43cfb7082796f 100644 --- a/docs/reference/mapping/params/analyzer.asciidoc +++ b/docs/reference/mapping/params/analyzer.asciidoc @@ -41,7 +41,7 @@ in the field mapping, as follows: [source,js] -------------------------------------------------- -PUT /my_index +PUT /my_index?include_type_name=true { "mappings": { "_doc": { @@ -93,7 +93,7 @@ To disable stop words for phrases a field utilising three analyzer settings will [source,js] -------------------------------------------------- -PUT my_index +PUT my_index?include_type_name=true { "settings":{ "analysis":{ diff --git a/docs/reference/mapping/params/boost.asciidoc b/docs/reference/mapping/params/boost.asciidoc index f2a5e48483c70..c6f6d104c0ca0 100644 --- a/docs/reference/mapping/params/boost.asciidoc +++ b/docs/reference/mapping/params/boost.asciidoc @@ -6,7 +6,7 @@ Individual fields can be _boosted_ automatically -- count more towards the relev [source,js] -------------------------------------------------- -PUT my_index +PUT my_index?include_type_name=true { "mappings": { "_doc": { diff --git a/docs/reference/mapping/params/coerce.asciidoc b/docs/reference/mapping/params/coerce.asciidoc index fff0d27b6ff27..dbb7ca84ba6fd 100644 --- a/docs/reference/mapping/params/coerce.asciidoc +++ b/docs/reference/mapping/params/coerce.asciidoc @@ -17,7 +17,7 @@ For instance: [source,js] -------------------------------------------------- -PUT my_index +PUT my_index?include_type_name=true { "mappings": { "_doc": { @@ -61,7 +61,7 @@ coercion globally across all mapping types: [source,js] -------------------------------------------------- -PUT my_index +PUT my_index?include_type_name=true { "settings": { "index.mapping.coerce": false diff --git a/docs/reference/mapping/params/copy-to.asciidoc b/docs/reference/mapping/params/copy-to.asciidoc index 6a1aef62bc391..d56258aa73324 100644 --- a/docs/reference/mapping/params/copy-to.asciidoc +++ b/docs/reference/mapping/params/copy-to.asciidoc @@ -8,7 +8,7 @@ the `full_name` field as follows: [source,js] -------------------------------------------------- -PUT my_index +PUT my_index?include_type_name=true { "mappings": { "_doc": { diff --git a/docs/reference/mapping/params/doc-values.asciidoc b/docs/reference/mapping/params/doc-values.asciidoc index 88bc944e3c589..74efc1bbbfabd 100644 --- a/docs/reference/mapping/params/doc-values.asciidoc +++ b/docs/reference/mapping/params/doc-values.asciidoc @@ -23,7 +23,7 @@ value from a script, you can disable doc values in order to save disk space: [source,js] -------------------------------------------------- -PUT my_index +PUT my_index?include_type_name=true { "mappings": { "_doc": { diff --git a/docs/reference/mapping/params/dynamic.asciidoc b/docs/reference/mapping/params/dynamic.asciidoc index 63303abdefd75..f7159724f48f4 100644 --- a/docs/reference/mapping/params/dynamic.asciidoc +++ b/docs/reference/mapping/params/dynamic.asciidoc @@ -58,7 +58,7 @@ object or from the mapping type. For instance: [source,js] -------------------------------------------------- -PUT my_index +PUT my_index?include_type_name=true { "mappings": { "_doc": { diff --git a/docs/reference/mapping/params/eager-global-ordinals.asciidoc b/docs/reference/mapping/params/eager-global-ordinals.asciidoc index 1b4cee65e6173..8973be951129c 100644 --- a/docs/reference/mapping/params/eager-global-ordinals.asciidoc +++ b/docs/reference/mapping/params/eager-global-ordinals.asciidoc @@ -36,7 +36,7 @@ aggregations: [source,js] ------------ -PUT my_index/_mapping/_doc +PUT my_index/_mapping { "properties": { "tags": { @@ -59,7 +59,7 @@ time: [source,js] ------------ -PUT my_index/_mapping/_doc +PUT my_index/_mapping { "properties": { "tags": { diff --git a/docs/reference/mapping/params/enabled.asciidoc b/docs/reference/mapping/params/enabled.asciidoc index 111d431cd25a1..fbcd25c716bbf 100644 --- a/docs/reference/mapping/params/enabled.asciidoc +++ b/docs/reference/mapping/params/enabled.asciidoc @@ -15,7 +15,7 @@ in any other way: [source,js] -------------------------------------------------- -PUT my_index +PUT my_index?include_type_name=true { "mappings": { "_doc": { @@ -63,7 +63,7 @@ retrieved, but none of its contents are indexed in any way: [source,js] -------------------------------------------------- -PUT my_index +PUT my_index?include_type_name=true { "mappings": { "_doc": { <1> diff --git a/docs/reference/mapping/params/fielddata.asciidoc b/docs/reference/mapping/params/fielddata.asciidoc index ff959b98b6e7f..eee7463c6400c 100644 --- a/docs/reference/mapping/params/fielddata.asciidoc +++ b/docs/reference/mapping/params/fielddata.asciidoc @@ -55,7 +55,7 @@ enabled for aggregations, as follows: [source,js] --------------------------------- -PUT my_index +PUT my_index?include_type_name=true { "mappings": { "_doc": { @@ -84,7 +84,7 @@ You can enable fielddata on an existing `text` field using the [source,js] ----------------------------------- -PUT my_index/_mapping/_doc +PUT my_index/_mapping { "properties": { "my_field": { <1> @@ -118,7 +118,7 @@ number of docs that the segment should contain with `min_segment_size`: [source,js] -------------------------------------------------- -PUT my_index +PUT my_index?include_type_name=true { "mappings": { "_doc": { diff --git a/docs/reference/mapping/params/format.asciidoc b/docs/reference/mapping/params/format.asciidoc index 120233a8f0fdd..641433685d3a2 100644 --- a/docs/reference/mapping/params/format.asciidoc +++ b/docs/reference/mapping/params/format.asciidoc @@ -11,7 +11,7 @@ Besides the <>, your own [source,js] -------------------------------------------------- -PUT my_index +PUT my_index?include_type_name=true { "mappings": { "_doc": { diff --git a/docs/reference/mapping/params/ignore-above.asciidoc b/docs/reference/mapping/params/ignore-above.asciidoc index fe28e77cd6322..19c275dac5d01 100644 --- a/docs/reference/mapping/params/ignore-above.asciidoc +++ b/docs/reference/mapping/params/ignore-above.asciidoc @@ -8,7 +8,7 @@ NOTE: All strings/array elements will still be present in the `_source` field, i [source,js] -------------------------------------------------- -PUT my_index +PUT my_index?include_type_name=true { "mappings": { "_doc": { diff --git a/docs/reference/mapping/params/ignore-malformed.asciidoc b/docs/reference/mapping/params/ignore-malformed.asciidoc index be0bdfe4ffaf8..84a53515d9ebe 100644 --- a/docs/reference/mapping/params/ignore-malformed.asciidoc +++ b/docs/reference/mapping/params/ignore-malformed.asciidoc @@ -14,7 +14,7 @@ For example: [source,js] -------------------------------------------------- -PUT my_index +PUT my_index?include_type_name=true { "mappings": { "_doc": { @@ -61,7 +61,7 @@ allow to ignore malformed content globally across all mapping types. [source,js] -------------------------------------------------- -PUT my_index +PUT my_index?include_type_name=true { "settings": { "index.mapping.ignore_malformed": true <1> diff --git a/docs/reference/mapping/params/index-options.asciidoc b/docs/reference/mapping/params/index-options.asciidoc index 9bbdd017e0249..cda680399dde2 100644 --- a/docs/reference/mapping/params/index-options.asciidoc +++ b/docs/reference/mapping/params/index-options.asciidoc @@ -35,7 +35,7 @@ all other fields use `docs` as the default. [source,js] -------------------------------------------------- -PUT my_index +PUT my_index?include_type_name=true { "mappings": { "_doc": { diff --git a/docs/reference/mapping/params/index-prefixes.asciidoc b/docs/reference/mapping/params/index-prefixes.asciidoc index 203e063bbfb6f..baca8606f5c03 100644 --- a/docs/reference/mapping/params/index-prefixes.asciidoc +++ b/docs/reference/mapping/params/index-prefixes.asciidoc @@ -19,7 +19,7 @@ This example creates a text field using the default prefix length settings: [source,js] -------------------------------- -PUT my_index +PUT my_index?include_type_name=true { "mappings": { "_doc": { @@ -42,7 +42,7 @@ This example uses custom prefix length settings: [source,js] -------------------------------- -PUT my_index +PUT my_index?include_type_name=true { "mappings": { "_doc": { diff --git a/docs/reference/mapping/params/multi-fields.asciidoc b/docs/reference/mapping/params/multi-fields.asciidoc index 2f973a3f62b3a..e4bdb04506d92 100644 --- a/docs/reference/mapping/params/multi-fields.asciidoc +++ b/docs/reference/mapping/params/multi-fields.asciidoc @@ -8,7 +8,7 @@ search, and as a `keyword` field for sorting or aggregations: [source,js] -------------------------------------------------- -PUT my_index +PUT my_index?include_type_name=true { "mappings": { "_doc": { @@ -76,7 +76,7 @@ which stems words into their root form: [source,js] -------------------------------------------------- -PUT my_index +PUT my_index?include_type_name=true { "mappings": { "_doc": { diff --git a/docs/reference/mapping/params/normalizer.asciidoc b/docs/reference/mapping/params/normalizer.asciidoc index bfd24381753f7..79ba39e194726 100644 --- a/docs/reference/mapping/params/normalizer.asciidoc +++ b/docs/reference/mapping/params/normalizer.asciidoc @@ -12,7 +12,7 @@ such as the <> query. [source,js] -------------------------------- -PUT index +PUT index?include_type_name=true { "settings": { "analysis": { diff --git a/docs/reference/mapping/params/norms.asciidoc b/docs/reference/mapping/params/norms.asciidoc index 842f0af64b537..8a7be4baef8c5 100644 --- a/docs/reference/mapping/params/norms.asciidoc +++ b/docs/reference/mapping/params/norms.asciidoc @@ -20,7 +20,7 @@ Norms can be disabled (but not reenabled) after the fact, using the [source,js] ------------ -PUT my_index/_mapping/_doc +PUT my_index/_mapping { "properties": { "title": { diff --git a/docs/reference/mapping/params/null-value.asciidoc b/docs/reference/mapping/params/null-value.asciidoc index f85724e66d0ae..b2f46a2c7cef3 100644 --- a/docs/reference/mapping/params/null-value.asciidoc +++ b/docs/reference/mapping/params/null-value.asciidoc @@ -10,7 +10,7 @@ the specified value so that it can be indexed and searched. For instance: [source,js] -------------------------------------------------- -PUT my_index +PUT my_index?include_type_name=true { "mappings": { "_doc": { diff --git a/docs/reference/mapping/params/position-increment-gap.asciidoc b/docs/reference/mapping/params/position-increment-gap.asciidoc index c002d5a861aa8..ae6cc85865709 100644 --- a/docs/reference/mapping/params/position-increment-gap.asciidoc +++ b/docs/reference/mapping/params/position-increment-gap.asciidoc @@ -51,7 +51,7 @@ The `position_increment_gap` can be specified in the mapping. For instance: [source,js] -------------------------------------------------- -PUT my_index +PUT my_index?include_type_name=true { "mappings": { "_doc": { diff --git a/docs/reference/mapping/params/properties.asciidoc b/docs/reference/mapping/params/properties.asciidoc index e50c0b3ac777f..2efef0abf3cdc 100644 --- a/docs/reference/mapping/params/properties.asciidoc +++ b/docs/reference/mapping/params/properties.asciidoc @@ -15,7 +15,7 @@ field, and a `nested` field: [source,js] -------------------------------------------------- -PUT my_index +PUT my_index?include_type_name=true { "mappings": { "_doc": { <1> diff --git a/docs/reference/mapping/params/search-analyzer.asciidoc b/docs/reference/mapping/params/search-analyzer.asciidoc index 1a67dfab63182..eb483a3af384f 100644 --- a/docs/reference/mapping/params/search-analyzer.asciidoc +++ b/docs/reference/mapping/params/search-analyzer.asciidoc @@ -14,7 +14,7 @@ this can be overridden with the `search_analyzer` setting: [source,js] -------------------------------------------------- -PUT my_index +PUT my_index?include_type_name=true { "settings": { "analysis": { diff --git a/docs/reference/mapping/params/similarity.asciidoc b/docs/reference/mapping/params/similarity.asciidoc index a0be0fb3ccbeb..9e6e4e0877830 100644 --- a/docs/reference/mapping/params/similarity.asciidoc +++ b/docs/reference/mapping/params/similarity.asciidoc @@ -36,7 +36,7 @@ as follows: [source,js] -------------------------------------------------- -PUT my_index +PUT my_index?include_type_name=true { "mappings": { "_doc": { diff --git a/docs/reference/mapping/params/store.asciidoc b/docs/reference/mapping/params/store.asciidoc index 56c62385b59c8..186666ef5dd73 100644 --- a/docs/reference/mapping/params/store.asciidoc +++ b/docs/reference/mapping/params/store.asciidoc @@ -18,7 +18,7 @@ to extract those fields from a large `_source` field: [source,js] -------------------------------------------------- -PUT my_index +PUT my_index?include_type_name=true { "mappings": { "_doc": { diff --git a/docs/reference/mapping/params/term-vector.asciidoc b/docs/reference/mapping/params/term-vector.asciidoc index c71c23f8a4881..491a002f3d612 100644 --- a/docs/reference/mapping/params/term-vector.asciidoc +++ b/docs/reference/mapping/params/term-vector.asciidoc @@ -29,7 +29,7 @@ index. [source,js] -------------------------------------------------- -PUT my_index +PUT my_index?include_type_name=true { "mappings": { "_doc": { diff --git a/docs/reference/mapping/removal_of_types.asciidoc b/docs/reference/mapping/removal_of_types.asciidoc index 75690875e0eb8..251025c1ba24c 100644 --- a/docs/reference/mapping/removal_of_types.asciidoc +++ b/docs/reference/mapping/removal_of_types.asciidoc @@ -112,7 +112,7 @@ have looked something like this: [source,js] ---- -PUT twitter +PUT twitter?include_type_name=true { "mappings": { "user": { @@ -161,7 +161,7 @@ You could achieve the same thing by adding a custom `type` field as follows: [source,js] ---- -PUT twitter +PUT twitter?include_type_name=true { "mappings": { "_doc": { @@ -299,7 +299,7 @@ This first example splits our `twitter` index into a `tweets` index and a [source,js] ---- -PUT users +PUT users?include_type_name=true { "settings": { "index.mapping.single_type": true @@ -321,7 +321,7 @@ PUT users } } -PUT tweets +PUT tweets?include_type_name=true { "settings": { "index.mapping.single_type": true @@ -376,7 +376,7 @@ documents of different types which have conflicting IDs: [source,js] ---- -PUT new_twitter +PUT new_twitter?include_type_name=true { "mappings": { "_doc": { diff --git a/docs/reference/mapping/types/alias.asciidoc b/docs/reference/mapping/types/alias.asciidoc index b4243d80e6d08..75a2b65e5c110 100644 --- a/docs/reference/mapping/types/alias.asciidoc +++ b/docs/reference/mapping/types/alias.asciidoc @@ -7,7 +7,7 @@ and selected other APIs like <>. [source,js] -------------------------------- -PUT trips +PUT trips?include_type_name=true { "mappings": { "_doc": { diff --git a/docs/reference/mapping/types/binary.asciidoc b/docs/reference/mapping/types/binary.asciidoc index 9b7b032d0abf4..ebd50802b8d84 100644 --- a/docs/reference/mapping/types/binary.asciidoc +++ b/docs/reference/mapping/types/binary.asciidoc @@ -7,7 +7,7 @@ stored by default and is not searchable: [source,js] -------------------------------------------------- -PUT my_index +PUT my_index?include_type_name=true { "mappings": { "_doc": { diff --git a/docs/reference/mapping/types/boolean.asciidoc b/docs/reference/mapping/types/boolean.asciidoc index 4ae350412bece..3a92ae23e2fd9 100644 --- a/docs/reference/mapping/types/boolean.asciidoc +++ b/docs/reference/mapping/types/boolean.asciidoc @@ -17,7 +17,7 @@ For example: [source,js] -------------------------------------------------- -PUT my_index +PUT my_index?include_type_name=true { "mappings": { "_doc": { diff --git a/docs/reference/mapping/types/date.asciidoc b/docs/reference/mapping/types/date.asciidoc index 1beb64083b8c8..03024d1383945 100644 --- a/docs/reference/mapping/types/date.asciidoc +++ b/docs/reference/mapping/types/date.asciidoc @@ -30,7 +30,7 @@ For instance: [source,js] -------------------------------------------------- -PUT my_index +PUT my_index?include_type_name=true { "mappings": { "_doc": { @@ -74,7 +74,7 @@ into a string. [source,js] -------------------------------------------------- -PUT my_index +PUT my_index?include_type_name=true { "mappings": { "_doc": { diff --git a/docs/reference/mapping/types/dense-vector.asciidoc b/docs/reference/mapping/types/dense-vector.asciidoc index f948968ed3b5c..b12e9e120a479 100644 --- a/docs/reference/mapping/types/dense-vector.asciidoc +++ b/docs/reference/mapping/types/dense-vector.asciidoc @@ -17,7 +17,7 @@ You index a dense vector as an array of floats. [source,js] -------------------------------------------------- -PUT my_index +PUT my_index?include_type_name=true { "mappings": { "_doc": { diff --git a/docs/reference/mapping/types/feature-vector.asciidoc b/docs/reference/mapping/types/feature-vector.asciidoc index 2ce3c017fe3e5..25358ab9ec3a6 100644 --- a/docs/reference/mapping/types/feature-vector.asciidoc +++ b/docs/reference/mapping/types/feature-vector.asciidoc @@ -11,7 +11,7 @@ one field to the mappings for each of them. [source,js] -------------------------------------------------- -PUT my_index +PUT my_index?include_type_name=true { "mappings": { "_doc": { diff --git a/docs/reference/mapping/types/feature.asciidoc b/docs/reference/mapping/types/feature.asciidoc index 3b5e78d5fb46b..76eada86c59e5 100644 --- a/docs/reference/mapping/types/feature.asciidoc +++ b/docs/reference/mapping/types/feature.asciidoc @@ -6,7 +6,7 @@ documents in queries with a <> query. [source,js] -------------------------------------------------- -PUT my_index +PUT my_index?include_type_name=true { "mappings": { "_doc": { diff --git a/docs/reference/mapping/types/geo-point.asciidoc b/docs/reference/mapping/types/geo-point.asciidoc index 97f2ddb52825b..752e611db811f 100644 --- a/docs/reference/mapping/types/geo-point.asciidoc +++ b/docs/reference/mapping/types/geo-point.asciidoc @@ -15,7 +15,7 @@ There are four ways that a geo-point may be specified, as demonstrated below: [source,js] -------------------------------------------------- -PUT my_index +PUT my_index?include_type_name=true { "mappings": { "_doc": { diff --git a/docs/reference/mapping/types/geo-shape.asciidoc b/docs/reference/mapping/types/geo-shape.asciidoc index 92ee2d1065100..11ba1c55cb228 100644 --- a/docs/reference/mapping/types/geo-shape.asciidoc +++ b/docs/reference/mapping/types/geo-shape.asciidoc @@ -198,7 +198,7 @@ the cell right next to it -- even though the shape is very close to the point. [source,js] -------------------------------------------------- -PUT /example +PUT /example?include_type_name=true { "mappings": { "doc": { diff --git a/docs/reference/mapping/types/ip.asciidoc b/docs/reference/mapping/types/ip.asciidoc index 695cd1c626bc3..6206df61ac3b6 100644 --- a/docs/reference/mapping/types/ip.asciidoc +++ b/docs/reference/mapping/types/ip.asciidoc @@ -6,7 +6,7 @@ https://en.wikipedia.org/wiki/IPv6[IPv6] addresses. [source,js] -------------------------------------------------- -PUT my_index +PUT my_index?include_type_name=true { "mappings": { "_doc": { diff --git a/docs/reference/mapping/types/keyword.asciidoc b/docs/reference/mapping/types/keyword.asciidoc index 09d540feed19b..e9e913be3a6d0 100644 --- a/docs/reference/mapping/types/keyword.asciidoc +++ b/docs/reference/mapping/types/keyword.asciidoc @@ -15,7 +15,7 @@ Below is an example of a mapping for a keyword field: [source,js] -------------------------------- -PUT my_index +PUT my_index?include_type_name=true { "mappings": { "_doc": { diff --git a/docs/reference/mapping/types/nested.asciidoc b/docs/reference/mapping/types/nested.asciidoc index 238e26bf3375b..4681897575189 100644 --- a/docs/reference/mapping/types/nested.asciidoc +++ b/docs/reference/mapping/types/nested.asciidoc @@ -75,7 +75,7 @@ queried independently of the others, with the < diff --git a/docs/reference/mapping/types/parent-join.asciidoc b/docs/reference/mapping/types/parent-join.asciidoc index ea2e1f719a7ca..dacef7c4bc7cb 100644 --- a/docs/reference/mapping/types/parent-join.asciidoc +++ b/docs/reference/mapping/types/parent-join.asciidoc @@ -9,7 +9,7 @@ A parent/child relation can be defined as follows: [source,js] -------------------------------------------------- -PUT my_index +PUT my_index?include_type_name=true { "mappings": { "_doc": { @@ -319,7 +319,7 @@ make sense to disable eager loading: [source,js] -------------------------------------------------- -PUT my_index +PUT my_index?include_type_name=true { "mappings": { "_doc": { @@ -358,7 +358,7 @@ It is also possible to define multiple children for a single parent: [source,js] -------------------------------------------------- -PUT my_index +PUT my_index?include_type_name=true { "mappings": { "_doc": { @@ -388,7 +388,7 @@ Multiple levels of parent/child: [source,js] -------------------------------------------------- -PUT my_index +PUT my_index?include_type_name=true { "mappings": { "_doc": { diff --git a/docs/reference/mapping/types/percolator.asciidoc b/docs/reference/mapping/types/percolator.asciidoc index c6d31d942b979..7324826eb44ef 100644 --- a/docs/reference/mapping/types/percolator.asciidoc +++ b/docs/reference/mapping/types/percolator.asciidoc @@ -15,7 +15,7 @@ If the following mapping configures the `percolator` field type for the [source,js] -------------------------------------------------- -PUT my_index +PUT my_index?include_type_name=true { "mappings": { "_doc": { @@ -69,7 +69,7 @@ Lets take a look at the following index with a percolator field type: [source,js] -------------------------------------------------- -PUT index +PUT index?include_type_name=true { "mappings": { "_doc" : { @@ -117,7 +117,7 @@ to read your queries you need to reindex your queries into a new index on the cu [source,js] -------------------------------------------------- -PUT new_index +PUT new_index?include_type_name=true { "mappings": { "_doc" : { @@ -269,7 +269,7 @@ with these settings and mapping: [source,js] -------------------------------------------------- -PUT /test_index +PUT /test_index?include_type_name=true { "settings": { "analysis": { @@ -442,7 +442,7 @@ Creating an index with custom analysis settings: [source,js] -------------------------------------------------- -PUT my_queries1 +PUT my_queries1?include_type_name=true { "settings": { "analysis": { @@ -595,7 +595,7 @@ before the `edge_ngram` token filter. [source,js] -------------------------------------------------- -PUT my_queries2 +PUT my_queries2?include_type_name=true { "settings": { "analysis": { diff --git a/docs/reference/mapping/types/range.asciidoc b/docs/reference/mapping/types/range.asciidoc index fcf13ed446b0a..630458b4866e5 100644 --- a/docs/reference/mapping/types/range.asciidoc +++ b/docs/reference/mapping/types/range.asciidoc @@ -16,7 +16,7 @@ Below is an example of configuring a mapping with various range fields followed [source,js] -------------------------------------------------- -PUT range_index +PUT range_index?include_type_name=true { "settings": { "number_of_shards": 2 @@ -184,7 +184,7 @@ https://en.wikipedia.org/wiki/Classless_Inter-Domain_Routing#CIDR_notation[CIDR] [source,js] -------------------------------------------------- -PUT range_index/_mapping/_doc +PUT range_index/_mapping { "properties": { "ip_whitelist": { diff --git a/docs/reference/mapping/types/sparse-vector.asciidoc b/docs/reference/mapping/types/sparse-vector.asciidoc index 1ef02d4738ce7..df374e857ba6a 100644 --- a/docs/reference/mapping/types/sparse-vector.asciidoc +++ b/docs/reference/mapping/types/sparse-vector.asciidoc @@ -20,7 +20,7 @@ Dimensions don't need to be in order. [source,js] -------------------------------------------------- -PUT my_index +PUT my_index?include_type_name=true { "mappings": { "_doc": { diff --git a/docs/reference/mapping/types/text.asciidoc b/docs/reference/mapping/types/text.asciidoc index dd7e6cbbacfff..1b3c0a0eaf112 100644 --- a/docs/reference/mapping/types/text.asciidoc +++ b/docs/reference/mapping/types/text.asciidoc @@ -17,7 +17,7 @@ Below is an example of a mapping for a text field: [source,js] -------------------------------- -PUT my_index +PUT my_index?include_type_name=true { "mappings": { "_doc": { diff --git a/docs/reference/mapping/types/token-count.asciidoc b/docs/reference/mapping/types/token-count.asciidoc index 6f3295fab5ebb..469583abf1bcd 100644 --- a/docs/reference/mapping/types/token-count.asciidoc +++ b/docs/reference/mapping/types/token-count.asciidoc @@ -9,7 +9,7 @@ For instance: [source,js] -------------------------------------------------- -PUT my_index +PUT my_index?include_type_name=true { "mappings": { "_doc": { diff --git a/docs/reference/ml/transforms.asciidoc b/docs/reference/ml/transforms.asciidoc index 6fc67fa7c4e4b..02dd4d5469cfd 100644 --- a/docs/reference/ml/transforms.asciidoc +++ b/docs/reference/ml/transforms.asciidoc @@ -26,7 +26,7 @@ subsequent examples: [source,js] ---------------------------------- -PUT /my_index +PUT /my_index?include_type_name=true { "mappings":{ "_doc":{ diff --git a/docs/reference/modules/indices/request_cache.asciidoc b/docs/reference/modules/indices/request_cache.asciidoc index fc04c5e9c63f6..d13c108589836 100644 --- a/docs/reference/modules/indices/request_cache.asciidoc +++ b/docs/reference/modules/indices/request_cache.asciidoc @@ -55,7 +55,7 @@ index as follows: [source,js] ----------------------------- -PUT /my_index +PUT /my_index?include_type_name=true { "settings": { "index.requests.cache.enable": false diff --git a/docs/reference/modules/scripting/fields.asciidoc b/docs/reference/modules/scripting/fields.asciidoc index 408707207f010..d615038baa25f 100644 --- a/docs/reference/modules/scripting/fields.asciidoc +++ b/docs/reference/modules/scripting/fields.asciidoc @@ -172,7 +172,7 @@ For instance: [source,js] ------------------------------- -PUT my_index +PUT my_index?include_type_name=true { "mappings": { "_doc": { diff --git a/docs/reference/query-dsl/exists-query.asciidoc b/docs/reference/query-dsl/exists-query.asciidoc index 3e0fd71cd58b0..16828c3751a39 100644 --- a/docs/reference/query-dsl/exists-query.asciidoc +++ b/docs/reference/query-dsl/exists-query.asciidoc @@ -52,7 +52,7 @@ instance, if the `user` field were mapped as follows: [source,js] -------------------------------------------------- -PUT /example +PUT /example?include_type_name=true { "mappings": { "_doc": { diff --git a/docs/reference/query-dsl/feature-query.asciidoc b/docs/reference/query-dsl/feature-query.asciidoc index 5d1683854375b..c261133cff412 100644 --- a/docs/reference/query-dsl/feature-query.asciidoc +++ b/docs/reference/query-dsl/feature-query.asciidoc @@ -27,7 +27,7 @@ based or `pagerank`, `url_length` and the `sports` topic. [source,js] -------------------------------------------------- -PUT test +PUT test?include_type_name=true { "mappings": { "_doc": { diff --git a/docs/reference/query-dsl/geo-bounding-box-query.asciidoc b/docs/reference/query-dsl/geo-bounding-box-query.asciidoc index fdf5ca5de16e5..8c2b25ea0ee7a 100644 --- a/docs/reference/query-dsl/geo-bounding-box-query.asciidoc +++ b/docs/reference/query-dsl/geo-bounding-box-query.asciidoc @@ -6,7 +6,7 @@ bounding box. Assuming the following indexed document: [source,js] -------------------------------------------------- -PUT /my_locations +PUT /my_locations?include_type_name=true { "mappings": { "_doc": { diff --git a/docs/reference/query-dsl/geo-distance-query.asciidoc b/docs/reference/query-dsl/geo-distance-query.asciidoc index 2bf24c942cbcc..da1f99f834d67 100644 --- a/docs/reference/query-dsl/geo-distance-query.asciidoc +++ b/docs/reference/query-dsl/geo-distance-query.asciidoc @@ -7,7 +7,7 @@ document: [source,js] -------------------------------------------------- -PUT /my_locations +PUT /my_locations?include_type_name=true { "mappings": { "_doc": { diff --git a/docs/reference/query-dsl/geo-shape-query.asciidoc b/docs/reference/query-dsl/geo-shape-query.asciidoc index 059d0db14b51a..4121ebfc45edd 100644 --- a/docs/reference/query-dsl/geo-shape-query.asciidoc +++ b/docs/reference/query-dsl/geo-shape-query.asciidoc @@ -24,7 +24,7 @@ Given the following index: [source,js] -------------------------------------------------- -PUT /example +PUT /example?include_type_name=true { "mappings": { "_doc": { @@ -99,7 +99,7 @@ shape: [source,js] -------------------------------------------------- -PUT /shapes +PUT /shapes?include_type_name=true { "mappings": { "_doc": { diff --git a/docs/reference/query-dsl/mlt-query.asciidoc b/docs/reference/query-dsl/mlt-query.asciidoc index 64a2a6052df71..3d8b08b8ff44e 100644 --- a/docs/reference/query-dsl/mlt-query.asciidoc +++ b/docs/reference/query-dsl/mlt-query.asciidoc @@ -119,7 +119,7 @@ default, but there will be no speed up on analysis for these fields. [source,js] -------------------------------------------------- -PUT /imdb +PUT /imdb?include_type_name=true { "mappings": { "movies": { diff --git a/docs/reference/query-dsl/nested-query.asciidoc b/docs/reference/query-dsl/nested-query.asciidoc index 9c6276a0565a2..f75348ca976ec 100644 --- a/docs/reference/query-dsl/nested-query.asciidoc +++ b/docs/reference/query-dsl/nested-query.asciidoc @@ -10,7 +10,7 @@ will work with: [source,js] -------------------------------------------------- -PUT /my_index +PUT /my_index?include_type_name=true { "mappings": { "_doc" : { diff --git a/docs/reference/query-dsl/parent-id-query.asciidoc b/docs/reference/query-dsl/parent-id-query.asciidoc index aa86923541254..ff306238ddea0 100644 --- a/docs/reference/query-dsl/parent-id-query.asciidoc +++ b/docs/reference/query-dsl/parent-id-query.asciidoc @@ -6,7 +6,7 @@ Given the following mapping definition: [source,js] -------------------------------------------- -PUT my_index +PUT my_index?include_type_name=true { "mappings": { "_doc": { diff --git a/docs/reference/query-dsl/percolate-query.asciidoc b/docs/reference/query-dsl/percolate-query.asciidoc index a343b4dbb89cb..9ae6dd802137c 100644 --- a/docs/reference/query-dsl/percolate-query.asciidoc +++ b/docs/reference/query-dsl/percolate-query.asciidoc @@ -13,7 +13,7 @@ Create an index with two fields: [source,js] -------------------------------------------------- -PUT /my-index +PUT /my-index?include_type_name=true { "mappings": { "_doc": { diff --git a/docs/reference/query-dsl/term-query.asciidoc b/docs/reference/query-dsl/term-query.asciidoc index f1224f33ca7d4..26dd6ee492f21 100644 --- a/docs/reference/query-dsl/term-query.asciidoc +++ b/docs/reference/query-dsl/term-query.asciidoc @@ -87,7 +87,7 @@ To demonstrate, try out the example below. First, create an index, specifying t [source,js] -------------------------------------------------- -PUT my_index +PUT my_index?include_type_name=true { "mappings": { "_doc": { diff --git a/docs/reference/query-dsl/terms-set-query.asciidoc b/docs/reference/query-dsl/terms-set-query.asciidoc index fa879bb068d34..5c0e00ce360fa 100644 --- a/docs/reference/query-dsl/terms-set-query.asciidoc +++ b/docs/reference/query-dsl/terms-set-query.asciidoc @@ -12,7 +12,7 @@ be a number field: [source,js] -------------------------------------------------- -PUT /my-index +PUT /my-index?include_type_name=true { "mappings": { "_doc": { diff --git a/docs/reference/search/request/highlighters-internal.asciidoc b/docs/reference/search/request/highlighters-internal.asciidoc index 651cdf917ced0..aa3377bdccf61 100644 --- a/docs/reference/search/request/highlighters-internal.asciidoc +++ b/docs/reference/search/request/highlighters-internal.asciidoc @@ -87,7 +87,7 @@ using `english` analyzer, and will be indexed without offsets or term vectors. [source,js] -------------------------------------------------- -PUT test_index +PUT test_index?include_type_name=true { "mappings": { "_doc": { diff --git a/docs/reference/search/request/highlighting.asciidoc b/docs/reference/search/request/highlighting.asciidoc index e798fcf186906..4dd037cf3c001 100644 --- a/docs/reference/search/request/highlighting.asciidoc +++ b/docs/reference/search/request/highlighting.asciidoc @@ -787,7 +787,7 @@ allow for highlighting using the postings: [source,js] -------------------------------------------------- -PUT /example +PUT /example?include_type_name=true { "mappings": { "doc" : { @@ -808,7 +808,7 @@ highlighting using the `term_vectors` (this will cause the index to be bigger): [source,js] -------------------------------------------------- -PUT /example +PUT /example?include_type_name=true { "mappings": { "doc" : { diff --git a/docs/reference/search/request/inner-hits.asciidoc b/docs/reference/search/request/inner-hits.asciidoc index a1eeeb8f06375..b67377edfe901 100644 --- a/docs/reference/search/request/inner-hits.asciidoc +++ b/docs/reference/search/request/inner-hits.asciidoc @@ -84,7 +84,7 @@ The nested `inner_hits` can be used to include nested inner objects as inner hit [source,js] -------------------------------------------------- -PUT test +PUT test?include_type_name=true { "mappings": { "_doc": { @@ -210,7 +210,7 @@ including the source and solely rely on doc values fields. Like this: [source,js] -------------------------------------------------- -PUT test +PUT test?include_type_name=true { "mappings": { "_doc": { @@ -329,7 +329,7 @@ with the root hits then the following path can be defined: [source,js] -------------------------------------------------- -PUT test +PUT test?include_type_name=true { "mappings": { "_doc": { @@ -451,7 +451,7 @@ The parent/child `inner_hits` can be used to include parent or child: [source,js] -------------------------------------------------- -PUT test +PUT test?include_type_name=true { "mappings": { "_doc": { diff --git a/docs/reference/search/request/post-filter.asciidoc b/docs/reference/search/request/post-filter.asciidoc index 636824bc067ca..b404a30a8250a 100644 --- a/docs/reference/search/request/post-filter.asciidoc +++ b/docs/reference/search/request/post-filter.asciidoc @@ -9,7 +9,7 @@ Imagine that you are selling shirts that have the following properties: [source,js] -------------------------------------------------- -PUT /shirts +PUT /shirts?include_type_name=true { "mappings": { "_doc": { diff --git a/docs/reference/search/request/sort.asciidoc b/docs/reference/search/request/sort.asciidoc index 1875c402adab4..fe7cf362ea36f 100644 --- a/docs/reference/search/request/sort.asciidoc +++ b/docs/reference/search/request/sort.asciidoc @@ -9,7 +9,7 @@ Assuming the following index mapping: [source,js] -------------------------------------------------- -PUT /my_index +PUT /my_index?include_type_name=true { "mappings": { "_doc": { diff --git a/docs/reference/search/suggesters/completion-suggest.asciidoc b/docs/reference/search/suggesters/completion-suggest.asciidoc index b2b7f7a443708..191f01f17394d 100644 --- a/docs/reference/search/suggesters/completion-suggest.asciidoc +++ b/docs/reference/search/suggesters/completion-suggest.asciidoc @@ -24,7 +24,7 @@ which indexes the field values for fast completions. [source,js] -------------------------------------------------- -PUT music +PUT music?include_type_name=true { "mappings": { "_doc" : { diff --git a/docs/reference/search/suggesters/context-suggest.asciidoc b/docs/reference/search/suggesters/context-suggest.asciidoc index ab52097a4c504..1ef8968188ccd 100644 --- a/docs/reference/search/suggesters/context-suggest.asciidoc +++ b/docs/reference/search/suggesters/context-suggest.asciidoc @@ -21,7 +21,7 @@ field: [source,js] -------------------------------------------------- -PUT place +PUT place?include_type_name=true { "mappings": { "_doc" : { @@ -44,7 +44,7 @@ PUT place } } } -PUT place_path_category +PUT place_path_category?include_type_name=true { "mappings": { "_doc" : { diff --git a/docs/reference/search/suggesters/phrase-suggest.asciidoc b/docs/reference/search/suggesters/phrase-suggest.asciidoc index 2bca0d81aa658..08a7aabd1c5de 100644 --- a/docs/reference/search/suggesters/phrase-suggest.asciidoc +++ b/docs/reference/search/suggesters/phrase-suggest.asciidoc @@ -23,7 +23,7 @@ work. The `reverse` analyzer is used only in the last example. [source,js] -------------------------------------------------- -PUT test +PUT test?include_type_name=true { "settings": { "index": { diff --git a/modules/lang-mustache/src/test/resources/rest-api-spec/test/lang_mustache/60_typed_keys.yml b/modules/lang-mustache/src/test/resources/rest-api-spec/test/lang_mustache/60_typed_keys.yml index 3ea1b31b04995..9c12aeaa530ea 100644 --- a/modules/lang-mustache/src/test/resources/rest-api-spec/test/lang_mustache/60_typed_keys.yml +++ b/modules/lang-mustache/src/test/resources/rest-api-spec/test/lang_mustache/60_typed_keys.yml @@ -1,36 +1,36 @@ setup: - do: indices.put_template: + include_type_name: false name: index_template body: index_patterns: test-* settings: number_of_replicas: 0 mappings: - user: - properties: - ip: - type: ip - integer: - type: integer - float: - type: float - name: - type: keyword + properties: + ip: + type: ip + integer: + type: integer + float: + type: float + name: + type: keyword - do: bulk: refresh: true body: - - '{"index": {"_index": "test-0", "_type": "user"}}' + - '{"index": {"_index": "test-0", "_type": "_doc"}}' - '{"ip": "10.0.0.1", "integer": 38, "float": 12.5713, "name": "Ruth", "bool": true}' - - '{"index": {"_index": "test-0", "_type": "user"}}' + - '{"index": {"_index": "test-0", "_type": "_doc"}}' - '{"ip": "10.0.0.2", "integer": 42, "float": 15.3393, "name": "Jackie", "surname": "Bowling", "bool": false}' - - '{"index": {"_index": "test-1", "_type": "user"}}' + - '{"index": {"_index": "test-1", "_type": "_doc"}}' - '{"ip": "10.0.0.3", "integer": 29, "float": 19.0517, "name": "Stephanie", "bool": true}' - - '{"index": {"_index": "test-1", "_type": "user"}}' + - '{"index": {"_index": "test-1", "_type": "_doc"}}' - '{"ip": "10.0.0.4", "integer": 19, "float": 19.3717, "surname": "Hamilton", "bool": true}' - - '{"index": {"_index": "test-2", "_type": "user"}}' + - '{"index": {"_index": "test-2", "_type": "_doc"}}' - '{"ip": "10.0.0.5", "integer": 0, "float": 17.3349, "name": "Natalie", "bool": false}' --- diff --git a/modules/reindex/src/test/resources/rest-api-spec/test/update_by_query/30_new_fields.yml b/modules/reindex/src/test/resources/rest-api-spec/test/update_by_query/30_new_fields.yml index ae6e7559e2dfd..d20208f68e229 100644 --- a/modules/reindex/src/test/resources/rest-api-spec/test/update_by_query/30_new_fields.yml +++ b/modules/reindex/src/test/resources/rest-api-spec/test/update_by_query/30_new_fields.yml @@ -19,8 +19,8 @@ - do: indices.put_mapping: + include_type_name: false index: test - type: _doc body: properties: name: diff --git a/modules/transport-netty4/src/test/java/org/elasticsearch/rest/Netty4HeadBodyIsEmptyIT.java b/modules/transport-netty4/src/test/java/org/elasticsearch/rest/Netty4HeadBodyIsEmptyIT.java index 79b75487f6568..623633f690fe0 100644 --- a/modules/transport-netty4/src/test/java/org/elasticsearch/rest/Netty4HeadBodyIsEmptyIT.java +++ b/modules/transport-netty4/src/test/java/org/elasticsearch/rest/Netty4HeadBodyIsEmptyIT.java @@ -44,17 +44,17 @@ public void testHeadRoot() throws IOException { } private void createTestDoc() throws IOException { - createTestDoc("test", "test"); + createTestDoc("test"); } - private void createTestDoc(final String indexName, final String typeName) throws IOException { + private void createTestDoc(final String indexName) throws IOException { try (XContentBuilder builder = jsonBuilder()) { builder.startObject(); { builder.field("test", "test"); } builder.endObject(); - Request request = new Request("PUT", "/" + indexName + "/" + typeName + "/" + "1"); + Request request = new Request("PUT", "/" + indexName + "/_doc/" + "1"); request.setJsonEntity(Strings.toString(builder)); client().performRequest(request); } @@ -62,9 +62,9 @@ private void createTestDoc(final String indexName, final String typeName) throws public void testDocumentExists() throws IOException { createTestDoc(); - headTestCase("/test/test/1", emptyMap(), greaterThan(0)); - headTestCase("/test/test/1", singletonMap("pretty", "true"), greaterThan(0)); - headTestCase("/test/test/2", emptyMap(), NOT_FOUND.getStatus(), greaterThan(0)); + headTestCase("/test/_doc/1", emptyMap(), greaterThan(0)); + headTestCase("/test/_doc/1", singletonMap("pretty", "true"), greaterThan(0)); + headTestCase("/test/_doc/2", emptyMap(), NOT_FOUND.getStatus(), greaterThan(0)); } public void testIndexExists() throws IOException { @@ -81,14 +81,14 @@ protected boolean getStrictDeprecationMode() { public void testTypeExists() throws IOException { createTestDoc(); - headTestCase("/test/_mapping/test", emptyMap(), greaterThan(0)); - headTestCase("/test/_mapping/test", singletonMap("pretty", "true"), greaterThan(0)); + headTestCase("/test/_mapping/_doc", emptyMap(), greaterThan(0)); + headTestCase("/test/_mapping/_doc", singletonMap("pretty", "true"), greaterThan(0)); } public void testTypeDoesNotExist() throws IOException { createTestDoc(); headTestCase("/test/_mapping/does-not-exist", emptyMap(), NOT_FOUND.getStatus(), greaterThan(0)); - headTestCase("/text/_mapping/test,does-not-exist", emptyMap(), NOT_FOUND.getStatus(), greaterThan(0)); + headTestCase("/text/_mapping/_doc,does-not-exist", emptyMap(), NOT_FOUND.getStatus(), greaterThan(0)); } public void testAliasExists() throws IOException { @@ -157,13 +157,9 @@ public void testGetSourceAction() throws IOException { { builder.startObject("mappings"); { - builder.startObject("test-no-source"); + builder.startObject("_source"); { - builder.startObject("_source"); - { - builder.field("enabled", false); - } - builder.endObject(); + builder.field("enabled", false); } builder.endObject(); } @@ -174,7 +170,7 @@ public void testGetSourceAction() throws IOException { Request request = new Request("PUT", "/test-no-source"); request.setJsonEntity(Strings.toString(builder)); client().performRequest(request); - createTestDoc("test-no-source", "test-no-source"); + createTestDoc("test-no-source"); headTestCase("/test-no-source/_source/1", emptyMap(), NOT_FOUND.getStatus(), greaterThan(0)); } } diff --git a/qa/full-cluster-restart/src/test/java/org/elasticsearch/upgrades/FullClusterRestartIT.java b/qa/full-cluster-restart/src/test/java/org/elasticsearch/upgrades/FullClusterRestartIT.java index e8bd2d95697a8..46038da21dfc0 100644 --- a/qa/full-cluster-restart/src/test/java/org/elasticsearch/upgrades/FullClusterRestartIT.java +++ b/qa/full-cluster-restart/src/test/java/org/elasticsearch/upgrades/FullClusterRestartIT.java @@ -60,6 +60,7 @@ import static org.elasticsearch.cluster.routing.UnassignedInfo.INDEX_DELAYED_NODE_LEFT_TIMEOUT_SETTING; import static org.elasticsearch.cluster.routing.allocation.decider.MaxRetryAllocationDecider.SETTING_ALLOCATION_MAX_RETRY; import static org.elasticsearch.common.xcontent.XContentFactory.jsonBuilder; +import static org.elasticsearch.rest.BaseRestHandler.INCLUDE_TYPE_NAME_PARAMETER; import static org.hamcrest.Matchers.containsString; import static org.hamcrest.Matchers.equalTo; import static org.hamcrest.Matchers.greaterThan; @@ -901,6 +902,13 @@ public void testSnapshotRestore() throws IOException { templateBuilder.endObject().endObject(); Request createTemplateRequest = new Request("PUT", "/_template/test_template"); createTemplateRequest.setJsonEntity(Strings.toString(templateBuilder)); + + // In 7.0, type names are no longer expected by default in put index template requests. + // We therefore use the deprecated typed APIs when running against the current version. + if (isRunningAgainstOldCluster() == false) { + createTemplateRequest.addParameter(INCLUDE_TYPE_NAME_PARAMETER, "true"); + } + client().performRequest(createTemplateRequest); if (isRunningAgainstOldCluster()) { @@ -1094,7 +1102,15 @@ private void checkSnapshot(String snapshotName, int count, Version tookOnVersion assertThat(persistentSettings.get("cluster.routing.allocation.exclude.test_attr"), equalTo(getOldClusterVersion().toString())); // Check that the template was restored successfully - Map getTemplateResponse = entityAsMap(client().performRequest(new Request("GET", "/_template/test_template"))); + Request getTemplateRequest = new Request("GET", "/_template/test_template"); + + // In 7.0, type names are no longer returned by default in get index template requests. + // We therefore use the deprecated typed APIs when running against the current version. + if (isRunningAgainstOldCluster() == false) { + getTemplateRequest.addParameter(INCLUDE_TYPE_NAME_PARAMETER, "true"); + } + + Map getTemplateResponse = entityAsMap(client().performRequest(getTemplateRequest)); Map expectedTemplate = new HashMap<>(); if (isRunningAgainstOldCluster() && getOldClusterVersion().before(Version.V_6_0_0_beta1)) { expectedTemplate.put("template", "evil_*"); diff --git a/rest-api-spec/src/main/resources/rest-api-spec/test/indices.create/10_basic.yml b/rest-api-spec/src/main/resources/rest-api-spec/test/indices.create/10_basic.yml index e58e2bd8aa1fc..19bc94870cb47 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/test/indices.create/10_basic.yml +++ b/rest-api-spec/src/main/resources/rest-api-spec/test/indices.create/10_basic.yml @@ -3,7 +3,7 @@ - skip: version: " - 6.99.99" - reason: types are required in requests before 7.0.0 + reason: include_type_name defaults to true before 7.0.0 - do: indices.create: include_type_name: false @@ -14,7 +14,6 @@ - do: indices.get_mapping: - include_type_name: false index: test_index - is_true: test_index.mappings @@ -24,7 +23,7 @@ - skip: version: " - 6.99.99" - reason: types are required in requests before 7.0.0 + reason: include_type_name defaults to true before 7.0.0 - do: indices.create: include_type_name: false @@ -44,7 +43,7 @@ - skip: version: " - 6.99.99" - reason: types are required in requests before 7.0.0 + reason: include_type_name defaults to true before 7.0.0 - do: indices.create: include_type_name: false @@ -58,7 +57,7 @@ - skip: version: " - 6.99.99" - reason: types are required in requests before 7.0.0 + reason: include_type_name defaults to true before 7.0.0 - do: indices.create: include_type_name: false @@ -76,7 +75,7 @@ - skip: version: " - 6.99.99" - reason: types are required in requests before 7.0.0 + reason: include_type_name defaults to true before 7.0.0 - do: indices.create: include_type_name: false diff --git a/rest-api-spec/src/main/resources/rest-api-spec/test/indices.create/11_basic_with_types.yml b/rest-api-spec/src/main/resources/rest-api-spec/test/indices.create/11_basic_with_types.yml index 6f7c5a6009386..f5aeb53751119 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/test/indices.create/11_basic_with_types.yml +++ b/rest-api-spec/src/main/resources/rest-api-spec/test/indices.create/11_basic_with_types.yml @@ -3,6 +3,7 @@ - do: indices.create: + include_type_name: true index: test_index body: mappings: @@ -10,6 +11,7 @@ - do: indices.get_mapping: + include_type_name: true index: test_index - is_true: test_index.mappings.type_1 @@ -19,6 +21,7 @@ - do: indices.create: + include_type_name: true index: test_index body: settings: @@ -35,6 +38,7 @@ - do: indices.create: + include_type_name: true index: test_index - match: { acknowledged: true } @@ -45,6 +49,7 @@ - do: indices.create: + include_type_name: true index: test_index wait_for_active_shards: all body: @@ -59,6 +64,7 @@ - do: indices.create: + include_type_name: true index: test_index body: mappings: @@ -93,6 +99,7 @@ reason: is_write_index is not implemented in ES <= 6.x - do: indices.create: + include_type_name: true index: test_index body: aliases: @@ -115,6 +122,7 @@ - do: catch: /illegal_argument_exception/ indices.create: + include_type_name: true index: test_index body: mappings: @@ -125,6 +133,7 @@ - do: catch: /illegal_argument_exception/ indices.create: + include_type_name: true index: test_index body: mappings: diff --git a/rest-api-spec/src/main/resources/rest-api-spec/test/indices.get_field_mapping/10_basic.yml b/rest-api-spec/src/main/resources/rest-api-spec/test/indices.get_field_mapping/10_basic.yml index b77c56d34160c..e215c43b4bdeb 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/test/indices.get_field_mapping/10_basic.yml +++ b/rest-api-spec/src/main/resources/rest-api-spec/test/indices.get_field_mapping/10_basic.yml @@ -2,7 +2,7 @@ setup: - skip: version: " - 6.99.99" - reason: include_type_name is not supported before 7.0.0 + reason: include_type_name defaults to true before 7.0 - do: indices.create: include_type_name: false @@ -18,7 +18,6 @@ setup: - do: indices.get_field_mapping: - include_type_name: false fields: text - match: {test_index.mappings.text.mapping.text.type: text} @@ -27,7 +26,6 @@ setup: "Get field mapping by index only": - do: indices.get_field_mapping: - include_type_name: false index: test_index fields: text @@ -38,7 +36,6 @@ setup: - do: indices.get_field_mapping: - include_type_name: false index: test_index fields: [ text , text1 ] @@ -50,7 +47,6 @@ setup: - do: indices.get_field_mapping: - include_type_name: false index: test_index fields: text include_defaults: true diff --git a/rest-api-spec/src/main/resources/rest-api-spec/test/indices.get_field_mapping/11_basic_with_types.yml b/rest-api-spec/src/main/resources/rest-api-spec/test/indices.get_field_mapping/11_basic_with_types.yml index 6cf0a0b7cf26c..d45f0671db5a6 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/test/indices.get_field_mapping/11_basic_with_types.yml +++ b/rest-api-spec/src/main/resources/rest-api-spec/test/indices.get_field_mapping/11_basic_with_types.yml @@ -15,6 +15,7 @@ setup: - do: indices.get_field_mapping: + include_type_name: true fields: text - match: {test_index.mappings.test_type.text.mapping.text.type: text} @@ -23,6 +24,7 @@ setup: "Get field mapping by index only": - do: indices.get_field_mapping: + include_type_name: true index: test_index fields: text @@ -33,6 +35,7 @@ setup: - do: indices.get_field_mapping: + include_type_name: true index: test_index type: test_type fields: text @@ -44,6 +47,7 @@ setup: - do: indices.get_field_mapping: + include_type_name: true index: test_index type: test_type fields: [ text , text1 ] @@ -56,6 +60,7 @@ setup: - do: indices.get_field_mapping: + include_type_name: true index: test_index type: test_type fields: text @@ -69,6 +74,7 @@ setup: - do: indices.get_field_mapping: + include_type_name: true type: test_type fields: text diff --git a/rest-api-spec/src/main/resources/rest-api-spec/test/indices.get_field_mapping/20_missing_field.yml b/rest-api-spec/src/main/resources/rest-api-spec/test/indices.get_field_mapping/20_missing_field.yml index 61f1f409d2939..3bc1c349039c0 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/test/indices.get_field_mapping/20_missing_field.yml +++ b/rest-api-spec/src/main/resources/rest-api-spec/test/indices.get_field_mapping/20_missing_field.yml @@ -2,7 +2,7 @@ "Return empty object if field doesn't exist, but type and index do": - skip: version: " - 6.99.99" - reason: types are required in requests before 7.0.0 + reason: include_type_name defaults to true before 7.0 - do: indices.create: include_type_name: false diff --git a/rest-api-spec/src/main/resources/rest-api-spec/test/indices.get_field_mapping/21_missing_field_with_types.yml b/rest-api-spec/src/main/resources/rest-api-spec/test/indices.get_field_mapping/21_missing_field_with_types.yml index c760561f09282..c715d27ebc26f 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/test/indices.get_field_mapping/21_missing_field_with_types.yml +++ b/rest-api-spec/src/main/resources/rest-api-spec/test/indices.get_field_mapping/21_missing_field_with_types.yml @@ -14,6 +14,7 @@ - do: indices.get_field_mapping: + include_type_name: true index: test_index type: test_type fields: not_existent diff --git a/rest-api-spec/src/main/resources/rest-api-spec/test/indices.get_field_mapping/30_missing_type.yml b/rest-api-spec/src/main/resources/rest-api-spec/test/indices.get_field_mapping/30_missing_type.yml index dfbd9a4af77a4..bbafb3e43f1a7 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/test/indices.get_field_mapping/30_missing_type.yml +++ b/rest-api-spec/src/main/resources/rest-api-spec/test/indices.get_field_mapping/30_missing_type.yml @@ -15,6 +15,7 @@ - do: catch: missing indices.get_field_mapping: + include_type_name: true index: test_index type: not_test_type fields: text diff --git a/rest-api-spec/src/main/resources/rest-api-spec/test/indices.get_field_mapping/40_missing_index.yml b/rest-api-spec/src/main/resources/rest-api-spec/test/indices.get_field_mapping/40_missing_index.yml index 7c7b07b587849..2273867816c6e 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/test/indices.get_field_mapping/40_missing_index.yml +++ b/rest-api-spec/src/main/resources/rest-api-spec/test/indices.get_field_mapping/40_missing_index.yml @@ -4,6 +4,7 @@ - do: catch: missing indices.get_field_mapping: + include_type_name: false index: test_index fields: field diff --git a/rest-api-spec/src/main/resources/rest-api-spec/test/indices.get_field_mapping/50_field_wildcards.yml b/rest-api-spec/src/main/resources/rest-api-spec/test/indices.get_field_mapping/50_field_wildcards.yml index 3ffecdcc72618..435259fb3b07d 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/test/indices.get_field_mapping/50_field_wildcards.yml +++ b/rest-api-spec/src/main/resources/rest-api-spec/test/indices.get_field_mapping/50_field_wildcards.yml @@ -2,7 +2,7 @@ setup: - skip: version: " - 6.99.99" - reason: types are required in requests before 7.0.0 + reason: include_type_name defaults to true before 7.0 - do: indices.create: include_type_name: false @@ -48,7 +48,6 @@ setup: - do: indices.get_field_mapping: - include_type_name: false fields: "*" - match: {test_index.mappings.t1.full_name: t1 } @@ -62,7 +61,6 @@ setup: - do: indices.get_field_mapping: - include_type_name: false index: test_index fields: "t*" @@ -75,7 +73,6 @@ setup: - do: indices.get_field_mapping: - include_type_name: false index: test_index fields: "*t1" - match: {test_index.mappings.t1.full_name: t1 } @@ -88,7 +85,6 @@ setup: - do: indices.get_field_mapping: - include_type_name: false index: test_index fields: "obj.i_*" - match: {test_index.mappings.obj\.i_t1.full_name: obj.i_t1 } @@ -100,7 +96,6 @@ setup: - do: indices.get_field_mapping: - include_type_name: false index: _all fields: "t*" - match: {test_index.mappings.t1.full_name: t1 } @@ -115,7 +110,6 @@ setup: - do: indices.get_field_mapping: - include_type_name: false index: '*' fields: "t*" - match: {test_index.mappings.t1.full_name: t1 } @@ -130,7 +124,6 @@ setup: - do: indices.get_field_mapping: - include_type_name: false index: 'test_index,test_index_2' fields: "t*" - match: {test_index.mappings.t1.full_name: t1 } diff --git a/rest-api-spec/src/main/resources/rest-api-spec/test/indices.get_field_mapping/51_field_wildcards_with_types.yml b/rest-api-spec/src/main/resources/rest-api-spec/test/indices.get_field_mapping/51_field_wildcards_with_types.yml index 9d62ab6101fc2..fe0c135f13ad7 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/test/indices.get_field_mapping/51_field_wildcards_with_types.yml +++ b/rest-api-spec/src/main/resources/rest-api-spec/test/indices.get_field_mapping/51_field_wildcards_with_types.yml @@ -45,6 +45,7 @@ setup: - do: indices.get_field_mapping: + include_type_name: true fields: "*" - match: {test_index.mappings.test_type.t1.full_name: t1 } @@ -58,6 +59,7 @@ setup: - do: indices.get_field_mapping: + include_type_name: true index: test_index fields: "t*" @@ -70,6 +72,7 @@ setup: - do: indices.get_field_mapping: + include_type_name: true index: test_index fields: "*t1" - match: {test_index.mappings.test_type.t1.full_name: t1 } @@ -82,6 +85,7 @@ setup: - do: indices.get_field_mapping: + include_type_name: true index: test_index fields: "obj.i_*" - match: {test_index.mappings.test_type.obj\.i_t1.full_name: obj.i_t1 } @@ -93,6 +97,7 @@ setup: - do: indices.get_field_mapping: + include_type_name: true index: _all type: _all fields: "t*" @@ -108,6 +113,7 @@ setup: - do: indices.get_field_mapping: + include_type_name: true index: '*' type: '*' fields: "t*" @@ -123,6 +129,7 @@ setup: - do: indices.get_field_mapping: + include_type_name: true index: 'test_index,test_index_2' type: 'test_type,test_type_2' fields: "t*" diff --git a/rest-api-spec/src/main/resources/rest-api-spec/test/indices.get_field_mapping/60_mix_typeless_typeful.yml b/rest-api-spec/src/main/resources/rest-api-spec/test/indices.get_field_mapping/60_mix_typeless_typeful.yml index d7ea620bb6c58..862075f450e73 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/test/indices.get_field_mapping/60_mix_typeless_typeful.yml +++ b/rest-api-spec/src/main/resources/rest-api-spec/test/indices.get_field_mapping/60_mix_typeless_typeful.yml @@ -1,10 +1,6 @@ --- "GET mapping with typeless API on an index that has types": - - skip: - version: " - 6.99.99" - reason: include_type_name was introduced in 7.0.0 - - do: indices.create: # not using include_type_name: false on purpose index: index diff --git a/rest-api-spec/src/main/resources/rest-api-spec/test/indices.get_mapping/10_basic.yml b/rest-api-spec/src/main/resources/rest-api-spec/test/indices.get_mapping/10_basic.yml index ccebfc7c9423b..d9ea7d325e3a8 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/test/indices.get_mapping/10_basic.yml +++ b/rest-api-spec/src/main/resources/rest-api-spec/test/indices.get_mapping/10_basic.yml @@ -2,7 +2,7 @@ setup: - skip: version: " - 6.99.99" - reason: types are required in requests before 7.0.0 + reason: include_type_name defaults to true before 7.0 - do: indices.create: include_type_name: false @@ -25,7 +25,6 @@ setup: - do: indices.get_mapping: - include_type_name: false index: t - match: { t.mappings: {}} @@ -34,8 +33,7 @@ setup: "Get /_mapping": - do: - indices.get_mapping: - include_type_name: false + indices.get_mapping: {} - is_true: test_1.mappings - is_true: test_2.mappings @@ -45,7 +43,6 @@ setup: - do: indices.get_mapping: - include_type_name: false index: test_1 - is_true: test_1.mappings @@ -58,7 +55,6 @@ setup: - do: indices.get_mapping: - include_type_name: false index: _all - is_true: test_1.mappings @@ -69,7 +65,6 @@ setup: - do: indices.get_mapping: - include_type_name: false index: '*' - is_true: test_1.mappings @@ -80,7 +75,6 @@ setup: - do: indices.get_mapping: - include_type_name: false index: test_1,test_2 - is_true: test_1.mappings @@ -91,7 +85,6 @@ setup: - do: indices.get_mapping: - include_type_name: false index: '*2' - is_true: test_2.mappings diff --git a/rest-api-spec/src/main/resources/rest-api-spec/test/indices.get_mapping/11_basic_with_types.yml b/rest-api-spec/src/main/resources/rest-api-spec/test/indices.get_mapping/11_basic_with_types.yml index 90bb2747a7bfc..1c26a2a3d3d13 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/test/indices.get_mapping/11_basic_with_types.yml +++ b/rest-api-spec/src/main/resources/rest-api-spec/test/indices.get_mapping/11_basic_with_types.yml @@ -21,6 +21,7 @@ setup: - do: indices.get_mapping: + include_type_name: true index: t - match: { t.mappings: {}} @@ -29,7 +30,8 @@ setup: "Get /_mapping": - do: - indices.get_mapping: {} + indices.get_mapping: + include_type_name: true - is_true: test_1.mappings.doc - is_true: test_2.mappings.doc @@ -39,6 +41,7 @@ setup: - do: indices.get_mapping: + include_type_name: true index: test_1 - is_true: test_1.mappings.doc @@ -50,6 +53,7 @@ setup: - do: indices.get_mapping: + include_type_name: true index: test_1 type: _all @@ -61,6 +65,7 @@ setup: - do: indices.get_mapping: + include_type_name: true index: test_1 type: '*' @@ -72,6 +77,7 @@ setup: - do: indices.get_mapping: + include_type_name: true index: test_1 type: doc @@ -83,6 +89,7 @@ setup: - do: indices.get_mapping: + include_type_name: true index: test_1 type: 'd*' @@ -94,6 +101,7 @@ setup: - do: indices.get_mapping: + include_type_name: true type: doc - is_true: test_1.mappings.doc @@ -104,6 +112,7 @@ setup: - do: indices.get_mapping: + include_type_name: true index: _all type: doc @@ -115,6 +124,7 @@ setup: - do: indices.get_mapping: + include_type_name: true index: '*' type: doc @@ -126,6 +136,7 @@ setup: - do: indices.get_mapping: + include_type_name: true index: test_1,test_2 type: doc @@ -137,6 +148,7 @@ setup: - do: indices.get_mapping: + include_type_name: true index: '*2' type: doc diff --git a/rest-api-spec/src/main/resources/rest-api-spec/test/indices.get_mapping/20_missing_type.yml b/rest-api-spec/src/main/resources/rest-api-spec/test/indices.get_mapping/20_missing_type.yml index eecf1786e00c7..2f03bf7df5014 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/test/indices.get_mapping/20_missing_type.yml +++ b/rest-api-spec/src/main/resources/rest-api-spec/test/indices.get_mapping/20_missing_type.yml @@ -14,6 +14,7 @@ - do: catch: missing indices.get_mapping: + include_type_name: true index: test_index type: not_test_type @@ -36,6 +37,7 @@ - do: catch: missing indices.get_mapping: + include_type_name: true index: test_index type: test*,not* @@ -59,6 +61,7 @@ - do: catch: missing indices.get_mapping: + include_type_name: true index: test_index type: test_type,not_test_type @@ -82,6 +85,7 @@ - do: catch: missing indices.get_mapping: + include_type_name: true index: test_index type: test_type,not_test_type,another_not_test_type @@ -94,5 +98,6 @@ - do: catch: missing indices.get_mapping: + include_type_name: true type: not_test_type diff --git a/rest-api-spec/src/main/resources/rest-api-spec/test/indices.get_mapping/30_missing_index.yml b/rest-api-spec/src/main/resources/rest-api-spec/test/indices.get_mapping/30_missing_index.yml index 4d3abb292f467..9b422ec8b9156 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/test/indices.get_mapping/30_missing_index.yml +++ b/rest-api-spec/src/main/resources/rest-api-spec/test/indices.get_mapping/30_missing_index.yml @@ -3,11 +3,11 @@ - do: catch: missing indices.get_mapping: + include_type_name: false index: test_index - type: not_test_type --- -"Index missing, no indexes, no types": +"Index missing, no indexes": - do: catch: missing indices.get_mapping: @@ -20,6 +20,7 @@ reason: ignore_unavailable was ignored in previous versions - do: indices.get_mapping: + include_type_name: false index: test_index ignore_unavailable: true @@ -30,6 +31,7 @@ - do: catch: missing indices.get_mapping: + include_type_name: false index: test_index ignore_unavailable: true allow_no_indices: false diff --git a/rest-api-spec/src/main/resources/rest-api-spec/test/indices.get_mapping/40_aliases.yml b/rest-api-spec/src/main/resources/rest-api-spec/test/indices.get_mapping/40_aliases.yml index a608566544182..7afd10e1109eb 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/test/indices.get_mapping/40_aliases.yml +++ b/rest-api-spec/src/main/resources/rest-api-spec/test/indices.get_mapping/40_aliases.yml @@ -19,8 +19,9 @@ - do: indices.get_mapping: + include_type_name: false index: test_alias - - match: {test_index.mappings.test_type.properties.text.type: text} - - match: {test_index.mappings.test_type.properties.text.analyzer: whitespace} + - match: {test_index.mappings.properties.text.type: text} + - match: {test_index.mappings.properties.text.analyzer: whitespace} diff --git a/rest-api-spec/src/main/resources/rest-api-spec/test/indices.get_mapping/50_wildcard_expansion.yml b/rest-api-spec/src/main/resources/rest-api-spec/test/indices.get_mapping/50_wildcard_expansion.yml index d1a95b2690745..8f8ff8f9fc102 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/test/indices.get_mapping/50_wildcard_expansion.yml +++ b/rest-api-spec/src/main/resources/rest-api-spec/test/indices.get_mapping/50_wildcard_expansion.yml @@ -54,43 +54,47 @@ setup: - do: indices.get_mapping: + include_type_name: false index: test-x* - - is_true: test-xxx.mappings.type_1 - - is_true: test-xxy.mappings.type_2 + - is_true: test-xxx.mappings + - is_true: test-xxy.mappings --- "Get test-* with wildcard_expansion=all": - do: indices.get_mapping: + include_type_name: false index: test-x* expand_wildcards: all - - is_true: test-xxx.mappings.type_1 - - is_true: test-xxy.mappings.type_2 - - is_true: test-xyy.mappings.type_3 + - is_true: test-xxx.mappings + - is_true: test-xxy.mappings + - is_true: test-xyy.mappings --- "Get test-* with wildcard_expansion=open": - do: indices.get_mapping: + include_type_name: false index: test-x* expand_wildcards: open - - is_true: test-xxx.mappings.type_1 - - is_true: test-xxy.mappings.type_2 + - is_true: test-xxx.mappings + - is_true: test-xxy.mappings --- "Get test-* with wildcard_expansion=closed": - do: indices.get_mapping: + include_type_name: false index: test-x* expand_wildcards: closed - - is_true: test-xyy.mappings.type_3 + - is_true: test-xyy.mappings --- "Get test-* with wildcard_expansion=none": @@ -99,6 +103,7 @@ setup: reason: allow_no_indices (defaults to true) was ignored in previous versions - do: indices.get_mapping: + include_type_name: false index: test-x* expand_wildcards: none @@ -111,6 +116,7 @@ setup: - do: catch: missing indices.get_mapping: + include_type_name: false index: test-x* expand_wildcards: none allow_no_indices: false @@ -119,9 +125,10 @@ setup: - do: indices.get_mapping: + include_type_name: false index: test-x* expand_wildcards: open,closed - - is_true: test-xxx.mappings.type_1 - - is_true: test-xxy.mappings.type_2 - - is_true: test-xyy.mappings.type_3 + - is_true: test-xxx.mappings + - is_true: test-xxy.mappings + - is_true: test-xyy.mappings diff --git a/rest-api-spec/src/main/resources/rest-api-spec/test/indices.get_mapping/60_empty.yml b/rest-api-spec/src/main/resources/rest-api-spec/test/indices.get_mapping/60_empty.yml index dfba78578bbfe..e2a502f30a84d 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/test/indices.get_mapping/60_empty.yml +++ b/rest-api-spec/src/main/resources/rest-api-spec/test/indices.get_mapping/60_empty.yml @@ -1,6 +1,8 @@ --- setup: - + - skip: + version: " - 6.99.99" + reason: include_type_name defaults to true before 7.0 - do: indices.create: index: test_1 diff --git a/rest-api-spec/src/main/resources/rest-api-spec/test/indices.get_mapping/61_empty_with_types.yml b/rest-api-spec/src/main/resources/rest-api-spec/test/indices.get_mapping/61_empty_with_types.yml new file mode 100644 index 0000000000000..6da7f4a2c6946 --- /dev/null +++ b/rest-api-spec/src/main/resources/rest-api-spec/test/indices.get_mapping/61_empty_with_types.yml @@ -0,0 +1,20 @@ +--- +setup: + + - do: + indices.create: + index: test_1 + + - do: + indices.create: + index: test_2 + +--- +"Check empty mapping when getting all mappings via /_mapping": + + - do: + indices.get_mapping: + include_type_name: true + + - match: { test_1.mappings: {}} + - match: { test_2.mappings: {}} diff --git a/rest-api-spec/src/main/resources/rest-api-spec/test/indices.get_template/10_basic.yml b/rest-api-spec/src/main/resources/rest-api-spec/test/indices.get_template/10_basic.yml index 8fe244e0c6323..c1aac94bf1d84 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/test/indices.get_template/10_basic.yml +++ b/rest-api-spec/src/main/resources/rest-api-spec/test/indices.get_template/10_basic.yml @@ -1,7 +1,7 @@ setup: - skip: version: " - 6.99.99" - reason: include_type_name is not supported before 7.0.0 + reason: include_type_name defaults to true before 7.0.0 - do: indices.put_template: name: test @@ -11,17 +11,15 @@ setup: number_of_shards: 1 number_of_replicas: 0 mappings: - _doc: - properties: - field: - type: keyword + properties: + field: + type: keyword --- "Get template": - do: indices.get_template: - include_type_name: false name: test - match: {test.index_patterns: ["test-*"]} @@ -42,7 +40,6 @@ setup: - do: indices.get_template: - include_type_name: false name: test_no_mappings - match: {test_no_mappings.index_patterns: ["test-*"]} diff --git a/rest-api-spec/src/main/resources/rest-api-spec/test/indices.get_template/11_basic_with_types.yml b/rest-api-spec/src/main/resources/rest-api-spec/test/indices.get_template/11_basic_with_types.yml index c15f5dc6de4f1..0ecf304b1ce70 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/test/indices.get_template/11_basic_with_types.yml +++ b/rest-api-spec/src/main/resources/rest-api-spec/test/indices.get_template/11_basic_with_types.yml @@ -1,6 +1,7 @@ setup: - do: indices.put_template: + include_type_name: true name: test body: index_patterns: test-* @@ -18,6 +19,7 @@ setup: - do: indices.get_template: + include_type_name: true name: test - match: {test.index_patterns: ["test-*"]} @@ -38,6 +40,7 @@ setup: - do: indices.get_template: + include_type_name: true name: test_no_mappings - match: {test_no_mappings.index_patterns: ["test-*"]} diff --git a/rest-api-spec/src/main/resources/rest-api-spec/test/indices.get_template/20_get_missing.yml b/rest-api-spec/src/main/resources/rest-api-spec/test/indices.get_template/20_get_missing.yml index 2751f57dacb6c..13d05efc67151 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/test/indices.get_template/20_get_missing.yml +++ b/rest-api-spec/src/main/resources/rest-api-spec/test/indices.get_template/20_get_missing.yml @@ -9,5 +9,6 @@ setup: - do: catch: missing indices.get_template: + include_type_name: false name: test diff --git a/rest-api-spec/src/main/resources/rest-api-spec/test/indices.put_mapping/10_basic.yml b/rest-api-spec/src/main/resources/rest-api-spec/test/indices.put_mapping/10_basic.yml index e4545320a973c..443e33d2c33a4 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/test/indices.put_mapping/10_basic.yml +++ b/rest-api-spec/src/main/resources/rest-api-spec/test/indices.put_mapping/10_basic.yml @@ -1,9 +1,8 @@ --- "Test Create and update mapping": - - skip: version: " - 6.99.99" - reason: types are required in requests before 7.0.0 + reason: include_type_name defaults to true before 7.0 - do: indices.create: include_type_name: false @@ -11,7 +10,6 @@ - do: indices.put_mapping: - include_type_name: false index: test_index body: properties: @@ -26,7 +24,6 @@ - do: indices.get_mapping: - include_type_name: false index: test_index - match: {test_index.mappings.properties.text1.type: text} @@ -36,7 +33,6 @@ - do: indices.put_mapping: - include_type_name: false index: test_index body: properties: @@ -50,7 +46,6 @@ - do: indices.get_mapping: - include_type_name: false index: test_index - match: {test_index.mappings.properties.text1.type: text} @@ -62,7 +57,7 @@ - skip: version: " - 6.99.99" - reason: types are required in requests before 7.0.0 + reason: include_type_name defaults to true before 7.0 - do: indices.create: include_type_name: false @@ -70,7 +65,6 @@ - do: catch: /illegal_argument_exception/ indices.put_mapping: - include_type_name: false index: test_index body: properties: @@ -82,7 +76,7 @@ - skip: version: " - 6.99.99" - reason: types are required in requests before 7.0.0 + reason: include_type_name defaults to true before 7.0 - do: indices.create: index: index @@ -93,8 +87,8 @@ indices.put_mapping: index: index type: _doc - include_type_name: false body: properties: bar: type: float + diff --git a/rest-api-spec/src/main/resources/rest-api-spec/test/indices.put_mapping/11_basic_with_types.yml b/rest-api-spec/src/main/resources/rest-api-spec/test/indices.put_mapping/11_basic_with_types.yml index 1d33f2d31bb15..5da9cd4bf707c 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/test/indices.put_mapping/11_basic_with_types.yml +++ b/rest-api-spec/src/main/resources/rest-api-spec/test/indices.put_mapping/11_basic_with_types.yml @@ -6,6 +6,7 @@ - do: indices.put_mapping: + include_type_name: true index: test_index type: test_type body: @@ -22,6 +23,7 @@ - do: indices.get_mapping: + include_type_name: true index: test_index - match: {test_index.mappings.test_type.properties.text1.type: text} @@ -31,6 +33,7 @@ - do: indices.put_mapping: + include_type_name: true index: test_index type: test_type body: @@ -46,6 +49,7 @@ - do: indices.get_mapping: + include_type_name: true index: test_index - match: {test_index.mappings.test_type.properties.text1.type: text} @@ -60,6 +64,7 @@ - do: catch: /illegal_argument_exception/ indices.put_mapping: + include_type_name: true index: test_index type: test_type body: diff --git a/rest-api-spec/src/main/resources/rest-api-spec/test/indices.put_mapping/20_mix_typeless_typeful.yml b/rest-api-spec/src/main/resources/rest-api-spec/test/indices.put_mapping/20_mix_typeless_typeful.yml index 5f9efb1a3750c..3aedff101110b 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/test/indices.put_mapping/20_mix_typeless_typeful.yml +++ b/rest-api-spec/src/main/resources/rest-api-spec/test/indices.put_mapping/20_mix_typeless_typeful.yml @@ -1,12 +1,9 @@ --- "PUT mapping with typeless API on an index that has types": - - skip: - version: " - 6.99.99" - reason: include_type_name was introduced in 7.0.0 - - do: indices.create: # not using include_type_name: false on purpose + include_type_name: true index: index body: mappings: @@ -44,6 +41,7 @@ - do: catch: bad_request indices.put_mapping: + include_type_name: true index: index body: some_other_type: diff --git a/rest-api-spec/src/main/resources/rest-api-spec/test/indices.put_mapping/all_path_options.yml b/rest-api-spec/src/main/resources/rest-api-spec/test/indices.put_mapping/all_path_options.yml index c6212fbbd8fa5..44796defef7c5 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/test/indices.put_mapping/all_path_options.yml +++ b/rest-api-spec/src/main/resources/rest-api-spec/test/indices.put_mapping/all_path_options.yml @@ -1,7 +1,7 @@ setup: - skip: version: " - 6.99.99" - reason: types are required in requests before 7.0.0 + reason: include_type_name defaults to true before 7.0 - do: indices.create: include_type_name: false @@ -20,7 +20,6 @@ setup: "put one mapping per index": - do: indices.put_mapping: - include_type_name: false index: test_index1 body: properties: @@ -29,7 +28,6 @@ setup: analyzer: whitespace - do: indices.put_mapping: - include_type_name: false index: test_index2 body: properties: @@ -39,8 +37,7 @@ setup: - do: - indices.get_mapping: - include_type_name: false + indices.get_mapping: {} - match: {test_index1.mappings.properties.text.type: text} - match: {test_index1.mappings.properties.text.analyzer: whitespace} @@ -55,7 +52,6 @@ setup: - do: indices.put_mapping: - include_type_name: false index: _all body: properties: @@ -64,8 +60,7 @@ setup: analyzer: whitespace - do: - indices.get_mapping: - include_type_name: false + indices.get_mapping: {} - match: {test_index1.mappings.properties.text.type: text} - match: {test_index1.mappings.properties.text.analyzer: whitespace} @@ -80,7 +75,6 @@ setup: "put mapping in * index": - do: indices.put_mapping: - include_type_name: false index: "*" body: properties: @@ -89,8 +83,7 @@ setup: analyzer: whitespace - do: - indices.get_mapping: - include_type_name: false + indices.get_mapping: {} - match: {test_index1.mappings.properties.text.type: text} - match: {test_index1.mappings.properties.text.analyzer: whitespace} @@ -105,7 +98,6 @@ setup: "put mapping in prefix* index": - do: indices.put_mapping: - include_type_name: false index: "test_index*" body: properties: @@ -114,8 +106,7 @@ setup: analyzer: whitespace - do: - indices.get_mapping: - include_type_name: false + indices.get_mapping: {} - match: {test_index1.mappings.properties.text.type: text} - match: {test_index1.mappings.properties.text.analyzer: whitespace} @@ -129,7 +120,6 @@ setup: "put mapping in list of indices": - do: indices.put_mapping: - include_type_name: false index: [test_index1, test_index2] body: properties: @@ -138,8 +128,7 @@ setup: analyzer: whitespace - do: - indices.get_mapping: - include_type_name: false + indices.get_mapping: {} - match: {test_index1.mappings.properties.text.type: text} - match: {test_index1.mappings.properties.text.analyzer: whitespace} @@ -154,7 +143,6 @@ setup: - do: indices.put_mapping: - include_type_name: false index: test_index1 body: dynamic: false @@ -165,7 +153,6 @@ setup: - do: indices.put_mapping: - include_type_name: false index: test_index1 body: dynamic: false @@ -175,8 +162,7 @@ setup: type: text - do: - indices.get_mapping: - include_type_name: false + indices.get_mapping: {} - match: {test_index1.mappings.properties.text.type: text} diff --git a/rest-api-spec/src/main/resources/rest-api-spec/test/indices.put_mapping/all_path_options_with_types.yml b/rest-api-spec/src/main/resources/rest-api-spec/test/indices.put_mapping/all_path_options_with_types.yml index e125ec2a70f4e..6f9b6f7d9ceef 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/test/indices.put_mapping/all_path_options_with_types.yml +++ b/rest-api-spec/src/main/resources/rest-api-spec/test/indices.put_mapping/all_path_options_with_types.yml @@ -14,6 +14,7 @@ setup: "put one mapping per index": - do: indices.put_mapping: + include_type_name: true index: test_index1 type: test_type body: @@ -24,6 +25,7 @@ setup: analyzer: whitespace - do: indices.put_mapping: + include_type_name: true index: test_index2 type: test_type body: @@ -35,7 +37,8 @@ setup: - do: - indices.get_mapping: {} + indices.get_mapping: + include_type_name: true - match: {test_index1.mappings.test_type.properties.text.type: text} - match: {test_index1.mappings.test_type.properties.text.analyzer: whitespace} @@ -50,6 +53,7 @@ setup: - do: indices.put_mapping: + include_type_name: true index: _all type: test_type body: @@ -60,7 +64,8 @@ setup: analyzer: whitespace - do: - indices.get_mapping: {} + indices.get_mapping: + include_type_name: true - match: {test_index1.mappings.test_type.properties.text.type: text} - match: {test_index1.mappings.test_type.properties.text.analyzer: whitespace} @@ -75,6 +80,7 @@ setup: "put mapping in * index": - do: indices.put_mapping: + include_type_name: true index: "*" type: test_type body: @@ -85,7 +91,8 @@ setup: analyzer: whitespace - do: - indices.get_mapping: {} + indices.get_mapping: + include_type_name: true - match: {test_index1.mappings.test_type.properties.text.type: text} - match: {test_index1.mappings.test_type.properties.text.analyzer: whitespace} @@ -100,6 +107,7 @@ setup: "put mapping in prefix* index": - do: indices.put_mapping: + include_type_name: true index: "test_index*" type: test_type body: @@ -110,7 +118,8 @@ setup: analyzer: whitespace - do: - indices.get_mapping: {} + indices.get_mapping: + include_type_name: true - match: {test_index1.mappings.test_type.properties.text.type: text} - match: {test_index1.mappings.test_type.properties.text.analyzer: whitespace} @@ -124,6 +133,7 @@ setup: "put mapping in list of indices": - do: indices.put_mapping: + include_type_name: true index: [test_index1, test_index2] type: test_type body: @@ -134,7 +144,8 @@ setup: analyzer: whitespace - do: - indices.get_mapping: {} + indices.get_mapping: + include_type_name: true - match: {test_index1.mappings.test_type.properties.text.type: text} - match: {test_index1.mappings.test_type.properties.text.analyzer: whitespace} @@ -148,6 +159,7 @@ setup: "put mapping with blank index": - do: indices.put_mapping: + include_type_name: true type: test_type body: test_type: @@ -157,7 +169,8 @@ setup: analyzer: whitespace - do: - indices.get_mapping: {} + indices.get_mapping: + include_type_name: true - match: {test_index1.mappings.test_type.properties.text.type: text} - match: {test_index1.mappings.test_type.properties.text.analyzer: whitespace} @@ -174,13 +187,15 @@ setup: - do: catch: param - indices.put_mapping: {} + indices.put_mapping: + include_type_name: true --- "post a mapping with default analyzer twice": - do: indices.put_mapping: + include_type_name: true index: test_index1 type: test_type body: @@ -193,6 +208,7 @@ setup: - do: indices.put_mapping: + include_type_name: true index: test_index1 type: test_type body: @@ -204,7 +220,8 @@ setup: type: text - do: - indices.get_mapping: {} + indices.get_mapping: + include_type_name: true - match: {test_index1.mappings.test_type.properties.text.type: text} diff --git a/rest-api-spec/src/main/resources/rest-api-spec/test/indices.put_template/10_basic.yml b/rest-api-spec/src/main/resources/rest-api-spec/test/indices.put_template/10_basic.yml index 8637b3e6d1864..c0b0bbe5d30b7 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/test/indices.put_template/10_basic.yml +++ b/rest-api-spec/src/main/resources/rest-api-spec/test/indices.put_template/10_basic.yml @@ -2,11 +2,10 @@ "Put template": - skip: version: " - 6.99.99" - reason: include_type_name is not supported before 7.0.0 + reason: include_type_name defaults to true before 7.0.0 - do: indices.put_template: - include_type_name: false name: test body: index_patterns: test-* @@ -20,7 +19,6 @@ - do: indices.get_template: - include_type_name: false name: test flat_settings: true @@ -32,11 +30,10 @@ "Put multiple template": - skip: version: " - 6.99.99" - reason: include_type_name was introduced in 7.0.0 + reason: include_type_name defaults to true before 7.0.0 - do: indices.put_template: - include_type_name: false name: test body: index_patterns: [test-*, test2-*] @@ -50,7 +47,6 @@ - do: indices.get_template: - include_type_name: false name: test flat_settings: true @@ -62,11 +58,10 @@ "Put template with empty mappings": - skip: version: " - 6.99.99" - reason: include_type_name was introduced in 7.0.0 + reason: include_type_name defaults to true before 7.0.0 - do: indices.put_template: - include_type_name: false name: test body: index_patterns: test-* @@ -77,7 +72,6 @@ - do: indices.get_template: - include_type_name: false name: test flat_settings: true diff --git a/rest-api-spec/src/main/resources/rest-api-spec/test/indices.put_template/11_basic_with_types.yml b/rest-api-spec/src/main/resources/rest-api-spec/test/indices.put_template/11_basic_with_types.yml index 1e14a9d3895a7..fde28db3c691d 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/test/indices.put_template/11_basic_with_types.yml +++ b/rest-api-spec/src/main/resources/rest-api-spec/test/indices.put_template/11_basic_with_types.yml @@ -2,6 +2,7 @@ "Put template": - do: indices.put_template: + include_type_name: true name: test body: index_patterns: test-* @@ -16,6 +17,7 @@ - do: indices.get_template: + include_type_name: true name: test flat_settings: true @@ -27,6 +29,7 @@ "Put multiple template": - do: indices.put_template: + include_type_name: true name: test body: index_patterns: [test-*, test2-*] @@ -41,6 +44,7 @@ - do: indices.get_template: + include_type_name: true name: test flat_settings: true @@ -52,6 +56,7 @@ "Put template with empty mappings": - do: indices.put_template: + include_type_name: true name: test body: index_patterns: test-* @@ -62,6 +67,7 @@ - do: indices.get_template: + include_type_name: true name: test flat_settings: true diff --git a/rest-api-spec/src/main/resources/rest-api-spec/test/indices.shrink/20_source_mapping.yml b/rest-api-spec/src/main/resources/rest-api-spec/test/indices.shrink/20_source_mapping.yml index 2e7a13b38e10e..c5715f7d4b2d5 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/test/indices.shrink/20_source_mapping.yml +++ b/rest-api-spec/src/main/resources/rest-api-spec/test/indices.shrink/20_source_mapping.yml @@ -39,14 +39,14 @@ # create template matching shrink target - do: indices.put_template: + include_type_name: false name: tpl1 body: index_patterns: targ* mappings: - test: - properties: - count: - type: integer + properties: + count: + type: integer # make it read-only - do: diff --git a/rest-api-spec/src/main/resources/rest-api-spec/test/indices.split/20_source_mapping.yml b/rest-api-spec/src/main/resources/rest-api-spec/test/indices.split/20_source_mapping.yml index 727e1e374ba65..0827860d88fdf 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/test/indices.split/20_source_mapping.yml +++ b/rest-api-spec/src/main/resources/rest-api-spec/test/indices.split/20_source_mapping.yml @@ -32,14 +32,14 @@ # create template matching shrink target - do: indices.put_template: + include_type_name: false name: tpl1 body: index_patterns: targ* mappings: - test: - properties: - count: - type: integer + properties: + count: + type: integer # make it read-only - do: diff --git a/rest-api-spec/src/main/resources/rest-api-spec/test/search/110_field_collapsing.yml b/rest-api-spec/src/main/resources/rest-api-spec/test/search/110_field_collapsing.yml index 885fdbdd06b1b..7ba1d9b62fbc7 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/test/search/110_field_collapsing.yml +++ b/rest-api-spec/src/main/resources/rest-api-spec/test/search/110_field_collapsing.yml @@ -383,12 +383,11 @@ setup: reason: Field aliases were introduced in 6.4.0. - do: indices.put_mapping: + include_type_name: false index: test - type: test body: - test: - properties: - group_alias: { type: alias, path: numeric_group } + properties: + group_alias: { type: alias, path: numeric_group } - do: search: rest_total_hits_as_int: true diff --git a/rest-api-spec/src/main/resources/rest-api-spec/test/suggest/30_context.yml b/rest-api-spec/src/main/resources/rest-api-spec/test/suggest/30_context.yml index 9ddbc922f614b..94233decd2217 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/test/suggest/30_context.yml +++ b/rest-api-spec/src/main/resources/rest-api-spec/test/suggest/30_context.yml @@ -195,7 +195,8 @@ setup: indices.refresh: {} - do: - indices.get_mapping: {} + indices.get_mapping: + include_type_name: false - do: search: @@ -247,7 +248,8 @@ setup: indices.refresh: {} - do: - indices.get_mapping: {} + indices.get_mapping: + include_type_name: false - do: search: 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 c7e64143f9c9a..9482a42a56e45 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 @@ -47,6 +47,7 @@ import java.util.Objects; import static org.elasticsearch.common.xcontent.XContentParserUtils.ensureExpectedToken; +import static org.elasticsearch.rest.BaseRestHandler.DEFAULT_INCLUDE_TYPE_NAME_POLICY; import static org.elasticsearch.rest.BaseRestHandler.INCLUDE_TYPE_NAME_PARAMETER; /** @@ -254,7 +255,8 @@ public XContentBuilder toXContent(XContentBuilder builder, Params params) throws builder.endObject(); ImmutableOpenMap indexMappings = mappings.get(index); - boolean includeTypeName = params.paramAsBoolean(INCLUDE_TYPE_NAME_PARAMETER, false); + boolean includeTypeName = params.paramAsBoolean(INCLUDE_TYPE_NAME_PARAMETER, + DEFAULT_INCLUDE_TYPE_NAME_POLICY); if (includeTypeName) { builder.startObject("mappings"); if (indexMappings != null) { diff --git a/server/src/main/java/org/elasticsearch/action/admin/indices/mapping/get/GetFieldMappingsResponse.java b/server/src/main/java/org/elasticsearch/action/admin/indices/mapping/get/GetFieldMappingsResponse.java index 2c07ebc68d0ef..f24c75d988e88 100644 --- a/server/src/main/java/org/elasticsearch/action/admin/indices/mapping/get/GetFieldMappingsResponse.java +++ b/server/src/main/java/org/elasticsearch/action/admin/indices/mapping/get/GetFieldMappingsResponse.java @@ -48,6 +48,7 @@ import static org.elasticsearch.common.xcontent.ConstructingObjectParser.optionalConstructorArg; import static org.elasticsearch.common.xcontent.XContentFactory.jsonBuilder; import static org.elasticsearch.common.xcontent.XContentParserUtils.ensureExpectedToken; +import static org.elasticsearch.rest.BaseRestHandler.DEFAULT_INCLUDE_TYPE_NAME_POLICY; /** Response object for {@link GetFieldMappingsRequest} API */ public class GetFieldMappingsResponse extends ActionResponse implements ToXContentObject { @@ -114,7 +115,8 @@ public FieldMappingMetaData fieldMappings(String index, String type, String fiel @Override public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException { - boolean includeTypeName = params.paramAsBoolean(BaseRestHandler.INCLUDE_TYPE_NAME_PARAMETER, true); + boolean includeTypeName = params.paramAsBoolean(BaseRestHandler.INCLUDE_TYPE_NAME_PARAMETER, + DEFAULT_INCLUDE_TYPE_NAME_POLICY); builder.startObject(); for (Map.Entry>> indexEntry : mappings.entrySet()) { diff --git a/server/src/main/java/org/elasticsearch/action/admin/indices/mapping/get/GetMappingsResponse.java b/server/src/main/java/org/elasticsearch/action/admin/indices/mapping/get/GetMappingsResponse.java index ad864c94e3693..4c037bd1d6df9 100644 --- a/server/src/main/java/org/elasticsearch/action/admin/indices/mapping/get/GetMappingsResponse.java +++ b/server/src/main/java/org/elasticsearch/action/admin/indices/mapping/get/GetMappingsResponse.java @@ -31,10 +31,13 @@ import org.elasticsearch.common.xcontent.ToXContentFragment; import org.elasticsearch.common.xcontent.XContentBuilder; import org.elasticsearch.common.xcontent.XContentParser; +import org.elasticsearch.rest.BaseRestHandler; import java.io.IOException; import java.util.Map; +import static org.elasticsearch.rest.BaseRestHandler.DEFAULT_INCLUDE_TYPE_NAME_POLICY; + public class GetMappingsResponse extends ActionResponse implements ToXContentFragment { private static final ParseField MAPPINGS = new ParseField("mappings"); @@ -119,10 +122,9 @@ public static GetMappingsResponse fromXContent(XContentParser parser) throws IOE @Override public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException { - return toXContent(builder, params, true); - } + boolean includeTypeName = params.paramAsBoolean(BaseRestHandler.INCLUDE_TYPE_NAME_PARAMETER, + DEFAULT_INCLUDE_TYPE_NAME_POLICY); - public XContentBuilder toXContent(XContentBuilder builder, Params params, boolean includeTypeName) throws IOException { for (final ObjectObjectCursor> indexEntry : getMappings()) { builder.startObject(indexEntry.key); { diff --git a/server/src/main/java/org/elasticsearch/action/admin/indices/template/get/GetIndexTemplatesResponse.java b/server/src/main/java/org/elasticsearch/action/admin/indices/template/get/GetIndexTemplatesResponse.java index 2bdc966c74e59..9749aaa05b1a4 100644 --- a/server/src/main/java/org/elasticsearch/action/admin/indices/template/get/GetIndexTemplatesResponse.java +++ b/server/src/main/java/org/elasticsearch/action/admin/indices/template/get/GetIndexTemplatesResponse.java @@ -32,6 +32,8 @@ import java.util.List; import static java.util.Collections.singletonMap; +import static org.elasticsearch.rest.BaseRestHandler.DEFAULT_INCLUDE_TYPE_NAME_POLICY; +import static org.elasticsearch.rest.BaseRestHandler.INCLUDE_TYPE_NAME_PARAMETER; public class GetIndexTemplatesResponse extends ActionResponse implements ToXContentObject { @@ -71,9 +73,17 @@ public void writeTo(StreamOutput out) throws IOException { @Override public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException { params = new ToXContent.DelegatingMapParams(singletonMap("reduce_mappings", "true"), params); + + boolean includeTypeName = params.paramAsBoolean(INCLUDE_TYPE_NAME_PARAMETER, + DEFAULT_INCLUDE_TYPE_NAME_POLICY); + builder.startObject(); for (IndexTemplateMetaData indexTemplateMetaData : getIndexTemplates()) { - IndexTemplateMetaData.Builder.toXContent(indexTemplateMetaData, builder, params); + if (includeTypeName) { + IndexTemplateMetaData.Builder.toXContentWithTypes(indexTemplateMetaData, builder, params); + } else { + IndexTemplateMetaData.Builder.toXContent(indexTemplateMetaData, builder, params); + } } builder.endObject(); return builder; diff --git a/server/src/main/java/org/elasticsearch/cluster/metadata/IndexTemplateMetaData.java b/server/src/main/java/org/elasticsearch/cluster/metadata/IndexTemplateMetaData.java index 4055af3e2f460..e21aa95865a6f 100644 --- a/server/src/main/java/org/elasticsearch/cluster/metadata/IndexTemplateMetaData.java +++ b/server/src/main/java/org/elasticsearch/cluster/metadata/IndexTemplateMetaData.java @@ -42,7 +42,6 @@ import org.elasticsearch.common.xcontent.XContentHelper; import org.elasticsearch.common.xcontent.XContentParser; import org.elasticsearch.index.mapper.MapperService; -import org.elasticsearch.rest.BaseRestHandler; import java.io.IOException; import java.util.ArrayList; @@ -331,19 +330,47 @@ public IndexTemplateMetaData build() { return new IndexTemplateMetaData(name, order, version, indexPatterns, settings, mappings.build(), aliases.build()); } - public static void toXContent(IndexTemplateMetaData indexTemplateMetaData, XContentBuilder builder, ToXContent.Params params) - throws IOException { + /** + * Serializes the template to xContent, using the legacy format where the mappings are + * nested under the type name. + * + * This method is used for serializing templates before storing them in the cluster metadata, + * and also in the REST layer when returning a deprecated typed response. + */ + public static void toXContentWithTypes(IndexTemplateMetaData indexTemplateMetaData, + XContentBuilder builder, + ToXContent.Params params) throws IOException { builder.startObject(indexTemplateMetaData.name()); + toInnerXContent(indexTemplateMetaData, builder, params, true); + builder.endObject(); + } - toInnerXContent(indexTemplateMetaData, builder, params); - + /** + * Serializes the template to xContent, making sure not to nest mappings under the + * type name. + * + * Note that this method should currently only be used for creating REST responses, + * and not when directly updating stored templates. Index templates are still stored + * in the old, typed format, and have yet to be migrated to be typeless. + */ + public static void toXContent(IndexTemplateMetaData indexTemplateMetaData, + XContentBuilder builder, + ToXContent.Params params) throws IOException { + builder.startObject(indexTemplateMetaData.name()); + toInnerXContent(indexTemplateMetaData, builder, params, false); builder.endObject(); } - public static void toInnerXContent(IndexTemplateMetaData indexTemplateMetaData, XContentBuilder builder, ToXContent.Params params) - throws IOException { + static void toInnerXContentWithTypes(IndexTemplateMetaData indexTemplateMetaData, + XContentBuilder builder, + ToXContent.Params params) throws IOException { + toInnerXContent(indexTemplateMetaData, builder, params, true); + } - boolean includeTypeName = params.paramAsBoolean(BaseRestHandler.INCLUDE_TYPE_NAME_PARAMETER, true); + private static void toInnerXContent(IndexTemplateMetaData indexTemplateMetaData, + XContentBuilder builder, + ToXContent.Params params, + boolean includeTypeName) throws IOException { builder.field("order", indexTemplateMetaData.order()); if (indexTemplateMetaData.version() != null) { diff --git a/server/src/main/java/org/elasticsearch/cluster/metadata/MetaData.java b/server/src/main/java/org/elasticsearch/cluster/metadata/MetaData.java index a9ec770720732..3cce3f791d2b8 100644 --- a/server/src/main/java/org/elasticsearch/cluster/metadata/MetaData.java +++ b/server/src/main/java/org/elasticsearch/cluster/metadata/MetaData.java @@ -1245,7 +1245,7 @@ public static void toXContent(MetaData metaData, XContentBuilder builder, ToXCon builder.startObject("templates"); for (ObjectCursor cursor : metaData.templates().values()) { - IndexTemplateMetaData.Builder.toXContent(cursor.value, builder, params); + IndexTemplateMetaData.Builder.toXContentWithTypes(cursor.value, builder, params); } builder.endObject(); diff --git a/server/src/main/java/org/elasticsearch/cluster/metadata/TemplateUpgradeService.java b/server/src/main/java/org/elasticsearch/cluster/metadata/TemplateUpgradeService.java index 227dca6b739bc..deb04067acb9a 100644 --- a/server/src/main/java/org/elasticsearch/cluster/metadata/TemplateUpgradeService.java +++ b/server/src/main/java/org/elasticsearch/cluster/metadata/TemplateUpgradeService.java @@ -253,7 +253,7 @@ Optional, Set>> calculateTemplateChang private BytesReference toBytesReference(IndexTemplateMetaData templateMetaData) { try { return XContentHelper.toXContent((builder, params) -> { - IndexTemplateMetaData.Builder.toInnerXContent(templateMetaData, builder, params); + IndexTemplateMetaData.Builder.toInnerXContentWithTypes(templateMetaData, builder, params); return builder; }, XContentType.JSON, PARAMS, false); } catch (IOException ex) { diff --git a/server/src/main/java/org/elasticsearch/rest/BaseRestHandler.java b/server/src/main/java/org/elasticsearch/rest/BaseRestHandler.java index 97b4e29d9a208..98611e4c9d8ab 100644 --- a/server/src/main/java/org/elasticsearch/rest/BaseRestHandler.java +++ b/server/src/main/java/org/elasticsearch/rest/BaseRestHandler.java @@ -63,6 +63,7 @@ public abstract class BaseRestHandler extends AbstractComponent implements RestH * Note: Support for this parameter will be removed after the transition period to typeless APIs. */ public static final String INCLUDE_TYPE_NAME_PARAMETER = "include_type_name"; + public static final boolean DEFAULT_INCLUDE_TYPE_NAME_POLICY = false; protected BaseRestHandler(Settings settings) { // TODO drop settings from ctor diff --git a/server/src/main/java/org/elasticsearch/rest/action/admin/indices/RestCreateIndexAction.java b/server/src/main/java/org/elasticsearch/rest/action/admin/indices/RestCreateIndexAction.java index aa0867696c674..ed2724b95bb47 100644 --- a/server/src/main/java/org/elasticsearch/rest/action/admin/indices/RestCreateIndexAction.java +++ b/server/src/main/java/org/elasticsearch/rest/action/admin/indices/RestCreateIndexAction.java @@ -49,7 +49,8 @@ public String getName() { @Override public RestChannelConsumer prepareRequest(final RestRequest request, final NodeClient client) throws IOException { - final boolean includeTypeName = request.paramAsBoolean(INCLUDE_TYPE_NAME_PARAMETER, true); + final boolean includeTypeName = request.paramAsBoolean(INCLUDE_TYPE_NAME_PARAMETER, + DEFAULT_INCLUDE_TYPE_NAME_POLICY); CreateIndexRequest createIndexRequest = new CreateIndexRequest(request.param("index")); if (request.hasContent()) { Map sourceAsMap = XContentHelper.convertToMap(request.content(), false, request.getXContentType()).v2(); diff --git a/server/src/main/java/org/elasticsearch/rest/action/admin/indices/RestGetFieldMappingAction.java b/server/src/main/java/org/elasticsearch/rest/action/admin/indices/RestGetFieldMappingAction.java index f3a73fa29fd98..54cc9fd988f13 100644 --- a/server/src/main/java/org/elasticsearch/rest/action/admin/indices/RestGetFieldMappingAction.java +++ b/server/src/main/java/org/elasticsearch/rest/action/admin/indices/RestGetFieldMappingAction.java @@ -63,10 +63,10 @@ public RestChannelConsumer prepareRequest(final RestRequest request, final NodeC final String[] types = request.paramAsStringArrayOrEmptyIfAll("type"); final String[] fields = Strings.splitStringByCommaToArray(request.param("fields")); - boolean includeTypeName = request.paramAsBoolean(INCLUDE_TYPE_NAME_PARAMETER, true); + boolean includeTypeName = request.paramAsBoolean(INCLUDE_TYPE_NAME_PARAMETER, DEFAULT_INCLUDE_TYPE_NAME_POLICY); if (includeTypeName == false && types.length > 0) { - throw new IllegalArgumentException("Cannot set include_type_name=false and specify" + - " types at the same time."); + throw new IllegalArgumentException("Types cannot be specified unless include_type_name" + + " is set to true."); } GetFieldMappingsRequest getMappingsRequest = new GetFieldMappingsRequest(); diff --git a/server/src/main/java/org/elasticsearch/rest/action/admin/indices/RestGetMappingAction.java b/server/src/main/java/org/elasticsearch/rest/action/admin/indices/RestGetMappingAction.java index ee052f96a1a33..da7f2af501db2 100644 --- a/server/src/main/java/org/elasticsearch/rest/action/admin/indices/RestGetMappingAction.java +++ b/server/src/main/java/org/elasticsearch/rest/action/admin/indices/RestGetMappingAction.java @@ -20,7 +20,6 @@ package org.elasticsearch.rest.action.admin.indices; import com.carrotsearch.hppc.cursors.ObjectCursor; - import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; import org.elasticsearch.action.admin.indices.mapping.get.GetMappingsRequest; @@ -34,7 +33,6 @@ import org.elasticsearch.common.regex.Regex; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.util.set.Sets; -import org.elasticsearch.common.xcontent.ToXContent; import org.elasticsearch.common.xcontent.XContentBuilder; import org.elasticsearch.indices.TypeMissingException; import org.elasticsearch.rest.BaseRestHandler; @@ -82,13 +80,17 @@ public String getName() { @Override public RestChannelConsumer prepareRequest(final RestRequest request, final NodeClient client) throws IOException { + final String[] indices = Strings.splitStringByCommaToArray(request.param("index")); + final String[] types = request.paramAsStringArrayOrEmptyIfAll("type"); + boolean includeTypeName = request.paramAsBoolean(INCLUDE_TYPE_NAME_PARAMETER, DEFAULT_INCLUDE_TYPE_NAME_POLICY); + if (request.method().equals(HEAD)) { deprecationLogger.deprecated("Type exists requests are deprecated, as types have been deprecated."); + } else if (includeTypeName == false && types.length > 0) { + throw new IllegalArgumentException("Types cannot be provided in get mapping requests, unless" + + " include_type_name is set to true."); } - final boolean includeTypeName = request.paramAsBoolean(INCLUDE_TYPE_NAME_PARAMETER, true); - final String[] indices = Strings.splitStringByCommaToArray(request.param("index")); - final String[] types = request.paramAsStringArrayOrEmptyIfAll("type"); final GetMappingsRequest getMappingsRequest = new GetMappingsRequest(); getMappingsRequest.indices(indices).types(types); getMappingsRequest.indicesOptions(IndicesOptions.fromRequest(request, getMappingsRequest.indicesOptions())); @@ -139,7 +141,7 @@ public RestResponse buildResponse(final GetMappingsResponse response, final XCon builder.field("error", message); builder.field("status", status.getStatus()); } - response.toXContent(builder, ToXContent.EMPTY_PARAMS, includeTypeName); + response.toXContent(builder, request); } builder.endObject(); diff --git a/server/src/main/java/org/elasticsearch/rest/action/admin/indices/RestPutIndexTemplateAction.java b/server/src/main/java/org/elasticsearch/rest/action/admin/indices/RestPutIndexTemplateAction.java index f5cc3c6aad26d..2b72a724a8906 100644 --- a/server/src/main/java/org/elasticsearch/rest/action/admin/indices/RestPutIndexTemplateAction.java +++ b/server/src/main/java/org/elasticsearch/rest/action/admin/indices/RestPutIndexTemplateAction.java @@ -68,7 +68,7 @@ public RestChannelConsumer prepareRequest(final RestRequest request, final NodeC putRequest.create(request.paramAsBoolean("create", false)); putRequest.cause(request.param("cause", "")); - boolean includeTypeName = request.paramAsBoolean(INCLUDE_TYPE_NAME_PARAMETER, true); + boolean includeTypeName = request.paramAsBoolean(INCLUDE_TYPE_NAME_PARAMETER, DEFAULT_INCLUDE_TYPE_NAME_POLICY); Map sourceAsMap = prepareRequestSource(request, includeTypeName); putRequest.source(sourceAsMap); diff --git a/server/src/main/java/org/elasticsearch/rest/action/admin/indices/RestPutMappingAction.java b/server/src/main/java/org/elasticsearch/rest/action/admin/indices/RestPutMappingAction.java index 286bd82fd3501..525b398d48b7a 100644 --- a/server/src/main/java/org/elasticsearch/rest/action/admin/indices/RestPutMappingAction.java +++ b/server/src/main/java/org/elasticsearch/rest/action/admin/indices/RestPutMappingAction.java @@ -68,11 +68,13 @@ public String getName() { @Override public RestChannelConsumer prepareRequest(final RestRequest request, final NodeClient client) throws IOException { - final boolean includeTypeName = request.paramAsBoolean(INCLUDE_TYPE_NAME_PARAMETER, true); + final boolean includeTypeName = request.paramAsBoolean(INCLUDE_TYPE_NAME_PARAMETER, + DEFAULT_INCLUDE_TYPE_NAME_POLICY); PutMappingRequest putMappingRequest = putMappingRequest(Strings.splitStringByCommaToArray(request.param("index"))); final String type = request.param("type"); if (type != null && includeTypeName == false) { - throw new IllegalArgumentException("Cannot set include_type_name=false and provide a type at the same time"); + throw new IllegalArgumentException("Types cannot be provided in put mapping requests, unless " + + "the include_type_name parameter is set to true."); } putMappingRequest.type(includeTypeName ? type : MapperService.SINGLE_MAPPING_NAME); putMappingRequest.source(request.requiredContent(), request.getXContentType()); diff --git a/server/src/test/java/org/elasticsearch/action/admin/indices/mapping/get/GetFieldMappingsResponseTests.java b/server/src/test/java/org/elasticsearch/action/admin/indices/mapping/get/GetFieldMappingsResponseTests.java index b6e785a4d05be..734de94b1c419 100644 --- a/server/src/test/java/org/elasticsearch/action/admin/indices/mapping/get/GetFieldMappingsResponseTests.java +++ b/server/src/test/java/org/elasticsearch/action/admin/indices/mapping/get/GetFieldMappingsResponseTests.java @@ -24,6 +24,7 @@ import org.elasticsearch.common.io.stream.BytesStreamOutput; import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.xcontent.LoggingDeprecationHandler; +import org.elasticsearch.common.xcontent.ToXContent; import org.elasticsearch.common.xcontent.XContentParser; import org.elasticsearch.common.xcontent.XContentType; import org.elasticsearch.test.AbstractStreamableXContentTestCase; @@ -34,6 +35,7 @@ import java.util.Map; import java.util.function.Predicate; +import static org.elasticsearch.rest.BaseRestHandler.INCLUDE_TYPE_NAME_PARAMETER; import static org.hamcrest.CoreMatchers.equalTo; public class GetFieldMappingsResponseTests extends AbstractStreamableXContentTestCase { @@ -121,6 +123,15 @@ protected Predicate getRandomFieldsExcludeFilter() { || s.matches("(?[^.]+)\\.mappings\\.(?[^.]+)\\.(?[^.]+)")); } + /** + * For now, we only unit test the legacy typed responses. This will soon no longer be the case, + * as we introduce support for typeless xContent parsing in {@link GetFieldMappingsResponse}. + */ + @Override + protected ToXContent.Params getToXContentParams() { + return new ToXContent.MapParams(Collections.singletonMap(INCLUDE_TYPE_NAME_PARAMETER, "true")); + } + private Map>> randomMapping() { Map>> mappings = new HashMap<>(); 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 481d05ea8cc07..633d74acde174 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 @@ -23,6 +23,7 @@ import org.elasticsearch.cluster.metadata.MappingMetaData; import org.elasticsearch.common.collect.ImmutableOpenMap; +import org.elasticsearch.common.xcontent.ToXContent; import org.elasticsearch.common.xcontent.XContentParser; import org.elasticsearch.index.mapper.MapperService; import org.elasticsearch.test.AbstractStreamableXContentTestCase; @@ -30,12 +31,15 @@ import java.io.IOException; import java.util.ArrayList; +import java.util.Collections; import java.util.HashMap; import java.util.Iterator; import java.util.List; import java.util.Map; import java.util.Objects; +import static org.elasticsearch.rest.BaseRestHandler.INCLUDE_TYPE_NAME_PARAMETER; + public class GetMappingsResponseTests extends AbstractStreamableXContentTestCase { @Override @@ -125,6 +129,15 @@ protected GetMappingsResponse createTestInstance() { return resp; } + /** + * For now, we only unit test the legacy typed responses. This will soon no longer be the + * case, as we introduce support for typeless xContent parsing in {@link GetMappingsResponse}. + */ + @Override + protected ToXContent.Params getToXContentParams() { + return new ToXContent.MapParams(Collections.singletonMap(INCLUDE_TYPE_NAME_PARAMETER, "true")); + } + // Not meant to be exhaustive private static Map randomFieldMapping() { Map mappings = new HashMap<>(); diff --git a/server/src/test/java/org/elasticsearch/action/admin/indices/template/get/GetIndexTemplatesResponseTests.java b/server/src/test/java/org/elasticsearch/action/admin/indices/template/get/GetIndexTemplatesResponseTests.java index 6cb26967d07fa..0806ff76906b5 100644 --- a/server/src/test/java/org/elasticsearch/action/admin/indices/template/get/GetIndexTemplatesResponseTests.java +++ b/server/src/test/java/org/elasticsearch/action/admin/indices/template/get/GetIndexTemplatesResponseTests.java @@ -19,20 +19,24 @@ package org.elasticsearch.action.admin.indices.template.get; +import org.elasticsearch.action.admin.indices.mapping.get.GetFieldMappingsResponse; import org.elasticsearch.cluster.metadata.AliasMetaData; import org.elasticsearch.cluster.metadata.IndexTemplateMetaData; import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.common.xcontent.ToXContent; import org.elasticsearch.common.xcontent.XContentParser; import org.elasticsearch.test.AbstractXContentTestCase; import java.io.IOException; import java.io.UncheckedIOException; import java.util.ArrayList; +import java.util.Collections; import java.util.HashSet; import java.util.List; import java.util.stream.Collectors; import java.util.stream.IntStream; +import static org.elasticsearch.rest.BaseRestHandler.INCLUDE_TYPE_NAME_PARAMETER; import static org.hamcrest.Matchers.equalTo; public class GetIndexTemplatesResponseTests extends AbstractXContentTestCase { @@ -81,6 +85,15 @@ protected boolean supportsUnknownFields() { return false; } + /** + * For now, we only unit test the legacy typed responses. This will soon no longer be the case, + * as we introduce support for typeless xContent parsing in {@link GetFieldMappingsResponse}. + */ + @Override + protected ToXContent.Params getToXContentParams() { + return new ToXContent.MapParams(Collections.singletonMap(INCLUDE_TYPE_NAME_PARAMETER, "true")); + } + @Override protected void assertEqualInstances(GetIndexTemplatesResponse expectedInstance, GetIndexTemplatesResponse newInstance) { assertNotSame(newInstance, expectedInstance); diff --git a/server/src/test/java/org/elasticsearch/cluster/metadata/IndexTemplateMetaDataTests.java b/server/src/test/java/org/elasticsearch/cluster/metadata/IndexTemplateMetaDataTests.java index 5fc0764235417..03d10aca7e812 100644 --- a/server/src/test/java/org/elasticsearch/cluster/metadata/IndexTemplateMetaDataTests.java +++ b/server/src/test/java/org/elasticsearch/cluster/metadata/IndexTemplateMetaDataTests.java @@ -62,7 +62,7 @@ public void testIndexTemplateMetaDataXContentRoundTrip() throws Exception { final BytesReference templateBytesRoundTrip; try (XContentBuilder builder = XContentBuilder.builder(JsonXContent.jsonXContent)) { builder.startObject(); - IndexTemplateMetaData.Builder.toXContent(indexTemplateMetaData, builder, params); + IndexTemplateMetaData.Builder.toXContentWithTypes(indexTemplateMetaData, builder, params); builder.endObject(); templateBytesRoundTrip = BytesReference.bytes(builder); } @@ -161,7 +161,7 @@ public void testFromToXContent() throws Exception { IndexTemplateMetaData template = templateBuilder.build(); XContentBuilder builder = XContentBuilder.builder(randomFrom(XContentType.JSON.xContent())); builder.startObject(); - IndexTemplateMetaData.Builder.toXContent(template, builder, ToXContent.EMPTY_PARAMS); + IndexTemplateMetaData.Builder.toXContentWithTypes(template, builder, ToXContent.EMPTY_PARAMS); builder.endObject(); try (XContentParser parser = createParser(shuffleXContent(builder))) { IndexTemplateMetaData parsed = IndexTemplateMetaData.Builder.fromXContent(parser, templateName); diff --git a/server/src/test/java/org/elasticsearch/rest/action/admin/indices/RestGetMappingActionTests.java b/server/src/test/java/org/elasticsearch/rest/action/admin/indices/RestGetMappingActionTests.java index 8f9093df2fc8b..8eea9dc34c2fb 100644 --- a/server/src/test/java/org/elasticsearch/rest/action/admin/indices/RestGetMappingActionTests.java +++ b/server/src/test/java/org/elasticsearch/rest/action/admin/indices/RestGetMappingActionTests.java @@ -21,17 +21,21 @@ import org.elasticsearch.client.node.NodeClient; import org.elasticsearch.common.settings.Settings; +import org.elasticsearch.common.util.concurrent.ThreadContext; import org.elasticsearch.rest.RestController; import org.elasticsearch.rest.RestRequest; -import org.elasticsearch.test.ESTestCase; +import org.elasticsearch.rest.RestStatus; +import org.elasticsearch.test.rest.FakeRestChannel; import org.elasticsearch.test.rest.FakeRestRequest; +import org.elasticsearch.test.rest.RestActionTestCase; 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 RestGetMappingActionTests extends ESTestCase { +public class RestGetMappingActionTests extends RestActionTestCase { public void testTypeExistsDeprecation() throws Exception { Map params = new HashMap<>(); @@ -46,4 +50,23 @@ public void testTypeExistsDeprecation() throws Exception { assertWarnings("Type exists requests are deprecated, as types have been deprecated."); } + + public void testTypeInPath() { + // Test that specifying a type while setting include_type_name to false + // results in an illegal argument exception. + Map params = new HashMap<>(); + params.put(INCLUDE_TYPE_NAME_PARAMETER, "false"); + RestRequest request = new FakeRestRequest.Builder(xContentRegistry()) + .withMethod(RestRequest.Method.GET) + .withPath("some_index/some_type/_mapping/some_field") + .withParams(params) + .build(); + + FakeRestChannel channel = new FakeRestChannel(request, false, 1); + ThreadContext threadContext = new ThreadContext(Settings.EMPTY); + controller().dispatchRequest(request, channel, threadContext); + + assertEquals(1, channel.errors().get()); + assertEquals(RestStatus.BAD_REQUEST, channel.capturedResponse().status()); + } } diff --git a/test/framework/src/main/java/org/elasticsearch/test/rest/ESRestTestCase.java b/test/framework/src/main/java/org/elasticsearch/test/rest/ESRestTestCase.java index c711019bf753c..668f837dcedf1 100644 --- a/test/framework/src/main/java/org/elasticsearch/test/rest/ESRestTestCase.java +++ b/test/framework/src/main/java/org/elasticsearch/test/rest/ESRestTestCase.java @@ -801,7 +801,9 @@ protected static void ensureNoInitializingShards() throws IOException { } protected static void createIndex(String name, Settings settings) throws IOException { - createIndex(name, settings, ""); + Request request = new Request("PUT", "/" + name); + request.setJsonEntity("{\n \"settings\": " + Strings.toString(settings) + "}"); + client().performRequest(request); } protected static void createIndex(String name, Settings settings, String mapping) throws IOException { diff --git a/test/framework/src/main/java/org/elasticsearch/test/rest/yaml/ClientYamlTestExecutionContext.java b/test/framework/src/main/java/org/elasticsearch/test/rest/yaml/ClientYamlTestExecutionContext.java index b1337172a5679..d7ff384837562 100644 --- a/test/framework/src/main/java/org/elasticsearch/test/rest/yaml/ClientYamlTestExecutionContext.java +++ b/test/framework/src/main/java/org/elasticsearch/test/rest/yaml/ClientYamlTestExecutionContext.java @@ -40,6 +40,8 @@ import java.util.List; import java.util.Map; +import static org.elasticsearch.rest.BaseRestHandler.INCLUDE_TYPE_NAME_PARAMETER; + /** * Execution context passed across the REST tests. * Holds the REST client used to communicate with elasticsearch. @@ -96,6 +98,14 @@ public ClientYamlTestResponse callApi(String apiName, Map params } } + // Although include_type_name defaults to false, there is a large number of typed index creations + // in REST tests that need to be manually converted to typeless calls. As a temporary measure, we + // specify include_type_name=true in indices.create calls, unless the parameter has been set otherwise. + // This workaround will be removed once we convert all index creations to be typeless. + if (apiName.equals("indices.create") && requestParams.containsKey(INCLUDE_TYPE_NAME_PARAMETER) == false) { + requestParams.put(INCLUDE_TYPE_NAME_PARAMETER, "true"); + } + HttpEntity entity = createEntity(bodies, requestHeaders); try { response = callApiInternal(apiName, requestParams, entity, requestHeaders, nodeSelector); diff --git a/x-pack/docs/en/security/authorization/alias-privileges.asciidoc b/x-pack/docs/en/security/authorization/alias-privileges.asciidoc index b9b6d44fd69b7..aca6fc27b1dd8 100644 --- a/x-pack/docs/en/security/authorization/alias-privileges.asciidoc +++ b/x-pack/docs/en/security/authorization/alias-privileges.asciidoc @@ -54,7 +54,7 @@ added to an index directly as part of the index creation: [source,shell] ------------------------------------------------------------------------------- -PUT /2015 +PUT /2015?include_type_name=true { "aliases" : { "current_year" : {} diff --git a/x-pack/plugin/ccr/qa/downgrade-to-basic-license/src/test/java/org/elasticsearch/xpack/ccr/FollowIndexIT.java b/x-pack/plugin/ccr/qa/downgrade-to-basic-license/src/test/java/org/elasticsearch/xpack/ccr/FollowIndexIT.java index 26d7825198d8a..b612d8822437e 100644 --- a/x-pack/plugin/ccr/qa/downgrade-to-basic-license/src/test/java/org/elasticsearch/xpack/ccr/FollowIndexIT.java +++ b/x-pack/plugin/ccr/qa/downgrade-to-basic-license/src/test/java/org/elasticsearch/xpack/ccr/FollowIndexIT.java @@ -114,7 +114,7 @@ private void createNewIndexAndIndexDocs(RestClient client, String index) throws .build(); Request request = new Request("PUT", "/" + index); request.setJsonEntity("{\"settings\": " + Strings.toString(settings) + - ", \"mappings\": {\"_doc\": {\"properties\": {\"field\": {\"type\": \"keyword\"}}}} }"); + ", \"mappings\": {\"properties\": {\"field\": {\"type\": \"keyword\"}}}}"); assertOK(client.performRequest(request)); for (int i = 0; i < 5; i++) { diff --git a/x-pack/plugin/ccr/qa/multi-cluster/src/test/java/org/elasticsearch/xpack/ccr/AutoFollowIT.java b/x-pack/plugin/ccr/qa/multi-cluster/src/test/java/org/elasticsearch/xpack/ccr/AutoFollowIT.java index d1f28fa0da918..055c536faac39 100644 --- a/x-pack/plugin/ccr/qa/multi-cluster/src/test/java/org/elasticsearch/xpack/ccr/AutoFollowIT.java +++ b/x-pack/plugin/ccr/qa/multi-cluster/src/test/java/org/elasticsearch/xpack/ccr/AutoFollowIT.java @@ -38,7 +38,7 @@ public void testMultipleAutoFollowPatternsDifferentClusters() throws Exception { .build(); Request request = new Request("PUT", "/logs-20190101"); request.setJsonEntity("{\"settings\": " + Strings.toString(settings) + - ", \"mappings\": {\"_doc\": {\"properties\": {\"field\": {\"type\": \"keyword\"}}}} }"); + ", \"mappings\": {\"properties\": {\"field\": {\"type\": \"keyword\"}}}}"); assertOK(leaderClient.performRequest(request)); for (int i = 0; i < 5; i++) { String id = Integer.toString(i); @@ -51,7 +51,7 @@ public void testMultipleAutoFollowPatternsDifferentClusters() throws Exception { .build(); Request request = new Request("PUT", "/logs-20200101"); request.setJsonEntity("{\"settings\": " + Strings.toString(settings) + - ", \"mappings\": {\"_doc\": {\"properties\": {\"field\": {\"type\": \"keyword\"}}}} }"); + ", \"mappings\": {\"properties\": {\"field\": {\"type\": \"keyword\"}}}}"); assertOK(middleClient.performRequest(request)); for (int i = 0; i < 5; i++) { String id = Integer.toString(i); @@ -85,7 +85,7 @@ public void testAutoFollowPatterns() throws Exception { .build(); request = new Request("PUT", "/metrics-20210101"); request.setJsonEntity("{\"settings\": " + Strings.toString(settings) + - ", \"mappings\": {\"_doc\": {\"properties\": {\"field\": {\"type\": \"keyword\"}}}} }"); + ", \"mappings\": {\"properties\": {\"field\": {\"type\": \"keyword\"}}}}"); assertOK(leaderClient.performRequest(request)); for (int i = 0; i < 5; i++) { diff --git a/x-pack/plugin/ccr/qa/multi-cluster/src/test/java/org/elasticsearch/xpack/ccr/ChainIT.java b/x-pack/plugin/ccr/qa/multi-cluster/src/test/java/org/elasticsearch/xpack/ccr/ChainIT.java index e5a37aa829bbf..b3e6384f7f635 100644 --- a/x-pack/plugin/ccr/qa/multi-cluster/src/test/java/org/elasticsearch/xpack/ccr/ChainIT.java +++ b/x-pack/plugin/ccr/qa/multi-cluster/src/test/java/org/elasticsearch/xpack/ccr/ChainIT.java @@ -20,12 +20,10 @@ public void testFollowIndex() throws Exception { String mapping = ""; if (randomBoolean()) { // randomly do source filtering on indexing mapping = - "\"_doc\": {" + - " \"_source\": {" + - " \"includes\": [\"field\"]," + - " \"excludes\": [\"filtered_field\"]" + - " }"+ - "}"; + "\"_source\": {" + + " \"includes\": [\"field\"]," + + " \"excludes\": [\"filtered_field\"]" + + "}"; } Settings indexSettings = Settings.builder() .put("index.soft_deletes.enabled", true) diff --git a/x-pack/plugin/ccr/qa/multi-cluster/src/test/java/org/elasticsearch/xpack/ccr/FollowIndexIT.java b/x-pack/plugin/ccr/qa/multi-cluster/src/test/java/org/elasticsearch/xpack/ccr/FollowIndexIT.java index b44abb15fad15..b50fe9233e8fa 100644 --- a/x-pack/plugin/ccr/qa/multi-cluster/src/test/java/org/elasticsearch/xpack/ccr/FollowIndexIT.java +++ b/x-pack/plugin/ccr/qa/multi-cluster/src/test/java/org/elasticsearch/xpack/ccr/FollowIndexIT.java @@ -25,11 +25,9 @@ public void testFollowIndex() throws Exception { String mapping = ""; if (randomBoolean()) { // randomly do source filtering on indexing mapping = - "\"_doc\": {" + - " \"_source\": {" + - " \"includes\": [\"field\"]," + - " \"excludes\": [\"filtered_field\"]" + - " }"+ + "\"_source\": {" + + " \"includes\": [\"field\"]," + + " \"excludes\": [\"filtered_field\"]" + "}"; } Settings indexSettings = Settings.builder() diff --git a/x-pack/plugin/ccr/qa/security/src/test/java/org/elasticsearch/xpack/ccr/FollowIndexSecurityIT.java b/x-pack/plugin/ccr/qa/security/src/test/java/org/elasticsearch/xpack/ccr/FollowIndexSecurityIT.java index 2b902d2a18399..91b94f1c4b57d 100644 --- a/x-pack/plugin/ccr/qa/security/src/test/java/org/elasticsearch/xpack/ccr/FollowIndexSecurityIT.java +++ b/x-pack/plugin/ccr/qa/security/src/test/java/org/elasticsearch/xpack/ccr/FollowIndexSecurityIT.java @@ -148,7 +148,7 @@ public void testAutoFollowPatterns() throws Exception { .put("index.soft_deletes.enabled", true) .build(); String requestBody = "{\"settings\": " + Strings.toString(settings) + - ", \"mappings\": {\"_doc\": {\"properties\": {\"field\": {\"type\": \"keyword\"}}}} }"; + ", \"mappings\": {\"properties\": {\"field\": {\"type\": \"keyword\"}}}}"; request = new Request("PUT", "/" + index); request.setJsonEntity(requestBody); assertOK(leaderClient.performRequest(request)); diff --git a/x-pack/plugin/ml/qa/basic-multi-node/src/test/java/org/elasticsearch/xpack/ml/integration/MlBasicMultiNodeIT.java b/x-pack/plugin/ml/qa/basic-multi-node/src/test/java/org/elasticsearch/xpack/ml/integration/MlBasicMultiNodeIT.java index 0cfa6f5ebf04e..31b43eb055523 100644 --- a/x-pack/plugin/ml/qa/basic-multi-node/src/test/java/org/elasticsearch/xpack/ml/integration/MlBasicMultiNodeIT.java +++ b/x-pack/plugin/ml/qa/basic-multi-node/src/test/java/org/elasticsearch/xpack/ml/integration/MlBasicMultiNodeIT.java @@ -106,12 +106,10 @@ public void testMiniFarequoteWithDatafeeder() throws Exception { Request createAirlineDataRequest = new Request("PUT", "/airline-data"); createAirlineDataRequest.setJsonEntity("{" + " \"mappings\": {" - + " \"_doc\": {" - + " \"properties\": {" - + " \"time\": { \"type\":\"date\"}," - + " \"airline\": { \"type\":\"keyword\"}," - + " \"responsetime\": { \"type\":\"float\"}" - + " }" + + " \"properties\": {" + + " \"time\": { \"type\":\"date\"}," + + " \"airline\": { \"type\":\"keyword\"}," + + " \"responsetime\": { \"type\":\"float\"}" + " }" + " }" + "}"); diff --git a/x-pack/plugin/ml/qa/native-multi-node-tests/src/test/java/org/elasticsearch/xpack/ml/integration/DatafeedJobsRestIT.java b/x-pack/plugin/ml/qa/native-multi-node-tests/src/test/java/org/elasticsearch/xpack/ml/integration/DatafeedJobsRestIT.java index 64d8f73c4a3e5..2e69702381bcf 100644 --- a/x-pack/plugin/ml/qa/native-multi-node-tests/src/test/java/org/elasticsearch/xpack/ml/integration/DatafeedJobsRestIT.java +++ b/x-pack/plugin/ml/qa/native-multi-node-tests/src/test/java/org/elasticsearch/xpack/ml/integration/DatafeedJobsRestIT.java @@ -13,7 +13,6 @@ import org.elasticsearch.client.RestClient; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.util.concurrent.ThreadContext; -import org.elasticsearch.rest.action.document.RestBulkAction; import org.elasticsearch.test.SecuritySettingsSourceField; import org.elasticsearch.test.rest.ESRestTestCase; import org.elasticsearch.xpack.core.ml.integration.MlRestTestStateCleaner; @@ -105,12 +104,10 @@ private void addAirlineData() throws IOException { Request createEmptyAirlineDataRequest = new Request("PUT", "/airline-data-empty"); createEmptyAirlineDataRequest.setJsonEntity("{" + " \"mappings\": {" - + " \"response\": {" - + " \"properties\": {" - + " \"time stamp\": { \"type\":\"date\"}," // space in 'time stamp' is intentional - + " \"airline\": { \"type\":\"keyword\"}," - + " \"responsetime\": { \"type\":\"float\"}" - + " }" + + " \"properties\": {" + + " \"time stamp\": { \"type\":\"date\"}," // space in 'time stamp' is intentional + + " \"airline\": { \"type\":\"keyword\"}," + + " \"responsetime\": { \"type\":\"float\"}" + " }" + " }" + "}"); @@ -120,116 +117,106 @@ private void addAirlineData() throws IOException { Request createAirlineDataRequest = new Request("PUT", "/airline-data"); createAirlineDataRequest.setJsonEntity("{" + " \"mappings\": {" - + " \"response\": {" - + " \"properties\": {" - + " \"time stamp\": { \"type\":\"date\"}," // space in 'time stamp' is intentional - + " \"airline\": {" - + " \"type\":\"text\"," - + " \"fields\":{" - + " \"text\":{\"type\":\"text\"}," - + " \"keyword\":{\"type\":\"keyword\"}" - + " }" - + " }," - + " \"responsetime\": { \"type\":\"float\"}" - + " }" + + " \"properties\": {" + + " \"time stamp\": { \"type\":\"date\"}," // space in 'time stamp' is intentional + + " \"airline\": {" + + " \"type\":\"text\"," + + " \"fields\":{" + + " \"text\":{\"type\":\"text\"}," + + " \"keyword\":{\"type\":\"keyword\"}" + + " }" + + " }," + + " \"responsetime\": { \"type\":\"float\"}" + " }" + " }" + "}"); client().performRequest(createAirlineDataRequest); - bulk.append("{\"index\": {\"_index\": \"airline-data\", \"_type\": \"response\", \"_id\": 1}}\n"); + bulk.append("{\"index\": {\"_index\": \"airline-data\", \"_id\": 1}}\n"); bulk.append("{\"time stamp\":\"2016-06-01T00:00:00Z\",\"airline\":\"AAA\",\"responsetime\":135.22}\n"); - bulk.append("{\"index\": {\"_index\": \"airline-data\", \"_type\": \"response\", \"_id\": 2}}\n"); + bulk.append("{\"index\": {\"_index\": \"airline-data\", \"_id\": 2}}\n"); bulk.append("{\"time stamp\":\"2016-06-01T01:59:00Z\",\"airline\":\"AAA\",\"responsetime\":541.76}\n"); // Create index with source = enabled, doc_values = disabled (except time), stored = false Request createAirlineDataDisabledDocValues = new Request("PUT", "/airline-data-disabled-doc-values"); createAirlineDataDisabledDocValues.setJsonEntity("{" + " \"mappings\": {" - + " \"response\": {" - + " \"properties\": {" - + " \"time stamp\": { \"type\":\"date\"}," - + " \"airline\": { \"type\":\"keyword\", \"doc_values\":false}," - + " \"responsetime\": { \"type\":\"float\", \"doc_values\":false}" - + " }" + + " \"properties\": {" + + " \"time stamp\": { \"type\":\"date\"}," + + " \"airline\": { \"type\":\"keyword\", \"doc_values\":false}," + + " \"responsetime\": { \"type\":\"float\", \"doc_values\":false}" + " }" + " }" + "}"); client().performRequest(createAirlineDataDisabledDocValues); - bulk.append("{\"index\": {\"_index\": \"airline-data-disabled-doc-values\", \"_type\": \"response\", \"_id\": 1}}\n"); + bulk.append("{\"index\": {\"_index\": \"airline-data-disabled-doc-values\", \"_id\": 1}}\n"); bulk.append("{\"time stamp\":\"2016-06-01T00:00:00Z\",\"airline\":\"AAA\",\"responsetime\":135.22}\n"); - bulk.append("{\"index\": {\"_index\": \"airline-data-disabled-doc-values\", \"_type\": \"response\", \"_id\": 2}}\n"); + bulk.append("{\"index\": {\"_index\": \"airline-data-disabled-doc-values\", \"_id\": 2}}\n"); bulk.append("{\"time stamp\":\"2016-06-01T01:59:00Z\",\"airline\":\"AAA\",\"responsetime\":541.76}\n"); // Create index with source = disabled, doc_values = enabled (except time), stored = true Request createAirlineDataDisabledSource = new Request("PUT", "/airline-data-disabled-source"); createAirlineDataDisabledSource.setJsonEntity("{" + " \"mappings\": {" - + " \"response\": {" - + " \"_source\":{\"enabled\":false}," - + " \"properties\": {" - + " \"time stamp\": { \"type\":\"date\", \"store\":true}," - + " \"airline\": { \"type\":\"keyword\", \"store\":true}," - + " \"responsetime\": { \"type\":\"float\", \"store\":true}" - + " }" + + " \"_source\":{\"enabled\":false}," + + " \"properties\": {" + + " \"time stamp\": { \"type\":\"date\", \"store\":true}," + + " \"airline\": { \"type\":\"keyword\", \"store\":true}," + + " \"responsetime\": { \"type\":\"float\", \"store\":true}" + " }" + " }" + "}"); - bulk.append("{\"index\": {\"_index\": \"airline-data-disabled-source\", \"_type\": \"response\", \"_id\": 1}}\n"); + bulk.append("{\"index\": {\"_index\": \"airline-data-disabled-source\", \"_id\": 1}}\n"); bulk.append("{\"time stamp\":\"2016-06-01T00:00:00Z\",\"airline\":\"AAA\",\"responsetime\":135.22}\n"); - bulk.append("{\"index\": {\"_index\": \"airline-data-disabled-source\", \"_type\": \"response\", \"_id\": 2}}\n"); + bulk.append("{\"index\": {\"_index\": \"airline-data-disabled-source\", \"_id\": 2}}\n"); bulk.append("{\"time stamp\":\"2016-06-01T01:59:00Z\",\"airline\":\"AAA\",\"responsetime\":541.76}\n"); // Create index with nested documents Request createAirlineDataNested = new Request("PUT", "/nested-data"); createAirlineDataNested.setJsonEntity("{" + " \"mappings\": {" - + " \"response\": {" - + " \"properties\": {" - + " \"time\": { \"type\":\"date\"}" - + " }" + + " \"properties\": {" + + " \"time\": { \"type\":\"date\"}" + " }" + " }" + "}"); client().performRequest(createAirlineDataNested); - bulk.append("{\"index\": {\"_index\": \"nested-data\", \"_type\": \"response\", \"_id\": 1}}\n"); + bulk.append("{\"index\": {\"_index\": \"nested-data\", \"_id\": 1}}\n"); bulk.append("{\"time\":\"2016-06-01T00:00:00Z\", \"responsetime\":{\"millis\":135.22}}\n"); - bulk.append("{\"index\": {\"_index\": \"nested-data\", \"_type\": \"response\", \"_id\": 2}}\n"); + bulk.append("{\"index\": {\"_index\": \"nested-data\", \"_id\": 2}}\n"); bulk.append("{\"time\":\"2016-06-01T01:59:00Z\",\"responsetime\":{\"millis\":222.0}}\n"); // Create index with multiple docs per time interval for aggregation testing Request createAirlineDataAggs = new Request("PUT", "/airline-data-aggs"); createAirlineDataAggs.setJsonEntity("{" + " \"mappings\": {" - + " \"response\": {" - + " \"properties\": {" - + " \"time stamp\": { \"type\":\"date\"}," // space in 'time stamp' is intentional - + " \"airline\": { \"type\":\"keyword\"}," - + " \"responsetime\": { \"type\":\"float\"}" - + " }" + + " \"properties\": {" + + " \"time stamp\": { \"type\":\"date\"}," // space in 'time stamp' is intentional + + " \"airline\": { \"type\":\"keyword\"}," + + " \"responsetime\": { \"type\":\"float\"}" + " }" + " }" + "}"); client().performRequest(createAirlineDataAggs); - bulk.append("{\"index\": {\"_index\": \"airline-data-aggs\", \"_type\": \"response\", \"_id\": 1}}\n"); + bulk.append("{\"index\": {\"_index\": \"airline-data-aggs\", \"_id\": 1}}\n"); bulk.append("{\"time stamp\":\"2016-06-01T00:00:00Z\",\"airline\":\"AAA\",\"responsetime\":100.0}\n"); - bulk.append("{\"index\": {\"_index\": \"airline-data-aggs\", \"_type\": \"response\", \"_id\": 2}}\n"); + bulk.append("{\"index\": {\"_index\": \"airline-data-aggs\", \"_id\": 2}}\n"); bulk.append("{\"time stamp\":\"2016-06-01T00:01:00Z\",\"airline\":\"AAA\",\"responsetime\":200.0}\n"); - bulk.append("{\"index\": {\"_index\": \"airline-data-aggs\", \"_type\": \"response\", \"_id\": 3}}\n"); + bulk.append("{\"index\": {\"_index\": \"airline-data-aggs\", \"_id\": 3}}\n"); bulk.append("{\"time stamp\":\"2016-06-01T00:00:00Z\",\"airline\":\"BBB\",\"responsetime\":1000.0}\n"); - bulk.append("{\"index\": {\"_index\": \"airline-data-aggs\", \"_type\": \"response\", \"_id\": 4}}\n"); + bulk.append("{\"index\": {\"_index\": \"airline-data-aggs\", \"_id\": 4}}\n"); bulk.append("{\"time stamp\":\"2016-06-01T00:01:00Z\",\"airline\":\"BBB\",\"responsetime\":2000.0}\n"); - bulk.append("{\"index\": {\"_index\": \"airline-data-aggs\", \"_type\": \"response\", \"_id\": 5}}\n"); + bulk.append("{\"index\": {\"_index\": \"airline-data-aggs\", \"_id\": 5}}\n"); bulk.append("{\"time stamp\":\"2016-06-01T01:00:00Z\",\"airline\":\"AAA\",\"responsetime\":300.0}\n"); - bulk.append("{\"index\": {\"_index\": \"airline-data-aggs\", \"_type\": \"response\", \"_id\": 6}}\n"); + bulk.append("{\"index\": {\"_index\": \"airline-data-aggs\", \"_id\": 6}}\n"); bulk.append("{\"time stamp\":\"2016-06-01T01:01:00Z\",\"airline\":\"AAA\",\"responsetime\":400.0}\n"); - bulk.append("{\"index\": {\"_index\": \"airline-data-aggs\", \"_type\": \"response\", \"_id\": 7}}\n"); + bulk.append("{\"index\": {\"_index\": \"airline-data-aggs\", \"_id\": 7}}\n"); bulk.append("{\"time stamp\":\"2016-06-01T01:00:00Z\",\"airline\":\"BBB\",\"responsetime\":3000.0}\n"); - bulk.append("{\"index\": {\"_index\": \"airline-data-aggs\", \"_type\": \"response\", \"_id\": 8}}\n"); + bulk.append("{\"index\": {\"_index\": \"airline-data-aggs\", \"_id\": 8}}\n"); bulk.append("{\"time stamp\":\"2016-06-01T01:01:00Z\",\"airline\":\"BBB\",\"responsetime\":4000.0}\n"); bulkIndex(bulk.toString()); @@ -240,18 +227,16 @@ private void addNetworkData(String index) throws IOException { Request createIndexRequest = new Request("PUT", index); createIndexRequest.setJsonEntity("{" + " \"mappings\": {" - + " \"doc\": {" - + " \"properties\": {" - + " \"timestamp\": { \"type\":\"date\"}," - + " \"host\": {" - + " \"type\":\"text\"," - + " \"fields\":{" - + " \"text\":{\"type\":\"text\"}," - + " \"keyword\":{\"type\":\"keyword\"}" - + " }" - + " }," - + " \"network_bytes_out\": { \"type\":\"long\"}" - + " }" + + " \"properties\": {" + + " \"timestamp\": { \"type\":\"date\"}," + + " \"host\": {" + + " \"type\":\"text\"," + + " \"fields\":{" + + " \"text\":{\"type\":\"text\"}," + + " \"keyword\":{\"type\":\"keyword\"}" + + " }" + + " }," + + " \"network_bytes_out\": { \"type\":\"long\"}" + " }" + " }" + "}");; @@ -262,11 +247,11 @@ private void addNetworkData(String index) throws IOException { Date date = new Date(1464739200735L); for (int i = 0; i < 120; i++) { long byteCount = randomNonNegativeLong(); - bulk.append("{\"index\": {\"_index\": \"").append(index).append("\", \"_type\": \"doc\"}}\n"); + bulk.append("{\"index\": {\"_index\": \"").append(index).append("\"}}\n"); bulk.append(String.format(Locale.ROOT, docTemplate, date.getTime(), "hostA", byteCount)).append('\n'); byteCount = randomNonNegativeLong(); - bulk.append("{\"index\": {\"_index\": \"").append(index).append("\", \"_type\": \"doc\"}}\n"); + bulk.append("{\"index\": {\"_index\": \"").append(index).append("\"}}\n"); bulk.append(String.format(Locale.ROOT, docTemplate, date.getTime(), "hostB", byteCount)).append('\n'); date = new Date(date.getTime() + 10_000); @@ -1197,7 +1182,6 @@ private void bulkIndex(String bulk) throws IOException { bulkRequest.setJsonEntity(bulk); bulkRequest.addParameter("refresh", "true"); bulkRequest.addParameter("pretty", null); - bulkRequest.setOptions(expectWarnings(RestBulkAction.TYPES_DEPRECATION_MESSAGE)); String bulkResponse = EntityUtils.toString(client().performRequest(bulkRequest).getEntity()); assertThat(bulkResponse, not(containsString("\"errors\": false"))); } diff --git a/x-pack/plugin/ml/qa/single-node-tests/src/test/java/org/elasticsearch/xpack/ml/transforms/PainlessDomainSplitIT.java b/x-pack/plugin/ml/qa/single-node-tests/src/test/java/org/elasticsearch/xpack/ml/transforms/PainlessDomainSplitIT.java index c895ab2d4f5d1..1453f59ed43e4 100644 --- a/x-pack/plugin/ml/qa/single-node-tests/src/test/java/org/elasticsearch/xpack/ml/transforms/PainlessDomainSplitIT.java +++ b/x-pack/plugin/ml/qa/single-node-tests/src/test/java/org/elasticsearch/xpack/ml/transforms/PainlessDomainSplitIT.java @@ -262,8 +262,8 @@ public void testHRDSplit() throws Exception { .put(IndexMetaData.INDEX_NUMBER_OF_SHARDS_SETTING.getKey(), 1) .put(IndexMetaData.INDEX_NUMBER_OF_REPLICAS_SETTING.getKey(), 0); - createIndex("painless", settings.build(), "\"_doc\": { \"properties\": { \"domain\": { \"type\": \"keyword\" }," + - "\"time\": { \"type\": \"date\" } } }"); + createIndex("painless", settings.build(), "\"properties\": { \"domain\": { \"type\": \"keyword\" }," + + "\"time\": { \"type\": \"date\" } }"); // Index some data DateTime baseTime = new DateTime().minusYears(1); diff --git a/x-pack/plugin/monitoring/src/main/java/org/elasticsearch/xpack/monitoring/exporter/http/TemplateHttpResource.java b/x-pack/plugin/monitoring/src/main/java/org/elasticsearch/xpack/monitoring/exporter/http/TemplateHttpResource.java index 5e6ed790b1a54..28bca87b0a096 100644 --- a/x-pack/plugin/monitoring/src/main/java/org/elasticsearch/xpack/monitoring/exporter/http/TemplateHttpResource.java +++ b/x-pack/plugin/monitoring/src/main/java/org/elasticsearch/xpack/monitoring/exporter/http/TemplateHttpResource.java @@ -17,9 +17,14 @@ import org.elasticsearch.common.xcontent.XContentType; import org.elasticsearch.xpack.core.monitoring.exporter.MonitoringTemplateUtils; +import java.util.Collections; +import java.util.Map; import java.util.Objects; +import java.util.TreeMap; import java.util.function.Supplier; +import static org.elasticsearch.rest.BaseRestHandler.INCLUDE_TYPE_NAME_PARAMETER; + /** * {@code TemplateHttpResource}s allow the checking and uploading of templates to a remote cluster. *

@@ -31,6 +36,15 @@ public class TemplateHttpResource extends PublishableHttpResource { private static final Logger logger = LogManager.getLogger(TemplateHttpResource.class); + public static final Map PARAMETERS; + + static { + Map parameters = new TreeMap<>(); + parameters.put("filter_path", FILTER_PATH_RESOURCE_VERSION); + parameters.put(INCLUDE_TYPE_NAME_PARAMETER, "true"); + PARAMETERS = Collections.unmodifiableMap(parameters); + } + /** * The name of the template that is sent to the remote cluster. */ @@ -50,7 +64,7 @@ public class TemplateHttpResource extends PublishableHttpResource { */ public TemplateHttpResource(final String resourceOwnerName, @Nullable final TimeValue masterTimeout, final String templateName, final Supplier template) { - super(resourceOwnerName, masterTimeout, PublishableHttpResource.RESOURCE_VERSION_PARAMETERS); + super(resourceOwnerName, masterTimeout, PARAMETERS); this.templateName = Objects.requireNonNull(templateName); this.template = Objects.requireNonNull(template); diff --git a/x-pack/plugin/monitoring/src/test/java/org/elasticsearch/xpack/monitoring/exporter/http/AbstractPublishableHttpResourceTestCase.java b/x-pack/plugin/monitoring/src/test/java/org/elasticsearch/xpack/monitoring/exporter/http/AbstractPublishableHttpResourceTestCase.java index 39f6c573fc403..798053d59be37 100644 --- a/x-pack/plugin/monitoring/src/test/java/org/elasticsearch/xpack/monitoring/exporter/http/AbstractPublishableHttpResourceTestCase.java +++ b/x-pack/plugin/monitoring/src/test/java/org/elasticsearch/xpack/monitoring/exporter/http/AbstractPublishableHttpResourceTestCase.java @@ -30,6 +30,7 @@ import java.util.function.Predicate; import java.util.stream.Collectors; +import static org.elasticsearch.rest.BaseRestHandler.INCLUDE_TYPE_NAME_PARAMETER; import static org.elasticsearch.xpack.monitoring.exporter.http.AsyncHttpResourceHelper.mockBooleanActionListener; import static org.elasticsearch.xpack.monitoring.exporter.http.AsyncHttpResourceHelper.whenPerformRequestAsyncWith; import static org.elasticsearch.xpack.monitoring.exporter.http.PublishableHttpResource.GET_DOES_NOT_EXIST; @@ -212,6 +213,11 @@ protected void assertVersionParameters(final PublishableHttpResource resource) { } assertThat(parameters.remove("filter_path"), is("*.version")); + + if (parameters.containsKey(INCLUDE_TYPE_NAME_PARAMETER)) { + assertThat(parameters.remove(INCLUDE_TYPE_NAME_PARAMETER), is("true")); + } + assertThat(parameters.isEmpty(), is(true)); } diff --git a/x-pack/plugin/monitoring/src/test/java/org/elasticsearch/xpack/monitoring/exporter/http/HttpExporterIT.java b/x-pack/plugin/monitoring/src/test/java/org/elasticsearch/xpack/monitoring/exporter/http/HttpExporterIT.java index 1f9ab90e86542..a6cddcf35d370 100644 --- a/x-pack/plugin/monitoring/src/test/java/org/elasticsearch/xpack/monitoring/exporter/http/HttpExporterIT.java +++ b/x-pack/plugin/monitoring/src/test/java/org/elasticsearch/xpack/monitoring/exporter/http/HttpExporterIT.java @@ -57,6 +57,7 @@ import java.util.stream.Collectors; import static org.elasticsearch.common.xcontent.XContentFactory.jsonBuilder; +import static org.elasticsearch.rest.BaseRestHandler.INCLUDE_TYPE_NAME_PARAMETER; import static org.elasticsearch.xpack.core.monitoring.exporter.MonitoringTemplateUtils.LAST_UPDATED_VERSION; import static org.elasticsearch.xpack.core.monitoring.exporter.MonitoringTemplateUtils.TEMPLATE_VERSION; import static org.elasticsearch.xpack.core.monitoring.exporter.MonitoringTemplateUtils.indexName; @@ -273,17 +274,18 @@ public void testHostChangeReChecksTemplate() throws Exception { assertMonitorVersion(secondWebServer); + String resourcePrefix = "/_template/"; for (Tuple template : monitoringTemplates(includeOldTemplates)) { MockRequest recordedRequest = secondWebServer.takeRequest(); assertThat(recordedRequest.getMethod(), equalTo("GET")); - assertThat(recordedRequest.getUri().getPath(), equalTo("/_template/" + template.v1())); - assertThat(recordedRequest.getUri().getQuery(), equalTo(resourceVersionQueryString())); + assertThat(recordedRequest.getUri().getPath(), equalTo(resourcePrefix + template.v1())); + assertMonitorVersionQueryString(resourcePrefix, recordedRequest.getUri().getQuery()); if (missingTemplate.equals(template.v1())) { recordedRequest = secondWebServer.takeRequest(); assertThat(recordedRequest.getMethod(), equalTo("PUT")); - assertThat(recordedRequest.getUri().getPath(), equalTo("/_template/" + template.v1())); - assertThat(recordedRequest.getUri().getQuery(), equalTo(resourceVersionQueryString())); + assertThat(recordedRequest.getUri().getPath(), equalTo(resourcePrefix + template.v1())); + assertMonitorVersionQueryString(resourcePrefix, recordedRequest.getUri().getQuery()); assertThat(recordedRequest.getBody(), equalTo(template.v2())); } } @@ -457,7 +459,7 @@ private void assertMonitorVersionResource(final MockWebServer webServer, final b assertThat(getRequest.getMethod(), equalTo("GET")); assertThat(getRequest.getUri().getPath(), equalTo(pathPrefix + resourcePrefix + resource.v1())); - assertThat(getRequest.getUri().getQuery(), equalTo(resourceVersionQueryString())); + assertMonitorVersionQueryString(resourcePrefix, getRequest.getUri().getQuery()); assertHeaders(getRequest, customHeaders); if (alreadyExists == false) { @@ -465,13 +467,21 @@ private void assertMonitorVersionResource(final MockWebServer webServer, final b assertThat(putRequest.getMethod(), equalTo("PUT")); assertThat(putRequest.getUri().getPath(), equalTo(pathPrefix + resourcePrefix + resource.v1())); - assertThat(putRequest.getUri().getQuery(), equalTo(resourceVersionQueryString())); + assertMonitorVersionQueryString(resourcePrefix, getRequest.getUri().getQuery()); assertThat(putRequest.getBody(), equalTo(resource.v2())); assertHeaders(putRequest, customHeaders); } } } + private void assertMonitorVersionQueryString(String resourcePrefix, String query) { + if (resourcePrefix.startsWith("/_template")) { + assertThat(query, equalTo(INCLUDE_TYPE_NAME_PARAMETER + "=true&" + resourceVersionQueryString())); + } else { + assertThat(query, equalTo(resourceVersionQueryString())); + } + } + private void assertMonitorWatches(final MockWebServer webServer, final boolean remoteClusterAllowsWatcher, final boolean currentLicenseAllowsWatcher, final boolean alreadyExists, diff --git a/x-pack/plugin/security/src/test/java/org/elasticsearch/integration/IndexPrivilegeTests.java b/x-pack/plugin/security/src/test/java/org/elasticsearch/integration/IndexPrivilegeTests.java index 8b96a780d1a8c..7305b2f1902cf 100644 --- a/x-pack/plugin/security/src/test/java/org/elasticsearch/integration/IndexPrivilegeTests.java +++ b/x-pack/plugin/security/src/test/java/org/elasticsearch/integration/IndexPrivilegeTests.java @@ -436,7 +436,7 @@ private void assertUserExecutes(String user, String action, String index, boolea // indexing a document to have the mapping available, and wait for green state to make sure index is created assertAccessIsAllowed("admin", "PUT", "/" + index + "/foo/1", jsonDoc); assertNoTimeout(client().admin().cluster().prepareHealth(index).setWaitForGreenStatus().get()); - assertAccessIsAllowed(user, "GET", "/" + index + "/_mapping/foo/field/name"); + assertAccessIsAllowed(user, "GET", "/" + index + "/_mapping/field/name"); assertAccessIsAllowed(user, "GET", "/" + index + "/_settings"); } else { assertAccessIsDenied(user, "DELETE", "/" + index); @@ -449,7 +449,7 @@ private void assertUserExecutes(String user, String action, String index, boolea assertAccessIsDenied(user, "POST", "/" + index + "/_close"); assertAccessIsDenied(user, "POST", "/" + index + "/_open"); assertAccessIsDenied(user, "POST", "/" + index + "/_cache/clear"); - assertAccessIsDenied(user, "GET", "/" + index + "/_mapping/foo/field/name"); + assertAccessIsDenied(user, "GET", "/" + index + "/_mapping/field/name"); assertAccessIsDenied(user, "GET", "/" + index + "/_settings"); } break; diff --git a/x-pack/plugin/sql/qa/single-node/src/test/java/org/elasticsearch/xpack/sql/qa/single_node/JdbcShardFailureIT.java b/x-pack/plugin/sql/qa/single-node/src/test/java/org/elasticsearch/xpack/sql/qa/single_node/JdbcShardFailureIT.java index f7c490cc8b490..19e16d1de2edb 100644 --- a/x-pack/plugin/sql/qa/single-node/src/test/java/org/elasticsearch/xpack/sql/qa/single_node/JdbcShardFailureIT.java +++ b/x-pack/plugin/sql/qa/single-node/src/test/java/org/elasticsearch/xpack/sql/qa/single_node/JdbcShardFailureIT.java @@ -20,12 +20,12 @@ public class JdbcShardFailureIT extends JdbcIntegrationTestCase { @Before public void createTestIndex() throws IOException { Request createTest1 = new Request("PUT", "/test1"); - String body1 = "{\"aliases\":{\"test\":{}}, \"mappings\": {\"doc\": {\"properties\": {\"test_field\":{\"type\":\"integer\"}}}}}"; + String body1 = "{\"aliases\":{\"test\":{}}, \"mappings\": {\"properties\": {\"test_field\":{\"type\":\"integer\"}}}}"; createTest1.setJsonEntity(body1); client().performRequest(createTest1); Request createTest2 = new Request("PUT", "/test2"); - String body2 = "{\"aliases\":{\"test\":{}}, \"mappings\": {\"doc\": {\"properties\": {\"test_field\":{\"type\":\"integer\"}}}}," + + String body2 = "{\"aliases\":{\"test\":{}}, \"mappings\": {\"properties\": {\"test_field\":{\"type\":\"integer\"}}}," + "\"settings\": {\"index.routing.allocation.include.node\": \"nowhere\"}}"; createTest2.setJsonEntity(body2); createTest2.addParameter("timeout", "100ms"); diff --git a/x-pack/plugin/sql/qa/src/main/java/org/elasticsearch/xpack/sql/qa/jdbc/ResultSetTestCase.java b/x-pack/plugin/sql/qa/src/main/java/org/elasticsearch/xpack/sql/qa/jdbc/ResultSetTestCase.java index 3bb710c998290..3d5308d148d57 100644 --- a/x-pack/plugin/sql/qa/src/main/java/org/elasticsearch/xpack/sql/qa/jdbc/ResultSetTestCase.java +++ b/x-pack/plugin/sql/qa/src/main/java/org/elasticsearch/xpack/sql/qa/jdbc/ResultSetTestCase.java @@ -1273,12 +1273,8 @@ private void createIndex(String index) throws Exception { createIndex.endObject(); createIndex.startObject("mappings"); { - createIndex.startObject("_doc"); - { - createIndex.startObject("properties"); - {} - createIndex.endObject(); - } + createIndex.startObject("properties"); + {} createIndex.endObject(); } createIndex.endObject().endObject(); @@ -1287,7 +1283,7 @@ private void createIndex(String index) throws Exception { } private void updateMapping(String index, CheckedConsumer body) throws Exception { - Request request = new Request("PUT", "/" + index + "/_mapping/_doc"); + Request request = new Request("PUT", "/" + index + "/_mapping"); XContentBuilder updateMapping = JsonXContent.contentBuilder().startObject(); updateMapping.startObject("properties"); { diff --git a/x-pack/plugin/src/test/resources/rest-api-spec/test/ml/ml_anomalies_default_mappings.yml b/x-pack/plugin/src/test/resources/rest-api-spec/test/ml/ml_anomalies_default_mappings.yml index b230e12589f53..91ba0ae17ce0d 100644 --- a/x-pack/plugin/src/test/resources/rest-api-spec/test/ml/ml_anomalies_default_mappings.yml +++ b/x-pack/plugin/src/test/resources/rest-api-spec/test/ml/ml_anomalies_default_mappings.yml @@ -42,10 +42,10 @@ setup: headers: Authorization: "Basic eF9wYWNrX3Jlc3RfdXNlcjp4LXBhY2stdGVzdC1wYXNzd29yZA==" # run as x_pack_rest_user, i.e. the test setup superuser indices.get_field_mapping: + include_type_name: false index: .ml-anomalies-shared - type: doc fields: new_field - - match: {\.ml-anomalies-shared.mappings.doc.new_field.mapping.new_field.type: keyword} + - match: {\.ml-anomalies-shared.mappings.new_field.mapping.new_field.type: keyword} --- "Test _meta exists when two jobs share an index": @@ -75,8 +75,9 @@ setup: headers: Authorization: "Basic eF9wYWNrX3Jlc3RfdXNlcjp4LXBhY2stdGVzdC1wYXNzd29yZA==" # run as x_pack_rest_user, i.e. the test setup superuser indices.get_mapping: + include_type_name: false index: .ml-anomalies-shared - - is_true: \.ml-anomalies-shared.mappings.doc._meta.version + - is_true: \.ml-anomalies-shared.mappings._meta.version - do: ml.put_job: @@ -103,5 +104,6 @@ setup: headers: Authorization: "Basic eF9wYWNrX3Jlc3RfdXNlcjp4LXBhY2stdGVzdC1wYXNzd29yZA==" # run as x_pack_rest_user, i.e. the test setup superuser indices.get_mapping: + include_type_name: false index: .ml-anomalies-shared - - is_true: \.ml-anomalies-shared.mappings.doc._meta.version + - is_true: \.ml-anomalies-shared.mappings._meta.version From 397f315f5691c6f57a4cf3a2199315eb693a819d Mon Sep 17 00:00:00 2001 From: Nhat Nguyen Date: Mon, 14 Jan 2019 18:20:54 -0500 Subject: [PATCH 155/186] Make finalize step of recovery source non-blocking (#37388) Relates #37291 --- .../elasticsearch/action/ActionListener.java | 13 ++++ .../action/ActionListenerResponseHandler.java | 10 ++- .../recovery/PeerRecoveryTargetService.java | 9 +-- .../recovery/RecoverySourceHandler.java | 68 +++++++++++-------- .../indices/recovery/RecoveryTarget.java | 15 ++-- .../recovery/RecoveryTargetHandler.java | 3 +- .../recovery/RemoteRecoveryTargetHandler.java | 6 +- .../action/ActionListenerTests.java | 16 +++++ .../RecoveryDuringReplicationTests.java | 5 +- .../index/shard/IndexShardTests.java | 5 +- .../recovery/RecoverySourceHandlerTests.java | 2 +- 11 files changed, 103 insertions(+), 49 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/action/ActionListener.java b/server/src/main/java/org/elasticsearch/action/ActionListener.java index 07bc519346f7c..34d90035ad6cc 100644 --- a/server/src/main/java/org/elasticsearch/action/ActionListener.java +++ b/server/src/main/java/org/elasticsearch/action/ActionListener.java @@ -21,6 +21,7 @@ import org.elasticsearch.ExceptionsHelper; import org.elasticsearch.common.CheckedConsumer; +import org.elasticsearch.common.CheckedSupplier; import java.util.ArrayList; import java.util.List; @@ -180,4 +181,16 @@ protected void innerOnFailure(Exception e) { } }; } + + /** + * Completes the given listener with the result from the provided supplier accordingly. + * This method is mainly used to complete a listener with a block of synchronous code. + */ + static void completeWith(ActionListener listener, CheckedSupplier supplier) { + try { + listener.onResponse(supplier.get()); + } catch (Exception e) { + listener.onFailure(e); + } + } } diff --git a/server/src/main/java/org/elasticsearch/action/ActionListenerResponseHandler.java b/server/src/main/java/org/elasticsearch/action/ActionListenerResponseHandler.java index 432cef6ad3029..19a0618e1c5a4 100644 --- a/server/src/main/java/org/elasticsearch/action/ActionListenerResponseHandler.java +++ b/server/src/main/java/org/elasticsearch/action/ActionListenerResponseHandler.java @@ -37,10 +37,16 @@ public class ActionListenerResponseHandler i private final ActionListener listener; private final Writeable.Reader reader; + private final String executor; - public ActionListenerResponseHandler(ActionListener listener, Writeable.Reader reader) { + public ActionListenerResponseHandler(ActionListener listener, Writeable.Reader reader, String executor) { this.listener = Objects.requireNonNull(listener); this.reader = Objects.requireNonNull(reader); + this.executor = Objects.requireNonNull(executor); + } + + public ActionListenerResponseHandler(ActionListener listener, Writeable.Reader reader) { + this(listener, reader, ThreadPool.Names.SAME); } @Override @@ -55,7 +61,7 @@ public void handleException(TransportException e) { @Override public String executor() { - return ThreadPool.Names.SAME; + return executor; } @Override diff --git a/server/src/main/java/org/elasticsearch/indices/recovery/PeerRecoveryTargetService.java b/server/src/main/java/org/elasticsearch/indices/recovery/PeerRecoveryTargetService.java index 8edaf0ef093ab..8579caecc8add 100644 --- a/server/src/main/java/org/elasticsearch/indices/recovery/PeerRecoveryTargetService.java +++ b/server/src/main/java/org/elasticsearch/indices/recovery/PeerRecoveryTargetService.java @@ -445,11 +445,12 @@ class FinalizeRecoveryRequestHandler implements TransportRequestHandler listener = + new HandledTransportAction.ChannelActionListener<>(channel, Actions.FINALIZE, request); + recoveryRef.target().finalizeRecovery(request.globalCheckpoint(), + ActionListener.wrap(nullVal -> listener.onResponse(TransportResponse.Empty.INSTANCE), listener::onFailure)); } - channel.sendResponse(TransportResponse.Empty.INSTANCE); } } diff --git a/server/src/main/java/org/elasticsearch/indices/recovery/RecoverySourceHandler.java b/server/src/main/java/org/elasticsearch/indices/recovery/RecoverySourceHandler.java index bdda5e8d8d4f8..b1585251934dc 100644 --- a/server/src/main/java/org/elasticsearch/indices/recovery/RecoverySourceHandler.java +++ b/server/src/main/java/org/elasticsearch/indices/recovery/RecoverySourceHandler.java @@ -32,6 +32,7 @@ import org.elasticsearch.ExceptionsHelper; import org.elasticsearch.Version; import org.elasticsearch.action.ActionListener; +import org.elasticsearch.action.StepListener; import org.elasticsearch.cluster.routing.IndexShardRoutingTable; import org.elasticsearch.cluster.routing.ShardRouting; import org.elasticsearch.common.StopWatch; @@ -71,6 +72,7 @@ import java.util.concurrent.CopyOnWriteArrayList; import java.util.concurrent.atomic.AtomicLong; import java.util.concurrent.atomic.AtomicReference; +import java.util.function.Consumer; import java.util.function.Supplier; import java.util.stream.StreamSupport; @@ -137,6 +139,9 @@ public void recoverToTarget(ActionListener listener) { IOUtils.closeWhileHandlingException(releaseResources, () -> wrappedListener.onFailure(e)); throw e; }); + final Consumer onFailure = e -> + IOUtils.closeWhileHandlingException(releaseResources, () -> wrappedListener.onFailure(e)); + runUnderPrimaryPermit(() -> { final IndexShardRoutingTable routingTable = shard.getReplicationGroup().getRoutingTable(); ShardRouting targetShardRouting = routingTable.getByAllocationId(request.targetAllocationId()); @@ -235,16 +240,21 @@ public void recoverToTarget(ActionListener listener) { throw new RecoveryEngineException(shard.shardId(), 2, "phase2 failed", e); } - finalizeRecovery(sendSnapshotResult.targetLocalCheckpoint); - final long phase1ThrottlingWaitTime = 0L; // TODO: return the actual throttle time - assert resources.isEmpty() : "not every resource is released [" + resources + "]"; - IOUtils.close(resources); - wrappedListener.onResponse( - new RecoveryResponse(sendFileResult.phase1FileNames, sendFileResult.phase1FileSizes, + final StepListener finalizeStep = new StepListener<>(); + finalizeRecovery(sendSnapshotResult.targetLocalCheckpoint, finalizeStep); + finalizeStep.whenComplete(r -> { + assert resources.isEmpty() : "not every resource is released [" + resources + "]"; + final long phase1ThrottlingWaitTime = 0L; // TODO: return the actual throttle time + final RecoveryResponse response = new RecoveryResponse(sendFileResult.phase1FileNames, sendFileResult.phase1FileSizes, sendFileResult.phase1ExistingFileNames, sendFileResult.phase1ExistingFileSizes, sendFileResult.totalSize, - sendFileResult.existingTotalSize, sendFileResult.took.millis(), phase1ThrottlingWaitTime, prepareEngineTime.millis(), - sendSnapshotResult.totalOperations, sendSnapshotResult.tookTime.millis()) - ); + sendFileResult.existingTotalSize, sendFileResult.took.millis(), phase1ThrottlingWaitTime, + prepareEngineTime.millis(), sendSnapshotResult.totalOperations, sendSnapshotResult.tookTime.millis()); + try { + wrappedListener.onResponse(response); + } finally { + IOUtils.close(resources); + } + }, onFailure); } catch (Exception e) { IOUtils.closeWhileHandlingException(releaseResources, () -> wrappedListener.onFailure(e)); } @@ -585,10 +595,7 @@ SendSnapshotResult phase2(long startingSeqNo, long requiredSeqNoRangeStart, long return new SendSnapshotResult(targetLocalCheckpoint.get(), totalSentOps, tookTime); } - /* - * finalizes the recovery process - */ - public void finalizeRecovery(final long targetLocalCheckpoint) throws IOException { + void finalizeRecovery(final long targetLocalCheckpoint, final ActionListener listener) throws IOException { if (shard.state() == IndexShardState.CLOSED) { throw new IndexShardClosedException(request.shardId()); } @@ -604,21 +611,26 @@ public void finalizeRecovery(final long targetLocalCheckpoint) throws IOExceptio runUnderPrimaryPermit(() -> shard.markAllocationIdAsInSync(request.targetAllocationId(), targetLocalCheckpoint), shardId + " marking " + request.targetAllocationId() + " as in sync", shard, cancellableThreads, logger); final long globalCheckpoint = shard.getGlobalCheckpoint(); - cancellableThreads.executeIO(() -> recoveryTarget.finalizeRecovery(globalCheckpoint)); - runUnderPrimaryPermit(() -> shard.updateGlobalCheckpointForShard(request.targetAllocationId(), globalCheckpoint), - shardId + " updating " + request.targetAllocationId() + "'s global checkpoint", shard, cancellableThreads, logger); - - if (request.isPrimaryRelocation()) { - logger.trace("performing relocation hand-off"); - // this acquires all IndexShard operation permits and will thus delay new recoveries until it is done - cancellableThreads.execute(() -> shard.relocated(recoveryTarget::handoffPrimaryContext)); - /* - * if the recovery process fails after disabling primary mode on the source shard, both relocation source and - * target are failed (see {@link IndexShard#updateRoutingEntry}). - */ - } - stopWatch.stop(); - logger.trace("finalizing recovery took [{}]", stopWatch.totalTime()); + final StepListener finalizeListener = new StepListener<>(); + cancellableThreads.executeIO(() -> recoveryTarget.finalizeRecovery(globalCheckpoint, finalizeListener)); + finalizeListener.whenComplete(r -> { + runUnderPrimaryPermit(() -> shard.updateGlobalCheckpointForShard(request.targetAllocationId(), globalCheckpoint), + shardId + " updating " + request.targetAllocationId() + "'s global checkpoint", shard, cancellableThreads, logger); + + if (request.isPrimaryRelocation()) { + logger.trace("performing relocation hand-off"); + // TODO: make relocated async + // this acquires all IndexShard operation permits and will thus delay new recoveries until it is done + cancellableThreads.execute(() -> shard.relocated(recoveryTarget::handoffPrimaryContext)); + /* + * if the recovery process fails after disabling primary mode on the source shard, both relocation source and + * target are failed (see {@link IndexShard#updateRoutingEntry}). + */ + } + stopWatch.stop(); + logger.trace("finalizing recovery took [{}]", stopWatch.totalTime()); + listener.onResponse(null); + }, listener::onFailure); } static final class SendSnapshotResult { diff --git a/server/src/main/java/org/elasticsearch/indices/recovery/RecoveryTarget.java b/server/src/main/java/org/elasticsearch/indices/recovery/RecoveryTarget.java index 54a42bcdc928a..73ad4c17594c1 100644 --- a/server/src/main/java/org/elasticsearch/indices/recovery/RecoveryTarget.java +++ b/server/src/main/java/org/elasticsearch/indices/recovery/RecoveryTarget.java @@ -372,12 +372,15 @@ public void prepareForTranslogOperations(boolean fileBasedRecovery, int totalTra } @Override - public void finalizeRecovery(final long globalCheckpoint) throws IOException { - final IndexShard indexShard = indexShard(); - indexShard.updateGlobalCheckpointOnReplica(globalCheckpoint, "finalizing recovery"); - // Persist the global checkpoint. - indexShard.sync(); - indexShard.finalizeRecovery(); + public void finalizeRecovery(final long globalCheckpoint, ActionListener listener) { + ActionListener.completeWith(listener, () -> { + final IndexShard indexShard = indexShard(); + indexShard.updateGlobalCheckpointOnReplica(globalCheckpoint, "finalizing recovery"); + // Persist the global checkpoint. + indexShard.sync(); + indexShard.finalizeRecovery(); + return null; + }); } @Override diff --git a/server/src/main/java/org/elasticsearch/indices/recovery/RecoveryTargetHandler.java b/server/src/main/java/org/elasticsearch/indices/recovery/RecoveryTargetHandler.java index c958665b04497..18e57866c68cf 100644 --- a/server/src/main/java/org/elasticsearch/indices/recovery/RecoveryTargetHandler.java +++ b/server/src/main/java/org/elasticsearch/indices/recovery/RecoveryTargetHandler.java @@ -42,8 +42,9 @@ public interface RecoveryTargetHandler { * updates the global checkpoint. * * @param globalCheckpoint the global checkpoint on the recovery source + * @param listener the listener which will be notified when this method is completed */ - void finalizeRecovery(long globalCheckpoint) throws IOException; + void finalizeRecovery(long globalCheckpoint, ActionListener listener); /** * Blockingly waits for cluster state with at least clusterStateVersion to be available diff --git a/server/src/main/java/org/elasticsearch/indices/recovery/RemoteRecoveryTargetHandler.java b/server/src/main/java/org/elasticsearch/indices/recovery/RemoteRecoveryTargetHandler.java index 560d679bbe7fe..53eb3e342face 100644 --- a/server/src/main/java/org/elasticsearch/indices/recovery/RemoteRecoveryTargetHandler.java +++ b/server/src/main/java/org/elasticsearch/indices/recovery/RemoteRecoveryTargetHandler.java @@ -30,6 +30,7 @@ import org.elasticsearch.index.store.Store; import org.elasticsearch.index.store.StoreFileMetaData; import org.elasticsearch.index.translog.Translog; +import org.elasticsearch.threadpool.ThreadPool; import org.elasticsearch.transport.EmptyTransportResponseHandler; import org.elasticsearch.transport.TransportFuture; import org.elasticsearch.transport.TransportRequestOptions; @@ -85,11 +86,12 @@ public void prepareForTranslogOperations(boolean fileBasedRecovery, int totalTra } @Override - public void finalizeRecovery(final long globalCheckpoint) { + public void finalizeRecovery(final long globalCheckpoint, final ActionListener listener) { transportService.submitRequest(targetNode, PeerRecoveryTargetService.Actions.FINALIZE, new RecoveryFinalizeRecoveryRequest(recoveryId, shardId, globalCheckpoint), TransportRequestOptions.builder().withTimeout(recoverySettings.internalActionLongTimeout()).build(), - EmptyTransportResponseHandler.INSTANCE_SAME).txGet(); + new ActionListenerResponseHandler<>(ActionListener.wrap(r -> listener.onResponse(null), listener::onFailure), + in -> TransportResponse.Empty.INSTANCE, ThreadPool.Names.GENERIC)); } @Override diff --git a/server/src/test/java/org/elasticsearch/action/ActionListenerTests.java b/server/src/test/java/org/elasticsearch/action/ActionListenerTests.java index 1d4d83457b20e..cd3735b4843e6 100644 --- a/server/src/test/java/org/elasticsearch/action/ActionListenerTests.java +++ b/server/src/test/java/org/elasticsearch/action/ActionListenerTests.java @@ -18,16 +18,20 @@ */ package org.elasticsearch.action; +import org.elasticsearch.action.support.PlainActionFuture; import org.elasticsearch.common.CheckedConsumer; import org.elasticsearch.test.ESTestCase; +import java.io.IOException; import java.util.ArrayList; import java.util.List; +import java.util.concurrent.ExecutionException; import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicReference; import static org.hamcrest.Matchers.equalTo; +import static org.hamcrest.Matchers.instanceOf; public class ActionListenerTests extends ESTestCase { @@ -201,4 +205,16 @@ public void onFailure(Exception e) { assertThat(onFailureTimes.get(), equalTo(1)); } } + + public void testCompleteWith() { + PlainActionFuture onResponseListener = new PlainActionFuture<>(); + ActionListener.completeWith(onResponseListener, () -> 100); + assertThat(onResponseListener.isDone(), equalTo(true)); + assertThat(onResponseListener.actionGet(), equalTo(100)); + + PlainActionFuture onFailureListener = new PlainActionFuture<>(); + ActionListener.completeWith(onFailureListener, () -> { throw new IOException("not found"); }); + assertThat(onFailureListener.isDone(), equalTo(true)); + assertThat(expectThrows(ExecutionException.class, onFailureListener::get).getCause(), instanceOf(IOException.class)); + } } diff --git a/server/src/test/java/org/elasticsearch/index/replication/RecoveryDuringReplicationTests.java b/server/src/test/java/org/elasticsearch/index/replication/RecoveryDuringReplicationTests.java index 6f38822092aea..0f591f6db54f2 100644 --- a/server/src/test/java/org/elasticsearch/index/replication/RecoveryDuringReplicationTests.java +++ b/server/src/test/java/org/elasticsearch/index/replication/RecoveryDuringReplicationTests.java @@ -24,6 +24,7 @@ import org.apache.lucene.index.IndexableField; import org.apache.lucene.store.AlreadyClosedException; import org.elasticsearch.Version; +import org.elasticsearch.action.ActionListener; import org.elasticsearch.action.admin.indices.flush.FlushRequest; import org.elasticsearch.action.bulk.BulkShardRequest; import org.elasticsearch.action.delete.DeleteRequest; @@ -847,13 +848,13 @@ public void cleanFiles(int totalTranslogOps, Store.MetadataSnapshot sourceMetaDa } @Override - public void finalizeRecovery(long globalCheckpoint) throws IOException { + public void finalizeRecovery(long globalCheckpoint, ActionListener listener) { if (hasBlocked() == false) { // it maybe that not ops have been transferred, block now blockIfNeeded(RecoveryState.Stage.TRANSLOG); } blockIfNeeded(RecoveryState.Stage.FINALIZE); - super.finalizeRecovery(globalCheckpoint); + super.finalizeRecovery(globalCheckpoint, listener); } } diff --git a/server/src/test/java/org/elasticsearch/index/shard/IndexShardTests.java b/server/src/test/java/org/elasticsearch/index/shard/IndexShardTests.java index 4745904a55467..5c7dd089534a2 100644 --- a/server/src/test/java/org/elasticsearch/index/shard/IndexShardTests.java +++ b/server/src/test/java/org/elasticsearch/index/shard/IndexShardTests.java @@ -2524,9 +2524,8 @@ public long indexTranslogOperations(List operations, int tot } @Override - public void finalizeRecovery(long globalCheckpoint) throws IOException { - super.finalizeRecovery(globalCheckpoint); - assertListenerCalled.accept(replica); + public void finalizeRecovery(long globalCheckpoint, ActionListener listener) { + super.finalizeRecovery(globalCheckpoint, ActionListener.runAfter(listener, () -> assertListenerCalled.accept(replica))); } }, false, true); diff --git a/server/src/test/java/org/elasticsearch/indices/recovery/RecoverySourceHandlerTests.java b/server/src/test/java/org/elasticsearch/indices/recovery/RecoverySourceHandlerTests.java index 7b9e8fe05dad7..29eb1466056e8 100644 --- a/server/src/test/java/org/elasticsearch/indices/recovery/RecoverySourceHandlerTests.java +++ b/server/src/test/java/org/elasticsearch/indices/recovery/RecoverySourceHandlerTests.java @@ -686,7 +686,7 @@ public void prepareForTranslogOperations(boolean fileBasedRecovery, int totalTra } @Override - public void finalizeRecovery(long globalCheckpoint) { + public void finalizeRecovery(long globalCheckpoint, ActionListener listener) { } @Override From cd78565acfe307549974b272138d2a3c9255e7ff Mon Sep 17 00:00:00 2001 From: Michael Basnight Date: Mon, 14 Jan 2019 19:50:20 -0600 Subject: [PATCH 156/186] Update Delete Watch to allow unknown fields (#37435) DeleteWatchResponse did not allow unknown fields. This commit fixes the test and ConstructingObjectParser such that it does now allow unknown fields. Relates #36938 --- .../client/watcher/DeleteWatchResponse.java | 15 +------ .../watcher/DeleteWatchResponseTests.java | 39 ++++++++++++------- 2 files changed, 26 insertions(+), 28 deletions(-) diff --git a/client/rest-high-level/src/main/java/org/elasticsearch/client/watcher/DeleteWatchResponse.java b/client/rest-high-level/src/main/java/org/elasticsearch/client/watcher/DeleteWatchResponse.java index 4e946ad459cf8..1d519773b32ce 100644 --- a/client/rest-high-level/src/main/java/org/elasticsearch/client/watcher/DeleteWatchResponse.java +++ b/client/rest-high-level/src/main/java/org/elasticsearch/client/watcher/DeleteWatchResponse.java @@ -20,17 +20,15 @@ import org.elasticsearch.common.ParseField; import org.elasticsearch.common.xcontent.ObjectParser; -import org.elasticsearch.common.xcontent.ToXContentObject; -import org.elasticsearch.common.xcontent.XContentBuilder; import org.elasticsearch.common.xcontent.XContentParser; import java.io.IOException; import java.util.Objects; -public class DeleteWatchResponse implements ToXContentObject { +public class DeleteWatchResponse { private static final ObjectParser PARSER - = new ObjectParser<>("x_pack_delete_watch_response", DeleteWatchResponse::new); + = new ObjectParser<>("x_pack_delete_watch_response", true, DeleteWatchResponse::new); static { PARSER.declareString(DeleteWatchResponse::setId, new ParseField("_id")); PARSER.declareLong(DeleteWatchResponse::setVersion, new ParseField("_version")); @@ -89,15 +87,6 @@ public int hashCode() { return Objects.hash(id, version, found); } - @Override - public XContentBuilder toXContent(XContentBuilder builder, Params params) throws IOException { - return builder.startObject() - .field("_id", id) - .field("_version", version) - .field("found", found) - .endObject(); - } - public static DeleteWatchResponse fromXContent(XContentParser parser) throws IOException { return PARSER.parse(parser, null); } diff --git a/client/rest-high-level/src/test/java/org/elasticsearch/client/watcher/DeleteWatchResponseTests.java b/client/rest-high-level/src/test/java/org/elasticsearch/client/watcher/DeleteWatchResponseTests.java index 3017b188292c0..45c3ef9dfdcfc 100644 --- a/client/rest-high-level/src/test/java/org/elasticsearch/client/watcher/DeleteWatchResponseTests.java +++ b/client/rest-high-level/src/test/java/org/elasticsearch/client/watcher/DeleteWatchResponseTests.java @@ -18,28 +18,37 @@ */ package org.elasticsearch.client.watcher; -import org.elasticsearch.common.xcontent.XContentParser; -import org.elasticsearch.test.AbstractXContentTestCase; +import org.elasticsearch.common.xcontent.XContentBuilder; +import org.elasticsearch.test.ESTestCase; import java.io.IOException; -public class DeleteWatchResponseTests extends AbstractXContentTestCase { +import static org.elasticsearch.test.AbstractXContentTestCase.xContentTester; - @Override - protected DeleteWatchResponse createTestInstance() { - String id = randomAlphaOfLength(10); - long version = randomLongBetween(1, 10); - boolean found = randomBoolean(); - return new DeleteWatchResponse(id, version, found); +public class DeleteWatchResponseTests extends ESTestCase { + + public void testFromXContent() throws IOException { + xContentTester(this::createParser, + DeleteWatchResponseTests::createTestInstance, + DeleteWatchResponseTests::toXContent, + DeleteWatchResponse::fromXContent) + .supportsUnknownFields(true) + .assertToXContentEquivalence(false) + .test(); } - @Override - protected DeleteWatchResponse doParseInstance(XContentParser parser) throws IOException { - return DeleteWatchResponse.fromXContent(parser); + private static XContentBuilder toXContent(DeleteWatchResponse response, XContentBuilder builder) throws IOException { + return builder.startObject() + .field("_id", response.getId()) + .field("_version", response.getVersion()) + .field("found", response.isFound()) + .endObject(); } - @Override - protected boolean supportsUnknownFields() { - return false; + private static DeleteWatchResponse createTestInstance() { + String id = randomAlphaOfLength(10); + long version = randomLongBetween(1, 10); + boolean found = randomBoolean(); + return new DeleteWatchResponse(id, version, found); } } From 74640d0ba7cbb0f6ffd78ff116990ae8d8e5cfde Mon Sep 17 00:00:00 2001 From: Jason Tedor Date: Mon, 14 Jan 2019 21:06:44 -0500 Subject: [PATCH 157/186] Introduce retention lease serialization (#37447) This commit is a simple introduction of the serialization of retention leases, which will be needed when they are sent across the wire while synchronizing retention leases to replicas. --- .../index/seqno/RetentionLease.java | 34 ++++++++++++++++++- .../index/seqno/RetentionLeaseTests.java | 17 ++++++++++ 2 files changed, 50 insertions(+), 1 deletion(-) diff --git a/server/src/main/java/org/elasticsearch/index/seqno/RetentionLease.java b/server/src/main/java/org/elasticsearch/index/seqno/RetentionLease.java index f763759261385..13e3381b11553 100644 --- a/server/src/main/java/org/elasticsearch/index/seqno/RetentionLease.java +++ b/server/src/main/java/org/elasticsearch/index/seqno/RetentionLease.java @@ -19,6 +19,11 @@ package org.elasticsearch.index.seqno; +import org.elasticsearch.common.io.stream.StreamInput; +import org.elasticsearch.common.io.stream.StreamOutput; +import org.elasticsearch.common.io.stream.Writeable; + +import java.io.IOException; import java.util.Arrays; import java.util.Collection; import java.util.Collections; @@ -32,7 +37,7 @@ * otherwise merge away operations that have been soft deleted). Each retention lease contains a unique identifier, the retaining sequence * number, the timestamp of when the lease was created or renewed, and the source of the retention lease (e.g., "ccr"). */ -public final class RetentionLease { +public final class RetentionLease implements Writeable { private final String id; @@ -116,6 +121,33 @@ public RetentionLease(final String id, final long retainingSequenceNumber, final this.source = source; } + /** + * Constructs a new retention lease from a stream. The retention lease should have been written via {@link #writeTo(StreamOutput)}. + * + * @param in the stream to construct the retention lease from + * @throws IOException if an I/O exception occurs reading from the stream + */ + public RetentionLease(final StreamInput in) throws IOException { + id = in.readString(); + retainingSequenceNumber = in.readZLong(); + timestamp = in.readVLong(); + source = in.readString(); + } + + /** + * Writes a retention lease to a stream in a manner suitable for later reconstruction via {@link #RetentionLease(StreamInput)}. + * + * @param out the stream to write the retention lease to + * @throws IOException if an I/O exception occurs writing to the stream + */ + @Override + public void writeTo(final StreamOutput out) throws IOException { + out.writeString(id); + out.writeZLong(retainingSequenceNumber); + out.writeVLong(timestamp); + out.writeString(source); + } + /** * Encodes a retention lease as a string. This encoding can be decoded by {@link #decodeRetentionLease(String)}. The retention lease is * encoded in the format id:{id};retaining_seq_no:{retainingSequenecNumber};timestamp:{timestamp};source:{source}. diff --git a/server/src/test/java/org/elasticsearch/index/seqno/RetentionLeaseTests.java b/server/src/test/java/org/elasticsearch/index/seqno/RetentionLeaseTests.java index c4340a381ce25..500393f2cfac2 100644 --- a/server/src/test/java/org/elasticsearch/index/seqno/RetentionLeaseTests.java +++ b/server/src/test/java/org/elasticsearch/index/seqno/RetentionLeaseTests.java @@ -19,8 +19,11 @@ package org.elasticsearch.index.seqno; +import org.elasticsearch.common.io.stream.BytesStreamOutput; +import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.test.ESTestCase; +import java.io.IOException; import java.util.ArrayList; import java.util.Collection; import java.util.List; @@ -82,6 +85,20 @@ public void testEmptySource() { assertThat(e, hasToString(containsString("retention lease source can not be empty"))); } + public void testRetentionLeaseSerialization() throws IOException { + final String id = randomAlphaOfLength(8); + final long retainingSequenceNumber = randomLongBetween(SequenceNumbers.NO_OPS_PERFORMED, Long.MAX_VALUE); + final long timestamp = randomNonNegativeLong(); + final String source = randomAlphaOfLength(8); + final RetentionLease retentionLease = new RetentionLease(id, retainingSequenceNumber, timestamp, source); + try (BytesStreamOutput out = new BytesStreamOutput()) { + retentionLease.writeTo(out); + try (StreamInput in = out.bytes().streamInput()) { + assertThat(retentionLease, equalTo(new RetentionLease(in))); + } + } + } + public void testRetentionLeaseEncoding() { final String id = randomAlphaOfLength(8); final long retainingSequenceNumber = randomNonNegativeLong(); From eb86b9f284ae0f272b51218efe68f36e294dedbc Mon Sep 17 00:00:00 2001 From: Jason Tedor Date: Mon, 14 Jan 2019 21:15:35 -0500 Subject: [PATCH 158/186] Fix retention lease commit test This commit fixes an issue with testing committed retention leases when they are not any retention leases (a deliberate edge case). Closes #37420 --- .../shard/IndexShardRetentionLeaseTests.java | 18 +++++++++++++----- 1 file changed, 13 insertions(+), 5 deletions(-) diff --git a/server/src/test/java/org/elasticsearch/index/shard/IndexShardRetentionLeaseTests.java b/server/src/test/java/org/elasticsearch/index/shard/IndexShardRetentionLeaseTests.java index e15f6c45ae640..d0018a0a864f8 100644 --- a/server/src/test/java/org/elasticsearch/index/shard/IndexShardRetentionLeaseTests.java +++ b/server/src/test/java/org/elasticsearch/index/shard/IndexShardRetentionLeaseTests.java @@ -43,6 +43,7 @@ import java.util.function.LongSupplier; import static org.hamcrest.Matchers.contains; +import static org.hamcrest.Matchers.empty; import static org.hamcrest.Matchers.equalTo; import static org.hamcrest.Matchers.hasItem; import static org.hamcrest.Matchers.hasSize; @@ -134,7 +135,6 @@ public void testExpiration() throws IOException { } } - @AwaitsFix(bugUrl = "https://github.com/elastic/elasticsearch/issues/37420") public void testCommit() throws IOException { final Settings settings = Settings.builder() .put(IndexSettings.INDEX_SOFT_DELETES_SETTING.getKey(), true) @@ -162,7 +162,11 @@ public void testCommit() throws IOException { final SegmentInfos segmentCommitInfos = indexShard.store().readLastCommittedSegmentsInfo(); assertTrue(segmentCommitInfos.getUserData().containsKey(Engine.RETENTION_LEASES)); final Collection retentionLeases = indexShard.getEngine().config().retentionLeasesSupplier().get(); - assertThat(IndexShard.getRetentionLeases(segmentCommitInfos), contains(retentionLeases.toArray(new RetentionLease[0]))); + if (retentionLeases.isEmpty()) { + assertThat(IndexShard.getRetentionLeases(segmentCommitInfos), empty()); + } else { + assertThat(IndexShard.getRetentionLeases(segmentCommitInfos), contains(retentionLeases.toArray(new RetentionLease[0]))); + } // when we recover, we should recover the retention leases final IndexShard recoveredShard = reinitShard( @@ -170,9 +174,13 @@ public void testCommit() throws IOException { ShardRoutingHelper.initWithSameId(indexShard.routingEntry(), RecoverySource.ExistingStoreRecoverySource.INSTANCE)); try { recoverShardFromStore(recoveredShard); - assertThat( - recoveredShard.getEngine().config().retentionLeasesSupplier().get(), - contains(retentionLeases.toArray(new RetentionLease[0]))); + if (retentionLeases.isEmpty()) { + assertThat(recoveredShard.getEngine().config().retentionLeasesSupplier().get(), empty()); + } else { + assertThat( + recoveredShard.getEngine().config().retentionLeasesSupplier().get(), + contains(retentionLeases.toArray(new RetentionLease[0]))); + } } finally { closeShards(recoveredShard); } From 3bc0711b908b3f68d3d0b52fae43f42aba8b8a92 Mon Sep 17 00:00:00 2001 From: Jason Tedor Date: Mon, 14 Jan 2019 21:28:28 -0500 Subject: [PATCH 159/186] Add simple method to write collection of writeables (#37448) This commit adds a simple convenience method for writing a collection of writeables, and replaces existing call sites with the new method. --- .../bootstrap/GetDiscoveredNodesResponse.java | 2 +- .../coordination/CoordinationMetaData.java | 2 +- .../common/io/stream/StreamOutput.java | 16 ++++-- .../common/io/stream/StreamTests.java | 52 +++++++++++++++++++ .../user/GetUserPrivilegesResponse.java | 4 +- 5 files changed, 68 insertions(+), 8 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/action/admin/cluster/bootstrap/GetDiscoveredNodesResponse.java b/server/src/main/java/org/elasticsearch/action/admin/cluster/bootstrap/GetDiscoveredNodesResponse.java index f1174002b6998..f697e16c03c2c 100644 --- a/server/src/main/java/org/elasticsearch/action/admin/cluster/bootstrap/GetDiscoveredNodesResponse.java +++ b/server/src/main/java/org/elasticsearch/action/admin/cluster/bootstrap/GetDiscoveredNodesResponse.java @@ -68,6 +68,6 @@ public void readFrom(StreamInput in) throws IOException { @Override public void writeTo(StreamOutput out) throws IOException { super.writeTo(out); - out.writeCollection(nodes, (o, v) -> v.writeTo(o)); + out.writeCollection(nodes); } } diff --git a/server/src/main/java/org/elasticsearch/cluster/coordination/CoordinationMetaData.java b/server/src/main/java/org/elasticsearch/cluster/coordination/CoordinationMetaData.java index 2bf5755df3144..01ef85b656d1e 100644 --- a/server/src/main/java/org/elasticsearch/cluster/coordination/CoordinationMetaData.java +++ b/server/src/main/java/org/elasticsearch/cluster/coordination/CoordinationMetaData.java @@ -117,7 +117,7 @@ public void writeTo(StreamOutput out) throws IOException { out.writeLong(term); lastCommittedConfiguration.writeTo(out); lastAcceptedConfiguration.writeTo(out); - out.writeCollection(votingConfigExclusions, (o, v) -> v.writeTo(o)); + out.writeCollection(votingConfigExclusions); } @Override diff --git a/server/src/main/java/org/elasticsearch/common/io/stream/StreamOutput.java b/server/src/main/java/org/elasticsearch/common/io/stream/StreamOutput.java index 4164e29a58279..8131335602693 100644 --- a/server/src/main/java/org/elasticsearch/common/io/stream/StreamOutput.java +++ b/server/src/main/java/org/elasticsearch/common/io/stream/StreamOutput.java @@ -1010,14 +1010,22 @@ public void writeStreamableList(List list) throws IOExcept } } + /** + * Writes a collection to this stream. The corresponding collection can be read from a stream input using + * {@link StreamInput#readList(Writeable.Reader)}. + * + * @param collection the collection to write to this stream + * @throws IOException if an I/O exception occurs writing the collection + */ + public void writeCollection(final Collection collection) throws IOException { + writeCollection(collection, (o, v) -> v.writeTo(o)); + } + /** * Writes a list of {@link Writeable} objects */ public void writeList(List list) throws IOException { - writeVInt(list.size()); - for (Writeable obj: list) { - obj.writeTo(this); - } + writeCollection(list); } /** diff --git a/server/src/test/java/org/elasticsearch/common/io/stream/StreamTests.java b/server/src/test/java/org/elasticsearch/common/io/stream/StreamTests.java index 05cc442c48e9e..53a00111c4289 100644 --- a/server/src/test/java/org/elasticsearch/common/io/stream/StreamTests.java +++ b/server/src/test/java/org/elasticsearch/common/io/stream/StreamTests.java @@ -30,12 +30,14 @@ import java.io.IOException; import java.util.ArrayList; import java.util.Arrays; +import java.util.Collection; import java.util.Collections; import java.util.HashSet; import java.util.LinkedHashMap; import java.util.List; import java.util.Locale; import java.util.Map; +import java.util.Objects; import java.util.Set; import java.util.stream.Collectors; import java.util.stream.IntStream; @@ -254,6 +256,56 @@ public void testArrays() throws IOException { assertThat(deserialized, equalTo(strings)); } + public void testCollection() throws IOException { + class FooBar implements Writeable { + + private final int foo; + private final int bar; + + private FooBar(final int foo, final int bar) { + this.foo = foo; + this.bar = bar; + } + + private FooBar(final StreamInput in) throws IOException { + this.foo = in.readInt(); + this.bar = in.readInt(); + } + + @Override + public void writeTo(final StreamOutput out) throws IOException { + out.writeInt(foo); + out.writeInt(bar); + } + + @Override + public boolean equals(final Object o) { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + final FooBar that = (FooBar) o; + return foo == that.foo && bar == that.bar; + } + + @Override + public int hashCode() { + return Objects.hash(foo, bar); + } + + } + + final int length = randomIntBetween(0, 16); + final Collection fooBars = new ArrayList<>(length); + for (int i = 0; i < length; i++) { + fooBars.add(new FooBar(randomInt(), randomInt())); + } + try (BytesStreamOutput out = new BytesStreamOutput()) { + out.writeCollection(fooBars); + try (StreamInput in = out.bytes().streamInput()) { + assertThat(fooBars, equalTo(in.readList(FooBar::new))); + } + } + } + public void testSetOfLongs() throws IOException { final int size = randomIntBetween(0, 6); final Set sourceSet = new HashSet<>(size); diff --git a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/security/action/user/GetUserPrivilegesResponse.java b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/security/action/user/GetUserPrivilegesResponse.java index fccb22e13065d..6a6f011a76ad2 100644 --- a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/security/action/user/GetUserPrivilegesResponse.java +++ b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/security/action/user/GetUserPrivilegesResponse.java @@ -86,8 +86,8 @@ public void writeTo(StreamOutput out) throws IOException { super.writeTo(out); out.writeCollection(cluster, StreamOutput::writeString); out.writeCollection(conditionalCluster, ConditionalClusterPrivileges.WRITER); - out.writeCollection(index, (o, p) -> p.writeTo(o)); - out.writeCollection(application, (o, p) -> p.writeTo(o)); + out.writeCollection(index); + out.writeCollection(application); out.writeCollection(runAs, StreamOutput::writeString); } From c0368a20864dbc8780508b88c668d8730c21c586 Mon Sep 17 00:00:00 2001 From: Michael Basnight Date: Mon, 14 Jan 2019 20:44:21 -0600 Subject: [PATCH 160/186] [DOCS] Add watcher context examples (#36565) --- ...ainless-watcher-condition-context.asciidoc | 144 +++++++++++++--- .../painless-watcher-context-example.asciidoc | 157 +++++++++++++++++ ...ainless-watcher-context-variables.asciidoc | 40 +++++ ...ainless-watcher-transform-context.asciidoc | 162 +++++++++++++++--- .../painless-weight-context.asciidoc | 2 +- 5 files changed, 458 insertions(+), 47 deletions(-) create mode 100644 docs/painless/painless-contexts/painless-watcher-context-example.asciidoc create mode 100644 docs/painless/painless-contexts/painless-watcher-context-variables.asciidoc diff --git a/docs/painless/painless-contexts/painless-watcher-condition-context.asciidoc b/docs/painless/painless-contexts/painless-watcher-condition-context.asciidoc index 713bcf32daec4..91ab51561ef88 100644 --- a/docs/painless/painless-contexts/painless-watcher-condition-context.asciidoc +++ b/docs/painless/painless-contexts/painless-watcher-condition-context.asciidoc @@ -1,38 +1,136 @@ [[painless-watcher-condition-context]] === Watcher condition context -Use a Painless script as a {xpack-ref}/condition-script.html[watcher condition] -to test if a response is necessary. +Use a Painless script as a {xpack-ref}/condition-script.html[watch condition] +that determines whether to execute a watch or a particular action within a watch. +Condition scripts return a Boolean value to indicate the status of the condition. -*Variables* +include::painless-watcher-context-variables.asciidoc[] -`params` (`Map`, read-only):: - User-defined parameters passed in as part of the query. +*Return* -`ctx['watch_id']` (`String`, read-only):: - The id of the watch. +`boolean`:: + Expects `true` if the condition is met, and `false` if it is not. -`ctx['execution_time']` (`ZonedDateTime`, read-only):: - The start time for the watch. +*API* -`ctx['trigger']['scheduled_time']` (`ZonedDateTime`, read-only):: - The scheduled trigger time for the watch. +The standard <> is available. -`ctx['trigger']['triggered_time']` (`ZonedDateTime`, read-only):: - The actual trigger time for the watch. +*Example* -`ctx['metadata']` (`Map`, read-only):: - Any metadata associated with the watch. +[source,Painless] +---- +POST _watcher/watch/_execute +{ + "watch" : { + "trigger" : { "schedule" : { "interval" : "24h" } }, + "input" : { + "search" : { + "request" : { + "indices" : [ "seats" ], + "body" : { + "query" : { + "term": { "sold": "true"} + }, + "aggs" : { + "theatres" : { + "terms" : { "field" : "play" }, + "aggs" : { + "money" : { + "sum": { "field" : "cost" } + } + } + } + } + } + } + } + }, + "condition" : { + "script" : + """ + return ctx.payload.aggregations.theatres.buckets.stream() <1> + .filter(theatre -> theatre.money.value < 15000 || + theatre.money.value > 50000) <2> + .count() > 0 <3> + """ + }, + "actions" : { + "my_log" : { + "logging" : { + "text" : "The output of the search was : {{ctx.payload.aggregations.theatres.buckets}}" + } + } + } + } +} +---- -`ctx['payload']` (`Map`, read-only):: - The accessible watch data based upon the - {xpack-ref}/input.html[watch input]. +<1> The Java Stream API is used in the condition. This API allows manipulation of +the elements of the list in a pipeline. +<2> The stream filter removes items that do not meet the filter criteria. +<3> If there is at least one item in the list, the condition evaluates to true and the watch is executed. -*Return* +The following action condition script controls execution of the my_log action based +on the value of the seats sold for the plays in the data set. The script aggregates +the total sold seats for each play and returns true if there is at least one play +that has sold over $50,000. -`boolean`:: - Expects `true` if the condition is met, and `false` otherwise. +[source,Painless] +---- +POST _watcher/watch/_execute +{ + "watch" : { + "trigger" : { "schedule" : { "interval" : "24h" } }, + "input" : { + "search" : { + "request" : { + "indices" : [ "seats" ], + "body" : { + "query" : { + "term": { "sold": "true"} + }, + "aggs" : { + "theatres" : { + "terms" : { "field" : "play" }, + "aggs" : { + "money" : { + "sum": { "field" : "cost" } + } + } + } + } + } + } + } + }, + "actions" : { + "my_log" : { + "condition": { <1> + "script" : + """ + return ctx.payload.aggregations.theatres.buckets.stream() + .anyMatch(theatre -> theatre.money.value > 50000) <2> + """ + }, + "logging" : { + "text" : "At least one play has grossed over $50,000: {{ctx.payload.aggregations.theatres.buckets}}" + } + } + } + } +} +---- -*API* +This example uses a nearly identical condition as the previous example. The +differences below are subtle and are worth calling out. + +<1> The location of the condition is no longer at the top level, but is within +an individual action. +<2> Instead of a filter, `anyMatch` is used to return a boolean value + +The following example shows scripted watch and action conditions within the +context of a complete watch. This watch also uses a scripted +<>. -The standard <> is available. \ No newline at end of file +include::painless-watcher-context-example.asciidoc[] diff --git a/docs/painless/painless-contexts/painless-watcher-context-example.asciidoc b/docs/painless/painless-contexts/painless-watcher-context-example.asciidoc new file mode 100644 index 0000000000000..2d2e3993e3233 --- /dev/null +++ b/docs/painless/painless-contexts/painless-watcher-context-example.asciidoc @@ -0,0 +1,157 @@ +[source,Painless] +---- +POST _watcher/watch/_execute +{ + "watch" : { + "metadata" : { "high_threshold": 50000, "low_threshold": 15000 }, + "trigger" : { "schedule" : { "interval" : "24h" } }, + "input" : { + "search" : { + "request" : { + "indices" : [ "seats" ], + "body" : { + "query" : { + "term": { "sold": "true"} + }, + "aggs" : { + "theatres" : { + "terms" : { "field" : "play" }, + "aggs" : { + "money" : { + "sum": { "field" : "cost" } + } + } + } + } + } + } + } + }, + "condition" : { + "script" : + """ + return ctx.payload.aggregations.theatres.buckets.stream() + .anyMatch(theatre -> theatre.money.value < ctx.metadata.low_threshold || + theatre.money.value > ctx.metadata.high_threshold) + """ + }, + "transform" : { + "script": + """ + return [ + 'money_makers': ctx.payload.aggregations.theatres.buckets.stream() + .filter(t -> { + return t.money.value > ctx.metadata.high_threshold + }) + .map(t -> { + return ['play': t.key, 'total_value': t.money.value ] + }).collect(Collectors.toList()), + 'duds' : ctx.payload.aggregations.theatres.buckets.stream() + .filter(t -> { + return t.money.value < ctx.metadata.low_threshold + }) + .map(t -> { + return ['play': t.key, 'total_value': t.money.value ] + }).collect(Collectors.toList()) + ] + """ + }, + "actions" : { + "log_money_makers" : { + "condition": { + "script" : "return ctx.payload.money_makers.size() > 0" + }, + "transform": { + "script" : + """ + def formatter = NumberFormat.getCurrencyInstance(); + return [ + 'plays_value': ctx.payload.money_makers.stream() + .map(t-> formatter.format(t.total_value) + ' for the play ' + t.play) + .collect(Collectors.joining(", ")) + ] + """ + }, + "logging" : { + "text" : "The following plays contain the highest grossing total income: {{ctx.payload.plays_value}}" + } + }, + "log_duds" : { + "condition": { + "script" : "return ctx.payload.duds.size() > 0" + }, + "transform": { + "script" : + """ + def formatter = NumberFormat.getCurrencyInstance(); + return [ + 'plays_value': ctx.payload.duds.stream() + .map(t-> formatter.format(t.total_value) + ' for the play ' + t.play) + .collect(Collectors.joining(", ")) + ] + """ + }, + "logging" : { + "text" : "The following plays need more advertising due to their low total income: {{ctx.payload.plays_value}}" + } + } + } + } +} +---- + +The following example shows the use of metadata and transforming dates into a readable format. + +[source,Painless] +---- +POST _xpack/watcher/watch/_execute +{ + "watch" : { + "metadata" : { "min_hits": 10000 }, + "trigger" : { "schedule" : { "interval" : "24h" } }, + "input" : { + "search" : { + "request" : { + "indices" : [ "seats" ], + "body" : { + "query" : { + "term": { "sold": "true"} + }, + "aggs" : { + "theatres" : { + "terms" : { "field" : "play" }, + "aggs" : { + "money" : { + "sum": { "field" : "cost" } + } + } + } + } + } + } + } + }, + "condition" : { + "script" : + """ + return ctx.payload.hits.total > ctx.metadata.min_hits + """ + }, + "transform" : { + "script" : + """ + def theDate = ZonedDateTime.ofInstant(ctx.execution_time.toInstant(), ctx.execution_time.getZone()); + return ['human_date': DateTimeFormatter.RFC_1123_DATE_TIME.format(theDate), + 'aggregations': ctx.payload.aggregations] + """ + }, + "actions" : { + "my_log" : { + "logging" : { + "text" : "The watch was successfully executed on {{ctx.payload.human_date}} and contained {{ctx.payload.aggregations.theatres.buckets.size}} buckets" + } + } + } + } +} +---- diff --git a/docs/painless/painless-contexts/painless-watcher-context-variables.asciidoc b/docs/painless/painless-contexts/painless-watcher-context-variables.asciidoc new file mode 100644 index 0000000000000..addfd11cab92e --- /dev/null +++ b/docs/painless/painless-contexts/painless-watcher-context-variables.asciidoc @@ -0,0 +1,40 @@ +The following variables are available in all watcher contexts. + +*Variables* + +`params` (`Map`, read-only):: + User-defined parameters passed in as part of the query. + +`ctx['watch_id']` (`String`, read-only):: + The id of the watch. + +`ctx['id']` (`String`, read-only):: + The server generated unique identifer for the run watch. + +`ctx['metadata']` (`Map`, read-only):: + Metadata can be added to the top level of the watch definition. This + is user defined and is typically used to consolidate duplicate values + in a watch. + +`ctx['execution_time']` (`ZonedDateTime`, read-only):: + The time the watch began execution. + +`ctx['trigger']['scheduled_time']` (`ZonedDateTime`, read-only):: + The scheduled trigger time for the watch. This is the time the + watch should be executed. + +`ctx['trigger']['triggered_time']` (`ZonedDateTime`, read-only):: + The actual trigger time for the watch. This is the time the + watch was triggered for execution. + +`ctx['payload']` (`Map`, read-only):: + The accessible watch data based upon the + {xpack-ref}/input.html[watch input]. + +*API* + + +The standard <> is available. + +To run this example, first follow the steps in +<>. diff --git a/docs/painless/painless-contexts/painless-watcher-transform-context.asciidoc b/docs/painless/painless-contexts/painless-watcher-transform-context.asciidoc index 27cb4eb15056d..92012720aa69e 100644 --- a/docs/painless/painless-contexts/painless-watcher-transform-context.asciidoc +++ b/docs/painless/painless-contexts/painless-watcher-transform-context.asciidoc @@ -1,39 +1,155 @@ [[painless-watcher-transform-context]] === Watcher transform context -Use a Painless script to {xpack-ref}/transform-script.html[transform] watch -data into a new payload for use in a response to a condition. +Use a Painless script as a {xpack-ref}/transform-script.html[watch transform] +to transform a payload into a new payload for further use in the watch. +Transform scripts return an Object value of the new payload. -*Variables* +include::painless-watcher-context-variables.asciidoc[] -`params` (`Map`, read-only):: - User-defined parameters passed in as part of the query. +*Return* -`ctx['watch_id']` (`String`, read-only):: - The id of the watch. +`Object`:: + The new payload. -`ctx['execution_time']` (`ZonedDateTime`, read-only):: - The start time for the watch. +*API* -`ctx['trigger']['scheduled_time']` (`ZonedDateTime`, read-only):: - The scheduled trigger time for the watch. +The standard <> is available. -`ctx['trigger']['triggered_time']` (`ZonedDateTime`, read-only):: - The actual trigger time for the watch. +*Example* -`ctx['metadata']` (`Map`, read-only):: - Any metadata associated with the watch. +[source,Painless] +---- +POST _watcher/watch/_execute +{ + "watch" : { + "trigger" : { "schedule" : { "interval" : "24h" } }, + "input" : { + "search" : { + "request" : { + "indices" : [ "seats" ], + "body" : { + "query" : { "term": { "sold": "true"} }, + "aggs" : { + "theatres" : { + "terms" : { "field" : "play" }, + "aggs" : { + "money" : { + "sum": { "field" : "cost" } + } + } + } + } + } + } + } + }, + "transform" : { + "script": + """ + return [ + 'money_makers': ctx.payload.aggregations.theatres.buckets.stream() <1> + .filter(t -> { <2> + return t.money.value > 50000 + }) + .map(t -> { <3> + return ['play': t.key, 'total_value': t.money.value ] + }).collect(Collectors.toList()), <4> + 'duds' : ctx.payload.aggregations.theatres.buckets.stream() <5> + .filter(t -> { + return t.money.value < 15000 + }) + .map(t -> { + return ['play': t.key, 'total_value': t.money.value ] + }).collect(Collectors.toList()) + ] + """ + }, + "actions" : { + "my_log" : { + "logging" : { + "text" : "The output of the payload was transformed to {{ctx.payload}}" + } + } + } + } +} +---- -`ctx['payload']` (`Map`, read-only):: - The accessible watch data based upon the - {xpack-ref}/input.html[watch input]. +<1> The Java Stream API is used in the transform. This API allows manipulation of +the elements of the list in a pipeline. +<2> The stream filter removes items that do not meet the filter criteria. +<3> The stream map transforms each element into a new object. +<4> The collector reduces the stream to a `java.util.List`. +<5> This is done again for the second set of values in the transform. +The following action transform changes each value in the mod_log action into a `String`. +This transform does not change the values in the unmod_log action. -*Return* +[source,Painless] +---- +POST _watcher/watch/_execute +{ + "watch" : { + "trigger" : { "schedule" : { "interval" : "24h" } }, + "input" : { + "search" : { + "request" : { + "indices" : [ "seats" ], + "body" : { + "query" : { + "term": { "sold": "true"} + }, + "aggs" : { + "theatres" : { + "terms" : { "field" : "play" }, + "aggs" : { + "money" : { + "sum": { "field" : "cost" } + } + } + } + } + } + } + } + }, + "actions" : { + "mod_log" : { + "transform": { <1> + "script" : + """ + def formatter = NumberFormat.getCurrencyInstance(); + return [ + 'msg': ctx.payload.aggregations.theatres.buckets.stream() + .map(t-> formatter.format(t.money.value) + ' for the play ' + t.key) + .collect(Collectors.joining(", ")) + ] + """ + }, + "logging" : { + "text" : "The output of the payload was transformed to: {{ctx.payload.msg}}" + } + }, + "unmod_log" : { <2> + "logging" : { + "text" : "The output of the payload was not transformed and this value should not exist: {{ctx.payload.msg}}" + } + } + } + } +} +---- -`Object`:: - The new payload. +This example uses the streaming API in a very similar manner. The differences below are +subtle and worth calling out. -*API* +<1> The location of the transform is no longer at the top level, but is within +an individual action. +<2> A second action that does not transform the payload is given for reference. + +The following example shows scripted watch and action transforms within the +context of a complete watch. This watch also uses a scripted +<>. -The standard <> is available. \ No newline at end of file +include::painless-watcher-context-example.asciidoc[] diff --git a/docs/painless/painless-contexts/painless-weight-context.asciidoc b/docs/painless/painless-contexts/painless-weight-context.asciidoc index 319b7999aa831..a3088db5f3ab0 100644 --- a/docs/painless/painless-contexts/painless-weight-context.asciidoc +++ b/docs/painless/painless-contexts/painless-weight-context.asciidoc @@ -42,4 +42,4 @@ Queries that contain multiple terms calculate a separate weight for each term. *API* -The standard <> is available. \ No newline at end of file +The standard <> is available. From e11a32eda8f6182d3ebf98955e3ec39daf6a398c Mon Sep 17 00:00:00 2001 From: Jason Tedor Date: Mon, 14 Jan 2019 21:43:56 -0500 Subject: [PATCH 161/186] Reformat some classes in the index universe This commit reformats some classes in the index universe with the purpose of breaking some long method definitions and invocations into a line per parameter. This has the advantage that for an upcoming change to these definitions and invocations, the diff for that change will be a single line per definition or invocation. That makes these sorts of changes easier to read. --- .../org/elasticsearch/index/IndexService.java | 24 +++- .../elasticsearch/index/shard/IndexShard.java | 36 +++--- .../elasticsearch/indices/IndicesService.java | 12 +- .../cluster/IndicesClusterStateService.java | 113 +++++++++++------- .../index/shard/IndexShardIT.java | 32 +++-- ...actIndicesClusterStateServiceTestCase.java | 14 ++- .../snapshots/SnapshotsServiceTests.java | 55 +++++---- .../index/shard/IndexShardTestCase.java | 23 +++- 8 files changed, 201 insertions(+), 108 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/index/IndexService.java b/server/src/main/java/org/elasticsearch/index/IndexService.java index 54bf5fa1aa18e..6c01e61586d18 100644 --- a/server/src/main/java/org/elasticsearch/index/IndexService.java +++ b/server/src/main/java/org/elasticsearch/index/IndexService.java @@ -382,11 +382,25 @@ public synchronized IndexShard createShard(ShardRouting routing, Consumer eventListener.onStoreClosed(shardId))); - indexShard = new IndexShard(routing, this.indexSettings, path, store, indexSortSupplier, - indexCache, mapperService, similarityService, engineFactory, - eventListener, searcherWrapper, threadPool, bigArrays, engineWarmer, - searchOperationListeners, indexingOperationListeners, () -> globalCheckpointSyncer.accept(shardId), - circuitBreakerService); + indexShard = new IndexShard( + routing, + this.indexSettings, + path, + store, + indexSortSupplier, + indexCache, + mapperService, + similarityService, + engineFactory, + eventListener, + searcherWrapper, + threadPool, + bigArrays, + engineWarmer, + searchOperationListeners, + indexingOperationListeners, + () -> globalCheckpointSyncer.accept(shardId), + circuitBreakerService); eventListener.indexShardStateChanged(indexShard, null, indexShard.state(), "shard created"); eventListener.afterIndexShardCreated(indexShard); shards = newMapBuilder(shards).put(shardId.id(), indexShard).immutableMap(); diff --git a/server/src/main/java/org/elasticsearch/index/shard/IndexShard.java b/server/src/main/java/org/elasticsearch/index/shard/IndexShard.java index 270ce20819102..c76c845328bd7 100644 --- a/server/src/main/java/org/elasticsearch/index/shard/IndexShard.java +++ b/server/src/main/java/org/elasticsearch/index/shard/IndexShard.java @@ -249,24 +249,24 @@ Runnable getGlobalCheckpointSyncer() { private final AtomicReference pendingRefreshLocation = new AtomicReference<>(); public IndexShard( - ShardRouting shardRouting, - IndexSettings indexSettings, - ShardPath path, - Store store, - Supplier indexSortSupplier, - IndexCache indexCache, - MapperService mapperService, - SimilarityService similarityService, - @Nullable EngineFactory engineFactory, - IndexEventListener indexEventListener, - IndexSearcherWrapper indexSearcherWrapper, - ThreadPool threadPool, - BigArrays bigArrays, - Engine.Warmer warmer, - List searchOperationListener, - List listeners, - Runnable globalCheckpointSyncer, - CircuitBreakerService circuitBreakerService) throws IOException { + final ShardRouting shardRouting, + final IndexSettings indexSettings, + final ShardPath path, + final Store store, + final Supplier indexSortSupplier, + final IndexCache indexCache, + final MapperService mapperService, + final SimilarityService similarityService, + final @Nullable EngineFactory engineFactory, + final IndexEventListener indexEventListener, + final IndexSearcherWrapper indexSearcherWrapper, + final ThreadPool threadPool, + final BigArrays bigArrays, + final Engine.Warmer warmer, + final List searchOperationListener, + final List listeners, + final Runnable globalCheckpointSyncer, + final CircuitBreakerService circuitBreakerService) throws IOException { super(shardRouting.shardId(), indexSettings); assert shardRouting.initializing(); this.shardRouting = shardRouting; diff --git a/server/src/main/java/org/elasticsearch/indices/IndicesService.java b/server/src/main/java/org/elasticsearch/indices/IndicesService.java index f0294535fa751..e3c1a77b84300 100644 --- a/server/src/main/java/org/elasticsearch/indices/IndicesService.java +++ b/server/src/main/java/org/elasticsearch/indices/IndicesService.java @@ -586,10 +586,14 @@ public synchronized void verifyIndexMetadata(IndexMetaData metaData, IndexMetaDa } @Override - public IndexShard createShard(ShardRouting shardRouting, RecoveryState recoveryState, PeerRecoveryTargetService recoveryTargetService, - PeerRecoveryTargetService.RecoveryListener recoveryListener, RepositoriesService repositoriesService, - Consumer onShardFailure, - Consumer globalCheckpointSyncer) throws IOException { + public IndexShard createShard( + final ShardRouting shardRouting, + final RecoveryState recoveryState, + final PeerRecoveryTargetService recoveryTargetService, + final PeerRecoveryTargetService.RecoveryListener recoveryListener, + final RepositoriesService repositoriesService, + final Consumer onShardFailure, + final Consumer globalCheckpointSyncer) throws IOException { ensureChangesAllowed(); IndexService indexService = indexService(shardRouting.index()); IndexShard indexShard = indexService.createShard(shardRouting, globalCheckpointSyncer); diff --git a/server/src/main/java/org/elasticsearch/indices/cluster/IndicesClusterStateService.java b/server/src/main/java/org/elasticsearch/indices/cluster/IndicesClusterStateService.java index 701690ed1d4f0..337c66812da81 100644 --- a/server/src/main/java/org/elasticsearch/indices/cluster/IndicesClusterStateService.java +++ b/server/src/main/java/org/elasticsearch/indices/cluster/IndicesClusterStateService.java @@ -43,7 +43,6 @@ import org.elasticsearch.cluster.service.ClusterService; import org.elasticsearch.common.Nullable; import org.elasticsearch.common.component.AbstractLifecycleComponent; -import org.elasticsearch.common.inject.Inject; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.unit.TimeValue; import org.elasticsearch.common.util.concurrent.AbstractRunnable; @@ -122,42 +121,54 @@ public class IndicesClusterStateService extends AbstractLifecycleComponent imple private final PrimaryReplicaSyncer primaryReplicaSyncer; private final Consumer globalCheckpointSyncer; - @Inject - public IndicesClusterStateService(Settings settings, - IndicesService indicesService, - ClusterService clusterService, - ThreadPool threadPool, - PeerRecoveryTargetService recoveryTargetService, - ShardStateAction shardStateAction, - NodeMappingRefreshAction nodeMappingRefreshAction, - RepositoriesService repositoriesService, - SearchService searchService, - SyncedFlushService syncedFlushService, - PeerRecoverySourceService peerRecoverySourceService, - SnapshotShardsService snapshotShardsService, - PrimaryReplicaSyncer primaryReplicaSyncer, - GlobalCheckpointSyncAction globalCheckpointSyncAction) { - this(settings, (AllocatedIndices>) indicesService, - clusterService, threadPool, recoveryTargetService, shardStateAction, - nodeMappingRefreshAction, repositoriesService, searchService, syncedFlushService, peerRecoverySourceService, - snapshotShardsService, primaryReplicaSyncer, globalCheckpointSyncAction::updateGlobalCheckpointForShard); + public IndicesClusterStateService( + final Settings settings, + final IndicesService indicesService, + final ClusterService clusterService, + final ThreadPool threadPool, + final PeerRecoveryTargetService recoveryTargetService, + final ShardStateAction shardStateAction, + final NodeMappingRefreshAction nodeMappingRefreshAction, + final RepositoriesService repositoriesService, + final SearchService searchService, + final SyncedFlushService syncedFlushService, + final PeerRecoverySourceService peerRecoverySourceService, + final SnapshotShardsService snapshotShardsService, + final PrimaryReplicaSyncer primaryReplicaSyncer, + final GlobalCheckpointSyncAction globalCheckpointSyncAction) { + this( + settings, + (AllocatedIndices>) indicesService, + clusterService, + threadPool, + recoveryTargetService, + shardStateAction, + nodeMappingRefreshAction, + repositoriesService, + searchService, + syncedFlushService, + peerRecoverySourceService, + snapshotShardsService, + primaryReplicaSyncer, + globalCheckpointSyncAction::updateGlobalCheckpointForShard); } // for tests - IndicesClusterStateService(Settings settings, - AllocatedIndices> indicesService, - ClusterService clusterService, - ThreadPool threadPool, - PeerRecoveryTargetService recoveryTargetService, - ShardStateAction shardStateAction, - NodeMappingRefreshAction nodeMappingRefreshAction, - RepositoriesService repositoriesService, - SearchService searchService, - SyncedFlushService syncedFlushService, - PeerRecoverySourceService peerRecoverySourceService, - SnapshotShardsService snapshotShardsService, - PrimaryReplicaSyncer primaryReplicaSyncer, - Consumer globalCheckpointSyncer) { + IndicesClusterStateService( + final Settings settings, + final AllocatedIndices> indicesService, + final ClusterService clusterService, + final ThreadPool threadPool, + final PeerRecoveryTargetService recoveryTargetService, + final ShardStateAction shardStateAction, + final NodeMappingRefreshAction nodeMappingRefreshAction, + final RepositoriesService repositoriesService, + final SearchService searchService, + final SyncedFlushService syncedFlushService, + final PeerRecoverySourceService peerRecoverySourceService, + final SnapshotShardsService snapshotShardsService, + final PrimaryReplicaSyncer primaryReplicaSyncer, + final Consumer globalCheckpointSyncer) { super(settings); this.settings = settings; this.buildInIndexListener = @@ -557,8 +568,14 @@ private void createShard(DiscoveryNodes nodes, RoutingTable routingTable, ShardR try { logger.debug("{} creating shard", shardRouting.shardId()); RecoveryState recoveryState = new RecoveryState(shardRouting, nodes.getLocalNode(), sourceNode); - indicesService.createShard(shardRouting, recoveryState, recoveryTargetService, new RecoveryListener(shardRouting), - repositoriesService, failedShardHandler, globalCheckpointSyncer); + indicesService.createShard( + shardRouting, + recoveryState, + recoveryTargetService, + new RecoveryListener(shardRouting), + repositoriesService, + failedShardHandler, + globalCheckpointSyncer); } catch (Exception e) { failAndRemoveShard(shardRouting, true, "failed to create shard", e, state); } @@ -843,12 +860,26 @@ U createIndex(IndexMetaData indexMetaData, @Nullable U indexService(Index index); /** - * Creates shard for the specified shard routing and starts recovery, + * Creates a shard for the specified shard routing and starts recovery. + * + * @param shardRouting the shard routing + * @param recoveryState the recovery state + * @param recoveryTargetService recovery service for the target + * @param recoveryListener a callback when recovery changes state (finishes or fails) + * @param repositoriesService service responsible for snapshot/restore + * @param onShardFailure a callback when this shard fails + * @param globalCheckpointSyncer a callback when this shard syncs the global checkpoint + * @return a new shard + * @throws IOException if an I/O exception occurs when creating the shard */ - T createShard(ShardRouting shardRouting, RecoveryState recoveryState, PeerRecoveryTargetService recoveryTargetService, - PeerRecoveryTargetService.RecoveryListener recoveryListener, RepositoriesService repositoriesService, - Consumer onShardFailure, - Consumer globalCheckpointSyncer) throws IOException; + T createShard( + ShardRouting shardRouting, + RecoveryState recoveryState, + PeerRecoveryTargetService recoveryTargetService, + PeerRecoveryTargetService.RecoveryListener recoveryListener, + RepositoriesService repositoriesService, + Consumer onShardFailure, + Consumer globalCheckpointSyncer) throws IOException; /** * Returns shard for the specified id if it exists otherwise returns null. diff --git a/server/src/test/java/org/elasticsearch/index/shard/IndexShardIT.java b/server/src/test/java/org/elasticsearch/index/shard/IndexShardIT.java index 475caf06e30a8..1bcb8cd29104f 100644 --- a/server/src/test/java/org/elasticsearch/index/shard/IndexShardIT.java +++ b/server/src/test/java/org/elasticsearch/index/shard/IndexShardIT.java @@ -638,15 +638,31 @@ public static final IndexShard recoverShard(IndexShard newShard) throws IOExcept return newShard; } - public static final IndexShard newIndexShard(IndexService indexService, IndexShard shard, IndexSearcherWrapper wrapper, - CircuitBreakerService cbs, IndexingOperationListener... listeners) throws IOException { + public static final IndexShard newIndexShard( + final IndexService indexService, + final IndexShard shard,IndexSearcherWrapper wrapper, + final CircuitBreakerService cbs, + final IndexingOperationListener... listeners) throws IOException { ShardRouting initializingShardRouting = getInitializingShardRouting(shard.routingEntry()); - IndexShard newShard = new IndexShard(initializingShardRouting, indexService.getIndexSettings(), shard.shardPath(), - shard.store(), indexService.getIndexSortSupplier(), indexService.cache(), indexService.mapperService(), - indexService.similarityService(), shard.getEngineFactory(), indexService.getIndexEventListener(), wrapper, - indexService.getThreadPool(), indexService.getBigArrays(), null, Collections.emptyList(), Arrays.asList(listeners), - () -> {}, cbs); - return newShard; + return new IndexShard( + initializingShardRouting, + indexService.getIndexSettings(), + shard.shardPath(), + shard.store(), + indexService.getIndexSortSupplier(), + indexService.cache(), + indexService.mapperService(), + indexService.similarityService(), + shard.getEngineFactory(), + indexService.getIndexEventListener(), + wrapper, + indexService.getThreadPool(), + indexService.getBigArrays(), + null, + Collections.emptyList(), + Arrays.asList(listeners), + () -> {}, + cbs); } private static ShardRouting getInitializingShardRouting(ShardRouting existingShardRouting) { diff --git a/server/src/test/java/org/elasticsearch/indices/cluster/AbstractIndicesClusterStateServiceTestCase.java b/server/src/test/java/org/elasticsearch/indices/cluster/AbstractIndicesClusterStateServiceTestCase.java index c480c5719035d..b4c3d65115155 100644 --- a/server/src/test/java/org/elasticsearch/indices/cluster/AbstractIndicesClusterStateServiceTestCase.java +++ b/server/src/test/java/org/elasticsearch/indices/cluster/AbstractIndicesClusterStateServiceTestCase.java @@ -226,12 +226,14 @@ public MockIndexService indexService(Index index) { } @Override - public MockIndexShard createShard(ShardRouting shardRouting, RecoveryState recoveryState, - PeerRecoveryTargetService recoveryTargetService, - PeerRecoveryTargetService.RecoveryListener recoveryListener, - RepositoriesService repositoriesService, - Consumer onShardFailure, - Consumer globalCheckpointSyncer) throws IOException { + public MockIndexShard createShard( + final ShardRouting shardRouting, + final RecoveryState recoveryState, + final PeerRecoveryTargetService recoveryTargetService, + final PeerRecoveryTargetService.RecoveryListener recoveryListener, + final RepositoriesService repositoriesService, + final Consumer onShardFailure, + final Consumer globalCheckpointSyncer) throws IOException { failRandomly(); MockIndexService indexService = indexService(recoveryState.getShardId().getIndex()); MockIndexShard indexShard = indexService.createShard(shardRouting); diff --git a/server/src/test/java/org/elasticsearch/snapshots/SnapshotsServiceTests.java b/server/src/test/java/org/elasticsearch/snapshots/SnapshotsServiceTests.java index b4dbbef65cce2..5cd75df22a48a 100644 --- a/server/src/test/java/org/elasticsearch/snapshots/SnapshotsServiceTests.java +++ b/server/src/test/java/org/elasticsearch/snapshots/SnapshotsServiceTests.java @@ -467,28 +467,39 @@ protected void assertSnapshotOrGenericThread() { deterministicTaskQueue.getThreadPool() ); indicesClusterStateService = new IndicesClusterStateService( - settings, indicesService, clusterService, threadPool, - new PeerRecoveryTargetService( - deterministicTaskQueue.getThreadPool(), transportService, recoverySettings, - clusterService - ), - shardStateAction, - new NodeMappingRefreshAction(transportService, new MetaDataMappingService(clusterService, indicesService)), - repositoriesService, - mock(SearchService.class), - new SyncedFlushService(indicesService, clusterService, transportService, indexNameExpressionResolver), - new PeerRecoverySourceService(transportService, indicesService, recoverySettings), - snapshotShardsService, - new PrimaryReplicaSyncer( - transportService, - new TransportResyncReplicationAction( - settings, transportService, clusterService, indicesService, threadPool, - shardStateAction, actionFilters, indexNameExpressionResolver) - ), - new GlobalCheckpointSyncAction( - settings, transportService, clusterService, indicesService, threadPool, - shardStateAction, actionFilters, indexNameExpressionResolver) - ); + settings, + indicesService, + clusterService, + threadPool, + new PeerRecoveryTargetService( + deterministicTaskQueue.getThreadPool(), transportService, recoverySettings, clusterService), + shardStateAction, + new NodeMappingRefreshAction(transportService, new MetaDataMappingService(clusterService, indicesService)), + repositoriesService, + mock(SearchService.class), + new SyncedFlushService(indicesService, clusterService, transportService, indexNameExpressionResolver), + new PeerRecoverySourceService(transportService, indicesService, recoverySettings), + snapshotShardsService, + new PrimaryReplicaSyncer( + transportService, + new TransportResyncReplicationAction( + settings, + transportService, + clusterService, + indicesService, + threadPool, + shardStateAction, + actionFilters, + indexNameExpressionResolver)), + new GlobalCheckpointSyncAction( + settings, + transportService, + clusterService, + indicesService, + threadPool, + shardStateAction, + actionFilters, + indexNameExpressionResolver)); Map actions = new HashMap<>(); actions.put(CreateIndexAction.INSTANCE, new TransportCreateIndexAction( diff --git a/test/framework/src/main/java/org/elasticsearch/index/shard/IndexShardTestCase.java b/test/framework/src/main/java/org/elasticsearch/index/shard/IndexShardTestCase.java index 273967509f4a1..4938b307e7578 100644 --- a/test/framework/src/main/java/org/elasticsearch/index/shard/IndexShardTestCase.java +++ b/test/framework/src/main/java/org/elasticsearch/index/shard/IndexShardTestCase.java @@ -367,10 +367,25 @@ protected IndexShard newShard(ShardRouting routing, ShardPath shardPath, IndexMe }; ClusterSettings clusterSettings = new ClusterSettings(nodeSettings, ClusterSettings.BUILT_IN_CLUSTER_SETTINGS); CircuitBreakerService breakerService = new HierarchyCircuitBreakerService(nodeSettings, clusterSettings); - indexShard = new IndexShard(routing, indexSettings, shardPath, store, () -> null, indexCache, mapperService, similarityService, - engineFactory, indexEventListener, indexSearcherWrapper, threadPool, - BigArrays.NON_RECYCLING_INSTANCE, warmer, Collections.emptyList(), Arrays.asList(listeners), globalCheckpointSyncer, - breakerService); + indexShard = new IndexShard( + routing, + indexSettings, + shardPath, + store, + () -> null, + indexCache, + mapperService, + similarityService, + engineFactory, + indexEventListener, + indexSearcherWrapper, + threadPool, + BigArrays.NON_RECYCLING_INSTANCE, + warmer, + Collections.emptyList(), + Arrays.asList(listeners), + globalCheckpointSyncer, + breakerService); indexShard.addShardFailureCallback(DEFAULT_SHARD_FAILURE_HANDLER); success = true; } finally { From 43bfdd32eea161a9084d7b49b12261ea32a7983c Mon Sep 17 00:00:00 2001 From: Jason Tedor Date: Mon, 14 Jan 2019 21:54:42 -0500 Subject: [PATCH 162/186] Add run under primary permit method (#37440) This commit adds a simple method for executing a runnable against a shard under a primary permit. Today there is only a single caller for this method, but this there are two upcoming use-cases for which having this method will help keep the code simpler. --- .../org/elasticsearch/index/IndexService.java | 31 ++++----- .../elasticsearch/index/shard/IndexShard.java | 28 ++++++++ .../index/shard/IndexShardTests.java | 65 +++++++++++++++++++ 3 files changed, 105 insertions(+), 19 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/index/IndexService.java b/server/src/main/java/org/elasticsearch/index/IndexService.java index 6c01e61586d18..3eaad1eee5460 100644 --- a/server/src/main/java/org/elasticsearch/index/IndexService.java +++ b/server/src/main/java/org/elasticsearch/index/IndexService.java @@ -26,13 +26,11 @@ import org.apache.lucene.util.Accountable; import org.elasticsearch.Assertions; import org.elasticsearch.Version; -import org.elasticsearch.action.ActionListener; import org.elasticsearch.client.Client; import org.elasticsearch.cluster.metadata.IndexMetaData; import org.elasticsearch.cluster.routing.ShardRouting; import org.elasticsearch.common.Nullable; import org.elasticsearch.common.io.stream.NamedWriteableRegistry; -import org.elasticsearch.common.lease.Releasable; import org.elasticsearch.common.settings.Setting; import org.elasticsearch.common.settings.Setting.Property; import org.elasticsearch.common.settings.Settings; @@ -790,23 +788,18 @@ private void maybeSyncGlobalCheckpoints() { continue; case STARTED: try { - shard.acquirePrimaryOperationPermit( - ActionListener.wrap( - releasable -> { - try (Releasable ignored = releasable) { - shard.maybeSyncGlobalCheckpoint("background"); - } - }, - e -> { - if (!(e instanceof AlreadyClosedException || e instanceof IndexShardClosedException)) { - logger.info( - new ParameterizedMessage( - "{} failed to execute background global checkpoint sync", - shard.shardId()), - e); - } - }), - ThreadPool.Names.SAME, "background global checkpoint sync"); + shard.runUnderPrimaryPermit( + () -> shard.maybeSyncGlobalCheckpoint("background"), + e -> { + if (e instanceof AlreadyClosedException == false + && e instanceof IndexShardClosedException == false) { + logger.warn( + new ParameterizedMessage( + "{} failed to execute background global checkpoint sync", shard.shardId()), e); + } + }, + ThreadPool.Names.SAME, + "background global checkpoint sync"); } catch (final AlreadyClosedException | IndexShardClosedException e) { // the shard was closed concurrently, continue } diff --git a/server/src/main/java/org/elasticsearch/index/shard/IndexShard.java b/server/src/main/java/org/elasticsearch/index/shard/IndexShard.java index c76c845328bd7..8a10f4021d20f 100644 --- a/server/src/main/java/org/elasticsearch/index/shard/IndexShard.java +++ b/server/src/main/java/org/elasticsearch/index/shard/IndexShard.java @@ -2395,6 +2395,34 @@ private void asyncBlockOperations(ActionListener onPermitAcquired, l } } + /** + * Runs the specified runnable under a permit and otherwise calling back the specified failure callback. This method is really a + * convenience for {@link #acquirePrimaryOperationPermit(ActionListener, String, Object)} where the listener equates to + * try-with-resources closing the releasable after executing the runnable on successfully acquiring the permit, an otherwise calling + * back the failure callback. + * + * @param runnable the runnable to execute under permit + * @param onFailure the callback on failure + * @param executorOnDelay the executor to execute the runnable on if permit acquisition is blocked + * @param debugInfo debug info + */ + public void runUnderPrimaryPermit( + final Runnable runnable, + final Consumer onFailure, + final String executorOnDelay, + final Object debugInfo) { + verifyNotClosed(); + assert shardRouting.primary() : "runUnderPrimaryPermit should only be called on primary shard but was " + shardRouting; + final ActionListener onPermitAcquired = ActionListener.wrap( + releasable -> { + try (Releasable ignore = releasable) { + runnable.run(); + } + }, + onFailure); + acquirePrimaryOperationPermit(onPermitAcquired, executorOnDelay, debugInfo); + } + private void bumpPrimaryTerm(final long newPrimaryTerm, final CheckedRunnable onBlocked, @Nullable ActionListener combineWithAction) { diff --git a/server/src/test/java/org/elasticsearch/index/shard/IndexShardTests.java b/server/src/test/java/org/elasticsearch/index/shard/IndexShardTests.java index 5c7dd089534a2..9bb98da6e9667 100644 --- a/server/src/test/java/org/elasticsearch/index/shard/IndexShardTests.java +++ b/server/src/test/java/org/elasticsearch/index/shard/IndexShardTests.java @@ -147,6 +147,7 @@ import java.util.HashSet; import java.util.Iterator; import java.util.List; +import java.util.Locale; import java.util.Map; import java.util.Set; import java.util.concurrent.BrokenBarrierException; @@ -331,6 +332,70 @@ public void testClosesPreventsNewOperations() throws Exception { randomNonNegativeLong(), null, TimeValue.timeValueSeconds(30L))); } + public void testRunUnderPrimaryPermitRunsUnderPrimaryPermit() throws IOException { + final IndexShard indexShard = newStartedShard(true); + try { + assertThat(indexShard.getActiveOperationsCount(), equalTo(0)); + indexShard.runUnderPrimaryPermit( + () -> assertThat(indexShard.getActiveOperationsCount(), equalTo(1)), + e -> fail(e.toString()), + ThreadPool.Names.SAME, + "test"); + assertThat(indexShard.getActiveOperationsCount(), equalTo(0)); + } finally { + closeShards(indexShard); + } + } + + public void testRunUnderPrimaryPermitOnFailure() throws IOException { + final IndexShard indexShard = newStartedShard(true); + final AtomicBoolean invoked = new AtomicBoolean(); + try { + indexShard.runUnderPrimaryPermit( + () -> { + throw new RuntimeException("failure"); + }, + e -> { + assertThat(e, instanceOf(RuntimeException.class)); + assertThat(e.getMessage(), equalTo("failure")); + invoked.set(true); + }, + ThreadPool.Names.SAME, + "test"); + assertTrue(invoked.get()); + } finally { + closeShards(indexShard); + } + } + + public void testRunUnderPrimaryPermitDelaysToExecutorWhenBlocked() throws Exception { + final IndexShard indexShard = newStartedShard(true); + try { + final PlainActionFuture onAcquired = new PlainActionFuture<>(); + indexShard.acquireAllPrimaryOperationsPermits(onAcquired, new TimeValue(Long.MAX_VALUE, TimeUnit.NANOSECONDS)); + final Releasable permit = onAcquired.actionGet(); + final CountDownLatch latch = new CountDownLatch(1); + final String executorOnDelay = + randomFrom(ThreadPool.Names.FLUSH, ThreadPool.Names.GENERIC, ThreadPool.Names.MANAGEMENT, ThreadPool.Names.SAME); + indexShard.runUnderPrimaryPermit( + () -> { + final String expectedThreadPoolName = + executorOnDelay.equals(ThreadPool.Names.SAME) ? "generic" : executorOnDelay.toLowerCase(Locale.ROOT); + assertThat(Thread.currentThread().getName(), containsString(expectedThreadPoolName)); + latch.countDown(); + }, + e -> fail(e.toString()), + executorOnDelay, + "test"); + permit.close(); + latch.await(); + // we could race and assert on the count before the permit is returned + assertBusy(() -> assertThat(indexShard.getActiveOperationsCount(), equalTo(0))); + } finally { + closeShards(indexShard); + } + } + public void testRejectOperationPermitWithHigherTermWhenNotStarted() throws IOException { IndexShard indexShard = newShard(false); expectThrows(IndexShardNotStartedException.class, () -> From b97245cfcd927d2326710de34b66b6cfd4b018a8 Mon Sep 17 00:00:00 2001 From: Tim Vernum Date: Tue, 15 Jan 2019 18:20:22 +1100 Subject: [PATCH 163/186] Restore lost @Inject annotation (#37452) The Inject Annotation was removed from IndicesClusterStateService as part of reformatting in e11a32e, but this causes CreationException on cluster startup. --- .../indices/cluster/IndicesClusterStateService.java | 2 ++ 1 file changed, 2 insertions(+) diff --git a/server/src/main/java/org/elasticsearch/indices/cluster/IndicesClusterStateService.java b/server/src/main/java/org/elasticsearch/indices/cluster/IndicesClusterStateService.java index 337c66812da81..41050d7d47486 100644 --- a/server/src/main/java/org/elasticsearch/indices/cluster/IndicesClusterStateService.java +++ b/server/src/main/java/org/elasticsearch/indices/cluster/IndicesClusterStateService.java @@ -43,6 +43,7 @@ import org.elasticsearch.cluster.service.ClusterService; import org.elasticsearch.common.Nullable; import org.elasticsearch.common.component.AbstractLifecycleComponent; +import org.elasticsearch.common.inject.Inject; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.unit.TimeValue; import org.elasticsearch.common.util.concurrent.AbstractRunnable; @@ -121,6 +122,7 @@ public class IndicesClusterStateService extends AbstractLifecycleComponent imple private final PrimaryReplicaSyncer primaryReplicaSyncer; private final Consumer globalCheckpointSyncer; + @Inject public IndicesClusterStateService( final Settings settings, final IndicesService indicesService, From b594e81c86b1032cb79e59a277fe9de49458ca62 Mon Sep 17 00:00:00 2001 From: Marios Trivyzas Date: Tue, 15 Jan 2019 09:41:41 +0200 Subject: [PATCH 164/186] SQL: Fix issue with field names containing "." (#37364) Adjust FieldExtractor to handle fields which contain `.` in their name, regardless where they fall in, in the document hierarchy. E.g.: ``` { "a.b": "Elastic Search" } { "a": { "b.c": "Elastic Search" } } { "a.b": { "c": { "d.e" : "Elastic Search" } } } ``` Fixes: #37128 --- .../search/extractor/FieldHitExtractor.java | 52 +++++++-- .../extractor/FieldHitExtractorTests.java | 105 +++++++++++++++++- 2 files changed, 143 insertions(+), 14 deletions(-) diff --git a/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/execution/search/extractor/FieldHitExtractor.java b/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/execution/search/extractor/FieldHitExtractor.java index 07bf0f15b0bd3..7cce05652dfc0 100644 --- a/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/execution/search/extractor/FieldHitExtractor.java +++ b/x-pack/plugin/sql/src/main/java/org/elasticsearch/xpack/sql/execution/search/extractor/FieldHitExtractor.java @@ -6,6 +6,7 @@ package org.elasticsearch.xpack.sql.execution.search.extractor; import org.elasticsearch.common.Strings; +import org.elasticsearch.common.collect.Tuple; import org.elasticsearch.common.document.DocumentField; import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.io.stream.StreamOutput; @@ -16,9 +17,12 @@ import org.joda.time.DateTime; import java.io.IOException; +import java.util.ArrayDeque; +import java.util.Deque; import java.util.List; import java.util.Map; import java.util.Objects; +import java.util.StringJoiner; /** * Extractor for ES fields. Works for both 'normal' fields but also nested ones (which require hitName to be set). @@ -141,17 +145,43 @@ private Object unwrapMultiValue(Object values) { @SuppressWarnings("unchecked") Object extractFromSource(Map map) { - Object value = map; - boolean first = true; - // each node is a key inside the map - for (String node : path) { - if (value == null) { - return null; - } else if (first || value instanceof Map) { - first = false; - value = ((Map) value).get(node); - } else { - throw new SqlIllegalArgumentException("Cannot extract value [{}] from source", fieldName); + Object value = null; + + // Used to avoid recursive method calls + // Holds the sub-maps in the document hierarchy that are pending to be inspected. + // along with the current index of the `path`. + Deque>> queue = new ArrayDeque<>(); + queue.add(new Tuple<>(-1, map)); + + while (!queue.isEmpty()) { + Tuple> tuple = queue.removeLast(); + int idx = tuple.v1(); + Map subMap = tuple.v2(); + + // Find all possible entries by examining all combinations under the current level ("idx") of the "path" + // e.g.: If the path == "a.b.c.d" and the idx == 0, we need to check the current subMap against the keys: + // "b", "b.c" and "b.c.d" + StringJoiner sj = new StringJoiner("."); + for (int i = idx + 1; i < path.length; i++) { + sj.add(path[i]); + Object node = subMap.get(sj.toString()); + if (node instanceof Map) { + // Add the sub-map to the queue along with the current path index + queue.add(new Tuple<>(i, (Map) node)); + } else if (node != null) { + if (i < path.length - 1) { + // If we reach a concrete value without exhausting the full path, something is wrong with the mapping + // e.g.: map is {"a" : { "b" : "value }} and we are looking for a path: "a.b.c.d" + throw new SqlIllegalArgumentException("Cannot extract value [{}] from source", fieldName); + } + if (value != null) { + // A value has already been found so this means that there are more than one + // values in the document for the same path but different hierarchy. + // e.g.: {"a" : {"b" : {"c" : "value"}}}, {"a.b" : {"c" : "value"}}, ... + throw new SqlIllegalArgumentException("Multiple values (returned by [{}]) are not supported", fieldName); + } + value = node; + } } } return unwrapMultiValue(value); diff --git a/x-pack/plugin/sql/src/test/java/org/elasticsearch/xpack/sql/execution/search/extractor/FieldHitExtractorTests.java b/x-pack/plugin/sql/src/test/java/org/elasticsearch/xpack/sql/execution/search/extractor/FieldHitExtractorTests.java index 5c3478eaea343..4f562e82b5c21 100644 --- a/x-pack/plugin/sql/src/test/java/org/elasticsearch/xpack/sql/execution/search/extractor/FieldHitExtractorTests.java +++ b/x-pack/plugin/sql/src/test/java/org/elasticsearch/xpack/sql/execution/search/extractor/FieldHitExtractorTests.java @@ -21,8 +21,10 @@ import java.util.ArrayList; import java.util.Arrays; import java.util.Collections; +import java.util.HashMap; import java.util.List; import java.util.Map; +import java.util.StringJoiner; import java.util.function.Supplier; import static java.util.Arrays.asList; @@ -47,7 +49,7 @@ protected Reader instanceReader() { } @Override - protected FieldHitExtractor mutateInstance(FieldHitExtractor instance) throws IOException { + protected FieldHitExtractor mutateInstance(FieldHitExtractor instance) { return new FieldHitExtractor(instance.fieldName() + "mutated", null, true, instance.hitName()); } @@ -237,7 +239,104 @@ public void testMultiValuedSource() { assertThat(ex.getMessage(), is("Arrays (returned by [a]) are not supported")); } - public Object randomValue() { + public void testFieldWithDots() { + FieldHitExtractor fe = new FieldHitExtractor("a.b", null, false); + Object value = randomValue(); + Map map = singletonMap("a.b", value); + assertEquals(value, fe.extractFromSource(map)); + } + + public void testNestedFieldWithDots() { + FieldHitExtractor fe = new FieldHitExtractor("a.b.c", null, false); + Object value = randomValue(); + Map map = singletonMap("a", singletonMap("b.c", value)); + assertEquals(value, fe.extractFromSource(map)); + } + + public void testNestedFieldWithDotsWithNestedField() { + FieldHitExtractor fe = new FieldHitExtractor("a.b.c.d", null, false); + Object value = randomValue(); + Map map = singletonMap("a", singletonMap("b.c", singletonMap("d", value))); + assertEquals(value, fe.extractFromSource(map)); + } + + public void testNestedFieldWithDotsWithNestedFieldWithDots() { + FieldHitExtractor fe = new FieldHitExtractor("a.b.c.d.e", null, false); + Object value = randomValue(); + Map map = singletonMap("a", singletonMap("b.c", singletonMap("d.e", value))); + assertEquals(value, fe.extractFromSource(map)); + } + + public void testNestedFieldsWithDotsAndRandomHiearachy() { + String[] path = new String[100]; + StringJoiner sj = new StringJoiner("."); + for (int i = 0; i < 100; i++) { + path[i] = randomAlphaOfLength(randomIntBetween(1, 10)); + sj.add(path[i]); + } + FieldHitExtractor fe = new FieldHitExtractor(sj.toString(), null, false); + + List paths = new ArrayList<>(path.length); + int start = 0; + while (start < path.length) { + int end = randomIntBetween(start + 1, path.length); + sj = new StringJoiner("."); + for (int j = start; j < end; j++) { + sj.add(path[j]); + } + paths.add(sj.toString()); + start = end; + } + + Object value = randomValue(); + Map map = singletonMap(paths.get(paths.size() - 1), value); + for (int i = paths.size() - 2; i >= 0; i--) { + map = singletonMap(paths.get(i), map); + } + assertEquals(value, fe.extractFromSource(map)); + } + + public void testExtractSourceIncorrectPathWithFieldWithDots() { + FieldHitExtractor fe = new FieldHitExtractor("a.b.c.d.e", null, false); + Object value = randomNonNullValue(); + Map map = singletonMap("a", singletonMap("b.c", singletonMap("d", value))); + SqlException ex = expectThrows(SqlException.class, () -> fe.extractFromSource(map)); + assertThat(ex.getMessage(), is("Cannot extract value [a.b.c.d.e] from source")); + } + + public void testFieldWithDotsAndCommonPrefix() { + FieldHitExtractor fe1 = new FieldHitExtractor("a.d", null, false); + FieldHitExtractor fe2 = new FieldHitExtractor("a.b.c", null, false); + Object value = randomNonNullValue(); + Map map = new HashMap<>(); + map.put("a", singletonMap("d", value)); + map.put("a.b", singletonMap("c", value)); + assertEquals(value, fe1.extractFromSource(map)); + assertEquals(value, fe2.extractFromSource(map)); + } + + public void testFieldWithDotsAndCommonPrefixes() { + FieldHitExtractor fe1 = new FieldHitExtractor("a1.b.c.d1.e.f.g1", null, false); + FieldHitExtractor fe2 = new FieldHitExtractor("a2.b.c.d2.e.f.g2", null, false); + Object value = randomNonNullValue(); + Map map = new HashMap<>(); + map.put("a1", singletonMap("b.c", singletonMap("d1", singletonMap("e.f", singletonMap("g1", value))))); + map.put("a2", singletonMap("b.c", singletonMap("d2", singletonMap("e.f", singletonMap("g2", value))))); + assertEquals(value, fe1.extractFromSource(map)); + assertEquals(value, fe2.extractFromSource(map)); + } + + public void testFieldWithDotsAndSamePathButDifferentHierarchy() { + FieldHitExtractor fe = new FieldHitExtractor("a.b.c.d.e.f.g", null, false); + Object value = randomNonNullValue(); + Map map = new HashMap<>(); + map.put("a.b", singletonMap("c", singletonMap("d.e", singletonMap("f.g", value)))); + map.put("a", singletonMap("b.c", singletonMap("d.e", singletonMap("f", singletonMap("g", value))))); + SqlException ex = expectThrows(SqlException.class, () -> fe.extractFromSource(map)); + assertThat(ex.getMessage(), is("Multiple values (returned by [a.b.c.d.e.f.g]) are not supported")); + } + + private Object randomValue() { Supplier value = randomFrom(Arrays.asList( () -> randomAlphaOfLength(10), ESTestCase::randomLong, @@ -246,7 +345,7 @@ public Object randomValue() { return value.get(); } - public Object randomNonNullValue() { + private Object randomNonNullValue() { Supplier value = randomFrom(Arrays.asList( () -> randomAlphaOfLength(10), ESTestCase::randomLong, From bf49f54456e3b9ad2bb5bd5b739b0306a4f437b2 Mon Sep 17 00:00:00 2001 From: Nhat Nguyen Date: Tue, 15 Jan 2019 03:07:56 -0500 Subject: [PATCH 165/186] Simplify testSendSnapshotSendsOps (#37445) The test testSendSnapshotSendsOps is currently using a mock instance of RecoveryTargetHandler which will be hard to modify when we make the RecoveryTargetHandler non-blocking. This commit prepares for the incoming changes by replacing the mock instance with a stub. --- .../recovery/RecoverySourceHandlerTests.java | 103 ++++++++---------- 1 file changed, 48 insertions(+), 55 deletions(-) diff --git a/server/src/test/java/org/elasticsearch/indices/recovery/RecoverySourceHandlerTests.java b/server/src/test/java/org/elasticsearch/indices/recovery/RecoverySourceHandlerTests.java index 29eb1466056e8..1d58e0be65a2a 100644 --- a/server/src/test/java/org/elasticsearch/indices/recovery/RecoverySourceHandlerTests.java +++ b/server/src/test/java/org/elasticsearch/indices/recovery/RecoverySourceHandlerTests.java @@ -76,7 +76,6 @@ import org.elasticsearch.test.DummyShardLock; import org.elasticsearch.test.ESTestCase; import org.elasticsearch.test.IndexSettingsModule; -import org.mockito.ArgumentCaptor; import java.io.IOException; import java.io.OutputStream; @@ -108,7 +107,6 @@ import static org.mockito.Matchers.anyString; import static org.mockito.Mockito.doAnswer; import static org.mockito.Mockito.mock; -import static org.mockito.Mockito.verify; import static org.mockito.Mockito.when; public class RecoverySourceHandlerTests extends ESTestCase { @@ -205,9 +203,6 @@ public void testSendSnapshotSendsOps() throws IOException { final StartRecoveryRequest request = getStartRecoveryRequest(); final IndexShard shard = mock(IndexShard.class); when(shard.state()).thenReturn(IndexShardState.STARTED); - final RecoveryTargetHandler recoveryTarget = mock(RecoveryTargetHandler.class); - final RecoverySourceHandler handler = - new RecoverySourceHandler(shard, recoveryTarget, request, fileChunkSizeInBytes, between(1, 10)); final List operations = new ArrayList<>(); final int initialNumberOfDocs = randomIntBetween(16, 64); for (int i = 0; i < initialNumberOfDocs; i++) { @@ -219,38 +214,23 @@ public void testSendSnapshotSendsOps() throws IOException { final Engine.Index index = getIndex(Integer.toString(i)); operations.add(new Translog.Index(index, new Engine.IndexResult(1, 1, i - initialNumberOfDocs, true))); } - operations.add(null); final long startingSeqNo = randomIntBetween(0, numberOfDocsWithValidSequenceNumbers - 1); final long requiredStartingSeqNo = randomIntBetween((int) startingSeqNo, numberOfDocsWithValidSequenceNumbers - 1); final long endingSeqNo = randomIntBetween((int) requiredStartingSeqNo - 1, numberOfDocsWithValidSequenceNumbers - 1); - RecoverySourceHandler.SendSnapshotResult result = handler.phase2(startingSeqNo, requiredStartingSeqNo, - endingSeqNo, new Translog.Snapshot() { - @Override - public void close() { - - } - - private int counter = 0; - @Override - public int totalOperations() { - return operations.size() - 1; - } - - @Override - public Translog.Operation next() throws IOException { - return operations.get(counter++); - } - }, randomNonNegativeLong(), randomNonNegativeLong()); + final List shippedOps = new ArrayList<>(); + RecoveryTargetHandler recoveryTarget = new TestRecoveryTargetHandler() { + @Override + public long indexTranslogOperations(List operations, int totalTranslogOps, long timestamp, long msu) { + shippedOps.addAll(operations); + return SequenceNumbers.NO_OPS_PERFORMED; + } + }; + RecoverySourceHandler handler = new RecoverySourceHandler(shard, recoveryTarget, request, fileChunkSizeInBytes, between(1, 10)); + RecoverySourceHandler.SendSnapshotResult result = handler.phase2(startingSeqNo, requiredStartingSeqNo, + endingSeqNo, newTranslogSnapshot(operations, Collections.emptyList()), randomNonNegativeLong(), randomNonNegativeLong()); final int expectedOps = (int) (endingSeqNo - startingSeqNo + 1); assertThat(result.totalOperations, equalTo(expectedOps)); - final ArgumentCaptor shippedOpsCaptor = ArgumentCaptor.forClass(List.class); - verify(recoveryTarget).indexTranslogOperations(shippedOpsCaptor.capture(), ArgumentCaptor.forClass(Integer.class).capture(), - ArgumentCaptor.forClass(Long.class).capture(), ArgumentCaptor.forClass(Long.class).capture()); - List shippedOps = new ArrayList<>(); - for (List list: shippedOpsCaptor.getAllValues()) { - shippedOps.addAll(list); - } shippedOps.sort(Comparator.comparing(Translog.Operation::seqNo)); assertThat(shippedOps.size(), equalTo(expectedOps)); for (int i = 0; i < shippedOps.size(); i++) { @@ -261,30 +241,8 @@ public Translog.Operation next() throws IOException { List requiredOps = operations.subList(0, operations.size() - 1).stream() // remove last null marker .filter(o -> o.seqNo() >= requiredStartingSeqNo && o.seqNo() <= endingSeqNo).collect(Collectors.toList()); List opsToSkip = randomSubsetOf(randomIntBetween(1, requiredOps.size()), requiredOps); - expectThrows(IllegalStateException.class, () -> - handler.phase2(startingSeqNo, requiredStartingSeqNo, - endingSeqNo, new Translog.Snapshot() { - @Override - public void close() { - - } - - private int counter = 0; - - @Override - public int totalOperations() { - return operations.size() - 1 - opsToSkip.size(); - } - - @Override - public Translog.Operation next() throws IOException { - Translog.Operation op; - do { - op = operations.get(counter++); - } while (op != null && opsToSkip.contains(op)); - return op; - } - }, randomNonNegativeLong(), randomNonNegativeLong())); + expectThrows(IllegalStateException.class, () -> handler.phase2(startingSeqNo, requiredStartingSeqNo, + endingSeqNo, newTranslogSnapshot(operations, opsToSkip), randomNonNegativeLong(), randomNonNegativeLong())); } } @@ -716,4 +674,39 @@ public void writeFileChunk(StoreFileMetaData fileMetaData, long position, BytesR int totalTranslogOps, ActionListener listener) { } } + + private Translog.Snapshot newTranslogSnapshot(List operations, List operationsToSkip) { + return new Translog.Snapshot() { + int index = 0; + int skippedCount = 0; + + @Override + public int totalOperations() { + return operations.size(); + } + + @Override + public int skippedOperations() { + return skippedCount; + } + + @Override + public Translog.Operation next() { + while (index < operations.size()) { + Translog.Operation op = operations.get(index++); + if (operationsToSkip.contains(op)) { + skippedCount++; + } else { + return op; + } + } + return null; + } + + @Override + public void close() { + + } + }; + } } From 147c5e65d35f4397b9fbeef56a8104a8c80ec741 Mon Sep 17 00:00:00 2001 From: Simon Willnauer Date: Tue, 15 Jan 2019 09:39:53 +0100 Subject: [PATCH 166/186] Remove dead code from ShardSearchStats (#37421) The clear methodsa are unused and unsafe at this point. This commit removes the dead code. --- .../index/search/stats/ShardSearchStats.java | 51 +++++-------------- 1 file changed, 12 insertions(+), 39 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/index/search/stats/ShardSearchStats.java b/server/src/main/java/org/elasticsearch/index/search/stats/ShardSearchStats.java index 342b638db3104..0a19702e59526 100644 --- a/server/src/main/java/org/elasticsearch/index/search/stats/ShardSearchStats.java +++ b/server/src/main/java/org/elasticsearch/index/search/stats/ShardSearchStats.java @@ -122,27 +122,11 @@ public void onFetchPhase(SearchContext searchContext, long tookInNanos) { }); } - public void clear() { - totalStats.clear(); - synchronized (this) { - if (!groupsStats.isEmpty()) { - MapBuilder typesStatsBuilder = MapBuilder.newMapBuilder(); - for (Map.Entry typeStats : groupsStats.entrySet()) { - if (typeStats.getValue().totalCurrent() > 0) { - typeStats.getValue().clear(); - typesStatsBuilder.put(typeStats.getKey(), typeStats.getValue()); - } - } - groupsStats = typesStatsBuilder.immutableMap(); - } - } - } - private void computeStats(SearchContext searchContext, Consumer consumer) { consumer.accept(totalStats); if (searchContext.groupStats() != null) { - for (int i = 0; i < searchContext.groupStats().size(); i++) { - consumer.accept(groupStats(searchContext.groupStats().get(i))); + for (String group : searchContext.groupStats()) { + consumer.accept(groupStats(group)); } } } @@ -184,8 +168,8 @@ public void onFreeScrollContext(SearchContext context) { } static final class StatsHolder { - public final MeanMetric queryMetric = new MeanMetric(); - public final MeanMetric fetchMetric = new MeanMetric(); + final MeanMetric queryMetric = new MeanMetric(); + final MeanMetric fetchMetric = new MeanMetric(); /* We store scroll statistics in microseconds because with nanoseconds we run the risk of overflowing the total stats if there are * many scrolls. For example, on a system with 2^24 scrolls that have been executed, each executing for 2^10 seconds, then using * nanoseconds would require a numeric representation that can represent at least 2^24 * 2^10 * 10^9 > 2^24 * 2^10 * 2^29 = 2^63 @@ -193,14 +177,14 @@ static final class StatsHolder { * times as many scrolls (i.e., billions of scrolls which at one per second would take 32 years to occur), or scrolls that execute * for one-thousand times as long (i.e., scrolls that execute for almost twelve days on average). */ - public final MeanMetric scrollMetric = new MeanMetric(); - public final MeanMetric suggestMetric = new MeanMetric(); - public final CounterMetric queryCurrent = new CounterMetric(); - public final CounterMetric fetchCurrent = new CounterMetric(); - public final CounterMetric scrollCurrent = new CounterMetric(); - public final CounterMetric suggestCurrent = new CounterMetric(); - - public SearchStats.Stats stats() { + final MeanMetric scrollMetric = new MeanMetric(); + final MeanMetric suggestMetric = new MeanMetric(); + final CounterMetric queryCurrent = new CounterMetric(); + final CounterMetric fetchCurrent = new CounterMetric(); + final CounterMetric scrollCurrent = new CounterMetric(); + final CounterMetric suggestCurrent = new CounterMetric(); + + SearchStats.Stats stats() { return new SearchStats.Stats( queryMetric.count(), TimeUnit.NANOSECONDS.toMillis(queryMetric.sum()), queryCurrent.count(), fetchMetric.count(), TimeUnit.NANOSECONDS.toMillis(fetchMetric.sum()), fetchCurrent.count(), @@ -208,16 +192,5 @@ public SearchStats.Stats stats() { suggestMetric.count(), TimeUnit.NANOSECONDS.toMillis(suggestMetric.sum()), suggestCurrent.count() ); } - - public long totalCurrent() { - return queryCurrent.count() + fetchCurrent.count() + scrollCurrent.count() + suggestCurrent.count(); - } - - public void clear() { - queryMetric.clear(); - fetchMetric.clear(); - scrollMetric.clear(); - suggestMetric.clear(); - } } } From d6a104f52bc047274b93a9274933b77fef004bba Mon Sep 17 00:00:00 2001 From: Marios Trivyzas Date: Tue, 15 Jan 2019 11:51:45 +0200 Subject: [PATCH 167/186] [TEST] Muted testDifferentRolesMaintainPathOnRestart Relates to #37462 --- .../org/elasticsearch/test/test/InternalTestClusterTests.java | 1 + 1 file changed, 1 insertion(+) diff --git a/test/framework/src/test/java/org/elasticsearch/test/test/InternalTestClusterTests.java b/test/framework/src/test/java/org/elasticsearch/test/test/InternalTestClusterTests.java index 01768edc5b25c..38d76423eea9c 100644 --- a/test/framework/src/test/java/org/elasticsearch/test/test/InternalTestClusterTests.java +++ b/test/framework/src/test/java/org/elasticsearch/test/test/InternalTestClusterTests.java @@ -393,6 +393,7 @@ private Path[] getNodePaths(InternalTestCluster cluster, String name) { } } + @AwaitsFix(bugUrl = "https://github.com/elastic/elasticsearch/issues/37462") public void testDifferentRolesMaintainPathOnRestart() throws Exception { final Path baseDir = createTempDir(); final int numNodes = 5; From f7eb517eb8fcd2109e776ffb354ed24240683967 Mon Sep 17 00:00:00 2001 From: Boaz Leskes Date: Tue, 15 Jan 2019 11:43:09 +0100 Subject: [PATCH 168/186] Update the Flush API documentation (#33551) The semantics of the API changed considerably since the documentation was written. The main change is to remove references to memory reduction (this is related to refresh). Instead, flush refers to recovery times. I also removed the references to trimming the translog as the translog may be required for other purposes (operation history for ops based recovery and complement ongoing file based recoveries). Closes #32869 --- docs/reference/indices/flush.asciidoc | 11 ++++++----- 1 file changed, 6 insertions(+), 5 deletions(-) diff --git a/docs/reference/indices/flush.asciidoc b/docs/reference/indices/flush.asciidoc index a0027756ab57a..fdfcd80ecd463 100644 --- a/docs/reference/indices/flush.asciidoc +++ b/docs/reference/indices/flush.asciidoc @@ -2,11 +2,12 @@ == Flush The flush API allows to flush one or more indices through an API. The -flush process of an index basically frees memory from the index by -flushing data to the index storage and clearing the internal -<>. By -default, Elasticsearch uses memory heuristics in order to automatically -trigger flush operations as required in order to clear memory. +flush process of an index makes sure that any data that is currently only +persisted in the <> is also permanently +persisted in Lucene. This reduces recovery times as that data doesn't need to be +reindexed from the transaction logs after the Lucene indexed is opened. By +default, Elasticsearch uses heuristics in order to automatically +trigger flushes as required. It is rare for users to need to call the API directly. [source,js] -------------------------------------------------- From 23ae9808bac69f1f4356af152c55de2632fcfed2 Mon Sep 17 00:00:00 2001 From: Tanguy Leroux Date: Tue, 15 Jan 2019 12:48:21 +0100 Subject: [PATCH 169/186] Fix IndexShardTestCase.recoverReplica(IndexShard, IndexShard, boolean) (#37414) This commit fixes the IndexShardTestCase.recoverReplica(IndexShard, IndexShard, boolean) method where the startReplica parameter was not correctly propagated and the value true always used instead. --- .../java/org/elasticsearch/index/shard/IndexShardTestCase.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/test/framework/src/main/java/org/elasticsearch/index/shard/IndexShardTestCase.java b/test/framework/src/main/java/org/elasticsearch/index/shard/IndexShardTestCase.java index 4938b307e7578..10c9f399d4cbb 100644 --- a/test/framework/src/main/java/org/elasticsearch/index/shard/IndexShardTestCase.java +++ b/test/framework/src/main/java/org/elasticsearch/index/shard/IndexShardTestCase.java @@ -565,7 +565,7 @@ protected void recoverReplica(IndexShard replica, IndexShard primary, boolean st recoverReplica(replica, primary, (r, sourceNode) -> new RecoveryTarget(r, sourceNode, recoveryListener, version -> { }), - true, true); + true, startReplica); } /** recovers a replica from the given primary **/ From 19fc59f0892bce149a7f609483bb97e0e57fa98e Mon Sep 17 00:00:00 2001 From: Dimitrios Liappis Date: Tue, 15 Jan 2019 14:18:15 +0200 Subject: [PATCH 170/186] Fix line length for monitor and remove suppressions (#37456) Relates #34884 --- .../resources/checkstyle_suppressions.xml | 3 --- .../elasticsearch/monitor/jvm/GcNames.java | 12 ++++++---- .../elasticsearch/monitor/jvm/HotThreads.java | 6 +++-- .../jvm/JvmGcMonitorServiceSettingsTests.java | 24 ++++++++++++------- 4 files changed, 28 insertions(+), 17 deletions(-) diff --git a/buildSrc/src/main/resources/checkstyle_suppressions.xml b/buildSrc/src/main/resources/checkstyle_suppressions.xml index 370a6bbcac901..3da6c32db995c 100644 --- a/buildSrc/src/main/resources/checkstyle_suppressions.xml +++ b/buildSrc/src/main/resources/checkstyle_suppressions.xml @@ -48,11 +48,8 @@ - - - diff --git a/server/src/main/java/org/elasticsearch/monitor/jvm/GcNames.java b/server/src/main/java/org/elasticsearch/monitor/jvm/GcNames.java index 9f6a2d3f2e7db..702550fb4bdac 100644 --- a/server/src/main/java/org/elasticsearch/monitor/jvm/GcNames.java +++ b/server/src/main/java/org/elasticsearch/monitor/jvm/GcNames.java @@ -29,13 +29,16 @@ public class GcNames { * Resolves the GC type by its memory pool name ({@link java.lang.management.MemoryPoolMXBean#getName()}. */ public static String getByMemoryPoolName(String poolName, String defaultName) { - if ("Eden Space".equals(poolName) || "PS Eden Space".equals(poolName) || "Par Eden Space".equals(poolName) || "G1 Eden Space".equals(poolName)) { + if ("Eden Space".equals(poolName) || "PS Eden Space".equals(poolName) + || "Par Eden Space".equals(poolName) || "G1 Eden Space".equals(poolName)) { return YOUNG; } - if ("Survivor Space".equals(poolName) || "PS Survivor Space".equals(poolName) || "Par Survivor Space".equals(poolName) || "G1 Survivor Space".equals(poolName)) { + if ("Survivor Space".equals(poolName) || "PS Survivor Space".equals(poolName) + || "Par Survivor Space".equals(poolName) || "G1 Survivor Space".equals(poolName)) { return SURVIVOR; } - if ("Tenured Gen".equals(poolName) || "PS Old Gen".equals(poolName) || "CMS Old Gen".equals(poolName) || "G1 Old Gen".equals(poolName)) { + if ("Tenured Gen".equals(poolName) || "PS Old Gen".equals(poolName) + || "CMS Old Gen".equals(poolName) || "G1 Old Gen".equals(poolName)) { return OLD; } return defaultName; @@ -45,7 +48,8 @@ public static String getByGcName(String gcName, String defaultName) { if ("Copy".equals(gcName) || "PS Scavenge".equals(gcName) || "ParNew".equals(gcName) || "G1 Young Generation".equals(gcName)) { return YOUNG; } - if ("MarkSweepCompact".equals(gcName) || "PS MarkSweep".equals(gcName) || "ConcurrentMarkSweep".equals(gcName) || "G1 Old Generation".equals(gcName)) { + if ("MarkSweepCompact".equals(gcName) || "PS MarkSweep".equals(gcName) + || "ConcurrentMarkSweep".equals(gcName) || "G1 Old Generation".equals(gcName)) { return OLD; } return defaultName; diff --git a/server/src/main/java/org/elasticsearch/monitor/jvm/HotThreads.java b/server/src/main/java/org/elasticsearch/monitor/jvm/HotThreads.java index 9d7571c6eefc9..5b55c00875d47 100644 --- a/server/src/main/java/org/elasticsearch/monitor/jvm/HotThreads.java +++ b/server/src/main/java/org/elasticsearch/monitor/jvm/HotThreads.java @@ -234,7 +234,8 @@ private String innerDetect() throws Exception { continue; // thread is not alive yet or died before the first snapshot - ignore it! } double percent = (((double) time) / interval.nanos()) * 100; - sb.append(String.format(Locale.ROOT, "%n%4.1f%% (%s out of %s) %s usage by thread '%s'%n", percent, TimeValue.timeValueNanos(time), interval, type, threadName)); + sb.append(String.format(Locale.ROOT, "%n%4.1f%% (%s out of %s) %s usage by thread '%s'%n", + percent, TimeValue.timeValueNanos(time), interval, type, threadName)); // for each snapshot (2nd array index) find later snapshot for same thread with max number of // identical StackTraceElements (starting from end of each) boolean[] done = new boolean[threadElementsSnapshotCount]; @@ -267,7 +268,8 @@ private String innerDetect() throws Exception { sb.append(String.format(Locale.ROOT, " %s%n", show[l])); } } else { - sb.append(String.format(Locale.ROOT, " %d/%d snapshots sharing following %d elements%n", count, threadElementsSnapshotCount, maxSim)); + sb.append(String.format(Locale.ROOT, " %d/%d snapshots sharing following %d elements%n", + count, threadElementsSnapshotCount, maxSim)); for (int l = show.length - maxSim; l < show.length; l++) { sb.append(String.format(Locale.ROOT, " %s%n", show[l])); } diff --git a/server/src/test/java/org/elasticsearch/monitor/jvm/JvmGcMonitorServiceSettingsTests.java b/server/src/test/java/org/elasticsearch/monitor/jvm/JvmGcMonitorServiceSettingsTests.java index 4431f1ead80f3..b412aa5755d4f 100644 --- a/server/src/test/java/org/elasticsearch/monitor/jvm/JvmGcMonitorServiceSettingsTests.java +++ b/server/src/test/java/org/elasticsearch/monitor/jvm/JvmGcMonitorServiceSettingsTests.java @@ -65,7 +65,8 @@ public void testNegativeSetting() throws InterruptedException { Settings settings = Settings.builder().put("monitor.jvm.gc.collector." + collector + ".warn", "-" + randomTimeValue()).build(); execute(settings, (command, interval, name) -> null, e -> { assertThat(e, instanceOf(IllegalArgumentException.class)); - assertThat(e.getMessage(), allOf(containsString("invalid gc_threshold"), containsString("for [monitor.jvm.gc.collector." + collector + "."))); + assertThat(e.getMessage(), allOf(containsString("invalid gc_threshold"), + containsString("for [monitor.jvm.gc.collector." + collector + "."))); }, true, null); } @@ -78,8 +79,9 @@ public void testMissingSetting() throws InterruptedException { Settings.Builder builder = Settings.builder(); // drop a random setting or two - for (@SuppressWarnings("unchecked") AbstractMap.SimpleEntry entry : randomSubsetOf(randomIntBetween(1, 2), entries.toArray(new AbstractMap.SimpleEntry[0]))) { - builder.put(entry.getKey(), entry.getValue()); + for (@SuppressWarnings("unchecked") AbstractMap.SimpleEntry entry : randomSubsetOf(randomIntBetween(1, 2), + entries.toArray(new AbstractMap.SimpleEntry[0]))) { + builder.put(entry.getKey(), entry.getValue()); } // we should get an exception that a setting is missing @@ -115,25 +117,31 @@ public void testIllegalOverheadSettings() throws InterruptedException { infoWarnOutOfOrderBuilder.put("monitor.jvm.gc.overhead.warn", warn); execute(infoWarnOutOfOrderBuilder.build(), (command, interval, name) -> null, e -> { assertThat(e, instanceOf(IllegalArgumentException.class)); - assertThat(e.getMessage(), containsString("[monitor.jvm.gc.overhead.warn] must be greater than [monitor.jvm.gc.overhead.info] [" + info + "] but was [" + warn + "]")); + assertThat(e.getMessage(), containsString("[monitor.jvm.gc.overhead.warn] must be greater than " + + "[monitor.jvm.gc.overhead.info] [" + info + "] but was [" + warn + "]")); }, true, null); final Settings.Builder debugInfoOutOfOrderBuilder = Settings.builder(); debugInfoOutOfOrderBuilder.put("monitor.jvm.gc.overhead.info", info); final int debug = randomIntBetween(info + 1, 99); debugInfoOutOfOrderBuilder.put("monitor.jvm.gc.overhead.debug", debug); - debugInfoOutOfOrderBuilder.put("monitor.jvm.gc.overhead.warn", randomIntBetween(debug + 1, 100)); // or the test will fail for the wrong reason + debugInfoOutOfOrderBuilder.put("monitor.jvm.gc.overhead.warn", + randomIntBetween(debug + 1, 100)); // or the test will fail for the wrong reason execute(debugInfoOutOfOrderBuilder.build(), (command, interval, name) -> null, e -> { assertThat(e, instanceOf(IllegalArgumentException.class)); - assertThat(e.getMessage(), containsString("[monitor.jvm.gc.overhead.info] must be greater than [monitor.jvm.gc.overhead.debug] [" + debug + "] but was [" + info + "]")); + assertThat(e.getMessage(), containsString("[monitor.jvm.gc.overhead.info] must be greater than " + + "[monitor.jvm.gc.overhead.debug] [" + debug + "] but was [" + info + "]")); }, true, null); } - private static void execute(Settings settings, TriFunction scheduler, Runnable asserts) throws InterruptedException { + private static void execute(Settings settings, TriFunction scheduler, + Runnable asserts) throws InterruptedException { execute(settings, scheduler, null, false, asserts); } - private static void execute(Settings settings, TriFunction scheduler, Consumer consumer, boolean constructionShouldFail, Runnable asserts) throws InterruptedException { + private static void execute(Settings settings, TriFunction scheduler, + Consumer consumer, boolean constructionShouldFail, + Runnable asserts) throws InterruptedException { assert constructionShouldFail == (consumer != null); assert constructionShouldFail == (asserts == null); ThreadPool threadPool = null; From a88c050a05cc2727225497cbe2ce16f0c5a1766f Mon Sep 17 00:00:00 2001 From: Albert Zaharovits Date: Tue, 15 Jan 2019 14:29:32 +0200 Subject: [PATCH 171/186] Docs be explicit on how to turn off deprecated auditing (#37316) Just be explicit about turning off the deprecated audit log appender because we really want people to turn it off. --- .../security/auditing/output-logfile.asciidoc | 35 ++++++++++++++++--- 1 file changed, 30 insertions(+), 5 deletions(-) diff --git a/x-pack/docs/en/security/auditing/output-logfile.asciidoc b/x-pack/docs/en/security/auditing/output-logfile.asciidoc index 382307e893f6d..ac7128852b942 100644 --- a/x-pack/docs/en/security/auditing/output-logfile.asciidoc +++ b/x-pack/docs/en/security/auditing/output-logfile.asciidoc @@ -7,11 +7,36 @@ the `_audit.log` file in the logs directory. To maintain compatibility with releases prior to 6.5.0, a `_access.log` file is also generated. They differ in the output format but the contents are similar. For systems that are not ingesting the audit file for search or -analytics it is strongly recommended to only keep the newer format. -Turning off the deprecated output format can be achieved by disabling the logger -in the `log4j2.properties` file (hint: there is a config comment -about it). -For more information, see {ref}/logging.html#configuring-logging-levels[configuring-logging]. +analytics it is strongly recommended to keep only the newer format. + +To turn off the deprecated output format, you can disable the logger in the +`log4j2.properties` file: + +[source, properties] +-------------------------------------------------- +# change info to off +# logger.xpack_security_audit_deprecated_logfile.level = info +logger.xpack_security_audit_deprecated_logfile.level = off +-------------------------------------------------- + +Alternatively, use the +{ref}/cluster-update-settings.html[cluster update settings API] to dynamically +configure the logger: + +[source,js] +-------------------------------------------------- +PUT /_cluster/settings +{ + "persistent": { + "logger.org.elasticsearch.xpack.security.audit.logfile.DeprecatedLoggingAuditTrail": "off" + } +} +-------------------------------------------------- +// CONSOLE + +NOTE: If you overwrite the `log4j2.properties` and do not specify appenders for +any of the audit trails, audit events are forwarded to the root appender, which +by default points to the `elasticsearch.log` file. [float] From e848388865e8936b5befd4586a910dbb50d3a307 Mon Sep 17 00:00:00 2001 From: Tanguy Leroux Date: Tue, 15 Jan 2019 15:03:09 +0100 Subject: [PATCH 172/186] Fix SourceOnlySnapshotIT (#37461) The SourceOnlySnapshotIT class tests a source only repository using the following scenario: starts a master node starts a data node creates a source only repository creates an index with documents snapshots the index to the source only repository deletes the index stops the data node starts a new data node restores the index Thanks to ESIntegTestCase the index is sometimes created using a custom data path. With such a setting, when a shard is assigned to one of the data node of the cluster the shard path is resolved using the index custom data path and the node's lock id by the NodeEnvironment#resolveCustomLocation(). It should work nicely but in SourceOnlySnapshotIT.snashotAndRestore(), b efore the change in this PR, the last data node was restarted using a different path.home. At startup time this node was assigned a node lock based on other locks in the data directory of this temporary path.home which is empty. So it always got the 0 lock id. And when this new data node is assigned a shard for the index and resolves it against the index custom data path, it also uses the node lock id 0 which conflicts with another node of the cluster, resulting in various errors with the most obvious one being LockObtainFailedException. This commit removes the temporary home path for the last data node so that it uses the same path home as other nodes of the cluster and then got assigned a correct node lock id at startup. Closes #36330 Closes #36276 --- .../snapshots/SourceOnlySnapshotIT.java | 27 ++++++------------- 1 file changed, 8 insertions(+), 19 deletions(-) diff --git a/x-pack/plugin/core/src/test/java/org/elasticsearch/snapshots/SourceOnlySnapshotIT.java b/x-pack/plugin/core/src/test/java/org/elasticsearch/snapshots/SourceOnlySnapshotIT.java index a025b07a8eb6d..00b199eef4419 100644 --- a/x-pack/plugin/core/src/test/java/org/elasticsearch/snapshots/SourceOnlySnapshotIT.java +++ b/x-pack/plugin/core/src/test/java/org/elasticsearch/snapshots/SourceOnlySnapshotIT.java @@ -28,7 +28,6 @@ import org.elasticsearch.index.engine.EngineFactory; import org.elasticsearch.index.mapper.SeqNoFieldMapper; import org.elasticsearch.index.query.QueryBuilders; -import org.elasticsearch.node.Node; import org.elasticsearch.plugins.EnginePlugin; import org.elasticsearch.plugins.Plugin; import org.elasticsearch.plugins.RepositoryPlugin; @@ -48,13 +47,13 @@ import java.util.List; import java.util.Map; import java.util.Optional; -import java.util.concurrent.ExecutionException; import java.util.function.BiConsumer; import static org.elasticsearch.common.xcontent.XContentFactory.jsonBuilder; import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertAcked; import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertHitCount; +@ESIntegTestCase.ClusterScope(numDataNodes = 0) public class SourceOnlySnapshotIT extends ESIntegTestCase { @Override @@ -92,12 +91,6 @@ public List> getSettings() { } } - public void testToStopSuiteFailing() { - // This is required because otherwise every test in the suite is muted - // TODO remove this when one of the other tests is fixed - } - - @AwaitsFix(bugUrl = "https://github.com/elastic/elasticsearch/issues/36330") public void testSnapshotAndRestore() throws Exception { final String sourceIdx = "test-idx"; boolean requireRouting = randomBoolean(); @@ -128,7 +121,6 @@ public void testSnapshotAndRestore() throws Exception { assertHits(sourceIdx, builders.length, sourceHadDeletions); } - @AwaitsFix(bugUrl = "https://github.com/elastic/elasticsearch/issues/36276") public void testSnapshotAndRestoreWithNested() throws Exception { final String sourceIdx = "test-idx"; boolean requireRouting = randomBoolean(); @@ -215,12 +207,13 @@ private void assertHits(String index, int numDocsExpected, boolean sourceHadDele client().prepareClearScroll().addScrollId(searchResponse.getScrollId()).get(); } } - } - private IndexRequestBuilder[] snashotAndRestore(String sourceIdx, int numShards, boolean minimal, boolean requireRouting, boolean - useNested) - throws ExecutionException, InterruptedException, IOException { + private IndexRequestBuilder[] snashotAndRestore(final String sourceIdx, + final int numShards, + final boolean minimal, + final boolean requireRouting, + final boolean useNested) throws InterruptedException, IOException { logger.info("--> starting a master node and a data node"); internalCluster().startMasterOnlyNode(); internalCluster().startDataOnlyNode(); @@ -284,12 +277,8 @@ private IndexRequestBuilder[] snashotAndRestore(String sourceIdx, int numShards, internalCluster().stopRandomDataNode(); client().admin().cluster().prepareHealth().setTimeout("30s").setWaitForNodes("1"); - logger.info("--> start a new data node"); - final Settings dataSettings = Settings.builder() - .put(Node.NODE_NAME_SETTING.getKey(), randomAlphaOfLength(5)) - .put(Environment.PATH_HOME_SETTING.getKey(), createTempDir()) // to get a new node id - .build(); - internalCluster().startDataOnlyNode(dataSettings); + final String newDataNode = internalCluster().startDataOnlyNode(); + logger.info("--> start a new data node " + newDataNode); client().admin().cluster().prepareHealth().setTimeout("30s").setWaitForNodes("2"); logger.info("--> restore the index and ensure all shards are allocated"); From 7cdf7f882b0debd42964a0d4300406cd5fc31297 Mon Sep 17 00:00:00 2001 From: David Roberts Date: Tue, 15 Jan 2019 14:19:05 +0000 Subject: [PATCH 173/186] [ML] Fix ML datafeed CCS with wildcarded cluster name (#37470) The test that remote clusters used by ML datafeeds have a license that allows ML was not accounting for the possibility that the remote cluster name could be wildcarded. This change fixes that omission. Fixes #36228 --- .../transport/RemoteClusterService.java | 8 +++++ .../license/RemoteClusterLicenseChecker.java | 13 +++++-- .../RemoteClusterLicenseCheckerTests.java | 35 ++++++++++++++++--- .../action/TransportStartDatafeedAction.java | 7 ++-- 4 files changed, 54 insertions(+), 9 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/transport/RemoteClusterService.java b/server/src/main/java/org/elasticsearch/transport/RemoteClusterService.java index ff9dea8fe45a7..d9fcb01df4ce8 100644 --- a/server/src/main/java/org/elasticsearch/transport/RemoteClusterService.java +++ b/server/src/main/java/org/elasticsearch/transport/RemoteClusterService.java @@ -303,6 +303,14 @@ boolean isRemoteClusterRegistered(String clusterName) { return remoteClusters.containsKey(clusterName); } + /** + * Returns the registered remote cluster names. + */ + public Set getRegisteredRemoteClusterNames() { + // remoteClusters is unmodifiable so its key set will be unmodifiable too + return remoteClusters.keySet(); + } + public void collectSearchShards(IndicesOptions indicesOptions, String preference, String routing, Map remoteIndicesByCluster, ActionListener> listener) { diff --git a/x-pack/plugin/core/src/main/java/org/elasticsearch/license/RemoteClusterLicenseChecker.java b/x-pack/plugin/core/src/main/java/org/elasticsearch/license/RemoteClusterLicenseChecker.java index b50d38c73c827..7d5a3b5e9a53d 100644 --- a/x-pack/plugin/core/src/main/java/org/elasticsearch/license/RemoteClusterLicenseChecker.java +++ b/x-pack/plugin/core/src/main/java/org/elasticsearch/license/RemoteClusterLicenseChecker.java @@ -11,6 +11,7 @@ import org.elasticsearch.action.ActionListener; import org.elasticsearch.action.support.ContextPreservingActionListener; import org.elasticsearch.client.Client; +import org.elasticsearch.cluster.metadata.ClusterNameExpressionResolver; import org.elasticsearch.common.util.concurrent.ThreadContext; import org.elasticsearch.protocol.xpack.XPackInfoRequest; import org.elasticsearch.protocol.xpack.XPackInfoResponse; @@ -22,6 +23,7 @@ import java.util.Iterator; import java.util.List; import java.util.Locale; +import java.util.Set; import java.util.concurrent.atomic.AtomicReference; import java.util.function.Predicate; import java.util.stream.Collectors; @@ -119,6 +121,7 @@ private LicenseCheck(final RemoteClusterLicenseInfo remoteClusterLicenseInfo) { } + private static final ClusterNameExpressionResolver clusterNameExpressionResolver = new ClusterNameExpressionResolver(); private final Client client; private final Predicate predicate; @@ -243,16 +246,20 @@ public static List remoteIndices(final List indices) { /** * Extract the list of remote cluster aliases from the list of index names. Remote index names are of the form - * {@code cluster_alias:index_name} and the cluster_alias is extracted for each index name that represents a remote index. + * {@code cluster_alias:index_name} and the cluster_alias is extracted (and expanded if it is a wildcard) for + * each index name that represents a remote index. * - * @param indices the collection of index names + * @param remoteClusters the aliases for remote clusters + * @param indices the collection of index names * @return the remote cluster names */ - public static List remoteClusterAliases(final List indices) { + public static List remoteClusterAliases(final Set remoteClusters, final List indices) { return indices.stream() .filter(RemoteClusterLicenseChecker::isRemoteIndex) .map(index -> index.substring(0, index.indexOf(RemoteClusterAware.REMOTE_CLUSTER_INDEX_SEPARATOR))) .distinct() + .flatMap(clusterExpression -> clusterNameExpressionResolver.resolveClusterNames(remoteClusters, clusterExpression).stream()) + .distinct() .collect(Collectors.toList()); } diff --git a/x-pack/plugin/core/src/test/java/org/elasticsearch/license/RemoteClusterLicenseCheckerTests.java b/x-pack/plugin/core/src/test/java/org/elasticsearch/license/RemoteClusterLicenseCheckerTests.java index 24a6960da4833..0f512a69f92d1 100644 --- a/x-pack/plugin/core/src/test/java/org/elasticsearch/license/RemoteClusterLicenseCheckerTests.java +++ b/x-pack/plugin/core/src/test/java/org/elasticsearch/license/RemoteClusterLicenseCheckerTests.java @@ -12,6 +12,7 @@ import org.elasticsearch.client.Client; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.util.concurrent.ThreadContext; +import org.elasticsearch.common.util.set.Sets; import org.elasticsearch.protocol.xpack.XPackInfoResponse; import org.elasticsearch.protocol.xpack.license.LicenseStatus; import org.elasticsearch.test.ESTestCase; @@ -23,6 +24,7 @@ import java.util.Arrays; import java.util.Collections; import java.util.List; +import java.util.Set; import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicReference; @@ -80,24 +82,49 @@ public void testRemoteIndices() { } public void testNoRemoteClusterAliases() { + final Set remoteClusters = Sets.newHashSet("remote-cluster1", "remote-cluster2"); final List indices = Arrays.asList("local-index1", "local-index2"); - assertThat(RemoteClusterLicenseChecker.remoteClusterAliases(indices), empty()); + assertThat(RemoteClusterLicenseChecker.remoteClusterAliases(remoteClusters, indices), empty()); } public void testOneRemoteClusterAlias() { + final Set remoteClusters = Sets.newHashSet("remote-cluster1", "remote-cluster2"); final List indices = Arrays.asList("local-index1", "remote-cluster1:remote-index1"); - assertThat(RemoteClusterLicenseChecker.remoteClusterAliases(indices), contains("remote-cluster1")); + assertThat(RemoteClusterLicenseChecker.remoteClusterAliases(remoteClusters, indices), contains("remote-cluster1")); } public void testMoreThanOneRemoteClusterAlias() { + final Set remoteClusters = Sets.newHashSet("remote-cluster1", "remote-cluster2"); final List indices = Arrays.asList("remote-cluster1:remote-index1", "local-index1", "remote-cluster2:remote-index1"); - assertThat(RemoteClusterLicenseChecker.remoteClusterAliases(indices), contains("remote-cluster1", "remote-cluster2")); + assertThat(RemoteClusterLicenseChecker.remoteClusterAliases(remoteClusters, indices), + containsInAnyOrder("remote-cluster1", "remote-cluster2")); } public void testDuplicateRemoteClusterAlias() { + final Set remoteClusters = Sets.newHashSet("remote-cluster1", "remote-cluster2"); final List indices = Arrays.asList( "remote-cluster1:remote-index1", "local-index1", "remote-cluster2:index1", "remote-cluster2:remote-index2"); - assertThat(RemoteClusterLicenseChecker.remoteClusterAliases(indices), contains("remote-cluster1", "remote-cluster2")); + assertThat(RemoteClusterLicenseChecker.remoteClusterAliases(remoteClusters, indices), + containsInAnyOrder("remote-cluster1", "remote-cluster2")); + } + + public void testSimpleWildcardRemoteClusterAlias() { + final Set remoteClusters = Sets.newHashSet("remote-cluster1", "remote-cluster2"); + final List indices = Arrays.asList("*:remote-index1", "local-index1"); + assertThat(RemoteClusterLicenseChecker.remoteClusterAliases(remoteClusters, indices), + containsInAnyOrder("remote-cluster1", "remote-cluster2")); + } + + public void testPartialWildcardRemoteClusterAlias() { + final Set remoteClusters = Sets.newHashSet("remote-cluster1", "remote-cluster2"); + final List indices = Arrays.asList("*2:remote-index1", "local-index1"); + assertThat(RemoteClusterLicenseChecker.remoteClusterAliases(remoteClusters, indices), contains("remote-cluster2")); + } + + public void testNonMatchingWildcardRemoteClusterAlias() { + final Set remoteClusters = Sets.newHashSet("remote-cluster1", "remote-cluster2"); + final List indices = Arrays.asList("*3:remote-index1", "local-index1"); + assertThat(RemoteClusterLicenseChecker.remoteClusterAliases(remoteClusters, indices), empty()); } public void testCheckRemoteClusterLicensesGivenCompatibleLicenses() { diff --git a/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/action/TransportStartDatafeedAction.java b/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/action/TransportStartDatafeedAction.java index 5867948bbad63..f81fcfbfb1d2e 100644 --- a/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/action/TransportStartDatafeedAction.java +++ b/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/action/TransportStartDatafeedAction.java @@ -174,7 +174,9 @@ public void onFailure(Exception e) { final RemoteClusterLicenseChecker remoteClusterLicenseChecker = new RemoteClusterLicenseChecker(client, XPackLicenseState::isMachineLearningAllowedForOperationMode); remoteClusterLicenseChecker.checkRemoteClusterLicenses( - RemoteClusterLicenseChecker.remoteClusterAliases(params.getDatafeedIndices()), + RemoteClusterLicenseChecker.remoteClusterAliases( + transportService.getRemoteClusterService().getRegisteredRemoteClusterNames(), + params.getDatafeedIndices()), ActionListener.wrap( response -> { if (response.isSuccess() == false) { @@ -311,7 +313,8 @@ private ElasticsearchStatusException createUnlicensedError( private ElasticsearchStatusException createUnknownLicenseError( final String datafeedId, final List remoteIndices, final Exception cause) { - final int numberOfRemoteClusters = RemoteClusterLicenseChecker.remoteClusterAliases(remoteIndices).size(); + final int numberOfRemoteClusters = RemoteClusterLicenseChecker.remoteClusterAliases( + transportService.getRemoteClusterService().getRegisteredRemoteClusterNames(), remoteIndices).size(); assert numberOfRemoteClusters > 0; final String remoteClusterQualifier = numberOfRemoteClusters == 1 ? "a remote cluster" : "remote clusters"; final String licenseTypeQualifier = numberOfRemoteClusters == 1 ? "" : "s"; From a56aa4f076959f834c226c911996053f95a955a6 Mon Sep 17 00:00:00 2001 From: Jay Modi Date: Tue, 15 Jan 2019 07:52:58 -0700 Subject: [PATCH 174/186] Remove SslNullCipherTests from codebase (#37431) This change deletes the SslNullCipherTests from our codebase since it will have issues with newer JDK versions and it is essentially testing JDK functionality rather than our own. The upstream JDK issue for disabling these ciphers by default is https://bugs.openjdk.java.net/browse/JDK-8212823. Closes #37403 --- .../transport/ssl/SslNullCipherTests.java | 61 ------------------- 1 file changed, 61 deletions(-) delete mode 100644 x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/transport/ssl/SslNullCipherTests.java diff --git a/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/transport/ssl/SslNullCipherTests.java b/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/transport/ssl/SslNullCipherTests.java deleted file mode 100644 index f57be31632868..0000000000000 --- a/x-pack/plugin/security/src/test/java/org/elasticsearch/xpack/security/transport/ssl/SslNullCipherTests.java +++ /dev/null @@ -1,61 +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.security.transport.ssl; - -import org.elasticsearch.action.DocWriteResponse.Result; -import org.elasticsearch.action.index.IndexResponse; -import org.elasticsearch.bootstrap.JavaVersion; -import org.elasticsearch.client.Client; -import org.elasticsearch.common.settings.Settings; -import org.elasticsearch.test.SecurityIntegTestCase; -import org.junit.BeforeClass; - -/** - * An extremely simple test that shows SSL will work with a cipher that does not perform encryption - */ -public class SslNullCipherTests extends SecurityIntegTestCase { - - @BeforeClass - public static void muteInFips() { - assumeFalse("Can't run in a FIPS JVM", inFipsJvm()); - } - - @BeforeClass - public static void muteInJDK12() { - assumeFalse("https://github.com/elastic/elasticsearch/issues/37403", JavaVersion.current().equals(JavaVersion.parse("12"))); - } - - @Override - public boolean transportSSLEnabled() { - return true; - } - - @Override - public Settings nodeSettings(int nodeOrdinal) { - Settings settings = super.nodeSettings(nodeOrdinal); - Settings.Builder builder = Settings.builder() - .put(settings); - builder.put("xpack.security.transport.ssl.cipher_suites", "TLS_RSA_WITH_NULL_SHA256"); - return builder.build(); - } - - @Override - public Settings transportClientSettings() { - Settings settings = super.transportClientSettings(); - Settings.Builder builder = Settings.builder() - .put(settings); - - builder.put("xpack.security.transport.ssl.cipher_suites", "TLS_RSA_WITH_NULL_SHA256"); - return builder.build(); - } - - public void testClusterIsFormed() { - ensureGreen(); - Client client = internalCluster().transportClient(); - IndexResponse response = client.prepareIndex("index", "type").setSource("foo", "bar").get(); - assertEquals(Result.CREATED, response.getResult()); - } -} From e71542ffd3e31335c8fc315d6f31e82d29ead4df Mon Sep 17 00:00:00 2001 From: niloct Date: Tue, 15 Jan 2019 13:04:32 -0200 Subject: [PATCH 175/186] Update delete-by-query.asciidoc (#37370) Tried my best to clarify sentence on `_delete_by_query` docs. --- docs/reference/docs/delete-by-query.asciidoc | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/docs/reference/docs/delete-by-query.asciidoc b/docs/reference/docs/delete-by-query.asciidoc index d883212a331cb..307d762abe79e 100644 --- a/docs/reference/docs/delete-by-query.asciidoc +++ b/docs/reference/docs/delete-by-query.asciidoc @@ -383,8 +383,9 @@ POST _tasks/r1A2WoRbTwKZ516z6NEs5A:36619/_cancel The task ID can be found using the <>. -Cancellation should happen quickly but might take a few seconds. The task status -API above will continue to list the task until it is wakes to cancel itself. +Cancellation should happen quickly but might take a few seconds. The task status +API above will continue to list the delete by query task until this task checks that it +has been cancelled and terminates itself. [float] From 3cc8f39532dac9768704d07c7611e6b1a162dbdc Mon Sep 17 00:00:00 2001 From: Fabricio Archanjo Fonseca Date: Tue, 15 Jan 2019 13:06:32 -0200 Subject: [PATCH 176/186] New mapping signature and mapping string source fixed. (#37401) * New mapping signature and mapping string source fixed. * Keep compatibility with CreateIndexRequest class. --- .../admin/indices/rollover/RolloverRequestBuilder.java | 8 +++++++- 1 file changed, 7 insertions(+), 1 deletion(-) diff --git a/server/src/main/java/org/elasticsearch/action/admin/indices/rollover/RolloverRequestBuilder.java b/server/src/main/java/org/elasticsearch/action/admin/indices/rollover/RolloverRequestBuilder.java index be331547b2a20..85867b11ad618 100644 --- a/server/src/main/java/org/elasticsearch/action/admin/indices/rollover/RolloverRequestBuilder.java +++ b/server/src/main/java/org/elasticsearch/action/admin/indices/rollover/RolloverRequestBuilder.java @@ -25,6 +25,7 @@ import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.unit.ByteSizeValue; import org.elasticsearch.common.unit.TimeValue; +import org.elasticsearch.common.xcontent.XContentType; public class RolloverRequestBuilder extends MasterNodeOperationRequestBuilder Date: Tue, 15 Jan 2019 16:24:19 +0100 Subject: [PATCH 177/186] When removing an AutoFollower also mark it as removed. (#37402) Currently when there are no more auto follow patterns for a remote cluster then the AutoFollower instance for this remote cluster will be removed. If a new auto follow pattern for this remote cluster gets added quickly enough after the last delete then there may be two AutoFollower instance running for this remote cluster instead of one. Each AutoFollower instance stops automatically after it sees in the start() method that there are no more auto follow patterns for the remote cluster it is tracking. However when an auto follow pattern gets removed and then added back quickly enough then old AutoFollower may never detect that at some point there were no auto follow patterns for the remote cluster it is monitoring. The creation and removal of an AutoFollower instance happens independently in the `updateAutoFollowers()` as part of a cluster state update. By adding the `removed` field, an AutoFollower instance will not miss the fact there were no auto follow patterns at some point in time. The `updateAutoFollowers()` method now marks an AutoFollower instance as removed when it sees that there are no more patterns for a remote cluster. The updateAutoFollowers() method can then safely start a new AutoFollower instance. Relates to #36761 --- .../ccr/action/AutoFollowCoordinator.java | 33 +++++++++++++++++++ .../elasticsearch/xpack/ccr/AutoFollowIT.java | 7 +++- .../action/AutoFollowCoordinatorTests.java | 13 ++++++++ 3 files changed, 52 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 55e24abc86c26..e3b008efc5657 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 @@ -258,6 +258,7 @@ public void clusterStateProcessed(String source, ClusterState oldState, ClusterS .anyMatch(pattern -> pattern.getRemoteCluster().equals(remoteCluster)); if (exist == false) { LOGGER.info("removing auto follower for remote cluster [{}]", remoteCluster); + autoFollower.removed = true; removedRemoteClusters.add(remoteCluster); } else if (autoFollower.remoteClusterConnectionMissing) { LOGGER.info("retrying auto follower [{}] after remote cluster connection was missing", remoteCluster); @@ -265,11 +266,25 @@ public void clusterStateProcessed(String source, ClusterState oldState, ClusterS autoFollower.start(); } } + assert assertNoOtherActiveAutoFollower(newAutoFollowers); this.autoFollowers = autoFollowers .copyAndPutAll(newAutoFollowers) .copyAndRemoveAll(removedRemoteClusters); } + private boolean assertNoOtherActiveAutoFollower(Map newAutoFollowers) { + for (AutoFollower newAutoFollower : newAutoFollowers.values()) { + AutoFollower previousInstance = autoFollowers.get(newAutoFollower.remoteCluster); + assert previousInstance == null || previousInstance.removed; + } + return true; + } + + + Map getAutoFollowers() { + return autoFollowers; + } + @Override public void clusterChanged(ClusterChangedEvent event) { if (event.localNodeMaster()) { @@ -295,6 +310,7 @@ abstract static class AutoFollower { private volatile long lastAutoFollowTimeInMillis = -1; private volatile long metadataVersion = 0; private volatile boolean remoteClusterConnectionMissing = false; + volatile boolean removed = false; private volatile CountDown autoFollowPatternsCountDown; private volatile AtomicArray autoFollowResults; @@ -309,6 +325,17 @@ abstract static class AutoFollower { } void start() { + if (removed) { + // This check exists to avoid two AutoFollower instances a single remote cluster. + // (If an auto follow pattern is deleted and then added back quickly enough then + // the old AutoFollower instance still sees that there is an auto follow pattern + // for the remote cluster it is tracking and will continue to operate, while in + // the meantime in updateAutoFollowers() method another AutoFollower instance has been + // started for the same remote cluster.) + LOGGER.info("AutoFollower instance for cluster [{}] has been removed", remoteCluster); + return; + } + lastAutoFollowTimeInMillis = relativeTimeProvider.getAsLong(); final ClusterState clusterState = followerClusterStateSupplier.get(); final AutoFollowMetadata autoFollowMetadata = clusterState.metaData().custom(AutoFollowMetadata.TYPE); @@ -330,6 +357,12 @@ void start() { this.autoFollowResults = new AtomicArray<>(patterns.size()); getRemoteClusterState(remoteCluster, metadataVersion + 1, (remoteClusterStateResponse, remoteError) -> { + // Also check removed flag here, as it may take a while for this remote cluster state api call to return: + if (removed) { + LOGGER.info("AutoFollower instance for cluster [{}] has been removed", remoteCluster); + return; + } + if (remoteClusterStateResponse != null) { assert remoteError == null; if (remoteClusterStateResponse.isWaitForTimedOut()) { diff --git a/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/AutoFollowIT.java b/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/AutoFollowIT.java index 286e5badee133..70f624392367a 100644 --- a/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/AutoFollowIT.java +++ b/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/AutoFollowIT.java @@ -157,15 +157,20 @@ public void testAutoFollowManyIndices() throws Exception { int expectedVal2 = numIndices; MetaData[] metaData = new MetaData[1]; + AutoFollowStats[] autoFollowStats = new AutoFollowStats[1]; try { assertBusy(() -> { metaData[0] = followerClient().admin().cluster().prepareState().get().getState().metaData(); + autoFollowStats[0] = getAutoFollowStats(); int count = (int) Arrays.stream(metaData[0].getConcreteAllIndices()).filter(s -> s.startsWith("copy-")).count(); assertThat(count, equalTo(expectedVal2)); + // Ensure that there are no auto follow errors: + // (added specifically to see that there are no leader indices auto followed multiple times) + assertThat(autoFollowStats[0].getRecentAutoFollowErrors().size(), equalTo(0)); }); } catch (AssertionError ae) { logger.warn("metadata={}", Strings.toString(metaData[0])); - logger.warn("auto follow stats={}", Strings.toString(getAutoFollowStats())); + logger.warn("auto follow stats={}", Strings.toString(autoFollowStats[0])); throw ae; } } diff --git a/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/action/AutoFollowCoordinatorTests.java b/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/action/AutoFollowCoordinatorTests.java index 1c6864088b5fe..2ac67a65fc1c6 100644 --- a/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/action/AutoFollowCoordinatorTests.java +++ b/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/action/AutoFollowCoordinatorTests.java @@ -620,6 +620,10 @@ public void testUpdateAutoFollowers() { assertThat(autoFollowCoordinator.getStats().getAutoFollowedClusters().size(), equalTo(2)); assertThat(autoFollowCoordinator.getStats().getAutoFollowedClusters().get("remote1"), notNullValue()); assertThat(autoFollowCoordinator.getStats().getAutoFollowedClusters().get("remote2"), notNullValue()); + // Get a reference to auto follower that will get removed, so that we can assert that it has been marked as removed, + // when pattern 1 and 3 are moved. (To avoid a edge case where multiple auto follow coordinators for the same remote cluster) + AutoFollowCoordinator.AutoFollower removedAutoFollower1 = autoFollowCoordinator.getAutoFollowers().get("remote1"); + assertThat(removedAutoFollower1.removed, is(false)); // Remove patterns 1 and 3: patterns.remove("pattern1"); patterns.remove("pattern3"); @@ -630,6 +634,7 @@ public void testUpdateAutoFollowers() { autoFollowCoordinator.updateAutoFollowers(clusterState); assertThat(autoFollowCoordinator.getStats().getAutoFollowedClusters().size(), equalTo(1)); assertThat(autoFollowCoordinator.getStats().getAutoFollowedClusters().get("remote2"), notNullValue()); + assertThat(removedAutoFollower1.removed, is(true)); // Add pattern 4: patterns.put("pattern4", new AutoFollowPattern("remote1", Collections.singletonList("metrics-*"), null, null, null, null, null, null, null, null, null, null, null)); @@ -641,7 +646,13 @@ public void testUpdateAutoFollowers() { assertThat(autoFollowCoordinator.getStats().getAutoFollowedClusters().size(), equalTo(2)); assertThat(autoFollowCoordinator.getStats().getAutoFollowedClusters().get("remote1"), notNullValue()); assertThat(autoFollowCoordinator.getStats().getAutoFollowedClusters().get("remote2"), notNullValue()); + // Get references to auto followers that will get removed, so that we can assert that those have been marked as removed, + // when pattern 2 and 4 are moved. (To avoid a edge case where multiple auto follow coordinators for the same remote cluster) + removedAutoFollower1 = autoFollowCoordinator.getAutoFollowers().get("remote1"); + AutoFollower removedAutoFollower2 = autoFollowCoordinator.getAutoFollowers().get("remote2"); // Remove patterns 2 and 4: + assertThat(removedAutoFollower1.removed, is(false)); + assertThat(removedAutoFollower2.removed, is(false)); patterns.remove("pattern2"); patterns.remove("pattern4"); clusterState = ClusterState.builder(new ClusterName("remote")) @@ -650,6 +661,8 @@ public void testUpdateAutoFollowers() { .build(); autoFollowCoordinator.updateAutoFollowers(clusterState); assertThat(autoFollowCoordinator.getStats().getAutoFollowedClusters().size(), equalTo(0)); + assertThat(removedAutoFollower1.removed, is(true)); + assertThat(removedAutoFollower2.removed, is(true)); } public void testUpdateAutoFollowersNoPatterns() { From 1a1dbf705faaff44d40bd24168ecce15fbfe9540 Mon Sep 17 00:00:00 2001 From: Julie Tibshirani Date: Tue, 15 Jan 2019 07:32:47 -0800 Subject: [PATCH 178/186] Make sure to use the resolved type in DocumentMapperService#extractMappings. (#37451) * Pull out a shared method MapperService#resolveDocumentType. * Make sure to resolve the type when extracting the mappings. Addresses #36811. --- .../test/index/70_mix_typeless_typeful.yml | 38 ++++++++++++++++ .../test/update/90_mix_typeless_typeful.yml | 45 +++++++++++++++++++ .../metadata/MetaDataMappingService.java | 21 ++++----- .../index/mapper/DocumentMapperParser.java | 2 +- .../index/mapper/MapperService.java | 17 +++++++ .../elasticsearch/index/shard/IndexShard.java | 30 ++++--------- 6 files changed, 119 insertions(+), 34 deletions(-) diff --git a/rest-api-spec/src/main/resources/rest-api-spec/test/index/70_mix_typeless_typeful.yml b/rest-api-spec/src/main/resources/rest-api-spec/test/index/70_mix_typeless_typeful.yml index 5e225ec1ad30a..9f4f68f1998ae 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/test/index/70_mix_typeless_typeful.yml +++ b/rest-api-spec/src/main/resources/rest-api-spec/test/index/70_mix_typeless_typeful.yml @@ -60,3 +60,41 @@ - match: { _id: $id} - match: { _version: 1} - match: { _source: { foo: bar }} + +--- +"Index call that introduces new field mappings": + + - skip: + version: " - 6.99.99" + reason: Typeless APIs were introduced in 7.0.0 + + - do: + indices.create: # not using include_type_name: false on purpose + index: index + body: + mappings: + not_doc: + properties: + foo: + type: "keyword" + - do: + index: + index: index + id: 2 + body: { new_field: value } + + - match: { _index: "index" } + - match: { _type: "_doc" } + - match: { _id: "2" } + - match: { _version: 1 } + + - do: + get: # using typeful API on purpose + index: index + type: not_doc + id: 2 + + - match: { _index: "index" } + - match: { _type: "not_doc" } + - match: { _id: "2" } + - match: { _version: 1} diff --git a/rest-api-spec/src/main/resources/rest-api-spec/test/update/90_mix_typeless_typeful.yml b/rest-api-spec/src/main/resources/rest-api-spec/test/update/90_mix_typeless_typeful.yml index 066f0989c35b2..4caeb712c2896 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/test/update/90_mix_typeless_typeful.yml +++ b/rest-api-spec/src/main/resources/rest-api-spec/test/update/90_mix_typeless_typeful.yml @@ -37,3 +37,48 @@ id: 1 - match: { _source.foo: baz } + +--- +"Update call that introduces new field mappings": + + - skip: + version: " - 6.99.99" + reason: Typeless APIs were introduced in 7.0.0 + + - do: + indices.create: # not using include_type_name: false on purpose + index: index + body: + mappings: + not_doc: + properties: + foo: + type: "keyword" + + - do: + index: + index: index + type: not_doc + id: 1 + body: { foo: bar } + + - do: + update: + index: index + id: 1 + body: + doc: + foo: baz + new_field: value + - do: + get: # using typeful API on purpose + index: index + type: not_doc + id: 1 + + - match: { _index: "index" } + - match: { _type: "not_doc" } + - match: { _id: "1" } + - match: { _version: 2} + - match: { _source.foo: baz } + - match: { _source.new_field: value } diff --git a/server/src/main/java/org/elasticsearch/cluster/metadata/MetaDataMappingService.java b/server/src/main/java/org/elasticsearch/cluster/metadata/MetaDataMappingService.java index 002ed86da34c2..06dda07d2289e 100644 --- a/server/src/main/java/org/elasticsearch/cluster/metadata/MetaDataMappingService.java +++ b/server/src/main/java/org/elasticsearch/cluster/metadata/MetaDataMappingService.java @@ -276,8 +276,9 @@ private ClusterState applyRequest(ClusterState currentState, PutMappingClusterSt } if (mappingType == null) { mappingType = newMapper.type(); - } else if (mappingType.equals(newMapper.type()) == false) { - throw new InvalidTypeNameException("Type name provided does not match type name within mapping definition"); + } else if (mappingType.equals(newMapper.type()) == false + && mapperService.resolveDocumentType(mappingType).equals(newMapper.type()) == false) { + throw new InvalidTypeNameException("Type name provided does not match type name within mapping definition."); } } assert mappingType != null; @@ -295,16 +296,12 @@ private ClusterState applyRequest(ClusterState currentState, PutMappingClusterSt // we use the exact same indexService and metadata we used to validate above here to actually apply the update final Index index = indexMetaData.getIndex(); final MapperService mapperService = indexMapperServices.get(index); - String typeForUpdate = mappingType; // the type to use to apply the mapping update - if (MapperService.SINGLE_MAPPING_NAME.equals(typeForUpdate)) { - // If the user gave _doc as a special type value or if (s)he is using the new typeless APIs, - // then we apply the mapping update to the existing type. This allows to move to typeless - // APIs with indices whose type name is different from `_doc`. - DocumentMapper mapper = mapperService.documentMapper(); - if (mapper != null) { - typeForUpdate = mapper.type(); - } - } + + // If the user gave _doc as a special type value or if they are using the new typeless APIs, + // then we apply the mapping update to the existing type. This allows to move to typeless + // APIs with indices whose type name is different from `_doc`. + String typeForUpdate = mapperService.resolveDocumentType(mappingType); // the type to use to apply the mapping update + CompressedXContent existingSource = null; DocumentMapper existingMapper = mapperService.documentMapper(typeForUpdate); if (existingMapper != null) { diff --git a/server/src/main/java/org/elasticsearch/index/mapper/DocumentMapperParser.java b/server/src/main/java/org/elasticsearch/index/mapper/DocumentMapperParser.java index e63d5a279f3cd..e388dd7ebcd00 100644 --- a/server/src/main/java/org/elasticsearch/index/mapper/DocumentMapperParser.java +++ b/server/src/main/java/org/elasticsearch/index/mapper/DocumentMapperParser.java @@ -197,7 +197,7 @@ private Tuple> extractMapping(String type, Map> mapping; - if (type == null || type.equals(rootName)) { + if (type == null || type.equals(rootName) || mapperService.resolveDocumentType(type).equals(rootName)) { mapping = new Tuple<>(rootName, (Map) root.get(rootName)); } else { mapping = new Tuple<>(type, root); diff --git a/server/src/main/java/org/elasticsearch/index/mapper/MapperService.java b/server/src/main/java/org/elasticsearch/index/mapper/MapperService.java index 6424e75eaf662..add313d7aa87b 100644 --- a/server/src/main/java/org/elasticsearch/index/mapper/MapperService.java +++ b/server/src/main/java/org/elasticsearch/index/mapper/MapperService.java @@ -665,6 +665,23 @@ public DocumentMapper documentMapper(String type) { return null; } + /** + * Resolves a type from a mapping-related request into the type that should be used when + * merging and updating mappings. + * + * If the special `_doc` type is provided, then we replace it with the actual type that is + * being used in the mappings. This allows typeless APIs such as 'index' or 'put mappings' + * to work against indices with a custom type name. + */ + public String resolveDocumentType(String type) { + if (MapperService.SINGLE_MAPPING_NAME.equals(type)) { + if (mapper != null) { + return mapper.type(); + } + } + return type; + } + /** * Returns the document mapper created, including a mapping update if the * type has been dynamically created. diff --git a/server/src/main/java/org/elasticsearch/index/shard/IndexShard.java b/server/src/main/java/org/elasticsearch/index/shard/IndexShard.java index 8a10f4021d20f..e8cb0f519dd13 100644 --- a/server/src/main/java/org/elasticsearch/index/shard/IndexShard.java +++ b/server/src/main/java/org/elasticsearch/index/shard/IndexShard.java @@ -721,7 +721,7 @@ private Engine.IndexResult applyIndexOperation(Engine engine, long seqNo, long o ensureWriteAllowed(origin); Engine.Index operation; try { - final String resolvedType = resolveType(sourceToParse.type()); + final String resolvedType = mapperService.resolveDocumentType(sourceToParse.type()); final SourceToParse sourceWithResolvedType; if (resolvedType.equals(sourceToParse.type())) { sourceWithResolvedType = sourceToParse; @@ -844,11 +844,12 @@ private Engine.DeleteResult applyDeleteOperation(Engine engine, long seqNo, long } catch (MapperParsingException | IllegalArgumentException | TypeMissingException e) { return new Engine.DeleteResult(e, version, operationPrimaryTerm, seqNo, false); } - if (resolveType(type).equals(mapperService.documentMapper().type()) == false) { + if (mapperService.resolveDocumentType(type).equals(mapperService.documentMapper().type()) == false) { // We should never get there due to the fact that we generate mapping updates on deletes, // but we still prefer to have a hard exception here as we would otherwise delete a // document in the wrong type. - throw new IllegalStateException("Deleting document from type [" + resolveType(type) + "] while current type is [" + + throw new IllegalStateException("Deleting document from type [" + + mapperService.resolveDocumentType(type) + "] while current type is [" + mapperService.documentMapper().type() + "]"); } final Term uid = new Term(IdFieldMapper.NAME, Uid.encodeId(id)); @@ -861,8 +862,8 @@ private Engine.Delete prepareDelete(String type, String id, Term uid, long seqNo VersionType versionType, Engine.Operation.Origin origin, long ifSeqNo, long ifPrimaryTerm) { long startTime = System.nanoTime(); - return new Engine.Delete(resolveType(type), id, uid, seqNo, primaryTerm, version, versionType, origin, startTime, - ifSeqNo, ifPrimaryTerm); + return new Engine.Delete(mapperService.resolveDocumentType(type), id, uid, seqNo, primaryTerm, version, versionType, + origin, startTime, ifSeqNo, ifPrimaryTerm); } private Engine.DeleteResult delete(Engine engine, Engine.Delete delete) throws IOException { @@ -885,7 +886,7 @@ private Engine.DeleteResult delete(Engine engine, Engine.Delete delete) throws I public Engine.GetResult get(Engine.Get get) { readAllowed(); DocumentMapper mapper = mapperService.documentMapper(); - if (mapper == null || mapper.type().equals(resolveType(get.type())) == false) { + if (mapper == null || mapper.type().equals(mapperService.resolveDocumentType(get.type())) == false) { return GetResult.NOT_EXISTS; } return getEngine().get(get, this::acquireSearcher); @@ -2319,23 +2320,10 @@ private static void persistMetadata( } } - /** - * If an index/update/get/delete operation is using the special `_doc` type, then we replace - * it with the actual type that is being used in the mappings so that users may use typeless - * APIs with indices that have types. - */ - private String resolveType(String type) { - if (MapperService.SINGLE_MAPPING_NAME.equals(type)) { - DocumentMapper docMapper = mapperService.documentMapper(); - if (docMapper != null) { - return docMapper.type(); - } - } - return type; - } private DocumentMapperForType docMapper(String type) { - return mapperService.documentMapperWithAutoCreate(resolveType(type)); + return mapperService.documentMapperWithAutoCreate( + mapperService.resolveDocumentType(type)); } private EngineConfig newEngineConfig() { From 0b396a0c5e5f5a463ccc629a7826a78389b4557f Mon Sep 17 00:00:00 2001 From: Luca Cavanna Date: Tue, 15 Jan 2019 16:42:55 +0100 Subject: [PATCH 179/186] Restore assertion on discount overlaps in SimilarityTests This assertion was commented out as the getDiscountOverlaps getter was missing from LegacyBm25Similarity. That has been fixed in lucene. --- .../org/elasticsearch/index/similarity/SimilarityTests.java | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/server/src/test/java/org/elasticsearch/index/similarity/SimilarityTests.java b/server/src/test/java/org/elasticsearch/index/similarity/SimilarityTests.java index fd5a77665fecd..d1fae9acceb67 100644 --- a/server/src/test/java/org/elasticsearch/index/similarity/SimilarityTests.java +++ b/server/src/test/java/org/elasticsearch/index/similarity/SimilarityTests.java @@ -99,8 +99,7 @@ public void testResolveSimilaritiesFromMapping_bm25() throws IOException { LegacyBM25Similarity similarity = (LegacyBM25Similarity) mapperService.fullName("field1").similarity().get(); assertThat(similarity.getK1(), equalTo(2.0f)); assertThat(similarity.getB(), equalTo(0.5f)); - // TODO: re-enable when we switch back to BM25Similarity - // assertThat(similarity.getDiscountOverlaps(), equalTo(false)); + assertThat(similarity.getDiscountOverlaps(), equalTo(false)); } public void testResolveSimilaritiesFromMapping_boolean() throws IOException { From 6f91f06d8679bbf2bfc7563d38cc500d83deba1c Mon Sep 17 00:00:00 2001 From: Igor Motov Date: Tue, 15 Jan 2019 10:52:46 -0500 Subject: [PATCH 180/186] Geo: Adds a set of no dependency geo classes for JDBC driver (#36477) Adds a set of geo classes to represent geo data in the JDBC driver and to be used as an intermediate format to pass geo shapes for indexing and query generation in #35320. Relates to #35767 and #35320 --- libs/geo/build.gradle | 49 ++ .../elasticsearch/geo/geometry/Circle.java | 104 ++++ .../elasticsearch/geo/geometry/Geometry.java | 32 + .../geo/geometry/GeometryCollection.java | 85 +++ .../geo/geometry/GeometryUtils.java | 70 +++ .../geo/geometry/GeometryVisitor.java | 69 +++ .../org/elasticsearch/geo/geometry/Line.java | 106 ++++ .../geo/geometry/LinearRing.java | 53 ++ .../elasticsearch/geo/geometry/MultiLine.java | 46 ++ .../geo/geometry/MultiPoint.java | 47 ++ .../geo/geometry/MultiPolygon.java | 46 ++ .../org/elasticsearch/geo/geometry/Point.java | 95 +++ .../elasticsearch/geo/geometry/Polygon.java | 122 ++++ .../elasticsearch/geo/geometry/Rectangle.java | 168 ++++++ .../elasticsearch/geo/geometry/ShapeType.java | 36 ++ .../org/elasticsearch/geo/package-info.java | 24 + .../geo/utils/WellKnownText.java | 560 ++++++++++++++++++ .../geo/geometry/BaseGeometryTestCase.java | 203 +++++++ .../geo/geometry/CircleTests.java | 51 ++ .../geo/geometry/GeometryCollectionTests.java | 54 ++ .../elasticsearch/geo/geometry/LineTests.java | 51 ++ .../geo/geometry/LinearRingTests.java | 52 ++ .../geo/geometry/MultiLineTests.java | 51 ++ .../geo/geometry/MultiPointTests.java | 51 ++ .../geo/geometry/MultiPolygonTests.java | 53 ++ .../geo/geometry/PointTests.java | 48 ++ .../geo/geometry/PolygonTests.java | 52 ++ .../geo/geometry/RectangleTests.java | 51 ++ settings.gradle | 7 +- 29 files changed, 2435 insertions(+), 1 deletion(-) create mode 100644 libs/geo/build.gradle create mode 100644 libs/geo/src/main/java/org/elasticsearch/geo/geometry/Circle.java create mode 100644 libs/geo/src/main/java/org/elasticsearch/geo/geometry/Geometry.java create mode 100644 libs/geo/src/main/java/org/elasticsearch/geo/geometry/GeometryCollection.java create mode 100644 libs/geo/src/main/java/org/elasticsearch/geo/geometry/GeometryUtils.java create mode 100644 libs/geo/src/main/java/org/elasticsearch/geo/geometry/GeometryVisitor.java create mode 100644 libs/geo/src/main/java/org/elasticsearch/geo/geometry/Line.java create mode 100644 libs/geo/src/main/java/org/elasticsearch/geo/geometry/LinearRing.java create mode 100644 libs/geo/src/main/java/org/elasticsearch/geo/geometry/MultiLine.java create mode 100644 libs/geo/src/main/java/org/elasticsearch/geo/geometry/MultiPoint.java create mode 100644 libs/geo/src/main/java/org/elasticsearch/geo/geometry/MultiPolygon.java create mode 100644 libs/geo/src/main/java/org/elasticsearch/geo/geometry/Point.java create mode 100644 libs/geo/src/main/java/org/elasticsearch/geo/geometry/Polygon.java create mode 100644 libs/geo/src/main/java/org/elasticsearch/geo/geometry/Rectangle.java create mode 100644 libs/geo/src/main/java/org/elasticsearch/geo/geometry/ShapeType.java create mode 100644 libs/geo/src/main/java/org/elasticsearch/geo/package-info.java create mode 100644 libs/geo/src/main/java/org/elasticsearch/geo/utils/WellKnownText.java create mode 100644 libs/geo/src/test/java/org/elasticsearch/geo/geometry/BaseGeometryTestCase.java create mode 100644 libs/geo/src/test/java/org/elasticsearch/geo/geometry/CircleTests.java create mode 100644 libs/geo/src/test/java/org/elasticsearch/geo/geometry/GeometryCollectionTests.java create mode 100644 libs/geo/src/test/java/org/elasticsearch/geo/geometry/LineTests.java create mode 100644 libs/geo/src/test/java/org/elasticsearch/geo/geometry/LinearRingTests.java create mode 100644 libs/geo/src/test/java/org/elasticsearch/geo/geometry/MultiLineTests.java create mode 100644 libs/geo/src/test/java/org/elasticsearch/geo/geometry/MultiPointTests.java create mode 100644 libs/geo/src/test/java/org/elasticsearch/geo/geometry/MultiPolygonTests.java create mode 100644 libs/geo/src/test/java/org/elasticsearch/geo/geometry/PointTests.java create mode 100644 libs/geo/src/test/java/org/elasticsearch/geo/geometry/PolygonTests.java create mode 100644 libs/geo/src/test/java/org/elasticsearch/geo/geometry/RectangleTests.java diff --git a/libs/geo/build.gradle b/libs/geo/build.gradle new file mode 100644 index 0000000000000..ab3419b93b9b8 --- /dev/null +++ b/libs/geo/build.gradle @@ -0,0 +1,49 @@ +/* + * 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. + */ + +apply plugin: 'elasticsearch.build' +apply plugin: 'nebula.maven-base-publish' +apply plugin: 'nebula.maven-scm' + +dependencies { + if (isEclipse == false || project.path == ":libs:geo-tests") { + testCompile("org.elasticsearch.test:framework:${version}") { + exclude group: 'org.elasticsearch', module: 'elasticsearch-geo' + } + } +} + +forbiddenApisMain { + // geo does not depend on server + // TODO: Need to decide how we want to handle for forbidden signatures with the changes to core + replaceSignatureFiles 'jdk-signatures' +} + +if (isEclipse) { + // in eclipse the project is under a fake root, we need to change around the source sets + sourceSets { + if (project.path == ":libs:geo") { + main.java.srcDirs = ['java'] + main.resources.srcDirs = ['resources'] + } else { + test.java.srcDirs = ['java'] + test.resources.srcDirs = ['resources'] + } + } +} diff --git a/libs/geo/src/main/java/org/elasticsearch/geo/geometry/Circle.java b/libs/geo/src/main/java/org/elasticsearch/geo/geometry/Circle.java new file mode 100644 index 0000000000000..fea582e07b3e2 --- /dev/null +++ b/libs/geo/src/main/java/org/elasticsearch/geo/geometry/Circle.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.geo.geometry; + +/** + * Circle geometry (not part of WKT standard, but used in elasticsearch) + */ +public class Circle implements Geometry { + public static final Circle EMPTY = new Circle(); + private final double lat; + private final double lon; + private final double radiusMeters; + + private Circle() { + lat = 0; + lon = 0; + radiusMeters = -1; + } + + public Circle(final double lat, final double lon, final double radiusMeters) { + this.lat = lat; + this.lon = lon; + this.radiusMeters = radiusMeters; + if (radiusMeters < 0 ) { + throw new IllegalArgumentException("Circle radius [" + radiusMeters + "] cannot be negative"); + } + GeometryUtils.checkLatitude(lat); + GeometryUtils.checkLongitude(lon); + } + + @Override + public ShapeType type() { + return ShapeType.CIRCLE; + } + + public double getLat() { + return lat; + } + + public double getLon() { + return lon; + } + + public double getRadiusMeters() { + return radiusMeters; + } + + @Override + public boolean equals(Object o) { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + + Circle circle = (Circle) o; + if (Double.compare(circle.lat, lat) != 0) return false; + if (Double.compare(circle.lon, lon) != 0) return false; + return (Double.compare(circle.radiusMeters, radiusMeters) == 0); + } + + @Override + public int hashCode() { + int result; + long temp; + temp = Double.doubleToLongBits(lat); + result = (int) (temp ^ (temp >>> 32)); + temp = Double.doubleToLongBits(lon); + result = 31 * result + (int) (temp ^ (temp >>> 32)); + temp = Double.doubleToLongBits(radiusMeters); + result = 31 * result + (int) (temp ^ (temp >>> 32)); + return result; + } + + @Override + public T visit(GeometryVisitor visitor) { + return visitor.visit(this); + } + + @Override + public boolean isEmpty() { + return radiusMeters < 0; + } + + @Override + public String toString() { + return "lat=" + lat + ", lon=" + lon + ", radius=" + radiusMeters; + } + +} diff --git a/libs/geo/src/main/java/org/elasticsearch/geo/geometry/Geometry.java b/libs/geo/src/main/java/org/elasticsearch/geo/geometry/Geometry.java new file mode 100644 index 0000000000000..4557780effcad --- /dev/null +++ b/libs/geo/src/main/java/org/elasticsearch/geo/geometry/Geometry.java @@ -0,0 +1,32 @@ +/* + * 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.geo.geometry; + +/** + * Base class for all Geometry objects supported by elasticsearch + */ +public interface Geometry { + + ShapeType type(); + + T visit(GeometryVisitor visitor); + + boolean isEmpty(); +} diff --git a/libs/geo/src/main/java/org/elasticsearch/geo/geometry/GeometryCollection.java b/libs/geo/src/main/java/org/elasticsearch/geo/geometry/GeometryCollection.java new file mode 100644 index 0000000000000..a6bad62efad30 --- /dev/null +++ b/libs/geo/src/main/java/org/elasticsearch/geo/geometry/GeometryCollection.java @@ -0,0 +1,85 @@ +/* + * 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.geo.geometry; + +import java.util.Collections; +import java.util.Iterator; +import java.util.List; +import java.util.Objects; + +/** + * Collection of arbitrary geometry classes + */ +public class GeometryCollection implements Geometry, Iterable { + public static final GeometryCollection EMPTY = new GeometryCollection<>(); + + private final List shapes; + + public GeometryCollection() { + shapes = Collections.emptyList(); + } + + public GeometryCollection(List shapes) { + if (shapes == null || shapes.isEmpty()) { + throw new IllegalArgumentException("the list of shapes cannot be null or empty"); + } + this.shapes = shapes; + } + + @Override + public ShapeType type() { + return ShapeType.GEOMETRYCOLLECTION; + } + + @Override + public T visit(GeometryVisitor visitor) { + return visitor.visit(this); + } + + @Override + public boolean isEmpty() { + return shapes.isEmpty(); + } + + public int size() { + return shapes.size(); + } + + public G get(int i) { + return shapes.get(i); + } + + @Override + public boolean equals(Object o) { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + GeometryCollection that = (GeometryCollection) o; + return Objects.equals(shapes, that.shapes); + } + + @Override + public int hashCode() { + return Objects.hash(shapes); + } + + @Override + public Iterator iterator() { + return shapes.iterator(); + } +} diff --git a/libs/geo/src/main/java/org/elasticsearch/geo/geometry/GeometryUtils.java b/libs/geo/src/main/java/org/elasticsearch/geo/geometry/GeometryUtils.java new file mode 100644 index 0000000000000..9a7d4b99d3e53 --- /dev/null +++ b/libs/geo/src/main/java/org/elasticsearch/geo/geometry/GeometryUtils.java @@ -0,0 +1,70 @@ +/* + * 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.geo.geometry; + +/** + * Geometry-related utility methods + */ +final class GeometryUtils { + /** + * Minimum longitude value. + */ + static final double MIN_LON_INCL = -180.0D; + + /** + * Maximum longitude value. + */ + static final double MAX_LON_INCL = 180.0D; + + /** + * Minimum latitude value. + */ + static final double MIN_LAT_INCL = -90.0D; + + /** + * Maximum latitude value. + */ + static final double MAX_LAT_INCL = 90.0D; + + // No instance: + private GeometryUtils() { + } + + /** + * validates latitude value is within standard +/-90 coordinate bounds + */ + static void checkLatitude(double latitude) { + if (Double.isNaN(latitude) || latitude < MIN_LAT_INCL || latitude > MAX_LAT_INCL) { + throw new IllegalArgumentException( + "invalid latitude " + latitude + "; must be between " + MIN_LAT_INCL + " and " + MAX_LAT_INCL); + } + } + + /** + * validates longitude value is within standard +/-180 coordinate bounds + */ + static void checkLongitude(double longitude) { + if (Double.isNaN(longitude) || longitude < MIN_LON_INCL || longitude > MAX_LON_INCL) { + throw new IllegalArgumentException( + "invalid longitude " + longitude + "; must be between " + MIN_LON_INCL + " and " + MAX_LON_INCL); + } + } + +} diff --git a/libs/geo/src/main/java/org/elasticsearch/geo/geometry/GeometryVisitor.java b/libs/geo/src/main/java/org/elasticsearch/geo/geometry/GeometryVisitor.java new file mode 100644 index 0000000000000..8317b23d1feca --- /dev/null +++ b/libs/geo/src/main/java/org/elasticsearch/geo/geometry/GeometryVisitor.java @@ -0,0 +1,69 @@ +/* + * 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.geo.geometry; + +/** + * Support class for creating Geometry Visitors. + *

+ * This is an implementation of the Visitor pattern. The basic idea is to simplify adding new operations on Geometries, without + * constantly modifying and adding new functionality to the Geometry hierarchy and keeping it as lightweight as possible. + *

+ * It is a more object-oriented alternative to structures like this: + *

+ * if (obj instanceof This) {
+ *   doThis((This) obj);
+ * } elseif (obj instanceof That) {
+ *   doThat((That) obj);
+ * ...
+ * } else {
+ *   throw new IllegalArgumentException("Unknown object " + obj);
+ * }
+ * 
+ *

+ * The Visitor Pattern replaces this structure with Interface inheritance making it easier to identify all places that are using this + * structure, and making a shape a compile-time failure instead of runtime. + *

+ * See {@link org.elasticsearch.geo.utils.WellKnownText#toWKT(Geometry, StringBuilder)} for an example of how this interface is used. + * + * @see Visitor Pattern + */ +public interface GeometryVisitor { + + T visit(Circle circle); + + T visit(GeometryCollection collection); + + T visit(Line line); + + T visit(LinearRing ring); + + T visit(MultiLine multiLine); + + T visit(MultiPoint multiPoint); + + T visit(MultiPolygon multiPolygon); + + T visit(Point point); + + T visit(Polygon polygon); + + T visit(Rectangle rectangle); + +} diff --git a/libs/geo/src/main/java/org/elasticsearch/geo/geometry/Line.java b/libs/geo/src/main/java/org/elasticsearch/geo/geometry/Line.java new file mode 100644 index 0000000000000..415dacfce9b3c --- /dev/null +++ b/libs/geo/src/main/java/org/elasticsearch/geo/geometry/Line.java @@ -0,0 +1,106 @@ +/* + * 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.geo.geometry; + +import java.util.Arrays; + +/** + * Represents a Line on the earth's surface in lat/lon decimal degrees. + */ +public class Line implements Geometry { + public static final Line EMPTY = new Line(); + private final double[] lats; + private final double[] lons; + + protected Line() { + lats = new double[0]; + lons = new double[0]; + } + + public Line(double[] lats, double[] lons) { + this.lats = lats; + this.lons = lons; + if (lats == null) { + throw new IllegalArgumentException("lats must not be null"); + } + if (lons == null) { + throw new IllegalArgumentException("lons must not be null"); + } + if (lats.length != lons.length) { + throw new IllegalArgumentException("lats and lons must be equal length"); + } + if (lats.length < 2) { + throw new IllegalArgumentException("at least two points in the line is required"); + } + for (int i = 0; i < lats.length; i++) { + GeometryUtils.checkLatitude(lats[i]); + GeometryUtils.checkLongitude(lons[i]); + } + } + + public int length() { + return lats.length; + } + + public double getLat(int i) { + return lats[i]; + } + + public double getLon(int i) { + return lons[i]; + } + + @Override + public ShapeType type() { + return ShapeType.LINESTRING; + } + + @Override + public T visit(GeometryVisitor visitor) { + return visitor.visit(this); + } + + @Override + public boolean isEmpty() { + return lats.length == 0; + } + + @Override + public boolean equals(Object o) { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + Line line = (Line) o; + return Arrays.equals(lats, line.lats) && + Arrays.equals(lons, line.lons); + } + + @Override + public int hashCode() { + int result = Arrays.hashCode(lats); + result = 31 * result + Arrays.hashCode(lons); + return result; + } + + @Override + public String toString() { + return "lats=" + Arrays.toString(lats) + + ", lons=" + Arrays.toString(lons); + } +} diff --git a/libs/geo/src/main/java/org/elasticsearch/geo/geometry/LinearRing.java b/libs/geo/src/main/java/org/elasticsearch/geo/geometry/LinearRing.java new file mode 100644 index 0000000000000..20b1a46dd9d31 --- /dev/null +++ b/libs/geo/src/main/java/org/elasticsearch/geo/geometry/LinearRing.java @@ -0,0 +1,53 @@ +/* + * 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.geo.geometry; + +/** + * Represents a closed line on the earth's surface in lat/lon decimal degrees. + *

+ * Cannot be serialized by WKT directly but used as a part of polygon + */ +public class LinearRing extends Line { + public static final LinearRing EMPTY = new LinearRing(); + + private LinearRing() { + } + + public LinearRing(double[] lats, double[] lons) { + super(lats, lons); + if (lats.length < 2) { + throw new IllegalArgumentException("linear ring cannot contain less than 2 points, found " + lats.length); + } + if (lats[0] != lats[lats.length - 1] || lons[0] != lons[lons.length - 1]) { + throw new IllegalArgumentException("first and last points of the linear ring must be the same (it must close itself): lats[0]=" + + lats[0] + " lats[" + (lats.length - 1) + "]=" + lats[lats.length - 1]); + } + } + + @Override + public ShapeType type() { + return ShapeType.LINEARRING; + } + + @Override + public T visit(GeometryVisitor visitor) { + return visitor.visit(this); + } +} diff --git a/libs/geo/src/main/java/org/elasticsearch/geo/geometry/MultiLine.java b/libs/geo/src/main/java/org/elasticsearch/geo/geometry/MultiLine.java new file mode 100644 index 0000000000000..995c43d0c1c80 --- /dev/null +++ b/libs/geo/src/main/java/org/elasticsearch/geo/geometry/MultiLine.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.geo.geometry; + +import java.util.List; + +/** + * Represents a MultiLine geometry object on the earth's surface. + */ +public class MultiLine extends GeometryCollection { + public static final MultiLine EMPTY = new MultiLine(); + + private MultiLine() { + } + + public MultiLine(List lines) { + super(lines); + } + + @Override + public ShapeType type() { + return ShapeType.MULTILINESTRING; + } + + @Override + public T visit(GeometryVisitor visitor) { + return visitor.visit(this); + } +} diff --git a/libs/geo/src/main/java/org/elasticsearch/geo/geometry/MultiPoint.java b/libs/geo/src/main/java/org/elasticsearch/geo/geometry/MultiPoint.java new file mode 100644 index 0000000000000..383fef81219aa --- /dev/null +++ b/libs/geo/src/main/java/org/elasticsearch/geo/geometry/MultiPoint.java @@ -0,0 +1,47 @@ +/* + * 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.geo.geometry; + +import java.util.List; + +/** + * Represents a MultiPoint object on the earth's surface in decimal degrees. + */ +public class MultiPoint extends GeometryCollection { + public static final MultiPoint EMPTY = new MultiPoint(); + + private MultiPoint() { + } + + public MultiPoint(List points) { + super(points); + } + + @Override + public ShapeType type() { + return ShapeType.MULTIPOINT; + } + + @Override + public T visit(GeometryVisitor visitor) { + return visitor.visit(this); + } + +} diff --git a/libs/geo/src/main/java/org/elasticsearch/geo/geometry/MultiPolygon.java b/libs/geo/src/main/java/org/elasticsearch/geo/geometry/MultiPolygon.java new file mode 100644 index 0000000000000..01c68d6dd0b32 --- /dev/null +++ b/libs/geo/src/main/java/org/elasticsearch/geo/geometry/MultiPolygon.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.geo.geometry; + +import java.util.List; + +/** + * Collection of polygons + */ +public class MultiPolygon extends GeometryCollection { + public static final MultiPolygon EMPTY = new MultiPolygon(); + + private MultiPolygon() { + } + + public MultiPolygon(List polygons) { + super(polygons); + } + + @Override + public ShapeType type() { + return ShapeType.MULTIPOLYGON; + } + + @Override + public T visit(GeometryVisitor visitor) { + return visitor.visit(this); + } +} diff --git a/libs/geo/src/main/java/org/elasticsearch/geo/geometry/Point.java b/libs/geo/src/main/java/org/elasticsearch/geo/geometry/Point.java new file mode 100644 index 0000000000000..d85d40c8dc789 --- /dev/null +++ b/libs/geo/src/main/java/org/elasticsearch/geo/geometry/Point.java @@ -0,0 +1,95 @@ +/* + * 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.geo.geometry; + +/** + * Represents a Point on the earth's surface in decimal degrees. + */ +public class Point implements Geometry { + public static final Point EMPTY = new Point(); + + private final double lat; + private final double lon; + private final boolean empty; + + private Point() { + lat = 0; + lon = 0; + empty = true; + } + + public Point(double lat, double lon) { + GeometryUtils.checkLatitude(lat); + GeometryUtils.checkLongitude(lon); + this.lat = lat; + this.lon = lon; + this.empty = false; + } + + @Override + public ShapeType type() { + return ShapeType.POINT; + } + + public double lat() { + return lat; + } + + public double lon() { + return lon; + } + + @Override + public boolean equals(Object o) { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + + Point point = (Point) o; + if (point.empty != empty) return false; + if (Double.compare(point.lat, lat) != 0) return false; + return Double.compare(point.lon, lon) == 0; + } + + @Override + public int hashCode() { + int result; + long temp; + temp = Double.doubleToLongBits(lat); + result = (int) (temp ^ (temp >>> 32)); + temp = Double.doubleToLongBits(lon); + result = 31 * result + (int) (temp ^ (temp >>> 32)); + return result; + } + + @Override + public T visit(GeometryVisitor visitor) { + return visitor.visit(this); + } + + @Override + public boolean isEmpty() { + return empty; + } + + @Override + public String toString() { + return "lat=" + lat + ", lon=" + lon; + } +} diff --git a/libs/geo/src/main/java/org/elasticsearch/geo/geometry/Polygon.java b/libs/geo/src/main/java/org/elasticsearch/geo/geometry/Polygon.java new file mode 100644 index 0000000000000..9f28c4b81b6a2 --- /dev/null +++ b/libs/geo/src/main/java/org/elasticsearch/geo/geometry/Polygon.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.geo.geometry; + +import java.util.Collections; +import java.util.List; +import java.util.Objects; + +/** + * Represents a closed polygon on the earth's surface with optional holes + */ +public final class Polygon implements Geometry { + public static final Polygon EMPTY = new Polygon(); + private final LinearRing polygon; + private final List holes; + + private Polygon() { + polygon = LinearRing.EMPTY; + holes = Collections.emptyList(); + } + + /** + * Creates a new Polygon from the supplied latitude/longitude array, and optionally any holes. + */ + public Polygon(LinearRing polygon, List holes) { + this.polygon = polygon; + this.holes = holes; + if (holes == null) { + throw new IllegalArgumentException("holes must not be null"); + } + checkRing(polygon); + for (LinearRing hole : holes) { + checkRing(hole); + } + } + + /** + * Creates a new Polygon from the supplied latitude/longitude array, and optionally any holes. + */ + public Polygon(LinearRing polygon) { + this(polygon, Collections.emptyList()); + } + + @Override + public ShapeType type() { + return ShapeType.POLYGON; + } + + private void checkRing(LinearRing ring) { + if (ring.length() < 4) { + throw new IllegalArgumentException("at least 4 polygon points required"); + } + } + + public int getNumberOfHoles() { + return holes.size(); + } + + public LinearRing getPolygon() { + return polygon; + } + + public LinearRing getHole(int i) { + if (i >= holes.size()) { + throw new IllegalArgumentException("Index " + i + " is outside the bounds of the " + holes.size() + " polygon holes"); + } + return holes.get(i); + } + + @Override + public T visit(GeometryVisitor visitor) { + return visitor.visit(this); + } + + @Override + public boolean isEmpty() { + return polygon.isEmpty(); + } + + + @Override + public String toString() { + StringBuilder sb = new StringBuilder(); + sb.append("polygon=").append(polygon); + if (holes.size() > 0) { + sb.append(", holes="); + sb.append(holes); + } + return sb.toString(); + } + + @Override + public boolean equals(Object o) { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + Polygon polygon1 = (Polygon) o; + return Objects.equals(polygon, polygon1.polygon) && + Objects.equals(holes, polygon1.holes); + } + + @Override + public int hashCode() { + return Objects.hash(polygon, holes); + } +} diff --git a/libs/geo/src/main/java/org/elasticsearch/geo/geometry/Rectangle.java b/libs/geo/src/main/java/org/elasticsearch/geo/geometry/Rectangle.java new file mode 100644 index 0000000000000..8170bf95a6e2d --- /dev/null +++ b/libs/geo/src/main/java/org/elasticsearch/geo/geometry/Rectangle.java @@ -0,0 +1,168 @@ +/* + * 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.geo.geometry; + +/** + * Represents a lat/lon rectangle in decimal degrees. + */ +public class Rectangle implements Geometry { + public static final Rectangle EMPTY = new Rectangle(); + /** + * maximum longitude value (in degrees) + */ + private final double minLat; + /** + * minimum longitude value (in degrees) + */ + private final double minLon; + /** + * maximum latitude value (in degrees) + */ + private final double maxLat; + /** + * minimum latitude value (in degrees) + */ + private final double maxLon; + + private final boolean empty; + + private Rectangle() { + minLat = 0; + minLon = 0; + maxLat = 0; + maxLon = 0; + empty = true; + } + + /** + * Constructs a bounding box by first validating the provided latitude and longitude coordinates + */ + public Rectangle(double minLat, double maxLat, double minLon, double maxLon) { + GeometryUtils.checkLatitude(minLat); + GeometryUtils.checkLatitude(maxLat); + GeometryUtils.checkLongitude(minLon); + GeometryUtils.checkLongitude(maxLon); + this.minLon = minLon; + this.maxLon = maxLon; + this.minLat = minLat; + this.maxLat = maxLat; + empty = false; + if (maxLat < minLat) { + throw new IllegalArgumentException("max lat cannot be less than min lat"); + } + } + + public double getWidth() { + if (crossesDateline()) { + return GeometryUtils.MAX_LON_INCL - minLon + maxLon - GeometryUtils.MIN_LON_INCL; + } + return maxLon - minLon; + } + + public double getHeight() { + return maxLat - minLat; + } + + public double getMinLat() { + return minLat; + } + + public double getMinLon() { + return minLon; + } + + public double getMaxLat() { + return maxLat; + } + + public double getMaxLon() { + return maxLon; + } + + @Override + public ShapeType type() { + return ShapeType.ENVELOPE; + } + + @Override + public String toString() { + StringBuilder b = new StringBuilder(); + b.append("Rectangle(lat="); + b.append(minLat); + b.append(" TO "); + b.append(maxLat); + b.append(" lon="); + b.append(minLon); + b.append(" TO "); + b.append(maxLon); + if (maxLon < minLon) { + b.append(" [crosses dateline!]"); + } + b.append(")"); + + return b.toString(); + } + + /** + * Returns true if this bounding box crosses the dateline + */ + public boolean crossesDateline() { + return maxLon < minLon; + } + + @Override + public boolean equals(Object o) { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + + Rectangle rectangle = (Rectangle) o; + + if (Double.compare(rectangle.minLat, minLat) != 0) return false; + if (Double.compare(rectangle.minLon, minLon) != 0) return false; + if (Double.compare(rectangle.maxLat, maxLat) != 0) return false; + return Double.compare(rectangle.maxLon, maxLon) == 0; + + } + + @Override + public int hashCode() { + int result; + long temp; + temp = Double.doubleToLongBits(minLat); + result = (int) (temp ^ (temp >>> 32)); + temp = Double.doubleToLongBits(minLon); + result = 31 * result + (int) (temp ^ (temp >>> 32)); + temp = Double.doubleToLongBits(maxLat); + result = 31 * result + (int) (temp ^ (temp >>> 32)); + temp = Double.doubleToLongBits(maxLon); + result = 31 * result + (int) (temp ^ (temp >>> 32)); + return result; + } + + @Override + public T visit(GeometryVisitor visitor) { + return visitor.visit(this); + } + + @Override + public boolean isEmpty() { + return empty; + } +} diff --git a/libs/geo/src/main/java/org/elasticsearch/geo/geometry/ShapeType.java b/libs/geo/src/main/java/org/elasticsearch/geo/geometry/ShapeType.java new file mode 100644 index 0000000000000..2272f1ad89410 --- /dev/null +++ b/libs/geo/src/main/java/org/elasticsearch/geo/geometry/ShapeType.java @@ -0,0 +1,36 @@ +/* + * 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.geo.geometry; + +/** + * Shape types supported by elasticsearch + */ +public enum ShapeType { + POINT, + MULTIPOINT, + LINESTRING, + MULTILINESTRING, + POLYGON, + MULTIPOLYGON, + GEOMETRYCOLLECTION, + LINEARRING, // not serialized by itself in WKT or WKB + ENVELOPE, // not part of the actual WKB spec + CIRCLE; // not part of the actual WKB spec +} diff --git a/libs/geo/src/main/java/org/elasticsearch/geo/package-info.java b/libs/geo/src/main/java/org/elasticsearch/geo/package-info.java new file mode 100644 index 0000000000000..3b59d36b46ae7 --- /dev/null +++ b/libs/geo/src/main/java/org/elasticsearch/geo/package-info.java @@ -0,0 +1,24 @@ +/* + * 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. + */ + + +/** + * Common Geo classes + */ +package org.elasticsearch.geo; diff --git a/libs/geo/src/main/java/org/elasticsearch/geo/utils/WellKnownText.java b/libs/geo/src/main/java/org/elasticsearch/geo/utils/WellKnownText.java new file mode 100644 index 0000000000000..5cf29065b006a --- /dev/null +++ b/libs/geo/src/main/java/org/elasticsearch/geo/utils/WellKnownText.java @@ -0,0 +1,560 @@ +/* + * 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.geo.utils; + +import org.elasticsearch.geo.geometry.Circle; +import org.elasticsearch.geo.geometry.Geometry; +import org.elasticsearch.geo.geometry.GeometryCollection; +import org.elasticsearch.geo.geometry.GeometryVisitor; +import org.elasticsearch.geo.geometry.Line; +import org.elasticsearch.geo.geometry.LinearRing; +import org.elasticsearch.geo.geometry.MultiLine; +import org.elasticsearch.geo.geometry.MultiPoint; +import org.elasticsearch.geo.geometry.MultiPolygon; +import org.elasticsearch.geo.geometry.Point; +import org.elasticsearch.geo.geometry.Polygon; +import org.elasticsearch.geo.geometry.Rectangle; + +import java.io.IOException; +import java.io.StreamTokenizer; +import java.io.StringReader; +import java.text.ParseException; +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; +import java.util.Locale; + +/** + * Utility class for converting to and from WKT + */ +public class WellKnownText { + public static final String EMPTY = "EMPTY"; + public static final String SPACE = " "; + public static final String LPAREN = "("; + public static final String RPAREN = ")"; + public static final String COMMA = ","; + public static final String NAN = "NaN"; + + private static final String NUMBER = ""; + private static final String EOF = "END-OF-STREAM"; + private static final String EOL = "END-OF-LINE"; + + public static String toWKT(Geometry geometry) { + StringBuilder builder = new StringBuilder(); + toWKT(geometry, builder); + return builder.toString(); + } + + public static void toWKT(Geometry geometry, StringBuilder sb) { + sb.append(getWKTName(geometry)); + sb.append(SPACE); + if (geometry.isEmpty()) { + sb.append(EMPTY); + } else { + geometry.visit(new GeometryVisitor() { + @Override + public Void visit(Circle circle) { + sb.append(LPAREN); + visitPoint(circle.getLon(), circle.getLat()); + sb.append(SPACE); + sb.append(circle.getRadiusMeters()); + sb.append(RPAREN); + return null; + } + + @Override + public Void visit(GeometryCollection collection) { + if (collection.size() == 0) { + sb.append(EMPTY); + } else { + sb.append(LPAREN); + toWKT(collection.get(0), sb); + for (int i = 1; i < collection.size(); ++i) { + sb.append(COMMA); + toWKT(collection.get(i), sb); + } + sb.append(RPAREN); + } + return null; + } + + @Override + public Void visit(Line line) { + sb.append(LPAREN); + visitPoint(line.getLon(0), line.getLat(0)); + for (int i = 1; i < line.length(); ++i) { + sb.append(COMMA); + sb.append(SPACE); + visitPoint(line.getLon(i), line.getLat(i)); + } + sb.append(RPAREN); + return null; + } + + @Override + public Void visit(LinearRing ring) { + throw new IllegalArgumentException("Linear ring is not supported by WKT"); + } + + @Override + public Void visit(MultiLine multiLine) { + visitCollection(multiLine); + return null; + } + + @Override + public Void visit(MultiPoint multiPoint) { + // walk through coordinates: + sb.append(LPAREN); + visitPoint(multiPoint.get(0).lon(), multiPoint.get(0).lat()); + for (int i = 1; i < multiPoint.size(); ++i) { + sb.append(COMMA); + sb.append(SPACE); + Point point = multiPoint.get(i); + visitPoint(point.lon(), point.lat()); + } + sb.append(RPAREN); + return null; + } + + @Override + public Void visit(MultiPolygon multiPolygon) { + visitCollection(multiPolygon); + return null; + } + + @Override + public Void visit(Point point) { + if (point.isEmpty()) { + sb.append(EMPTY); + } else { + sb.append(LPAREN); + visitPoint(point.lon(), point.lat()); + sb.append(RPAREN); + } + return null; + } + + private void visitPoint(double lon, double lat) { + sb.append(lon).append(SPACE).append(lat); + } + + private void visitCollection(GeometryCollection collection) { + if (collection.size() == 0) { + sb.append(EMPTY); + } else { + sb.append(LPAREN); + collection.get(0).visit(this); + for (int i = 1; i < collection.size(); ++i) { + sb.append(COMMA); + collection.get(i).visit(this); + } + sb.append(RPAREN); + } + } + + @Override + public Void visit(Polygon polygon) { + sb.append(LPAREN); + visit((Line) polygon.getPolygon()); + int numberOfHoles = polygon.getNumberOfHoles(); + for (int i = 0; i < numberOfHoles; ++i) { + sb.append(", "); + visit((Line) polygon.getHole(i)); + } + sb.append(RPAREN); + return null; + } + + @Override + public Void visit(Rectangle rectangle) { + sb.append(LPAREN); + // minX, maxX, maxY, minY + sb.append(rectangle.getMinLon()); + sb.append(COMMA); + sb.append(SPACE); + sb.append(rectangle.getMaxLon()); + sb.append(COMMA); + sb.append(SPACE); + sb.append(rectangle.getMaxLat()); + sb.append(COMMA); + sb.append(SPACE); + sb.append(rectangle.getMinLat()); + sb.append(RPAREN); + return null; + } + }); + } + } + + public static Geometry fromWKT(String wkt) throws IOException, ParseException { + StringReader reader = new StringReader(wkt); + try { + // setup the tokenizer; configured to read words w/o numbers + StreamTokenizer tokenizer = new StreamTokenizer(reader); + tokenizer.resetSyntax(); + tokenizer.wordChars('a', 'z'); + tokenizer.wordChars('A', 'Z'); + tokenizer.wordChars(128 + 32, 255); + tokenizer.wordChars('0', '9'); + tokenizer.wordChars('-', '-'); + tokenizer.wordChars('+', '+'); + tokenizer.wordChars('.', '.'); + tokenizer.whitespaceChars(' ', ' '); + tokenizer.whitespaceChars('\t', '\t'); + tokenizer.whitespaceChars('\r', '\r'); + tokenizer.whitespaceChars('\n', '\n'); + tokenizer.commentChar('#'); + return parseGeometry(tokenizer); + } finally { + reader.close(); + } + } + + /** + * parse geometry from the stream tokenizer + */ + private static Geometry parseGeometry(StreamTokenizer stream) throws IOException, ParseException { + final String type = nextWord(stream).toLowerCase(Locale.ROOT); + switch (type) { + case "point": + return parsePoint(stream); + case "multipoint": + return parseMultiPoint(stream); + case "linestring": + return parseLine(stream); + case "multilinestring": + return parseMultiLine(stream); + case "polygon": + return parsePolygon(stream); + case "multipolygon": + return parseMultiPolygon(stream); + case "bbox": + return parseBBox(stream); + case "geometrycollection": + return parseGeometryCollection(stream); + case "circle": // Not part of the standard, but we need it for internal serialization + return parseCircle(stream); + } + throw new IllegalArgumentException("Unknown geometry type: " + type); + } + + private static GeometryCollection parseGeometryCollection(StreamTokenizer stream) throws IOException, ParseException { + if (nextEmptyOrOpen(stream).equals(EMPTY)) { + return GeometryCollection.EMPTY; + } + List shapes = new ArrayList<>(); + shapes.add(parseGeometry(stream)); + while (nextCloserOrComma(stream).equals(COMMA)) { + shapes.add(parseGeometry(stream)); + } + return new GeometryCollection<>(shapes); + } + + private static Point parsePoint(StreamTokenizer stream) throws IOException, ParseException { + if (nextEmptyOrOpen(stream).equals(EMPTY)) { + return Point.EMPTY; + } + double lon = nextNumber(stream); + double lat = nextNumber(stream); + Point pt = new Point(lat, lon); + if (isNumberNext(stream) == true) { + nextNumber(stream); + } + nextCloser(stream); + return pt; + } + + private static void parseCoordinates(StreamTokenizer stream, ArrayList lats, ArrayList lons) + throws IOException, ParseException { + parseCoordinate(stream, lats, lons); + while (nextCloserOrComma(stream).equals(COMMA)) { + parseCoordinate(stream, lats, lons); + } + } + + private static void parseCoordinate(StreamTokenizer stream, ArrayList lats, ArrayList lons) + throws IOException, ParseException { + lons.add(nextNumber(stream)); + lats.add(nextNumber(stream)); + if (isNumberNext(stream)) { + nextNumber(stream); + } + } + + private static MultiPoint parseMultiPoint(StreamTokenizer stream) throws IOException, ParseException { + String token = nextEmptyOrOpen(stream); + if (token.equals(EMPTY)) { + return MultiPoint.EMPTY; + } + ArrayList lats = new ArrayList<>(); + ArrayList lons = new ArrayList<>(); + ArrayList points = new ArrayList<>(); + parseCoordinates(stream, lats, lons); + for (int i = 0; i < lats.size(); i++) { + points.add(new Point(lats.get(i), lons.get(i))); + } + return new MultiPoint(Collections.unmodifiableList(points)); + } + + private static Line parseLine(StreamTokenizer stream) throws IOException, ParseException { + String token = nextEmptyOrOpen(stream); + if (token.equals(EMPTY)) { + return Line.EMPTY; + } + ArrayList lats = new ArrayList<>(); + ArrayList lons = new ArrayList<>(); + parseCoordinates(stream, lats, lons); + return new Line(lats.stream().mapToDouble(i -> i).toArray(), lons.stream().mapToDouble(i -> i).toArray()); + } + + private static MultiLine parseMultiLine(StreamTokenizer stream) throws IOException, ParseException { + String token = nextEmptyOrOpen(stream); + if (token.equals(EMPTY)) { + return MultiLine.EMPTY; + } + ArrayList lines = new ArrayList<>(); + lines.add(parseLine(stream)); + while (nextCloserOrComma(stream).equals(COMMA)) { + lines.add(parseLine(stream)); + } + return new MultiLine(Collections.unmodifiableList(lines)); + } + + private static LinearRing parsePolygonHole(StreamTokenizer stream) throws IOException, ParseException { + nextOpener(stream); + ArrayList lats = new ArrayList<>(); + ArrayList lons = new ArrayList<>(); + parseCoordinates(stream, lats, lons); + return new LinearRing(lats.stream().mapToDouble(i -> i).toArray(), lons.stream().mapToDouble(i -> i).toArray()); + } + + private static Polygon parsePolygon(StreamTokenizer stream) throws IOException, ParseException { + if (nextEmptyOrOpen(stream).equals(EMPTY)) { + return Polygon.EMPTY; + } + nextOpener(stream); + ArrayList lats = new ArrayList<>(); + ArrayList lons = new ArrayList<>(); + parseCoordinates(stream, lats, lons); + ArrayList holes = new ArrayList<>(); + while (nextCloserOrComma(stream).equals(COMMA)) { + holes.add(parsePolygonHole(stream)); + } + if (holes.isEmpty()) { + return new Polygon(new LinearRing(lats.stream().mapToDouble(i -> i).toArray(), lons.stream().mapToDouble(i -> i).toArray())); + } else { + return new Polygon( + new LinearRing(lats.stream().mapToDouble(i -> i).toArray(), lons.stream().mapToDouble(i -> i).toArray()), + Collections.unmodifiableList(holes)); + } + } + + private static MultiPolygon parseMultiPolygon(StreamTokenizer stream) throws IOException, ParseException { + String token = nextEmptyOrOpen(stream); + if (token.equals(EMPTY)) { + return MultiPolygon.EMPTY; + } + ArrayList polygons = new ArrayList<>(); + polygons.add(parsePolygon(stream)); + while (nextCloserOrComma(stream).equals(COMMA)) { + polygons.add(parsePolygon(stream)); + } + return new MultiPolygon(Collections.unmodifiableList(polygons)); + } + + private static Rectangle parseBBox(StreamTokenizer stream) throws IOException, ParseException { + if (nextEmptyOrOpen(stream).equals(EMPTY)) { + return Rectangle.EMPTY; + } + double minLon = nextNumber(stream); + nextComma(stream); + double maxLon = nextNumber(stream); + nextComma(stream); + double maxLat = nextNumber(stream); + nextComma(stream); + double minLat = nextNumber(stream); + nextCloser(stream); + return new Rectangle(minLat, maxLat, minLon, maxLon); + } + + + private static Circle parseCircle(StreamTokenizer stream) throws IOException, ParseException { + if (nextEmptyOrOpen(stream).equals(EMPTY)) { + return Circle.EMPTY; + } + double lon = nextNumber(stream); + double lat = nextNumber(stream); + double radius = nextNumber(stream); + Circle circle = new Circle(lat, lon, radius); + if (isNumberNext(stream) == true) { + nextNumber(stream); + } + nextCloser(stream); + return circle; + } + + /** + * next word in the stream + */ + private static String nextWord(StreamTokenizer stream) throws ParseException, IOException { + switch (stream.nextToken()) { + case StreamTokenizer.TT_WORD: + final String word = stream.sval; + return word.equalsIgnoreCase(EMPTY) ? EMPTY : word; + case '(': + return LPAREN; + case ')': + return RPAREN; + case ',': + return COMMA; + } + throw new ParseException("expected word but found: " + tokenString(stream), stream.lineno()); + } + + private static double nextNumber(StreamTokenizer stream) throws IOException, ParseException { + if (stream.nextToken() == StreamTokenizer.TT_WORD) { + if (stream.sval.equalsIgnoreCase(NAN)) { + return Double.NaN; + } else { + try { + return Double.parseDouble(stream.sval); + } catch (NumberFormatException e) { + throw new ParseException("invalid number found: " + stream.sval, stream.lineno()); + } + } + } + throw new ParseException("expected number but found: " + tokenString(stream), stream.lineno()); + } + + private static String tokenString(StreamTokenizer stream) { + switch (stream.ttype) { + case StreamTokenizer.TT_WORD: + return stream.sval; + case StreamTokenizer.TT_EOF: + return EOF; + case StreamTokenizer.TT_EOL: + return EOL; + case StreamTokenizer.TT_NUMBER: + return NUMBER; + } + return "'" + (char) stream.ttype + "'"; + } + + private static boolean isNumberNext(StreamTokenizer stream) throws IOException { + final int type = stream.nextToken(); + stream.pushBack(); + return type == StreamTokenizer.TT_WORD; + } + + private static String nextEmptyOrOpen(StreamTokenizer stream) throws IOException, ParseException { + final String next = nextWord(stream); + if (next.equals(EMPTY) || next.equals(LPAREN)) { + return next; + } + throw new ParseException("expected " + EMPTY + " or " + LPAREN + + " but found: " + tokenString(stream), stream.lineno()); + } + + private static String nextCloser(StreamTokenizer stream) throws IOException, ParseException { + if (nextWord(stream).equals(RPAREN)) { + return RPAREN; + } + throw new ParseException("expected " + RPAREN + " but found: " + tokenString(stream), stream.lineno()); + } + + private static String nextComma(StreamTokenizer stream) throws IOException, ParseException { + if (nextWord(stream).equals(COMMA) == true) { + return COMMA; + } + throw new ParseException("expected " + COMMA + " but found: " + tokenString(stream), stream.lineno()); + } + + private static String nextOpener(StreamTokenizer stream) throws IOException, ParseException { + if (nextWord(stream).equals(LPAREN)) { + return LPAREN; + } + throw new ParseException("expected " + LPAREN + " but found: " + tokenString(stream), stream.lineno()); + } + + private static String nextCloserOrComma(StreamTokenizer stream) throws IOException, ParseException { + String token = nextWord(stream); + if (token.equals(COMMA) || token.equals(RPAREN)) { + return token; + } + throw new ParseException("expected " + COMMA + " or " + RPAREN + + " but found: " + tokenString(stream), stream.lineno()); + } + + public static String getWKTName(Geometry geometry) { + return geometry.visit(new GeometryVisitor() { + @Override + public String visit(Circle circle) { + return "circle"; + } + + @Override + public String visit(GeometryCollection collection) { + return "geometrycollection"; + } + + @Override + public String visit(Line line) { + return "linestring"; + } + + @Override + public String visit(LinearRing ring) { + throw new UnsupportedOperationException("line ring cannot be serialized using WKT"); + } + + @Override + public String visit(MultiLine multiLine) { + return "multilinestring"; + } + + @Override + public String visit(MultiPoint multiPoint) { + return "multipoint"; + } + + @Override + public String visit(MultiPolygon multiPolygon) { + return "multipolygon"; + } + + @Override + public String visit(Point point) { + return "point"; + } + + @Override + public String visit(Polygon polygon) { + return "polygon"; + } + + @Override + public String visit(Rectangle rectangle) { + return "bbox"; + } + }); + } + +} diff --git a/libs/geo/src/test/java/org/elasticsearch/geo/geometry/BaseGeometryTestCase.java b/libs/geo/src/test/java/org/elasticsearch/geo/geometry/BaseGeometryTestCase.java new file mode 100644 index 0000000000000..3ffabd2134323 --- /dev/null +++ b/libs/geo/src/test/java/org/elasticsearch/geo/geometry/BaseGeometryTestCase.java @@ -0,0 +1,203 @@ +/* + * 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.geo.geometry; + +import org.elasticsearch.ElasticsearchException; +import org.elasticsearch.Version; +import org.elasticsearch.common.io.stream.Writeable; +import org.elasticsearch.geo.utils.WellKnownText; +import org.elasticsearch.test.AbstractWireTestCase; + +import java.io.IOException; +import java.text.ParseException; +import java.util.ArrayList; +import java.util.List; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.function.Supplier; + +abstract class BaseGeometryTestCase extends AbstractWireTestCase { + + @Override + protected Writeable.Reader instanceReader() { + throw new IllegalStateException("shouldn't be called in this test"); + } + + + @SuppressWarnings("unchecked") + @Override + protected T copyInstance(T instance, Version version) throws IOException { + String text = WellKnownText.toWKT(instance); + try { + return (T) WellKnownText.fromWKT(text); + } catch (ParseException e) { + throw new ElasticsearchException(e); + } + } + + public void testVisitor() { + testVisitor(createTestInstance()); + } + + public static void testVisitor(Geometry geom) { + AtomicBoolean called = new AtomicBoolean(false); + Object result = geom.visit(new GeometryVisitor() { + private Object verify(Geometry geometry, String expectedClass) { + assertFalse("Visitor should be called only once", called.getAndSet(true)); + assertSame(geom, geometry); + assertEquals(geometry.getClass().getName(), "org.elasticsearch.geo.geometry." + expectedClass); + return "result"; + } + + @Override + public Object visit(Circle circle) { + return verify(circle, "Circle"); + } + + @Override + public Object visit(GeometryCollection collection) { + return verify(collection, "GeometryCollection"); } + + @Override + public Object visit(Line line) { + return verify(line, "Line"); + } + + @Override + public Object visit(LinearRing ring) { + return verify(ring, "LinearRing"); + } + + @Override + public Object visit(MultiLine multiLine) { + return verify(multiLine, "MultiLine"); + } + + @Override + public Object visit(MultiPoint multiPoint) { + return verify(multiPoint, "MultiPoint"); + } + + @Override + public Object visit(MultiPolygon multiPolygon) { + return verify(multiPolygon, "MultiPolygon"); + } + + @Override + public Object visit(Point point) { + return verify(point, "Point"); + } + + @Override + public Object visit(Polygon polygon) { + return verify(polygon, "Polygon"); + } + + @Override + public Object visit(Rectangle rectangle) { + return verify(rectangle, "Rectangle"); + } + }); + + assertTrue("visitor wasn't called", called.get()); + assertEquals("result", result); + } + + public static double randomLat() { + return randomDoubleBetween(-90, 90, true); + } + + public static double randomLon() { + return randomDoubleBetween(-180, 180, true); + } + + public static Circle randomCircle() { + return new Circle(randomDoubleBetween(-90, 90, true), randomDoubleBetween(-180, 180, true), randomDoubleBetween(0, 100, false)); + } + + public static Line randomLine() { + int size = randomIntBetween(2, 10); + double[] lats = new double[size]; + double[] lons = new double[size]; + for (int i = 0; i < size; i++) { + lats[i] = randomLat(); + lons[i] = randomLon(); + } + return new Line(lats, lons); + } + + public static Point randomPoint() { + return new Point(randomLat(), randomLon()); + } + + public static LinearRing randomLinearRing() { + int size = randomIntBetween(3, 10); + double[] lats = new double[size + 1]; + double[] lons = new double[size + 1]; + for (int i = 0; i < size; i++) { + lats[i] = randomLat(); + lons[i] = randomLon(); + } + lats[size] = lats[0]; + lons[size] = lons[0]; + return new LinearRing(lats, lons); + } + + public static Polygon randomPolygon() { + int size = randomIntBetween(0, 10); + List holes = new ArrayList<>(); + for (int i = 0; i < size; i++) { + holes.add(randomLinearRing()); + } + if (holes.size() > 0) { + return new Polygon(randomLinearRing(), holes); + } else { + return new Polygon(randomLinearRing()); + } + } + + public static Rectangle randomRectangle() { + double lat1 = randomLat(); + double lat2 = randomLat(); + double minLon = randomLon(); + double maxLon = randomLon(); + return new Rectangle(Math.min(lat1, lat2), Math.max(lat1, lat2), minLon, maxLon); + } + + public static GeometryCollection randomGeometryCollection() { + return randomGeometryCollection(0); + } + + private static GeometryCollection randomGeometryCollection(int level) { + int size = randomIntBetween(1, 10); + List shapes = new ArrayList<>(); + for (int i = 0; i < size; i++) { + @SuppressWarnings("unchecked") Supplier geometry = randomFrom( + BaseGeometryTestCase::randomCircle, + BaseGeometryTestCase::randomLine, + BaseGeometryTestCase::randomPoint, + BaseGeometryTestCase::randomPolygon, + BaseGeometryTestCase::randomRectangle, + level < 3 ? () -> randomGeometryCollection(level + 1) : BaseGeometryTestCase::randomPoint // don't build too deep + ); + shapes.add(geometry.get()); + } + return new GeometryCollection<>(shapes); + } +} diff --git a/libs/geo/src/test/java/org/elasticsearch/geo/geometry/CircleTests.java b/libs/geo/src/test/java/org/elasticsearch/geo/geometry/CircleTests.java new file mode 100644 index 0000000000000..0f2292792a743 --- /dev/null +++ b/libs/geo/src/test/java/org/elasticsearch/geo/geometry/CircleTests.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.geo.geometry; + +import org.elasticsearch.geo.utils.WellKnownText; + +import java.io.IOException; +import java.text.ParseException; + +public class CircleTests extends BaseGeometryTestCase { + @Override + protected Circle createTestInstance() { + return new Circle(randomDoubleBetween(-90, 90, true), randomDoubleBetween(-180, 180, true), randomDoubleBetween(0, 100, false)); + } + + public void testBasicSerialization() throws IOException, ParseException { + assertEquals("circle (20.0 10.0 15.0)", WellKnownText.toWKT(new Circle(10, 20, 15))); + assertEquals(new Circle(10, 20, 15), WellKnownText.fromWKT("circle (20.0 10.0 15.0)")); + + assertEquals("circle EMPTY", WellKnownText.toWKT(Circle.EMPTY)); + assertEquals(Circle.EMPTY, WellKnownText.fromWKT("circle EMPTY)")); + } + + public void testInitValidation() { + IllegalArgumentException ex = expectThrows(IllegalArgumentException.class, () -> new Circle(10, 20, -1)); + assertEquals("Circle radius [-1.0] cannot be negative", ex.getMessage()); + + ex = expectThrows(IllegalArgumentException.class, () -> new Circle(100, 20, 1)); + assertEquals("invalid latitude 100.0; must be between -90.0 and 90.0", ex.getMessage()); + + ex = expectThrows(IllegalArgumentException.class, () -> new Circle(10, 200, 1)); + assertEquals("invalid longitude 200.0; must be between -180.0 and 180.0", ex.getMessage()); + } +} diff --git a/libs/geo/src/test/java/org/elasticsearch/geo/geometry/GeometryCollectionTests.java b/libs/geo/src/test/java/org/elasticsearch/geo/geometry/GeometryCollectionTests.java new file mode 100644 index 0000000000000..21b3c65410372 --- /dev/null +++ b/libs/geo/src/test/java/org/elasticsearch/geo/geometry/GeometryCollectionTests.java @@ -0,0 +1,54 @@ +/* + * 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.geo.geometry; + +import org.elasticsearch.geo.utils.WellKnownText; + +import java.io.IOException; +import java.text.ParseException; +import java.util.Arrays; +import java.util.Collections; + +public class GeometryCollectionTests extends BaseGeometryTestCase> { + @Override + protected GeometryCollection createTestInstance() { + return randomGeometryCollection(); + } + + public void testBasicSerialization() throws IOException, ParseException { + assertEquals("geometrycollection (point (20.0 10.0),point EMPTY)", + WellKnownText.toWKT(new GeometryCollection(Arrays.asList(new Point(10, 20), Point.EMPTY)))); + + assertEquals(new GeometryCollection(Arrays.asList(new Point(10, 20), Point.EMPTY)), + WellKnownText.fromWKT("geometrycollection (point (20.0 10.0),point EMPTY)")); + + assertEquals("geometrycollection EMPTY", WellKnownText.toWKT(GeometryCollection.EMPTY)); + assertEquals(GeometryCollection.EMPTY, WellKnownText.fromWKT("geometrycollection EMPTY)")); + } + + @SuppressWarnings("ConstantConditions") + public void testInitValidation() { + IllegalArgumentException ex = expectThrows(IllegalArgumentException.class, () -> new GeometryCollection<>(Collections.emptyList())); + assertEquals("the list of shapes cannot be null or empty", ex.getMessage()); + + ex = expectThrows(IllegalArgumentException.class, () -> new GeometryCollection<>(null)); + assertEquals("the list of shapes cannot be null or empty", ex.getMessage()); + } +} diff --git a/libs/geo/src/test/java/org/elasticsearch/geo/geometry/LineTests.java b/libs/geo/src/test/java/org/elasticsearch/geo/geometry/LineTests.java new file mode 100644 index 0000000000000..9914481df44e1 --- /dev/null +++ b/libs/geo/src/test/java/org/elasticsearch/geo/geometry/LineTests.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.geo.geometry; + +import org.elasticsearch.geo.utils.WellKnownText; + +import java.io.IOException; +import java.text.ParseException; + +public class LineTests extends BaseGeometryTestCase { + @Override + protected Line createTestInstance() { + return randomLine(); + } + + public void testBasicSerialization() throws IOException, ParseException { + assertEquals("linestring (3.0 1.0, 4.0 2.0)", WellKnownText.toWKT(new Line(new double[]{1, 2}, new double[]{3, 4}))); + assertEquals(new Line(new double[]{1, 2}, new double[]{3, 4}), WellKnownText.fromWKT("linestring (3 1, 4 2)")); + + assertEquals("linestring EMPTY", WellKnownText.toWKT(Line.EMPTY)); + assertEquals(Line.EMPTY, WellKnownText.fromWKT("linestring EMPTY)")); + } + + public void testInitValidation() { + IllegalArgumentException ex = expectThrows(IllegalArgumentException.class, () -> new Line(new double[]{1}, new double[]{3})); + assertEquals("at least two points in the line is required", ex.getMessage()); + + ex = expectThrows(IllegalArgumentException.class, () -> new Line(new double[]{1, 2, 3, 1}, new double[]{3, 4, 500, 3})); + assertEquals("invalid longitude 500.0; must be between -180.0 and 180.0", ex.getMessage()); + + ex = expectThrows(IllegalArgumentException.class, () -> new Line(new double[]{1, 100, 3, 1}, new double[]{3, 4, 5, 3})); + assertEquals("invalid latitude 100.0; must be between -90.0 and 90.0", ex.getMessage()); + } +} diff --git a/libs/geo/src/test/java/org/elasticsearch/geo/geometry/LinearRingTests.java b/libs/geo/src/test/java/org/elasticsearch/geo/geometry/LinearRingTests.java new file mode 100644 index 0000000000000..73f6ac9a2f97a --- /dev/null +++ b/libs/geo/src/test/java/org/elasticsearch/geo/geometry/LinearRingTests.java @@ -0,0 +1,52 @@ +/* + * 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.geo.geometry; + +import org.elasticsearch.geo.utils.WellKnownText; +import org.elasticsearch.test.ESTestCase; + +public class LinearRingTests extends ESTestCase { + + public void testBasicSerialization() { + UnsupportedOperationException ex = expectThrows(UnsupportedOperationException.class, + () -> WellKnownText.toWKT(new LinearRing(new double[]{1, 2, 3, 1}, new double[]{3, 4, 5, 3}))); + assertEquals("line ring cannot be serialized using WKT", ex.getMessage()); + } + + public void testInitValidation() { + IllegalArgumentException ex = expectThrows(IllegalArgumentException.class, + () -> new LinearRing(new double[]{1, 2, 3}, new double[]{3, 4, 5})); + assertEquals("first and last points of the linear ring must be the same (it must close itself): lats[0]=1.0 lats[2]=3.0", + ex.getMessage()); + + ex = expectThrows(IllegalArgumentException.class, () -> new LinearRing(new double[]{1}, new double[]{3})); + assertEquals("at least two points in the line is required", ex.getMessage()); + + ex = expectThrows(IllegalArgumentException.class, () -> new LinearRing(new double[]{1, 2, 3, 1}, new double[]{3, 4, 500, 3})); + assertEquals("invalid longitude 500.0; must be between -180.0 and 180.0", ex.getMessage()); + + ex = expectThrows(IllegalArgumentException.class, () -> new LinearRing(new double[]{1, 100, 3, 1}, new double[]{3, 4, 5, 3})); + assertEquals("invalid latitude 100.0; must be between -90.0 and 90.0", ex.getMessage()); + } + + public void testVisitor() { + BaseGeometryTestCase.testVisitor(new LinearRing(new double[]{1, 2, 3, 1}, new double[]{3, 4, 5, 3})); + } +} diff --git a/libs/geo/src/test/java/org/elasticsearch/geo/geometry/MultiLineTests.java b/libs/geo/src/test/java/org/elasticsearch/geo/geometry/MultiLineTests.java new file mode 100644 index 0000000000000..ec5a8c2dd3860 --- /dev/null +++ b/libs/geo/src/test/java/org/elasticsearch/geo/geometry/MultiLineTests.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.geo.geometry; + +import org.elasticsearch.geo.utils.WellKnownText; + +import java.io.IOException; +import java.text.ParseException; +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; + +public class MultiLineTests extends BaseGeometryTestCase { + + @Override + protected MultiLine createTestInstance() { + int size = randomIntBetween(1, 10); + List arr = new ArrayList(); + for (int i = 0; i < size; i++) { + arr.add(randomLine()); + } + return new MultiLine(arr); + } + + public void testBasicSerialization() throws IOException, ParseException { + assertEquals("multilinestring ((3.0 1.0, 4.0 2.0))", WellKnownText.toWKT( + new MultiLine(Collections.singletonList(new Line(new double[]{1, 2}, new double[]{3, 4}))))); + assertEquals(new MultiLine(Collections.singletonList(new Line(new double[]{1, 2}, new double[]{3, 4}))), + WellKnownText.fromWKT("multilinestring ((3 1, 4 2))")); + + assertEquals("multilinestring EMPTY", WellKnownText.toWKT(MultiLine.EMPTY)); + assertEquals(MultiLine.EMPTY, WellKnownText.fromWKT("multilinestring EMPTY)")); + } +} diff --git a/libs/geo/src/test/java/org/elasticsearch/geo/geometry/MultiPointTests.java b/libs/geo/src/test/java/org/elasticsearch/geo/geometry/MultiPointTests.java new file mode 100644 index 0000000000000..81c8c6f3facab --- /dev/null +++ b/libs/geo/src/test/java/org/elasticsearch/geo/geometry/MultiPointTests.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.geo.geometry; + +import org.elasticsearch.geo.utils.WellKnownText; + +import java.io.IOException; +import java.text.ParseException; +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; + +public class MultiPointTests extends BaseGeometryTestCase { + + @Override + protected MultiPoint createTestInstance() { + int size = randomIntBetween(1, 10); + List arr = new ArrayList<>(); + for (int i = 0; i < size; i++) { + arr.add(randomPoint()); + } + return new MultiPoint(arr); + } + + public void testBasicSerialization() throws IOException, ParseException { + assertEquals("multipoint (2.0 1.0)", WellKnownText.toWKT( + new MultiPoint(Collections.singletonList(new Point(1, 2))))); + assertEquals(new MultiPoint(Collections.singletonList(new Point(1 ,2))), + WellKnownText.fromWKT("multipoint (2 1)")); + + assertEquals("multipoint EMPTY", WellKnownText.toWKT(MultiPoint.EMPTY)); + assertEquals(MultiPoint.EMPTY, WellKnownText.fromWKT("multipoint EMPTY)")); + } +} diff --git a/libs/geo/src/test/java/org/elasticsearch/geo/geometry/MultiPolygonTests.java b/libs/geo/src/test/java/org/elasticsearch/geo/geometry/MultiPolygonTests.java new file mode 100644 index 0000000000000..382572456032c --- /dev/null +++ b/libs/geo/src/test/java/org/elasticsearch/geo/geometry/MultiPolygonTests.java @@ -0,0 +1,53 @@ +/* + * 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.geo.geometry; + +import org.elasticsearch.geo.utils.WellKnownText; + +import java.io.IOException; +import java.text.ParseException; +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; + +public class MultiPolygonTests extends BaseGeometryTestCase { + + @Override + protected MultiPolygon createTestInstance() { + int size = randomIntBetween(1, 10); + List arr = new ArrayList<>(); + for (int i = 0; i < size; i++) { + arr.add(randomPolygon()); + } + return new MultiPolygon(arr); + } + + public void testBasicSerialization() throws IOException, ParseException { + assertEquals("multipolygon (((3.0 1.0, 4.0 2.0, 5.0 3.0, 3.0 1.0)))", + WellKnownText.toWKT(new MultiPolygon(Collections.singletonList( + new Polygon(new LinearRing(new double[]{1, 2, 3, 1}, new double[]{3, 4, 5, 3})))))); + assertEquals(new MultiPolygon(Collections.singletonList( + new Polygon(new LinearRing(new double[]{1, 2, 3, 1}, new double[]{3, 4, 5, 3})))), + WellKnownText.fromWKT("multipolygon (((3.0 1.0, 4.0 2.0, 5.0 3.0, 3.0 1.0)))")); + + assertEquals("multipolygon EMPTY", WellKnownText.toWKT(MultiPolygon.EMPTY)); + assertEquals(MultiPolygon.EMPTY, WellKnownText.fromWKT("multipolygon EMPTY)")); + } +} diff --git a/libs/geo/src/test/java/org/elasticsearch/geo/geometry/PointTests.java b/libs/geo/src/test/java/org/elasticsearch/geo/geometry/PointTests.java new file mode 100644 index 0000000000000..bfdb369d7a839 --- /dev/null +++ b/libs/geo/src/test/java/org/elasticsearch/geo/geometry/PointTests.java @@ -0,0 +1,48 @@ +/* + * 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.geo.geometry; + +import org.elasticsearch.geo.utils.WellKnownText; + +import java.io.IOException; +import java.text.ParseException; + +public class PointTests extends BaseGeometryTestCase { + @Override + protected Point createTestInstance() { + return randomPoint(); + } + + public void testBasicSerialization() throws IOException, ParseException { + assertEquals("point (20.0 10.0)", WellKnownText.toWKT(new Point(10, 20))); + assertEquals(new Point(10, 20), WellKnownText.fromWKT("point (20.0 10.0)")); + + assertEquals("point EMPTY", WellKnownText.toWKT(Point.EMPTY)); + assertEquals(Point.EMPTY, WellKnownText.fromWKT("point EMPTY)")); + } + + public void testInitValidation() { + IllegalArgumentException ex = expectThrows(IllegalArgumentException.class, () -> new Point(100, 10)); + assertEquals("invalid latitude 100.0; must be between -90.0 and 90.0", ex.getMessage()); + + ex = expectThrows(IllegalArgumentException.class, () -> new Point(10, 500)); + assertEquals("invalid longitude 500.0; must be between -180.0 and 180.0", ex.getMessage()); + } +} diff --git a/libs/geo/src/test/java/org/elasticsearch/geo/geometry/PolygonTests.java b/libs/geo/src/test/java/org/elasticsearch/geo/geometry/PolygonTests.java new file mode 100644 index 0000000000000..69a6d232083fe --- /dev/null +++ b/libs/geo/src/test/java/org/elasticsearch/geo/geometry/PolygonTests.java @@ -0,0 +1,52 @@ +/* + * 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.geo.geometry; + +import org.elasticsearch.geo.utils.WellKnownText; + +import java.io.IOException; +import java.text.ParseException; + +public class PolygonTests extends BaseGeometryTestCase { + @Override + protected Polygon createTestInstance() { + return randomPolygon(); + } + + public void testBasicSerialization() throws IOException, ParseException { + assertEquals("polygon ((3.0 1.0, 4.0 2.0, 5.0 3.0, 3.0 1.0))", + WellKnownText.toWKT(new Polygon(new LinearRing(new double[]{1, 2, 3, 1}, new double[]{3, 4, 5, 3})))); + assertEquals(new Polygon(new LinearRing(new double[]{1, 2, 3, 1}, new double[]{3, 4, 5, 3})), + WellKnownText.fromWKT("polygon ((3 1, 4 2, 5 3, 3 1))")); + + assertEquals("polygon EMPTY", WellKnownText.toWKT(Polygon.EMPTY)); + assertEquals(Polygon.EMPTY, WellKnownText.fromWKT("polygon EMPTY)")); + } + + public void testInitValidation() { + IllegalArgumentException ex = expectThrows(IllegalArgumentException.class, + () -> new Polygon(new LinearRing(new double[]{1, 2, 1}, new double[]{3, 4, 3}))); + assertEquals("at least 4 polygon points required", ex.getMessage()); + + ex = expectThrows(IllegalArgumentException.class, + () -> new Polygon(new LinearRing(new double[]{1, 2, 3, 1}, new double[]{3, 4, 5, 3}), null)); + assertEquals("holes must not be null", ex.getMessage()); + } +} diff --git a/libs/geo/src/test/java/org/elasticsearch/geo/geometry/RectangleTests.java b/libs/geo/src/test/java/org/elasticsearch/geo/geometry/RectangleTests.java new file mode 100644 index 0000000000000..fa5cbcd0a8f05 --- /dev/null +++ b/libs/geo/src/test/java/org/elasticsearch/geo/geometry/RectangleTests.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.geo.geometry; + +import org.elasticsearch.geo.utils.WellKnownText; + +import java.io.IOException; +import java.text.ParseException; + +public class RectangleTests extends BaseGeometryTestCase { + @Override + protected Rectangle createTestInstance() { + return randomRectangle(); + } + + public void testBasicSerialization() throws IOException, ParseException { + assertEquals("bbox (10.0, 20.0, 40.0, 30.0)", WellKnownText.toWKT(new Rectangle(30, 40, 10, 20))); + assertEquals(new Rectangle(30, 40, 10, 20), WellKnownText.fromWKT("bbox (10.0, 20.0, 40.0, 30.0)")); + + assertEquals("bbox EMPTY", WellKnownText.toWKT(Rectangle.EMPTY)); + assertEquals(Rectangle.EMPTY, WellKnownText.fromWKT("bbox EMPTY)")); + } + + public void testInitValidation() { + IllegalArgumentException ex = expectThrows(IllegalArgumentException.class, () -> new Rectangle(100, 1, 2, 3)); + assertEquals("invalid latitude 100.0; must be between -90.0 and 90.0", ex.getMessage()); + + ex = expectThrows(IllegalArgumentException.class, () -> new Rectangle(1, 2, 200, 3)); + assertEquals("invalid longitude 200.0; must be between -180.0 and 180.0", ex.getMessage()); + + ex = expectThrows(IllegalArgumentException.class, () -> new Rectangle(2, 1, 2, 3)); + assertEquals("max lat cannot be less than min lat", ex.getMessage()); + } +} diff --git a/settings.gradle b/settings.gradle index 43313f7236cb7..40fb419b0bf3c 100644 --- a/settings.gradle +++ b/settings.gradle @@ -95,6 +95,7 @@ if (isEclipse) { projects << 'libs:x-content-tests' projects << 'libs:secure-sm-tests' projects << 'libs:grok-tests' + projects << 'libs:geo-tests' } include projects.toArray(new String[0]) @@ -130,7 +131,10 @@ if (isEclipse) { project(":libs:grok").buildFileName = 'eclipse-build.gradle' project(":libs:grok-tests").projectDir = new File(rootProject.projectDir, 'libs/grok/src/test') project(":libs:grok-tests").buildFileName = 'eclipse-build.gradle' -} + project(":libs:geo").projectDir = new File(rootProject.projectDir, 'libs/geo/src/main') + project(":libs:geo").buildFileName = 'eclipse-build.gradle' + project(":libs:geo-tests").projectDir = new File(rootProject.projectDir, 'libs/geo/src/test') + project(":libs:geo-tests").buildFileName = 'eclipse-build.gradle'} // look for extra plugins for elasticsearch File extraProjects = new File(rootProject.projectDir.parentFile, "${dirName}-extra") @@ -141,3 +145,4 @@ if (extraProjects.exists()) { } project(":libs:cli").name = 'elasticsearch-cli' +project(":libs:geo").name = 'elasticsearch-geo' From 65e42ab63bb6e764ea9f547b42bf82b3ef465881 Mon Sep 17 00:00:00 2001 From: Marios Trivyzas Date: Tue, 15 Jan 2019 18:04:31 +0200 Subject: [PATCH 181/186] [TEST] Muted TokenBackwardsCompatibilityIT.* Relates to #37379 --- .../elasticsearch/upgrades/TokenBackwardsCompatibilityIT.java | 2 ++ 1 file changed, 2 insertions(+) diff --git a/x-pack/qa/rolling-upgrade/src/test/java/org/elasticsearch/upgrades/TokenBackwardsCompatibilityIT.java b/x-pack/qa/rolling-upgrade/src/test/java/org/elasticsearch/upgrades/TokenBackwardsCompatibilityIT.java index 1f0b8cffe20b0..b2eff72c71551 100644 --- a/x-pack/qa/rolling-upgrade/src/test/java/org/elasticsearch/upgrades/TokenBackwardsCompatibilityIT.java +++ b/x-pack/qa/rolling-upgrade/src/test/java/org/elasticsearch/upgrades/TokenBackwardsCompatibilityIT.java @@ -7,6 +7,7 @@ import org.apache.http.HttpHeaders; import org.apache.http.HttpHost; +import org.apache.lucene.util.LuceneTestCase.AwaitsFix; import org.elasticsearch.Version; import org.elasticsearch.client.Request; import org.elasticsearch.client.RequestOptions; @@ -26,6 +27,7 @@ import static org.elasticsearch.common.xcontent.XContentFactory.jsonBuilder; import static org.hamcrest.Matchers.equalTo; +@AwaitsFix(bugUrl = "https://github.com/elastic/elasticsearch/issues/37379") public class TokenBackwardsCompatibilityIT extends AbstractUpgradeTestCase { public void testGeneratingTokenInOldCluster() throws Exception { From 68e2d36fa301de9d85ec01861cbc8f706296d07c Mon Sep 17 00:00:00 2001 From: Nhat Nguyen Date: Tue, 15 Jan 2019 11:18:55 -0500 Subject: [PATCH 182/186] Adjust bwc version for max_concurrent_file_chunks Relates #36981 --- .../elasticsearch/indices/recovery/RecoverySourceHandler.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/server/src/main/java/org/elasticsearch/indices/recovery/RecoverySourceHandler.java b/server/src/main/java/org/elasticsearch/indices/recovery/RecoverySourceHandler.java index b1585251934dc..e3fd1bf73d307 100644 --- a/server/src/main/java/org/elasticsearch/indices/recovery/RecoverySourceHandler.java +++ b/server/src/main/java/org/elasticsearch/indices/recovery/RecoverySourceHandler.java @@ -111,7 +111,7 @@ public RecoverySourceHandler(final IndexShard shard, RecoveryTargetHandler recov this.logger = Loggers.getLogger(getClass(), request.shardId(), "recover to " + request.targetNode().getName()); this.chunkSizeInBytes = fileChunkSizeInBytes; // if the target is on an old version, it won't be able to handle out-of-order file chunks. - this.maxConcurrentFileChunks = request.targetNode().getVersion().onOrAfter(Version.V_7_0_0) ? maxConcurrentFileChunks : 1; + this.maxConcurrentFileChunks = request.targetNode().getVersion().onOrAfter(Version.V_6_7_0) ? maxConcurrentFileChunks : 1; } public StartRecoveryRequest getRequest() { From 6129e9d9ddeb38be1ba88b2451503c7faa35db32 Mon Sep 17 00:00:00 2001 From: Marios Trivyzas Date: Tue, 15 Jan 2019 18:24:42 +0200 Subject: [PATCH 183/186] Revert "[TEST] Muted TokenBackwardsCompatibilityIT.*" This reverts commit 65e42ab63bb6e764ea9f547b42bf82b3ef465881. The test is only failing in 6.x not master. --- .../elasticsearch/upgrades/TokenBackwardsCompatibilityIT.java | 2 -- 1 file changed, 2 deletions(-) diff --git a/x-pack/qa/rolling-upgrade/src/test/java/org/elasticsearch/upgrades/TokenBackwardsCompatibilityIT.java b/x-pack/qa/rolling-upgrade/src/test/java/org/elasticsearch/upgrades/TokenBackwardsCompatibilityIT.java index b2eff72c71551..1f0b8cffe20b0 100644 --- a/x-pack/qa/rolling-upgrade/src/test/java/org/elasticsearch/upgrades/TokenBackwardsCompatibilityIT.java +++ b/x-pack/qa/rolling-upgrade/src/test/java/org/elasticsearch/upgrades/TokenBackwardsCompatibilityIT.java @@ -7,7 +7,6 @@ import org.apache.http.HttpHeaders; import org.apache.http.HttpHost; -import org.apache.lucene.util.LuceneTestCase.AwaitsFix; import org.elasticsearch.Version; import org.elasticsearch.client.Request; import org.elasticsearch.client.RequestOptions; @@ -27,7 +26,6 @@ import static org.elasticsearch.common.xcontent.XContentFactory.jsonBuilder; import static org.hamcrest.Matchers.equalTo; -@AwaitsFix(bugUrl = "https://github.com/elastic/elasticsearch/issues/37379") public class TokenBackwardsCompatibilityIT extends AbstractUpgradeTestCase { public void testGeneratingTokenInOldCluster() throws Exception { From 44e83f30e2e11c22f9bd49ef9354e9b97f261f78 Mon Sep 17 00:00:00 2001 From: Lisa Cawley Date: Tue, 15 Jan 2019 08:46:36 -0800 Subject: [PATCH 184/186] [DOCS] Edits rollup API description (#37444) --- docs/reference/rollup/apis/rollup-caps.asciidoc | 17 ++++++++++------- 1 file changed, 10 insertions(+), 7 deletions(-) diff --git a/docs/reference/rollup/apis/rollup-caps.asciidoc b/docs/reference/rollup/apis/rollup-caps.asciidoc index b4a0d4470a7bd..a0de0f99f9872 100644 --- a/docs/reference/rollup/apis/rollup-caps.asciidoc +++ b/docs/reference/rollup/apis/rollup-caps.asciidoc @@ -8,16 +8,19 @@ experimental[] -This API returns the rollup capabilities that have been configured for an index or index pattern. This API is useful -because a rollup job is often configured to rollup only a subset of fields from the source index. Furthermore, only -certain aggregations can be configured for various fields, leading to a limited subset of functionality depending on -that configuration. +This API returns the capabilities of any rollup jobs that have been configured +for a specific index or index pattern. -This API will allow you to inspect an index and determine: +This API is useful because a rollup job is often configured to rollup only a +subset of fields from the source index. Furthermore, only certain aggregations +can be configured for various fields, leading to a limited subset of +functionality depending on that configuration. + +This API enables you to inspect an index and determine: 1. Does this index have associated rollup data somewhere in the cluster? -2. If yes to the first question, what fields were rolled up, what aggregations can be performed, and where does the data -live? +2. If yes to the first question, what fields were rolled up, what aggregations +can be performed, and where does the data live? ==== Request From 7c11b05c2877114c8003aa233587d8d3021ea013 Mon Sep 17 00:00:00 2001 From: David Kyle Date: Tue, 15 Jan 2019 17:19:58 +0000 Subject: [PATCH 185/186] [ML] Remove unused code from the JIndex project (#37477) --- .../xpack/core/ml/MlMetadata.java | 113 +------ .../ml/job/persistence/JobConfigProvider.java | 48 --- .../xpack/ml/MlMetadataTests.java | 298 +----------------- .../ml/integration/JobConfigProviderIT.java | 17 - 4 files changed, 11 insertions(+), 465 deletions(-) diff --git a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/ml/MlMetadata.java b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/ml/MlMetadata.java index 76de682e99d35..efd78cfaf7586 100644 --- a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/ml/MlMetadata.java +++ b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/ml/MlMetadata.java @@ -5,7 +5,6 @@ */ package org.elasticsearch.xpack.core.ml; -import org.elasticsearch.ResourceNotFoundException; import org.elasticsearch.Version; import org.elasticsearch.cluster.AbstractDiffable; import org.elasticsearch.cluster.ClusterState; @@ -22,19 +21,12 @@ import org.elasticsearch.common.xcontent.ObjectParser; import org.elasticsearch.common.xcontent.ToXContent; import org.elasticsearch.common.xcontent.XContentBuilder; -import org.elasticsearch.persistent.PersistentTasksCustomMetaData; -import org.elasticsearch.persistent.PersistentTasksCustomMetaData.PersistentTask; import org.elasticsearch.xpack.core.ClientHelper; import org.elasticsearch.xpack.core.XPackPlugin; import org.elasticsearch.xpack.core.ml.datafeed.DatafeedConfig; import org.elasticsearch.xpack.core.ml.datafeed.DatafeedJobValidator; -import org.elasticsearch.xpack.core.ml.datafeed.DatafeedState; -import org.elasticsearch.xpack.core.ml.datafeed.DatafeedUpdate; 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.config.JobTaskState; import org.elasticsearch.xpack.core.ml.job.groups.GroupOrJobLookup; -import org.elasticsearch.xpack.core.ml.job.messages.Messages; import org.elasticsearch.xpack.core.ml.utils.ExceptionsHelper; import org.elasticsearch.xpack.core.ml.utils.NameResolver; import org.elasticsearch.xpack.core.ml.utils.ToXContentParams; @@ -49,7 +41,6 @@ import java.util.Set; import java.util.SortedMap; import java.util.TreeMap; -import java.util.function.Supplier; import java.util.stream.Collectors; public class MlMetadata implements XPackPlugin.XPackMetaDataCustom { @@ -82,19 +73,10 @@ public Map getJobs() { return jobs; } - public boolean isGroupOrJob(String id) { - return groupOrJobLookup.isGroupOrJob(id); - } - public Set expandJobIds(String expression, boolean allowNoJobs) { return groupOrJobLookup.expandJobIds(expression, allowNoJobs); } - public boolean isJobDeleting(String jobId) { - Job job = jobs.get(jobId); - return job == null || job.isDeleting(); - } - public SortedMap getDatafeeds() { return datafeeds; } @@ -278,20 +260,9 @@ public Builder putJob(Job job, boolean overwrite) { return this; } - public Builder deleteJob(String jobId, PersistentTasksCustomMetaData tasks) { - checkJobHasNoDatafeed(jobId); - - JobState jobState = MlTasks.getJobState(jobId, tasks); - if (jobState.isAnyOf(JobState.CLOSED, JobState.FAILED) == false) { - throw ExceptionsHelper.conflictStatusException("Unexpected job state [" + jobState + "], expected [" + - JobState.CLOSED + " or " + JobState.FAILED + "]"); - } - Job job = jobs.remove(jobId); - if (job == null) { - throw new ResourceNotFoundException("job [" + jobId + "] does not exist"); - } - if (job.isDeleting() == false) { - throw ExceptionsHelper.conflictStatusException("Cannot delete job [" + jobId + "] because it hasn't marked as deleted"); + public Builder putJobs(Collection jobs) { + for (Job job : jobs) { + putJob(job, true); } return this; } @@ -300,6 +271,7 @@ public Builder putDatafeed(DatafeedConfig datafeedConfig, Map he if (datafeeds.containsKey(datafeedConfig.getId())) { throw ExceptionsHelper.datafeedAlreadyExists(datafeedConfig.getId()); } + String jobId = datafeedConfig.getJobId(); checkJobIsAvailableForDatafeed(jobId); Job job = jobs.get(jobId); @@ -331,54 +303,10 @@ private void checkJobIsAvailableForDatafeed(String jobId) { } } - public Builder updateDatafeed(DatafeedUpdate update, PersistentTasksCustomMetaData persistentTasks, Map headers) { - String datafeedId = update.getId(); - DatafeedConfig oldDatafeedConfig = datafeeds.get(datafeedId); - if (oldDatafeedConfig == null) { - throw ExceptionsHelper.missingDatafeedException(datafeedId); - } - checkDatafeedIsStopped(() -> Messages.getMessage(Messages.DATAFEED_CANNOT_UPDATE_IN_CURRENT_STATE, datafeedId, - DatafeedState.STARTED), datafeedId, persistentTasks); - DatafeedConfig newDatafeedConfig = update.apply(oldDatafeedConfig, headers); - if (newDatafeedConfig.getJobId().equals(oldDatafeedConfig.getJobId()) == false) { - checkJobIsAvailableForDatafeed(newDatafeedConfig.getJobId()); - } - Job job = jobs.get(newDatafeedConfig.getJobId()); - DatafeedJobValidator.validate(newDatafeedConfig, job); - datafeeds.put(datafeedId, newDatafeedConfig); - return this; - } - - public Builder removeDatafeed(String datafeedId, PersistentTasksCustomMetaData persistentTasks) { - DatafeedConfig datafeed = datafeeds.get(datafeedId); - if (datafeed == null) { - throw ExceptionsHelper.missingDatafeedException(datafeedId); - } - checkDatafeedIsStopped(() -> Messages.getMessage(Messages.DATAFEED_CANNOT_DELETE_IN_CURRENT_STATE, datafeedId, - DatafeedState.STARTED), datafeedId, persistentTasks); - datafeeds.remove(datafeedId); - return this; - } - private Optional getDatafeedByJobId(String jobId) { return datafeeds.values().stream().filter(s -> s.getJobId().equals(jobId)).findFirst(); } - private void checkDatafeedIsStopped(Supplier msg, String datafeedId, PersistentTasksCustomMetaData persistentTasks) { - if (persistentTasks != null) { - if (persistentTasks.getTask(MlTasks.datafeedTaskId(datafeedId)) != null) { - throw ExceptionsHelper.conflictStatusException(msg.get()); - } - } - } - - public Builder putJobs(Collection jobs) { - for (Job job : jobs) { - putJob(job, true); - } - return this; - } - public Builder putDatafeeds(Collection datafeeds) { for (DatafeedConfig datafeed : datafeeds) { this.datafeeds.put(datafeed.getId(), datafeed); @@ -389,39 +317,6 @@ public Builder putDatafeeds(Collection datafeeds) { public MlMetadata build() { return new MlMetadata(jobs, datafeeds); } - - public void markJobAsDeleting(String jobId, PersistentTasksCustomMetaData tasks, boolean allowDeleteOpenJob) { - Job job = jobs.get(jobId); - if (job == null) { - throw ExceptionsHelper.missingJobException(jobId); - } - if (job.isDeleting()) { - // Job still exists but is already being deleted - return; - } - - checkJobHasNoDatafeed(jobId); - - if (allowDeleteOpenJob == false) { - PersistentTask jobTask = MlTasks.getJobTask(jobId, tasks); - if (jobTask != null) { - JobTaskState jobTaskState = (JobTaskState) jobTask.getState(); - throw ExceptionsHelper.conflictStatusException("Cannot delete job [" + jobId + "] because the job is " - + ((jobTaskState == null) ? JobState.OPENING : jobTaskState.getState())); - } - } - Job.Builder jobBuilder = new Job.Builder(job); - jobBuilder.setDeleting(true); - putJob(jobBuilder.build(), true); - } - - void checkJobHasNoDatafeed(String jobId) { - Optional datafeed = getDatafeedByJobId(jobId); - if (datafeed.isPresent()) { - throw ExceptionsHelper.conflictStatusException("Cannot delete job [" + jobId + "] because datafeed [" - + datafeed.get().getId() + "] refers to it"); - } - } } public static MlMetadata getMlMetadata(ClusterState state) { diff --git a/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/job/persistence/JobConfigProvider.java b/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/job/persistence/JobConfigProvider.java index 73b1fe155fc3c..a21e6bc8c133c 100644 --- a/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/job/persistence/JobConfigProvider.java +++ b/x-pack/plugin/ml/src/main/java/org/elasticsearch/xpack/ml/job/persistence/JobConfigProvider.java @@ -19,9 +19,6 @@ import org.elasticsearch.action.get.GetAction; import org.elasticsearch.action.get.GetRequest; import org.elasticsearch.action.get.GetResponse; -import org.elasticsearch.action.get.MultiGetItemResponse; -import org.elasticsearch.action.get.MultiGetRequest; -import org.elasticsearch.action.get.MultiGetResponse; import org.elasticsearch.action.index.IndexAction; import org.elasticsearch.action.index.IndexRequest; import org.elasticsearch.action.index.IndexResponse; @@ -184,51 +181,6 @@ public void onFailure(Exception e) { }, client::get); } - /** - * Get the list anomaly detector jobs specified by {@code jobIds}. - * - * WARNING: errors are silently ignored, if a job is not found a - * {@code ResourceNotFoundException} is not thrown. Only found - * jobs are returned, this size of the returned jobs list could - * be different to the size of the requested ids list. - * - * @param jobIds The jobs to get - * @param listener Jobs listener - */ - public void getJobs(List jobIds, ActionListener> listener) { - MultiGetRequest multiGetRequest = new MultiGetRequest(); - jobIds.forEach(jobId -> multiGetRequest.add(AnomalyDetectorsIndex.configIndexName(), - ElasticsearchMappings.DOC_TYPE, Job.documentId(jobId))); - - List jobs = new ArrayList<>(); - executeAsyncWithOrigin(client.threadPool().getThreadContext(), ML_ORIGIN, multiGetRequest, new ActionListener() { - @Override - public void onResponse(MultiGetResponse multiGetResponse) { - - MultiGetItemResponse[] responses = multiGetResponse.getResponses(); - for (MultiGetItemResponse response : responses) { - GetResponse getResponse = response.getResponse(); - if (getResponse.isExists()) { - BytesReference source = getResponse.getSourceAsBytesRef(); - try { - Job.Builder job = parseJobLenientlyFromSource(source); - jobs.add(job); - } catch (IOException e) { - logger.error("Error parsing job configuration [" + response.getId() + "]"); - } - } - } - - listener.onResponse(jobs); - } - - @Override - public void onFailure(Exception e) { - listener.onFailure(e); - } - }, client::multiGet); - } - /** * Delete the anomaly detector job config document. * {@code errorIfMissing} controls whether or not an error is returned diff --git a/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/MlMetadataTests.java b/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/MlMetadataTests.java index 2377bc5921eb2..637b1089d9eb2 100644 --- a/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/MlMetadataTests.java +++ b/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/MlMetadataTests.java @@ -5,44 +5,30 @@ */ package org.elasticsearch.xpack.ml; -import org.elasticsearch.ElasticsearchStatusException; import org.elasticsearch.ResourceAlreadyExistsException; -import org.elasticsearch.ResourceNotFoundException; import org.elasticsearch.common.io.stream.NamedWriteableRegistry; import org.elasticsearch.common.io.stream.Writeable; import org.elasticsearch.common.settings.Settings; -import org.elasticsearch.common.unit.TimeValue; import org.elasticsearch.common.xcontent.NamedXContentRegistry; import org.elasticsearch.common.xcontent.XContentParser; -import org.elasticsearch.rest.RestStatus; import org.elasticsearch.search.SearchModule; import org.elasticsearch.test.AbstractSerializingTestCase; import org.elasticsearch.xpack.core.ml.MlMetadata; -import org.elasticsearch.xpack.core.ml.MlTasks; -import org.elasticsearch.xpack.core.ml.action.StartDatafeedAction; import org.elasticsearch.xpack.core.ml.datafeed.DatafeedConfig; import org.elasticsearch.xpack.core.ml.datafeed.DatafeedConfigTests; -import org.elasticsearch.xpack.core.ml.datafeed.DatafeedUpdate; import org.elasticsearch.xpack.core.ml.job.config.AnalysisConfig; 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.config.JobTests; -import org.elasticsearch.persistent.PersistentTasksCustomMetaData; -import org.elasticsearch.xpack.core.security.authc.AuthenticationServiceField; +import java.util.ArrayList; import java.util.Collections; -import java.util.Date; -import java.util.HashMap; +import java.util.List; import java.util.Map; import static org.elasticsearch.xpack.core.ml.job.config.JobTests.buildJobBuilder; -import static org.elasticsearch.persistent.PersistentTasksCustomMetaData.INITIAL_ASSIGNMENT; -import static org.elasticsearch.xpack.ml.action.TransportOpenJobActionTests.addJobTask; import static org.elasticsearch.xpack.ml.datafeed.DatafeedManagerTests.createDatafeedConfig; -import static org.elasticsearch.xpack.ml.datafeed.DatafeedManagerTests.createDatafeedJob; import static org.hamcrest.Matchers.contains; import static org.hamcrest.Matchers.equalTo; -import static org.hamcrest.Matchers.hasEntry; import static org.hamcrest.Matchers.nullValue; import static org.hamcrest.Matchers.sameInstance; @@ -122,277 +108,6 @@ public void testPutJob() { assertThat(result.getJobs().get("2"), sameInstance(job2Attempt2)); } - public void testRemoveJob() { - Job.Builder jobBuilder = buildJobBuilder("1"); - jobBuilder.setDeleting(true); - Job job1 = jobBuilder.build(); - MlMetadata.Builder builder = new MlMetadata.Builder(); - builder.putJob(job1, false); - - MlMetadata result = builder.build(); - assertThat(result.getJobs().get("1"), sameInstance(job1)); - assertThat(result.getDatafeeds().get("1"), nullValue()); - - builder = new MlMetadata.Builder(result); - assertThat(result.getJobs().get("1"), sameInstance(job1)); - assertThat(result.getDatafeeds().get("1"), nullValue()); - - builder.deleteJob("1", new PersistentTasksCustomMetaData(0L, Collections.emptyMap())); - result = builder.build(); - assertThat(result.getJobs().get("1"), nullValue()); - assertThat(result.getDatafeeds().get("1"), nullValue()); - } - - public void testRemoveJob_failBecauseJobIsOpen() { - Job job1 = buildJobBuilder("1").build(); - MlMetadata.Builder builder1 = new MlMetadata.Builder(); - builder1.putJob(job1, false); - - MlMetadata result = builder1.build(); - assertThat(result.getJobs().get("1"), sameInstance(job1)); - assertThat(result.getDatafeeds().get("1"), nullValue()); - - PersistentTasksCustomMetaData.Builder tasksBuilder = PersistentTasksCustomMetaData.builder(); - addJobTask("1", null, JobState.CLOSED, tasksBuilder); - MlMetadata.Builder builder2 = new MlMetadata.Builder(result); - ElasticsearchStatusException e = expectThrows(ElasticsearchStatusException.class, - () -> builder2.deleteJob("1", tasksBuilder.build())); - assertThat(e.status(), equalTo(RestStatus.CONFLICT)); - } - - public void testRemoveJob_failDatafeedRefersToJob() { - Job job1 = createDatafeedJob().build(new Date()); - DatafeedConfig datafeedConfig1 = createDatafeedConfig("datafeed1", job1.getId()).build(); - MlMetadata.Builder builder = new MlMetadata.Builder(); - builder.putJob(job1, false); - builder.putDatafeed(datafeedConfig1, Collections.emptyMap()); - - ElasticsearchStatusException e = expectThrows(ElasticsearchStatusException.class, - () -> builder.deleteJob(job1.getId(), new PersistentTasksCustomMetaData(0L, Collections.emptyMap()))); - assertThat(e.status(), equalTo(RestStatus.CONFLICT)); - String expectedMsg = "Cannot delete job [" + job1.getId() + "] because datafeed [" + datafeedConfig1.getId() + "] refers to it"; - assertThat(e.getMessage(), equalTo(expectedMsg)); - } - - public void testRemoveJob_failBecauseJobDoesNotExist() { - MlMetadata.Builder builder1 = new MlMetadata.Builder(); - expectThrows(ResourceNotFoundException.class, - () -> builder1.deleteJob("1", new PersistentTasksCustomMetaData(0L, Collections.emptyMap()))); - } - - public void testCrudDatafeed() { - Job job1 = createDatafeedJob().build(new Date()); - DatafeedConfig datafeedConfig1 = createDatafeedConfig("datafeed1", job1.getId()).build(); - MlMetadata.Builder builder = new MlMetadata.Builder(); - builder.putJob(job1, false); - builder.putDatafeed(datafeedConfig1, Collections.emptyMap()); - - MlMetadata result = builder.build(); - assertThat(result.getJobs().get("job_id"), sameInstance(job1)); - assertThat(result.getDatafeeds().get("datafeed1"), sameInstance(datafeedConfig1)); - - builder = new MlMetadata.Builder(result); - builder.removeDatafeed("datafeed1", new PersistentTasksCustomMetaData(0, Collections.emptyMap())); - result = builder.build(); - assertThat(result.getJobs().get("job_id"), sameInstance(job1)); - assertThat(result.getDatafeeds().get("datafeed1"), nullValue()); - } - - public void testPutDatafeed_failBecauseJobDoesNotExist() { - DatafeedConfig datafeedConfig1 = createDatafeedConfig("datafeed1", "missing-job").build(); - MlMetadata.Builder builder = new MlMetadata.Builder(); - - expectThrows(ResourceNotFoundException.class, () -> builder.putDatafeed(datafeedConfig1, Collections.emptyMap())); - } - - public void testPutDatafeed_failBecauseJobIsBeingDeleted() { - Job job1 = createDatafeedJob().setDeleting(true).build(new Date()); - DatafeedConfig datafeedConfig1 = createDatafeedConfig("datafeed1", job1.getId()).build(); - MlMetadata.Builder builder = new MlMetadata.Builder(); - builder.putJob(job1, false); - - expectThrows(ResourceNotFoundException.class, () -> builder.putDatafeed(datafeedConfig1, Collections.emptyMap())); - } - - public void testPutDatafeed_failBecauseDatafeedIdIsAlreadyTaken() { - Job job1 = createDatafeedJob().build(new Date()); - DatafeedConfig datafeedConfig1 = createDatafeedConfig("datafeed1", job1.getId()).build(); - MlMetadata.Builder builder = new MlMetadata.Builder(); - builder.putJob(job1, false); - builder.putDatafeed(datafeedConfig1, Collections.emptyMap()); - - expectThrows(ResourceAlreadyExistsException.class, () -> builder.putDatafeed(datafeedConfig1, Collections.emptyMap())); - } - - public void testPutDatafeed_failBecauseJobAlreadyHasDatafeed() { - Job job1 = createDatafeedJob().build(new Date()); - DatafeedConfig datafeedConfig1 = createDatafeedConfig("datafeed1", job1.getId()).build(); - DatafeedConfig datafeedConfig2 = createDatafeedConfig("datafeed2", job1.getId()).build(); - MlMetadata.Builder builder = new MlMetadata.Builder(); - builder.putJob(job1, false); - builder.putDatafeed(datafeedConfig1, Collections.emptyMap()); - - ElasticsearchStatusException e = expectThrows(ElasticsearchStatusException.class, - () -> builder.putDatafeed(datafeedConfig2, Collections.emptyMap())); - assertThat(e.status(), equalTo(RestStatus.CONFLICT)); - } - - public void testPutDatafeed_failBecauseJobIsNotCompatibleForDatafeed() { - Job.Builder job1 = createDatafeedJob(); - Date now = new Date(); - AnalysisConfig.Builder analysisConfig = new AnalysisConfig.Builder(job1.build(now).getAnalysisConfig()); - analysisConfig.setLatency(TimeValue.timeValueHours(1)); - job1.setAnalysisConfig(analysisConfig); - DatafeedConfig datafeedConfig1 = createDatafeedConfig("datafeed1", job1.getId()).build(); - MlMetadata.Builder builder = new MlMetadata.Builder(); - builder.putJob(job1.build(now), false); - - expectThrows(ElasticsearchStatusException.class, () -> builder.putDatafeed(datafeedConfig1, Collections.emptyMap())); - } - - public void testPutDatafeed_setsSecurityHeaders() { - Job datafeedJob = createDatafeedJob().build(new Date()); - DatafeedConfig datafeedConfig = createDatafeedConfig("datafeed1", datafeedJob.getId()).build(); - MlMetadata.Builder builder = new MlMetadata.Builder(); - builder.putJob(datafeedJob, false); - - Map headers = new HashMap<>(); - headers.put("unrelated_header", "unrelated_header_value"); - headers.put(AuthenticationServiceField.RUN_AS_USER_HEADER, "permitted_run_as_user"); - builder.putDatafeed(datafeedConfig, headers); - MlMetadata metadata = builder.build(); - assertThat(metadata.getDatafeed("datafeed1").getHeaders().size(), equalTo(1)); - assertThat(metadata.getDatafeed("datafeed1").getHeaders(), - hasEntry(AuthenticationServiceField.RUN_AS_USER_HEADER, "permitted_run_as_user")); - } - - public void testUpdateDatafeed() { - Job job1 = createDatafeedJob().build(new Date()); - DatafeedConfig datafeedConfig1 = createDatafeedConfig("datafeed1", job1.getId()).build(); - MlMetadata.Builder builder = new MlMetadata.Builder(); - builder.putJob(job1, false); - builder.putDatafeed(datafeedConfig1, Collections.emptyMap()); - MlMetadata beforeMetadata = builder.build(); - - DatafeedUpdate.Builder update = new DatafeedUpdate.Builder(datafeedConfig1.getId()); - update.setScrollSize(5000); - MlMetadata updatedMetadata = - new MlMetadata.Builder(beforeMetadata).updateDatafeed(update.build(), null, Collections.emptyMap()).build(); - - DatafeedConfig updatedDatafeed = updatedMetadata.getDatafeed(datafeedConfig1.getId()); - assertThat(updatedDatafeed.getJobId(), equalTo(datafeedConfig1.getJobId())); - assertThat(updatedDatafeed.getIndices(), equalTo(datafeedConfig1.getIndices())); - assertThat(updatedDatafeed.getScrollSize(), equalTo(5000)); - } - - public void testUpdateDatafeed_failBecauseDatafeedDoesNotExist() { - DatafeedUpdate.Builder update = new DatafeedUpdate.Builder("job_id"); - update.setScrollSize(5000); - expectThrows(ResourceNotFoundException.class, - () -> new MlMetadata.Builder().updateDatafeed(update.build(), null, Collections.emptyMap()).build()); - } - - public void testUpdateDatafeed_failBecauseDatafeedIsNotStopped() { - Job job1 = createDatafeedJob().build(new Date()); - DatafeedConfig datafeedConfig1 = createDatafeedConfig("datafeed1", job1.getId()).build(); - MlMetadata.Builder builder = new MlMetadata.Builder(); - builder.putJob(job1, false); - builder.putDatafeed(datafeedConfig1, Collections.emptyMap()); - MlMetadata beforeMetadata = builder.build(); - - PersistentTasksCustomMetaData.Builder tasksBuilder = PersistentTasksCustomMetaData.builder(); - StartDatafeedAction.DatafeedParams params = new StartDatafeedAction.DatafeedParams(datafeedConfig1.getId(), 0L); - tasksBuilder.addTask(MlTasks.datafeedTaskId("datafeed1"), MlTasks.DATAFEED_TASK_NAME, params, INITIAL_ASSIGNMENT); - PersistentTasksCustomMetaData tasksInProgress = tasksBuilder.build(); - - DatafeedUpdate.Builder update = new DatafeedUpdate.Builder(datafeedConfig1.getId()); - update.setScrollSize(5000); - - ElasticsearchStatusException e = expectThrows(ElasticsearchStatusException.class, - () -> new MlMetadata.Builder(beforeMetadata).updateDatafeed(update.build(), tasksInProgress, null)); - assertThat(e.status(), equalTo(RestStatus.CONFLICT)); - } - - public void testUpdateDatafeed_failBecauseNewJobIdDoesNotExist() { - Job job1 = createDatafeedJob().build(new Date()); - DatafeedConfig datafeedConfig1 = createDatafeedConfig("datafeed1", job1.getId()).build(); - MlMetadata.Builder builder = new MlMetadata.Builder(); - builder.putJob(job1, false); - builder.putDatafeed(datafeedConfig1, Collections.emptyMap()); - MlMetadata beforeMetadata = builder.build(); - - DatafeedUpdate.Builder update = new DatafeedUpdate.Builder(datafeedConfig1.getId()); - update.setJobId(job1.getId() + "_2"); - - expectThrows(ResourceNotFoundException.class, - () -> new MlMetadata.Builder(beforeMetadata).updateDatafeed(update.build(), null, Collections.emptyMap())); - } - - public void testUpdateDatafeed_failBecauseNewJobHasAnotherDatafeedAttached() { - Job job1 = createDatafeedJob().build(new Date()); - Job.Builder job2 = new Job.Builder(job1); - job2.setId(job1.getId() + "_2"); - DatafeedConfig datafeedConfig1 = createDatafeedConfig("datafeed1", job1.getId()).build(); - DatafeedConfig datafeedConfig2 = createDatafeedConfig("datafeed2", job2.getId()).build(); - MlMetadata.Builder builder = new MlMetadata.Builder(); - builder.putJob(job1, false); - builder.putJob(job2.build(), false); - builder.putDatafeed(datafeedConfig1, Collections.emptyMap()); - builder.putDatafeed(datafeedConfig2, Collections.emptyMap()); - MlMetadata beforeMetadata = builder.build(); - - DatafeedUpdate.Builder update = new DatafeedUpdate.Builder(datafeedConfig1.getId()); - update.setJobId(job2.getId()); - - ElasticsearchStatusException e = expectThrows(ElasticsearchStatusException.class, - () -> new MlMetadata.Builder(beforeMetadata).updateDatafeed(update.build(), null, Collections.emptyMap())); - assertThat(e.status(), equalTo(RestStatus.CONFLICT)); - assertThat(e.getMessage(), equalTo("A datafeed [datafeed2] already exists for job [job_id_2]")); - } - - public void testUpdateDatafeed_setsSecurityHeaders() { - Job datafeedJob = createDatafeedJob().build(new Date()); - DatafeedConfig datafeedConfig = createDatafeedConfig("datafeed1", datafeedJob.getId()).build(); - MlMetadata.Builder builder = new MlMetadata.Builder(); - builder.putJob(datafeedJob, false); - builder.putDatafeed(datafeedConfig, Collections.emptyMap()); - MlMetadata beforeMetadata = builder.build(); - assertTrue(beforeMetadata.getDatafeed("datafeed1").getHeaders().isEmpty()); - - DatafeedUpdate.Builder update = new DatafeedUpdate.Builder(datafeedConfig.getId()); - update.setQueryDelay(TimeValue.timeValueMinutes(5)); - - Map headers = new HashMap<>(); - headers.put("unrelated_header", "unrelated_header_value"); - headers.put(AuthenticationServiceField.RUN_AS_USER_HEADER, "permitted_run_as_user"); - MlMetadata afterMetadata = new MlMetadata.Builder(beforeMetadata).updateDatafeed(update.build(), null, headers).build(); - Map updatedHeaders = afterMetadata.getDatafeed("datafeed1").getHeaders(); - assertThat(updatedHeaders.size(), equalTo(1)); - assertThat(updatedHeaders, hasEntry(AuthenticationServiceField.RUN_AS_USER_HEADER, "permitted_run_as_user")); - } - - public void testRemoveDatafeed_failBecauseDatafeedStarted() { - Job job1 = createDatafeedJob().build(new Date()); - DatafeedConfig datafeedConfig1 = createDatafeedConfig("datafeed1", job1.getId()).build(); - MlMetadata.Builder builder = new MlMetadata.Builder(); - builder.putJob(job1, false); - builder.putDatafeed(datafeedConfig1, Collections.emptyMap()); - - MlMetadata result = builder.build(); - assertThat(result.getJobs().get("job_id"), sameInstance(job1)); - assertThat(result.getDatafeeds().get("datafeed1"), sameInstance(datafeedConfig1)); - - PersistentTasksCustomMetaData.Builder tasksBuilder = PersistentTasksCustomMetaData.builder(); - StartDatafeedAction.DatafeedParams params = new StartDatafeedAction.DatafeedParams("datafeed1", 0L); - tasksBuilder.addTask(MlTasks.datafeedTaskId("datafeed1"), MlTasks.DATAFEED_TASK_NAME, params, INITIAL_ASSIGNMENT); - PersistentTasksCustomMetaData tasksInProgress = tasksBuilder.build(); - - MlMetadata.Builder builder2 = new MlMetadata.Builder(result); - ElasticsearchStatusException e = expectThrows(ElasticsearchStatusException.class, - () -> builder2.removeDatafeed("datafeed1", tasksInProgress)); - assertThat(e.status(), equalTo(RestStatus.CONFLICT)); - } - public void testExpandJobIds() { MlMetadata mlMetadata = newMlMetadataWithJobs("bar-1", "foo-1", "foo-2").build(); @@ -404,12 +119,13 @@ public void testExpandJobIds() { public void testExpandDatafeedIds() { MlMetadata.Builder mlMetadataBuilder = newMlMetadataWithJobs("bar-1", "foo-1", "foo-2"); - mlMetadataBuilder.putDatafeed(createDatafeedConfig("bar-1-feed", "bar-1").build(), Collections.emptyMap()); - mlMetadataBuilder.putDatafeed(createDatafeedConfig("foo-1-feed", "foo-1").build(), Collections.emptyMap()); - mlMetadataBuilder.putDatafeed(createDatafeedConfig("foo-2-feed", "foo-2").build(), Collections.emptyMap()); + List datafeeds = new ArrayList<>(); + datafeeds.add(createDatafeedConfig("bar-1-feed", "bar-1").build()); + datafeeds.add(createDatafeedConfig("foo-1-feed", "foo-1").build()); + datafeeds.add(createDatafeedConfig("foo-2-feed", "foo-2").build()); + mlMetadataBuilder.putDatafeeds(datafeeds); MlMetadata mlMetadata = mlMetadataBuilder.build(); - assertThat(mlMetadata.expandDatafeedIds("_all", false), contains("bar-1-feed", "foo-1-feed", "foo-2-feed")); assertThat(mlMetadata.expandDatafeedIds("*", false), contains("bar-1-feed", "foo-1-feed", "foo-2-feed")); assertThat(mlMetadata.expandDatafeedIds("foo-*", false), contains("foo-1-feed", "foo-2-feed")); diff --git a/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/integration/JobConfigProviderIT.java b/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/integration/JobConfigProviderIT.java index c19e8b12295a8..0266247714dfe 100644 --- a/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/integration/JobConfigProviderIT.java +++ b/x-pack/plugin/ml/src/test/java/org/elasticsearch/xpack/ml/integration/JobConfigProviderIT.java @@ -186,23 +186,6 @@ public void testCrud() throws InterruptedException { assertEquals(DocWriteResponse.Result.NOT_FOUND, deleteJobResponseHolder.get().getResult()); } - public void testGetJobs() throws Exception { - putJob(createJob("nginx", null)); - putJob(createJob("tomcat", null)); - putJob(createJob("mysql", null)); - - List jobsToGet = Arrays.asList("nginx", "tomcat", "unknown-job"); - - AtomicReference> jobsHolder = new AtomicReference<>(); - AtomicReference exceptionHolder = new AtomicReference<>(); - blockingCall(actionListener -> jobConfigProvider.getJobs(jobsToGet, actionListener), jobsHolder, exceptionHolder); - assertNull(exceptionHolder.get()); - assertNotNull(jobsHolder.get()); - assertThat(jobsHolder.get(), hasSize(2)); - List foundIds = jobsHolder.get().stream().map(Job.Builder::getId).collect(Collectors.toList()); - assertThat(foundIds, containsInAnyOrder("nginx", "tomcat")); - } - public void testUpdateWithAValidationError() throws Exception { final String jobId = "bad-update-job"; From a2a40c50a0db185e34c317a7cf666de7717be707 Mon Sep 17 00:00:00 2001 From: David Turner Date: Tue, 15 Jan 2019 17:32:08 +0000 Subject: [PATCH 186/186] Report terms and version if cluster does not form (#37473) Adds the node's current term and the term and version of the the last-accepted cluster state to the message reported by the `ClusterFormationFailureHelper`, since these values may be of importance when tracking down a cluster formation failure. --- .../ClusterFormationFailureHelper.java | 9 +- .../cluster/coordination/Coordinator.java | 2 +- .../ClusterFormationFailureHelperTests.java | 104 ++++++++++-------- 3 files changed, 68 insertions(+), 47 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/cluster/coordination/ClusterFormationFailureHelper.java b/server/src/main/java/org/elasticsearch/cluster/coordination/ClusterFormationFailureHelper.java index 6338b49f5d14f..9fc408fc9479c 100644 --- a/server/src/main/java/org/elasticsearch/cluster/coordination/ClusterFormationFailureHelper.java +++ b/server/src/main/java/org/elasticsearch/cluster/coordination/ClusterFormationFailureHelper.java @@ -117,13 +117,15 @@ static class ClusterFormationState { private final ClusterState clusterState; private final List resolvedAddresses; private final List foundPeers; + private final long currentTerm; ClusterFormationState(Settings settings, ClusterState clusterState, List resolvedAddresses, - List foundPeers) { + List foundPeers, long currentTerm) { this.settings = settings; this.clusterState = clusterState; this.resolvedAddresses = resolvedAddresses; this.foundPeers = foundPeers; + this.currentTerm = currentTerm; } String getDescription() { @@ -131,8 +133,9 @@ String getDescription() { = StreamSupport.stream(clusterState.nodes().spliterator(), false).map(DiscoveryNode::toString).collect(Collectors.toList()); final String discoveryWillContinueDescription = String.format(Locale.ROOT, - "discovery will continue using %s from hosts providers and %s from last-known cluster state", - resolvedAddresses, clusterStateNodes); + "discovery will continue using %s from hosts providers and %s from last-known cluster state; " + + "node term %d, last-accepted version %d in term %d", + resolvedAddresses, clusterStateNodes, currentTerm, clusterState.version(), clusterState.term()); final String discoveryStateIgnoringQuorum = String.format(Locale.ROOT, "have discovered %s; %s", foundPeers, discoveryWillContinueDescription); 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 1d7ed0bdc7c2b..805eef9a89c06 100644 --- a/server/src/main/java/org/elasticsearch/cluster/coordination/Coordinator.java +++ b/server/src/main/java/org/elasticsearch/cluster/coordination/Coordinator.java @@ -182,7 +182,7 @@ public Coordinator(String nodeName, Settings settings, ClusterSettings clusterSe private ClusterFormationState getClusterFormationState() { return new ClusterFormationState(settings, getStateForMasterService(), peerFinder.getLastResolvedAddresses(), - StreamSupport.stream(peerFinder.getFoundPeers().spliterator(), false).collect(Collectors.toList())); + StreamSupport.stream(peerFinder.getFoundPeers().spliterator(), false).collect(Collectors.toList()), getCurrentTerm()); } private Runnable getOnLeaderFailure() { diff --git a/server/src/test/java/org/elasticsearch/cluster/coordination/ClusterFormationFailureHelperTests.java b/server/src/test/java/org/elasticsearch/cluster/coordination/ClusterFormationFailureHelperTests.java index 48e9ba42933df..129b29e1f21e5 100644 --- a/server/src/test/java/org/elasticsearch/cluster/coordination/ClusterFormationFailureHelperTests.java +++ b/server/src/test/java/org/elasticsearch/cluster/coordination/ClusterFormationFailureHelperTests.java @@ -68,7 +68,7 @@ public void testScheduling() { final ClusterFormationFailureHelper clusterFormationFailureHelper = new ClusterFormationFailureHelper(settingsBuilder.build(), () -> { warningCount.incrementAndGet(); - return new ClusterFormationState(Settings.EMPTY, clusterState, emptyList(), emptyList()); + return new ClusterFormationState(Settings.EMPTY, clusterState, emptyList(), emptyList(), 0L); }, deterministicTaskQueue.getThreadPool()); @@ -131,51 +131,57 @@ public void testScheduling() { public void testDescriptionOnMasterIneligibleNodes() { final DiscoveryNode localNode = new DiscoveryNode("local", buildNewFakeTransportAddress(), emptyMap(), emptySet(), Version.CURRENT); final ClusterState clusterState = ClusterState.builder(ClusterName.DEFAULT) - .nodes(DiscoveryNodes.builder().add(localNode).localNodeId(localNode.getId())).build(); + .version(12L).nodes(DiscoveryNodes.builder().add(localNode).localNodeId(localNode.getId())).build(); - assertThat(new ClusterFormationState(Settings.EMPTY, clusterState, emptyList(), emptyList()).getDescription(), + assertThat(new ClusterFormationState(Settings.EMPTY, clusterState, emptyList(), emptyList(), 15L).getDescription(), is("master not discovered yet: have discovered []; discovery will continue using [] from hosts providers and [" + localNode + - "] from last-known cluster state")); + "] from last-known cluster state; node term 15, last-accepted version 12 in term 0")); final TransportAddress otherAddress = buildNewFakeTransportAddress(); - assertThat(new ClusterFormationState(Settings.EMPTY, clusterState, singletonList(otherAddress), emptyList()).getDescription(), + assertThat(new ClusterFormationState(Settings.EMPTY, clusterState, singletonList(otherAddress), emptyList(), 16L).getDescription(), is("master not discovered yet: have discovered []; discovery will continue using [" + otherAddress + - "] from hosts providers and [" + localNode + "] from last-known cluster state")); + "] from hosts providers and [" + localNode + + "] from last-known cluster state; node term 16, last-accepted version 12 in term 0")); final DiscoveryNode otherNode = new DiscoveryNode("other", buildNewFakeTransportAddress(), Version.CURRENT); - assertThat(new ClusterFormationState(Settings.EMPTY, clusterState, emptyList(), singletonList(otherNode)).getDescription(), + assertThat(new ClusterFormationState(Settings.EMPTY, clusterState, emptyList(), singletonList(otherNode), 17L).getDescription(), is("master not discovered yet: have discovered [" + otherNode + "]; discovery will continue using [] from hosts providers and [" - + localNode + "] from last-known cluster state")); + + localNode + "] from last-known cluster state; node term 17, last-accepted version 12 in term 0")); } public void testDescriptionBeforeBootstrapping() { final DiscoveryNode localNode = new DiscoveryNode("local", buildNewFakeTransportAddress(), Version.CURRENT); final ClusterState clusterState = ClusterState.builder(ClusterName.DEFAULT) + .version(7L) + .metaData(MetaData.builder().coordinationMetaData(CoordinationMetaData.builder().term(4L).build())) .nodes(DiscoveryNodes.builder().add(localNode).localNodeId(localNode.getId())).build(); - assertThat(new ClusterFormationState(Settings.EMPTY, clusterState, emptyList(), emptyList()).getDescription(), + assertThat(new ClusterFormationState(Settings.EMPTY, clusterState, emptyList(), emptyList(), 1L).getDescription(), is("master not discovered yet, this node has not previously joined a bootstrapped (v7+) cluster, and " + "[cluster.initial_master_nodes] is empty on this node: have discovered []; " + - "discovery will continue using [] from hosts providers and [" + localNode + "] from last-known cluster state")); + "discovery will continue using [] from hosts providers and [" + localNode + + "] from last-known cluster state; node term 1, last-accepted version 7 in term 4")); final TransportAddress otherAddress = buildNewFakeTransportAddress(); - assertThat(new ClusterFormationState(Settings.EMPTY, clusterState, singletonList(otherAddress), emptyList()).getDescription(), + assertThat(new ClusterFormationState(Settings.EMPTY, clusterState, singletonList(otherAddress), emptyList(), 2L).getDescription(), is("master not discovered yet, this node has not previously joined a bootstrapped (v7+) cluster, and " + "[cluster.initial_master_nodes] is empty on this node: have discovered []; " + "discovery will continue using [" + otherAddress + "] from hosts providers and [" + localNode + - "] from last-known cluster state")); + "] from last-known cluster state; node term 2, last-accepted version 7 in term 4")); final DiscoveryNode otherNode = new DiscoveryNode("other", buildNewFakeTransportAddress(), Version.CURRENT); - assertThat(new ClusterFormationState(Settings.EMPTY, clusterState, emptyList(), singletonList(otherNode)).getDescription(), + assertThat(new ClusterFormationState(Settings.EMPTY, clusterState, emptyList(), singletonList(otherNode), 3L).getDescription(), is("master not discovered yet, this node has not previously joined a bootstrapped (v7+) cluster, and " + "[cluster.initial_master_nodes] is empty on this node: have discovered [" + otherNode + "]; " + - "discovery will continue using [] from hosts providers and [" + localNode + "] from last-known cluster state")); + "discovery will continue using [] from hosts providers and [" + localNode + + "] from last-known cluster state; node term 3, last-accepted version 7 in term 4")); assertThat(new ClusterFormationState(Settings.builder().putList(INITIAL_MASTER_NODES_SETTING.getKey(), "other").build(), - clusterState, emptyList(), emptyList()).getDescription(), + clusterState, emptyList(), emptyList(), 4L).getDescription(), is("master not discovered yet, this node has not previously joined a bootstrapped (v7+) cluster, and " + "this node must discover master-eligible nodes [other] to bootstrap a cluster: have discovered []; " + - "discovery will continue using [] from hosts providers and [" + localNode + "] from last-known cluster state")); + "discovery will continue using [] from hosts providers and [" + localNode + + "] from last-known cluster state; node term 4, last-accepted version 7 in term 4")); } private static VotingConfiguration config(String[] nodeIds) { @@ -199,75 +205,87 @@ public void testDescriptionAfterBootstrapping() { final ClusterState clusterState = state(localNode, "otherNode"); - assertThat(new ClusterFormationState(Settings.EMPTY, clusterState, emptyList(), emptyList()).getDescription(), + assertThat(new ClusterFormationState(Settings.EMPTY, clusterState, emptyList(), emptyList(), 0L).getDescription(), is("master not discovered or elected yet, an election requires a node with id [otherNode], " + "have discovered [] which is not a quorum; " + - "discovery will continue using [] from hosts providers and [" + localNode + "] from last-known cluster state")); + "discovery will continue using [] from hosts providers and [" + localNode + + "] from last-known cluster state; node term 0, last-accepted version 0 in term 0")); final TransportAddress otherAddress = buildNewFakeTransportAddress(); - assertThat(new ClusterFormationState(Settings.EMPTY, clusterState, singletonList(otherAddress), emptyList()).getDescription(), + assertThat(new ClusterFormationState(Settings.EMPTY, clusterState, singletonList(otherAddress), emptyList(), 0L).getDescription(), is("master not discovered or elected yet, an election requires a node with id [otherNode], " + "have discovered [] which is not a quorum; " + "discovery will continue using [" + otherAddress + "] from hosts providers and [" + localNode + - "] from last-known cluster state")); + "] from last-known cluster state; node term 0, last-accepted version 0 in term 0")); final DiscoveryNode otherNode = new DiscoveryNode("otherNode", buildNewFakeTransportAddress(), Version.CURRENT); - assertThat(new ClusterFormationState(Settings.EMPTY, clusterState, emptyList(), singletonList(otherNode)).getDescription(), + assertThat(new ClusterFormationState(Settings.EMPTY, clusterState, emptyList(), singletonList(otherNode), 0L).getDescription(), is("master not discovered or elected yet, an election requires a node with id [otherNode], " + "have discovered [" + otherNode + "] which is a quorum; " + - "discovery will continue using [] from hosts providers and [" + localNode + "] from last-known cluster state")); + "discovery will continue using [] from hosts providers and [" + localNode + + "] from last-known cluster state; node term 0, last-accepted version 0 in term 0")); final DiscoveryNode yetAnotherNode = new DiscoveryNode("yetAnotherNode", buildNewFakeTransportAddress(), Version.CURRENT); - assertThat(new ClusterFormationState(Settings.EMPTY, clusterState, emptyList(), singletonList(yetAnotherNode)).getDescription(), + assertThat(new ClusterFormationState(Settings.EMPTY, clusterState, emptyList(), singletonList(yetAnotherNode), 0L).getDescription(), is("master not discovered or elected yet, an election requires a node with id [otherNode], " + "have discovered [" + yetAnotherNode + "] which is not a quorum; " + - "discovery will continue using [] from hosts providers and [" + localNode + "] from last-known cluster state")); + "discovery will continue using [] from hosts providers and [" + localNode + + "] from last-known cluster state; node term 0, last-accepted version 0 in term 0")); - assertThat(new ClusterFormationState(Settings.EMPTY, state(localNode, "n1", "n2"), emptyList(), emptyList()).getDescription(), + assertThat(new ClusterFormationState(Settings.EMPTY, state(localNode, "n1", "n2"), emptyList(), emptyList(), 0L).getDescription(), is("master not discovered or elected yet, an election requires two nodes with ids [n1, n2], " + "have discovered [] which is not a quorum; " + - "discovery will continue using [] from hosts providers and [" + localNode + "] from last-known cluster state")); + "discovery will continue using [] from hosts providers and [" + localNode + + "] from last-known cluster state; node term 0, last-accepted version 0 in term 0")); - assertThat(new ClusterFormationState(Settings.EMPTY, state(localNode, "n1", "n2", "n3"), emptyList(), emptyList()).getDescription(), + assertThat(new ClusterFormationState(Settings.EMPTY, state(localNode, "n1", "n2", "n3"), emptyList(), emptyList(), 0L) + .getDescription(), is("master not discovered or elected yet, an election requires at least 2 nodes with ids from [n1, n2, n3], " + "have discovered [] which is not a quorum; " + - "discovery will continue using [] from hosts providers and [" + localNode + "] from last-known cluster state")); + "discovery will continue using [] from hosts providers and [" + localNode + + "] from last-known cluster state; node term 0, last-accepted version 0 in term 0")); - assertThat(new ClusterFormationState(Settings.EMPTY, state(localNode, "n1", "n2", "n3", "n4"), emptyList(), emptyList()) + assertThat(new ClusterFormationState(Settings.EMPTY, state(localNode, "n1", "n2", "n3", "n4"), emptyList(), emptyList(), 0L) .getDescription(), is("master not discovered or elected yet, an election requires at least 3 nodes with ids from [n1, n2, n3, n4], " + "have discovered [] which is not a quorum; " + - "discovery will continue using [] from hosts providers and [" + localNode + "] from last-known cluster state")); + "discovery will continue using [] from hosts providers and [" + localNode + + "] from last-known cluster state; node term 0, last-accepted version 0 in term 0")); - assertThat(new ClusterFormationState(Settings.EMPTY, state(localNode, "n1", "n2", "n3", "n4", "n5"), emptyList(), emptyList()) + assertThat(new ClusterFormationState(Settings.EMPTY, state(localNode, "n1", "n2", "n3", "n4", "n5"), emptyList(), emptyList(), 0L) .getDescription(), is("master not discovered or elected yet, an election requires at least 3 nodes with ids from [n1, n2, n3, n4, n5], " + "have discovered [] which is not a quorum; " + - "discovery will continue using [] from hosts providers and [" + localNode + "] from last-known cluster state")); + "discovery will continue using [] from hosts providers and [" + localNode + + "] from last-known cluster state; node term 0, last-accepted version 0 in term 0")); - assertThat(new ClusterFormationState(Settings.EMPTY, state(localNode, new String[]{"n1"}, new String[]{"n1"}), - emptyList(), emptyList()).getDescription(), + assertThat(new ClusterFormationState(Settings.EMPTY, state(localNode, new String[]{"n1"}, new String[]{"n1"}), emptyList(), + emptyList(), 0L).getDescription(), is("master not discovered or elected yet, an election requires a node with id [n1], " + "have discovered [] which is not a quorum; " + - "discovery will continue using [] from hosts providers and [" + localNode + "] from last-known cluster state")); + "discovery will continue using [] from hosts providers and [" + localNode + + "] from last-known cluster state; node term 0, last-accepted version 0 in term 0")); - assertThat(new ClusterFormationState(Settings.EMPTY, state(localNode, new String[]{"n1"}, new String[]{"n2"}), - emptyList(), emptyList()).getDescription(), + assertThat(new ClusterFormationState(Settings.EMPTY, state(localNode, new String[]{"n1"}, new String[]{"n2"}), emptyList(), + emptyList(), 0L).getDescription(), is("master not discovered or elected yet, an election requires a node with id [n1] and a node with id [n2], " + "have discovered [] which is not a quorum; " + - "discovery will continue using [] from hosts providers and [" + localNode + "] from last-known cluster state")); + "discovery will continue using [] from hosts providers and [" + localNode + + "] from last-known cluster state; node term 0, last-accepted version 0 in term 0")); - assertThat(new ClusterFormationState(Settings.EMPTY, state(localNode, new String[]{"n1"}, new String[]{"n2", "n3"}), - emptyList(), emptyList()).getDescription(), + assertThat(new ClusterFormationState(Settings.EMPTY, state(localNode, new String[]{"n1"}, new String[]{"n2", "n3"}), emptyList(), + emptyList(), 0L).getDescription(), is("master not discovered or elected yet, an election requires a node with id [n1] and two nodes with ids [n2, n3], " + "have discovered [] which is not a quorum; " + - "discovery will continue using [] from hosts providers and [" + localNode + "] from last-known cluster state")); + "discovery will continue using [] from hosts providers and [" + localNode + + "] from last-known cluster state; node term 0, last-accepted version 0 in term 0")); assertThat(new ClusterFormationState(Settings.EMPTY, state(localNode, new String[]{"n1"}, new String[]{"n2", "n3", "n4"}), - emptyList(), emptyList()).getDescription(), + emptyList(), emptyList(), 0L).getDescription(), is("master not discovered or elected yet, an election requires a node with id [n1] and " + "at least 2 nodes with ids from [n2, n3, n4], " + "have discovered [] which is not a quorum; " + - "discovery will continue using [] from hosts providers and [" + localNode + "] from last-known cluster state")); + "discovery will continue using [] from hosts providers and [" + localNode + + "] from last-known cluster state; node term 0, last-accepted version 0 in term 0")); } }