From 554928fe375358d6e0f196f7678f538bc3c82500 Mon Sep 17 00:00:00 2001 From: Carter Kozak Date: Tue, 25 Feb 2020 15:35:55 -0500 Subject: [PATCH 01/17] Replace QueuedChannel with a backoff based retryer --- ...va => ChannelToLimitedChannelAdapter.java} | 6 +- .../com/palantir/dialogue/core/Channels.java | 26 +- .../core/LimitedChannelToChannelAdapter.java | 51 +++ .../palantir/dialogue/core/QueuedChannel.java | 299 ------------------ .../dialogue/core/RetryingChannel.java | 79 ++++- .../palantir/dialogue/core/ChannelsTest.java | 8 +- .../core/ConcurrencyLimitedChannelTest.java | 4 +- .../dialogue/core/QueuedChannelTest.java | 286 ----------------- .../dialogue/core/RetryingChannelTest.java | 25 +- .../ChannelsTest/traces_on_retries.log | 48 +-- .../dialogue/core/SimulationServer.java | 2 +- .../dialogue/core/TestCaffeineTicker.java | 4 +- .../com/palantir/dialogue/core/Strategy.java | 32 +- 13 files changed, 182 insertions(+), 688 deletions(-) rename dialogue-core/src/main/java/com/palantir/dialogue/core/{LimitedChannelAdapter.java => ChannelToLimitedChannelAdapter.java} (87%) create mode 100644 dialogue-core/src/main/java/com/palantir/dialogue/core/LimitedChannelToChannelAdapter.java delete mode 100644 dialogue-core/src/main/java/com/palantir/dialogue/core/QueuedChannel.java delete mode 100644 dialogue-core/src/test/java/com/palantir/dialogue/core/QueuedChannelTest.java diff --git a/dialogue-core/src/main/java/com/palantir/dialogue/core/LimitedChannelAdapter.java b/dialogue-core/src/main/java/com/palantir/dialogue/core/ChannelToLimitedChannelAdapter.java similarity index 87% rename from dialogue-core/src/main/java/com/palantir/dialogue/core/LimitedChannelAdapter.java rename to dialogue-core/src/main/java/com/palantir/dialogue/core/ChannelToLimitedChannelAdapter.java index 42c05205e..261b64558 100644 --- a/dialogue-core/src/main/java/com/palantir/dialogue/core/LimitedChannelAdapter.java +++ b/dialogue-core/src/main/java/com/palantir/dialogue/core/ChannelToLimitedChannelAdapter.java @@ -24,11 +24,11 @@ import java.util.Optional; /** Adapter from {@link Channel} to {@link LimitedChannel} which always returns a {@link Optional#isPresent() value}. */ -final class LimitedChannelAdapter implements LimitedChannel { +final class ChannelToLimitedChannelAdapter implements LimitedChannel { private final Channel delegate; - LimitedChannelAdapter(Channel delegate) { + ChannelToLimitedChannelAdapter(Channel delegate) { this.delegate = Preconditions.checkNotNull(delegate, "Channel"); } @@ -39,6 +39,6 @@ public Optional> maybeExecute(Endpoint endpoint, Requ @Override public String toString() { - return "LimitedChannelAdapter{delegate=" + delegate + '}'; + return "ChannelToLimitedChannelAdapter{delegate=" + delegate + '}'; } } diff --git a/dialogue-core/src/main/java/com/palantir/dialogue/core/Channels.java b/dialogue-core/src/main/java/com/palantir/dialogue/core/Channels.java index 124621c3b..d9e3d7a1b 100644 --- a/dialogue-core/src/main/java/com/palantir/dialogue/core/Channels.java +++ b/dialogue-core/src/main/java/com/palantir/dialogue/core/Channels.java @@ -26,7 +26,6 @@ import java.util.Collection; import java.util.List; import java.util.function.Function; -import javax.annotation.Nullable; public final class Channels { @@ -39,26 +38,24 @@ public static Channel create(Collection channels, ClientConfi Preconditions.checkArgument(config.userAgent().isPresent(), "config.userAgent() must be specified"); DialogueClientMetrics clientMetrics = DialogueClientMetrics.of(config.taggedMetricRegistry()); - // n.b. This becomes cleaner once we support reloadable channels, the queue can be created first, and - // each limited channel can be created later and passed a method reference to the queued channel. - DeferredLimitedChannelListener queueListener = new DeferredLimitedChannelListener(); List limitedChannels = channels.stream() // Instrument inner-most channel with metrics so that we measure only the over-the-wire-time .map(channel -> new InstrumentedChannel(channel, clientMetrics)) // TracedChannel must wrap TracedRequestChannel to ensure requests have tracing headers. .map(TracedRequestChannel::new) .map(channel -> new TracedChannel(channel, "Dialogue-http-request")) - .map(LimitedChannelAdapter::new) + .map(ChannelToLimitedChannelAdapter::new) .map(concurrencyLimiter(config, clientMetrics)) .map(channel -> new FixedLimitedChannel(channel, MAX_REQUESTS_PER_CHANNEL, clientMetrics)) .collect(ImmutableList.toImmutableList()); LimitedChannel limited = nodeSelectionStrategy(config, limitedChannels); - QueuedChannel queuedChannel = new QueuedChannel(limited, DispatcherMetrics.of(config.taggedMetricRegistry())); - queueListener.delegate = queuedChannel::schedule; - Channel channel = queuedChannel; + Channel channel = new LimitedChannelToChannelAdapter(limited); channel = new TracedChannel(channel, "Dialogue-request-attempt"); - channel = new RetryingChannel(channel, config.maxNumRetries(), config.serverQoS()); + if (config.maxNumRetries() > 0) { + channel = + new RetryingChannel(channel, config.maxNumRetries(), config.backoffSlotSize(), config.serverQoS()); + } channel = new UserAgentChannel(channel, config.userAgent().get()); channel = new DeprecationWarningChannel(channel, clientMetrics); channel = new ContentDecodingChannel(channel); @@ -97,15 +94,4 @@ private static Function concurrencyLimiter( throw new SafeIllegalStateException( "Encountered unknown client QoS configuration", SafeArg.of("ClientQoS", clientQoS)); } - - private static final class DeferredLimitedChannelListener implements LimitedChannelListener { - @Nullable - private LimitedChannelListener delegate; - - @Override - public void onChannelReady() { - Preconditions.checkNotNull(delegate, "Delegate listener has not been initialized") - .onChannelReady(); - } - } } diff --git a/dialogue-core/src/main/java/com/palantir/dialogue/core/LimitedChannelToChannelAdapter.java b/dialogue-core/src/main/java/com/palantir/dialogue/core/LimitedChannelToChannelAdapter.java new file mode 100644 index 000000000..692794718 --- /dev/null +++ b/dialogue-core/src/main/java/com/palantir/dialogue/core/LimitedChannelToChannelAdapter.java @@ -0,0 +1,51 @@ +/* + * (c) Copyright 2020 Palantir Technologies Inc. All rights reserved. + * + * Licensed 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 com.palantir.dialogue.core; + +import com.google.common.util.concurrent.Futures; +import com.google.common.util.concurrent.ListenableFuture; +import com.palantir.dialogue.Channel; +import com.palantir.dialogue.Endpoint; +import com.palantir.dialogue.Request; +import com.palantir.dialogue.Response; +import com.palantir.logsafe.Preconditions; +import com.palantir.logsafe.exceptions.SafeRuntimeException; +import java.util.function.Supplier; + +/** Adapter from {@link LimitedChannel} to {@link Channel} which produces a failed future when results are limited. */ +final class LimitedChannelToChannelAdapter implements Channel { + + // Avoid method reference allocations + @SuppressWarnings("UnnecessaryLambda") + private static final Supplier> limitedResultSupplier = + () -> Futures.immediateFailedFuture(new SafeRuntimeException("Failed to make a request")); + + private final LimitedChannel delegate; + + LimitedChannelToChannelAdapter(LimitedChannel delegate) { + this.delegate = Preconditions.checkNotNull(delegate, "LimitedChannel"); + } + + @Override + public ListenableFuture execute(Endpoint endpoint, Request request) { + return delegate.maybeExecute(endpoint, request).orElseGet(limitedResultSupplier); + } + + @Override + public String toString() { + return "LimitedChannelToChannelAdapter{delegate=" + delegate + '}'; + } +} diff --git a/dialogue-core/src/main/java/com/palantir/dialogue/core/QueuedChannel.java b/dialogue-core/src/main/java/com/palantir/dialogue/core/QueuedChannel.java deleted file mode 100644 index 9fbf39575..000000000 --- a/dialogue-core/src/main/java/com/palantir/dialogue/core/QueuedChannel.java +++ /dev/null @@ -1,299 +0,0 @@ -/* - * (c) Copyright 2019 Palantir Technologies Inc. All rights reserved. - * - * Licensed 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 com.palantir.dialogue.core; - -import com.google.common.annotations.VisibleForTesting; -import com.google.common.collect.ImmutableMap; -import com.google.common.util.concurrent.FutureCallback; -import com.google.common.util.concurrent.Futures; -import com.google.common.util.concurrent.ListenableFuture; -import com.google.common.util.concurrent.MoreExecutors; -import com.google.common.util.concurrent.SettableFuture; -import com.palantir.dialogue.Channel; -import com.palantir.dialogue.Endpoint; -import com.palantir.dialogue.Request; -import com.palantir.dialogue.Response; -import com.palantir.logsafe.SafeArg; -import com.palantir.logsafe.exceptions.SafeRuntimeException; -import com.palantir.tracing.CloseableSpan; -import com.palantir.tracing.DetachedSpan; -import java.io.ByteArrayInputStream; -import java.io.InputStream; -import java.util.Deque; -import java.util.List; -import java.util.Map; -import java.util.Optional; -import java.util.concurrent.ConcurrentLinkedDeque; -import java.util.concurrent.Executor; -import java.util.concurrent.atomic.AtomicInteger; -import org.immutables.value.Value; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -/** - * A {@link Channel} that queues requests while the underlying {@link LimitedChannel} is unable to accept any new - * requests. This is done by enqueueing requests on submission, and then running the schedule loop in one of 3 ways: - *
    - *
  1. On submission - allows execution when there is available capacity
  2. - *
  3. On request completion - allows execution when capacity has now become available
  4. - *
- * - * This implementation was chosen over alternatives for the following reasons: - *
    - *
  • Always periodically schedule: this decreases throughout as requests that may be able to run will have to - * wait until the next scheduling period
  • - *
  • Schedule in a spin loop: this would allow us to schedule without delay, but requires a thread constantly - * doing work, much of which will be wasted
  • - *
- * - * TODO(jellis): record metrics for queue sizes, num requests in flight, time spent in queue, etc. - */ -final class QueuedChannel implements Channel { - private static final Logger log = LoggerFactory.getLogger(QueuedChannel.class); - private static final Executor DIRECT = MoreExecutors.directExecutor(); - - private final Deque queuedCalls; - private final LimitedChannel delegate; - // Tracks requests that are current executing in delegate and are not tracked in queuedCalls - private final AtomicInteger numRunningRequests = new AtomicInteger(0); - private final AtomicInteger queueSizeEstimate = new AtomicInteger(0); - private final int maxQueueSize; - - QueuedChannel(LimitedChannel channel, DispatcherMetrics metrics) { - this(channel, 1_000, metrics); - } - - @VisibleForTesting - QueuedChannel(LimitedChannel delegate, int maxQueueSize, DispatcherMetrics metrics) { - this.delegate = new NeverThrowLimitedChannel(delegate); - // Do _not_ call size on a ConcurrentLinkedDeque. Unlike other collections, size is an O(n) operation. - this.queuedCalls = new ProtectedConcurrentLinkedDeque<>(); - this.maxQueueSize = maxQueueSize; - - metrics.callsQueued(queueSizeEstimate::get); - metrics.callsRunning(numRunningRequests::get); - } - - /** - * Enqueues and tries to schedule as many queued tasks as possible. - */ - @Override - public ListenableFuture execute(Endpoint endpoint, Request request) { - // Optimistically avoid the queue in the fast path. - // Queuing adds contention between threads and should be avoided unless we need to shed load. - if (queueSizeEstimate.get() <= 0) { - Optional> maybeResult = delegate.maybeExecute(endpoint, request); - if (maybeResult.isPresent()) { - ListenableFuture result = maybeResult.get(); - numRunningRequests.incrementAndGet(); - result.addListener(this::onCompletion, DIRECT); - return result; - } - } - - // Important to read the queue size here as well as prior to the optimistic maybeExecute because - // maybeExecute may take sufficiently long that other requests could be queued. - if (queueSizeEstimate.get() >= maxQueueSize) { - return queueFullResponse(); - } - - DeferredCall components = DeferredCall.builder() - .endpoint(endpoint) - .request(request) - .response(SettableFuture.create()) - .span(DetachedSpan.start("Dialogue-request-enqueued")) - .build(); - - if (!queuedCalls.offer(components)) { - // Should never happen, ConcurrentLinkedDeque has no maximum size - return queueFullResponse(); - } - queueSizeEstimate.incrementAndGet(); - - schedule(); - - return components.response(); - } - - private ListenableFuture queueFullResponse() { - return Futures.immediateFuture(RateLimitedResponse.INSTANCE); - } - - private void onCompletion() { - numRunningRequests.decrementAndGet(); - schedule(); - } - - /** - * Try to schedule as many tasks as possible. Called when requests are submitted and when they complete. - */ - void schedule() { - int numScheduled = 0; - while (scheduleNextTask()) { - numScheduled++; - } - - if (log.isDebugEnabled()) { - log.debug("Scheduled {} requests", SafeArg.of("numScheduled", numScheduled)); - } - } - - /** - * Get the next call and attempt to execute it. If it is runnable, wire up the underlying future to the one - * previously returned to the caller. If it is not runnable, add it back into the queue. Returns true if more - * tasks may be able to be scheduled, and false otherwise. - */ - private boolean scheduleNextTask() { - DeferredCall queueHead = queuedCalls.poll(); - if (queueHead == null) { - return false; - } - SettableFuture queuedResponse = queueHead.response(); - // If the future has been completed (most likely via cancel) the call should not be queued. - // There's a race where cancel may be invoked between this check and execution, but the scheduled - // request will be quickly cancelled in that case. - if (queuedResponse.isDone()) { - queueSizeEstimate.decrementAndGet(); - return true; - } - try (CloseableSpan ignored = queueHead.span().childSpan("Dialogue-request-scheduled")) { - Endpoint endpoint = queueHead.endpoint(); - Optional> maybeResponse = delegate.maybeExecute(endpoint, queueHead.request()); - - if (maybeResponse.isPresent()) { - queueSizeEstimate.decrementAndGet(); - ListenableFuture response = maybeResponse.get(); - queueHead.span().complete(); - numRunningRequests.incrementAndGet(); - response.addListener(numRunningRequests::decrementAndGet, DIRECT); - Futures.addCallback(response, new ForwardAndSchedule(queuedResponse), DIRECT); - queuedResponse.addListener( - () -> { - if (queuedResponse.isCancelled()) { - // TODO(ckozak): Consider capturing the argument value provided to cancel to propagate - // here. - // Currently cancel(false) will be converted to cancel(true) - if (!response.cancel(true) && log.isDebugEnabled()) { - log.debug( - "Failed to cancel delegate response, it should be reported by" - + " ForwardAndSchedule logging", - SafeArg.of("service", endpoint.serviceName()), - SafeArg.of("endpoint", endpoint.endpointName())); - } - } - }, - DIRECT); - return true; - } else { - if (!queuedCalls.offerFirst(queueHead)) { - // Should never happen, ConcurrentLinkedDeque has no maximum size - log.error( - "Failed to add an attempted call back to the deque", - SafeArg.of("service", endpoint.serviceName()), - SafeArg.of("endpoint", endpoint.endpointName())); - queueSizeEstimate.decrementAndGet(); - if (!queuedResponse.setException( - new SafeRuntimeException( - "Failed to req-queue request", - SafeArg.of("service", endpoint.serviceName()), - SafeArg.of("endpoint", endpoint.endpointName())))) { - log.debug( - "Queued response has already been completed", - SafeArg.of("service", endpoint.serviceName()), - SafeArg.of("endpoint", endpoint.endpointName())); - } - } - return false; - } - } - } - - /** - * Forward the success or failure of the call to the SettableFuture that was previously returned to the caller. - * This also schedules the next set of requests to be run. - */ - private class ForwardAndSchedule implements FutureCallback { - private final SettableFuture response; - - ForwardAndSchedule(SettableFuture response) { - this.response = response; - } - - @Override - public void onSuccess(Response result) { - if (!response.set(result)) { - result.close(); - } - schedule(); - } - - @Override - public void onFailure(Throwable throwable) { - if (!response.setException(throwable)) { - log.info("Call failed after the future completed", throwable); - } - schedule(); - } - } - - private enum RateLimitedResponse implements Response { - INSTANCE; - - @Override - public InputStream body() { - return new ByteArrayInputStream(new byte[0]); - } - - @Override - public int code() { - return 429; - } - - @Override - public Map> headers() { - return ImmutableMap.of(); - } - - @Override - public void close() {} - } - - @Value.Immutable - interface DeferredCall { - Endpoint endpoint(); - - Request request(); - - SettableFuture response(); - - DetachedSpan span(); - - class Builder extends ImmutableDeferredCall.Builder {} - - static Builder builder() { - return new Builder(); - } - } - - private static final class ProtectedConcurrentLinkedDeque extends ConcurrentLinkedDeque { - - @Override - public int size() { - throw new UnsupportedOperationException("size should never be called on a ConcurrentLinkedDeque"); - } - } -} diff --git a/dialogue-core/src/main/java/com/palantir/dialogue/core/RetryingChannel.java b/dialogue-core/src/main/java/com/palantir/dialogue/core/RetryingChannel.java index f1cb32fab..38a8df297 100644 --- a/dialogue-core/src/main/java/com/palantir/dialogue/core/RetryingChannel.java +++ b/dialogue-core/src/main/java/com/palantir/dialogue/core/RetryingChannel.java @@ -16,9 +16,13 @@ package com.palantir.dialogue.core; +import com.google.common.base.Suppliers; import com.google.common.util.concurrent.Futures; import com.google.common.util.concurrent.ListenableFuture; +import com.google.common.util.concurrent.ListenableScheduledFuture; +import com.google.common.util.concurrent.ListeningScheduledExecutorService; import com.google.common.util.concurrent.MoreExecutors; +import com.google.common.util.concurrent.ThreadFactoryBuilder; import com.palantir.conjure.java.client.config.ClientConfiguration; import com.palantir.dialogue.Channel; import com.palantir.dialogue.Endpoint; @@ -27,6 +31,13 @@ import com.palantir.logsafe.SafeArg; import com.palantir.logsafe.exceptions.SafeIllegalStateException; import com.palantir.logsafe.exceptions.SafeRuntimeException; +import com.palantir.tracing.Tracers; +import java.time.Duration; +import java.util.concurrent.Executors; +import java.util.concurrent.ThreadLocalRandom; +import java.util.concurrent.TimeUnit; +import java.util.function.DoubleSupplier; +import java.util.function.Supplier; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -37,44 +48,80 @@ final class RetryingChannel implements Channel { private static final Logger log = LoggerFactory.getLogger(RetryingChannel.class); + /* + * Shared single thread executor is reused between all retrying channels. If it becomes oversaturated + * we may wait longer than expected before resuming requests, but this is an + * edge case where services are already operating in a degraded state and we should not + * spam servers. + */ + private static final Supplier sharedScheduler = Suppliers.memoize( + () -> MoreExecutors.listeningDecorator(Tracers.wrap( + "dialogue-BlacklistingChannel-scheduler", + Executors.newSingleThreadScheduledExecutor(new ThreadFactoryBuilder() + .setNameFormat("dialogue-BlacklistingChannel-scheduler-%d") + .setDaemon(false) + .build())))); + + private final ListeningScheduledExecutorService scheduler; private final Channel delegate; private final int maxRetries; private final ClientConfiguration.ServerQoS serverQoS; + private final Duration backoffSlotSize; + private final DoubleSupplier random; - RetryingChannel(Channel delegate, int maxRetries, ClientConfiguration.ServerQoS serverQoS) { + RetryingChannel( + Channel delegate, int maxRetries, Duration backoffSlotSize, ClientConfiguration.ServerQoS serverQoS) { + this(delegate, maxRetries, backoffSlotSize, serverQoS, sharedScheduler.get(), () -> + ThreadLocalRandom.current().nextDouble()); + } + + RetryingChannel( + Channel delegate, + int maxRetries, + Duration backoffSlotSize, + ClientConfiguration.ServerQoS serverQoS, + ListeningScheduledExecutorService scheduler, + DoubleSupplier random) { this.delegate = delegate; this.maxRetries = maxRetries; + this.backoffSlotSize = backoffSlotSize; this.serverQoS = serverQoS; + this.scheduler = scheduler; + this.random = random; } @Override public ListenableFuture execute(Endpoint endpoint, Request request) { - return new RetryingCallback(delegate, endpoint, request, maxRetries, serverQoS).execute(); + return new RetryingCallback(endpoint, request).execute(); } - private static final class RetryingCallback { - private final Channel delegate; + private final class RetryingCallback { private final Endpoint endpoint; private final Request request; - private final int maxRetries; - private final ClientConfiguration.ServerQoS serverQoS; private int failures = 0; - private RetryingCallback( - Channel delegate, - Endpoint endpoint, - Request request, - int maxRetries, - ClientConfiguration.ServerQoS serverQoS) { - this.delegate = delegate; + private RetryingCallback(Endpoint endpoint, Request request) { this.endpoint = endpoint; this.request = request; - this.maxRetries = maxRetries; - this.serverQoS = serverQoS; } + @SuppressWarnings("FutureReturnValueIgnored") // error-prone bug ListenableFuture execute() { - return wrap(delegate.execute(endpoint, request)); + long backoffNanoseconds = getBackoffNanoseconds(); + if (backoffNanoseconds <= 0) { + return wrap(delegate.execute(endpoint, request)); + } + ListenableScheduledFuture> scheduled = scheduler.schedule( + () -> delegate.execute(endpoint, request), backoffNanoseconds, TimeUnit.NANOSECONDS); + return wrap(Futures.transformAsync(scheduled, input -> input, MoreExecutors.directExecutor())); + } + + private long getBackoffNanoseconds() { + if (failures == 0) { + return 0L; + } + int upperBound = (int) Math.pow(2, failures - 1); + return Math.round(backoffSlotSize.toNanos() * random.getAsDouble() * upperBound); } ListenableFuture success(Response response) { diff --git a/dialogue-core/src/test/java/com/palantir/dialogue/core/ChannelsTest.java b/dialogue-core/src/test/java/com/palantir/dialogue/core/ChannelsTest.java index d2fdb2e1c..eb5e60563 100644 --- a/dialogue-core/src/test/java/com/palantir/dialogue/core/ChannelsTest.java +++ b/dialogue-core/src/test/java/com/palantir/dialogue/core/ChannelsTest.java @@ -148,13 +148,17 @@ public ListenableFuture execute(Endpoint _endpoint, Request _request) @TestTracing(snapshot = true) public void traces_on_retries() throws Exception { when(response.code()).thenReturn(429); - channel.execute(endpoint, request).get(); + try (Response response = channel.execute(endpoint, request).get()) { + assertThat(response.code()).isEqualTo(429); + } } @Test @TestTracing(snapshot = true) public void traces_on_succes() throws Exception { when(response.code()).thenReturn(200); - channel.execute(endpoint, request).get(); + try (Response response = channel.execute(endpoint, request).get()) { + assertThat(response.code()).isEqualTo(200); + } } } diff --git a/dialogue-core/src/test/java/com/palantir/dialogue/core/ConcurrencyLimitedChannelTest.java b/dialogue-core/src/test/java/com/palantir/dialogue/core/ConcurrencyLimitedChannelTest.java index 6347865b0..ebe489c89 100644 --- a/dialogue-core/src/test/java/com/palantir/dialogue/core/ConcurrencyLimitedChannelTest.java +++ b/dialogue-core/src/test/java/com/palantir/dialogue/core/ConcurrencyLimitedChannelTest.java @@ -65,7 +65,7 @@ public class ConcurrencyLimitedChannelTest { @BeforeEach public void before() { - channel = new ConcurrencyLimitedChannel(new LimitedChannelAdapter(delegate), limiter, metrics); + channel = new ConcurrencyLimitedChannel(new ChannelToLimitedChannelAdapter(delegate), limiter, metrics); responseFuture = SettableFuture.create(); lenient().when(delegate.execute(endpoint, request)).thenReturn(responseFuture); @@ -108,7 +108,7 @@ public void testUnavailable() { @Test public void testWithDefaultLimiter() { - channel = ConcurrencyLimitedChannel.create(new LimitedChannelAdapter(delegate), metrics); + channel = ConcurrencyLimitedChannel.create(new ChannelToLimitedChannelAdapter(delegate), metrics); assertThat(channel.maybeExecute(endpoint, request)).contains(responseFuture); } diff --git a/dialogue-core/src/test/java/com/palantir/dialogue/core/QueuedChannelTest.java b/dialogue-core/src/test/java/com/palantir/dialogue/core/QueuedChannelTest.java deleted file mode 100644 index dae175de0..000000000 --- a/dialogue-core/src/test/java/com/palantir/dialogue/core/QueuedChannelTest.java +++ /dev/null @@ -1,286 +0,0 @@ -/* - * (c) Copyright 2019 Palantir Technologies Inc. All rights reserved. - * - * Licensed 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 com.palantir.dialogue.core; - -import static org.assertj.core.api.Assertions.assertThat; -import static org.assertj.core.api.Assertions.assertThatThrownBy; -import static org.mockito.Mockito.times; -import static org.mockito.Mockito.verify; -import static org.mockito.Mockito.when; - -import com.codahale.metrics.Gauge; -import com.google.common.collect.ImmutableMap; -import com.google.common.util.concurrent.Futures; -import com.google.common.util.concurrent.ListenableFuture; -import com.google.common.util.concurrent.SettableFuture; -import com.palantir.dialogue.Endpoint; -import com.palantir.dialogue.Request; -import com.palantir.dialogue.Response; -import com.palantir.tracing.TestTracing; -import com.palantir.tritium.metrics.registry.DefaultTaggedMetricRegistry; -import com.palantir.tritium.metrics.registry.MetricName; -import com.palantir.tritium.metrics.registry.TaggedMetricRegistry; -import java.util.Optional; -import java.util.concurrent.ExecutionException; -import org.junit.jupiter.api.BeforeEach; -import org.junit.jupiter.api.Test; -import org.junit.jupiter.api.extension.ExtendWith; -import org.mockito.Mock; -import org.mockito.Mockito; -import org.mockito.junit.jupiter.MockitoExtension; -import org.mockito.stubbing.Answer; -import org.mockito.stubbing.OngoingStubbing; - -@ExtendWith(MockitoExtension.class) -@SuppressWarnings("FutureReturnValueIgnored") -public class QueuedChannelTest { - private static final MetricName NUM_QUEUED_METRIC = MetricName.builder() - .safeName("com.palantir.conjure.java.dispatcher.calls.queued") - .build(); - private static final MetricName NUM_RUNNING_METRICS = MetricName.builder() - .safeName("com.palantir.conjure.java.dispatcher.calls.running") - .build(); - - @Mock - private LimitedChannel delegate; - - @Mock - private Endpoint endpoint; - - @Mock - private Request request; - - @Mock - private Response mockResponse; - - private TaggedMetricRegistry metrics; - private QueuedChannel queuedChannel; - private SettableFuture futureResponse; - private Optional> maybeResponse; - - @BeforeEach - public void before() { - metrics = new DefaultTaggedMetricRegistry(); - queuedChannel = new QueuedChannel(delegate, DispatcherMetrics.of(metrics)); - futureResponse = SettableFuture.create(); - maybeResponse = Optional.of(futureResponse); - - mockHasCapacity(); - } - - @Test - public void testReceivesSuccessfulResponse() throws ExecutionException, InterruptedException { - ListenableFuture response = queuedChannel.execute(endpoint, request); - assertThat(response.isDone()).isFalse(); - - futureResponse.set(mockResponse); - - assertThat(response.isDone()).isTrue(); - assertThat(response.get()).isEqualTo(mockResponse); - } - - @Test - public void testReceivesExceptionalResponse() { - ListenableFuture response = queuedChannel.execute(endpoint, request); - assertThat(response.isDone()).isFalse(); - - futureResponse.setException(new IllegalArgumentException()); - - assertThat(response.isDone()).isTrue(); - assertThatThrownBy(() -> response.get()) - .isInstanceOf(ExecutionException.class) - .hasCauseInstanceOf(IllegalArgumentException.class); - } - - @Test - @SuppressWarnings("FutureReturnValueIgnored") - public void testQueuedRequestExecutedOnNextSubmission() { - mockNoCapacity(); - queuedChannel.execute(endpoint, request); - verify(delegate, times(2)).maybeExecute(endpoint, request); - - mockHasCapacity(); - queuedChannel.execute(endpoint, request); - verify(delegate, times(4)).maybeExecute(endpoint, request); - } - - @Test - public void testQueuedRequestExecutedOnNextSubmission_throws() throws ExecutionException, InterruptedException { - // First request is limited by the channel and queued - Request queuedRequest = Mockito.mock(Request.class); - when(delegate.maybeExecute(endpoint, queuedRequest)).thenReturn(Optional.empty()); - ListenableFuture queuedFuture = queuedChannel.execute(endpoint, queuedRequest); - verify(delegate, times(2)).maybeExecute(endpoint, queuedRequest); - assertThat(queuedFuture).isNotDone(); - - // Second request succeeds and the queued request is attempted, but throws an exception - futureResponse.set(mockResponse); - when(delegate.maybeExecute(endpoint, request)).thenReturn(maybeResponse); - when(delegate.maybeExecute(endpoint, queuedRequest)).thenThrow(new NullPointerException("expected")); - ListenableFuture completed = queuedChannel.execute(endpoint, request); - // Both results should be completed. The thrown exception should - // be converted into a failed future by NeverThrowLimitedChannel - assertThat(completed).isDone(); - assertThat(queuedFuture).isDone(); - assertThat(completed.get()).isEqualTo(mockResponse); - assertThatThrownBy(queuedFuture::get).hasRootCauseMessage("expected"); - verify(delegate, times(1)).maybeExecute(endpoint, request); - verify(delegate, times(3)).maybeExecute(endpoint, queuedRequest); - } - - @Test - @SuppressWarnings("FutureReturnValueIgnored") - public void testQueuedRequestExecutedWhenRunningRequestCompletes() { - mockHasCapacity(); - queuedChannel.execute(endpoint, request); - verify(delegate, times(1)).maybeExecute(endpoint, request); - - mockNoCapacity(); - queuedChannel.execute(endpoint, request); - verify(delegate, times(3)).maybeExecute(endpoint, request); - futureResponse.set(mockResponse); - - verify(delegate, times(4)).maybeExecute(endpoint, request); - } - - @Test - @TestTracing(snapshot = true) - public void testQueueTracing() { - // Put requests on queue - mockNoCapacity(); - queuedChannel.execute(endpoint, request); - queuedChannel.execute(endpoint, request); - verify(delegate, times(3)).maybeExecute(endpoint, request); - - // flush queue by completing a request - mockHasCapacity(); - queuedChannel.execute(endpoint, request); - verify(delegate, times(6)).maybeExecute(endpoint, request); - futureResponse.set(mockResponse); - - verify(delegate, times(6)).maybeExecute(endpoint, request); - } - - @Test - @SuppressWarnings("FutureReturnValueIgnored") - public void testQueueFullReturns429() throws ExecutionException, InterruptedException { - queuedChannel = new QueuedChannel(delegate, 1, DispatcherMetrics.of(metrics)); - - mockNoCapacity(); - queuedChannel.execute(endpoint, request); - - assertThat(queuedChannel.execute(endpoint, request).get().code()).isEqualTo(429); - } - - @Test - public void emitsQueueCapacityMetrics_whenChannelsHasNoCapacity() { - mockNoCapacity(); - - queuedChannel.execute(endpoint, request); - assertThat(gaugeValue(NUM_QUEUED_METRIC)).isEqualTo(1); - assertThat(gaugeValue(NUM_RUNNING_METRICS)).isZero(); - - queuedChannel.execute(endpoint, request); - assertThat(gaugeValue(NUM_QUEUED_METRIC)).isEqualTo(2); - assertThat(gaugeValue(NUM_RUNNING_METRICS)).isZero(); - } - - @Test - public void emitsQueueCapacityMetrics_whenChannelHasCapacity() { - mockHasCapacity(); - - queuedChannel.execute(endpoint, request); - assertThat(gaugeValue(NUM_QUEUED_METRIC)).isZero(); - assertThat(gaugeValue(NUM_RUNNING_METRICS)).isEqualTo(1); - - futureResponse.set(mockResponse); - assertThat(gaugeValue(NUM_QUEUED_METRIC)).isZero(); - assertThat(gaugeValue(NUM_RUNNING_METRICS)).isZero(); - } - - @Test - @SuppressWarnings("FutureReturnValueIgnored") - public void testQueuedResponseClosedOnCancel() { - Request queuedRequest = - Request.builder().pathParams(ImmutableMap.of("foo", "bar")).build(); - when(delegate.maybeExecute(endpoint, queuedRequest)).thenReturn(Optional.empty()); - ListenableFuture result = queuedChannel.execute(endpoint, queuedRequest); - verify(delegate, times(2)).maybeExecute(endpoint, queuedRequest); - - when(delegate.maybeExecute(endpoint, request)) - .thenReturn(Optional.of(Futures.immediateFuture(Mockito.mock(Response.class)))); - when(delegate.maybeExecute(endpoint, queuedRequest)) - .thenAnswer((Answer>>) _invocation -> { - // cancel from this invocation to simulate the race between cancellation and execution - assertThat(result.cancel(true)).isTrue(); - return Optional.of(Futures.immediateFuture(mockResponse)); - }); - // Force scheduling - queuedChannel.execute(endpoint, request); - assertThat(result).isCancelled(); - verify(delegate, times(1)).maybeExecute(endpoint, request); - verify(mockResponse, times(1)).close(); - } - - @Test - @SuppressWarnings("FutureReturnValueIgnored") - public void testQueuedResponsePropagatesCancel() { - Request queued = Request.builder().putHeaderParams("key", "val").build(); - when(delegate.maybeExecute(endpoint, queued)).thenReturn(Optional.empty()); - ListenableFuture result = queuedChannel.execute(endpoint, queued); - verify(delegate, times(2)).maybeExecute(endpoint, queued); - - when(delegate.maybeExecute(endpoint, request)) - .thenReturn(Optional.of(Futures.immediateFuture(Mockito.mock(Response.class)))); - when(delegate.maybeExecute(endpoint, queued)).thenReturn(maybeResponse); - queuedChannel.execute(endpoint, request); - result.cancel(true); - assertThat(futureResponse).isCancelled(); - verify(delegate, times(1)).maybeExecute(endpoint, request); - verify(delegate, times(3)).maybeExecute(endpoint, queued); - } - - @Test - @SuppressWarnings("FutureReturnValueIgnored") - public void testQueuedResponseAvoidsExecutingCancelled() { - Request queued = Request.builder().putHeaderParams("key", "val").build(); - when(delegate.maybeExecute(endpoint, queued)).thenReturn(Optional.empty()); - ListenableFuture result = queuedChannel.execute(endpoint, queued); - verify(delegate, times(2)).maybeExecute(endpoint, queued); - - assertThat(result.cancel(true)).isTrue(); - when(delegate.maybeExecute(endpoint, request)) - .thenReturn(Optional.of(Futures.immediateFuture(Mockito.mock(Response.class)))); - queuedChannel.execute(endpoint, request); - verify(delegate, times(1)).maybeExecute(endpoint, request); - // Should not have been invoked any more. - verify(delegate, times(2)).maybeExecute(endpoint, queued); - } - - @SuppressWarnings("unchecked") - private Integer gaugeValue(MetricName metric) { - return ((Gauge) metrics.getMetrics().get(metric)).getValue(); - } - - private OngoingStubbing>> mockHasCapacity() { - return when(delegate.maybeExecute(endpoint, request)).thenReturn(maybeResponse); - } - - private OngoingStubbing>> mockNoCapacity() { - return when(delegate.maybeExecute(endpoint, request)).thenReturn(Optional.empty()); - } -} diff --git a/dialogue-core/src/test/java/com/palantir/dialogue/core/RetryingChannelTest.java b/dialogue-core/src/test/java/com/palantir/dialogue/core/RetryingChannelTest.java index 5c6cd00d1..5a7d5a6a7 100644 --- a/dialogue-core/src/test/java/com/palantir/dialogue/core/RetryingChannelTest.java +++ b/dialogue-core/src/test/java/com/palantir/dialogue/core/RetryingChannelTest.java @@ -37,6 +37,7 @@ import com.palantir.dialogue.UrlBuilder; import java.io.ByteArrayInputStream; import java.io.InputStream; +import java.time.Duration; import java.util.List; import java.util.Map; import java.util.concurrent.ExecutionException; @@ -62,7 +63,7 @@ public class RetryingChannelTest { public void testNoFailures() throws ExecutionException, InterruptedException { when(channel.execute(any(), any())).thenReturn(SUCCESS); - Channel retryer = new RetryingChannel(channel, 3, ClientConfiguration.ServerQoS.AUTOMATIC_RETRY); + Channel retryer = new RetryingChannel(channel, 3, Duration.ZERO, ClientConfiguration.ServerQoS.AUTOMATIC_RETRY); ListenableFuture response = retryer.execute(ENDPOINT, REQUEST); assertThat(response.get()).isEqualTo(EXPECTED_RESPONSE); } @@ -72,7 +73,7 @@ public void testRetriesUpToMaxRetries() throws ExecutionException, InterruptedEx when(channel.execute(any(), any())).thenReturn(FAILED).thenReturn(SUCCESS); // One retry allows an initial request (not a retry) and a single retry. - Channel retryer = new RetryingChannel(channel, 1, ClientConfiguration.ServerQoS.AUTOMATIC_RETRY); + Channel retryer = new RetryingChannel(channel, 1, Duration.ZERO, ClientConfiguration.ServerQoS.AUTOMATIC_RETRY); ListenableFuture response = retryer.execute(ENDPOINT, REQUEST); assertThat(response).isDone(); assertThat(response.get()).isEqualTo(EXPECTED_RESPONSE); @@ -86,7 +87,7 @@ public void testRetriesUpToMaxRetriesAndFails() throws ExecutionException, Inter .thenReturn(SUCCESS); // One retry allows an initial request (not a retry) and a single retry. - Channel retryer = new RetryingChannel(channel, 1, ClientConfiguration.ServerQoS.AUTOMATIC_RETRY); + Channel retryer = new RetryingChannel(channel, 1, Duration.ZERO, ClientConfiguration.ServerQoS.AUTOMATIC_RETRY); ListenableFuture response = retryer.execute(ENDPOINT, REQUEST); assertThatThrownBy(response::get) .hasRootCauseExactlyInstanceOf(IllegalArgumentException.class) @@ -97,7 +98,7 @@ public void testRetriesUpToMaxRetriesAndFails() throws ExecutionException, Inter public void testRetriesMax() { when(channel.execute(any(), any())).thenReturn(FAILED); - Channel retryer = new RetryingChannel(channel, 3, ClientConfiguration.ServerQoS.AUTOMATIC_RETRY); + Channel retryer = new RetryingChannel(channel, 3, Duration.ZERO, ClientConfiguration.ServerQoS.AUTOMATIC_RETRY); ListenableFuture response = retryer.execute(ENDPOINT, REQUEST); assertThatThrownBy(response::get).hasCauseInstanceOf(IllegalArgumentException.class); verify(channel, times(4)).execute(ENDPOINT, REQUEST); @@ -109,7 +110,7 @@ public void retries_429s() throws Exception { when(mockResponse.code()).thenReturn(429); when(channel.execute(any(), any())).thenReturn(Futures.immediateFuture(mockResponse)); - Channel retryer = new RetryingChannel(channel, 3, ClientConfiguration.ServerQoS.AUTOMATIC_RETRY); + Channel retryer = new RetryingChannel(channel, 3, Duration.ZERO, ClientConfiguration.ServerQoS.AUTOMATIC_RETRY); ListenableFuture response = retryer.execute(ENDPOINT, REQUEST); assertThat(response).isDone(); assertThat(response.get()) @@ -124,7 +125,7 @@ public void retries_503s() throws Exception { when(mockResponse.code()).thenReturn(503); when(channel.execute(any(), any())).thenReturn(Futures.immediateFuture(mockResponse)); - Channel retryer = new RetryingChannel(channel, 3, ClientConfiguration.ServerQoS.AUTOMATIC_RETRY); + Channel retryer = new RetryingChannel(channel, 3, Duration.ZERO, ClientConfiguration.ServerQoS.AUTOMATIC_RETRY); ListenableFuture response = retryer.execute(ENDPOINT, REQUEST); assertThat(response).isDone(); assertThat(response.get()) @@ -139,8 +140,8 @@ public void retries_429s_when_requested() throws Exception { when(mockResponse.code()).thenReturn(429); when(channel.execute(any(), any())).thenReturn(Futures.immediateFuture(mockResponse)); - Channel retryer = - new RetryingChannel(channel, 3, ClientConfiguration.ServerQoS.PROPAGATE_429_and_503_TO_CALLER); + Channel retryer = new RetryingChannel( + channel, 3, Duration.ZERO, ClientConfiguration.ServerQoS.PROPAGATE_429_and_503_TO_CALLER); ListenableFuture response = retryer.execute(ENDPOINT, REQUEST); assertThat(response).isDone(); assertThat(response.get().code()).isEqualTo(429); @@ -153,8 +154,8 @@ public void returns_503s_when_requested() throws Exception { when(mockResponse.code()).thenReturn(503); when(channel.execute(any(), any())).thenReturn(Futures.immediateFuture(mockResponse)); - Channel retryer = - new RetryingChannel(channel, 3, ClientConfiguration.ServerQoS.PROPAGATE_429_and_503_TO_CALLER); + Channel retryer = new RetryingChannel( + channel, 3, Duration.ZERO, ClientConfiguration.ServerQoS.PROPAGATE_429_and_503_TO_CALLER); ListenableFuture response = retryer.execute(ENDPOINT, REQUEST); assertThat(response).isDone(); assertThat(response.get().code()).isEqualTo(503); @@ -172,7 +173,7 @@ public void response_bodies_are_closed() throws Exception { .thenReturn(Futures.immediateFuture(response2)) .thenReturn(Futures.immediateFuture(eventualSuccess)); - Channel retryer = new RetryingChannel(channel, 3, ClientConfiguration.ServerQoS.AUTOMATIC_RETRY); + Channel retryer = new RetryingChannel(channel, 3, Duration.ZERO, ClientConfiguration.ServerQoS.AUTOMATIC_RETRY); ListenableFuture response = retryer.execute(ENDPOINT, REQUEST); assertThat(response.get(1, TimeUnit.SECONDS).code()).isEqualTo(200); @@ -184,7 +185,7 @@ public void response_bodies_are_closed() throws Exception { public void testPropagatesCancel() { ListenableFuture delegateResult = SettableFuture.create(); when(channel.execute(any(), any())).thenReturn(delegateResult); - Channel retryer = new RetryingChannel(channel, 3, ClientConfiguration.ServerQoS.AUTOMATIC_RETRY); + Channel retryer = new RetryingChannel(channel, 3, Duration.ZERO, ClientConfiguration.ServerQoS.AUTOMATIC_RETRY); ListenableFuture retryingResult = retryer.execute(ENDPOINT, REQUEST); assertThat(retryingResult.cancel(true)).isTrue(); assertThat(delegateResult).as("Failed to cancel the delegate future").isCancelled(); diff --git a/dialogue-core/src/test/resources/tracing/ChannelsTest/traces_on_retries.log b/dialogue-core/src/test/resources/tracing/ChannelsTest/traces_on_retries.log index 589b43a14..c3626d590 100644 --- a/dialogue-core/src/test/resources/tracing/ChannelsTest/traces_on_retries.log +++ b/dialogue-core/src/test/resources/tracing/ChannelsTest/traces_on_retries.log @@ -1,22 +1,26 @@ -{"traceId":"3af31cbecda2b46c","parentSpanId":"8f6fc520968546da","spanId":"7bd25af853d37ea5","type":"LOCAL","operation":"Dialogue-http-request initial","startTimeMicroSeconds":1582562924525750,"durationNanoSeconds":2639094,"metadata":{}} -{"traceId":"3af31cbecda2b46c","parentSpanId":"5d0cbcb5c2f6ec79","spanId":"8f6fc520968546da","type":"LOCAL","operation":"Dialogue-http-request","startTimeMicroSeconds":1582562924525740,"durationNanoSeconds":2853554,"metadata":{}} -{"traceId":"3af31cbecda2b46c","parentSpanId":"394e7bce220a625e","spanId":"5d0cbcb5c2f6ec79","type":"LOCAL","operation":"Dialogue-request-attempt initial","startTimeMicroSeconds":1582562924525618,"durationNanoSeconds":3474891,"metadata":{}} -{"traceId":"3af31cbecda2b46c","parentSpanId":"697e878d5bfc5628","spanId":"394e7bce220a625e","type":"LOCAL","operation":"Dialogue-request-attempt","startTimeMicroSeconds":1582562924525604,"durationNanoSeconds":3501888,"metadata":{}} -{"traceId":"3af31cbecda2b46c","parentSpanId":"eba063d026b23b23","spanId":"81ba619a8c099225","type":"LOCAL","operation":"Dialogue-http-request initial","startTimeMicroSeconds":1582562924531144,"durationNanoSeconds":338716,"metadata":{}} -{"traceId":"3af31cbecda2b46c","parentSpanId":"7cbe8cd625965cd7","spanId":"eba063d026b23b23","type":"LOCAL","operation":"Dialogue-http-request","startTimeMicroSeconds":1582562924531125,"durationNanoSeconds":370301,"metadata":{}} -{"traceId":"3af31cbecda2b46c","parentSpanId":"d567331da4f0ac55","spanId":"7cbe8cd625965cd7","type":"LOCAL","operation":"Dialogue-request-attempt initial","startTimeMicroSeconds":1582562924530316,"durationNanoSeconds":1265906,"metadata":{}} -{"traceId":"3af31cbecda2b46c","parentSpanId":"697e878d5bfc5628","spanId":"d567331da4f0ac55","type":"LOCAL","operation":"Dialogue-request-attempt","startTimeMicroSeconds":1582562924530282,"durationNanoSeconds":1304889,"metadata":{}} -{"traceId":"3af31cbecda2b46c","parentSpanId":"46aedc07f001de9c","spanId":"9a5602d68db3d08f","type":"LOCAL","operation":"Dialogue-http-request initial","startTimeMicroSeconds":1582562924532418,"durationNanoSeconds":235251,"metadata":{}} -{"traceId":"3af31cbecda2b46c","parentSpanId":"c4541e32bbc6bfe8","spanId":"46aedc07f001de9c","type":"LOCAL","operation":"Dialogue-http-request","startTimeMicroSeconds":1582562924532409,"durationNanoSeconds":256966,"metadata":{}} -{"traceId":"3af31cbecda2b46c","parentSpanId":"28f98da9021fd19b","spanId":"c4541e32bbc6bfe8","type":"LOCAL","operation":"Dialogue-request-attempt initial","startTimeMicroSeconds":1582562924532356,"durationNanoSeconds":386694,"metadata":{}} -{"traceId":"3af31cbecda2b46c","parentSpanId":"697e878d5bfc5628","spanId":"28f98da9021fd19b","type":"LOCAL","operation":"Dialogue-request-attempt","startTimeMicroSeconds":1582562924532338,"durationNanoSeconds":408967,"metadata":{}} -{"traceId":"3af31cbecda2b46c","parentSpanId":"b430069d0282a2c9","spanId":"bfd65f0cca09c11b","type":"LOCAL","operation":"Dialogue-http-request initial","startTimeMicroSeconds":1582562924533529,"durationNanoSeconds":208896,"metadata":{}} -{"traceId":"3af31cbecda2b46c","parentSpanId":"d7faec679d55f2d4","spanId":"b430069d0282a2c9","type":"LOCAL","operation":"Dialogue-http-request","startTimeMicroSeconds":1582562924533520,"durationNanoSeconds":229953,"metadata":{}} -{"traceId":"3af31cbecda2b46c","parentSpanId":"d5319f25e6638359","spanId":"d7faec679d55f2d4","type":"LOCAL","operation":"Dialogue-request-attempt initial","startTimeMicroSeconds":1582562924533488,"durationNanoSeconds":334775,"metadata":{}} -{"traceId":"3af31cbecda2b46c","parentSpanId":"697e878d5bfc5628","spanId":"d5319f25e6638359","type":"LOCAL","operation":"Dialogue-request-attempt","startTimeMicroSeconds":1582562924533473,"durationNanoSeconds":355345,"metadata":{}} -{"traceId":"3af31cbecda2b46c","parentSpanId":"4a1d1c1893452979","spanId":"87b2d147957639a2","type":"LOCAL","operation":"Dialogue-http-request initial","startTimeMicroSeconds":1582562924534981,"durationNanoSeconds":333038,"metadata":{}} -{"traceId":"3af31cbecda2b46c","parentSpanId":"749bb99b3951da3f","spanId":"4a1d1c1893452979","type":"LOCAL","operation":"Dialogue-http-request","startTimeMicroSeconds":1582562924534969,"durationNanoSeconds":363180,"metadata":{}} -{"traceId":"3af31cbecda2b46c","parentSpanId":"28686cc7c2695fb2","spanId":"749bb99b3951da3f","type":"LOCAL","operation":"Dialogue-request-attempt initial","startTimeMicroSeconds":1582562924534927,"durationNanoSeconds":544058,"metadata":{}} -{"traceId":"3af31cbecda2b46c","parentSpanId":"697e878d5bfc5628","spanId":"28686cc7c2695fb2","type":"LOCAL","operation":"Dialogue-request-attempt","startTimeMicroSeconds":1582562924534905,"durationNanoSeconds":577068,"metadata":{}} -{"traceId":"3af31cbecda2b46c","parentSpanId":"efaf41a21cc7c6a9","spanId":"697e878d5bfc5628","type":"LOCAL","operation":"Dialogue-request initial","startTimeMicroSeconds":1582562924525312,"durationNanoSeconds":13461016,"metadata":{}} -{"traceId":"3af31cbecda2b46c","parentSpanId":null,"spanId":"efaf41a21cc7c6a9","type":"LOCAL","operation":"Dialogue-request","startTimeMicroSeconds":1582562924525264,"durationNanoSeconds":13530208,"metadata":{}} +{"traceId":"cc2af584ecf2a705","parentSpanId":"a87826fa8551ef00","spanId":"c4b3149ae692711c","type":"LOCAL","operation":"Dialogue-http-request initial","startTimeMicroSeconds":1582666422499681,"durationNanoSeconds":5720159,"metadata":{}} +{"traceId":"cc2af584ecf2a705","parentSpanId":"be80183dfeed0f90","spanId":"a87826fa8551ef00","type":"LOCAL","operation":"Dialogue-http-request","startTimeMicroSeconds":1582666422499663,"durationNanoSeconds":6209895,"metadata":{}} +{"traceId":"cc2af584ecf2a705","parentSpanId":"83267beb8636c288","spanId":"be80183dfeed0f90","type":"LOCAL","operation":"Dialogue-request-attempt initial","startTimeMicroSeconds":1582666422499420,"durationNanoSeconds":7916145,"metadata":{}} +{"traceId":"cc2af584ecf2a705","parentSpanId":"cc4c5dcb09dddef9","spanId":"83267beb8636c288","type":"LOCAL","operation":"Dialogue-request-attempt","startTimeMicroSeconds":1582666422499398,"durationNanoSeconds":7957300,"metadata":{}} +{"traceId":"cc2af584ecf2a705","parentSpanId":"0662ae5d80d73804","spanId":"cc4c5dcb09dddef9","type":"LOCAL","operation":"Dialogue-request initial","startTimeMicroSeconds":1582666422495802,"durationNanoSeconds":23563978,"metadata":{}} +{"traceId":"cc2af584ecf2a705","parentSpanId":"57cd304d8aa2828b","spanId":"dec18ab7ec0993f3","type":"LOCAL","operation":"Dialogue-http-request initial","startTimeMicroSeconds":1582666422723483,"durationNanoSeconds":2243957,"metadata":{}} +{"traceId":"cc2af584ecf2a705","parentSpanId":"9fad1346f7475bb9","spanId":"57cd304d8aa2828b","type":"LOCAL","operation":"Dialogue-http-request","startTimeMicroSeconds":1582666422723472,"durationNanoSeconds":2283742,"metadata":{}} +{"traceId":"cc2af584ecf2a705","parentSpanId":"a854c890e137267a","spanId":"9fad1346f7475bb9","type":"LOCAL","operation":"Dialogue-request-attempt initial","startTimeMicroSeconds":1582666422723449,"durationNanoSeconds":2714734,"metadata":{}} +{"traceId":"cc2af584ecf2a705","parentSpanId":"0a741a3522748074","spanId":"a854c890e137267a","type":"LOCAL","operation":"Dialogue-request-attempt","startTimeMicroSeconds":1582666422723420,"durationNanoSeconds":2753169,"metadata":{}} +{"traceId":"cc2af584ecf2a705","parentSpanId":"cc4c5dcb09dddef9","spanId":"0a741a3522748074","type":"LOCAL","operation":"dialogue-BlacklistingChannel-scheduler","startTimeMicroSeconds":1582666422722160,"durationNanoSeconds":5403844,"metadata":{}} +{"traceId":"cc2af584ecf2a705","parentSpanId":"eea9a0d6db97c500","spanId":"9e14558d567d876c","type":"LOCAL","operation":"Dialogue-http-request initial","startTimeMicroSeconds":1582666422907935,"durationNanoSeconds":702085,"metadata":{}} +{"traceId":"cc2af584ecf2a705","parentSpanId":"e40527d29024b72b","spanId":"eea9a0d6db97c500","type":"LOCAL","operation":"Dialogue-http-request","startTimeMicroSeconds":1582666422907901,"durationNanoSeconds":757730,"metadata":{}} +{"traceId":"cc2af584ecf2a705","parentSpanId":"9a2dfa85700147f7","spanId":"e40527d29024b72b","type":"LOCAL","operation":"Dialogue-request-attempt initial","startTimeMicroSeconds":1582666422907861,"durationNanoSeconds":886895,"metadata":{}} +{"traceId":"cc2af584ecf2a705","parentSpanId":"8daeb0b90c77c79e","spanId":"9a2dfa85700147f7","type":"LOCAL","operation":"Dialogue-request-attempt","startTimeMicroSeconds":1582666422907810,"durationNanoSeconds":945628,"metadata":{}} +{"traceId":"cc2af584ecf2a705","parentSpanId":"0a741a3522748074","spanId":"8daeb0b90c77c79e","type":"LOCAL","operation":"dialogue-BlacklistingChannel-scheduler","startTimeMicroSeconds":1582666422907732,"durationNanoSeconds":1770029,"metadata":{}} +{"traceId":"cc2af584ecf2a705","parentSpanId":"ae91c04dc48d3823","spanId":"f8fae2d39ed7f596","type":"LOCAL","operation":"Dialogue-http-request initial","startTimeMicroSeconds":1582666423404387,"durationNanoSeconds":549102,"metadata":{}} +{"traceId":"cc2af584ecf2a705","parentSpanId":"e6e5748ef6b892ea","spanId":"ae91c04dc48d3823","type":"LOCAL","operation":"Dialogue-http-request","startTimeMicroSeconds":1582666423404347,"durationNanoSeconds":634413,"metadata":{}} +{"traceId":"cc2af584ecf2a705","parentSpanId":"8ccbd85267240f19","spanId":"e6e5748ef6b892ea","type":"LOCAL","operation":"Dialogue-request-attempt initial","startTimeMicroSeconds":1582666423404296,"durationNanoSeconds":791806,"metadata":{}} +{"traceId":"cc2af584ecf2a705","parentSpanId":"5489902871835754","spanId":"8ccbd85267240f19","type":"LOCAL","operation":"Dialogue-request-attempt","startTimeMicroSeconds":1582666423404214,"durationNanoSeconds":883959,"metadata":{}} +{"traceId":"cc2af584ecf2a705","parentSpanId":"8daeb0b90c77c79e","spanId":"5489902871835754","type":"LOCAL","operation":"dialogue-BlacklistingChannel-scheduler","startTimeMicroSeconds":1582666423404156,"durationNanoSeconds":1560627,"metadata":{}} +{"traceId":"cc2af584ecf2a705","parentSpanId":"52619fc7b27ab654","spanId":"96a31783efa996de","type":"LOCAL","operation":"Dialogue-http-request initial","startTimeMicroSeconds":1582666424863107,"durationNanoSeconds":415054,"metadata":{}} +{"traceId":"cc2af584ecf2a705","parentSpanId":"ea8aa8eacfd27d5d","spanId":"52619fc7b27ab654","type":"LOCAL","operation":"Dialogue-http-request","startTimeMicroSeconds":1582666424863095,"durationNanoSeconds":452835,"metadata":{}} +{"traceId":"cc2af584ecf2a705","parentSpanId":"4e43b755683cdcb2","spanId":"ea8aa8eacfd27d5d","type":"LOCAL","operation":"Dialogue-request-attempt initial","startTimeMicroSeconds":1582666424863070,"durationNanoSeconds":581276,"metadata":{}} +{"traceId":"cc2af584ecf2a705","parentSpanId":"eb3dc7fe2b367564","spanId":"4e43b755683cdcb2","type":"LOCAL","operation":"Dialogue-request-attempt","startTimeMicroSeconds":1582666424863050,"durationNanoSeconds":610269,"metadata":{}} +{"traceId":"cc2af584ecf2a705","parentSpanId":null,"spanId":"0662ae5d80d73804","type":"LOCAL","operation":"Dialogue-request","startTimeMicroSeconds":1582666422492806,"durationNanoSeconds":2375555274,"metadata":{}} +{"traceId":"cc2af584ecf2a705","parentSpanId":"5489902871835754","spanId":"eb3dc7fe2b367564","type":"LOCAL","operation":"dialogue-BlacklistingChannel-scheduler","startTimeMicroSeconds":1582666424863016,"durationNanoSeconds":5393223,"metadata":{}} diff --git a/simulation/src/main/java/com/palantir/dialogue/core/SimulationServer.java b/simulation/src/main/java/com/palantir/dialogue/core/SimulationServer.java index cd8de946a..439ac2164 100644 --- a/simulation/src/main/java/com/palantir/dialogue/core/SimulationServer.java +++ b/simulation/src/main/java/com/palantir/dialogue/core/SimulationServer.java @@ -84,9 +84,9 @@ public ListenableFuture execute(Endpoint endpoint, Request request) { } ListenableFuture resp = maybeResp.get(); + DialogueFutures.addDirectCallback(resp, DialogueFutures.onSuccess(ignored -> globalResponses.inc())); resp.addListener( () -> { - globalResponses.inc(); activeRequests.dec(); globalServerTimeNanos.inc(simulation.clock().read() - beforeNanos); }, diff --git a/simulation/src/main/java/com/palantir/dialogue/core/TestCaffeineTicker.java b/simulation/src/main/java/com/palantir/dialogue/core/TestCaffeineTicker.java index 6093c3d21..ec53f77af 100644 --- a/simulation/src/main/java/com/palantir/dialogue/core/TestCaffeineTicker.java +++ b/simulation/src/main/java/com/palantir/dialogue/core/TestCaffeineTicker.java @@ -17,6 +17,7 @@ import com.github.benmanes.caffeine.cache.Ticker; import com.palantir.logsafe.Preconditions; +import com.palantir.logsafe.SafeArg; import java.time.Duration; import java.util.concurrent.TimeUnit; import org.slf4j.Logger; @@ -38,7 +39,8 @@ void advanceTo(Duration duration) { long difference = nanos - newNanos; Preconditions.checkState( difference < Duration.ofMillis(1).toNanos(), - "Large time rewind - this is likely a bug in the test harness"); + "Large time rewind - this is likely a bug in the test harness", + SafeArg.of("difference", difference)); log.debug( "Tried to rewind time by {} micros - no-op as this is deterministic and harmless", TimeUnit.MICROSECONDS.convert(difference, TimeUnit.NANOSECONDS)); diff --git a/simulation/src/test/java/com/palantir/dialogue/core/Strategy.java b/simulation/src/test/java/com/palantir/dialogue/core/Strategy.java index 95865009e..cf094e34f 100644 --- a/simulation/src/test/java/com/palantir/dialogue/core/Strategy.java +++ b/simulation/src/test/java/com/palantir/dialogue/core/Strategy.java @@ -24,7 +24,6 @@ import com.palantir.dialogue.Endpoint; import com.palantir.dialogue.Request; import com.palantir.dialogue.Response; -import com.palantir.logsafe.Preconditions; import java.time.Duration; import java.util.List; import java.util.Optional; @@ -67,15 +66,14 @@ private static Channel concurrencyLimiter(Simulation sim, Supplier> channelSupplier) { - DeferredLimitedChannelListener listener = new DeferredLimitedChannelListener(); return RefreshingChannelFactory.RefreshingChannel.create(channelSupplier, channels -> { List limitedChannels = channels.stream() .map(addConcurrencyLimiter(sim)) .map(addFixedLimiter(sim)) - .map(c -> new BlacklistingChannel(c, Duration.ofSeconds(1), listener, sim.clock(), sim.scheduler())) + .map(c -> new BlacklistingChannel(c, Duration.ofSeconds(1), () -> {}, sim.clock(), sim.scheduler())) .collect(Collectors.toList()); LimitedChannel limited1 = new RoundRobinChannel(limitedChannels); - return queuedChannelAndRetrying(sim, limited1, listener); + return queuedChannelAndRetrying(sim, limited1); }); } @@ -105,7 +103,7 @@ private static Channel roundRobin(Simulation sim, Supplier addConcurrencyLimiter(Simulation sim) { return channel -> new ConcurrencyLimitedChannel( - new LimitedChannelAdapter(channel), + new ChannelToLimitedChannelAdapter(channel), ConcurrencyLimitedChannel.createLimiter(sim.clock()), DialogueClientMetrics.of(sim.taggedMetrics())); } @@ -115,18 +113,14 @@ private static Function addFixedLimiter(Simulati } private static Channel queuedChannelAndRetrying(Simulation sim, LimitedChannel limited) { - return queuedChannelAndRetrying(sim, limited, new DeferredLimitedChannelListener()); - } - - private static Channel queuedChannelAndRetrying( - Simulation sim, LimitedChannel limited, DeferredLimitedChannelListener listener) { LimitedChannel limited1 = instrumentClient(limited, sim.taggedMetrics()); - QueuedChannel channel = new QueuedChannel(limited1, DispatcherMetrics.of(sim.taggedMetrics())); - listener.delegate = channel::schedule; return new RetryingChannel( - channel, + new LimitedChannelToChannelAdapter(limited1), 4 /* ClientConfigurations.DEFAULT_MAX_NUM_RETRIES */, - ClientConfiguration.ServerQoS.AUTOMATIC_RETRY); + Duration.ofMillis(250) /* ClientConfigurations.DEFAULT_BACKOFF_SLOT_SIZE */, + ClientConfiguration.ServerQoS.AUTOMATIC_RETRY, + sim.scheduler(), + new Random(8 /* Guaranteed lucky */)::nextDouble); } private static LimitedChannel instrumentClient(LimitedChannel delegate, TaggedMetrics metrics) { @@ -167,14 +161,4 @@ public String toString() { } }; } - - private static final class DeferredLimitedChannelListener implements LimitedChannelListener { - private LimitedChannelListener delegate; - - @Override - public void onChannelReady() { - Preconditions.checkNotNull(delegate, "Delegate listener has not been initialized") - .onChannelReady(); - } - } } From b697561c81af5a020662be5e43d05ae58c87ccc2 Mon Sep 17 00:00:00 2001 From: Carter Kozak Date: Tue, 25 Feb 2020 16:50:11 -0500 Subject: [PATCH 02/17] reduce crufy from the queue --- .../dialogue/core/BlacklistingChannel.java | 44 ++----------------- .../dialogue/core/LimitedChannel.java | 3 -- .../dialogue/core/LimitedChannelListener.java | 29 ------------ .../core/BlacklistingChannelTest.java | 2 +- .../com/palantir/dialogue/core/Strategy.java | 2 +- 5 files changed, 6 insertions(+), 74 deletions(-) delete mode 100644 dialogue-core/src/main/java/com/palantir/dialogue/core/LimitedChannelListener.java diff --git a/dialogue-core/src/main/java/com/palantir/dialogue/core/BlacklistingChannel.java b/dialogue-core/src/main/java/com/palantir/dialogue/core/BlacklistingChannel.java index 74cbcb361..36aaa43e4 100644 --- a/dialogue-core/src/main/java/com/palantir/dialogue/core/BlacklistingChannel.java +++ b/dialogue-core/src/main/java/com/palantir/dialogue/core/BlacklistingChannel.java @@ -18,10 +18,8 @@ import com.github.benmanes.caffeine.cache.Ticker; import com.google.common.annotations.VisibleForTesting; -import com.google.common.base.Suppliers; import com.google.common.util.concurrent.FutureCallback; import com.google.common.util.concurrent.ListenableFuture; -import com.google.common.util.concurrent.ThreadFactoryBuilder; import com.palantir.dialogue.Endpoint; import com.palantir.dialogue.Request; import com.palantir.dialogue.Response; @@ -29,13 +27,8 @@ import com.palantir.logsafe.UnsafeArg; import java.time.Duration; import java.util.Optional; -import java.util.concurrent.Executors; -import java.util.concurrent.ScheduledExecutorService; -import java.util.concurrent.ScheduledFuture; -import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicReference; -import java.util.function.Supplier; import javax.annotation.Nullable; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -53,45 +46,21 @@ final class BlacklistingChannel implements LimitedChannel { private static final Logger log = LoggerFactory.getLogger(BlacklistingChannel.class); private static final int NUM_PROBATION_REQUESTS = 5; - /* - * Shared single thread executor is reused between all blacklisting channels. If it becomes oversaturated - * we may wait longer than expected before resuming requests to blacklisted channels, but this is an - * edge case where things are already operating in a degraded state. - */ - private static final Supplier sharedScheduler = Suppliers.memoize( - () -> Executors.newSingleThreadScheduledExecutor(new ThreadFactoryBuilder() - .setNameFormat("dialogue-BlacklistingChannel-scheduler-%d") - .setDaemon(false) - .build())); private final LimitedChannel delegate; private final Duration duration; private final Ticker ticker; - private final LimitedChannelListener listener; - private final ScheduledExecutorService scheduler; private final AtomicReference channelBlacklistState; - BlacklistingChannel(LimitedChannel delegate, Duration duration, LimitedChannelListener listener) { - this(delegate, duration, listener, Ticker.systemTicker(), sharedScheduler.get()); + BlacklistingChannel(LimitedChannel delegate, Duration duration) { + this(delegate, duration, Ticker.systemTicker()); } @VisibleForTesting - BlacklistingChannel(LimitedChannel delegate, Duration duration, LimitedChannelListener listener, Ticker ticker) { - this(delegate, duration, listener, ticker, sharedScheduler.get()); - } - - @VisibleForTesting - BlacklistingChannel( - LimitedChannel delegate, - Duration duration, - LimitedChannelListener listener, - Ticker ticker, - ScheduledExecutorService scheduler) { + BlacklistingChannel(LimitedChannel delegate, Duration duration, Ticker ticker) { this.delegate = delegate; this.duration = duration; this.ticker = ticker; - this.listener = listener; - this.scheduler = scheduler; this.channelBlacklistState = new AtomicReference<>(); } @@ -112,15 +81,12 @@ public String toString() { private final class BlacklistState { private final AtomicReference probation = new AtomicReference<>(); - private final ScheduledFuture scheduledFuture; private final long blacklistUntilNanos; private final int probationPermits; BlacklistState(Duration duration, int probationPermits) { this.blacklistUntilNanos = ticker.read() + duration.toNanos(); this.probationPermits = probationPermits; - this.scheduledFuture = - scheduler.schedule(this::maybeBeginProbation, duration.toNanos(), TimeUnit.NANOSECONDS); } Optional> maybeExecute(Endpoint endpoint, Request request) { @@ -148,8 +114,6 @@ Optional maybeBeginProbation() { if (log.isDebugEnabled()) { log.debug("Channel {} is entering probation", UnsafeArg.of("channel", delegate)); } - listener.onChannelReady(); - scheduledFuture.cancel(false); } return Optional.ofNullable(probation.get()); } @@ -161,7 +125,7 @@ void markSuccess() { if (maybeProbation != null && maybeProbation.checkIfProbationIsComplete()) { log.debug("Clearing probation state"); if (channelBlacklistState.compareAndSet(this, null)) { - listener.onChannelReady(); + log.debug("Channel is no longer blacklisted"); } else { log.debug("Blacklist state has already been updated"); } diff --git a/dialogue-core/src/main/java/com/palantir/dialogue/core/LimitedChannel.java b/dialogue-core/src/main/java/com/palantir/dialogue/core/LimitedChannel.java index a110842ab..0b615ac5d 100644 --- a/dialogue-core/src/main/java/com/palantir/dialogue/core/LimitedChannel.java +++ b/dialogue-core/src/main/java/com/palantir/dialogue/core/LimitedChannel.java @@ -26,9 +26,6 @@ * Similar to {@link com.palantir.dialogue.Channel}, but may not actually execute the call (eg: when the channel is * overloaded). Semantics match {@link com.palantir.dialogue.Channel} aside from returning an * {@link Optional optional response}. - * Limited channels must limit exclusively based on the state of the {@link com.palantir.dialogue.Channel}, not - * the {@link Endpoint} or {@link Request} arguments, otherwise the caller (generally a {@link QueuedChannel}) - * may prevent all requests from proceeding. */ interface LimitedChannel { Optional> maybeExecute(Endpoint endpoint, Request request); diff --git a/dialogue-core/src/main/java/com/palantir/dialogue/core/LimitedChannelListener.java b/dialogue-core/src/main/java/com/palantir/dialogue/core/LimitedChannelListener.java deleted file mode 100644 index bcb31c2f2..000000000 --- a/dialogue-core/src/main/java/com/palantir/dialogue/core/LimitedChannelListener.java +++ /dev/null @@ -1,29 +0,0 @@ -/* - * (c) Copyright 2020 Palantir Technologies Inc. All rights reserved. - * - * Licensed 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 com.palantir.dialogue.core; - -@FunctionalInterface -interface LimitedChannelListener { - - /** - * Invoked when requests may succeed. There is no guarantee that requests will be accepted. - * This is only necessary if edge-triggering is not sufficient. For example if permits are based - * on the number of active requests, when existing requests complete this is triggered automatically. - * LimitedChannel implementations should invoke {@link #onChannelReady()} when another - * event (scheduled timeout) allows requests to proceed. - */ - void onChannelReady(); -} diff --git a/dialogue-core/src/test/java/com/palantir/dialogue/core/BlacklistingChannelTest.java b/dialogue-core/src/test/java/com/palantir/dialogue/core/BlacklistingChannelTest.java index d03269187..c11e38886 100644 --- a/dialogue-core/src/test/java/com/palantir/dialogue/core/BlacklistingChannelTest.java +++ b/dialogue-core/src/test/java/com/palantir/dialogue/core/BlacklistingChannelTest.java @@ -57,7 +57,7 @@ public class BlacklistingChannelTest { @BeforeEach public void before() { - channel = new BlacklistingChannel(delegate, BLACKLIST_DURATION, () -> {}, ticker); + channel = new BlacklistingChannel(delegate, BLACKLIST_DURATION, ticker); futureResponse = SettableFuture.create(); when(delegate.maybeExecute(endpoint, request)).thenReturn(Optional.of(futureResponse)); diff --git a/simulation/src/test/java/com/palantir/dialogue/core/Strategy.java b/simulation/src/test/java/com/palantir/dialogue/core/Strategy.java index cf094e34f..2d5af8fee 100644 --- a/simulation/src/test/java/com/palantir/dialogue/core/Strategy.java +++ b/simulation/src/test/java/com/palantir/dialogue/core/Strategy.java @@ -70,7 +70,7 @@ private static Channel concurrencyLimiterBlacklistRoundRobin( List limitedChannels = channels.stream() .map(addConcurrencyLimiter(sim)) .map(addFixedLimiter(sim)) - .map(c -> new BlacklistingChannel(c, Duration.ofSeconds(1), () -> {}, sim.clock(), sim.scheduler())) + .map(c -> new BlacklistingChannel(c, Duration.ofSeconds(1), sim.clock())) .collect(Collectors.toList()); LimitedChannel limited1 = new RoundRobinChannel(limitedChannels); return queuedChannelAndRetrying(sim, limited1); From e83923fc13ee81fd75cdefcea4ed2f42980a4c2f Mon Sep 17 00:00:00 2001 From: Carter Kozak Date: Tue, 25 Feb 2020 17:10:50 -0500 Subject: [PATCH 03/17] Fix the simulation scheduler to avoid fast-forward in external use --- .../com/palantir/dialogue/core/Benchmark.java | 4 +- .../core/ExternalDeterministicScheduler.java | 17 ++-- .../palantir/dialogue/core/Simulation.java | 5 +- ...URRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].png | 4 +- ...0[CONCURRENCY_LIMITER_PIN_UNTIL_ERROR].png | 4 +- ...s_500[CONCURRENCY_LIMITER_ROUND_ROBIN].png | 4 +- .../all_nodes_500[UNLIMITED_ROUND_ROBIN].png | 4 +- ...URRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].png | 4 +- ...e[CONCURRENCY_LIMITER_PIN_UNTIL_ERROR].png | 4 +- ..._hole[CONCURRENCY_LIMITER_ROUND_ROBIN].png | 4 +- .../black_hole[UNLIMITED_ROUND_ROBIN].png | 4 +- ...URRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].png | 4 +- ...n[CONCURRENCY_LIMITER_PIN_UNTIL_ERROR].png | 4 +- ...wdown[CONCURRENCY_LIMITER_ROUND_ROBIN].png | 4 +- ...rastic_slowdown[UNLIMITED_ROUND_ROBIN].png | 4 +- ...URRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].png | 4 +- ...t[CONCURRENCY_LIMITER_PIN_UNTIL_ERROR].png | 4 +- ...evert[CONCURRENCY_LIMITER_ROUND_ROBIN].png | 4 +- ...00s_then_revert[UNLIMITED_ROUND_ROBIN].png | 4 +- ...URRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].png | 4 +- ...g[CONCURRENCY_LIMITER_PIN_UNTIL_ERROR].png | 4 +- ...ading[CONCURRENCY_LIMITER_ROUND_ROBIN].png | 4 +- .../live_reloading[UNLIMITED_ROUND_ROBIN].png | 4 +- ...URRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].png | 4 +- ...r[CONCURRENCY_LIMITER_PIN_UNTIL_ERROR].png | 4 +- ...erver[CONCURRENCY_LIMITER_ROUND_ROBIN].png | 4 +- ..._on_each_server[UNLIMITED_ROUND_ROBIN].png | 4 +- simulation/src/test/resources/report.md | 80 +++++++++---------- ...URRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].png | 4 +- ...e[CONCURRENCY_LIMITER_PIN_UNTIL_ERROR].png | 4 +- ..._case[CONCURRENCY_LIMITER_ROUND_ROBIN].png | 4 +- ...t_possible_case[UNLIMITED_ROUND_ROBIN].png | 4 +- ...URRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].png | 4 +- ...t[CONCURRENCY_LIMITER_PIN_UNTIL_ERROR].png | 4 +- ...evert[CONCURRENCY_LIMITER_ROUND_ROBIN].png | 4 +- ...03s_then_revert[UNLIMITED_ROUND_ROBIN].png | 4 +- ...URRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].png | 4 +- ...s[CONCURRENCY_LIMITER_PIN_UNTIL_ERROR].png | 4 +- ...holds[CONCURRENCY_LIMITER_ROUND_ROBIN].png | 4 +- ...rror_thresholds[UNLIMITED_ROUND_ROBIN].png | 4 +- ...URRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].txt | 2 +- ...0[CONCURRENCY_LIMITER_PIN_UNTIL_ERROR].txt | 2 +- ...s_500[CONCURRENCY_LIMITER_ROUND_ROBIN].txt | 2 +- .../all_nodes_500[UNLIMITED_ROUND_ROBIN].txt | 2 +- ...URRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].txt | 2 +- ...e[CONCURRENCY_LIMITER_PIN_UNTIL_ERROR].txt | 2 +- ..._hole[CONCURRENCY_LIMITER_ROUND_ROBIN].txt | 2 +- .../txt/black_hole[UNLIMITED_ROUND_ROBIN].txt | 2 +- ...URRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].txt | 2 +- ...n[CONCURRENCY_LIMITER_PIN_UNTIL_ERROR].txt | 2 +- ...wdown[CONCURRENCY_LIMITER_ROUND_ROBIN].txt | 2 +- ...rastic_slowdown[UNLIMITED_ROUND_ROBIN].txt | 2 +- ...URRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].txt | 2 +- ...t[CONCURRENCY_LIMITER_PIN_UNTIL_ERROR].txt | 2 +- ...evert[CONCURRENCY_LIMITER_ROUND_ROBIN].txt | 2 +- ...00s_then_revert[UNLIMITED_ROUND_ROBIN].txt | 2 +- ...URRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].txt | 2 +- ...g[CONCURRENCY_LIMITER_PIN_UNTIL_ERROR].txt | 2 +- ...ading[CONCURRENCY_LIMITER_ROUND_ROBIN].txt | 2 +- .../live_reloading[UNLIMITED_ROUND_ROBIN].txt | 2 +- ...URRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].txt | 2 +- ...r[CONCURRENCY_LIMITER_PIN_UNTIL_ERROR].txt | 2 +- ...erver[CONCURRENCY_LIMITER_ROUND_ROBIN].txt | 2 +- ..._on_each_server[UNLIMITED_ROUND_ROBIN].txt | 2 +- ...URRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].txt | 2 +- ...e[CONCURRENCY_LIMITER_PIN_UNTIL_ERROR].txt | 2 +- ..._case[CONCURRENCY_LIMITER_ROUND_ROBIN].txt | 2 +- ...t_possible_case[UNLIMITED_ROUND_ROBIN].txt | 2 +- ...URRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].txt | 2 +- ...t[CONCURRENCY_LIMITER_PIN_UNTIL_ERROR].txt | 2 +- ...evert[CONCURRENCY_LIMITER_ROUND_ROBIN].txt | 2 +- ...03s_then_revert[UNLIMITED_ROUND_ROBIN].txt | 2 +- ...URRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].txt | 2 +- ...s[CONCURRENCY_LIMITER_PIN_UNTIL_ERROR].txt | 2 +- ...holds[CONCURRENCY_LIMITER_ROUND_ROBIN].txt | 2 +- ...rror_thresholds[UNLIMITED_ROUND_ROBIN].txt | 2 +- ...URRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].txt | 2 +- ...s[CONCURRENCY_LIMITER_PIN_UNTIL_ERROR].txt | 2 +- ...lakes[CONCURRENCY_LIMITER_ROUND_ROBIN].txt | 2 +- ...uncommon_flakes[UNLIMITED_ROUND_ROBIN].txt | 2 +- ...URRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].png | 4 +- ...s[CONCURRENCY_LIMITER_PIN_UNTIL_ERROR].png | 4 +- ...lakes[CONCURRENCY_LIMITER_ROUND_ROBIN].png | 4 +- ...uncommon_flakes[UNLIMITED_ROUND_ROBIN].png | 4 +- 84 files changed, 172 insertions(+), 174 deletions(-) diff --git a/simulation/src/main/java/com/palantir/dialogue/core/Benchmark.java b/simulation/src/main/java/com/palantir/dialogue/core/Benchmark.java index 6d6fb3e12..10425e8f8 100644 --- a/simulation/src/main/java/com/palantir/dialogue/core/Benchmark.java +++ b/simulation/src/main/java/com/palantir/dialogue/core/Benchmark.java @@ -142,7 +142,7 @@ public void update(Duration _time, long _requestsStarted, long responsesReceived public Benchmark abortAfter(Duration cutoff) { simulation .scheduler() - .schedule( + .scheduleAndFastForward( () -> { log.warn( "Aborted running benchmark after cutoff reached - strategy might be buggy {}", @@ -192,7 +192,7 @@ public void onFailure(Throwable throwable) { simulation .scheduler() - .schedule( + .scheduleAndFastForward( () -> { log.debug( "time={} starting num={} {}", diff --git a/simulation/src/main/java/com/palantir/dialogue/core/ExternalDeterministicScheduler.java b/simulation/src/main/java/com/palantir/dialogue/core/ExternalDeterministicScheduler.java index 48d5a7595..0266b54b0 100644 --- a/simulation/src/main/java/com/palantir/dialogue/core/ExternalDeterministicScheduler.java +++ b/simulation/src/main/java/com/palantir/dialogue/core/ExternalDeterministicScheduler.java @@ -39,26 +39,25 @@ final class ExternalDeterministicScheduler implements ListeningScheduledExecutor @Override public ListenableScheduledFuture schedule(Runnable command, long delay, TimeUnit unit) { - return schedule( - () -> { - command.run(); - return null; - }, - delay, - unit); + return delegate.schedule(command, delay, unit); } @Override public ListenableScheduledFuture schedule(Callable command, long delay, TimeUnit unit) { + return delegate.schedule(command, delay, unit); + } + + @SuppressWarnings({"CheckReturnValue", "FutureReturnValueIgnored"}) + public void scheduleAndFastForward(Runnable command, long delay, TimeUnit unit) { long scheduleTime = ticker.read(); long delayNanos = unit.toNanos(delay); RuntimeException exceptionForStackTrace = new RuntimeException(); - return delegate.schedule( + delegate.schedule( () -> { try { ticker.advanceTo(Duration.ofNanos(scheduleTime + delayNanos)); - return command.call(); + command.run(); } catch (Throwable e) { e.addSuppressed(exceptionForStackTrace); throw e; diff --git a/simulation/src/main/java/com/palantir/dialogue/core/Simulation.java b/simulation/src/main/java/com/palantir/dialogue/core/Simulation.java index 023ec5925..7926c60a3 100644 --- a/simulation/src/main/java/com/palantir/dialogue/core/Simulation.java +++ b/simulation/src/main/java/com/palantir/dialogue/core/Simulation.java @@ -17,7 +17,6 @@ package com.palantir.dialogue.core; import com.github.benmanes.caffeine.cache.Ticker; -import com.google.common.util.concurrent.ListeningScheduledExecutorService; import com.google.common.util.concurrent.MoreExecutors; import java.time.Duration; import java.util.concurrent.TimeUnit; @@ -30,7 +29,7 @@ final class Simulation { private static final Logger log = LoggerFactory.getLogger(Simulation.class); private final DeterministicScheduler deterministicExecutor = new DeterministicScheduler(); - private final ListeningScheduledExecutorService listenableExecutor; + private final ExternalDeterministicScheduler listenableExecutor; private final TestCaffeineTicker ticker = new TestCaffeineTicker(); private final SimulationMetricsReporter metrics = new SimulationMetricsReporter(this); @@ -48,7 +47,7 @@ public Ticker clock() { return ticker; // read only! } - public ListeningScheduledExecutorService scheduler() { + public ExternalDeterministicScheduler scheduler() { return listenableExecutor; } diff --git a/simulation/src/test/resources/all_nodes_500[CONCURRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].png b/simulation/src/test/resources/all_nodes_500[CONCURRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].png index 6ca270542..5acb32325 100644 --- a/simulation/src/test/resources/all_nodes_500[CONCURRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].png +++ b/simulation/src/test/resources/all_nodes_500[CONCURRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].png @@ -1,3 +1,3 @@ version https://git-lfs.github.com/spec/v1 -oid sha256:6d277fb4db30e57d37781b561e6a75549322c9d4a7efcf074de1ec1388886563 -size 116386 +oid sha256:7a84d2aae6f10659e002899e1c04cff6ccba471f985b7860726055026251c7c1 +size 112141 diff --git a/simulation/src/test/resources/all_nodes_500[CONCURRENCY_LIMITER_PIN_UNTIL_ERROR].png b/simulation/src/test/resources/all_nodes_500[CONCURRENCY_LIMITER_PIN_UNTIL_ERROR].png index 2dee3a969..69e68cf86 100644 --- a/simulation/src/test/resources/all_nodes_500[CONCURRENCY_LIMITER_PIN_UNTIL_ERROR].png +++ b/simulation/src/test/resources/all_nodes_500[CONCURRENCY_LIMITER_PIN_UNTIL_ERROR].png @@ -1,3 +1,3 @@ version https://git-lfs.github.com/spec/v1 -oid sha256:64a876183e62ccf0399d76ce6674f3235f28af6d37f5d8159600ddd0de7a3b4d -size 159328 +oid sha256:7dcf77450d2446563ebc46c0cf74c8ccc883f45810cd84d57a9401afd9ad1ad4 +size 155428 diff --git a/simulation/src/test/resources/all_nodes_500[CONCURRENCY_LIMITER_ROUND_ROBIN].png b/simulation/src/test/resources/all_nodes_500[CONCURRENCY_LIMITER_ROUND_ROBIN].png index 36277564c..81269dfef 100644 --- a/simulation/src/test/resources/all_nodes_500[CONCURRENCY_LIMITER_ROUND_ROBIN].png +++ b/simulation/src/test/resources/all_nodes_500[CONCURRENCY_LIMITER_ROUND_ROBIN].png @@ -1,3 +1,3 @@ version https://git-lfs.github.com/spec/v1 -oid sha256:c1cf35f4684c0ef9ebcef8d97cd094fab3b3d3036e93a757771b9d452ba6a936 -size 119011 +oid sha256:c8d45d45ff5b5556e1bdaf585e841e7d347a3ba71caa4d4f55e1c7ef370bea83 +size 114957 diff --git a/simulation/src/test/resources/all_nodes_500[UNLIMITED_ROUND_ROBIN].png b/simulation/src/test/resources/all_nodes_500[UNLIMITED_ROUND_ROBIN].png index a9e751070..05b5871cf 100644 --- a/simulation/src/test/resources/all_nodes_500[UNLIMITED_ROUND_ROBIN].png +++ b/simulation/src/test/resources/all_nodes_500[UNLIMITED_ROUND_ROBIN].png @@ -1,3 +1,3 @@ version https://git-lfs.github.com/spec/v1 -oid sha256:9206e52a7e6fd933424f902b47bb04bd72a6b940c7a077c35f34c08f35d4c5f9 -size 118264 +oid sha256:364db8979470ce2b022af4a94a99151170d233feff4562125bfef6d5c10119c9 +size 114590 diff --git a/simulation/src/test/resources/black_hole[CONCURRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].png b/simulation/src/test/resources/black_hole[CONCURRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].png index 378513ee2..956aa48e2 100644 --- a/simulation/src/test/resources/black_hole[CONCURRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].png +++ b/simulation/src/test/resources/black_hole[CONCURRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].png @@ -1,3 +1,3 @@ version https://git-lfs.github.com/spec/v1 -oid sha256:2bd44c6da370c5c5d32b4d517968b517322985bd02b1fa52ff0ba51ea0bf6c9e -size 92351 +oid sha256:2628352ee622ebb23bc8db8e55a113e0f866eef897c0a75a35dba8e2ae188f2f +size 92380 diff --git a/simulation/src/test/resources/black_hole[CONCURRENCY_LIMITER_PIN_UNTIL_ERROR].png b/simulation/src/test/resources/black_hole[CONCURRENCY_LIMITER_PIN_UNTIL_ERROR].png index 8216783ca..cf6def8de 100644 --- a/simulation/src/test/resources/black_hole[CONCURRENCY_LIMITER_PIN_UNTIL_ERROR].png +++ b/simulation/src/test/resources/black_hole[CONCURRENCY_LIMITER_PIN_UNTIL_ERROR].png @@ -1,3 +1,3 @@ version https://git-lfs.github.com/spec/v1 -oid sha256:4dadf1d8214146660398adb4480ce8e23b7e421591e0fcd63b3fe1d473e4ac49 -size 99715 +oid sha256:77df5f8afe5c6493aed12915ad7de18febe14bac80deae28e61cf7bce2f378cb +size 89878 diff --git a/simulation/src/test/resources/black_hole[CONCURRENCY_LIMITER_ROUND_ROBIN].png b/simulation/src/test/resources/black_hole[CONCURRENCY_LIMITER_ROUND_ROBIN].png index ac59336ed..417677410 100644 --- a/simulation/src/test/resources/black_hole[CONCURRENCY_LIMITER_ROUND_ROBIN].png +++ b/simulation/src/test/resources/black_hole[CONCURRENCY_LIMITER_ROUND_ROBIN].png @@ -1,3 +1,3 @@ version https://git-lfs.github.com/spec/v1 -oid sha256:3fcb1a7c1c92bab157ee4d7327ed599f616b0225b6f2b0d5cd89af851bb9a5ae -size 101855 +oid sha256:4bf6fcbe5dd7ba92c082d81a3eb26f030aa1672cd06171139e9fea303e12ba12 +size 92167 diff --git a/simulation/src/test/resources/black_hole[UNLIMITED_ROUND_ROBIN].png b/simulation/src/test/resources/black_hole[UNLIMITED_ROUND_ROBIN].png index 24e148ac3..fdb030b33 100644 --- a/simulation/src/test/resources/black_hole[UNLIMITED_ROUND_ROBIN].png +++ b/simulation/src/test/resources/black_hole[UNLIMITED_ROUND_ROBIN].png @@ -1,3 +1,3 @@ version https://git-lfs.github.com/spec/v1 -oid sha256:66ee7c54b8a509c06d54dfc5647cadc85c83e8aea425da35200612210d3b0a78 -size 98870 +oid sha256:a11ebbd8893e89a96cdefb16e1b54816d1b76ce38340372e2d083e9fb171ac72 +size 90701 diff --git a/simulation/src/test/resources/drastic_slowdown[CONCURRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].png b/simulation/src/test/resources/drastic_slowdown[CONCURRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].png index 754bcf2f0..fd9a8081b 100644 --- a/simulation/src/test/resources/drastic_slowdown[CONCURRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].png +++ b/simulation/src/test/resources/drastic_slowdown[CONCURRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].png @@ -1,3 +1,3 @@ version https://git-lfs.github.com/spec/v1 -oid sha256:63b1307da66a21ca8f467bd2848edca400abe6d4a45598c1fcb2cf7359d56ab5 -size 91914 +oid sha256:d9480c7f5981778bbbdf4ae0876f382eff0a07e1f9a674ea94d8869c68da9ea9 +size 91734 diff --git a/simulation/src/test/resources/drastic_slowdown[CONCURRENCY_LIMITER_PIN_UNTIL_ERROR].png b/simulation/src/test/resources/drastic_slowdown[CONCURRENCY_LIMITER_PIN_UNTIL_ERROR].png index c79b89ad5..17372b788 100644 --- a/simulation/src/test/resources/drastic_slowdown[CONCURRENCY_LIMITER_PIN_UNTIL_ERROR].png +++ b/simulation/src/test/resources/drastic_slowdown[CONCURRENCY_LIMITER_PIN_UNTIL_ERROR].png @@ -1,3 +1,3 @@ version https://git-lfs.github.com/spec/v1 -oid sha256:49f238323493f96e1e18cf9435d81a3c28aee7200f64ef0d20fe3518f8fda777 -size 99474 +oid sha256:0787ef84ca0da4b55a4524c4febe92f9d9485c93535246596737deeaee12a64c +size 88105 diff --git a/simulation/src/test/resources/drastic_slowdown[CONCURRENCY_LIMITER_ROUND_ROBIN].png b/simulation/src/test/resources/drastic_slowdown[CONCURRENCY_LIMITER_ROUND_ROBIN].png index bffc4af86..ed70347b4 100644 --- a/simulation/src/test/resources/drastic_slowdown[CONCURRENCY_LIMITER_ROUND_ROBIN].png +++ b/simulation/src/test/resources/drastic_slowdown[CONCURRENCY_LIMITER_ROUND_ROBIN].png @@ -1,3 +1,3 @@ version https://git-lfs.github.com/spec/v1 -oid sha256:4d720842d172faba5aad6feb1302ed0a976790208a368315ede0530610042621 -size 103423 +oid sha256:d66270aefd6b655298a7b1040be6fa4d98d1f166fc7b1d2d2f8acc282eb18968 +size 91630 diff --git a/simulation/src/test/resources/drastic_slowdown[UNLIMITED_ROUND_ROBIN].png b/simulation/src/test/resources/drastic_slowdown[UNLIMITED_ROUND_ROBIN].png index 0485b08f9..d46b4a2c5 100644 --- a/simulation/src/test/resources/drastic_slowdown[UNLIMITED_ROUND_ROBIN].png +++ b/simulation/src/test/resources/drastic_slowdown[UNLIMITED_ROUND_ROBIN].png @@ -1,3 +1,3 @@ version https://git-lfs.github.com/spec/v1 -oid sha256:8135e8843614ab19264be1fdc9f01a7f6458151994db52d472ac52e0e684ead8 -size 101162 +oid sha256:67b5d3a78d4ab3826780e3e9cf4a742afba9193b3ec3dcb9dcad26a5bdf9e84d +size 91271 diff --git a/simulation/src/test/resources/fast_500s_then_revert[CONCURRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].png b/simulation/src/test/resources/fast_500s_then_revert[CONCURRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].png index 85599efc6..7ed0805ea 100644 --- a/simulation/src/test/resources/fast_500s_then_revert[CONCURRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].png +++ b/simulation/src/test/resources/fast_500s_then_revert[CONCURRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].png @@ -1,3 +1,3 @@ version https://git-lfs.github.com/spec/v1 -oid sha256:8ddb45a3f1d7a276490cae23ceac6cbc7645b43e56fa541da9dfc60d1364446e -size 118966 +oid sha256:ae302eb217e284df84d90541843ba5c19ed2333129b6f377880b9c4094302ce1 +size 119266 diff --git a/simulation/src/test/resources/fast_500s_then_revert[CONCURRENCY_LIMITER_PIN_UNTIL_ERROR].png b/simulation/src/test/resources/fast_500s_then_revert[CONCURRENCY_LIMITER_PIN_UNTIL_ERROR].png index 401e5b12a..cca6be792 100644 --- a/simulation/src/test/resources/fast_500s_then_revert[CONCURRENCY_LIMITER_PIN_UNTIL_ERROR].png +++ b/simulation/src/test/resources/fast_500s_then_revert[CONCURRENCY_LIMITER_PIN_UNTIL_ERROR].png @@ -1,3 +1,3 @@ version https://git-lfs.github.com/spec/v1 -oid sha256:eddee277800705d8c3e197a55fe4de3c3af5e0ee63fa4727f4f45b713031e6e5 -size 94151 +oid sha256:6acb15dcb1c0ccce74b95b1201adf53c9a6fb1214bc757332449ab8285a281b9 +size 82977 diff --git a/simulation/src/test/resources/fast_500s_then_revert[CONCURRENCY_LIMITER_ROUND_ROBIN].png b/simulation/src/test/resources/fast_500s_then_revert[CONCURRENCY_LIMITER_ROUND_ROBIN].png index 2dc2cceba..2e5982211 100644 --- a/simulation/src/test/resources/fast_500s_then_revert[CONCURRENCY_LIMITER_ROUND_ROBIN].png +++ b/simulation/src/test/resources/fast_500s_then_revert[CONCURRENCY_LIMITER_ROUND_ROBIN].png @@ -1,3 +1,3 @@ version https://git-lfs.github.com/spec/v1 -oid sha256:562699fcaa3aa8030b2667090dcadeb9e671bbce4778daa512c6cd195311d366 -size 176556 +oid sha256:128a4ec3eea36f41a3adf456397b2aa06a3800b4e7180361b02f7a012ea796f9 +size 150416 diff --git a/simulation/src/test/resources/fast_500s_then_revert[UNLIMITED_ROUND_ROBIN].png b/simulation/src/test/resources/fast_500s_then_revert[UNLIMITED_ROUND_ROBIN].png index 5a63b5b18..b8ca36581 100644 --- a/simulation/src/test/resources/fast_500s_then_revert[UNLIMITED_ROUND_ROBIN].png +++ b/simulation/src/test/resources/fast_500s_then_revert[UNLIMITED_ROUND_ROBIN].png @@ -1,3 +1,3 @@ version https://git-lfs.github.com/spec/v1 -oid sha256:0ef9efd54ba5ffd4d0e309fa5e5ba7eea38bc6004c4ff794b488608a76a40576 -size 175716 +oid sha256:82747f174c2e9e3b6ac49527625b67fae6ea29846b449a51cd7ce5bd88a4f595 +size 150032 diff --git a/simulation/src/test/resources/live_reloading[CONCURRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].png b/simulation/src/test/resources/live_reloading[CONCURRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].png index 943cffac4..47a883da5 100644 --- a/simulation/src/test/resources/live_reloading[CONCURRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].png +++ b/simulation/src/test/resources/live_reloading[CONCURRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].png @@ -1,3 +1,3 @@ version https://git-lfs.github.com/spec/v1 -oid sha256:94ccf0b415c2d89aba022b855bfe1f573a5f222a82717f567ec748bfd9c4cbfe -size 119577 +oid sha256:84873a37e5ae3b240af0addf29c5bcd1c7b1bc3f2b84f562a3d100d82649c6f1 +size 108678 diff --git a/simulation/src/test/resources/live_reloading[CONCURRENCY_LIMITER_PIN_UNTIL_ERROR].png b/simulation/src/test/resources/live_reloading[CONCURRENCY_LIMITER_PIN_UNTIL_ERROR].png index 6f39009c4..d8e882b85 100644 --- a/simulation/src/test/resources/live_reloading[CONCURRENCY_LIMITER_PIN_UNTIL_ERROR].png +++ b/simulation/src/test/resources/live_reloading[CONCURRENCY_LIMITER_PIN_UNTIL_ERROR].png @@ -1,3 +1,3 @@ version https://git-lfs.github.com/spec/v1 -oid sha256:79e8c58cc7f329b4cfc9c271dbc17fca278c4384a6301b4349ac49fab7da451b -size 117229 +oid sha256:6be6d8e39bad03dce2e67c0934f49a61faa6ba017daa37ea2196d1511890640e +size 117475 diff --git a/simulation/src/test/resources/live_reloading[CONCURRENCY_LIMITER_ROUND_ROBIN].png b/simulation/src/test/resources/live_reloading[CONCURRENCY_LIMITER_ROUND_ROBIN].png index 4165d3d09..46876cac5 100644 --- a/simulation/src/test/resources/live_reloading[CONCURRENCY_LIMITER_ROUND_ROBIN].png +++ b/simulation/src/test/resources/live_reloading[CONCURRENCY_LIMITER_ROUND_ROBIN].png @@ -1,3 +1,3 @@ version https://git-lfs.github.com/spec/v1 -oid sha256:a02ca5b1f32ec2c07a3ee7b2cf986ee30280d7cd080e20cac08e1eeae3fb9736 -size 119001 +oid sha256:db54f48e239d2cbb952fdd65bcaff9061f7471eee79bd216ae21f754233875dd +size 108969 diff --git a/simulation/src/test/resources/live_reloading[UNLIMITED_ROUND_ROBIN].png b/simulation/src/test/resources/live_reloading[UNLIMITED_ROUND_ROBIN].png index 4c3110b6e..8320f324f 100644 --- a/simulation/src/test/resources/live_reloading[UNLIMITED_ROUND_ROBIN].png +++ b/simulation/src/test/resources/live_reloading[UNLIMITED_ROUND_ROBIN].png @@ -1,3 +1,3 @@ version https://git-lfs.github.com/spec/v1 -oid sha256:476ef8be829752a0a64dab4199dc58689da8ca6f7da6f9f0e24384693278f63a -size 126261 +oid sha256:1ad481026594b9c6eed222589147158d429bf4d1041dfa0efb2f81b8a85c9e98 +size 115690 diff --git a/simulation/src/test/resources/one_endpoint_dies_on_each_server[CONCURRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].png b/simulation/src/test/resources/one_endpoint_dies_on_each_server[CONCURRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].png index 637d072bd..bf7c161ab 100644 --- a/simulation/src/test/resources/one_endpoint_dies_on_each_server[CONCURRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].png +++ b/simulation/src/test/resources/one_endpoint_dies_on_each_server[CONCURRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].png @@ -1,3 +1,3 @@ version https://git-lfs.github.com/spec/v1 -oid sha256:a2b4674191786ac8c740fe0d5436d510246260257c34c574c315a0c17a8d30fe -size 198566 +oid sha256:db1b1baf41c42a9e7ec0688c8b73f757aceb4c5312302a31cfd21c493860e185 +size 141939 diff --git a/simulation/src/test/resources/one_endpoint_dies_on_each_server[CONCURRENCY_LIMITER_PIN_UNTIL_ERROR].png b/simulation/src/test/resources/one_endpoint_dies_on_each_server[CONCURRENCY_LIMITER_PIN_UNTIL_ERROR].png index ff2052111..b260fb194 100644 --- a/simulation/src/test/resources/one_endpoint_dies_on_each_server[CONCURRENCY_LIMITER_PIN_UNTIL_ERROR].png +++ b/simulation/src/test/resources/one_endpoint_dies_on_each_server[CONCURRENCY_LIMITER_PIN_UNTIL_ERROR].png @@ -1,3 +1,3 @@ version https://git-lfs.github.com/spec/v1 -oid sha256:4e14b1833a27716ef986e16b44924b357a9c55e7866c56a000a712ea3a150c5b -size 149667 +oid sha256:316170018d54b7fa457d65cf74522c7db0eb377c1d035e5678e1a4d71cb166d9 +size 149974 diff --git a/simulation/src/test/resources/one_endpoint_dies_on_each_server[CONCURRENCY_LIMITER_ROUND_ROBIN].png b/simulation/src/test/resources/one_endpoint_dies_on_each_server[CONCURRENCY_LIMITER_ROUND_ROBIN].png index b5872ea53..03bd899ca 100644 --- a/simulation/src/test/resources/one_endpoint_dies_on_each_server[CONCURRENCY_LIMITER_ROUND_ROBIN].png +++ b/simulation/src/test/resources/one_endpoint_dies_on_each_server[CONCURRENCY_LIMITER_ROUND_ROBIN].png @@ -1,3 +1,3 @@ version https://git-lfs.github.com/spec/v1 -oid sha256:91894f217722789062b6527ef42638c7f9c56b8aea2b7c4d107c5ee7395fe1c9 -size 121368 +oid sha256:a9aae71317e5c64bb62195afc6851f1bdbef2430767b7963b7840a4427ed081c +size 113216 diff --git a/simulation/src/test/resources/one_endpoint_dies_on_each_server[UNLIMITED_ROUND_ROBIN].png b/simulation/src/test/resources/one_endpoint_dies_on_each_server[UNLIMITED_ROUND_ROBIN].png index 5d2489845..e0349a6a8 100644 --- a/simulation/src/test/resources/one_endpoint_dies_on_each_server[UNLIMITED_ROUND_ROBIN].png +++ b/simulation/src/test/resources/one_endpoint_dies_on_each_server[UNLIMITED_ROUND_ROBIN].png @@ -1,3 +1,3 @@ version https://git-lfs.github.com/spec/v1 -oid sha256:ddc3e68fa8491589a6e25bc0bd8cc375cf1726d54042ebbbf3295fd9963e0ffb -size 120527 +oid sha256:27db45d539791d10bda2f2a3a7f7565b605049c7e6f3dc7211c22530a85aea26 +size 112929 diff --git a/simulation/src/test/resources/report.md b/simulation/src/test/resources/report.md index a41a9dcfe..9c034433b 100644 --- a/simulation/src/test/resources/report.md +++ b/simulation/src/test/resources/report.md @@ -1,46 +1,46 @@ # Report ``` - all_nodes_500[CONCURRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].txt: success=66.3% client_mean=PT1.87116S server_cpu=PT20M client_received=2000/2000 server_resps=2000 codes={200=1326, 500=674} - all_nodes_500[CONCURRENCY_LIMITER_PIN_UNTIL_ERROR].txt: success=50.0% client_mean=PT0.6S server_cpu=PT20M client_received=2000/2000 server_resps=2000 codes={200=1000, 500=1000} - all_nodes_500[CONCURRENCY_LIMITER_ROUND_ROBIN].txt: success=50.0% client_mean=PT0.6S server_cpu=PT20M client_received=2000/2000 server_resps=2000 codes={200=1000, 500=1000} - all_nodes_500[UNLIMITED_ROUND_ROBIN].txt: success=50.0% client_mean=PT0.6S server_cpu=PT20M client_received=2000/2000 server_resps=2000 codes={200=1000, 500=1000} - black_hole[CONCURRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].txt: success=89.9% client_mean=PT0.600086254S server_cpu=PT17M58.2S client_received=1797/2000 server_resps=1797 codes={200=1797} - black_hole[CONCURRENCY_LIMITER_PIN_UNTIL_ERROR].txt: success=88.7% client_mean=PT0.600191765S server_cpu=PT17M43.8S client_received=1773/2000 server_resps=1773 codes={200=1773} - black_hole[CONCURRENCY_LIMITER_ROUND_ROBIN].txt: success=89.9% client_mean=PT0.600086254S server_cpu=PT17M58.2S client_received=1797/2000 server_resps=1797 codes={200=1797} - black_hole[UNLIMITED_ROUND_ROBIN].txt: success=65.0% client_mean=PT0.6S server_cpu=PT12M59.4S client_received=1299/2000 server_resps=1299 codes={200=1299} - drastic_slowdown[CONCURRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].txt: success=100.0% client_mean=PT2.069939083S server_cpu=PT2H17M59.756333311S client_received=4000/4000 server_resps=4000 codes={200=4000} - drastic_slowdown[CONCURRENCY_LIMITER_PIN_UNTIL_ERROR].txt: success=100.0% client_mean=PT2.053277999S server_cpu=PT2H16M53.111999959S client_received=4000/4000 server_resps=4000 codes={200=4000} - drastic_slowdown[CONCURRENCY_LIMITER_ROUND_ROBIN].txt: success=100.0% client_mean=PT2.069939083S server_cpu=PT2H17M59.756333311S client_received=4000/4000 server_resps=4000 codes={200=4000} - drastic_slowdown[UNLIMITED_ROUND_ROBIN].txt: success=100.0% client_mean=PT8.353421749S server_cpu=PT9H16M53.686999978S client_received=4000/4000 server_resps=4000 codes={200=4000} - fast_500s_then_revert[CONCURRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].txt: success=98.1% client_mean=PT0.073223422S server_cpu=PT4M34.587833291S client_received=3750/3750 server_resps=3750 codes={200=3679, 500=71} - fast_500s_then_revert[CONCURRENCY_LIMITER_PIN_UNTIL_ERROR].txt: success=99.7% client_mean=PT0.080628266S server_cpu=PT5M2.355999997S client_received=3750/3750 server_resps=3750 codes={200=3739, 500=11} - fast_500s_then_revert[CONCURRENCY_LIMITER_ROUND_ROBIN].txt: success=76.7% client_mean=PT0.055463644S server_cpu=PT3M27.988666346S client_received=3750/3750 server_resps=3750 codes={200=2876, 500=874} - fast_500s_then_revert[UNLIMITED_ROUND_ROBIN].txt: success=76.7% client_mean=PT0.055463644S server_cpu=PT3M27.988666346S client_received=3750/3750 server_resps=3750 codes={200=2876, 500=874} - live_reloading[CONCURRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].txt: success=82.4% client_mean=PT4.5664688S server_cpu=PT1H52M14.26S client_received=2500/2500 server_resps=2500 codes={200=2061, 500=439} - live_reloading[CONCURRENCY_LIMITER_PIN_UNTIL_ERROR].txt: success=59.0% client_mean=PT3.5693656S server_cpu=PT1H58M42.9S client_received=2500/2500 server_resps=2500 codes={200=1476, 500=1024} - live_reloading[CONCURRENCY_LIMITER_ROUND_ROBIN].txt: success=58.6% client_mean=PT3.5376608S server_cpu=PT1H58M19S client_received=2500/2500 server_resps=2500 codes={200=1466, 500=1034} - live_reloading[UNLIMITED_ROUND_ROBIN].txt: success=58.4% client_mean=PT2.8396S server_cpu=PT1H58M19S client_received=2500/2500 server_resps=2500 codes={200=1461, 500=1039} -one_endpoint_dies_on_each_server[CONCURRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].txt: success=65.8% client_mean=PT6.0343024S server_cpu=PT25M client_received=2500/2500 server_resps=2500 codes={200=1645, 500=855} -one_endpoint_dies_on_each_server[CONCURRENCY_LIMITER_PIN_UNTIL_ERROR].txt: success=63.8% client_mean=PT0.6S server_cpu=PT25M client_received=2500/2500 server_resps=2500 codes={200=1594, 500=906} - one_endpoint_dies_on_each_server[CONCURRENCY_LIMITER_ROUND_ROBIN].txt: success=65.5% client_mean=PT0.6S server_cpu=PT25M client_received=2500/2500 server_resps=2500 codes={200=1638, 500=862} - one_endpoint_dies_on_each_server[UNLIMITED_ROUND_ROBIN].txt: success=65.5% client_mean=PT0.6S server_cpu=PT25M client_received=2500/2500 server_resps=2500 codes={200=1638, 500=862} - simplest_possible_case[CONCURRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].txt: success=100.0% client_mean=PT0.799939393S server_cpu=PT2H55M59.2S client_received=13200/13200 server_resps=13200 codes={200=13200} - simplest_possible_case[CONCURRENCY_LIMITER_PIN_UNTIL_ERROR].txt: success=100.0% client_mean=PT0.998696969S server_cpu=PT3H39M42.8S client_received=13200/13200 server_resps=13200 codes={200=13200} - simplest_possible_case[CONCURRENCY_LIMITER_ROUND_ROBIN].txt: success=100.0% client_mean=PT0.799939393S server_cpu=PT2H55M59.2S client_received=13200/13200 server_resps=13200 codes={200=13200} - simplest_possible_case[UNLIMITED_ROUND_ROBIN].txt: success=100.0% client_mean=PT0.799939393S server_cpu=PT2H55M59.2S client_received=13200/13200 server_resps=13200 codes={200=13200} - slow_503s_then_revert[CONCURRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].txt: success=100.0% client_mean=PT0.294783777S server_cpu=PT14M44.351333306S client_received=3000/3000 server_resps=3175 codes={200=3000} - slow_503s_then_revert[CONCURRENCY_LIMITER_PIN_UNTIL_ERROR].txt: success=100.0% client_mean=PT0.337337888S server_cpu=PT16M52.013666631S client_received=3000/3000 server_resps=3197 codes={200=3000} - slow_503s_then_revert[CONCURRENCY_LIMITER_ROUND_ROBIN].txt: success=100.0% client_mean=PT0.736112444S server_cpu=PT36M48.33733331S client_received=3000/3000 server_resps=3416 codes={200=3000} - slow_503s_then_revert[UNLIMITED_ROUND_ROBIN].txt: success=100.0% client_mean=PT1.410522888S server_cpu=PT1H10M31.568666642S client_received=3000/3000 server_resps=3810 codes={200=3000} -slowdown_and_error_thresholds[CONCURRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].txt: success=22.4% client_mean=PT1M35.145090132S server_cpu=PT10H19M23.903333214S client_received=10000/10000 server_resps=10000 codes={200=2235, 500=7765} -slowdown_and_error_thresholds[CONCURRENCY_LIMITER_PIN_UNTIL_ERROR].txt: success=1.4% client_mean=PT20.017261999S server_cpu=PT10H35M25.600666645S client_received=10000/10000 server_resps=10000 codes={200=142, 500=9858} - slowdown_and_error_thresholds[CONCURRENCY_LIMITER_ROUND_ROBIN].txt: success=1.2% client_mean=PT16.859225466S server_cpu=PT10H30M49.207333306S client_received=10000/10000 server_resps=10000 codes={200=120, 500=9880} - slowdown_and_error_thresholds[UNLIMITED_ROUND_ROBIN].txt: success=1.2% client_mean=PT3.974119999S server_cpu=PT11H2M21.19999998S client_received=10000/10000 server_resps=10000 codes={200=120, 500=9880} - uncommon_flakes[CONCURRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].txt: success=97.9% client_mean=PT0.098022879S server_cpu=PT0.010031S client_received=9893/10000 server_resps=9893 codes={200=9794, 500=99} - uncommon_flakes[CONCURRENCY_LIMITER_PIN_UNTIL_ERROR].txt: success=99.0% client_mean=PT0.000001S server_cpu=PT0.01S client_received=10000/10000 server_resps=10000 codes={200=9902, 500=98} - uncommon_flakes[CONCURRENCY_LIMITER_ROUND_ROBIN].txt: success=99.0% client_mean=PT0.000001S server_cpu=PT0.01S client_received=10000/10000 server_resps=10000 codes={200=9900, 500=100} - uncommon_flakes[UNLIMITED_ROUND_ROBIN].txt: success=99.0% client_mean=PT0.000001S server_cpu=PT0.01S client_received=10000/10000 server_resps=10000 codes={200=9900, 500=100} + all_nodes_500[CONCURRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].txt: success=66.3% client_mean=PT1.86201S server_cpu=PT19M41.7S client_received=2000/2000 server_resps=2000 codes={200=1326, 500=674} + all_nodes_500[CONCURRENCY_LIMITER_PIN_UNTIL_ERROR].txt: success=50.0% client_mean=PT0.59085S server_cpu=PT19M41.7S client_received=2000/2000 server_resps=2000 codes={200=1000, 500=1000} + all_nodes_500[CONCURRENCY_LIMITER_ROUND_ROBIN].txt: success=50.0% client_mean=PT0.59085S server_cpu=PT19M41.7S client_received=2000/2000 server_resps=2000 codes={200=1000, 500=1000} + all_nodes_500[UNLIMITED_ROUND_ROBIN].txt: success=50.0% client_mean=PT0.59085S server_cpu=PT19M41.7S client_received=2000/2000 server_resps=2000 codes={200=1000, 500=1000} + black_hole[CONCURRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].txt: success=89.9% client_mean=PT0.57988592S server_cpu=PT17M21.9S client_received=1797/2000 server_resps=1797 codes={200=1797} + black_hole[CONCURRENCY_LIMITER_PIN_UNTIL_ERROR].txt: success=88.7% client_mean=PT0.579717992S server_cpu=PT17M7.5S client_received=1773/2000 server_resps=1773 codes={200=1773} + black_hole[CONCURRENCY_LIMITER_ROUND_ROBIN].txt: success=89.9% client_mean=PT0.57988592S server_cpu=PT17M21.9S client_received=1797/2000 server_resps=1797 codes={200=1797} + black_hole[UNLIMITED_ROUND_ROBIN].txt: success=65.0% client_mean=PT0.586343341S server_cpu=PT12M41.66S client_received=1299/2000 server_resps=1299 codes={200=1299} + drastic_slowdown[CONCURRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].txt: success=100.0% client_mean=PT0.85580125S server_cpu=PT57M3.205S client_received=4000/4000 server_resps=4000 codes={200=4000} + drastic_slowdown[CONCURRENCY_LIMITER_PIN_UNTIL_ERROR].txt: success=100.0% client_mean=PT0.86453125S server_cpu=PT57M38.125S client_received=4000/4000 server_resps=4000 codes={200=4000} + drastic_slowdown[CONCURRENCY_LIMITER_ROUND_ROBIN].txt: success=100.0% client_mean=PT0.85580125S server_cpu=PT57M3.205S client_received=4000/4000 server_resps=4000 codes={200=4000} + drastic_slowdown[UNLIMITED_ROUND_ROBIN].txt: success=100.0% client_mean=PT2.44372125S server_cpu=PT2H42M54.885S client_received=4000/4000 server_resps=4000 codes={200=4000} + fast_500s_then_revert[CONCURRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].txt: success=98.1% client_mean=PT0.071725866S server_cpu=PT4M28.972S client_received=3750/3750 server_resps=3750 codes={200=3679, 500=71} + fast_500s_then_revert[CONCURRENCY_LIMITER_PIN_UNTIL_ERROR].txt: success=99.7% client_mean=PT0.079397333S server_cpu=PT4M57.74S client_received=3750/3750 server_resps=3750 codes={200=3739, 500=11} + fast_500s_then_revert[CONCURRENCY_LIMITER_ROUND_ROBIN].txt: success=76.7% client_mean=PT0.053707733S server_cpu=PT3M21.404S client_received=3750/3750 server_resps=3750 codes={200=2876, 500=874} + fast_500s_then_revert[UNLIMITED_ROUND_ROBIN].txt: success=76.7% client_mean=PT0.053707733S server_cpu=PT3M21.404S client_received=3750/3750 server_resps=3750 codes={200=2876, 500=874} + live_reloading[CONCURRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].txt: success=84.9% client_mean=PT3.2184032S server_cpu=PT1H7M31.656S client_received=2500/2500 server_resps=2500 codes={200=2123, 500=377} + live_reloading[CONCURRENCY_LIMITER_PIN_UNTIL_ERROR].txt: success=59.0% client_mean=PT2.93804S server_cpu=PT1H34M6.156S client_received=2500/2500 server_resps=2500 codes={200=1474, 500=1026} + live_reloading[CONCURRENCY_LIMITER_ROUND_ROBIN].txt: success=58.6% client_mean=PT2.9248512S server_cpu=PT1H33M32.58S client_received=2500/2500 server_resps=2500 codes={200=1465, 500=1035} + live_reloading[UNLIMITED_ROUND_ROBIN].txt: success=58.4% client_mean=PT2.3889216S server_cpu=PT1H39M32.304S client_received=2500/2500 server_resps=2500 codes={200=1461, 500=1039} +one_endpoint_dies_on_each_server[CONCURRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].txt: success=42.6% client_mean=PT1.31733431S server_cpu=PT13M22.32S client_received=1364/2500 server_resps=1364 codes={200=1066, 500=298} +one_endpoint_dies_on_each_server[CONCURRENCY_LIMITER_PIN_UNTIL_ERROR].txt: success=63.8% client_mean=PT0.58188S server_cpu=PT24M14.7S client_received=2500/2500 server_resps=2500 codes={200=1594, 500=906} + one_endpoint_dies_on_each_server[CONCURRENCY_LIMITER_ROUND_ROBIN].txt: success=65.5% client_mean=PT0.58188S server_cpu=PT24M14.7S client_received=2500/2500 server_resps=2500 codes={200=1638, 500=862} + one_endpoint_dies_on_each_server[UNLIMITED_ROUND_ROBIN].txt: success=65.5% client_mean=PT0.58188S server_cpu=PT24M14.7S client_received=2500/2500 server_resps=2500 codes={200=1638, 500=862} + simplest_possible_case[CONCURRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].txt: success=100.0% client_mean=PT0.757203849S server_cpu=PT2H46M35.09080914S client_received=13200/13200 server_resps=13200 codes={200=13200} + simplest_possible_case[CONCURRENCY_LIMITER_PIN_UNTIL_ERROR].txt: success=100.0% client_mean=PT0.997975196S server_cpu=PT3H39M33.27259554S client_received=13200/13200 server_resps=13200 codes={200=13200} + simplest_possible_case[CONCURRENCY_LIMITER_ROUND_ROBIN].txt: success=100.0% client_mean=PT0.757203849S server_cpu=PT2H46M35.09080914S client_received=13200/13200 server_resps=13200 codes={200=13200} + simplest_possible_case[UNLIMITED_ROUND_ROBIN].txt: success=100.0% client_mean=PT0.757203849S server_cpu=PT2H46M35.09080914S client_received=13200/13200 server_resps=13200 codes={200=13200} + slow_503s_then_revert[CONCURRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].txt: success=100.0% client_mean=PT0.292731666S server_cpu=PT14M38.195S client_received=3000/3000 server_resps=3175 codes={200=3000} + slow_503s_then_revert[CONCURRENCY_LIMITER_PIN_UNTIL_ERROR].txt: success=100.0% client_mean=PT0.335935S server_cpu=PT16M47.805S client_received=3000/3000 server_resps=3197 codes={200=3000} + slow_503s_then_revert[CONCURRENCY_LIMITER_ROUND_ROBIN].txt: success=100.0% client_mean=PT0.733185S server_cpu=PT36M39.555S client_received=3000/3000 server_resps=3416 codes={200=3000} + slow_503s_then_revert[UNLIMITED_ROUND_ROBIN].txt: success=100.0% client_mean=PT1.406521666S server_cpu=PT1H10M19.565S client_received=3000/3000 server_resps=3810 codes={200=3000} +slowdown_and_error_thresholds[CONCURRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].txt: success=12.3% client_mean=PT9.156086414S server_cpu=PT1H22M10.65S client_received=2407/10000 server_resps=2407 codes={200=1227, 500=1180} +slowdown_and_error_thresholds[CONCURRENCY_LIMITER_PIN_UNTIL_ERROR].txt: success=1.6% client_mean=PT7.337106S server_cpu=PT3H41M38.344S client_received=10000/10000 server_resps=10000 codes={200=163, 500=9837} + slowdown_and_error_thresholds[CONCURRENCY_LIMITER_ROUND_ROBIN].txt: success=1.2% client_mean=PT7.3037772S server_cpu=PT3H44M17.984S client_received=10000/10000 server_resps=10000 codes={200=120, 500=9880} + slowdown_and_error_thresholds[UNLIMITED_ROUND_ROBIN].txt: success=1.2% client_mean=PT3.5573984S server_cpu=PT9H52M53.984S client_received=10000/10000 server_resps=10000 codes={200=120, 500=9880} + uncommon_flakes[CONCURRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].txt: success=97.9% client_mean=PT0.09807059S server_cpu=PT0S client_received=9888/10000 server_resps=9888 codes={200=9789, 500=99} + uncommon_flakes[CONCURRENCY_LIMITER_PIN_UNTIL_ERROR].txt: success=99.0% client_mean=PT0S server_cpu=PT0S client_received=10000/10000 server_resps=10000 codes={200=9902, 500=98} + uncommon_flakes[CONCURRENCY_LIMITER_ROUND_ROBIN].txt: success=99.0% client_mean=PT0S server_cpu=PT0S client_received=10000/10000 server_resps=10000 codes={200=9900, 500=100} + uncommon_flakes[UNLIMITED_ROUND_ROBIN].txt: success=99.0% client_mean=PT0S server_cpu=PT0S client_received=10000/10000 server_resps=10000 codes={200=9900, 500=100} ``` diff --git a/simulation/src/test/resources/simplest_possible_case[CONCURRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].png b/simulation/src/test/resources/simplest_possible_case[CONCURRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].png index d1263d50f..7d697adfe 100644 --- a/simulation/src/test/resources/simplest_possible_case[CONCURRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].png +++ b/simulation/src/test/resources/simplest_possible_case[CONCURRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].png @@ -1,3 +1,3 @@ version https://git-lfs.github.com/spec/v1 -oid sha256:600df4a83fbc71d93fa1a2d8666c305f31abb22ac43176c4fe52370af2406dbe -size 291857 +oid sha256:93bba21a142c67b4d969d3b29f4cf8560f07499f7427229e47ef79ad6775e4d3 +size 190694 diff --git a/simulation/src/test/resources/simplest_possible_case[CONCURRENCY_LIMITER_PIN_UNTIL_ERROR].png b/simulation/src/test/resources/simplest_possible_case[CONCURRENCY_LIMITER_PIN_UNTIL_ERROR].png index 461c27b6b..d7080b525 100644 --- a/simulation/src/test/resources/simplest_possible_case[CONCURRENCY_LIMITER_PIN_UNTIL_ERROR].png +++ b/simulation/src/test/resources/simplest_possible_case[CONCURRENCY_LIMITER_PIN_UNTIL_ERROR].png @@ -1,3 +1,3 @@ version https://git-lfs.github.com/spec/v1 -oid sha256:51f1f618292cdbe2391701fcbd9a0bd1489376108e45f23606af0aeda178c31f -size 101182 +oid sha256:2d2c9396115c7f175d975ffc22ff3560a6be16616901b39a63a55f566c2430eb +size 84234 diff --git a/simulation/src/test/resources/simplest_possible_case[CONCURRENCY_LIMITER_ROUND_ROBIN].png b/simulation/src/test/resources/simplest_possible_case[CONCURRENCY_LIMITER_ROUND_ROBIN].png index 843cd7204..e7afae119 100644 --- a/simulation/src/test/resources/simplest_possible_case[CONCURRENCY_LIMITER_ROUND_ROBIN].png +++ b/simulation/src/test/resources/simplest_possible_case[CONCURRENCY_LIMITER_ROUND_ROBIN].png @@ -1,3 +1,3 @@ version https://git-lfs.github.com/spec/v1 -oid sha256:9c536e45a198a97fbf544fd0e743a930720cb619f1ba286bd6bcba746fd18a04 -size 291828 +oid sha256:7ec16b22a0d1f5c30d56663412e4e67f21369669cc8018c30129828fea7eb935 +size 190552 diff --git a/simulation/src/test/resources/simplest_possible_case[UNLIMITED_ROUND_ROBIN].png b/simulation/src/test/resources/simplest_possible_case[UNLIMITED_ROUND_ROBIN].png index 2c5bf661c..3ae5cc324 100644 --- a/simulation/src/test/resources/simplest_possible_case[UNLIMITED_ROUND_ROBIN].png +++ b/simulation/src/test/resources/simplest_possible_case[UNLIMITED_ROUND_ROBIN].png @@ -1,3 +1,3 @@ version https://git-lfs.github.com/spec/v1 -oid sha256:f2764ed93da8990aa01026425643baf92a82c71d9d39633b59497772efe4527b -size 290867 +oid sha256:e25182694fbbd715cb075798ecc59165901ffe77202b789ac6c787e32385ea79 +size 190371 diff --git a/simulation/src/test/resources/slow_503s_then_revert[CONCURRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].png b/simulation/src/test/resources/slow_503s_then_revert[CONCURRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].png index 0dc47c0a7..8c3f2dcd3 100644 --- a/simulation/src/test/resources/slow_503s_then_revert[CONCURRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].png +++ b/simulation/src/test/resources/slow_503s_then_revert[CONCURRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].png @@ -1,3 +1,3 @@ version https://git-lfs.github.com/spec/v1 -oid sha256:7d99ddb30447022d71c69b382fd88b0405690c7dbb053ed3c182c0e21538ddcb -size 102542 +oid sha256:0358d9610ac45107c623f5d356b24e24dc5001f3e3f1e0d78bd03dfd2e2b2159 +size 94772 diff --git a/simulation/src/test/resources/slow_503s_then_revert[CONCURRENCY_LIMITER_PIN_UNTIL_ERROR].png b/simulation/src/test/resources/slow_503s_then_revert[CONCURRENCY_LIMITER_PIN_UNTIL_ERROR].png index c7902eab4..a87c52109 100644 --- a/simulation/src/test/resources/slow_503s_then_revert[CONCURRENCY_LIMITER_PIN_UNTIL_ERROR].png +++ b/simulation/src/test/resources/slow_503s_then_revert[CONCURRENCY_LIMITER_PIN_UNTIL_ERROR].png @@ -1,3 +1,3 @@ version https://git-lfs.github.com/spec/v1 -oid sha256:911e1b5b129bc0e83774a5d5f091ce10b556cbd9954f6503db344ea5fc56c474 -size 96626 +oid sha256:03f57dcf8f19648356fce21a41abeb74edb975fa4e657385ea4a41fd1d0424cb +size 88133 diff --git a/simulation/src/test/resources/slow_503s_then_revert[CONCURRENCY_LIMITER_ROUND_ROBIN].png b/simulation/src/test/resources/slow_503s_then_revert[CONCURRENCY_LIMITER_ROUND_ROBIN].png index 5fd891b08..dacb102ac 100644 --- a/simulation/src/test/resources/slow_503s_then_revert[CONCURRENCY_LIMITER_ROUND_ROBIN].png +++ b/simulation/src/test/resources/slow_503s_then_revert[CONCURRENCY_LIMITER_ROUND_ROBIN].png @@ -1,3 +1,3 @@ version https://git-lfs.github.com/spec/v1 -oid sha256:2e6f5ab3a3d2ba9126bf98063ab4f1182e262e10e56d48bd6dfc7736c30e901a -size 100596 +oid sha256:f53c5034b2ecc1ac76a7832e620a6b4c56a77bb8d0063494ff8c6afa8195f362 +size 90655 diff --git a/simulation/src/test/resources/slow_503s_then_revert[UNLIMITED_ROUND_ROBIN].png b/simulation/src/test/resources/slow_503s_then_revert[UNLIMITED_ROUND_ROBIN].png index 4cac8cedb..b6c67ad6d 100644 --- a/simulation/src/test/resources/slow_503s_then_revert[UNLIMITED_ROUND_ROBIN].png +++ b/simulation/src/test/resources/slow_503s_then_revert[UNLIMITED_ROUND_ROBIN].png @@ -1,3 +1,3 @@ version https://git-lfs.github.com/spec/v1 -oid sha256:58e1aedbc6bb3967eeff2ec72fbb4e795a3181dee672221fbc1edf3adce574e2 -size 97196 +oid sha256:b3b436c8664265dbb3ec2ef30ffa5dd2962163c262e4878de14518a0c5d5bcdf +size 88573 diff --git a/simulation/src/test/resources/slowdown_and_error_thresholds[CONCURRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].png b/simulation/src/test/resources/slowdown_and_error_thresholds[CONCURRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].png index 5c613797e..b7bbdf787 100644 --- a/simulation/src/test/resources/slowdown_and_error_thresholds[CONCURRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].png +++ b/simulation/src/test/resources/slowdown_and_error_thresholds[CONCURRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].png @@ -1,3 +1,3 @@ version https://git-lfs.github.com/spec/v1 -oid sha256:e73bc142c2cae69dd1445e2ce886f8ad5affca6fe46b49403c5c914eebc599eb -size 166539 +oid sha256:196b3d4b6ffa86ccdd5f5bdeef09601678091862b2622538d4994ab29c557bde +size 119480 diff --git a/simulation/src/test/resources/slowdown_and_error_thresholds[CONCURRENCY_LIMITER_PIN_UNTIL_ERROR].png b/simulation/src/test/resources/slowdown_and_error_thresholds[CONCURRENCY_LIMITER_PIN_UNTIL_ERROR].png index 0d10662f4..e1e1831c8 100644 --- a/simulation/src/test/resources/slowdown_and_error_thresholds[CONCURRENCY_LIMITER_PIN_UNTIL_ERROR].png +++ b/simulation/src/test/resources/slowdown_and_error_thresholds[CONCURRENCY_LIMITER_PIN_UNTIL_ERROR].png @@ -1,3 +1,3 @@ version https://git-lfs.github.com/spec/v1 -oid sha256:d3008d3a0e58b253a2444571a86ecb01d01ff794e1131d0f7528733c3f0be491 -size 193671 +oid sha256:50fd3f744979e7fd90402bf4613ee71b6cc8bc61f472c6b41e90e9c5a99c0399 +size 109522 diff --git a/simulation/src/test/resources/slowdown_and_error_thresholds[CONCURRENCY_LIMITER_ROUND_ROBIN].png b/simulation/src/test/resources/slowdown_and_error_thresholds[CONCURRENCY_LIMITER_ROUND_ROBIN].png index 7db97a1b4..41b7cccc5 100644 --- a/simulation/src/test/resources/slowdown_and_error_thresholds[CONCURRENCY_LIMITER_ROUND_ROBIN].png +++ b/simulation/src/test/resources/slowdown_and_error_thresholds[CONCURRENCY_LIMITER_ROUND_ROBIN].png @@ -1,3 +1,3 @@ version https://git-lfs.github.com/spec/v1 -oid sha256:fc483a8e6120f7f984a97192f893e6d9ad4a9920cdee8c92d6be8a048222d912 -size 112793 +oid sha256:59614e81380ed975eea9721d9fd671e0eb986000f413b3a444fc91facb69f345 +size 97437 diff --git a/simulation/src/test/resources/slowdown_and_error_thresholds[UNLIMITED_ROUND_ROBIN].png b/simulation/src/test/resources/slowdown_and_error_thresholds[UNLIMITED_ROUND_ROBIN].png index 9e6d5fbca..1d2115bde 100644 --- a/simulation/src/test/resources/slowdown_and_error_thresholds[UNLIMITED_ROUND_ROBIN].png +++ b/simulation/src/test/resources/slowdown_and_error_thresholds[UNLIMITED_ROUND_ROBIN].png @@ -1,3 +1,3 @@ version https://git-lfs.github.com/spec/v1 -oid sha256:23849150c829909864781e14e96f33a702fb77ec483453d35a09c6e43c206e0d -size 110252 +oid sha256:cc611164ef9124452fbb968cbdc4e96e9fdef17988aa52d379e6e5cbed100a50 +size 101684 diff --git a/simulation/src/test/resources/txt/all_nodes_500[CONCURRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].txt b/simulation/src/test/resources/txt/all_nodes_500[CONCURRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].txt index 84e72386c..3cb98c004 100644 --- a/simulation/src/test/resources/txt/all_nodes_500[CONCURRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].txt +++ b/simulation/src/test/resources/txt/all_nodes_500[CONCURRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].txt @@ -1 +1 @@ -success=66.3% client_mean=PT1.87116S server_cpu=PT20M client_received=2000/2000 server_resps=2000 codes={200=1326, 500=674} \ No newline at end of file +success=66.3% client_mean=PT1.86201S server_cpu=PT19M41.7S client_received=2000/2000 server_resps=2000 codes={200=1326, 500=674} \ No newline at end of file diff --git a/simulation/src/test/resources/txt/all_nodes_500[CONCURRENCY_LIMITER_PIN_UNTIL_ERROR].txt b/simulation/src/test/resources/txt/all_nodes_500[CONCURRENCY_LIMITER_PIN_UNTIL_ERROR].txt index 21cae27db..da96b9a82 100644 --- a/simulation/src/test/resources/txt/all_nodes_500[CONCURRENCY_LIMITER_PIN_UNTIL_ERROR].txt +++ b/simulation/src/test/resources/txt/all_nodes_500[CONCURRENCY_LIMITER_PIN_UNTIL_ERROR].txt @@ -1 +1 @@ -success=50.0% client_mean=PT0.6S server_cpu=PT20M client_received=2000/2000 server_resps=2000 codes={200=1000, 500=1000} \ No newline at end of file +success=50.0% client_mean=PT0.59085S server_cpu=PT19M41.7S client_received=2000/2000 server_resps=2000 codes={200=1000, 500=1000} \ No newline at end of file diff --git a/simulation/src/test/resources/txt/all_nodes_500[CONCURRENCY_LIMITER_ROUND_ROBIN].txt b/simulation/src/test/resources/txt/all_nodes_500[CONCURRENCY_LIMITER_ROUND_ROBIN].txt index 21cae27db..da96b9a82 100644 --- a/simulation/src/test/resources/txt/all_nodes_500[CONCURRENCY_LIMITER_ROUND_ROBIN].txt +++ b/simulation/src/test/resources/txt/all_nodes_500[CONCURRENCY_LIMITER_ROUND_ROBIN].txt @@ -1 +1 @@ -success=50.0% client_mean=PT0.6S server_cpu=PT20M client_received=2000/2000 server_resps=2000 codes={200=1000, 500=1000} \ No newline at end of file +success=50.0% client_mean=PT0.59085S server_cpu=PT19M41.7S client_received=2000/2000 server_resps=2000 codes={200=1000, 500=1000} \ No newline at end of file diff --git a/simulation/src/test/resources/txt/all_nodes_500[UNLIMITED_ROUND_ROBIN].txt b/simulation/src/test/resources/txt/all_nodes_500[UNLIMITED_ROUND_ROBIN].txt index 21cae27db..da96b9a82 100644 --- a/simulation/src/test/resources/txt/all_nodes_500[UNLIMITED_ROUND_ROBIN].txt +++ b/simulation/src/test/resources/txt/all_nodes_500[UNLIMITED_ROUND_ROBIN].txt @@ -1 +1 @@ -success=50.0% client_mean=PT0.6S server_cpu=PT20M client_received=2000/2000 server_resps=2000 codes={200=1000, 500=1000} \ No newline at end of file +success=50.0% client_mean=PT0.59085S server_cpu=PT19M41.7S client_received=2000/2000 server_resps=2000 codes={200=1000, 500=1000} \ No newline at end of file diff --git a/simulation/src/test/resources/txt/black_hole[CONCURRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].txt b/simulation/src/test/resources/txt/black_hole[CONCURRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].txt index d94b6183d..bc5acb9e4 100644 --- a/simulation/src/test/resources/txt/black_hole[CONCURRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].txt +++ b/simulation/src/test/resources/txt/black_hole[CONCURRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].txt @@ -1 +1 @@ -success=89.9% client_mean=PT0.600086254S server_cpu=PT17M58.2S client_received=1797/2000 server_resps=1797 codes={200=1797} \ No newline at end of file +success=89.9% client_mean=PT0.57988592S server_cpu=PT17M21.9S client_received=1797/2000 server_resps=1797 codes={200=1797} \ No newline at end of file diff --git a/simulation/src/test/resources/txt/black_hole[CONCURRENCY_LIMITER_PIN_UNTIL_ERROR].txt b/simulation/src/test/resources/txt/black_hole[CONCURRENCY_LIMITER_PIN_UNTIL_ERROR].txt index d0623d4c4..8eabf16a8 100644 --- a/simulation/src/test/resources/txt/black_hole[CONCURRENCY_LIMITER_PIN_UNTIL_ERROR].txt +++ b/simulation/src/test/resources/txt/black_hole[CONCURRENCY_LIMITER_PIN_UNTIL_ERROR].txt @@ -1 +1 @@ -success=88.7% client_mean=PT0.600191765S server_cpu=PT17M43.8S client_received=1773/2000 server_resps=1773 codes={200=1773} \ No newline at end of file +success=88.7% client_mean=PT0.579717992S server_cpu=PT17M7.5S client_received=1773/2000 server_resps=1773 codes={200=1773} \ No newline at end of file diff --git a/simulation/src/test/resources/txt/black_hole[CONCURRENCY_LIMITER_ROUND_ROBIN].txt b/simulation/src/test/resources/txt/black_hole[CONCURRENCY_LIMITER_ROUND_ROBIN].txt index d94b6183d..bc5acb9e4 100644 --- a/simulation/src/test/resources/txt/black_hole[CONCURRENCY_LIMITER_ROUND_ROBIN].txt +++ b/simulation/src/test/resources/txt/black_hole[CONCURRENCY_LIMITER_ROUND_ROBIN].txt @@ -1 +1 @@ -success=89.9% client_mean=PT0.600086254S server_cpu=PT17M58.2S client_received=1797/2000 server_resps=1797 codes={200=1797} \ No newline at end of file +success=89.9% client_mean=PT0.57988592S server_cpu=PT17M21.9S client_received=1797/2000 server_resps=1797 codes={200=1797} \ No newline at end of file diff --git a/simulation/src/test/resources/txt/black_hole[UNLIMITED_ROUND_ROBIN].txt b/simulation/src/test/resources/txt/black_hole[UNLIMITED_ROUND_ROBIN].txt index 4a2f468d0..32f0da954 100644 --- a/simulation/src/test/resources/txt/black_hole[UNLIMITED_ROUND_ROBIN].txt +++ b/simulation/src/test/resources/txt/black_hole[UNLIMITED_ROUND_ROBIN].txt @@ -1 +1 @@ -success=65.0% client_mean=PT0.6S server_cpu=PT12M59.4S client_received=1299/2000 server_resps=1299 codes={200=1299} \ No newline at end of file +success=65.0% client_mean=PT0.586343341S server_cpu=PT12M41.66S client_received=1299/2000 server_resps=1299 codes={200=1299} \ No newline at end of file diff --git a/simulation/src/test/resources/txt/drastic_slowdown[CONCURRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].txt b/simulation/src/test/resources/txt/drastic_slowdown[CONCURRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].txt index 6812e4deb..8c1ea1ae6 100644 --- a/simulation/src/test/resources/txt/drastic_slowdown[CONCURRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].txt +++ b/simulation/src/test/resources/txt/drastic_slowdown[CONCURRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].txt @@ -1 +1 @@ -success=100.0% client_mean=PT2.069939083S server_cpu=PT2H17M59.756333311S client_received=4000/4000 server_resps=4000 codes={200=4000} \ No newline at end of file +success=100.0% client_mean=PT0.85580125S server_cpu=PT57M3.205S client_received=4000/4000 server_resps=4000 codes={200=4000} \ No newline at end of file diff --git a/simulation/src/test/resources/txt/drastic_slowdown[CONCURRENCY_LIMITER_PIN_UNTIL_ERROR].txt b/simulation/src/test/resources/txt/drastic_slowdown[CONCURRENCY_LIMITER_PIN_UNTIL_ERROR].txt index 59465c642..7bb25dbab 100644 --- a/simulation/src/test/resources/txt/drastic_slowdown[CONCURRENCY_LIMITER_PIN_UNTIL_ERROR].txt +++ b/simulation/src/test/resources/txt/drastic_slowdown[CONCURRENCY_LIMITER_PIN_UNTIL_ERROR].txt @@ -1 +1 @@ -success=100.0% client_mean=PT2.053277999S server_cpu=PT2H16M53.111999959S client_received=4000/4000 server_resps=4000 codes={200=4000} \ No newline at end of file +success=100.0% client_mean=PT0.86453125S server_cpu=PT57M38.125S client_received=4000/4000 server_resps=4000 codes={200=4000} \ No newline at end of file diff --git a/simulation/src/test/resources/txt/drastic_slowdown[CONCURRENCY_LIMITER_ROUND_ROBIN].txt b/simulation/src/test/resources/txt/drastic_slowdown[CONCURRENCY_LIMITER_ROUND_ROBIN].txt index 6812e4deb..8c1ea1ae6 100644 --- a/simulation/src/test/resources/txt/drastic_slowdown[CONCURRENCY_LIMITER_ROUND_ROBIN].txt +++ b/simulation/src/test/resources/txt/drastic_slowdown[CONCURRENCY_LIMITER_ROUND_ROBIN].txt @@ -1 +1 @@ -success=100.0% client_mean=PT2.069939083S server_cpu=PT2H17M59.756333311S client_received=4000/4000 server_resps=4000 codes={200=4000} \ No newline at end of file +success=100.0% client_mean=PT0.85580125S server_cpu=PT57M3.205S client_received=4000/4000 server_resps=4000 codes={200=4000} \ No newline at end of file diff --git a/simulation/src/test/resources/txt/drastic_slowdown[UNLIMITED_ROUND_ROBIN].txt b/simulation/src/test/resources/txt/drastic_slowdown[UNLIMITED_ROUND_ROBIN].txt index a129c4e00..85c13a532 100644 --- a/simulation/src/test/resources/txt/drastic_slowdown[UNLIMITED_ROUND_ROBIN].txt +++ b/simulation/src/test/resources/txt/drastic_slowdown[UNLIMITED_ROUND_ROBIN].txt @@ -1 +1 @@ -success=100.0% client_mean=PT8.353421749S server_cpu=PT9H16M53.686999978S client_received=4000/4000 server_resps=4000 codes={200=4000} \ No newline at end of file +success=100.0% client_mean=PT2.44372125S server_cpu=PT2H42M54.885S client_received=4000/4000 server_resps=4000 codes={200=4000} \ No newline at end of file diff --git a/simulation/src/test/resources/txt/fast_500s_then_revert[CONCURRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].txt b/simulation/src/test/resources/txt/fast_500s_then_revert[CONCURRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].txt index ad3460bb0..c61146e83 100644 --- a/simulation/src/test/resources/txt/fast_500s_then_revert[CONCURRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].txt +++ b/simulation/src/test/resources/txt/fast_500s_then_revert[CONCURRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].txt @@ -1 +1 @@ -success=98.1% client_mean=PT0.073223422S server_cpu=PT4M34.587833291S client_received=3750/3750 server_resps=3750 codes={200=3679, 500=71} \ No newline at end of file +success=98.1% client_mean=PT0.071725866S server_cpu=PT4M28.972S client_received=3750/3750 server_resps=3750 codes={200=3679, 500=71} \ No newline at end of file diff --git a/simulation/src/test/resources/txt/fast_500s_then_revert[CONCURRENCY_LIMITER_PIN_UNTIL_ERROR].txt b/simulation/src/test/resources/txt/fast_500s_then_revert[CONCURRENCY_LIMITER_PIN_UNTIL_ERROR].txt index 8077268c6..e34c29e1e 100644 --- a/simulation/src/test/resources/txt/fast_500s_then_revert[CONCURRENCY_LIMITER_PIN_UNTIL_ERROR].txt +++ b/simulation/src/test/resources/txt/fast_500s_then_revert[CONCURRENCY_LIMITER_PIN_UNTIL_ERROR].txt @@ -1 +1 @@ -success=99.7% client_mean=PT0.080628266S server_cpu=PT5M2.355999997S client_received=3750/3750 server_resps=3750 codes={200=3739, 500=11} \ No newline at end of file +success=99.7% client_mean=PT0.079397333S server_cpu=PT4M57.74S client_received=3750/3750 server_resps=3750 codes={200=3739, 500=11} \ No newline at end of file diff --git a/simulation/src/test/resources/txt/fast_500s_then_revert[CONCURRENCY_LIMITER_ROUND_ROBIN].txt b/simulation/src/test/resources/txt/fast_500s_then_revert[CONCURRENCY_LIMITER_ROUND_ROBIN].txt index 5f71a3396..fb5cc428a 100644 --- a/simulation/src/test/resources/txt/fast_500s_then_revert[CONCURRENCY_LIMITER_ROUND_ROBIN].txt +++ b/simulation/src/test/resources/txt/fast_500s_then_revert[CONCURRENCY_LIMITER_ROUND_ROBIN].txt @@ -1 +1 @@ -success=76.7% client_mean=PT0.055463644S server_cpu=PT3M27.988666346S client_received=3750/3750 server_resps=3750 codes={200=2876, 500=874} \ No newline at end of file +success=76.7% client_mean=PT0.053707733S server_cpu=PT3M21.404S client_received=3750/3750 server_resps=3750 codes={200=2876, 500=874} \ No newline at end of file diff --git a/simulation/src/test/resources/txt/fast_500s_then_revert[UNLIMITED_ROUND_ROBIN].txt b/simulation/src/test/resources/txt/fast_500s_then_revert[UNLIMITED_ROUND_ROBIN].txt index 5f71a3396..fb5cc428a 100644 --- a/simulation/src/test/resources/txt/fast_500s_then_revert[UNLIMITED_ROUND_ROBIN].txt +++ b/simulation/src/test/resources/txt/fast_500s_then_revert[UNLIMITED_ROUND_ROBIN].txt @@ -1 +1 @@ -success=76.7% client_mean=PT0.055463644S server_cpu=PT3M27.988666346S client_received=3750/3750 server_resps=3750 codes={200=2876, 500=874} \ No newline at end of file +success=76.7% client_mean=PT0.053707733S server_cpu=PT3M21.404S client_received=3750/3750 server_resps=3750 codes={200=2876, 500=874} \ No newline at end of file diff --git a/simulation/src/test/resources/txt/live_reloading[CONCURRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].txt b/simulation/src/test/resources/txt/live_reloading[CONCURRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].txt index da2a0d574..05b2ed2d7 100644 --- a/simulation/src/test/resources/txt/live_reloading[CONCURRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].txt +++ b/simulation/src/test/resources/txt/live_reloading[CONCURRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].txt @@ -1 +1 @@ -success=82.4% client_mean=PT4.5664688S server_cpu=PT1H52M14.26S client_received=2500/2500 server_resps=2500 codes={200=2061, 500=439} \ No newline at end of file +success=84.9% client_mean=PT3.2184032S server_cpu=PT1H7M31.656S client_received=2500/2500 server_resps=2500 codes={200=2123, 500=377} \ No newline at end of file diff --git a/simulation/src/test/resources/txt/live_reloading[CONCURRENCY_LIMITER_PIN_UNTIL_ERROR].txt b/simulation/src/test/resources/txt/live_reloading[CONCURRENCY_LIMITER_PIN_UNTIL_ERROR].txt index 85c8bb451..f0c7c9af5 100644 --- a/simulation/src/test/resources/txt/live_reloading[CONCURRENCY_LIMITER_PIN_UNTIL_ERROR].txt +++ b/simulation/src/test/resources/txt/live_reloading[CONCURRENCY_LIMITER_PIN_UNTIL_ERROR].txt @@ -1 +1 @@ -success=59.0% client_mean=PT3.5693656S server_cpu=PT1H58M42.9S client_received=2500/2500 server_resps=2500 codes={200=1476, 500=1024} \ No newline at end of file +success=59.0% client_mean=PT2.93804S server_cpu=PT1H34M6.156S client_received=2500/2500 server_resps=2500 codes={200=1474, 500=1026} \ No newline at end of file diff --git a/simulation/src/test/resources/txt/live_reloading[CONCURRENCY_LIMITER_ROUND_ROBIN].txt b/simulation/src/test/resources/txt/live_reloading[CONCURRENCY_LIMITER_ROUND_ROBIN].txt index ac29f4d53..83b99fdec 100644 --- a/simulation/src/test/resources/txt/live_reloading[CONCURRENCY_LIMITER_ROUND_ROBIN].txt +++ b/simulation/src/test/resources/txt/live_reloading[CONCURRENCY_LIMITER_ROUND_ROBIN].txt @@ -1 +1 @@ -success=58.6% client_mean=PT3.5376608S server_cpu=PT1H58M19S client_received=2500/2500 server_resps=2500 codes={200=1466, 500=1034} \ No newline at end of file +success=58.6% client_mean=PT2.9248512S server_cpu=PT1H33M32.58S client_received=2500/2500 server_resps=2500 codes={200=1465, 500=1035} \ No newline at end of file diff --git a/simulation/src/test/resources/txt/live_reloading[UNLIMITED_ROUND_ROBIN].txt b/simulation/src/test/resources/txt/live_reloading[UNLIMITED_ROUND_ROBIN].txt index e289c4621..61780b182 100644 --- a/simulation/src/test/resources/txt/live_reloading[UNLIMITED_ROUND_ROBIN].txt +++ b/simulation/src/test/resources/txt/live_reloading[UNLIMITED_ROUND_ROBIN].txt @@ -1 +1 @@ -success=58.4% client_mean=PT2.8396S server_cpu=PT1H58M19S client_received=2500/2500 server_resps=2500 codes={200=1461, 500=1039} \ No newline at end of file +success=58.4% client_mean=PT2.3889216S server_cpu=PT1H39M32.304S client_received=2500/2500 server_resps=2500 codes={200=1461, 500=1039} \ No newline at end of file diff --git a/simulation/src/test/resources/txt/one_endpoint_dies_on_each_server[CONCURRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].txt b/simulation/src/test/resources/txt/one_endpoint_dies_on_each_server[CONCURRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].txt index aed3457e7..1d1554d92 100644 --- a/simulation/src/test/resources/txt/one_endpoint_dies_on_each_server[CONCURRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].txt +++ b/simulation/src/test/resources/txt/one_endpoint_dies_on_each_server[CONCURRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].txt @@ -1 +1 @@ -success=65.8% client_mean=PT6.0343024S server_cpu=PT25M client_received=2500/2500 server_resps=2500 codes={200=1645, 500=855} \ No newline at end of file +success=42.6% client_mean=PT1.31733431S server_cpu=PT13M22.32S client_received=1364/2500 server_resps=1364 codes={200=1066, 500=298} \ No newline at end of file diff --git a/simulation/src/test/resources/txt/one_endpoint_dies_on_each_server[CONCURRENCY_LIMITER_PIN_UNTIL_ERROR].txt b/simulation/src/test/resources/txt/one_endpoint_dies_on_each_server[CONCURRENCY_LIMITER_PIN_UNTIL_ERROR].txt index a9e98a08d..b15205fa2 100644 --- a/simulation/src/test/resources/txt/one_endpoint_dies_on_each_server[CONCURRENCY_LIMITER_PIN_UNTIL_ERROR].txt +++ b/simulation/src/test/resources/txt/one_endpoint_dies_on_each_server[CONCURRENCY_LIMITER_PIN_UNTIL_ERROR].txt @@ -1 +1 @@ -success=63.8% client_mean=PT0.6S server_cpu=PT25M client_received=2500/2500 server_resps=2500 codes={200=1594, 500=906} \ No newline at end of file +success=63.8% client_mean=PT0.58188S server_cpu=PT24M14.7S client_received=2500/2500 server_resps=2500 codes={200=1594, 500=906} \ No newline at end of file diff --git a/simulation/src/test/resources/txt/one_endpoint_dies_on_each_server[CONCURRENCY_LIMITER_ROUND_ROBIN].txt b/simulation/src/test/resources/txt/one_endpoint_dies_on_each_server[CONCURRENCY_LIMITER_ROUND_ROBIN].txt index 6e102680e..06793cc99 100644 --- a/simulation/src/test/resources/txt/one_endpoint_dies_on_each_server[CONCURRENCY_LIMITER_ROUND_ROBIN].txt +++ b/simulation/src/test/resources/txt/one_endpoint_dies_on_each_server[CONCURRENCY_LIMITER_ROUND_ROBIN].txt @@ -1 +1 @@ -success=65.5% client_mean=PT0.6S server_cpu=PT25M client_received=2500/2500 server_resps=2500 codes={200=1638, 500=862} \ No newline at end of file +success=65.5% client_mean=PT0.58188S server_cpu=PT24M14.7S client_received=2500/2500 server_resps=2500 codes={200=1638, 500=862} \ No newline at end of file diff --git a/simulation/src/test/resources/txt/one_endpoint_dies_on_each_server[UNLIMITED_ROUND_ROBIN].txt b/simulation/src/test/resources/txt/one_endpoint_dies_on_each_server[UNLIMITED_ROUND_ROBIN].txt index 6e102680e..06793cc99 100644 --- a/simulation/src/test/resources/txt/one_endpoint_dies_on_each_server[UNLIMITED_ROUND_ROBIN].txt +++ b/simulation/src/test/resources/txt/one_endpoint_dies_on_each_server[UNLIMITED_ROUND_ROBIN].txt @@ -1 +1 @@ -success=65.5% client_mean=PT0.6S server_cpu=PT25M client_received=2500/2500 server_resps=2500 codes={200=1638, 500=862} \ No newline at end of file +success=65.5% client_mean=PT0.58188S server_cpu=PT24M14.7S client_received=2500/2500 server_resps=2500 codes={200=1638, 500=862} \ No newline at end of file diff --git a/simulation/src/test/resources/txt/simplest_possible_case[CONCURRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].txt b/simulation/src/test/resources/txt/simplest_possible_case[CONCURRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].txt index 68dc39802..273eb5824 100644 --- a/simulation/src/test/resources/txt/simplest_possible_case[CONCURRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].txt +++ b/simulation/src/test/resources/txt/simplest_possible_case[CONCURRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].txt @@ -1 +1 @@ -success=100.0% client_mean=PT0.799939393S server_cpu=PT2H55M59.2S client_received=13200/13200 server_resps=13200 codes={200=13200} \ No newline at end of file +success=100.0% client_mean=PT0.757203849S server_cpu=PT2H46M35.09080914S client_received=13200/13200 server_resps=13200 codes={200=13200} \ No newline at end of file diff --git a/simulation/src/test/resources/txt/simplest_possible_case[CONCURRENCY_LIMITER_PIN_UNTIL_ERROR].txt b/simulation/src/test/resources/txt/simplest_possible_case[CONCURRENCY_LIMITER_PIN_UNTIL_ERROR].txt index 363d31abd..ef0f3cc04 100644 --- a/simulation/src/test/resources/txt/simplest_possible_case[CONCURRENCY_LIMITER_PIN_UNTIL_ERROR].txt +++ b/simulation/src/test/resources/txt/simplest_possible_case[CONCURRENCY_LIMITER_PIN_UNTIL_ERROR].txt @@ -1 +1 @@ -success=100.0% client_mean=PT0.998696969S server_cpu=PT3H39M42.8S client_received=13200/13200 server_resps=13200 codes={200=13200} \ No newline at end of file +success=100.0% client_mean=PT0.997975196S server_cpu=PT3H39M33.27259554S client_received=13200/13200 server_resps=13200 codes={200=13200} \ No newline at end of file diff --git a/simulation/src/test/resources/txt/simplest_possible_case[CONCURRENCY_LIMITER_ROUND_ROBIN].txt b/simulation/src/test/resources/txt/simplest_possible_case[CONCURRENCY_LIMITER_ROUND_ROBIN].txt index 68dc39802..273eb5824 100644 --- a/simulation/src/test/resources/txt/simplest_possible_case[CONCURRENCY_LIMITER_ROUND_ROBIN].txt +++ b/simulation/src/test/resources/txt/simplest_possible_case[CONCURRENCY_LIMITER_ROUND_ROBIN].txt @@ -1 +1 @@ -success=100.0% client_mean=PT0.799939393S server_cpu=PT2H55M59.2S client_received=13200/13200 server_resps=13200 codes={200=13200} \ No newline at end of file +success=100.0% client_mean=PT0.757203849S server_cpu=PT2H46M35.09080914S client_received=13200/13200 server_resps=13200 codes={200=13200} \ No newline at end of file diff --git a/simulation/src/test/resources/txt/simplest_possible_case[UNLIMITED_ROUND_ROBIN].txt b/simulation/src/test/resources/txt/simplest_possible_case[UNLIMITED_ROUND_ROBIN].txt index 68dc39802..273eb5824 100644 --- a/simulation/src/test/resources/txt/simplest_possible_case[UNLIMITED_ROUND_ROBIN].txt +++ b/simulation/src/test/resources/txt/simplest_possible_case[UNLIMITED_ROUND_ROBIN].txt @@ -1 +1 @@ -success=100.0% client_mean=PT0.799939393S server_cpu=PT2H55M59.2S client_received=13200/13200 server_resps=13200 codes={200=13200} \ No newline at end of file +success=100.0% client_mean=PT0.757203849S server_cpu=PT2H46M35.09080914S client_received=13200/13200 server_resps=13200 codes={200=13200} \ No newline at end of file diff --git a/simulation/src/test/resources/txt/slow_503s_then_revert[CONCURRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].txt b/simulation/src/test/resources/txt/slow_503s_then_revert[CONCURRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].txt index e287a0afe..5a46d61f0 100644 --- a/simulation/src/test/resources/txt/slow_503s_then_revert[CONCURRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].txt +++ b/simulation/src/test/resources/txt/slow_503s_then_revert[CONCURRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].txt @@ -1 +1 @@ -success=100.0% client_mean=PT0.294783777S server_cpu=PT14M44.351333306S client_received=3000/3000 server_resps=3175 codes={200=3000} \ No newline at end of file +success=100.0% client_mean=PT0.292731666S server_cpu=PT14M38.195S client_received=3000/3000 server_resps=3175 codes={200=3000} \ No newline at end of file diff --git a/simulation/src/test/resources/txt/slow_503s_then_revert[CONCURRENCY_LIMITER_PIN_UNTIL_ERROR].txt b/simulation/src/test/resources/txt/slow_503s_then_revert[CONCURRENCY_LIMITER_PIN_UNTIL_ERROR].txt index af2e71f1f..d67e6acd8 100644 --- a/simulation/src/test/resources/txt/slow_503s_then_revert[CONCURRENCY_LIMITER_PIN_UNTIL_ERROR].txt +++ b/simulation/src/test/resources/txt/slow_503s_then_revert[CONCURRENCY_LIMITER_PIN_UNTIL_ERROR].txt @@ -1 +1 @@ -success=100.0% client_mean=PT0.337337888S server_cpu=PT16M52.013666631S client_received=3000/3000 server_resps=3197 codes={200=3000} \ No newline at end of file +success=100.0% client_mean=PT0.335935S server_cpu=PT16M47.805S client_received=3000/3000 server_resps=3197 codes={200=3000} \ No newline at end of file diff --git a/simulation/src/test/resources/txt/slow_503s_then_revert[CONCURRENCY_LIMITER_ROUND_ROBIN].txt b/simulation/src/test/resources/txt/slow_503s_then_revert[CONCURRENCY_LIMITER_ROUND_ROBIN].txt index 7cf1c3bf1..90db4d9b9 100644 --- a/simulation/src/test/resources/txt/slow_503s_then_revert[CONCURRENCY_LIMITER_ROUND_ROBIN].txt +++ b/simulation/src/test/resources/txt/slow_503s_then_revert[CONCURRENCY_LIMITER_ROUND_ROBIN].txt @@ -1 +1 @@ -success=100.0% client_mean=PT0.736112444S server_cpu=PT36M48.33733331S client_received=3000/3000 server_resps=3416 codes={200=3000} \ No newline at end of file +success=100.0% client_mean=PT0.733185S server_cpu=PT36M39.555S client_received=3000/3000 server_resps=3416 codes={200=3000} \ No newline at end of file diff --git a/simulation/src/test/resources/txt/slow_503s_then_revert[UNLIMITED_ROUND_ROBIN].txt b/simulation/src/test/resources/txt/slow_503s_then_revert[UNLIMITED_ROUND_ROBIN].txt index 507b078d9..51a3a6b1d 100644 --- a/simulation/src/test/resources/txt/slow_503s_then_revert[UNLIMITED_ROUND_ROBIN].txt +++ b/simulation/src/test/resources/txt/slow_503s_then_revert[UNLIMITED_ROUND_ROBIN].txt @@ -1 +1 @@ -success=100.0% client_mean=PT1.410522888S server_cpu=PT1H10M31.568666642S client_received=3000/3000 server_resps=3810 codes={200=3000} \ No newline at end of file +success=100.0% client_mean=PT1.406521666S server_cpu=PT1H10M19.565S client_received=3000/3000 server_resps=3810 codes={200=3000} \ No newline at end of file diff --git a/simulation/src/test/resources/txt/slowdown_and_error_thresholds[CONCURRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].txt b/simulation/src/test/resources/txt/slowdown_and_error_thresholds[CONCURRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].txt index 10386fd80..f2460deb4 100644 --- a/simulation/src/test/resources/txt/slowdown_and_error_thresholds[CONCURRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].txt +++ b/simulation/src/test/resources/txt/slowdown_and_error_thresholds[CONCURRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].txt @@ -1 +1 @@ -success=22.4% client_mean=PT1M35.145090132S server_cpu=PT10H19M23.903333214S client_received=10000/10000 server_resps=10000 codes={200=2235, 500=7765} \ No newline at end of file +success=12.3% client_mean=PT9.156086414S server_cpu=PT1H22M10.65S client_received=2407/10000 server_resps=2407 codes={200=1227, 500=1180} \ No newline at end of file diff --git a/simulation/src/test/resources/txt/slowdown_and_error_thresholds[CONCURRENCY_LIMITER_PIN_UNTIL_ERROR].txt b/simulation/src/test/resources/txt/slowdown_and_error_thresholds[CONCURRENCY_LIMITER_PIN_UNTIL_ERROR].txt index eff2ada05..b6bbd457e 100644 --- a/simulation/src/test/resources/txt/slowdown_and_error_thresholds[CONCURRENCY_LIMITER_PIN_UNTIL_ERROR].txt +++ b/simulation/src/test/resources/txt/slowdown_and_error_thresholds[CONCURRENCY_LIMITER_PIN_UNTIL_ERROR].txt @@ -1 +1 @@ -success=1.4% client_mean=PT20.017261999S server_cpu=PT10H35M25.600666645S client_received=10000/10000 server_resps=10000 codes={200=142, 500=9858} \ No newline at end of file +success=1.6% client_mean=PT7.337106S server_cpu=PT3H41M38.344S client_received=10000/10000 server_resps=10000 codes={200=163, 500=9837} \ No newline at end of file diff --git a/simulation/src/test/resources/txt/slowdown_and_error_thresholds[CONCURRENCY_LIMITER_ROUND_ROBIN].txt b/simulation/src/test/resources/txt/slowdown_and_error_thresholds[CONCURRENCY_LIMITER_ROUND_ROBIN].txt index a7b894f83..d8b78369b 100644 --- a/simulation/src/test/resources/txt/slowdown_and_error_thresholds[CONCURRENCY_LIMITER_ROUND_ROBIN].txt +++ b/simulation/src/test/resources/txt/slowdown_and_error_thresholds[CONCURRENCY_LIMITER_ROUND_ROBIN].txt @@ -1 +1 @@ -success=1.2% client_mean=PT16.859225466S server_cpu=PT10H30M49.207333306S client_received=10000/10000 server_resps=10000 codes={200=120, 500=9880} \ No newline at end of file +success=1.2% client_mean=PT7.3037772S server_cpu=PT3H44M17.984S client_received=10000/10000 server_resps=10000 codes={200=120, 500=9880} \ No newline at end of file diff --git a/simulation/src/test/resources/txt/slowdown_and_error_thresholds[UNLIMITED_ROUND_ROBIN].txt b/simulation/src/test/resources/txt/slowdown_and_error_thresholds[UNLIMITED_ROUND_ROBIN].txt index 319ba6acb..eb1f1413c 100644 --- a/simulation/src/test/resources/txt/slowdown_and_error_thresholds[UNLIMITED_ROUND_ROBIN].txt +++ b/simulation/src/test/resources/txt/slowdown_and_error_thresholds[UNLIMITED_ROUND_ROBIN].txt @@ -1 +1 @@ -success=1.2% client_mean=PT3.974119999S server_cpu=PT11H2M21.19999998S client_received=10000/10000 server_resps=10000 codes={200=120, 500=9880} \ No newline at end of file +success=1.2% client_mean=PT3.5573984S server_cpu=PT9H52M53.984S client_received=10000/10000 server_resps=10000 codes={200=120, 500=9880} \ No newline at end of file diff --git a/simulation/src/test/resources/txt/uncommon_flakes[CONCURRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].txt b/simulation/src/test/resources/txt/uncommon_flakes[CONCURRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].txt index dd0f42ca1..72f0ed1b2 100644 --- a/simulation/src/test/resources/txt/uncommon_flakes[CONCURRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].txt +++ b/simulation/src/test/resources/txt/uncommon_flakes[CONCURRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].txt @@ -1 +1 @@ -success=97.9% client_mean=PT0.098022879S server_cpu=PT0.010031S client_received=9893/10000 server_resps=9893 codes={200=9794, 500=99} \ No newline at end of file +success=97.9% client_mean=PT0.09807059S server_cpu=PT0S client_received=9888/10000 server_resps=9888 codes={200=9789, 500=99} \ No newline at end of file diff --git a/simulation/src/test/resources/txt/uncommon_flakes[CONCURRENCY_LIMITER_PIN_UNTIL_ERROR].txt b/simulation/src/test/resources/txt/uncommon_flakes[CONCURRENCY_LIMITER_PIN_UNTIL_ERROR].txt index ff3d253c9..eff571464 100644 --- a/simulation/src/test/resources/txt/uncommon_flakes[CONCURRENCY_LIMITER_PIN_UNTIL_ERROR].txt +++ b/simulation/src/test/resources/txt/uncommon_flakes[CONCURRENCY_LIMITER_PIN_UNTIL_ERROR].txt @@ -1 +1 @@ -success=99.0% client_mean=PT0.000001S server_cpu=PT0.01S client_received=10000/10000 server_resps=10000 codes={200=9902, 500=98} \ No newline at end of file +success=99.0% client_mean=PT0S server_cpu=PT0S client_received=10000/10000 server_resps=10000 codes={200=9902, 500=98} \ No newline at end of file diff --git a/simulation/src/test/resources/txt/uncommon_flakes[CONCURRENCY_LIMITER_ROUND_ROBIN].txt b/simulation/src/test/resources/txt/uncommon_flakes[CONCURRENCY_LIMITER_ROUND_ROBIN].txt index 0da17a935..8ad9b33ff 100644 --- a/simulation/src/test/resources/txt/uncommon_flakes[CONCURRENCY_LIMITER_ROUND_ROBIN].txt +++ b/simulation/src/test/resources/txt/uncommon_flakes[CONCURRENCY_LIMITER_ROUND_ROBIN].txt @@ -1 +1 @@ -success=99.0% client_mean=PT0.000001S server_cpu=PT0.01S client_received=10000/10000 server_resps=10000 codes={200=9900, 500=100} \ No newline at end of file +success=99.0% client_mean=PT0S server_cpu=PT0S client_received=10000/10000 server_resps=10000 codes={200=9900, 500=100} \ No newline at end of file diff --git a/simulation/src/test/resources/txt/uncommon_flakes[UNLIMITED_ROUND_ROBIN].txt b/simulation/src/test/resources/txt/uncommon_flakes[UNLIMITED_ROUND_ROBIN].txt index 0da17a935..8ad9b33ff 100644 --- a/simulation/src/test/resources/txt/uncommon_flakes[UNLIMITED_ROUND_ROBIN].txt +++ b/simulation/src/test/resources/txt/uncommon_flakes[UNLIMITED_ROUND_ROBIN].txt @@ -1 +1 @@ -success=99.0% client_mean=PT0.000001S server_cpu=PT0.01S client_received=10000/10000 server_resps=10000 codes={200=9900, 500=100} \ No newline at end of file +success=99.0% client_mean=PT0S server_cpu=PT0S client_received=10000/10000 server_resps=10000 codes={200=9900, 500=100} \ No newline at end of file diff --git a/simulation/src/test/resources/uncommon_flakes[CONCURRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].png b/simulation/src/test/resources/uncommon_flakes[CONCURRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].png index 5f6e80bb6..124999ab1 100644 --- a/simulation/src/test/resources/uncommon_flakes[CONCURRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].png +++ b/simulation/src/test/resources/uncommon_flakes[CONCURRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].png @@ -1,3 +1,3 @@ version https://git-lfs.github.com/spec/v1 -oid sha256:d78ecb9511c2d78f6b061a78cff7d3b30f09939d7cc00034d970981838d94ebb -size 177565 +oid sha256:f31a10dba7edff3be88d724160002e1fe580718190fc9c147ab32111249726cc +size 177090 diff --git a/simulation/src/test/resources/uncommon_flakes[CONCURRENCY_LIMITER_PIN_UNTIL_ERROR].png b/simulation/src/test/resources/uncommon_flakes[CONCURRENCY_LIMITER_PIN_UNTIL_ERROR].png index 6429bc076..77f608733 100644 --- a/simulation/src/test/resources/uncommon_flakes[CONCURRENCY_LIMITER_PIN_UNTIL_ERROR].png +++ b/simulation/src/test/resources/uncommon_flakes[CONCURRENCY_LIMITER_PIN_UNTIL_ERROR].png @@ -1,3 +1,3 @@ version https://git-lfs.github.com/spec/v1 -oid sha256:590964246fe4179967c26695a2a113b626c108a1f8caaf0552c4c1814574e07e -size 159079 +oid sha256:d516991606a31781d0e2ce8010f0b1135c6e50275b249e81b7b8f2fce6ecb425 +size 159060 diff --git a/simulation/src/test/resources/uncommon_flakes[CONCURRENCY_LIMITER_ROUND_ROBIN].png b/simulation/src/test/resources/uncommon_flakes[CONCURRENCY_LIMITER_ROUND_ROBIN].png index 1291b8eb6..63cc86c05 100644 --- a/simulation/src/test/resources/uncommon_flakes[CONCURRENCY_LIMITER_ROUND_ROBIN].png +++ b/simulation/src/test/resources/uncommon_flakes[CONCURRENCY_LIMITER_ROUND_ROBIN].png @@ -1,3 +1,3 @@ version https://git-lfs.github.com/spec/v1 -oid sha256:7648cb73b70b7bf603fe6db654f22d25c2dc30f6fba21e2fe8b23fb4bfd3f927 -size 191296 +oid sha256:0ac7eae26070636698baf3b859e29b4c527e1707c67459c577671b77528607b7 +size 191237 diff --git a/simulation/src/test/resources/uncommon_flakes[UNLIMITED_ROUND_ROBIN].png b/simulation/src/test/resources/uncommon_flakes[UNLIMITED_ROUND_ROBIN].png index 2e95c5bcc..2a3c15dc7 100644 --- a/simulation/src/test/resources/uncommon_flakes[UNLIMITED_ROUND_ROBIN].png +++ b/simulation/src/test/resources/uncommon_flakes[UNLIMITED_ROUND_ROBIN].png @@ -1,3 +1,3 @@ version https://git-lfs.github.com/spec/v1 -oid sha256:f1a8e520c3fafa5e36fda8d8d4bf06e56746a2f1e7e065f9524e2b962b572776 -size 190800 +oid sha256:e1cc501073dccde2d69dbdd11bf0066d59f873f7c16b5f96c0390ee0c9181a59 +size 190722 From bf0bd479bbce0ecc0d49f93767603afd3dde8216 Mon Sep 17 00:00:00 2001 From: Carter Kozak Date: Tue, 25 Feb 2020 17:14:38 -0500 Subject: [PATCH 04/17] update simulations --- ...URRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].png | 4 +-- ...URRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].png | 4 +-- ...e[CONCURRENCY_LIMITER_PIN_UNTIL_ERROR].png | 4 +-- ..._hole[CONCURRENCY_LIMITER_ROUND_ROBIN].png | 4 +-- ...n[CONCURRENCY_LIMITER_PIN_UNTIL_ERROR].png | 4 +-- ...URRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].png | 4 +-- ...g[CONCURRENCY_LIMITER_PIN_UNTIL_ERROR].png | 4 +-- ...ading[CONCURRENCY_LIMITER_ROUND_ROBIN].png | 4 +-- ...URRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].png | 4 +-- ...r[CONCURRENCY_LIMITER_PIN_UNTIL_ERROR].png | 4 +-- simulation/src/test/resources/report.md | 36 +++++++++---------- ...URRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].png | 4 +-- ...t[CONCURRENCY_LIMITER_PIN_UNTIL_ERROR].png | 4 +-- ...evert[CONCURRENCY_LIMITER_ROUND_ROBIN].png | 4 +-- ...03s_then_revert[UNLIMITED_ROUND_ROBIN].png | 4 +-- ...URRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].png | 4 +-- ...s[CONCURRENCY_LIMITER_PIN_UNTIL_ERROR].png | 4 +-- ...holds[CONCURRENCY_LIMITER_ROUND_ROBIN].png | 4 +-- ...URRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].txt | 2 +- ...URRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].txt | 2 +- ...e[CONCURRENCY_LIMITER_PIN_UNTIL_ERROR].txt | 2 +- ..._hole[CONCURRENCY_LIMITER_ROUND_ROBIN].txt | 2 +- ...n[CONCURRENCY_LIMITER_PIN_UNTIL_ERROR].txt | 2 +- ...URRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].txt | 2 +- ...g[CONCURRENCY_LIMITER_PIN_UNTIL_ERROR].txt | 2 +- ...ading[CONCURRENCY_LIMITER_ROUND_ROBIN].txt | 2 +- ...URRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].txt | 2 +- ...r[CONCURRENCY_LIMITER_PIN_UNTIL_ERROR].txt | 2 +- ...URRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].txt | 2 +- ...t[CONCURRENCY_LIMITER_PIN_UNTIL_ERROR].txt | 2 +- ...evert[CONCURRENCY_LIMITER_ROUND_ROBIN].txt | 2 +- ...03s_then_revert[UNLIMITED_ROUND_ROBIN].txt | 2 +- ...URRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].txt | 2 +- ...s[CONCURRENCY_LIMITER_PIN_UNTIL_ERROR].txt | 2 +- ...holds[CONCURRENCY_LIMITER_ROUND_ROBIN].txt | 2 +- ...URRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].txt | 2 +- ...URRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].png | 4 +-- 37 files changed, 72 insertions(+), 72 deletions(-) diff --git a/simulation/src/test/resources/all_nodes_500[CONCURRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].png b/simulation/src/test/resources/all_nodes_500[CONCURRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].png index 5acb32325..f8fdc1db5 100644 --- a/simulation/src/test/resources/all_nodes_500[CONCURRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].png +++ b/simulation/src/test/resources/all_nodes_500[CONCURRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].png @@ -1,3 +1,3 @@ version https://git-lfs.github.com/spec/v1 -oid sha256:7a84d2aae6f10659e002899e1c04cff6ccba471f985b7860726055026251c7c1 -size 112141 +oid sha256:a6f2a2dbc8b54bbb053edfa34a57052a8b9cc4260f6ac819a39d6448d61c38f3 +size 166274 diff --git a/simulation/src/test/resources/black_hole[CONCURRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].png b/simulation/src/test/resources/black_hole[CONCURRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].png index 956aa48e2..6ae68db14 100644 --- a/simulation/src/test/resources/black_hole[CONCURRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].png +++ b/simulation/src/test/resources/black_hole[CONCURRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].png @@ -1,3 +1,3 @@ version https://git-lfs.github.com/spec/v1 -oid sha256:2628352ee622ebb23bc8db8e55a113e0f866eef897c0a75a35dba8e2ae188f2f -size 92380 +oid sha256:94a4a05d4ecde191b6f175e14d556951f7cb75a170b3595d83dce3a776e3fdfb +size 92638 diff --git a/simulation/src/test/resources/black_hole[CONCURRENCY_LIMITER_PIN_UNTIL_ERROR].png b/simulation/src/test/resources/black_hole[CONCURRENCY_LIMITER_PIN_UNTIL_ERROR].png index cf6def8de..90ec61f01 100644 --- a/simulation/src/test/resources/black_hole[CONCURRENCY_LIMITER_PIN_UNTIL_ERROR].png +++ b/simulation/src/test/resources/black_hole[CONCURRENCY_LIMITER_PIN_UNTIL_ERROR].png @@ -1,3 +1,3 @@ version https://git-lfs.github.com/spec/v1 -oid sha256:77df5f8afe5c6493aed12915ad7de18febe14bac80deae28e61cf7bce2f378cb -size 89878 +oid sha256:5af7613c0c4fa2a991f8c1d289f4cc593abdeb7eac2c4078728cd080caee4c15 +size 90357 diff --git a/simulation/src/test/resources/black_hole[CONCURRENCY_LIMITER_ROUND_ROBIN].png b/simulation/src/test/resources/black_hole[CONCURRENCY_LIMITER_ROUND_ROBIN].png index 417677410..e79f25474 100644 --- a/simulation/src/test/resources/black_hole[CONCURRENCY_LIMITER_ROUND_ROBIN].png +++ b/simulation/src/test/resources/black_hole[CONCURRENCY_LIMITER_ROUND_ROBIN].png @@ -1,3 +1,3 @@ version https://git-lfs.github.com/spec/v1 -oid sha256:4bf6fcbe5dd7ba92c082d81a3eb26f030aa1672cd06171139e9fea303e12ba12 -size 92167 +oid sha256:c2738827f335ef407b88334b71052f01dde4bd3478b98c66e188f905b8b5985e +size 92471 diff --git a/simulation/src/test/resources/drastic_slowdown[CONCURRENCY_LIMITER_PIN_UNTIL_ERROR].png b/simulation/src/test/resources/drastic_slowdown[CONCURRENCY_LIMITER_PIN_UNTIL_ERROR].png index 17372b788..8696021bd 100644 --- a/simulation/src/test/resources/drastic_slowdown[CONCURRENCY_LIMITER_PIN_UNTIL_ERROR].png +++ b/simulation/src/test/resources/drastic_slowdown[CONCURRENCY_LIMITER_PIN_UNTIL_ERROR].png @@ -1,3 +1,3 @@ version https://git-lfs.github.com/spec/v1 -oid sha256:0787ef84ca0da4b55a4524c4febe92f9d9485c93535246596737deeaee12a64c -size 88105 +oid sha256:96903770263f5efe494822e165ea74845dbdbe8083ca3461ccb11e2770c89e32 +size 88274 diff --git a/simulation/src/test/resources/live_reloading[CONCURRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].png b/simulation/src/test/resources/live_reloading[CONCURRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].png index 47a883da5..f3b391f04 100644 --- a/simulation/src/test/resources/live_reloading[CONCURRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].png +++ b/simulation/src/test/resources/live_reloading[CONCURRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].png @@ -1,3 +1,3 @@ version https://git-lfs.github.com/spec/v1 -oid sha256:84873a37e5ae3b240af0addf29c5bcd1c7b1bc3f2b84f562a3d100d82649c6f1 -size 108678 +oid sha256:692a74113932cacdee9f72ac5884d77bab03ade004f4c50ba33c14cbe4a6fb68 +size 120176 diff --git a/simulation/src/test/resources/live_reloading[CONCURRENCY_LIMITER_PIN_UNTIL_ERROR].png b/simulation/src/test/resources/live_reloading[CONCURRENCY_LIMITER_PIN_UNTIL_ERROR].png index d8e882b85..9b2d4c474 100644 --- a/simulation/src/test/resources/live_reloading[CONCURRENCY_LIMITER_PIN_UNTIL_ERROR].png +++ b/simulation/src/test/resources/live_reloading[CONCURRENCY_LIMITER_PIN_UNTIL_ERROR].png @@ -1,3 +1,3 @@ version https://git-lfs.github.com/spec/v1 -oid sha256:6be6d8e39bad03dce2e67c0934f49a61faa6ba017daa37ea2196d1511890640e -size 117475 +oid sha256:640e5599fd511d9d8f4718e6119496efbdb116d1d4e54099c85aa9799154aa13 +size 114692 diff --git a/simulation/src/test/resources/live_reloading[CONCURRENCY_LIMITER_ROUND_ROBIN].png b/simulation/src/test/resources/live_reloading[CONCURRENCY_LIMITER_ROUND_ROBIN].png index 46876cac5..fa2e7d8d2 100644 --- a/simulation/src/test/resources/live_reloading[CONCURRENCY_LIMITER_ROUND_ROBIN].png +++ b/simulation/src/test/resources/live_reloading[CONCURRENCY_LIMITER_ROUND_ROBIN].png @@ -1,3 +1,3 @@ version https://git-lfs.github.com/spec/v1 -oid sha256:db54f48e239d2cbb952fdd65bcaff9061f7471eee79bd216ae21f754233875dd -size 108969 +oid sha256:98ad2da05348bd1d6374e7ff56f323452652b615725cbdfc6c9d98fbd84793b4 +size 114801 diff --git a/simulation/src/test/resources/one_endpoint_dies_on_each_server[CONCURRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].png b/simulation/src/test/resources/one_endpoint_dies_on_each_server[CONCURRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].png index bf7c161ab..fb3aaf588 100644 --- a/simulation/src/test/resources/one_endpoint_dies_on_each_server[CONCURRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].png +++ b/simulation/src/test/resources/one_endpoint_dies_on_each_server[CONCURRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].png @@ -1,3 +1,3 @@ version https://git-lfs.github.com/spec/v1 -oid sha256:db1b1baf41c42a9e7ec0688c8b73f757aceb4c5312302a31cfd21c493860e185 -size 141939 +oid sha256:63294f8c6b34a486eeddec5d238ad2dec01d3fd32025f6a66cc967f170900e1c +size 146213 diff --git a/simulation/src/test/resources/one_endpoint_dies_on_each_server[CONCURRENCY_LIMITER_PIN_UNTIL_ERROR].png b/simulation/src/test/resources/one_endpoint_dies_on_each_server[CONCURRENCY_LIMITER_PIN_UNTIL_ERROR].png index b260fb194..c93f2b9d6 100644 --- a/simulation/src/test/resources/one_endpoint_dies_on_each_server[CONCURRENCY_LIMITER_PIN_UNTIL_ERROR].png +++ b/simulation/src/test/resources/one_endpoint_dies_on_each_server[CONCURRENCY_LIMITER_PIN_UNTIL_ERROR].png @@ -1,3 +1,3 @@ version https://git-lfs.github.com/spec/v1 -oid sha256:316170018d54b7fa457d65cf74522c7db0eb377c1d035e5678e1a4d71cb166d9 -size 149974 +oid sha256:efd56b9ca9e6fb59c8d73b16d1eb79ff956f3e4a0fbfa08ef4793ef3f2945417 +size 141995 diff --git a/simulation/src/test/resources/report.md b/simulation/src/test/resources/report.md index 9c034433b..0e49f28ae 100644 --- a/simulation/src/test/resources/report.md +++ b/simulation/src/test/resources/report.md @@ -1,43 +1,43 @@ # Report ``` - all_nodes_500[CONCURRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].txt: success=66.3% client_mean=PT1.86201S server_cpu=PT19M41.7S client_received=2000/2000 server_resps=2000 codes={200=1326, 500=674} + all_nodes_500[CONCURRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].txt: success=52.2% client_mean=PT1.016145S server_cpu=PT16M24.9S client_received=2000/2000 server_resps=1672 codes={200=1043, 500=629, Failed to make a request=328} all_nodes_500[CONCURRENCY_LIMITER_PIN_UNTIL_ERROR].txt: success=50.0% client_mean=PT0.59085S server_cpu=PT19M41.7S client_received=2000/2000 server_resps=2000 codes={200=1000, 500=1000} all_nodes_500[CONCURRENCY_LIMITER_ROUND_ROBIN].txt: success=50.0% client_mean=PT0.59085S server_cpu=PT19M41.7S client_received=2000/2000 server_resps=2000 codes={200=1000, 500=1000} all_nodes_500[UNLIMITED_ROUND_ROBIN].txt: success=50.0% client_mean=PT0.59085S server_cpu=PT19M41.7S client_received=2000/2000 server_resps=2000 codes={200=1000, 500=1000} - black_hole[CONCURRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].txt: success=89.9% client_mean=PT0.57988592S server_cpu=PT17M21.9S client_received=1797/2000 server_resps=1797 codes={200=1797} - black_hole[CONCURRENCY_LIMITER_PIN_UNTIL_ERROR].txt: success=88.7% client_mean=PT0.579717992S server_cpu=PT17M7.5S client_received=1773/2000 server_resps=1773 codes={200=1773} - black_hole[CONCURRENCY_LIMITER_ROUND_ROBIN].txt: success=89.9% client_mean=PT0.57988592S server_cpu=PT17M21.9S client_received=1797/2000 server_resps=1797 codes={200=1797} + black_hole[CONCURRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].txt: success=89.9% client_mean=PT0.58008069S server_cpu=PT17M21.9S client_received=1797/2000 server_resps=1797 codes={200=1797} + black_hole[CONCURRENCY_LIMITER_PIN_UNTIL_ERROR].txt: success=88.7% client_mean=PT0.581249294S server_cpu=PT17M7.5S client_received=1773/2000 server_resps=1773 codes={200=1773} + black_hole[CONCURRENCY_LIMITER_ROUND_ROBIN].txt: success=89.9% client_mean=PT0.58008069S server_cpu=PT17M21.9S client_received=1797/2000 server_resps=1797 codes={200=1797} black_hole[UNLIMITED_ROUND_ROBIN].txt: success=65.0% client_mean=PT0.586343341S server_cpu=PT12M41.66S client_received=1299/2000 server_resps=1299 codes={200=1299} drastic_slowdown[CONCURRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].txt: success=100.0% client_mean=PT0.85580125S server_cpu=PT57M3.205S client_received=4000/4000 server_resps=4000 codes={200=4000} - drastic_slowdown[CONCURRENCY_LIMITER_PIN_UNTIL_ERROR].txt: success=100.0% client_mean=PT0.86453125S server_cpu=PT57M38.125S client_received=4000/4000 server_resps=4000 codes={200=4000} + drastic_slowdown[CONCURRENCY_LIMITER_PIN_UNTIL_ERROR].txt: success=100.0% client_mean=PT0.86496875S server_cpu=PT57M38.075S client_received=4000/4000 server_resps=4000 codes={200=4000} drastic_slowdown[CONCURRENCY_LIMITER_ROUND_ROBIN].txt: success=100.0% client_mean=PT0.85580125S server_cpu=PT57M3.205S client_received=4000/4000 server_resps=4000 codes={200=4000} drastic_slowdown[UNLIMITED_ROUND_ROBIN].txt: success=100.0% client_mean=PT2.44372125S server_cpu=PT2H42M54.885S client_received=4000/4000 server_resps=4000 codes={200=4000} fast_500s_then_revert[CONCURRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].txt: success=98.1% client_mean=PT0.071725866S server_cpu=PT4M28.972S client_received=3750/3750 server_resps=3750 codes={200=3679, 500=71} fast_500s_then_revert[CONCURRENCY_LIMITER_PIN_UNTIL_ERROR].txt: success=99.7% client_mean=PT0.079397333S server_cpu=PT4M57.74S client_received=3750/3750 server_resps=3750 codes={200=3739, 500=11} fast_500s_then_revert[CONCURRENCY_LIMITER_ROUND_ROBIN].txt: success=76.7% client_mean=PT0.053707733S server_cpu=PT3M21.404S client_received=3750/3750 server_resps=3750 codes={200=2876, 500=874} fast_500s_then_revert[UNLIMITED_ROUND_ROBIN].txt: success=76.7% client_mean=PT0.053707733S server_cpu=PT3M21.404S client_received=3750/3750 server_resps=3750 codes={200=2876, 500=874} - live_reloading[CONCURRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].txt: success=84.9% client_mean=PT3.2184032S server_cpu=PT1H7M31.656S client_received=2500/2500 server_resps=2500 codes={200=2123, 500=377} - live_reloading[CONCURRENCY_LIMITER_PIN_UNTIL_ERROR].txt: success=59.0% client_mean=PT2.93804S server_cpu=PT1H34M6.156S client_received=2500/2500 server_resps=2500 codes={200=1474, 500=1026} - live_reloading[CONCURRENCY_LIMITER_ROUND_ROBIN].txt: success=58.6% client_mean=PT2.9248512S server_cpu=PT1H33M32.58S client_received=2500/2500 server_resps=2500 codes={200=1465, 500=1035} + live_reloading[CONCURRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].txt: success=59.5% client_mean=PT2.1742928S server_cpu=PT1H1M39.384S client_received=2500/2500 server_resps=1832 codes={200=1488, 500=344, Failed to make a request=668} + live_reloading[CONCURRENCY_LIMITER_PIN_UNTIL_ERROR].txt: success=53.8% client_mean=PT2.3703696S server_cpu=PT1H21M50.076S client_received=2500/2500 server_resps=2176 codes={200=1346, 500=830, Failed to make a request=324} + live_reloading[CONCURRENCY_LIMITER_ROUND_ROBIN].txt: success=52.1% client_mean=PT2.3660912S server_cpu=PT1H21M12.564S client_received=2500/2500 server_resps=2193 codes={200=1302, 500=891, Failed to make a request=307} live_reloading[UNLIMITED_ROUND_ROBIN].txt: success=58.4% client_mean=PT2.3889216S server_cpu=PT1H39M32.304S client_received=2500/2500 server_resps=2500 codes={200=1461, 500=1039} -one_endpoint_dies_on_each_server[CONCURRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].txt: success=42.6% client_mean=PT1.31733431S server_cpu=PT13M22.32S client_received=1364/2500 server_resps=1364 codes={200=1066, 500=298} -one_endpoint_dies_on_each_server[CONCURRENCY_LIMITER_PIN_UNTIL_ERROR].txt: success=63.8% client_mean=PT0.58188S server_cpu=PT24M14.7S client_received=2500/2500 server_resps=2500 codes={200=1594, 500=906} +one_endpoint_dies_on_each_server[CONCURRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].txt: success=44.1% client_mean=PT1.0998096S server_cpu=PT13M25.476S client_received=2500/2500 server_resps=1426 codes={200=1103, 500=323, Failed to make a request=1074} +one_endpoint_dies_on_each_server[CONCURRENCY_LIMITER_PIN_UNTIL_ERROR].txt: success=63.8% client_mean=PT0.5829248S server_cpu=PT24M14.688S client_received=2500/2500 server_resps=2500 codes={200=1595, 500=905} one_endpoint_dies_on_each_server[CONCURRENCY_LIMITER_ROUND_ROBIN].txt: success=65.5% client_mean=PT0.58188S server_cpu=PT24M14.7S client_received=2500/2500 server_resps=2500 codes={200=1638, 500=862} one_endpoint_dies_on_each_server[UNLIMITED_ROUND_ROBIN].txt: success=65.5% client_mean=PT0.58188S server_cpu=PT24M14.7S client_received=2500/2500 server_resps=2500 codes={200=1638, 500=862} simplest_possible_case[CONCURRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].txt: success=100.0% client_mean=PT0.757203849S server_cpu=PT2H46M35.09080914S client_received=13200/13200 server_resps=13200 codes={200=13200} simplest_possible_case[CONCURRENCY_LIMITER_PIN_UNTIL_ERROR].txt: success=100.0% client_mean=PT0.997975196S server_cpu=PT3H39M33.27259554S client_received=13200/13200 server_resps=13200 codes={200=13200} simplest_possible_case[CONCURRENCY_LIMITER_ROUND_ROBIN].txt: success=100.0% client_mean=PT0.757203849S server_cpu=PT2H46M35.09080914S client_received=13200/13200 server_resps=13200 codes={200=13200} simplest_possible_case[UNLIMITED_ROUND_ROBIN].txt: success=100.0% client_mean=PT0.757203849S server_cpu=PT2H46M35.09080914S client_received=13200/13200 server_resps=13200 codes={200=13200} - slow_503s_then_revert[CONCURRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].txt: success=100.0% client_mean=PT0.292731666S server_cpu=PT14M38.195S client_received=3000/3000 server_resps=3175 codes={200=3000} - slow_503s_then_revert[CONCURRENCY_LIMITER_PIN_UNTIL_ERROR].txt: success=100.0% client_mean=PT0.335935S server_cpu=PT16M47.805S client_received=3000/3000 server_resps=3197 codes={200=3000} - slow_503s_then_revert[CONCURRENCY_LIMITER_ROUND_ROBIN].txt: success=100.0% client_mean=PT0.733185S server_cpu=PT36M39.555S client_received=3000/3000 server_resps=3416 codes={200=3000} - slow_503s_then_revert[UNLIMITED_ROUND_ROBIN].txt: success=100.0% client_mean=PT1.406521666S server_cpu=PT1H10M19.565S client_received=3000/3000 server_resps=3810 codes={200=3000} -slowdown_and_error_thresholds[CONCURRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].txt: success=12.3% client_mean=PT9.156086414S server_cpu=PT1H22M10.65S client_received=2407/10000 server_resps=2407 codes={200=1227, 500=1180} -slowdown_and_error_thresholds[CONCURRENCY_LIMITER_PIN_UNTIL_ERROR].txt: success=1.6% client_mean=PT7.337106S server_cpu=PT3H41M38.344S client_received=10000/10000 server_resps=10000 codes={200=163, 500=9837} - slowdown_and_error_thresholds[CONCURRENCY_LIMITER_ROUND_ROBIN].txt: success=1.2% client_mean=PT7.3037772S server_cpu=PT3H44M17.984S client_received=10000/10000 server_resps=10000 codes={200=120, 500=9880} + slow_503s_then_revert[CONCURRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].txt: success=100.0% client_mean=PT0.30077S server_cpu=PT14M38.66S client_received=3000/3000 server_resps=3175 codes={200=3000} + slow_503s_then_revert[CONCURRENCY_LIMITER_PIN_UNTIL_ERROR].txt: success=100.0% client_mean=PT0.345103333S server_cpu=PT16M47.965S client_received=3000/3000 server_resps=3197 codes={200=3000} + slow_503s_then_revert[CONCURRENCY_LIMITER_ROUND_ROBIN].txt: success=100.0% client_mean=PT0.741096666S server_cpu=PT36M14.075S client_received=3000/3000 server_resps=3411 codes={200=3000} + slow_503s_then_revert[UNLIMITED_ROUND_ROBIN].txt: success=100.0% client_mean=PT1.423741666S server_cpu=PT1H9M35.875S client_received=3000/3000 server_resps=3802 codes={200=3000} +slowdown_and_error_thresholds[CONCURRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].txt: success=3.7% client_mean=PT2.0471542S server_cpu=PT1H26M34.726S client_received=10000/10000 server_resps=1614 codes={200=370, 500=1244, Failed to make a request=8386} +slowdown_and_error_thresholds[CONCURRENCY_LIMITER_PIN_UNTIL_ERROR].txt: success=1.2% client_mean=PT2.5592244S server_cpu=PT3H42M1.968S client_received=10000/10000 server_resps=4248 codes={200=121, 500=4127, Failed to make a request=5752} + slowdown_and_error_thresholds[CONCURRENCY_LIMITER_ROUND_ROBIN].txt: success=1.2% client_mean=PT2.57855S server_cpu=PT3H43M45.466S client_received=10000/10000 server_resps=4432 codes={200=120, 500=4312, Failed to make a request=5568} slowdown_and_error_thresholds[UNLIMITED_ROUND_ROBIN].txt: success=1.2% client_mean=PT3.5573984S server_cpu=PT9H52M53.984S client_received=10000/10000 server_resps=10000 codes={200=120, 500=9880} - uncommon_flakes[CONCURRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].txt: success=97.9% client_mean=PT0.09807059S server_cpu=PT0S client_received=9888/10000 server_resps=9888 codes={200=9789, 500=99} + uncommon_flakes[CONCURRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].txt: success=94.1% client_mean=PT0.207311264S server_cpu=PT0S client_received=9712/10000 server_resps=9499 codes={200=9405, 500=94, Failed to make a request=213} uncommon_flakes[CONCURRENCY_LIMITER_PIN_UNTIL_ERROR].txt: success=99.0% client_mean=PT0S server_cpu=PT0S client_received=10000/10000 server_resps=10000 codes={200=9902, 500=98} uncommon_flakes[CONCURRENCY_LIMITER_ROUND_ROBIN].txt: success=99.0% client_mean=PT0S server_cpu=PT0S client_received=10000/10000 server_resps=10000 codes={200=9900, 500=100} uncommon_flakes[UNLIMITED_ROUND_ROBIN].txt: success=99.0% client_mean=PT0S server_cpu=PT0S client_received=10000/10000 server_resps=10000 codes={200=9900, 500=100} diff --git a/simulation/src/test/resources/slow_503s_then_revert[CONCURRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].png b/simulation/src/test/resources/slow_503s_then_revert[CONCURRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].png index 8c3f2dcd3..a451b68bb 100644 --- a/simulation/src/test/resources/slow_503s_then_revert[CONCURRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].png +++ b/simulation/src/test/resources/slow_503s_then_revert[CONCURRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].png @@ -1,3 +1,3 @@ version https://git-lfs.github.com/spec/v1 -oid sha256:0358d9610ac45107c623f5d356b24e24dc5001f3e3f1e0d78bd03dfd2e2b2159 -size 94772 +oid sha256:3636a9586208994f13967dc034ba4eaea07e08db77b6ba801b2b0de45599c61e +size 94680 diff --git a/simulation/src/test/resources/slow_503s_then_revert[CONCURRENCY_LIMITER_PIN_UNTIL_ERROR].png b/simulation/src/test/resources/slow_503s_then_revert[CONCURRENCY_LIMITER_PIN_UNTIL_ERROR].png index a87c52109..9880964ad 100644 --- a/simulation/src/test/resources/slow_503s_then_revert[CONCURRENCY_LIMITER_PIN_UNTIL_ERROR].png +++ b/simulation/src/test/resources/slow_503s_then_revert[CONCURRENCY_LIMITER_PIN_UNTIL_ERROR].png @@ -1,3 +1,3 @@ version https://git-lfs.github.com/spec/v1 -oid sha256:03f57dcf8f19648356fce21a41abeb74edb975fa4e657385ea4a41fd1d0424cb -size 88133 +oid sha256:e0b066d38f44ab189d7bea2bc37928f5928be8e1b164d580aaa530c173444ca7 +size 88773 diff --git a/simulation/src/test/resources/slow_503s_then_revert[CONCURRENCY_LIMITER_ROUND_ROBIN].png b/simulation/src/test/resources/slow_503s_then_revert[CONCURRENCY_LIMITER_ROUND_ROBIN].png index dacb102ac..8385c6071 100644 --- a/simulation/src/test/resources/slow_503s_then_revert[CONCURRENCY_LIMITER_ROUND_ROBIN].png +++ b/simulation/src/test/resources/slow_503s_then_revert[CONCURRENCY_LIMITER_ROUND_ROBIN].png @@ -1,3 +1,3 @@ version https://git-lfs.github.com/spec/v1 -oid sha256:f53c5034b2ecc1ac76a7832e620a6b4c56a77bb8d0063494ff8c6afa8195f362 -size 90655 +oid sha256:153b09df70f5be8ba932b90b23cea6da963d09f9005054470eaf8f8300d0cd58 +size 92083 diff --git a/simulation/src/test/resources/slow_503s_then_revert[UNLIMITED_ROUND_ROBIN].png b/simulation/src/test/resources/slow_503s_then_revert[UNLIMITED_ROUND_ROBIN].png index b6c67ad6d..d57e55859 100644 --- a/simulation/src/test/resources/slow_503s_then_revert[UNLIMITED_ROUND_ROBIN].png +++ b/simulation/src/test/resources/slow_503s_then_revert[UNLIMITED_ROUND_ROBIN].png @@ -1,3 +1,3 @@ version https://git-lfs.github.com/spec/v1 -oid sha256:b3b436c8664265dbb3ec2ef30ffa5dd2962163c262e4878de14518a0c5d5bcdf -size 88573 +oid sha256:31700b0f6c307ebbae6582d7ecfc857fac4cc60291a0eb3362596da3efce6e0c +size 88793 diff --git a/simulation/src/test/resources/slowdown_and_error_thresholds[CONCURRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].png b/simulation/src/test/resources/slowdown_and_error_thresholds[CONCURRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].png index b7bbdf787..771dcf649 100644 --- a/simulation/src/test/resources/slowdown_and_error_thresholds[CONCURRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].png +++ b/simulation/src/test/resources/slowdown_and_error_thresholds[CONCURRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].png @@ -1,3 +1,3 @@ version https://git-lfs.github.com/spec/v1 -oid sha256:196b3d4b6ffa86ccdd5f5bdeef09601678091862b2622538d4994ab29c557bde -size 119480 +oid sha256:ce9a0884e66131ca152ead9558d44db32b9bbae4c2f139e54ee7e48b40e2248f +size 131006 diff --git a/simulation/src/test/resources/slowdown_and_error_thresholds[CONCURRENCY_LIMITER_PIN_UNTIL_ERROR].png b/simulation/src/test/resources/slowdown_and_error_thresholds[CONCURRENCY_LIMITER_PIN_UNTIL_ERROR].png index e1e1831c8..c5ead4f4d 100644 --- a/simulation/src/test/resources/slowdown_and_error_thresholds[CONCURRENCY_LIMITER_PIN_UNTIL_ERROR].png +++ b/simulation/src/test/resources/slowdown_and_error_thresholds[CONCURRENCY_LIMITER_PIN_UNTIL_ERROR].png @@ -1,3 +1,3 @@ version https://git-lfs.github.com/spec/v1 -oid sha256:50fd3f744979e7fd90402bf4613ee71b6cc8bc61f472c6b41e90e9c5a99c0399 -size 109522 +oid sha256:833f4f817657d7a0a597fbe30d5f20066225be58644b1041f8b5751061de43c7 +size 126043 diff --git a/simulation/src/test/resources/slowdown_and_error_thresholds[CONCURRENCY_LIMITER_ROUND_ROBIN].png b/simulation/src/test/resources/slowdown_and_error_thresholds[CONCURRENCY_LIMITER_ROUND_ROBIN].png index 41b7cccc5..567ac1303 100644 --- a/simulation/src/test/resources/slowdown_and_error_thresholds[CONCURRENCY_LIMITER_ROUND_ROBIN].png +++ b/simulation/src/test/resources/slowdown_and_error_thresholds[CONCURRENCY_LIMITER_ROUND_ROBIN].png @@ -1,3 +1,3 @@ version https://git-lfs.github.com/spec/v1 -oid sha256:59614e81380ed975eea9721d9fd671e0eb986000f413b3a444fc91facb69f345 -size 97437 +oid sha256:d8b084962ed20324ab05dd9f2b4d93261e9418ea9bba240278014b968ba0c382 +size 105756 diff --git a/simulation/src/test/resources/txt/all_nodes_500[CONCURRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].txt b/simulation/src/test/resources/txt/all_nodes_500[CONCURRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].txt index 3cb98c004..ea29fd3ac 100644 --- a/simulation/src/test/resources/txt/all_nodes_500[CONCURRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].txt +++ b/simulation/src/test/resources/txt/all_nodes_500[CONCURRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].txt @@ -1 +1 @@ -success=66.3% client_mean=PT1.86201S server_cpu=PT19M41.7S client_received=2000/2000 server_resps=2000 codes={200=1326, 500=674} \ No newline at end of file +success=52.2% client_mean=PT1.016145S server_cpu=PT16M24.9S client_received=2000/2000 server_resps=1672 codes={200=1043, 500=629, Failed to make a request=328} \ No newline at end of file diff --git a/simulation/src/test/resources/txt/black_hole[CONCURRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].txt b/simulation/src/test/resources/txt/black_hole[CONCURRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].txt index bc5acb9e4..175d57d57 100644 --- a/simulation/src/test/resources/txt/black_hole[CONCURRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].txt +++ b/simulation/src/test/resources/txt/black_hole[CONCURRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].txt @@ -1 +1 @@ -success=89.9% client_mean=PT0.57988592S server_cpu=PT17M21.9S client_received=1797/2000 server_resps=1797 codes={200=1797} \ No newline at end of file +success=89.9% client_mean=PT0.58008069S server_cpu=PT17M21.9S client_received=1797/2000 server_resps=1797 codes={200=1797} \ No newline at end of file diff --git a/simulation/src/test/resources/txt/black_hole[CONCURRENCY_LIMITER_PIN_UNTIL_ERROR].txt b/simulation/src/test/resources/txt/black_hole[CONCURRENCY_LIMITER_PIN_UNTIL_ERROR].txt index 8eabf16a8..77f533c2a 100644 --- a/simulation/src/test/resources/txt/black_hole[CONCURRENCY_LIMITER_PIN_UNTIL_ERROR].txt +++ b/simulation/src/test/resources/txt/black_hole[CONCURRENCY_LIMITER_PIN_UNTIL_ERROR].txt @@ -1 +1 @@ -success=88.7% client_mean=PT0.579717992S server_cpu=PT17M7.5S client_received=1773/2000 server_resps=1773 codes={200=1773} \ No newline at end of file +success=88.7% client_mean=PT0.581249294S server_cpu=PT17M7.5S client_received=1773/2000 server_resps=1773 codes={200=1773} \ No newline at end of file diff --git a/simulation/src/test/resources/txt/black_hole[CONCURRENCY_LIMITER_ROUND_ROBIN].txt b/simulation/src/test/resources/txt/black_hole[CONCURRENCY_LIMITER_ROUND_ROBIN].txt index bc5acb9e4..175d57d57 100644 --- a/simulation/src/test/resources/txt/black_hole[CONCURRENCY_LIMITER_ROUND_ROBIN].txt +++ b/simulation/src/test/resources/txt/black_hole[CONCURRENCY_LIMITER_ROUND_ROBIN].txt @@ -1 +1 @@ -success=89.9% client_mean=PT0.57988592S server_cpu=PT17M21.9S client_received=1797/2000 server_resps=1797 codes={200=1797} \ No newline at end of file +success=89.9% client_mean=PT0.58008069S server_cpu=PT17M21.9S client_received=1797/2000 server_resps=1797 codes={200=1797} \ No newline at end of file diff --git a/simulation/src/test/resources/txt/drastic_slowdown[CONCURRENCY_LIMITER_PIN_UNTIL_ERROR].txt b/simulation/src/test/resources/txt/drastic_slowdown[CONCURRENCY_LIMITER_PIN_UNTIL_ERROR].txt index 7bb25dbab..0db33de67 100644 --- a/simulation/src/test/resources/txt/drastic_slowdown[CONCURRENCY_LIMITER_PIN_UNTIL_ERROR].txt +++ b/simulation/src/test/resources/txt/drastic_slowdown[CONCURRENCY_LIMITER_PIN_UNTIL_ERROR].txt @@ -1 +1 @@ -success=100.0% client_mean=PT0.86453125S server_cpu=PT57M38.125S client_received=4000/4000 server_resps=4000 codes={200=4000} \ No newline at end of file +success=100.0% client_mean=PT0.86496875S server_cpu=PT57M38.075S client_received=4000/4000 server_resps=4000 codes={200=4000} \ No newline at end of file diff --git a/simulation/src/test/resources/txt/live_reloading[CONCURRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].txt b/simulation/src/test/resources/txt/live_reloading[CONCURRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].txt index 05b2ed2d7..eb31bc3dd 100644 --- a/simulation/src/test/resources/txt/live_reloading[CONCURRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].txt +++ b/simulation/src/test/resources/txt/live_reloading[CONCURRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].txt @@ -1 +1 @@ -success=84.9% client_mean=PT3.2184032S server_cpu=PT1H7M31.656S client_received=2500/2500 server_resps=2500 codes={200=2123, 500=377} \ No newline at end of file +success=59.5% client_mean=PT2.1742928S server_cpu=PT1H1M39.384S client_received=2500/2500 server_resps=1832 codes={200=1488, 500=344, Failed to make a request=668} \ No newline at end of file diff --git a/simulation/src/test/resources/txt/live_reloading[CONCURRENCY_LIMITER_PIN_UNTIL_ERROR].txt b/simulation/src/test/resources/txt/live_reloading[CONCURRENCY_LIMITER_PIN_UNTIL_ERROR].txt index f0c7c9af5..4507301ce 100644 --- a/simulation/src/test/resources/txt/live_reloading[CONCURRENCY_LIMITER_PIN_UNTIL_ERROR].txt +++ b/simulation/src/test/resources/txt/live_reloading[CONCURRENCY_LIMITER_PIN_UNTIL_ERROR].txt @@ -1 +1 @@ -success=59.0% client_mean=PT2.93804S server_cpu=PT1H34M6.156S client_received=2500/2500 server_resps=2500 codes={200=1474, 500=1026} \ No newline at end of file +success=53.8% client_mean=PT2.3703696S server_cpu=PT1H21M50.076S client_received=2500/2500 server_resps=2176 codes={200=1346, 500=830, Failed to make a request=324} \ No newline at end of file diff --git a/simulation/src/test/resources/txt/live_reloading[CONCURRENCY_LIMITER_ROUND_ROBIN].txt b/simulation/src/test/resources/txt/live_reloading[CONCURRENCY_LIMITER_ROUND_ROBIN].txt index 83b99fdec..f3f236699 100644 --- a/simulation/src/test/resources/txt/live_reloading[CONCURRENCY_LIMITER_ROUND_ROBIN].txt +++ b/simulation/src/test/resources/txt/live_reloading[CONCURRENCY_LIMITER_ROUND_ROBIN].txt @@ -1 +1 @@ -success=58.6% client_mean=PT2.9248512S server_cpu=PT1H33M32.58S client_received=2500/2500 server_resps=2500 codes={200=1465, 500=1035} \ No newline at end of file +success=52.1% client_mean=PT2.3660912S server_cpu=PT1H21M12.564S client_received=2500/2500 server_resps=2193 codes={200=1302, 500=891, Failed to make a request=307} \ No newline at end of file diff --git a/simulation/src/test/resources/txt/one_endpoint_dies_on_each_server[CONCURRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].txt b/simulation/src/test/resources/txt/one_endpoint_dies_on_each_server[CONCURRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].txt index 1d1554d92..5a9f1ee59 100644 --- a/simulation/src/test/resources/txt/one_endpoint_dies_on_each_server[CONCURRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].txt +++ b/simulation/src/test/resources/txt/one_endpoint_dies_on_each_server[CONCURRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].txt @@ -1 +1 @@ -success=42.6% client_mean=PT1.31733431S server_cpu=PT13M22.32S client_received=1364/2500 server_resps=1364 codes={200=1066, 500=298} \ No newline at end of file +success=44.1% client_mean=PT1.0998096S server_cpu=PT13M25.476S client_received=2500/2500 server_resps=1426 codes={200=1103, 500=323, Failed to make a request=1074} \ No newline at end of file diff --git a/simulation/src/test/resources/txt/one_endpoint_dies_on_each_server[CONCURRENCY_LIMITER_PIN_UNTIL_ERROR].txt b/simulation/src/test/resources/txt/one_endpoint_dies_on_each_server[CONCURRENCY_LIMITER_PIN_UNTIL_ERROR].txt index b15205fa2..8853e6b16 100644 --- a/simulation/src/test/resources/txt/one_endpoint_dies_on_each_server[CONCURRENCY_LIMITER_PIN_UNTIL_ERROR].txt +++ b/simulation/src/test/resources/txt/one_endpoint_dies_on_each_server[CONCURRENCY_LIMITER_PIN_UNTIL_ERROR].txt @@ -1 +1 @@ -success=63.8% client_mean=PT0.58188S server_cpu=PT24M14.7S client_received=2500/2500 server_resps=2500 codes={200=1594, 500=906} \ No newline at end of file +success=63.8% client_mean=PT0.5829248S server_cpu=PT24M14.688S client_received=2500/2500 server_resps=2500 codes={200=1595, 500=905} \ No newline at end of file diff --git a/simulation/src/test/resources/txt/slow_503s_then_revert[CONCURRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].txt b/simulation/src/test/resources/txt/slow_503s_then_revert[CONCURRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].txt index 5a46d61f0..5ef4258a7 100644 --- a/simulation/src/test/resources/txt/slow_503s_then_revert[CONCURRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].txt +++ b/simulation/src/test/resources/txt/slow_503s_then_revert[CONCURRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].txt @@ -1 +1 @@ -success=100.0% client_mean=PT0.292731666S server_cpu=PT14M38.195S client_received=3000/3000 server_resps=3175 codes={200=3000} \ No newline at end of file +success=100.0% client_mean=PT0.30077S server_cpu=PT14M38.66S client_received=3000/3000 server_resps=3175 codes={200=3000} \ No newline at end of file diff --git a/simulation/src/test/resources/txt/slow_503s_then_revert[CONCURRENCY_LIMITER_PIN_UNTIL_ERROR].txt b/simulation/src/test/resources/txt/slow_503s_then_revert[CONCURRENCY_LIMITER_PIN_UNTIL_ERROR].txt index d67e6acd8..4ac527b5f 100644 --- a/simulation/src/test/resources/txt/slow_503s_then_revert[CONCURRENCY_LIMITER_PIN_UNTIL_ERROR].txt +++ b/simulation/src/test/resources/txt/slow_503s_then_revert[CONCURRENCY_LIMITER_PIN_UNTIL_ERROR].txt @@ -1 +1 @@ -success=100.0% client_mean=PT0.335935S server_cpu=PT16M47.805S client_received=3000/3000 server_resps=3197 codes={200=3000} \ No newline at end of file +success=100.0% client_mean=PT0.345103333S server_cpu=PT16M47.965S client_received=3000/3000 server_resps=3197 codes={200=3000} \ No newline at end of file diff --git a/simulation/src/test/resources/txt/slow_503s_then_revert[CONCURRENCY_LIMITER_ROUND_ROBIN].txt b/simulation/src/test/resources/txt/slow_503s_then_revert[CONCURRENCY_LIMITER_ROUND_ROBIN].txt index 90db4d9b9..bf995233d 100644 --- a/simulation/src/test/resources/txt/slow_503s_then_revert[CONCURRENCY_LIMITER_ROUND_ROBIN].txt +++ b/simulation/src/test/resources/txt/slow_503s_then_revert[CONCURRENCY_LIMITER_ROUND_ROBIN].txt @@ -1 +1 @@ -success=100.0% client_mean=PT0.733185S server_cpu=PT36M39.555S client_received=3000/3000 server_resps=3416 codes={200=3000} \ No newline at end of file +success=100.0% client_mean=PT0.741096666S server_cpu=PT36M14.075S client_received=3000/3000 server_resps=3411 codes={200=3000} \ No newline at end of file diff --git a/simulation/src/test/resources/txt/slow_503s_then_revert[UNLIMITED_ROUND_ROBIN].txt b/simulation/src/test/resources/txt/slow_503s_then_revert[UNLIMITED_ROUND_ROBIN].txt index 51a3a6b1d..013ac0362 100644 --- a/simulation/src/test/resources/txt/slow_503s_then_revert[UNLIMITED_ROUND_ROBIN].txt +++ b/simulation/src/test/resources/txt/slow_503s_then_revert[UNLIMITED_ROUND_ROBIN].txt @@ -1 +1 @@ -success=100.0% client_mean=PT1.406521666S server_cpu=PT1H10M19.565S client_received=3000/3000 server_resps=3810 codes={200=3000} \ No newline at end of file +success=100.0% client_mean=PT1.423741666S server_cpu=PT1H9M35.875S client_received=3000/3000 server_resps=3802 codes={200=3000} \ No newline at end of file diff --git a/simulation/src/test/resources/txt/slowdown_and_error_thresholds[CONCURRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].txt b/simulation/src/test/resources/txt/slowdown_and_error_thresholds[CONCURRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].txt index f2460deb4..5f1cea526 100644 --- a/simulation/src/test/resources/txt/slowdown_and_error_thresholds[CONCURRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].txt +++ b/simulation/src/test/resources/txt/slowdown_and_error_thresholds[CONCURRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].txt @@ -1 +1 @@ -success=12.3% client_mean=PT9.156086414S server_cpu=PT1H22M10.65S client_received=2407/10000 server_resps=2407 codes={200=1227, 500=1180} \ No newline at end of file +success=3.7% client_mean=PT2.0471542S server_cpu=PT1H26M34.726S client_received=10000/10000 server_resps=1614 codes={200=370, 500=1244, Failed to make a request=8386} \ No newline at end of file diff --git a/simulation/src/test/resources/txt/slowdown_and_error_thresholds[CONCURRENCY_LIMITER_PIN_UNTIL_ERROR].txt b/simulation/src/test/resources/txt/slowdown_and_error_thresholds[CONCURRENCY_LIMITER_PIN_UNTIL_ERROR].txt index b6bbd457e..5ab2395e2 100644 --- a/simulation/src/test/resources/txt/slowdown_and_error_thresholds[CONCURRENCY_LIMITER_PIN_UNTIL_ERROR].txt +++ b/simulation/src/test/resources/txt/slowdown_and_error_thresholds[CONCURRENCY_LIMITER_PIN_UNTIL_ERROR].txt @@ -1 +1 @@ -success=1.6% client_mean=PT7.337106S server_cpu=PT3H41M38.344S client_received=10000/10000 server_resps=10000 codes={200=163, 500=9837} \ No newline at end of file +success=1.2% client_mean=PT2.5592244S server_cpu=PT3H42M1.968S client_received=10000/10000 server_resps=4248 codes={200=121, 500=4127, Failed to make a request=5752} \ No newline at end of file diff --git a/simulation/src/test/resources/txt/slowdown_and_error_thresholds[CONCURRENCY_LIMITER_ROUND_ROBIN].txt b/simulation/src/test/resources/txt/slowdown_and_error_thresholds[CONCURRENCY_LIMITER_ROUND_ROBIN].txt index d8b78369b..6d7c95429 100644 --- a/simulation/src/test/resources/txt/slowdown_and_error_thresholds[CONCURRENCY_LIMITER_ROUND_ROBIN].txt +++ b/simulation/src/test/resources/txt/slowdown_and_error_thresholds[CONCURRENCY_LIMITER_ROUND_ROBIN].txt @@ -1 +1 @@ -success=1.2% client_mean=PT7.3037772S server_cpu=PT3H44M17.984S client_received=10000/10000 server_resps=10000 codes={200=120, 500=9880} \ No newline at end of file +success=1.2% client_mean=PT2.57855S server_cpu=PT3H43M45.466S client_received=10000/10000 server_resps=4432 codes={200=120, 500=4312, Failed to make a request=5568} \ No newline at end of file diff --git a/simulation/src/test/resources/txt/uncommon_flakes[CONCURRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].txt b/simulation/src/test/resources/txt/uncommon_flakes[CONCURRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].txt index 72f0ed1b2..c1c24eb80 100644 --- a/simulation/src/test/resources/txt/uncommon_flakes[CONCURRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].txt +++ b/simulation/src/test/resources/txt/uncommon_flakes[CONCURRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].txt @@ -1 +1 @@ -success=97.9% client_mean=PT0.09807059S server_cpu=PT0S client_received=9888/10000 server_resps=9888 codes={200=9789, 500=99} \ No newline at end of file +success=94.1% client_mean=PT0.207311264S server_cpu=PT0S client_received=9712/10000 server_resps=9499 codes={200=9405, 500=94, Failed to make a request=213} \ No newline at end of file diff --git a/simulation/src/test/resources/uncommon_flakes[CONCURRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].png b/simulation/src/test/resources/uncommon_flakes[CONCURRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].png index 124999ab1..9fdc8d489 100644 --- a/simulation/src/test/resources/uncommon_flakes[CONCURRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].png +++ b/simulation/src/test/resources/uncommon_flakes[CONCURRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].png @@ -1,3 +1,3 @@ version https://git-lfs.github.com/spec/v1 -oid sha256:f31a10dba7edff3be88d724160002e1fe580718190fc9c147ab32111249726cc -size 177090 +oid sha256:0b42d3b4831c79ba983ff4e35a2034a240914e28d53acb3fd2994804b302272e +size 181233 From 2aede1d215c07ce95ccf1093c4b01adfa185984b Mon Sep 17 00:00:00 2001 From: Carter Kozak Date: Tue, 25 Feb 2020 18:53:23 -0500 Subject: [PATCH 05/17] improved backoff logging --- .../dialogue/core/BackoffStrategy.java | 26 ++++++++++++ .../core/ExponentialBackoffStrategy.java | 40 +++++++++++++++++++ .../dialogue/core/RetryingChannel.java | 19 ++++++--- 3 files changed, 79 insertions(+), 6 deletions(-) create mode 100644 dialogue-core/src/main/java/com/palantir/dialogue/core/BackoffStrategy.java create mode 100644 dialogue-core/src/main/java/com/palantir/dialogue/core/ExponentialBackoffStrategy.java diff --git a/dialogue-core/src/main/java/com/palantir/dialogue/core/BackoffStrategy.java b/dialogue-core/src/main/java/com/palantir/dialogue/core/BackoffStrategy.java new file mode 100644 index 000000000..5800fffc7 --- /dev/null +++ b/dialogue-core/src/main/java/com/palantir/dialogue/core/BackoffStrategy.java @@ -0,0 +1,26 @@ +/* + * (c) Copyright 2020 Palantir Technologies Inc. All rights reserved. + * + * Licensed 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 com.palantir.dialogue.core; + +import java.time.Duration; + +/** Defines a strategy for waiting in between successive retries of an operation that is subject to failure. */ +interface BackoffStrategy { + + /** Returns the next suggested backoff duration. */ + Duration backoffDuration(int failures); +} diff --git a/dialogue-core/src/main/java/com/palantir/dialogue/core/ExponentialBackoffStrategy.java b/dialogue-core/src/main/java/com/palantir/dialogue/core/ExponentialBackoffStrategy.java new file mode 100644 index 000000000..71383d2c0 --- /dev/null +++ b/dialogue-core/src/main/java/com/palantir/dialogue/core/ExponentialBackoffStrategy.java @@ -0,0 +1,40 @@ +/* + * (c) Copyright 2020 Palantir Technologies Inc. All rights reserved. + * + * Licensed 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 com.palantir.dialogue.core; + +import java.time.Duration; +import java.util.function.DoubleSupplier; + +final class ExponentialBackoffStrategy implements BackoffStrategy { + + private final Duration backoffSlotSize; + private final DoubleSupplier random; + + ExponentialBackoffStrategy(Duration backoffSlotSize, DoubleSupplier random) { + this.backoffSlotSize = backoffSlotSize; + this.random = random; + } + + @Override + public Duration backoffDuration(int failures) { + if (failures == 0) { + return Duration.ZERO; + } + int upperBound = (int) Math.pow(2, failures - 1); + return Duration.ofNanos(Math.round(backoffSlotSize.toNanos() * random.getAsDouble() * upperBound)); + } +} diff --git a/dialogue-core/src/main/java/com/palantir/dialogue/core/RetryingChannel.java b/dialogue-core/src/main/java/com/palantir/dialogue/core/RetryingChannel.java index 38a8df297..965bfd7cd 100644 --- a/dialogue-core/src/main/java/com/palantir/dialogue/core/RetryingChannel.java +++ b/dialogue-core/src/main/java/com/palantir/dialogue/core/RetryingChannel.java @@ -38,6 +38,7 @@ import java.util.concurrent.TimeUnit; import java.util.function.DoubleSupplier; import java.util.function.Supplier; +import javax.annotation.Nullable; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -105,9 +106,16 @@ private RetryingCallback(Endpoint endpoint, Request request) { this.request = request; } - @SuppressWarnings("FutureReturnValueIgnored") // error-prone bug ListenableFuture execute() { + return execute(null); + } + + @SuppressWarnings("FutureReturnValueIgnored") // error-prone bug + ListenableFuture execute(@Nullable Throwable cause) { long backoffNanoseconds = getBackoffNanoseconds(); + if (failures > 0) { + logRetry(cause, backoffNanoseconds); + } if (backoffNanoseconds <= 0) { return wrap(delegate.execute(endpoint, request)); } @@ -132,8 +140,7 @@ ListenableFuture success(Response response) { Throwable failure = new SafeRuntimeException("Received retryable response", SafeArg.of("status", response.code())); if (++failures <= maxRetries) { - logRetry(failure); - return execute(); + return execute(failure); } if (log.isDebugEnabled()) { log.debug( @@ -150,18 +157,18 @@ ListenableFuture success(Response response) { ListenableFuture failure(Throwable throwable) { if (++failures <= maxRetries) { - logRetry(throwable); - return execute(); + return execute(throwable); } return Futures.immediateFailedFuture(throwable); } - private void logRetry(Throwable throwable) { + private void logRetry(@Nullable Throwable throwable, long backoffNanoseconds) { if (log.isInfoEnabled()) { log.info( "Retrying call after failure", SafeArg.of("failures", failures), SafeArg.of("maxRetries", maxRetries), + SafeArg.of("backoffNanoseconds", backoffNanoseconds), SafeArg.of("serviceName", endpoint.serviceName()), SafeArg.of("endpoint", endpoint.endpointName()), throwable); From 9d689895da13f26fab542edba234984451680be1 Mon Sep 17 00:00:00 2001 From: Carter Kozak Date: Tue, 25 Feb 2020 19:05:39 -0500 Subject: [PATCH 06/17] simplify --- .../palantir/dialogue/core/RetryingChannel.java | 15 ++++++--------- 1 file changed, 6 insertions(+), 9 deletions(-) diff --git a/dialogue-core/src/main/java/com/palantir/dialogue/core/RetryingChannel.java b/dialogue-core/src/main/java/com/palantir/dialogue/core/RetryingChannel.java index 965bfd7cd..aca6d8354 100644 --- a/dialogue-core/src/main/java/com/palantir/dialogue/core/RetryingChannel.java +++ b/dialogue-core/src/main/java/com/palantir/dialogue/core/RetryingChannel.java @@ -38,7 +38,6 @@ import java.util.concurrent.TimeUnit; import java.util.function.DoubleSupplier; import java.util.function.Supplier; -import javax.annotation.Nullable; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -107,15 +106,13 @@ private RetryingCallback(Endpoint endpoint, Request request) { } ListenableFuture execute() { - return execute(null); + return wrap(delegate.execute(endpoint, request)); } @SuppressWarnings("FutureReturnValueIgnored") // error-prone bug - ListenableFuture execute(@Nullable Throwable cause) { + ListenableFuture retry(Throwable cause) { long backoffNanoseconds = getBackoffNanoseconds(); - if (failures > 0) { - logRetry(cause, backoffNanoseconds); - } + logRetry(backoffNanoseconds, cause); if (backoffNanoseconds <= 0) { return wrap(delegate.execute(endpoint, request)); } @@ -140,7 +137,7 @@ ListenableFuture success(Response response) { Throwable failure = new SafeRuntimeException("Received retryable response", SafeArg.of("status", response.code())); if (++failures <= maxRetries) { - return execute(failure); + return retry(failure); } if (log.isDebugEnabled()) { log.debug( @@ -157,12 +154,12 @@ ListenableFuture success(Response response) { ListenableFuture failure(Throwable throwable) { if (++failures <= maxRetries) { - return execute(throwable); + return retry(throwable); } return Futures.immediateFailedFuture(throwable); } - private void logRetry(@Nullable Throwable throwable, long backoffNanoseconds) { + private void logRetry(long backoffNanoseconds, Throwable throwable) { if (log.isInfoEnabled()) { log.info( "Retrying call after failure", From e912ef3b586c52b491479835822a73246f418b62 Mon Sep 17 00:00:00 2001 From: Carter Kozak Date: Tue, 25 Feb 2020 20:49:02 -0500 Subject: [PATCH 07/17] cr --- .../dialogue/core/RetryingChannel.java | 12 ++--- .../palantir/dialogue/core/ChannelsTest.java | 2 +- .../ChannelsTest/traces_on_retries.log | 52 +++++++++---------- 3 files changed, 33 insertions(+), 33 deletions(-) diff --git a/dialogue-core/src/main/java/com/palantir/dialogue/core/RetryingChannel.java b/dialogue-core/src/main/java/com/palantir/dialogue/core/RetryingChannel.java index aca6d8354..54b16837b 100644 --- a/dialogue-core/src/main/java/com/palantir/dialogue/core/RetryingChannel.java +++ b/dialogue-core/src/main/java/com/palantir/dialogue/core/RetryingChannel.java @@ -56,9 +56,9 @@ final class RetryingChannel implements Channel { */ private static final Supplier sharedScheduler = Suppliers.memoize( () -> MoreExecutors.listeningDecorator(Tracers.wrap( - "dialogue-BlacklistingChannel-scheduler", + "dialogue-RetryingChannel-scheduler", Executors.newSingleThreadScheduledExecutor(new ThreadFactoryBuilder() - .setNameFormat("dialogue-BlacklistingChannel-scheduler-%d") + .setNameFormat("dialogue-RetryingChannel-scheduler-%d") .setDaemon(false) .build())))); @@ -67,7 +67,7 @@ final class RetryingChannel implements Channel { private final int maxRetries; private final ClientConfiguration.ServerQoS serverQoS; private final Duration backoffSlotSize; - private final DoubleSupplier random; + private final DoubleSupplier jitter; RetryingChannel( Channel delegate, int maxRetries, Duration backoffSlotSize, ClientConfiguration.ServerQoS serverQoS) { @@ -81,13 +81,13 @@ final class RetryingChannel implements Channel { Duration backoffSlotSize, ClientConfiguration.ServerQoS serverQoS, ListeningScheduledExecutorService scheduler, - DoubleSupplier random) { + DoubleSupplier jitter) { this.delegate = delegate; this.maxRetries = maxRetries; this.backoffSlotSize = backoffSlotSize; this.serverQoS = serverQoS; this.scheduler = scheduler; - this.random = random; + this.jitter = jitter; } @Override @@ -126,7 +126,7 @@ private long getBackoffNanoseconds() { return 0L; } int upperBound = (int) Math.pow(2, failures - 1); - return Math.round(backoffSlotSize.toNanos() * random.getAsDouble() * upperBound); + return Math.round(backoffSlotSize.toNanos() * jitter.getAsDouble() * upperBound); } ListenableFuture success(Response response) { diff --git a/dialogue-core/src/test/java/com/palantir/dialogue/core/ChannelsTest.java b/dialogue-core/src/test/java/com/palantir/dialogue/core/ChannelsTest.java index eb5e60563..97e711afe 100644 --- a/dialogue-core/src/test/java/com/palantir/dialogue/core/ChannelsTest.java +++ b/dialogue-core/src/test/java/com/palantir/dialogue/core/ChannelsTest.java @@ -155,7 +155,7 @@ public void traces_on_retries() throws Exception { @Test @TestTracing(snapshot = true) - public void traces_on_succes() throws Exception { + public void traces_on_success() throws Exception { when(response.code()).thenReturn(200); try (Response response = channel.execute(endpoint, request).get()) { assertThat(response.code()).isEqualTo(200); diff --git a/dialogue-core/src/test/resources/tracing/ChannelsTest/traces_on_retries.log b/dialogue-core/src/test/resources/tracing/ChannelsTest/traces_on_retries.log index c3626d590..c7ed86647 100644 --- a/dialogue-core/src/test/resources/tracing/ChannelsTest/traces_on_retries.log +++ b/dialogue-core/src/test/resources/tracing/ChannelsTest/traces_on_retries.log @@ -1,26 +1,26 @@ -{"traceId":"cc2af584ecf2a705","parentSpanId":"a87826fa8551ef00","spanId":"c4b3149ae692711c","type":"LOCAL","operation":"Dialogue-http-request initial","startTimeMicroSeconds":1582666422499681,"durationNanoSeconds":5720159,"metadata":{}} -{"traceId":"cc2af584ecf2a705","parentSpanId":"be80183dfeed0f90","spanId":"a87826fa8551ef00","type":"LOCAL","operation":"Dialogue-http-request","startTimeMicroSeconds":1582666422499663,"durationNanoSeconds":6209895,"metadata":{}} -{"traceId":"cc2af584ecf2a705","parentSpanId":"83267beb8636c288","spanId":"be80183dfeed0f90","type":"LOCAL","operation":"Dialogue-request-attempt initial","startTimeMicroSeconds":1582666422499420,"durationNanoSeconds":7916145,"metadata":{}} -{"traceId":"cc2af584ecf2a705","parentSpanId":"cc4c5dcb09dddef9","spanId":"83267beb8636c288","type":"LOCAL","operation":"Dialogue-request-attempt","startTimeMicroSeconds":1582666422499398,"durationNanoSeconds":7957300,"metadata":{}} -{"traceId":"cc2af584ecf2a705","parentSpanId":"0662ae5d80d73804","spanId":"cc4c5dcb09dddef9","type":"LOCAL","operation":"Dialogue-request initial","startTimeMicroSeconds":1582666422495802,"durationNanoSeconds":23563978,"metadata":{}} -{"traceId":"cc2af584ecf2a705","parentSpanId":"57cd304d8aa2828b","spanId":"dec18ab7ec0993f3","type":"LOCAL","operation":"Dialogue-http-request initial","startTimeMicroSeconds":1582666422723483,"durationNanoSeconds":2243957,"metadata":{}} -{"traceId":"cc2af584ecf2a705","parentSpanId":"9fad1346f7475bb9","spanId":"57cd304d8aa2828b","type":"LOCAL","operation":"Dialogue-http-request","startTimeMicroSeconds":1582666422723472,"durationNanoSeconds":2283742,"metadata":{}} -{"traceId":"cc2af584ecf2a705","parentSpanId":"a854c890e137267a","spanId":"9fad1346f7475bb9","type":"LOCAL","operation":"Dialogue-request-attempt initial","startTimeMicroSeconds":1582666422723449,"durationNanoSeconds":2714734,"metadata":{}} -{"traceId":"cc2af584ecf2a705","parentSpanId":"0a741a3522748074","spanId":"a854c890e137267a","type":"LOCAL","operation":"Dialogue-request-attempt","startTimeMicroSeconds":1582666422723420,"durationNanoSeconds":2753169,"metadata":{}} -{"traceId":"cc2af584ecf2a705","parentSpanId":"cc4c5dcb09dddef9","spanId":"0a741a3522748074","type":"LOCAL","operation":"dialogue-BlacklistingChannel-scheduler","startTimeMicroSeconds":1582666422722160,"durationNanoSeconds":5403844,"metadata":{}} -{"traceId":"cc2af584ecf2a705","parentSpanId":"eea9a0d6db97c500","spanId":"9e14558d567d876c","type":"LOCAL","operation":"Dialogue-http-request initial","startTimeMicroSeconds":1582666422907935,"durationNanoSeconds":702085,"metadata":{}} -{"traceId":"cc2af584ecf2a705","parentSpanId":"e40527d29024b72b","spanId":"eea9a0d6db97c500","type":"LOCAL","operation":"Dialogue-http-request","startTimeMicroSeconds":1582666422907901,"durationNanoSeconds":757730,"metadata":{}} -{"traceId":"cc2af584ecf2a705","parentSpanId":"9a2dfa85700147f7","spanId":"e40527d29024b72b","type":"LOCAL","operation":"Dialogue-request-attempt initial","startTimeMicroSeconds":1582666422907861,"durationNanoSeconds":886895,"metadata":{}} -{"traceId":"cc2af584ecf2a705","parentSpanId":"8daeb0b90c77c79e","spanId":"9a2dfa85700147f7","type":"LOCAL","operation":"Dialogue-request-attempt","startTimeMicroSeconds":1582666422907810,"durationNanoSeconds":945628,"metadata":{}} -{"traceId":"cc2af584ecf2a705","parentSpanId":"0a741a3522748074","spanId":"8daeb0b90c77c79e","type":"LOCAL","operation":"dialogue-BlacklistingChannel-scheduler","startTimeMicroSeconds":1582666422907732,"durationNanoSeconds":1770029,"metadata":{}} -{"traceId":"cc2af584ecf2a705","parentSpanId":"ae91c04dc48d3823","spanId":"f8fae2d39ed7f596","type":"LOCAL","operation":"Dialogue-http-request initial","startTimeMicroSeconds":1582666423404387,"durationNanoSeconds":549102,"metadata":{}} -{"traceId":"cc2af584ecf2a705","parentSpanId":"e6e5748ef6b892ea","spanId":"ae91c04dc48d3823","type":"LOCAL","operation":"Dialogue-http-request","startTimeMicroSeconds":1582666423404347,"durationNanoSeconds":634413,"metadata":{}} -{"traceId":"cc2af584ecf2a705","parentSpanId":"8ccbd85267240f19","spanId":"e6e5748ef6b892ea","type":"LOCAL","operation":"Dialogue-request-attempt initial","startTimeMicroSeconds":1582666423404296,"durationNanoSeconds":791806,"metadata":{}} -{"traceId":"cc2af584ecf2a705","parentSpanId":"5489902871835754","spanId":"8ccbd85267240f19","type":"LOCAL","operation":"Dialogue-request-attempt","startTimeMicroSeconds":1582666423404214,"durationNanoSeconds":883959,"metadata":{}} -{"traceId":"cc2af584ecf2a705","parentSpanId":"8daeb0b90c77c79e","spanId":"5489902871835754","type":"LOCAL","operation":"dialogue-BlacklistingChannel-scheduler","startTimeMicroSeconds":1582666423404156,"durationNanoSeconds":1560627,"metadata":{}} -{"traceId":"cc2af584ecf2a705","parentSpanId":"52619fc7b27ab654","spanId":"96a31783efa996de","type":"LOCAL","operation":"Dialogue-http-request initial","startTimeMicroSeconds":1582666424863107,"durationNanoSeconds":415054,"metadata":{}} -{"traceId":"cc2af584ecf2a705","parentSpanId":"ea8aa8eacfd27d5d","spanId":"52619fc7b27ab654","type":"LOCAL","operation":"Dialogue-http-request","startTimeMicroSeconds":1582666424863095,"durationNanoSeconds":452835,"metadata":{}} -{"traceId":"cc2af584ecf2a705","parentSpanId":"4e43b755683cdcb2","spanId":"ea8aa8eacfd27d5d","type":"LOCAL","operation":"Dialogue-request-attempt initial","startTimeMicroSeconds":1582666424863070,"durationNanoSeconds":581276,"metadata":{}} -{"traceId":"cc2af584ecf2a705","parentSpanId":"eb3dc7fe2b367564","spanId":"4e43b755683cdcb2","type":"LOCAL","operation":"Dialogue-request-attempt","startTimeMicroSeconds":1582666424863050,"durationNanoSeconds":610269,"metadata":{}} -{"traceId":"cc2af584ecf2a705","parentSpanId":null,"spanId":"0662ae5d80d73804","type":"LOCAL","operation":"Dialogue-request","startTimeMicroSeconds":1582666422492806,"durationNanoSeconds":2375555274,"metadata":{}} -{"traceId":"cc2af584ecf2a705","parentSpanId":"5489902871835754","spanId":"eb3dc7fe2b367564","type":"LOCAL","operation":"dialogue-BlacklistingChannel-scheduler","startTimeMicroSeconds":1582666424863016,"durationNanoSeconds":5393223,"metadata":{}} +{"traceId":"09b26a1be498af5c","parentSpanId":"49c88b0d4a7b609a","spanId":"802d8f9c0d0333a7","type":"LOCAL","operation":"Dialogue-http-request initial","startTimeMicroSeconds":1582681721129000,"durationNanoSeconds":7754255,"metadata":{}} +{"traceId":"09b26a1be498af5c","parentSpanId":"9e61fc51aa9ff7d5","spanId":"49c88b0d4a7b609a","type":"LOCAL","operation":"Dialogue-http-request","startTimeMicroSeconds":1582681721129000,"durationNanoSeconds":8326537,"metadata":{}} +{"traceId":"09b26a1be498af5c","parentSpanId":"abd6c8fb7ea1964b","spanId":"9e61fc51aa9ff7d5","type":"LOCAL","operation":"Dialogue-request-attempt initial","startTimeMicroSeconds":1582681721129000,"durationNanoSeconds":10882650,"metadata":{}} +{"traceId":"09b26a1be498af5c","parentSpanId":"3c790821bbcc49be","spanId":"abd6c8fb7ea1964b","type":"LOCAL","operation":"Dialogue-request-attempt","startTimeMicroSeconds":1582681721129000,"durationNanoSeconds":10924205,"metadata":{}} +{"traceId":"09b26a1be498af5c","parentSpanId":"e18bf8527dd88bf6","spanId":"3c790821bbcc49be","type":"LOCAL","operation":"Dialogue-request initial","startTimeMicroSeconds":1582681721123000,"durationNanoSeconds":33250364,"metadata":{}} +{"traceId":"09b26a1be498af5c","parentSpanId":"d4b5027896e2ee6f","spanId":"1b3ff835c00dc3a9","type":"LOCAL","operation":"Dialogue-http-request initial","startTimeMicroSeconds":1582681721268000,"durationNanoSeconds":918989,"metadata":{}} +{"traceId":"09b26a1be498af5c","parentSpanId":"0c6b2f7895022814","spanId":"d4b5027896e2ee6f","type":"LOCAL","operation":"Dialogue-http-request","startTimeMicroSeconds":1582681721268000,"durationNanoSeconds":968054,"metadata":{}} +{"traceId":"09b26a1be498af5c","parentSpanId":"653f1954e48d901d","spanId":"0c6b2f7895022814","type":"LOCAL","operation":"Dialogue-request-attempt initial","startTimeMicroSeconds":1582681721268000,"durationNanoSeconds":1121840,"metadata":{}} +{"traceId":"09b26a1be498af5c","parentSpanId":"a86d7d00725e5e0d","spanId":"653f1954e48d901d","type":"LOCAL","operation":"Dialogue-request-attempt","startTimeMicroSeconds":1582681721268000,"durationNanoSeconds":1174029,"metadata":{}} +{"traceId":"09b26a1be498af5c","parentSpanId":"3c790821bbcc49be","spanId":"a86d7d00725e5e0d","type":"LOCAL","operation":"dialogue-RetryingChannel-scheduler","startTimeMicroSeconds":1582681721267000,"durationNanoSeconds":3972227,"metadata":{}} +{"traceId":"09b26a1be498af5c","parentSpanId":"e33738f0e8df87ed","spanId":"d78edab07c8adbb9","type":"LOCAL","operation":"Dialogue-http-request initial","startTimeMicroSeconds":1582681721609000,"durationNanoSeconds":375103,"metadata":{}} +{"traceId":"09b26a1be498af5c","parentSpanId":"594e8148899ae2fb","spanId":"e33738f0e8df87ed","type":"LOCAL","operation":"Dialogue-http-request","startTimeMicroSeconds":1582681721609000,"durationNanoSeconds":417055,"metadata":{}} +{"traceId":"09b26a1be498af5c","parentSpanId":"8c7cf0ad97a48bab","spanId":"594e8148899ae2fb","type":"LOCAL","operation":"Dialogue-request-attempt initial","startTimeMicroSeconds":1582681721609000,"durationNanoSeconds":550461,"metadata":{}} +{"traceId":"09b26a1be498af5c","parentSpanId":"b1e940cc9ff1f912","spanId":"8c7cf0ad97a48bab","type":"LOCAL","operation":"Dialogue-request-attempt","startTimeMicroSeconds":1582681721609000,"durationNanoSeconds":588480,"metadata":{}} +{"traceId":"09b26a1be498af5c","parentSpanId":"a86d7d00725e5e0d","spanId":"b1e940cc9ff1f912","type":"LOCAL","operation":"dialogue-RetryingChannel-scheduler","startTimeMicroSeconds":1582681721609000,"durationNanoSeconds":1597817,"metadata":{}} +{"traceId":"09b26a1be498af5c","parentSpanId":"6cf1bbc0db4fc84d","spanId":"211a731a733c77ee","type":"LOCAL","operation":"Dialogue-http-request initial","startTimeMicroSeconds":1582681722515000,"durationNanoSeconds":441865,"metadata":{}} +{"traceId":"09b26a1be498af5c","parentSpanId":"0c711e3862bc8847","spanId":"6cf1bbc0db4fc84d","type":"LOCAL","operation":"Dialogue-http-request","startTimeMicroSeconds":1582681722515000,"durationNanoSeconds":493989,"metadata":{}} +{"traceId":"09b26a1be498af5c","parentSpanId":"339a06e6a9db75d0","spanId":"0c711e3862bc8847","type":"LOCAL","operation":"Dialogue-request-attempt initial","startTimeMicroSeconds":1582681722515000,"durationNanoSeconds":644308,"metadata":{}} +{"traceId":"09b26a1be498af5c","parentSpanId":"5a7a77d22c1efbe3","spanId":"339a06e6a9db75d0","type":"LOCAL","operation":"Dialogue-request-attempt","startTimeMicroSeconds":1582681722515000,"durationNanoSeconds":693042,"metadata":{}} +{"traceId":"09b26a1be498af5c","parentSpanId":"b1e940cc9ff1f912","spanId":"5a7a77d22c1efbe3","type":"LOCAL","operation":"dialogue-RetryingChannel-scheduler","startTimeMicroSeconds":1582681722515000,"durationNanoSeconds":1927394,"metadata":{}} +{"traceId":"09b26a1be498af5c","parentSpanId":"25637786765abfdf","spanId":"8765caa53a1e0127","type":"LOCAL","operation":"Dialogue-http-request initial","startTimeMicroSeconds":1582681723414000,"durationNanoSeconds":492161,"metadata":{}} +{"traceId":"09b26a1be498af5c","parentSpanId":"fb99e9d80bd4160d","spanId":"25637786765abfdf","type":"LOCAL","operation":"Dialogue-http-request","startTimeMicroSeconds":1582681723414000,"durationNanoSeconds":543936,"metadata":{}} +{"traceId":"09b26a1be498af5c","parentSpanId":"64281a7f7d042379","spanId":"fb99e9d80bd4160d","type":"LOCAL","operation":"Dialogue-request-attempt initial","startTimeMicroSeconds":1582681723414000,"durationNanoSeconds":715880,"metadata":{}} +{"traceId":"09b26a1be498af5c","parentSpanId":"304ac4d52e61ba1c","spanId":"64281a7f7d042379","type":"LOCAL","operation":"Dialogue-request-attempt","startTimeMicroSeconds":1582681723414000,"durationNanoSeconds":771963,"metadata":{}} +{"traceId":"09b26a1be498af5c","parentSpanId":null,"spanId":"e18bf8527dd88bf6","type":"LOCAL","operation":"Dialogue-request","startTimeMicroSeconds":1582681721118000,"durationNanoSeconds":2302777772,"metadata":{}} +{"traceId":"09b26a1be498af5c","parentSpanId":"5a7a77d22c1efbe3","spanId":"304ac4d52e61ba1c","type":"LOCAL","operation":"dialogue-RetryingChannel-scheduler","startTimeMicroSeconds":1582681723414000,"durationNanoSeconds":8052790,"metadata":{}} From 310cfba0e6d7d56193ff8e9b4090dd901be2e38f Mon Sep 17 00:00:00 2001 From: Carter Kozak Date: Wed, 26 Feb 2020 01:49:02 +0000 Subject: [PATCH 08/17] Add generated changelog entries --- changelog/@unreleased/pr-432.v2.yml | 5 +++++ 1 file changed, 5 insertions(+) create mode 100644 changelog/@unreleased/pr-432.v2.yml diff --git a/changelog/@unreleased/pr-432.v2.yml b/changelog/@unreleased/pr-432.v2.yml new file mode 100644 index 000000000..828e77d61 --- /dev/null +++ b/changelog/@unreleased/pr-432.v2.yml @@ -0,0 +1,5 @@ +type: improvement +improvement: + description: Replace QueuedChannel with a backoff based retryer + links: + - https://github.com/palantir/dialogue/pull/432 From 0081d8ab9b5d8a67486cf001eeb7b0b7b23c316a Mon Sep 17 00:00:00 2001 From: Carter Kozak Date: Tue, 25 Feb 2020 21:48:59 -0500 Subject: [PATCH 09/17] Revert "Fix the simulation scheduler to avoid fast-forward in external use" This reverts commit e83923fc13ee81fd75cdefcea4ed2f42980a4c2f. --- .../com/palantir/dialogue/core/Benchmark.java | 4 ++-- .../core/ExternalDeterministicScheduler.java | 17 +++++++++-------- .../com/palantir/dialogue/core/Simulation.java | 5 +++-- 3 files changed, 14 insertions(+), 12 deletions(-) diff --git a/simulation/src/main/java/com/palantir/dialogue/core/Benchmark.java b/simulation/src/main/java/com/palantir/dialogue/core/Benchmark.java index 10425e8f8..6d6fb3e12 100644 --- a/simulation/src/main/java/com/palantir/dialogue/core/Benchmark.java +++ b/simulation/src/main/java/com/palantir/dialogue/core/Benchmark.java @@ -142,7 +142,7 @@ public void update(Duration _time, long _requestsStarted, long responsesReceived public Benchmark abortAfter(Duration cutoff) { simulation .scheduler() - .scheduleAndFastForward( + .schedule( () -> { log.warn( "Aborted running benchmark after cutoff reached - strategy might be buggy {}", @@ -192,7 +192,7 @@ public void onFailure(Throwable throwable) { simulation .scheduler() - .scheduleAndFastForward( + .schedule( () -> { log.debug( "time={} starting num={} {}", diff --git a/simulation/src/main/java/com/palantir/dialogue/core/ExternalDeterministicScheduler.java b/simulation/src/main/java/com/palantir/dialogue/core/ExternalDeterministicScheduler.java index 0266b54b0..48d5a7595 100644 --- a/simulation/src/main/java/com/palantir/dialogue/core/ExternalDeterministicScheduler.java +++ b/simulation/src/main/java/com/palantir/dialogue/core/ExternalDeterministicScheduler.java @@ -39,25 +39,26 @@ final class ExternalDeterministicScheduler implements ListeningScheduledExecutor @Override public ListenableScheduledFuture schedule(Runnable command, long delay, TimeUnit unit) { - return delegate.schedule(command, delay, unit); + return schedule( + () -> { + command.run(); + return null; + }, + delay, + unit); } @Override public ListenableScheduledFuture schedule(Callable command, long delay, TimeUnit unit) { - return delegate.schedule(command, delay, unit); - } - - @SuppressWarnings({"CheckReturnValue", "FutureReturnValueIgnored"}) - public void scheduleAndFastForward(Runnable command, long delay, TimeUnit unit) { long scheduleTime = ticker.read(); long delayNanos = unit.toNanos(delay); RuntimeException exceptionForStackTrace = new RuntimeException(); - delegate.schedule( + return delegate.schedule( () -> { try { ticker.advanceTo(Duration.ofNanos(scheduleTime + delayNanos)); - command.run(); + return command.call(); } catch (Throwable e) { e.addSuppressed(exceptionForStackTrace); throw e; diff --git a/simulation/src/main/java/com/palantir/dialogue/core/Simulation.java b/simulation/src/main/java/com/palantir/dialogue/core/Simulation.java index 7926c60a3..023ec5925 100644 --- a/simulation/src/main/java/com/palantir/dialogue/core/Simulation.java +++ b/simulation/src/main/java/com/palantir/dialogue/core/Simulation.java @@ -17,6 +17,7 @@ package com.palantir.dialogue.core; import com.github.benmanes.caffeine.cache.Ticker; +import com.google.common.util.concurrent.ListeningScheduledExecutorService; import com.google.common.util.concurrent.MoreExecutors; import java.time.Duration; import java.util.concurrent.TimeUnit; @@ -29,7 +30,7 @@ final class Simulation { private static final Logger log = LoggerFactory.getLogger(Simulation.class); private final DeterministicScheduler deterministicExecutor = new DeterministicScheduler(); - private final ExternalDeterministicScheduler listenableExecutor; + private final ListeningScheduledExecutorService listenableExecutor; private final TestCaffeineTicker ticker = new TestCaffeineTicker(); private final SimulationMetricsReporter metrics = new SimulationMetricsReporter(this); @@ -47,7 +48,7 @@ public Ticker clock() { return ticker; // read only! } - public ExternalDeterministicScheduler scheduler() { + public ListeningScheduledExecutorService scheduler() { return listenableExecutor; } From f841e9350315383e02d5debc4b90c9efdaf92869 Mon Sep 17 00:00:00 2001 From: Carter Kozak Date: Tue, 25 Feb 2020 21:58:18 -0500 Subject: [PATCH 10/17] revert simulation change, comment checkState: allow rewind --- .../ChannelsTest/traces_on_success.log | 6 ++ .../core/ExternalDeterministicScheduler.java | 12 +-- .../dialogue/core/TestCaffeineTicker.java | 14 ++-- ...URRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].png | 4 +- ...0[CONCURRENCY_LIMITER_PIN_UNTIL_ERROR].png | 4 +- ...s_500[CONCURRENCY_LIMITER_ROUND_ROBIN].png | 4 +- .../all_nodes_500[UNLIMITED_ROUND_ROBIN].png | 4 +- ...URRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].png | 4 +- ...e[CONCURRENCY_LIMITER_PIN_UNTIL_ERROR].png | 4 +- ..._hole[CONCURRENCY_LIMITER_ROUND_ROBIN].png | 4 +- .../black_hole[UNLIMITED_ROUND_ROBIN].png | 4 +- ...URRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].png | 4 +- ...n[CONCURRENCY_LIMITER_PIN_UNTIL_ERROR].png | 4 +- ...wdown[CONCURRENCY_LIMITER_ROUND_ROBIN].png | 4 +- ...rastic_slowdown[UNLIMITED_ROUND_ROBIN].png | 4 +- ...URRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].png | 4 +- ...t[CONCURRENCY_LIMITER_PIN_UNTIL_ERROR].png | 4 +- ...evert[CONCURRENCY_LIMITER_ROUND_ROBIN].png | 4 +- ...00s_then_revert[UNLIMITED_ROUND_ROBIN].png | 4 +- ...URRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].png | 4 +- ...g[CONCURRENCY_LIMITER_PIN_UNTIL_ERROR].png | 4 +- ...ading[CONCURRENCY_LIMITER_ROUND_ROBIN].png | 4 +- .../live_reloading[UNLIMITED_ROUND_ROBIN].png | 4 +- ...URRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].png | 4 +- ...r[CONCURRENCY_LIMITER_PIN_UNTIL_ERROR].png | 4 +- ...erver[CONCURRENCY_LIMITER_ROUND_ROBIN].png | 4 +- ..._on_each_server[UNLIMITED_ROUND_ROBIN].png | 4 +- simulation/src/test/resources/report.md | 80 +++++++++---------- ...URRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].png | 4 +- ...e[CONCURRENCY_LIMITER_PIN_UNTIL_ERROR].png | 4 +- ..._case[CONCURRENCY_LIMITER_ROUND_ROBIN].png | 4 +- ...t_possible_case[UNLIMITED_ROUND_ROBIN].png | 4 +- ...URRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].png | 4 +- ...t[CONCURRENCY_LIMITER_PIN_UNTIL_ERROR].png | 4 +- ...evert[CONCURRENCY_LIMITER_ROUND_ROBIN].png | 4 +- ...03s_then_revert[UNLIMITED_ROUND_ROBIN].png | 4 +- ...URRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].png | 4 +- ...s[CONCURRENCY_LIMITER_PIN_UNTIL_ERROR].png | 4 +- ...holds[CONCURRENCY_LIMITER_ROUND_ROBIN].png | 4 +- ...rror_thresholds[UNLIMITED_ROUND_ROBIN].png | 4 +- ...URRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].txt | 2 +- ...0[CONCURRENCY_LIMITER_PIN_UNTIL_ERROR].txt | 2 +- ...s_500[CONCURRENCY_LIMITER_ROUND_ROBIN].txt | 2 +- .../all_nodes_500[UNLIMITED_ROUND_ROBIN].txt | 2 +- ...URRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].txt | 2 +- ...e[CONCURRENCY_LIMITER_PIN_UNTIL_ERROR].txt | 2 +- ..._hole[CONCURRENCY_LIMITER_ROUND_ROBIN].txt | 2 +- .../txt/black_hole[UNLIMITED_ROUND_ROBIN].txt | 2 +- ...URRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].txt | 2 +- ...n[CONCURRENCY_LIMITER_PIN_UNTIL_ERROR].txt | 2 +- ...wdown[CONCURRENCY_LIMITER_ROUND_ROBIN].txt | 2 +- ...rastic_slowdown[UNLIMITED_ROUND_ROBIN].txt | 2 +- ...URRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].txt | 2 +- ...t[CONCURRENCY_LIMITER_PIN_UNTIL_ERROR].txt | 2 +- ...evert[CONCURRENCY_LIMITER_ROUND_ROBIN].txt | 2 +- ...00s_then_revert[UNLIMITED_ROUND_ROBIN].txt | 2 +- ...URRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].txt | 2 +- ...g[CONCURRENCY_LIMITER_PIN_UNTIL_ERROR].txt | 2 +- ...ading[CONCURRENCY_LIMITER_ROUND_ROBIN].txt | 2 +- .../live_reloading[UNLIMITED_ROUND_ROBIN].txt | 2 +- ...URRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].txt | 2 +- ...r[CONCURRENCY_LIMITER_PIN_UNTIL_ERROR].txt | 2 +- ...erver[CONCURRENCY_LIMITER_ROUND_ROBIN].txt | 2 +- ..._on_each_server[UNLIMITED_ROUND_ROBIN].txt | 2 +- ...URRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].txt | 2 +- ...e[CONCURRENCY_LIMITER_PIN_UNTIL_ERROR].txt | 2 +- ..._case[CONCURRENCY_LIMITER_ROUND_ROBIN].txt | 2 +- ...t_possible_case[UNLIMITED_ROUND_ROBIN].txt | 2 +- ...URRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].txt | 2 +- ...t[CONCURRENCY_LIMITER_PIN_UNTIL_ERROR].txt | 2 +- ...evert[CONCURRENCY_LIMITER_ROUND_ROBIN].txt | 2 +- ...03s_then_revert[UNLIMITED_ROUND_ROBIN].txt | 2 +- ...URRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].txt | 2 +- ...s[CONCURRENCY_LIMITER_PIN_UNTIL_ERROR].txt | 2 +- ...holds[CONCURRENCY_LIMITER_ROUND_ROBIN].txt | 2 +- ...rror_thresholds[UNLIMITED_ROUND_ROBIN].txt | 2 +- ...URRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].txt | 2 +- ...s[CONCURRENCY_LIMITER_PIN_UNTIL_ERROR].txt | 2 +- ...lakes[CONCURRENCY_LIMITER_ROUND_ROBIN].txt | 2 +- ...uncommon_flakes[UNLIMITED_ROUND_ROBIN].txt | 2 +- ...URRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].png | 4 +- ...s[CONCURRENCY_LIMITER_PIN_UNTIL_ERROR].png | 4 +- ...lakes[CONCURRENCY_LIMITER_ROUND_ROBIN].png | 4 +- ...uncommon_flakes[UNLIMITED_ROUND_ROBIN].png | 4 +- 84 files changed, 173 insertions(+), 179 deletions(-) create mode 100644 dialogue-core/src/test/resources/tracing/ChannelsTest/traces_on_success.log diff --git a/dialogue-core/src/test/resources/tracing/ChannelsTest/traces_on_success.log b/dialogue-core/src/test/resources/tracing/ChannelsTest/traces_on_success.log new file mode 100644 index 000000000..0256c3bf1 --- /dev/null +++ b/dialogue-core/src/test/resources/tracing/ChannelsTest/traces_on_success.log @@ -0,0 +1,6 @@ +{"traceId":"fd8b24d382531fa0","parentSpanId":"ea0900dd50ba18fd","spanId":"f656a157f0b91edc","type":"LOCAL","operation":"Dialogue-http-request initial","startTimeMicroSeconds":1582685430723968,"durationNanoSeconds":404196,"metadata":{}} +{"traceId":"fd8b24d382531fa0","parentSpanId":"0291503916ed4578","spanId":"ea0900dd50ba18fd","type":"LOCAL","operation":"Dialogue-http-request","startTimeMicroSeconds":1582685430723945,"durationNanoSeconds":448189,"metadata":{}} +{"traceId":"fd8b24d382531fa0","parentSpanId":"afd7e0b178bf1210","spanId":"0291503916ed4578","type":"LOCAL","operation":"Dialogue-request-attempt initial","startTimeMicroSeconds":1582685430723918,"durationNanoSeconds":1841451,"metadata":{}} +{"traceId":"fd8b24d382531fa0","parentSpanId":"d3d308fa976128d2","spanId":"afd7e0b178bf1210","type":"LOCAL","operation":"Dialogue-request-attempt","startTimeMicroSeconds":1582685430723889,"durationNanoSeconds":1890797,"metadata":{}} +{"traceId":"fd8b24d382531fa0","parentSpanId":"74528c96ab18dc25","spanId":"d3d308fa976128d2","type":"LOCAL","operation":"Dialogue-request initial","startTimeMicroSeconds":1582685430723622,"durationNanoSeconds":2826486,"metadata":{}} +{"traceId":"fd8b24d382531fa0","parentSpanId":null,"spanId":"74528c96ab18dc25","type":"LOCAL","operation":"Dialogue-request","startTimeMicroSeconds":1582685430723593,"durationNanoSeconds":2889237,"metadata":{}} diff --git a/simulation/src/main/java/com/palantir/dialogue/core/ExternalDeterministicScheduler.java b/simulation/src/main/java/com/palantir/dialogue/core/ExternalDeterministicScheduler.java index 48d5a7595..b5a8b2523 100644 --- a/simulation/src/main/java/com/palantir/dialogue/core/ExternalDeterministicScheduler.java +++ b/simulation/src/main/java/com/palantir/dialogue/core/ExternalDeterministicScheduler.java @@ -18,7 +18,6 @@ import com.google.common.util.concurrent.ListenableFuture; import com.google.common.util.concurrent.ListenableScheduledFuture; import com.google.common.util.concurrent.ListeningScheduledExecutorService; -import java.time.Duration; import java.util.Collection; import java.util.List; import java.util.concurrent.Callable; @@ -52,17 +51,10 @@ public ListenableScheduledFuture schedule(Runnable command, long delay, TimeU public ListenableScheduledFuture schedule(Callable command, long delay, TimeUnit unit) { long scheduleTime = ticker.read(); long delayNanos = unit.toNanos(delay); - - RuntimeException exceptionForStackTrace = new RuntimeException(); return delegate.schedule( () -> { - try { - ticker.advanceTo(Duration.ofNanos(scheduleTime + delayNanos)); - return command.call(); - } catch (Throwable e) { - e.addSuppressed(exceptionForStackTrace); - throw e; - } + ticker.advanceTo(scheduleTime + delayNanos); + return command.call(); }, delay, unit); diff --git a/simulation/src/main/java/com/palantir/dialogue/core/TestCaffeineTicker.java b/simulation/src/main/java/com/palantir/dialogue/core/TestCaffeineTicker.java index ec53f77af..606e495b8 100644 --- a/simulation/src/main/java/com/palantir/dialogue/core/TestCaffeineTicker.java +++ b/simulation/src/main/java/com/palantir/dialogue/core/TestCaffeineTicker.java @@ -16,9 +16,6 @@ package com.palantir.dialogue.core; import com.github.benmanes.caffeine.cache.Ticker; -import com.palantir.logsafe.Preconditions; -import com.palantir.logsafe.SafeArg; -import java.time.Duration; import java.util.concurrent.TimeUnit; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -33,14 +30,13 @@ public long read() { return nanos; } - void advanceTo(Duration duration) { - long newNanos = duration.toNanos(); + void advanceTo(long newNanos) { if (newNanos < nanos) { long difference = nanos - newNanos; - Preconditions.checkState( - difference < Duration.ofMillis(1).toNanos(), - "Large time rewind - this is likely a bug in the test harness", - SafeArg.of("difference", difference)); + // Preconditions.checkState( + // difference < Duration.ofMillis(2).toNanos(), + // "Large time rewind - this is likely a bug in the test harness", + // SafeArg.of("difference", difference)); log.debug( "Tried to rewind time by {} micros - no-op as this is deterministic and harmless", TimeUnit.MICROSECONDS.convert(difference, TimeUnit.NANOSECONDS)); diff --git a/simulation/src/test/resources/all_nodes_500[CONCURRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].png b/simulation/src/test/resources/all_nodes_500[CONCURRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].png index f8fdc1db5..11b7216fe 100644 --- a/simulation/src/test/resources/all_nodes_500[CONCURRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].png +++ b/simulation/src/test/resources/all_nodes_500[CONCURRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].png @@ -1,3 +1,3 @@ version https://git-lfs.github.com/spec/v1 -oid sha256:a6f2a2dbc8b54bbb053edfa34a57052a8b9cc4260f6ac819a39d6448d61c38f3 -size 166274 +oid sha256:e26afa8c5053623cb1ede2032a7ec8a083b9de75e03d3d26508e0094b05aae88 +size 180149 diff --git a/simulation/src/test/resources/all_nodes_500[CONCURRENCY_LIMITER_PIN_UNTIL_ERROR].png b/simulation/src/test/resources/all_nodes_500[CONCURRENCY_LIMITER_PIN_UNTIL_ERROR].png index 69e68cf86..c8b37306d 100644 --- a/simulation/src/test/resources/all_nodes_500[CONCURRENCY_LIMITER_PIN_UNTIL_ERROR].png +++ b/simulation/src/test/resources/all_nodes_500[CONCURRENCY_LIMITER_PIN_UNTIL_ERROR].png @@ -1,3 +1,3 @@ version https://git-lfs.github.com/spec/v1 -oid sha256:7dcf77450d2446563ebc46c0cf74c8ccc883f45810cd84d57a9401afd9ad1ad4 -size 155428 +oid sha256:45ad237bb93fbae6fbf9bc4040c2ba77ba906187db9bc9f08d8a03a842a0fad5 +size 162951 diff --git a/simulation/src/test/resources/all_nodes_500[CONCURRENCY_LIMITER_ROUND_ROBIN].png b/simulation/src/test/resources/all_nodes_500[CONCURRENCY_LIMITER_ROUND_ROBIN].png index 81269dfef..c67e30726 100644 --- a/simulation/src/test/resources/all_nodes_500[CONCURRENCY_LIMITER_ROUND_ROBIN].png +++ b/simulation/src/test/resources/all_nodes_500[CONCURRENCY_LIMITER_ROUND_ROBIN].png @@ -1,3 +1,3 @@ version https://git-lfs.github.com/spec/v1 -oid sha256:c8d45d45ff5b5556e1bdaf585e841e7d347a3ba71caa4d4f55e1c7ef370bea83 -size 114957 +oid sha256:70cd5a8a4a136e0b43d7790fee60d842a3f020a9ea0480778ecfcdf8c2d6130e +size 120912 diff --git a/simulation/src/test/resources/all_nodes_500[UNLIMITED_ROUND_ROBIN].png b/simulation/src/test/resources/all_nodes_500[UNLIMITED_ROUND_ROBIN].png index 05b5871cf..6cc4b2a73 100644 --- a/simulation/src/test/resources/all_nodes_500[UNLIMITED_ROUND_ROBIN].png +++ b/simulation/src/test/resources/all_nodes_500[UNLIMITED_ROUND_ROBIN].png @@ -1,3 +1,3 @@ version https://git-lfs.github.com/spec/v1 -oid sha256:364db8979470ce2b022af4a94a99151170d233feff4562125bfef6d5c10119c9 -size 114590 +oid sha256:7ec61dce7640e9e5e1ce9e79c8e816cac5fdfa1ec2520a63c6b12265def0be42 +size 120164 diff --git a/simulation/src/test/resources/black_hole[CONCURRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].png b/simulation/src/test/resources/black_hole[CONCURRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].png index 6ae68db14..37fd513e0 100644 --- a/simulation/src/test/resources/black_hole[CONCURRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].png +++ b/simulation/src/test/resources/black_hole[CONCURRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].png @@ -1,3 +1,3 @@ version https://git-lfs.github.com/spec/v1 -oid sha256:94a4a05d4ecde191b6f175e14d556951f7cb75a170b3595d83dce3a776e3fdfb -size 92638 +oid sha256:75d7402f0a3c1361242b07b6726679614e8cae5ead990832d336158bc0f599bb +size 101885 diff --git a/simulation/src/test/resources/black_hole[CONCURRENCY_LIMITER_PIN_UNTIL_ERROR].png b/simulation/src/test/resources/black_hole[CONCURRENCY_LIMITER_PIN_UNTIL_ERROR].png index 90ec61f01..6483e788a 100644 --- a/simulation/src/test/resources/black_hole[CONCURRENCY_LIMITER_PIN_UNTIL_ERROR].png +++ b/simulation/src/test/resources/black_hole[CONCURRENCY_LIMITER_PIN_UNTIL_ERROR].png @@ -1,3 +1,3 @@ version https://git-lfs.github.com/spec/v1 -oid sha256:5af7613c0c4fa2a991f8c1d289f4cc593abdeb7eac2c4078728cd080caee4c15 -size 90357 +oid sha256:644711d014217b5f939b15e753df3290a35c83a056324ac11533307106b28985 +size 100371 diff --git a/simulation/src/test/resources/black_hole[CONCURRENCY_LIMITER_ROUND_ROBIN].png b/simulation/src/test/resources/black_hole[CONCURRENCY_LIMITER_ROUND_ROBIN].png index e79f25474..ad24dc73e 100644 --- a/simulation/src/test/resources/black_hole[CONCURRENCY_LIMITER_ROUND_ROBIN].png +++ b/simulation/src/test/resources/black_hole[CONCURRENCY_LIMITER_ROUND_ROBIN].png @@ -1,3 +1,3 @@ version https://git-lfs.github.com/spec/v1 -oid sha256:c2738827f335ef407b88334b71052f01dde4bd3478b98c66e188f905b8b5985e -size 92471 +oid sha256:79e89ac6f0b8dbebcb0e6da25e23ea9bd7763eb89b6e0e4fd42daca07736b194 +size 101435 diff --git a/simulation/src/test/resources/black_hole[UNLIMITED_ROUND_ROBIN].png b/simulation/src/test/resources/black_hole[UNLIMITED_ROUND_ROBIN].png index fdb030b33..c53da382f 100644 --- a/simulation/src/test/resources/black_hole[UNLIMITED_ROUND_ROBIN].png +++ b/simulation/src/test/resources/black_hole[UNLIMITED_ROUND_ROBIN].png @@ -1,3 +1,3 @@ version https://git-lfs.github.com/spec/v1 -oid sha256:a11ebbd8893e89a96cdefb16e1b54816d1b76ce38340372e2d083e9fb171ac72 -size 90701 +oid sha256:b7dd26b83eeae2f4be53a2fbf11621e1f0d0231ddaddec0a6a8c440f5f697a58 +size 98156 diff --git a/simulation/src/test/resources/drastic_slowdown[CONCURRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].png b/simulation/src/test/resources/drastic_slowdown[CONCURRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].png index fd9a8081b..b8c6110ad 100644 --- a/simulation/src/test/resources/drastic_slowdown[CONCURRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].png +++ b/simulation/src/test/resources/drastic_slowdown[CONCURRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].png @@ -1,3 +1,3 @@ version https://git-lfs.github.com/spec/v1 -oid sha256:d9480c7f5981778bbbdf4ae0876f382eff0a07e1f9a674ea94d8869c68da9ea9 -size 91734 +oid sha256:893e8bdd5be5410e8deb00877f0a00dc12b3ff77da1e4d912a765910bad8f9f8 +size 102075 diff --git a/simulation/src/test/resources/drastic_slowdown[CONCURRENCY_LIMITER_PIN_UNTIL_ERROR].png b/simulation/src/test/resources/drastic_slowdown[CONCURRENCY_LIMITER_PIN_UNTIL_ERROR].png index 8696021bd..ffdc4fdd8 100644 --- a/simulation/src/test/resources/drastic_slowdown[CONCURRENCY_LIMITER_PIN_UNTIL_ERROR].png +++ b/simulation/src/test/resources/drastic_slowdown[CONCURRENCY_LIMITER_PIN_UNTIL_ERROR].png @@ -1,3 +1,3 @@ version https://git-lfs.github.com/spec/v1 -oid sha256:96903770263f5efe494822e165ea74845dbdbe8083ca3461ccb11e2770c89e32 -size 88274 +oid sha256:2f53134c29c11c30e5204dd9cb259d257ac115b0a657b65cdd4db718fce8e0c6 +size 99435 diff --git a/simulation/src/test/resources/drastic_slowdown[CONCURRENCY_LIMITER_ROUND_ROBIN].png b/simulation/src/test/resources/drastic_slowdown[CONCURRENCY_LIMITER_ROUND_ROBIN].png index ed70347b4..7fc476523 100644 --- a/simulation/src/test/resources/drastic_slowdown[CONCURRENCY_LIMITER_ROUND_ROBIN].png +++ b/simulation/src/test/resources/drastic_slowdown[CONCURRENCY_LIMITER_ROUND_ROBIN].png @@ -1,3 +1,3 @@ version https://git-lfs.github.com/spec/v1 -oid sha256:d66270aefd6b655298a7b1040be6fa4d98d1f166fc7b1d2d2f8acc282eb18968 -size 91630 +oid sha256:1a29c87e19b3f08c8d659fada1fd6e12bbea1261074681c3ed14d64f534e47c3 +size 102242 diff --git a/simulation/src/test/resources/drastic_slowdown[UNLIMITED_ROUND_ROBIN].png b/simulation/src/test/resources/drastic_slowdown[UNLIMITED_ROUND_ROBIN].png index d46b4a2c5..06881d168 100644 --- a/simulation/src/test/resources/drastic_slowdown[UNLIMITED_ROUND_ROBIN].png +++ b/simulation/src/test/resources/drastic_slowdown[UNLIMITED_ROUND_ROBIN].png @@ -1,3 +1,3 @@ version https://git-lfs.github.com/spec/v1 -oid sha256:67b5d3a78d4ab3826780e3e9cf4a742afba9193b3ec3dcb9dcad26a5bdf9e84d -size 91271 +oid sha256:b39bcf627f1b8902764dca84a2dc25526be77a9e56696c51d04b7e0d466c3a2b +size 100524 diff --git a/simulation/src/test/resources/fast_500s_then_revert[CONCURRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].png b/simulation/src/test/resources/fast_500s_then_revert[CONCURRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].png index 7ed0805ea..4e2423c3f 100644 --- a/simulation/src/test/resources/fast_500s_then_revert[CONCURRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].png +++ b/simulation/src/test/resources/fast_500s_then_revert[CONCURRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].png @@ -1,3 +1,3 @@ version https://git-lfs.github.com/spec/v1 -oid sha256:ae302eb217e284df84d90541843ba5c19ed2333129b6f377880b9c4094302ce1 -size 119266 +oid sha256:73b7c4898d860f5d2696a8196f780e468afd57b105b6851fc9df65d9e6a0d965 +size 128918 diff --git a/simulation/src/test/resources/fast_500s_then_revert[CONCURRENCY_LIMITER_PIN_UNTIL_ERROR].png b/simulation/src/test/resources/fast_500s_then_revert[CONCURRENCY_LIMITER_PIN_UNTIL_ERROR].png index cca6be792..8448562de 100644 --- a/simulation/src/test/resources/fast_500s_then_revert[CONCURRENCY_LIMITER_PIN_UNTIL_ERROR].png +++ b/simulation/src/test/resources/fast_500s_then_revert[CONCURRENCY_LIMITER_PIN_UNTIL_ERROR].png @@ -1,3 +1,3 @@ version https://git-lfs.github.com/spec/v1 -oid sha256:6acb15dcb1c0ccce74b95b1201adf53c9a6fb1214bc757332449ab8285a281b9 -size 82977 +oid sha256:78f63d5dca3b52185166060f293b89f0d3019e18fe90a1585a77c56ed789c937 +size 92743 diff --git a/simulation/src/test/resources/fast_500s_then_revert[CONCURRENCY_LIMITER_ROUND_ROBIN].png b/simulation/src/test/resources/fast_500s_then_revert[CONCURRENCY_LIMITER_ROUND_ROBIN].png index 2e5982211..050b1008a 100644 --- a/simulation/src/test/resources/fast_500s_then_revert[CONCURRENCY_LIMITER_ROUND_ROBIN].png +++ b/simulation/src/test/resources/fast_500s_then_revert[CONCURRENCY_LIMITER_ROUND_ROBIN].png @@ -1,3 +1,3 @@ version https://git-lfs.github.com/spec/v1 -oid sha256:128a4ec3eea36f41a3adf456397b2aa06a3800b4e7180361b02f7a012ea796f9 -size 150416 +oid sha256:4b9b7fdb450a567dd53bc6b22e5506f4479bba879d59e257c5195bd6b141ff4b +size 156809 diff --git a/simulation/src/test/resources/fast_500s_then_revert[UNLIMITED_ROUND_ROBIN].png b/simulation/src/test/resources/fast_500s_then_revert[UNLIMITED_ROUND_ROBIN].png index b8ca36581..de3c167bb 100644 --- a/simulation/src/test/resources/fast_500s_then_revert[UNLIMITED_ROUND_ROBIN].png +++ b/simulation/src/test/resources/fast_500s_then_revert[UNLIMITED_ROUND_ROBIN].png @@ -1,3 +1,3 @@ version https://git-lfs.github.com/spec/v1 -oid sha256:82747f174c2e9e3b6ac49527625b67fae6ea29846b449a51cd7ce5bd88a4f595 -size 150032 +oid sha256:46eb269bdf57afe844cc7339b521e63fc5ebdd8dc394616f132fad6ffe73bc6c +size 156086 diff --git a/simulation/src/test/resources/live_reloading[CONCURRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].png b/simulation/src/test/resources/live_reloading[CONCURRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].png index f3b391f04..1b3fb130d 100644 --- a/simulation/src/test/resources/live_reloading[CONCURRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].png +++ b/simulation/src/test/resources/live_reloading[CONCURRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].png @@ -1,3 +1,3 @@ version https://git-lfs.github.com/spec/v1 -oid sha256:692a74113932cacdee9f72ac5884d77bab03ade004f4c50ba33c14cbe4a6fb68 -size 120176 +oid sha256:2d9cc4200431e2a56147d08f622fd1b4b040dc3c98d2fc6dc12076518d1c71d7 +size 131916 diff --git a/simulation/src/test/resources/live_reloading[CONCURRENCY_LIMITER_PIN_UNTIL_ERROR].png b/simulation/src/test/resources/live_reloading[CONCURRENCY_LIMITER_PIN_UNTIL_ERROR].png index 9b2d4c474..0c42736e1 100644 --- a/simulation/src/test/resources/live_reloading[CONCURRENCY_LIMITER_PIN_UNTIL_ERROR].png +++ b/simulation/src/test/resources/live_reloading[CONCURRENCY_LIMITER_PIN_UNTIL_ERROR].png @@ -1,3 +1,3 @@ version https://git-lfs.github.com/spec/v1 -oid sha256:640e5599fd511d9d8f4718e6119496efbdb116d1d4e54099c85aa9799154aa13 -size 114692 +oid sha256:96cf6edd6e9656958c38eda3dd0074ed17bf927a48710b22017bcba2cb2f9094 +size 122196 diff --git a/simulation/src/test/resources/live_reloading[CONCURRENCY_LIMITER_ROUND_ROBIN].png b/simulation/src/test/resources/live_reloading[CONCURRENCY_LIMITER_ROUND_ROBIN].png index fa2e7d8d2..4a5576fba 100644 --- a/simulation/src/test/resources/live_reloading[CONCURRENCY_LIMITER_ROUND_ROBIN].png +++ b/simulation/src/test/resources/live_reloading[CONCURRENCY_LIMITER_ROUND_ROBIN].png @@ -1,3 +1,3 @@ version https://git-lfs.github.com/spec/v1 -oid sha256:98ad2da05348bd1d6374e7ff56f323452652b615725cbdfc6c9d98fbd84793b4 -size 114801 +oid sha256:e47fc243b06ce2ae6f044cfcbdc0fb7c571f36581eb285b22347d8b17c334014 +size 124308 diff --git a/simulation/src/test/resources/live_reloading[UNLIMITED_ROUND_ROBIN].png b/simulation/src/test/resources/live_reloading[UNLIMITED_ROUND_ROBIN].png index 8320f324f..82462c883 100644 --- a/simulation/src/test/resources/live_reloading[UNLIMITED_ROUND_ROBIN].png +++ b/simulation/src/test/resources/live_reloading[UNLIMITED_ROUND_ROBIN].png @@ -1,3 +1,3 @@ version https://git-lfs.github.com/spec/v1 -oid sha256:1ad481026594b9c6eed222589147158d429bf4d1041dfa0efb2f81b8a85c9e98 -size 115690 +oid sha256:f3ac1372a12be003925c3b73c2c7e525a0ba8e36a87df84617c5f3aca2bb1d51 +size 125717 diff --git a/simulation/src/test/resources/one_endpoint_dies_on_each_server[CONCURRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].png b/simulation/src/test/resources/one_endpoint_dies_on_each_server[CONCURRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].png index fb3aaf588..fe04fbab1 100644 --- a/simulation/src/test/resources/one_endpoint_dies_on_each_server[CONCURRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].png +++ b/simulation/src/test/resources/one_endpoint_dies_on_each_server[CONCURRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].png @@ -1,3 +1,3 @@ version https://git-lfs.github.com/spec/v1 -oid sha256:63294f8c6b34a486eeddec5d238ad2dec01d3fd32025f6a66cc967f170900e1c -size 146213 +oid sha256:7ec7ceb400f5c5ebeea8a3429ae20f7650b4ca5e685cda6236f42f82f2172cb5 +size 161956 diff --git a/simulation/src/test/resources/one_endpoint_dies_on_each_server[CONCURRENCY_LIMITER_PIN_UNTIL_ERROR].png b/simulation/src/test/resources/one_endpoint_dies_on_each_server[CONCURRENCY_LIMITER_PIN_UNTIL_ERROR].png index c93f2b9d6..3f90ba8f2 100644 --- a/simulation/src/test/resources/one_endpoint_dies_on_each_server[CONCURRENCY_LIMITER_PIN_UNTIL_ERROR].png +++ b/simulation/src/test/resources/one_endpoint_dies_on_each_server[CONCURRENCY_LIMITER_PIN_UNTIL_ERROR].png @@ -1,3 +1,3 @@ version https://git-lfs.github.com/spec/v1 -oid sha256:efd56b9ca9e6fb59c8d73b16d1eb79ff956f3e4a0fbfa08ef4793ef3f2945417 -size 141995 +oid sha256:0b69edf22458ead638d0f5eb50526659e53065b360e2f76a5ac7d0b31496cc35 +size 149729 diff --git a/simulation/src/test/resources/one_endpoint_dies_on_each_server[CONCURRENCY_LIMITER_ROUND_ROBIN].png b/simulation/src/test/resources/one_endpoint_dies_on_each_server[CONCURRENCY_LIMITER_ROUND_ROBIN].png index 03bd899ca..082729cdc 100644 --- a/simulation/src/test/resources/one_endpoint_dies_on_each_server[CONCURRENCY_LIMITER_ROUND_ROBIN].png +++ b/simulation/src/test/resources/one_endpoint_dies_on_each_server[CONCURRENCY_LIMITER_ROUND_ROBIN].png @@ -1,3 +1,3 @@ version https://git-lfs.github.com/spec/v1 -oid sha256:a9aae71317e5c64bb62195afc6851f1bdbef2430767b7963b7840a4427ed081c -size 113216 +oid sha256:46822d816c73084513d745608cd06b2c36e315fd278b83441f7f4b3237508319 +size 120782 diff --git a/simulation/src/test/resources/one_endpoint_dies_on_each_server[UNLIMITED_ROUND_ROBIN].png b/simulation/src/test/resources/one_endpoint_dies_on_each_server[UNLIMITED_ROUND_ROBIN].png index e0349a6a8..046dea7a3 100644 --- a/simulation/src/test/resources/one_endpoint_dies_on_each_server[UNLIMITED_ROUND_ROBIN].png +++ b/simulation/src/test/resources/one_endpoint_dies_on_each_server[UNLIMITED_ROUND_ROBIN].png @@ -1,3 +1,3 @@ version https://git-lfs.github.com/spec/v1 -oid sha256:27db45d539791d10bda2f2a3a7f7565b605049c7e6f3dc7211c22530a85aea26 -size 112929 +oid sha256:413989eb32845bd4fc9a3deb4aaefb8b63e07095a145089881f0e8514ce75b57 +size 119885 diff --git a/simulation/src/test/resources/report.md b/simulation/src/test/resources/report.md index 0e49f28ae..c55bcaea4 100644 --- a/simulation/src/test/resources/report.md +++ b/simulation/src/test/resources/report.md @@ -1,46 +1,46 @@ # Report ``` - all_nodes_500[CONCURRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].txt: success=52.2% client_mean=PT1.016145S server_cpu=PT16M24.9S client_received=2000/2000 server_resps=1672 codes={200=1043, 500=629, Failed to make a request=328} - all_nodes_500[CONCURRENCY_LIMITER_PIN_UNTIL_ERROR].txt: success=50.0% client_mean=PT0.59085S server_cpu=PT19M41.7S client_received=2000/2000 server_resps=2000 codes={200=1000, 500=1000} - all_nodes_500[CONCURRENCY_LIMITER_ROUND_ROBIN].txt: success=50.0% client_mean=PT0.59085S server_cpu=PT19M41.7S client_received=2000/2000 server_resps=2000 codes={200=1000, 500=1000} - all_nodes_500[UNLIMITED_ROUND_ROBIN].txt: success=50.0% client_mean=PT0.59085S server_cpu=PT19M41.7S client_received=2000/2000 server_resps=2000 codes={200=1000, 500=1000} - black_hole[CONCURRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].txt: success=89.9% client_mean=PT0.58008069S server_cpu=PT17M21.9S client_received=1797/2000 server_resps=1797 codes={200=1797} - black_hole[CONCURRENCY_LIMITER_PIN_UNTIL_ERROR].txt: success=88.7% client_mean=PT0.581249294S server_cpu=PT17M7.5S client_received=1773/2000 server_resps=1773 codes={200=1773} - black_hole[CONCURRENCY_LIMITER_ROUND_ROBIN].txt: success=89.9% client_mean=PT0.58008069S server_cpu=PT17M21.9S client_received=1797/2000 server_resps=1797 codes={200=1797} - black_hole[UNLIMITED_ROUND_ROBIN].txt: success=65.0% client_mean=PT0.586343341S server_cpu=PT12M41.66S client_received=1299/2000 server_resps=1299 codes={200=1299} - drastic_slowdown[CONCURRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].txt: success=100.0% client_mean=PT0.85580125S server_cpu=PT57M3.205S client_received=4000/4000 server_resps=4000 codes={200=4000} - drastic_slowdown[CONCURRENCY_LIMITER_PIN_UNTIL_ERROR].txt: success=100.0% client_mean=PT0.86496875S server_cpu=PT57M38.075S client_received=4000/4000 server_resps=4000 codes={200=4000} - drastic_slowdown[CONCURRENCY_LIMITER_ROUND_ROBIN].txt: success=100.0% client_mean=PT0.85580125S server_cpu=PT57M3.205S client_received=4000/4000 server_resps=4000 codes={200=4000} - drastic_slowdown[UNLIMITED_ROUND_ROBIN].txt: success=100.0% client_mean=PT2.44372125S server_cpu=PT2H42M54.885S client_received=4000/4000 server_resps=4000 codes={200=4000} - fast_500s_then_revert[CONCURRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].txt: success=98.1% client_mean=PT0.071725866S server_cpu=PT4M28.972S client_received=3750/3750 server_resps=3750 codes={200=3679, 500=71} - fast_500s_then_revert[CONCURRENCY_LIMITER_PIN_UNTIL_ERROR].txt: success=99.7% client_mean=PT0.079397333S server_cpu=PT4M57.74S client_received=3750/3750 server_resps=3750 codes={200=3739, 500=11} - fast_500s_then_revert[CONCURRENCY_LIMITER_ROUND_ROBIN].txt: success=76.7% client_mean=PT0.053707733S server_cpu=PT3M21.404S client_received=3750/3750 server_resps=3750 codes={200=2876, 500=874} - fast_500s_then_revert[UNLIMITED_ROUND_ROBIN].txt: success=76.7% client_mean=PT0.053707733S server_cpu=PT3M21.404S client_received=3750/3750 server_resps=3750 codes={200=2876, 500=874} - live_reloading[CONCURRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].txt: success=59.5% client_mean=PT2.1742928S server_cpu=PT1H1M39.384S client_received=2500/2500 server_resps=1832 codes={200=1488, 500=344, Failed to make a request=668} - live_reloading[CONCURRENCY_LIMITER_PIN_UNTIL_ERROR].txt: success=53.8% client_mean=PT2.3703696S server_cpu=PT1H21M50.076S client_received=2500/2500 server_resps=2176 codes={200=1346, 500=830, Failed to make a request=324} - live_reloading[CONCURRENCY_LIMITER_ROUND_ROBIN].txt: success=52.1% client_mean=PT2.3660912S server_cpu=PT1H21M12.564S client_received=2500/2500 server_resps=2193 codes={200=1302, 500=891, Failed to make a request=307} - live_reloading[UNLIMITED_ROUND_ROBIN].txt: success=58.4% client_mean=PT2.3889216S server_cpu=PT1H39M32.304S client_received=2500/2500 server_resps=2500 codes={200=1461, 500=1039} -one_endpoint_dies_on_each_server[CONCURRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].txt: success=44.1% client_mean=PT1.0998096S server_cpu=PT13M25.476S client_received=2500/2500 server_resps=1426 codes={200=1103, 500=323, Failed to make a request=1074} -one_endpoint_dies_on_each_server[CONCURRENCY_LIMITER_PIN_UNTIL_ERROR].txt: success=63.8% client_mean=PT0.5829248S server_cpu=PT24M14.688S client_received=2500/2500 server_resps=2500 codes={200=1595, 500=905} - one_endpoint_dies_on_each_server[CONCURRENCY_LIMITER_ROUND_ROBIN].txt: success=65.5% client_mean=PT0.58188S server_cpu=PT24M14.7S client_received=2500/2500 server_resps=2500 codes={200=1638, 500=862} - one_endpoint_dies_on_each_server[UNLIMITED_ROUND_ROBIN].txt: success=65.5% client_mean=PT0.58188S server_cpu=PT24M14.7S client_received=2500/2500 server_resps=2500 codes={200=1638, 500=862} - simplest_possible_case[CONCURRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].txt: success=100.0% client_mean=PT0.757203849S server_cpu=PT2H46M35.09080914S client_received=13200/13200 server_resps=13200 codes={200=13200} - simplest_possible_case[CONCURRENCY_LIMITER_PIN_UNTIL_ERROR].txt: success=100.0% client_mean=PT0.997975196S server_cpu=PT3H39M33.27259554S client_received=13200/13200 server_resps=13200 codes={200=13200} - simplest_possible_case[CONCURRENCY_LIMITER_ROUND_ROBIN].txt: success=100.0% client_mean=PT0.757203849S server_cpu=PT2H46M35.09080914S client_received=13200/13200 server_resps=13200 codes={200=13200} - simplest_possible_case[UNLIMITED_ROUND_ROBIN].txt: success=100.0% client_mean=PT0.757203849S server_cpu=PT2H46M35.09080914S client_received=13200/13200 server_resps=13200 codes={200=13200} - slow_503s_then_revert[CONCURRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].txt: success=100.0% client_mean=PT0.30077S server_cpu=PT14M38.66S client_received=3000/3000 server_resps=3175 codes={200=3000} - slow_503s_then_revert[CONCURRENCY_LIMITER_PIN_UNTIL_ERROR].txt: success=100.0% client_mean=PT0.345103333S server_cpu=PT16M47.965S client_received=3000/3000 server_resps=3197 codes={200=3000} - slow_503s_then_revert[CONCURRENCY_LIMITER_ROUND_ROBIN].txt: success=100.0% client_mean=PT0.741096666S server_cpu=PT36M14.075S client_received=3000/3000 server_resps=3411 codes={200=3000} - slow_503s_then_revert[UNLIMITED_ROUND_ROBIN].txt: success=100.0% client_mean=PT1.423741666S server_cpu=PT1H9M35.875S client_received=3000/3000 server_resps=3802 codes={200=3000} -slowdown_and_error_thresholds[CONCURRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].txt: success=3.7% client_mean=PT2.0471542S server_cpu=PT1H26M34.726S client_received=10000/10000 server_resps=1614 codes={200=370, 500=1244, Failed to make a request=8386} -slowdown_and_error_thresholds[CONCURRENCY_LIMITER_PIN_UNTIL_ERROR].txt: success=1.2% client_mean=PT2.5592244S server_cpu=PT3H42M1.968S client_received=10000/10000 server_resps=4248 codes={200=121, 500=4127, Failed to make a request=5752} - slowdown_and_error_thresholds[CONCURRENCY_LIMITER_ROUND_ROBIN].txt: success=1.2% client_mean=PT2.57855S server_cpu=PT3H43M45.466S client_received=10000/10000 server_resps=4432 codes={200=120, 500=4312, Failed to make a request=5568} - slowdown_and_error_thresholds[UNLIMITED_ROUND_ROBIN].txt: success=1.2% client_mean=PT3.5573984S server_cpu=PT9H52M53.984S client_received=10000/10000 server_resps=10000 codes={200=120, 500=9880} - uncommon_flakes[CONCURRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].txt: success=94.1% client_mean=PT0.207311264S server_cpu=PT0S client_received=9712/10000 server_resps=9499 codes={200=9405, 500=94, Failed to make a request=213} - uncommon_flakes[CONCURRENCY_LIMITER_PIN_UNTIL_ERROR].txt: success=99.0% client_mean=PT0S server_cpu=PT0S client_received=10000/10000 server_resps=10000 codes={200=9902, 500=98} - uncommon_flakes[CONCURRENCY_LIMITER_ROUND_ROBIN].txt: success=99.0% client_mean=PT0S server_cpu=PT0S client_received=10000/10000 server_resps=10000 codes={200=9900, 500=100} - uncommon_flakes[UNLIMITED_ROUND_ROBIN].txt: success=99.0% client_mean=PT0S server_cpu=PT0S client_received=10000/10000 server_resps=10000 codes={200=9900, 500=100} + all_nodes_500[CONCURRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].txt: success=52.6% client_mean=PT1.027869274S server_cpu=PT16M44.041614481S client_received=2000/2000 server_resps=1673 codes={200=1051, 500=622, Failed to make a request=327} + all_nodes_500[CONCURRENCY_LIMITER_PIN_UNTIL_ERROR].txt: success=50.0% client_mean=PT0.6S server_cpu=PT20M client_received=2000/2000 server_resps=2000 codes={200=1000, 500=1000} + all_nodes_500[CONCURRENCY_LIMITER_ROUND_ROBIN].txt: success=50.0% client_mean=PT0.6S server_cpu=PT20M client_received=2000/2000 server_resps=2000 codes={200=1000, 500=1000} + all_nodes_500[UNLIMITED_ROUND_ROBIN].txt: success=50.0% client_mean=PT0.6S server_cpu=PT20M client_received=2000/2000 server_resps=2000 codes={200=1000, 500=1000} + black_hole[CONCURRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].txt: success=89.9% client_mean=PT0.600286226S server_cpu=PT17M58.2S client_received=1797/2000 server_resps=1797 codes={200=1797} + black_hole[CONCURRENCY_LIMITER_PIN_UNTIL_ERROR].txt: success=88.7% client_mean=PT0.601752452S server_cpu=PT17M43.8S client_received=1773/2000 server_resps=1773 codes={200=1773} + black_hole[CONCURRENCY_LIMITER_ROUND_ROBIN].txt: success=89.9% client_mean=PT0.600286226S server_cpu=PT17M58.2S client_received=1797/2000 server_resps=1797 codes={200=1797} + black_hole[UNLIMITED_ROUND_ROBIN].txt: success=65.0% client_mean=PT0.6S server_cpu=PT12M59.4S client_received=1299/2000 server_resps=1299 codes={200=1299} + drastic_slowdown[CONCURRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].txt: success=100.0% client_mean=PT2.069939083S server_cpu=PT2H17M59.756333311S client_received=4000/4000 server_resps=4000 codes={200=4000} + drastic_slowdown[CONCURRENCY_LIMITER_PIN_UNTIL_ERROR].txt: success=100.0% client_mean=PT2.053746297S server_cpu=PT2H16M53.159562742S client_received=4000/4000 server_resps=4000 codes={200=4000} + drastic_slowdown[CONCURRENCY_LIMITER_ROUND_ROBIN].txt: success=100.0% client_mean=PT2.069939083S server_cpu=PT2H17M59.756333311S client_received=4000/4000 server_resps=4000 codes={200=4000} + drastic_slowdown[UNLIMITED_ROUND_ROBIN].txt: success=100.0% client_mean=PT8.353421749S server_cpu=PT9H16M53.686999978S client_received=4000/4000 server_resps=4000 codes={200=4000} + fast_500s_then_revert[CONCURRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].txt: success=98.1% client_mean=PT0.073222888S server_cpu=PT4M34.585833294S client_received=3750/3750 server_resps=3750 codes={200=3679, 500=71} + fast_500s_then_revert[CONCURRENCY_LIMITER_PIN_UNTIL_ERROR].txt: success=99.7% client_mean=PT0.080628266S server_cpu=PT5M2.355999997S client_received=3750/3750 server_resps=3750 codes={200=3739, 500=11} + fast_500s_then_revert[CONCURRENCY_LIMITER_ROUND_ROBIN].txt: success=76.7% client_mean=PT0.055463644S server_cpu=PT3M27.988666346S client_received=3750/3750 server_resps=3750 codes={200=2876, 500=874} + fast_500s_then_revert[UNLIMITED_ROUND_ROBIN].txt: success=76.7% client_mean=PT0.055463644S server_cpu=PT3M27.988666346S client_received=3750/3750 server_resps=3750 codes={200=2876, 500=874} + live_reloading[CONCURRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].txt: success=59.2% client_mean=PT2.707428425S server_cpu=PT1H20M58.880261641S client_received=2500/2500 server_resps=1860 codes={200=1480, 500=380, Failed to make a request=640} + live_reloading[CONCURRENCY_LIMITER_PIN_UNTIL_ERROR].txt: success=54.0% client_mean=PT2.903672277S server_cpu=PT1H42M35.433984304S client_received=2500/2500 server_resps=2177 codes={200=1349, 500=828, Failed to make a request=323} + live_reloading[CONCURRENCY_LIMITER_ROUND_ROBIN].txt: success=52.1% client_mean=PT2.908560636S server_cpu=PT1H42M58.764371458S client_received=2500/2500 server_resps=2193 codes={200=1302, 500=891, Failed to make a request=307} + live_reloading[UNLIMITED_ROUND_ROBIN].txt: success=58.4% client_mean=PT2.8396S server_cpu=PT1H58M19S client_received=2500/2500 server_resps=2500 codes={200=1461, 500=1039} +one_endpoint_dies_on_each_server[CONCURRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].txt: success=47.0% client_mean=PT1.284406604S server_cpu=PT15M41.717604108S client_received=2500/2500 server_resps=1569 codes={200=1176, 500=393, Failed to make a request=931} +one_endpoint_dies_on_each_server[CONCURRENCY_LIMITER_PIN_UNTIL_ERROR].txt: success=63.8% client_mean=PT0.601140264S server_cpu=PT25M client_received=2500/2500 server_resps=2500 codes={200=1595, 500=905} + one_endpoint_dies_on_each_server[CONCURRENCY_LIMITER_ROUND_ROBIN].txt: success=65.5% client_mean=PT0.6S server_cpu=PT25M client_received=2500/2500 server_resps=2500 codes={200=1638, 500=862} + one_endpoint_dies_on_each_server[UNLIMITED_ROUND_ROBIN].txt: success=65.5% client_mean=PT0.6S server_cpu=PT25M client_received=2500/2500 server_resps=2500 codes={200=1638, 500=862} + simplest_possible_case[CONCURRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].txt: success=100.0% client_mean=PT0.799939393S server_cpu=PT2H55M59.2S client_received=13200/13200 server_resps=13200 codes={200=13200} + simplest_possible_case[CONCURRENCY_LIMITER_PIN_UNTIL_ERROR].txt: success=100.0% client_mean=PT0.998696969S server_cpu=PT3H39M42.8S client_received=13200/13200 server_resps=13200 codes={200=13200} + simplest_possible_case[CONCURRENCY_LIMITER_ROUND_ROBIN].txt: success=100.0% client_mean=PT0.799939393S server_cpu=PT2H55M59.2S client_received=13200/13200 server_resps=13200 codes={200=13200} + simplest_possible_case[UNLIMITED_ROUND_ROBIN].txt: success=100.0% client_mean=PT0.799939393S server_cpu=PT2H55M59.2S client_received=13200/13200 server_resps=13200 codes={200=13200} + slow_503s_then_revert[CONCURRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].txt: success=100.0% client_mean=PT0.302783916S server_cpu=PT14M44.375304373S client_received=3000/3000 server_resps=3175 codes={200=3000} + slow_503s_then_revert[CONCURRENCY_LIMITER_PIN_UNTIL_ERROR].txt: success=100.0% client_mean=PT0.346572928S server_cpu=PT16M51.987159344S client_received=3000/3000 server_resps=3197 codes={200=3000} + slow_503s_then_revert[CONCURRENCY_LIMITER_ROUND_ROBIN].txt: success=100.0% client_mean=PT0.745544946S server_cpu=PT36M23.421729078S client_received=3000/3000 server_resps=3411 codes={200=3000} + slow_503s_then_revert[UNLIMITED_ROUND_ROBIN].txt: success=100.0% client_mean=PT1.43038075S server_cpu=PT1H9M49.707166439S client_received=3000/3000 server_resps=3802 codes={200=3000} +slowdown_and_error_thresholds[CONCURRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].txt: success=3.3% client_mean=PT2.227891278S server_cpu=PT1H40M10.840754535S client_received=10000/10000 server_resps=1617 codes={200=327, 500=1290, Failed to make a request=8383} +slowdown_and_error_thresholds[CONCURRENCY_LIMITER_PIN_UNTIL_ERROR].txt: success=1.2% client_mean=PT2.916010959S server_cpu=PT4H30M39.598982029S client_received=10000/10000 server_resps=4255 codes={200=120, 500=4135, Failed to make a request=5745} + slowdown_and_error_thresholds[CONCURRENCY_LIMITER_ROUND_ROBIN].txt: success=1.2% client_mean=PT2.990999813S server_cpu=PT4H41M15.507187288S client_received=10000/10000 server_resps=4433 codes={200=120, 500=4313, Failed to make a request=5567} + slowdown_and_error_thresholds[UNLIMITED_ROUND_ROBIN].txt: success=1.2% client_mean=PT3.974119999S server_cpu=PT11H2M21.19999998S client_received=10000/10000 server_resps=10000 codes={200=120, 500=9880} + uncommon_flakes[CONCURRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].txt: success=93.6% client_mean=PT0.210193295S server_cpu=PT6.063774874S client_received=9730/10000 server_resps=9456 codes={200=9363, 500=93, Failed to make a request=274} + uncommon_flakes[CONCURRENCY_LIMITER_PIN_UNTIL_ERROR].txt: success=99.0% client_mean=PT0.000001S server_cpu=PT0.01S client_received=10000/10000 server_resps=10000 codes={200=9902, 500=98} + uncommon_flakes[CONCURRENCY_LIMITER_ROUND_ROBIN].txt: success=99.0% client_mean=PT0.000001S server_cpu=PT0.01S client_received=10000/10000 server_resps=10000 codes={200=9900, 500=100} + uncommon_flakes[UNLIMITED_ROUND_ROBIN].txt: success=99.0% client_mean=PT0.000001S server_cpu=PT0.01S client_received=10000/10000 server_resps=10000 codes={200=9900, 500=100} ``` diff --git a/simulation/src/test/resources/simplest_possible_case[CONCURRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].png b/simulation/src/test/resources/simplest_possible_case[CONCURRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].png index 7d697adfe..ba39dc6ea 100644 --- a/simulation/src/test/resources/simplest_possible_case[CONCURRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].png +++ b/simulation/src/test/resources/simplest_possible_case[CONCURRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].png @@ -1,3 +1,3 @@ version https://git-lfs.github.com/spec/v1 -oid sha256:93bba21a142c67b4d969d3b29f4cf8560f07499f7427229e47ef79ad6775e4d3 -size 190694 +oid sha256:75ab5905eeb5edf37fc0075c62953fd9dbce73f6ca53837f7a552165616f5d7b +size 199770 diff --git a/simulation/src/test/resources/simplest_possible_case[CONCURRENCY_LIMITER_PIN_UNTIL_ERROR].png b/simulation/src/test/resources/simplest_possible_case[CONCURRENCY_LIMITER_PIN_UNTIL_ERROR].png index d7080b525..bee8d84dd 100644 --- a/simulation/src/test/resources/simplest_possible_case[CONCURRENCY_LIMITER_PIN_UNTIL_ERROR].png +++ b/simulation/src/test/resources/simplest_possible_case[CONCURRENCY_LIMITER_PIN_UNTIL_ERROR].png @@ -1,3 +1,3 @@ version https://git-lfs.github.com/spec/v1 -oid sha256:2d2c9396115c7f175d975ffc22ff3560a6be16616901b39a63a55f566c2430eb -size 84234 +oid sha256:a63d38543fbf49e275de3738fb3c463ace39bb5484ecf04ab0521ba30b71a5e0 +size 100223 diff --git a/simulation/src/test/resources/simplest_possible_case[CONCURRENCY_LIMITER_ROUND_ROBIN].png b/simulation/src/test/resources/simplest_possible_case[CONCURRENCY_LIMITER_ROUND_ROBIN].png index e7afae119..1640bbc59 100644 --- a/simulation/src/test/resources/simplest_possible_case[CONCURRENCY_LIMITER_ROUND_ROBIN].png +++ b/simulation/src/test/resources/simplest_possible_case[CONCURRENCY_LIMITER_ROUND_ROBIN].png @@ -1,3 +1,3 @@ version https://git-lfs.github.com/spec/v1 -oid sha256:7ec16b22a0d1f5c30d56663412e4e67f21369669cc8018c30129828fea7eb935 -size 190552 +oid sha256:48353dd885320ec53e1bc26ebfaa34a10aa4a6b46a96e9c7ef0284df417b6c6c +size 199683 diff --git a/simulation/src/test/resources/simplest_possible_case[UNLIMITED_ROUND_ROBIN].png b/simulation/src/test/resources/simplest_possible_case[UNLIMITED_ROUND_ROBIN].png index 3ae5cc324..1a8de60fa 100644 --- a/simulation/src/test/resources/simplest_possible_case[UNLIMITED_ROUND_ROBIN].png +++ b/simulation/src/test/resources/simplest_possible_case[UNLIMITED_ROUND_ROBIN].png @@ -1,3 +1,3 @@ version https://git-lfs.github.com/spec/v1 -oid sha256:e25182694fbbd715cb075798ecc59165901ffe77202b789ac6c787e32385ea79 -size 190371 +oid sha256:fee8d875ef5696bc9385cf0a8f0988520313218d2fb50afe83bda80c63bc0ba1 +size 198817 diff --git a/simulation/src/test/resources/slow_503s_then_revert[CONCURRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].png b/simulation/src/test/resources/slow_503s_then_revert[CONCURRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].png index a451b68bb..925348469 100644 --- a/simulation/src/test/resources/slow_503s_then_revert[CONCURRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].png +++ b/simulation/src/test/resources/slow_503s_then_revert[CONCURRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].png @@ -1,3 +1,3 @@ version https://git-lfs.github.com/spec/v1 -oid sha256:3636a9586208994f13967dc034ba4eaea07e08db77b6ba801b2b0de45599c61e -size 94680 +oid sha256:b3f233b8faf6b39ee52f94177e18c354c05d18311eb9fe57f0c1a5e26d3e40da +size 102677 diff --git a/simulation/src/test/resources/slow_503s_then_revert[CONCURRENCY_LIMITER_PIN_UNTIL_ERROR].png b/simulation/src/test/resources/slow_503s_then_revert[CONCURRENCY_LIMITER_PIN_UNTIL_ERROR].png index 9880964ad..7c718ec63 100644 --- a/simulation/src/test/resources/slow_503s_then_revert[CONCURRENCY_LIMITER_PIN_UNTIL_ERROR].png +++ b/simulation/src/test/resources/slow_503s_then_revert[CONCURRENCY_LIMITER_PIN_UNTIL_ERROR].png @@ -1,3 +1,3 @@ version https://git-lfs.github.com/spec/v1 -oid sha256:e0b066d38f44ab189d7bea2bc37928f5928be8e1b164d580aaa530c173444ca7 -size 88773 +oid sha256:6038e449309374e1b76fc60d4c1370c9a2988b1e46ba28dcf7ed318372bacb36 +size 97048 diff --git a/simulation/src/test/resources/slow_503s_then_revert[CONCURRENCY_LIMITER_ROUND_ROBIN].png b/simulation/src/test/resources/slow_503s_then_revert[CONCURRENCY_LIMITER_ROUND_ROBIN].png index 8385c6071..5d970fc49 100644 --- a/simulation/src/test/resources/slow_503s_then_revert[CONCURRENCY_LIMITER_ROUND_ROBIN].png +++ b/simulation/src/test/resources/slow_503s_then_revert[CONCURRENCY_LIMITER_ROUND_ROBIN].png @@ -1,3 +1,3 @@ version https://git-lfs.github.com/spec/v1 -oid sha256:153b09df70f5be8ba932b90b23cea6da963d09f9005054470eaf8f8300d0cd58 -size 92083 +oid sha256:984763e1a5ae4fa6993da2ea2ed4bdc26359129767c683468859ffe1e532a9bc +size 99932 diff --git a/simulation/src/test/resources/slow_503s_then_revert[UNLIMITED_ROUND_ROBIN].png b/simulation/src/test/resources/slow_503s_then_revert[UNLIMITED_ROUND_ROBIN].png index d57e55859..1ae063926 100644 --- a/simulation/src/test/resources/slow_503s_then_revert[UNLIMITED_ROUND_ROBIN].png +++ b/simulation/src/test/resources/slow_503s_then_revert[UNLIMITED_ROUND_ROBIN].png @@ -1,3 +1,3 @@ version https://git-lfs.github.com/spec/v1 -oid sha256:31700b0f6c307ebbae6582d7ecfc857fac4cc60291a0eb3362596da3efce6e0c -size 88793 +oid sha256:d40179c954a45a26f6f84bca1c066de7e3b3b614f434e2ee2d684cee23129fd6 +size 96740 diff --git a/simulation/src/test/resources/slowdown_and_error_thresholds[CONCURRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].png b/simulation/src/test/resources/slowdown_and_error_thresholds[CONCURRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].png index 771dcf649..4ffb3e9e2 100644 --- a/simulation/src/test/resources/slowdown_and_error_thresholds[CONCURRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].png +++ b/simulation/src/test/resources/slowdown_and_error_thresholds[CONCURRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].png @@ -1,3 +1,3 @@ version https://git-lfs.github.com/spec/v1 -oid sha256:ce9a0884e66131ca152ead9558d44db32b9bbae4c2f139e54ee7e48b40e2248f -size 131006 +oid sha256:fa46699f003e9244ba2b56ae137d276c8edf1e170871676875e985cd0f906fb0 +size 141184 diff --git a/simulation/src/test/resources/slowdown_and_error_thresholds[CONCURRENCY_LIMITER_PIN_UNTIL_ERROR].png b/simulation/src/test/resources/slowdown_and_error_thresholds[CONCURRENCY_LIMITER_PIN_UNTIL_ERROR].png index c5ead4f4d..76e853958 100644 --- a/simulation/src/test/resources/slowdown_and_error_thresholds[CONCURRENCY_LIMITER_PIN_UNTIL_ERROR].png +++ b/simulation/src/test/resources/slowdown_and_error_thresholds[CONCURRENCY_LIMITER_PIN_UNTIL_ERROR].png @@ -1,3 +1,3 @@ version https://git-lfs.github.com/spec/v1 -oid sha256:833f4f817657d7a0a597fbe30d5f20066225be58644b1041f8b5751061de43c7 -size 126043 +oid sha256:337014604e03f79e26284b124ddeb846b0d2b7b559204da16b367427996cb741 +size 133416 diff --git a/simulation/src/test/resources/slowdown_and_error_thresholds[CONCURRENCY_LIMITER_ROUND_ROBIN].png b/simulation/src/test/resources/slowdown_and_error_thresholds[CONCURRENCY_LIMITER_ROUND_ROBIN].png index 567ac1303..7cf2726ae 100644 --- a/simulation/src/test/resources/slowdown_and_error_thresholds[CONCURRENCY_LIMITER_ROUND_ROBIN].png +++ b/simulation/src/test/resources/slowdown_and_error_thresholds[CONCURRENCY_LIMITER_ROUND_ROBIN].png @@ -1,3 +1,3 @@ version https://git-lfs.github.com/spec/v1 -oid sha256:d8b084962ed20324ab05dd9f2b4d93261e9418ea9bba240278014b968ba0c382 -size 105756 +oid sha256:ded99e4cf11f4b35537791d501969bf8020145240b5241a62c2ccb3516d5d082 +size 116435 diff --git a/simulation/src/test/resources/slowdown_and_error_thresholds[UNLIMITED_ROUND_ROBIN].png b/simulation/src/test/resources/slowdown_and_error_thresholds[UNLIMITED_ROUND_ROBIN].png index 1d2115bde..c60a5221a 100644 --- a/simulation/src/test/resources/slowdown_and_error_thresholds[UNLIMITED_ROUND_ROBIN].png +++ b/simulation/src/test/resources/slowdown_and_error_thresholds[UNLIMITED_ROUND_ROBIN].png @@ -1,3 +1,3 @@ version https://git-lfs.github.com/spec/v1 -oid sha256:cc611164ef9124452fbb968cbdc4e96e9fdef17988aa52d379e6e5cbed100a50 -size 101684 +oid sha256:5a1e9ae3deacd69bb9f0cb02057f4ac7af5d1388557362ea8c7174336cf2b8db +size 110209 diff --git a/simulation/src/test/resources/txt/all_nodes_500[CONCURRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].txt b/simulation/src/test/resources/txt/all_nodes_500[CONCURRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].txt index ea29fd3ac..20efdd3c1 100644 --- a/simulation/src/test/resources/txt/all_nodes_500[CONCURRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].txt +++ b/simulation/src/test/resources/txt/all_nodes_500[CONCURRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].txt @@ -1 +1 @@ -success=52.2% client_mean=PT1.016145S server_cpu=PT16M24.9S client_received=2000/2000 server_resps=1672 codes={200=1043, 500=629, Failed to make a request=328} \ No newline at end of file +success=52.6% client_mean=PT1.027869274S server_cpu=PT16M44.041614481S client_received=2000/2000 server_resps=1673 codes={200=1051, 500=622, Failed to make a request=327} \ No newline at end of file diff --git a/simulation/src/test/resources/txt/all_nodes_500[CONCURRENCY_LIMITER_PIN_UNTIL_ERROR].txt b/simulation/src/test/resources/txt/all_nodes_500[CONCURRENCY_LIMITER_PIN_UNTIL_ERROR].txt index da96b9a82..21cae27db 100644 --- a/simulation/src/test/resources/txt/all_nodes_500[CONCURRENCY_LIMITER_PIN_UNTIL_ERROR].txt +++ b/simulation/src/test/resources/txt/all_nodes_500[CONCURRENCY_LIMITER_PIN_UNTIL_ERROR].txt @@ -1 +1 @@ -success=50.0% client_mean=PT0.59085S server_cpu=PT19M41.7S client_received=2000/2000 server_resps=2000 codes={200=1000, 500=1000} \ No newline at end of file +success=50.0% client_mean=PT0.6S server_cpu=PT20M client_received=2000/2000 server_resps=2000 codes={200=1000, 500=1000} \ No newline at end of file diff --git a/simulation/src/test/resources/txt/all_nodes_500[CONCURRENCY_LIMITER_ROUND_ROBIN].txt b/simulation/src/test/resources/txt/all_nodes_500[CONCURRENCY_LIMITER_ROUND_ROBIN].txt index da96b9a82..21cae27db 100644 --- a/simulation/src/test/resources/txt/all_nodes_500[CONCURRENCY_LIMITER_ROUND_ROBIN].txt +++ b/simulation/src/test/resources/txt/all_nodes_500[CONCURRENCY_LIMITER_ROUND_ROBIN].txt @@ -1 +1 @@ -success=50.0% client_mean=PT0.59085S server_cpu=PT19M41.7S client_received=2000/2000 server_resps=2000 codes={200=1000, 500=1000} \ No newline at end of file +success=50.0% client_mean=PT0.6S server_cpu=PT20M client_received=2000/2000 server_resps=2000 codes={200=1000, 500=1000} \ No newline at end of file diff --git a/simulation/src/test/resources/txt/all_nodes_500[UNLIMITED_ROUND_ROBIN].txt b/simulation/src/test/resources/txt/all_nodes_500[UNLIMITED_ROUND_ROBIN].txt index da96b9a82..21cae27db 100644 --- a/simulation/src/test/resources/txt/all_nodes_500[UNLIMITED_ROUND_ROBIN].txt +++ b/simulation/src/test/resources/txt/all_nodes_500[UNLIMITED_ROUND_ROBIN].txt @@ -1 +1 @@ -success=50.0% client_mean=PT0.59085S server_cpu=PT19M41.7S client_received=2000/2000 server_resps=2000 codes={200=1000, 500=1000} \ No newline at end of file +success=50.0% client_mean=PT0.6S server_cpu=PT20M client_received=2000/2000 server_resps=2000 codes={200=1000, 500=1000} \ No newline at end of file diff --git a/simulation/src/test/resources/txt/black_hole[CONCURRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].txt b/simulation/src/test/resources/txt/black_hole[CONCURRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].txt index 175d57d57..1a614c149 100644 --- a/simulation/src/test/resources/txt/black_hole[CONCURRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].txt +++ b/simulation/src/test/resources/txt/black_hole[CONCURRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].txt @@ -1 +1 @@ -success=89.9% client_mean=PT0.58008069S server_cpu=PT17M21.9S client_received=1797/2000 server_resps=1797 codes={200=1797} \ No newline at end of file +success=89.9% client_mean=PT0.600286226S server_cpu=PT17M58.2S client_received=1797/2000 server_resps=1797 codes={200=1797} \ No newline at end of file diff --git a/simulation/src/test/resources/txt/black_hole[CONCURRENCY_LIMITER_PIN_UNTIL_ERROR].txt b/simulation/src/test/resources/txt/black_hole[CONCURRENCY_LIMITER_PIN_UNTIL_ERROR].txt index 77f533c2a..733d3cc7d 100644 --- a/simulation/src/test/resources/txt/black_hole[CONCURRENCY_LIMITER_PIN_UNTIL_ERROR].txt +++ b/simulation/src/test/resources/txt/black_hole[CONCURRENCY_LIMITER_PIN_UNTIL_ERROR].txt @@ -1 +1 @@ -success=88.7% client_mean=PT0.581249294S server_cpu=PT17M7.5S client_received=1773/2000 server_resps=1773 codes={200=1773} \ No newline at end of file +success=88.7% client_mean=PT0.601752452S server_cpu=PT17M43.8S client_received=1773/2000 server_resps=1773 codes={200=1773} \ No newline at end of file diff --git a/simulation/src/test/resources/txt/black_hole[CONCURRENCY_LIMITER_ROUND_ROBIN].txt b/simulation/src/test/resources/txt/black_hole[CONCURRENCY_LIMITER_ROUND_ROBIN].txt index 175d57d57..1a614c149 100644 --- a/simulation/src/test/resources/txt/black_hole[CONCURRENCY_LIMITER_ROUND_ROBIN].txt +++ b/simulation/src/test/resources/txt/black_hole[CONCURRENCY_LIMITER_ROUND_ROBIN].txt @@ -1 +1 @@ -success=89.9% client_mean=PT0.58008069S server_cpu=PT17M21.9S client_received=1797/2000 server_resps=1797 codes={200=1797} \ No newline at end of file +success=89.9% client_mean=PT0.600286226S server_cpu=PT17M58.2S client_received=1797/2000 server_resps=1797 codes={200=1797} \ No newline at end of file diff --git a/simulation/src/test/resources/txt/black_hole[UNLIMITED_ROUND_ROBIN].txt b/simulation/src/test/resources/txt/black_hole[UNLIMITED_ROUND_ROBIN].txt index 32f0da954..4a2f468d0 100644 --- a/simulation/src/test/resources/txt/black_hole[UNLIMITED_ROUND_ROBIN].txt +++ b/simulation/src/test/resources/txt/black_hole[UNLIMITED_ROUND_ROBIN].txt @@ -1 +1 @@ -success=65.0% client_mean=PT0.586343341S server_cpu=PT12M41.66S client_received=1299/2000 server_resps=1299 codes={200=1299} \ No newline at end of file +success=65.0% client_mean=PT0.6S server_cpu=PT12M59.4S client_received=1299/2000 server_resps=1299 codes={200=1299} \ No newline at end of file diff --git a/simulation/src/test/resources/txt/drastic_slowdown[CONCURRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].txt b/simulation/src/test/resources/txt/drastic_slowdown[CONCURRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].txt index 8c1ea1ae6..6812e4deb 100644 --- a/simulation/src/test/resources/txt/drastic_slowdown[CONCURRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].txt +++ b/simulation/src/test/resources/txt/drastic_slowdown[CONCURRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].txt @@ -1 +1 @@ -success=100.0% client_mean=PT0.85580125S server_cpu=PT57M3.205S client_received=4000/4000 server_resps=4000 codes={200=4000} \ No newline at end of file +success=100.0% client_mean=PT2.069939083S server_cpu=PT2H17M59.756333311S client_received=4000/4000 server_resps=4000 codes={200=4000} \ No newline at end of file diff --git a/simulation/src/test/resources/txt/drastic_slowdown[CONCURRENCY_LIMITER_PIN_UNTIL_ERROR].txt b/simulation/src/test/resources/txt/drastic_slowdown[CONCURRENCY_LIMITER_PIN_UNTIL_ERROR].txt index 0db33de67..0a149cd99 100644 --- a/simulation/src/test/resources/txt/drastic_slowdown[CONCURRENCY_LIMITER_PIN_UNTIL_ERROR].txt +++ b/simulation/src/test/resources/txt/drastic_slowdown[CONCURRENCY_LIMITER_PIN_UNTIL_ERROR].txt @@ -1 +1 @@ -success=100.0% client_mean=PT0.86496875S server_cpu=PT57M38.075S client_received=4000/4000 server_resps=4000 codes={200=4000} \ No newline at end of file +success=100.0% client_mean=PT2.053746297S server_cpu=PT2H16M53.159562742S client_received=4000/4000 server_resps=4000 codes={200=4000} \ No newline at end of file diff --git a/simulation/src/test/resources/txt/drastic_slowdown[CONCURRENCY_LIMITER_ROUND_ROBIN].txt b/simulation/src/test/resources/txt/drastic_slowdown[CONCURRENCY_LIMITER_ROUND_ROBIN].txt index 8c1ea1ae6..6812e4deb 100644 --- a/simulation/src/test/resources/txt/drastic_slowdown[CONCURRENCY_LIMITER_ROUND_ROBIN].txt +++ b/simulation/src/test/resources/txt/drastic_slowdown[CONCURRENCY_LIMITER_ROUND_ROBIN].txt @@ -1 +1 @@ -success=100.0% client_mean=PT0.85580125S server_cpu=PT57M3.205S client_received=4000/4000 server_resps=4000 codes={200=4000} \ No newline at end of file +success=100.0% client_mean=PT2.069939083S server_cpu=PT2H17M59.756333311S client_received=4000/4000 server_resps=4000 codes={200=4000} \ No newline at end of file diff --git a/simulation/src/test/resources/txt/drastic_slowdown[UNLIMITED_ROUND_ROBIN].txt b/simulation/src/test/resources/txt/drastic_slowdown[UNLIMITED_ROUND_ROBIN].txt index 85c13a532..a129c4e00 100644 --- a/simulation/src/test/resources/txt/drastic_slowdown[UNLIMITED_ROUND_ROBIN].txt +++ b/simulation/src/test/resources/txt/drastic_slowdown[UNLIMITED_ROUND_ROBIN].txt @@ -1 +1 @@ -success=100.0% client_mean=PT2.44372125S server_cpu=PT2H42M54.885S client_received=4000/4000 server_resps=4000 codes={200=4000} \ No newline at end of file +success=100.0% client_mean=PT8.353421749S server_cpu=PT9H16M53.686999978S client_received=4000/4000 server_resps=4000 codes={200=4000} \ No newline at end of file diff --git a/simulation/src/test/resources/txt/fast_500s_then_revert[CONCURRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].txt b/simulation/src/test/resources/txt/fast_500s_then_revert[CONCURRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].txt index c61146e83..694839c1d 100644 --- a/simulation/src/test/resources/txt/fast_500s_then_revert[CONCURRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].txt +++ b/simulation/src/test/resources/txt/fast_500s_then_revert[CONCURRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].txt @@ -1 +1 @@ -success=98.1% client_mean=PT0.071725866S server_cpu=PT4M28.972S client_received=3750/3750 server_resps=3750 codes={200=3679, 500=71} \ No newline at end of file +success=98.1% client_mean=PT0.073222888S server_cpu=PT4M34.585833294S client_received=3750/3750 server_resps=3750 codes={200=3679, 500=71} \ No newline at end of file diff --git a/simulation/src/test/resources/txt/fast_500s_then_revert[CONCURRENCY_LIMITER_PIN_UNTIL_ERROR].txt b/simulation/src/test/resources/txt/fast_500s_then_revert[CONCURRENCY_LIMITER_PIN_UNTIL_ERROR].txt index e34c29e1e..8077268c6 100644 --- a/simulation/src/test/resources/txt/fast_500s_then_revert[CONCURRENCY_LIMITER_PIN_UNTIL_ERROR].txt +++ b/simulation/src/test/resources/txt/fast_500s_then_revert[CONCURRENCY_LIMITER_PIN_UNTIL_ERROR].txt @@ -1 +1 @@ -success=99.7% client_mean=PT0.079397333S server_cpu=PT4M57.74S client_received=3750/3750 server_resps=3750 codes={200=3739, 500=11} \ No newline at end of file +success=99.7% client_mean=PT0.080628266S server_cpu=PT5M2.355999997S client_received=3750/3750 server_resps=3750 codes={200=3739, 500=11} \ No newline at end of file diff --git a/simulation/src/test/resources/txt/fast_500s_then_revert[CONCURRENCY_LIMITER_ROUND_ROBIN].txt b/simulation/src/test/resources/txt/fast_500s_then_revert[CONCURRENCY_LIMITER_ROUND_ROBIN].txt index fb5cc428a..5f71a3396 100644 --- a/simulation/src/test/resources/txt/fast_500s_then_revert[CONCURRENCY_LIMITER_ROUND_ROBIN].txt +++ b/simulation/src/test/resources/txt/fast_500s_then_revert[CONCURRENCY_LIMITER_ROUND_ROBIN].txt @@ -1 +1 @@ -success=76.7% client_mean=PT0.053707733S server_cpu=PT3M21.404S client_received=3750/3750 server_resps=3750 codes={200=2876, 500=874} \ No newline at end of file +success=76.7% client_mean=PT0.055463644S server_cpu=PT3M27.988666346S client_received=3750/3750 server_resps=3750 codes={200=2876, 500=874} \ No newline at end of file diff --git a/simulation/src/test/resources/txt/fast_500s_then_revert[UNLIMITED_ROUND_ROBIN].txt b/simulation/src/test/resources/txt/fast_500s_then_revert[UNLIMITED_ROUND_ROBIN].txt index fb5cc428a..5f71a3396 100644 --- a/simulation/src/test/resources/txt/fast_500s_then_revert[UNLIMITED_ROUND_ROBIN].txt +++ b/simulation/src/test/resources/txt/fast_500s_then_revert[UNLIMITED_ROUND_ROBIN].txt @@ -1 +1 @@ -success=76.7% client_mean=PT0.053707733S server_cpu=PT3M21.404S client_received=3750/3750 server_resps=3750 codes={200=2876, 500=874} \ No newline at end of file +success=76.7% client_mean=PT0.055463644S server_cpu=PT3M27.988666346S client_received=3750/3750 server_resps=3750 codes={200=2876, 500=874} \ No newline at end of file diff --git a/simulation/src/test/resources/txt/live_reloading[CONCURRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].txt b/simulation/src/test/resources/txt/live_reloading[CONCURRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].txt index eb31bc3dd..422b786ca 100644 --- a/simulation/src/test/resources/txt/live_reloading[CONCURRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].txt +++ b/simulation/src/test/resources/txt/live_reloading[CONCURRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].txt @@ -1 +1 @@ -success=59.5% client_mean=PT2.1742928S server_cpu=PT1H1M39.384S client_received=2500/2500 server_resps=1832 codes={200=1488, 500=344, Failed to make a request=668} \ No newline at end of file +success=59.2% client_mean=PT2.707428425S server_cpu=PT1H20M58.880261641S client_received=2500/2500 server_resps=1860 codes={200=1480, 500=380, Failed to make a request=640} \ No newline at end of file diff --git a/simulation/src/test/resources/txt/live_reloading[CONCURRENCY_LIMITER_PIN_UNTIL_ERROR].txt b/simulation/src/test/resources/txt/live_reloading[CONCURRENCY_LIMITER_PIN_UNTIL_ERROR].txt index 4507301ce..e7fefd8d4 100644 --- a/simulation/src/test/resources/txt/live_reloading[CONCURRENCY_LIMITER_PIN_UNTIL_ERROR].txt +++ b/simulation/src/test/resources/txt/live_reloading[CONCURRENCY_LIMITER_PIN_UNTIL_ERROR].txt @@ -1 +1 @@ -success=53.8% client_mean=PT2.3703696S server_cpu=PT1H21M50.076S client_received=2500/2500 server_resps=2176 codes={200=1346, 500=830, Failed to make a request=324} \ No newline at end of file +success=54.0% client_mean=PT2.903672277S server_cpu=PT1H42M35.433984304S client_received=2500/2500 server_resps=2177 codes={200=1349, 500=828, Failed to make a request=323} \ No newline at end of file diff --git a/simulation/src/test/resources/txt/live_reloading[CONCURRENCY_LIMITER_ROUND_ROBIN].txt b/simulation/src/test/resources/txt/live_reloading[CONCURRENCY_LIMITER_ROUND_ROBIN].txt index f3f236699..928d0f0c6 100644 --- a/simulation/src/test/resources/txt/live_reloading[CONCURRENCY_LIMITER_ROUND_ROBIN].txt +++ b/simulation/src/test/resources/txt/live_reloading[CONCURRENCY_LIMITER_ROUND_ROBIN].txt @@ -1 +1 @@ -success=52.1% client_mean=PT2.3660912S server_cpu=PT1H21M12.564S client_received=2500/2500 server_resps=2193 codes={200=1302, 500=891, Failed to make a request=307} \ No newline at end of file +success=52.1% client_mean=PT2.908560636S server_cpu=PT1H42M58.764371458S client_received=2500/2500 server_resps=2193 codes={200=1302, 500=891, Failed to make a request=307} \ No newline at end of file diff --git a/simulation/src/test/resources/txt/live_reloading[UNLIMITED_ROUND_ROBIN].txt b/simulation/src/test/resources/txt/live_reloading[UNLIMITED_ROUND_ROBIN].txt index 61780b182..e289c4621 100644 --- a/simulation/src/test/resources/txt/live_reloading[UNLIMITED_ROUND_ROBIN].txt +++ b/simulation/src/test/resources/txt/live_reloading[UNLIMITED_ROUND_ROBIN].txt @@ -1 +1 @@ -success=58.4% client_mean=PT2.3889216S server_cpu=PT1H39M32.304S client_received=2500/2500 server_resps=2500 codes={200=1461, 500=1039} \ No newline at end of file +success=58.4% client_mean=PT2.8396S server_cpu=PT1H58M19S client_received=2500/2500 server_resps=2500 codes={200=1461, 500=1039} \ No newline at end of file diff --git a/simulation/src/test/resources/txt/one_endpoint_dies_on_each_server[CONCURRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].txt b/simulation/src/test/resources/txt/one_endpoint_dies_on_each_server[CONCURRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].txt index 5a9f1ee59..2df5e16dd 100644 --- a/simulation/src/test/resources/txt/one_endpoint_dies_on_each_server[CONCURRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].txt +++ b/simulation/src/test/resources/txt/one_endpoint_dies_on_each_server[CONCURRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].txt @@ -1 +1 @@ -success=44.1% client_mean=PT1.0998096S server_cpu=PT13M25.476S client_received=2500/2500 server_resps=1426 codes={200=1103, 500=323, Failed to make a request=1074} \ No newline at end of file +success=47.0% client_mean=PT1.284406604S server_cpu=PT15M41.717604108S client_received=2500/2500 server_resps=1569 codes={200=1176, 500=393, Failed to make a request=931} \ No newline at end of file diff --git a/simulation/src/test/resources/txt/one_endpoint_dies_on_each_server[CONCURRENCY_LIMITER_PIN_UNTIL_ERROR].txt b/simulation/src/test/resources/txt/one_endpoint_dies_on_each_server[CONCURRENCY_LIMITER_PIN_UNTIL_ERROR].txt index 8853e6b16..fdf15dc60 100644 --- a/simulation/src/test/resources/txt/one_endpoint_dies_on_each_server[CONCURRENCY_LIMITER_PIN_UNTIL_ERROR].txt +++ b/simulation/src/test/resources/txt/one_endpoint_dies_on_each_server[CONCURRENCY_LIMITER_PIN_UNTIL_ERROR].txt @@ -1 +1 @@ -success=63.8% client_mean=PT0.5829248S server_cpu=PT24M14.688S client_received=2500/2500 server_resps=2500 codes={200=1595, 500=905} \ No newline at end of file +success=63.8% client_mean=PT0.601140264S server_cpu=PT25M client_received=2500/2500 server_resps=2500 codes={200=1595, 500=905} \ No newline at end of file diff --git a/simulation/src/test/resources/txt/one_endpoint_dies_on_each_server[CONCURRENCY_LIMITER_ROUND_ROBIN].txt b/simulation/src/test/resources/txt/one_endpoint_dies_on_each_server[CONCURRENCY_LIMITER_ROUND_ROBIN].txt index 06793cc99..6e102680e 100644 --- a/simulation/src/test/resources/txt/one_endpoint_dies_on_each_server[CONCURRENCY_LIMITER_ROUND_ROBIN].txt +++ b/simulation/src/test/resources/txt/one_endpoint_dies_on_each_server[CONCURRENCY_LIMITER_ROUND_ROBIN].txt @@ -1 +1 @@ -success=65.5% client_mean=PT0.58188S server_cpu=PT24M14.7S client_received=2500/2500 server_resps=2500 codes={200=1638, 500=862} \ No newline at end of file +success=65.5% client_mean=PT0.6S server_cpu=PT25M client_received=2500/2500 server_resps=2500 codes={200=1638, 500=862} \ No newline at end of file diff --git a/simulation/src/test/resources/txt/one_endpoint_dies_on_each_server[UNLIMITED_ROUND_ROBIN].txt b/simulation/src/test/resources/txt/one_endpoint_dies_on_each_server[UNLIMITED_ROUND_ROBIN].txt index 06793cc99..6e102680e 100644 --- a/simulation/src/test/resources/txt/one_endpoint_dies_on_each_server[UNLIMITED_ROUND_ROBIN].txt +++ b/simulation/src/test/resources/txt/one_endpoint_dies_on_each_server[UNLIMITED_ROUND_ROBIN].txt @@ -1 +1 @@ -success=65.5% client_mean=PT0.58188S server_cpu=PT24M14.7S client_received=2500/2500 server_resps=2500 codes={200=1638, 500=862} \ No newline at end of file +success=65.5% client_mean=PT0.6S server_cpu=PT25M client_received=2500/2500 server_resps=2500 codes={200=1638, 500=862} \ No newline at end of file diff --git a/simulation/src/test/resources/txt/simplest_possible_case[CONCURRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].txt b/simulation/src/test/resources/txt/simplest_possible_case[CONCURRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].txt index 273eb5824..68dc39802 100644 --- a/simulation/src/test/resources/txt/simplest_possible_case[CONCURRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].txt +++ b/simulation/src/test/resources/txt/simplest_possible_case[CONCURRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].txt @@ -1 +1 @@ -success=100.0% client_mean=PT0.757203849S server_cpu=PT2H46M35.09080914S client_received=13200/13200 server_resps=13200 codes={200=13200} \ No newline at end of file +success=100.0% client_mean=PT0.799939393S server_cpu=PT2H55M59.2S client_received=13200/13200 server_resps=13200 codes={200=13200} \ No newline at end of file diff --git a/simulation/src/test/resources/txt/simplest_possible_case[CONCURRENCY_LIMITER_PIN_UNTIL_ERROR].txt b/simulation/src/test/resources/txt/simplest_possible_case[CONCURRENCY_LIMITER_PIN_UNTIL_ERROR].txt index ef0f3cc04..363d31abd 100644 --- a/simulation/src/test/resources/txt/simplest_possible_case[CONCURRENCY_LIMITER_PIN_UNTIL_ERROR].txt +++ b/simulation/src/test/resources/txt/simplest_possible_case[CONCURRENCY_LIMITER_PIN_UNTIL_ERROR].txt @@ -1 +1 @@ -success=100.0% client_mean=PT0.997975196S server_cpu=PT3H39M33.27259554S client_received=13200/13200 server_resps=13200 codes={200=13200} \ No newline at end of file +success=100.0% client_mean=PT0.998696969S server_cpu=PT3H39M42.8S client_received=13200/13200 server_resps=13200 codes={200=13200} \ No newline at end of file diff --git a/simulation/src/test/resources/txt/simplest_possible_case[CONCURRENCY_LIMITER_ROUND_ROBIN].txt b/simulation/src/test/resources/txt/simplest_possible_case[CONCURRENCY_LIMITER_ROUND_ROBIN].txt index 273eb5824..68dc39802 100644 --- a/simulation/src/test/resources/txt/simplest_possible_case[CONCURRENCY_LIMITER_ROUND_ROBIN].txt +++ b/simulation/src/test/resources/txt/simplest_possible_case[CONCURRENCY_LIMITER_ROUND_ROBIN].txt @@ -1 +1 @@ -success=100.0% client_mean=PT0.757203849S server_cpu=PT2H46M35.09080914S client_received=13200/13200 server_resps=13200 codes={200=13200} \ No newline at end of file +success=100.0% client_mean=PT0.799939393S server_cpu=PT2H55M59.2S client_received=13200/13200 server_resps=13200 codes={200=13200} \ No newline at end of file diff --git a/simulation/src/test/resources/txt/simplest_possible_case[UNLIMITED_ROUND_ROBIN].txt b/simulation/src/test/resources/txt/simplest_possible_case[UNLIMITED_ROUND_ROBIN].txt index 273eb5824..68dc39802 100644 --- a/simulation/src/test/resources/txt/simplest_possible_case[UNLIMITED_ROUND_ROBIN].txt +++ b/simulation/src/test/resources/txt/simplest_possible_case[UNLIMITED_ROUND_ROBIN].txt @@ -1 +1 @@ -success=100.0% client_mean=PT0.757203849S server_cpu=PT2H46M35.09080914S client_received=13200/13200 server_resps=13200 codes={200=13200} \ No newline at end of file +success=100.0% client_mean=PT0.799939393S server_cpu=PT2H55M59.2S client_received=13200/13200 server_resps=13200 codes={200=13200} \ No newline at end of file diff --git a/simulation/src/test/resources/txt/slow_503s_then_revert[CONCURRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].txt b/simulation/src/test/resources/txt/slow_503s_then_revert[CONCURRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].txt index 5ef4258a7..9a05293ca 100644 --- a/simulation/src/test/resources/txt/slow_503s_then_revert[CONCURRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].txt +++ b/simulation/src/test/resources/txt/slow_503s_then_revert[CONCURRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].txt @@ -1 +1 @@ -success=100.0% client_mean=PT0.30077S server_cpu=PT14M38.66S client_received=3000/3000 server_resps=3175 codes={200=3000} \ No newline at end of file +success=100.0% client_mean=PT0.302783916S server_cpu=PT14M44.375304373S client_received=3000/3000 server_resps=3175 codes={200=3000} \ No newline at end of file diff --git a/simulation/src/test/resources/txt/slow_503s_then_revert[CONCURRENCY_LIMITER_PIN_UNTIL_ERROR].txt b/simulation/src/test/resources/txt/slow_503s_then_revert[CONCURRENCY_LIMITER_PIN_UNTIL_ERROR].txt index 4ac527b5f..2e3aa5ba1 100644 --- a/simulation/src/test/resources/txt/slow_503s_then_revert[CONCURRENCY_LIMITER_PIN_UNTIL_ERROR].txt +++ b/simulation/src/test/resources/txt/slow_503s_then_revert[CONCURRENCY_LIMITER_PIN_UNTIL_ERROR].txt @@ -1 +1 @@ -success=100.0% client_mean=PT0.345103333S server_cpu=PT16M47.965S client_received=3000/3000 server_resps=3197 codes={200=3000} \ No newline at end of file +success=100.0% client_mean=PT0.346572928S server_cpu=PT16M51.987159344S client_received=3000/3000 server_resps=3197 codes={200=3000} \ No newline at end of file diff --git a/simulation/src/test/resources/txt/slow_503s_then_revert[CONCURRENCY_LIMITER_ROUND_ROBIN].txt b/simulation/src/test/resources/txt/slow_503s_then_revert[CONCURRENCY_LIMITER_ROUND_ROBIN].txt index bf995233d..39248acea 100644 --- a/simulation/src/test/resources/txt/slow_503s_then_revert[CONCURRENCY_LIMITER_ROUND_ROBIN].txt +++ b/simulation/src/test/resources/txt/slow_503s_then_revert[CONCURRENCY_LIMITER_ROUND_ROBIN].txt @@ -1 +1 @@ -success=100.0% client_mean=PT0.741096666S server_cpu=PT36M14.075S client_received=3000/3000 server_resps=3411 codes={200=3000} \ No newline at end of file +success=100.0% client_mean=PT0.745544946S server_cpu=PT36M23.421729078S client_received=3000/3000 server_resps=3411 codes={200=3000} \ No newline at end of file diff --git a/simulation/src/test/resources/txt/slow_503s_then_revert[UNLIMITED_ROUND_ROBIN].txt b/simulation/src/test/resources/txt/slow_503s_then_revert[UNLIMITED_ROUND_ROBIN].txt index 013ac0362..5cfeced77 100644 --- a/simulation/src/test/resources/txt/slow_503s_then_revert[UNLIMITED_ROUND_ROBIN].txt +++ b/simulation/src/test/resources/txt/slow_503s_then_revert[UNLIMITED_ROUND_ROBIN].txt @@ -1 +1 @@ -success=100.0% client_mean=PT1.423741666S server_cpu=PT1H9M35.875S client_received=3000/3000 server_resps=3802 codes={200=3000} \ No newline at end of file +success=100.0% client_mean=PT1.43038075S server_cpu=PT1H9M49.707166439S client_received=3000/3000 server_resps=3802 codes={200=3000} \ No newline at end of file diff --git a/simulation/src/test/resources/txt/slowdown_and_error_thresholds[CONCURRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].txt b/simulation/src/test/resources/txt/slowdown_and_error_thresholds[CONCURRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].txt index 5f1cea526..43e6c7f90 100644 --- a/simulation/src/test/resources/txt/slowdown_and_error_thresholds[CONCURRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].txt +++ b/simulation/src/test/resources/txt/slowdown_and_error_thresholds[CONCURRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].txt @@ -1 +1 @@ -success=3.7% client_mean=PT2.0471542S server_cpu=PT1H26M34.726S client_received=10000/10000 server_resps=1614 codes={200=370, 500=1244, Failed to make a request=8386} \ No newline at end of file +success=3.3% client_mean=PT2.227891278S server_cpu=PT1H40M10.840754535S client_received=10000/10000 server_resps=1617 codes={200=327, 500=1290, Failed to make a request=8383} \ No newline at end of file diff --git a/simulation/src/test/resources/txt/slowdown_and_error_thresholds[CONCURRENCY_LIMITER_PIN_UNTIL_ERROR].txt b/simulation/src/test/resources/txt/slowdown_and_error_thresholds[CONCURRENCY_LIMITER_PIN_UNTIL_ERROR].txt index 5ab2395e2..75f8cb42f 100644 --- a/simulation/src/test/resources/txt/slowdown_and_error_thresholds[CONCURRENCY_LIMITER_PIN_UNTIL_ERROR].txt +++ b/simulation/src/test/resources/txt/slowdown_and_error_thresholds[CONCURRENCY_LIMITER_PIN_UNTIL_ERROR].txt @@ -1 +1 @@ -success=1.2% client_mean=PT2.5592244S server_cpu=PT3H42M1.968S client_received=10000/10000 server_resps=4248 codes={200=121, 500=4127, Failed to make a request=5752} \ No newline at end of file +success=1.2% client_mean=PT2.916010959S server_cpu=PT4H30M39.598982029S client_received=10000/10000 server_resps=4255 codes={200=120, 500=4135, Failed to make a request=5745} \ No newline at end of file diff --git a/simulation/src/test/resources/txt/slowdown_and_error_thresholds[CONCURRENCY_LIMITER_ROUND_ROBIN].txt b/simulation/src/test/resources/txt/slowdown_and_error_thresholds[CONCURRENCY_LIMITER_ROUND_ROBIN].txt index 6d7c95429..281fc3f15 100644 --- a/simulation/src/test/resources/txt/slowdown_and_error_thresholds[CONCURRENCY_LIMITER_ROUND_ROBIN].txt +++ b/simulation/src/test/resources/txt/slowdown_and_error_thresholds[CONCURRENCY_LIMITER_ROUND_ROBIN].txt @@ -1 +1 @@ -success=1.2% client_mean=PT2.57855S server_cpu=PT3H43M45.466S client_received=10000/10000 server_resps=4432 codes={200=120, 500=4312, Failed to make a request=5568} \ No newline at end of file +success=1.2% client_mean=PT2.990999813S server_cpu=PT4H41M15.507187288S client_received=10000/10000 server_resps=4433 codes={200=120, 500=4313, Failed to make a request=5567} \ No newline at end of file diff --git a/simulation/src/test/resources/txt/slowdown_and_error_thresholds[UNLIMITED_ROUND_ROBIN].txt b/simulation/src/test/resources/txt/slowdown_and_error_thresholds[UNLIMITED_ROUND_ROBIN].txt index eb1f1413c..319ba6acb 100644 --- a/simulation/src/test/resources/txt/slowdown_and_error_thresholds[UNLIMITED_ROUND_ROBIN].txt +++ b/simulation/src/test/resources/txt/slowdown_and_error_thresholds[UNLIMITED_ROUND_ROBIN].txt @@ -1 +1 @@ -success=1.2% client_mean=PT3.5573984S server_cpu=PT9H52M53.984S client_received=10000/10000 server_resps=10000 codes={200=120, 500=9880} \ No newline at end of file +success=1.2% client_mean=PT3.974119999S server_cpu=PT11H2M21.19999998S client_received=10000/10000 server_resps=10000 codes={200=120, 500=9880} \ No newline at end of file diff --git a/simulation/src/test/resources/txt/uncommon_flakes[CONCURRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].txt b/simulation/src/test/resources/txt/uncommon_flakes[CONCURRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].txt index c1c24eb80..6a61f1195 100644 --- a/simulation/src/test/resources/txt/uncommon_flakes[CONCURRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].txt +++ b/simulation/src/test/resources/txt/uncommon_flakes[CONCURRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].txt @@ -1 +1 @@ -success=94.1% client_mean=PT0.207311264S server_cpu=PT0S client_received=9712/10000 server_resps=9499 codes={200=9405, 500=94, Failed to make a request=213} \ No newline at end of file +success=93.6% client_mean=PT0.210193295S server_cpu=PT6.063774874S client_received=9730/10000 server_resps=9456 codes={200=9363, 500=93, Failed to make a request=274} \ No newline at end of file diff --git a/simulation/src/test/resources/txt/uncommon_flakes[CONCURRENCY_LIMITER_PIN_UNTIL_ERROR].txt b/simulation/src/test/resources/txt/uncommon_flakes[CONCURRENCY_LIMITER_PIN_UNTIL_ERROR].txt index eff571464..ff3d253c9 100644 --- a/simulation/src/test/resources/txt/uncommon_flakes[CONCURRENCY_LIMITER_PIN_UNTIL_ERROR].txt +++ b/simulation/src/test/resources/txt/uncommon_flakes[CONCURRENCY_LIMITER_PIN_UNTIL_ERROR].txt @@ -1 +1 @@ -success=99.0% client_mean=PT0S server_cpu=PT0S client_received=10000/10000 server_resps=10000 codes={200=9902, 500=98} \ No newline at end of file +success=99.0% client_mean=PT0.000001S server_cpu=PT0.01S client_received=10000/10000 server_resps=10000 codes={200=9902, 500=98} \ No newline at end of file diff --git a/simulation/src/test/resources/txt/uncommon_flakes[CONCURRENCY_LIMITER_ROUND_ROBIN].txt b/simulation/src/test/resources/txt/uncommon_flakes[CONCURRENCY_LIMITER_ROUND_ROBIN].txt index 8ad9b33ff..0da17a935 100644 --- a/simulation/src/test/resources/txt/uncommon_flakes[CONCURRENCY_LIMITER_ROUND_ROBIN].txt +++ b/simulation/src/test/resources/txt/uncommon_flakes[CONCURRENCY_LIMITER_ROUND_ROBIN].txt @@ -1 +1 @@ -success=99.0% client_mean=PT0S server_cpu=PT0S client_received=10000/10000 server_resps=10000 codes={200=9900, 500=100} \ No newline at end of file +success=99.0% client_mean=PT0.000001S server_cpu=PT0.01S client_received=10000/10000 server_resps=10000 codes={200=9900, 500=100} \ No newline at end of file diff --git a/simulation/src/test/resources/txt/uncommon_flakes[UNLIMITED_ROUND_ROBIN].txt b/simulation/src/test/resources/txt/uncommon_flakes[UNLIMITED_ROUND_ROBIN].txt index 8ad9b33ff..0da17a935 100644 --- a/simulation/src/test/resources/txt/uncommon_flakes[UNLIMITED_ROUND_ROBIN].txt +++ b/simulation/src/test/resources/txt/uncommon_flakes[UNLIMITED_ROUND_ROBIN].txt @@ -1 +1 @@ -success=99.0% client_mean=PT0S server_cpu=PT0S client_received=10000/10000 server_resps=10000 codes={200=9900, 500=100} \ No newline at end of file +success=99.0% client_mean=PT0.000001S server_cpu=PT0.01S client_received=10000/10000 server_resps=10000 codes={200=9900, 500=100} \ No newline at end of file diff --git a/simulation/src/test/resources/uncommon_flakes[CONCURRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].png b/simulation/src/test/resources/uncommon_flakes[CONCURRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].png index 9fdc8d489..1dcf74585 100644 --- a/simulation/src/test/resources/uncommon_flakes[CONCURRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].png +++ b/simulation/src/test/resources/uncommon_flakes[CONCURRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].png @@ -1,3 +1,3 @@ version https://git-lfs.github.com/spec/v1 -oid sha256:0b42d3b4831c79ba983ff4e35a2034a240914e28d53acb3fd2994804b302272e -size 181233 +oid sha256:a19bf7e56543297a68014f98765b86207631ce6e63d32c6d9bd5edd13ef6b78c +size 195134 diff --git a/simulation/src/test/resources/uncommon_flakes[CONCURRENCY_LIMITER_PIN_UNTIL_ERROR].png b/simulation/src/test/resources/uncommon_flakes[CONCURRENCY_LIMITER_PIN_UNTIL_ERROR].png index 77f608733..8486f89c5 100644 --- a/simulation/src/test/resources/uncommon_flakes[CONCURRENCY_LIMITER_PIN_UNTIL_ERROR].png +++ b/simulation/src/test/resources/uncommon_flakes[CONCURRENCY_LIMITER_PIN_UNTIL_ERROR].png @@ -1,3 +1,3 @@ version https://git-lfs.github.com/spec/v1 -oid sha256:d516991606a31781d0e2ce8010f0b1135c6e50275b249e81b7b8f2fce6ecb425 -size 159060 +oid sha256:b295a6d609022f7d5b9c612483d85dba4297afdf2ca90828b2743a45c01d5dc5 +size 164347 diff --git a/simulation/src/test/resources/uncommon_flakes[CONCURRENCY_LIMITER_ROUND_ROBIN].png b/simulation/src/test/resources/uncommon_flakes[CONCURRENCY_LIMITER_ROUND_ROBIN].png index 63cc86c05..0002da436 100644 --- a/simulation/src/test/resources/uncommon_flakes[CONCURRENCY_LIMITER_ROUND_ROBIN].png +++ b/simulation/src/test/resources/uncommon_flakes[CONCURRENCY_LIMITER_ROUND_ROBIN].png @@ -1,3 +1,3 @@ version https://git-lfs.github.com/spec/v1 -oid sha256:0ac7eae26070636698baf3b859e29b4c527e1707c67459c577671b77528607b7 -size 191237 +oid sha256:3f369eb0ea53482757807683dfcdefd9dfeb19dd743ce052ab1727a021f1f179 +size 197139 diff --git a/simulation/src/test/resources/uncommon_flakes[UNLIMITED_ROUND_ROBIN].png b/simulation/src/test/resources/uncommon_flakes[UNLIMITED_ROUND_ROBIN].png index 2a3c15dc7..f15e745ee 100644 --- a/simulation/src/test/resources/uncommon_flakes[UNLIMITED_ROUND_ROBIN].png +++ b/simulation/src/test/resources/uncommon_flakes[UNLIMITED_ROUND_ROBIN].png @@ -1,3 +1,3 @@ version https://git-lfs.github.com/spec/v1 -oid sha256:e1cc501073dccde2d69dbdd11bf0066d59f873f7c16b5f96c0390ee0c9181a59 -size 190722 +oid sha256:aa35570b89c8a862377f7a470b2ddf6b9d31f32977f60f3758d02f70fef0aae6 +size 196040 From f036e9deb9d3552adb412b2d31ef29574cbcd8c0 Mon Sep 17 00:00:00 2001 From: Carter Kozak Date: Wed, 26 Feb 2020 09:08:42 -0500 Subject: [PATCH 11/17] remove old queue metrics --- .../src/main/metrics/dialogue-core-metrics.yml | 10 ---------- 1 file changed, 10 deletions(-) diff --git a/dialogue-core/src/main/metrics/dialogue-core-metrics.yml b/dialogue-core/src/main/metrics/dialogue-core-metrics.yml index 9bd0aae21..5174bc285 100644 --- a/dialogue-core/src/main/metrics/dialogue-core-metrics.yml +++ b/dialogue-core/src/main/metrics/dialogue-core-metrics.yml @@ -2,16 +2,6 @@ options: javaPackage: com.palantir.dialogue.core javaVisibility: packagePrivate namespaces: - com.palantir.conjure.java.dispatcher: - shortName: Dispatcher - docs: Dialogue client dispatcher metrics - metrics: - calls.queued: - type: gauge - docs: Number of queued calls waiting to execute. - calls.running: - type: gauge - docs: Number of active outgoing requests. dialogue.client: docs: Dialogue client response metrics. metrics: From ff0e69285c8b7980fa02a720a5a7bac282688a5d Mon Sep 17 00:00:00 2001 From: Carter Kozak Date: Wed, 26 Feb 2020 09:32:42 -0500 Subject: [PATCH 12/17] Use the abort time as infinity --- .../java/com/palantir/dialogue/core/Benchmark.java | 11 +++++++---- .../java/com/palantir/dialogue/core/Simulation.java | 5 +++-- simulation/src/test/resources/report.md | 2 +- ...kes[CONCURRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].txt | 2 +- ...kes[CONCURRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].png | 4 ++-- 5 files changed, 14 insertions(+), 10 deletions(-) diff --git a/simulation/src/main/java/com/palantir/dialogue/core/Benchmark.java b/simulation/src/main/java/com/palantir/dialogue/core/Benchmark.java index 6d6fb3e12..c3f9b85f5 100644 --- a/simulation/src/main/java/com/palantir/dialogue/core/Benchmark.java +++ b/simulation/src/main/java/com/palantir/dialogue/core/Benchmark.java @@ -31,6 +31,7 @@ import java.time.Duration; import java.util.Arrays; import java.util.Map; +import java.util.Optional; import java.util.Random; import java.util.TreeMap; import java.util.concurrent.TimeUnit; @@ -59,6 +60,7 @@ public final class Benchmark { private Stream requestStream; private Function requestSupplier = Benchmark::constructRequest; private ShouldStopPredicate benchmarkFinished; + private Duration abortAfter; private Benchmark() {} @@ -139,17 +141,18 @@ public void update(Duration _time, long _requestsStarted, long responsesReceived } @SuppressWarnings({"FutureReturnValueIgnored", "CheckReturnValue"}) - public Benchmark abortAfter(Duration cutoff) { + public Benchmark abortAfter(Duration value) { + this.abortAfter = value; simulation .scheduler() .schedule( () -> { log.warn( "Aborted running benchmark after cutoff reached - strategy might be buggy {}", - cutoff); + value); benchmarkFinished.getFuture().set(null); }, - cutoff.toNanos(), + value.toNanos(), TimeUnit.NANOSECONDS); return this; } @@ -157,7 +160,7 @@ public Benchmark abortAfter(Duration cutoff) { public BenchmarkResult run() { ListenableFuture result = schedule(); Stopwatch run = Stopwatch.createStarted(); - simulation.runClockToInfinity(); + simulation.runClockToInfinity(Optional.ofNullable(abortAfter)); log.info("Ran clock to infinity ({} ms)", run.elapsed(TimeUnit.MILLISECONDS)); return Futures.getUnchecked(result); } diff --git a/simulation/src/main/java/com/palantir/dialogue/core/Simulation.java b/simulation/src/main/java/com/palantir/dialogue/core/Simulation.java index 023ec5925..f0903bafa 100644 --- a/simulation/src/main/java/com/palantir/dialogue/core/Simulation.java +++ b/simulation/src/main/java/com/palantir/dialogue/core/Simulation.java @@ -20,6 +20,7 @@ import com.google.common.util.concurrent.ListeningScheduledExecutorService; import com.google.common.util.concurrent.MoreExecutors; import java.time.Duration; +import java.util.Optional; import java.util.concurrent.TimeUnit; import org.jmock.lib.concurrent.DeterministicScheduler; import org.slf4j.Logger; @@ -68,7 +69,7 @@ public EventMarkers events() { return eventMarkers; } - public void runClockToInfinity() { - deterministicExecutor.tick(Duration.ofDays(1).toNanos(), TimeUnit.NANOSECONDS); + public void runClockToInfinity(Optional infinity) { + deterministicExecutor.tick(infinity.orElseGet(() -> Duration.ofDays(1)).toNanos(), TimeUnit.NANOSECONDS); } } diff --git a/simulation/src/test/resources/report.md b/simulation/src/test/resources/report.md index c55bcaea4..eb6a6b0a1 100644 --- a/simulation/src/test/resources/report.md +++ b/simulation/src/test/resources/report.md @@ -37,7 +37,7 @@ slowdown_and_error_thresholds[CONCURRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].txt: su slowdown_and_error_thresholds[CONCURRENCY_LIMITER_PIN_UNTIL_ERROR].txt: success=1.2% client_mean=PT2.916010959S server_cpu=PT4H30M39.598982029S client_received=10000/10000 server_resps=4255 codes={200=120, 500=4135, Failed to make a request=5745} slowdown_and_error_thresholds[CONCURRENCY_LIMITER_ROUND_ROBIN].txt: success=1.2% client_mean=PT2.990999813S server_cpu=PT4H41M15.507187288S client_received=10000/10000 server_resps=4433 codes={200=120, 500=4313, Failed to make a request=5567} slowdown_and_error_thresholds[UNLIMITED_ROUND_ROBIN].txt: success=1.2% client_mean=PT3.974119999S server_cpu=PT11H2M21.19999998S client_received=10000/10000 server_resps=10000 codes={200=120, 500=9880} - uncommon_flakes[CONCURRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].txt: success=93.6% client_mean=PT0.210193295S server_cpu=PT6.063774874S client_received=9730/10000 server_resps=9456 codes={200=9363, 500=93, Failed to make a request=274} + uncommon_flakes[CONCURRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].txt: success=93.6% client_mean=PT0.210193295S server_cpu=PT6.024159232S client_received=9730/10000 server_resps=9456 codes={200=9363, 500=93, Failed to make a request=274} uncommon_flakes[CONCURRENCY_LIMITER_PIN_UNTIL_ERROR].txt: success=99.0% client_mean=PT0.000001S server_cpu=PT0.01S client_received=10000/10000 server_resps=10000 codes={200=9902, 500=98} uncommon_flakes[CONCURRENCY_LIMITER_ROUND_ROBIN].txt: success=99.0% client_mean=PT0.000001S server_cpu=PT0.01S client_received=10000/10000 server_resps=10000 codes={200=9900, 500=100} uncommon_flakes[UNLIMITED_ROUND_ROBIN].txt: success=99.0% client_mean=PT0.000001S server_cpu=PT0.01S client_received=10000/10000 server_resps=10000 codes={200=9900, 500=100} diff --git a/simulation/src/test/resources/txt/uncommon_flakes[CONCURRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].txt b/simulation/src/test/resources/txt/uncommon_flakes[CONCURRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].txt index 6a61f1195..b02247270 100644 --- a/simulation/src/test/resources/txt/uncommon_flakes[CONCURRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].txt +++ b/simulation/src/test/resources/txt/uncommon_flakes[CONCURRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].txt @@ -1 +1 @@ -success=93.6% client_mean=PT0.210193295S server_cpu=PT6.063774874S client_received=9730/10000 server_resps=9456 codes={200=9363, 500=93, Failed to make a request=274} \ No newline at end of file +success=93.6% client_mean=PT0.210193295S server_cpu=PT6.024159232S client_received=9730/10000 server_resps=9456 codes={200=9363, 500=93, Failed to make a request=274} \ No newline at end of file diff --git a/simulation/src/test/resources/uncommon_flakes[CONCURRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].png b/simulation/src/test/resources/uncommon_flakes[CONCURRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].png index 1dcf74585..1886d064c 100644 --- a/simulation/src/test/resources/uncommon_flakes[CONCURRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].png +++ b/simulation/src/test/resources/uncommon_flakes[CONCURRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].png @@ -1,3 +1,3 @@ version https://git-lfs.github.com/spec/v1 -oid sha256:a19bf7e56543297a68014f98765b86207631ce6e63d32c6d9bd5edd13ef6b78c -size 195134 +oid sha256:31decac5193da2cae57af434398fcd4385e62068b76a1fad269080db18b95afb +size 193416 From 95d73f77c60e90eb9ba30d07b54d9e1a788477fb Mon Sep 17 00:00:00 2001 From: Carter Kozak Date: Wed, 26 Feb 2020 10:50:28 -0500 Subject: [PATCH 13/17] pre/post tick --- .../core/ExternalDeterministicScheduler.java | 29 +++++++++----- .../palantir/dialogue/core/Simulation.java | 4 +- .../dialogue/core/TestCaffeineTicker.java | 11 +++-- ...URRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].png | 4 +- ...n[CONCURRENCY_LIMITER_PIN_UNTIL_ERROR].png | 4 +- ...t[CONCURRENCY_LIMITER_PIN_UNTIL_ERROR].png | 4 +- ...URRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].png | 4 +- ...g[CONCURRENCY_LIMITER_PIN_UNTIL_ERROR].png | 4 +- ...ading[CONCURRENCY_LIMITER_ROUND_ROBIN].png | 4 +- ...URRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].png | 4 +- simulation/src/test/resources/report.md | 40 +++++++++---------- ...URRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].png | 4 +- ...e[CONCURRENCY_LIMITER_PIN_UNTIL_ERROR].png | 4 +- ..._case[CONCURRENCY_LIMITER_ROUND_ROBIN].png | 4 +- ...t_possible_case[UNLIMITED_ROUND_ROBIN].png | 4 +- ...URRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].png | 4 +- ...t[CONCURRENCY_LIMITER_PIN_UNTIL_ERROR].png | 4 +- ...evert[CONCURRENCY_LIMITER_ROUND_ROBIN].png | 4 +- ...03s_then_revert[UNLIMITED_ROUND_ROBIN].png | 4 +- ...URRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].png | 4 +- ...s[CONCURRENCY_LIMITER_PIN_UNTIL_ERROR].png | 4 +- ...holds[CONCURRENCY_LIMITER_ROUND_ROBIN].png | 4 +- ...rror_thresholds[UNLIMITED_ROUND_ROBIN].png | 4 +- ...URRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].txt | 2 +- ...n[CONCURRENCY_LIMITER_PIN_UNTIL_ERROR].txt | 2 +- ...t[CONCURRENCY_LIMITER_PIN_UNTIL_ERROR].txt | 2 +- ...URRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].txt | 2 +- ...g[CONCURRENCY_LIMITER_PIN_UNTIL_ERROR].txt | 2 +- ...ading[CONCURRENCY_LIMITER_ROUND_ROBIN].txt | 2 +- ...URRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].txt | 2 +- ...URRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].txt | 2 +- ...e[CONCURRENCY_LIMITER_PIN_UNTIL_ERROR].txt | 2 +- ..._case[CONCURRENCY_LIMITER_ROUND_ROBIN].txt | 2 +- ...t_possible_case[UNLIMITED_ROUND_ROBIN].txt | 2 +- ...URRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].txt | 2 +- ...t[CONCURRENCY_LIMITER_PIN_UNTIL_ERROR].txt | 2 +- ...evert[CONCURRENCY_LIMITER_ROUND_ROBIN].txt | 2 +- ...03s_then_revert[UNLIMITED_ROUND_ROBIN].txt | 2 +- ...URRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].txt | 2 +- ...s[CONCURRENCY_LIMITER_PIN_UNTIL_ERROR].txt | 2 +- ...holds[CONCURRENCY_LIMITER_ROUND_ROBIN].txt | 2 +- ...rror_thresholds[UNLIMITED_ROUND_ROBIN].txt | 2 +- ...URRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].txt | 2 +- ...URRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].png | 4 +- 44 files changed, 107 insertions(+), 97 deletions(-) diff --git a/simulation/src/main/java/com/palantir/dialogue/core/ExternalDeterministicScheduler.java b/simulation/src/main/java/com/palantir/dialogue/core/ExternalDeterministicScheduler.java index b5a8b2523..43c3d5f6e 100644 --- a/simulation/src/main/java/com/palantir/dialogue/core/ExternalDeterministicScheduler.java +++ b/simulation/src/main/java/com/palantir/dialogue/core/ExternalDeterministicScheduler.java @@ -18,6 +18,7 @@ import com.google.common.util.concurrent.ListenableFuture; import com.google.common.util.concurrent.ListenableScheduledFuture; import com.google.common.util.concurrent.ListeningScheduledExecutorService; +import com.google.common.util.concurrent.MoreExecutors; import java.util.Collection; import java.util.List; import java.util.concurrent.Callable; @@ -25,14 +26,17 @@ import java.util.concurrent.Future; import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeoutException; +import org.jmock.lib.concurrent.DeterministicScheduler; final class ExternalDeterministicScheduler implements ListeningScheduledExecutorService { + private final DeterministicScheduler deterministicExecutor; private final ListeningScheduledExecutorService delegate; private final TestCaffeineTicker ticker; - ExternalDeterministicScheduler(ListeningScheduledExecutorService delegate, TestCaffeineTicker ticker) { - this.delegate = delegate; + ExternalDeterministicScheduler(DeterministicScheduler deterministicExecutor, TestCaffeineTicker ticker) { + this.deterministicExecutor = deterministicExecutor; + this.delegate = MoreExecutors.listeningDecorator(deterministicExecutor); this.ticker = ticker; } @@ -49,15 +53,20 @@ public ListenableScheduledFuture schedule(Runnable command, long delay, TimeU @Override public ListenableScheduledFuture schedule(Callable command, long delay, TimeUnit unit) { + deterministicExecutor.tick(0, TimeUnit.NANOSECONDS); long scheduleTime = ticker.read(); - long delayNanos = unit.toNanos(delay); - return delegate.schedule( - () -> { - ticker.advanceTo(scheduleTime + delayNanos); - return command.call(); - }, - delay, - unit); + long delayNanos = Math.max(0L, unit.toNanos(delay)); + try { + return delegate.schedule( + () -> { + ticker.advanceTo(scheduleTime + delayNanos); + return command.call(); + }, + delayNanos, + TimeUnit.NANOSECONDS); + } finally { + deterministicExecutor.tick(0, TimeUnit.NANOSECONDS); + } } @Override diff --git a/simulation/src/main/java/com/palantir/dialogue/core/Simulation.java b/simulation/src/main/java/com/palantir/dialogue/core/Simulation.java index f0903bafa..6902483ac 100644 --- a/simulation/src/main/java/com/palantir/dialogue/core/Simulation.java +++ b/simulation/src/main/java/com/palantir/dialogue/core/Simulation.java @@ -18,7 +18,6 @@ import com.github.benmanes.caffeine.cache.Ticker; import com.google.common.util.concurrent.ListeningScheduledExecutorService; -import com.google.common.util.concurrent.MoreExecutors; import java.time.Duration; import java.util.Optional; import java.util.concurrent.TimeUnit; @@ -41,8 +40,7 @@ final class Simulation { Simulation() { Thread.currentThread().setUncaughtExceptionHandler((t, e) -> log.error("Uncaught throwable", e)); - this.listenableExecutor = - new ExternalDeterministicScheduler(MoreExecutors.listeningDecorator(deterministicExecutor), ticker); + this.listenableExecutor = new ExternalDeterministicScheduler(deterministicExecutor, ticker); } public Ticker clock() { diff --git a/simulation/src/main/java/com/palantir/dialogue/core/TestCaffeineTicker.java b/simulation/src/main/java/com/palantir/dialogue/core/TestCaffeineTicker.java index 606e495b8..ac849758d 100644 --- a/simulation/src/main/java/com/palantir/dialogue/core/TestCaffeineTicker.java +++ b/simulation/src/main/java/com/palantir/dialogue/core/TestCaffeineTicker.java @@ -16,6 +16,9 @@ package com.palantir.dialogue.core; import com.github.benmanes.caffeine.cache.Ticker; +import com.palantir.logsafe.Preconditions; +import com.palantir.logsafe.SafeArg; +import java.time.Duration; import java.util.concurrent.TimeUnit; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -33,10 +36,10 @@ public long read() { void advanceTo(long newNanos) { if (newNanos < nanos) { long difference = nanos - newNanos; - // Preconditions.checkState( - // difference < Duration.ofMillis(2).toNanos(), - // "Large time rewind - this is likely a bug in the test harness", - // SafeArg.of("difference", difference)); + Preconditions.checkState( + difference < Duration.ofMillis(1).toNanos(), + "Large time rewind - this is likely a bug in the test harness", + SafeArg.of("difference", difference)); log.debug( "Tried to rewind time by {} micros - no-op as this is deterministic and harmless", TimeUnit.MICROSECONDS.convert(difference, TimeUnit.NANOSECONDS)); diff --git a/simulation/src/test/resources/all_nodes_500[CONCURRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].png b/simulation/src/test/resources/all_nodes_500[CONCURRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].png index 11b7216fe..b30eabae1 100644 --- a/simulation/src/test/resources/all_nodes_500[CONCURRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].png +++ b/simulation/src/test/resources/all_nodes_500[CONCURRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].png @@ -1,3 +1,3 @@ version https://git-lfs.github.com/spec/v1 -oid sha256:e26afa8c5053623cb1ede2032a7ec8a083b9de75e03d3d26508e0094b05aae88 -size 180149 +oid sha256:9c24fab00c7c62446e2b515554d66dfb09faf8144d78845c6dc072837ef1875c +size 133376 diff --git a/simulation/src/test/resources/drastic_slowdown[CONCURRENCY_LIMITER_PIN_UNTIL_ERROR].png b/simulation/src/test/resources/drastic_slowdown[CONCURRENCY_LIMITER_PIN_UNTIL_ERROR].png index ffdc4fdd8..b88e5788f 100644 --- a/simulation/src/test/resources/drastic_slowdown[CONCURRENCY_LIMITER_PIN_UNTIL_ERROR].png +++ b/simulation/src/test/resources/drastic_slowdown[CONCURRENCY_LIMITER_PIN_UNTIL_ERROR].png @@ -1,3 +1,3 @@ version https://git-lfs.github.com/spec/v1 -oid sha256:2f53134c29c11c30e5204dd9cb259d257ac115b0a657b65cdd4db718fce8e0c6 -size 99435 +oid sha256:08ca57ae0af3172a46478026ef1fa5556747baf977cd61c02e4ef5567d52665a +size 66576 diff --git a/simulation/src/test/resources/fast_500s_then_revert[CONCURRENCY_LIMITER_PIN_UNTIL_ERROR].png b/simulation/src/test/resources/fast_500s_then_revert[CONCURRENCY_LIMITER_PIN_UNTIL_ERROR].png index 8448562de..be433c40e 100644 --- a/simulation/src/test/resources/fast_500s_then_revert[CONCURRENCY_LIMITER_PIN_UNTIL_ERROR].png +++ b/simulation/src/test/resources/fast_500s_then_revert[CONCURRENCY_LIMITER_PIN_UNTIL_ERROR].png @@ -1,3 +1,3 @@ version https://git-lfs.github.com/spec/v1 -oid sha256:78f63d5dca3b52185166060f293b89f0d3019e18fe90a1585a77c56ed789c937 -size 92743 +oid sha256:2baed73f8c00d2ad33fdca2192b7f975e001dfae9b7fd884f740d4bad5b87603 +size 58071 diff --git a/simulation/src/test/resources/live_reloading[CONCURRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].png b/simulation/src/test/resources/live_reloading[CONCURRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].png index 1b3fb130d..df48e8a5a 100644 --- a/simulation/src/test/resources/live_reloading[CONCURRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].png +++ b/simulation/src/test/resources/live_reloading[CONCURRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].png @@ -1,3 +1,3 @@ version https://git-lfs.github.com/spec/v1 -oid sha256:2d9cc4200431e2a56147d08f622fd1b4b040dc3c98d2fc6dc12076518d1c71d7 -size 131916 +oid sha256:4bbc6bc614dfc8e04b01600cc3034bc5cf511027eaeb72ec64c8cf148c5a7447 +size 101878 diff --git a/simulation/src/test/resources/live_reloading[CONCURRENCY_LIMITER_PIN_UNTIL_ERROR].png b/simulation/src/test/resources/live_reloading[CONCURRENCY_LIMITER_PIN_UNTIL_ERROR].png index 0c42736e1..65dd09784 100644 --- a/simulation/src/test/resources/live_reloading[CONCURRENCY_LIMITER_PIN_UNTIL_ERROR].png +++ b/simulation/src/test/resources/live_reloading[CONCURRENCY_LIMITER_PIN_UNTIL_ERROR].png @@ -1,3 +1,3 @@ version https://git-lfs.github.com/spec/v1 -oid sha256:96cf6edd6e9656958c38eda3dd0074ed17bf927a48710b22017bcba2cb2f9094 -size 122196 +oid sha256:bce2af3b1d7313311c72e26aff06ec3e4a0d715cc3b670bbbe1804605dfb4b3c +size 96317 diff --git a/simulation/src/test/resources/live_reloading[CONCURRENCY_LIMITER_ROUND_ROBIN].png b/simulation/src/test/resources/live_reloading[CONCURRENCY_LIMITER_ROUND_ROBIN].png index 4a5576fba..603c53447 100644 --- a/simulation/src/test/resources/live_reloading[CONCURRENCY_LIMITER_ROUND_ROBIN].png +++ b/simulation/src/test/resources/live_reloading[CONCURRENCY_LIMITER_ROUND_ROBIN].png @@ -1,3 +1,3 @@ version https://git-lfs.github.com/spec/v1 -oid sha256:e47fc243b06ce2ae6f044cfcbdc0fb7c571f36581eb285b22347d8b17c334014 -size 124308 +oid sha256:703aa9b8cd3ec7099c513ad74204278eba4d9fe0da56c28d3538c7e1d4574577 +size 95611 diff --git a/simulation/src/test/resources/one_endpoint_dies_on_each_server[CONCURRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].png b/simulation/src/test/resources/one_endpoint_dies_on_each_server[CONCURRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].png index fe04fbab1..c9bf6cbb5 100644 --- a/simulation/src/test/resources/one_endpoint_dies_on_each_server[CONCURRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].png +++ b/simulation/src/test/resources/one_endpoint_dies_on_each_server[CONCURRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].png @@ -1,3 +1,3 @@ version https://git-lfs.github.com/spec/v1 -oid sha256:7ec7ceb400f5c5ebeea8a3429ae20f7650b4ca5e685cda6236f42f82f2172cb5 -size 161956 +oid sha256:511590a3c0ffdcd2e06aea8908acd0aa152895e3021d85fce754943bd7dd0ab3 +size 126298 diff --git a/simulation/src/test/resources/report.md b/simulation/src/test/resources/report.md index eb6a6b0a1..12e3eb263 100644 --- a/simulation/src/test/resources/report.md +++ b/simulation/src/test/resources/report.md @@ -1,7 +1,7 @@ # Report ``` - all_nodes_500[CONCURRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].txt: success=52.6% client_mean=PT1.027869274S server_cpu=PT16M44.041614481S client_received=2000/2000 server_resps=1673 codes={200=1051, 500=622, Failed to make a request=327} + all_nodes_500[CONCURRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].txt: success=51.9% client_mean=PT1.032634888S server_cpu=PT16M43.011354078S client_received=2000/2000 server_resps=1662 codes={200=1037, 500=625, Failed to make a request=338} all_nodes_500[CONCURRENCY_LIMITER_PIN_UNTIL_ERROR].txt: success=50.0% client_mean=PT0.6S server_cpu=PT20M client_received=2000/2000 server_resps=2000 codes={200=1000, 500=1000} all_nodes_500[CONCURRENCY_LIMITER_ROUND_ROBIN].txt: success=50.0% client_mean=PT0.6S server_cpu=PT20M client_received=2000/2000 server_resps=2000 codes={200=1000, 500=1000} all_nodes_500[UNLIMITED_ROUND_ROBIN].txt: success=50.0% client_mean=PT0.6S server_cpu=PT20M client_received=2000/2000 server_resps=2000 codes={200=1000, 500=1000} @@ -10,34 +10,34 @@ black_hole[CONCURRENCY_LIMITER_ROUND_ROBIN].txt: success=89.9% client_mean=PT0.600286226S server_cpu=PT17M58.2S client_received=1797/2000 server_resps=1797 codes={200=1797} black_hole[UNLIMITED_ROUND_ROBIN].txt: success=65.0% client_mean=PT0.6S server_cpu=PT12M59.4S client_received=1299/2000 server_resps=1299 codes={200=1299} drastic_slowdown[CONCURRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].txt: success=100.0% client_mean=PT2.069939083S server_cpu=PT2H17M59.756333311S client_received=4000/4000 server_resps=4000 codes={200=4000} - drastic_slowdown[CONCURRENCY_LIMITER_PIN_UNTIL_ERROR].txt: success=100.0% client_mean=PT2.053746297S server_cpu=PT2H16M53.159562742S client_received=4000/4000 server_resps=4000 codes={200=4000} + drastic_slowdown[CONCURRENCY_LIMITER_PIN_UNTIL_ERROR].txt: success=100.0% client_mean=PT2.053746438S server_cpu=PT2H16M53.160125525S client_received=4000/4000 server_resps=4000 codes={200=4000} drastic_slowdown[CONCURRENCY_LIMITER_ROUND_ROBIN].txt: success=100.0% client_mean=PT2.069939083S server_cpu=PT2H17M59.756333311S client_received=4000/4000 server_resps=4000 codes={200=4000} drastic_slowdown[UNLIMITED_ROUND_ROBIN].txt: success=100.0% client_mean=PT8.353421749S server_cpu=PT9H16M53.686999978S client_received=4000/4000 server_resps=4000 codes={200=4000} fast_500s_then_revert[CONCURRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].txt: success=98.1% client_mean=PT0.073222888S server_cpu=PT4M34.585833294S client_received=3750/3750 server_resps=3750 codes={200=3679, 500=71} - fast_500s_then_revert[CONCURRENCY_LIMITER_PIN_UNTIL_ERROR].txt: success=99.7% client_mean=PT0.080628266S server_cpu=PT5M2.355999997S client_received=3750/3750 server_resps=3750 codes={200=3739, 500=11} + fast_500s_then_revert[CONCURRENCY_LIMITER_PIN_UNTIL_ERROR].txt: success=99.7% client_mean=PT0.080628355S server_cpu=PT5M2.35633333S client_received=3750/3750 server_resps=3750 codes={200=3739, 500=11} fast_500s_then_revert[CONCURRENCY_LIMITER_ROUND_ROBIN].txt: success=76.7% client_mean=PT0.055463644S server_cpu=PT3M27.988666346S client_received=3750/3750 server_resps=3750 codes={200=2876, 500=874} fast_500s_then_revert[UNLIMITED_ROUND_ROBIN].txt: success=76.7% client_mean=PT0.055463644S server_cpu=PT3M27.988666346S client_received=3750/3750 server_resps=3750 codes={200=2876, 500=874} - live_reloading[CONCURRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].txt: success=59.2% client_mean=PT2.707428425S server_cpu=PT1H20M58.880261641S client_received=2500/2500 server_resps=1860 codes={200=1480, 500=380, Failed to make a request=640} - live_reloading[CONCURRENCY_LIMITER_PIN_UNTIL_ERROR].txt: success=54.0% client_mean=PT2.903672277S server_cpu=PT1H42M35.433984304S client_received=2500/2500 server_resps=2177 codes={200=1349, 500=828, Failed to make a request=323} - live_reloading[CONCURRENCY_LIMITER_ROUND_ROBIN].txt: success=52.1% client_mean=PT2.908560636S server_cpu=PT1H42M58.764371458S client_received=2500/2500 server_resps=2193 codes={200=1302, 500=891, Failed to make a request=307} + live_reloading[CONCURRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].txt: success=59.3% client_mean=PT2.717022131S server_cpu=PT1H21M18.049029538S client_received=2500/2500 server_resps=1865 codes={200=1483, 500=382, Failed to make a request=635} + live_reloading[CONCURRENCY_LIMITER_PIN_UNTIL_ERROR].txt: success=54.2% client_mean=PT2.907371536S server_cpu=PT1H42M40.098775697S client_received=2500/2500 server_resps=2177 codes={200=1355, 500=822, Failed to make a request=323} + live_reloading[CONCURRENCY_LIMITER_ROUND_ROBIN].txt: success=52.0% client_mean=PT2.915893225S server_cpu=PT1H43M13.659598627S client_received=2500/2500 server_resps=2197 codes={200=1301, 500=896, Failed to make a request=303} live_reloading[UNLIMITED_ROUND_ROBIN].txt: success=58.4% client_mean=PT2.8396S server_cpu=PT1H58M19S client_received=2500/2500 server_resps=2500 codes={200=1461, 500=1039} -one_endpoint_dies_on_each_server[CONCURRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].txt: success=47.0% client_mean=PT1.284406604S server_cpu=PT15M41.717604108S client_received=2500/2500 server_resps=1569 codes={200=1176, 500=393, Failed to make a request=931} +one_endpoint_dies_on_each_server[CONCURRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].txt: success=46.4% client_mean=PT1.302954516S server_cpu=PT15M22.396603327S client_received=2500/2500 server_resps=1536 codes={200=1161, 500=375, Failed to make a request=964} one_endpoint_dies_on_each_server[CONCURRENCY_LIMITER_PIN_UNTIL_ERROR].txt: success=63.8% client_mean=PT0.601140264S server_cpu=PT25M client_received=2500/2500 server_resps=2500 codes={200=1595, 500=905} one_endpoint_dies_on_each_server[CONCURRENCY_LIMITER_ROUND_ROBIN].txt: success=65.5% client_mean=PT0.6S server_cpu=PT25M client_received=2500/2500 server_resps=2500 codes={200=1638, 500=862} one_endpoint_dies_on_each_server[UNLIMITED_ROUND_ROBIN].txt: success=65.5% client_mean=PT0.6S server_cpu=PT25M client_received=2500/2500 server_resps=2500 codes={200=1638, 500=862} - simplest_possible_case[CONCURRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].txt: success=100.0% client_mean=PT0.799939393S server_cpu=PT2H55M59.2S client_received=13200/13200 server_resps=13200 codes={200=13200} - simplest_possible_case[CONCURRENCY_LIMITER_PIN_UNTIL_ERROR].txt: success=100.0% client_mean=PT0.998696969S server_cpu=PT3H39M42.8S client_received=13200/13200 server_resps=13200 codes={200=13200} - simplest_possible_case[CONCURRENCY_LIMITER_ROUND_ROBIN].txt: success=100.0% client_mean=PT0.799939393S server_cpu=PT2H55M59.2S client_received=13200/13200 server_resps=13200 codes={200=13200} - simplest_possible_case[UNLIMITED_ROUND_ROBIN].txt: success=100.0% client_mean=PT0.799939393S server_cpu=PT2H55M59.2S client_received=13200/13200 server_resps=13200 codes={200=13200} - slow_503s_then_revert[CONCURRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].txt: success=100.0% client_mean=PT0.302783916S server_cpu=PT14M44.375304373S client_received=3000/3000 server_resps=3175 codes={200=3000} - slow_503s_then_revert[CONCURRENCY_LIMITER_PIN_UNTIL_ERROR].txt: success=100.0% client_mean=PT0.346572928S server_cpu=PT16M51.987159344S client_received=3000/3000 server_resps=3197 codes={200=3000} - slow_503s_then_revert[CONCURRENCY_LIMITER_ROUND_ROBIN].txt: success=100.0% client_mean=PT0.745544946S server_cpu=PT36M23.421729078S client_received=3000/3000 server_resps=3411 codes={200=3000} - slow_503s_then_revert[UNLIMITED_ROUND_ROBIN].txt: success=100.0% client_mean=PT1.43038075S server_cpu=PT1H9M49.707166439S client_received=3000/3000 server_resps=3802 codes={200=3000} -slowdown_and_error_thresholds[CONCURRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].txt: success=3.3% client_mean=PT2.227891278S server_cpu=PT1H40M10.840754535S client_received=10000/10000 server_resps=1617 codes={200=327, 500=1290, Failed to make a request=8383} -slowdown_and_error_thresholds[CONCURRENCY_LIMITER_PIN_UNTIL_ERROR].txt: success=1.2% client_mean=PT2.916010959S server_cpu=PT4H30M39.598982029S client_received=10000/10000 server_resps=4255 codes={200=120, 500=4135, Failed to make a request=5745} - slowdown_and_error_thresholds[CONCURRENCY_LIMITER_ROUND_ROBIN].txt: success=1.2% client_mean=PT2.990999813S server_cpu=PT4H41M15.507187288S client_received=10000/10000 server_resps=4433 codes={200=120, 500=4313, Failed to make a request=5567} - slowdown_and_error_thresholds[UNLIMITED_ROUND_ROBIN].txt: success=1.2% client_mean=PT3.974119999S server_cpu=PT11H2M21.19999998S client_received=10000/10000 server_resps=10000 codes={200=120, 500=9880} - uncommon_flakes[CONCURRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].txt: success=93.6% client_mean=PT0.210193295S server_cpu=PT6.024159232S client_received=9730/10000 server_resps=9456 codes={200=9363, 500=93, Failed to make a request=274} + simplest_possible_case[CONCURRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].txt: success=100.0% client_mean=PT0.799939398S server_cpu=PT2H55M59.2000635S client_received=13200/13200 server_resps=13200 codes={200=13200} + simplest_possible_case[CONCURRENCY_LIMITER_PIN_UNTIL_ERROR].txt: success=100.0% client_mean=PT0.998702894S server_cpu=PT3H39M42.87820128S client_received=13200/13200 server_resps=13200 codes={200=13200} + simplest_possible_case[CONCURRENCY_LIMITER_ROUND_ROBIN].txt: success=100.0% client_mean=PT0.799939398S server_cpu=PT2H55M59.2000635S client_received=13200/13200 server_resps=13200 codes={200=13200} + simplest_possible_case[UNLIMITED_ROUND_ROBIN].txt: success=100.0% client_mean=PT0.799939398S server_cpu=PT2H55M59.2000635S client_received=13200/13200 server_resps=13200 codes={200=13200} + slow_503s_then_revert[CONCURRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].txt: success=100.0% client_mean=PT0.30283067S server_cpu=PT14M44.486985327S client_received=3000/3000 server_resps=3175 codes={200=3000} + slow_503s_then_revert[CONCURRENCY_LIMITER_PIN_UNTIL_ERROR].txt: success=100.0% client_mean=PT0.346630773S server_cpu=PT16M52.121612924S client_received=3000/3000 server_resps=3197 codes={200=3000} + slow_503s_then_revert[CONCURRENCY_LIMITER_ROUND_ROBIN].txt: success=100.0% client_mean=PT0.745660042S server_cpu=PT36M23.70242803S client_received=3000/3000 server_resps=3411 codes={200=3000} + slow_503s_then_revert[UNLIMITED_ROUND_ROBIN].txt: success=100.0% client_mean=PT1.430720789S server_cpu=PT1H9M50.430528642S client_received=3000/3000 server_resps=3802 codes={200=3000} +slowdown_and_error_thresholds[CONCURRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].txt: success=3.4% client_mean=PT2.255528943S server_cpu=PT1H43M47.583860348S client_received=10000/10000 server_resps=1676 codes={200=344, 500=1332, Failed to make a request=8324} +slowdown_and_error_thresholds[CONCURRENCY_LIMITER_PIN_UNTIL_ERROR].txt: success=1.2% client_mean=PT2.928958527S server_cpu=PT4H31M58.68391554S client_received=10000/10000 server_resps=4264 codes={200=120, 500=4144, Failed to make a request=5736} + slowdown_and_error_thresholds[CONCURRENCY_LIMITER_ROUND_ROBIN].txt: success=1.2% client_mean=PT2.987017569S server_cpu=PT4H40M48.239132959S client_received=10000/10000 server_resps=4417 codes={200=120, 500=4297, Failed to make a request=5583} + slowdown_and_error_thresholds[UNLIMITED_ROUND_ROBIN].txt: success=1.2% client_mean=PT3.974129199S server_cpu=PT11H2M21.291999888S client_received=10000/10000 server_resps=10000 codes={200=120, 500=9880} + uncommon_flakes[CONCURRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].txt: success=93.5% client_mean=PT0.203113054S server_cpu=PT7.376462491S client_received=9774/10000 server_resps=9441 codes={200=9348, 500=93, Failed to make a request=333} uncommon_flakes[CONCURRENCY_LIMITER_PIN_UNTIL_ERROR].txt: success=99.0% client_mean=PT0.000001S server_cpu=PT0.01S client_received=10000/10000 server_resps=10000 codes={200=9902, 500=98} uncommon_flakes[CONCURRENCY_LIMITER_ROUND_ROBIN].txt: success=99.0% client_mean=PT0.000001S server_cpu=PT0.01S client_received=10000/10000 server_resps=10000 codes={200=9900, 500=100} uncommon_flakes[UNLIMITED_ROUND_ROBIN].txt: success=99.0% client_mean=PT0.000001S server_cpu=PT0.01S client_received=10000/10000 server_resps=10000 codes={200=9900, 500=100} diff --git a/simulation/src/test/resources/simplest_possible_case[CONCURRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].png b/simulation/src/test/resources/simplest_possible_case[CONCURRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].png index ba39dc6ea..8dd9b147f 100644 --- a/simulation/src/test/resources/simplest_possible_case[CONCURRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].png +++ b/simulation/src/test/resources/simplest_possible_case[CONCURRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].png @@ -1,3 +1,3 @@ version https://git-lfs.github.com/spec/v1 -oid sha256:75ab5905eeb5edf37fc0075c62953fd9dbce73f6ca53837f7a552165616f5d7b -size 199770 +oid sha256:ff06cbe9e9ff96ab23d3410584327a28fd8a6fd0819121a7e6786f338ceacb4a +size 151514 diff --git a/simulation/src/test/resources/simplest_possible_case[CONCURRENCY_LIMITER_PIN_UNTIL_ERROR].png b/simulation/src/test/resources/simplest_possible_case[CONCURRENCY_LIMITER_PIN_UNTIL_ERROR].png index bee8d84dd..6c3d6cea8 100644 --- a/simulation/src/test/resources/simplest_possible_case[CONCURRENCY_LIMITER_PIN_UNTIL_ERROR].png +++ b/simulation/src/test/resources/simplest_possible_case[CONCURRENCY_LIMITER_PIN_UNTIL_ERROR].png @@ -1,3 +1,3 @@ version https://git-lfs.github.com/spec/v1 -oid sha256:a63d38543fbf49e275de3738fb3c463ace39bb5484ecf04ab0521ba30b71a5e0 -size 100223 +oid sha256:adb59311305fc3b0949c84f3881caefa36a2bf7acac006e5e4eb2a756612daf1 +size 59511 diff --git a/simulation/src/test/resources/simplest_possible_case[CONCURRENCY_LIMITER_ROUND_ROBIN].png b/simulation/src/test/resources/simplest_possible_case[CONCURRENCY_LIMITER_ROUND_ROBIN].png index 1640bbc59..f89b6a00f 100644 --- a/simulation/src/test/resources/simplest_possible_case[CONCURRENCY_LIMITER_ROUND_ROBIN].png +++ b/simulation/src/test/resources/simplest_possible_case[CONCURRENCY_LIMITER_ROUND_ROBIN].png @@ -1,3 +1,3 @@ version https://git-lfs.github.com/spec/v1 -oid sha256:48353dd885320ec53e1bc26ebfaa34a10aa4a6b46a96e9c7ef0284df417b6c6c -size 199683 +oid sha256:fd30e04d6f5c3b6c62c6f3a2b15087bf6b2c6ea99f8a290771205e21770b669f +size 151449 diff --git a/simulation/src/test/resources/simplest_possible_case[UNLIMITED_ROUND_ROBIN].png b/simulation/src/test/resources/simplest_possible_case[UNLIMITED_ROUND_ROBIN].png index 1a8de60fa..29b3ec717 100644 --- a/simulation/src/test/resources/simplest_possible_case[UNLIMITED_ROUND_ROBIN].png +++ b/simulation/src/test/resources/simplest_possible_case[UNLIMITED_ROUND_ROBIN].png @@ -1,3 +1,3 @@ version https://git-lfs.github.com/spec/v1 -oid sha256:fee8d875ef5696bc9385cf0a8f0988520313218d2fb50afe83bda80c63bc0ba1 -size 198817 +oid sha256:49f93488c846010ed6e0d0b04b3de008a580047959b74c89d74a6c07967372c5 +size 151226 diff --git a/simulation/src/test/resources/slow_503s_then_revert[CONCURRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].png b/simulation/src/test/resources/slow_503s_then_revert[CONCURRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].png index 925348469..ecb0d13f6 100644 --- a/simulation/src/test/resources/slow_503s_then_revert[CONCURRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].png +++ b/simulation/src/test/resources/slow_503s_then_revert[CONCURRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].png @@ -1,3 +1,3 @@ version https://git-lfs.github.com/spec/v1 -oid sha256:b3f233b8faf6b39ee52f94177e18c354c05d18311eb9fe57f0c1a5e26d3e40da -size 102677 +oid sha256:3c8be79ecd24d91ea775788459db4692eb0ba15170e4db90bb66205b700e25c1 +size 77254 diff --git a/simulation/src/test/resources/slow_503s_then_revert[CONCURRENCY_LIMITER_PIN_UNTIL_ERROR].png b/simulation/src/test/resources/slow_503s_then_revert[CONCURRENCY_LIMITER_PIN_UNTIL_ERROR].png index 7c718ec63..ac6edba3c 100644 --- a/simulation/src/test/resources/slow_503s_then_revert[CONCURRENCY_LIMITER_PIN_UNTIL_ERROR].png +++ b/simulation/src/test/resources/slow_503s_then_revert[CONCURRENCY_LIMITER_PIN_UNTIL_ERROR].png @@ -1,3 +1,3 @@ version https://git-lfs.github.com/spec/v1 -oid sha256:6038e449309374e1b76fc60d4c1370c9a2988b1e46ba28dcf7ed318372bacb36 -size 97048 +oid sha256:c2ef5bde030775058c6fcbd439285ac8507a0ff274cb5e6191c11d7ce5540991 +size 67624 diff --git a/simulation/src/test/resources/slow_503s_then_revert[CONCURRENCY_LIMITER_ROUND_ROBIN].png b/simulation/src/test/resources/slow_503s_then_revert[CONCURRENCY_LIMITER_ROUND_ROBIN].png index 5d970fc49..a141f623e 100644 --- a/simulation/src/test/resources/slow_503s_then_revert[CONCURRENCY_LIMITER_ROUND_ROBIN].png +++ b/simulation/src/test/resources/slow_503s_then_revert[CONCURRENCY_LIMITER_ROUND_ROBIN].png @@ -1,3 +1,3 @@ version https://git-lfs.github.com/spec/v1 -oid sha256:984763e1a5ae4fa6993da2ea2ed4bdc26359129767c683468859ffe1e532a9bc -size 99932 +oid sha256:c4b2b426b34f2259bac4acf0b2832f5d511118cc8350f365101e41af41b02535 +size 75729 diff --git a/simulation/src/test/resources/slow_503s_then_revert[UNLIMITED_ROUND_ROBIN].png b/simulation/src/test/resources/slow_503s_then_revert[UNLIMITED_ROUND_ROBIN].png index 1ae063926..f076ab457 100644 --- a/simulation/src/test/resources/slow_503s_then_revert[UNLIMITED_ROUND_ROBIN].png +++ b/simulation/src/test/resources/slow_503s_then_revert[UNLIMITED_ROUND_ROBIN].png @@ -1,3 +1,3 @@ version https://git-lfs.github.com/spec/v1 -oid sha256:d40179c954a45a26f6f84bca1c066de7e3b3b614f434e2ee2d684cee23129fd6 -size 96740 +oid sha256:8727fb4d90bbcdba6756dbaa716804d92a96459a35e18abb272d1bf1c1d54c1b +size 73043 diff --git a/simulation/src/test/resources/slowdown_and_error_thresholds[CONCURRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].png b/simulation/src/test/resources/slowdown_and_error_thresholds[CONCURRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].png index 4ffb3e9e2..9758ab01c 100644 --- a/simulation/src/test/resources/slowdown_and_error_thresholds[CONCURRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].png +++ b/simulation/src/test/resources/slowdown_and_error_thresholds[CONCURRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].png @@ -1,3 +1,3 @@ version https://git-lfs.github.com/spec/v1 -oid sha256:fa46699f003e9244ba2b56ae137d276c8edf1e170871676875e985cd0f906fb0 -size 141184 +oid sha256:727828c804562522d61dc8a8d3c8cc689b5514dacefd78e6b38e93c6efaf8b6a +size 113667 diff --git a/simulation/src/test/resources/slowdown_and_error_thresholds[CONCURRENCY_LIMITER_PIN_UNTIL_ERROR].png b/simulation/src/test/resources/slowdown_and_error_thresholds[CONCURRENCY_LIMITER_PIN_UNTIL_ERROR].png index 76e853958..d5e071b4f 100644 --- a/simulation/src/test/resources/slowdown_and_error_thresholds[CONCURRENCY_LIMITER_PIN_UNTIL_ERROR].png +++ b/simulation/src/test/resources/slowdown_and_error_thresholds[CONCURRENCY_LIMITER_PIN_UNTIL_ERROR].png @@ -1,3 +1,3 @@ version https://git-lfs.github.com/spec/v1 -oid sha256:337014604e03f79e26284b124ddeb846b0d2b7b559204da16b367427996cb741 -size 133416 +oid sha256:1d603486e69cbaf422aff6177dad2756f1141701b3eb2b225516e3dcd6723469 +size 102745 diff --git a/simulation/src/test/resources/slowdown_and_error_thresholds[CONCURRENCY_LIMITER_ROUND_ROBIN].png b/simulation/src/test/resources/slowdown_and_error_thresholds[CONCURRENCY_LIMITER_ROUND_ROBIN].png index 7cf2726ae..c86a7cb40 100644 --- a/simulation/src/test/resources/slowdown_and_error_thresholds[CONCURRENCY_LIMITER_ROUND_ROBIN].png +++ b/simulation/src/test/resources/slowdown_and_error_thresholds[CONCURRENCY_LIMITER_ROUND_ROBIN].png @@ -1,3 +1,3 @@ version https://git-lfs.github.com/spec/v1 -oid sha256:ded99e4cf11f4b35537791d501969bf8020145240b5241a62c2ccb3516d5d082 -size 116435 +oid sha256:1c588e31891e6feccb44e5c7b222dd9136059f619d6326d2da65a68ef338c4e0 +size 91557 diff --git a/simulation/src/test/resources/slowdown_and_error_thresholds[UNLIMITED_ROUND_ROBIN].png b/simulation/src/test/resources/slowdown_and_error_thresholds[UNLIMITED_ROUND_ROBIN].png index c60a5221a..5c0cd68df 100644 --- a/simulation/src/test/resources/slowdown_and_error_thresholds[UNLIMITED_ROUND_ROBIN].png +++ b/simulation/src/test/resources/slowdown_and_error_thresholds[UNLIMITED_ROUND_ROBIN].png @@ -1,3 +1,3 @@ version https://git-lfs.github.com/spec/v1 -oid sha256:5a1e9ae3deacd69bb9f0cb02057f4ac7af5d1388557362ea8c7174336cf2b8db -size 110209 +oid sha256:9eba92136268be9fbce8b6ab9083a2d971b27c7d341084bf02b7006e241774e3 +size 80204 diff --git a/simulation/src/test/resources/txt/all_nodes_500[CONCURRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].txt b/simulation/src/test/resources/txt/all_nodes_500[CONCURRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].txt index 20efdd3c1..7ae5f3dcf 100644 --- a/simulation/src/test/resources/txt/all_nodes_500[CONCURRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].txt +++ b/simulation/src/test/resources/txt/all_nodes_500[CONCURRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].txt @@ -1 +1 @@ -success=52.6% client_mean=PT1.027869274S server_cpu=PT16M44.041614481S client_received=2000/2000 server_resps=1673 codes={200=1051, 500=622, Failed to make a request=327} \ No newline at end of file +success=51.9% client_mean=PT1.032634888S server_cpu=PT16M43.011354078S client_received=2000/2000 server_resps=1662 codes={200=1037, 500=625, Failed to make a request=338} \ No newline at end of file diff --git a/simulation/src/test/resources/txt/drastic_slowdown[CONCURRENCY_LIMITER_PIN_UNTIL_ERROR].txt b/simulation/src/test/resources/txt/drastic_slowdown[CONCURRENCY_LIMITER_PIN_UNTIL_ERROR].txt index 0a149cd99..b802c8f38 100644 --- a/simulation/src/test/resources/txt/drastic_slowdown[CONCURRENCY_LIMITER_PIN_UNTIL_ERROR].txt +++ b/simulation/src/test/resources/txt/drastic_slowdown[CONCURRENCY_LIMITER_PIN_UNTIL_ERROR].txt @@ -1 +1 @@ -success=100.0% client_mean=PT2.053746297S server_cpu=PT2H16M53.159562742S client_received=4000/4000 server_resps=4000 codes={200=4000} \ No newline at end of file +success=100.0% client_mean=PT2.053746438S server_cpu=PT2H16M53.160125525S client_received=4000/4000 server_resps=4000 codes={200=4000} \ No newline at end of file diff --git a/simulation/src/test/resources/txt/fast_500s_then_revert[CONCURRENCY_LIMITER_PIN_UNTIL_ERROR].txt b/simulation/src/test/resources/txt/fast_500s_then_revert[CONCURRENCY_LIMITER_PIN_UNTIL_ERROR].txt index 8077268c6..5fa650be6 100644 --- a/simulation/src/test/resources/txt/fast_500s_then_revert[CONCURRENCY_LIMITER_PIN_UNTIL_ERROR].txt +++ b/simulation/src/test/resources/txt/fast_500s_then_revert[CONCURRENCY_LIMITER_PIN_UNTIL_ERROR].txt @@ -1 +1 @@ -success=99.7% client_mean=PT0.080628266S server_cpu=PT5M2.355999997S client_received=3750/3750 server_resps=3750 codes={200=3739, 500=11} \ No newline at end of file +success=99.7% client_mean=PT0.080628355S server_cpu=PT5M2.35633333S client_received=3750/3750 server_resps=3750 codes={200=3739, 500=11} \ No newline at end of file diff --git a/simulation/src/test/resources/txt/live_reloading[CONCURRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].txt b/simulation/src/test/resources/txt/live_reloading[CONCURRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].txt index 422b786ca..d59ae58c5 100644 --- a/simulation/src/test/resources/txt/live_reloading[CONCURRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].txt +++ b/simulation/src/test/resources/txt/live_reloading[CONCURRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].txt @@ -1 +1 @@ -success=59.2% client_mean=PT2.707428425S server_cpu=PT1H20M58.880261641S client_received=2500/2500 server_resps=1860 codes={200=1480, 500=380, Failed to make a request=640} \ No newline at end of file +success=59.3% client_mean=PT2.717022131S server_cpu=PT1H21M18.049029538S client_received=2500/2500 server_resps=1865 codes={200=1483, 500=382, Failed to make a request=635} \ No newline at end of file diff --git a/simulation/src/test/resources/txt/live_reloading[CONCURRENCY_LIMITER_PIN_UNTIL_ERROR].txt b/simulation/src/test/resources/txt/live_reloading[CONCURRENCY_LIMITER_PIN_UNTIL_ERROR].txt index e7fefd8d4..87ac9ec2b 100644 --- a/simulation/src/test/resources/txt/live_reloading[CONCURRENCY_LIMITER_PIN_UNTIL_ERROR].txt +++ b/simulation/src/test/resources/txt/live_reloading[CONCURRENCY_LIMITER_PIN_UNTIL_ERROR].txt @@ -1 +1 @@ -success=54.0% client_mean=PT2.903672277S server_cpu=PT1H42M35.433984304S client_received=2500/2500 server_resps=2177 codes={200=1349, 500=828, Failed to make a request=323} \ No newline at end of file +success=54.2% client_mean=PT2.907371536S server_cpu=PT1H42M40.098775697S client_received=2500/2500 server_resps=2177 codes={200=1355, 500=822, Failed to make a request=323} \ No newline at end of file diff --git a/simulation/src/test/resources/txt/live_reloading[CONCURRENCY_LIMITER_ROUND_ROBIN].txt b/simulation/src/test/resources/txt/live_reloading[CONCURRENCY_LIMITER_ROUND_ROBIN].txt index 928d0f0c6..f9d442f81 100644 --- a/simulation/src/test/resources/txt/live_reloading[CONCURRENCY_LIMITER_ROUND_ROBIN].txt +++ b/simulation/src/test/resources/txt/live_reloading[CONCURRENCY_LIMITER_ROUND_ROBIN].txt @@ -1 +1 @@ -success=52.1% client_mean=PT2.908560636S server_cpu=PT1H42M58.764371458S client_received=2500/2500 server_resps=2193 codes={200=1302, 500=891, Failed to make a request=307} \ No newline at end of file +success=52.0% client_mean=PT2.915893225S server_cpu=PT1H43M13.659598627S client_received=2500/2500 server_resps=2197 codes={200=1301, 500=896, Failed to make a request=303} \ No newline at end of file diff --git a/simulation/src/test/resources/txt/one_endpoint_dies_on_each_server[CONCURRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].txt b/simulation/src/test/resources/txt/one_endpoint_dies_on_each_server[CONCURRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].txt index 2df5e16dd..1db6ef93a 100644 --- a/simulation/src/test/resources/txt/one_endpoint_dies_on_each_server[CONCURRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].txt +++ b/simulation/src/test/resources/txt/one_endpoint_dies_on_each_server[CONCURRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].txt @@ -1 +1 @@ -success=47.0% client_mean=PT1.284406604S server_cpu=PT15M41.717604108S client_received=2500/2500 server_resps=1569 codes={200=1176, 500=393, Failed to make a request=931} \ No newline at end of file +success=46.4% client_mean=PT1.302954516S server_cpu=PT15M22.396603327S client_received=2500/2500 server_resps=1536 codes={200=1161, 500=375, Failed to make a request=964} \ No newline at end of file diff --git a/simulation/src/test/resources/txt/simplest_possible_case[CONCURRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].txt b/simulation/src/test/resources/txt/simplest_possible_case[CONCURRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].txt index 68dc39802..71f58f138 100644 --- a/simulation/src/test/resources/txt/simplest_possible_case[CONCURRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].txt +++ b/simulation/src/test/resources/txt/simplest_possible_case[CONCURRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].txt @@ -1 +1 @@ -success=100.0% client_mean=PT0.799939393S server_cpu=PT2H55M59.2S client_received=13200/13200 server_resps=13200 codes={200=13200} \ No newline at end of file +success=100.0% client_mean=PT0.799939398S server_cpu=PT2H55M59.2000635S client_received=13200/13200 server_resps=13200 codes={200=13200} \ No newline at end of file diff --git a/simulation/src/test/resources/txt/simplest_possible_case[CONCURRENCY_LIMITER_PIN_UNTIL_ERROR].txt b/simulation/src/test/resources/txt/simplest_possible_case[CONCURRENCY_LIMITER_PIN_UNTIL_ERROR].txt index 363d31abd..f779e471b 100644 --- a/simulation/src/test/resources/txt/simplest_possible_case[CONCURRENCY_LIMITER_PIN_UNTIL_ERROR].txt +++ b/simulation/src/test/resources/txt/simplest_possible_case[CONCURRENCY_LIMITER_PIN_UNTIL_ERROR].txt @@ -1 +1 @@ -success=100.0% client_mean=PT0.998696969S server_cpu=PT3H39M42.8S client_received=13200/13200 server_resps=13200 codes={200=13200} \ No newline at end of file +success=100.0% client_mean=PT0.998702894S server_cpu=PT3H39M42.87820128S client_received=13200/13200 server_resps=13200 codes={200=13200} \ No newline at end of file diff --git a/simulation/src/test/resources/txt/simplest_possible_case[CONCURRENCY_LIMITER_ROUND_ROBIN].txt b/simulation/src/test/resources/txt/simplest_possible_case[CONCURRENCY_LIMITER_ROUND_ROBIN].txt index 68dc39802..71f58f138 100644 --- a/simulation/src/test/resources/txt/simplest_possible_case[CONCURRENCY_LIMITER_ROUND_ROBIN].txt +++ b/simulation/src/test/resources/txt/simplest_possible_case[CONCURRENCY_LIMITER_ROUND_ROBIN].txt @@ -1 +1 @@ -success=100.0% client_mean=PT0.799939393S server_cpu=PT2H55M59.2S client_received=13200/13200 server_resps=13200 codes={200=13200} \ No newline at end of file +success=100.0% client_mean=PT0.799939398S server_cpu=PT2H55M59.2000635S client_received=13200/13200 server_resps=13200 codes={200=13200} \ No newline at end of file diff --git a/simulation/src/test/resources/txt/simplest_possible_case[UNLIMITED_ROUND_ROBIN].txt b/simulation/src/test/resources/txt/simplest_possible_case[UNLIMITED_ROUND_ROBIN].txt index 68dc39802..71f58f138 100644 --- a/simulation/src/test/resources/txt/simplest_possible_case[UNLIMITED_ROUND_ROBIN].txt +++ b/simulation/src/test/resources/txt/simplest_possible_case[UNLIMITED_ROUND_ROBIN].txt @@ -1 +1 @@ -success=100.0% client_mean=PT0.799939393S server_cpu=PT2H55M59.2S client_received=13200/13200 server_resps=13200 codes={200=13200} \ No newline at end of file +success=100.0% client_mean=PT0.799939398S server_cpu=PT2H55M59.2000635S client_received=13200/13200 server_resps=13200 codes={200=13200} \ No newline at end of file diff --git a/simulation/src/test/resources/txt/slow_503s_then_revert[CONCURRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].txt b/simulation/src/test/resources/txt/slow_503s_then_revert[CONCURRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].txt index 9a05293ca..47cd1c3d8 100644 --- a/simulation/src/test/resources/txt/slow_503s_then_revert[CONCURRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].txt +++ b/simulation/src/test/resources/txt/slow_503s_then_revert[CONCURRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].txt @@ -1 +1 @@ -success=100.0% client_mean=PT0.302783916S server_cpu=PT14M44.375304373S client_received=3000/3000 server_resps=3175 codes={200=3000} \ No newline at end of file +success=100.0% client_mean=PT0.30283067S server_cpu=PT14M44.486985327S client_received=3000/3000 server_resps=3175 codes={200=3000} \ No newline at end of file diff --git a/simulation/src/test/resources/txt/slow_503s_then_revert[CONCURRENCY_LIMITER_PIN_UNTIL_ERROR].txt b/simulation/src/test/resources/txt/slow_503s_then_revert[CONCURRENCY_LIMITER_PIN_UNTIL_ERROR].txt index 2e3aa5ba1..fa0ff7664 100644 --- a/simulation/src/test/resources/txt/slow_503s_then_revert[CONCURRENCY_LIMITER_PIN_UNTIL_ERROR].txt +++ b/simulation/src/test/resources/txt/slow_503s_then_revert[CONCURRENCY_LIMITER_PIN_UNTIL_ERROR].txt @@ -1 +1 @@ -success=100.0% client_mean=PT0.346572928S server_cpu=PT16M51.987159344S client_received=3000/3000 server_resps=3197 codes={200=3000} \ No newline at end of file +success=100.0% client_mean=PT0.346630773S server_cpu=PT16M52.121612924S client_received=3000/3000 server_resps=3197 codes={200=3000} \ No newline at end of file diff --git a/simulation/src/test/resources/txt/slow_503s_then_revert[CONCURRENCY_LIMITER_ROUND_ROBIN].txt b/simulation/src/test/resources/txt/slow_503s_then_revert[CONCURRENCY_LIMITER_ROUND_ROBIN].txt index 39248acea..0ec1120fb 100644 --- a/simulation/src/test/resources/txt/slow_503s_then_revert[CONCURRENCY_LIMITER_ROUND_ROBIN].txt +++ b/simulation/src/test/resources/txt/slow_503s_then_revert[CONCURRENCY_LIMITER_ROUND_ROBIN].txt @@ -1 +1 @@ -success=100.0% client_mean=PT0.745544946S server_cpu=PT36M23.421729078S client_received=3000/3000 server_resps=3411 codes={200=3000} \ No newline at end of file +success=100.0% client_mean=PT0.745660042S server_cpu=PT36M23.70242803S client_received=3000/3000 server_resps=3411 codes={200=3000} \ No newline at end of file diff --git a/simulation/src/test/resources/txt/slow_503s_then_revert[UNLIMITED_ROUND_ROBIN].txt b/simulation/src/test/resources/txt/slow_503s_then_revert[UNLIMITED_ROUND_ROBIN].txt index 5cfeced77..dbb005d9a 100644 --- a/simulation/src/test/resources/txt/slow_503s_then_revert[UNLIMITED_ROUND_ROBIN].txt +++ b/simulation/src/test/resources/txt/slow_503s_then_revert[UNLIMITED_ROUND_ROBIN].txt @@ -1 +1 @@ -success=100.0% client_mean=PT1.43038075S server_cpu=PT1H9M49.707166439S client_received=3000/3000 server_resps=3802 codes={200=3000} \ No newline at end of file +success=100.0% client_mean=PT1.430720789S server_cpu=PT1H9M50.430528642S client_received=3000/3000 server_resps=3802 codes={200=3000} \ No newline at end of file diff --git a/simulation/src/test/resources/txt/slowdown_and_error_thresholds[CONCURRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].txt b/simulation/src/test/resources/txt/slowdown_and_error_thresholds[CONCURRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].txt index 43e6c7f90..5089265e9 100644 --- a/simulation/src/test/resources/txt/slowdown_and_error_thresholds[CONCURRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].txt +++ b/simulation/src/test/resources/txt/slowdown_and_error_thresholds[CONCURRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].txt @@ -1 +1 @@ -success=3.3% client_mean=PT2.227891278S server_cpu=PT1H40M10.840754535S client_received=10000/10000 server_resps=1617 codes={200=327, 500=1290, Failed to make a request=8383} \ No newline at end of file +success=3.4% client_mean=PT2.255528943S server_cpu=PT1H43M47.583860348S client_received=10000/10000 server_resps=1676 codes={200=344, 500=1332, Failed to make a request=8324} \ No newline at end of file diff --git a/simulation/src/test/resources/txt/slowdown_and_error_thresholds[CONCURRENCY_LIMITER_PIN_UNTIL_ERROR].txt b/simulation/src/test/resources/txt/slowdown_and_error_thresholds[CONCURRENCY_LIMITER_PIN_UNTIL_ERROR].txt index 75f8cb42f..758fdfba2 100644 --- a/simulation/src/test/resources/txt/slowdown_and_error_thresholds[CONCURRENCY_LIMITER_PIN_UNTIL_ERROR].txt +++ b/simulation/src/test/resources/txt/slowdown_and_error_thresholds[CONCURRENCY_LIMITER_PIN_UNTIL_ERROR].txt @@ -1 +1 @@ -success=1.2% client_mean=PT2.916010959S server_cpu=PT4H30M39.598982029S client_received=10000/10000 server_resps=4255 codes={200=120, 500=4135, Failed to make a request=5745} \ No newline at end of file +success=1.2% client_mean=PT2.928958527S server_cpu=PT4H31M58.68391554S client_received=10000/10000 server_resps=4264 codes={200=120, 500=4144, Failed to make a request=5736} \ No newline at end of file diff --git a/simulation/src/test/resources/txt/slowdown_and_error_thresholds[CONCURRENCY_LIMITER_ROUND_ROBIN].txt b/simulation/src/test/resources/txt/slowdown_and_error_thresholds[CONCURRENCY_LIMITER_ROUND_ROBIN].txt index 281fc3f15..3c8a33385 100644 --- a/simulation/src/test/resources/txt/slowdown_and_error_thresholds[CONCURRENCY_LIMITER_ROUND_ROBIN].txt +++ b/simulation/src/test/resources/txt/slowdown_and_error_thresholds[CONCURRENCY_LIMITER_ROUND_ROBIN].txt @@ -1 +1 @@ -success=1.2% client_mean=PT2.990999813S server_cpu=PT4H41M15.507187288S client_received=10000/10000 server_resps=4433 codes={200=120, 500=4313, Failed to make a request=5567} \ No newline at end of file +success=1.2% client_mean=PT2.987017569S server_cpu=PT4H40M48.239132959S client_received=10000/10000 server_resps=4417 codes={200=120, 500=4297, Failed to make a request=5583} \ No newline at end of file diff --git a/simulation/src/test/resources/txt/slowdown_and_error_thresholds[UNLIMITED_ROUND_ROBIN].txt b/simulation/src/test/resources/txt/slowdown_and_error_thresholds[UNLIMITED_ROUND_ROBIN].txt index 319ba6acb..92b1cdc70 100644 --- a/simulation/src/test/resources/txt/slowdown_and_error_thresholds[UNLIMITED_ROUND_ROBIN].txt +++ b/simulation/src/test/resources/txt/slowdown_and_error_thresholds[UNLIMITED_ROUND_ROBIN].txt @@ -1 +1 @@ -success=1.2% client_mean=PT3.974119999S server_cpu=PT11H2M21.19999998S client_received=10000/10000 server_resps=10000 codes={200=120, 500=9880} \ No newline at end of file +success=1.2% client_mean=PT3.974129199S server_cpu=PT11H2M21.291999888S client_received=10000/10000 server_resps=10000 codes={200=120, 500=9880} \ No newline at end of file diff --git a/simulation/src/test/resources/txt/uncommon_flakes[CONCURRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].txt b/simulation/src/test/resources/txt/uncommon_flakes[CONCURRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].txt index b02247270..ab7b1ce60 100644 --- a/simulation/src/test/resources/txt/uncommon_flakes[CONCURRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].txt +++ b/simulation/src/test/resources/txt/uncommon_flakes[CONCURRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].txt @@ -1 +1 @@ -success=93.6% client_mean=PT0.210193295S server_cpu=PT6.024159232S client_received=9730/10000 server_resps=9456 codes={200=9363, 500=93, Failed to make a request=274} \ No newline at end of file +success=93.5% client_mean=PT0.203113054S server_cpu=PT7.376462491S client_received=9774/10000 server_resps=9441 codes={200=9348, 500=93, Failed to make a request=333} \ No newline at end of file diff --git a/simulation/src/test/resources/uncommon_flakes[CONCURRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].png b/simulation/src/test/resources/uncommon_flakes[CONCURRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].png index 1886d064c..cec7d034c 100644 --- a/simulation/src/test/resources/uncommon_flakes[CONCURRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].png +++ b/simulation/src/test/resources/uncommon_flakes[CONCURRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].png @@ -1,3 +1,3 @@ version https://git-lfs.github.com/spec/v1 -oid sha256:31decac5193da2cae57af434398fcd4385e62068b76a1fad269080db18b95afb -size 193416 +oid sha256:e7bba1f20d180ba0c54cf2a45c739ac6ad5fbd722eb22963481e765508206710 +size 128863 From 628bc346c6a28859e38fe0628f2dd84f7187f85b Mon Sep 17 00:00:00 2001 From: Carter Kozak Date: Wed, 26 Feb 2020 10:56:17 -0500 Subject: [PATCH 14/17] comment again --- .../palantir/dialogue/core/TestCaffeineTicker.java | 11 ++++------- ...500[CONCURRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].png | 4 ++-- ...ing[CONCURRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].png | 4 ++-- ...reloading[CONCURRENCY_LIMITER_PIN_UNTIL_ERROR].png | 4 ++-- ...ive_reloading[CONCURRENCY_LIMITER_ROUND_ROBIN].png | 4 ++-- ...ver[CONCURRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].png | 4 ++-- ...ert[CONCURRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].png | 4 ++-- ...en_revert[CONCURRENCY_LIMITER_PIN_UNTIL_ERROR].png | 4 ++-- ...s_then_revert[CONCURRENCY_LIMITER_ROUND_ROBIN].png | 4 ++-- .../slow_503s_then_revert[UNLIMITED_ROUND_ROBIN].png | 4 ++-- ...lds[CONCURRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].png | 4 ++-- ...hresholds[CONCURRENCY_LIMITER_PIN_UNTIL_ERROR].png | 4 ++-- ...or_thresholds[CONCURRENCY_LIMITER_ROUND_ROBIN].png | 4 ++-- ...kes[CONCURRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].png | 4 ++-- 14 files changed, 30 insertions(+), 33 deletions(-) diff --git a/simulation/src/main/java/com/palantir/dialogue/core/TestCaffeineTicker.java b/simulation/src/main/java/com/palantir/dialogue/core/TestCaffeineTicker.java index ac849758d..b73879218 100644 --- a/simulation/src/main/java/com/palantir/dialogue/core/TestCaffeineTicker.java +++ b/simulation/src/main/java/com/palantir/dialogue/core/TestCaffeineTicker.java @@ -16,9 +16,6 @@ package com.palantir.dialogue.core; import com.github.benmanes.caffeine.cache.Ticker; -import com.palantir.logsafe.Preconditions; -import com.palantir.logsafe.SafeArg; -import java.time.Duration; import java.util.concurrent.TimeUnit; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -36,10 +33,10 @@ public long read() { void advanceTo(long newNanos) { if (newNanos < nanos) { long difference = nanos - newNanos; - Preconditions.checkState( - difference < Duration.ofMillis(1).toNanos(), - "Large time rewind - this is likely a bug in the test harness", - SafeArg.of("difference", difference)); + // Preconditions.checkState( + // difference < Duration.ofMillis(1).toNanos(), + // "Large time rewind - this is likely a bug in the test harness", + // SafeArg.of("difference", difference)); log.debug( "Tried to rewind time by {} micros - no-op as this is deterministic and harmless", TimeUnit.MICROSECONDS.convert(difference, TimeUnit.NANOSECONDS)); diff --git a/simulation/src/test/resources/all_nodes_500[CONCURRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].png b/simulation/src/test/resources/all_nodes_500[CONCURRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].png index b30eabae1..cedcf5cf0 100644 --- a/simulation/src/test/resources/all_nodes_500[CONCURRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].png +++ b/simulation/src/test/resources/all_nodes_500[CONCURRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].png @@ -1,3 +1,3 @@ version https://git-lfs.github.com/spec/v1 -oid sha256:9c24fab00c7c62446e2b515554d66dfb09faf8144d78845c6dc072837ef1875c -size 133376 +oid sha256:8e65ad6138cb108b12069368d43ee9fa3bebf947b7d26491d38210ca659e69dc +size 179489 diff --git a/simulation/src/test/resources/live_reloading[CONCURRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].png b/simulation/src/test/resources/live_reloading[CONCURRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].png index df48e8a5a..30511bcea 100644 --- a/simulation/src/test/resources/live_reloading[CONCURRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].png +++ b/simulation/src/test/resources/live_reloading[CONCURRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].png @@ -1,3 +1,3 @@ version https://git-lfs.github.com/spec/v1 -oid sha256:4bbc6bc614dfc8e04b01600cc3034bc5cf511027eaeb72ec64c8cf148c5a7447 -size 101878 +oid sha256:faa3e3081d7668bc44cf7c4118ce91e0a275ca9b59a4164a05c198e437de559c +size 134225 diff --git a/simulation/src/test/resources/live_reloading[CONCURRENCY_LIMITER_PIN_UNTIL_ERROR].png b/simulation/src/test/resources/live_reloading[CONCURRENCY_LIMITER_PIN_UNTIL_ERROR].png index 65dd09784..61730f03f 100644 --- a/simulation/src/test/resources/live_reloading[CONCURRENCY_LIMITER_PIN_UNTIL_ERROR].png +++ b/simulation/src/test/resources/live_reloading[CONCURRENCY_LIMITER_PIN_UNTIL_ERROR].png @@ -1,3 +1,3 @@ version https://git-lfs.github.com/spec/v1 -oid sha256:bce2af3b1d7313311c72e26aff06ec3e4a0d715cc3b670bbbe1804605dfb4b3c -size 96317 +oid sha256:cbc7dd1ecc3d4b58f3cd2ff4fe33340c1e985bb369f3021b032a59491d77b749 +size 125651 diff --git a/simulation/src/test/resources/live_reloading[CONCURRENCY_LIMITER_ROUND_ROBIN].png b/simulation/src/test/resources/live_reloading[CONCURRENCY_LIMITER_ROUND_ROBIN].png index 603c53447..686c2daf6 100644 --- a/simulation/src/test/resources/live_reloading[CONCURRENCY_LIMITER_ROUND_ROBIN].png +++ b/simulation/src/test/resources/live_reloading[CONCURRENCY_LIMITER_ROUND_ROBIN].png @@ -1,3 +1,3 @@ version https://git-lfs.github.com/spec/v1 -oid sha256:703aa9b8cd3ec7099c513ad74204278eba4d9fe0da56c28d3538c7e1d4574577 -size 95611 +oid sha256:4bcd3efff7e36d5668b6459e4cc35de30ab0550bb04d76ddb7d97e7835cb1573 +size 124186 diff --git a/simulation/src/test/resources/one_endpoint_dies_on_each_server[CONCURRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].png b/simulation/src/test/resources/one_endpoint_dies_on_each_server[CONCURRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].png index c9bf6cbb5..3adb8c3af 100644 --- a/simulation/src/test/resources/one_endpoint_dies_on_each_server[CONCURRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].png +++ b/simulation/src/test/resources/one_endpoint_dies_on_each_server[CONCURRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].png @@ -1,3 +1,3 @@ version https://git-lfs.github.com/spec/v1 -oid sha256:511590a3c0ffdcd2e06aea8908acd0aa152895e3021d85fce754943bd7dd0ab3 -size 126298 +oid sha256:69ba98cd730d079c3321506085324428e1f0efb7493aa99636b4adb9965a2aee +size 161896 diff --git a/simulation/src/test/resources/slow_503s_then_revert[CONCURRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].png b/simulation/src/test/resources/slow_503s_then_revert[CONCURRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].png index ecb0d13f6..9b4019a59 100644 --- a/simulation/src/test/resources/slow_503s_then_revert[CONCURRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].png +++ b/simulation/src/test/resources/slow_503s_then_revert[CONCURRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].png @@ -1,3 +1,3 @@ version https://git-lfs.github.com/spec/v1 -oid sha256:3c8be79ecd24d91ea775788459db4692eb0ba15170e4db90bb66205b700e25c1 -size 77254 +oid sha256:d0b2e2e4485b4f1cd08845cccfe74d70fd69776ef87c819dd7551b544ee3f78f +size 102738 diff --git a/simulation/src/test/resources/slow_503s_then_revert[CONCURRENCY_LIMITER_PIN_UNTIL_ERROR].png b/simulation/src/test/resources/slow_503s_then_revert[CONCURRENCY_LIMITER_PIN_UNTIL_ERROR].png index ac6edba3c..3c6db795a 100644 --- a/simulation/src/test/resources/slow_503s_then_revert[CONCURRENCY_LIMITER_PIN_UNTIL_ERROR].png +++ b/simulation/src/test/resources/slow_503s_then_revert[CONCURRENCY_LIMITER_PIN_UNTIL_ERROR].png @@ -1,3 +1,3 @@ version https://git-lfs.github.com/spec/v1 -oid sha256:c2ef5bde030775058c6fcbd439285ac8507a0ff274cb5e6191c11d7ce5540991 -size 67624 +oid sha256:fb53c89a5f7f22cecdfdc52e2ed90506900813d2c5d4c37360c5827ecc23bec0 +size 97013 diff --git a/simulation/src/test/resources/slow_503s_then_revert[CONCURRENCY_LIMITER_ROUND_ROBIN].png b/simulation/src/test/resources/slow_503s_then_revert[CONCURRENCY_LIMITER_ROUND_ROBIN].png index a141f623e..14919381c 100644 --- a/simulation/src/test/resources/slow_503s_then_revert[CONCURRENCY_LIMITER_ROUND_ROBIN].png +++ b/simulation/src/test/resources/slow_503s_then_revert[CONCURRENCY_LIMITER_ROUND_ROBIN].png @@ -1,3 +1,3 @@ version https://git-lfs.github.com/spec/v1 -oid sha256:c4b2b426b34f2259bac4acf0b2832f5d511118cc8350f365101e41af41b02535 -size 75729 +oid sha256:c1c5deca4e73be049ccf9f4d573d2965eb5aa86eb869192307f28c68e99a985b +size 99949 diff --git a/simulation/src/test/resources/slow_503s_then_revert[UNLIMITED_ROUND_ROBIN].png b/simulation/src/test/resources/slow_503s_then_revert[UNLIMITED_ROUND_ROBIN].png index f076ab457..8b167214d 100644 --- a/simulation/src/test/resources/slow_503s_then_revert[UNLIMITED_ROUND_ROBIN].png +++ b/simulation/src/test/resources/slow_503s_then_revert[UNLIMITED_ROUND_ROBIN].png @@ -1,3 +1,3 @@ version https://git-lfs.github.com/spec/v1 -oid sha256:8727fb4d90bbcdba6756dbaa716804d92a96459a35e18abb272d1bf1c1d54c1b -size 73043 +oid sha256:e3b21460fb804cf9c7e61176e15eefe49e77f197759125ed7206bfa66479f953 +size 96628 diff --git a/simulation/src/test/resources/slowdown_and_error_thresholds[CONCURRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].png b/simulation/src/test/resources/slowdown_and_error_thresholds[CONCURRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].png index 9758ab01c..d2c433db8 100644 --- a/simulation/src/test/resources/slowdown_and_error_thresholds[CONCURRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].png +++ b/simulation/src/test/resources/slowdown_and_error_thresholds[CONCURRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].png @@ -1,3 +1,3 @@ version https://git-lfs.github.com/spec/v1 -oid sha256:727828c804562522d61dc8a8d3c8cc689b5514dacefd78e6b38e93c6efaf8b6a -size 113667 +oid sha256:aa260c5b42b1807933d0a83ce450d7a686493220fc481672d973a17c267c2aee +size 141914 diff --git a/simulation/src/test/resources/slowdown_and_error_thresholds[CONCURRENCY_LIMITER_PIN_UNTIL_ERROR].png b/simulation/src/test/resources/slowdown_and_error_thresholds[CONCURRENCY_LIMITER_PIN_UNTIL_ERROR].png index d5e071b4f..c9058cc49 100644 --- a/simulation/src/test/resources/slowdown_and_error_thresholds[CONCURRENCY_LIMITER_PIN_UNTIL_ERROR].png +++ b/simulation/src/test/resources/slowdown_and_error_thresholds[CONCURRENCY_LIMITER_PIN_UNTIL_ERROR].png @@ -1,3 +1,3 @@ version https://git-lfs.github.com/spec/v1 -oid sha256:1d603486e69cbaf422aff6177dad2756f1141701b3eb2b225516e3dcd6723469 -size 102745 +oid sha256:3019e6e78c94557271633b930f569edafc92de1ba0ce9e709de5bc50799a1f20 +size 134706 diff --git a/simulation/src/test/resources/slowdown_and_error_thresholds[CONCURRENCY_LIMITER_ROUND_ROBIN].png b/simulation/src/test/resources/slowdown_and_error_thresholds[CONCURRENCY_LIMITER_ROUND_ROBIN].png index c86a7cb40..69f5e0eb1 100644 --- a/simulation/src/test/resources/slowdown_and_error_thresholds[CONCURRENCY_LIMITER_ROUND_ROBIN].png +++ b/simulation/src/test/resources/slowdown_and_error_thresholds[CONCURRENCY_LIMITER_ROUND_ROBIN].png @@ -1,3 +1,3 @@ version https://git-lfs.github.com/spec/v1 -oid sha256:1c588e31891e6feccb44e5c7b222dd9136059f619d6326d2da65a68ef338c4e0 -size 91557 +oid sha256:8c9e197a1af30696df9eb81d5549fd89d2d1c13d369b3c58acd6bdeeceb8d9cd +size 116519 diff --git a/simulation/src/test/resources/uncommon_flakes[CONCURRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].png b/simulation/src/test/resources/uncommon_flakes[CONCURRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].png index cec7d034c..81bff56b5 100644 --- a/simulation/src/test/resources/uncommon_flakes[CONCURRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].png +++ b/simulation/src/test/resources/uncommon_flakes[CONCURRENCY_LIMITER_BLACKLIST_ROUND_ROBIN].png @@ -1,3 +1,3 @@ version https://git-lfs.github.com/spec/v1 -oid sha256:e7bba1f20d180ba0c54cf2a45c739ac6ad5fbd722eb22963481e765508206710 -size 128863 +oid sha256:972af6e8776e3ab51171e87d22a2c90d41a23a02c82d3baa0c681c98497057f4 +size 184380 From 91456c6f88f4c1fb3d0e1822a067384c2cbaa2c7 Mon Sep 17 00:00:00 2001 From: Carter Kozak Date: Wed, 26 Feb 2020 11:20:50 -0500 Subject: [PATCH 15/17] delete unused --- .../dialogue/core/BackoffStrategy.java | 26 ------------ .../core/ExponentialBackoffStrategy.java | 40 ------------------- 2 files changed, 66 deletions(-) delete mode 100644 dialogue-core/src/main/java/com/palantir/dialogue/core/BackoffStrategy.java delete mode 100644 dialogue-core/src/main/java/com/palantir/dialogue/core/ExponentialBackoffStrategy.java diff --git a/dialogue-core/src/main/java/com/palantir/dialogue/core/BackoffStrategy.java b/dialogue-core/src/main/java/com/palantir/dialogue/core/BackoffStrategy.java deleted file mode 100644 index 5800fffc7..000000000 --- a/dialogue-core/src/main/java/com/palantir/dialogue/core/BackoffStrategy.java +++ /dev/null @@ -1,26 +0,0 @@ -/* - * (c) Copyright 2020 Palantir Technologies Inc. All rights reserved. - * - * Licensed 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 com.palantir.dialogue.core; - -import java.time.Duration; - -/** Defines a strategy for waiting in between successive retries of an operation that is subject to failure. */ -interface BackoffStrategy { - - /** Returns the next suggested backoff duration. */ - Duration backoffDuration(int failures); -} diff --git a/dialogue-core/src/main/java/com/palantir/dialogue/core/ExponentialBackoffStrategy.java b/dialogue-core/src/main/java/com/palantir/dialogue/core/ExponentialBackoffStrategy.java deleted file mode 100644 index 71383d2c0..000000000 --- a/dialogue-core/src/main/java/com/palantir/dialogue/core/ExponentialBackoffStrategy.java +++ /dev/null @@ -1,40 +0,0 @@ -/* - * (c) Copyright 2020 Palantir Technologies Inc. All rights reserved. - * - * Licensed 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 com.palantir.dialogue.core; - -import java.time.Duration; -import java.util.function.DoubleSupplier; - -final class ExponentialBackoffStrategy implements BackoffStrategy { - - private final Duration backoffSlotSize; - private final DoubleSupplier random; - - ExponentialBackoffStrategy(Duration backoffSlotSize, DoubleSupplier random) { - this.backoffSlotSize = backoffSlotSize; - this.random = random; - } - - @Override - public Duration backoffDuration(int failures) { - if (failures == 0) { - return Duration.ZERO; - } - int upperBound = (int) Math.pow(2, failures - 1); - return Duration.ofNanos(Math.round(backoffSlotSize.toNanos() * random.getAsDouble() * upperBound)); - } -} From 45623975cbab8f1366c0b0dc6e3014c756ec5041 Mon Sep 17 00:00:00 2001 From: Carter Kozak Date: Wed, 26 Feb 2020 11:28:41 -0500 Subject: [PATCH 16/17] Log backoff milliseconds --- .../java/com/palantir/dialogue/core/RetryingChannel.java | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/dialogue-core/src/main/java/com/palantir/dialogue/core/RetryingChannel.java b/dialogue-core/src/main/java/com/palantir/dialogue/core/RetryingChannel.java index 54b16837b..d2771c574 100644 --- a/dialogue-core/src/main/java/com/palantir/dialogue/core/RetryingChannel.java +++ b/dialogue-core/src/main/java/com/palantir/dialogue/core/RetryingChannel.java @@ -141,7 +141,8 @@ ListenableFuture success(Response response) { } if (log.isDebugEnabled()) { log.debug( - "Retries exhausted, returning a retryable response with status {}", + "Exhausted {} retries, returning a retryable response with status {}", + SafeArg.of("retries", maxRetries), SafeArg.of("status", response.code())); } return Futures.immediateFuture(response); @@ -165,7 +166,7 @@ private void logRetry(long backoffNanoseconds, Throwable throwable) { "Retrying call after failure", SafeArg.of("failures", failures), SafeArg.of("maxRetries", maxRetries), - SafeArg.of("backoffNanoseconds", backoffNanoseconds), + SafeArg.of("backoffMillis", TimeUnit.NANOSECONDS.toMillis(backoffNanoseconds)), SafeArg.of("serviceName", endpoint.serviceName()), SafeArg.of("endpoint", endpoint.endpointName()), throwable); From f4f999a6325814ebbe64f6b5920c2fd9977e316a Mon Sep 17 00:00:00 2001 From: Carter Kozak Date: Wed, 26 Feb 2020 11:31:10 -0500 Subject: [PATCH 17/17] queuedChannelAndRetrying -> retryingChannel --- .../test/java/com/palantir/dialogue/core/Strategy.java | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/simulation/src/test/java/com/palantir/dialogue/core/Strategy.java b/simulation/src/test/java/com/palantir/dialogue/core/Strategy.java index 2d5af8fee..bc64d033d 100644 --- a/simulation/src/test/java/com/palantir/dialogue/core/Strategy.java +++ b/simulation/src/test/java/com/palantir/dialogue/core/Strategy.java @@ -60,7 +60,7 @@ private static Channel concurrencyLimiter(Simulation sim, Supplier new BlacklistingChannel(c, Duration.ofSeconds(1), sim.clock())) .collect(Collectors.toList()); LimitedChannel limited1 = new RoundRobinChannel(limitedChannels); - return queuedChannelAndRetrying(sim, limited1); + return retryingChannel(sim, limited1); }); } @@ -86,7 +86,7 @@ private static Channel pinUntilError(Simulation sim, Supplier addFixedLimiter(Simulati return channel -> new FixedLimitedChannel(channel, 256, DialogueClientMetrics.of(sim.taggedMetrics())); } - private static Channel queuedChannelAndRetrying(Simulation sim, LimitedChannel limited) { + private static Channel retryingChannel(Simulation sim, LimitedChannel limited) { LimitedChannel limited1 = instrumentClient(limited, sim.taggedMetrics()); return new RetryingChannel( new LimitedChannelToChannelAdapter(limited1),