From e581c56f571e2e9910595b5ff1fc19ac9811bad0 Mon Sep 17 00:00:00 2001 From: javanna Date: Wed, 4 Apr 2018 15:14:27 +0200 Subject: [PATCH 01/13] Simplify `Sniffer` and begin testing it properly Introduced `Scheduler` abstraction to make `Sniffer` testable so that tasks scheduling is isolated and easily mockable. Also added a bunch of TODOs mainly around new tests that should be added soon-ish. --- .../elasticsearch/client/sniff/Sniffer.java | 170 ++++++---- .../client/sniff/MockHostsSniffer.java | 3 +- .../client/sniff/SnifferTests.java | 312 ++++++++++++++++++ 3 files changed, 415 insertions(+), 70 deletions(-) create mode 100644 client/sniffer/src/test/java/org/elasticsearch/client/sniff/SnifferTests.java diff --git a/client/sniffer/src/main/java/org/elasticsearch/client/sniff/Sniffer.java b/client/sniffer/src/main/java/org/elasticsearch/client/sniff/Sniffer.java index c655babd9ed3d..65239dd008b49 100644 --- a/client/sniffer/src/main/java/org/elasticsearch/client/sniff/Sniffer.java +++ b/client/sniffer/src/main/java/org/elasticsearch/client/sniff/Sniffer.java @@ -26,7 +26,6 @@ import org.elasticsearch.client.RestClientBuilder; import java.io.Closeable; -import java.io.IOException; import java.security.AccessController; import java.security.PrivilegedAction; import java.util.List; @@ -51,91 +50,137 @@ public class Sniffer implements Closeable { private static final Log logger = LogFactory.getLog(Sniffer.class); private static final String SNIFFER_THREAD_NAME = "es_rest_client_sniffer"; - private final Task task; + private final HostsSniffer hostsSniffer; + private final RestClient restClient; + + private final long sniffIntervalMillis; + private final long sniffAfterFailureDelayMillis; + private final Scheduler scheduler; + + private final AtomicBoolean running = new AtomicBoolean(false); Sniffer(RestClient restClient, HostsSniffer hostsSniffer, long sniffInterval, long sniffAfterFailureDelay) { - this.task = new Task(hostsSniffer, restClient, sniffInterval, sniffAfterFailureDelay); + this(restClient, hostsSniffer, new DefaultScheduler(), sniffInterval, sniffAfterFailureDelay); + } + + Sniffer(RestClient restClient, HostsSniffer hostsSniffer, Scheduler scheduler, long sniffInterval, long sniffAfterFailureDelay) { + this.hostsSniffer = hostsSniffer; + this.restClient = restClient; + this.sniffIntervalMillis = sniffInterval; + this.sniffAfterFailureDelayMillis = sniffAfterFailureDelay; + this.scheduler = scheduler; + scheduleNextRun(0); + } + + private void scheduleNextRun(long delayMillis) { + scheduler.schedule(new Runnable() { + @Override + public void run() { + sniff(null, sniffIntervalMillis); + } + }, delayMillis); } /** * Triggers a new sniffing round and explicitly takes out the failed host provided as argument */ public void sniffOnFailure(HttpHost failedHost) { - this.task.sniffOnFailure(failedHost); + sniff(failedHost, sniffAfterFailureDelayMillis); + } + + private void sniff(HttpHost excludeHost, long nextSniffDelayMillis) { + //If a sniffing round is already running nothing happens, it makes no sense to start or wait to start another round + if (running.compareAndSet(false, true)) { + try { + List sniffedHosts = hostsSniffer.sniffHosts(); + logger.debug("sniffed hosts: " + sniffedHosts); + if (excludeHost != null) { + sniffedHosts.remove(excludeHost); + } + if (sniffedHosts.isEmpty()) { + logger.warn("no hosts to set, hosts will be updated at the next sniffing round"); + } else { + this.restClient.setHosts(sniffedHosts.toArray(new HttpHost[sniffedHosts.size()])); + } + } catch (Exception e) { + logger.error("error while sniffing nodes", e); + } finally { + //TODO potential problem here if this doesn't happen last? though tests become complicated + running.set(false); + //TODO do we want to also test that this never gets called concurrently? + scheduleNextRun(nextSniffDelayMillis); + } + } } @Override - public void close() throws IOException { - task.shutdown(); + public void close() { + this.scheduler.shutdown(); } - private static class Task implements Runnable { - private final HostsSniffer hostsSniffer; - private final RestClient restClient; + /** + * Returns a new {@link SnifferBuilder} to help with {@link Sniffer} creation. + * + * @param restClient the client that gets its hosts set (via {@link RestClient#setHosts(HttpHost...)}) once they are fetched + * @return a new instance of {@link SnifferBuilder} + */ + public static SnifferBuilder builder(RestClient restClient) { + return new SnifferBuilder(restClient); + } + + /** + * The Scheduler interface allows to isolate the sniffing scheduling aspects so that we can test + * the sniffer by injecting a custom scheduler that is more suited for testing. + */ + interface Scheduler { + /** + * Schedules the provided {@link Runnable} to run in delayMillis milliseconds + */ + void schedule(Runnable runnable, long delayMillis); + + /** + * Shuts this scheduler down + */ + void shutdown(); + } - private final long sniffIntervalMillis; - private final long sniffAfterFailureDelayMillis; - private final ScheduledExecutorService scheduledExecutorService; - private final AtomicBoolean running = new AtomicBoolean(false); + /** + * Default implementation of {@link Scheduler}, based on {@link ScheduledExecutorService} + */ + static final class DefaultScheduler implements Scheduler { + final ScheduledExecutorService scheduledExecutorService; private ScheduledFuture scheduledFuture; - private Task(HostsSniffer hostsSniffer, RestClient restClient, long sniffIntervalMillis, long sniffAfterFailureDelayMillis) { - this.hostsSniffer = hostsSniffer; - this.restClient = restClient; - this.sniffIntervalMillis = sniffIntervalMillis; - this.sniffAfterFailureDelayMillis = sniffAfterFailureDelayMillis; - SnifferThreadFactory threadFactory = new SnifferThreadFactory(SNIFFER_THREAD_NAME); - this.scheduledExecutorService = Executors.newScheduledThreadPool(1, threadFactory); - scheduleNextRun(0); + DefaultScheduler() { + this(Executors.newScheduledThreadPool(1, new SnifferThreadFactory(SNIFFER_THREAD_NAME))); + } + + DefaultScheduler(ScheduledExecutorService scheduledExecutorService) { + this.scheduledExecutorService = scheduledExecutorService; } - synchronized void scheduleNextRun(long delayMillis) { + //TODO test concurrent calls to schedule? + @Override + public synchronized void schedule(Runnable runnable, long delayMillis) { + //TODO maybe this is not even needed, just let it throw rejected execution exception instead? if (scheduledExecutorService.isShutdown() == false) { try { - if (scheduledFuture != null) { - //regardless of when the next sniff is scheduled, cancel it and schedule a new one with updated delay + if (this.scheduledFuture != null) { + //regardless of when the next sniff is scheduled, cancel it and schedule a new one with the latest delay. + //instead of piling up sniff rounds to be run, the last run decides when the following execution will be. this.scheduledFuture.cancel(false); } logger.debug("scheduling next sniff in " + delayMillis + " ms"); - this.scheduledFuture = this.scheduledExecutorService.schedule(this, delayMillis, TimeUnit.MILLISECONDS); + this.scheduledFuture = scheduledExecutorService.schedule(runnable, delayMillis, TimeUnit.MILLISECONDS); } catch(Exception e) { logger.error("error while scheduling next sniffer task", e); } } } + //TODO test concurrent calls to shutdown? @Override - public void run() { - sniff(null, sniffIntervalMillis); - } - - void sniffOnFailure(HttpHost failedHost) { - sniff(failedHost, sniffAfterFailureDelayMillis); - } - - void sniff(HttpHost excludeHost, long nextSniffDelayMillis) { - if (running.compareAndSet(false, true)) { - try { - List sniffedHosts = hostsSniffer.sniffHosts(); - logger.debug("sniffed hosts: " + sniffedHosts); - if (excludeHost != null) { - sniffedHosts.remove(excludeHost); - } - if (sniffedHosts.isEmpty()) { - logger.warn("no hosts to set, hosts will be updated at the next sniffing round"); - } else { - this.restClient.setHosts(sniffedHosts.toArray(new HttpHost[sniffedHosts.size()])); - } - } catch (Exception e) { - logger.error("error while sniffing nodes", e); - } finally { - scheduleNextRun(nextSniffDelayMillis); - running.set(false); - } - } - } - - synchronized void shutdown() { + public synchronized void shutdown() { scheduledExecutorService.shutdown(); try { if (scheduledExecutorService.awaitTermination(1000, TimeUnit.MILLISECONDS)) { @@ -148,18 +193,7 @@ synchronized void shutdown() { } } - /** - * Returns a new {@link SnifferBuilder} to help with {@link Sniffer} creation. - * - * @param restClient the client that gets its hosts set (via {@link RestClient#setHosts(HttpHost...)}) once they are fetched - * @return a new instance of {@link SnifferBuilder} - */ - public static SnifferBuilder builder(RestClient restClient) { - return new SnifferBuilder(restClient); - } - - private static class SnifferThreadFactory implements ThreadFactory { - + static class SnifferThreadFactory implements ThreadFactory { private final AtomicInteger threadNumber = new AtomicInteger(1); private final String namePrefix; private final ThreadFactory originalThreadFactory; diff --git a/client/sniffer/src/test/java/org/elasticsearch/client/sniff/MockHostsSniffer.java b/client/sniffer/src/test/java/org/elasticsearch/client/sniff/MockHostsSniffer.java index 5a52151d76e01..7550459e9ea50 100644 --- a/client/sniffer/src/test/java/org/elasticsearch/client/sniff/MockHostsSniffer.java +++ b/client/sniffer/src/test/java/org/elasticsearch/client/sniff/MockHostsSniffer.java @@ -21,7 +21,6 @@ import org.apache.http.HttpHost; -import java.io.IOException; import java.util.Collections; import java.util.List; @@ -30,7 +29,7 @@ */ class MockHostsSniffer implements HostsSniffer { @Override - public List sniffHosts() throws IOException { + public List sniffHosts() { return Collections.singletonList(new HttpHost("localhost", 9200)); } } diff --git a/client/sniffer/src/test/java/org/elasticsearch/client/sniff/SnifferTests.java b/client/sniffer/src/test/java/org/elasticsearch/client/sniff/SnifferTests.java new file mode 100644 index 0000000000000..8a90c098340fc --- /dev/null +++ b/client/sniffer/src/test/java/org/elasticsearch/client/sniff/SnifferTests.java @@ -0,0 +1,312 @@ +/* + * 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.sniff; + +import org.apache.http.HttpHost; +import org.elasticsearch.client.RestClient; +import org.elasticsearch.client.RestClientTestCase; +import org.elasticsearch.client.sniff.Sniffer.DefaultScheduler; +import org.elasticsearch.client.sniff.Sniffer.Scheduler; + +import java.io.IOException; +import java.util.Collections; +import java.util.List; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.ScheduledFuture; +import java.util.concurrent.ScheduledThreadPoolExecutor; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.atomic.AtomicInteger; + +import static org.hamcrest.CoreMatchers.equalTo; +import static org.hamcrest.CoreMatchers.instanceOf; +import static org.hamcrest.Matchers.is; +import static org.hamcrest.Matchers.lessThanOrEqualTo; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertThat; +import static org.junit.Assert.assertTrue; +import static org.mockito.Matchers.any; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.times; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.verifyNoMoreInteractions; +import static org.mockito.Mockito.when; + +public class SnifferTests extends RestClientTestCase { + + /** + * Test multiple sniffing rounds by mocking the {@link Scheduler} as well as + * the {@link HostsSniffer}. + * The {@link CountingHostsSniffer} doesn't make any connection but may throw exception or return no hosts. + * The {@link Scheduler} implementation doesn't respect requested sniff delays but rather immediately runs them while + * allowing to assert that the requested delay for each schedule run is the expected one. + */ + public void testOrdinarySniffingRounds() throws Exception { + final long sniffInterval = randomLongBetween(1, Long.MAX_VALUE); + long sniffAfterFailureDelay = randomLongBetween(1, Long.MAX_VALUE); + RestClient restClient = mock(RestClient.class); + CountingHostsSniffer hostsSniffer = new CountingHostsSniffer(); + final int iters = randomIntBetween(30, 100); + final CountDownLatch latch = new CountDownLatch(1); + final AtomicInteger runs = new AtomicInteger(iters); + final ExecutorService executor = Executors.newSingleThreadExecutor(); + try { + Scheduler scheduler = new Scheduler() { + @Override + public void schedule(Runnable runnable, long delayMillis) { + int numberOfRuns = runs.getAndDecrement(); + if (numberOfRuns == 0) { + latch.countDown(); + return; + } + if (numberOfRuns == iters) { + //the first time "schedule" gets called from the Sniffer constructor with delay set to 0 + assertEquals(0L, delayMillis); + } else { + //all of the subsequent times "schedule" is called with delay set to the configured sniff interval + assertEquals(sniffInterval, delayMillis); + } + //we immediately run rather than scheduling + executor.execute(runnable); + } + + @Override + public void shutdown() { + //the executor is closed externally, shutdown is tested separately + } + }; + + //all we need to do is initialize the sniffer, sniffing will start automatically in the background + new Sniffer(restClient, hostsSniffer, scheduler, sniffInterval, sniffAfterFailureDelay); + assertTrue(latch.await(1000, TimeUnit.MILLISECONDS)); + } finally { + executor.shutdown(); + executor.awaitTermination(1000, TimeUnit.MILLISECONDS); + } + int totalRuns = hostsSniffer.runs.get(); + assertEquals(iters, totalRuns); + int setHostsRuns = totalRuns - hostsSniffer.failures.get() - hostsSniffer.emptyList.get(); + verify(restClient, times(setHostsRuns)).setHosts(any(HttpHost.class)); + verifyNoMoreInteractions(restClient); + } + + /** + * Test that {@link Sniffer#close()} shuts down the underlying {@link Scheduler}, and that such calls are idempotent + */ + public void testClose() { + long sniffInterval = randomLongBetween(1, Long.MAX_VALUE); + long sniffAfterFailureDelay = randomLongBetween(1, Long.MAX_VALUE); + RestClient restClient = mock(RestClient.class); + final AtomicInteger shutdown = new AtomicInteger(0); + Scheduler scheduler = new Scheduler() { + @Override + public void schedule(Runnable runnable, long delayMillis) { + } + + @Override + public void shutdown() { + shutdown.incrementAndGet(); + } + }; + + Sniffer sniffer = new Sniffer(restClient, new MockHostsSniffer(), scheduler, sniffInterval, sniffAfterFailureDelay); + assertEquals(0, shutdown.get()); + int iters = randomIntBetween(3, 10); + for (int i = 0; i < iters; i++) { + sniffer.close(); + assertEquals(i + 1, shutdown.get()); + } + } + + /** + * Test concurrent calls to Sniffer#sniff. They may happen if you set a super low sniffInterval + * and/or the {@link HostsSniffer} implementation is very slow. + */ + public void testConcurrentSniffRounds() throws Exception { + long sniffInterval = randomLongBetween(1, Long.MAX_VALUE); + long sniffAfterFailureDelay = randomLongBetween(1, Long.MAX_VALUE); + RestClient restClient = mock(RestClient.class); + CountingHostsSniffer hostsSniffer = new CountingHostsSniffer(); + + final int numThreads = randomIntBetween(10, 30); + final ExecutorService executor = Executors.newFixedThreadPool(numThreads); + final AtomicBoolean firstRun = new AtomicBoolean(true); + + try { + Scheduler scheduler = new Scheduler() { + @Override + public void schedule(Runnable runnable, long delayMillis) { + if (firstRun.compareAndSet(true, false)) { + for (int i = 0; i < numThreads; i++) { + //this will only run the same runnable n times, to simulate concurrent calls to the sniff method + executor.submit(runnable); + } + } + } + + @Override + public void shutdown() { + } + }; + new Sniffer(restClient, hostsSniffer, scheduler, sniffInterval, sniffAfterFailureDelay); + } finally { + executor.shutdown(); + executor.awaitTermination(1000, TimeUnit.MILLISECONDS); + } + + int totalRuns = hostsSniffer.runs.get(); + //out of n concurrent threads trying to run Sniffer#sniff, only the first one will go through. + //That's why the total number of runs is less or equal to the number of threads. + //The important part is also that the assertion on no concurrent runs in the below CountingHostsSniffer never trips + assertThat(totalRuns, lessThanOrEqualTo(numThreads)); + int setHostsRuns = totalRuns - hostsSniffer.failures.get() - hostsSniffer.emptyList.get(); + verify(restClient, times(setHostsRuns)).setHosts(any(HttpHost.class)); + verifyNoMoreInteractions(restClient); + } + + /** + * Mock {@link HostsSniffer} implementation used for testing, which most of the times return a fixed host. + * It rarely throws exception or return an empty list of hosts, to make sure that such situations are properly handled. + * It also asserts that it never gets called concurrently, based on the assumption that only one sniff run can be run + * at a given point in time. + */ + private static class CountingHostsSniffer implements HostsSniffer { + private final AtomicBoolean running = new AtomicBoolean(false); + private final AtomicInteger runs = new AtomicInteger(0); + private final AtomicInteger failures = new AtomicInteger(0); + private final AtomicInteger emptyList = new AtomicInteger(0); + + @Override + public List sniffHosts() throws IOException { + //check that this method is never called concurrently + assertTrue(running.compareAndSet(false, true)); + try { + runs.incrementAndGet(); + if (rarely()) { + failures.incrementAndGet(); + //check that if communication breaks, sniffer keeps on working + throw new IOException("communication breakdown"); + } + if (rarely()) { + emptyList.incrementAndGet(); + return Collections.emptyList(); + } + return Collections.singletonList(new HttpHost("localhost", 9200)); + } finally { + assertTrue(running.compareAndSet(true, false)); + } + } + } + + @SuppressWarnings("unchecked") + public void testDefaultSchedulerSchedule() { + ScheduledExecutorService scheduledExecutorService = mock(ScheduledExecutorService.class); + ScheduledFuture scheduledFuture = mock(ScheduledFuture.class); + when(scheduledExecutorService.isShutdown()).thenReturn(false); + when(scheduledExecutorService.schedule(any(Runnable.class), any(Long.class), any(TimeUnit.class))).thenReturn(scheduledFuture); + + DefaultScheduler defaultScheduler = new DefaultScheduler(scheduledExecutorService); + Runnable runnable = new Runnable() { + @Override + public void run() { + + } + }; + { + long delayMillis = randomLongBetween(0L, Long.MAX_VALUE); + defaultScheduler.schedule(runnable, delayMillis); + verify(scheduledExecutorService).isShutdown(); + verify(scheduledExecutorService).schedule(runnable, delayMillis, TimeUnit.MILLISECONDS); + verifyNoMoreInteractions(scheduledExecutorService); + verifyNoMoreInteractions(scheduledFuture); + } + { + long delayMillis = randomLongBetween(0L, Long.MAX_VALUE); + defaultScheduler.schedule(runnable, delayMillis); + verify(scheduledExecutorService, times(2)).isShutdown(); + verify(scheduledExecutorService).schedule(runnable, delayMillis, TimeUnit.MILLISECONDS); + verifyNoMoreInteractions(scheduledExecutorService); + verify(scheduledFuture).cancel(false); + verifyNoMoreInteractions(scheduledFuture); + } + { + when(scheduledExecutorService.schedule(any(Runnable.class), any(Long.class), any(TimeUnit.class))) + .thenThrow(new IllegalArgumentException()); + long delayMillis = randomLongBetween(0L, Long.MAX_VALUE); + defaultScheduler.schedule(runnable, delayMillis); + verify(scheduledExecutorService, times(3)).isShutdown(); + verify(scheduledExecutorService).schedule(runnable, delayMillis, TimeUnit.MILLISECONDS); + verifyNoMoreInteractions(scheduledExecutorService); + verify(scheduledFuture, times(2)).cancel(false); + verifyNoMoreInteractions(scheduledFuture); + } + { + when(scheduledExecutorService.isShutdown()).thenReturn(true); + long delayMillis = randomLongBetween(0L, Long.MAX_VALUE); + defaultScheduler.schedule(runnable, delayMillis); + verify(scheduledExecutorService, times(4)).isShutdown(); + verifyNoMoreInteractions(scheduledExecutorService); + verifyNoMoreInteractions(scheduledFuture); + } + } + + public void testDefaultSchedulerThreadFactory() { + DefaultScheduler defaultScheduler = new DefaultScheduler(); + assertThat(defaultScheduler.scheduledExecutorService, instanceOf(ScheduledThreadPoolExecutor.class)); + ScheduledThreadPoolExecutor scheduledThreadPoolExecutor = (ScheduledThreadPoolExecutor) defaultScheduler.scheduledExecutorService; + assertThat(scheduledThreadPoolExecutor.getThreadFactory(), instanceOf(Sniffer.SnifferThreadFactory.class)); + int iters = randomIntBetween(3, 10); + for (int i = 1; i <= iters; i++) { + Thread thread = scheduledThreadPoolExecutor.getThreadFactory().newThread(new Runnable() { + @Override + public void run() { + + } + }); + assertThat(thread.getName(), equalTo("es_rest_client_sniffer[T#" + i + "]")); + assertThat(thread.isDaemon(), is(true)); + } + } + + public void testDefaultSchedulerShutdown() throws Exception { + ScheduledExecutorService scheduledExecutorService = mock(ScheduledExecutorService.class); + DefaultScheduler defaultScheduler = new DefaultScheduler(scheduledExecutorService); + + defaultScheduler.shutdown(); + verify(scheduledExecutorService).shutdown(); + verify(scheduledExecutorService).awaitTermination(1000, TimeUnit.MILLISECONDS); + verify(scheduledExecutorService).shutdownNow(); + verifyNoMoreInteractions(scheduledExecutorService); + + when(scheduledExecutorService.awaitTermination(1000, TimeUnit.MILLISECONDS)).thenReturn(true); + defaultScheduler.shutdown(); + verify(scheduledExecutorService, times(2)).shutdown(); + verify(scheduledExecutorService, times(2)).awaitTermination(1000, TimeUnit.MILLISECONDS); + verifyNoMoreInteractions(scheduledExecutorService); + } + + //TODO test on failure intervals + + //TODO add a sniffer test against proper http server, or maybe just throw exception and trigger on failure + +} From 0d8a270b8b9671c7460f299279803f779090b1a4 Mon Sep 17 00:00:00 2001 From: javanna Date: Tue, 10 Apr 2018 16:29:36 +0200 Subject: [PATCH 02/13] some improvements --- .../client/sniff/SniffOnFailureListener.java | 3 +- .../elasticsearch/client/sniff/Sniffer.java | 173 ++++++++++++------ .../client/sniff/SnifferTests.java | 4 +- 3 files changed, 119 insertions(+), 61 deletions(-) diff --git a/client/sniffer/src/main/java/org/elasticsearch/client/sniff/SniffOnFailureListener.java b/client/sniffer/src/main/java/org/elasticsearch/client/sniff/SniffOnFailureListener.java index cbc77351de98b..41051555bae2c 100644 --- a/client/sniffer/src/main/java/org/elasticsearch/client/sniff/SniffOnFailureListener.java +++ b/client/sniffer/src/main/java/org/elasticsearch/client/sniff/SniffOnFailureListener.java @@ -58,7 +58,6 @@ public void onFailure(HttpHost host) { if (sniffer == null) { throw new IllegalStateException("sniffer was not set, unable to sniff on failure"); } - //re-sniff immediately but take out the node that failed - sniffer.sniffOnFailure(host); + sniffer.sniffOnFailure(); } } diff --git a/client/sniffer/src/main/java/org/elasticsearch/client/sniff/Sniffer.java b/client/sniffer/src/main/java/org/elasticsearch/client/sniff/Sniffer.java index 65239dd008b49..891b302179091 100644 --- a/client/sniffer/src/main/java/org/elasticsearch/client/sniff/Sniffer.java +++ b/client/sniffer/src/main/java/org/elasticsearch/client/sniff/Sniffer.java @@ -29,9 +29,11 @@ import java.security.AccessController; import java.security.PrivilegedAction; import java.util.List; +import java.util.concurrent.Delayed; import java.util.concurrent.Executors; import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.ScheduledFuture; +import java.util.concurrent.ScheduledThreadPoolExecutor; import java.util.concurrent.ThreadFactory; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; @@ -69,48 +71,40 @@ public class Sniffer implements Closeable { this.sniffIntervalMillis = sniffInterval; this.sniffAfterFailureDelayMillis = sniffAfterFailureDelay; this.scheduler = scheduler; - scheduleNextRun(0); - } - - private void scheduleNextRun(long delayMillis) { - scheduler.schedule(new Runnable() { - @Override - public void run() { - sniff(null, sniffIntervalMillis); - } - }, delayMillis); + scheduler.submit(createRunnable(sniffIntervalMillis)); } /** - * Triggers a new sniffing round and explicitly takes out the failed host provided as argument + * Triggers a new immediate sniffing round, which will schedule a new round in sniffAfterFailureDelayMillis ms */ - public void sniffOnFailure(HttpHost failedHost) { - sniff(failedHost, sniffAfterFailureDelayMillis); + public final void sniffOnFailure() { + scheduler.submit(createRunnable(sniffAfterFailureDelayMillis)); } - private void sniff(HttpHost excludeHost, long nextSniffDelayMillis) { - //If a sniffing round is already running nothing happens, it makes no sense to start or wait to start another round - if (running.compareAndSet(false, true)) { - try { - List sniffedHosts = hostsSniffer.sniffHosts(); - logger.debug("sniffed hosts: " + sniffedHosts); - if (excludeHost != null) { - sniffedHosts.remove(excludeHost); - } - if (sniffedHosts.isEmpty()) { - logger.warn("no hosts to set, hosts will be updated at the next sniffing round"); - } else { - this.restClient.setHosts(sniffedHosts.toArray(new HttpHost[sniffedHosts.size()])); + private Runnable createRunnable(final long nextSniffDelayMillis) { + assert nextSniffDelayMillis > 0 : "delay must be greater than 0 when scheduling a task"; + return new Runnable() { + @Override + public void run() { + assert running.compareAndSet(false, true) : "multiple sniff rounds must not be executed in parallel, that should " + + "be guaranteed by using a single threaded executor"; + try { + List sniffedHosts = hostsSniffer.sniffHosts(); + logger.debug("sniffed hosts: " + sniffedHosts); + if (sniffedHosts.isEmpty()) { + logger.warn("no hosts to set, hosts will be updated at the next sniffing round"); + } else { + restClient.setHosts(sniffedHosts.toArray(new HttpHost[sniffedHosts.size()])); + } + } catch (Exception e) { + logger.error("error while sniffing nodes", e); + } finally { + //schedule ordinary sniff run, will happen unless sniffing on failure kicks in first + scheduler.schedule(createRunnable(sniffIntervalMillis), nextSniffDelayMillis); + assert running.compareAndSet(true, false); } - } catch (Exception e) { - logger.error("error while sniffing nodes", e); - } finally { - //TODO potential problem here if this doesn't happen last? though tests become complicated - running.set(false); - //TODO do we want to also test that this never gets called concurrently? - scheduleNextRun(nextSniffDelayMillis); } - } + }; } @Override @@ -134,7 +128,12 @@ public static SnifferBuilder builder(RestClient restClient) { */ interface Scheduler { /** - * Schedules the provided {@link Runnable} to run in delayMillis milliseconds + * Schedules the provided {@link Runnable} to be executed straight-away + */ + void submit(Runnable runnable); + + /** + * Schedules the provided {@link Runnable} to be executed in delayMillis milliseconds */ void schedule(Runnable runnable, long delayMillis); @@ -148,51 +147,111 @@ interface Scheduler { * Default implementation of {@link Scheduler}, based on {@link ScheduledExecutorService} */ static final class DefaultScheduler implements Scheduler { - final ScheduledExecutorService scheduledExecutorService; - private ScheduledFuture scheduledFuture; + final ScheduledThreadPoolExecutor executor; + private ScheduledFuture scheduledFuture = DummyScheduledFuture.INSTANCE; DefaultScheduler() { - this(Executors.newScheduledThreadPool(1, new SnifferThreadFactory(SNIFFER_THREAD_NAME))); + this(initScheduledExecutorService()); + } + + DefaultScheduler(ScheduledThreadPoolExecutor executor) { + this.executor = executor; + } + + //TODO test this + static ScheduledThreadPoolExecutor initScheduledExecutorService() { + ScheduledThreadPoolExecutor executor = new ScheduledThreadPoolExecutor(1, new SnifferThreadFactory(SNIFFER_THREAD_NAME)); + executor.setRemoveOnCancelPolicy(true); + //TODO does this have any effect? + executor.setExecuteExistingDelayedTasksAfterShutdownPolicy(false); + return executor; } - DefaultScheduler(ScheduledExecutorService scheduledExecutorService) { - this.scheduledExecutorService = scheduledExecutorService; + //TODO can it happen that we get so many failures that we keep on cancelling without ever getting to execute the next round? + // no, because we add each failed node to the blacklist, and it stays there till setHosts is called, at the end of the sniff round + //may happen if we end up reviving nodes from the blacklist as all of the sniffed nodes are marked dead. + //in that case the sniff round won't work either though. + + //TODO this can be called concurrently when sniffing on failure, test concurrent calls + @Override + public synchronized void submit(Runnable runnable) { + ScheduledFuture scheduledFuture = this.scheduledFuture; + assert scheduledFuture != DummyScheduledFuture.INSTANCE + || executor.getTaskCount() == 0 : "next task may only be null at the very first run"; + //regardless of when the next sniff is scheduled, cancel it and schedule a new one. + //this happens when sniffing on failure is enabled and a failure happens, the following scheduled ordinary + //round gets cancelled in favour of the immediate round caused by the failure + scheduledFuture.cancel(false); + schedule(runnable, 0L); } - //TODO test concurrent calls to schedule? + //TODO test concurrent calls to schedule @Override public synchronized void schedule(Runnable runnable, long delayMillis) { - //TODO maybe this is not even needed, just let it throw rejected execution exception instead? - if (scheduledExecutorService.isShutdown() == false) { - try { - if (this.scheduledFuture != null) { - //regardless of when the next sniff is scheduled, cancel it and schedule a new one with the latest delay. - //instead of piling up sniff rounds to be run, the last run decides when the following execution will be. - this.scheduledFuture.cancel(false); - } - logger.debug("scheduling next sniff in " + delayMillis + " ms"); - this.scheduledFuture = scheduledExecutorService.schedule(runnable, delayMillis, TimeUnit.MILLISECONDS); - } catch(Exception e) { - logger.error("error while scheduling next sniffer task", e); - } + assert scheduledFuture.isDone() : "the task being replaced must either be running or cancelled"; + logger.debug("scheduling next sniff round in " + delayMillis + " ms"); + try { + this.scheduledFuture = executor.schedule(runnable, delayMillis, TimeUnit.MILLISECONDS); + } catch(Exception e) { + logger.error("error while scheduling next sniffer round", e); } } //TODO test concurrent calls to shutdown? @Override public synchronized void shutdown() { - scheduledExecutorService.shutdown(); + //TODO cancel here too, no need to try and wait + executor.shutdown(); try { - if (scheduledExecutorService.awaitTermination(1000, TimeUnit.MILLISECONDS)) { + if (executor.awaitTermination(1000, TimeUnit.MILLISECONDS)) { return; } - scheduledExecutorService.shutdownNow(); + executor.shutdownNow(); } catch (InterruptedException e) { Thread.currentThread().interrupt(); } } } + private static class DummyScheduledFuture implements ScheduledFuture { + private static final DummyScheduledFuture INSTANCE = new DummyScheduledFuture(); + + @Override + public long getDelay(TimeUnit unit) { + return 0; + } + + @Override + public int compareTo(Delayed o) { + return 0; + } + + @Override + public boolean cancel(boolean mayInterruptIfRunning) { + return true; + } + + @Override + public boolean isCancelled() { + return false; + } + + @Override + public boolean isDone() { + return true; + } + + @Override + public Object get() { + return null; + } + + @Override + public Object get(long timeout, TimeUnit unit) { + return null; + } + } + static class SnifferThreadFactory implements ThreadFactory { private final AtomicInteger threadNumber = new AtomicInteger(1); private final String namePrefix; diff --git a/client/sniffer/src/test/java/org/elasticsearch/client/sniff/SnifferTests.java b/client/sniffer/src/test/java/org/elasticsearch/client/sniff/SnifferTests.java index 8a90c098340fc..93d7605e0a407 100644 --- a/client/sniffer/src/test/java/org/elasticsearch/client/sniff/SnifferTests.java +++ b/client/sniffer/src/test/java/org/elasticsearch/client/sniff/SnifferTests.java @@ -272,8 +272,8 @@ public void run() { public void testDefaultSchedulerThreadFactory() { DefaultScheduler defaultScheduler = new DefaultScheduler(); - assertThat(defaultScheduler.scheduledExecutorService, instanceOf(ScheduledThreadPoolExecutor.class)); - ScheduledThreadPoolExecutor scheduledThreadPoolExecutor = (ScheduledThreadPoolExecutor) defaultScheduler.scheduledExecutorService; + assertThat(defaultScheduler.executor, instanceOf(ScheduledThreadPoolExecutor.class)); + ScheduledThreadPoolExecutor scheduledThreadPoolExecutor = (ScheduledThreadPoolExecutor) defaultScheduler.executor; assertThat(scheduledThreadPoolExecutor.getThreadFactory(), instanceOf(Sniffer.SnifferThreadFactory.class)); int iters = randomIntBetween(3, 10); for (int i = 1; i <= iters; i++) { From c9ec5a87e780af549cf15cb17fca720520c0a0ee Mon Sep 17 00:00:00 2001 From: javanna Date: Thu, 12 Apr 2018 12:41:38 +0200 Subject: [PATCH 03/13] tests rewritten --- .../client/sniff/SnifferTests.java | 262 ++++++++++++------ 1 file changed, 175 insertions(+), 87 deletions(-) diff --git a/client/sniffer/src/test/java/org/elasticsearch/client/sniff/SnifferTests.java b/client/sniffer/src/test/java/org/elasticsearch/client/sniff/SnifferTests.java index 93d7605e0a407..4a0a8882b824a 100644 --- a/client/sniffer/src/test/java/org/elasticsearch/client/sniff/SnifferTests.java +++ b/client/sniffer/src/test/java/org/elasticsearch/client/sniff/SnifferTests.java @@ -29,10 +29,10 @@ import java.util.Collections; import java.util.List; import java.util.concurrent.CountDownLatch; +import java.util.concurrent.ExecutionException; import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; -import java.util.concurrent.ScheduledExecutorService; -import java.util.concurrent.ScheduledFuture; +import java.util.concurrent.Future; import java.util.concurrent.ScheduledThreadPoolExecutor; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; @@ -40,7 +40,9 @@ import static org.hamcrest.CoreMatchers.equalTo; import static org.hamcrest.CoreMatchers.instanceOf; +import static org.hamcrest.Matchers.greaterThan; import static org.hamcrest.Matchers.is; +import static org.hamcrest.Matchers.lessThan; import static org.hamcrest.Matchers.lessThanOrEqualTo; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertThat; @@ -50,7 +52,6 @@ import static org.mockito.Mockito.times; import static org.mockito.Mockito.verify; import static org.mockito.Mockito.verifyNoMoreInteractions; -import static org.mockito.Mockito.when; public class SnifferTests extends RestClientTestCase { @@ -61,41 +62,45 @@ public class SnifferTests extends RestClientTestCase { * The {@link Scheduler} implementation doesn't respect requested sniff delays but rather immediately runs them while * allowing to assert that the requested delay for each schedule run is the expected one. */ - public void testOrdinarySniffingRounds() throws Exception { + public void testOrdinarySniffRounds() throws Exception { final long sniffInterval = randomLongBetween(1, Long.MAX_VALUE); long sniffAfterFailureDelay = randomLongBetween(1, Long.MAX_VALUE); RestClient restClient = mock(RestClient.class); CountingHostsSniffer hostsSniffer = new CountingHostsSniffer(); final int iters = randomIntBetween(30, 100); final CountDownLatch latch = new CountDownLatch(1); + final AtomicBoolean submitCalled = new AtomicBoolean(false); final AtomicInteger runs = new AtomicInteger(iters); final ExecutorService executor = Executors.newSingleThreadExecutor(); - try { - Scheduler scheduler = new Scheduler() { - @Override - public void schedule(Runnable runnable, long delayMillis) { - int numberOfRuns = runs.getAndDecrement(); - if (numberOfRuns == 0) { - latch.countDown(); - return; - } - if (numberOfRuns == iters) { - //the first time "schedule" gets called from the Sniffer constructor with delay set to 0 - assertEquals(0L, delayMillis); - } else { - //all of the subsequent times "schedule" is called with delay set to the configured sniff interval - assertEquals(sniffInterval, delayMillis); - } - //we immediately run rather than scheduling - executor.execute(runnable); - } + Scheduler scheduler = new Scheduler() { + @Override + public void submit(Runnable runnable) { + //the first call is to "submit" the first sniff round from the Sniffer constructor + assertTrue(submitCalled.compareAndSet(false, true)); + assertEquals(iters, runs.getAndDecrement()); + executor.execute(runnable); + } - @Override - public void shutdown() { - //the executor is closed externally, shutdown is tested separately + @Override + public void schedule(Runnable runnable, long delayMillis) { + int numberOfRuns = runs.getAndDecrement(); + if (numberOfRuns == 0) { + latch.countDown(); + return; } - }; + assertThat(numberOfRuns, lessThan(iters)); + //all of the subsequent times "schedule" is called with delay set to the configured sniff interval + assertEquals(sniffInterval, delayMillis); + //we immediately run rather than scheduling + executor.execute(runnable); + } + @Override + public void shutdown() { + //the executor is closed externally, shutdown is tested separately + } + }; + try { //all we need to do is initialize the sniffer, sniffing will start automatically in the background new Sniffer(restClient, hostsSniffer, scheduler, sniffInterval, sniffAfterFailureDelay); assertTrue(latch.await(1000, TimeUnit.MILLISECONDS)); @@ -119,6 +124,11 @@ public void testClose() { RestClient restClient = mock(RestClient.class); final AtomicInteger shutdown = new AtomicInteger(0); Scheduler scheduler = new Scheduler() { + @Override + public void submit(Runnable runnable) { + + } + @Override public void schedule(Runnable runnable, long delayMillis) { } @@ -139,8 +149,8 @@ public void shutdown() { } /** - * Test concurrent calls to Sniffer#sniff. They may happen if you set a super low sniffInterval - * and/or the {@link HostsSniffer} implementation is very slow. + * Test concurrent calls to the sniffing code. They should not happen as we are using a single threaded executor. + * This test makes sure that concurrent calls are not supported and such assumption is enforced. */ public void testConcurrentSniffRounds() throws Exception { long sniffInterval = randomLongBetween(1, Long.MAX_VALUE); @@ -150,20 +160,22 @@ public void testConcurrentSniffRounds() throws Exception { final int numThreads = randomIntBetween(10, 30); final ExecutorService executor = Executors.newFixedThreadPool(numThreads); - final AtomicBoolean firstRun = new AtomicBoolean(true); - + final Future[] futures = new Future[numThreads]; try { Scheduler scheduler = new Scheduler() { @Override - public void schedule(Runnable runnable, long delayMillis) { - if (firstRun.compareAndSet(true, false)) { - for (int i = 0; i < numThreads; i++) { - //this will only run the same runnable n times, to simulate concurrent calls to the sniff method - executor.submit(runnable); - } + public void submit(Runnable runnable) { + for (int i = 0; i < numThreads; i++) { + //this will only submit the same runnable n times, to simulate concurrent calls to the sniffing code + futures[i] = executor.submit(runnable); } } + @Override + public void schedule(Runnable runnable, long delayMillis) { + //do nothing + } + @Override public void shutdown() { } @@ -174,16 +186,79 @@ public void shutdown() { executor.awaitTermination(1000, TimeUnit.MILLISECONDS); } + int failures = 0; + int successfulRuns = 0; + for (Future future : futures) { + try { + future.get(); + successfulRuns++; + } catch (ExecutionException e) { + if (e.getCause() instanceof UnsupportedOperationException) { + failures++; + } else { + throw e; + } + } + } + int totalRuns = hostsSniffer.runs.get(); - //out of n concurrent threads trying to run Sniffer#sniff, only the first one will go through. + //out of n concurrent threads trying to run a sniff round, two will never be run in parallel. //That's why the total number of runs is less or equal to the number of threads. //The important part is also that the assertion on no concurrent runs in the below CountingHostsSniffer never trips assertThat(totalRuns, lessThanOrEqualTo(numThreads)); + assertThat(successfulRuns, greaterThan(0)); + //the successful runs that we counted match with the hosts sniffer calls + assertThat(totalRuns, equalTo(successfulRuns)); + //and the failures that we counted due to the another thread already sniffing are the rest of the threads + assertThat(numThreads - totalRuns, equalTo(failures)); + //also check out how many times setHosts is called on the underlying client int setHostsRuns = totalRuns - hostsSniffer.failures.get() - hostsSniffer.emptyList.get(); verify(restClient, times(setHostsRuns)).setHosts(any(HttpHost.class)); verifyNoMoreInteractions(restClient); } + public void testSniffOnFailure() throws Exception { + RestClient restClient = mock(RestClient.class); + CountingHostsSniffer hostsSniffer = new CountingHostsSniffer(); + final long sniffInterval = randomLongBetween(1, Long.MAX_VALUE); + long sniffAfterFailureDelay = randomLongBetween(1, Long.MAX_VALUE); + final ExecutorService executor = Executors.newSingleThreadExecutor(); + final AtomicInteger submitRuns = new AtomicInteger(0); + final AtomicBoolean sniffingOnFailure = new AtomicBoolean(false); + Scheduler scheduler = new Scheduler() { + @Override + public void submit(Runnable runnable) { + submitRuns.incrementAndGet(); + } + + @Override + public void schedule(Runnable runnable, long delayMillis) { + + } + + @Override + public void shutdown() { + + } + }; + int numThreads = randomIntBetween(10, 20); + final ExecutorService onFailureExecutor = Executors.newFixedThreadPool(numThreads); + try { + final Sniffer sniffer = new Sniffer(restClient, hostsSniffer, scheduler, sniffInterval, sniffAfterFailureDelay); + for (int i = 0; i < numThreads; i++) { + onFailureExecutor.execute(new Runnable() { + @Override + public void run() { + sniffer.sniffOnFailure(); + } + }); + } + } finally { + executor.shutdown(); + executor.awaitTermination(1000, TimeUnit.MILLISECONDS); + } + } + /** * Mock {@link HostsSniffer} implementation used for testing, which most of the times return a fixed host. * It rarely throws exception or return an empty list of hosts, to make sure that such situations are properly handled. @@ -218,92 +293,105 @@ public List sniffHosts() throws IOException { } } + //TODO adapt this test @SuppressWarnings("unchecked") +/* public void testDefaultSchedulerSchedule() { - ScheduledExecutorService scheduledExecutorService = mock(ScheduledExecutorService.class); + ScheduledThreadPoolExecutor executor = mock(ScheduledThreadPoolExecutor.class); ScheduledFuture scheduledFuture = mock(ScheduledFuture.class); - when(scheduledExecutorService.isShutdown()).thenReturn(false); - when(scheduledExecutorService.schedule(any(Runnable.class), any(Long.class), any(TimeUnit.class))).thenReturn(scheduledFuture); + when(executor.isShutdown()).thenReturn(false); + when(executor.schedule(any(Runnable.class), any(Long.class), any(TimeUnit.class))).thenReturn(scheduledFuture); - DefaultScheduler defaultScheduler = new DefaultScheduler(scheduledExecutorService); + DefaultScheduler defaultScheduler = new DefaultScheduler(executor); Runnable runnable = new Runnable() { @Override public void run() { } }; + int iters = randomIntBetween(3, 10); { - long delayMillis = randomLongBetween(0L, Long.MAX_VALUE); - defaultScheduler.schedule(runnable, delayMillis); - verify(scheduledExecutorService).isShutdown(); - verify(scheduledExecutorService).schedule(runnable, delayMillis, TimeUnit.MILLISECONDS); - verifyNoMoreInteractions(scheduledExecutorService); - verifyNoMoreInteractions(scheduledFuture); - } - { - long delayMillis = randomLongBetween(0L, Long.MAX_VALUE); - defaultScheduler.schedule(runnable, delayMillis); - verify(scheduledExecutorService, times(2)).isShutdown(); - verify(scheduledExecutorService).schedule(runnable, delayMillis, TimeUnit.MILLISECONDS); - verifyNoMoreInteractions(scheduledExecutorService); - verify(scheduledFuture).cancel(false); - verifyNoMoreInteractions(scheduledFuture); + for (int i = 1; i <= iters; i++) { + long delayMillis = randomLongBetween(0L, Long.MAX_VALUE); + defaultScheduler.schedule(runnable, delayMillis); + verify(executor, times(i)).schedule(runnable, delayMillis, TimeUnit.MILLISECONDS); + verifyNoMoreInteractions(executor); + verify(scheduledFuture, times(i)).cancel(false); + verifyNoMoreInteractions(scheduledFuture); + } } { - when(scheduledExecutorService.schedule(any(Runnable.class), any(Long.class), any(TimeUnit.class))) + when(executor.schedule(any(Runnable.class), any(Long.class), any(TimeUnit.class))) .thenThrow(new IllegalArgumentException()); long delayMillis = randomLongBetween(0L, Long.MAX_VALUE); defaultScheduler.schedule(runnable, delayMillis); - verify(scheduledExecutorService, times(3)).isShutdown(); - verify(scheduledExecutorService).schedule(runnable, delayMillis, TimeUnit.MILLISECONDS); - verifyNoMoreInteractions(scheduledExecutorService); + verify(executor).schedule(runnable, delayMillis, TimeUnit.MILLISECONDS); + verifyNoMoreInteractions(executor); verify(scheduledFuture, times(2)).cancel(false); verifyNoMoreInteractions(scheduledFuture); } { - when(scheduledExecutorService.isShutdown()).thenReturn(true); + when(executor.isShutdown()).thenReturn(true); long delayMillis = randomLongBetween(0L, Long.MAX_VALUE); defaultScheduler.schedule(runnable, delayMillis); - verify(scheduledExecutorService, times(4)).isShutdown(); - verifyNoMoreInteractions(scheduledExecutorService); + verifyNoMoreInteractions(executor); verifyNoMoreInteractions(scheduledFuture); } } +*/ public void testDefaultSchedulerThreadFactory() { DefaultScheduler defaultScheduler = new DefaultScheduler(); - assertThat(defaultScheduler.executor, instanceOf(ScheduledThreadPoolExecutor.class)); - ScheduledThreadPoolExecutor scheduledThreadPoolExecutor = (ScheduledThreadPoolExecutor) defaultScheduler.executor; - assertThat(scheduledThreadPoolExecutor.getThreadFactory(), instanceOf(Sniffer.SnifferThreadFactory.class)); - int iters = randomIntBetween(3, 10); - for (int i = 1; i <= iters; i++) { - Thread thread = scheduledThreadPoolExecutor.getThreadFactory().newThread(new Runnable() { - @Override - public void run() { + try { + ScheduledThreadPoolExecutor scheduledThreadPoolExecutor = defaultScheduler.executor; + assertThat(scheduledThreadPoolExecutor.getThreadFactory(), instanceOf(Sniffer.SnifferThreadFactory.class)); + int iters = randomIntBetween(3, 10); + for (int i = 1; i <= iters; i++) { + Thread thread = scheduledThreadPoolExecutor.getThreadFactory().newThread(new Runnable() { + @Override + public void run() { - } - }); - assertThat(thread.getName(), equalTo("es_rest_client_sniffer[T#" + i + "]")); - assertThat(thread.isDaemon(), is(true)); + } + }); + assertThat(thread.getName(), equalTo("es_rest_client_sniffer[T#" + i + "]")); + assertThat(thread.isDaemon(), is(true)); + } + } finally { + defaultScheduler.shutdown(); } } + //TODO adapt this, it's not a good test though +/* @SuppressWarnings("unchecked") public void testDefaultSchedulerShutdown() throws Exception { - ScheduledExecutorService scheduledExecutorService = mock(ScheduledExecutorService.class); - DefaultScheduler defaultScheduler = new DefaultScheduler(scheduledExecutorService); + ScheduledThreadPoolExecutor executor = mock(ScheduledThreadPoolExecutor.class); + when(executor.isShutdown()).thenReturn(false); + ScheduledFuture scheduledFuture = mock(ScheduledFuture.class); + when(executor.schedule(any(Runnable.class), any(Long.class), any(TimeUnit.class))).thenReturn(scheduledFuture); + DefaultScheduler defaultScheduler = new DefaultScheduler(executor); + + defaultScheduler.schedule(new Runnable() { + @Override + public void run() { + + } + }, 0L); + verify(executor.schedule(any(Runnable.class), any(Long.class), any(TimeUnit.class))); defaultScheduler.shutdown(); - verify(scheduledExecutorService).shutdown(); - verify(scheduledExecutorService).awaitTermination(1000, TimeUnit.MILLISECONDS); - verify(scheduledExecutorService).shutdownNow(); - verifyNoMoreInteractions(scheduledExecutorService); + verify(scheduledFuture).cancel(false); + verify(executor).shutdown(); + verify(executor).awaitTermination(1000, TimeUnit.MILLISECONDS); + verify(executor).shutdownNow(); + verifyNoMoreInteractions(executor, scheduledFuture); - when(scheduledExecutorService.awaitTermination(1000, TimeUnit.MILLISECONDS)).thenReturn(true); + when(executor.awaitTermination(1000, TimeUnit.MILLISECONDS)).thenReturn(true); defaultScheduler.shutdown(); - verify(scheduledExecutorService, times(2)).shutdown(); - verify(scheduledExecutorService, times(2)).awaitTermination(1000, TimeUnit.MILLISECONDS); - verifyNoMoreInteractions(scheduledExecutorService); - } + verify(scheduledFuture, times(2)).cancel(false); + verify(executor, times(2)).shutdown(); + verify(executor, times(2)).awaitTermination(1000, TimeUnit.MILLISECONDS); + verifyNoMoreInteractions(executor, scheduledFuture); + }*/ //TODO test on failure intervals From 4c886a0e767309cec327f557be5b58e6f275a7bb Mon Sep 17 00:00:00 2001 From: javanna Date: Fri, 20 Apr 2018 10:23:28 +0200 Subject: [PATCH 04/13] refactor and add tests --- .../org/elasticsearch/client/RestClient.java | 7 + .../elasticsearch/client/sniff/Sniffer.java | 167 +++---- .../client/sniff/SnifferTests.java | 413 +++++++++++++----- 3 files changed, 371 insertions(+), 216 deletions(-) diff --git a/client/rest/src/main/java/org/elasticsearch/client/RestClient.java b/client/rest/src/main/java/org/elasticsearch/client/RestClient.java index 48349c3858938..c57daecc29ca5 100644 --- a/client/rest/src/main/java/org/elasticsearch/client/RestClient.java +++ b/client/rest/src/main/java/org/elasticsearch/client/RestClient.java @@ -143,6 +143,13 @@ public synchronized void setHosts(HttpHost... hosts) { this.blacklist.clear(); } + /** + * Returns the configured hosts + */ + public List getHosts() { + return new ArrayList<>(hostTuple.hosts); + } + /** * Sends a request to the Elasticsearch cluster that the client points to and waits for the corresponding response * to be returned. Shortcut to {@link #performRequest(String, String, Map, HttpEntity, Header...)} but without parameters diff --git a/client/sniffer/src/main/java/org/elasticsearch/client/sniff/Sniffer.java b/client/sniffer/src/main/java/org/elasticsearch/client/sniff/Sniffer.java index 891b302179091..5f75474440104 100644 --- a/client/sniffer/src/main/java/org/elasticsearch/client/sniff/Sniffer.java +++ b/client/sniffer/src/main/java/org/elasticsearch/client/sniff/Sniffer.java @@ -26,11 +26,12 @@ import org.elasticsearch.client.RestClientBuilder; import java.io.Closeable; +import java.io.IOException; import java.security.AccessController; import java.security.PrivilegedAction; import java.util.List; -import java.util.concurrent.Delayed; import java.util.concurrent.Executors; +import java.util.concurrent.Future; import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.ScheduledFuture; import java.util.concurrent.ScheduledThreadPoolExecutor; @@ -38,6 +39,7 @@ import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicInteger; +import java.util.concurrent.atomic.AtomicReference; /** * Class responsible for sniffing nodes from some source (default is elasticsearch itself) and setting them to a provided instance of @@ -60,6 +62,7 @@ public class Sniffer implements Closeable { private final Scheduler scheduler; private final AtomicBoolean running = new AtomicBoolean(false); + private final AtomicReference nextTask = new AtomicReference<>(); Sniffer(RestClient restClient, HostsSniffer hostsSniffer, long sniffInterval, long sniffAfterFailureDelay) { this(restClient, hostsSniffer, new DefaultScheduler(), sniffInterval, sniffAfterFailureDelay); @@ -71,44 +74,78 @@ public class Sniffer implements Closeable { this.sniffIntervalMillis = sniffInterval; this.sniffAfterFailureDelayMillis = sniffAfterFailureDelay; this.scheduler = scheduler; - scheduler.submit(createRunnable(sniffIntervalMillis)); + //first sniffing round is immediately executed, next one will be executed depending on the configured sniff interval + scheduleNextRound(0L, sniffIntervalMillis, false); } /** * Triggers a new immediate sniffing round, which will schedule a new round in sniffAfterFailureDelayMillis ms */ public final void sniffOnFailure() { - scheduler.submit(createRunnable(sniffAfterFailureDelayMillis)); + scheduleNextRound(0L, sniffAfterFailureDelayMillis, true); } - private Runnable createRunnable(final long nextSniffDelayMillis) { - assert nextSniffDelayMillis > 0 : "delay must be greater than 0 when scheduling a task"; - return new Runnable() { - @Override - public void run() { - assert running.compareAndSet(false, true) : "multiple sniff rounds must not be executed in parallel, that should " + - "be guaranteed by using a single threaded executor"; - try { - List sniffedHosts = hostsSniffer.sniffHosts(); - logger.debug("sniffed hosts: " + sniffedHosts); - if (sniffedHosts.isEmpty()) { - logger.warn("no hosts to set, hosts will be updated at the next sniffing round"); - } else { - restClient.setHosts(sniffedHosts.toArray(new HttpHost[sniffedHosts.size()])); - } - } catch (Exception e) { - logger.error("error while sniffing nodes", e); - } finally { - //schedule ordinary sniff run, will happen unless sniffing on failure kicks in first - scheduler.schedule(createRunnable(sniffIntervalMillis), nextSniffDelayMillis); - assert running.compareAndSet(true, false); - } + //TODO test concurrency on this method + private void scheduleNextRound(long delay, long nextDelay, boolean mustCancelNextRound) { + Task task = new Task(nextDelay); + ScheduledTask scheduledTask = task.schedule(delay); + assert scheduledTask.task == task; + ScheduledTask previousTask = nextTask.getAndSet(scheduledTask); + if (mustCancelNextRound) { + previousTask.cancelIfNotYetStarted(); + } + } + + final class Task implements Runnable { + final long nextTaskDelay; + + Task(long nextTaskDelay) { + this.nextTaskDelay = nextTaskDelay; + } + + ScheduledTask schedule(long delay) { + return scheduler.schedule(this, delay); + } + + @Override + public void run() { + try { + sniff(); + } catch (Exception e) { + logger.error("error while sniffing nodes", e); + } finally { + scheduleNextRound(nextTaskDelay, sniffIntervalMillis, false); } - }; + } + } + + static final class ScheduledTask { + final Task task; + final Future future; + + ScheduledTask(Task task, Future future) { + this.task = task; + this.future = future; + } + + void cancelIfNotYetStarted() { + this.future.cancel(false); + } + } + + final void sniff() throws IOException { + List sniffedHosts = hostsSniffer.sniffHosts(); + logger.debug("sniffed hosts: " + sniffedHosts); + if (sniffedHosts.isEmpty()) { + logger.warn("no hosts to set, hosts will be updated at the next sniffing round"); + } else { + restClient.setHosts(sniffedHosts.toArray(new HttpHost[sniffedHosts.size()])); + } } @Override public void close() { + nextTask.get().cancelIfNotYetStarted(); this.scheduler.shutdown(); } @@ -124,18 +161,13 @@ public static SnifferBuilder builder(RestClient restClient) { /** * The Scheduler interface allows to isolate the sniffing scheduling aspects so that we can test - * the sniffer by injecting a custom scheduler that is more suited for testing. + * the sniffer by injecting when needed a custom scheduler that is more suited for testing. */ interface Scheduler { - /** - * Schedules the provided {@link Runnable} to be executed straight-away - */ - void submit(Runnable runnable); - /** * Schedules the provided {@link Runnable} to be executed in delayMillis milliseconds */ - void schedule(Runnable runnable, long delayMillis); + ScheduledTask schedule(Task task, long delayMillis); /** * Shuts this scheduler down @@ -148,7 +180,6 @@ interface Scheduler { */ static final class DefaultScheduler implements Scheduler { final ScheduledThreadPoolExecutor executor; - private ScheduledFuture scheduledFuture = DummyScheduledFuture.INSTANCE; DefaultScheduler() { this(initScheduledExecutorService()); @@ -172,35 +203,14 @@ static ScheduledThreadPoolExecutor initScheduledExecutorService() { //may happen if we end up reviving nodes from the blacklist as all of the sniffed nodes are marked dead. //in that case the sniff round won't work either though. - //TODO this can be called concurrently when sniffing on failure, test concurrent calls - @Override - public synchronized void submit(Runnable runnable) { - ScheduledFuture scheduledFuture = this.scheduledFuture; - assert scheduledFuture != DummyScheduledFuture.INSTANCE - || executor.getTaskCount() == 0 : "next task may only be null at the very first run"; - //regardless of when the next sniff is scheduled, cancel it and schedule a new one. - //this happens when sniffing on failure is enabled and a failure happens, the following scheduled ordinary - //round gets cancelled in favour of the immediate round caused by the failure - scheduledFuture.cancel(false); - schedule(runnable, 0L); - } - - //TODO test concurrent calls to schedule @Override - public synchronized void schedule(Runnable runnable, long delayMillis) { - assert scheduledFuture.isDone() : "the task being replaced must either be running or cancelled"; - logger.debug("scheduling next sniff round in " + delayMillis + " ms"); - try { - this.scheduledFuture = executor.schedule(runnable, delayMillis, TimeUnit.MILLISECONDS); - } catch(Exception e) { - logger.error("error while scheduling next sniffer round", e); - } + public ScheduledTask schedule(Task task, long delayMillis) { + ScheduledFuture future = executor.schedule(task, delayMillis, TimeUnit.MILLISECONDS); + return new ScheduledTask(task, future); } - //TODO test concurrent calls to shutdown? @Override - public synchronized void shutdown() { - //TODO cancel here too, no need to try and wait + public void shutdown() { executor.shutdown(); try { if (executor.awaitTermination(1000, TimeUnit.MILLISECONDS)) { @@ -213,45 +223,6 @@ public synchronized void shutdown() { } } - private static class DummyScheduledFuture implements ScheduledFuture { - private static final DummyScheduledFuture INSTANCE = new DummyScheduledFuture(); - - @Override - public long getDelay(TimeUnit unit) { - return 0; - } - - @Override - public int compareTo(Delayed o) { - return 0; - } - - @Override - public boolean cancel(boolean mayInterruptIfRunning) { - return true; - } - - @Override - public boolean isCancelled() { - return false; - } - - @Override - public boolean isDone() { - return true; - } - - @Override - public Object get() { - return null; - } - - @Override - public Object get(long timeout, TimeUnit unit) { - return null; - } - } - static class SnifferThreadFactory implements ThreadFactory { private final AtomicInteger threadNumber = new AtomicInteger(1); private final String namePrefix; diff --git a/client/sniffer/src/test/java/org/elasticsearch/client/sniff/SnifferTests.java b/client/sniffer/src/test/java/org/elasticsearch/client/sniff/SnifferTests.java index 4a0a8882b824a..59a4c45118ec2 100644 --- a/client/sniffer/src/test/java/org/elasticsearch/client/sniff/SnifferTests.java +++ b/client/sniffer/src/test/java/org/elasticsearch/client/sniff/SnifferTests.java @@ -24,12 +24,15 @@ import org.elasticsearch.client.RestClientTestCase; import org.elasticsearch.client.sniff.Sniffer.DefaultScheduler; import org.elasticsearch.client.sniff.Sniffer.Scheduler; +import org.mockito.invocation.InvocationOnMock; +import org.mockito.stubbing.Answer; import java.io.IOException; import java.util.Collections; import java.util.List; +import java.util.concurrent.CancellationException; +import java.util.concurrent.CopyOnWriteArrayList; import java.util.concurrent.CountDownLatch; -import java.util.concurrent.ExecutionException; import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; import java.util.concurrent.Future; @@ -37,30 +40,106 @@ import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicInteger; +import java.util.concurrent.atomic.AtomicReference; import static org.hamcrest.CoreMatchers.equalTo; import static org.hamcrest.CoreMatchers.instanceOf; -import static org.hamcrest.Matchers.greaterThan; import static org.hamcrest.Matchers.is; -import static org.hamcrest.Matchers.lessThan; -import static org.hamcrest.Matchers.lessThanOrEqualTo; import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertThat; import static org.junit.Assert.assertTrue; +import static org.junit.Assert.fail; import static org.mockito.Matchers.any; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.times; import static org.mockito.Mockito.verify; import static org.mockito.Mockito.verifyNoMoreInteractions; +import static org.mockito.Mockito.when; public class SnifferTests extends RestClientTestCase { /** - * Test multiple sniffing rounds by mocking the {@link Scheduler} as well as - * the {@link HostsSniffer}. - * The {@link CountingHostsSniffer} doesn't make any connection but may throw exception or return no hosts. - * The {@link Scheduler} implementation doesn't respect requested sniff delays but rather immediately runs them while - * allowing to assert that the requested delay for each schedule run is the expected one. + * Tests the {@link Sniffer#sniff()} method in isolation. Verifies that it uses the {@link HostsSniffer} implementation + * to retrieve nodes and set them (when not empty) to the provided {@link RestClient} instance. + */ + public void testSniff() throws IOException { + try (RestClient restClient = RestClient.builder(new HttpHost("localhost", 9200)).build()) { + Scheduler noOpScheduler = new Scheduler() { + @Override + public Sniffer.ScheduledTask schedule(Sniffer.Task task, long delayMillis) { + return new Sniffer.ScheduledTask(task, null); + } + + @Override + public void shutdown() { + + } + }; + HostsSniffer emptyHostsSniffer = new HostsSniffer() { + @Override + public List sniffHosts() { + return Collections.emptyList(); + } + }; + Sniffer sniffer = new Sniffer(restClient, emptyHostsSniffer, noOpScheduler, 0, 0); + { + assertEquals(1, restClient.getHosts().size()); + HttpHost httpHost = restClient.getHosts().get(0); + assertEquals("localhost", httpHost.getHostName()); + assertEquals(9200, httpHost.getPort()); + } + sniffer.sniff(); + { + assertEquals(1, restClient.getHosts().size()); + HttpHost httpHost = restClient.getHosts().get(0); + assertEquals("localhost", httpHost.getHostName()); + assertEquals(9200, httpHost.getPort()); + } + + HostsSniffer hostsSniffer = new HostsSniffer() { + @Override + public List sniffHosts() { + return Collections.singletonList(new HttpHost("sniffed", 9210)); + } + }; + new Sniffer(restClient, hostsSniffer, noOpScheduler, 0, 0).sniff(); + { + assertEquals(1, restClient.getHosts().size()); + HttpHost httpHost = restClient.getHosts().get(0); + assertEquals("sniffed", httpHost.getHostName()); + assertEquals(9210, httpHost.getPort()); + } + + HostsSniffer throwingHostsSniffer = new HostsSniffer() { + @Override + public List sniffHosts() throws IOException { + throw new IOException("communication breakdown"); + } + }; + Sniffer throwingSniffer = new Sniffer(restClient, throwingHostsSniffer, noOpScheduler, 0, 0); + try { + throwingSniffer.sniff(); + fail("should have failed"); + } catch(IOException e) { + assertEquals("communication breakdown", e.getMessage()); + } + { + assertEquals(1, restClient.getHosts().size()); + HttpHost httpHost = restClient.getHosts().get(0); + assertEquals("sniffed", httpHost.getHostName()); + assertEquals(9210, httpHost.getPort()); + } + } + } + + /** + * Test multiple sniffing rounds by mocking the {@link Scheduler} as well as the {@link HostsSniffer}. + * Simulates the ordinary behaviour of {@link Sniffer} when sniffing on failure is not enabled. + * The {@link CountingHostsSniffer} doesn't make any network connection but may throw exception or return no hosts, which makes + * it possible to verify that errors are properly handled and don't affect subsequent runs and their scheduling. + * The {@link Scheduler} implementation submits rather than scheduling runs, meaning that it doesn't respect the requested sniff + * delays while allowing to assert that the requested delays for each requested run and the following one are the expected values. */ public void testOrdinarySniffRounds() throws Exception { final long sniffInterval = randomLongBetween(1, Long.MAX_VALUE); @@ -68,31 +147,36 @@ public void testOrdinarySniffRounds() throws Exception { RestClient restClient = mock(RestClient.class); CountingHostsSniffer hostsSniffer = new CountingHostsSniffer(); final int iters = randomIntBetween(30, 100); + final List> futures = new CopyOnWriteArrayList<>(); final CountDownLatch latch = new CountDownLatch(1); - final AtomicBoolean submitCalled = new AtomicBoolean(false); final AtomicInteger runs = new AtomicInteger(iters); final ExecutorService executor = Executors.newSingleThreadExecutor(); + final AtomicReference lastFuture = new AtomicReference<>(); Scheduler scheduler = new Scheduler() { @Override - public void submit(Runnable runnable) { - //the first call is to "submit" the first sniff round from the Sniffer constructor - assertTrue(submitCalled.compareAndSet(false, true)); - assertEquals(iters, runs.getAndDecrement()); - executor.execute(runnable); - } - - @Override - public void schedule(Runnable runnable, long delayMillis) { + public Sniffer.ScheduledTask schedule(Sniffer.Task task, long delayMillis) { + assertEquals(sniffInterval, task.nextTaskDelay); int numberOfRuns = runs.getAndDecrement(); if (numberOfRuns == 0) { latch.countDown(); - return; + return new Sniffer.ScheduledTask(task, null); + } + if (numberOfRuns == iters) { + //the first call is to schedule the first sniff round from the Sniffer constructor, with delay O + assertEquals(0L, delayMillis); + assertEquals(sniffInterval, task.nextTaskDelay); + } else { + //all of the subsequent times "schedule" is called with delay set to the configured sniff interval + assertEquals(sniffInterval, delayMillis); + assertEquals(sniffInterval, task.nextTaskDelay); + } + //we submit rather than scheduling to make the test quick and not depend on time + Future future = executor.submit(task); + futures.add(future); + if (numberOfRuns == 1) { + lastFuture.compareAndSet(null, future); } - assertThat(numberOfRuns, lessThan(iters)); - //all of the subsequent times "schedule" is called with delay set to the configured sniff interval - assertEquals(sniffInterval, delayMillis); - //we immediately run rather than scheduling - executor.execute(runnable); + return new Sniffer.ScheduledTask(task, future); } @Override @@ -104,9 +188,18 @@ public void shutdown() { //all we need to do is initialize the sniffer, sniffing will start automatically in the background new Sniffer(restClient, hostsSniffer, scheduler, sniffInterval, sniffAfterFailureDelay); assertTrue(latch.await(1000, TimeUnit.MILLISECONDS)); + assertEquals(iters, futures.size()); + //the last future is the only one that may not be completed yet, as the count down happens + //while scheduling the next round which is still part of the execution of the runnable itself. + //we don't take the last item from the list as futures may be ouf of order in there + lastFuture.get().get(); + for (Future future : futures) { + assertTrue(future.isDone()); + future.get(); + } } finally { executor.shutdown(); - executor.awaitTermination(1000, TimeUnit.MILLISECONDS); + assertTrue(executor.awaitTermination(1000, TimeUnit.MILLISECONDS)); } int totalRuns = hostsSniffer.runs.get(); assertEquals(iters, totalRuns); @@ -116,21 +209,19 @@ public void shutdown() { } /** - * Test that {@link Sniffer#close()} shuts down the underlying {@link Scheduler}, and that such calls are idempotent + * Test that {@link Sniffer#close()} shuts down the underlying {@link Scheduler}, and that such calls are idempotent. + * Also verifies that the next scheduled round gets cancelled. */ public void testClose() { + final Future future = mock(Future.class); long sniffInterval = randomLongBetween(1, Long.MAX_VALUE); long sniffAfterFailureDelay = randomLongBetween(1, Long.MAX_VALUE); RestClient restClient = mock(RestClient.class); final AtomicInteger shutdown = new AtomicInteger(0); Scheduler scheduler = new Scheduler() { @Override - public void submit(Runnable runnable) { - - } - - @Override - public void schedule(Runnable runnable, long delayMillis) { + public Sniffer.ScheduledTask schedule(Sniffer.Task task, long delayMillis) { + return new Sniffer.ScheduledTask(task, future); } @Override @@ -142,98 +233,162 @@ public void shutdown() { Sniffer sniffer = new Sniffer(restClient, new MockHostsSniffer(), scheduler, sniffInterval, sniffAfterFailureDelay); assertEquals(0, shutdown.get()); int iters = randomIntBetween(3, 10); - for (int i = 0; i < iters; i++) { + for (int i = 1; i <= iters; i++) { sniffer.close(); - assertEquals(i + 1, shutdown.get()); + verify(future, times(i)).cancel(false); + assertEquals(i, shutdown.get()); } } - /** - * Test concurrent calls to the sniffing code. They should not happen as we are using a single threaded executor. - * This test makes sure that concurrent calls are not supported and such assumption is enforced. - */ - public void testConcurrentSniffRounds() throws Exception { - long sniffInterval = randomLongBetween(1, Long.MAX_VALUE); - long sniffAfterFailureDelay = randomLongBetween(1, Long.MAX_VALUE); + //TODO test what happens when on failure is called while another round is already running https://github.com/elastic/elasticsearch/issues/27697 + + //TODO test more on failure rounds? + + //TODO do we need ScheduledTask or is the future enough? + + //TODO test that sniffOnFailure doesn't do any IO blocking. https://github.com/elastic/elasticsearch/issues/25701 + + public void test() throws Exception { + final Future mockedFuture = mock(Future.class); RestClient restClient = mock(RestClient.class); CountingHostsSniffer hostsSniffer = new CountingHostsSniffer(); - - final int numThreads = randomIntBetween(10, 30); - final ExecutorService executor = Executors.newFixedThreadPool(numThreads); - final Future[] futures = new Future[numThreads]; + final long sniffInterval = randomLongBetween(1, Long.MAX_VALUE); + final long sniffAfterFailureDelay = randomLongBetween(1, Long.MAX_VALUE); + final AtomicBoolean initializing = new AtomicBoolean(true); + final AtomicBoolean ongoingOnFailure = new AtomicBoolean(true); + final AtomicBoolean afterFailureExpected = new AtomicBoolean(false); + final CountDownLatch initializingLatch = new CountDownLatch(1); + final CountDownLatch cancelLatch = new CountDownLatch(1); + when(mockedFuture.cancel(false)).thenAnswer(new Answer() { + @Override + public Void answer(InvocationOnMock invocationOnMock) { + cancelLatch.countDown(); + return null; + } + }); + final CountDownLatch doneLatch = new CountDownLatch(1); + final List futures = new CopyOnWriteArrayList<>(); + final AtomicReference lastFuture = new AtomicReference<>(); + final ExecutorService executor = Executors.newSingleThreadExecutor(); try { Scheduler scheduler = new Scheduler() { @Override - public void submit(Runnable runnable) { - for (int i = 0; i < numThreads; i++) { - //this will only submit the same runnable n times, to simulate concurrent calls to the sniffing code - futures[i] = executor.submit(runnable); + public Sniffer.ScheduledTask schedule(Sniffer.Task task, long delayMillis) { + if (initializing.compareAndSet(true, false)) { + assertEquals(0L, delayMillis); + assertEquals(sniffInterval, task.nextTaskDelay); + initializingLatch.countDown(); + return new Sniffer.ScheduledTask(task, mockedFuture); + } + if (ongoingOnFailure.compareAndSet(true, false)) { + assertEquals(0L, delayMillis); + assertEquals(sniffAfterFailureDelay, task.nextTaskDelay); + afterFailureExpected.set(true); + Future future = executor.submit(task); + futures.add(future); + return new Sniffer.ScheduledTask(task, future); + } else if (afterFailureExpected.compareAndSet(true, false)) { + //onFailure is called in another thread (not from the single threaded executor), hence we need to wait for + //it to be completed or the rest of the rounds (after failure etc.) may be executed before the onFailure + //scheduling is completed. This is a problem only when testing as we submit tasks instead of scheduling. + try { + cancelLatch.await(500, TimeUnit.MILLISECONDS); + } catch (InterruptedException e) { + fail(e.getMessage()); + } + assertEquals(sniffAfterFailureDelay, delayMillis); + assertEquals(sniffInterval, task.nextTaskDelay); + Future future = executor.submit(task); + futures.add(future); + lastFuture.compareAndSet(null, future); + return new Sniffer.ScheduledTask(task, future); + } else { + assertEquals(sniffInterval, delayMillis); + assertEquals(sniffInterval, task.nextTaskDelay); + doneLatch.countDown(); + return new Sniffer.ScheduledTask(task, null); } - } - - @Override - public void schedule(Runnable runnable, long delayMillis) { - //do nothing } @Override public void shutdown() { + } }; - new Sniffer(restClient, hostsSniffer, scheduler, sniffInterval, sniffAfterFailureDelay); - } finally { - executor.shutdown(); - executor.awaitTermination(1000, TimeUnit.MILLISECONDS); - } + Sniffer sniffer = new Sniffer(restClient, hostsSniffer, scheduler, sniffInterval, sniffAfterFailureDelay); + assertTrue(initializingLatch.await(1000, TimeUnit.MILLISECONDS)); - int failures = 0; - int successfulRuns = 0; - for (Future future : futures) { - try { + sniffer.sniffOnFailure(); + + assertTrue(doneLatch.await(1000, TimeUnit.MILLISECONDS)); + + assertEquals(2, futures.size()); + lastFuture.get().get(); + for (Future future : futures) { + assertTrue(future.isDone()); future.get(); - successfulRuns++; - } catch (ExecutionException e) { - if (e.getCause() instanceof UnsupportedOperationException) { - failures++; - } else { - throw e; - } } - } - int totalRuns = hostsSniffer.runs.get(); - //out of n concurrent threads trying to run a sniff round, two will never be run in parallel. - //That's why the total number of runs is less or equal to the number of threads. - //The important part is also that the assertion on no concurrent runs in the below CountingHostsSniffer never trips - assertThat(totalRuns, lessThanOrEqualTo(numThreads)); - assertThat(successfulRuns, greaterThan(0)); - //the successful runs that we counted match with the hosts sniffer calls - assertThat(totalRuns, equalTo(successfulRuns)); - //and the failures that we counted due to the another thread already sniffing are the rest of the threads - assertThat(numThreads - totalRuns, equalTo(failures)); - //also check out how many times setHosts is called on the underlying client - int setHostsRuns = totalRuns - hostsSniffer.failures.get() - hostsSniffer.emptyList.get(); - verify(restClient, times(setHostsRuns)).setHosts(any(HttpHost.class)); - verifyNoMoreInteractions(restClient); + int totalRuns = hostsSniffer.runs.get(); + assertEquals(2, totalRuns); + int setHostsRuns = totalRuns - hostsSniffer.failures.get() - hostsSniffer.emptyList.get(); + verify(restClient, times(setHostsRuns)).setHosts(any(HttpHost.class)); + verifyNoMoreInteractions(restClient); + + } finally { + executor.shutdown(); + assertTrue(executor.awaitTermination(1000, TimeUnit.MILLISECONDS)); + } } public void testSniffOnFailure() throws Exception { RestClient restClient = mock(RestClient.class); + CountingHostsSniffer hostsSniffer = new CountingHostsSniffer(); + int ordinaryRuns = randomIntBetween(20, 50); + int onFailureRuns = randomIntBetween(5, 10); + + final AtomicBoolean initializing = new AtomicBoolean(true); + final AtomicInteger runs = new AtomicInteger(ordinaryRuns); final long sniffInterval = randomLongBetween(1, Long.MAX_VALUE); - long sniffAfterFailureDelay = randomLongBetween(1, Long.MAX_VALUE); + final long sniffAfterFailureDelay = randomLongBetween(1, Long.MAX_VALUE); + final List> ordinaryFutures = new CopyOnWriteArrayList<>(); + final List> onFailureFutures = new CopyOnWriteArrayList<>(); + final List> afterFailureFutures = new CopyOnWriteArrayList<>(); + final AtomicInteger ongoingOnFailures = new AtomicInteger(0); + + final AtomicBoolean sniffAfterFailure = new AtomicBoolean(false); + final ExecutorService executor = Executors.newSingleThreadExecutor(); - final AtomicInteger submitRuns = new AtomicInteger(0); - final AtomicBoolean sniffingOnFailure = new AtomicBoolean(false); Scheduler scheduler = new Scheduler() { @Override - public void submit(Runnable runnable) { - submitRuns.incrementAndGet(); - } - - @Override - public void schedule(Runnable runnable, long delayMillis) { + public Sniffer.ScheduledTask schedule(Sniffer.Task task, long delayMillis) { + int numberOfRuns = runs.getAndDecrement(); + if (numberOfRuns == 0) { + return new Sniffer.ScheduledTask(task, null); + } + Future future = executor.submit(task); + if (ongoingOnFailures.get() > 0) { + onFailureFutures.add(future); + assertEquals(0L, delayMillis); + assertEquals(sniffAfterFailureDelay, task.nextTaskDelay); + sniffAfterFailure.set(true); + } else if (sniffAfterFailure.compareAndSet(true, false)) { + afterFailureFutures.add(future); + assertEquals(sniffAfterFailureDelay, delayMillis); + assertEquals(sniffInterval, task.nextTaskDelay); + } else { + ordinaryFutures.add(future); + if (initializing.compareAndSet(true, false)) { + assertEquals(0L, delayMillis); + assertEquals(sniffInterval, task.nextTaskDelay); + } else { + assertEquals(sniffInterval, delayMillis); + assertEquals(sniffInterval, task.nextTaskDelay); + } + } + return new Sniffer.ScheduledTask(task, future); } @Override @@ -241,21 +396,50 @@ public void shutdown() { } }; - int numThreads = randomIntBetween(10, 20); - final ExecutorService onFailureExecutor = Executors.newFixedThreadPool(numThreads); + + + ExecutorService onFailureExecutor = Executors.newFixedThreadPool(randomIntBetween(1, onFailureRuns)); + try { final Sniffer sniffer = new Sniffer(restClient, hostsSniffer, scheduler, sniffInterval, sniffAfterFailureDelay); - for (int i = 0; i < numThreads; i++) { - onFailureExecutor.execute(new Runnable() { + Future[] onFailureExecutorFutures = new Future[onFailureRuns]; + for (int i = 0; i < onFailureExecutorFutures.length; i++) { + onFailureExecutorFutures[i] = onFailureExecutor.submit(new Runnable() { @Override public void run() { - sniffer.sniffOnFailure(); + ongoingOnFailures.incrementAndGet(); + try { + sniffer.sniffOnFailure(); + } finally { + ongoingOnFailures.decrementAndGet(); + } } }); } + for (Future onFailureFuture : onFailureExecutorFutures) { + onFailureFuture.get(); + } + assertEquals(0, ongoingOnFailures.get()); + assertFalse(sniffAfterFailure.get()); + assertEquals(ordinaryRuns, ordinaryFutures.size()); + for (Future future : ordinaryFutures) { + assertTrue("Future is cancelled: " + future.isCancelled(), future.isDone()); + try { + future.get(); + } catch(CancellationException e) { + //all good + + //TODO shall we count these? + } + } + + //TODO check other futures } finally { + onFailureExecutor.shutdown(); + assertTrue(onFailureExecutor.awaitTermination(1000, TimeUnit.MILLISECONDS)); executor.shutdown(); - executor.awaitTermination(1000, TimeUnit.MILLISECONDS); + assertTrue(executor.awaitTermination(1000, TimeUnit.MILLISECONDS)); + } } @@ -266,30 +450,23 @@ public void run() { * at a given point in time. */ private static class CountingHostsSniffer implements HostsSniffer { - private final AtomicBoolean running = new AtomicBoolean(false); private final AtomicInteger runs = new AtomicInteger(0); private final AtomicInteger failures = new AtomicInteger(0); private final AtomicInteger emptyList = new AtomicInteger(0); @Override public List sniffHosts() throws IOException { - //check that this method is never called concurrently - assertTrue(running.compareAndSet(false, true)); - try { - runs.incrementAndGet(); - if (rarely()) { - failures.incrementAndGet(); - //check that if communication breaks, sniffer keeps on working - throw new IOException("communication breakdown"); - } - if (rarely()) { - emptyList.incrementAndGet(); - return Collections.emptyList(); - } - return Collections.singletonList(new HttpHost("localhost", 9200)); - } finally { - assertTrue(running.compareAndSet(true, false)); + runs.incrementAndGet(); + if (rarely()) { + failures.incrementAndGet(); + //check that if communication breaks, sniffer keeps on working + throw new IOException("communication breakdown"); + } + if (rarely()) { + emptyList.incrementAndGet(); + return Collections.emptyList(); } + return Collections.singletonList(new HttpHost("localhost", 9200)); } } From c2a88a3be32f5d6bd4aef0b93d1f3589657565cf Mon Sep 17 00:00:00 2001 From: javanna Date: Fri, 20 Apr 2018 17:24:09 +0200 Subject: [PATCH 05/13] improve tests --- .../org/elasticsearch/client/RestClient.java | 3 +- .../elasticsearch/client/RestClientTests.java | 32 ++ .../elasticsearch/client/sniff/Sniffer.java | 54 +-- .../client/sniff/SnifferTests.java | 408 ++++++++---------- 4 files changed, 236 insertions(+), 261 deletions(-) diff --git a/client/rest/src/main/java/org/elasticsearch/client/RestClient.java b/client/rest/src/main/java/org/elasticsearch/client/RestClient.java index c57daecc29ca5..fb4a2fa403903 100644 --- a/client/rest/src/main/java/org/elasticsearch/client/RestClient.java +++ b/client/rest/src/main/java/org/elasticsearch/client/RestClient.java @@ -61,6 +61,7 @@ import java.util.HashMap; import java.util.HashSet; import java.util.Iterator; +import java.util.LinkedHashSet; import java.util.List; import java.util.Locale; import java.util.Map; @@ -132,7 +133,7 @@ public synchronized void setHosts(HttpHost... hosts) { if (hosts == null || hosts.length == 0) { throw new IllegalArgumentException("hosts must not be null nor empty"); } - Set httpHosts = new HashSet<>(); + Set httpHosts = new LinkedHashSet<>(); AuthCache authCache = new BasicAuthCache(); for (HttpHost host : hosts) { Objects.requireNonNull(host, "host cannot be null"); diff --git a/client/rest/src/test/java/org/elasticsearch/client/RestClientTests.java b/client/rest/src/test/java/org/elasticsearch/client/RestClientTests.java index ee6dbf449bd56..75d79c658ca6b 100644 --- a/client/rest/src/test/java/org/elasticsearch/client/RestClientTests.java +++ b/client/rest/src/test/java/org/elasticsearch/client/RestClientTests.java @@ -25,6 +25,7 @@ import java.io.IOException; import java.net.URI; +import java.util.Arrays; import java.util.Collections; import java.util.concurrent.CountDownLatch; @@ -175,6 +176,37 @@ public void testSetHostsWrongArguments() throws IOException { } } + public void testSetHostsPreservesOrdering() throws Exception { + try (RestClient restClient = createRestClient()) { + HttpHost[] hosts = randomHosts(); + restClient.setHosts(hosts); + assertEquals(Arrays.asList(hosts), restClient.getHosts()); + } + } + + private static HttpHost[] randomHosts() { + int numHosts = randomIntBetween(1, 10); + HttpHost[] hosts = new HttpHost[numHosts]; + for (int i = 0; i < hosts.length; i++) { + hosts[i] = new HttpHost("host-" + i, 9200); + } + return hosts; + } + + public void testSetHostsDuplicatedHosts() throws Exception { + try (RestClient restClient = createRestClient()) { + int numHosts = randomIntBetween(1, 10); + HttpHost[] hosts = new HttpHost[numHosts]; + HttpHost host = new HttpHost("host", 9200); + for (int i = 0; i < hosts.length; i++) { + hosts[i] = host; + } + restClient.setHosts(hosts); + assertEquals(1, restClient.getHosts().size()); + assertEquals(host, restClient.getHosts().get(0)); + } + } + public void testNullPath() throws IOException { try (RestClient restClient = createRestClient()) { for (String method : getHttpMethods()) { diff --git a/client/sniffer/src/main/java/org/elasticsearch/client/sniff/Sniffer.java b/client/sniffer/src/main/java/org/elasticsearch/client/sniff/Sniffer.java index 5f75474440104..c682c0b180251 100644 --- a/client/sniffer/src/main/java/org/elasticsearch/client/sniff/Sniffer.java +++ b/client/sniffer/src/main/java/org/elasticsearch/client/sniff/Sniffer.java @@ -33,11 +33,9 @@ import java.util.concurrent.Executors; import java.util.concurrent.Future; import java.util.concurrent.ScheduledExecutorService; -import java.util.concurrent.ScheduledFuture; import java.util.concurrent.ScheduledThreadPoolExecutor; import java.util.concurrent.ThreadFactory; import java.util.concurrent.TimeUnit; -import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicReference; @@ -61,8 +59,7 @@ public class Sniffer implements Closeable { private final long sniffAfterFailureDelayMillis; private final Scheduler scheduler; - private final AtomicBoolean running = new AtomicBoolean(false); - private final AtomicReference nextTask = new AtomicReference<>(); + private final AtomicReference nextTask = new AtomicReference<>(); Sniffer(RestClient restClient, HostsSniffer hostsSniffer, long sniffInterval, long sniffAfterFailureDelay) { this(restClient, hostsSniffer, new DefaultScheduler(), sniffInterval, sniffAfterFailureDelay); @@ -85,14 +82,12 @@ public final void sniffOnFailure() { scheduleNextRound(0L, sniffAfterFailureDelayMillis, true); } - //TODO test concurrency on this method private void scheduleNextRound(long delay, long nextDelay, boolean mustCancelNextRound) { Task task = new Task(nextDelay); - ScheduledTask scheduledTask = task.schedule(delay); - assert scheduledTask.task == task; - ScheduledTask previousTask = nextTask.getAndSet(scheduledTask); + Future nextFuture = task.schedule(delay); + Future previousFuture = nextTask.getAndSet(nextFuture); if (mustCancelNextRound) { - previousTask.cancelIfNotYetStarted(); + previousFuture.cancel(false); } } @@ -103,7 +98,7 @@ final class Task implements Runnable { this.nextTaskDelay = nextTaskDelay; } - ScheduledTask schedule(long delay) { + Future schedule(long delay) { return scheduler.schedule(this, delay); } @@ -119,20 +114,6 @@ public void run() { } } - static final class ScheduledTask { - final Task task; - final Future future; - - ScheduledTask(Task task, Future future) { - this.task = task; - this.future = future; - } - - void cancelIfNotYetStarted() { - this.future.cancel(false); - } - } - final void sniff() throws IOException { List sniffedHosts = hostsSniffer.sniffHosts(); logger.debug("sniffed hosts: " + sniffedHosts); @@ -145,7 +126,7 @@ final void sniff() throws IOException { @Override public void close() { - nextTask.get().cancelIfNotYetStarted(); + nextTask.get().cancel(false); this.scheduler.shutdown(); } @@ -167,7 +148,7 @@ interface Scheduler { /** * Schedules the provided {@link Runnable} to be executed in delayMillis milliseconds */ - ScheduledTask schedule(Task task, long delayMillis); + Future schedule(Task task, long delayMillis); /** * Shuts this scheduler down @@ -179,34 +160,25 @@ interface Scheduler { * Default implementation of {@link Scheduler}, based on {@link ScheduledExecutorService} */ static final class DefaultScheduler implements Scheduler { - final ScheduledThreadPoolExecutor executor; + final ScheduledExecutorService executor; DefaultScheduler() { this(initScheduledExecutorService()); } - DefaultScheduler(ScheduledThreadPoolExecutor executor) { + DefaultScheduler(ScheduledExecutorService executor) { this.executor = executor; } - //TODO test this - static ScheduledThreadPoolExecutor initScheduledExecutorService() { + private static ScheduledExecutorService initScheduledExecutorService() { ScheduledThreadPoolExecutor executor = new ScheduledThreadPoolExecutor(1, new SnifferThreadFactory(SNIFFER_THREAD_NAME)); executor.setRemoveOnCancelPolicy(true); - //TODO does this have any effect? - executor.setExecuteExistingDelayedTasksAfterShutdownPolicy(false); return executor; } - //TODO can it happen that we get so many failures that we keep on cancelling without ever getting to execute the next round? - // no, because we add each failed node to the blacklist, and it stays there till setHosts is called, at the end of the sniff round - //may happen if we end up reviving nodes from the blacklist as all of the sniffed nodes are marked dead. - //in that case the sniff round won't work either though. - @Override - public ScheduledTask schedule(Task task, long delayMillis) { - ScheduledFuture future = executor.schedule(task, delayMillis, TimeUnit.MILLISECONDS); - return new ScheduledTask(task, future); + public Future schedule(Task task, long delayMillis) { + return executor.schedule(task, delayMillis, TimeUnit.MILLISECONDS); } @Override @@ -217,7 +189,7 @@ public void shutdown() { return; } executor.shutdownNow(); - } catch (InterruptedException e) { + } catch (InterruptedException ignore) { Thread.currentThread().interrupt(); } } diff --git a/client/sniffer/src/test/java/org/elasticsearch/client/sniff/SnifferTests.java b/client/sniffer/src/test/java/org/elasticsearch/client/sniff/SnifferTests.java index 59a4c45118ec2..9114450f4bef2 100644 --- a/client/sniffer/src/test/java/org/elasticsearch/client/sniff/SnifferTests.java +++ b/client/sniffer/src/test/java/org/elasticsearch/client/sniff/SnifferTests.java @@ -24,18 +24,22 @@ import org.elasticsearch.client.RestClientTestCase; import org.elasticsearch.client.sniff.Sniffer.DefaultScheduler; import org.elasticsearch.client.sniff.Sniffer.Scheduler; +import org.mockito.Matchers; +import org.mockito.exceptions.verification.WantedButNotInvoked; import org.mockito.invocation.InvocationOnMock; import org.mockito.stubbing.Answer; import java.io.IOException; +import java.util.ArrayList; import java.util.Collections; import java.util.List; -import java.util.concurrent.CancellationException; import java.util.concurrent.CopyOnWriteArrayList; import java.util.concurrent.CountDownLatch; import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; import java.util.concurrent.Future; +import java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.ScheduledFuture; import java.util.concurrent.ScheduledThreadPoolExecutor; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; @@ -47,6 +51,8 @@ import static org.hamcrest.Matchers.is; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertNotEquals; +import static org.junit.Assert.assertSame; import static org.junit.Assert.assertThat; import static org.junit.Assert.assertTrue; import static org.junit.Assert.fail; @@ -64,11 +70,12 @@ public class SnifferTests extends RestClientTestCase { * to retrieve nodes and set them (when not empty) to the provided {@link RestClient} instance. */ public void testSniff() throws IOException { - try (RestClient restClient = RestClient.builder(new HttpHost("localhost", 9200)).build()) { + HttpHost initialHost = new HttpHost("localhost", 9200); + try (RestClient restClient = RestClient.builder(initialHost).build()) { Scheduler noOpScheduler = new Scheduler() { @Override - public Sniffer.ScheduledTask schedule(Sniffer.Task task, long delayMillis) { - return new Sniffer.ScheduledTask(task, null); + public Future schedule(Sniffer.Task task, long delayMillis) { + return mock(Future.class); } @Override @@ -76,59 +83,45 @@ public void shutdown() { } }; - HostsSniffer emptyHostsSniffer = new HostsSniffer() { - @Override - public List sniffHosts() { - return Collections.emptyList(); - } - }; - Sniffer sniffer = new Sniffer(restClient, emptyHostsSniffer, noOpScheduler, 0, 0); - { - assertEquals(1, restClient.getHosts().size()); - HttpHost httpHost = restClient.getHosts().get(0); - assertEquals("localhost", httpHost.getHostName()); - assertEquals(9200, httpHost.getPort()); - } - sniffer.sniff(); - { - assertEquals(1, restClient.getHosts().size()); - HttpHost httpHost = restClient.getHosts().get(0); - assertEquals("localhost", httpHost.getHostName()); - assertEquals(9200, httpHost.getPort()); - } - - HostsSniffer hostsSniffer = new HostsSniffer() { - @Override - public List sniffHosts() { - return Collections.singletonList(new HttpHost("sniffed", 9210)); + CountingHostsSniffer hostsSniffer = new CountingHostsSniffer(); + int iters = randomIntBetween(5, 30); + try (Sniffer sniffer = new Sniffer(restClient, hostsSniffer, noOpScheduler, 1000L, -1)){ + { + assertEquals(1, restClient.getHosts().size()); + HttpHost httpHost = restClient.getHosts().get(0); + assertEquals("localhost", httpHost.getHostName()); + assertEquals(9200, httpHost.getPort()); } - }; - new Sniffer(restClient, hostsSniffer, noOpScheduler, 0, 0).sniff(); - { - assertEquals(1, restClient.getHosts().size()); - HttpHost httpHost = restClient.getHosts().get(0); - assertEquals("sniffed", httpHost.getHostName()); - assertEquals(9210, httpHost.getPort()); - } - - HostsSniffer throwingHostsSniffer = new HostsSniffer() { - @Override - public List sniffHosts() throws IOException { - throw new IOException("communication breakdown"); + int emptyList = 0; + int failures = 0; + int runs = 0; + List lastHosts = Collections.singletonList(initialHost); + for (int i = 0; i < iters; i++) { + try { + runs++; + sniffer.sniff(); + if (hostsSniffer.failures.get() > failures) { + failures++; + fail("should have failed"); + } else if (hostsSniffer.emptyList.get() > emptyList) { + emptyList++; + assertEquals(lastHosts, restClient.getHosts()); + } else { + assertNotEquals(lastHosts, restClient.getHosts()); + List expectedHosts = CountingHostsSniffer.buildHosts(runs); + assertEquals(expectedHosts, restClient.getHosts()); + lastHosts = restClient.getHosts(); + } + } catch(IOException e) { + if (hostsSniffer.failures.get() > failures) { + failures++; + assertEquals("communication breakdown", e.getMessage()); + } + } } - }; - Sniffer throwingSniffer = new Sniffer(restClient, throwingHostsSniffer, noOpScheduler, 0, 0); - try { - throwingSniffer.sniff(); - fail("should have failed"); - } catch(IOException e) { - assertEquals("communication breakdown", e.getMessage()); - } - { - assertEquals(1, restClient.getHosts().size()); - HttpHost httpHost = restClient.getHosts().get(0); - assertEquals("sniffed", httpHost.getHostName()); - assertEquals(9210, httpHost.getPort()); + assertEquals(hostsSniffer.emptyList.get(), emptyList); + assertEquals(hostsSniffer.failures.get(), failures); + assertEquals(hostsSniffer.runs.get(), runs); } } } @@ -154,12 +147,12 @@ public void testOrdinarySniffRounds() throws Exception { final AtomicReference lastFuture = new AtomicReference<>(); Scheduler scheduler = new Scheduler() { @Override - public Sniffer.ScheduledTask schedule(Sniffer.Task task, long delayMillis) { + public Future schedule(Sniffer.Task task, long delayMillis) { assertEquals(sniffInterval, task.nextTaskDelay); int numberOfRuns = runs.getAndDecrement(); if (numberOfRuns == 0) { latch.countDown(); - return new Sniffer.ScheduledTask(task, null); + return null; } if (numberOfRuns == iters) { //the first call is to schedule the first sniff round from the Sniffer constructor, with delay O @@ -176,7 +169,7 @@ public Sniffer.ScheduledTask schedule(Sniffer.Task task, long delayMillis) { if (numberOfRuns == 1) { lastFuture.compareAndSet(null, future); } - return new Sniffer.ScheduledTask(task, future); + return future; } @Override @@ -204,7 +197,7 @@ public void shutdown() { int totalRuns = hostsSniffer.runs.get(); assertEquals(iters, totalRuns); int setHostsRuns = totalRuns - hostsSniffer.failures.get() - hostsSniffer.emptyList.get(); - verify(restClient, times(setHostsRuns)).setHosts(any(HttpHost.class)); + verify(restClient, times(setHostsRuns)).setHosts(Matchers.anyVararg()); verifyNoMoreInteractions(restClient); } @@ -220,8 +213,8 @@ public void testClose() { final AtomicInteger shutdown = new AtomicInteger(0); Scheduler scheduler = new Scheduler() { @Override - public Sniffer.ScheduledTask schedule(Sniffer.Task task, long delayMillis) { - return new Sniffer.ScheduledTask(task, future); + public Future schedule(Sniffer.Task task, long delayMillis) { + return future; } @Override @@ -240,15 +233,12 @@ public void shutdown() { } } - //TODO test what happens when on failure is called while another round is already running https://github.com/elastic/elasticsearch/issues/27697 - - //TODO test more on failure rounds? - - //TODO do we need ScheduledTask or is the future enough? - - //TODO test that sniffOnFailure doesn't do any IO blocking. https://github.com/elastic/elasticsearch/issues/25701 - - public void test() throws Exception { + /** + * Test calling {@link Sniffer#sniffOnFailure()} once. The next scheduled sniffing round is cancelled, + * a new round is scheduled with delay 0, a following one is scheduled with sniffAfterFailure delay, after + * which the ordinary sniffing rounds get scheduled with sniffInterval delay + */ + public void testOnFailureSingleRound() throws Exception { final Future mockedFuture = mock(Future.class); RestClient restClient = mock(RestClient.class); CountingHostsSniffer hostsSniffer = new CountingHostsSniffer(); @@ -273,12 +263,12 @@ public Void answer(InvocationOnMock invocationOnMock) { try { Scheduler scheduler = new Scheduler() { @Override - public Sniffer.ScheduledTask schedule(Sniffer.Task task, long delayMillis) { + public Future schedule(Sniffer.Task task, long delayMillis) { if (initializing.compareAndSet(true, false)) { assertEquals(0L, delayMillis); assertEquals(sniffInterval, task.nextTaskDelay); initializingLatch.countDown(); - return new Sniffer.ScheduledTask(task, mockedFuture); + return mockedFuture; } if (ongoingOnFailure.compareAndSet(true, false)) { assertEquals(0L, delayMillis); @@ -286,11 +276,12 @@ public Sniffer.ScheduledTask schedule(Sniffer.Task task, long delayMillis) { afterFailureExpected.set(true); Future future = executor.submit(task); futures.add(future); - return new Sniffer.ScheduledTask(task, future); + return future; } else if (afterFailureExpected.compareAndSet(true, false)) { - //onFailure is called in another thread (not from the single threaded executor), hence we need to wait for - //it to be completed or the rest of the rounds (after failure etc.) may be executed before the onFailure - //scheduling is completed. This is a problem only when testing as we submit tasks instead of scheduling. + //onFailure is called in another thread (not from the single threaded executor), hence we need to + //wait for it to be completed or the rest of the rounds (after failure etc.) may be executed + //before the onFailure scheduling is completed. This is a problem only when testing as we submit + //tasks instead of scheduling them with some delay. try { cancelLatch.await(500, TimeUnit.MILLISECONDS); } catch (InterruptedException e) { @@ -301,12 +292,12 @@ public Sniffer.ScheduledTask schedule(Sniffer.Task task, long delayMillis) { Future future = executor.submit(task); futures.add(future); lastFuture.compareAndSet(null, future); - return new Sniffer.ScheduledTask(task, future); + return future; } else { assertEquals(sniffInterval, delayMillis); assertEquals(sniffInterval, task.nextTaskDelay); doneLatch.countDown(); - return new Sniffer.ScheduledTask(task, null); + return null; } } @@ -321,74 +312,46 @@ public void shutdown() { sniffer.sniffOnFailure(); assertTrue(doneLatch.await(1000, TimeUnit.MILLISECONDS)); - assertEquals(2, futures.size()); lastFuture.get().get(); for (Future future : futures) { assertTrue(future.isDone()); future.get(); } - int totalRuns = hostsSniffer.runs.get(); assertEquals(2, totalRuns); int setHostsRuns = totalRuns - hostsSniffer.failures.get() - hostsSniffer.emptyList.get(); - verify(restClient, times(setHostsRuns)).setHosts(any(HttpHost.class)); + verify(restClient, times(setHostsRuns)).setHosts(Matchers.anyVararg()); verifyNoMoreInteractions(restClient); - } finally { executor.shutdown(); assertTrue(executor.awaitTermination(1000, TimeUnit.MILLISECONDS)); } } - public void testSniffOnFailure() throws Exception { + /** + * Test that when invoking {@link Sniffer#sniffOnFailure()} concurrently, the next scheduled sniffing round is always cancelled. + */ + public void testSniffOnFailureCancelsNextRound() throws Exception { RestClient restClient = mock(RestClient.class); - - CountingHostsSniffer hostsSniffer = new CountingHostsSniffer(); - int ordinaryRuns = randomIntBetween(20, 50); - int onFailureRuns = randomIntBetween(5, 10); - - final AtomicBoolean initializing = new AtomicBoolean(true); - final AtomicInteger runs = new AtomicInteger(ordinaryRuns); + HostsSniffer hostsSniffer = mock(HostsSniffer.class); + int onFailureRounds = randomIntBetween(30, 50); final long sniffInterval = randomLongBetween(1, Long.MAX_VALUE); final long sniffAfterFailureDelay = randomLongBetween(1, Long.MAX_VALUE); - final List> ordinaryFutures = new CopyOnWriteArrayList<>(); - final List> onFailureFutures = new CopyOnWriteArrayList<>(); - final List> afterFailureFutures = new CopyOnWriteArrayList<>(); - final AtomicInteger ongoingOnFailures = new AtomicInteger(0); - - final AtomicBoolean sniffAfterFailure = new AtomicBoolean(false); - - final ExecutorService executor = Executors.newSingleThreadExecutor(); + final AtomicBoolean initialized = new AtomicBoolean(false); + final List futures = new CopyOnWriteArrayList<>(); Scheduler scheduler = new Scheduler() { @Override - public Sniffer.ScheduledTask schedule(Sniffer.Task task, long delayMillis) { - int numberOfRuns = runs.getAndDecrement(); - if (numberOfRuns == 0) { - return new Sniffer.ScheduledTask(task, null); - } - - Future future = executor.submit(task); - if (ongoingOnFailures.get() > 0) { - onFailureFutures.add(future); - assertEquals(0L, delayMillis); - assertEquals(sniffAfterFailureDelay, task.nextTaskDelay); - sniffAfterFailure.set(true); - } else if (sniffAfterFailure.compareAndSet(true, false)) { - afterFailureFutures.add(future); - assertEquals(sniffAfterFailureDelay, delayMillis); + public Future schedule(Sniffer.Task task, long delayMillis) { + assertEquals(0L, delayMillis); + if (initialized.compareAndSet(false, true)) { assertEquals(sniffInterval, task.nextTaskDelay); } else { - ordinaryFutures.add(future); - if (initializing.compareAndSet(true, false)) { - assertEquals(0L, delayMillis); - assertEquals(sniffInterval, task.nextTaskDelay); - } else { - assertEquals(sniffInterval, delayMillis); - assertEquals(sniffInterval, task.nextTaskDelay); - } + assertEquals(sniffAfterFailureDelay, task.nextTaskDelay); } - return new Sniffer.ScheduledTask(task, future); + Future mockedFuture = mock(Future.class); + futures.add(mockedFuture); + return mockedFuture; } @Override @@ -397,49 +360,79 @@ public void shutdown() { } }; - - ExecutorService onFailureExecutor = Executors.newFixedThreadPool(randomIntBetween(1, onFailureRuns)); - + final Sniffer sniffer = new Sniffer(restClient, hostsSniffer, scheduler, sniffInterval, sniffAfterFailureDelay); + ExecutorService onFailureExecutor = Executors.newFixedThreadPool(randomIntBetween(2, 10)); try { - final Sniffer sniffer = new Sniffer(restClient, hostsSniffer, scheduler, sniffInterval, sniffAfterFailureDelay); - Future[] onFailureExecutorFutures = new Future[onFailureRuns]; - for (int i = 0; i < onFailureExecutorFutures.length; i++) { - onFailureExecutorFutures[i] = onFailureExecutor.submit(new Runnable() { + Future[] onFailureFutures = new Future[onFailureRounds]; + for (int i = 0; i < onFailureFutures.length; i++) { + onFailureFutures[i] = onFailureExecutor.submit(new Runnable() { @Override public void run() { - ongoingOnFailures.incrementAndGet(); - try { - sniffer.sniffOnFailure(); - } finally { - ongoingOnFailures.decrementAndGet(); - } + sniffer.sniffOnFailure(); } }); } - for (Future onFailureFuture : onFailureExecutorFutures) { + for (Future onFailureFuture : onFailureFutures) { onFailureFuture.get(); } - assertEquals(0, ongoingOnFailures.get()); - assertFalse(sniffAfterFailure.get()); - assertEquals(ordinaryRuns, ordinaryFutures.size()); - for (Future future : ordinaryFutures) { - assertTrue("Future is cancelled: " + future.isCancelled(), future.isDone()); - try { - future.get(); - } catch(CancellationException e) { - //all good - - //TODO shall we count these? + } finally { + onFailureExecutor.shutdown(); + onFailureExecutor.awaitTermination(1000, TimeUnit.MILLISECONDS); + } + + //we have as many futures as the onFailure rounds, plus the initial one obtained at Sniffer construction + assertEquals(onFailureRounds + 1, futures.size()); + boolean notInvokedFound = false; + //all of these futures but one must be cancelled, one single time. The one that's not is the last scheduled but not + //necessarily the last one in our list + for (Future future : futures) { + try { + verify(future).cancel(false); + } catch(WantedButNotInvoked e) { + if (notInvokedFound) { + throw new AssertionError("future was cancelled more than once", e); } + notInvokedFound = true; } + } + } + + /** + * Test that no matter when an onFailure sniffing round is triggered, it will run and it will always + * schedule a subsequent afterFailure round. See https://github.com/elastic/elasticsearch/issues/27697 . + * The difference in this test compared to the other ones above is that we keep on scheduling ordinary sniffing + * rounds to show that they don't interfere with the onFailure round that is scheduled at some point. + */ + public void testSniffOnFailureIsAlwaysExecuted() throws Exception { + RestClient restClient = mock(RestClient.class); + CountingHostsSniffer hostsSniffer = new CountingHostsSniffer(); + long sniffInterval = randomLongBetween(1, Long.MAX_VALUE); + final long sniffAfterFailureDelay = randomLongBetween(1, Long.MAX_VALUE); + final CountDownLatch latch = new CountDownLatch(1); + final ExecutorService executor = Executors.newSingleThreadExecutor(); + try { + Scheduler scheduler = new Scheduler() { + @Override + public Future schedule(Sniffer.Task task, long delayMillis) { + //if we are scheduling a next round with delay sniffAfterFailureDelay it means that we are executing the onFailure task + if (delayMillis == sniffAfterFailureDelay) { + latch.countDown(); + return null; + } + return executor.submit(task); + } + + @Override + public void shutdown() { - //TODO check other futures + } + }; + Sniffer sniffer = new Sniffer(restClient, hostsSniffer, scheduler, sniffInterval, sniffAfterFailureDelay); + sniffer.sniffOnFailure(); + assertTrue(latch.await(1000L, TimeUnit.MILLISECONDS)); } finally { - onFailureExecutor.shutdown(); - assertTrue(onFailureExecutor.awaitTermination(1000, TimeUnit.MILLISECONDS)); executor.shutdown(); - assertTrue(executor.awaitTermination(1000, TimeUnit.MILLISECONDS)); - + executor.awaitTermination(1000L, TimeUnit.MILLISECONDS); } } @@ -456,7 +449,7 @@ private static class CountingHostsSniffer implements HostsSniffer { @Override public List sniffHosts() throws IOException { - runs.incrementAndGet(); + int run = runs.incrementAndGet(); if (rarely()) { failures.incrementAndGet(); //check that if communication breaks, sniffer keeps on working @@ -466,65 +459,63 @@ public List sniffHosts() throws IOException { emptyList.incrementAndGet(); return Collections.emptyList(); } - return Collections.singletonList(new HttpHost("localhost", 9200)); + return buildHosts(run); + } + + private static List buildHosts(int run) { + int size = run % 5 + 1; + assert size > 0; + List hosts = new ArrayList<>(size); + for (int i = 0; i < size; i++) { + hosts.add(new HttpHost("sniffed-" + run, 9200 + i)); + } + return hosts; } } - //TODO adapt this test @SuppressWarnings("unchecked") -/* public void testDefaultSchedulerSchedule() { - ScheduledThreadPoolExecutor executor = mock(ScheduledThreadPoolExecutor.class); - ScheduledFuture scheduledFuture = mock(ScheduledFuture.class); - when(executor.isShutdown()).thenReturn(false); - when(executor.schedule(any(Runnable.class), any(Long.class), any(TimeUnit.class))).thenReturn(scheduledFuture); + RestClient restClient = mock(RestClient.class); + HostsSniffer hostsSniffer = mock(HostsSniffer.class); + Scheduler noOpScheduler = new Scheduler() { + @Override + public Future schedule(Sniffer.Task task, long delayMillis) { + return mock(Future.class); + } - DefaultScheduler defaultScheduler = new DefaultScheduler(executor); - Runnable runnable = new Runnable() { @Override - public void run() { + public void shutdown() { } }; - int iters = randomIntBetween(3, 10); - { - for (int i = 1; i <= iters; i++) { - long delayMillis = randomLongBetween(0L, Long.MAX_VALUE); - defaultScheduler.schedule(runnable, delayMillis); - verify(executor, times(i)).schedule(runnable, delayMillis, TimeUnit.MILLISECONDS); - verifyNoMoreInteractions(executor); - verify(scheduledFuture, times(i)).cancel(false); - verifyNoMoreInteractions(scheduledFuture); - } - } - { - when(executor.schedule(any(Runnable.class), any(Long.class), any(TimeUnit.class))) - .thenThrow(new IllegalArgumentException()); - long delayMillis = randomLongBetween(0L, Long.MAX_VALUE); - defaultScheduler.schedule(runnable, delayMillis); - verify(executor).schedule(runnable, delayMillis, TimeUnit.MILLISECONDS); - verifyNoMoreInteractions(executor); - verify(scheduledFuture, times(2)).cancel(false); - verifyNoMoreInteractions(scheduledFuture); - } - { - when(executor.isShutdown()).thenReturn(true); - long delayMillis = randomLongBetween(0L, Long.MAX_VALUE); - defaultScheduler.schedule(runnable, delayMillis); - verifyNoMoreInteractions(executor); - verifyNoMoreInteractions(scheduledFuture); - } + Sniffer sniffer = new Sniffer(restClient, hostsSniffer, noOpScheduler, 0L, 0L); + Sniffer.Task task = sniffer.new Task(randomLongBetween(1, Long.MAX_VALUE)); + + ScheduledExecutorService scheduledExecutorService = mock(ScheduledExecutorService.class); + ScheduledFuture mockedFuture = mock(ScheduledFuture.class); + when(scheduledExecutorService.schedule(any(Runnable.class), any(Long.class), any(TimeUnit.class))).thenReturn(mockedFuture); + DefaultScheduler scheduler = new DefaultScheduler(scheduledExecutorService); + long delay = randomLongBetween(1, Long.MAX_VALUE); + Future future = scheduler.schedule(task, delay); + assertSame(mockedFuture, future); + verify(scheduledExecutorService).schedule(task, delay, TimeUnit.MILLISECONDS); + verifyNoMoreInteractions(scheduledExecutorService, mockedFuture); } -*/ public void testDefaultSchedulerThreadFactory() { DefaultScheduler defaultScheduler = new DefaultScheduler(); try { - ScheduledThreadPoolExecutor scheduledThreadPoolExecutor = defaultScheduler.executor; - assertThat(scheduledThreadPoolExecutor.getThreadFactory(), instanceOf(Sniffer.SnifferThreadFactory.class)); + ScheduledExecutorService executorService = defaultScheduler.executor; + assertThat(executorService, instanceOf(ScheduledThreadPoolExecutor.class)); + assertThat(executorService, instanceOf(ScheduledThreadPoolExecutor.class)); + ScheduledThreadPoolExecutor executor = (ScheduledThreadPoolExecutor) executorService; + assertTrue(executor.getRemoveOnCancelPolicy()); + assertFalse(executor.getContinueExistingPeriodicTasksAfterShutdownPolicy()); + assertTrue(executor.getExecuteExistingDelayedTasksAfterShutdownPolicy()); + assertThat(executor.getThreadFactory(), instanceOf(Sniffer.SnifferThreadFactory.class)); int iters = randomIntBetween(3, 10); for (int i = 1; i <= iters; i++) { - Thread thread = scheduledThreadPoolExecutor.getThreadFactory().newThread(new Runnable() { + Thread thread = executor.getThreadFactory().newThread(new Runnable() { @Override public void run() { @@ -538,40 +529,19 @@ public void run() { } } - //TODO adapt this, it's not a good test though -/* @SuppressWarnings("unchecked") public void testDefaultSchedulerShutdown() throws Exception { ScheduledThreadPoolExecutor executor = mock(ScheduledThreadPoolExecutor.class); - when(executor.isShutdown()).thenReturn(false); - ScheduledFuture scheduledFuture = mock(ScheduledFuture.class); - when(executor.schedule(any(Runnable.class), any(Long.class), any(TimeUnit.class))).thenReturn(scheduledFuture); DefaultScheduler defaultScheduler = new DefaultScheduler(executor); - - defaultScheduler.schedule(new Runnable() { - @Override - public void run() { - - } - }, 0L); - verify(executor.schedule(any(Runnable.class), any(Long.class), any(TimeUnit.class))); - defaultScheduler.shutdown(); - verify(scheduledFuture).cancel(false); verify(executor).shutdown(); verify(executor).awaitTermination(1000, TimeUnit.MILLISECONDS); verify(executor).shutdownNow(); - verifyNoMoreInteractions(executor, scheduledFuture); + verifyNoMoreInteractions(executor); when(executor.awaitTermination(1000, TimeUnit.MILLISECONDS)).thenReturn(true); defaultScheduler.shutdown(); - verify(scheduledFuture, times(2)).cancel(false); verify(executor, times(2)).shutdown(); verify(executor, times(2)).awaitTermination(1000, TimeUnit.MILLISECONDS); - verifyNoMoreInteractions(executor, scheduledFuture); - }*/ - - //TODO test on failure intervals - - //TODO add a sniffer test against proper http server, or maybe just throw exception and trigger on failure - + verifyNoMoreInteractions(executor); + } } From 5d7c6360bf5761d9497922db9a26b2aca1204510 Mon Sep 17 00:00:00 2001 From: javanna Date: Fri, 20 Apr 2018 21:04:51 +0200 Subject: [PATCH 06/13] address first review comments --- .../java/org/elasticsearch/client/sniff/Sniffer.java | 12 +++++------- .../org/elasticsearch/client/sniff/SnifferTests.java | 2 +- 2 files changed, 6 insertions(+), 8 deletions(-) diff --git a/client/sniffer/src/main/java/org/elasticsearch/client/sniff/Sniffer.java b/client/sniffer/src/main/java/org/elasticsearch/client/sniff/Sniffer.java index c682c0b180251..556801daafef4 100644 --- a/client/sniffer/src/main/java/org/elasticsearch/client/sniff/Sniffer.java +++ b/client/sniffer/src/main/java/org/elasticsearch/client/sniff/Sniffer.java @@ -59,7 +59,7 @@ public class Sniffer implements Closeable { private final long sniffAfterFailureDelayMillis; private final Scheduler scheduler; - private final AtomicReference nextTask = new AtomicReference<>(); + private final AtomicReference> nextTask = new AtomicReference<>(); Sniffer(RestClient restClient, HostsSniffer hostsSniffer, long sniffInterval, long sniffAfterFailureDelay) { this(restClient, hostsSniffer, new DefaultScheduler(), sniffInterval, sniffAfterFailureDelay); @@ -84,7 +84,7 @@ public final void sniffOnFailure() { private void scheduleNextRound(long delay, long nextDelay, boolean mustCancelNextRound) { Task task = new Task(nextDelay); - Future nextFuture = task.schedule(delay); + Future nextFuture = scheduler.schedule(task, delay); Future previousFuture = nextTask.getAndSet(nextFuture); if (mustCancelNextRound) { previousFuture.cancel(false); @@ -98,10 +98,6 @@ final class Task implements Runnable { this.nextTaskDelay = nextTaskDelay; } - Future schedule(long delay) { - return scheduler.schedule(this, delay); - } - @Override public void run() { try { @@ -116,7 +112,9 @@ public void run() { final void sniff() throws IOException { List sniffedHosts = hostsSniffer.sniffHosts(); - logger.debug("sniffed hosts: " + sniffedHosts); + if (logger.isDebugEnabled()) { + logger.debug("sniffed hosts: " + sniffedHosts); + } if (sniffedHosts.isEmpty()) { logger.warn("no hosts to set, hosts will be updated at the next sniffing round"); } else { diff --git a/client/sniffer/src/test/java/org/elasticsearch/client/sniff/SnifferTests.java b/client/sniffer/src/test/java/org/elasticsearch/client/sniff/SnifferTests.java index 9114450f4bef2..6dfe833df2c24 100644 --- a/client/sniffer/src/test/java/org/elasticsearch/client/sniff/SnifferTests.java +++ b/client/sniffer/src/test/java/org/elasticsearch/client/sniff/SnifferTests.java @@ -102,7 +102,7 @@ public void shutdown() { sniffer.sniff(); if (hostsSniffer.failures.get() > failures) { failures++; - fail("should have failed"); + fail("should have failed given that hostsSniffer says it threw an exception"); } else if (hostsSniffer.emptyList.get() > emptyList) { emptyList++; assertEquals(lastHosts, restClient.getHosts()); From 9658886889c47a6e720083113e3a3605efc85187 Mon Sep 17 00:00:00 2001 From: javanna Date: Mon, 30 Apr 2018 12:15:43 +0200 Subject: [PATCH 07/13] adapt NodeFailureListener --- .../xpack/monitoring/exporter/http/NodeFailureListener.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/x-pack/plugin/monitoring/src/main/java/org/elasticsearch/xpack/monitoring/exporter/http/NodeFailureListener.java b/x-pack/plugin/monitoring/src/main/java/org/elasticsearch/xpack/monitoring/exporter/http/NodeFailureListener.java index 6590232fda1ff..92febdf3561f8 100644 --- a/x-pack/plugin/monitoring/src/main/java/org/elasticsearch/xpack/monitoring/exporter/http/NodeFailureListener.java +++ b/x-pack/plugin/monitoring/src/main/java/org/elasticsearch/xpack/monitoring/exporter/http/NodeFailureListener.java @@ -86,7 +86,7 @@ public void onFailure(final HttpHost host) { resource.markDirty(); } if (sniffer != null) { - sniffer.sniffOnFailure(host); + sniffer.sniffOnFailure(); } } From 4094a4e575049e9bb20beb42c1d188a063064384 Mon Sep 17 00:00:00 2001 From: javanna Date: Mon, 30 Apr 2018 15:13:19 +0200 Subject: [PATCH 08/13] fixed warnings --- .../client/sniff/SnifferTests.java | 32 +++++++++++-------- 1 file changed, 19 insertions(+), 13 deletions(-) diff --git a/client/sniffer/src/test/java/org/elasticsearch/client/sniff/SnifferTests.java b/client/sniffer/src/test/java/org/elasticsearch/client/sniff/SnifferTests.java index 6dfe833df2c24..49852ba32dad2 100644 --- a/client/sniffer/src/test/java/org/elasticsearch/client/sniff/SnifferTests.java +++ b/client/sniffer/src/test/java/org/elasticsearch/client/sniff/SnifferTests.java @@ -144,7 +144,7 @@ public void testOrdinarySniffRounds() throws Exception { final CountDownLatch latch = new CountDownLatch(1); final AtomicInteger runs = new AtomicInteger(iters); final ExecutorService executor = Executors.newSingleThreadExecutor(); - final AtomicReference lastFuture = new AtomicReference<>(); + final AtomicReference> lastFuture = new AtomicReference<>(); Scheduler scheduler = new Scheduler() { @Override public Future schedule(Sniffer.Task task, long delayMillis) { @@ -206,7 +206,7 @@ public void shutdown() { * Also verifies that the next scheduled round gets cancelled. */ public void testClose() { - final Future future = mock(Future.class); + final Future future = mock(Future.class); long sniffInterval = randomLongBetween(1, Long.MAX_VALUE); long sniffAfterFailureDelay = randomLongBetween(1, Long.MAX_VALUE); RestClient restClient = mock(RestClient.class); @@ -239,7 +239,7 @@ public void shutdown() { * which the ordinary sniffing rounds get scheduled with sniffInterval delay */ public void testOnFailureSingleRound() throws Exception { - final Future mockedFuture = mock(Future.class); + final Future mockedFuture = mock(Future.class); RestClient restClient = mock(RestClient.class); CountingHostsSniffer hostsSniffer = new CountingHostsSniffer(); final long sniffInterval = randomLongBetween(1, Long.MAX_VALUE); @@ -257,8 +257,8 @@ public Void answer(InvocationOnMock invocationOnMock) { } }); final CountDownLatch doneLatch = new CountDownLatch(1); - final List futures = new CopyOnWriteArrayList<>(); - final AtomicReference lastFuture = new AtomicReference<>(); + final List> futures = new CopyOnWriteArrayList<>(); + final AtomicReference> lastFuture = new AtomicReference<>(); final ExecutorService executor = Executors.newSingleThreadExecutor(); try { Scheduler scheduler = new Scheduler() { @@ -314,7 +314,7 @@ public void shutdown() { assertTrue(doneLatch.await(1000, TimeUnit.MILLISECONDS)); assertEquals(2, futures.size()); lastFuture.get().get(); - for (Future future : futures) { + for (Future future : futures) { assertTrue(future.isDone()); future.get(); } @@ -339,7 +339,7 @@ public void testSniffOnFailureCancelsNextRound() throws Exception { final long sniffInterval = randomLongBetween(1, Long.MAX_VALUE); final long sniffAfterFailureDelay = randomLongBetween(1, Long.MAX_VALUE); final AtomicBoolean initialized = new AtomicBoolean(false); - final List futures = new CopyOnWriteArrayList<>(); + final List> futures = new CopyOnWriteArrayList<>(); Scheduler scheduler = new Scheduler() { @Override public Future schedule(Sniffer.Task task, long delayMillis) { @@ -349,7 +349,7 @@ public Future schedule(Sniffer.Task task, long delayMillis) { } else { assertEquals(sniffAfterFailureDelay, task.nextTaskDelay); } - Future mockedFuture = mock(Future.class); + Future mockedFuture = mock(Future.class); futures.add(mockedFuture); return mockedFuture; } @@ -363,7 +363,7 @@ public void shutdown() { final Sniffer sniffer = new Sniffer(restClient, hostsSniffer, scheduler, sniffInterval, sniffAfterFailureDelay); ExecutorService onFailureExecutor = Executors.newFixedThreadPool(randomIntBetween(2, 10)); try { - Future[] onFailureFutures = new Future[onFailureRounds]; + Future[] onFailureFutures = new Future[onFailureRounds]; for (int i = 0; i < onFailureFutures.length; i++) { onFailureFutures[i] = onFailureExecutor.submit(new Runnable() { @Override @@ -372,7 +372,7 @@ public void run() { } }); } - for (Future onFailureFuture : onFailureFutures) { + for (Future onFailureFuture : onFailureFutures) { onFailureFuture.get(); } } finally { @@ -385,7 +385,7 @@ public void run() { boolean notInvokedFound = false; //all of these futures but one must be cancelled, one single time. The one that's not is the last scheduled but not //necessarily the last one in our list - for (Future future : futures) { + for (Future future : futures) { try { verify(future).cancel(false); } catch(WantedButNotInvoked e) { @@ -492,8 +492,14 @@ public void shutdown() { Sniffer.Task task = sniffer.new Task(randomLongBetween(1, Long.MAX_VALUE)); ScheduledExecutorService scheduledExecutorService = mock(ScheduledExecutorService.class); - ScheduledFuture mockedFuture = mock(ScheduledFuture.class); - when(scheduledExecutorService.schedule(any(Runnable.class), any(Long.class), any(TimeUnit.class))).thenReturn(mockedFuture); + final ScheduledFuture mockedFuture = mock(ScheduledFuture.class); + when(scheduledExecutorService.schedule(any(Runnable.class), any(Long.class), any(TimeUnit.class))) + .then(new Answer>() { + @Override + public ScheduledFuture answer(InvocationOnMock invocationOnMock) { + return mockedFuture; + } + }); DefaultScheduler scheduler = new DefaultScheduler(scheduledExecutorService); long delay = randomLongBetween(1, Long.MAX_VALUE); Future future = scheduler.schedule(task, delay); From f3baab39c193d0843120d8cf8f48530d1024ed6a Mon Sep 17 00:00:00 2001 From: javanna Date: Mon, 30 Apr 2018 15:31:46 +0200 Subject: [PATCH 09/13] Adapt HttpExporter --- .../xpack/monitoring/exporter/http/HttpExporter.java | 6 ++---- 1 file changed, 2 insertions(+), 4 deletions(-) diff --git a/x-pack/plugin/monitoring/src/main/java/org/elasticsearch/xpack/monitoring/exporter/http/HttpExporter.java b/x-pack/plugin/monitoring/src/main/java/org/elasticsearch/xpack/monitoring/exporter/http/HttpExporter.java index 88b1a8a9db18f..fe00e8ce83ec6 100644 --- a/x-pack/plugin/monitoring/src/main/java/org/elasticsearch/xpack/monitoring/exporter/http/HttpExporter.java +++ b/x-pack/plugin/monitoring/src/main/java/org/elasticsearch/xpack/monitoring/exporter/http/HttpExporter.java @@ -41,8 +41,6 @@ import org.joda.time.format.DateTimeFormatter; import javax.net.ssl.SSLContext; - -import java.io.IOException; import java.util.ArrayList; import java.util.Arrays; import java.util.Collections; @@ -658,12 +656,12 @@ public void doClose() { if (sniffer != null) { sniffer.close(); } - } catch (IOException | RuntimeException e) { + } catch (Exception e) { logger.error("an error occurred while closing the internal client sniffer", e); } finally { try { client.close(); - } catch (IOException | RuntimeException e) { + } catch (Exception e) { logger.error("an error occurred while closing the internal client", e); } } From 07dce21909301b165eb5ee41c592206c6df1091f Mon Sep 17 00:00:00 2001 From: javanna Date: Mon, 30 Apr 2018 15:44:36 +0200 Subject: [PATCH 10/13] adapt NodeFailureListenerTests --- .../src/main/java/org/elasticsearch/client/sniff/Sniffer.java | 2 +- .../monitoring/exporter/http/NodeFailureListenerTests.java | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/client/sniffer/src/main/java/org/elasticsearch/client/sniff/Sniffer.java b/client/sniffer/src/main/java/org/elasticsearch/client/sniff/Sniffer.java index 556801daafef4..11cbd9e4c0b82 100644 --- a/client/sniffer/src/main/java/org/elasticsearch/client/sniff/Sniffer.java +++ b/client/sniffer/src/main/java/org/elasticsearch/client/sniff/Sniffer.java @@ -78,7 +78,7 @@ public class Sniffer implements Closeable { /** * Triggers a new immediate sniffing round, which will schedule a new round in sniffAfterFailureDelayMillis ms */ - public final void sniffOnFailure() { + public void sniffOnFailure() { scheduleNextRound(0L, sniffAfterFailureDelayMillis, true); } diff --git a/x-pack/plugin/monitoring/src/test/java/org/elasticsearch/xpack/monitoring/exporter/http/NodeFailureListenerTests.java b/x-pack/plugin/monitoring/src/test/java/org/elasticsearch/xpack/monitoring/exporter/http/NodeFailureListenerTests.java index f1ecb799406e8..08512e82e145d 100644 --- a/x-pack/plugin/monitoring/src/test/java/org/elasticsearch/xpack/monitoring/exporter/http/NodeFailureListenerTests.java +++ b/x-pack/plugin/monitoring/src/test/java/org/elasticsearch/xpack/monitoring/exporter/http/NodeFailureListenerTests.java @@ -46,7 +46,7 @@ public void testSnifferNotifiedOnFailure() { listener.onFailure(host); - verify(sniffer).sniffOnFailure(host); + verify(sniffer).sniffOnFailure(); } public void testResourceNotifiedOnFailure() { @@ -71,7 +71,7 @@ public void testResourceAndSnifferNotifiedOnFailure() { } if (optionalSniffer != null) { - verify(sniffer).sniffOnFailure(host); + verify(sniffer).sniffOnFailure(); } } From dccb6730d95cd8751ff60e97a3639c3a882e7826 Mon Sep 17 00:00:00 2001 From: javanna Date: Fri, 25 May 2018 20:03:44 +0200 Subject: [PATCH 11/13] Add tests around cancelling tasks and make impl more robust --- .../elasticsearch/client/sniff/Sniffer.java | 106 ++++- .../client/sniff/SnifferTests.java | 419 +++++++++++------- 2 files changed, 350 insertions(+), 175 deletions(-) diff --git a/client/sniffer/src/main/java/org/elasticsearch/client/sniff/Sniffer.java b/client/sniffer/src/main/java/org/elasticsearch/client/sniff/Sniffer.java index 11cbd9e4c0b82..bb62ad4ce0d39 100644 --- a/client/sniffer/src/main/java/org/elasticsearch/client/sniff/Sniffer.java +++ b/client/sniffer/src/main/java/org/elasticsearch/client/sniff/Sniffer.java @@ -36,6 +36,7 @@ import java.util.concurrent.ScheduledThreadPoolExecutor; import java.util.concurrent.ThreadFactory; import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicReference; @@ -54,12 +55,12 @@ public class Sniffer implements Closeable { private final HostsSniffer hostsSniffer; private final RestClient restClient; - private final long sniffIntervalMillis; private final long sniffAfterFailureDelayMillis; private final Scheduler scheduler; - - private final AtomicReference> nextTask = new AtomicReference<>(); + private final AtomicBoolean initialized = new AtomicBoolean(false); + //this is just a placeholder, it will be replaced once the first sniffing round runs + private volatile ScheduledTask nextScheduledTask = new ScheduledTask(null, null); Sniffer(RestClient restClient, HostsSniffer hostsSniffer, long sniffInterval, long sniffAfterFailureDelay) { this(restClient, hostsSniffer, new DefaultScheduler(), sniffInterval, sniffAfterFailureDelay); @@ -71,28 +72,47 @@ public class Sniffer implements Closeable { this.sniffIntervalMillis = sniffInterval; this.sniffAfterFailureDelayMillis = sniffAfterFailureDelay; this.scheduler = scheduler; - //first sniffing round is immediately executed, next one will be executed depending on the configured sniff interval - scheduleNextRound(0L, sniffIntervalMillis, false); + //the first sniffing round is scheduled, so this constructor returns before nextScheduledTask is assigned to a proper task + Task task = new Task(sniffIntervalMillis) { + @Override + public void run() { + super.run(); + initialized.compareAndSet(false, true); + } + }; + //we do not keep track of the returned future as we never intend to cancel the initial sniffing round, we rather + //prevent any other operation from being executed till the sniffer is properly initialized + scheduler.schedule(task, 0L); } /** - * Triggers a new immediate sniffing round, which will schedule a new round in sniffAfterFailureDelayMillis ms + * Triggers a new immediate sniffing round, which will schedule a new round in sniffAfterFailureDelayMillis */ public void sniffOnFailure() { - scheduleNextRound(0L, sniffAfterFailureDelayMillis, true); + //we don't want sniffOnFailure to do anything until the initial sniffing round has been completed + if (initialized.get()) { + //we want to limit sniffing rounds triggered by concurrent calls of this method. + //if sniffing is already running, there is no point in scheduling another round right after the current one. + ScheduledTask scheduledTask = this.nextScheduledTask; + //concurrent calls may be checking the same task state, but only the first cancelOrSKip call on the same task returns true. + //It can also happen that the task gets replaced while we check its state, in which case the original task is completed + //and calling cancelOrSKip on it returns false. + if (scheduledTask.cancelOrSKip()) { + //we do not keep track of this future as the task will immediately run and we don't intend to cancel it + //due to concurrent sniffOnFailure runs. Effectively the previous cancelled task will stay assigned to nextTask + //till this onFailure round is run and schedules its corresponding following afterFailure round. + scheduler.schedule(new Task(sniffAfterFailureDelayMillis), 0L); + } + } } - private void scheduleNextRound(long delay, long nextDelay, boolean mustCancelNextRound) { - Task task = new Task(nextDelay); - Future nextFuture = scheduler.schedule(task, delay); - Future previousFuture = nextTask.getAndSet(nextFuture); - if (mustCancelNextRound) { - previousFuture.cancel(false); - } + enum TaskState { + WAITING, SKIPPED, STARTED } - final class Task implements Runnable { + class Task implements Runnable { final long nextTaskDelay; + final AtomicReference taskState = new AtomicReference<>(TaskState.WAITING); Task(long nextTaskDelay) { this.nextTaskDelay = nextTaskDelay; @@ -100,14 +120,64 @@ final class Task implements Runnable { @Override public void run() { + if (taskState.compareAndSet(TaskState.WAITING, TaskState.STARTED) == false) { + return; + } try { sniff(); } catch (Exception e) { logger.error("error while sniffing nodes", e); } finally { - scheduleNextRound(nextTaskDelay, sniffIntervalMillis, false); + Task task = new Task(sniffIntervalMillis); + Future future = scheduler.schedule(task, nextTaskDelay); + //tasks are run by a single threaded executor, so swapping is safe with a simple volatile variable + ScheduledTask previousTask = nextScheduledTask; + nextScheduledTask = new ScheduledTask(task, future); + assert initialized.get() == false || + previousTask.task.isSkipped() || previousTask.task.hasStarted() : "task that we are replacing is neither " + + "cancelled nor has it ever started"; } } + + /** + * Returns true if the task has started, false in case it didn't start (yet?) or it was skipped + */ + boolean hasStarted() { + return taskState.get() == TaskState.STARTED; + } + + /** + * Sets this task to be skipped. Returns true if the task will be skipped, false if the task has already started or completed. + */ + boolean skip() { + return taskState.compareAndSet(TaskState.WAITING, TaskState.SKIPPED); + } + + /** + * Returns true if the task was set to be skipped before it was started + */ + boolean isSkipped() { + return taskState.get() == TaskState.SKIPPED; + } + } + + static final class ScheduledTask { + final Task task; + final Future future; + + ScheduledTask(Task task, Future future) { + this.task = task; + this.future = future; + } + + /** + * Cancels this task. Returns true if the task has been successfully cancelled, meaning it won't be executed + * or if it is its execution won't have any effect. Returns false if the task cannot be cancelled (possibly it was + * already cancelled or already completed). + */ + boolean cancelOrSKip() { + return future.cancel(false) && task.skip(); + } } final void sniff() throws IOException { @@ -124,7 +194,9 @@ final void sniff() throws IOException { @Override public void close() { - nextTask.get().cancel(false); + if (initialized.get()) { + nextScheduledTask.future.cancel(false); + } this.scheduler.shutdown(); } diff --git a/client/sniffer/src/test/java/org/elasticsearch/client/sniff/SnifferTests.java b/client/sniffer/src/test/java/org/elasticsearch/client/sniff/SnifferTests.java index 49852ba32dad2..132130e4193db 100644 --- a/client/sniffer/src/test/java/org/elasticsearch/client/sniff/SnifferTests.java +++ b/client/sniffer/src/test/java/org/elasticsearch/client/sniff/SnifferTests.java @@ -25,7 +25,6 @@ import org.elasticsearch.client.sniff.Sniffer.DefaultScheduler; import org.elasticsearch.client.sniff.Sniffer.Scheduler; import org.mockito.Matchers; -import org.mockito.exceptions.verification.WantedButNotInvoked; import org.mockito.invocation.InvocationOnMock; import org.mockito.stubbing.Answer; @@ -33,8 +32,11 @@ import java.util.ArrayList; import java.util.Collections; import java.util.List; -import java.util.concurrent.CopyOnWriteArrayList; +import java.util.Set; +import java.util.concurrent.CancellationException; +import java.util.concurrent.CopyOnWriteArraySet; import java.util.concurrent.CountDownLatch; +import java.util.concurrent.ExecutionException; import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; import java.util.concurrent.Future; @@ -48,10 +50,13 @@ import static org.hamcrest.CoreMatchers.equalTo; import static org.hamcrest.CoreMatchers.instanceOf; +import static org.hamcrest.Matchers.greaterThan; +import static org.hamcrest.Matchers.greaterThanOrEqualTo; import static org.hamcrest.Matchers.is; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertNotEquals; +import static org.junit.Assert.assertNull; import static org.junit.Assert.assertSame; import static org.junit.Assert.assertThat; import static org.junit.Assert.assertTrue; @@ -131,7 +136,7 @@ public void shutdown() { * Simulates the ordinary behaviour of {@link Sniffer} when sniffing on failure is not enabled. * The {@link CountingHostsSniffer} doesn't make any network connection but may throw exception or return no hosts, which makes * it possible to verify that errors are properly handled and don't affect subsequent runs and their scheduling. - * The {@link Scheduler} implementation submits rather than scheduling runs, meaning that it doesn't respect the requested sniff + * The {@link Scheduler} implementation submits rather than scheduling tasks, meaning that it doesn't respect the requested sniff * delays while allowing to assert that the requested delays for each requested run and the following one are the expected values. */ public void testOrdinarySniffRounds() throws Exception { @@ -140,20 +145,17 @@ public void testOrdinarySniffRounds() throws Exception { RestClient restClient = mock(RestClient.class); CountingHostsSniffer hostsSniffer = new CountingHostsSniffer(); final int iters = randomIntBetween(30, 100); - final List> futures = new CopyOnWriteArrayList<>(); - final CountDownLatch latch = new CountDownLatch(1); + final Set> futures = new CopyOnWriteArraySet<>(); + final CountDownLatch completionLatch = new CountDownLatch(1); final AtomicInteger runs = new AtomicInteger(iters); final ExecutorService executor = Executors.newSingleThreadExecutor(); final AtomicReference> lastFuture = new AtomicReference<>(); + final AtomicReference lastTask = new AtomicReference<>(); Scheduler scheduler = new Scheduler() { @Override public Future schedule(Sniffer.Task task, long delayMillis) { assertEquals(sniffInterval, task.nextTaskDelay); int numberOfRuns = runs.getAndDecrement(); - if (numberOfRuns == 0) { - latch.countDown(); - return null; - } if (numberOfRuns == iters) { //the first call is to schedule the first sniff round from the Sniffer constructor, with delay O assertEquals(0L, delayMillis); @@ -162,12 +164,17 @@ public Future schedule(Sniffer.Task task, long delayMillis) { //all of the subsequent times "schedule" is called with delay set to the configured sniff interval assertEquals(sniffInterval, delayMillis); assertEquals(sniffInterval, task.nextTaskDelay); + if (numberOfRuns == 0) { + completionLatch.countDown(); + return null; + } } //we submit rather than scheduling to make the test quick and not depend on time Future future = executor.submit(task); futures.add(future); if (numberOfRuns == 1) { - lastFuture.compareAndSet(null, future); + lastFuture.set(future); + lastTask.set(task); } return future; } @@ -178,13 +185,12 @@ public void shutdown() { } }; try { - //all we need to do is initialize the sniffer, sniffing will start automatically in the background new Sniffer(restClient, hostsSniffer, scheduler, sniffInterval, sniffAfterFailureDelay); - assertTrue(latch.await(1000, TimeUnit.MILLISECONDS)); + assertTrue("timeout waiting for sniffing rounds to be completed", completionLatch.await(1000, TimeUnit.MILLISECONDS)); assertEquals(iters, futures.size()); //the last future is the only one that may not be completed yet, as the count down happens //while scheduling the next round which is still part of the execution of the runnable itself. - //we don't take the last item from the list as futures may be ouf of order in there + assertTrue(lastTask.get().hasStarted()); lastFuture.get().get(); for (Future future : futures) { assertTrue(future.isDone()); @@ -211,9 +217,14 @@ public void testClose() { long sniffAfterFailureDelay = randomLongBetween(1, Long.MAX_VALUE); RestClient restClient = mock(RestClient.class); final AtomicInteger shutdown = new AtomicInteger(0); + final AtomicBoolean initialized = new AtomicBoolean(false); Scheduler scheduler = new Scheduler() { @Override public Future schedule(Sniffer.Task task, long delayMillis) { + if (initialized.compareAndSet(false, true)) { + //run from the same thread so the sniffer gets for sure initialized and the scheduled task gets cancelled on close + task.run(); + } return future; } @@ -233,206 +244,298 @@ public void shutdown() { } } + public void testSniffOnFailureNotInitialized() { + RestClient restClient = mock(RestClient.class); + CountingHostsSniffer hostsSniffer = new CountingHostsSniffer(); + long sniffInterval = randomLongBetween(1, Long.MAX_VALUE); + long sniffAfterFailureDelay = randomLongBetween(1, Long.MAX_VALUE); + final AtomicInteger scheduleCalls = new AtomicInteger(0); + Scheduler scheduler = new Scheduler() { + @Override + public Future schedule(Sniffer.Task task, long delayMillis) { + scheduleCalls.incrementAndGet(); + return null; + } + + @Override + public void shutdown() { + } + }; + + Sniffer sniffer = new Sniffer(restClient, hostsSniffer, scheduler, sniffInterval, sniffAfterFailureDelay); + for (int i = 0; i < 10; i++) { + sniffer.sniffOnFailure(); + } + assertEquals(1, scheduleCalls.get()); + int totalRuns = hostsSniffer.runs.get(); + assertEquals(0, totalRuns); + int setHostsRuns = totalRuns - hostsSniffer.failures.get() - hostsSniffer.emptyList.get(); + verify(restClient, times(setHostsRuns)).setHosts(Matchers.anyVararg()); + verifyNoMoreInteractions(restClient); + } + /** - * Test calling {@link Sniffer#sniffOnFailure()} once. The next scheduled sniffing round is cancelled, - * a new round is scheduled with delay 0, a following one is scheduled with sniffAfterFailure delay, after - * which the ordinary sniffing rounds get scheduled with sniffInterval delay + * Test behaviour when a bunch of onFailure sniffing rounds are triggered in parallel. Each run will always + * schedule a subsequent afterFailure round. Also, for each onFailure round that starts, the net scheduled round + * (either afterFailure or ordinary) gets cancelled. */ - public void testOnFailureSingleRound() throws Exception { - final Future mockedFuture = mock(Future.class); + public void testSniffOnFailure() throws Exception { RestClient restClient = mock(RestClient.class); CountingHostsSniffer hostsSniffer = new CountingHostsSniffer(); + final AtomicBoolean initializing = new AtomicBoolean(true); final long sniffInterval = randomLongBetween(1, Long.MAX_VALUE); final long sniffAfterFailureDelay = randomLongBetween(1, Long.MAX_VALUE); - final AtomicBoolean initializing = new AtomicBoolean(true); - final AtomicBoolean ongoingOnFailure = new AtomicBoolean(true); - final AtomicBoolean afterFailureExpected = new AtomicBoolean(false); + int minNumOnFailureRounds = randomIntBetween(5, 10); final CountDownLatch initializingLatch = new CountDownLatch(1); - final CountDownLatch cancelLatch = new CountDownLatch(1); - when(mockedFuture.cancel(false)).thenAnswer(new Answer() { - @Override - public Void answer(InvocationOnMock invocationOnMock) { - cancelLatch.countDown(); - return null; - } - }); - final CountDownLatch doneLatch = new CountDownLatch(1); - final List> futures = new CopyOnWriteArrayList<>(); - final AtomicReference> lastFuture = new AtomicReference<>(); - final ExecutorService executor = Executors.newSingleThreadExecutor(); + final Set ordinaryRoundsTasks = new CopyOnWriteArraySet<>(); + final AtomicReference> initializingFuture = new AtomicReference<>(); + final Set onFailureTasks = new CopyOnWriteArraySet<>(); + final Set afterFailureTasks = new CopyOnWriteArraySet<>(); + final AtomicBoolean onFailureCompleted = new AtomicBoolean(false); + final CountDownLatch completionLatch = new CountDownLatch(1); + final ScheduledExecutorService executor = Executors.newSingleThreadScheduledExecutor(); try { Scheduler scheduler = new Scheduler() { @Override - public Future schedule(Sniffer.Task task, long delayMillis) { + public Future schedule(final Sniffer.Task task, long delayMillis) { if (initializing.compareAndSet(true, false)) { assertEquals(0L, delayMillis); - assertEquals(sniffInterval, task.nextTaskDelay); - initializingLatch.countDown(); - return mockedFuture; + Future future = executor.submit(new Runnable() { + @Override + public void run() { + try { + task.run(); + } finally { + //we need to make sure that the sniffer is initialized, so the sniffOnFailure + //call does what it needs to do. Otherwise nothing happens until initialized. + initializingLatch.countDown(); + } + } + }); + assertTrue(initializingFuture.compareAndSet(null, future)); + return future; } - if (ongoingOnFailure.compareAndSet(true, false)) { - assertEquals(0L, delayMillis); - assertEquals(sniffAfterFailureDelay, task.nextTaskDelay); - afterFailureExpected.set(true); + if (delayMillis == 0L) { Future future = executor.submit(task); - futures.add(future); + onFailureTasks.add(new Sniffer.ScheduledTask(task, future)); return future; - } else if (afterFailureExpected.compareAndSet(true, false)) { - //onFailure is called in another thread (not from the single threaded executor), hence we need to - //wait for it to be completed or the rest of the rounds (after failure etc.) may be executed - //before the onFailure scheduling is completed. This is a problem only when testing as we submit - //tasks instead of scheduling them with some delay. - try { - cancelLatch.await(500, TimeUnit.MILLISECONDS); - } catch (InterruptedException e) { - fail(e.getMessage()); - } - assertEquals(sniffAfterFailureDelay, delayMillis); - assertEquals(sniffInterval, task.nextTaskDelay); - Future future = executor.submit(task); - futures.add(future); - lastFuture.compareAndSet(null, future); + } + if (delayMillis == sniffAfterFailureDelay) { + Future future = scheduleOrSubmit(task); + afterFailureTasks.add(new Sniffer.ScheduledTask(task, future)); return future; + } + + assertEquals(sniffInterval, delayMillis); + assertEquals(sniffInterval, task.nextTaskDelay); + + if (onFailureCompleted.get() && onFailureTasks.size() == afterFailureTasks.size()) { + completionLatch.countDown(); + return mock(Future.class); + } + + Future future = scheduleOrSubmit(task); + ordinaryRoundsTasks.add(new Sniffer.ScheduledTask(task, future)); + return future; + } + + private Future scheduleOrSubmit(Sniffer.Task task) { + if (randomBoolean()) { + return executor.schedule(task, randomLongBetween(0L, 200L), TimeUnit.MILLISECONDS); } else { - assertEquals(sniffInterval, delayMillis); - assertEquals(sniffInterval, task.nextTaskDelay); - doneLatch.countDown(); - return null; + return executor.submit(task); } } @Override public void shutdown() { - } }; - Sniffer sniffer = new Sniffer(restClient, hostsSniffer, scheduler, sniffInterval, sniffAfterFailureDelay); - assertTrue(initializingLatch.await(1000, TimeUnit.MILLISECONDS)); + final Sniffer sniffer = new Sniffer(restClient, hostsSniffer, scheduler, sniffInterval, sniffAfterFailureDelay); + assertTrue("timeout waiting for sniffer to get initialized", initializingLatch.await(1000, TimeUnit.MILLISECONDS)); - sniffer.sniffOnFailure(); + ExecutorService onFailureExecutor = Executors.newFixedThreadPool(randomIntBetween(5, 20)); + Set> onFailureFutures = new CopyOnWriteArraySet<>(); + try { + //with tasks executing quickly one after each other, it is very likely that the onFailure round gets skipped + //as another round is already running. We retry till enough runs get through as that's what we want to test. + while (onFailureTasks.size() < minNumOnFailureRounds) { + onFailureFutures.add(onFailureExecutor.submit(new Runnable() { + @Override + public void run() { + sniffer.sniffOnFailure(); + } + })); + } + assertThat(onFailureFutures.size(), greaterThanOrEqualTo(minNumOnFailureRounds)); + for (Future onFailureFuture : onFailureFutures) { + assertNull(onFailureFuture.get()); + } + onFailureCompleted.set(true); + } finally { + onFailureExecutor.shutdown(); + onFailureExecutor.awaitTermination(1000, TimeUnit.MILLISECONDS); + } - assertTrue(doneLatch.await(1000, TimeUnit.MILLISECONDS)); - assertEquals(2, futures.size()); - lastFuture.get().get(); - for (Future future : futures) { - assertTrue(future.isDone()); - future.get(); + assertFalse(initializingFuture.get().isCancelled()); + assertTrue(initializingFuture.get().isDone()); + assertNull(initializingFuture.get().get()); + + assertTrue("timeout waiting for sniffing rounds to be completed", completionLatch.await(1000, TimeUnit.MILLISECONDS)); + assertThat(onFailureTasks.size(), greaterThanOrEqualTo(minNumOnFailureRounds)); + assertEquals(onFailureTasks.size(), afterFailureTasks.size()); + + for (Sniffer.ScheduledTask onFailureTask : onFailureTasks) { + assertFalse(onFailureTask.future.isCancelled()); + assertTrue(onFailureTask.future.isDone()); + assertNull(onFailureTask.future.get()); + assertTrue(onFailureTask.task.hasStarted()); + assertFalse(onFailureTask.task.isSkipped()); + } + + int cancelledTasks = 0; + int completedTasks = onFailureTasks.size() + 1; + for (Sniffer.ScheduledTask afterFailureTask : afterFailureTasks) { + if (assertTaskCancelledOrCompleted(afterFailureTask)) { + completedTasks++; + } else { + cancelledTasks++; + } } - int totalRuns = hostsSniffer.runs.get(); - assertEquals(2, totalRuns); - int setHostsRuns = totalRuns - hostsSniffer.failures.get() - hostsSniffer.emptyList.get(); + + assertThat(ordinaryRoundsTasks.size(), greaterThan(0)); + for (Sniffer.ScheduledTask task : ordinaryRoundsTasks) { + if (assertTaskCancelledOrCompleted(task)) { + completedTasks++; + } else { + cancelledTasks++; + } + } + assertEquals(onFailureTasks.size(), cancelledTasks); + + assertEquals(completedTasks, hostsSniffer.runs.get()); + int setHostsRuns = hostsSniffer.runs.get() - hostsSniffer.failures.get() - hostsSniffer.emptyList.get(); verify(restClient, times(setHostsRuns)).setHosts(Matchers.anyVararg()); verifyNoMoreInteractions(restClient); } finally { executor.shutdown(); - assertTrue(executor.awaitTermination(1000, TimeUnit.MILLISECONDS)); + executor.awaitTermination(1000L, TimeUnit.MILLISECONDS); } } - /** - * Test that when invoking {@link Sniffer#sniffOnFailure()} concurrently, the next scheduled sniffing round is always cancelled. - */ - public void testSniffOnFailureCancelsNextRound() throws Exception { + private static boolean assertTaskCancelledOrCompleted(Sniffer.ScheduledTask task) throws ExecutionException, InterruptedException { + if (task.task.isSkipped()) { + assertTrue(task.future.isCancelled()); + try { + task.future.get(); + fail("cancellation exception should have been thrown"); + } catch(CancellationException ignore) { + } + return false; + } else { + try { + assertNull(task.future.get()); + } catch(CancellationException ignore) { + assertTrue(task.future.isCancelled()); + } + assertTrue(task.future.isDone()); + assertTrue(task.task.hasStarted()); + return true; + } + } + + public void testTaskCancelling() throws Exception { RestClient restClient = mock(RestClient.class); HostsSniffer hostsSniffer = mock(HostsSniffer.class); - int onFailureRounds = randomIntBetween(30, 50); - final long sniffInterval = randomLongBetween(1, Long.MAX_VALUE); - final long sniffAfterFailureDelay = randomLongBetween(1, Long.MAX_VALUE); - final AtomicBoolean initialized = new AtomicBoolean(false); - final List> futures = new CopyOnWriteArrayList<>(); - Scheduler scheduler = new Scheduler() { + Scheduler noOpScheduler = new Scheduler() { @Override public Future schedule(Sniffer.Task task, long delayMillis) { - assertEquals(0L, delayMillis); - if (initialized.compareAndSet(false, true)) { - assertEquals(sniffInterval, task.nextTaskDelay); - } else { - assertEquals(sniffAfterFailureDelay, task.nextTaskDelay); - } - Future mockedFuture = mock(Future.class); - futures.add(mockedFuture); - return mockedFuture; + return null; } @Override public void shutdown() { - } }; - - final Sniffer sniffer = new Sniffer(restClient, hostsSniffer, scheduler, sniffInterval, sniffAfterFailureDelay); - ExecutorService onFailureExecutor = Executors.newFixedThreadPool(randomIntBetween(2, 10)); + Sniffer sniffer = new Sniffer(restClient, hostsSniffer, noOpScheduler, 0L, 0L); + ScheduledExecutorService executor = Executors.newSingleThreadScheduledExecutor(); try { - Future[] onFailureFutures = new Future[onFailureRounds]; - for (int i = 0; i < onFailureFutures.length; i++) { - onFailureFutures[i] = onFailureExecutor.submit(new Runnable() { - @Override - public void run() { - sniffer.sniffOnFailure(); - } - }); - } - for (Future onFailureFuture : onFailureFutures) { - onFailureFuture.get(); - } - } finally { - onFailureExecutor.shutdown(); - onFailureExecutor.awaitTermination(1000, TimeUnit.MILLISECONDS); - } + int numIters = randomIntBetween(50, 100); + for (int i = 0; i < numIters; i++) { + Sniffer.Task task = sniffer.new Task(0L); + TaskWrapper wrapper = new TaskWrapper(task); + Future future; + if (rarely()) { + future = executor.schedule(wrapper, randomLongBetween(0L, 200L), TimeUnit.MILLISECONDS); + } else { + future = executor.submit(wrapper); + } + Sniffer.ScheduledTask scheduledTask = new Sniffer.ScheduledTask(task, future); + boolean cancelOrSKip = scheduledTask.cancelOrSKip(); + try { + assertNull(future.get()); + } catch(CancellationException ignore) { + assertTrue(future.isCancelled()); + } - //we have as many futures as the onFailure rounds, plus the initial one obtained at Sniffer construction - assertEquals(onFailureRounds + 1, futures.size()); - boolean notInvokedFound = false; - //all of these futures but one must be cancelled, one single time. The one that's not is the last scheduled but not - //necessarily the last one in our list - for (Future future : futures) { - try { - verify(future).cancel(false); - } catch(WantedButNotInvoked e) { - if (notInvokedFound) { - throw new AssertionError("future was cancelled more than once", e); + if (cancelOrSKip) { + //the task was either cancelled before starting, in which case it will never start (thanks to Future#cancel), + //or skipped, in which case it will run but do nothing (thanks to Task#skip). + //Here we want to make sure that whenever cancelOrSkip returns true, the task either won't run or it won't do anything, + //otherwise we may end up with parallel sniffing tracks given that each task schedules the following one. We need to + // make sure that onFailure takes scheduling over while at the same time ordinary rounds don't go on. + assertFalse(task.hasStarted()); + assertTrue(task.isSkipped()); + assertTrue(future.isCancelled()); + assertTrue(future.isDone()); + } else { + //if a future is cancelled when its execution has already started, future#get throws CancellationException before + //completion. The execution continues though so we use a latch to try and wait for the task to be completed. + //Here we want to make sure that whenever cancelOrSkip returns false, the task will be completed, otherwise we may be + //missing to schedule the following round, which means no sniffing will ever happen again besides on failure sniffing. + assertTrue(wrapper.await()); + //the future may or may not be cancelled but the task has for sure started and completed + assertTrue(task.toString(), task.hasStarted()); + assertFalse(task.isSkipped()); + assertTrue(future.isDone()); + } + //subsequent cancel calls return false for sure + int cancelCalls = randomIntBetween(1, 10); + for (int j = 0; j < cancelCalls; j++) { + assertFalse(scheduledTask.cancelOrSKip()); } - notInvokedFound = true; } + } finally { + executor.shutdown(); + executor.awaitTermination(1000, TimeUnit.MILLISECONDS); } } /** - * Test that no matter when an onFailure sniffing round is triggered, it will run and it will always - * schedule a subsequent afterFailure round. See https://github.com/elastic/elasticsearch/issues/27697 . - * The difference in this test compared to the other ones above is that we keep on scheduling ordinary sniffing - * rounds to show that they don't interfere with the onFailure round that is scheduled at some point. + * Wraps a {@link Sniffer.Task} and allows to wait for its completion. This is needed to verify + * that tasks are either never started or always completed. Calling {@link Future#get()} against a cancelled future will + * throw {@link CancellationException} straight-away but the execution of the task will continue if it had already started, + * in which case {@link Future#cancel(boolean)} returns true which is not very helpful. */ - public void testSniffOnFailureIsAlwaysExecuted() throws Exception { - RestClient restClient = mock(RestClient.class); - CountingHostsSniffer hostsSniffer = new CountingHostsSniffer(); - long sniffInterval = randomLongBetween(1, Long.MAX_VALUE); - final long sniffAfterFailureDelay = randomLongBetween(1, Long.MAX_VALUE); - final CountDownLatch latch = new CountDownLatch(1); - final ExecutorService executor = Executors.newSingleThreadExecutor(); - try { - Scheduler scheduler = new Scheduler() { - @Override - public Future schedule(Sniffer.Task task, long delayMillis) { - //if we are scheduling a next round with delay sniffAfterFailureDelay it means that we are executing the onFailure task - if (delayMillis == sniffAfterFailureDelay) { - latch.countDown(); - return null; - } - return executor.submit(task); - } + private static final class TaskWrapper implements Runnable { + final Sniffer.Task task; + final CountDownLatch completionLatch = new CountDownLatch(1); - @Override - public void shutdown() { + TaskWrapper(Sniffer.Task task) { + this.task = task; + } - } - }; - Sniffer sniffer = new Sniffer(restClient, hostsSniffer, scheduler, sniffInterval, sniffAfterFailureDelay); - sniffer.sniffOnFailure(); - assertTrue(latch.await(1000L, TimeUnit.MILLISECONDS)); - } finally { - executor.shutdown(); - executor.awaitTermination(1000L, TimeUnit.MILLISECONDS); + @Override + public void run() { + try { + task.run(); + } finally { + completionLatch.countDown(); + } + } + + boolean await() throws InterruptedException { + return completionLatch.await(1000, TimeUnit.MILLISECONDS); } } From 1759215c3e528d8b481bb20f054555d13800746c Mon Sep 17 00:00:00 2001 From: javanna Date: Mon, 28 May 2018 11:39:31 +0200 Subject: [PATCH 12/13] update comments --- .../elasticsearch/client/sniff/Sniffer.java | 52 ++++++++++++------- 1 file changed, 34 insertions(+), 18 deletions(-) diff --git a/client/sniffer/src/main/java/org/elasticsearch/client/sniff/Sniffer.java b/client/sniffer/src/main/java/org/elasticsearch/client/sniff/Sniffer.java index bb62ad4ce0d39..cf357b66475ce 100644 --- a/client/sniffer/src/main/java/org/elasticsearch/client/sniff/Sniffer.java +++ b/client/sniffer/src/main/java/org/elasticsearch/client/sniff/Sniffer.java @@ -59,8 +59,7 @@ public class Sniffer implements Closeable { private final long sniffAfterFailureDelayMillis; private final Scheduler scheduler; private final AtomicBoolean initialized = new AtomicBoolean(false); - //this is just a placeholder, it will be replaced once the first sniffing round runs - private volatile ScheduledTask nextScheduledTask = new ScheduledTask(null, null); + private volatile ScheduledTask nextScheduledTask; Sniffer(RestClient restClient, HostsSniffer hostsSniffer, long sniffInterval, long sniffAfterFailureDelay) { this(restClient, hostsSniffer, new DefaultScheduler(), sniffInterval, sniffAfterFailureDelay); @@ -72,7 +71,10 @@ public class Sniffer implements Closeable { this.sniffIntervalMillis = sniffInterval; this.sniffAfterFailureDelayMillis = sniffAfterFailureDelay; this.scheduler = scheduler; - //the first sniffing round is scheduled, so this constructor returns before nextScheduledTask is assigned to a proper task + /* + The first sniffing round is async, so this constructor returns before nextScheduledTask is assigned to a task. + The initialized flag is a protection against NPE due to that. + */ Task task = new Task(sniffIntervalMillis) { @Override public void run() { @@ -80,8 +82,10 @@ public void run() { initialized.compareAndSet(false, true); } }; - //we do not keep track of the returned future as we never intend to cancel the initial sniffing round, we rather - //prevent any other operation from being executed till the sniffer is properly initialized + /* + We do not keep track of the returned future as we never intend to cancel the initial sniffing round, we rather + prevent any other operation from being executed till the sniffer is properly initialized + */ scheduler.schedule(task, 0L); } @@ -89,18 +93,19 @@ public void run() { * Triggers a new immediate sniffing round, which will schedule a new round in sniffAfterFailureDelayMillis */ public void sniffOnFailure() { - //we don't want sniffOnFailure to do anything until the initial sniffing round has been completed + //sniffOnFailure does nothing until the initial sniffing round has been completed if (initialized.get()) { - //we want to limit sniffing rounds triggered by concurrent calls of this method. - //if sniffing is already running, there is no point in scheduling another round right after the current one. - ScheduledTask scheduledTask = this.nextScheduledTask; - //concurrent calls may be checking the same task state, but only the first cancelOrSKip call on the same task returns true. - //It can also happen that the task gets replaced while we check its state, in which case the original task is completed - //and calling cancelOrSKip on it returns false. - if (scheduledTask.cancelOrSKip()) { - //we do not keep track of this future as the task will immediately run and we don't intend to cancel it - //due to concurrent sniffOnFailure runs. Effectively the previous cancelled task will stay assigned to nextTask - //till this onFailure round is run and schedules its corresponding following afterFailure round. + /* + If sniffing is already running, there is no point in scheduling another round right after the current one. + Concurrent calls may be checking the same task state, but only the first cancelOrSKip call on the same task returns true. + The task may also get replaced while we check its state, in which case calling cancelOrSKip on it returns false. + */ + if (this.nextScheduledTask.cancelOrSKip()) { + /* + We do not keep track of this future as the task will immediately run and we don't intend to cancel it + due to concurrent sniffOnFailure runs. Effectively the previous (now cancelled or skipped) task will stay + assigned to nextTask till this onFailure round gets run and schedules its corresponding afterFailure round. + */ scheduler.schedule(new Task(sniffAfterFailureDelayMillis), 0L); } } @@ -120,6 +125,11 @@ class Task implements Runnable { @Override public void run() { + /* + Skipped or already started tasks do nothing. In most cases tasks will be cancelled and not run, but we want to protect for + cases where future#cancel returns true yet the task runs. We want to make sure that such tasks do nothing otherwise they will + schedule another round at the end and so on, leaving us with multiple parallel sniffing "tracks" whish is undesirable. + */ if (taskState.compareAndSet(TaskState.WAITING, TaskState.STARTED) == false) { return; } @@ -147,9 +157,15 @@ boolean hasStarted() { } /** - * Sets this task to be skipped. Returns true if the task will be skipped, false if the task has already started or completed. + * Sets this task to be skipped. Returns true if the task will be skipped, false if the task has already started. */ boolean skip() { + /* + Threads may still get run although future#cancel returns true. We make sure that a task is either cancelled (or skipped), + or entirely run. In the odd case that future#cancel returns true and the thread still runs, the task won't do anything. + In case future#cancel returns true but the task has already started, this state change will not succeed hence this method + returns false and the task will normally run. + */ return taskState.compareAndSet(TaskState.WAITING, TaskState.SKIPPED); } @@ -195,7 +211,7 @@ final void sniff() throws IOException { @Override public void close() { if (initialized.get()) { - nextScheduledTask.future.cancel(false); + nextScheduledTask.cancelOrSKip(); } this.scheduler.shutdown(); } From 44b517817ff6e8aa2af2fe55492bb94c154c3b61 Mon Sep 17 00:00:00 2001 From: javanna Date: Wed, 30 May 2018 11:57:47 +0200 Subject: [PATCH 13/13] addressed comments --- .../elasticsearch/client/RequestTests.java | 9 ++-- .../elasticsearch/client/sniff/Sniffer.java | 52 +++++++++++-------- .../client/sniff/SnifferTests.java | 10 ++-- 3 files changed, 39 insertions(+), 32 deletions(-) diff --git a/client/rest/src/test/java/org/elasticsearch/client/RequestTests.java b/client/rest/src/test/java/org/elasticsearch/client/RequestTests.java index 75ddfa0a15c58..29bbf23a1f20e 100644 --- a/client/rest/src/test/java/org/elasticsearch/client/RequestTests.java +++ b/client/rest/src/test/java/org/elasticsearch/client/RequestTests.java @@ -24,16 +24,15 @@ import org.apache.http.entity.ByteArrayEntity; import org.apache.http.entity.ContentType; import org.apache.http.entity.StringEntity; - -import java.io.ByteArrayOutputStream; -import java.io.IOException; -import java.util.HashMap; -import java.util.Map; import org.apache.http.nio.entity.NStringEntity; import org.elasticsearch.client.HttpAsyncResponseConsumerFactory.HeapBufferedResponseConsumerFactory; +import java.io.ByteArrayOutputStream; +import java.io.IOException; import java.util.ArrayList; +import java.util.HashMap; import java.util.List; +import java.util.Map; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertNotEquals; diff --git a/client/sniffer/src/main/java/org/elasticsearch/client/sniff/Sniffer.java b/client/sniffer/src/main/java/org/elasticsearch/client/sniff/Sniffer.java index cf357b66475ce..dc873ccd44e10 100644 --- a/client/sniffer/src/main/java/org/elasticsearch/client/sniff/Sniffer.java +++ b/client/sniffer/src/main/java/org/elasticsearch/client/sniff/Sniffer.java @@ -72,8 +72,8 @@ public class Sniffer implements Closeable { this.sniffAfterFailureDelayMillis = sniffAfterFailureDelay; this.scheduler = scheduler; /* - The first sniffing round is async, so this constructor returns before nextScheduledTask is assigned to a task. - The initialized flag is a protection against NPE due to that. + * The first sniffing round is async, so this constructor returns before nextScheduledTask is assigned to a task. + * The initialized flag is a protection against NPE due to that. */ Task task = new Task(sniffIntervalMillis) { @Override @@ -83,28 +83,29 @@ public void run() { } }; /* - We do not keep track of the returned future as we never intend to cancel the initial sniffing round, we rather - prevent any other operation from being executed till the sniffer is properly initialized + * We do not keep track of the returned future as we never intend to cancel the initial sniffing round, we rather + * prevent any other operation from being executed till the sniffer is properly initialized */ scheduler.schedule(task, 0L); } /** - * Triggers a new immediate sniffing round, which will schedule a new round in sniffAfterFailureDelayMillis + * Schedule sniffing to run as soon as possible if it isn't already running. Once such sniffing round runs + * it will also schedule a new round after sniffAfterFailureDelay ms. */ public void sniffOnFailure() { //sniffOnFailure does nothing until the initial sniffing round has been completed if (initialized.get()) { /* - If sniffing is already running, there is no point in scheduling another round right after the current one. - Concurrent calls may be checking the same task state, but only the first cancelOrSKip call on the same task returns true. - The task may also get replaced while we check its state, in which case calling cancelOrSKip on it returns false. + * If sniffing is already running, there is no point in scheduling another round right after the current one. + * Concurrent calls may be checking the same task state, but only the first skip call on the same task returns true. + * The task may also get replaced while we check its state, in which case calling skip on it returns false. */ - if (this.nextScheduledTask.cancelOrSKip()) { + if (this.nextScheduledTask.skip()) { /* - We do not keep track of this future as the task will immediately run and we don't intend to cancel it - due to concurrent sniffOnFailure runs. Effectively the previous (now cancelled or skipped) task will stay - assigned to nextTask till this onFailure round gets run and schedules its corresponding afterFailure round. + * We do not keep track of this future as the task will immediately run and we don't intend to cancel it + * due to concurrent sniffOnFailure runs. Effectively the previous (now cancelled or skipped) task will stay + * assigned to nextTask till this onFailure round gets run and schedules its corresponding afterFailure round. */ scheduler.schedule(new Task(sniffAfterFailureDelayMillis), 0L); } @@ -126,9 +127,9 @@ class Task implements Runnable { @Override public void run() { /* - Skipped or already started tasks do nothing. In most cases tasks will be cancelled and not run, but we want to protect for - cases where future#cancel returns true yet the task runs. We want to make sure that such tasks do nothing otherwise they will - schedule another round at the end and so on, leaving us with multiple parallel sniffing "tracks" whish is undesirable. + * Skipped or already started tasks do nothing. In most cases tasks will be cancelled and not run, but we want to protect for + * cases where future#cancel returns true yet the task runs. We want to make sure that such tasks do nothing otherwise they will + * schedule another round at the end and so on, leaving us with multiple parallel sniffing "tracks" whish is undesirable. */ if (taskState.compareAndSet(TaskState.WAITING, TaskState.STARTED) == false) { return; @@ -161,10 +162,10 @@ boolean hasStarted() { */ boolean skip() { /* - Threads may still get run although future#cancel returns true. We make sure that a task is either cancelled (or skipped), - or entirely run. In the odd case that future#cancel returns true and the thread still runs, the task won't do anything. - In case future#cancel returns true but the task has already started, this state change will not succeed hence this method - returns false and the task will normally run. + * Threads may still get run although future#cancel returns true. We make sure that a task is either cancelled (or skipped), + * or entirely run. In the odd case that future#cancel returns true and the thread still runs, the task won't do anything. + * In case future#cancel returns true but the task has already started, this state change will not succeed hence this method + * returns false and the task will normally run. */ return taskState.compareAndSet(TaskState.WAITING, TaskState.SKIPPED); } @@ -191,8 +192,15 @@ static final class ScheduledTask { * or if it is its execution won't have any effect. Returns false if the task cannot be cancelled (possibly it was * already cancelled or already completed). */ - boolean cancelOrSKip() { - return future.cancel(false) && task.skip(); + boolean skip() { + /* + * Future#cancel should return false whenever a task cannot be cancelled, most likely as it has already started. We don't + * trust it much though so we try to cancel hoping that it will work. At the same time we always call skip too, which means + * that if the task has already started the state change will fail. We could potentially not call skip when cancel returns + * false but we prefer to stay on the safe side. + */ + future.cancel(false); + return task.skip(); } } @@ -211,7 +219,7 @@ final void sniff() throws IOException { @Override public void close() { if (initialized.get()) { - nextScheduledTask.cancelOrSKip(); + nextScheduledTask.skip(); } this.scheduler.shutdown(); } diff --git a/client/sniffer/src/test/java/org/elasticsearch/client/sniff/SnifferTests.java b/client/sniffer/src/test/java/org/elasticsearch/client/sniff/SnifferTests.java index 132130e4193db..8172774a77d80 100644 --- a/client/sniffer/src/test/java/org/elasticsearch/client/sniff/SnifferTests.java +++ b/client/sniffer/src/test/java/org/elasticsearch/client/sniff/SnifferTests.java @@ -471,17 +471,17 @@ public void shutdown() { future = executor.submit(wrapper); } Sniffer.ScheduledTask scheduledTask = new Sniffer.ScheduledTask(task, future); - boolean cancelOrSKip = scheduledTask.cancelOrSKip(); + boolean skip = scheduledTask.skip(); try { assertNull(future.get()); } catch(CancellationException ignore) { assertTrue(future.isCancelled()); } - if (cancelOrSKip) { + if (skip) { //the task was either cancelled before starting, in which case it will never start (thanks to Future#cancel), //or skipped, in which case it will run but do nothing (thanks to Task#skip). - //Here we want to make sure that whenever cancelOrSkip returns true, the task either won't run or it won't do anything, + //Here we want to make sure that whenever skip returns true, the task either won't run or it won't do anything, //otherwise we may end up with parallel sniffing tracks given that each task schedules the following one. We need to // make sure that onFailure takes scheduling over while at the same time ordinary rounds don't go on. assertFalse(task.hasStarted()); @@ -491,7 +491,7 @@ public void shutdown() { } else { //if a future is cancelled when its execution has already started, future#get throws CancellationException before //completion. The execution continues though so we use a latch to try and wait for the task to be completed. - //Here we want to make sure that whenever cancelOrSkip returns false, the task will be completed, otherwise we may be + //Here we want to make sure that whenever skip returns false, the task will be completed, otherwise we may be //missing to schedule the following round, which means no sniffing will ever happen again besides on failure sniffing. assertTrue(wrapper.await()); //the future may or may not be cancelled but the task has for sure started and completed @@ -502,7 +502,7 @@ public void shutdown() { //subsequent cancel calls return false for sure int cancelCalls = randomIntBetween(1, 10); for (int j = 0; j < cancelCalls; j++) { - assertFalse(scheduledTask.cancelOrSKip()); + assertFalse(scheduledTask.skip()); } } } finally {