From a070b8acc004740a001bd3b01e6fc6417a9e7919 Mon Sep 17 00:00:00 2001 From: Armin Braun Date: Wed, 30 Jan 2019 19:21:09 +0100 Subject: [PATCH 01/16] Extract TransportRequestDeduplication from ShardStateAction (#37870) * Extracted the logic for master request duplication so it can be reused by the snapshotting logic * Removed custom listener used by `ShardStateAction` to not leak these into future users of this class * Changed semantics slightly to get rid of redundant instantiations of the composite listener * Relates #37686 --- .../TransportReplicationAction.java | 6 +- .../action/shard/ShardStateAction.java | 145 ++---------------- .../cluster/IndicesClusterStateService.java | 3 +- .../TransportRequestDeduplicator.java | 114 ++++++++++++++ .../action/shard/ShardStateActionTests.java | 81 ++-------- .../discovery/ClusterDisruptionIT.java | 7 +- .../TransportRequestDeduplicatorTests.java | 91 +++++++++++ 7 files changed, 241 insertions(+), 206 deletions(-) create mode 100644 server/src/main/java/org/elasticsearch/transport/TransportRequestDeduplicator.java create mode 100644 server/src/test/java/org/elasticsearch/transport/TransportRequestDeduplicatorTests.java diff --git a/server/src/main/java/org/elasticsearch/action/support/replication/TransportReplicationAction.java b/server/src/main/java/org/elasticsearch/action/support/replication/TransportReplicationAction.java index 4894ca1f772c4..c0f0278479a0c 100644 --- a/server/src/main/java/org/elasticsearch/action/support/replication/TransportReplicationAction.java +++ b/server/src/main/java/org/elasticsearch/action/support/replication/TransportReplicationAction.java @@ -1192,12 +1192,12 @@ public void markShardCopyAsStaleIfNeeded(ShardId shardId, String allocationId, R onSuccess.run(); } - protected final ShardStateAction.Listener createShardActionListener(final Runnable onSuccess, + protected final ActionListener createShardActionListener(final Runnable onSuccess, final Consumer onPrimaryDemoted, final Consumer onIgnoredFailure) { - return new ShardStateAction.Listener() { + return new ActionListener() { @Override - public void onSuccess() { + public void onResponse(Void aVoid) { onSuccess.run(); } diff --git a/server/src/main/java/org/elasticsearch/cluster/action/shard/ShardStateAction.java b/server/src/main/java/org/elasticsearch/cluster/action/shard/ShardStateAction.java index 4419d921a3b4a..071885202c458 100644 --- a/server/src/main/java/org/elasticsearch/cluster/action/shard/ShardStateAction.java +++ b/server/src/main/java/org/elasticsearch/cluster/action/shard/ShardStateAction.java @@ -25,6 +25,7 @@ import org.elasticsearch.ElasticsearchException; import org.elasticsearch.ExceptionsHelper; import org.elasticsearch.Version; +import org.elasticsearch.action.ActionListener; import org.elasticsearch.cluster.ClusterChangedEvent; import org.elasticsearch.cluster.ClusterState; import org.elasticsearch.cluster.ClusterStateObserver; @@ -48,18 +49,17 @@ import org.elasticsearch.common.io.stream.StreamInput; import org.elasticsearch.common.io.stream.StreamOutput; import org.elasticsearch.common.unit.TimeValue; -import org.elasticsearch.common.util.concurrent.ConcurrentCollections; import org.elasticsearch.index.shard.ShardId; import org.elasticsearch.node.NodeClosedException; import org.elasticsearch.tasks.Task; import org.elasticsearch.threadpool.ThreadPool; import org.elasticsearch.transport.ConnectTransportException; import org.elasticsearch.transport.EmptyTransportResponseHandler; -import org.elasticsearch.transport.NodeDisconnectedException; import org.elasticsearch.transport.RemoteTransportException; import org.elasticsearch.transport.TransportChannel; import org.elasticsearch.transport.TransportException; import org.elasticsearch.transport.TransportRequest; +import org.elasticsearch.transport.TransportRequestDeduplicator; import org.elasticsearch.transport.TransportRequestHandler; import org.elasticsearch.transport.TransportResponse; import org.elasticsearch.transport.TransportService; @@ -71,7 +71,6 @@ import java.util.Locale; import java.util.Objects; import java.util.Set; -import java.util.concurrent.ConcurrentMap; import java.util.function.Predicate; import static org.elasticsearch.index.seqno.SequenceNumbers.UNASSIGNED_PRIMARY_TERM; @@ -89,7 +88,7 @@ public class ShardStateAction { // a list of shards that failed during replication // we keep track of these shards in order to avoid sending duplicate failed shard requests for a single failing shard. - private final ConcurrentMap remoteFailedShardsCache = ConcurrentCollections.newConcurrentMap(); + private final TransportRequestDeduplicator remoteFailedShardsDeduplicator = new TransportRequestDeduplicator<>(); @Inject public ShardStateAction(ClusterService clusterService, TransportService transportService, @@ -106,7 +105,7 @@ public ShardStateAction(ClusterService clusterService, TransportService transpor } private void sendShardAction(final String actionName, final ClusterState currentState, - final TransportRequest request, final Listener listener) { + final TransportRequest request, final ActionListener listener) { ClusterStateObserver observer = new ClusterStateObserver(currentState, clusterService, null, logger, threadPool.getThreadContext()); DiscoveryNode masterNode = currentState.nodes().getMasterNode(); @@ -120,7 +119,7 @@ private void sendShardAction(final String actionName, final ClusterState current actionName, request, new EmptyTransportResponseHandler(ThreadPool.Names.SAME) { @Override public void handleResponse(TransportResponse.Empty response) { - listener.onSuccess(); + listener.onResponse(null); } @Override @@ -163,44 +162,22 @@ private static boolean isMasterChannelException(TransportException exp) { * @param listener callback upon completion of the request */ public void remoteShardFailed(final ShardId shardId, String allocationId, long primaryTerm, boolean markAsStale, final String message, - @Nullable final Exception failure, Listener listener) { + @Nullable final Exception failure, ActionListener listener) { assert primaryTerm > 0L : "primary term should be strictly positive"; - final FailedShardEntry shardEntry = new FailedShardEntry(shardId, allocationId, primaryTerm, message, failure, markAsStale); - final CompositeListener compositeListener = new CompositeListener(listener); - final CompositeListener existingListener = remoteFailedShardsCache.putIfAbsent(shardEntry, compositeListener); - if (existingListener == null) { - sendShardAction(SHARD_FAILED_ACTION_NAME, clusterService.state(), shardEntry, new Listener() { - @Override - public void onSuccess() { - try { - compositeListener.onSuccess(); - } finally { - remoteFailedShardsCache.remove(shardEntry); - } - } - @Override - public void onFailure(Exception e) { - try { - compositeListener.onFailure(e); - } finally { - remoteFailedShardsCache.remove(shardEntry); - } - } - }); - } else { - existingListener.addListener(listener); - } + remoteFailedShardsDeduplicator.executeOnce( + new FailedShardEntry(shardId, allocationId, primaryTerm, message, failure, markAsStale), listener, + (req, reqListener) -> sendShardAction(SHARD_FAILED_ACTION_NAME, clusterService.state(), req, reqListener)); } int remoteShardFailedCacheSize() { - return remoteFailedShardsCache.size(); + return remoteFailedShardsDeduplicator.size(); } /** * Send a shard failed request to the master node to update the cluster state when a shard on the local node failed. */ public void localShardFailed(final ShardRouting shardRouting, final String message, - @Nullable final Exception failure, Listener listener) { + @Nullable final Exception failure, ActionListener listener) { localShardFailed(shardRouting, message, failure, listener, clusterService.state()); } @@ -208,7 +185,7 @@ public void localShardFailed(final ShardRouting shardRouting, final String messa * Send a shard failed request to the master node to update the cluster state when a shard on the local node failed. */ public void localShardFailed(final ShardRouting shardRouting, final String message, @Nullable final Exception failure, - Listener listener, final ClusterState currentState) { + ActionListener listener, final ClusterState currentState) { FailedShardEntry shardEntry = new FailedShardEntry(shardRouting.shardId(), shardRouting.allocationId().getId(), 0L, message, failure, true); sendShardAction(SHARD_FAILED_ACTION_NAME, currentState, shardEntry, listener); @@ -216,7 +193,8 @@ public void localShardFailed(final ShardRouting shardRouting, final String messa // visible for testing protected void waitForNewMasterAndRetry(String actionName, ClusterStateObserver observer, - TransportRequest request, Listener listener, Predicate changePredicate) { + TransportRequest request, ActionListener listener, + Predicate changePredicate) { observer.waitForNextChange(new ClusterStateObserver.Listener() { @Override public void onNewClusterState(ClusterState state) { @@ -497,14 +475,14 @@ public int hashCode() { public void shardStarted(final ShardRouting shardRouting, final long primaryTerm, final String message, - final Listener listener) { + final ActionListener listener) { shardStarted(shardRouting, primaryTerm, message, listener, clusterService.state()); } public void shardStarted(final ShardRouting shardRouting, final long primaryTerm, final String message, - final Listener listener, + final ActionListener listener, final ClusterState currentState) { StartedShardEntry entry = new StartedShardEntry(shardRouting.shardId(), shardRouting.allocationId().getId(), primaryTerm, message); sendShardAction(SHARD_STARTED_ACTION_NAME, currentState, entry, listener); @@ -670,97 +648,6 @@ public String toString() { } } - public interface Listener { - - default void onSuccess() { - } - - /** - * Notification for non-channel exceptions that are not handled - * by {@link ShardStateAction}. - * - * The exceptions that are handled by {@link ShardStateAction} - * are: - * - {@link NotMasterException} - * - {@link NodeDisconnectedException} - * - {@link FailedToCommitClusterStateException} - * - * Any other exception is communicated to the requester via - * this notification. - * - * @param e the unexpected cause of the failure on the master - */ - default void onFailure(final Exception e) { - } - - } - - /** - * A composite listener that allows registering multiple listeners dynamically. - */ - static final class CompositeListener implements Listener { - private boolean isNotified = false; - private Exception failure = null; - private final List listeners = new ArrayList<>(); - - CompositeListener(Listener listener) { - listeners.add(listener); - } - - void addListener(Listener listener) { - final boolean ready; - synchronized (this) { - ready = this.isNotified; - if (ready == false) { - listeners.add(listener); - } - } - if (ready) { - if (failure != null) { - listener.onFailure(failure); - } else { - listener.onSuccess(); - } - } - } - - private void onCompleted(Exception failure) { - synchronized (this) { - this.failure = failure; - this.isNotified = true; - } - RuntimeException firstException = null; - for (Listener listener : listeners) { - try { - if (failure != null) { - listener.onFailure(failure); - } else { - listener.onSuccess(); - } - } catch (RuntimeException innerEx) { - if (firstException == null) { - firstException = innerEx; - } else { - firstException.addSuppressed(innerEx); - } - } - } - if (firstException != null) { - throw firstException; - } - } - - @Override - public void onSuccess() { - onCompleted(null); - } - - @Override - public void onFailure(Exception failure) { - onCompleted(failure); - } - } - public static class NoLongerPrimaryShardException extends ElasticsearchException { public NoLongerPrimaryShardException(ShardId shardId, String msg) { 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 5955a749fea34..57ec87d1c6493 100644 --- a/server/src/main/java/org/elasticsearch/indices/cluster/IndicesClusterStateService.java +++ b/server/src/main/java/org/elasticsearch/indices/cluster/IndicesClusterStateService.java @@ -109,8 +109,7 @@ public class IndicesClusterStateService extends AbstractLifecycleComponent imple private final ShardStateAction shardStateAction; private final NodeMappingRefreshAction nodeMappingRefreshAction; - private static final ShardStateAction.Listener SHARD_STATE_ACTION_LISTENER = new ShardStateAction.Listener() { - }; + private static final ActionListener SHARD_STATE_ACTION_LISTENER = ActionListener.wrap(() -> {}); private final Settings settings; // a list of shards that failed during recovery diff --git a/server/src/main/java/org/elasticsearch/transport/TransportRequestDeduplicator.java b/server/src/main/java/org/elasticsearch/transport/TransportRequestDeduplicator.java new file mode 100644 index 0000000000000..d929ef34ce2c3 --- /dev/null +++ b/server/src/main/java/org/elasticsearch/transport/TransportRequestDeduplicator.java @@ -0,0 +1,114 @@ +/* + * 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.transport; + +import org.elasticsearch.action.ActionListener; +import org.elasticsearch.common.util.concurrent.ConcurrentCollections; + +import java.util.ArrayList; +import java.util.List; +import java.util.concurrent.ConcurrentMap; +import java.util.function.BiConsumer; + +/** + * Deduplicator for {@link TransportRequest}s that keeps track of {@link TransportRequest}s that should + * not be sent in parallel. + * @param Transport Request Class + */ +public final class TransportRequestDeduplicator { + + private final ConcurrentMap requests = ConcurrentCollections.newConcurrentMap(); + + /** + * Ensures a given request not executed multiple times when another equal request is already in-flight. + * If the request is not yet known to the deduplicator it will invoke the passed callback with an {@link ActionListener} + * that must be completed by the caller when the request completes. Once that listener is completed the request will be removed from + * the deduplicator's internal state. If the request is already known to the deduplicator it will keep + * track of the given listener and invoke it when the listener passed to the callback on first invocation is completed. + * @param request Request to deduplicate + * @param listener Listener to invoke on request completion + * @param callback Callback to be invoked with request and completion listener the first time the request is added to the deduplicator + */ + public void executeOnce(T request, ActionListener listener, BiConsumer> callback) { + ActionListener completionListener = requests.computeIfAbsent(request, CompositeListener::new).addListener(listener); + if (completionListener != null) { + callback.accept(request, completionListener); + } + } + + public int size() { + return requests.size(); + } + + private final class CompositeListener implements ActionListener { + + private final List> listeners = new ArrayList<>(); + + private final T request; + + private boolean isNotified; + private Exception failure; + + CompositeListener(T request) { + this.request = request; + } + + CompositeListener addListener(ActionListener listener) { + synchronized (this) { + if (this.isNotified == false) { + listeners.add(listener); + return listeners.size() == 1 ? this : null; + } + } + if (failure != null) { + listener.onFailure(failure); + } else { + listener.onResponse(null); + } + return null; + } + + private void onCompleted(Exception failure) { + synchronized (this) { + this.failure = failure; + this.isNotified = true; + } + try { + if (failure == null) { + ActionListener.onResponse(listeners, null); + } else { + ActionListener.onFailure(listeners, failure); + } + } finally { + requests.remove(request); + } + } + + @Override + public void onResponse(final Void aVoid) { + onCompleted(null); + } + + @Override + public void onFailure(Exception failure) { + onCompleted(failure); + } + } +} diff --git a/server/src/test/java/org/elasticsearch/cluster/action/shard/ShardStateActionTests.java b/server/src/test/java/org/elasticsearch/cluster/action/shard/ShardStateActionTests.java index a800c0c79929c..69743c101ee10 100644 --- a/server/src/test/java/org/elasticsearch/cluster/action/shard/ShardStateActionTests.java +++ b/server/src/test/java/org/elasticsearch/cluster/action/shard/ShardStateActionTests.java @@ -22,6 +22,7 @@ import org.apache.lucene.index.CorruptIndexException; import org.apache.lucene.util.SetOnce; import org.elasticsearch.Version; +import org.elasticsearch.action.ActionListener; import org.elasticsearch.action.support.replication.ClusterStateCreationUtils; import org.elasticsearch.cluster.ClusterState; import org.elasticsearch.cluster.ClusterStateObserver; @@ -75,7 +76,6 @@ import static org.elasticsearch.test.VersionUtils.randomCompatibleVersion; import static org.hamcrest.CoreMatchers.equalTo; import static org.hamcrest.CoreMatchers.instanceOf; -import static org.hamcrest.CoreMatchers.sameInstance; import static org.hamcrest.Matchers.arrayWithSize; import static org.hamcrest.Matchers.greaterThanOrEqualTo; import static org.hamcrest.Matchers.is; @@ -110,7 +110,7 @@ public void setOnAfterWaitForNewMasterAndRetry(Runnable onAfterWaitForNewMasterA @Override protected void waitForNewMasterAndRetry(String actionName, ClusterStateObserver observer, TransportRequest request, - Listener listener, Predicate changePredicate) { + ActionListener listener, Predicate changePredicate) { onBeforeWaitForNewMasterAndRetry.run(); super.waitForNewMasterAndRetry(actionName, observer, request, listener, changePredicate); onAfterWaitForNewMasterAndRetry.run(); @@ -197,9 +197,9 @@ public void testNoMaster() throws InterruptedException { }); ShardRouting failedShard = getRandomShardRouting(index); - shardStateAction.localShardFailed(failedShard, "test", getSimulatedFailure(), new ShardStateAction.Listener() { + shardStateAction.localShardFailed(failedShard, "test", getSimulatedFailure(), new ActionListener() { @Override - public void onSuccess() { + public void onResponse(Void aVoid) { success.set(true); latch.countDown(); } @@ -246,9 +246,9 @@ public void testMasterChannelException() throws InterruptedException { setUpMasterRetryVerification(numberOfRetries, retries, latch, retryLoop); ShardRouting failedShard = getRandomShardRouting(index); - shardStateAction.localShardFailed(failedShard, "test", getSimulatedFailure(), new ShardStateAction.Listener() { + shardStateAction.localShardFailed(failedShard, "test", getSimulatedFailure(), new ActionListener() { @Override - public void onSuccess() { + public void onResponse(Void aVoid) { success.set(true); latch.countDown(); } @@ -343,9 +343,9 @@ public void testCacheRemoteShardFailed() throws Exception { long primaryTerm = randomLongBetween(1, Long.MAX_VALUE); for (int i = 0; i < numListeners; i++) { shardStateAction.remoteShardFailed(failedShard.shardId(), failedShard.allocationId().getId(), - primaryTerm, markAsStale, "test", getSimulatedFailure(), new ShardStateAction.Listener() { + primaryTerm, markAsStale, "test", getSimulatedFailure(), new ActionListener() { @Override - public void onSuccess() { + public void onResponse(Void aVoid) { latch.countDown(); } @Override @@ -394,9 +394,9 @@ public void testRemoteShardFailedConcurrently() throws Exception { ShardRouting failedShard = randomFrom(failedShards); shardStateAction.remoteShardFailed(failedShard.shardId(), failedShard.allocationId().getId(), randomLongBetween(1, Long.MAX_VALUE), randomBoolean(), "test", getSimulatedFailure(), - new ShardStateAction.Listener() { + new ActionListener() { @Override - public void onSuccess() { + public void onResponse(Void aVoid) { notifiedResponses.incrementAndGet(); } @Override @@ -561,70 +561,13 @@ BytesReference serialize(Writeable writeable, Version version) throws IOExceptio } } - public void testCompositeListener() throws Exception { - AtomicInteger successCount = new AtomicInteger(); - AtomicInteger failureCount = new AtomicInteger(); - Exception failure = randomBoolean() ? getSimulatedFailure() : null; - ShardStateAction.CompositeListener compositeListener = new ShardStateAction.CompositeListener(new ShardStateAction.Listener() { - @Override - public void onSuccess() { - successCount.incrementAndGet(); - } - @Override - public void onFailure(Exception e) { - assertThat(e, sameInstance(failure)); - failureCount.incrementAndGet(); - } - }); - int iterationsPerThread = scaledRandomIntBetween(100, 1000); - Thread[] threads = new Thread[between(1, 4)]; - Phaser barrier = new Phaser(threads.length + 1); - for (int i = 0; i < threads.length; i++) { - threads[i] = new Thread(() -> { - barrier.arriveAndAwaitAdvance(); - for (int n = 0; n < iterationsPerThread; n++) { - compositeListener.addListener(new ShardStateAction.Listener() { - @Override - public void onSuccess() { - successCount.incrementAndGet(); - } - @Override - public void onFailure(Exception e) { - assertThat(e, sameInstance(failure)); - failureCount.incrementAndGet(); - } - }); - } - }); - threads[i].start(); - } - barrier.arriveAndAwaitAdvance(); - if (failure != null) { - compositeListener.onFailure(failure); - } else { - compositeListener.onSuccess(); - } - for (Thread t : threads) { - t.join(); - } - assertBusy(() -> { - if (failure != null) { - assertThat(successCount.get(), equalTo(0)); - assertThat(failureCount.get(), equalTo(threads.length*iterationsPerThread + 1)); - } else { - assertThat(successCount.get(), equalTo(threads.length*iterationsPerThread + 1)); - assertThat(failureCount.get(), equalTo(0)); - } - }); - } - - private static class TestListener implements ShardStateAction.Listener { + private static class TestListener implements ActionListener { private final SetOnce failure = new SetOnce<>(); private final CountDownLatch latch = new CountDownLatch(1); @Override - public void onSuccess() { + public void onResponse(Void aVoid) { try { failure.set(null); } finally { diff --git a/server/src/test/java/org/elasticsearch/discovery/ClusterDisruptionIT.java b/server/src/test/java/org/elasticsearch/discovery/ClusterDisruptionIT.java index 330c73b9c02c5..0a9016c20111b 100644 --- a/server/src/test/java/org/elasticsearch/discovery/ClusterDisruptionIT.java +++ b/server/src/test/java/org/elasticsearch/discovery/ClusterDisruptionIT.java @@ -22,6 +22,7 @@ import org.apache.logging.log4j.message.ParameterizedMessage; import org.apache.lucene.index.CorruptIndexException; import org.elasticsearch.ElasticsearchException; +import org.elasticsearch.action.ActionListener; import org.elasticsearch.action.NoShardAvailableActionException; import org.elasticsearch.action.get.GetResponse; import org.elasticsearch.action.index.IndexResponse; @@ -317,10 +318,10 @@ public void testSendingShardFailure() throws Exception { setDisruptionScheme(networkDisruption); networkDisruption.startDisrupting(); - service.localShardFailed(failedShard, "simulated", new CorruptIndexException("simulated", (String) null), new - ShardStateAction.Listener() { + service.localShardFailed(failedShard, "simulated", new CorruptIndexException("simulated", (String) null), + new ActionListener() { @Override - public void onSuccess() { + public void onResponse(final Void aVoid) { success.set(true); latch.countDown(); } diff --git a/server/src/test/java/org/elasticsearch/transport/TransportRequestDeduplicatorTests.java b/server/src/test/java/org/elasticsearch/transport/TransportRequestDeduplicatorTests.java new file mode 100644 index 0000000000000..ab178134995a7 --- /dev/null +++ b/server/src/test/java/org/elasticsearch/transport/TransportRequestDeduplicatorTests.java @@ -0,0 +1,91 @@ +/* + * Licensed to Elasticsearch under one or more contributor + * license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright + * ownership. Elasticsearch licenses this file to you under + * the Apache License, Version 2.0 (the "License"); you may + * not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.elasticsearch.transport; + +import org.apache.lucene.util.SetOnce; +import org.elasticsearch.action.ActionListener; +import org.elasticsearch.tasks.TaskId; +import org.elasticsearch.test.ESTestCase; + +import java.util.concurrent.Phaser; +import java.util.concurrent.atomic.AtomicInteger; + +import static org.hamcrest.CoreMatchers.equalTo; +import static org.hamcrest.CoreMatchers.sameInstance; + +public class TransportRequestDeduplicatorTests extends ESTestCase { + + public void testRequestDeduplication() throws Exception { + AtomicInteger successCount = new AtomicInteger(); + AtomicInteger failureCount = new AtomicInteger(); + Exception failure = randomBoolean() ? new TransportException("simulated") : null; + final TransportRequest request = new TransportRequest() { + @Override + public void setParentTask(final TaskId taskId) { + } + }; + final TransportRequestDeduplicator deduplicator = new TransportRequestDeduplicator<>(); + final SetOnce> listenerHolder = new SetOnce<>(); + int iterationsPerThread = scaledRandomIntBetween(100, 1000); + Thread[] threads = new Thread[between(1, 4)]; + Phaser barrier = new Phaser(threads.length + 1); + for (int i = 0; i < threads.length; i++) { + threads[i] = new Thread(() -> { + barrier.arriveAndAwaitAdvance(); + for (int n = 0; n < iterationsPerThread; n++) { + deduplicator.executeOnce(request, new ActionListener() { + @Override + public void onResponse(Void aVoid) { + successCount.incrementAndGet(); + } + + @Override + public void onFailure(Exception e) { + assertThat(e, sameInstance(failure)); + failureCount.incrementAndGet(); + } + }, (req, reqListener) -> listenerHolder.set(reqListener)); + } + }); + threads[i].start(); + } + barrier.arriveAndAwaitAdvance(); + for (Thread t : threads) { + t.join(); + } + final ActionListener listener = listenerHolder.get(); + assertThat(deduplicator.size(), equalTo(1)); + if (failure != null) { + listener.onFailure(failure); + } else { + listener.onResponse(null); + } + assertThat(deduplicator.size(), equalTo(0)); + assertBusy(() -> { + if (failure != null) { + assertThat(successCount.get(), equalTo(0)); + assertThat(failureCount.get(), equalTo(threads.length * iterationsPerThread)); + } else { + assertThat(successCount.get(), equalTo(threads.length * iterationsPerThread)); + assertThat(failureCount.get(), equalTo(0)); + } + }); + } + +} From cac6b8e06f051d68919faf6081f1c87fa5b6757d Mon Sep 17 00:00:00 2001 From: Lee Hinman Date: Wed, 30 Jan 2019 11:24:18 -0700 Subject: [PATCH 02/16] Add ECS schema for user-agent ingest processor (#37727) (#37984) * Add ECS schema for user-agent ingest processor (#37727) This switches the format of the user agent processor to use the schema from [ECS](https://github.com/elastic/ecs). So rather than something like this: ``` { "patch" : "3538", "major" : "70", "minor" : "0", "os" : "Mac OS X 10.14.1", "os_minor" : "14", "os_major" : "10", "name" : "Chrome", "os_name" : "Mac OS X", "device" : "Other" } ``` The structure is now like this: ``` { "name" : "Chrome", "original" : "Mozilla/5.0 (Macintosh; Intel Mac OS X 10_14_1) AppleWebKit/537.36 (KHTML, like Gecko) Chrome/70.0.3538.102 Safari/537.36", "os" : { "name" : "Mac OS X", "version" : "10.14.1", "full" : "Mac OS X 10.14.1" }, "device" : "Other", "version" : "70.0.3538.102" } ``` This is now the default for 7.0. The deprecated `ecs` setting in 6.x is not supported. Resolves #37329 * Remove `ecs` setting from docs --- .../ingest/processors/user-agent.asciidoc | 14 +-- .../migration/migrate_7_0/settings.asciidoc | 6 + .../ingest/useragent/UserAgentProcessor.java | 103 ++++++++++-------- .../UserAgentProcessorFactoryTests.java | 4 +- .../useragent/UserAgentProcessorTests.java | 44 +++----- .../20_useragent_processor.yml | 19 +--- .../test/ingest-useragent/30_custom_regex.yml | 9 +- 7 files changed, 92 insertions(+), 107 deletions(-) diff --git a/docs/reference/ingest/processors/user-agent.asciidoc b/docs/reference/ingest/processors/user-agent.asciidoc index 201e3beab8313..f6b6d46fe7b9d 100644 --- a/docs/reference/ingest/processors/user-agent.asciidoc +++ b/docs/reference/ingest/processors/user-agent.asciidoc @@ -60,13 +60,13 @@ Which returns "agent": "Mozilla/5.0 (Macintosh; Intel Mac OS X 10_10_5) AppleWebKit/537.36 (KHTML, like Gecko) Chrome/51.0.2704.103 Safari/537.36", "user_agent": { "name": "Chrome", - "major": "51", - "minor": "0", - "patch": "2704", - "os_name": "Mac OS X", - "os": "Mac OS X 10.10.5", - "os_major": "10", - "os_minor": "10", + "original": "Mozilla/5.0 (Macintosh; Intel Mac OS X 10_10_5) AppleWebKit/537.36 (KHTML, like Gecko) Chrome/51.0.2704.103 Safari/537.36", + "version": "51.0.2704", + "os": { + "name": "Mac OS X", + "version": "10.10.5", + "full": "Mac OS X 10.10.5" + }, "device": "Other" } } diff --git a/docs/reference/migration/migrate_7_0/settings.asciidoc b/docs/reference/migration/migrate_7_0/settings.asciidoc index 6e9f7451e094f..c6874856011ce 100644 --- a/docs/reference/migration/migrate_7_0/settings.asciidoc +++ b/docs/reference/migration/migrate_7_0/settings.asciidoc @@ -182,3 +182,9 @@ could have lead to dropping audit events while the operations on the system were allowed to continue as usual. The recommended replacement is the use of the `logfile` audit output type and using other components from the Elastic Stack to handle the indexing part. + +[float] +[[ingest-user-agent-ecs-always]] +==== Ingest User Agent processor always uses `ecs` output format +The deprecated `ecs` setting for the user agent ingest processor has been +removed. https://github.com/elastic/ecs[ECS] format is now the default. diff --git a/modules/ingest-user-agent/src/main/java/org/elasticsearch/ingest/useragent/UserAgentProcessor.java b/modules/ingest-user-agent/src/main/java/org/elasticsearch/ingest/useragent/UserAgentProcessor.java index 6e7f588f0bd8a..6f2518eede673 100644 --- a/modules/ingest-user-agent/src/main/java/org/elasticsearch/ingest/useragent/UserAgentProcessor.java +++ b/modules/ingest-user-agent/src/main/java/org/elasticsearch/ingest/useragent/UserAgentProcessor.java @@ -19,6 +19,8 @@ package org.elasticsearch.ingest.useragent; +import org.apache.logging.log4j.LogManager; +import org.elasticsearch.common.logging.DeprecationLogger; import org.elasticsearch.ingest.AbstractProcessor; import org.elasticsearch.ingest.IngestDocument; import org.elasticsearch.ingest.Processor; @@ -40,6 +42,8 @@ public class UserAgentProcessor extends AbstractProcessor { + private static final DeprecationLogger deprecationLogger = new DeprecationLogger(LogManager.getLogger(UserAgentProcessor.class)); + public static final String TYPE = "user_agent"; private final String field; @@ -63,7 +67,7 @@ boolean isIgnoreMissing() { } @Override - public IngestDocument execute(IngestDocument ingestDocument) throws Exception { + public IngestDocument execute(IngestDocument ingestDocument) { String userAgent = ingestDocument.getFieldValue(field, String.class, ignoreMissing); if (userAgent == null && ignoreMissing) { @@ -75,68 +79,64 @@ public IngestDocument execute(IngestDocument ingestDocument) throws Exception { Details uaClient = parser.parse(userAgent); Map uaDetails = new HashMap<>(); + + // Parse the user agent in the ECS (Elastic Common Schema) format for (Property property : this.properties) { switch (property) { + case ORIGINAL: + uaDetails.put("original", userAgent); + break; case NAME: if (uaClient.userAgent != null && uaClient.userAgent.name != null) { uaDetails.put("name", uaClient.userAgent.name); - } - else { + } else { uaDetails.put("name", "Other"); } break; - case MAJOR: + case VERSION: + StringBuilder version = new StringBuilder(); if (uaClient.userAgent != null && uaClient.userAgent.major != null) { - uaDetails.put("major", uaClient.userAgent.major); - } - break; - case MINOR: - if (uaClient.userAgent != null && uaClient.userAgent.minor != null) { - uaDetails.put("minor", uaClient.userAgent.minor); - } - break; - case PATCH: - if (uaClient.userAgent != null && uaClient.userAgent.patch != null) { - uaDetails.put("patch", uaClient.userAgent.patch); - } - break; - case BUILD: - if (uaClient.userAgent != null && uaClient.userAgent.build != null) { - uaDetails.put("build", uaClient.userAgent.build); + version.append(uaClient.userAgent.major); + if (uaClient.userAgent.minor != null) { + version.append(".").append(uaClient.userAgent.minor); + if (uaClient.userAgent.patch != null) { + version.append(".").append(uaClient.userAgent.patch); + if (uaClient.userAgent.build != null) { + version.append(".").append(uaClient.userAgent.build); + } + } + } + uaDetails.put("version", version.toString()); } break; case OS: if (uaClient.operatingSystem != null) { - uaDetails.put("os", buildFullOSName(uaClient.operatingSystem)); - } - else { - uaDetails.put("os", "Other"); - } - - break; - case OS_NAME: - if (uaClient.operatingSystem != null && uaClient.operatingSystem.name != null) { - uaDetails.put("os_name", uaClient.operatingSystem.name); - } - else { - uaDetails.put("os_name", "Other"); - } - break; - case OS_MAJOR: - if (uaClient.operatingSystem != null && uaClient.operatingSystem.major != null) { - uaDetails.put("os_major", uaClient.operatingSystem.major); - } - break; - case OS_MINOR: - if (uaClient.operatingSystem != null && uaClient.operatingSystem.minor != null) { - uaDetails.put("os_minor", uaClient.operatingSystem.minor); + Map osDetails = new HashMap<>(3); + if (uaClient.operatingSystem.name != null) { + osDetails.put("name", uaClient.operatingSystem.name); + StringBuilder sb = new StringBuilder(); + if (uaClient.operatingSystem.major != null) { + sb.append(uaClient.operatingSystem.major); + if (uaClient.operatingSystem.minor != null) { + sb.append(".").append(uaClient.operatingSystem.minor); + if (uaClient.operatingSystem.patch != null) { + sb.append(".").append(uaClient.operatingSystem.patch); + if (uaClient.operatingSystem.build != null) { + sb.append(".").append(uaClient.operatingSystem.build); + } + } + } + osDetails.put("version", sb.toString()); + osDetails.put("full", uaClient.operatingSystem.name + " " + sb.toString()); + } + uaDetails.put("os", osDetails); + } } break; case DEVICE: if (uaClient.device != null && uaClient.device.name != null) { uaDetails.put("device", uaClient.device.name); - } - else { + } else { uaDetails.put("device", "Other"); } break; @@ -215,6 +215,10 @@ public UserAgentProcessor create(Map factories, Strin String regexFilename = readStringProperty(TYPE, processorTag, config, "regex_file", IngestUserAgentPlugin.DEFAULT_PARSER_NAME); List propertyNames = readOptionalList(TYPE, processorTag, config, "properties"); boolean ignoreMissing = readBooleanProperty(TYPE, processorTag, config, "ignore_missing", false); + Object ecsValue = config.remove("ecs"); + if (ecsValue != null) { + deprecationLogger.deprecated("setting [ecs] is deprecated as ECS format is the default and only option"); + } UserAgentParser parser = userAgentParsers.get(regexFilename); if (parser == null) { @@ -242,13 +246,16 @@ public UserAgentProcessor create(Map factories, Strin enum Property { - NAME, MAJOR, MINOR, PATCH, OS, OS_NAME, OS_MAJOR, OS_MINOR, DEVICE, BUILD; + NAME, + OS, + DEVICE, + ORIGINAL, + VERSION; public static Property parseProperty(String propertyName) { try { return valueOf(propertyName.toUpperCase(Locale.ROOT)); - } - catch (IllegalArgumentException e) { + } catch (IllegalArgumentException e) { throw new IllegalArgumentException("illegal property value [" + propertyName + "]. valid values are " + Arrays.toString(EnumSet.allOf(Property.class).toArray())); } diff --git a/modules/ingest-user-agent/src/test/java/org/elasticsearch/ingest/useragent/UserAgentProcessorFactoryTests.java b/modules/ingest-user-agent/src/test/java/org/elasticsearch/ingest/useragent/UserAgentProcessorFactoryTests.java index d9c6fc17620da..f723c13f23022 100644 --- a/modules/ingest-user-agent/src/test/java/org/elasticsearch/ingest/useragent/UserAgentProcessorFactoryTests.java +++ b/modules/ingest-user-agent/src/test/java/org/elasticsearch/ingest/useragent/UserAgentProcessorFactoryTests.java @@ -178,8 +178,8 @@ public void testInvalidProperty() throws Exception { config.put("properties", Collections.singletonList("invalid")); ElasticsearchParseException e = expectThrows(ElasticsearchParseException.class, () -> factory.create(null, null, config)); - assertThat(e.getMessage(), equalTo("[properties] illegal property value [invalid]. valid values are [NAME, MAJOR, MINOR, " - + "PATCH, OS, OS_NAME, OS_MAJOR, OS_MINOR, DEVICE, BUILD]")); + assertThat(e.getMessage(), equalTo("[properties] illegal property value [invalid]. valid values are [NAME, OS, DEVICE, " + + "ORIGINAL, VERSION]")); } public void testInvalidPropertiesType() throws Exception { diff --git a/modules/ingest-user-agent/src/test/java/org/elasticsearch/ingest/useragent/UserAgentProcessorTests.java b/modules/ingest-user-agent/src/test/java/org/elasticsearch/ingest/useragent/UserAgentProcessorTests.java index 0a8b453724c90..3938fccd832a3 100644 --- a/modules/ingest-user-agent/src/test/java/org/elasticsearch/ingest/useragent/UserAgentProcessorTests.java +++ b/modules/ingest-user-agent/src/test/java/org/elasticsearch/ingest/useragent/UserAgentProcessorTests.java @@ -103,16 +103,13 @@ public void testCommonBrowser() throws Exception { Map target = (Map) data.get("target_field"); assertThat(target.get("name"), is("Chrome")); - assertThat(target.get("major"), is("33")); - assertThat(target.get("minor"), is("0")); - assertThat(target.get("patch"), is("1750")); - assertNull(target.get("build")); - - assertThat(target.get("os"), is("Mac OS X 10.9.2")); - assertThat(target.get("os_name"), is("Mac OS X")); - assertThat(target.get("os_major"), is("10")); - assertThat(target.get("os_minor"), is("9")); + assertThat(target.get("version"), is("33.0.1750")); + Map os = new HashMap<>(); + os.put("name", "Mac OS X"); + os.put("version", "10.9.2"); + os.put("full", "Mac OS X 10.9.2"); + assertThat(target.get("os"), is(os)); assertThat(target.get("device"), is("Other")); } @@ -131,15 +128,13 @@ public void testUncommonDevice() throws Exception { Map target = (Map) data.get("target_field"); assertThat(target.get("name"), is("Android")); - assertThat(target.get("major"), is("3")); - assertThat(target.get("minor"), is("0")); - assertNull(target.get("patch")); - assertNull(target.get("build")); + assertThat(target.get("version"), is("3.0")); - assertThat(target.get("os"), is("Android 3.0")); - assertThat(target.get("os_name"), is("Android")); - assertThat(target.get("os_major"), is("3")); - assertThat(target.get("os_minor"), is("0")); + Map os = new HashMap<>(); + os.put("name", "Android"); + os.put("version", "3.0"); + os.put("full", "Android 3.0"); + assertThat(target.get("os"), is(os)); assertThat(target.get("device"), is("Motorola Xoom")); } @@ -158,15 +153,9 @@ public void testSpider() throws Exception { Map target = (Map) data.get("target_field"); assertThat(target.get("name"), is("EasouSpider")); - assertNull(target.get("major")); - assertNull(target.get("minor")); - assertNull(target.get("patch")); - assertNull(target.get("build")); - assertThat(target.get("os"), is("Other")); - assertThat(target.get("os_name"), is("Other")); - assertNull(target.get("os_major")); - assertNull(target.get("os_minor")); + assertNull(target.get("version")); + assertNull(target.get("os")); assertThat(target.get("device"), is("Spider")); } @@ -190,10 +179,7 @@ public void testUnknown() throws Exception { assertNull(target.get("patch")); assertNull(target.get("build")); - assertThat(target.get("os"), is("Other")); - assertThat(target.get("os_name"), is("Other")); - assertNull(target.get("os_major")); - assertNull(target.get("os_minor")); + assertNull(target.get("os")); assertThat(target.get("device"), is("Other")); } diff --git a/modules/ingest-user-agent/src/test/resources/rest-api-spec/test/ingest-useragent/20_useragent_processor.yml b/modules/ingest-user-agent/src/test/resources/rest-api-spec/test/ingest-useragent/20_useragent_processor.yml index 28c218edd6935..fc44d7261e80f 100644 --- a/modules/ingest-user-agent/src/test/resources/rest-api-spec/test/ingest-useragent/20_useragent_processor.yml +++ b/modules/ingest-user-agent/src/test/resources/rest-api-spec/test/ingest-useragent/20_useragent_processor.yml @@ -29,13 +29,9 @@ id: 1 - match: { _source.field1: "Mozilla/5.0 (Macintosh; Intel Mac OS X 10_9_2) AppleWebKit/537.36 (KHTML, like Gecko) Chrome/33.0.1750.149 Safari/537.36" } - match: { _source.user_agent.name: "Chrome" } - - match: { _source.user_agent.os: "Mac OS X 10.9.2" } - - match: { _source.user_agent.os_name: "Mac OS X" } - - match: { _source.user_agent.os_major: "10" } - - match: { _source.user_agent.os_minor: "9" } - - match: { _source.user_agent.major: "33" } - - match: { _source.user_agent.minor: "0" } - - match: { _source.user_agent.patch: "1750" } + - match: { _source.user_agent.original: "Mozilla/5.0 (Macintosh; Intel Mac OS X 10_9_2) AppleWebKit/537.36 (KHTML, like Gecko) Chrome/33.0.1750.149 Safari/537.36" } + - match: { _source.user_agent.os: {"name":"Mac OS X", "version":"10.9.2", "full":"Mac OS X 10.9.2"} } + - match: { _source.user_agent.version: "33.0.1750" } - match: { _source.user_agent.device: "Other" } --- @@ -70,13 +66,8 @@ index: test id: 1 - match: { _source.field1: "Mozilla/5.0 (Macintosh; Intel Mac OS X 10_9_2) AppleWebKit/537.36 (KHTML, like Gecko) Chrome/33.0.1750.149 Safari/537.36" } - - match: { _source.field2.os: "Mac OS X 10.9.2" } + - match: { _source.field2.os.full: "Mac OS X 10.9.2" } - is_false: _source.user_agent - is_false: _source.field2.name - - is_false: _source.field2.os_name - - is_false: _source.field2.os_major - - is_false: _source.field2.os_minor - - is_false: _source.field2.major - - is_false: _source.field2.minor - - is_false: _source.field2.patch - is_false: _source.field2.device + - is_false: _source.field2.original diff --git a/modules/ingest-user-agent/src/test/resources/rest-api-spec/test/ingest-useragent/30_custom_regex.yml b/modules/ingest-user-agent/src/test/resources/rest-api-spec/test/ingest-useragent/30_custom_regex.yml index 22df584e13166..ac90a3457fa65 100644 --- a/modules/ingest-user-agent/src/test/resources/rest-api-spec/test/ingest-useragent/30_custom_regex.yml +++ b/modules/ingest-user-agent/src/test/resources/rest-api-spec/test/ingest-useragent/30_custom_regex.yml @@ -30,11 +30,6 @@ id: 1 - match: { _source.field1: "Mozilla/5.0 (Macintosh; Intel Mac OS X 10_9_2) AppleWebKit/537.36 (KHTML, like Gecko) Chrome/33.0.1750.149 Safari/537.36" } - match: { _source.user_agent.name: "Test" } - - match: { _source.user_agent.os: "Other" } - - match: { _source.user_agent.os_name: "Other" } - match: { _source.user_agent.device: "Other" } - - is_false: _source.user_agent.os_major - - is_false: _source.user_agent.os_minor - - is_false: _source.user_agent.major - - is_false: _source.user_agent.minor - - is_false: _source.user_agent.patch + - is_false: _source.user_agent.os + - is_false: _source.user_agent.version From 5433af28e33daeda6d30e8cd41ac1e69a548334c Mon Sep 17 00:00:00 2001 From: Martijn van Groningen Date: Wed, 30 Jan 2019 19:33:00 +0100 Subject: [PATCH 03/16] Fixed test bug, lastFollowTime is null if there are no follower indices. --- .../org/elasticsearch/xpack/ccr/CCRFeatureSetTests.java | 9 +++++++-- 1 file changed, 7 insertions(+), 2 deletions(-) diff --git a/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/CCRFeatureSetTests.java b/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/CCRFeatureSetTests.java index b95e8fc7c4008..f2fb475816ec2 100644 --- a/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/CCRFeatureSetTests.java +++ b/x-pack/plugin/ccr/src/test/java/org/elasticsearch/xpack/ccr/CCRFeatureSetTests.java @@ -27,6 +27,7 @@ import static org.hamcrest.Matchers.equalTo; import static org.hamcrest.Matchers.greaterThanOrEqualTo; +import static org.hamcrest.Matchers.nullValue; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.when; @@ -36,7 +37,7 @@ public class CCRFeatureSetTests extends ESTestCase { private ClusterService clusterService; @Before - public void init() throws Exception { + public void init() { licenseState = mock(XPackLicenseState.class); clusterService = mock(ClusterService.class); } @@ -116,7 +117,11 @@ public void testUsageStats() throws Exception { assertThat(ccrUsage.available(), equalTo(ccrFeatureSet.available())); assertThat(ccrUsage.getNumberOfFollowerIndices(), equalTo(numFollowerIndices)); - assertThat(ccrUsage.getLastFollowTimeInMillis(), greaterThanOrEqualTo(0L)); + if (numFollowerIndices != 0) { + assertThat(ccrUsage.getLastFollowTimeInMillis(), greaterThanOrEqualTo(0L)); + } else { + assertThat(ccrUsage.getLastFollowTimeInMillis(), nullValue()); + } assertThat(ccrUsage.getNumberOfAutoFollowPatterns(), equalTo(numAutoFollowPatterns)); } From 36ee78d92468901be372b5218890f4c3e59f564d Mon Sep 17 00:00:00 2001 From: Jack Conradson Date: Wed, 30 Jan 2019 11:01:45 -0800 Subject: [PATCH 04/16] Add test coverage for Painless general casting of boolean and Boolean (#37780) This adds test coverage for general casts in Painless between boolean and other types and Boolean and other types. --- .../painless/StandardCastTests.java | 154 ++++++++++++++++++ 1 file changed, 154 insertions(+) diff --git a/modules/lang-painless/src/test/java/org/elasticsearch/painless/StandardCastTests.java b/modules/lang-painless/src/test/java/org/elasticsearch/painless/StandardCastTests.java index 739d9d021a427..d19ba0847b2de 100644 --- a/modules/lang-painless/src/test/java/org/elasticsearch/painless/StandardCastTests.java +++ b/modules/lang-painless/src/test/java/org/elasticsearch/painless/StandardCastTests.java @@ -641,4 +641,158 @@ public void testStringCasts() { expectScriptThrows(ClassCastException.class, () -> exec("String o = 'string'; ArrayList b = o;")); expectScriptThrows(ClassCastException.class, () -> exec("String o = 'string'; ArrayList b = (ArrayList)o;")); } + + public void testPrimitiveBooleanCasts() { + expectScriptThrows(ClassCastException.class, () -> exec("boolean o = true; Object n = o;")); + expectScriptThrows(ClassCastException.class, () -> exec("boolean o = true; Object n = (Object)o;")); + + expectScriptThrows(ClassCastException.class, () -> exec("boolean o = true; Number n = o;")); + expectScriptThrows(ClassCastException.class, () -> exec("boolean o = true; Number n = (boolean)o;")); + + exec("boolean o = true; boolean b = o;"); + exec("boolean o = true; boolean b = (boolean)o;"); + + expectScriptThrows(ClassCastException.class, () -> exec("boolean o = true; byte b = o;")); + expectScriptThrows(ClassCastException.class, () -> exec("boolean o = true; byte b = (byte)o;")); + + expectScriptThrows(ClassCastException.class, () -> exec("boolean o = true; short b = o;")); + expectScriptThrows(ClassCastException.class, () -> exec("boolean o = true; short b = (short)o;")); + + expectScriptThrows(ClassCastException.class, () -> exec("boolean o = true; char b = o;")); + expectScriptThrows(ClassCastException.class, () -> exec("boolean o = true; char b = (char)o;")); + + expectScriptThrows(ClassCastException.class, () -> exec("boolean o = true; int b = o;")); + expectScriptThrows(ClassCastException.class, () -> exec("boolean o = true; int b = (int)o;")); + + expectScriptThrows(ClassCastException.class, () -> exec("boolean o = true; long b = o;")); + expectScriptThrows(ClassCastException.class, () -> exec("boolean o = true; long b = (long)o;")); + + expectScriptThrows(ClassCastException.class, () -> exec("boolean o = true; float b = o;")); + expectScriptThrows(ClassCastException.class, () -> exec("boolean o = true; float b = (float)o;")); + + expectScriptThrows(ClassCastException.class, () -> exec("boolean o = true; double b = o;")); + expectScriptThrows(ClassCastException.class, () -> exec("boolean o = true; double b = (double)o;")); + + expectScriptThrows(ClassCastException.class, () -> exec("boolean o = true; Boolean b = o;")); + expectScriptThrows(ClassCastException.class, () -> exec("boolean o = true; Boolean b = (Boolean)o;")); + + expectScriptThrows(ClassCastException.class, () -> exec("boolean o = true; Byte b = o;")); + expectScriptThrows(ClassCastException.class, () -> exec("boolean o = true; Byte b = (Byte)o;")); + + expectScriptThrows(ClassCastException.class, () -> exec("boolean o = true; Short b = o;")); + expectScriptThrows(ClassCastException.class, () -> exec("boolean o = true; Short b = (Short)o;")); + + expectScriptThrows(ClassCastException.class, () -> exec("boolean o = true; Character b = o;")); + expectScriptThrows(ClassCastException.class, () -> exec("boolean o = true; Character b = (Character)o;")); + + expectScriptThrows(ClassCastException.class, () -> exec("boolean o = true; Integer b = o;")); + expectScriptThrows(ClassCastException.class, () -> exec("boolean o = true; Integer b = (Integer)o;")); + + expectScriptThrows(ClassCastException.class, () -> exec("boolean o = true; Long b = o;")); + expectScriptThrows(ClassCastException.class, () -> exec("boolean o = true; Long b = (Long)o;")); + + expectScriptThrows(ClassCastException.class, () -> exec("boolean o = true; Float b = o;")); + expectScriptThrows(ClassCastException.class, () -> exec("boolean o = true; Float b = (Float)o;")); + + expectScriptThrows(ClassCastException.class, () -> exec("boolean o = true; Double b = o;")); + expectScriptThrows(ClassCastException.class, () -> exec("boolean o = true; Double b = (Double)o;")); + + expectScriptThrows(ClassCastException.class, () -> exec("boolean o = true; ArrayList b = o;")); + expectScriptThrows(ClassCastException.class, () -> exec("boolean o = true; ArrayList b = (ArrayList)o;")); + } + + public void testBoxedBooleanCasts() { + exec("Boolean o = Boolean.valueOf(true); Object n = o;"); + exec("Boolean o = null; Object n = o;"); + exec("Boolean o = Boolean.valueOf(true); Object n = (Object)o;"); + exec("Boolean o = null; Object n = (Object)o;"); + + expectScriptThrows(ClassCastException.class, () -> exec("Boolean o = Boolean.valueOf(true); Number n = o;")); + expectScriptThrows(ClassCastException.class, () -> exec("Boolean o = null; Number n = o;")); + expectScriptThrows(ClassCastException.class, () -> exec("Boolean o = Boolean.valueOf(true); Number n = (Boolean)o;")); + expectScriptThrows(ClassCastException.class, () -> exec("Boolean o = null; Number n = (Boolean)o;")); + + expectScriptThrows(ClassCastException.class, () -> exec("Boolean o = Boolean.valueOf(true); boolean b = o;")); + expectScriptThrows(ClassCastException.class, () -> exec("Boolean o = null; boolean b = o;")); + expectScriptThrows(ClassCastException.class, () -> exec("Boolean o = Boolean.valueOf(true); boolean b = (boolean)o;")); + expectScriptThrows(ClassCastException.class, () -> exec("Boolean o = null; boolean b = (boolean)o;")); + + expectScriptThrows(ClassCastException.class, () -> exec("Boolean o = Boolean.valueOf(true); byte b = o;")); + expectScriptThrows(ClassCastException.class, () -> exec("Boolean o = null; byte b = o;")); + expectScriptThrows(ClassCastException.class, () -> exec("Boolean o = Boolean.valueOf(true); byte b = (byte)o;")); + expectScriptThrows(ClassCastException.class, () -> exec("Boolean o = null; byte b = (byte)o;")); + + expectScriptThrows(ClassCastException.class, () -> exec("Boolean o = Boolean.valueOf(true); short b = o;")); + expectScriptThrows(ClassCastException.class, () -> exec("Boolean o = null; short b = o;")); + expectScriptThrows(ClassCastException.class, () -> exec("Boolean o = Boolean.valueOf(true); short b = (short)o;")); + expectScriptThrows(ClassCastException.class, () -> exec("Boolean o = null; short b = (short)o;")); + + expectScriptThrows(ClassCastException.class, () -> exec("Boolean o = Boolean.valueOf(true); char b = o;")); + expectScriptThrows(ClassCastException.class, () -> exec("Boolean o = null; char b = o;")); + expectScriptThrows(ClassCastException.class, () -> exec("Boolean o = Boolean.valueOf(true); char b = (char)o;")); + expectScriptThrows(ClassCastException.class, () -> exec("Boolean o = null; char b = (char)o;")); + + expectScriptThrows(ClassCastException.class, () -> exec("Boolean o = Boolean.valueOf(true); int b = o;")); + expectScriptThrows(ClassCastException.class, () -> exec("Boolean o = null; int b = o;")); + expectScriptThrows(ClassCastException.class, () -> exec("Boolean o = Boolean.valueOf(true); int b = (int)o;")); + expectScriptThrows(ClassCastException.class, () -> exec("Boolean o = null; int b = (int)o;")); + + expectScriptThrows(ClassCastException.class, () -> exec("Boolean o = Boolean.valueOf(true); long b = o;")); + expectScriptThrows(ClassCastException.class, () -> exec("Boolean o = null; long b = o;")); + expectScriptThrows(ClassCastException.class, () -> exec("Boolean o = Boolean.valueOf(true); long b = (long)o;")); + expectScriptThrows(ClassCastException.class, () -> exec("Boolean o = null; long b = (long)o;")); + + expectScriptThrows(ClassCastException.class, () -> exec("Boolean o = Boolean.valueOf(true); float b = o;")); + expectScriptThrows(ClassCastException.class, () -> exec("Boolean o = null; float b = o;")); + expectScriptThrows(ClassCastException.class, () -> exec("Boolean o = Boolean.valueOf(true); float b = (float)o;")); + expectScriptThrows(ClassCastException.class, () -> exec("Boolean o = null; float b = (float)o;")); + + expectScriptThrows(ClassCastException.class, () -> exec("Boolean o = Boolean.valueOf(true); double b = o;")); + expectScriptThrows(ClassCastException.class, () -> exec("Boolean o = null; double b = o;")); + expectScriptThrows(ClassCastException.class, () -> exec("Boolean o = Boolean.valueOf(true); double b = (double)o;")); + expectScriptThrows(ClassCastException.class, () -> exec("Boolean o = null; double b = (double)o;")); + + exec("Boolean o = Boolean.valueOf(true); Boolean b = o;"); + exec("Boolean o = null; Boolean b = o;"); + exec("Boolean o = Boolean.valueOf(true); Boolean b = (Boolean)o;"); + exec("Boolean o = null; Boolean b = (Boolean)o;"); + + expectScriptThrows(ClassCastException.class, () -> exec("Boolean o = Boolean.valueOf(true); Byte b = o;")); + expectScriptThrows(ClassCastException.class, () -> exec("Boolean o = null; Byte b = o;")); + expectScriptThrows(ClassCastException.class, () -> exec("Boolean o = Boolean.valueOf(true); Byte b = (Byte)o;")); + expectScriptThrows(ClassCastException.class, () -> exec("Boolean o = null; Byte b = (Byte)o;")); + + expectScriptThrows(ClassCastException.class, () -> exec("Boolean o = Boolean.valueOf(true); Short b = o;")); + expectScriptThrows(ClassCastException.class, () -> exec("Boolean o = null; Short b = o;")); + expectScriptThrows(ClassCastException.class, () -> exec("Boolean o = Boolean.valueOf(true); Short b = (Short)o;")); + expectScriptThrows(ClassCastException.class, () -> exec("Boolean o = null; Short b = (Short)o;")); + + expectScriptThrows(ClassCastException.class, () -> exec("Boolean o = Boolean.valueOf(true); Character b = o;")); + expectScriptThrows(ClassCastException.class, () -> exec("Boolean o = null; Character b = o;")); + expectScriptThrows(ClassCastException.class, () -> exec("Boolean o = Boolean.valueOf(true); Character b = (Character)o;")); + expectScriptThrows(ClassCastException.class, () -> exec("Boolean o = null; Character b = (Character)o;")); + + expectScriptThrows(ClassCastException.class, () -> exec("Boolean o = Boolean.valueOf(true); Integer b = o;")); + expectScriptThrows(ClassCastException.class, () -> exec("Boolean o = null; Integer b = o;")); + expectScriptThrows(ClassCastException.class, () -> exec("Boolean o = Boolean.valueOf(true); Integer b = (Integer)o;")); + expectScriptThrows(ClassCastException.class, () -> exec("Boolean o = null; Integer b = (Integer)o;")); + + expectScriptThrows(ClassCastException.class, () -> exec("Boolean o = Boolean.valueOf(true); Long b = o;")); + expectScriptThrows(ClassCastException.class, () -> exec("Boolean o = null; Long b = o;")); + expectScriptThrows(ClassCastException.class, () -> exec("Boolean o = Boolean.valueOf(true); Long b = (Long)o;")); + expectScriptThrows(ClassCastException.class, () -> exec("Boolean o = null; Long b = (Long)o;")); + + expectScriptThrows(ClassCastException.class, () -> exec("Boolean o = Boolean.valueOf(true); Float b = o;")); + expectScriptThrows(ClassCastException.class, () -> exec("Boolean o = null; Float b = o;")); + expectScriptThrows(ClassCastException.class, () -> exec("Boolean o = Boolean.valueOf(true); Float b = (Float)o;")); + expectScriptThrows(ClassCastException.class, () -> exec("Boolean o = null; Float b = (Float)o;")); + + expectScriptThrows(ClassCastException.class, () -> exec("Boolean o = Boolean.valueOf(true); Double b = o;")); + expectScriptThrows(ClassCastException.class, () -> exec("Boolean o = null; Double b = o;")); + expectScriptThrows(ClassCastException.class, () -> exec("Boolean o = Boolean.valueOf(true); Double b = (Double)o;")); + expectScriptThrows(ClassCastException.class, () -> exec("Boolean o = null; Double b = (Double)o;")); + + expectScriptThrows(ClassCastException.class, () -> exec("Boolean o = Boolean.valueOf(true); ArrayList b = o;")); + expectScriptThrows(ClassCastException.class, () -> exec("Boolean o = Boolean.valueOf(true); ArrayList b = (ArrayList)o;")); + } } From 6a78b6a31cfe0f06f2656672db12f1d5edfb5aaf Mon Sep 17 00:00:00 2001 From: Jake Landis Date: Wed, 30 Jan 2019 13:12:13 -0600 Subject: [PATCH 05/16] Remove types from watcher docs (#38002) Types have been deprecated and this commit removes the documentation for specifying types in the index action, and search input/transform. Relates #37594 #35190 --- x-pack/docs/build.gradle | 3 +-- x-pack/docs/en/rest-api/watcher/ack-watch.asciidoc | 3 +-- x-pack/docs/en/rest-api/watcher/execute-watch.asciidoc | 2 +- x-pack/docs/en/rest-api/watcher/get-watch.asciidoc | 3 +-- x-pack/docs/en/watcher/actions/index.asciidoc | 9 +++------ x-pack/docs/en/watcher/input/search.asciidoc | 4 ---- x-pack/docs/en/watcher/transform/search.asciidoc | 6 ------ 7 files changed, 7 insertions(+), 23 deletions(-) diff --git a/x-pack/docs/build.gradle b/x-pack/docs/build.gradle index de2400c0e85f0..27f815b1637f1 100644 --- a/x-pack/docs/build.gradle +++ b/x-pack/docs/build.gradle @@ -122,8 +122,7 @@ setups['my_inactive_watch'] = ''' "actions": { "test_index": { "index": { - "index": "test", - "doc_type": "test2" + "index": "test" } } } diff --git a/x-pack/docs/en/rest-api/watcher/ack-watch.asciidoc b/x-pack/docs/en/rest-api/watcher/ack-watch.asciidoc index a1704e9acc329..f1e281819eec3 100644 --- a/x-pack/docs/en/rest-api/watcher/ack-watch.asciidoc +++ b/x-pack/docs/en/rest-api/watcher/ack-watch.asciidoc @@ -68,8 +68,7 @@ PUT _watcher/watch/my_watch "test_index": { "throttle_period": "15m", "index": { - "index": "test", - "doc_type": "test2" + "index": "test" } } } diff --git a/x-pack/docs/en/rest-api/watcher/execute-watch.asciidoc b/x-pack/docs/en/rest-api/watcher/execute-watch.asciidoc index ca3a0de0af781..8c7f747969373 100644 --- a/x-pack/docs/en/rest-api/watcher/execute-watch.asciidoc +++ b/x-pack/docs/en/rest-api/watcher/execute-watch.asciidoc @@ -255,7 +255,7 @@ This is an example of the output: "index": { "response": { "index": "test", - "type": "test2", + "type": "_doc", "version": 1, "created": true, "result": "created", diff --git a/x-pack/docs/en/rest-api/watcher/get-watch.asciidoc b/x-pack/docs/en/rest-api/watcher/get-watch.asciidoc index 7d62b5c76c41d..87a13d0829f9b 100644 --- a/x-pack/docs/en/rest-api/watcher/get-watch.asciidoc +++ b/x-pack/docs/en/rest-api/watcher/get-watch.asciidoc @@ -81,8 +81,7 @@ Response: "actions": { "test_index": { "index": { - "index": "test", - "doc_type": "test2" + "index": "test" } } } diff --git a/x-pack/docs/en/watcher/actions/index.asciidoc b/x-pack/docs/en/watcher/actions/index.asciidoc index 8a31b150f22cb..34fdad7c50d0b 100644 --- a/x-pack/docs/en/watcher/actions/index.asciidoc +++ b/x-pack/docs/en/watcher/actions/index.asciidoc @@ -16,8 +16,7 @@ The following snippet shows a simple `index` action definition: "transform": { ... }, <3> "index" : { "index" : "my-index", <4> - "doc_type" : "my-type", <5> - "doc_id": "my-id" <6> + "doc_id": "my-id" <5> } } } @@ -27,8 +26,7 @@ The following snippet shows a simple `index` action definition: <2> An optional <> to restrict action execution <3> An optional <> to transform the payload and prepare the data that should be indexed <4> The elasticsearch index to store the data to -<5> The document type to store the data as -<6> An optional `_id` for the document, if it should always be the same document. +<5> An optional `_id` for the document, if it should always be the same document. [[index-action-attributes]] @@ -40,7 +38,6 @@ The following snippet shows a simple `index` action definition: | `index` | yes | - | The Elasticsearch index to index into. -| `doc_type` | yes | - | The type of the document the data will be indexed as. | `doc_id` | no | - | The optional `_id` of the document. @@ -75,5 +72,5 @@ When a `_doc` field exists, if the field holds an object, it is extracted and in as a single document. If the field holds an array of objects, each object is treated as a document and the index action indexes all of them in a bulk. -An `_index`, `_type` or `_id` value can be added per document to dynamically set the ID +An `_index`, or `_id` value can be added per document to dynamically set the ID of the indexed document. diff --git a/x-pack/docs/en/watcher/input/search.asciidoc b/x-pack/docs/en/watcher/input/search.asciidoc index 3b50b22081b60..d4548a159a640 100644 --- a/x-pack/docs/en/watcher/input/search.asciidoc +++ b/x-pack/docs/en/watcher/input/search.asciidoc @@ -24,7 +24,6 @@ documents from the `logs` index: "search" : { "request" : { "indices" : [ "logs" ], - "types" : [ "event" ], "body" : { "query" : { "match_all" : {}} } @@ -172,9 +171,6 @@ accurately. | `request.indices` | no | - | The indices to search. If omitted, all indices are searched, which is the default behaviour in Elasticsearch. -| `request.types` | no | - | The document types to search for. If omitted, all document types are are - searched, which is the default behaviour in Elasticsearch. - | `request.body` | no | - | The body of the request. The {ref}/search-request-body.html[request body] follows the same structure you normally send in the body of a REST `_search` request. The body can be static text or include `mustache` <>. diff --git a/x-pack/docs/en/watcher/transform/search.asciidoc b/x-pack/docs/en/watcher/transform/search.asciidoc index 56f9304d986ce..d7f468f183182 100644 --- a/x-pack/docs/en/watcher/transform/search.asciidoc +++ b/x-pack/docs/en/watcher/transform/search.asciidoc @@ -56,10 +56,6 @@ The following table lists all available settings for the search transform: | `request.indices` | no | all indices | One or more indices to search on. -| `request.types` | no | all types | One or more document types to search on (may be a - comma-delimited string or an array of document types - names) - | `request.body` | no | `match_all` query | The body of the request. The {ref}/search-request-body.html[request body] follows the same structure you normally send in the body of @@ -105,7 +101,6 @@ time of the watch: "search" : { "request" : { "indices" : [ "logstash-*" ], - "types" : [ "event" ], "body" : { "size" : 0, "query" : { @@ -145,7 +140,6 @@ The following is an example of using templates that refer to provided parameters "search" : { "request" : { "indices" : [ "logstash-*" ], - "types" : [ "event" ], "template" : { "source" : { "size" : 0, From 81c443c9de81f3cd27b2050c0a85ff29d147cd73 Mon Sep 17 00:00:00 2001 From: David Turner Date: Wed, 30 Jan 2019 20:09:15 +0000 Subject: [PATCH 06/16] Deprecate minimum_master_nodes (#37868) Today we pass `discovery.zen.minimum_master_nodes` to nodes started up in tests, but for 7.x nodes this setting is not required as it has no effect. This commit removes this setting so that nodes are started with more realistic configurations, and deprecates it. --- .../gradle/test/ClusterFormationTasks.groovy | 13 ++- .../migration/migrate_7_0/discovery.asciidoc | 4 +- .../rest/discovery/Zen2RestApiIT.java | 8 +- .../ec2/Ec2DiscoveryUpdateSettingsTests.java | 14 ++-- .../custom_logging/CustomLoggingConfigIT.java | 2 +- qa/rolling-upgrade/build.gradle | 1 - .../test/cluster.put_settings/10_basic.yml | 16 ++-- .../discovery/zen/ElectMasterService.java | 2 +- .../admin/indices/exists/IndicesExistsIT.java | 4 +- .../cluster/MinimumMasterNodesIT.java | 7 +- .../cluster/SpecificMasterNodesIT.java | 7 -- .../coordination/UnsafeBootstrapMasterIT.java | 26 ++---- .../cluster/coordination/Zen1IT.java | 6 +- .../discovery/MasterDisruptionIT.java | 8 +- .../MinimumMasterNodesInClusterStateIT.java | 66 --------------- .../discovery/zen/ZenDiscoveryUnitTests.java | 6 ++ .../gateway/RecoverAfterNodesIT.java | 31 ++++--- .../elasticsearch/test/ESIntegTestCase.java | 11 --- .../test/InternalTestCluster.java | 70 +++++++++++----- .../test/discovery/TestZenDiscovery.java | 8 +- .../test/test/InternalTestClusterTests.java | 80 ++----------------- .../elasticsearch/license/LicensingTests.java | 15 ++-- ...ServerTransportFilterIntegrationTests.java | 13 ++- 23 files changed, 140 insertions(+), 278 deletions(-) delete mode 100644 server/src/test/java/org/elasticsearch/discovery/zen/MinimumMasterNodesInClusterStateIT.java diff --git a/buildSrc/src/main/groovy/org/elasticsearch/gradle/test/ClusterFormationTasks.groovy b/buildSrc/src/main/groovy/org/elasticsearch/gradle/test/ClusterFormationTasks.groovy index 2fffd67215581..f32d0d858f81f 100644 --- a/buildSrc/src/main/groovy/org/elasticsearch/gradle/test/ClusterFormationTasks.groovy +++ b/buildSrc/src/main/groovy/org/elasticsearch/gradle/test/ClusterFormationTasks.groovy @@ -373,18 +373,15 @@ class ClusterFormationTasks { 'path.repo' : "${node.sharedDir}/repo", 'path.shared_data' : "${node.sharedDir}/", // Define a node attribute so we can test that it exists - 'node.attr.testattr' : 'test' + 'node.attr.testattr' : 'test', + // Don't wait for state, just start up quickly. This will also allow new and old nodes in the BWC case to become the master + 'discovery.initial_state_timeout' : '0s' ] int minimumMasterNodes = node.config.minimumMasterNodes.call() - if (minimumMasterNodes > 0) { + if (node.nodeVersion.before("7.0.0") && minimumMasterNodes > 0) { esConfig['discovery.zen.minimum_master_nodes'] = minimumMasterNodes } - if (minimumMasterNodes > 1) { - // don't wait for state.. just start up quickly - // this will also allow new and old nodes in the BWC case to become the master - esConfig['discovery.initial_state_timeout'] = '0s' - } - if (esConfig.containsKey('discovery.zen.master_election.wait_for_joins_timeout') == false) { + if (node.nodeVersion.before("7.0.0") && esConfig.containsKey('discovery.zen.master_election.wait_for_joins_timeout') == false) { // If a node decides to become master based on partial information from the pinging, don't let it hang for 30 seconds to correct // its mistake. Instead, only wait 5s to do another round of pinging. // This is necessary since we use 30s as the default timeout in REST requests waiting for cluster formation diff --git a/docs/reference/migration/migrate_7_0/discovery.asciidoc b/docs/reference/migration/migrate_7_0/discovery.asciidoc index d568e7fe32c25..193f6bdd86a6f 100644 --- a/docs/reference/migration/migrate_7_0/discovery.asciidoc +++ b/docs/reference/migration/migrate_7_0/discovery.asciidoc @@ -13,8 +13,8 @@ settings summary>> for an example, and the <> describes this setting in more detail. -The `discovery.zen.minimum_master_nodes` setting is required during a rolling -upgrade from 6.x, but can be removed in all other circumstances. +The `discovery.zen.minimum_master_nodes` setting is permitted, but ignored, on +7.x nodes. [float] ==== Removing master-eligible nodes sometimes requires voting exclusions diff --git a/modules/transport-netty4/src/test/java/org/elasticsearch/rest/discovery/Zen2RestApiIT.java b/modules/transport-netty4/src/test/java/org/elasticsearch/rest/discovery/Zen2RestApiIT.java index cfa5a3f6d79c8..88afa57e83e23 100644 --- a/modules/transport-netty4/src/test/java/org/elasticsearch/rest/discovery/Zen2RestApiIT.java +++ b/modules/transport-netty4/src/test/java/org/elasticsearch/rest/discovery/Zen2RestApiIT.java @@ -33,9 +33,7 @@ import org.elasticsearch.cluster.routing.UnassignedInfo; import org.elasticsearch.common.Priority; import org.elasticsearch.common.settings.Settings; -import org.elasticsearch.common.settings.Settings.Builder; import org.elasticsearch.common.unit.TimeValue; -import org.elasticsearch.discovery.zen.ElectMasterService; import org.elasticsearch.http.HttpServerTransport; import org.elasticsearch.test.ESIntegTestCase; import org.elasticsearch.test.InternalTestCluster; @@ -58,11 +56,7 @@ public class Zen2RestApiIT extends ESNetty4IntegTestCase { @Override protected Settings nodeSettings(int nodeOrdinal) { - final Builder builder = Settings.builder().put(super.nodeSettings(nodeOrdinal)) - .put(TestZenDiscovery.USE_ZEN2.getKey(), true) - .put(ElectMasterService.DISCOVERY_ZEN_MINIMUM_MASTER_NODES_SETTING.getKey(), Integer.MAX_VALUE); - - return builder.build(); + return Settings.builder().put(super.nodeSettings(nodeOrdinal)).put(TestZenDiscovery.USE_ZEN2.getKey(), true).build(); } @Override diff --git a/plugins/discovery-ec2/src/test/java/org/elasticsearch/discovery/ec2/Ec2DiscoveryUpdateSettingsTests.java b/plugins/discovery-ec2/src/test/java/org/elasticsearch/discovery/ec2/Ec2DiscoveryUpdateSettingsTests.java index e91dff713b4f0..f11bd539fba7d 100644 --- a/plugins/discovery-ec2/src/test/java/org/elasticsearch/discovery/ec2/Ec2DiscoveryUpdateSettingsTests.java +++ b/plugins/discovery-ec2/src/test/java/org/elasticsearch/discovery/ec2/Ec2DiscoveryUpdateSettingsTests.java @@ -21,6 +21,7 @@ import org.elasticsearch.action.admin.cluster.settings.ClusterUpdateSettingsResponse; +import org.elasticsearch.common.UUIDs; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.discovery.DiscoveryModule; import org.elasticsearch.test.ESIntegTestCase.ClusterScope; @@ -41,13 +42,14 @@ public void testMinimumMasterNodesStart() { .build(); internalCluster().startNode(nodeSettings); - // We try to update minimum_master_nodes now - ClusterUpdateSettingsResponse response = client().admin().cluster().prepareUpdateSettings() - .setPersistentSettings(Settings.builder().put("discovery.zen.minimum_master_nodes", 1)) - .setTransientSettings(Settings.builder().put("discovery.zen.minimum_master_nodes", 1)) + // We try to update a setting now + final String expectedValue = UUIDs.randomBase64UUID(random()); + final String settingName = "cluster.routing.allocation.exclude.any_attribute"; + final ClusterUpdateSettingsResponse response = client().admin().cluster().prepareUpdateSettings() + .setPersistentSettings(Settings.builder().put(settingName, expectedValue)) .get(); - Integer min = response.getPersistentSettings().getAsInt("discovery.zen.minimum_master_nodes", null); - assertThat(min, is(1)); + final String value = response.getPersistentSettings().get(settingName); + assertThat(value, is(expectedValue)); } } diff --git a/qa/logging-config/src/test/java/org/elasticsearch/qa/custom_logging/CustomLoggingConfigIT.java b/qa/logging-config/src/test/java/org/elasticsearch/qa/custom_logging/CustomLoggingConfigIT.java index 407d23de99769..2e6b32d79fc41 100644 --- a/qa/logging-config/src/test/java/org/elasticsearch/qa/custom_logging/CustomLoggingConfigIT.java +++ b/qa/logging-config/src/test/java/org/elasticsearch/qa/custom_logging/CustomLoggingConfigIT.java @@ -40,7 +40,7 @@ * The intention is to confirm that users can still run their Elasticsearch instances with previous configurations. */ public class CustomLoggingConfigIT extends ESRestTestCase { - private static final String NODE_STARTED = ".*node-0.*cluster.uuid.*node.id.*started.*"; + private static final String NODE_STARTED = ".*node-0.*cluster.uuid.*node.id.*recovered.*cluster_state.*"; public void testSuccessfulStartupWithCustomConfig() throws Exception { assertBusy(() -> { diff --git a/qa/rolling-upgrade/build.gradle b/qa/rolling-upgrade/build.gradle index 6894463dd6fa6..160edea6a7898 100644 --- a/qa/rolling-upgrade/build.gradle +++ b/qa/rolling-upgrade/build.gradle @@ -77,7 +77,6 @@ for (Version version : bwcVersions.wireCompatible) { dependsOn lastRunner, "${baseName}#oldClusterTestCluster#node${stopNode}.stop" clusterName = 'rolling-upgrade' otherUnicastHostAddresses = { getOtherUnicastHostAddresses() } - minimumMasterNodes = { 2 } autoSetInitialMasterNodes = false /* Override the data directory so the new node always gets the node we * just stopped's data directory. */ diff --git a/rest-api-spec/src/main/resources/rest-api-spec/test/cluster.put_settings/10_basic.yml b/rest-api-spec/src/main/resources/rest-api-spec/test/cluster.put_settings/10_basic.yml index d801f3aeac89f..825bac9f91649 100644 --- a/rest-api-spec/src/main/resources/rest-api-spec/test/cluster.put_settings/10_basic.yml +++ b/rest-api-spec/src/main/resources/rest-api-spec/test/cluster.put_settings/10_basic.yml @@ -4,22 +4,22 @@ cluster.put_settings: body: transient: - discovery.zen.minimum_master_nodes: 1 + cluster.routing.allocation.enable: "none" flat_settings: true - - match: {transient: {discovery.zen.minimum_master_nodes: "1"}} + - match: {transient: {cluster.routing.allocation.enable: "none"}} - do: cluster.get_settings: flat_settings: true - - match: {transient: {discovery.zen.minimum_master_nodes: "1"}} + - match: {transient: {cluster.routing.allocation.enable: "none"}} - do: cluster.put_settings: body: transient: - discovery.zen.minimum_master_nodes: null + cluster.routing.allocation.enable: null flat_settings: true - match: {transient: {}} @@ -35,22 +35,22 @@ cluster.put_settings: body: persistent: - cluster.routing.allocation.disk.threshold_enabled: false + cluster.routing.allocation.enable: "none" flat_settings: true - - match: {persistent: {cluster.routing.allocation.disk.threshold_enabled: "false"}} + - match: {persistent: {cluster.routing.allocation.enable: "none"}} - do: cluster.get_settings: flat_settings: true - - match: {persistent: {cluster.routing.allocation.disk.threshold_enabled: "false"}} + - match: {persistent: {cluster.routing.allocation.enable: "none"}} - do: cluster.put_settings: body: persistent: - cluster.routing.allocation.disk.threshold_enabled: null + cluster.routing.allocation.enable: null flat_settings: true - match: {persistent: {}} diff --git a/server/src/main/java/org/elasticsearch/discovery/zen/ElectMasterService.java b/server/src/main/java/org/elasticsearch/discovery/zen/ElectMasterService.java index 87ad0a396ca76..7e0f0cfca2a99 100644 --- a/server/src/main/java/org/elasticsearch/discovery/zen/ElectMasterService.java +++ b/server/src/main/java/org/elasticsearch/discovery/zen/ElectMasterService.java @@ -42,7 +42,7 @@ public class ElectMasterService { private static final Logger logger = LogManager.getLogger(ElectMasterService.class); public static final Setting DISCOVERY_ZEN_MINIMUM_MASTER_NODES_SETTING = - Setting.intSetting("discovery.zen.minimum_master_nodes", -1, Property.Dynamic, Property.NodeScope); + Setting.intSetting("discovery.zen.minimum_master_nodes", -1, Property.Dynamic, Property.NodeScope, Property.Deprecated); private volatile int minimumMasterNodes; diff --git a/server/src/test/java/org/elasticsearch/action/admin/indices/exists/IndicesExistsIT.java b/server/src/test/java/org/elasticsearch/action/admin/indices/exists/IndicesExistsIT.java index cd90cda2ba286..33c0d22473c65 100644 --- a/server/src/test/java/org/elasticsearch/action/admin/indices/exists/IndicesExistsIT.java +++ b/server/src/test/java/org/elasticsearch/action/admin/indices/exists/IndicesExistsIT.java @@ -23,7 +23,6 @@ import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.unit.TimeValue; import org.elasticsearch.discovery.MasterNotDiscoveredException; -import org.elasticsearch.discovery.zen.ElectMasterService; import org.elasticsearch.gateway.GatewayService; import org.elasticsearch.test.ESIntegTestCase; import org.elasticsearch.test.ESIntegTestCase.ClusterScope; @@ -51,8 +50,7 @@ protected List addExtraClusterBootstrapSettings(List allNode public void testIndexExistsWithBlocksInPlace() throws IOException { Settings settings = Settings.builder() - .put(GatewayService.RECOVER_AFTER_NODES_SETTING.getKey(), 99) - .put(ElectMasterService.DISCOVERY_ZEN_MINIMUM_MASTER_NODES_SETTING.getKey(), Integer.MAX_VALUE).build(); + .put(GatewayService.RECOVER_AFTER_NODES_SETTING.getKey(), 99).build(); String node = internalCluster().startNode(settings); assertThrows(client(node).admin().indices().prepareExists("test").setMasterNodeTimeout(TimeValue.timeValueSeconds(0)), diff --git a/server/src/test/java/org/elasticsearch/cluster/MinimumMasterNodesIT.java b/server/src/test/java/org/elasticsearch/cluster/MinimumMasterNodesIT.java index c16d9b11e0d91..26b8ae88d266d 100644 --- a/server/src/test/java/org/elasticsearch/cluster/MinimumMasterNodesIT.java +++ b/server/src/test/java/org/elasticsearch/cluster/MinimumMasterNodesIT.java @@ -26,14 +26,13 @@ import org.elasticsearch.action.admin.cluster.health.ClusterHealthResponse; import org.elasticsearch.client.Client; import org.elasticsearch.cluster.coordination.ClusterBootstrapService; +import org.elasticsearch.cluster.coordination.FailedToCommitClusterStateException; import org.elasticsearch.cluster.metadata.MetaData; import org.elasticsearch.cluster.node.DiscoveryNode; import org.elasticsearch.cluster.service.ClusterService; import org.elasticsearch.common.Priority; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.discovery.DiscoverySettings; -import org.elasticsearch.cluster.coordination.FailedToCommitClusterStateException; -import org.elasticsearch.discovery.zen.ElectMasterService; import org.elasticsearch.discovery.zen.ZenDiscovery; import org.elasticsearch.index.query.QueryBuilders; import org.elasticsearch.node.Node; @@ -102,7 +101,6 @@ public void testTwoNodesNoMasterBlock() throws Exception { bootstrapNodeId = 2; Settings settings = Settings.builder() - .put("discovery.zen.minimum_master_nodes", 2) .put(ZenDiscovery.PING_TIMEOUT_SETTING.getKey(), "200ms") .put("discovery.initial_state_timeout", "500ms") .build(); @@ -237,7 +235,6 @@ public void testThreeNodesNoMasterBlock() throws Exception { bootstrapNodeId = 3; Settings settings = Settings.builder() - .put("discovery.zen.minimum_master_nodes", 3) .put(ZenDiscovery.PING_TIMEOUT_SETTING.getKey(), "1s") .put("discovery.initial_state_timeout", "500ms") .build(); @@ -316,11 +313,9 @@ public void testCannotCommitStateThreeNodes() throws Exception { Settings settings = Settings.builder() .put(ZenDiscovery.PING_TIMEOUT_SETTING.getKey(), "200ms") .put("discovery.initial_state_timeout", "500ms") - .put(ElectMasterService.DISCOVERY_ZEN_MINIMUM_MASTER_NODES_SETTING.getKey(), 2) .put(DiscoverySettings.COMMIT_TIMEOUT_SETTING.getKey(), "100ms") // speed things up .build(); - internalCluster().startNodes(3, settings); ensureGreen(); // ensure cluster state is recovered before we disrupt things diff --git a/server/src/test/java/org/elasticsearch/cluster/SpecificMasterNodesIT.java b/server/src/test/java/org/elasticsearch/cluster/SpecificMasterNodesIT.java index aaf01b5e6e079..8758e169b5124 100644 --- a/server/src/test/java/org/elasticsearch/cluster/SpecificMasterNodesIT.java +++ b/server/src/test/java/org/elasticsearch/cluster/SpecificMasterNodesIT.java @@ -38,7 +38,6 @@ import java.util.Collections; import java.util.List; -import static org.elasticsearch.discovery.zen.ElectMasterService.DISCOVERY_ZEN_MINIMUM_MASTER_NODES_SETTING; import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertAcked; import static org.hamcrest.Matchers.equalTo; import static org.hamcrest.Matchers.nullValue; @@ -47,12 +46,6 @@ @TestLogging("_root:DEBUG,org.elasticsearch.action.admin.cluster.state:TRACE") public class SpecificMasterNodesIT extends ESIntegTestCase { - @Override - protected Settings nodeSettings(int nodeOrdinal) { - return Settings.builder().put(super.nodeSettings(nodeOrdinal)) - .put(DISCOVERY_ZEN_MINIMUM_MASTER_NODES_SETTING.getKey(), 1).build(); - } - @Override protected List addExtraClusterBootstrapSettings(List allNodesSettings) { // if it's the first master in the cluster bootstrap the cluster with this node name diff --git a/server/src/test/java/org/elasticsearch/cluster/coordination/UnsafeBootstrapMasterIT.java b/server/src/test/java/org/elasticsearch/cluster/coordination/UnsafeBootstrapMasterIT.java index 73add5ba83520..334d392b1793d 100644 --- a/server/src/test/java/org/elasticsearch/cluster/coordination/UnsafeBootstrapMasterIT.java +++ b/server/src/test/java/org/elasticsearch/cluster/coordination/UnsafeBootstrapMasterIT.java @@ -28,7 +28,6 @@ import org.elasticsearch.cluster.metadata.MetaData; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.discovery.DiscoverySettings; -import org.elasticsearch.discovery.zen.ElectMasterService; import org.elasticsearch.env.Environment; import org.elasticsearch.env.NodeEnvironment; import org.elasticsearch.env.NodeMetaData; @@ -155,7 +154,6 @@ public void testNoNodeMetaData() throws IOException { public void testNotBootstrappedCluster() throws Exception { internalCluster().startNode( Settings.builder() - .put(ElectMasterService.DISCOVERY_ZEN_MINIMUM_MASTER_NODES_SETTING.getKey(), Integer.MAX_VALUE) .put(DiscoverySettings.INITIAL_STATE_TIMEOUT_SETTING.getKey(), "0s") // to ensure quick node startup .build()); assertBusy(() -> { @@ -172,9 +170,7 @@ public void testNotBootstrappedCluster() throws Exception { public void testNoManifestFile() throws IOException { bootstrapNodeId = 1; - internalCluster().startNode(Settings.builder() - .put(ElectMasterService.DISCOVERY_ZEN_MINIMUM_MASTER_NODES_SETTING.getKey(), Integer.MAX_VALUE) - .build()); + internalCluster().startNode(); ensureStableCluster(1); NodeEnvironment nodeEnvironment = internalCluster().getMasterNodeInstance(NodeEnvironment.class); internalCluster().stopRandomDataNode(); @@ -186,9 +182,7 @@ public void testNoManifestFile() throws IOException { public void testNoMetaData() throws IOException { bootstrapNodeId = 1; - internalCluster().startNode(Settings.builder() - .put(ElectMasterService.DISCOVERY_ZEN_MINIMUM_MASTER_NODES_SETTING.getKey(), Integer.MAX_VALUE) - .build()); + internalCluster().startNode(); ensureStableCluster(1); NodeEnvironment nodeEnvironment = internalCluster().getMasterNodeInstance(NodeEnvironment.class); internalCluster().stopRandomDataNode(); @@ -201,9 +195,7 @@ public void testNoMetaData() throws IOException { public void testAbortedByUser() throws IOException { bootstrapNodeId = 1; - internalCluster().startNode(Settings.builder() - .put(ElectMasterService.DISCOVERY_ZEN_MINIMUM_MASTER_NODES_SETTING.getKey(), Integer.MAX_VALUE) - .build()); + internalCluster().startNode(); ensureStableCluster(1); internalCluster().stopRandomDataNode(); @@ -213,13 +205,9 @@ public void testAbortedByUser() throws IOException { public void test3MasterNodes2Failed() throws Exception { bootstrapNodeId = 3; - List masterNodes = internalCluster().startMasterOnlyNodes(3, Settings.builder() - .put(ElectMasterService.DISCOVERY_ZEN_MINIMUM_MASTER_NODES_SETTING.getKey(), Integer.MAX_VALUE) - .build()); + List masterNodes = internalCluster().startMasterOnlyNodes(3, Settings.EMPTY); - String dataNode = internalCluster().startDataOnlyNode(Settings.builder() - .put(ElectMasterService.DISCOVERY_ZEN_MINIMUM_MASTER_NODES_SETTING.getKey(), Integer.MAX_VALUE) - .build()); + String dataNode = internalCluster().startDataOnlyNode(); createIndex("test"); Client dataNodeClient = internalCluster().client(dataNode); @@ -246,9 +234,7 @@ public void test3MasterNodes2Failed() throws Exception { String.format(Locale.ROOT, UnsafeBootstrapMasterCommand.CLUSTER_STATE_TERM_VERSION_MSG_FORMAT, metaData.coordinationMetaData().term(), metaData.version()))); - internalCluster().startMasterOnlyNode(Settings.builder() - .put(ElectMasterService.DISCOVERY_ZEN_MINIMUM_MASTER_NODES_SETTING.getKey(), Integer.MAX_VALUE) - .build()); + internalCluster().startMasterOnlyNode(); assertBusy(() -> { ClusterState state = dataNodeClient.admin().cluster().prepareState().setLocal(true) diff --git a/server/src/test/java/org/elasticsearch/cluster/coordination/Zen1IT.java b/server/src/test/java/org/elasticsearch/cluster/coordination/Zen1IT.java index eb4b2d75c73f9..e8cd691129745 100644 --- a/server/src/test/java/org/elasticsearch/cluster/coordination/Zen1IT.java +++ b/server/src/test/java/org/elasticsearch/cluster/coordination/Zen1IT.java @@ -34,6 +34,7 @@ import org.elasticsearch.common.Priority; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.unit.TimeValue; +import org.elasticsearch.discovery.zen.ElectMasterService; import org.elasticsearch.env.NodeEnvironment; import org.elasticsearch.gateway.MetaStateService; import org.elasticsearch.test.ESIntegTestCase; @@ -50,6 +51,7 @@ import static org.elasticsearch.cluster.routing.allocation.decider.EnableAllocationDecider.CLUSTER_ROUTING_ALLOCATION_ENABLE_SETTING; import static org.elasticsearch.cluster.routing.allocation.decider.FilterAllocationDecider.CLUSTER_ROUTING_EXCLUDE_GROUP_SETTING; import static org.elasticsearch.node.Node.NODE_NAME_SETTING; +import static org.elasticsearch.test.InternalTestCluster.REMOVED_MINIMUM_MASTER_NODES; import static org.hamcrest.Matchers.equalTo; import static org.hamcrest.Matchers.is; @@ -194,7 +196,8 @@ public Settings onNodeStopped(String nodeName) { } ClusterHealthResponse clusterHealthResponse = clusterHealthRequestBuilder.get(); assertFalse(nodeName, clusterHealthResponse.isTimedOut()); - return Coordinator.addZen1Attribute(false, Settings.builder().put(ZEN2_SETTINGS)).build(); + return Coordinator.addZen1Attribute(false, Settings.builder().put(ZEN2_SETTINGS) + .put(ElectMasterService.DISCOVERY_ZEN_MINIMUM_MASTER_NODES_SETTING.getKey(), REMOVED_MINIMUM_MASTER_NODES)).build(); } }); @@ -289,6 +292,7 @@ public Settings onNodeStopped(String nodeName) throws Exception { return Coordinator.addZen1Attribute(false, Settings.builder()) .put(ZEN2_SETTINGS) .putList(INITIAL_MASTER_NODES_SETTING.getKey(), nodeNames) + .put(ElectMasterService.DISCOVERY_ZEN_MINIMUM_MASTER_NODES_SETTING.getKey(), REMOVED_MINIMUM_MASTER_NODES) .build(); } }); diff --git a/server/src/test/java/org/elasticsearch/discovery/MasterDisruptionIT.java b/server/src/test/java/org/elasticsearch/discovery/MasterDisruptionIT.java index 652ce1fca1e14..718904eecb5bb 100644 --- a/server/src/test/java/org/elasticsearch/discovery/MasterDisruptionIT.java +++ b/server/src/test/java/org/elasticsearch/discovery/MasterDisruptionIT.java @@ -35,7 +35,6 @@ import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.unit.TimeValue; import org.elasticsearch.common.xcontent.XContentType; -import org.elasticsearch.discovery.zen.ElectMasterService; import org.elasticsearch.discovery.zen.ZenDiscovery; import org.elasticsearch.monitor.jvm.HotThreads; import org.elasticsearch.test.ESIntegTestCase; @@ -125,11 +124,6 @@ public void testNodesFDAfterMasterReelection() throws Exception { ensureStableCluster(3); - logger.info("--> reducing min master nodes to 2"); - assertAcked(client().admin().cluster().prepareUpdateSettings() - .setTransientSettings(Settings.builder().put(ElectMasterService.DISCOVERY_ZEN_MINIMUM_MASTER_NODES_SETTING.getKey(), 2)) - .get()); - String master = internalCluster().getMasterName(); String nonMaster = null; for (String node : internalCluster().getNodeNames()) { @@ -154,7 +148,7 @@ public void testNodesFDAfterMasterReelection() throws Exception { */ @TestLogging("_root:DEBUG,org.elasticsearch.cluster.service:TRACE,org.elasticsearch.test.disruption:TRACE") public void testStaleMasterNotHijackingMajority() throws Exception { - // 3 node cluster with unicast discovery and minimum_master_nodes set to the default of 2: + // 3 node cluster with unicast discovery: final List nodes = startCluster(3); // Save the current master node as old master node, because that node will get frozen diff --git a/server/src/test/java/org/elasticsearch/discovery/zen/MinimumMasterNodesInClusterStateIT.java b/server/src/test/java/org/elasticsearch/discovery/zen/MinimumMasterNodesInClusterStateIT.java deleted file mode 100644 index f60a313a5d41e..0000000000000 --- a/server/src/test/java/org/elasticsearch/discovery/zen/MinimumMasterNodesInClusterStateIT.java +++ /dev/null @@ -1,66 +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.discovery.zen; - -import org.elasticsearch.action.admin.cluster.state.ClusterStateRequest; -import org.elasticsearch.cluster.ClusterState; -import org.elasticsearch.test.ESIntegTestCase; - -import java.util.List; -import java.util.stream.Collectors; -import java.util.stream.Stream; - -import static org.elasticsearch.discovery.zen.ElectMasterService.DISCOVERY_ZEN_MINIMUM_MASTER_NODES_SETTING; -import static org.elasticsearch.test.InternalTestCluster.nameFilter; -import static org.hamcrest.Matchers.equalTo; -import static org.hamcrest.Matchers.isIn; - -@ESIntegTestCase.ClusterScope(numDataNodes = 0, numClientNodes = 0) -public class MinimumMasterNodesInClusterStateIT extends ESIntegTestCase { - - public void testMasterPublishes() throws Exception { - final String firstNode = internalCluster().startNode(); - - { - final ClusterState localState - = client(firstNode).admin().cluster().state(new ClusterStateRequest().local(true)).get().getState(); - assertThat(localState.getMinimumMasterNodesOnPublishingMaster(), equalTo(1)); - assertFalse(DISCOVERY_ZEN_MINIMUM_MASTER_NODES_SETTING.exists(localState.metaData().settings())); - } - - final List secondThirdNodes = internalCluster().startNodes(2); - assertThat(internalCluster().getMasterName(), equalTo(firstNode)); - - final List allNodes = Stream.concat(Stream.of(firstNode), secondThirdNodes.stream()).collect(Collectors.toList()); - for (final String node : allNodes) { - final ClusterState localState = client(node).admin().cluster().state(new ClusterStateRequest().local(true)).get().getState(); - assertThat(localState.getMinimumMasterNodesOnPublishingMaster(), equalTo(1)); - assertThat(DISCOVERY_ZEN_MINIMUM_MASTER_NODES_SETTING.get(localState.metaData().settings()), equalTo(2)); - } - - internalCluster().stopRandomNode(nameFilter(firstNode)); - assertThat(internalCluster().getMasterName(), isIn(secondThirdNodes)); - - for (final String node : secondThirdNodes) { - final ClusterState localState = client(node).admin().cluster().state(new ClusterStateRequest().local(true)).get().getState(); - assertThat(localState.getMinimumMasterNodesOnPublishingMaster(), equalTo(2)); - assertThat(DISCOVERY_ZEN_MINIMUM_MASTER_NODES_SETTING.get(localState.metaData().settings()), equalTo(2)); - } - } -} 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 c9a2f7dc58388..084ba62c4792d 100644 --- a/server/src/test/java/org/elasticsearch/discovery/zen/ZenDiscoveryUnitTests.java +++ b/server/src/test/java/org/elasticsearch/discovery/zen/ZenDiscoveryUnitTests.java @@ -268,6 +268,9 @@ public void testNodesUpdatedAfterClusterStatePublished() throws Exception { IOUtils.close(toClose); terminate(threadPool); } + + assertWarnings("[discovery.zen.minimum_master_nodes] setting was deprecated in Elasticsearch and will be removed in a future " + + "release! See the breaking changes documentation for the next major version."); } public void testPendingCSQueueIsClearedWhenClusterStatePublished() throws Exception { @@ -318,6 +321,9 @@ public void testPendingCSQueueIsClearedWhenClusterStatePublished() throws Except IOUtils.close(toClose); terminate(threadPool); } + + assertWarnings("[discovery.zen.minimum_master_nodes] setting was deprecated in Elasticsearch and will be removed in a future " + + "release! See the breaking changes documentation for the next major version."); } private class AwaitingPublishListener implements ActionListener { diff --git a/server/src/test/java/org/elasticsearch/gateway/RecoverAfterNodesIT.java b/server/src/test/java/org/elasticsearch/gateway/RecoverAfterNodesIT.java index e97f69b6d4965..e6fc2ed975fbb 100644 --- a/server/src/test/java/org/elasticsearch/gateway/RecoverAfterNodesIT.java +++ b/server/src/test/java/org/elasticsearch/gateway/RecoverAfterNodesIT.java @@ -25,7 +25,6 @@ import org.elasticsearch.cluster.coordination.ClusterBootstrapService; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.unit.TimeValue; -import org.elasticsearch.discovery.zen.ElectMasterService; import org.elasticsearch.node.Node; import org.elasticsearch.test.ESIntegTestCase; import org.elasticsearch.test.ESIntegTestCase.ClusterScope; @@ -59,7 +58,7 @@ protected List addExtraClusterBootstrapSettings(List allNode return super.addExtraClusterBootstrapSettings(allNodesSettings); } - public Set waitForNoBlocksOnNode(TimeValue timeout, Client nodeClient) throws InterruptedException { + public Set waitForNoBlocksOnNode(TimeValue timeout, Client nodeClient) { long start = System.currentTimeMillis(); Set blocks; do { @@ -70,22 +69,20 @@ public Set waitForNoBlocksOnNode(TimeValue timeout, Client nodeCli return blocks; } - public Client startNode(Settings.Builder settings, int minMasterNodes) { - String name = internalCluster().startNode( - Settings.builder().put(ElectMasterService.DISCOVERY_ZEN_MINIMUM_MASTER_NODES_SETTING.getKey(), minMasterNodes) - .put(settings.build())); + public Client startNode(Settings.Builder settings) { + String name = internalCluster().startNode(Settings.builder().put(settings.build())); return internalCluster().client(name); } public void testRecoverAfterNodes() throws Exception { logger.info("--> start node (1)"); - Client clientNode1 = startNode(Settings.builder().put("gateway.recover_after_nodes", 3), 1); + Client clientNode1 = startNode(Settings.builder().put("gateway.recover_after_nodes", 3)); assertThat(clientNode1.admin().cluster().prepareState().setLocal(true).execute().actionGet() .getState().blocks().global(ClusterBlockLevel.METADATA_WRITE), hasItem(GatewayService.STATE_NOT_RECOVERED_BLOCK)); logger.info("--> start node (2)"); - Client clientNode2 = startNode(Settings.builder().put("gateway.recover_after_nodes", 3), 1); + Client clientNode2 = startNode(Settings.builder().put("gateway.recover_after_nodes", 3)); Thread.sleep(BLOCK_WAIT_TIMEOUT.millis()); assertThat(clientNode1.admin().cluster().prepareState().setLocal(true).execute().actionGet() .getState().blocks().global(ClusterBlockLevel.METADATA_WRITE), @@ -95,7 +92,7 @@ public void testRecoverAfterNodes() throws Exception { hasItem(GatewayService.STATE_NOT_RECOVERED_BLOCK)); logger.info("--> start node (3)"); - Client clientNode3 = startNode(Settings.builder().put("gateway.recover_after_nodes", 3), 1); + Client clientNode3 = startNode(Settings.builder().put("gateway.recover_after_nodes", 3)); assertThat(waitForNoBlocksOnNode(BLOCK_WAIT_TIMEOUT, clientNode1).isEmpty(), equalTo(true)); assertThat(waitForNoBlocksOnNode(BLOCK_WAIT_TIMEOUT, clientNode2).isEmpty(), equalTo(true)); @@ -106,7 +103,7 @@ public void testRecoverAfterMasterNodes() throws Exception { logger.info("--> start master_node (1)"); Client master1 = startNode(Settings.builder() .put("gateway.recover_after_master_nodes", 2).put(Node.NODE_DATA_SETTING.getKey(), false) - .put(Node.NODE_MASTER_SETTING.getKey(), true), 1); + .put(Node.NODE_MASTER_SETTING.getKey(), true)); assertThat(master1.admin().cluster().prepareState().setLocal(true).execute().actionGet() .getState().blocks().global(ClusterBlockLevel.METADATA_WRITE), hasItem(GatewayService.STATE_NOT_RECOVERED_BLOCK)); @@ -114,7 +111,7 @@ public void testRecoverAfterMasterNodes() throws Exception { logger.info("--> start data_node (1)"); Client data1 = startNode(Settings.builder() .put("gateway.recover_after_master_nodes", 2) - .put(Node.NODE_DATA_SETTING.getKey(), true).put(Node.NODE_MASTER_SETTING.getKey(), false), 1); + .put(Node.NODE_DATA_SETTING.getKey(), true).put(Node.NODE_MASTER_SETTING.getKey(), false)); assertThat(master1.admin().cluster().prepareState().setLocal(true).execute().actionGet() .getState().blocks().global(ClusterBlockLevel.METADATA_WRITE), hasItem(GatewayService.STATE_NOT_RECOVERED_BLOCK)); @@ -125,7 +122,7 @@ public void testRecoverAfterMasterNodes() throws Exception { logger.info("--> start data_node (2)"); Client data2 = startNode(Settings.builder() .put("gateway.recover_after_master_nodes", 2).put(Node.NODE_DATA_SETTING.getKey(), true) - .put(Node.NODE_MASTER_SETTING.getKey(), false), 1); + .put(Node.NODE_MASTER_SETTING.getKey(), false)); assertThat(master1.admin().cluster().prepareState().setLocal(true).execute().actionGet() .getState().blocks().global(ClusterBlockLevel.METADATA_WRITE), hasItem(GatewayService.STATE_NOT_RECOVERED_BLOCK)); @@ -140,7 +137,7 @@ public void testRecoverAfterMasterNodes() throws Exception { Client master2 = startNode(Settings.builder() .put("gateway.recover_after_master_nodes", 2) .put(Node.NODE_DATA_SETTING.getKey(), false) - .put(Node.NODE_MASTER_SETTING.getKey(), true), 1); + .put(Node.NODE_MASTER_SETTING.getKey(), true)); assertThat(waitForNoBlocksOnNode(BLOCK_WAIT_TIMEOUT, master1).isEmpty(), equalTo(true)); assertThat(waitForNoBlocksOnNode(BLOCK_WAIT_TIMEOUT, master2).isEmpty(), equalTo(true)); assertThat(waitForNoBlocksOnNode(BLOCK_WAIT_TIMEOUT, data1).isEmpty(), equalTo(true)); @@ -152,7 +149,7 @@ public void testRecoverAfterDataNodes() throws Exception { Client master1 = startNode(Settings.builder() .put("gateway.recover_after_data_nodes", 2) .put(Node.NODE_DATA_SETTING.getKey(), false) - .put(Node.NODE_MASTER_SETTING.getKey(), true), 1); + .put(Node.NODE_MASTER_SETTING.getKey(), true)); assertThat(master1.admin().cluster().prepareState().setLocal(true).execute().actionGet() .getState().blocks().global(ClusterBlockLevel.METADATA_WRITE), hasItem(GatewayService.STATE_NOT_RECOVERED_BLOCK)); @@ -161,7 +158,7 @@ public void testRecoverAfterDataNodes() throws Exception { Client data1 = startNode(Settings.builder() .put("gateway.recover_after_data_nodes", 2) .put(Node.NODE_DATA_SETTING.getKey(), true) - .put(Node.NODE_MASTER_SETTING.getKey(), false), 1); + .put(Node.NODE_MASTER_SETTING.getKey(), false)); assertThat(master1.admin().cluster().prepareState().setLocal(true).execute().actionGet() .getState().blocks().global(ClusterBlockLevel.METADATA_WRITE), hasItem(GatewayService.STATE_NOT_RECOVERED_BLOCK)); @@ -173,7 +170,7 @@ public void testRecoverAfterDataNodes() throws Exception { Client master2 = startNode(Settings.builder() .put("gateway.recover_after_data_nodes", 2) .put(Node.NODE_DATA_SETTING.getKey(), false) - .put(Node.NODE_MASTER_SETTING.getKey(), true), 1); + .put(Node.NODE_MASTER_SETTING.getKey(), true)); assertThat(master2.admin().cluster().prepareState().setLocal(true).execute().actionGet() .getState().blocks().global(ClusterBlockLevel.METADATA_WRITE), hasItem(GatewayService.STATE_NOT_RECOVERED_BLOCK)); @@ -188,7 +185,7 @@ public void testRecoverAfterDataNodes() throws Exception { Client data2 = startNode(Settings.builder() .put("gateway.recover_after_data_nodes", 2) .put(Node.NODE_DATA_SETTING.getKey(), true) - .put(Node.NODE_MASTER_SETTING.getKey(), false), 1); + .put(Node.NODE_MASTER_SETTING.getKey(), false)); assertThat(waitForNoBlocksOnNode(BLOCK_WAIT_TIMEOUT, master1).isEmpty(), equalTo(true)); assertThat(waitForNoBlocksOnNode(BLOCK_WAIT_TIMEOUT, master2).isEmpty(), equalTo(true)); assertThat(waitForNoBlocksOnNode(BLOCK_WAIT_TIMEOUT, data1).isEmpty(), equalTo(true)); diff --git a/test/framework/src/main/java/org/elasticsearch/test/ESIntegTestCase.java b/test/framework/src/main/java/org/elasticsearch/test/ESIntegTestCase.java index 6e3089a6d63e1..45f8682dc5e61 100644 --- a/test/framework/src/main/java/org/elasticsearch/test/ESIntegTestCase.java +++ b/test/framework/src/main/java/org/elasticsearch/test/ESIntegTestCase.java @@ -1087,17 +1087,6 @@ public long waitForDocs(final long numDocs, int maxWaitTime, TimeUnit maxWaitTim return lastKnownCount.get(); } - - /** - * Sets the cluster's minimum master node and make sure the response is acknowledge. - * Note: this doesn't guarantee that the new setting has taken effect, just that it has been received by all nodes. - */ - public void setMinimumMasterNodes(int n) { - assertTrue(client().admin().cluster().prepareUpdateSettings().setTransientSettings( - Settings.builder().put(ElectMasterService.DISCOVERY_ZEN_MINIMUM_MASTER_NODES_SETTING.getKey(), n)) - .get().isAcknowledged()); - } - /** * Prints the current cluster state as debug logging. */ 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 55e356d093e6e..9313d9389d49c 100644 --- a/test/framework/src/main/java/org/elasticsearch/test/InternalTestCluster.java +++ b/test/framework/src/main/java/org/elasticsearch/test/InternalTestCluster.java @@ -33,8 +33,8 @@ import org.elasticsearch.ElasticsearchException; import org.elasticsearch.action.admin.cluster.configuration.AddVotingConfigExclusionsAction; import org.elasticsearch.action.admin.cluster.configuration.AddVotingConfigExclusionsRequest; -import org.elasticsearch.action.admin.cluster.configuration.ClearVotingConfigExclusionsRequest; import org.elasticsearch.action.admin.cluster.configuration.ClearVotingConfigExclusionsAction; +import org.elasticsearch.action.admin.cluster.configuration.ClearVotingConfigExclusionsRequest; import org.elasticsearch.action.admin.cluster.node.stats.NodeStats; import org.elasticsearch.action.admin.indices.stats.CommonStatsFlags; import org.elasticsearch.action.admin.indices.stats.CommonStatsFlags.Flag; @@ -157,11 +157,15 @@ import static org.elasticsearch.test.ESTestCase.awaitBusy; import static org.elasticsearch.test.ESTestCase.getTestTransportType; import static org.elasticsearch.test.ESTestCase.randomFrom; +import static org.elasticsearch.test.discovery.TestZenDiscovery.USE_ZEN2; +import static org.elasticsearch.test.discovery.TestZenDiscovery.usingZen1; import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertAcked; import static org.hamcrest.Matchers.equalTo; import static org.hamcrest.Matchers.greaterThan; import static org.hamcrest.Matchers.greaterThanOrEqualTo; import static org.hamcrest.Matchers.not; +import static org.hamcrest.Matchers.nullValue; +import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertThat; import static org.junit.Assert.assertTrue; import static org.junit.Assert.fail; @@ -615,19 +619,28 @@ private Settings getNodeSettings(final int nodeId, final long seed, final Settin .put("node.name", name) .put(NodeEnvironment.NODE_ID_SEED_SETTING.getKey(), seed); - final boolean usingSingleNodeDiscovery = DiscoveryModule.DISCOVERY_TYPE_SETTING.get(updatedSettings.build()).equals("single-node"); - if (!usingSingleNodeDiscovery && autoManageMinMasterNodes) { - assert updatedSettings.get(DISCOVERY_ZEN_MINIMUM_MASTER_NODES_SETTING.getKey()) == null : - "min master nodes may not be set when auto managed"; - assert updatedSettings.get(INITIAL_STATE_TIMEOUT_SETTING.getKey()) == null : - "automatically managing min master nodes require nodes to complete a join cycle" + - " when starting"; - updatedSettings - // don't wait too long not to slow down tests - .put(ZenDiscovery.MASTER_ELECTION_WAIT_FOR_JOINS_TIMEOUT_SETTING.getKey(), "5s") - .put(DISCOVERY_ZEN_MINIMUM_MASTER_NODES_SETTING.getKey(), defaultMinMasterNodes); - } else if (!usingSingleNodeDiscovery && updatedSettings.get(DISCOVERY_ZEN_MINIMUM_MASTER_NODES_SETTING.getKey()) == null) { - throw new IllegalArgumentException(DISCOVERY_ZEN_MINIMUM_MASTER_NODES_SETTING.getKey() + " must be configured"); + final String discoveryType = DiscoveryModule.DISCOVERY_TYPE_SETTING.get(updatedSettings.build()); + final boolean usingSingleNodeDiscovery = discoveryType.equals("single-node"); + final boolean usingZen1 = TestZenDiscovery.usingZen1(updatedSettings.build()); + if (usingSingleNodeDiscovery == false) { + if (autoManageMinMasterNodes) { + assertThat("min master nodes may not be set when auto managed", + updatedSettings.get(DISCOVERY_ZEN_MINIMUM_MASTER_NODES_SETTING.getKey()), nullValue()); + assertThat("automatically managing min master nodes require nodes to complete a join cycle when starting", + updatedSettings.get(INITIAL_STATE_TIMEOUT_SETTING.getKey()), nullValue()); + + if (usingZen1) { + updatedSettings + // don't wait too long not to slow down tests + .put(ZenDiscovery.MASTER_ELECTION_WAIT_FOR_JOINS_TIMEOUT_SETTING.getKey(), "5s") + .put(DISCOVERY_ZEN_MINIMUM_MASTER_NODES_SETTING.getKey(), defaultMinMasterNodes); + } + } else { + if (usingZen1) { + assertThat(DISCOVERY_ZEN_MINIMUM_MASTER_NODES_SETTING.getKey() + " must be configured", + updatedSettings.get(DISCOVERY_ZEN_MINIMUM_MASTER_NODES_SETTING.getKey()), not(nullValue())); + } + } } return updatedSettings.build(); @@ -822,6 +835,8 @@ public synchronized void close() { } } + public static final int REMOVED_MINIMUM_MASTER_NODES = Integer.MAX_VALUE; + private final class NodeAndClient implements Closeable { private MockNode node; private final Settings originalNodeSettings; @@ -933,8 +948,10 @@ Settings closeForRestart(RestartCallback callback, int minMasterNodes) throws Ex Settings.Builder newSettings = Settings.builder(); newSettings.put(callbackSettings); if (minMasterNodes >= 0) { - assert DISCOVERY_ZEN_MINIMUM_MASTER_NODES_SETTING.exists(newSettings.build()) == false : "min master nodes is auto managed"; - newSettings.put(DISCOVERY_ZEN_MINIMUM_MASTER_NODES_SETTING.getKey(), minMasterNodes); + if (usingZen1(newSettings.build())) { + assertFalse("min master nodes is auto managed", DISCOVERY_ZEN_MINIMUM_MASTER_NODES_SETTING.exists(newSettings.build())); + newSettings.put(DISCOVERY_ZEN_MINIMUM_MASTER_NODES_SETTING.getKey(), minMasterNodes); + } if (INITIAL_MASTER_NODES_SETTING.exists(callbackSettings) == false) { newSettings.putList(INITIAL_MASTER_NODES_SETTING.getKey()); } @@ -966,10 +983,23 @@ private void recreateNode(final Settings newSettings, final Runnable onTransport .put(newSettings) .put(NodeEnvironment.NODE_ID_SEED_SETTING.getKey(), newIdSeed) .build(); - final boolean usingSingleNodeDiscovery = DiscoveryModule.DISCOVERY_TYPE_SETTING.get(finalSettings).equals("single-node"); - if (usingSingleNodeDiscovery == false && DISCOVERY_ZEN_MINIMUM_MASTER_NODES_SETTING.exists(finalSettings) == false) { - throw new IllegalStateException(DISCOVERY_ZEN_MINIMUM_MASTER_NODES_SETTING.getKey() + - " is not configured after restart of [" + name + "]"); + if (usingZen1(finalSettings)) { + if (DISCOVERY_ZEN_MINIMUM_MASTER_NODES_SETTING.exists(finalSettings) == false) { + throw new IllegalStateException(DISCOVERY_ZEN_MINIMUM_MASTER_NODES_SETTING.getKey() + + " is not configured after restart of [" + name + "]"); + } + } else { + if (DISCOVERY_ZEN_MINIMUM_MASTER_NODES_SETTING.exists(finalSettings)) { + // simulating an upgrade from Zen1 to Zen2, but there's no way to remove a setting when restarting a node, so + // you have to set it to REMOVED_MINIMUM_MASTER_NODES (== Integer.MAX_VALUE) to indicate its removal: + assertTrue(USE_ZEN2.exists(finalSettings)); + assertTrue(USE_ZEN2.get(finalSettings)); + assertThat(DISCOVERY_ZEN_MINIMUM_MASTER_NODES_SETTING.get(finalSettings), equalTo(REMOVED_MINIMUM_MASTER_NODES)); + + final Builder builder = Settings.builder().put(finalSettings); + builder.remove(DISCOVERY_ZEN_MINIMUM_MASTER_NODES_SETTING.getKey()); + finalSettings = builder.build(); + } } Collection> plugins = node.getClasspathPlugins(); node = new MockNode(finalSettings, plugins); 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 53be34c0b40c8..56e6c24571715 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 @@ -60,6 +60,7 @@ public class TestZenDiscovery extends ZenDiscovery { public static final Setting USE_ZEN2 = Setting.boolSetting("discovery.zen.use_zen2", true, Setting.Property.NodeScope); + private static final String TEST_ZEN_DISCOVERY_TYPE = "test-zen"; /** A plugin which installs mock discovery and configures it to be used. */ public static class TestPlugin extends Plugin implements DiscoveryPlugin { @@ -97,7 +98,7 @@ public List> getSettings() { @Override public Settings additionalSettings() { return Settings.builder() - .put(DiscoveryModule.DISCOVERY_TYPE_SETTING.getKey(), "test-zen") + .put(DiscoveryModule.DISCOVERY_TYPE_SETTING.getKey(), TEST_ZEN_DISCOVERY_TYPE) .putList(DISCOVERY_ZEN_PING_UNICAST_HOSTS_SETTING.getKey()) .build(); } @@ -124,4 +125,9 @@ protected ZenPing newZenPing(Settings settings, ThreadPool threadPool, Transport public ZenPing getZenPing() { return zenPing; } + + public static boolean usingZen1(Settings settings) { + return DiscoveryModule.ZEN_DISCOVERY_TYPE.equals(DiscoveryModule.DISCOVERY_TYPE_SETTING.get(settings)) + || USE_ZEN2.get(settings) == false; + } } 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 38d76423eea9c..ca2fe8c753e44 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 @@ -63,7 +63,6 @@ import static org.elasticsearch.cluster.node.DiscoveryNode.Role.INGEST; import static org.elasticsearch.cluster.node.DiscoveryNode.Role.MASTER; import static org.elasticsearch.discovery.DiscoveryModule.DISCOVERY_HOSTS_PROVIDER_SETTING; -import static org.elasticsearch.discovery.zen.ElectMasterService.DISCOVERY_ZEN_MINIMUM_MASTER_NODES_SETTING; import static org.elasticsearch.node.Node.NODE_MASTER_SETTING; import static org.elasticsearch.node.Node.NODE_NAME_SETTING; import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertFileExists; @@ -101,7 +100,6 @@ public void testInitializiationIsConsistent() { nodePrefix, Collections.emptyList(), Function.identity()); // TODO: this is not ideal - we should have a way to make sure ports are initialized in the same way assertClusters(cluster0, cluster1, false); - } /** @@ -140,36 +138,6 @@ public static void assertSettings(Settings left, Settings right, boolean checkCl } } - private void assertMMNinNodeSetting(InternalTestCluster cluster, int masterNodes) { - for (final String node : cluster.getNodeNames()) { - assertMMNinNodeSetting(node, cluster, masterNodes); - } - } - - private void assertMMNinNodeSetting(String node, InternalTestCluster cluster, int masterNodes) { - final int minMasterNodes = masterNodes / 2 + 1; - Settings nodeSettings = cluster.client(node).admin().cluster().prepareNodesInfo(node).get().getNodes().get(0).getSettings(); - assertEquals("node setting of node [" + node + "] has the wrong min_master_node setting: [" - + nodeSettings.get(DISCOVERY_ZEN_MINIMUM_MASTER_NODES_SETTING.getKey()) + "]", - DISCOVERY_ZEN_MINIMUM_MASTER_NODES_SETTING.get(nodeSettings).intValue(), minMasterNodes); - } - - private void assertMMNinClusterSetting(InternalTestCluster cluster, int masterNodes) { - for (final String node : cluster.getNodeNames()) { - assertMMNinClusterSetting(node, cluster, masterNodes); - } - } - - private void assertMMNinClusterSetting(String node, InternalTestCluster cluster, int masterNodes) { - final int minMasterNodes = masterNodes / 2 + 1; - Settings stateSettings = cluster.client(node).admin().cluster().prepareState().setLocal(true) - .get().getState().getMetaData().settings(); - - assertEquals("dynamic setting for node [" + node + "] has the wrong min_master_node setting : [" - + stateSettings.get(DISCOVERY_ZEN_MINIMUM_MASTER_NODES_SETTING.getKey()) + "]", - DISCOVERY_ZEN_MINIMUM_MASTER_NODES_SETTING.get(stateSettings).intValue(), minMasterNodes); - } - public void testBeforeTest() throws Exception { final boolean autoManageMinMasterNodes = randomBoolean(); long clusterSeed = randomLong(); @@ -204,7 +172,6 @@ public Settings nodeSettings(int nodeOrdinal) { if (autoManageMinMasterNodes == false) { assert minNumDataNodes == maxNumDataNodes; assert masterNodes == false; - settings.put(DISCOVERY_ZEN_MINIMUM_MASTER_NODES_SETTING.getKey(), minNumDataNodes / 2 + 1); } return settings.build(); } @@ -257,8 +224,6 @@ public Settings transportClientSettings() { Client other = iterator1.next(); assertSettings(client.settings(), other.settings(), false); } - assertMMNinNodeSetting(cluster0, cluster0.numMasterNodes()); - assertMMNinNodeSetting(cluster1, cluster0.numMasterNodes()); cluster0.afterTest(); cluster1.afterTest(); } finally { @@ -320,7 +285,6 @@ public Settings transportClientSettings() { try { cluster.beforeTest(random(), 0.0); final int originalMasterCount = cluster.numMasterNodes(); - assertMMNinNodeSetting(cluster, originalMasterCount); final Map shardNodePaths = new HashMap<>(); for (String name: cluster.getNodeNames()) { shardNodePaths.put(name, getNodePaths(cluster, name)); @@ -335,10 +299,6 @@ public Settings transportClientSettings() { expectedMasterCount--; } cluster.stopRandomNode(InternalTestCluster.nameFilter(poorNode)); - if (expectedMasterCount != originalMasterCount) { - // check for updated - assertMMNinClusterSetting(cluster, expectedMasterCount); - } assertFileExists(testMarker); // stopping a node half way shouldn't clean data final String stableNode = randomFrom(cluster.getNodeNames()); @@ -351,14 +311,8 @@ public Settings transportClientSettings() { expectedMasterCount++; assertThat(getNodePaths(cluster, newNode1)[0], equalTo(dataPath)); assertFileExists(testMarker); // starting a node should re-use data folders and not clean it - if (expectedMasterCount > 1) { // this is the first master, it's in cluster state settings won't be updated - assertMMNinClusterSetting(cluster, expectedMasterCount); - } - assertMMNinNodeSetting(newNode1, cluster, expectedMasterCount); - final String newNode2 = cluster.startNode(); expectedMasterCount++; - assertMMNinClusterSetting(cluster, expectedMasterCount); final Path newDataPath = getNodePaths(cluster, newNode2)[0]; final Path newTestMarker = newDataPath.resolve("newTestMarker"); assertThat(newDataPath, not(dataPath)); @@ -377,8 +331,6 @@ public Settings transportClientSettings() { assertThat("data paths for " + name + " changed", getNodePaths(cluster, name), equalTo(shardNodePaths.get(name))); } - assertMMNinNodeSetting(cluster, originalMasterCount); - } finally { cluster.close(); } @@ -393,7 +345,6 @@ 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; @@ -448,11 +399,6 @@ public Settings transportClientSettings() { roles.add(role); } - final long masterCount = roles.stream().filter(role -> role == MASTER).count(); - final Settings minMasterNodes = Settings.builder() - .put(DISCOVERY_ZEN_MINIMUM_MASTER_NODES_SETTING.getKey(), masterCount / 2 + 1) - .build(); - try { Map> pathsPerRole = new HashMap<>(); for (int i = 0; i < numNodes; i++) { @@ -460,13 +406,13 @@ public Settings transportClientSettings() { final String node; switch (role) { case MASTER: - node = cluster.startMasterOnlyNode(minMasterNodes); + node = cluster.startMasterOnlyNode(); break; case DATA: - node = cluster.startDataOnlyNode(minMasterNodes); + node = cluster.startDataOnlyNode(); break; case INGEST: - node = cluster.startCoordinatingOnlyNode(minMasterNodes); + node = cluster.startCoordinatingOnlyNode(Settings.EMPTY); break; default: throw new IllegalStateException("get your story straight"); @@ -476,6 +422,7 @@ public Settings transportClientSettings() { assertTrue(rolePaths.add(path.toString())); } } + cluster.validateClusterFormed(); cluster.fullRestart(); Map> result = new HashMap<>(); @@ -533,34 +480,18 @@ public Settings transportClientSettings() { plugins, Function.identity()); try { cluster.beforeTest(random(), 0.0); - assertMMNinNodeSetting(cluster, 2); switch (randomInt(2)) { case 0: cluster.stopRandomDataNode(); - assertMMNinClusterSetting(cluster, 1); cluster.startNode(); - assertMMNinClusterSetting(cluster, 2); - assertMMNinNodeSetting(cluster, 2); break; case 1: - cluster.rollingRestart(new InternalTestCluster.RestartCallback() { - @Override - public Settings onNodeStopped(String nodeName) throws Exception { - for (String name : cluster.getNodeNames()) { - if (name.equals(nodeName) == false) { - assertMMNinClusterSetting(name, cluster, 1); - } - } - return super.onNodeStopped(nodeName); - } - }); - assertMMNinClusterSetting(cluster, 2); + cluster.rollingRestart(InternalTestCluster.EMPTY_CALLBACK); break; case 2: cluster.fullRestart(); break; } - assertMMNinNodeSetting(cluster, 2); } finally { cluster.close(); } @@ -585,6 +516,5 @@ public Settings additionalSettings() { } return Settings.builder().put("node.attr.dummy", true).build(); } - } } diff --git a/x-pack/plugin/security/src/test/java/org/elasticsearch/license/LicensingTests.java b/x-pack/plugin/security/src/test/java/org/elasticsearch/license/LicensingTests.java index e79ec2e2fc059..f146f12245e1f 100644 --- a/x-pack/plugin/security/src/test/java/org/elasticsearch/license/LicensingTests.java +++ b/x-pack/plugin/security/src/test/java/org/elasticsearch/license/LicensingTests.java @@ -26,6 +26,7 @@ import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.util.concurrent.ThreadContext; import org.elasticsearch.discovery.DiscoveryModule; +import org.elasticsearch.discovery.zen.ElectMasterService; import org.elasticsearch.node.MockNode; import org.elasticsearch.node.Node; import org.elasticsearch.plugins.Plugin; @@ -290,23 +291,25 @@ public void testNodeJoinWithoutSecurityExplicitlyEnabled() throws Exception { Path home = createTempDir(); Path conf = home.resolve("config"); Files.createDirectories(conf); - Settings nodeSettings = Settings.builder() + Settings.Builder nodeSettings = Settings.builder() .put(nodeSettings(maxNumberOfNodes() - 1).filter(s -> "xpack.security.enabled".equals(s) == false)) .put("node.name", "my-test-node") .put("network.host", "localhost") .put("cluster.name", internalCluster().getClusterName()) - .put("discovery.zen.minimum_master_nodes", - internalCluster().getInstance(Settings.class).get("discovery.zen.minimum_master_nodes")) .put("path.home", home) .put(TestZenDiscovery.USE_MOCK_PINGS.getKey(), false) .put(DiscoveryModule.DISCOVERY_TYPE_SETTING.getKey(), "test-zen") .put(TestZenDiscovery.USE_ZEN2.getKey(), getUseZen2()) .putList(DiscoveryModule.DISCOVERY_HOSTS_PROVIDER_SETTING.getKey()) - .putList(DISCOVERY_ZEN_PING_UNICAST_HOSTS_SETTING.getKey(), unicastHostsList) - .build(); + .putList(DISCOVERY_ZEN_PING_UNICAST_HOSTS_SETTING.getKey(), unicastHostsList); + if (getUseZen2() == false) { + nodeSettings.put(ElectMasterService.DISCOVERY_ZEN_MINIMUM_MASTER_NODES_SETTING.getKey(), + ElectMasterService.DISCOVERY_ZEN_MINIMUM_MASTER_NODES_SETTING.get(internalCluster().getInstance(Settings.class))); + } + Collection> mockPlugins = Arrays.asList(LocalStateSecurity.class, TestZenDiscovery.TestPlugin.class, MockHttpTransport.TestPlugin.class); - try (Node node = new MockNode(nodeSettings, mockPlugins)) { + try (Node node = new MockNode(nodeSettings.build(), mockPlugins)) { node.start(); ensureStableCluster(cluster().size() + 1); } 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 2383f3b3ac739..83640d9e931b5 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 @@ -13,6 +13,7 @@ import org.elasticsearch.common.network.NetworkAddress; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.transport.TransportAddress; +import org.elasticsearch.discovery.zen.ElectMasterService; import org.elasticsearch.node.MockNode; import org.elasticsearch.node.Node; import org.elasticsearch.node.NodeValidationException; @@ -101,8 +102,6 @@ public void testThatConnectionToServerTypeConnectionWorks() throws IOException, .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) @@ -111,6 +110,10 @@ public void testThatConnectionToServerTypeConnectionWorks() throws IOException, .put(Node.NODE_MASTER_SETTING.getKey(), false) .put(TestZenDiscovery.USE_ZEN2.getKey(), getUseZen2()) .put(TestZenDiscovery.USE_MOCK_PINGS.getKey(), false); + if (getUseZen2() == false) { + nodeSettings.put(ElectMasterService.DISCOVERY_ZEN_MINIMUM_MASTER_NODES_SETTING.getKey(), + ElectMasterService.DISCOVERY_ZEN_MINIMUM_MASTER_NODES_SETTING.get(internalCluster().getInstance(Settings.class))); + } Collection> mockPlugins = Arrays.asList( LocalStateSecurity.class, TestZenDiscovery.TestPlugin.class, MockHttpTransport.TestPlugin.class); addSSLSettingsForPEMFiles( @@ -144,8 +147,6 @@ public void testThatConnectionToClientTypeConnectionIsRejected() throws IOExcept .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) @@ -155,6 +156,10 @@ public void testThatConnectionToClientTypeConnectionIsRejected() throws IOExcept .put(Node.NODE_MASTER_SETTING.getKey(), false) .put(TestZenDiscovery.USE_ZEN2.getKey(), getUseZen2()) .put(TestZenDiscovery.USE_MOCK_PINGS.getKey(), false); + if (getUseZen2() == false) { + nodeSettings.put(ElectMasterService.DISCOVERY_ZEN_MINIMUM_MASTER_NODES_SETTING.getKey(), + ElectMasterService.DISCOVERY_ZEN_MINIMUM_MASTER_NODES_SETTING.get(internalCluster().getInstance(Settings.class))); + } Collection> mockPlugins = Arrays.asList( LocalStateSecurity.class, TestZenDiscovery.TestPlugin.class, MockHttpTransport.TestPlugin.class); addSSLSettingsForPEMFiles( From be788160efd23fed3eba5ef1d431cb1cb3e88cf6 Mon Sep 17 00:00:00 2001 From: David Roberts Date: Wed, 30 Jan 2019 20:12:20 +0000 Subject: [PATCH 07/16] [ML] Datafeed deprecation checks (#38026) Deprecation checks for the ML datafeed query and aggregations. --- .../deprecation/DeprecationInfoAction.java | 48 +++++++++++++++---- .../core/ml/datafeed/DatafeedConfig.java | 2 +- .../DeprecationInfoActionResponseTests.java | 22 +++++++-- .../xpack/deprecation/DeprecationChecks.java | 7 +++ .../deprecation/MlDeprecationChecks.java | 44 +++++++++++++++++ .../TransportDeprecationInfoAction.java | 42 +++++++++++++--- .../deprecation/MlDeprecationChecksTests.java | 48 +++++++++++++++++++ .../test/deprecation/10_basic.yml | 46 ++++++++++++++++++ 8 files changed, 239 insertions(+), 20 deletions(-) create mode 100644 x-pack/plugin/deprecation/src/main/java/org/elasticsearch/xpack/deprecation/MlDeprecationChecks.java create mode 100644 x-pack/plugin/deprecation/src/test/java/org/elasticsearch/xpack/deprecation/MlDeprecationChecksTests.java diff --git a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/deprecation/DeprecationInfoAction.java b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/deprecation/DeprecationInfoAction.java index 09c6a0d57524e..1241b136c7a6e 100644 --- a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/deprecation/DeprecationInfoAction.java +++ b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/deprecation/DeprecationInfoAction.java @@ -5,6 +5,7 @@ */ package org.elasticsearch.xpack.core.deprecation; +import org.elasticsearch.Version; import org.elasticsearch.action.Action; import org.elasticsearch.action.ActionRequestValidationException; import org.elasticsearch.action.ActionResponse; @@ -23,9 +24,12 @@ import org.elasticsearch.common.io.stream.StreamOutput; import org.elasticsearch.common.xcontent.ToXContentObject; import org.elasticsearch.common.xcontent.XContentBuilder; +import org.elasticsearch.xpack.core.ml.datafeed.DatafeedConfig; import java.io.IOException; +import java.util.ArrayList; import java.util.Arrays; +import java.util.Collections; import java.util.HashMap; import java.util.List; import java.util.Map; @@ -67,16 +71,19 @@ public static class Response extends ActionResponse implements ToXContentObject private List clusterSettingsIssues; private List nodeSettingsIssues; private Map> indexSettingsIssues; + private List mlSettingsIssues; public Response() { } public Response(List clusterSettingsIssues, List nodeSettingsIssues, - Map> indexSettingsIssues) { + Map> indexSettingsIssues, + List mlSettingsIssues) { this.clusterSettingsIssues = clusterSettingsIssues; this.nodeSettingsIssues = nodeSettingsIssues; this.indexSettingsIssues = indexSettingsIssues; + this.mlSettingsIssues = mlSettingsIssues; } public List getClusterSettingsIssues() { @@ -91,12 +98,21 @@ public Map> getIndexSettingsIssues() { return indexSettingsIssues; } + public List getMlSettingsIssues() { + return mlSettingsIssues; + } + @Override public void readFrom(StreamInput in) throws IOException { super.readFrom(in); clusterSettingsIssues = in.readList(DeprecationIssue::new); nodeSettingsIssues = in.readList(DeprecationIssue::new); indexSettingsIssues = in.readMapOfLists(StreamInput::readString, DeprecationIssue::new); + if (in.getVersion().onOrAfter(Version.V_6_7_0)) { + mlSettingsIssues = in.readList(DeprecationIssue::new); + } else { + mlSettingsIssues = Collections.emptyList(); + } } @Override @@ -105,6 +121,9 @@ public void writeTo(StreamOutput out) throws IOException { out.writeList(clusterSettingsIssues); out.writeList(nodeSettingsIssues); out.writeMapOfLists(indexSettingsIssues, StreamOutput::writeString, (o, v) -> v.writeTo(o)); + if (out.getVersion().onOrAfter(Version.V_6_7_0)) { + out.writeList(mlSettingsIssues); + } } @Override @@ -114,10 +133,10 @@ public XContentBuilder toXContent(XContentBuilder builder, Params params) throws .array("node_settings", nodeSettingsIssues.toArray()) .field("index_settings") .map(indexSettingsIssues) + .array("ml_settings", mlSettingsIssues.toArray()) .endObject(); } - @Override public boolean equals(Object o) { if (this == o) return true; @@ -125,12 +144,13 @@ public boolean equals(Object o) { Response response = (Response) o; return Objects.equals(clusterSettingsIssues, response.clusterSettingsIssues) && Objects.equals(nodeSettingsIssues, response.nodeSettingsIssues) && - Objects.equals(indexSettingsIssues, response.indexSettingsIssues); + Objects.equals(indexSettingsIssues, response.indexSettingsIssues) && + Objects.equals(mlSettingsIssues, response.mlSettingsIssues); } @Override public int hashCode() { - return Objects.hash(clusterSettingsIssues, nodeSettingsIssues, indexSettingsIssues); + return Objects.hash(clusterSettingsIssues, nodeSettingsIssues, indexSettingsIssues, mlSettingsIssues); } /** @@ -145,22 +165,30 @@ public int hashCode() { * @param indexNameExpressionResolver Used to resolve indices into their concrete names * @param indices The list of index expressions to evaluate using `indexNameExpressionResolver` * @param indicesOptions The options to use when resolving and filtering which indices to check + * @param datafeeds The ml datafeed configurations * @param clusterSettingsChecks The list of cluster-level checks * @param nodeSettingsChecks The list of node-level checks * @param indexSettingsChecks The list of index-level checks that will be run across all specified * concrete indices + * @param mlSettingsCheck The list of ml checks * @return The list of deprecation issues found in the cluster */ public static DeprecationInfoAction.Response from(List nodesInfo, List nodesStats, ClusterState state, - IndexNameExpressionResolver indexNameExpressionResolver, - String[] indices, IndicesOptions indicesOptions, - List>clusterSettingsChecks, - List, List, DeprecationIssue>> nodeSettingsChecks, - List> indexSettingsChecks) { + IndexNameExpressionResolver indexNameExpressionResolver, + String[] indices, IndicesOptions indicesOptions, + List datafeeds, + List>clusterSettingsChecks, + List, List, DeprecationIssue>> nodeSettingsChecks, + List> indexSettingsChecks, + List> mlSettingsCheck) { List clusterSettingsIssues = filterChecks(clusterSettingsChecks, (c) -> c.apply(state)); List nodeSettingsIssues = filterChecks(nodeSettingsChecks, (c) -> c.apply(nodesInfo, nodesStats)); + List mlSettingsIssues = new ArrayList<>(); + for (DatafeedConfig config : datafeeds) { + mlSettingsIssues.addAll(filterChecks(mlSettingsCheck, (c) -> c.apply(config))); + } String[] concreteIndexNames = indexNameExpressionResolver.concreteIndexNames(state, indicesOptions, indices); @@ -174,7 +202,7 @@ public static DeprecationInfoAction.Response from(List nodesInfo, List } } - return new DeprecationInfoAction.Response(clusterSettingsIssues, nodeSettingsIssues, indexSettingsIssues); + return new DeprecationInfoAction.Response(clusterSettingsIssues, nodeSettingsIssues, indexSettingsIssues, mlSettingsIssues); } } diff --git a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/ml/datafeed/DatafeedConfig.java b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/ml/datafeed/DatafeedConfig.java index 938452d27cc58..ed858b58dd484 100644 --- a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/ml/datafeed/DatafeedConfig.java +++ b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/ml/datafeed/DatafeedConfig.java @@ -665,7 +665,7 @@ public void setParsedQuery(QueryBuilder query) { } } - void setQuery(Map query) { + public void setQuery(Map query) { this.query = ExceptionsHelper.requireNonNull(query, QUERY.getPreferredName()); } diff --git a/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/deprecation/DeprecationInfoActionResponseTests.java b/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/deprecation/DeprecationInfoActionResponseTests.java index 5267e5dc2ff42..b878f1c5d404d 100644 --- a/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/deprecation/DeprecationInfoActionResponseTests.java +++ b/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/deprecation/DeprecationInfoActionResponseTests.java @@ -22,6 +22,8 @@ import org.elasticsearch.common.xcontent.XContentBuilder; import org.elasticsearch.common.xcontent.XContentFactory; import org.elasticsearch.test.AbstractStreamableTestCase; +import org.elasticsearch.xpack.core.ml.datafeed.DatafeedConfig; +import org.elasticsearch.xpack.core.ml.datafeed.DatafeedConfigTests; import java.io.IOException; import java.util.Arrays; @@ -45,13 +47,15 @@ protected DeprecationInfoAction.Response createTestInstance() { .limit(randomIntBetween(0, 10)).collect(Collectors.toList()); List nodeIssues = Stream.generate(DeprecationIssueTests::createTestInstance) .limit(randomIntBetween(0, 10)).collect(Collectors.toList()); + List mlIssues = Stream.generate(DeprecationIssueTests::createTestInstance) + .limit(randomIntBetween(0, 10)).collect(Collectors.toList()); Map> indexIssues = new HashMap<>(); for (int i = 0; i < randomIntBetween(0, 10); i++) { List perIndexIssues = Stream.generate(DeprecationIssueTests::createTestInstance) .limit(randomIntBetween(0, 10)).collect(Collectors.toList()); indexIssues.put(randomAlphaOfLength(10), perIndexIssues); } - return new DeprecationInfoAction.Response(clusterIssues, nodeIssues, indexIssues); + return new DeprecationInfoAction.Response(clusterIssues, nodeIssues, indexIssues, mlIssues); } @Override @@ -80,12 +84,14 @@ public void testFrom() throws IOException { List nodeStats = Collections.singletonList(new NodeStats(discoveryNode, 0L, null, null, null, null, null, null, null, null, null, null, null, null, null)); + List datafeeds = Collections.singletonList(DatafeedConfigTests.createRandomizedDatafeedConfig("foo")); IndexNameExpressionResolver resolver = new IndexNameExpressionResolver(); IndicesOptions indicesOptions = IndicesOptions.fromOptions(false, false, true, true); boolean clusterIssueFound = randomBoolean(); boolean nodeIssueFound = randomBoolean(); boolean indexIssueFound = randomBoolean(); + boolean mlIssueFound = randomBoolean(); DeprecationIssue foundIssue = DeprecationIssueTests.createTestInstance(); List> clusterSettingsChecks = Collections.unmodifiableList(Arrays.asList( @@ -100,10 +106,14 @@ public void testFrom() throws IOException { Collections.unmodifiableList(Arrays.asList( (idx) -> indexIssueFound ? foundIssue : null )); + List> mlSettingsChecks = + Collections.unmodifiableList(Arrays.asList( + (idx) -> mlIssueFound ? foundIssue : null + )); DeprecationInfoAction.Response response = DeprecationInfoAction.Response.from(nodeInfos, nodeStats, state, - resolver, Strings.EMPTY_ARRAY, indicesOptions, - clusterSettingsChecks, nodeSettingsChecks, indexSettingsChecks); + resolver, Strings.EMPTY_ARRAY, indicesOptions, datafeeds, + clusterSettingsChecks, nodeSettingsChecks, indexSettingsChecks, mlSettingsChecks); if (clusterIssueFound) { assertThat(response.getClusterSettingsIssues(), equalTo(Collections.singletonList(foundIssue))); @@ -123,5 +133,11 @@ public void testFrom() throws IOException { } else { assertTrue(response.getIndexSettingsIssues().isEmpty()); } + + if (mlIssueFound) { + assertThat(response.getMlSettingsIssues(), equalTo(Collections.singletonList(foundIssue))); + } else { + assertTrue(response.getMlSettingsIssues().isEmpty()); + } } } diff --git a/x-pack/plugin/deprecation/src/main/java/org/elasticsearch/xpack/deprecation/DeprecationChecks.java b/x-pack/plugin/deprecation/src/main/java/org/elasticsearch/xpack/deprecation/DeprecationChecks.java index 1363d3a09a03f..c6c3d5fd840c0 100644 --- a/x-pack/plugin/deprecation/src/main/java/org/elasticsearch/xpack/deprecation/DeprecationChecks.java +++ b/x-pack/plugin/deprecation/src/main/java/org/elasticsearch/xpack/deprecation/DeprecationChecks.java @@ -11,6 +11,7 @@ import org.elasticsearch.cluster.metadata.IndexMetaData; import org.elasticsearch.xpack.core.deprecation.DeprecationInfoAction; import org.elasticsearch.xpack.core.deprecation.DeprecationIssue; +import org.elasticsearch.xpack.core.ml.datafeed.DatafeedConfig; import java.util.Arrays; import java.util.Collections; @@ -41,6 +42,12 @@ private DeprecationChecks() { Collections.unmodifiableList(Arrays.asList( IndexDeprecationChecks::oldIndicesCheck)); + static List> ML_SETTINGS_CHECKS = + Collections.unmodifiableList(Arrays.asList( + MlDeprecationChecks::checkDataFeedAggregations, + MlDeprecationChecks::checkDataFeedQuery + )); + /** * helper utility function to reduce repeat of running a specific {@link List} of checks. * diff --git a/x-pack/plugin/deprecation/src/main/java/org/elasticsearch/xpack/deprecation/MlDeprecationChecks.java b/x-pack/plugin/deprecation/src/main/java/org/elasticsearch/xpack/deprecation/MlDeprecationChecks.java new file mode 100644 index 0000000000000..187a8669574cd --- /dev/null +++ b/x-pack/plugin/deprecation/src/main/java/org/elasticsearch/xpack/deprecation/MlDeprecationChecks.java @@ -0,0 +1,44 @@ +/* + * 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.deprecation; + +import org.elasticsearch.xpack.core.deprecation.DeprecationIssue; +import org.elasticsearch.xpack.core.ml.datafeed.DatafeedConfig; + +import java.util.List; + +/** + * Check the {@link DatafeedConfig} query and aggregations for deprecated usages. + */ +final class MlDeprecationChecks { + + private MlDeprecationChecks() { + } + + static DeprecationIssue checkDataFeedQuery(DatafeedConfig datafeedConfig) { + List deprecations = datafeedConfig.getQueryDeprecations(); + if (deprecations.isEmpty()) { + return null; + } else { + return new DeprecationIssue(DeprecationIssue.Level.WARNING, + "Datafeed [" + datafeedConfig.getId() + "] uses deprecated query options", + "https://www.elastic.co/guide/en/elasticsearch/reference/master/breaking-changes-7.0.html#breaking_70_search_changes", + deprecations.toString()); + } + } + + static DeprecationIssue checkDataFeedAggregations(DatafeedConfig datafeedConfig) { + List deprecations = datafeedConfig.getAggDeprecations(); + if (deprecations.isEmpty()) { + return null; + } else { + return new DeprecationIssue(DeprecationIssue.Level.WARNING, + "Datafeed [" + datafeedConfig.getId() + "] uses deprecated aggregation options", + "https://www.elastic.co/guide/en/elasticsearch/reference/master/breaking-changes-7.0.html" + + "#breaking_70_aggregations_changes", deprecations.toString()); + } + } +} diff --git a/x-pack/plugin/deprecation/src/main/java/org/elasticsearch/xpack/deprecation/TransportDeprecationInfoAction.java b/x-pack/plugin/deprecation/src/main/java/org/elasticsearch/xpack/deprecation/TransportDeprecationInfoAction.java index 29bd948ab6c33..6ae416248e9b7 100644 --- a/x-pack/plugin/deprecation/src/main/java/org/elasticsearch/xpack/deprecation/TransportDeprecationInfoAction.java +++ b/x-pack/plugin/deprecation/src/main/java/org/elasticsearch/xpack/deprecation/TransportDeprecationInfoAction.java @@ -19,6 +19,7 @@ import org.elasticsearch.cluster.metadata.IndexNameExpressionResolver; import org.elasticsearch.cluster.service.ClusterService; import org.elasticsearch.common.inject.Inject; +import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.util.concurrent.ThreadContext; import org.elasticsearch.license.LicenseUtils; import org.elasticsearch.license.XPackLicenseState; @@ -26,7 +27,13 @@ import org.elasticsearch.transport.TransportService; import org.elasticsearch.xpack.core.ClientHelper; import org.elasticsearch.xpack.core.XPackField; +import org.elasticsearch.xpack.core.XPackSettings; import org.elasticsearch.xpack.core.deprecation.DeprecationInfoAction; +import org.elasticsearch.xpack.core.ml.action.GetDatafeedsAction; +import org.elasticsearch.xpack.core.ml.datafeed.DatafeedConfig; + +import java.util.Collections; +import java.util.List; public class TransportDeprecationInfoAction extends TransportMasterNodeReadAction { @@ -34,9 +41,10 @@ public class TransportDeprecationInfoAction extends TransportMasterNodeReadActio private final XPackLicenseState licenseState; private final NodeClient client; private final IndexNameExpressionResolver indexNameExpressionResolver; + private final Settings settings; @Inject - public TransportDeprecationInfoAction(TransportService transportService, ClusterService clusterService, + public TransportDeprecationInfoAction(Settings settings, TransportService transportService, ClusterService clusterService, ThreadPool threadPool, ActionFilters actionFilters, IndexNameExpressionResolver indexNameExpressionResolver, XPackLicenseState licenseState, NodeClient client) { @@ -45,6 +53,7 @@ public TransportDeprecationInfoAction(TransportService transportService, Cluster this.licenseState = licenseState; this.client = client; this.indexNameExpressionResolver = indexNameExpressionResolver; + this.settings = settings; } @Override @@ -83,11 +92,20 @@ protected final void masterOperation(final DeprecationInfoAction.Request request if (nodesStatsResponse.hasFailures()) { throw nodesStatsResponse.failures().get(0); } - listener.onResponse(DeprecationInfoAction.Response.from(nodesInfoResponse.getNodes(), - nodesStatsResponse.getNodes(), state, indexNameExpressionResolver, - request.indices(), request.indicesOptions(), - DeprecationChecks.CLUSTER_SETTINGS_CHECKS, DeprecationChecks.NODE_SETTINGS_CHECKS, - DeprecationChecks.INDEX_SETTINGS_CHECKS)); + + getDatafeedConfigs(ActionListener.wrap( + datafeeds -> { + listener.onResponse( + DeprecationInfoAction.Response.from(nodesInfoResponse.getNodes(), + nodesStatsResponse.getNodes(), state, indexNameExpressionResolver, + request.indices(), request.indicesOptions(), datafeeds, + DeprecationChecks.CLUSTER_SETTINGS_CHECKS, + DeprecationChecks.NODE_SETTINGS_CHECKS, + DeprecationChecks.INDEX_SETTINGS_CHECKS, + DeprecationChecks.ML_SETTINGS_CHECKS)); + }, + listener::onFailure + )); }, listener::onFailure), client.admin().cluster()::nodesStats); }, listener::onFailure), client.admin().cluster()::nodesInfo); @@ -95,4 +113,16 @@ protected final void masterOperation(final DeprecationInfoAction.Request request listener.onFailure(LicenseUtils.newComplianceException(XPackField.DEPRECATION)); } } + + private void getDatafeedConfigs(ActionListener> listener) { + if (XPackSettings.MACHINE_LEARNING_ENABLED.get(settings) == false) { + listener.onResponse(Collections.emptyList()); + } else { + ClientHelper.executeAsyncWithOrigin(client, ClientHelper.DEPRECATION_ORIGIN, GetDatafeedsAction.INSTANCE, + new GetDatafeedsAction.Request(GetDatafeedsAction.ALL), ActionListener.wrap( + datafeedsResponse -> listener.onResponse(datafeedsResponse.getResponse().results()), + listener::onFailure + )); + } + } } diff --git a/x-pack/plugin/deprecation/src/test/java/org/elasticsearch/xpack/deprecation/MlDeprecationChecksTests.java b/x-pack/plugin/deprecation/src/test/java/org/elasticsearch/xpack/deprecation/MlDeprecationChecksTests.java new file mode 100644 index 0000000000000..6d93ed1873184 --- /dev/null +++ b/x-pack/plugin/deprecation/src/test/java/org/elasticsearch/xpack/deprecation/MlDeprecationChecksTests.java @@ -0,0 +1,48 @@ +/* + * 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.deprecation; + +import org.elasticsearch.index.query.TermQueryBuilder; +import org.elasticsearch.test.ESTestCase; +import org.elasticsearch.xpack.core.ml.datafeed.DatafeedConfig; + +import java.util.Collections; + +public class MlDeprecationChecksTests extends ESTestCase { + + @Override + protected boolean enableWarningsCheck() { + return false; + } + + public void testCheckDataFeedQuery() { + DatafeedConfig.Builder goodDatafeed = new DatafeedConfig.Builder("good-df", "job-id"); + goodDatafeed.setIndices(Collections.singletonList("some-index")); + goodDatafeed.setParsedQuery(new TermQueryBuilder("foo", "bar")); + assertNull(MlDeprecationChecks.checkDataFeedQuery(goodDatafeed.build())); + + DatafeedConfig.Builder deprecatedDatafeed = new DatafeedConfig.Builder("df-with-deprecated-query", "job-id"); + deprecatedDatafeed.setIndices(Collections.singletonList("some-index")); + // TODO: once some query syntax has been removed from 8.0 and deprecated in 7.x reinstate this test + // to check that particular query syntax causes a deprecation warning + /* + Map qs = new HashMap<>(); + qs.put("query", "foo"); + qs.put("use_dis_max", true); + Map query = Collections.singletonMap("query_string", qs); + deprecatedDatafeed.setQuery(query); + + DeprecationIssue issue = MlDeprecationChecks.checkDataFeedQuery(deprecatedDatafeed.build()); + assertNotNull(issue); + assertThat(issue.getDetails(), equalTo("[Deprecated field [use_dis_max] used, replaced by [Set [tie_breaker] to 1 instead]]")); + assertThat(issue.getLevel(), equalTo(DeprecationIssue.Level.WARNING)); + assertThat(issue.getMessage(), equalTo("Datafeed [df-with-deprecated-query] uses deprecated query options")); + assertThat(issue.getUrl(), equalTo("https://www.elastic.co/guide/en/elasticsearch/reference/master/breaking-changes-8.0.html" + + "#breaking_80_search_changes")); + */ + } +} diff --git a/x-pack/plugin/src/test/resources/rest-api-spec/test/deprecation/10_basic.yml b/x-pack/plugin/src/test/resources/rest-api-spec/test/deprecation/10_basic.yml index dad0c3b08eb57..1cbb310bb4a08 100644 --- a/x-pack/plugin/src/test/resources/rest-api-spec/test/deprecation/10_basic.yml +++ b/x-pack/plugin/src/test/resources/rest-api-spec/test/deprecation/10_basic.yml @@ -12,4 +12,50 @@ setup: - length: { cluster_settings: 0 } - length: { node_settings: 0 } - length: { index_settings: 0 } + - length: { ml_settings: 0 } +--- +"Test ml": + - skip: + version: "7.0.0 - " + features: ["headers", "warnings"] + reason: this test needs adjusting to contain syntax deprecated in 7.x and removed in 8.0 + +# Index the config directly to prevent the deprecated +# use_dis_max field being rewritten by the parser. This +# simulates the config being created in an older version +# of elasticsearch + - do: + headers: + Content-Type: application/json + index: + index: .ml-config + type: doc + id: deprecation-datafeed-datafeed + body: > + { + "datafeed_id" : "deprecation-datafeed", + "config_type" : "datafeed", + "job_id" : "deprecation-job", + "indices" : ["index-foo"], + "query" : { + "query_string" : { + "query" : "foo", + "use_dis_max" : true + } + } + } + + - do: + indices.refresh: + index: [.ml-config] + +# TODO: change the query and expected warnings to one that makes sense for 7.x + - do: + warnings: + - Deprecated field [use_dis_max] used, replaced by [Set [tie_breaker] to 1 instead] + xpack.migration.deprecations: + index: "*" + - length: { ml_settings: 1 } + - match: { ml_settings.0.level : warning } + - match: { ml_settings.0.message : "Datafeed [deprecation-datafeed] uses deprecated query options" } From 945ad05d54c3dabc5ab0a39e6bac6dd287f2b00b Mon Sep 17 00:00:00 2001 From: Michael Basnight Date: Wed, 30 Jan 2019 14:31:16 -0600 Subject: [PATCH 08/16] Update verify repository to allow unknown fields (#37619) The subparser in verify repository allows for unknown fields. This commit sets the value to true for the parser and modifies the test such that it accurately tests it. Relates #36938 --- .../VerifyRepositoryResponseTests.java | 59 +++++++++++++++++++ .../verify/VerifyRepositoryResponse.java | 24 ++++---- 2 files changed, 71 insertions(+), 12 deletions(-) create mode 100644 client/rest-high-level/src/test/java/org/elasticsearch/client/watcher/VerifyRepositoryResponseTests.java diff --git a/client/rest-high-level/src/test/java/org/elasticsearch/client/watcher/VerifyRepositoryResponseTests.java b/client/rest-high-level/src/test/java/org/elasticsearch/client/watcher/VerifyRepositoryResponseTests.java new file mode 100644 index 0000000000000..72193dc55e9e1 --- /dev/null +++ b/client/rest-high-level/src/test/java/org/elasticsearch/client/watcher/VerifyRepositoryResponseTests.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.client.watcher; + +import org.elasticsearch.action.admin.cluster.repositories.verify.VerifyRepositoryResponse; +import org.elasticsearch.common.xcontent.ToXContent; +import org.elasticsearch.common.xcontent.XContentBuilder; +import org.elasticsearch.test.ESTestCase; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; + +import static org.elasticsearch.test.AbstractXContentTestCase.xContentTester; + +public class VerifyRepositoryResponseTests extends ESTestCase { + + public void testFromXContent() throws IOException { + xContentTester(this::createParser, + VerifyRepositoryResponseTests::createTestInstance, + VerifyRepositoryResponseTests::toXContent, + VerifyRepositoryResponse::fromXContent) + .supportsUnknownFields(true) + .shuffleFieldsExceptions(new String[] {"nodes"}) // do not mix up the order of nodes, it will cause the tests to fail + .randomFieldsExcludeFilter((f) -> f.equals("nodes")) // everything in nodes needs to be a particular parseable object + .assertToXContentEquivalence(false) + .test(); + } + + private static VerifyRepositoryResponse createTestInstance() { + List nodes = new ArrayList<>(); + for (int i = 0; i < randomIntBetween(0, 2); i++) { + nodes.add(new VerifyRepositoryResponse.NodeView(randomAlphaOfLength(5), randomAlphaOfLength(5))); + } + + return new VerifyRepositoryResponse(nodes); + } + + private static XContentBuilder toXContent(VerifyRepositoryResponse response, XContentBuilder builder) throws IOException { + return response.toXContent(builder, ToXContent.EMPTY_PARAMS); + } +} diff --git a/server/src/main/java/org/elasticsearch/action/admin/cluster/repositories/verify/VerifyRepositoryResponse.java b/server/src/main/java/org/elasticsearch/action/admin/cluster/repositories/verify/VerifyRepositoryResponse.java index 41835d3e11255..d72136852631f 100644 --- a/server/src/main/java/org/elasticsearch/action/admin/cluster/repositories/verify/VerifyRepositoryResponse.java +++ b/server/src/main/java/org/elasticsearch/action/admin/cluster/repositories/verify/VerifyRepositoryResponse.java @@ -48,7 +48,7 @@ public class VerifyRepositoryResponse extends ActionResponse implements ToXConte public static class NodeView implements Writeable, ToXContentObject { private static final ObjectParser.NamedObjectParser PARSER; static { - ObjectParser internalParser = new ObjectParser<>(NODES); + ObjectParser internalParser = new ObjectParser<>(NODES, true, null); internalParser.declareString(NodeView::setName, new ParseField(NAME)); PARSER = (p, v, name) -> internalParser.parse(p, new NodeView(name), null); } @@ -110,7 +110,7 @@ public int hashCode() { private List nodes; private static final ObjectParser PARSER = - new ObjectParser<>(VerifyRepositoryResponse.class.getName(), VerifyRepositoryResponse::new); + new ObjectParser<>(VerifyRepositoryResponse.class.getName(), true, VerifyRepositoryResponse::new); static { PARSER.declareNamedObjects(VerifyRepositoryResponse::setNodes, NodeView.PARSER, new ParseField("nodes")); } @@ -122,6 +122,10 @@ public VerifyRepositoryResponse(DiscoveryNode[] nodes) { this.nodes = Arrays.stream(nodes).map(dn -> new NodeView(dn.getId(), dn.getName())).collect(Collectors.toList()); } + public VerifyRepositoryResponse(List nodes) { + this.nodes = nodes; + } + @Override public void readFrom(StreamInput in) throws IOException { super.readFrom(in); @@ -168,19 +172,15 @@ public String toString() { } @Override - public boolean equals(Object obj) { - if (obj == null) { - return false; - } - if (getClass() != obj.getClass()) { - return false; - } - VerifyRepositoryResponse other = (VerifyRepositoryResponse) obj; - return nodes.equals(other.nodes); + public boolean equals(Object o) { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + VerifyRepositoryResponse that = (VerifyRepositoryResponse) o; + return Objects.equals(nodes, that.nodes); } @Override public int hashCode() { - return nodes.hashCode(); + return Objects.hash(nodes); } } From 14c571532a1557424e02d05382c758233134e781 Mon Sep 17 00:00:00 2001 From: Michael Basnight Date: Wed, 30 Jan 2019 14:31:55 -0600 Subject: [PATCH 09/16] Fix ILM Lifecycle Policy to allow unknown fields (#38041) A few of the ILM Lifecycle Policy and classes did not allow for unknown fields. This commit sets those parsers and fixes the tests for the parsers. Relates #36938 --- .../client/indexlifecycle/AllocateAction.java | 2 +- .../client/indexlifecycle/DeleteAction.java | 2 +- .../indexlifecycle/ForceMergeAction.java | 2 +- .../client/indexlifecycle/FreezeAction.java | 2 +- .../indexlifecycle/LifecyclePolicy.java | 2 +- .../LifecyclePolicyMetadata.java | 3 ++- .../client/indexlifecycle/Phase.java | 2 +- .../indexlifecycle/PhaseExecutionInfo.java | 2 +- .../client/indexlifecycle/ReadOnlyAction.java | 2 +- .../client/indexlifecycle/RolloverAction.java | 2 +- .../client/indexlifecycle/ShrinkAction.java | 2 +- .../client/indexlifecycle/UnfollowAction.java | 2 +- .../indexlifecycle/AllocateActionTests.java | 10 +++++++++- .../indexlifecycle/DeleteActionTests.java | 2 +- .../indexlifecycle/ForceMergeActionTests.java | 2 +- .../indexlifecycle/FreezeActionTests.java | 2 +- .../GetLifecyclePolicyResponseTests.java | 19 ++++++++++++++++++- .../LifecyclePolicyMetadataTests.java | 17 ++++++++++++++++- .../indexlifecycle/LifecyclePolicyTests.java | 9 ++++++++- .../PhaseExecutionInfoTests.java | 9 ++++++++- .../client/indexlifecycle/PhaseTests.java | 9 ++++++++- .../indexlifecycle/ReadOnlyActionTests.java | 2 +- .../indexlifecycle/RolloverActionTests.java | 2 +- .../indexlifecycle/ShrinkActionTests.java | 2 +- .../indexlifecycle/UnfollowActionTests.java | 2 +- 25 files changed, 87 insertions(+), 25 deletions(-) diff --git a/client/rest-high-level/src/main/java/org/elasticsearch/client/indexlifecycle/AllocateAction.java b/client/rest-high-level/src/main/java/org/elasticsearch/client/indexlifecycle/AllocateAction.java index 702db15b965c7..a297e9b550e4e 100644 --- a/client/rest-high-level/src/main/java/org/elasticsearch/client/indexlifecycle/AllocateAction.java +++ b/client/rest-high-level/src/main/java/org/elasticsearch/client/indexlifecycle/AllocateAction.java @@ -40,7 +40,7 @@ public class AllocateAction implements LifecycleAction, ToXContentObject { static final ParseField REQUIRE_FIELD = new ParseField("require"); @SuppressWarnings("unchecked") - private static final ConstructingObjectParser PARSER = new ConstructingObjectParser<>(NAME, + private static final ConstructingObjectParser PARSER = new ConstructingObjectParser<>(NAME, true, a -> new AllocateAction((Integer) a[0], (Map) a[1], (Map) a[2], (Map) a[3])); static { diff --git a/client/rest-high-level/src/main/java/org/elasticsearch/client/indexlifecycle/DeleteAction.java b/client/rest-high-level/src/main/java/org/elasticsearch/client/indexlifecycle/DeleteAction.java index 299b0ac582771..9592b2edda0e4 100644 --- a/client/rest-high-level/src/main/java/org/elasticsearch/client/indexlifecycle/DeleteAction.java +++ b/client/rest-high-level/src/main/java/org/elasticsearch/client/indexlifecycle/DeleteAction.java @@ -30,7 +30,7 @@ public class DeleteAction implements LifecycleAction, ToXContentObject { public static final String NAME = "delete"; - private static final ObjectParser PARSER = new ObjectParser<>(NAME, DeleteAction::new); + private static final ObjectParser PARSER = new ObjectParser<>(NAME, true, DeleteAction::new); public static DeleteAction parse(XContentParser parser) { return PARSER.apply(parser, null); diff --git a/client/rest-high-level/src/main/java/org/elasticsearch/client/indexlifecycle/ForceMergeAction.java b/client/rest-high-level/src/main/java/org/elasticsearch/client/indexlifecycle/ForceMergeAction.java index eb564b7cd27b6..8b05b16eebafe 100644 --- a/client/rest-high-level/src/main/java/org/elasticsearch/client/indexlifecycle/ForceMergeAction.java +++ b/client/rest-high-level/src/main/java/org/elasticsearch/client/indexlifecycle/ForceMergeAction.java @@ -33,7 +33,7 @@ public class ForceMergeAction implements LifecycleAction, ToXContentObject { private static final ParseField MAX_NUM_SEGMENTS_FIELD = new ParseField("max_num_segments"); private static final ConstructingObjectParser PARSER = new ConstructingObjectParser<>(NAME, - false, a -> { + true, a -> { int maxNumSegments = (int) a[0]; return new ForceMergeAction(maxNumSegments); }); diff --git a/client/rest-high-level/src/main/java/org/elasticsearch/client/indexlifecycle/FreezeAction.java b/client/rest-high-level/src/main/java/org/elasticsearch/client/indexlifecycle/FreezeAction.java index ecc054c132d67..3e5952539bb89 100644 --- a/client/rest-high-level/src/main/java/org/elasticsearch/client/indexlifecycle/FreezeAction.java +++ b/client/rest-high-level/src/main/java/org/elasticsearch/client/indexlifecycle/FreezeAction.java @@ -29,7 +29,7 @@ public class FreezeAction implements LifecycleAction, ToXContentObject { public static final String NAME = "freeze"; - private static final ObjectParser PARSER = new ObjectParser<>(NAME, FreezeAction::new); + private static final ObjectParser PARSER = new ObjectParser<>(NAME, true, FreezeAction::new); public static FreezeAction parse(XContentParser parser) { return PARSER.apply(parser, null); diff --git a/client/rest-high-level/src/main/java/org/elasticsearch/client/indexlifecycle/LifecyclePolicy.java b/client/rest-high-level/src/main/java/org/elasticsearch/client/indexlifecycle/LifecyclePolicy.java index 5e4ae1f36bcbc..9032afd7ba417 100644 --- a/client/rest-high-level/src/main/java/org/elasticsearch/client/indexlifecycle/LifecyclePolicy.java +++ b/client/rest-high-level/src/main/java/org/elasticsearch/client/indexlifecycle/LifecyclePolicy.java @@ -44,7 +44,7 @@ public class LifecyclePolicy implements ToXContentObject { static final ParseField PHASES_FIELD = new ParseField("phases"); @SuppressWarnings("unchecked") - public static ConstructingObjectParser PARSER = new ConstructingObjectParser<>("lifecycle_policy", false, + public static ConstructingObjectParser PARSER = new ConstructingObjectParser<>("lifecycle_policy", true, (a, name) -> { List phases = (List) a[0]; Map phaseMap = phases.stream().collect(Collectors.toMap(Phase::getName, Function.identity())); diff --git a/client/rest-high-level/src/main/java/org/elasticsearch/client/indexlifecycle/LifecyclePolicyMetadata.java b/client/rest-high-level/src/main/java/org/elasticsearch/client/indexlifecycle/LifecyclePolicyMetadata.java index 84de81437065d..b58594e5756c9 100644 --- a/client/rest-high-level/src/main/java/org/elasticsearch/client/indexlifecycle/LifecyclePolicyMetadata.java +++ b/client/rest-high-level/src/main/java/org/elasticsearch/client/indexlifecycle/LifecyclePolicyMetadata.java @@ -38,7 +38,8 @@ public class LifecyclePolicyMetadata implements ToXContentObject { static final ParseField MODIFIED_DATE = new ParseField("modified_date"); @SuppressWarnings("unchecked") - public static final ConstructingObjectParser PARSER = new ConstructingObjectParser<>("policy_metadata", + public static final ConstructingObjectParser PARSER = new ConstructingObjectParser<>( + "policy_metadata", true, a -> { LifecyclePolicy policy = (LifecyclePolicy) a[0]; return new LifecyclePolicyMetadata(policy, (long) a[1], ZonedDateTime.parse((String) a[2]).toInstant().toEpochMilli()); diff --git a/client/rest-high-level/src/main/java/org/elasticsearch/client/indexlifecycle/Phase.java b/client/rest-high-level/src/main/java/org/elasticsearch/client/indexlifecycle/Phase.java index 0c19d39c85964..f6d3e80644764 100644 --- a/client/rest-high-level/src/main/java/org/elasticsearch/client/indexlifecycle/Phase.java +++ b/client/rest-high-level/src/main/java/org/elasticsearch/client/indexlifecycle/Phase.java @@ -44,7 +44,7 @@ public class Phase implements ToXContentObject { static final ParseField ACTIONS_FIELD = new ParseField("actions"); @SuppressWarnings("unchecked") - private static final ConstructingObjectParser PARSER = new ConstructingObjectParser<>("phase", false, + private static final ConstructingObjectParser PARSER = new ConstructingObjectParser<>("phase", true, (a, name) -> new Phase(name, (TimeValue) a[0], ((List) a[1]).stream() .collect(Collectors.toMap(LifecycleAction::getName, Function.identity())))); static { diff --git a/client/rest-high-level/src/main/java/org/elasticsearch/client/indexlifecycle/PhaseExecutionInfo.java b/client/rest-high-level/src/main/java/org/elasticsearch/client/indexlifecycle/PhaseExecutionInfo.java index 802ca8834cdd3..681f79c67829c 100644 --- a/client/rest-high-level/src/main/java/org/elasticsearch/client/indexlifecycle/PhaseExecutionInfo.java +++ b/client/rest-high-level/src/main/java/org/elasticsearch/client/indexlifecycle/PhaseExecutionInfo.java @@ -40,7 +40,7 @@ public class PhaseExecutionInfo implements ToXContentObject { private static final ParseField MODIFIED_DATE_IN_MILLIS_FIELD = new ParseField("modified_date_in_millis"); private static final ConstructingObjectParser PARSER = new ConstructingObjectParser<>( - "phase_execution_info", false, + "phase_execution_info", true, (a, name) -> new PhaseExecutionInfo((String) a[0], (Phase) a[1], (long) a[2], (long) a[3])); static { PARSER.declareString(ConstructingObjectParser.constructorArg(), POLICY_NAME_FIELD); diff --git a/client/rest-high-level/src/main/java/org/elasticsearch/client/indexlifecycle/ReadOnlyAction.java b/client/rest-high-level/src/main/java/org/elasticsearch/client/indexlifecycle/ReadOnlyAction.java index 7734e792bbc5b..cf364af6d9e5b 100644 --- a/client/rest-high-level/src/main/java/org/elasticsearch/client/indexlifecycle/ReadOnlyAction.java +++ b/client/rest-high-level/src/main/java/org/elasticsearch/client/indexlifecycle/ReadOnlyAction.java @@ -29,7 +29,7 @@ public class ReadOnlyAction implements LifecycleAction, ToXContentObject { public static final String NAME = "readonly"; - private static final ObjectParser PARSER = new ObjectParser<>(NAME, false, ReadOnlyAction::new); + private static final ObjectParser PARSER = new ObjectParser<>(NAME, true, ReadOnlyAction::new); public static ReadOnlyAction parse(XContentParser parser) { return PARSER.apply(parser, null); diff --git a/client/rest-high-level/src/main/java/org/elasticsearch/client/indexlifecycle/RolloverAction.java b/client/rest-high-level/src/main/java/org/elasticsearch/client/indexlifecycle/RolloverAction.java index 0cc9dcf234969..e84cc6921440a 100644 --- a/client/rest-high-level/src/main/java/org/elasticsearch/client/indexlifecycle/RolloverAction.java +++ b/client/rest-high-level/src/main/java/org/elasticsearch/client/indexlifecycle/RolloverAction.java @@ -38,7 +38,7 @@ public class RolloverAction implements LifecycleAction, ToXContentObject { private static final ParseField MAX_DOCS_FIELD = new ParseField("max_docs"); private static final ParseField MAX_AGE_FIELD = new ParseField("max_age"); - private static final ConstructingObjectParser PARSER = new ConstructingObjectParser<>(NAME, + private static final ConstructingObjectParser PARSER = new ConstructingObjectParser<>(NAME, true, a -> new RolloverAction((ByteSizeValue) a[0], (TimeValue) a[1], (Long) a[2])); static { PARSER.declareField(ConstructingObjectParser.optionalConstructorArg(), diff --git a/client/rest-high-level/src/main/java/org/elasticsearch/client/indexlifecycle/ShrinkAction.java b/client/rest-high-level/src/main/java/org/elasticsearch/client/indexlifecycle/ShrinkAction.java index 345356380145e..920fe4a46257f 100644 --- a/client/rest-high-level/src/main/java/org/elasticsearch/client/indexlifecycle/ShrinkAction.java +++ b/client/rest-high-level/src/main/java/org/elasticsearch/client/indexlifecycle/ShrinkAction.java @@ -33,7 +33,7 @@ public class ShrinkAction implements LifecycleAction, ToXContentObject { private static final ParseField NUMBER_OF_SHARDS_FIELD = new ParseField("number_of_shards"); private static final ConstructingObjectParser PARSER = - new ConstructingObjectParser<>(NAME, a -> new ShrinkAction((Integer) a[0])); + new ConstructingObjectParser<>(NAME, true, a -> new ShrinkAction((Integer) a[0])); static { PARSER.declareInt(ConstructingObjectParser.constructorArg(), NUMBER_OF_SHARDS_FIELD); diff --git a/client/rest-high-level/src/main/java/org/elasticsearch/client/indexlifecycle/UnfollowAction.java b/client/rest-high-level/src/main/java/org/elasticsearch/client/indexlifecycle/UnfollowAction.java index ba25cf937ec8f..11e9836efc02d 100644 --- a/client/rest-high-level/src/main/java/org/elasticsearch/client/indexlifecycle/UnfollowAction.java +++ b/client/rest-high-level/src/main/java/org/elasticsearch/client/indexlifecycle/UnfollowAction.java @@ -31,7 +31,7 @@ public class UnfollowAction implements LifecycleAction, ToXContentObject { public static final String NAME = "unfollow"; - private static final ObjectParser PARSER = new ObjectParser<>(NAME, UnfollowAction::new); + private static final ObjectParser PARSER = new ObjectParser<>(NAME, true, UnfollowAction::new); public UnfollowAction() {} diff --git a/client/rest-high-level/src/test/java/org/elasticsearch/client/indexlifecycle/AllocateActionTests.java b/client/rest-high-level/src/test/java/org/elasticsearch/client/indexlifecycle/AllocateActionTests.java index e44eb0da0e188..1cbfdf4f369cc 100644 --- a/client/rest-high-level/src/test/java/org/elasticsearch/client/indexlifecycle/AllocateActionTests.java +++ b/client/rest-high-level/src/test/java/org/elasticsearch/client/indexlifecycle/AllocateActionTests.java @@ -24,6 +24,7 @@ import java.util.Collections; import java.util.HashMap; import java.util.Map; +import java.util.function.Predicate; public class AllocateActionTests extends AbstractXContentTestCase { @@ -65,7 +66,14 @@ protected AllocateAction doParseInstance(XContentParser parser) { @Override protected boolean supportsUnknownFields() { - return false; + return true; + } + + @Override + protected Predicate getRandomFieldsExcludeFilter() { + // this whole structure expects to be maps of strings, so more complex objects would just mess that up. + // setting it this way allows for new fields at the root + return (field) -> field.isEmpty() == false; } public void testAllMapsNullOrEmpty() { diff --git a/client/rest-high-level/src/test/java/org/elasticsearch/client/indexlifecycle/DeleteActionTests.java b/client/rest-high-level/src/test/java/org/elasticsearch/client/indexlifecycle/DeleteActionTests.java index fb7deb97a2787..9f4dcf6bc641a 100644 --- a/client/rest-high-level/src/test/java/org/elasticsearch/client/indexlifecycle/DeleteActionTests.java +++ b/client/rest-high-level/src/test/java/org/elasticsearch/client/indexlifecycle/DeleteActionTests.java @@ -35,6 +35,6 @@ protected DeleteAction doParseInstance(XContentParser parser) { @Override protected boolean supportsUnknownFields() { - return false; + return true; } } diff --git a/client/rest-high-level/src/test/java/org/elasticsearch/client/indexlifecycle/ForceMergeActionTests.java b/client/rest-high-level/src/test/java/org/elasticsearch/client/indexlifecycle/ForceMergeActionTests.java index 16fafcfa24015..7c3181a61dfd3 100644 --- a/client/rest-high-level/src/test/java/org/elasticsearch/client/indexlifecycle/ForceMergeActionTests.java +++ b/client/rest-high-level/src/test/java/org/elasticsearch/client/indexlifecycle/ForceMergeActionTests.java @@ -39,7 +39,7 @@ protected ForceMergeAction doParseInstance(XContentParser parser) { @Override protected boolean supportsUnknownFields() { - return false; + return true; } @Override diff --git a/client/rest-high-level/src/test/java/org/elasticsearch/client/indexlifecycle/FreezeActionTests.java b/client/rest-high-level/src/test/java/org/elasticsearch/client/indexlifecycle/FreezeActionTests.java index 3fc40ee137b53..1a92db1cdf1f5 100644 --- a/client/rest-high-level/src/test/java/org/elasticsearch/client/indexlifecycle/FreezeActionTests.java +++ b/client/rest-high-level/src/test/java/org/elasticsearch/client/indexlifecycle/FreezeActionTests.java @@ -35,6 +35,6 @@ protected FreezeAction doParseInstance(XContentParser parser) { @Override protected boolean supportsUnknownFields() { - return false; + return true; } } diff --git a/client/rest-high-level/src/test/java/org/elasticsearch/client/indexlifecycle/GetLifecyclePolicyResponseTests.java b/client/rest-high-level/src/test/java/org/elasticsearch/client/indexlifecycle/GetLifecyclePolicyResponseTests.java index c16c270512ca6..ff1733498e37d 100644 --- a/client/rest-high-level/src/test/java/org/elasticsearch/client/indexlifecycle/GetLifecyclePolicyResponseTests.java +++ b/client/rest-high-level/src/test/java/org/elasticsearch/client/indexlifecycle/GetLifecyclePolicyResponseTests.java @@ -30,6 +30,7 @@ import java.util.ArrayList; import java.util.Arrays; import java.util.List; +import java.util.function.Predicate; import static org.elasticsearch.client.indexlifecycle.LifecyclePolicyTests.createRandomPolicy; @@ -54,7 +55,23 @@ protected GetLifecyclePolicyResponse doParseInstance(XContentParser parser) thro @Override protected boolean supportsUnknownFields() { - return false; + return true; + } + + @Override + protected Predicate getRandomFieldsExcludeFilter() { + return (field) -> + // phases is a list of Phase parsable entries only + field.endsWith(".phases") + // these are all meant to be maps of strings, so complex objects will confuse the parser + || field.endsWith(".include") + || field.endsWith(".exclude") + || field.endsWith(".require") + // actions are meant to be a list of LifecycleAction parsable entries only + || field.endsWith(".actions") + // field.isEmpty() means do not insert an object at the root of the json. This parser expects + // every root level named object to be parsable as a specific type + || field.isEmpty(); } @Override diff --git a/client/rest-high-level/src/test/java/org/elasticsearch/client/indexlifecycle/LifecyclePolicyMetadataTests.java b/client/rest-high-level/src/test/java/org/elasticsearch/client/indexlifecycle/LifecyclePolicyMetadataTests.java index 6d8014c432c28..eba3113f1777d 100644 --- a/client/rest-high-level/src/test/java/org/elasticsearch/client/indexlifecycle/LifecyclePolicyMetadataTests.java +++ b/client/rest-high-level/src/test/java/org/elasticsearch/client/indexlifecycle/LifecyclePolicyMetadataTests.java @@ -29,6 +29,7 @@ import java.util.ArrayList; import java.util.Arrays; import java.util.List; +import java.util.function.Predicate; import static org.elasticsearch.client.indexlifecycle.LifecyclePolicyTests.createRandomPolicy; @@ -50,7 +51,21 @@ protected LifecyclePolicyMetadata doParseInstance(XContentParser parser) throws @Override protected boolean supportsUnknownFields() { - return false; + return true; + } + + @Override + protected Predicate getRandomFieldsExcludeFilter() { + return (field) -> + // phases is a list of Phase parsable entries only + field.endsWith(".phases") + // these are all meant to be maps of strings, so complex objects will confuse the parser + || field.endsWith(".include") + || field.endsWith(".exclude") + || field.endsWith(".require") + // actions are meant to be a list of LifecycleAction parsable entries only + || field.endsWith(".actions"); + } @Override diff --git a/client/rest-high-level/src/test/java/org/elasticsearch/client/indexlifecycle/LifecyclePolicyTests.java b/client/rest-high-level/src/test/java/org/elasticsearch/client/indexlifecycle/LifecyclePolicyTests.java index 1690f66572142..6451b4bbbef59 100644 --- a/client/rest-high-level/src/test/java/org/elasticsearch/client/indexlifecycle/LifecyclePolicyTests.java +++ b/client/rest-high-level/src/test/java/org/elasticsearch/client/indexlifecycle/LifecyclePolicyTests.java @@ -34,6 +34,7 @@ import java.util.Map; import java.util.Set; import java.util.function.Function; +import java.util.function.Predicate; import java.util.stream.Collectors; import static org.hamcrest.Matchers.equalTo; @@ -55,7 +56,13 @@ protected LifecyclePolicy doParseInstance(XContentParser parser) { @Override protected boolean supportsUnknownFields() { - return false; + return true; + } + + @Override + protected Predicate getRandomFieldsExcludeFilter() { + // these items all have some specific parsing that does not allow them to have additional objects within them. + return (field) -> field.contains("allocate.") || field.equals("phases") || field.endsWith("actions"); } @Override diff --git a/client/rest-high-level/src/test/java/org/elasticsearch/client/indexlifecycle/PhaseExecutionInfoTests.java b/client/rest-high-level/src/test/java/org/elasticsearch/client/indexlifecycle/PhaseExecutionInfoTests.java index 0db9b56aea93c..fea740a442c95 100644 --- a/client/rest-high-level/src/test/java/org/elasticsearch/client/indexlifecycle/PhaseExecutionInfoTests.java +++ b/client/rest-high-level/src/test/java/org/elasticsearch/client/indexlifecycle/PhaseExecutionInfoTests.java @@ -28,6 +28,7 @@ import java.io.IOException; import java.util.ArrayList; import java.util.List; +import java.util.function.Predicate; public class PhaseExecutionInfoTests extends AbstractXContentTestCase { @@ -53,9 +54,15 @@ protected PhaseExecutionInfo doParseInstance(XContentParser parser) throws IOExc return PhaseExecutionInfo.parse(parser, phaseName); } + @Override + protected Predicate getRandomFieldsExcludeFilter() { + // actions are plucked from the named registry, and it fails if the action is not in the named registry + return (field) -> field.equals("phase_definition.actions"); + } + @Override protected boolean supportsUnknownFields() { - return false; + return true; } @Override diff --git a/client/rest-high-level/src/test/java/org/elasticsearch/client/indexlifecycle/PhaseTests.java b/client/rest-high-level/src/test/java/org/elasticsearch/client/indexlifecycle/PhaseTests.java index 3b4fc2fec6059..df4f11d18d028 100644 --- a/client/rest-high-level/src/test/java/org/elasticsearch/client/indexlifecycle/PhaseTests.java +++ b/client/rest-high-level/src/test/java/org/elasticsearch/client/indexlifecycle/PhaseTests.java @@ -30,6 +30,7 @@ import java.util.Collections; import java.util.List; import java.util.Map; +import java.util.function.Predicate; public class PhaseTests extends AbstractXContentTestCase { private String phaseName; @@ -61,6 +62,12 @@ protected Phase doParseInstance(XContentParser parser) { return Phase.parse(parser, phaseName); } + @Override + protected Predicate getRandomFieldsExcludeFilter() { + // actions are plucked from the named registry, and it fails if the action is not in the named registry + return (field) -> field.equals("actions"); + } + @Override protected NamedXContentRegistry xContentRegistry() { List entries = new ArrayList<>(ClusterModule.getNamedXWriteables()); @@ -70,7 +77,7 @@ protected NamedXContentRegistry xContentRegistry() { @Override protected boolean supportsUnknownFields() { - return false; + return true; } public void testDefaultAfter() { diff --git a/client/rest-high-level/src/test/java/org/elasticsearch/client/indexlifecycle/ReadOnlyActionTests.java b/client/rest-high-level/src/test/java/org/elasticsearch/client/indexlifecycle/ReadOnlyActionTests.java index bf57478425cc9..dd6f62a447b39 100644 --- a/client/rest-high-level/src/test/java/org/elasticsearch/client/indexlifecycle/ReadOnlyActionTests.java +++ b/client/rest-high-level/src/test/java/org/elasticsearch/client/indexlifecycle/ReadOnlyActionTests.java @@ -30,7 +30,7 @@ protected ReadOnlyAction doParseInstance(XContentParser parser) { @Override protected boolean supportsUnknownFields() { - return false; + return true; } @Override diff --git a/client/rest-high-level/src/test/java/org/elasticsearch/client/indexlifecycle/RolloverActionTests.java b/client/rest-high-level/src/test/java/org/elasticsearch/client/indexlifecycle/RolloverActionTests.java index bbbdba37e5640..833321d702c63 100644 --- a/client/rest-high-level/src/test/java/org/elasticsearch/client/indexlifecycle/RolloverActionTests.java +++ b/client/rest-high-level/src/test/java/org/elasticsearch/client/indexlifecycle/RolloverActionTests.java @@ -33,7 +33,7 @@ protected RolloverAction doParseInstance(XContentParser parser) { @Override protected boolean supportsUnknownFields() { - return false; + return true; } @Override diff --git a/client/rest-high-level/src/test/java/org/elasticsearch/client/indexlifecycle/ShrinkActionTests.java b/client/rest-high-level/src/test/java/org/elasticsearch/client/indexlifecycle/ShrinkActionTests.java index adeec1ff825a9..d796221518fe1 100644 --- a/client/rest-high-level/src/test/java/org/elasticsearch/client/indexlifecycle/ShrinkActionTests.java +++ b/client/rest-high-level/src/test/java/org/elasticsearch/client/indexlifecycle/ShrinkActionTests.java @@ -43,7 +43,7 @@ static ShrinkAction randomInstance() { @Override protected boolean supportsUnknownFields() { - return false; + return true; } public void testNonPositiveShardNumber() { diff --git a/client/rest-high-level/src/test/java/org/elasticsearch/client/indexlifecycle/UnfollowActionTests.java b/client/rest-high-level/src/test/java/org/elasticsearch/client/indexlifecycle/UnfollowActionTests.java index 4dd73c5a08ec2..715a692213232 100644 --- a/client/rest-high-level/src/test/java/org/elasticsearch/client/indexlifecycle/UnfollowActionTests.java +++ b/client/rest-high-level/src/test/java/org/elasticsearch/client/indexlifecycle/UnfollowActionTests.java @@ -38,6 +38,6 @@ protected UnfollowAction doParseInstance(XContentParser parser) throws IOExcepti @Override protected boolean supportsUnknownFields() { - return false; + return true; } } From daafcb66256aa209673b3c5b7bc57af8ed51a3b5 Mon Sep 17 00:00:00 2001 From: Michael Basnight Date: Wed, 30 Jan 2019 14:32:17 -0600 Subject: [PATCH 10/16] Fix ILM status to allow unknown fields (#38043) The ILM status parser did not allow for unknown fields. This commit fixes that and adds an xContentTester to the response test. Relates #36938 --- .../LifecycleManagementStatusResponse.java | 2 +- ...ifecycleManagementStatusResponseTests.java | 24 +++++++++++++++++++ 2 files changed, 25 insertions(+), 1 deletion(-) diff --git a/client/rest-high-level/src/main/java/org/elasticsearch/client/indexlifecycle/LifecycleManagementStatusResponse.java b/client/rest-high-level/src/main/java/org/elasticsearch/client/indexlifecycle/LifecycleManagementStatusResponse.java index c1586d7e1c738..d084113853f32 100644 --- a/client/rest-high-level/src/main/java/org/elasticsearch/client/indexlifecycle/LifecycleManagementStatusResponse.java +++ b/client/rest-high-level/src/main/java/org/elasticsearch/client/indexlifecycle/LifecycleManagementStatusResponse.java @@ -34,7 +34,7 @@ public class LifecycleManagementStatusResponse { private static final String OPERATION_MODE = "operation_mode"; @SuppressWarnings("unchecked") private static final ConstructingObjectParser PARSER = new ConstructingObjectParser<>( - OPERATION_MODE, a -> new LifecycleManagementStatusResponse((String) a[0])); + OPERATION_MODE, true, a -> new LifecycleManagementStatusResponse((String) a[0])); static { PARSER.declareString(ConstructingObjectParser.constructorArg(), new ParseField(OPERATION_MODE)); diff --git a/client/rest-high-level/src/test/java/org/elasticsearch/client/indexlifecycle/LifecycleManagementStatusResponseTests.java b/client/rest-high-level/src/test/java/org/elasticsearch/client/indexlifecycle/LifecycleManagementStatusResponseTests.java index 144039b8995c6..d027454453aca 100644 --- a/client/rest-high-level/src/test/java/org/elasticsearch/client/indexlifecycle/LifecycleManagementStatusResponseTests.java +++ b/client/rest-high-level/src/test/java/org/elasticsearch/client/indexlifecycle/LifecycleManagementStatusResponseTests.java @@ -21,6 +21,7 @@ import org.elasticsearch.common.xcontent.DeprecationHandler; import org.elasticsearch.common.xcontent.NamedXContentRegistry; +import org.elasticsearch.common.xcontent.XContentBuilder; import org.elasticsearch.common.xcontent.XContentParser; import org.elasticsearch.common.xcontent.XContentType; import org.elasticsearch.test.ESTestCase; @@ -30,8 +31,31 @@ import java.util.EnumSet; import java.util.stream.Collectors; +import static org.elasticsearch.test.AbstractXContentTestCase.xContentTester; + public class LifecycleManagementStatusResponseTests extends ESTestCase { + public void testFromXContent() throws IOException { + xContentTester(this::createParser, + LifecycleManagementStatusResponseTests::createTestInstance, + LifecycleManagementStatusResponseTests::toXContent, + LifecycleManagementStatusResponse::fromXContent) + .supportsUnknownFields(true) + .assertToXContentEquivalence(false) + .test(); + } + + private static XContentBuilder toXContent(LifecycleManagementStatusResponse response, XContentBuilder builder) throws IOException { + builder.startObject(); + builder.field("operation_mode", response.getOperationMode()); + builder.endObject(); + return builder; + } + + private static LifecycleManagementStatusResponse createTestInstance() { + return new LifecycleManagementStatusResponse(randomFrom(OperationMode.values()).name()); + } + public void testAllValidStatuses() { EnumSet.allOf(OperationMode.class) .forEach(e -> assertEquals(new LifecycleManagementStatusResponse(e.name()).getOperationMode(), e)); From aeab55e8d125efdfe66e4c9ad784c9d8c7fe6736 Mon Sep 17 00:00:00 2001 From: Tim Brooks Date: Wed, 30 Jan 2019 14:13:23 -0700 Subject: [PATCH 11/16] Reduce flaxiness of ccr recovery timeouts test (#38035) This fixes #38027. Currently we assert that all shards have failed. However, it is possible that some shards do not have segement files created yet. The action that we block is fetching these segement files so it is possible that some shards successfully recover. This commit changes the assertion to ensure that at least some of the shards have failed. --- .../java/org/elasticsearch/xpack/ccr/CcrRepositoryIT.java | 7 +++++-- 1 file changed, 5 insertions(+), 2 deletions(-) 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 f22857939e0d1..fdf2de6d6775f 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 @@ -55,7 +55,9 @@ import static org.elasticsearch.snapshots.RestoreService.restoreInProgress; import static org.elasticsearch.test.hamcrest.ElasticsearchAssertions.assertAcked; import static org.hamcrest.Matchers.equalTo; +import static org.hamcrest.Matchers.greaterThan; import static org.hamcrest.Matchers.instanceOf; +import static org.hamcrest.Matchers.lessThan; // TODO: Fold this integration test into a more expansive integration test as more bootstrap from remote work // TODO: is completed. @@ -367,8 +369,9 @@ public void testIndividualActionsTimeout() throws Exception { // be marked as failed. Either one is a success for the purpose of this test. try { RestoreInfo restoreInfo = future.actionGet(); - assertEquals(0, restoreInfo.successfulShards()); - assertEquals(numberOfPrimaryShards, restoreInfo.failedShards()); + assertThat(restoreInfo.failedShards(), greaterThan(0)); + assertThat(restoreInfo.successfulShards(), lessThan(restoreInfo.totalShards())); + assertEquals(numberOfPrimaryShards, restoreInfo.totalShards()); } catch (Exception e) { assertThat(ExceptionsHelper.unwrapCause(e), instanceOf(ElasticsearchTimeoutException.class)); } From 54dbf9469cb31b9b832f3e02243329a8dd0235b6 Mon Sep 17 00:00:00 2001 From: Jay Modi Date: Wed, 30 Jan 2019 14:24:29 -0700 Subject: [PATCH 12/16] Update httpclient for JDK 11 TLS engine (#37994) The apache commons http client implementations recently released versions that solve TLS compatibility issues with the new TLS engine that supports TLSv1.3 with JDK 11. This change updates our code to use these versions since JDK 11 is a supported JDK and we should allow the use of TLSv1.3. --- buildSrc/version.properties | 11 +++---- .../rest/licenses/commons-codec-1.10.jar.sha1 | 1 - .../rest/licenses/commons-codec-1.11.jar.sha1 | 1 + .../licenses/httpasyncclient-4.1.2.jar.sha1 | 1 - .../licenses/httpasyncclient-4.1.4.jar.sha1 | 1 + .../rest/licenses/httpclient-4.5.2.jar.sha1 | 1 - .../rest/licenses/httpclient-4.5.7.jar.sha1 | 1 + client/rest/licenses/httpcore-4.4.11.jar.sha1 | 1 + client/rest/licenses/httpcore-4.4.5.jar.sha1 | 1 - .../licenses/httpcore-nio-4.4.11.jar.sha1 | 1 + .../rest/licenses/httpcore-nio-4.4.5.jar.sha1 | 1 - .../licenses/commons-codec-1.10.jar.sha1 | 1 - .../licenses/commons-codec-1.11.jar.sha1 | 1 + .../licenses/httpclient-4.5.2.jar.sha1 | 1 - .../licenses/httpclient-4.5.7.jar.sha1 | 1 + .../sniffer/licenses/httpcore-4.4.11.jar.sha1 | 1 + .../sniffer/licenses/httpcore-4.4.5.jar.sha1 | 1 - .../licenses/commons-codec-1.10.jar.sha1 | 1 - .../licenses/commons-codec-1.11.jar.sha1 | 1 + .../licenses/commons-codec-1.10.jar.sha1 | 1 - .../licenses/commons-codec-1.11.jar.sha1 | 1 + .../licenses/httpclient-4.5.2.jar.sha1 | 1 - .../licenses/httpclient-4.5.7.jar.sha1 | 1 + .../licenses/httpcore-4.4.11.jar.sha1 | 1 + .../licenses/httpcore-4.4.5.jar.sha1 | 1 - .../licenses/commons-codec-1.10.jar.sha1 | 1 - .../licenses/commons-codec-1.11.jar.sha1 | 1 + .../licenses/httpclient-4.5.2.jar.sha1 | 1 - .../licenses/httpclient-4.5.7.jar.sha1 | 1 + .../licenses/httpcore-4.4.11.jar.sha1 | 1 + .../licenses/httpcore-4.4.5.jar.sha1 | 1 - .../licenses/commons-codec-1.10.jar.sha1 | 1 - .../licenses/commons-codec-1.11.jar.sha1 | 1 + .../licenses/httpclient-4.5.2.jar.sha1 | 1 - .../licenses/httpclient-4.5.7.jar.sha1 | 1 + .../licenses/httpcore-4.4.11.jar.sha1 | 1 + .../licenses/httpcore-4.4.5.jar.sha1 | 1 - .../licenses/commons-codec-1.10.jar.sha1 | 1 - .../licenses/commons-codec-1.11.jar.sha1 | 1 + .../licenses/commons-codec-1.10.jar.sha1 | 1 - .../licenses/commons-codec-1.11.jar.sha1 | 1 + .../licenses/httpclient-4.5.2.jar.sha1 | 1 - .../licenses/httpclient-4.5.7.jar.sha1 | 1 + .../licenses/httpcore-4.4.11.jar.sha1 | 1 + .../licenses/httpcore-4.4.5.jar.sha1 | 1 - plugins/repository-hdfs/build.gradle | 2 +- .../licenses/commons-codec-1.10.jar.sha1 | 1 - .../licenses/commons-codec-1.11.jar.sha1 | 1 + .../licenses/commons-codec-1.10.jar.sha1 | 1 - .../licenses/commons-codec-1.11.jar.sha1 | 1 + .../licenses/httpclient-4.5.2.jar.sha1 | 1 - .../licenses/httpclient-4.5.7.jar.sha1 | 1 + .../licenses/httpcore-4.4.11.jar.sha1 | 1 + .../licenses/httpcore-4.4.5.jar.sha1 | 1 - .../core/licenses/commons-codec-1.10.jar.sha1 | 1 - .../core/licenses/commons-codec-1.11.jar.sha1 | 1 + .../licenses/httpasyncclient-4.1.2.jar.sha1 | 1 - .../licenses/httpasyncclient-4.1.4.jar.sha1 | 1 + .../core/licenses/httpclient-4.5.2.jar.sha1 | 1 - .../core/licenses/httpclient-4.5.7.jar.sha1 | 1 + .../core/licenses/httpcore-4.4.11.jar.sha1 | 1 + .../core/licenses/httpcore-4.4.5.jar.sha1 | 1 - .../licenses/httpcore-nio-4.4.11.jar.sha1 | 1 + .../core/licenses/httpcore-nio-4.4.5.jar.sha1 | 1 - .../licenses/httpclient-cache-4.5.2.jar.sha1 | 1 - .../licenses/httpclient-cache-4.5.7.jar.sha1 | 1 + .../xpack/watcher/common/http/HttpClient.java | 29 +++++++++++++++---- 67 files changed, 61 insertions(+), 45 deletions(-) delete mode 100644 client/rest/licenses/commons-codec-1.10.jar.sha1 create mode 100644 client/rest/licenses/commons-codec-1.11.jar.sha1 delete mode 100644 client/rest/licenses/httpasyncclient-4.1.2.jar.sha1 create mode 100644 client/rest/licenses/httpasyncclient-4.1.4.jar.sha1 delete mode 100644 client/rest/licenses/httpclient-4.5.2.jar.sha1 create mode 100644 client/rest/licenses/httpclient-4.5.7.jar.sha1 create mode 100644 client/rest/licenses/httpcore-4.4.11.jar.sha1 delete mode 100644 client/rest/licenses/httpcore-4.4.5.jar.sha1 create mode 100644 client/rest/licenses/httpcore-nio-4.4.11.jar.sha1 delete mode 100644 client/rest/licenses/httpcore-nio-4.4.5.jar.sha1 delete mode 100644 client/sniffer/licenses/commons-codec-1.10.jar.sha1 create mode 100644 client/sniffer/licenses/commons-codec-1.11.jar.sha1 delete mode 100644 client/sniffer/licenses/httpclient-4.5.2.jar.sha1 create mode 100644 client/sniffer/licenses/httpclient-4.5.7.jar.sha1 create mode 100644 client/sniffer/licenses/httpcore-4.4.11.jar.sha1 delete mode 100644 client/sniffer/licenses/httpcore-4.4.5.jar.sha1 delete mode 100644 plugins/analysis-phonetic/licenses/commons-codec-1.10.jar.sha1 create mode 100644 plugins/analysis-phonetic/licenses/commons-codec-1.11.jar.sha1 delete mode 100644 plugins/discovery-azure-classic/licenses/commons-codec-1.10.jar.sha1 create mode 100644 plugins/discovery-azure-classic/licenses/commons-codec-1.11.jar.sha1 delete mode 100644 plugins/discovery-azure-classic/licenses/httpclient-4.5.2.jar.sha1 create mode 100644 plugins/discovery-azure-classic/licenses/httpclient-4.5.7.jar.sha1 create mode 100644 plugins/discovery-azure-classic/licenses/httpcore-4.4.11.jar.sha1 delete mode 100644 plugins/discovery-azure-classic/licenses/httpcore-4.4.5.jar.sha1 delete mode 100644 plugins/discovery-ec2/licenses/commons-codec-1.10.jar.sha1 create mode 100644 plugins/discovery-ec2/licenses/commons-codec-1.11.jar.sha1 delete mode 100644 plugins/discovery-ec2/licenses/httpclient-4.5.2.jar.sha1 create mode 100644 plugins/discovery-ec2/licenses/httpclient-4.5.7.jar.sha1 create mode 100644 plugins/discovery-ec2/licenses/httpcore-4.4.11.jar.sha1 delete mode 100644 plugins/discovery-ec2/licenses/httpcore-4.4.5.jar.sha1 delete mode 100644 plugins/discovery-gce/licenses/commons-codec-1.10.jar.sha1 create mode 100644 plugins/discovery-gce/licenses/commons-codec-1.11.jar.sha1 delete mode 100644 plugins/discovery-gce/licenses/httpclient-4.5.2.jar.sha1 create mode 100644 plugins/discovery-gce/licenses/httpclient-4.5.7.jar.sha1 create mode 100644 plugins/discovery-gce/licenses/httpcore-4.4.11.jar.sha1 delete mode 100644 plugins/discovery-gce/licenses/httpcore-4.4.5.jar.sha1 delete mode 100644 plugins/ingest-attachment/licenses/commons-codec-1.10.jar.sha1 create mode 100644 plugins/ingest-attachment/licenses/commons-codec-1.11.jar.sha1 delete mode 100644 plugins/repository-gcs/licenses/commons-codec-1.10.jar.sha1 create mode 100644 plugins/repository-gcs/licenses/commons-codec-1.11.jar.sha1 delete mode 100644 plugins/repository-gcs/licenses/httpclient-4.5.2.jar.sha1 create mode 100644 plugins/repository-gcs/licenses/httpclient-4.5.7.jar.sha1 create mode 100644 plugins/repository-gcs/licenses/httpcore-4.4.11.jar.sha1 delete mode 100644 plugins/repository-gcs/licenses/httpcore-4.4.5.jar.sha1 delete mode 100644 plugins/repository-hdfs/licenses/commons-codec-1.10.jar.sha1 create mode 100644 plugins/repository-hdfs/licenses/commons-codec-1.11.jar.sha1 delete mode 100644 plugins/repository-s3/licenses/commons-codec-1.10.jar.sha1 create mode 100644 plugins/repository-s3/licenses/commons-codec-1.11.jar.sha1 delete mode 100644 plugins/repository-s3/licenses/httpclient-4.5.2.jar.sha1 create mode 100644 plugins/repository-s3/licenses/httpclient-4.5.7.jar.sha1 create mode 100644 plugins/repository-s3/licenses/httpcore-4.4.11.jar.sha1 delete mode 100644 plugins/repository-s3/licenses/httpcore-4.4.5.jar.sha1 delete mode 100644 x-pack/plugin/core/licenses/commons-codec-1.10.jar.sha1 create mode 100644 x-pack/plugin/core/licenses/commons-codec-1.11.jar.sha1 delete mode 100644 x-pack/plugin/core/licenses/httpasyncclient-4.1.2.jar.sha1 create mode 100644 x-pack/plugin/core/licenses/httpasyncclient-4.1.4.jar.sha1 delete mode 100644 x-pack/plugin/core/licenses/httpclient-4.5.2.jar.sha1 create mode 100644 x-pack/plugin/core/licenses/httpclient-4.5.7.jar.sha1 create mode 100644 x-pack/plugin/core/licenses/httpcore-4.4.11.jar.sha1 delete mode 100644 x-pack/plugin/core/licenses/httpcore-4.4.5.jar.sha1 create mode 100644 x-pack/plugin/core/licenses/httpcore-nio-4.4.11.jar.sha1 delete mode 100644 x-pack/plugin/core/licenses/httpcore-nio-4.4.5.jar.sha1 delete mode 100644 x-pack/plugin/security/licenses/httpclient-cache-4.5.2.jar.sha1 create mode 100644 x-pack/plugin/security/licenses/httpclient-cache-4.5.7.jar.sha1 diff --git a/buildSrc/version.properties b/buildSrc/version.properties index 778f29686ad67..118ab2f905f74 100644 --- a/buildSrc/version.properties +++ b/buildSrc/version.properties @@ -21,16 +21,13 @@ joda = 2.10.1 # test dependencies randomizedrunner = 2.7.1 junit = 4.12 -httpclient = 4.5.2 -# When updating httpcore, please also update server/src/main/resources/org/elasticsearch/bootstrap/test-framework.policy -httpcore = 4.4.5 -# When updating httpasyncclient, please also update server/src/main/resources/org/elasticsearch/bootstrap/test-framework.policy -httpasyncclient = 4.1.2 +httpclient = 4.5.7 +httpcore = 4.4.11 +httpasyncclient = 4.1.4 commonslogging = 1.1.3 -commonscodec = 1.10 +commonscodec = 1.11 hamcrest = 1.3 securemock = 1.2 -# When updating mocksocket, please also update server/src/main/resources/org/elasticsearch/bootstrap/test-framework.policy mocksocket = 1.2 # benchmark dependencies diff --git a/client/rest/licenses/commons-codec-1.10.jar.sha1 b/client/rest/licenses/commons-codec-1.10.jar.sha1 deleted file mode 100644 index 3fe8682a1b0f9..0000000000000 --- a/client/rest/licenses/commons-codec-1.10.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -4b95f4897fa13f2cd904aee711aeafc0c5295cd8 \ No newline at end of file diff --git a/client/rest/licenses/commons-codec-1.11.jar.sha1 b/client/rest/licenses/commons-codec-1.11.jar.sha1 new file mode 100644 index 0000000000000..b08f71a5babf0 --- /dev/null +++ b/client/rest/licenses/commons-codec-1.11.jar.sha1 @@ -0,0 +1 @@ +3acb4705652e16236558f0f4f2192cc33c3bd189 \ No newline at end of file diff --git a/client/rest/licenses/httpasyncclient-4.1.2.jar.sha1 b/client/rest/licenses/httpasyncclient-4.1.2.jar.sha1 deleted file mode 100644 index 065ed920a1773..0000000000000 --- a/client/rest/licenses/httpasyncclient-4.1.2.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -95aa3e6fb520191a0970a73cf09f62948ee614be \ No newline at end of file diff --git a/client/rest/licenses/httpasyncclient-4.1.4.jar.sha1 b/client/rest/licenses/httpasyncclient-4.1.4.jar.sha1 new file mode 100644 index 0000000000000..8360ab45c7ab3 --- /dev/null +++ b/client/rest/licenses/httpasyncclient-4.1.4.jar.sha1 @@ -0,0 +1 @@ +f3a3240681faae3fa46b573a4c7e50cec9db0d86 \ No newline at end of file diff --git a/client/rest/licenses/httpclient-4.5.2.jar.sha1 b/client/rest/licenses/httpclient-4.5.2.jar.sha1 deleted file mode 100644 index 6937112a09fb6..0000000000000 --- a/client/rest/licenses/httpclient-4.5.2.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -733db77aa8d9b2d68015189df76ab06304406e50 \ No newline at end of file diff --git a/client/rest/licenses/httpclient-4.5.7.jar.sha1 b/client/rest/licenses/httpclient-4.5.7.jar.sha1 new file mode 100644 index 0000000000000..a8b7cc0d994d3 --- /dev/null +++ b/client/rest/licenses/httpclient-4.5.7.jar.sha1 @@ -0,0 +1 @@ +dda059f4908e1b548b7ba68d81a3b05897f27cb0 \ No newline at end of file diff --git a/client/rest/licenses/httpcore-4.4.11.jar.sha1 b/client/rest/licenses/httpcore-4.4.11.jar.sha1 new file mode 100644 index 0000000000000..6d64372bfccd8 --- /dev/null +++ b/client/rest/licenses/httpcore-4.4.11.jar.sha1 @@ -0,0 +1 @@ +de748cf874e4e193b42eceea9fe5574fabb9d4df \ No newline at end of file diff --git a/client/rest/licenses/httpcore-4.4.5.jar.sha1 b/client/rest/licenses/httpcore-4.4.5.jar.sha1 deleted file mode 100644 index 581726601745b..0000000000000 --- a/client/rest/licenses/httpcore-4.4.5.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -e7501a1b34325abb00d17dde96150604a0658b54 \ No newline at end of file diff --git a/client/rest/licenses/httpcore-nio-4.4.11.jar.sha1 b/client/rest/licenses/httpcore-nio-4.4.11.jar.sha1 new file mode 100644 index 0000000000000..9e8777cb3da1c --- /dev/null +++ b/client/rest/licenses/httpcore-nio-4.4.11.jar.sha1 @@ -0,0 +1 @@ +7d0a97d01d39cff9aa3e6db81f21fddb2435f4e6 \ No newline at end of file diff --git a/client/rest/licenses/httpcore-nio-4.4.5.jar.sha1 b/client/rest/licenses/httpcore-nio-4.4.5.jar.sha1 deleted file mode 100644 index d6a80bf100de3..0000000000000 --- a/client/rest/licenses/httpcore-nio-4.4.5.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -f4be009e7505f6ceddf21e7960c759f413f15056 \ No newline at end of file diff --git a/client/sniffer/licenses/commons-codec-1.10.jar.sha1 b/client/sniffer/licenses/commons-codec-1.10.jar.sha1 deleted file mode 100644 index 3fe8682a1b0f9..0000000000000 --- a/client/sniffer/licenses/commons-codec-1.10.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -4b95f4897fa13f2cd904aee711aeafc0c5295cd8 \ No newline at end of file diff --git a/client/sniffer/licenses/commons-codec-1.11.jar.sha1 b/client/sniffer/licenses/commons-codec-1.11.jar.sha1 new file mode 100644 index 0000000000000..b08f71a5babf0 --- /dev/null +++ b/client/sniffer/licenses/commons-codec-1.11.jar.sha1 @@ -0,0 +1 @@ +3acb4705652e16236558f0f4f2192cc33c3bd189 \ No newline at end of file diff --git a/client/sniffer/licenses/httpclient-4.5.2.jar.sha1 b/client/sniffer/licenses/httpclient-4.5.2.jar.sha1 deleted file mode 100644 index 6937112a09fb6..0000000000000 --- a/client/sniffer/licenses/httpclient-4.5.2.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -733db77aa8d9b2d68015189df76ab06304406e50 \ No newline at end of file diff --git a/client/sniffer/licenses/httpclient-4.5.7.jar.sha1 b/client/sniffer/licenses/httpclient-4.5.7.jar.sha1 new file mode 100644 index 0000000000000..a8b7cc0d994d3 --- /dev/null +++ b/client/sniffer/licenses/httpclient-4.5.7.jar.sha1 @@ -0,0 +1 @@ +dda059f4908e1b548b7ba68d81a3b05897f27cb0 \ No newline at end of file diff --git a/client/sniffer/licenses/httpcore-4.4.11.jar.sha1 b/client/sniffer/licenses/httpcore-4.4.11.jar.sha1 new file mode 100644 index 0000000000000..6d64372bfccd8 --- /dev/null +++ b/client/sniffer/licenses/httpcore-4.4.11.jar.sha1 @@ -0,0 +1 @@ +de748cf874e4e193b42eceea9fe5574fabb9d4df \ No newline at end of file diff --git a/client/sniffer/licenses/httpcore-4.4.5.jar.sha1 b/client/sniffer/licenses/httpcore-4.4.5.jar.sha1 deleted file mode 100644 index 581726601745b..0000000000000 --- a/client/sniffer/licenses/httpcore-4.4.5.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -e7501a1b34325abb00d17dde96150604a0658b54 \ No newline at end of file diff --git a/plugins/analysis-phonetic/licenses/commons-codec-1.10.jar.sha1 b/plugins/analysis-phonetic/licenses/commons-codec-1.10.jar.sha1 deleted file mode 100644 index ebd32cee72347..0000000000000 --- a/plugins/analysis-phonetic/licenses/commons-codec-1.10.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -4b95f4897fa13f2cd904aee711aeafc0c5295cd8 diff --git a/plugins/analysis-phonetic/licenses/commons-codec-1.11.jar.sha1 b/plugins/analysis-phonetic/licenses/commons-codec-1.11.jar.sha1 new file mode 100644 index 0000000000000..b08f71a5babf0 --- /dev/null +++ b/plugins/analysis-phonetic/licenses/commons-codec-1.11.jar.sha1 @@ -0,0 +1 @@ +3acb4705652e16236558f0f4f2192cc33c3bd189 \ No newline at end of file diff --git a/plugins/discovery-azure-classic/licenses/commons-codec-1.10.jar.sha1 b/plugins/discovery-azure-classic/licenses/commons-codec-1.10.jar.sha1 deleted file mode 100644 index 3fe8682a1b0f9..0000000000000 --- a/plugins/discovery-azure-classic/licenses/commons-codec-1.10.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -4b95f4897fa13f2cd904aee711aeafc0c5295cd8 \ No newline at end of file diff --git a/plugins/discovery-azure-classic/licenses/commons-codec-1.11.jar.sha1 b/plugins/discovery-azure-classic/licenses/commons-codec-1.11.jar.sha1 new file mode 100644 index 0000000000000..b08f71a5babf0 --- /dev/null +++ b/plugins/discovery-azure-classic/licenses/commons-codec-1.11.jar.sha1 @@ -0,0 +1 @@ +3acb4705652e16236558f0f4f2192cc33c3bd189 \ No newline at end of file diff --git a/plugins/discovery-azure-classic/licenses/httpclient-4.5.2.jar.sha1 b/plugins/discovery-azure-classic/licenses/httpclient-4.5.2.jar.sha1 deleted file mode 100644 index 6937112a09fb6..0000000000000 --- a/plugins/discovery-azure-classic/licenses/httpclient-4.5.2.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -733db77aa8d9b2d68015189df76ab06304406e50 \ No newline at end of file diff --git a/plugins/discovery-azure-classic/licenses/httpclient-4.5.7.jar.sha1 b/plugins/discovery-azure-classic/licenses/httpclient-4.5.7.jar.sha1 new file mode 100644 index 0000000000000..a8b7cc0d994d3 --- /dev/null +++ b/plugins/discovery-azure-classic/licenses/httpclient-4.5.7.jar.sha1 @@ -0,0 +1 @@ +dda059f4908e1b548b7ba68d81a3b05897f27cb0 \ No newline at end of file diff --git a/plugins/discovery-azure-classic/licenses/httpcore-4.4.11.jar.sha1 b/plugins/discovery-azure-classic/licenses/httpcore-4.4.11.jar.sha1 new file mode 100644 index 0000000000000..6d64372bfccd8 --- /dev/null +++ b/plugins/discovery-azure-classic/licenses/httpcore-4.4.11.jar.sha1 @@ -0,0 +1 @@ +de748cf874e4e193b42eceea9fe5574fabb9d4df \ No newline at end of file diff --git a/plugins/discovery-azure-classic/licenses/httpcore-4.4.5.jar.sha1 b/plugins/discovery-azure-classic/licenses/httpcore-4.4.5.jar.sha1 deleted file mode 100644 index 581726601745b..0000000000000 --- a/plugins/discovery-azure-classic/licenses/httpcore-4.4.5.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -e7501a1b34325abb00d17dde96150604a0658b54 \ No newline at end of file diff --git a/plugins/discovery-ec2/licenses/commons-codec-1.10.jar.sha1 b/plugins/discovery-ec2/licenses/commons-codec-1.10.jar.sha1 deleted file mode 100644 index 3fe8682a1b0f9..0000000000000 --- a/plugins/discovery-ec2/licenses/commons-codec-1.10.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -4b95f4897fa13f2cd904aee711aeafc0c5295cd8 \ No newline at end of file diff --git a/plugins/discovery-ec2/licenses/commons-codec-1.11.jar.sha1 b/plugins/discovery-ec2/licenses/commons-codec-1.11.jar.sha1 new file mode 100644 index 0000000000000..b08f71a5babf0 --- /dev/null +++ b/plugins/discovery-ec2/licenses/commons-codec-1.11.jar.sha1 @@ -0,0 +1 @@ +3acb4705652e16236558f0f4f2192cc33c3bd189 \ No newline at end of file diff --git a/plugins/discovery-ec2/licenses/httpclient-4.5.2.jar.sha1 b/plugins/discovery-ec2/licenses/httpclient-4.5.2.jar.sha1 deleted file mode 100644 index 6937112a09fb6..0000000000000 --- a/plugins/discovery-ec2/licenses/httpclient-4.5.2.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -733db77aa8d9b2d68015189df76ab06304406e50 \ No newline at end of file diff --git a/plugins/discovery-ec2/licenses/httpclient-4.5.7.jar.sha1 b/plugins/discovery-ec2/licenses/httpclient-4.5.7.jar.sha1 new file mode 100644 index 0000000000000..a8b7cc0d994d3 --- /dev/null +++ b/plugins/discovery-ec2/licenses/httpclient-4.5.7.jar.sha1 @@ -0,0 +1 @@ +dda059f4908e1b548b7ba68d81a3b05897f27cb0 \ No newline at end of file diff --git a/plugins/discovery-ec2/licenses/httpcore-4.4.11.jar.sha1 b/plugins/discovery-ec2/licenses/httpcore-4.4.11.jar.sha1 new file mode 100644 index 0000000000000..6d64372bfccd8 --- /dev/null +++ b/plugins/discovery-ec2/licenses/httpcore-4.4.11.jar.sha1 @@ -0,0 +1 @@ +de748cf874e4e193b42eceea9fe5574fabb9d4df \ No newline at end of file diff --git a/plugins/discovery-ec2/licenses/httpcore-4.4.5.jar.sha1 b/plugins/discovery-ec2/licenses/httpcore-4.4.5.jar.sha1 deleted file mode 100644 index 581726601745b..0000000000000 --- a/plugins/discovery-ec2/licenses/httpcore-4.4.5.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -e7501a1b34325abb00d17dde96150604a0658b54 \ No newline at end of file diff --git a/plugins/discovery-gce/licenses/commons-codec-1.10.jar.sha1 b/plugins/discovery-gce/licenses/commons-codec-1.10.jar.sha1 deleted file mode 100644 index 3fe8682a1b0f9..0000000000000 --- a/plugins/discovery-gce/licenses/commons-codec-1.10.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -4b95f4897fa13f2cd904aee711aeafc0c5295cd8 \ No newline at end of file diff --git a/plugins/discovery-gce/licenses/commons-codec-1.11.jar.sha1 b/plugins/discovery-gce/licenses/commons-codec-1.11.jar.sha1 new file mode 100644 index 0000000000000..b08f71a5babf0 --- /dev/null +++ b/plugins/discovery-gce/licenses/commons-codec-1.11.jar.sha1 @@ -0,0 +1 @@ +3acb4705652e16236558f0f4f2192cc33c3bd189 \ No newline at end of file diff --git a/plugins/discovery-gce/licenses/httpclient-4.5.2.jar.sha1 b/plugins/discovery-gce/licenses/httpclient-4.5.2.jar.sha1 deleted file mode 100644 index 6937112a09fb6..0000000000000 --- a/plugins/discovery-gce/licenses/httpclient-4.5.2.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -733db77aa8d9b2d68015189df76ab06304406e50 \ No newline at end of file diff --git a/plugins/discovery-gce/licenses/httpclient-4.5.7.jar.sha1 b/plugins/discovery-gce/licenses/httpclient-4.5.7.jar.sha1 new file mode 100644 index 0000000000000..a8b7cc0d994d3 --- /dev/null +++ b/plugins/discovery-gce/licenses/httpclient-4.5.7.jar.sha1 @@ -0,0 +1 @@ +dda059f4908e1b548b7ba68d81a3b05897f27cb0 \ No newline at end of file diff --git a/plugins/discovery-gce/licenses/httpcore-4.4.11.jar.sha1 b/plugins/discovery-gce/licenses/httpcore-4.4.11.jar.sha1 new file mode 100644 index 0000000000000..6d64372bfccd8 --- /dev/null +++ b/plugins/discovery-gce/licenses/httpcore-4.4.11.jar.sha1 @@ -0,0 +1 @@ +de748cf874e4e193b42eceea9fe5574fabb9d4df \ No newline at end of file diff --git a/plugins/discovery-gce/licenses/httpcore-4.4.5.jar.sha1 b/plugins/discovery-gce/licenses/httpcore-4.4.5.jar.sha1 deleted file mode 100644 index 581726601745b..0000000000000 --- a/plugins/discovery-gce/licenses/httpcore-4.4.5.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -e7501a1b34325abb00d17dde96150604a0658b54 \ No newline at end of file diff --git a/plugins/ingest-attachment/licenses/commons-codec-1.10.jar.sha1 b/plugins/ingest-attachment/licenses/commons-codec-1.10.jar.sha1 deleted file mode 100644 index 3fe8682a1b0f9..0000000000000 --- a/plugins/ingest-attachment/licenses/commons-codec-1.10.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -4b95f4897fa13f2cd904aee711aeafc0c5295cd8 \ No newline at end of file diff --git a/plugins/ingest-attachment/licenses/commons-codec-1.11.jar.sha1 b/plugins/ingest-attachment/licenses/commons-codec-1.11.jar.sha1 new file mode 100644 index 0000000000000..b08f71a5babf0 --- /dev/null +++ b/plugins/ingest-attachment/licenses/commons-codec-1.11.jar.sha1 @@ -0,0 +1 @@ +3acb4705652e16236558f0f4f2192cc33c3bd189 \ No newline at end of file diff --git a/plugins/repository-gcs/licenses/commons-codec-1.10.jar.sha1 b/plugins/repository-gcs/licenses/commons-codec-1.10.jar.sha1 deleted file mode 100644 index 3fe8682a1b0f9..0000000000000 --- a/plugins/repository-gcs/licenses/commons-codec-1.10.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -4b95f4897fa13f2cd904aee711aeafc0c5295cd8 \ No newline at end of file diff --git a/plugins/repository-gcs/licenses/commons-codec-1.11.jar.sha1 b/plugins/repository-gcs/licenses/commons-codec-1.11.jar.sha1 new file mode 100644 index 0000000000000..b08f71a5babf0 --- /dev/null +++ b/plugins/repository-gcs/licenses/commons-codec-1.11.jar.sha1 @@ -0,0 +1 @@ +3acb4705652e16236558f0f4f2192cc33c3bd189 \ No newline at end of file diff --git a/plugins/repository-gcs/licenses/httpclient-4.5.2.jar.sha1 b/plugins/repository-gcs/licenses/httpclient-4.5.2.jar.sha1 deleted file mode 100644 index 6937112a09fb6..0000000000000 --- a/plugins/repository-gcs/licenses/httpclient-4.5.2.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -733db77aa8d9b2d68015189df76ab06304406e50 \ No newline at end of file diff --git a/plugins/repository-gcs/licenses/httpclient-4.5.7.jar.sha1 b/plugins/repository-gcs/licenses/httpclient-4.5.7.jar.sha1 new file mode 100644 index 0000000000000..a8b7cc0d994d3 --- /dev/null +++ b/plugins/repository-gcs/licenses/httpclient-4.5.7.jar.sha1 @@ -0,0 +1 @@ +dda059f4908e1b548b7ba68d81a3b05897f27cb0 \ No newline at end of file diff --git a/plugins/repository-gcs/licenses/httpcore-4.4.11.jar.sha1 b/plugins/repository-gcs/licenses/httpcore-4.4.11.jar.sha1 new file mode 100644 index 0000000000000..6d64372bfccd8 --- /dev/null +++ b/plugins/repository-gcs/licenses/httpcore-4.4.11.jar.sha1 @@ -0,0 +1 @@ +de748cf874e4e193b42eceea9fe5574fabb9d4df \ No newline at end of file diff --git a/plugins/repository-gcs/licenses/httpcore-4.4.5.jar.sha1 b/plugins/repository-gcs/licenses/httpcore-4.4.5.jar.sha1 deleted file mode 100644 index 581726601745b..0000000000000 --- a/plugins/repository-gcs/licenses/httpcore-4.4.5.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -e7501a1b34325abb00d17dde96150604a0658b54 \ No newline at end of file diff --git a/plugins/repository-hdfs/build.gradle b/plugins/repository-hdfs/build.gradle index ac22fa389c614..34323fb930fce 100644 --- a/plugins/repository-hdfs/build.gradle +++ b/plugins/repository-hdfs/build.gradle @@ -52,7 +52,7 @@ dependencies { compile 'com.google.protobuf:protobuf-java:2.5.0' compile 'commons-logging:commons-logging:1.1.3' compile 'commons-cli:commons-cli:1.2' - compile 'commons-codec:commons-codec:1.10' + compile "commons-codec:commons-codec:${versions.commonscodec}" compile 'commons-collections:commons-collections:3.2.2' compile 'commons-configuration:commons-configuration:1.6' compile 'commons-io:commons-io:2.4' diff --git a/plugins/repository-hdfs/licenses/commons-codec-1.10.jar.sha1 b/plugins/repository-hdfs/licenses/commons-codec-1.10.jar.sha1 deleted file mode 100644 index 3fe8682a1b0f9..0000000000000 --- a/plugins/repository-hdfs/licenses/commons-codec-1.10.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -4b95f4897fa13f2cd904aee711aeafc0c5295cd8 \ No newline at end of file diff --git a/plugins/repository-hdfs/licenses/commons-codec-1.11.jar.sha1 b/plugins/repository-hdfs/licenses/commons-codec-1.11.jar.sha1 new file mode 100644 index 0000000000000..b08f71a5babf0 --- /dev/null +++ b/plugins/repository-hdfs/licenses/commons-codec-1.11.jar.sha1 @@ -0,0 +1 @@ +3acb4705652e16236558f0f4f2192cc33c3bd189 \ No newline at end of file diff --git a/plugins/repository-s3/licenses/commons-codec-1.10.jar.sha1 b/plugins/repository-s3/licenses/commons-codec-1.10.jar.sha1 deleted file mode 100644 index 3fe8682a1b0f9..0000000000000 --- a/plugins/repository-s3/licenses/commons-codec-1.10.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -4b95f4897fa13f2cd904aee711aeafc0c5295cd8 \ No newline at end of file diff --git a/plugins/repository-s3/licenses/commons-codec-1.11.jar.sha1 b/plugins/repository-s3/licenses/commons-codec-1.11.jar.sha1 new file mode 100644 index 0000000000000..b08f71a5babf0 --- /dev/null +++ b/plugins/repository-s3/licenses/commons-codec-1.11.jar.sha1 @@ -0,0 +1 @@ +3acb4705652e16236558f0f4f2192cc33c3bd189 \ No newline at end of file diff --git a/plugins/repository-s3/licenses/httpclient-4.5.2.jar.sha1 b/plugins/repository-s3/licenses/httpclient-4.5.2.jar.sha1 deleted file mode 100644 index 6937112a09fb6..0000000000000 --- a/plugins/repository-s3/licenses/httpclient-4.5.2.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -733db77aa8d9b2d68015189df76ab06304406e50 \ No newline at end of file diff --git a/plugins/repository-s3/licenses/httpclient-4.5.7.jar.sha1 b/plugins/repository-s3/licenses/httpclient-4.5.7.jar.sha1 new file mode 100644 index 0000000000000..a8b7cc0d994d3 --- /dev/null +++ b/plugins/repository-s3/licenses/httpclient-4.5.7.jar.sha1 @@ -0,0 +1 @@ +dda059f4908e1b548b7ba68d81a3b05897f27cb0 \ No newline at end of file diff --git a/plugins/repository-s3/licenses/httpcore-4.4.11.jar.sha1 b/plugins/repository-s3/licenses/httpcore-4.4.11.jar.sha1 new file mode 100644 index 0000000000000..6d64372bfccd8 --- /dev/null +++ b/plugins/repository-s3/licenses/httpcore-4.4.11.jar.sha1 @@ -0,0 +1 @@ +de748cf874e4e193b42eceea9fe5574fabb9d4df \ No newline at end of file diff --git a/plugins/repository-s3/licenses/httpcore-4.4.5.jar.sha1 b/plugins/repository-s3/licenses/httpcore-4.4.5.jar.sha1 deleted file mode 100644 index 581726601745b..0000000000000 --- a/plugins/repository-s3/licenses/httpcore-4.4.5.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -e7501a1b34325abb00d17dde96150604a0658b54 \ No newline at end of file diff --git a/x-pack/plugin/core/licenses/commons-codec-1.10.jar.sha1 b/x-pack/plugin/core/licenses/commons-codec-1.10.jar.sha1 deleted file mode 100644 index 3fe8682a1b0f9..0000000000000 --- a/x-pack/plugin/core/licenses/commons-codec-1.10.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -4b95f4897fa13f2cd904aee711aeafc0c5295cd8 \ No newline at end of file diff --git a/x-pack/plugin/core/licenses/commons-codec-1.11.jar.sha1 b/x-pack/plugin/core/licenses/commons-codec-1.11.jar.sha1 new file mode 100644 index 0000000000000..b08f71a5babf0 --- /dev/null +++ b/x-pack/plugin/core/licenses/commons-codec-1.11.jar.sha1 @@ -0,0 +1 @@ +3acb4705652e16236558f0f4f2192cc33c3bd189 \ No newline at end of file diff --git a/x-pack/plugin/core/licenses/httpasyncclient-4.1.2.jar.sha1 b/x-pack/plugin/core/licenses/httpasyncclient-4.1.2.jar.sha1 deleted file mode 100644 index 065ed920a1773..0000000000000 --- a/x-pack/plugin/core/licenses/httpasyncclient-4.1.2.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -95aa3e6fb520191a0970a73cf09f62948ee614be \ No newline at end of file diff --git a/x-pack/plugin/core/licenses/httpasyncclient-4.1.4.jar.sha1 b/x-pack/plugin/core/licenses/httpasyncclient-4.1.4.jar.sha1 new file mode 100644 index 0000000000000..8360ab45c7ab3 --- /dev/null +++ b/x-pack/plugin/core/licenses/httpasyncclient-4.1.4.jar.sha1 @@ -0,0 +1 @@ +f3a3240681faae3fa46b573a4c7e50cec9db0d86 \ No newline at end of file diff --git a/x-pack/plugin/core/licenses/httpclient-4.5.2.jar.sha1 b/x-pack/plugin/core/licenses/httpclient-4.5.2.jar.sha1 deleted file mode 100644 index 6937112a09fb6..0000000000000 --- a/x-pack/plugin/core/licenses/httpclient-4.5.2.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -733db77aa8d9b2d68015189df76ab06304406e50 \ No newline at end of file diff --git a/x-pack/plugin/core/licenses/httpclient-4.5.7.jar.sha1 b/x-pack/plugin/core/licenses/httpclient-4.5.7.jar.sha1 new file mode 100644 index 0000000000000..a8b7cc0d994d3 --- /dev/null +++ b/x-pack/plugin/core/licenses/httpclient-4.5.7.jar.sha1 @@ -0,0 +1 @@ +dda059f4908e1b548b7ba68d81a3b05897f27cb0 \ No newline at end of file diff --git a/x-pack/plugin/core/licenses/httpcore-4.4.11.jar.sha1 b/x-pack/plugin/core/licenses/httpcore-4.4.11.jar.sha1 new file mode 100644 index 0000000000000..6d64372bfccd8 --- /dev/null +++ b/x-pack/plugin/core/licenses/httpcore-4.4.11.jar.sha1 @@ -0,0 +1 @@ +de748cf874e4e193b42eceea9fe5574fabb9d4df \ No newline at end of file diff --git a/x-pack/plugin/core/licenses/httpcore-4.4.5.jar.sha1 b/x-pack/plugin/core/licenses/httpcore-4.4.5.jar.sha1 deleted file mode 100644 index 581726601745b..0000000000000 --- a/x-pack/plugin/core/licenses/httpcore-4.4.5.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -e7501a1b34325abb00d17dde96150604a0658b54 \ No newline at end of file diff --git a/x-pack/plugin/core/licenses/httpcore-nio-4.4.11.jar.sha1 b/x-pack/plugin/core/licenses/httpcore-nio-4.4.11.jar.sha1 new file mode 100644 index 0000000000000..9e8777cb3da1c --- /dev/null +++ b/x-pack/plugin/core/licenses/httpcore-nio-4.4.11.jar.sha1 @@ -0,0 +1 @@ +7d0a97d01d39cff9aa3e6db81f21fddb2435f4e6 \ No newline at end of file diff --git a/x-pack/plugin/core/licenses/httpcore-nio-4.4.5.jar.sha1 b/x-pack/plugin/core/licenses/httpcore-nio-4.4.5.jar.sha1 deleted file mode 100644 index d6a80bf100de3..0000000000000 --- a/x-pack/plugin/core/licenses/httpcore-nio-4.4.5.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -f4be009e7505f6ceddf21e7960c759f413f15056 \ No newline at end of file diff --git a/x-pack/plugin/security/licenses/httpclient-cache-4.5.2.jar.sha1 b/x-pack/plugin/security/licenses/httpclient-cache-4.5.2.jar.sha1 deleted file mode 100644 index 75fbd3009da8e..0000000000000 --- a/x-pack/plugin/security/licenses/httpclient-cache-4.5.2.jar.sha1 +++ /dev/null @@ -1 +0,0 @@ -bd50ea83908dbf2f387a333216e66d2f0c5079bd \ No newline at end of file diff --git a/x-pack/plugin/security/licenses/httpclient-cache-4.5.7.jar.sha1 b/x-pack/plugin/security/licenses/httpclient-cache-4.5.7.jar.sha1 new file mode 100644 index 0000000000000..b121bd654212b --- /dev/null +++ b/x-pack/plugin/security/licenses/httpclient-cache-4.5.7.jar.sha1 @@ -0,0 +1 @@ +c13a0ce27c17831e5e5be6c751842006dcecb270 \ No newline at end of file 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 10fb8889fae33..654bc6b757dde 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 @@ -47,6 +47,7 @@ import org.elasticsearch.ElasticsearchException; import org.elasticsearch.cluster.service.ClusterService; import org.elasticsearch.common.Strings; +import org.elasticsearch.common.collect.Tuple; import org.elasticsearch.common.regex.Regex; import org.elasticsearch.common.settings.Settings; import org.elasticsearch.common.unit.ByteSizeValue; @@ -162,7 +163,9 @@ private void setWhitelistAutomaton(List whiteListedHosts) { } public HttpResponse execute(HttpRequest request) throws IOException { - URI uri = createURI(request); + Tuple tuple = createURI(request); + final URI uri = tuple.v2(); + final HttpHost httpHost = tuple.v1(); HttpRequestBase internalRequest; if (request.method == HttpMethod.HEAD) { @@ -212,7 +215,7 @@ public HttpResponse execute(HttpRequest request) throws IOException { // preemptive auth, no need to wait for a 401 first AuthCache authCache = new BasicAuthCache(); BasicScheme basicAuth = new BasicScheme(); - authCache.put(new HttpHost(request.host, request.port, request.scheme.scheme()), basicAuth); + authCache.put(httpHost, basicAuth); localContext.setAuthCache(authCache); } @@ -233,7 +236,7 @@ public HttpResponse execute(HttpRequest request) throws IOException { internalRequest.setConfig(config.build()); - try (CloseableHttpResponse response = SocketAccess.doPrivileged(() -> client.execute(internalRequest, localContext))) { + try (CloseableHttpResponse response = SocketAccess.doPrivileged(() -> client.execute(httpHost, internalRequest, localContext))) { // headers Header[] headers = response.getAllHeaders(); Map responseHeaders = new HashMap<>(headers.length); @@ -310,7 +313,7 @@ private HttpProxy getProxyFromSettings(Settings settings) { return HttpProxy.NO_PROXY; } - private URI createURI(HttpRequest request) { + private Tuple createURI(HttpRequest request) { // this could be really simple, as the apache http client has a UriBuilder class, however this class is always doing // url path escaping, and we have done this already, so this would result in double escaping try { @@ -320,7 +323,23 @@ private URI createURI(HttpRequest request) { URI uri = URIUtils.createURI(request.scheme.scheme(), request.host, request.port, request.path, Strings.isNullOrEmpty(format) ? null : format, null); - return uri; + if (uri.isAbsolute() == false) { + throw new IllegalStateException("URI [" + uri.toASCIIString() + "] must be absolute"); + } + final HttpHost httpHost = URIUtils.extractHost(uri); + // what a mess that we need to do this to workaround https://issues.apache.org/jira/browse/HTTPCLIENT-1968 + // in some cases the HttpClient will re-write the URI which drops the escaping for + // slashes within a path. This rewriting is done to obtain a relative URI when + // a proxy is not being used. To avoid this we can handle making it relative ourselves + if (request.path != null && request.path.contains("%2F")) { + final boolean isUsingProxy = (request.proxy != null && request.proxy.equals(HttpProxy.NO_PROXY) == false) || + HttpProxy.NO_PROXY.equals(settingsProxy) == false; + if (isUsingProxy == false) { + // we need a relative uri + uri = URIUtils.createURI(null, null, -1, request.path, Strings.isNullOrEmpty(format) ? null : format, null); + } + } + return new Tuple<>(httpHost, uri); } catch (URISyntaxException e) { throw new IllegalArgumentException(e); } From b88bdfe958ccedb96f624bbd796ab10c5e5f70cb Mon Sep 17 00:00:00 2001 From: Tim Brooks Date: Wed, 30 Jan 2019 15:40:49 -0700 Subject: [PATCH 13/16] Add dispatching to `HandledTransportAction` (#38050) This commit allows implementors of the `HandledTransportAction` to specify what thread the action should be executed on. The motivation for this commit is that certain CCR requests should be performed on the generic threadpool. --- .../support/HandledTransportAction.java | 13 ++++++- .../ClearCcrRestoreSessionAction.java | 12 ++---- .../GetCcrRestoreFileChunkAction.java | 39 +++++++------------ 3 files changed, 30 insertions(+), 34 deletions(-) diff --git a/server/src/main/java/org/elasticsearch/action/support/HandledTransportAction.java b/server/src/main/java/org/elasticsearch/action/support/HandledTransportAction.java index 0b35bc8fb89d6..c0bc0af839967 100644 --- a/server/src/main/java/org/elasticsearch/action/support/HandledTransportAction.java +++ b/server/src/main/java/org/elasticsearch/action/support/HandledTransportAction.java @@ -45,6 +45,11 @@ protected HandledTransportAction(String actionName, TransportService transportSe this(actionName, true, transportService, actionFilters, requestReader); } + protected HandledTransportAction(String actionName, TransportService transportService, + ActionFilters actionFilters, Writeable.Reader requestReader, String executor) { + this(actionName, true, transportService, actionFilters, requestReader, executor); + } + protected HandledTransportAction(String actionName, boolean canTripCircuitBreaker, TransportService transportService, ActionFilters actionFilters, Supplier request) { super(actionName, actionFilters, transportService.getTaskManager()); @@ -55,8 +60,14 @@ protected HandledTransportAction(String actionName, boolean canTripCircuitBreake protected HandledTransportAction(String actionName, boolean canTripCircuitBreaker, TransportService transportService, ActionFilters actionFilters, Writeable.Reader requestReader) { + this(actionName, canTripCircuitBreaker, transportService, actionFilters, requestReader, ThreadPool.Names.SAME); + } + + protected HandledTransportAction(String actionName, boolean canTripCircuitBreaker, + TransportService transportService, ActionFilters actionFilters, + Writeable.Reader requestReader, String executor) { super(actionName, actionFilters, transportService.getTaskManager()); - transportService.registerRequestHandler(actionName, ThreadPool.Names.SAME, false, canTripCircuitBreaker, requestReader, + transportService.registerRequestHandler(actionName, executor, false, canTripCircuitBreaker, requestReader, new TransportHandler()); } diff --git a/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/repositories/ClearCcrRestoreSessionAction.java b/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/repositories/ClearCcrRestoreSessionAction.java index 81cde2984f500..317890edb4206 100644 --- a/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/repositories/ClearCcrRestoreSessionAction.java +++ b/x-pack/plugin/ccr/src/main/java/org/elasticsearch/xpack/ccr/action/repositories/ClearCcrRestoreSessionAction.java @@ -43,26 +43,20 @@ public static class TransportDeleteCcrRestoreSessionAction extends HandledTransportAction { private final CcrRestoreSourceService ccrRestoreService; - private final ThreadPool threadPool; @Inject public TransportDeleteCcrRestoreSessionAction(ActionFilters actionFilters, TransportService transportService, CcrRestoreSourceService ccrRestoreService) { - super(NAME, transportService, actionFilters, ClearCcrRestoreSessionRequest::new); + super(NAME, transportService, actionFilters, ClearCcrRestoreSessionRequest::new, ThreadPool.Names.GENERIC); TransportActionProxy.registerProxyAction(transportService, NAME, ClearCcrRestoreSessionResponse::new); this.ccrRestoreService = ccrRestoreService; - this.threadPool = transportService.getThreadPool(); } @Override protected void doExecute(Task task, ClearCcrRestoreSessionRequest request, ActionListener listener) { - // TODO: Currently blocking actions might occur in the session closed callbacks. This dispatch - // may be unnecessary when we remove these callbacks. - threadPool.generic().execute(() -> { - ccrRestoreService.closeSession(request.getSessionUUID()); - listener.onResponse(new ClearCcrRestoreSessionResponse()); - }); + ccrRestoreService.closeSession(request.getSessionUUID()); + listener.onResponse(new ClearCcrRestoreSessionResponse()); } } 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 index 3f473f25c2411..cf8d2e5c55f48 100644 --- 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 @@ -19,7 +19,6 @@ 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; @@ -58,7 +57,7 @@ public static class TransportGetCcrRestoreFileChunkAction @Inject public TransportGetCcrRestoreFileChunkAction(BigArrays bigArrays, TransportService transportService, ActionFilters actionFilters, CcrRestoreSourceService restoreSourceService) { - super(NAME, transportService, actionFilters, GetCcrRestoreFileChunkRequest::new); + super(NAME, transportService, actionFilters, GetCcrRestoreFileChunkRequest::new, ThreadPool.Names.GENERIC); TransportActionProxy.registerProxyAction(transportService, NAME, GetCcrRestoreFileChunkResponse::new); this.threadPool = transportService.getThreadPool(); this.restoreSourceService = restoreSourceService; @@ -68,29 +67,21 @@ public TransportGetCcrRestoreFileChunkAction(BigArrays bigArrays, TransportServi @Override protected void doExecute(Task task, GetCcrRestoreFileChunkRequest request, ActionListener listener) { - threadPool.generic().execute(new AbstractRunnable() { - @Override - public void onFailure(Exception e) { - listener.onFailure(e); + 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)); } - - @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)); - } - } - } - }); + } catch (IOException e) { + listener.onFailure(e); + } } } From 7c738fd2414a5a11954d201a4af49307472b99fc Mon Sep 17 00:00:00 2001 From: Tal Levy Date: Wed, 30 Jan 2019 15:09:17 -0800 Subject: [PATCH 14/16] Skip Shrink when numberOfShards not changed (#37953) Previously, ShrinkAction would fail if it was executed on an index that had the same number of shards as the target shrunken number. This PR introduced a new BranchingStep that is used inside of ShrinkAction to branch which step to move to next, depending on the shard values. So no shrink will occur if the shard count is unchanged. --- .../documentation/ILMDocumentationIT.java | 4 +- .../core/indexlifecycle/BranchingStep.java | 114 +++++++++++++++ .../core/indexlifecycle/ShrinkAction.java | 10 +- .../xpack/core/indexlifecycle/Step.java | 2 +- .../indexlifecycle/BranchingStepTests.java | 85 +++++++++++ .../indexlifecycle/ShrinkActionTests.java | 133 ++++++++++++++---- .../TimeSeriesLifecycleActionsIT.java | 18 +++ .../ExecuteStepsUpdateTask.java | 16 ++- .../ExecuteStepsUpdateTaskTests.java | 2 +- 9 files changed, 346 insertions(+), 38 deletions(-) create mode 100644 x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/indexlifecycle/BranchingStep.java create mode 100644 x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/indexlifecycle/BranchingStepTests.java diff --git a/client/rest-high-level/src/test/java/org/elasticsearch/client/documentation/ILMDocumentationIT.java b/client/rest-high-level/src/test/java/org/elasticsearch/client/documentation/ILMDocumentationIT.java index 5ccb0c8393304..db9df0ac24c78 100644 --- a/client/rest-high-level/src/test/java/org/elasticsearch/client/documentation/ILMDocumentationIT.java +++ b/client/rest-high-level/src/test/java/org/elasticsearch/client/documentation/ILMDocumentationIT.java @@ -591,7 +591,7 @@ public void testRetryPolicy() throws Exception { { Map phases = new HashMap<>(); Map warmActions = new HashMap<>(); - warmActions.put(ShrinkAction.NAME, new ShrinkAction(1)); + warmActions.put(ShrinkAction.NAME, new ShrinkAction(3)); phases.put("warm", new Phase("warm", TimeValue.ZERO, warmActions)); LifecyclePolicy policy = new LifecyclePolicy("my_policy", @@ -602,7 +602,7 @@ public void testRetryPolicy() throws Exception { CreateIndexRequest createIndexRequest = new CreateIndexRequest("my_index") .settings(Settings.builder() - .put(IndexMetaData.SETTING_NUMBER_OF_SHARDS, 1) + .put(IndexMetaData.SETTING_NUMBER_OF_SHARDS, 2) .put("index.lifecycle.name", "my_policy") .build()); client.indices().create(createIndexRequest, RequestOptions.DEFAULT); diff --git a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/indexlifecycle/BranchingStep.java b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/indexlifecycle/BranchingStep.java new file mode 100644 index 0000000000000..2514492520200 --- /dev/null +++ b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/indexlifecycle/BranchingStep.java @@ -0,0 +1,114 @@ +/* + * 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.indexlifecycle; + +import org.apache.logging.log4j.LogManager; +import org.apache.logging.log4j.Logger; +import org.apache.lucene.util.SetOnce; +import org.elasticsearch.cluster.ClusterState; +import org.elasticsearch.cluster.metadata.IndexMetaData; +import org.elasticsearch.index.Index; + +import java.util.Objects; +import java.util.function.BiPredicate; + +/** + * This step changes its {@link #getNextStepKey()} depending on the + * outcome of a defined predicate. It performs no changes to the + * cluster state. + */ +public class BranchingStep extends ClusterStateActionStep { + public static final String NAME = "branch"; + + private static final Logger logger = LogManager.getLogger(BranchingStep.class); + + private StepKey nextStepKeyOnFalse; + private StepKey nextStepKeyOnTrue; + private BiPredicate predicate; + private SetOnce predicateValue; + + /** + * {@link BranchingStep} is a step whose next step is based on + * the return value of a specific predicate. + * + * @param key the step's key + * @param nextStepKeyOnFalse the key of the step to run if predicate returns false + * @param nextStepKeyOnTrue the key of the step to run if predicate returns true + * @param predicate the condition to check when deciding which step to run next + */ + public BranchingStep(StepKey key, StepKey nextStepKeyOnFalse, StepKey nextStepKeyOnTrue, BiPredicate predicate) { + // super.nextStepKey is set to null since it is not used by this step + super(key, null); + this.nextStepKeyOnFalse = nextStepKeyOnFalse; + this.nextStepKeyOnTrue = nextStepKeyOnTrue; + this.predicate = predicate; + this.predicateValue = new SetOnce<>(); + } + + @Override + public ClusterState performAction(Index index, ClusterState clusterState) { + IndexMetaData indexMetaData = clusterState.metaData().index(index); + if (indexMetaData == null) { + // Index must have been since deleted, ignore it + logger.debug("[{}] lifecycle action for index [{}] executed but index no longer exists", getKey().getAction(), index.getName()); + return clusterState; + } + predicateValue.set(predicate.test(index, clusterState)); + return clusterState; + } + + /** + * This method returns the next step to execute based on the predicate. If + * the predicate returned true, then nextStepKeyOnTrue is the key of the + * next step to run, otherwise nextStepKeyOnFalse is. + * + * throws {@link UnsupportedOperationException} if performAction was not called yet + * + * @return next step to execute + */ + @Override + public final StepKey getNextStepKey() { + if (predicateValue.get() == null) { + throw new IllegalStateException("Cannot call getNextStepKey before performAction"); + } + return predicateValue.get() ? nextStepKeyOnTrue : nextStepKeyOnFalse; + } + + /** + * @return the next step if {@code predicate} is false + */ + final StepKey getNextStepKeyOnFalse() { + return nextStepKeyOnFalse; + } + + /** + * @return the next step if {@code predicate} is true + */ + final StepKey getNextStepKeyOnTrue() { + return nextStepKeyOnTrue; + } + + public final BiPredicate getPredicate() { + return predicate; + } + + @Override + public boolean equals(Object o) { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + if (!super.equals(o)) return false; + BranchingStep that = (BranchingStep) o; + return super.equals(o) + && Objects.equals(nextStepKeyOnFalse, that.nextStepKeyOnFalse) + && Objects.equals(nextStepKeyOnTrue, that.nextStepKeyOnTrue); + } + + @Override + public int hashCode() { + return Objects.hash(super.hashCode(), nextStepKeyOnFalse, nextStepKeyOnTrue); + } +} diff --git a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/indexlifecycle/ShrinkAction.java b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/indexlifecycle/ShrinkAction.java index a79383c24de8b..51f24e6d65254 100644 --- a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/indexlifecycle/ShrinkAction.java +++ b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/indexlifecycle/ShrinkAction.java @@ -85,6 +85,7 @@ public boolean isSafeAction() { public List toSteps(Client client, String phase, Step.StepKey nextStepKey) { Settings readOnlySettings = Settings.builder().put(IndexMetaData.SETTING_BLOCKS_WRITE, true).build(); + StepKey branchingKey = new StepKey(phase, NAME, BranchingStep.NAME); StepKey readOnlyKey = new StepKey(phase, NAME, ReadOnlyAction.NAME); StepKey setSingleNodeKey = new StepKey(phase, NAME, SetSingleNodeAllocateStep.NAME); StepKey allocationRoutedKey = new StepKey(phase, NAME, CheckShrinkReadyStep.NAME); @@ -94,6 +95,8 @@ public List toSteps(Client client, String phase, Step.StepKey nextStepKey) StepKey aliasKey = new StepKey(phase, NAME, ShrinkSetAliasStep.NAME); StepKey isShrunkIndexKey = new StepKey(phase, NAME, ShrunkenIndexCheckStep.NAME); + BranchingStep conditionalSkipShrinkStep = new BranchingStep(branchingKey, readOnlyKey, nextStepKey, + (index, clusterState) -> clusterState.getMetaData().index(index).getNumberOfShards() == numberOfShards); UpdateSettingsStep readOnlyStep = new UpdateSettingsStep(readOnlyKey, setSingleNodeKey, client, readOnlySettings); SetSingleNodeAllocateStep setSingleNodeStep = new SetSingleNodeAllocateStep(setSingleNodeKey, allocationRoutedKey, client); CheckShrinkReadyStep checkShrinkReadyStep = new CheckShrinkReadyStep(allocationRoutedKey, shrinkKey); @@ -102,12 +105,13 @@ public List toSteps(Client client, String phase, Step.StepKey nextStepKey) CopyExecutionStateStep copyMetadata = new CopyExecutionStateStep(copyMetadataKey, aliasKey, SHRUNKEN_INDEX_PREFIX); ShrinkSetAliasStep aliasSwapAndDelete = new ShrinkSetAliasStep(aliasKey, isShrunkIndexKey, client, SHRUNKEN_INDEX_PREFIX); ShrunkenIndexCheckStep waitOnShrinkTakeover = new ShrunkenIndexCheckStep(isShrunkIndexKey, nextStepKey, SHRUNKEN_INDEX_PREFIX); - return Arrays.asList(readOnlyStep, setSingleNodeStep, checkShrinkReadyStep, shrink, allocated, copyMetadata, - aliasSwapAndDelete, waitOnShrinkTakeover); + return Arrays.asList(conditionalSkipShrinkStep, readOnlyStep, setSingleNodeStep, checkShrinkReadyStep, shrink, allocated, + copyMetadata, aliasSwapAndDelete, waitOnShrinkTakeover); } @Override public List toStepKeys(String phase) { + StepKey conditionalSkipKey = new StepKey(phase, NAME, BranchingStep.NAME); StepKey readOnlyKey = new StepKey(phase, NAME, ReadOnlyAction.NAME); StepKey setSingleNodeKey = new StepKey(phase, NAME, SetSingleNodeAllocateStep.NAME); StepKey checkShrinkReadyKey = new StepKey(phase, NAME, CheckShrinkReadyStep.NAME); @@ -116,7 +120,7 @@ public List toStepKeys(String phase) { StepKey copyMetadataKey = new StepKey(phase, NAME, CopyExecutionStateStep.NAME); StepKey aliasKey = new StepKey(phase, NAME, ShrinkSetAliasStep.NAME); StepKey isShrunkIndexKey = new StepKey(phase, NAME, ShrunkenIndexCheckStep.NAME); - return Arrays.asList(readOnlyKey, setSingleNodeKey, checkShrinkReadyKey, shrinkKey, enoughShardsKey, + return Arrays.asList(conditionalSkipKey, readOnlyKey, setSingleNodeKey, checkShrinkReadyKey, shrinkKey, enoughShardsKey, copyMetadataKey, aliasKey, isShrunkIndexKey); } diff --git a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/indexlifecycle/Step.java b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/indexlifecycle/Step.java index 5f24ab29d0284..4917a2aafd433 100644 --- a/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/indexlifecycle/Step.java +++ b/x-pack/plugin/core/src/main/java/org/elasticsearch/xpack/core/indexlifecycle/Step.java @@ -34,7 +34,7 @@ public final StepKey getKey() { return key; } - public final StepKey getNextStepKey() { + public StepKey getNextStepKey() { return nextStepKey; } diff --git a/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/indexlifecycle/BranchingStepTests.java b/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/indexlifecycle/BranchingStepTests.java new file mode 100644 index 0000000000000..6c354b79203cd --- /dev/null +++ b/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/indexlifecycle/BranchingStepTests.java @@ -0,0 +1,85 @@ +/* + * 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.indexlifecycle; + +import org.apache.lucene.util.SetOnce; +import org.elasticsearch.Version; +import org.elasticsearch.cluster.ClusterName; +import org.elasticsearch.cluster.ClusterState; +import org.elasticsearch.cluster.metadata.IndexMetaData; +import org.elasticsearch.cluster.metadata.MetaData; +import org.elasticsearch.index.Index; +import org.elasticsearch.xpack.core.indexlifecycle.Step.StepKey; + +import java.util.function.BiPredicate; + +import static org.hamcrest.Matchers.equalTo; + +public class BranchingStepTests extends AbstractStepTestCase { + + public void testPredicateNextStepChange() { + String indexName = randomAlphaOfLength(5); + ClusterState state = ClusterState.builder(ClusterName.DEFAULT).metaData(MetaData.builder() + .put(IndexMetaData.builder(indexName).settings(settings(Version.CURRENT)) + .numberOfShards(1).numberOfReplicas(0))).build(); + StepKey stepKey = new StepKey(randomAlphaOfLength(5), randomAlphaOfLength(5), BranchingStep.NAME); + StepKey nextStepKey = new StepKey(randomAlphaOfLength(6), randomAlphaOfLength(6), BranchingStep.NAME); + StepKey nextSkipKey = new StepKey(randomAlphaOfLength(7), randomAlphaOfLength(7), BranchingStep.NAME); + { + BranchingStep step = new BranchingStep(stepKey, nextStepKey, nextSkipKey, (i, c) -> true); + expectThrows(IllegalStateException.class, step::getNextStepKey); + step.performAction(state.metaData().index(indexName).getIndex(), state); + assertThat(step.getNextStepKey(), equalTo(step.getNextStepKeyOnTrue())); + expectThrows(SetOnce.AlreadySetException.class, () -> step.performAction(state.metaData().index(indexName).getIndex(), state)); + } + { + BranchingStep step = new BranchingStep(stepKey, nextStepKey, nextSkipKey, (i, c) -> false); + expectThrows(IllegalStateException.class, step::getNextStepKey); + step.performAction(state.metaData().index(indexName).getIndex(), state); + assertThat(step.getNextStepKey(), equalTo(step.getNextStepKeyOnFalse())); + expectThrows(SetOnce.AlreadySetException.class, () -> step.performAction(state.metaData().index(indexName).getIndex(), state)); + } + } + + @Override + public BranchingStep createRandomInstance() { + StepKey stepKey = new StepKey(randomAlphaOfLength(5), randomAlphaOfLength(5), BranchingStep.NAME); + StepKey nextStepKey = new StepKey(randomAlphaOfLength(6), randomAlphaOfLength(6), BranchingStep.NAME); + StepKey nextSkipKey = new StepKey(randomAlphaOfLength(7), randomAlphaOfLength(7), BranchingStep.NAME); + return new BranchingStep(stepKey, nextStepKey, nextSkipKey, (i, c) -> randomBoolean()); + } + + @Override + public BranchingStep mutateInstance(BranchingStep instance) { + StepKey key = instance.getKey(); + StepKey nextStepKey = instance.getNextStepKeyOnFalse(); + StepKey nextSkipStepKey = instance.getNextStepKeyOnTrue(); + BiPredicate predicate = instance.getPredicate(); + + switch (between(0, 2)) { + case 0: + key = new StepKey(key.getPhase(), key.getAction(), key.getName() + randomAlphaOfLength(5)); + break; + case 1: + nextStepKey = new StepKey(nextStepKey.getPhase(), nextStepKey.getAction(), nextStepKey.getName() + randomAlphaOfLength(5)); + break; + case 2: + nextSkipStepKey = new StepKey(nextSkipStepKey.getPhase(), nextSkipStepKey.getAction(), + nextSkipStepKey.getName() + randomAlphaOfLength(5)); + break; + default: + throw new AssertionError("Illegal randomisation branch"); + } + + return new BranchingStep(key, nextStepKey, nextSkipStepKey, predicate); + } + + @Override + public BranchingStep copyInstance(BranchingStep instance) { + return new BranchingStep(instance.getKey(), instance.getNextStepKeyOnFalse(), instance.getNextStepKeyOnTrue(), + instance.getPredicate()); + } +} diff --git a/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/indexlifecycle/ShrinkActionTests.java b/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/indexlifecycle/ShrinkActionTests.java index 658f8bef6d47b..be512c87d8548 100644 --- a/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/indexlifecycle/ShrinkActionTests.java +++ b/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/indexlifecycle/ShrinkActionTests.java @@ -5,12 +5,18 @@ */ package org.elasticsearch.xpack.core.indexlifecycle; +import org.elasticsearch.Version; +import org.elasticsearch.cluster.ClusterName; +import org.elasticsearch.cluster.ClusterState; import org.elasticsearch.cluster.metadata.IndexMetaData; +import org.elasticsearch.cluster.metadata.MetaData; import org.elasticsearch.common.io.stream.Writeable.Reader; +import org.elasticsearch.common.unit.TimeValue; import org.elasticsearch.common.xcontent.XContentParser; import org.elasticsearch.xpack.core.indexlifecycle.Step.StepKey; import java.io.IOException; +import java.util.Collections; import java.util.List; import static org.hamcrest.Matchers.equalTo; @@ -46,59 +52,134 @@ public void testNonPositiveShardNumber() { assertThat(e.getMessage(), equalTo("[number_of_shards] must be greater than 0")); } + public void testPerformActionWithSkip() { + String lifecycleName = randomAlphaOfLengthBetween(4, 10); + int numberOfShards = randomIntBetween(1, 10); + ShrinkAction action = new ShrinkAction(numberOfShards); + String phase = randomAlphaOfLengthBetween(1, 10); + StepKey nextStepKey = new StepKey(randomAlphaOfLengthBetween(1, 10), randomAlphaOfLengthBetween(1, 10), + randomAlphaOfLengthBetween(1, 10)); + List steps = action.toSteps(null, phase, nextStepKey); + BranchingStep step = ((BranchingStep) steps.get(0)); + + LifecyclePolicy policy = new LifecyclePolicy(lifecycleName, Collections.singletonMap("warm", + new Phase("warm", TimeValue.ZERO, Collections.singletonMap(action.getWriteableName(), action)))); + LifecyclePolicyMetadata policyMetadata = new LifecyclePolicyMetadata(policy, Collections.emptyMap(), + randomNonNegativeLong(), randomNonNegativeLong()); + String indexName = randomAlphaOfLength(5); + ClusterState state = ClusterState.builder(ClusterName.DEFAULT).metaData(MetaData.builder() + .putCustom(IndexLifecycleMetadata.TYPE, new IndexLifecycleMetadata( + Collections.singletonMap(policyMetadata.getName(), policyMetadata), OperationMode.RUNNING)) + .put(IndexMetaData.builder(indexName).settings(settings(Version.CURRENT).put(LifecycleSettings.LIFECYCLE_NAME, lifecycleName)) + .putCustom(LifecycleExecutionState.ILM_CUSTOM_METADATA_KEY, + LifecycleExecutionState.builder() + .setPhase(step.getKey().getPhase()) + .setPhaseTime(0L) + .setAction(step.getKey().getAction()) + .setActionTime(0L) + .setStep(step.getKey().getName()) + .setStepTime(0L) + .build().asMap()) + .numberOfShards(numberOfShards).numberOfReplicas(0))).build(); + step.performAction(state.metaData().index(indexName).getIndex(), state); + assertThat(step.getNextStepKey(), equalTo(nextStepKey)); + } + + public void testPerformActionWithoutSkip() { + int numShards = 6; + int divisor = randomFrom(2, 3, 6); + int expectedFinalShards = numShards / divisor; + String lifecycleName = randomAlphaOfLengthBetween(4, 10); + ShrinkAction action = new ShrinkAction(expectedFinalShards); + String phase = randomAlphaOfLengthBetween(1, 10); + StepKey nextStepKey = new StepKey(randomAlphaOfLengthBetween(1, 10), randomAlphaOfLengthBetween(1, 10), + randomAlphaOfLengthBetween(1, 10)); + List steps = action.toSteps(null, phase, nextStepKey); + BranchingStep step = ((BranchingStep) steps.get(0)); + + LifecyclePolicy policy = new LifecyclePolicy(lifecycleName, Collections.singletonMap("warm", + new Phase("warm", TimeValue.ZERO, Collections.singletonMap(action.getWriteableName(), action)))); + LifecyclePolicyMetadata policyMetadata = new LifecyclePolicyMetadata(policy, Collections.emptyMap(), + randomNonNegativeLong(), randomNonNegativeLong()); + String indexName = randomAlphaOfLength(5); + ClusterState state = ClusterState.builder(ClusterName.DEFAULT).metaData(MetaData.builder() + .putCustom(IndexLifecycleMetadata.TYPE, new IndexLifecycleMetadata( + Collections.singletonMap(policyMetadata.getName(), policyMetadata), OperationMode.RUNNING)) + .put(IndexMetaData.builder(indexName).settings(settings(Version.CURRENT).put(LifecycleSettings.LIFECYCLE_NAME, lifecycleName)) + .putCustom(LifecycleExecutionState.ILM_CUSTOM_METADATA_KEY, + LifecycleExecutionState.builder() + .setPhase(step.getKey().getPhase()) + .setPhaseTime(0L) + .setAction(step.getKey().getAction()) + .setActionTime(0L) + .setStep(step.getKey().getName()) + .setStepTime(0L) + .build().asMap()) + .numberOfShards(numShards).numberOfReplicas(0))).build(); + ClusterState newState = step.performAction(state.metaData().index(indexName).getIndex(), state); + assertThat(step.getNextStepKey(), equalTo(steps.get(1).getKey())); + } + public void testToSteps() { ShrinkAction action = createTestInstance(); String phase = randomAlphaOfLengthBetween(1, 10); StepKey nextStepKey = new StepKey(randomAlphaOfLengthBetween(1, 10), randomAlphaOfLengthBetween(1, 10), randomAlphaOfLengthBetween(1, 10)); List steps = action.toSteps(null, phase, nextStepKey); - assertThat(steps.size(), equalTo(8)); - StepKey expectedFirstKey = new StepKey(phase, ShrinkAction.NAME, ReadOnlyAction.NAME); - StepKey expectedSecondKey = new StepKey(phase, ShrinkAction.NAME, SetSingleNodeAllocateStep.NAME); - StepKey expectedThirdKey = new StepKey(phase, ShrinkAction.NAME, CheckShrinkReadyStep.NAME); - StepKey expectedFourthKey = new StepKey(phase, ShrinkAction.NAME, ShrinkStep.NAME); - StepKey expectedFifthKey = new StepKey(phase, ShrinkAction.NAME, ShrunkShardsAllocatedStep.NAME); - StepKey expectedSixthKey = new StepKey(phase, ShrinkAction.NAME, CopyExecutionStateStep.NAME); - StepKey expectedSeventhKey = new StepKey(phase, ShrinkAction.NAME, ShrinkSetAliasStep.NAME); - StepKey expectedEighthKey = new StepKey(phase, ShrinkAction.NAME, ShrunkenIndexCheckStep.NAME); - - assertTrue(steps.get(0) instanceof UpdateSettingsStep); + assertThat(steps.size(), equalTo(9)); + StepKey expectedFirstKey = new StepKey(phase, ShrinkAction.NAME, BranchingStep.NAME); + StepKey expectedSecondKey = new StepKey(phase, ShrinkAction.NAME, ReadOnlyAction.NAME); + StepKey expectedThirdKey = new StepKey(phase, ShrinkAction.NAME, SetSingleNodeAllocateStep.NAME); + StepKey expectedFourthKey = new StepKey(phase, ShrinkAction.NAME, CheckShrinkReadyStep.NAME); + StepKey expectedFifthKey = new StepKey(phase, ShrinkAction.NAME, ShrinkStep.NAME); + StepKey expectedSixthKey = new StepKey(phase, ShrinkAction.NAME, ShrunkShardsAllocatedStep.NAME); + StepKey expectedSeventhKey = new StepKey(phase, ShrinkAction.NAME, CopyExecutionStateStep.NAME); + StepKey expectedEighthKey = new StepKey(phase, ShrinkAction.NAME, ShrinkSetAliasStep.NAME); + StepKey expectedNinthKey = new StepKey(phase, ShrinkAction.NAME, ShrunkenIndexCheckStep.NAME); + + assertTrue(steps.get(0) instanceof BranchingStep); assertThat(steps.get(0).getKey(), equalTo(expectedFirstKey)); - assertThat(steps.get(0).getNextStepKey(), equalTo(expectedSecondKey)); - assertTrue(IndexMetaData.INDEX_BLOCKS_WRITE_SETTING.get(((UpdateSettingsStep)steps.get(0)).getSettings())); + expectThrows(IllegalStateException.class, () -> steps.get(0).getNextStepKey()); + assertThat(((BranchingStep) steps.get(0)).getNextStepKeyOnFalse(), equalTo(expectedSecondKey)); + assertThat(((BranchingStep) steps.get(0)).getNextStepKeyOnTrue(), equalTo(nextStepKey)); - assertTrue(steps.get(1) instanceof SetSingleNodeAllocateStep); + assertTrue(steps.get(1) instanceof UpdateSettingsStep); assertThat(steps.get(1).getKey(), equalTo(expectedSecondKey)); assertThat(steps.get(1).getNextStepKey(), equalTo(expectedThirdKey)); + assertTrue(IndexMetaData.INDEX_BLOCKS_WRITE_SETTING.get(((UpdateSettingsStep)steps.get(1)).getSettings())); - assertTrue(steps.get(2) instanceof CheckShrinkReadyStep); + assertTrue(steps.get(2) instanceof SetSingleNodeAllocateStep); assertThat(steps.get(2).getKey(), equalTo(expectedThirdKey)); assertThat(steps.get(2).getNextStepKey(), equalTo(expectedFourthKey)); - assertTrue(steps.get(3) instanceof ShrinkStep); + assertTrue(steps.get(3) instanceof CheckShrinkReadyStep); assertThat(steps.get(3).getKey(), equalTo(expectedFourthKey)); assertThat(steps.get(3).getNextStepKey(), equalTo(expectedFifthKey)); - assertThat(((ShrinkStep) steps.get(3)).getShrunkIndexPrefix(), equalTo(ShrinkAction.SHRUNKEN_INDEX_PREFIX)); - assertTrue(steps.get(4) instanceof ShrunkShardsAllocatedStep); + assertTrue(steps.get(4) instanceof ShrinkStep); assertThat(steps.get(4).getKey(), equalTo(expectedFifthKey)); assertThat(steps.get(4).getNextStepKey(), equalTo(expectedSixthKey)); - assertThat(((ShrunkShardsAllocatedStep) steps.get(4)).getShrunkIndexPrefix(), equalTo(ShrinkAction.SHRUNKEN_INDEX_PREFIX)); + assertThat(((ShrinkStep) steps.get(4)).getShrunkIndexPrefix(), equalTo(ShrinkAction.SHRUNKEN_INDEX_PREFIX)); - assertTrue(steps.get(5) instanceof CopyExecutionStateStep); + assertTrue(steps.get(5) instanceof ShrunkShardsAllocatedStep); assertThat(steps.get(5).getKey(), equalTo(expectedSixthKey)); assertThat(steps.get(5).getNextStepKey(), equalTo(expectedSeventhKey)); - assertThat(((CopyExecutionStateStep) steps.get(5)).getShrunkIndexPrefix(), equalTo(ShrinkAction.SHRUNKEN_INDEX_PREFIX)); + assertThat(((ShrunkShardsAllocatedStep) steps.get(5)).getShrunkIndexPrefix(), equalTo(ShrinkAction.SHRUNKEN_INDEX_PREFIX)); - assertTrue(steps.get(6) instanceof ShrinkSetAliasStep); + assertTrue(steps.get(6) instanceof CopyExecutionStateStep); assertThat(steps.get(6).getKey(), equalTo(expectedSeventhKey)); assertThat(steps.get(6).getNextStepKey(), equalTo(expectedEighthKey)); - assertThat(((ShrinkSetAliasStep) steps.get(6)).getShrunkIndexPrefix(), equalTo(ShrinkAction.SHRUNKEN_INDEX_PREFIX)); + assertThat(((CopyExecutionStateStep) steps.get(6)).getShrunkIndexPrefix(), equalTo(ShrinkAction.SHRUNKEN_INDEX_PREFIX)); - assertTrue(steps.get(7) instanceof ShrunkenIndexCheckStep); + assertTrue(steps.get(7) instanceof ShrinkSetAliasStep); assertThat(steps.get(7).getKey(), equalTo(expectedEighthKey)); - assertThat(steps.get(7).getNextStepKey(), equalTo(nextStepKey)); - assertThat(((ShrunkenIndexCheckStep) steps.get(7)).getShrunkIndexPrefix(), equalTo(ShrinkAction.SHRUNKEN_INDEX_PREFIX)); + assertThat(steps.get(7).getNextStepKey(), equalTo(expectedNinthKey)); + assertThat(((ShrinkSetAliasStep) steps.get(7)).getShrunkIndexPrefix(), equalTo(ShrinkAction.SHRUNKEN_INDEX_PREFIX)); + + assertTrue(steps.get(8) instanceof ShrunkenIndexCheckStep); + assertThat(steps.get(8).getKey(), equalTo(expectedNinthKey)); + assertThat(steps.get(8).getNextStepKey(), equalTo(nextStepKey)); + assertThat(((ShrunkenIndexCheckStep) steps.get(8)).getShrunkIndexPrefix(), equalTo(ShrinkAction.SHRUNKEN_INDEX_PREFIX)); } @Override diff --git a/x-pack/plugin/ilm/qa/multi-node/src/test/java/org/elasticsearch/xpack/indexlifecycle/TimeSeriesLifecycleActionsIT.java b/x-pack/plugin/ilm/qa/multi-node/src/test/java/org/elasticsearch/xpack/indexlifecycle/TimeSeriesLifecycleActionsIT.java index 675c24a4195b7..24c1ab1c1cbf1 100644 --- a/x-pack/plugin/ilm/qa/multi-node/src/test/java/org/elasticsearch/xpack/indexlifecycle/TimeSeriesLifecycleActionsIT.java +++ b/x-pack/plugin/ilm/qa/multi-node/src/test/java/org/elasticsearch/xpack/indexlifecycle/TimeSeriesLifecycleActionsIT.java @@ -466,6 +466,24 @@ public void testShrinkAction() throws Exception { expectThrows(ResponseException.class, this::indexDocument); } + public void testShrinkSameShards() throws Exception { + int numberOfShards = randomFrom(1, 2); + String shrunkenIndex = ShrinkAction.SHRUNKEN_INDEX_PREFIX + index; + createIndexWithSettings(index, Settings.builder().put(IndexMetaData.SETTING_NUMBER_OF_SHARDS, numberOfShards) + .put(IndexMetaData.SETTING_NUMBER_OF_REPLICAS, 0)); + createNewSingletonPolicy("warm", new ShrinkAction(numberOfShards)); + updatePolicy(index, policy); + assertBusy(() -> { + assertTrue(indexExists(index)); + assertFalse(indexExists(shrunkenIndex)); + assertFalse(aliasExists(shrunkenIndex, index)); + Map settings = getOnlyIndexSettings(index); + assertThat(getStepKeyForIndex(index), equalTo(TerminalPolicyStep.KEY)); + assertThat(settings.get(IndexMetaData.SETTING_NUMBER_OF_SHARDS), equalTo(String.valueOf(numberOfShards))); + assertNull(settings.get(IndexMetaData.INDEX_BLOCKS_WRITE_SETTING.getKey())); + }); + } + public void testShrinkDuringSnapshot() throws Exception { String shrunkenIndex = ShrinkAction.SHRUNKEN_INDEX_PREFIX + index; // Create the repository before taking the snapshot. diff --git a/x-pack/plugin/ilm/src/main/java/org/elasticsearch/xpack/indexlifecycle/ExecuteStepsUpdateTask.java b/x-pack/plugin/ilm/src/main/java/org/elasticsearch/xpack/indexlifecycle/ExecuteStepsUpdateTask.java index 70aa9af2c7277..131330bcb9c99 100644 --- a/x-pack/plugin/ilm/src/main/java/org/elasticsearch/xpack/indexlifecycle/ExecuteStepsUpdateTask.java +++ b/x-pack/plugin/ilm/src/main/java/org/elasticsearch/xpack/indexlifecycle/ExecuteStepsUpdateTask.java @@ -86,24 +86,30 @@ public ClusterState execute(final ClusterState currentState) throws IOException // either get to a step that isn't a cluster state step or a // cluster state wait step returns not completed while (currentStep instanceof ClusterStateActionStep || currentStep instanceof ClusterStateWaitStep) { - nextStepKey = currentStep.getNextStepKey(); if (currentStep instanceof ClusterStateActionStep) { // cluster state action step so do the action and // move the cluster state to the next step - logger.trace("[{}] performing cluster state action ({}) [{}], next: [{}]", - index.getName(), currentStep.getClass().getSimpleName(), currentStep.getKey(), currentStep.getNextStepKey()); + logger.trace("[{}] performing cluster state action ({}) [{}]", + index.getName(), currentStep.getClass().getSimpleName(), currentStep.getKey()); try { state = ((ClusterStateActionStep) currentStep).performAction(index, state); } catch (Exception exception) { return moveToErrorStep(state, currentStep.getKey(), exception); } - if (currentStep.getNextStepKey() == null) { + // set here to make sure that the clusterProcessed knows to execute the + // correct step if it an async action + nextStepKey = currentStep.getNextStepKey(); + if (nextStepKey == null) { return state; } else { + logger.trace("[{}] moving cluster state to next step [{}]", index.getName(), nextStepKey); state = IndexLifecycleRunner.moveClusterStateToNextStep(index, state, currentStep.getKey(), - currentStep.getNextStepKey(), nowSupplier, false); + nextStepKey, nowSupplier, false); } } else { + // set here to make sure that the clusterProcessed knows to execute the + // correct step if it an async action + nextStepKey = currentStep.getNextStepKey(); // cluster state wait step so evaluate the // condition, if the condition is met move to the // next step, if its not met return the current diff --git a/x-pack/plugin/ilm/src/test/java/org/elasticsearch/xpack/indexlifecycle/ExecuteStepsUpdateTaskTests.java b/x-pack/plugin/ilm/src/test/java/org/elasticsearch/xpack/indexlifecycle/ExecuteStepsUpdateTaskTests.java index 4611618b2cd24..963ce5d2e2a6f 100644 --- a/x-pack/plugin/ilm/src/test/java/org/elasticsearch/xpack/indexlifecycle/ExecuteStepsUpdateTaskTests.java +++ b/x-pack/plugin/ilm/src/test/java/org/elasticsearch/xpack/indexlifecycle/ExecuteStepsUpdateTaskTests.java @@ -268,7 +268,7 @@ public void testClusterActionStepThrowsException() throws IOException { assertThat(currentStepKey, equalTo(new StepKey(firstStepKey.getPhase(), firstStepKey.getAction(), ErrorStep.NAME))); assertThat(firstStep.getExecuteCount(), equalTo(1L)); assertThat(secondStep.getExecuteCount(), equalTo(0L)); - assertThat(task.getNextStepKey(), equalTo(secondStep.getKey())); + assertNull(task.getNextStepKey()); assertThat(lifecycleState.getPhaseTime(), nullValue()); assertThat(lifecycleState.getActionTime(), nullValue()); assertThat(lifecycleState.getStepInfo(), From 6935d3d5fa020055145dfff6b5a7cdeac5ac3d83 Mon Sep 17 00:00:00 2001 From: Jason Tedor Date: Wed, 30 Jan 2019 18:18:30 -0500 Subject: [PATCH 15/16] Temporarily disable BWC for retention lease stats (#38049) This commit temporarily disables BWC testing while backporting a change to expose retention leases in stats. --- build.gradle | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/build.gradle b/build.gradle index 4bd211a12b3b0..164add875eb3b 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 = true -final String bwc_tests_disabled_issue = "" /* place a PR link here when committing bwc changes */ +final boolean bwc_tests_enabled = false +final String bwc_tests_disabled_issue = "https://github.com/elastic/elasticsearch/pull/37991" /* 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 dad41c2b7f1c2bb0c2396036d550c5bd5fb381a5 Mon Sep 17 00:00:00 2001 From: Jake Landis Date: Wed, 30 Jan 2019 17:38:47 -0600 Subject: [PATCH 16/16] ILM setPriority corrections for a 0 value (#38001) This commit fixes the test case that ensures only a priority less then 0 is used with testNonPositivePriority. This also allows the HLRC to support a value of 0. Closes #37652 --- .../elasticsearch/client/indexlifecycle/SetPriorityAction.java | 2 +- .../client/indexlifecycle/SetPriorityActionTests.java | 2 +- .../xpack/core/indexlifecycle/SetPriorityActionTests.java | 3 +-- 3 files changed, 3 insertions(+), 4 deletions(-) diff --git a/client/rest-high-level/src/main/java/org/elasticsearch/client/indexlifecycle/SetPriorityAction.java b/client/rest-high-level/src/main/java/org/elasticsearch/client/indexlifecycle/SetPriorityAction.java index 414d2a52ad048..7989c8ee9f145 100644 --- a/client/rest-high-level/src/main/java/org/elasticsearch/client/indexlifecycle/SetPriorityAction.java +++ b/client/rest-high-level/src/main/java/org/elasticsearch/client/indexlifecycle/SetPriorityAction.java @@ -55,7 +55,7 @@ public static SetPriorityAction parse(XContentParser parser) { } public SetPriorityAction(@Nullable Integer recoveryPriority) { - if (recoveryPriority != null && recoveryPriority <= 0) { + if (recoveryPriority != null && recoveryPriority < 0) { throw new IllegalArgumentException("[" + RECOVERY_PRIORITY_FIELD.getPreferredName() + "] must be 0 or greater"); } this.recoveryPriority = recoveryPriority; diff --git a/client/rest-high-level/src/test/java/org/elasticsearch/client/indexlifecycle/SetPriorityActionTests.java b/client/rest-high-level/src/test/java/org/elasticsearch/client/indexlifecycle/SetPriorityActionTests.java index f50935a87d398..bec0dd3276400 100644 --- a/client/rest-high-level/src/test/java/org/elasticsearch/client/indexlifecycle/SetPriorityActionTests.java +++ b/client/rest-high-level/src/test/java/org/elasticsearch/client/indexlifecycle/SetPriorityActionTests.java @@ -48,7 +48,7 @@ protected boolean supportsUnknownFields() { } public void testNonPositivePriority() { - Exception e = expectThrows(Exception.class, () -> new SetPriorityAction(randomIntBetween(-100, 0))); + Exception e = expectThrows(Exception.class, () -> new SetPriorityAction(randomIntBetween(-100, -1))); assertThat(e.getMessage(), equalTo("[priority] must be 0 or greater")); } diff --git a/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/indexlifecycle/SetPriorityActionTests.java b/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/indexlifecycle/SetPriorityActionTests.java index c0848f5326c40..302cf8cb9c6b8 100644 --- a/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/indexlifecycle/SetPriorityActionTests.java +++ b/x-pack/plugin/core/src/test/java/org/elasticsearch/xpack/core/indexlifecycle/SetPriorityActionTests.java @@ -38,9 +38,8 @@ protected Reader instanceReader() { return SetPriorityAction::new; } - @AwaitsFix(bugUrl="https://github.com/elastic/elasticsearch/issues/37652") public void testNonPositivePriority() { - Exception e = expectThrows(Exception.class, () -> new SetPriorityAction(randomIntBetween(-100, 0))); + Exception e = expectThrows(Exception.class, () -> new SetPriorityAction(randomIntBetween(-100, -1))); assertThat(e.getMessage(), equalTo("[priority] must be 0 or greater")); }