From 72534d7d0570ae4da1a356e1e23354e67453e1e1 Mon Sep 17 00:00:00 2001 From: Kyle Vosper Date: Wed, 27 Feb 2019 14:43:11 +0000 Subject: [PATCH 1/7] Squashed many commits - change how styx starts up, allow us to decouple starting of different services --- .../service/spi/AbstractStyxService.java | 38 ++-- .../extension/service/spi/StyxService.java | 82 ++++++-- .../service/spi/AbstractStyxServiceTest.java | 43 ++++- ...alyExcludingOriginHealthStatusMonitor.java | 2 +- .../monitors/NoOriginHealthStatusMonitor.java | 2 +- .../styx/common/FailureHandlingStrategy.java | 145 -------------- .../hotels/styx/common/SequenceProcessor.java | 179 ++++++++++++++++++ .../common/lambdas/RecordingSupplier.java | 78 ++++++++ .../lambdas/SupplierWithCheckedException.java | 58 ++++++ .../lambdas/UncheckedWrapperException.java | 35 ++++ .../styx/common/SequenceProcessorTest.java | 76 ++++++++ .../common/lambdas/RecordingSupplierTest.java | 79 ++++++++ .../main/java/com/hotels/styx/StyxServer.java | 110 +++++++++-- .../hotels/styx/admin/AdminServerBuilder.java | 3 + .../admin/handlers/StartupStatusHandler.java | 93 +++++++++ .../graphite/GraphiteReporterService.java | 12 +- .../hotels/styx/proxy/ProxyServerBuilder.java | 10 +- .../hotels/styx/startup/PipelineFactory.java | 7 +- .../styx/startup/PluginStartupService.java | 102 ++++++++++ .../hotels/styx/startup/ProxyServerSetUp.java | 61 ++++-- .../startup/ProxyStatusNotifications.java | 95 ++++++++++ .../hotels/styx/startup/ServerService.java | 105 ++++++++++ .../hotels/styx/startup/StartupException.java | 36 ++++ .../styx/startup/StyxPipelineFactory.java | 11 +- .../styx/startup/StyxServerComponents.java | 14 +- .../startup/extensions/FailureHandling.java | 88 --------- .../extensions/PluginLoadingForStartup.java | 87 ++++++++- .../extensions/PluginStatusNotifications.java | 93 +++++++++ .../java/com/hotels/styx/StyxServerTest.java | 58 +++--- .../handlers/StartupStatusHandlerTest.java | 81 ++++++++ .../graphite/GraphiteReporterServiceTest.java | 4 +- .../startup/PluginStartupServiceTest.java | 165 ++++++++++++++++ .../styx/startup/ProxyServerSetUpTest.java | 75 ++------ .../styx/startup/ServerServiceTest.java | 105 ++++++++++ .../startup/StyxServerComponentsTest.java | 16 -- .../PluginLoadingForStartupTest.java | 29 +-- .../hotels/styx/server/netty/NettyServer.java | 20 +- .../styx/server/netty/NettyServerBuilder.java | 17 +- distribution/conf/default.yml | 24 +-- .../com/hotels/styx/testapi/StyxServer.java | 15 ++ .../com/hotels/styx/startup/StartupTest.java | 151 +++++++++++++++ .../plugins/PluginAdminInterfaceSpec.scala | 17 ++ 42 files changed, 2004 insertions(+), 517 deletions(-) delete mode 100644 components/common/src/main/java/com/hotels/styx/common/FailureHandlingStrategy.java create mode 100644 components/common/src/main/java/com/hotels/styx/common/SequenceProcessor.java create mode 100644 components/common/src/main/java/com/hotels/styx/common/lambdas/RecordingSupplier.java create mode 100644 components/common/src/main/java/com/hotels/styx/common/lambdas/SupplierWithCheckedException.java create mode 100644 components/common/src/main/java/com/hotels/styx/common/lambdas/UncheckedWrapperException.java create mode 100644 components/common/src/test/java/com/hotels/styx/common/SequenceProcessorTest.java create mode 100644 components/common/src/test/java/com/hotels/styx/common/lambdas/RecordingSupplierTest.java create mode 100644 components/proxy/src/main/java/com/hotels/styx/admin/handlers/StartupStatusHandler.java create mode 100644 components/proxy/src/main/java/com/hotels/styx/startup/PluginStartupService.java create mode 100644 components/proxy/src/main/java/com/hotels/styx/startup/ProxyStatusNotifications.java create mode 100644 components/proxy/src/main/java/com/hotels/styx/startup/ServerService.java create mode 100644 components/proxy/src/main/java/com/hotels/styx/startup/StartupException.java delete mode 100644 components/proxy/src/main/java/com/hotels/styx/startup/extensions/FailureHandling.java create mode 100644 components/proxy/src/main/java/com/hotels/styx/startup/extensions/PluginStatusNotifications.java create mode 100644 components/proxy/src/test/java/com/hotels/styx/admin/handlers/StartupStatusHandlerTest.java create mode 100644 components/proxy/src/test/java/com/hotels/styx/startup/PluginStartupServiceTest.java create mode 100644 components/proxy/src/test/java/com/hotels/styx/startup/ServerServiceTest.java create mode 100644 system-tests/e2e-suite/src/test/java/com/hotels/styx/startup/StartupTest.java diff --git a/components/api/src/main/java/com/hotels/styx/api/extension/service/spi/AbstractStyxService.java b/components/api/src/main/java/com/hotels/styx/api/extension/service/spi/AbstractStyxService.java index ea45fe3365..0f83d83613 100644 --- a/components/api/src/main/java/com/hotels/styx/api/extension/service/spi/AbstractStyxService.java +++ b/components/api/src/main/java/com/hotels/styx/api/extension/service/spi/AbstractStyxService.java @@ -1,5 +1,5 @@ /* - Copyright (C) 2013-2018 Expedia Inc. + Copyright (C) 2013-2019 Expedia Inc. Licensed under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with the License. @@ -18,11 +18,11 @@ import com.google.common.collect.ImmutableMap; import com.hotels.styx.api.Eventual; import com.hotels.styx.api.HttpHandler; +import org.slf4j.Logger; import java.util.Map; import java.util.concurrent.CompletableFuture; import java.util.concurrent.atomic.AtomicReference; -import java.util.function.Function; import static com.hotels.styx.api.HttpHeaderNames.CONTENT_TYPE; import static com.hotels.styx.api.HttpHeaderValues.APPLICATION_JSON; @@ -36,16 +36,18 @@ import static java.lang.String.format; import static java.nio.charset.StandardCharsets.UTF_8; import static java.util.concurrent.CompletableFuture.completedFuture; +import static org.slf4j.LoggerFactory.getLogger; /** * A helper class for implementing StyxService interface. - * + *

* AbstractStyxService provides service state management facilities * for implementing a StyxSerive interface. */ public abstract class AbstractStyxService implements StyxService { private final String name; private final AtomicReference status = new AtomicReference<>(CREATED); + private final Logger logger = getLogger(getClass()); public AbstractStyxService(String name) { this.name = name; @@ -68,9 +70,17 @@ public CompletableFuture start() { boolean changed = status.compareAndSet(CREATED, STARTING); if (changed) { + logger.info("Starting serviceName={}...", serviceName()); return startService() - .exceptionally(failWithMessage("Service failed to start.")) - .thenAccept(na -> status.compareAndSet(STARTING, RUNNING)); + .exceptionally(cause -> { + status.set(StyxServiceStatus.FAILED); + logger.error("Failed to start serviceName=" + serviceName(), cause); + throw new ServiceFailureException("Service failed to start.", cause); + }) + .thenAccept(na -> { + logger.info("Started serviceName={}", serviceName()); + status.compareAndSet(STARTING, RUNNING); + }); } else { throw new IllegalStateException(format("Start called in %s state", status.get())); } @@ -82,20 +92,17 @@ public CompletableFuture stop() { if (changed) { return stopService() - .exceptionally(failWithMessage("Service failed to stop.")) + .exceptionally(cause -> { + status.set(StyxServiceStatus.FAILED); + logger.error("Failed to stop serviceName=" + serviceName(), cause); + throw new ServiceFailureException("Service failed to stop.", cause); + }) .thenAccept(na -> status.compareAndSet(STOPPING, STOPPED)); } else { throw new IllegalStateException(format("Stop called in %s state", status.get())); } } - private Function failWithMessage(String message) { - return cause -> { - status.set(StyxServiceStatus.FAILED); - throw new ServiceFailureException(message, cause); - }; - } - @Override public Map adminInterfaceHandlers() { return ImmutableMap.of("status", (request, context) -> Eventual.of( @@ -109,4 +116,9 @@ public Map adminInterfaceHandlers() { public String serviceName() { return name; } + + @Override + public String toString() { + return "service:" + serviceName(); + } } diff --git a/components/api/src/main/java/com/hotels/styx/api/extension/service/spi/StyxService.java b/components/api/src/main/java/com/hotels/styx/api/extension/service/spi/StyxService.java index c7e9cad040..e3084df4a8 100644 --- a/components/api/src/main/java/com/hotels/styx/api/extension/service/spi/StyxService.java +++ b/components/api/src/main/java/com/hotels/styx/api/extension/service/spi/StyxService.java @@ -1,5 +1,5 @@ /* - Copyright (C) 2013-2018 Expedia Inc. + Copyright (C) 2013-2019 Expedia Inc. Licensed under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with the License. @@ -20,6 +20,8 @@ import java.util.Map; import java.util.concurrent.CompletableFuture; +import java.util.function.Consumer; + import static java.util.Collections.emptyMap; /** @@ -30,38 +32,38 @@ public interface StyxService extends StyxLifecycleListener { /** * Invoked when Styx core starts the service. - * + *

* An implementation of start() should: - * + *

* - Asynchronously initialise all resources that are necessary for running - * the service. Especially resources that involve IO, such as opening files - * or establishing network connections, etc. - * - * @return StyxFuture associated to the asynchronous initialisation task. + * the service. Especially resources that involve IO, such as opening files + * or establishing network connections, etc. * + * @return StyxFuture associated to the asynchronous initialisation task. + *

* - The returned StyxFuture must be completed with a *null* value upon - * successful initialisation. - * + * successful initialisation. + *

* - The returned StyxFuture must be completed exceptionally with a failure - * cause when the initialisation fails. + * cause when the initialisation fails. */ CompletableFuture start(); /** * Invoked when Styx core stops the service. - * + *

* An implementation of stop() should: - * + *

* - Create an asynchronous task to initialise the service. The stop() method - * should tear down any resources that are associated with the service. + * should tear down any resources that are associated with the service. * * @return StyxFuture associated to the asynchronous teardown task. - * + *

* - The returned StyxFuture must be completed with a *null* value when - * successfully released all resources. - * + * successfully released all resources. + *

* - The returned StyxFuture must be completed exceptionally with a failure - * cause when the resource release fails. + * cause when the resource release fails. */ CompletableFuture stop(); @@ -73,4 +75,50 @@ public interface StyxService extends StyxLifecycleListener { default Map adminInterfaceHandlers() { return emptyMap(); } + + /** + * Derives a new service interface with added side-effects for errors. + * This could be used for logging, metrics, etc. + * + * @param consumer error consumer + * @return a new service interface + */ + default StyxService doOnError(Consumer consumer) { + StyxService parent = this; + + return new StyxService() { + @Override + public CompletableFuture start() { + return parent.start().exceptionally(throwable -> { + consumer.accept(throwable); + + if (throwable instanceof RuntimeException) { + throw (RuntimeException) throwable; + } + + if (throwable instanceof Error) { + throw (Error) throwable; + } + + throw new RuntimeException(throwable); + }); + } + + @Override + public CompletableFuture stop() { + return parent.stop(); + } + + @Override + public Map adminInterfaceHandlers() { + return parent.adminInterfaceHandlers(); + } + + @Override + public String toString() { + return parent.toString(); + } + + }; + } } diff --git a/components/api/src/test/java/com/hotels/styx/api/extension/service/spi/AbstractStyxServiceTest.java b/components/api/src/test/java/com/hotels/styx/api/extension/service/spi/AbstractStyxServiceTest.java index 5232c0ec0e..f6453fcc28 100644 --- a/components/api/src/test/java/com/hotels/styx/api/extension/service/spi/AbstractStyxServiceTest.java +++ b/components/api/src/test/java/com/hotels/styx/api/extension/service/spi/AbstractStyxServiceTest.java @@ -1,5 +1,5 @@ /* - Copyright (C) 2013-2018 Expedia Inc. + Copyright (C) 2013-2019 Expedia Inc. Licensed under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with the License. @@ -15,15 +15,14 @@ */ package com.hotels.styx.api.extension.service.spi; -import com.hotels.styx.api.Eventual; import com.hotels.styx.api.HttpResponse; import com.hotels.styx.api.LiveHttpRequest; import org.testng.annotations.Test; import reactor.core.publisher.Mono; -import reactor.test.StepVerifier; import java.util.concurrent.CompletableFuture; import java.util.concurrent.ExecutionException; +import java.util.concurrent.atomic.AtomicReference; import static com.hotels.styx.api.extension.service.spi.MockContext.MOCK_CONTEXT; import static com.hotels.styx.api.extension.service.spi.StyxServiceStatus.FAILED; @@ -33,19 +32,21 @@ import static com.hotels.styx.api.extension.service.spi.StyxServiceStatus.STOPPING; import static java.nio.charset.StandardCharsets.UTF_8; import static java.util.concurrent.CompletableFuture.completedFuture; +import static java.util.concurrent.TimeUnit.SECONDS; import static org.hamcrest.MatcherAssert.assertThat; +import static org.hamcrest.Matchers.instanceOf; import static org.hamcrest.Matchers.is; +import static org.testng.Assert.fail; public class AbstractStyxServiceTest { - private final LiveHttpRequest get = LiveHttpRequest.get("/").build(); @Test - public void exposesNameAndStatusViaAdminInterface() throws ExecutionException, InterruptedException { + public void exposesNameAndStatusViaAdminInterface() { DerivedStyxService service = new DerivedStyxService("derived-service", new CompletableFuture<>()); HttpResponse response = Mono.from(service.adminInterfaceHandlers().get("status").handle(get, MOCK_CONTEXT) - .flatMap(r -> r.aggregate(1024))).block(); + .flatMap(r -> r.aggregate(1024))).block(); assertThat(response.bodyAs(UTF_8), is("{ name: \"derived-service\" status: \"CREATED\" }")); } @@ -182,6 +183,36 @@ public void throwsExceptionWhenStopIsCalledInFailedState() { service.stop(); } + @Test(expectedExceptions = ServiceFailureException.class) + public void propagatesExceptions() throws Throwable { + CompletableFuture future = new CompletableFuture<>(); + DerivedStyxService derivedStyxService = new DerivedStyxService("foo", future); + future.completeExceptionally(new RuntimeException("This is just a test")); + + try { + derivedStyxService.start().get(1, SECONDS); + } catch (ExecutionException e) { + throw e.getCause(); + } + } + + @Test + public void canAttachDoOnErrorBehaviour() throws Throwable { + CompletableFuture future = new CompletableFuture<>(); + DerivedStyxService derivedStyxService = new DerivedStyxService("foo", future); + future.completeExceptionally(new RuntimeException("This is just a test")); + + AtomicReference caught = new AtomicReference<>(); + + try { + derivedStyxService + .doOnError(caught::set) + .start().get(1, SECONDS); + fail("Expected exception"); + } catch (ExecutionException e) { + assertThat(caught.get(), is(instanceOf(RuntimeException.class))); + } + } static class DerivedStyxService extends AbstractStyxService { private final CompletableFuture startFuture; diff --git a/components/client/src/main/java/com/hotels/styx/client/healthcheck/monitors/AnomalyExcludingOriginHealthStatusMonitor.java b/components/client/src/main/java/com/hotels/styx/client/healthcheck/monitors/AnomalyExcludingOriginHealthStatusMonitor.java index d6aa32a0b9..84b8f6f11d 100644 --- a/components/client/src/main/java/com/hotels/styx/client/healthcheck/monitors/AnomalyExcludingOriginHealthStatusMonitor.java +++ b/components/client/src/main/java/com/hotels/styx/client/healthcheck/monitors/AnomalyExcludingOriginHealthStatusMonitor.java @@ -1,5 +1,5 @@ /* - Copyright (C) 2013-2018 Expedia Inc. + Copyright (C) 2013-2019 Expedia Inc. Licensed under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with the License. diff --git a/components/client/src/main/java/com/hotels/styx/client/healthcheck/monitors/NoOriginHealthStatusMonitor.java b/components/client/src/main/java/com/hotels/styx/client/healthcheck/monitors/NoOriginHealthStatusMonitor.java index 7e45b82397..894a560e0a 100644 --- a/components/client/src/main/java/com/hotels/styx/client/healthcheck/monitors/NoOriginHealthStatusMonitor.java +++ b/components/client/src/main/java/com/hotels/styx/client/healthcheck/monitors/NoOriginHealthStatusMonitor.java @@ -1,5 +1,5 @@ /* - Copyright (C) 2013-2018 Expedia Inc. + Copyright (C) 2013-2019 Expedia Inc. Licensed under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with the License. diff --git a/components/common/src/main/java/com/hotels/styx/common/FailureHandlingStrategy.java b/components/common/src/main/java/com/hotels/styx/common/FailureHandlingStrategy.java deleted file mode 100644 index 2c24b5cffb..0000000000 --- a/components/common/src/main/java/com/hotels/styx/common/FailureHandlingStrategy.java +++ /dev/null @@ -1,145 +0,0 @@ -/* - Copyright (C) 2013-2019 Expedia Inc. - - 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.hotels.styx.common; - -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import java.util.ArrayList; -import java.util.LinkedHashMap; -import java.util.List; -import java.util.Map; -import java.util.function.BiConsumer; -import java.util.function.Consumer; - -import static java.util.Objects.requireNonNull; - -/** - * Allows various types of failure handling to be applied when applying a function over a list of inputs. - *

- * It can be as strict or lenient as desired - from failing the whole run immediately as soon as a single iteration fails, - * to just logging failures and returning successful results. - *

- * The behaviour is configured by injecting lambdas when an instance of this class is built using the {@link Builder}. - * - * @param input type - * @param output type - */ -public class FailureHandlingStrategy { - private static final Logger LOGGER = LoggerFactory.getLogger(FailureHandlingStrategy.class); - - private final BiConsumer onEachFailure; - private final Consumer> failuresPostProcessing; - - private FailureHandlingStrategy(Builder builder) { - this.onEachFailure = builder.onEachFailure; - this.failuresPostProcessing = builder.failuresPostProcessing; - } - - /** - * Execute a function on each item in a list of inputs, using the configured failure handling. - * - * @param inputs a list of inputs - * @param function a function to execute - * @return a list of outputs - */ - public List process(List inputs, FallibleFunction function) { - List successes = new ArrayList<>(); - Map failures = new LinkedHashMap<>(); - - inputs.forEach(input -> { - try { - successes.add(function.execute(input)); - } catch (Exception t) { - onEachFailure.accept(input, t); - failures.put(input, t); - } - }); - - if (!failures.isEmpty()) { - failuresPostProcessing.accept(failures); - } - - return successes; - } - - /** - * Builds {@link FailureHandlingStrategy}. - * - * @param input type - * @param output type - */ - public static final class Builder { - private BiConsumer onEachFailure = (input, err) -> LOGGER.error("Failed on input " + input, err); - private Consumer> failuresPostProcessing = failures -> { - throw new RuntimeException("Failure during execution: " + failures); - }; - - /** - * An action to take immediately after a failure. For example, this can be used to fail fast by rethrowing the error, - * or to log the failure as soon as it happens without waiting. - * - * @param onEachFailure an action to take - * @return this builder - */ - public Builder doImmediatelyOnEachFailure(BiConsumer onEachFailure) { - this.onEachFailure = requireNonNull(onEachFailure); - return this; - } - - /** - * An action to take after processing all list items, if there were failures. - * This will only execute if the individual failures did not rethrow their exceptions. - *

- * This action may be used to throw an exception, thereby causing an overall failure, or just to perform side-actions - * like logging. - * - * @param failuresPostProcessing an action to take - * @return this builder - */ - public Builder doOnFailuresAfterAllProcessing(Consumer> failuresPostProcessing) { - this.failuresPostProcessing = requireNonNull(failuresPostProcessing); - return this; - } - - /** - * Build an instance of {@link FailureHandlingStrategy} using the configuration provided. - * - * @return a new instance - */ - public FailureHandlingStrategy build() { - return new FailureHandlingStrategy<>(this); - } - } - - /** - * A function to execute. Similar to {@link java.util.function.Function}, except it is allowed to throw any type of - * {@link Exception}. - * - * @param input type - * @param output type - */ - public interface FallibleFunction { - /** - * Execute action. - * - * @param input input - * @return output - * @throws Exception if there is a failure - */ - R execute(T input) throws Exception; - } -} diff --git a/components/common/src/main/java/com/hotels/styx/common/SequenceProcessor.java b/components/common/src/main/java/com/hotels/styx/common/SequenceProcessor.java new file mode 100644 index 0000000000..905d8d9e01 --- /dev/null +++ b/components/common/src/main/java/com/hotels/styx/common/SequenceProcessor.java @@ -0,0 +1,179 @@ +/* + Copyright (C) 2013-2019 Expedia Inc. + + 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.hotels.styx.common; + +import reactor.core.publisher.Flux; + +import java.util.List; +import java.util.Map; +import java.util.function.BiConsumer; +import java.util.function.Consumer; +import java.util.function.Function; + +import static java.util.Objects.requireNonNull; +import static java.util.stream.Collectors.toList; +import static java.util.stream.Collectors.toMap; + +/** + * Provides the ability to easily apply side effects and failure handling to the processing of a sequence of inputs + * without cluttering code with nested loops, try-catch and if-else statements. + * + * @param input type + * @param output type + */ +public class SequenceProcessor { + private final Flux> flux; + private final Consumer> failuresPostProcessing; + + private SequenceProcessor(Flux> flux, Consumer> failuresPostProcessing) { + this.flux = requireNonNull(flux); + this.failuresPostProcessing = requireNonNull(failuresPostProcessing); + } + + /** + * Create a SequenceProcessor, handling a list of inputs. + * If no mapping is added, all inputs will be returned as outputs with no failures. + * + * @param inputs inputs + * @param input type + * @return a new SequenceProcessor + */ + public static SequenceProcessor processSequence(List inputs) { + return new SequenceProcessor<>( + Flux.fromIterable(inputs).map(element -> new Attempt<>(element, element)), + any -> { + // do nothing + }); + } + + /** + * Maps a function onto the SequenceProcessor so that it will be performed on every input that goes through. + * It will run on every input even if the function throws exceptions - these can be handled later with the + * {@link #onEachFailure(BiConsumer)} and {@link #failuresPostProcessing(Consumer)} methods. + * + * @param function function + * @param output type + * @return transformed SequenceProcessor + */ + public SequenceProcessor map(Function function) { + Flux> mapped = this.flux.map(item -> { + if (item.failed()) { + return (Attempt) item; + } + + try { + return new Attempt<>(item.input, function.apply(item.output)); + } catch (Exception e) { + return new Attempt<>(item.input, e); + } + }); + + return new SequenceProcessor<>(mapped, failuresPostProcessing); + } + + /** + * Adds a side-effect to each successful input and its output. + * + * @param consumer side-effect + * @return transformed SequenceProcessor + */ + public SequenceProcessor onEachSuccess(BiConsumer consumer) { + return new SequenceProcessor<>(flux.doOnNext(attempt -> { + if (attempt.success()) { + consumer.accept(attempt.input, attempt.output); + } + }), failuresPostProcessing); + } + + /** + * Adds a side-effect to each failed input and its exception. + * This can be used to log, collect metrics, or fail-fast by rethrowing the exception (or throwing a new exception). + * + * @param consumer side-effect + * @return transformed SequenceProcessor + */ + public SequenceProcessor onEachFailure(BiConsumer consumer) { + return new SequenceProcessor<>(flux.doOnNext(attempt -> { + if (attempt.failed()) { + consumer.accept(attempt.input, attempt.exception); + } + }), failuresPostProcessing); + } + + /** + * Adds behaviour to execute on failed input and its exception, after all processing is finished. + * It is only executed if at least one input resulted in an exception. + *

+ * This is useful for when you choose not to fail-fast on individual inputs. + * By throwing an exception in the provided lambda, you can fail instead of allowing the outputs to be collected. + * + * @param failuresPostProcessing behaviour to execute + * @return transformed SequenceProcessor + */ + public SequenceProcessor failuresPostProcessing(Consumer> failuresPostProcessing) { + return new SequenceProcessor<>(flux, failuresPostProcessing); + } + + /** + * Collect up all outputs from successful inputs. + * + * @return list of outputs + */ + public List collect() { + List> attempts = flux.collectList().block(); + + Map failures = attempts.stream() + .filter(Attempt::failed) + .collect(toMap( + attempt -> attempt.input, + attempt -> attempt.exception + )); + + if (!failures.isEmpty()) { + failuresPostProcessing.accept(failures); + } + + return attempts.stream().filter(Attempt::success) + .map(attempt -> attempt.output) + .collect(toList()); + } + + private static final class Attempt { + private final T input; + private final E output; + private final Exception exception; + + private Attempt(T input, E output) { + this.input = requireNonNull(input); + this.output = output; + this.exception = null; + } + + private Attempt(T input, Exception exception) { + this.input = requireNonNull(input); + this.output = null; + this.exception = requireNonNull(exception); + } + + private boolean failed() { + return exception != null; + } + + private boolean success() { + return !failed(); + } + } +} diff --git a/components/common/src/main/java/com/hotels/styx/common/lambdas/RecordingSupplier.java b/components/common/src/main/java/com/hotels/styx/common/lambdas/RecordingSupplier.java new file mode 100644 index 0000000000..024657f5ec --- /dev/null +++ b/components/common/src/main/java/com/hotels/styx/common/lambdas/RecordingSupplier.java @@ -0,0 +1,78 @@ +/* + Copyright (C) 2013-2019 Expedia Inc. + + 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.hotels.styx.common.lambdas; + +import java.util.function.Supplier; + +import static com.google.common.base.Suppliers.memoize; +import static java.util.Objects.requireNonNull; + +/** + * A supplier that makes its first request to a delegate and then records the result. + */ +class RecordingSupplier implements SupplierWithCheckedException { + private final Supplier> memoizer; + + RecordingSupplier(SupplierWithCheckedException supplier, boolean recordExceptions) { + Supplier> wrapped = () -> { + try { + return new Outcome<>(supplier.get()); + } catch (Exception e) { + if (recordExceptions) { + return new Outcome<>(e); + } + + throw new UncheckedWrapperException(e); + } + }; + + this.memoizer = memoize(wrapped::get)::get; + } + + @Override + public T get() throws Exception { + try { + Outcome outcome = memoizer.get(); + + if (outcome.failed()) { + throw outcome.exception; + } + + return outcome.output; + } catch (UncheckedWrapperException e) { + throw e.wrapped(); + } + } + + private static final class Outcome { + private final E output; + private final Exception exception; + + private Outcome(E output) { + this.output = output; + this.exception = null; + } + + private Outcome(Exception exception) { + this.output = null; + this.exception = requireNonNull(exception); + } + + private boolean failed() { + return exception != null; + } + } +} diff --git a/components/common/src/main/java/com/hotels/styx/common/lambdas/SupplierWithCheckedException.java b/components/common/src/main/java/com/hotels/styx/common/lambdas/SupplierWithCheckedException.java new file mode 100644 index 0000000000..3a0032a934 --- /dev/null +++ b/components/common/src/main/java/com/hotels/styx/common/lambdas/SupplierWithCheckedException.java @@ -0,0 +1,58 @@ +/* + Copyright (C) 2013-2019 Expedia Inc. + + 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.hotels.styx.common.lambdas; + +import java.util.function.Supplier; + +/** + * Like {@link java.util.function.Supplier} but allowed to throw checked exceptions. + * + * @param type of supplied value + */ +public interface SupplierWithCheckedException { + T get() throws Exception; + + /** + * Derives a Java 8+ standard library supplier from this object. + * If this object throws an Exception, it will be wrapped in a {@link UncheckedWrapperException} before being rethrown. + * + * @return a standard supplier + * @throws UncheckedWrapperException an exception for wrapping exceptions + */ + default Supplier toStandardSupplier() throws UncheckedWrapperException { + SupplierWithCheckedException original = this; + + return () -> { + try { + return original.get(); + } catch (Exception e) { + throw new UncheckedWrapperException(e); + } + }; + } + + /** + * Derives a supplier that will delegate to this object once for a value, then return that value every subsequent time it is called. + * A parameter allows the caller to determine if the same applies to exceptions - if the first call throws an Exception, should + * it rethrow that exception every time, or call the delegate again until it gets a value? + * + * @param recordExceptions true if a first-call exception should be rethrown on subsequent calls + * @return a recording supplier + */ + default SupplierWithCheckedException recordFirstOutput(boolean recordExceptions) { + return new RecordingSupplier<>(this, recordExceptions); + } +} diff --git a/components/common/src/main/java/com/hotels/styx/common/lambdas/UncheckedWrapperException.java b/components/common/src/main/java/com/hotels/styx/common/lambdas/UncheckedWrapperException.java new file mode 100644 index 0000000000..dbf47acd0f --- /dev/null +++ b/components/common/src/main/java/com/hotels/styx/common/lambdas/UncheckedWrapperException.java @@ -0,0 +1,35 @@ +/* + Copyright (C) 2013-2019 Expedia Inc. + + 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.hotels.styx.common.lambdas; + +import static java.util.Objects.requireNonNull; + +/** + * A wrapper for exceptions when transforming lambdas with exceptions into java SL lambdas. + * Note that unlike a typical exception, this only wraps Exceptions, not other Throwables. + */ +public class UncheckedWrapperException extends RuntimeException { + private final Exception wrapped; + + UncheckedWrapperException(Exception wrap) { + super(wrap); + this.wrapped = requireNonNull(wrap); + } + + public Exception wrapped() { + return wrapped; + } +} diff --git a/components/common/src/test/java/com/hotels/styx/common/SequenceProcessorTest.java b/components/common/src/test/java/com/hotels/styx/common/SequenceProcessorTest.java new file mode 100644 index 0000000000..1f392e805c --- /dev/null +++ b/components/common/src/test/java/com/hotels/styx/common/SequenceProcessorTest.java @@ -0,0 +1,76 @@ +/* + Copyright (C) 2013-2019 Expedia Inc. + + 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.hotels.styx.common; + +import org.testng.annotations.Test; + +import java.util.List; +import java.util.Map; +import java.util.concurrent.atomic.AtomicReference; +import java.util.function.BiConsumer; + +import static java.util.Arrays.asList; +import static org.hamcrest.MatcherAssert.assertThat; +import static org.hamcrest.Matchers.contains; +import static org.hamcrest.Matchers.containsInAnyOrder; +import static org.hamcrest.Matchers.is; +import static org.mockito.Matchers.any; +import static org.mockito.Matchers.eq; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.verifyNoMoreInteractions; + +public class SequenceProcessorTest { + @Test + public void countsSuccessesAndFailures() { + BiConsumer onSuccess = mock(BiConsumer.class); + BiConsumer onFailure = mock(BiConsumer.class); + + AtomicReference> collectedFailures = new AtomicReference<>(); + + List inputs = asList(1, 2, 3, 4, 5, 6, 7); + + List outputs = SequenceProcessor.processSequence(inputs).map(number -> { + // throw exception if even + + if (number % 2 == 0) { + throw new IllegalStateException(); + } + + return String.valueOf(number); + }) + .onEachSuccess(onSuccess) + .onEachFailure(onFailure) + .failuresPostProcessing(collectedFailures::set) + .collect(); + + assertThat(outputs, contains("1", "3", "5", "7")); + + verify(onSuccess).accept(1, "1"); + verify(onSuccess).accept(3, "3"); + verify(onSuccess).accept(5, "5"); + verify(onSuccess).accept(7, "7"); + verifyNoMoreInteractions(onSuccess); + + verify(onFailure).accept(eq(2), any(IllegalStateException.class)); + verify(onFailure).accept(eq(4), any(IllegalStateException.class)); + verify(onFailure).accept(eq(6), any(IllegalStateException.class)); + verifyNoMoreInteractions(onFailure); + + assertThat(collectedFailures.get().keySet(), containsInAnyOrder(2, 4, 6)); + assertThat(collectedFailures.get().values().stream().allMatch(err -> err instanceof IllegalStateException), is(true)); + } +} \ No newline at end of file diff --git a/components/common/src/test/java/com/hotels/styx/common/lambdas/RecordingSupplierTest.java b/components/common/src/test/java/com/hotels/styx/common/lambdas/RecordingSupplierTest.java new file mode 100644 index 0000000000..639218eeaa --- /dev/null +++ b/components/common/src/test/java/com/hotels/styx/common/lambdas/RecordingSupplierTest.java @@ -0,0 +1,79 @@ +/* + Copyright (C) 2013-2019 Expedia Inc. + + 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.hotels.styx.common.lambdas; + +import org.testng.annotations.Test; + +import java.util.concurrent.atomic.AtomicInteger; + +import static org.hamcrest.MatcherAssert.assertThat; +import static org.hamcrest.Matchers.is; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; +import static org.testng.Assert.fail; + +public class RecordingSupplierTest { + @Test + public void alwaysOutputsFirstResult() throws Exception { + AtomicInteger increment = new AtomicInteger(1); + SupplierWithCheckedException supplier = increment::getAndIncrement; + + RecordingSupplier recordingSupplier = new RecordingSupplier<>(supplier, false); + + assertThat(recordingSupplier.get(), is(1)); + assertThat(recordingSupplier.get(), is(1)); + assertThat(recordingSupplier.get(), is(1)); + } + + @Test + public void willRecordExceptionsIfConfigured() throws Exception { + SupplierWithCheckedException supplier = mock(SupplierWithCheckedException.class); + + when(supplier.get()).thenThrow(new IllegalStateException("Test")) + .thenReturn(1); + + RecordingSupplier recordingSupplier = new RecordingSupplier<>(supplier, true); + + try { + recordingSupplier.get(); + fail("Did not throw initial exception"); + } catch(IllegalStateException e) { + try { + recordingSupplier.get(); + fail("Did not record exception"); + } catch(IllegalStateException e2) { + // Expected outcome + } + } + } + + @Test + public void willNotRecordExceptionsIfNotConfigured() throws Exception { + SupplierWithCheckedException supplier = mock(SupplierWithCheckedException.class); + + when(supplier.get()).thenThrow(new IllegalStateException("Test")) + .thenReturn(1); + + RecordingSupplier recordingSupplier = new RecordingSupplier<>(supplier, false); + + try { + recordingSupplier.get(); + fail("Did not throw initial exception"); + } catch(IllegalStateException e) { + assertThat(recordingSupplier.get(), is(1)); + } + } +} \ No newline at end of file diff --git a/components/proxy/src/main/java/com/hotels/styx/StyxServer.java b/components/proxy/src/main/java/com/hotels/styx/StyxServer.java index 54ae0c4c7a..eb002aa8dc 100644 --- a/components/proxy/src/main/java/com/hotels/styx/StyxServer.java +++ b/components/proxy/src/main/java/com/hotels/styx/StyxServer.java @@ -25,10 +25,13 @@ import com.hotels.styx.api.extension.service.spi.Registry; import com.hotels.styx.api.extension.service.spi.StyxService; import com.hotels.styx.config.schema.SchemaValidationException; +import com.hotels.styx.configstore.ConfigStore; import com.hotels.styx.infrastructure.configuration.ConfigurationParser; import com.hotels.styx.infrastructure.configuration.yaml.YamlConfiguration; import com.hotels.styx.server.HttpServer; +import com.hotels.styx.startup.PluginStartupService; import com.hotels.styx.startup.ProxyServerSetUp; +import com.hotels.styx.startup.ServerService; import com.hotels.styx.startup.StyxPipelineFactory; import com.hotels.styx.startup.StyxServerComponents; import io.netty.util.ResourceLeakDetector; @@ -40,18 +43,26 @@ import java.net.InetSocketAddress; import java.util.ArrayList; import java.util.Map; +import java.util.concurrent.atomic.AtomicReference; import static com.hotels.styx.ServerConfigSchema.validateServerConfiguration; import static com.hotels.styx.infrastructure.configuration.ConfigurationSource.configSource; import static com.hotels.styx.infrastructure.configuration.yaml.YamlConfigurationFormat.YAML; import static com.hotels.styx.infrastructure.logging.LOGBackConfigurer.shutdownLogging; import static com.hotels.styx.startup.CoreMetrics.registerCoreMetrics; +import static com.hotels.styx.startup.ProxyStatusNotifications.notifyProxyFailed; +import static com.hotels.styx.startup.ProxyStatusNotifications.notifyProxyStarted; import static com.hotels.styx.startup.StyxServerComponents.LoggingSetUp.FROM_CONFIG; import static io.netty.util.ResourceLeakDetector.Level.DISABLED; +import static java.lang.Long.MAX_VALUE; import static java.lang.Runtime.getRuntime; import static java.lang.String.format; import static java.lang.System.getProperty; +import static java.lang.Thread.currentThread; +import static java.util.Objects.requireNonNull; +import static java.util.concurrent.Executors.newSingleThreadExecutor; import static java.util.concurrent.TimeUnit.MILLISECONDS; +import static java.util.concurrent.TimeUnit.SECONDS; import static org.slf4j.LoggerFactory.getLogger; /** @@ -77,7 +88,7 @@ public static void main(String[] args) { StyxServer styxServer = createStyxServer(args); getRuntime().addShutdownHook(new Thread(() -> styxServer.stopAsync().awaitTerminated())); - styxServer.startAsync().awaitRunning(); + styxServer.startAsync().awaitRunning(styxServer.startupTimeoutSeconds, SECONDS); } catch (SchemaValidationException cause) { System.exit(2); } catch (Throwable cause) { @@ -131,34 +142,47 @@ private static boolean skipServerConfigValidation() { return "n".equals(validate) || "no".equals(validate); } - private final HttpServer proxyServer; - private final HttpServer adminServer; + private final ConfigStore configStore; + private final long startupTimeoutSeconds; private final ServiceManager serviceManager; private final Stopwatch stopwatch; + private final ServerReference proxyServer = new ServerReference("Proxy server"); + private final ServerReference adminServer = new ServerReference("Admin server"); + public StyxServer(StyxServerComponents config) { this(config, null); } public StyxServer(StyxServerComponents components, Stopwatch stopwatch) { + this.startupTimeoutSeconds = components.environment().configuration() + .get("startup.timeoutSeconds", Integer.class) + .map(timeout -> (long) timeout) + .orElse(MAX_VALUE); + this.stopwatch = stopwatch; registerCoreMetrics(components.environment().buildInfo(), components.environment().metricRegistry()); Map servicesFromConfig = components.services(); - ProxyServerSetUp proxyServerSetUp = new ProxyServerSetUp(new StyxPipelineFactory()); + PluginStartupService pluginStartupService = new PluginStartupService(components); + + this.configStore = components.environment().configStore(); - components.plugins().forEach(plugin -> components.environment().configStore().set("plugins." + plugin.name(), plugin)); + StyxService adminServerService = new ServerService("adminServer", () -> + this.adminServer.store(createAdminServer(components))); - this.proxyServer = proxyServerSetUp.createProxyServer(components); - this.adminServer = createAdminServer(components); + StyxService proxyServerService = new ServerService("proxyServer", () -> + this.proxyServer.store(createProxyServer(components))) + .doOnError(err -> notifyProxyFailed(configStore)); this.serviceManager = new ServiceManager(new ArrayList() { { - add(proxyServer); - add(adminServer); + add(toGuavaService(pluginStartupService)); + add(toGuavaService(proxyServerService)); + add(toGuavaService(adminServerService)); servicesFromConfig.values().stream() .map(StyxServer::toGuavaService) .forEach(this::add); @@ -167,19 +191,19 @@ public StyxServer(StyxServerComponents components, Stopwatch stopwatch) { } public InetSocketAddress proxyHttpAddress() { - return proxyServer.httpAddress(); + return proxyServer.get().httpAddress(); } public InetSocketAddress proxyHttpsAddress() { - return proxyServer.httpsAddress(); + return proxyServer.get().httpsAddress(); } public InetSocketAddress adminHttpAddress() { - return adminServer.httpAddress(); + return adminServer.get().httpAddress(); } public InetSocketAddress adminHttpsAddress() { - return adminServer.httpsAddress(); + return adminServer.get().httpsAddress(); } private static StartupConfig parseStartupConfig(String[] args) { @@ -197,15 +221,19 @@ private static StartupConfig parseStartupConfig(String[] args) { @Override protected void doStart() { - printBanner(); - this.serviceManager.addListener(new ServerStartListener(this)); - this.serviceManager.startAsync().awaitHealthy(); + newSingleThreadExecutor().submit(() -> { + printBanner(); + this.serviceManager.addListener(new ServerStartListener(this)); + this.serviceManager.startAsync().awaitHealthy(); - if (stopwatch == null) { - LOG.info("Started Styx server"); - } else { - LOG.info("Started Styx server in {} ms", stopwatch.elapsed(MILLISECONDS)); - } + notifyProxyStarted(this.configStore, proxyServer.get()); + + if (stopwatch == null) { + LOG.info("Started Styx server"); + } else { + LOG.info("Started Styx server in {} ms", stopwatch.elapsed(MILLISECONDS)); + } + }); } private void printBanner() { @@ -232,6 +260,10 @@ protected void doStart() { styxService.start() .thenAccept(x -> notifyStarted()) .exceptionally(e -> { + if (e instanceof InterruptedException) { + currentThread().interrupt(); + } + notifyFailed(e); return null; }); @@ -242,6 +274,10 @@ protected void doStop() { styxService.stop() .thenAccept(x -> notifyStopped()) .exceptionally(e -> { + if (e instanceof InterruptedException) { + currentThread().interrupt(); + } + notifyFailed(e); return null; }); @@ -249,12 +285,44 @@ protected void doStop() { }; } + private static HttpServer createProxyServer(StyxServerComponents components) throws InterruptedException { + return new ProxyServerSetUp(new StyxPipelineFactory()) + .createProxyServer(components); + } + private static HttpServer createAdminServer(StyxServerComponents config) { return new AdminServerBuilder(config.environment()) .backendServicesRegistry((Registry) config.services().get("backendServiceRegistry")) .build(); } + // Add during implementation of liveness/readiness checks. This saves us from having to re-architect all the tests that rely on + // synchronous methods like proxyHttpAddress. + private static class ServerReference { + private final AtomicReference reference; + private final String name; + + ServerReference(String name) { + this.reference = new AtomicReference<>(); + this.name = requireNonNull(name); + } + + HttpServer store(HttpServer server) { + this.reference.set(server); + return server; + } + + HttpServer get() { + HttpServer server = reference.get(); + + if (server == null) { + throw new IllegalStateException(name + " has not finished starting up"); + } + + return server; + } + } + private static class ServerStartListener extends ServiceManager.Listener { private final StyxServer styxServer; diff --git a/components/proxy/src/main/java/com/hotels/styx/admin/AdminServerBuilder.java b/components/proxy/src/main/java/com/hotels/styx/admin/AdminServerBuilder.java index e244427b13..8cbef8ce91 100644 --- a/components/proxy/src/main/java/com/hotels/styx/admin/AdminServerBuilder.java +++ b/components/proxy/src/main/java/com/hotels/styx/admin/AdminServerBuilder.java @@ -33,6 +33,7 @@ import com.hotels.styx.admin.handlers.PluginListHandler; import com.hotels.styx.admin.handlers.PluginToggleHandler; import com.hotels.styx.admin.handlers.StartupConfigHandler; +import com.hotels.styx.admin.handlers.StartupStatusHandler; import com.hotels.styx.admin.handlers.StyxConfigurationHandler; import com.hotels.styx.admin.handlers.ThreadsHandler; import com.hotels.styx.admin.handlers.VersionTextHandler; @@ -115,6 +116,8 @@ public HttpServer build() { httpRouter.add("/admin/configuration/logging", new LoggingConfigurationHandler(styxConfig.startupConfig().logConfigLocation())); httpRouter.add("/admin/configuration/startup", new StartupConfigHandler(styxConfig.startupConfig())); + httpRouter.add("/admin/startup", new StartupStatusHandler(environment.configStore())); + // Dashboard httpRouter.add("/admin/dashboard/data.json", dashboardDataHandler(styxConfig)); httpRouter.add("/admin/dashboard/", new ClassPathResourceHandler("/admin/dashboard/")); diff --git a/components/proxy/src/main/java/com/hotels/styx/admin/handlers/StartupStatusHandler.java b/components/proxy/src/main/java/com/hotels/styx/admin/handlers/StartupStatusHandler.java new file mode 100644 index 0000000000..171d225a43 --- /dev/null +++ b/components/proxy/src/main/java/com/hotels/styx/admin/handlers/StartupStatusHandler.java @@ -0,0 +1,93 @@ +/* + Copyright (C) 2013-2019 Expedia Inc. + + 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.hotels.styx.admin.handlers; + +import com.fasterxml.jackson.core.JsonProcessingException; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.collect.ImmutableMap; +import com.hotels.styx.api.LiveHttpRequest; +import com.hotels.styx.api.LiveHttpResponse; +import com.hotels.styx.common.http.handler.BaseHttpHandler; +import com.hotels.styx.configstore.ConfigStore; +import com.hotels.styx.startup.ProxyStatusNotifications.ProxyStatus; +import com.hotels.styx.startup.extensions.PluginStatusNotifications.PluginStatus; +import org.slf4j.Logger; + +import java.util.HashMap; +import java.util.LinkedHashMap; +import java.util.Map; + +import static com.hotels.styx.api.HttpResponse.response; +import static com.hotels.styx.api.HttpResponseStatus.INTERNAL_SERVER_ERROR; +import static com.hotels.styx.startup.ProxyStatusNotifications.PROXY_HTTPS_STATUS_KEY; +import static com.hotels.styx.startup.ProxyStatusNotifications.PROXY_HTTP_STATUS_KEY; +import static com.hotels.styx.startup.extensions.PluginStatusNotifications.PLUGIN_STATUS_KEY_PREFIX; +import static java.nio.charset.StandardCharsets.UTF_8; +import static java.util.Objects.requireNonNull; +import static org.slf4j.LoggerFactory.getLogger; + +/** + * A handler that provides information on the status of the proxy server and plugins start-up. + */ +public class StartupStatusHandler extends BaseHttpHandler { + private static final Logger LOGGER = getLogger(StartupStatusHandler.class); + private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper(); + + private final ConfigStore configStore; + + public StartupStatusHandler(ConfigStore configStore) { + this.configStore = requireNonNull(configStore); + } + + @Override + protected LiveHttpResponse doHandle(LiveHttpRequest request) { + Map connectors = new LinkedHashMap<>(); + Map plugins = new HashMap<>(); + + connectors.put("http", configStore.get(PROXY_HTTP_STATUS_KEY, ProxyStatus.class) + .map(Object::toString) + .orElse("unknown")); + + connectors.put("https", configStore.get(PROXY_HTTPS_STATUS_KEY, ProxyStatus.class) + .map(Object::toString) + .orElse("unknown")); + + configStore.startingWith(PLUGIN_STATUS_KEY_PREFIX, PluginStatus.class).forEach(entry -> { + String pluginName = entry.key().substring(entry.key().lastIndexOf('.') + 1); + plugins.put(pluginName, entry.value().toString()); + }); + + try { + String body = OBJECT_MAPPER.writerWithDefaultPrettyPrinter() + .writeValueAsString(ImmutableMap.of( + "connectors", connectors, + "plugins", plugins)); + + return response() + .body(body, UTF_8) + .disableCaching() + .build() + .stream(); + } catch (JsonProcessingException e) { + LOGGER.error("Could not generate response", e); + return response(INTERNAL_SERVER_ERROR) + .body(e.getMessage(), UTF_8) + .build() + .stream(); + } + } + +} diff --git a/components/proxy/src/main/java/com/hotels/styx/metrics/reporting/graphite/GraphiteReporterService.java b/components/proxy/src/main/java/com/hotels/styx/metrics/reporting/graphite/GraphiteReporterService.java index ccb0391636..fde397521d 100644 --- a/components/proxy/src/main/java/com/hotels/styx/metrics/reporting/graphite/GraphiteReporterService.java +++ b/components/proxy/src/main/java/com/hotels/styx/metrics/reporting/graphite/GraphiteReporterService.java @@ -1,5 +1,5 @@ /* - Copyright (C) 2013-2018 Expedia Inc. + Copyright (C) 2013-2019 Expedia Inc. Licensed under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with the License. @@ -57,18 +57,12 @@ private GraphiteReporterService(Builder builder) { @Override protected CompletableFuture startService() { - return CompletableFuture.runAsync(() -> { - this.reporter.start(reportingIntervalMillis, MILLISECONDS); - LOGGER.info("Graphite service started, service name=\"{}\"", serviceName()); - }); + return CompletableFuture.runAsync(() -> this.reporter.start(reportingIntervalMillis, MILLISECONDS)); } @Override protected CompletableFuture stopService() { - return CompletableFuture.runAsync(() -> { - this.reporter.stop(); - LOGGER.info("Graphite service stopped, service name=\"{}\"", serviceName()); - }); + return CompletableFuture.runAsync(this.reporter::stop); } @VisibleForTesting diff --git a/components/proxy/src/main/java/com/hotels/styx/proxy/ProxyServerBuilder.java b/components/proxy/src/main/java/com/hotels/styx/proxy/ProxyServerBuilder.java index de14822041..e6faf941d5 100644 --- a/components/proxy/src/main/java/com/hotels/styx/proxy/ProxyServerBuilder.java +++ b/components/proxy/src/main/java/com/hotels/styx/proxy/ProxyServerBuilder.java @@ -1,5 +1,5 @@ /* - Copyright (C) 2013-2018 Expedia Inc. + Copyright (C) 2013-2019 Expedia Inc. Licensed under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with the License. @@ -34,8 +34,6 @@ public final class ProxyServerBuilder { private final CharSequence styxInfoHeaderName; private HttpHandler httpHandler; - private Runnable onStartupAction = () -> { - }; public ProxyServerBuilder(Environment environment) { this.environment = requireNonNull(environment); @@ -53,7 +51,6 @@ public HttpServer build() { .toNettyServerBuilder(proxyConfig) .httpHandler(httpHandler) // register health check - .doOnStartUp(onStartupAction) .build(); } @@ -65,9 +62,4 @@ public ProxyServerBuilder httpHandler(HttpHandler httpHandler) { this.httpHandler = httpHandler; return this; } - - public ProxyServerBuilder onStartup(Runnable startupAction) { - this.onStartupAction = startupAction; - return this; - } } diff --git a/components/proxy/src/main/java/com/hotels/styx/startup/PipelineFactory.java b/components/proxy/src/main/java/com/hotels/styx/startup/PipelineFactory.java index 8b8dfed764..4c38808613 100644 --- a/components/proxy/src/main/java/com/hotels/styx/startup/PipelineFactory.java +++ b/components/proxy/src/main/java/com/hotels/styx/startup/PipelineFactory.java @@ -1,5 +1,5 @@ /* - Copyright (C) 2013-2018 Expedia Inc. + Copyright (C) 2013-2019 Expedia Inc. Licensed under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with the License. @@ -16,10 +16,13 @@ package com.hotels.styx.startup; import com.hotels.styx.api.HttpHandler; +import com.hotels.styx.proxy.plugin.NamedPlugin; + +import java.util.List; /** * Factory that makes an HTTP pipeline. */ public interface PipelineFactory { - HttpHandler create(StyxServerComponents config); + HttpHandler create(StyxServerComponents config, List plugins); } diff --git a/components/proxy/src/main/java/com/hotels/styx/startup/PluginStartupService.java b/components/proxy/src/main/java/com/hotels/styx/startup/PluginStartupService.java new file mode 100644 index 0000000000..a45a847bd8 --- /dev/null +++ b/components/proxy/src/main/java/com/hotels/styx/startup/PluginStartupService.java @@ -0,0 +1,102 @@ +/* + Copyright (C) 2013-2019 Expedia Inc. + + 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.hotels.styx.startup; + +import com.hotels.styx.Environment; +import com.hotels.styx.api.extension.service.spi.AbstractStyxService; +import com.hotels.styx.proxy.plugin.NamedPlugin; +import com.hotels.styx.startup.extensions.ConfiguredPluginFactory; +import com.hotels.styx.startup.extensions.PluginStatusNotifications; +import org.slf4j.Logger; + +import java.util.List; +import java.util.concurrent.CompletableFuture; + +import static com.hotels.styx.common.SequenceProcessor.processSequence; +import static com.hotels.styx.startup.extensions.PluginLoadingForStartup.loadPlugins; +import static com.hotels.styx.startup.extensions.PluginStatusNotifications.PluginPipelineStatus.ALL_PLUGINS_COMPLETE; +import static com.hotels.styx.startup.extensions.PluginStatusNotifications.PluginPipelineStatus.AT_LEAST_ONE_PLUGIN_FAILED; +import static com.hotels.styx.startup.extensions.PluginStatusNotifications.PluginPipelineStatus.INCOMPLETE; +import static com.hotels.styx.startup.extensions.PluginStatusNotifications.PluginStatus.COMPLETE; +import static com.hotels.styx.startup.extensions.PluginStatusNotifications.PluginStatus.FAILED_WHILE_LIFECYCLE_STARTING; +import static com.hotels.styx.startup.extensions.PluginStatusNotifications.PluginStatus.LIFECYCLE_STARTING; +import static java.lang.String.format; +import static org.slf4j.LoggerFactory.getLogger; + +/** + * Service to start up plugins and add them to the config store. + */ +public class PluginStartupService extends AbstractStyxService { + public static final String PLUGIN_STATUS_KEY_FORMAT = "startup.plugins.%s"; + private static final String PLUGIN_KEY_FORMAT = "plugins.%s"; + + private static final Logger LOGGER = getLogger(PluginStartupService.class); + + private final List pluginFactories; + private final Environment environment; + + /** + * Constructs an service instance. + * If the components object has plugin factories explicitly set, those factories will be used, + * otherwise the service will get them from the configuration. + * + * @param components server components + */ + public PluginStartupService(StyxServerComponents components) { + super("PluginStartupService"); + this.pluginFactories = components.pluginFactories().orElse(null); + this.environment = components.environment(); + } + + protected CompletableFuture startService() { + return CompletableFuture.runAsync(() -> { + List plugins = pluginFactories == null + ? loadPlugins(environment) + : loadPlugins(environment, pluginFactories); + + PluginStatusNotifications notifications = new PluginStatusNotifications(environment.configStore()); + notifications.notifyPluginPipelineStatus(INCOMPLETE); + + processSequence(plugins) + .map(plugin -> { + notifications.notifyPluginStatus(plugin.name(), LIFECYCLE_STARTING); + plugin.styxStarting(); + return null; + }) + + .onEachSuccess((plugin, ignore) -> { + registerPlugin(plugin); + notifications.notifyPluginStatus(plugin.name(), COMPLETE); + }) + + .onEachFailure((plugin, err) -> { + notifications.notifyPluginStatus(plugin.name(), FAILED_WHILE_LIFECYCLE_STARTING); + LOGGER.error("Error starting plugin '{}'", plugin.name(), err); + }) + + .failuresPostProcessing(failures -> { + notifications.notifyPluginPipelineStatus(AT_LEAST_ONE_PLUGIN_FAILED); + throw new StartupException(format("%s plugins failed to start", failures.size())); + }).collect(); + + notifications.notifyPluginPipelineStatus(ALL_PLUGINS_COMPLETE); + }); + } + + private void registerPlugin(NamedPlugin plugin) { + environment.configStore().set(format(PLUGIN_KEY_FORMAT, plugin.name()), plugin); + } +} diff --git a/components/proxy/src/main/java/com/hotels/styx/startup/ProxyServerSetUp.java b/components/proxy/src/main/java/com/hotels/styx/startup/ProxyServerSetUp.java index d60097ba80..d3816e3e1c 100644 --- a/components/proxy/src/main/java/com/hotels/styx/startup/ProxyServerSetUp.java +++ b/components/proxy/src/main/java/com/hotels/styx/startup/ProxyServerSetUp.java @@ -20,10 +20,18 @@ import com.hotels.styx.proxy.ProxyServerBuilder; import com.hotels.styx.proxy.plugin.NamedPlugin; import com.hotels.styx.server.HttpServer; +import com.hotels.styx.startup.extensions.PluginStatusNotifications.PluginPipelineStatus; import org.slf4j.Logger; +import java.util.List; + import static com.google.common.util.concurrent.MoreExecutors.sameThreadExecutor; -import static java.lang.String.format; +import static com.hotels.styx.startup.ProxyStatusNotifications.notifyProxyStatus; +import static com.hotels.styx.startup.extensions.PluginStatusNotifications.PLUGIN_PIPELINE_STATUS_KEY; +import static com.hotels.styx.startup.extensions.PluginStatusNotifications.PluginPipelineStatus.ALL_PLUGINS_COMPLETE; +import static com.hotels.styx.startup.extensions.PluginStatusNotifications.PluginPipelineStatus.AT_LEAST_ONE_PLUGIN_FAILED; +import static com.hotels.styx.startup.extensions.PluginStatusNotifications.PluginPipelineStatus.INCOMPLETE; +import static java.lang.Thread.sleep; import static java.util.Objects.requireNonNull; import static org.slf4j.LoggerFactory.getLogger; @@ -39,33 +47,44 @@ public ProxyServerSetUp(PipelineFactory pipelineFactory) { this.pipelineFactory = requireNonNull(pipelineFactory); } - public HttpServer createProxyServer(StyxServerComponents config) { - HttpHandler pipeline = pipelineFactory.create(config); + public HttpServer createProxyServer(StyxServerComponents components) throws InterruptedException { + notifyProxyStatus( + components.environment().configStore(), + components.environment().configuration().proxyServerConfig()); - HttpServer proxyServer = new ProxyServerBuilder(config.environment()) - .httpHandler(pipeline) - .onStartup(() -> initialisePlugins(config.plugins())) - .build(); + // TODO see https://github.com/HotelsDotCom/styx/issues/382 - proxyServer.addListener(new PluginsNotifierOfProxyState(config.plugins()), sameThreadExecutor()); - return proxyServer; + while (!arePluginsLoaded(components)) { + sleep(100L); + } + + List plugins = components.environment().configStore() + .valuesStartingWith("plugins", NamedPlugin.class); + + return createProxyServer0(components, plugins); } - private static void initialisePlugins(Iterable plugins) { - int exceptions = 0; + private static boolean arePluginsLoaded(StyxServerComponents components) { + PluginPipelineStatus status = components.environment().configStore() + .get(PLUGIN_PIPELINE_STATUS_KEY, PluginPipelineStatus.class) + .orElse(INCOMPLETE); - for (NamedPlugin plugin : plugins) { - try { - plugin.styxStarting(); - } catch (Exception e) { - exceptions++; - LOG.error("Error starting plugin '{}'", plugin.name(), e); - } + if (status == AT_LEAST_ONE_PLUGIN_FAILED) { + throw new IllegalStateException("One or more plugins failed to start"); } - if (exceptions > 0) { - throw new RuntimeException(format("%s plugins failed to start", exceptions)); - } + return status == ALL_PLUGINS_COMPLETE; + } + + private HttpServer createProxyServer0(StyxServerComponents components, List plugins) { + HttpHandler pipeline = pipelineFactory.create(components, plugins); + + HttpServer proxyServer = new ProxyServerBuilder(components.environment()) + .httpHandler(pipeline) + .build(); + + proxyServer.addListener(new PluginsNotifierOfProxyState(plugins), sameThreadExecutor()); + return proxyServer; } private static class PluginsNotifierOfProxyState extends Service.Listener { diff --git a/components/proxy/src/main/java/com/hotels/styx/startup/ProxyStatusNotifications.java b/components/proxy/src/main/java/com/hotels/styx/startup/ProxyStatusNotifications.java new file mode 100644 index 0000000000..015cc93160 --- /dev/null +++ b/components/proxy/src/main/java/com/hotels/styx/startup/ProxyStatusNotifications.java @@ -0,0 +1,95 @@ +/* + Copyright (C) 2013-2019 Expedia Inc. + + 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.hotels.styx.startup; + +import com.hotels.styx.configstore.ConfigStore; +import com.hotels.styx.proxy.ProxyServerConfig; +import com.hotels.styx.server.HttpServer; + +import static com.hotels.styx.startup.ProxyStatusNotifications.ProxyStatus.COMPLETE; +import static com.hotels.styx.startup.ProxyStatusNotifications.ProxyStatus.DISABLED; +import static com.hotels.styx.startup.ProxyStatusNotifications.ProxyStatus.FAILED; +import static com.hotels.styx.startup.ProxyStatusNotifications.ProxyStatus.INCOMPLETE; +import static java.util.Objects.requireNonNull; + +/** + * For setting the proxy startup status in the config store. + */ +public final class ProxyStatusNotifications { + public static final String PROXY_HTTP_STATUS_KEY = "startup.proxy.http"; + public static final String PROXY_HTTPS_STATUS_KEY = "startup.proxy.https"; + + private ProxyStatusNotifications() { + } + + /** + * Notify config store of the initial state of both proxy connectors - are they in progress (incomplete) or disabled. + * + * @param configStore config store + * @param config proxy server config + */ + public static void notifyProxyStatus(ConfigStore configStore, ProxyServerConfig config) { + configStore.set(PROXY_HTTP_STATUS_KEY, config.httpConnectorConfig().map(any -> INCOMPLETE).orElse(DISABLED)); + configStore.set(PROXY_HTTPS_STATUS_KEY, config.httpsConnectorConfig().map(any -> INCOMPLETE).orElse(DISABLED)); + } + + /** + * Notify the config store when the proxy server has finished starting up. + * + * @param configStore config store + * @param server proxy server + */ + public static void notifyProxyStarted(ConfigStore configStore, HttpServer server) { + if (server.httpAddress() != null) { + configStore.set(PROXY_HTTP_STATUS_KEY, COMPLETE); + } + + if (server.httpsAddress() != null) { + configStore.set(PROXY_HTTPS_STATUS_KEY, COMPLETE); + } + } + + /** + * Notify the config store when the proxy server has failed to start up. + * + * @param configStore config store + */ + public static void notifyProxyFailed(ConfigStore configStore) { + configStore.set(PROXY_HTTP_STATUS_KEY, FAILED); + configStore.set(PROXY_HTTPS_STATUS_KEY, FAILED); + } + + /** + * Status for the proxy to be in during start-up. + */ + public enum ProxyStatus { + INCOMPLETE("incomplete"), + COMPLETE("complete"), + FAILED("failed"), + DISABLED("disabled"); + + private final String description; + + ProxyStatus(String description) { + this.description = requireNonNull(description); + } + + @Override + public String toString() { + return description; + } + } +} diff --git a/components/proxy/src/main/java/com/hotels/styx/startup/ServerService.java b/components/proxy/src/main/java/com/hotels/styx/startup/ServerService.java new file mode 100644 index 0000000000..0e2604900a --- /dev/null +++ b/components/proxy/src/main/java/com/hotels/styx/startup/ServerService.java @@ -0,0 +1,105 @@ +/* + Copyright (C) 2013-2019 Expedia Inc. + + 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.hotels.styx.startup; + +import com.google.common.util.concurrent.Service; +import com.hotels.styx.api.extension.service.spi.AbstractStyxService; +import com.hotels.styx.common.lambdas.SupplierWithCheckedException; +import com.hotels.styx.server.HttpServer; + +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; + +import static java.util.concurrent.CompletableFuture.completedFuture; + +/** + * A service that creates AND starts an HTTP server when started. + */ +public class ServerService extends AbstractStyxService { + private final SupplierWithCheckedException serverSupplier; + private final ExecutorService executor = Executors.newCachedThreadPool(); + + /** + * Construct a service. The supplier will be called once, when the service starts up. + * The server instance returned by that call will continue to be used thereafter. + * + * @param name service name + * @param serverSupplier server supplier + */ + public ServerService(String name, SupplierWithCheckedException serverSupplier) { + super(name); + this.serverSupplier = serverSupplier.recordFirstOutput(true); + } + + @Override + protected CompletableFuture startService() { + return server().thenCompose(server -> { + CompletableFuture future = new CompletableFuture<>(); + + server.addListener(new Service.Listener() { + @Override + public void running() { + future.complete(null); + } + + @Override + public void failed(Service.State from, Throwable failure) { + failure.printStackTrace(); + future.completeExceptionally(failure); + } + }, executor); + + server.startAsync(); + + return future; + }); + } + + @Override + protected CompletableFuture stopService() { + return server().thenCompose(server -> { + CompletableFuture future = new CompletableFuture<>(); + + server.addListener(new Service.Listener() { + @Override + public void terminated(Service.State from) { + future.complete(null); + } + + @Override + public void failed(Service.State from, Throwable failure) { + future.completeExceptionally(failure); + } + }, executor); + + server.stopAsync(); + + return future; + }); + } + + private CompletableFuture server() { + try { + HttpServer server = serverSupplier.get(); + return completedFuture(server); + } catch (Exception e) { + CompletableFuture future = new CompletableFuture<>(); + future.completeExceptionally(e); + return future; + } + } +} diff --git a/components/proxy/src/main/java/com/hotels/styx/startup/StartupException.java b/components/proxy/src/main/java/com/hotels/styx/startup/StartupException.java new file mode 100644 index 0000000000..07431fb46a --- /dev/null +++ b/components/proxy/src/main/java/com/hotels/styx/startup/StartupException.java @@ -0,0 +1,36 @@ +/* + Copyright (C) 2013-2019 Expedia Inc. + + 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.hotels.styx.startup; + +/** + * An exception that occurs during start-up. + */ +public class StartupException extends RuntimeException { + public StartupException() { + } + + public StartupException(String message) { + super(message); + } + + public StartupException(String message, Throwable cause) { + super(message, cause); + } + + public StartupException(Throwable cause) { + super(cause); + } +} diff --git a/components/proxy/src/main/java/com/hotels/styx/startup/StyxPipelineFactory.java b/components/proxy/src/main/java/com/hotels/styx/startup/StyxPipelineFactory.java index 5f4394a0f1..77009e99a6 100644 --- a/components/proxy/src/main/java/com/hotels/styx/startup/StyxPipelineFactory.java +++ b/components/proxy/src/main/java/com/hotels/styx/startup/StyxPipelineFactory.java @@ -1,5 +1,5 @@ /* - Copyright (C) 2013-2018 Expedia Inc. + Copyright (C) 2013-2019 Expedia Inc. Licensed under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with the License. @@ -46,6 +46,7 @@ import com.hotels.styx.routing.handlers.StaticResponseHandler; import com.hotels.styx.routing.interceptors.RewriteInterceptor; +import java.util.List; import java.util.Map; import java.util.concurrent.ConcurrentHashMap; @@ -60,7 +61,7 @@ public StyxPipelineFactory() { } @Override - public HttpHandler create(StyxServerComponents config) { + public HttpHandler create(StyxServerComponents config, List plugins) { BuiltinInterceptorsFactory builtinInterceptorsFactory = new BuiltinInterceptorsFactory( ImmutableMap.of("Rewrite", new RewriteInterceptor.ConfigFactory())); @@ -69,15 +70,15 @@ public HttpHandler create(StyxServerComponents config) { Map objectFactories = createBuiltinRoutingObjectFactories( config.environment(), config.services(), - config.plugins(), + plugins, builtinInterceptorsFactory, requestTracking); RouteHandlerFactory routeHandlerFactory = new RouteHandlerFactory(objectFactories, new ConcurrentHashMap<>()); return styxHttpPipeline( - config.environment().styxConfig(), - configuredPipeline(config.environment(), config.services(), config.plugins(), routeHandlerFactory), + config.environment().configuration(), + configuredPipeline(config.environment(), config.services(), plugins, routeHandlerFactory), requestTracking); } diff --git a/components/proxy/src/main/java/com/hotels/styx/startup/StyxServerComponents.java b/components/proxy/src/main/java/com/hotels/styx/startup/StyxServerComponents.java index 3a3d3d91e1..8a557f3ee3 100644 --- a/components/proxy/src/main/java/com/hotels/styx/startup/StyxServerComponents.java +++ b/components/proxy/src/main/java/com/hotels/styx/startup/StyxServerComponents.java @@ -26,18 +26,17 @@ import com.hotels.styx.api.extension.service.spi.StyxService; import com.hotels.styx.api.metrics.codahale.CodaHaleMetricRegistry; import com.hotels.styx.api.plugins.spi.Plugin; -import com.hotels.styx.proxy.plugin.NamedPlugin; import com.hotels.styx.startup.extensions.ConfiguredPluginFactory; import java.util.HashMap; import java.util.List; import java.util.Map; +import java.util.Optional; import static com.hotels.styx.Version.readVersionFrom; import static com.hotels.styx.infrastructure.logging.LOGBackConfigurer.initLogging; import static com.hotels.styx.startup.ServicesLoader.SERVICES_FROM_CONFIG; import static com.hotels.styx.startup.StyxServerComponents.LoggingSetUp.DO_NOT_MODIFY; -import static com.hotels.styx.startup.extensions.PluginLoadingForStartup.loadPlugins; import static java.util.Objects.requireNonNull; import static java.util.concurrent.Executors.newSingleThreadExecutor; import static java.util.stream.Collectors.toList; @@ -48,7 +47,7 @@ public class StyxServerComponents { private final Environment environment; private final Map services; - private final List plugins; + private final List pluginFactories; private StyxServerComponents(Builder builder) { StyxConfig styxConfig = requireNonNull(builder.styxConfig); @@ -56,10 +55,7 @@ private StyxServerComponents(Builder builder) { this.environment = newEnvironment(styxConfig, builder.metricRegistry); builder.loggingSetUp.setUp(environment); - // TODO In further refactoring, we will probably want this loading to happen outside of this constructor call, so that it doesn't delay the admin server from starting up - this.plugins = builder.configuredPluginFactories == null - ? loadPlugins(environment) - : loadPlugins(environment, builder.configuredPluginFactories); + this.pluginFactories = builder.configuredPluginFactories; this.services = mergeServices( builder.servicesLoader.load(environment), @@ -75,8 +71,8 @@ public Map services() { return services; } - public List plugins() { - return plugins; + public Optional> pluginFactories() { + return Optional.ofNullable(pluginFactories); } private static Environment newEnvironment(StyxConfig styxConfig, MetricRegistry metricRegistry) { diff --git a/components/proxy/src/main/java/com/hotels/styx/startup/extensions/FailureHandling.java b/components/proxy/src/main/java/com/hotels/styx/startup/extensions/FailureHandling.java deleted file mode 100644 index 799ee71127..0000000000 --- a/components/proxy/src/main/java/com/hotels/styx/startup/extensions/FailureHandling.java +++ /dev/null @@ -1,88 +0,0 @@ -/* - Copyright (C) 2013-2019 Expedia Inc. - - 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.hotels.styx.startup.extensions; - -import com.hotels.styx.common.FailureHandlingStrategy; -import com.hotels.styx.common.Pair; -import com.hotels.styx.proxy.plugin.NamedPlugin; -import com.hotels.styx.proxy.plugin.PluginStartupException; -import com.hotels.styx.spi.config.SpiExtension; -import org.slf4j.Logger; - -import java.util.List; -import java.util.Map; -import java.util.function.BiFunction; -import java.util.function.Function; - -import static com.hotels.styx.common.MapStream.stream; -import static java.lang.String.format; -import static java.util.stream.Collectors.toList; -import static org.slf4j.LoggerFactory.getLogger; - -/** - * Strategies for handling failure during start-up. - */ -final class FailureHandling { - private static final Logger LOGGER = getLogger(FailureHandling.class); - - static final FailureHandlingStrategy, ConfiguredPluginFactory> PLUGIN_FACTORY_LOADING_FAILURE_HANDLING_STRATEGY = - new FailureHandlingStrategy.Builder, ConfiguredPluginFactory>() - - .doImmediatelyOnEachFailure((plugin, err) -> - LOGGER.error(format("Could not load plugin: pluginName=%s; factoryClass=%s", plugin.key(), plugin.value().factory().factoryClass()), err)) - - .doOnFailuresAfterAllProcessing(failures -> { - throw new PluginStartupException(afterFailuresErrorMessage(failures, Pair::key)); - - }).build(); - - static final FailureHandlingStrategy PLUGIN_STARTUP_FAILURE_HANDLING_STRATEGY = - new FailureHandlingStrategy.Builder() - - .doImmediatelyOnEachFailure((plugin, err) -> - LOGGER.error(format("Could not load plugin: pluginName=%s; factoryClass=%s", plugin.name(), plugin.pluginFactory().getClass().getName()), err)) - - .doOnFailuresAfterAllProcessing(failures -> { - throw new PluginStartupException(afterFailuresErrorMessage(failures, ConfiguredPluginFactory::name)); - - }).build(); - - private FailureHandling() { - } - - private static String afterFailuresErrorMessage(Map failures, Function getPluginName) { - List failedPlugins = mapKeys(failures, getPluginName); - - List causes = mapEntries(failures, (key, err) -> { - // please note, transforming the exception to a String (as is done here indirectly) will not include the stack trace - return format("%s: %s", getPluginName.apply(key), err); - }); - - return format("%s plugin(s) could not be loaded: failedPlugins=%s; failureCauses=%s", failures.size(), failedPlugins, causes); - } - - private static List mapKeys(Map map, Function function) { - return stream(map) - .mapToObject((k, v) -> function.apply(k)) - .collect(toList()); - } - - private static List mapEntries(Map map, BiFunction function) { - return stream(map) - .mapToObject(function) - .collect(toList()); - } -} diff --git a/components/proxy/src/main/java/com/hotels/styx/startup/extensions/PluginLoadingForStartup.java b/components/proxy/src/main/java/com/hotels/styx/startup/extensions/PluginLoadingForStartup.java index 0864541030..eebf61651c 100644 --- a/components/proxy/src/main/java/com/hotels/styx/startup/extensions/PluginLoadingForStartup.java +++ b/components/proxy/src/main/java/com/hotels/styx/startup/extensions/PluginLoadingForStartup.java @@ -23,16 +23,29 @@ import com.hotels.styx.common.Pair; import com.hotels.styx.proxy.plugin.FileSystemPluginFactoryLoader; import com.hotels.styx.proxy.plugin.NamedPlugin; +import com.hotels.styx.proxy.plugin.PluginStartupException; import com.hotels.styx.proxy.plugin.PluginsMetadata; import com.hotels.styx.spi.config.SpiExtension; import org.slf4j.Logger; import java.util.List; +import java.util.Map; +import java.util.function.BiFunction; +import java.util.function.Function; +import static com.hotels.styx.common.MapStream.stream; +import static com.hotels.styx.common.SequenceProcessor.processSequence; import static com.hotels.styx.proxy.plugin.NamedPlugin.namedPlugin; -import static com.hotels.styx.startup.extensions.FailureHandling.PLUGIN_FACTORY_LOADING_FAILURE_HANDLING_STRATEGY; -import static com.hotels.styx.startup.extensions.FailureHandling.PLUGIN_STARTUP_FAILURE_HANDLING_STRATEGY; +import static com.hotels.styx.startup.extensions.PluginStatusNotifications.PluginPipelineStatus.AT_LEAST_ONE_PLUGIN_FAILED; +import static com.hotels.styx.startup.extensions.PluginStatusNotifications.PluginStatus.CONSTRUCTED; +import static com.hotels.styx.startup.extensions.PluginStatusNotifications.PluginStatus.CONSTRUCTING; +import static com.hotels.styx.startup.extensions.PluginStatusNotifications.PluginStatus.FAILED_WHILE_CONSTRUCTING; +import static com.hotels.styx.startup.extensions.PluginStatusNotifications.PluginStatus.FAILED_WHILE_LOADING_CLASSES; +import static com.hotels.styx.startup.extensions.PluginStatusNotifications.PluginStatus.LOADED_CLASSES; +import static com.hotels.styx.startup.extensions.PluginStatusNotifications.PluginStatus.LOADING_CLASSES; +import static java.lang.String.format; import static java.util.Collections.emptyList; +import static java.util.stream.Collectors.toList; import static org.slf4j.LoggerFactory.getLogger; /** @@ -64,10 +77,30 @@ public static List loadPlugins(Environment environment) { } private static List loadFactoriesFromConfig(Environment environment) { - return environment.configuration().get("plugins", PluginsMetadata.class) + List> configList = environment.configuration().get("plugins", PluginsMetadata.class) .map(PluginsMetadata::activePlugins) - .map(inputs -> PLUGIN_FACTORY_LOADING_FAILURE_HANDLING_STRATEGY.process(inputs, PluginLoadingForStartup::loadPluginFactory)) .orElse(emptyList()); + + PluginStatusNotifications notifications = new PluginStatusNotifications(environment.configStore()); + + return processSequence(configList) + .map(config -> { + notifications.notifyPluginStatus(config.key(), LOADING_CLASSES); + return loadPluginFactory(config); + }) + + .onEachSuccess((config, factory) -> notifications.notifyPluginStatus(config.key(), LOADED_CLASSES)) + + .onEachFailure((config, err) -> { + notifications.notifyPluginStatus(config.key(), FAILED_WHILE_LOADING_CLASSES); + environment.configStore().set("startup.plugins." + config.key(), "failed-while-loading"); + LOGGER.error(format("Could not load plugin: pluginName=%s; factoryClass=%s", config.key(), config.value().factory().factoryClass()), err); + }) + + .failuresPostProcessing(failures -> { + notifications.notifyPluginPipelineStatus(AT_LEAST_ONE_PLUGIN_FAILED); + throw new PluginStartupException(afterFailuresErrorMessage(failures, Pair::key)); + }).collect(); } private static ConfiguredPluginFactory loadPluginFactory(Pair pair) { @@ -80,14 +113,25 @@ private static ConfiguredPluginFactory loadPluginFactory(Pair loadPluginsFromFactories(Environment environment, List factories) { - return PLUGIN_STARTUP_FAILURE_HANDLING_STRATEGY.process(factories, factory -> { + PluginStatusNotifications notifications = new PluginStatusNotifications(environment.configStore()); - LOGGER.info("Instantiating Plugin, pluginName={}...", factory.name()); - NamedPlugin plugin = loadPlugin(environment, factory); + return processSequence(factories) + .map(factory -> { + notifications.notifyPluginStatus(factory.name(), CONSTRUCTING); + return loadPlugin(environment, factory); + }) - LOGGER.info("Instantiated Plugin, pluginName={}", factory.name()); - return plugin; - }); + .onEachSuccess((factory, plugin) -> notifications.notifyPluginStatus(factory.name(), CONSTRUCTED)) + + .onEachFailure((factory, err) -> { + notifications.notifyPluginStatus(factory.name(), FAILED_WHILE_CONSTRUCTING); + LOGGER.error(format("Could not load plugin: pluginName=%s; factoryClass=%s", factory.name(), factory.pluginFactory().getClass().getName()), err); + }) + + .failuresPostProcessing(failures -> { + notifications.notifyPluginPipelineStatus(AT_LEAST_ONE_PLUGIN_FAILED); + throw new PluginStartupException(afterFailuresErrorMessage(failures, ConfiguredPluginFactory::name)); + }).collect(); } private static NamedPlugin loadPlugin(Environment environment, ConfiguredPluginFactory factory) { @@ -112,4 +156,27 @@ public MetricRegistry metricRegistry() { return namedPlugin(factory.name(), plugin); } + + private static String afterFailuresErrorMessage(Map failures, Function getPluginName) { + List failedPlugins = mapKeys(failures, getPluginName); + + List causes = mapEntries(failures, (key, err) -> { + // please note, transforming the exception to a String (as is done here indirectly) will not include the stack trace + return format("%s: %s", getPluginName.apply(key), err); + }); + + return format("%s plugin(s) could not be loaded: failedPlugins=%s; failureCauses=%s", failures.size(), failedPlugins, causes); + } + + private static List mapKeys(Map map, Function function) { + return stream(map) + .mapToObject((k, v) -> function.apply(k)) + .collect(toList()); + } + + private static List mapEntries(Map map, BiFunction function) { + return stream(map) + .mapToObject(function) + .collect(toList()); + } } diff --git a/components/proxy/src/main/java/com/hotels/styx/startup/extensions/PluginStatusNotifications.java b/components/proxy/src/main/java/com/hotels/styx/startup/extensions/PluginStatusNotifications.java new file mode 100644 index 0000000000..d4218348e3 --- /dev/null +++ b/components/proxy/src/main/java/com/hotels/styx/startup/extensions/PluginStatusNotifications.java @@ -0,0 +1,93 @@ +/* + Copyright (C) 2013-2019 Expedia Inc. + + 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.hotels.styx.startup.extensions; + +import com.hotels.styx.configstore.ConfigStore; + +import static java.lang.String.format; +import static java.util.Objects.requireNonNull; + +/** + * For setting the plugin startup status in the config store. + */ +public class PluginStatusNotifications { + public static final String PLUGIN_STATUS_KEY_PREFIX = "startup.plugins"; + public static final String PLUGIN_STATUS_KEY_FORMAT = PLUGIN_STATUS_KEY_PREFIX + ".%s"; + public static final String PLUGIN_PIPELINE_STATUS_KEY = "startup.plugin-pipeline"; + + private final ConfigStore configStore; + + /** + * Construct a new instance. + * + * @param configStore config store. + */ + public PluginStatusNotifications(ConfigStore configStore) { + this.configStore = requireNonNull(configStore); + } + + /** + * Notifies of plugin status using config store. + * + * @param pluginName name of plugin + * @param status status + */ + public void notifyPluginStatus(String pluginName, PluginStatus status) { + configStore.set(format(PLUGIN_STATUS_KEY_FORMAT, pluginName), status); + } + + /** + * Notifies of overall plugin pipeline status using config store. + * + * @param status status + */ + public void notifyPluginPipelineStatus(PluginPipelineStatus status) { + configStore.set(PLUGIN_PIPELINE_STATUS_KEY, status); + } + + /** + * Status for overall plugin pipeline during start-up. + */ + public enum PluginPipelineStatus { + INCOMPLETE, ALL_PLUGINS_COMPLETE, AT_LEAST_ONE_PLUGIN_FAILED + } + + /** + * Status for a plugin to be in during start-up. + */ + public enum PluginStatus { + LOADING_CLASSES("incomplete:loading-classes"), + LOADED_CLASSES("incomplete:loaded-classes"), + CONSTRUCTING("incomplete:constructing"), + CONSTRUCTED("incomplete:constructed"), + LIFECYCLE_STARTING("incomplete:lifecycle-starting"), + COMPLETE("complete"), + FAILED_WHILE_LOADING_CLASSES("failed:loading-classes"), + FAILED_WHILE_CONSTRUCTING("failed:constructing"), + FAILED_WHILE_LIFECYCLE_STARTING("failed:lifecycle-starting"); + + private final String description; + + PluginStatus(String description) { + this.description = requireNonNull(description); + } + + @Override + public String toString() { + return description; + } + } +} diff --git a/components/proxy/src/test/java/com/hotels/styx/StyxServerTest.java b/components/proxy/src/test/java/com/hotels/styx/StyxServerTest.java index 6ffe5250c0..24c1c55d9b 100644 --- a/components/proxy/src/test/java/com/hotels/styx/StyxServerTest.java +++ b/components/proxy/src/test/java/com/hotels/styx/StyxServerTest.java @@ -16,7 +16,6 @@ package com.hotels.styx; import com.google.common.collect.ImmutableMap; -import com.google.common.util.concurrent.Service; import com.hotels.styx.admin.AdminServerConfig; import com.hotels.styx.api.Eventual; import com.hotels.styx.api.LiveHttpRequest; @@ -30,9 +29,8 @@ import com.hotels.styx.infrastructure.MemoryBackedRegistry; import com.hotels.styx.infrastructure.RegistryServiceAdapter; import com.hotels.styx.proxy.ProxyServerConfig; -import com.hotels.styx.proxy.plugin.NamedPlugin; import com.hotels.styx.server.HttpConnectorConfig; -import com.hotels.styx.startup.ProxyServerSetUp; +import com.hotels.styx.startup.PluginStartupService; import com.hotels.styx.startup.StyxServerComponents; import com.hotels.styx.support.matchers.LoggingTestSupport; import io.netty.util.ResourceLeakDetector; @@ -43,21 +41,22 @@ import java.lang.reflect.Field; import java.util.Map; import java.util.concurrent.CompletableFuture; +import java.util.concurrent.TimeoutException; import static ch.qos.logback.classic.Level.ERROR; import static com.google.common.util.concurrent.Service.State.FAILED; import static com.hotels.styx.api.configuration.Configuration.EMPTY_CONFIGURATION; -import static com.hotels.styx.proxy.plugin.NamedPlugin.namedPlugin; import static com.hotels.styx.support.matchers.LoggingEventMatcher.loggingEvent; import static io.netty.util.ResourceLeakDetector.Level.DISABLED; -import static java.lang.System.currentTimeMillis; import static java.util.Collections.emptyList; +import static java.util.concurrent.TimeUnit.SECONDS; import static org.hamcrest.MatcherAssert.assertThat; import static org.hamcrest.Matchers.hasItem; import static org.hamcrest.Matchers.is; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.verify; import static org.mockito.Mockito.when; +import static org.testng.Assert.fail; public class StyxServerTest { private LoggingTestSupport log; @@ -66,7 +65,7 @@ public class StyxServerTest { @BeforeMethod public void setUp() { log = new LoggingTestSupport(StyxServer.class); - pssLog = new LoggingTestSupport(ProxyServerSetUp.class); + pssLog = new LoggingTestSupport(PluginStartupService.class); } @AfterMethod @@ -76,7 +75,7 @@ public void removeAppender() { } @Test - public void invokesPluginLifecycleMethods() { + public void invokesPluginLifecycleMethods() throws TimeoutException { Plugin pluginMock1 = mock(Plugin.class); Plugin pluginMock2 = mock(Plugin.class); @@ -85,11 +84,11 @@ public void invokesPluginLifecycleMethods() { "mockplugin2", pluginMock2 )); try { - styxServer.startAsync().awaitRunning(); + styxServer.startAsync().awaitRunning(10, SECONDS); verify(pluginMock1).styxStarting(); verify(pluginMock2).styxStarting(); - styxServer.stopAsync().awaitTerminated(); + styxServer.stopAsync().awaitTerminated(10, SECONDS); verify(pluginMock1).styxStopping(); verify(pluginMock2).styxStopping(); } finally { @@ -110,15 +109,14 @@ public void disablesResourceLeakDetectionByDefault() { } @Test - public void stopsTheServerWhenPluginFailsToStart() { + public void stopsTheServerWhenPluginFailsToStart() throws TimeoutException { StyxServer styxServer = null; try { styxServer = styxServerWithPlugins(ImmutableMap.of( "foo", new NonStarterPlugin("foo"), "mockplugin3", mock(Plugin.class))); - Service service = styxServer.startAsync(); - eventually(() -> assertThat(service.state(), is(FAILED))); + tryToStartExpectingFailure(styxServer); assertThat(pssLog.log(), hasItem( loggingEvent(ERROR, "Error starting plugin 'foo'", RuntimeException.class, "Plugin start test error: foo"))); @@ -130,7 +128,7 @@ public void stopsTheServerWhenPluginFailsToStart() { } @Test - public void allPluginsAreStartedEvenIfSomeFail() { + public void allPluginsAreStartedEvenIfSomeFail() throws TimeoutException { StyxServer styxServer = null; try { Plugin pluginMock2 = mock(Plugin.class); @@ -142,8 +140,7 @@ public void allPluginsAreStartedEvenIfSomeFail() { "plug3", new NonStarterPlugin("plug3"), "plug4", pluginMock4)); - Service service = styxServer.startAsync(); - eventually(() -> assertThat(service.state(), is(FAILED))); + tryToStartExpectingFailure(styxServer); assertThat(pssLog.log(), hasItem(loggingEvent(ERROR, "Error starting plugin 'plug1'", RuntimeException.class, "Plugin start test error: plug1"))); assertThat(pssLog.log(), hasItem(loggingEvent(ERROR, "Error starting plugin 'plug3'", RuntimeException.class, "Plugin start test error: plug3"))); @@ -162,14 +159,13 @@ public void systemExitIsCalledWhenCreateStyxServerFails() { } @Test - public void serverDoesNotStartIfServiceFails() { + public void serverDoesNotStartIfServiceFails() throws TimeoutException { StyxServer styxServer = null; try { StyxService testService = registryThatFailsToStart(); styxServer = styxServerWithBackendServiceRegistry(testService); - Service serverService = styxServer.startAsync(); - eventually(() -> assertThat(serverService.state(), is(FAILED))); + tryToStartExpectingFailure(styxServer); assertThat(styxServer.state(), is(FAILED)); } finally { @@ -177,6 +173,15 @@ public void serverDoesNotStartIfServiceFails() { } } + private static void tryToStartExpectingFailure(StyxServer server) throws TimeoutException { + try { + server.startAsync().awaitRunning(3, SECONDS); + fail("Expected failure but succeeded"); + } catch (IllegalStateException e) { + // Desired behaviour, so no action taken + } + } + private static StyxService registryThatFailsToStart() { Registry registry = mock(Registry.class); when(registry.get()).thenReturn(emptyList()); @@ -232,10 +237,6 @@ private static void stopIfRunning(StyxServer styxServer) { } } - private static NamedPlugin failingPlugin(String id) { - return namedPlugin(id, new NonStarterPlugin(id)); - } - static class NonStarterPlugin implements Plugin { private final String id; @@ -254,19 +255,6 @@ public void styxStarting() { } } - private static void eventually(Runnable block) { - long startTime = currentTimeMillis(); - while (currentTimeMillis() - startTime < 3000) { - try { - block.run(); - return; - } catch (Exception e) { - // pass - } - } - throw new AssertionError("Eventually block did not complete in 3 seconds."); - } - private static Runtime captureSystemExit(Runnable block) { try { Runtime originalRuntime = Runtime.getRuntime(); diff --git a/components/proxy/src/test/java/com/hotels/styx/admin/handlers/StartupStatusHandlerTest.java b/components/proxy/src/test/java/com/hotels/styx/admin/handlers/StartupStatusHandlerTest.java new file mode 100644 index 0000000000..75c7b48226 --- /dev/null +++ b/components/proxy/src/test/java/com/hotels/styx/admin/handlers/StartupStatusHandlerTest.java @@ -0,0 +1,81 @@ +/* + Copyright (C) 2013-2019 Expedia Inc. + + 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.hotels.styx.admin.handlers; + +import com.fasterxml.jackson.databind.ObjectMapper; +import com.hotels.styx.api.HttpInterceptor; +import com.hotels.styx.api.HttpResponse; +import com.hotels.styx.configstore.ConfigStore; +import com.hotels.styx.proxy.ProxyServerConfig; +import com.hotels.styx.server.HttpConnectorConfig; +import com.hotels.styx.startup.extensions.PluginStatusNotifications; +import org.testng.annotations.Test; + +import java.io.IOException; +import java.util.Map; + +import static com.hotels.styx.api.HttpResponseStatus.OK; +import static com.hotels.styx.api.LiveHttpRequest.get; +import static com.hotels.styx.startup.ProxyStatusNotifications.notifyProxyStatus; +import static com.hotels.styx.startup.extensions.PluginStatusNotifications.PluginStatus.CONSTRUCTING; +import static com.hotels.styx.startup.extensions.PluginStatusNotifications.PluginStatus.FAILED_WHILE_CONSTRUCTING; +import static com.hotels.styx.support.api.BlockingObservables.waitForResponse; +import static java.nio.charset.StandardCharsets.UTF_8; +import static org.hamcrest.MatcherAssert.assertThat; +import static org.hamcrest.Matchers.contains; +import static org.hamcrest.Matchers.hasEntry; +import static org.hamcrest.Matchers.instanceOf; +import static org.hamcrest.Matchers.is; +import static org.mockito.Mockito.mock; + +public class StartupStatusHandlerTest { + @Test + public void showsStatuses() throws IOException { + ConfigStore configStore = new ConfigStore(); + + PluginStatusNotifications notifications = new PluginStatusNotifications(configStore); + notifications.notifyPluginStatus("myPlugin1", CONSTRUCTING); + notifications.notifyPluginStatus("myPlugin2", FAILED_WHILE_CONSTRUCTING); + + notifyProxyStatus(configStore, new ProxyServerConfig.Builder() + .setHttpConnector(new HttpConnectorConfig(8080)) + .build()); + + StartupStatusHandler handler = new StartupStatusHandler(configStore); + + HttpResponse response = waitForResponse(handler.handle( + get("/").build(), mock(HttpInterceptor.Context.class))); + + assertThat(response.status(), is(OK)); + + Map map = new ObjectMapper().readValue(response.bodyAs(UTF_8), Map.class); + + assertThat(map.keySet(), contains("connectors", "plugins")); + assertThat(map.get("connectors"), is(instanceOf(Map.class))); + assertThat(map.get("plugins"), is(instanceOf(Map.class))); + + Map connectors = (Map) map.get("connectors"); + assertThat(connectors.size(), is(2)); + assertThat(connectors, hasEntry("http", "incomplete")); + assertThat(connectors, hasEntry("https", "disabled")); + + Map plugins = (Map) map.get("plugins"); + + assertThat(plugins.size(), is(2)); + assertThat(plugins, hasEntry("myPlugin1", "incomplete:constructing")); + assertThat(plugins, hasEntry("myPlugin2", "failed:constructing")); + } +} \ No newline at end of file diff --git a/components/proxy/src/test/java/com/hotels/styx/metrics/reporting/graphite/GraphiteReporterServiceTest.java b/components/proxy/src/test/java/com/hotels/styx/metrics/reporting/graphite/GraphiteReporterServiceTest.java index b9f7edda66..ae55a50530 100644 --- a/components/proxy/src/test/java/com/hotels/styx/metrics/reporting/graphite/GraphiteReporterServiceTest.java +++ b/components/proxy/src/test/java/com/hotels/styx/metrics/reporting/graphite/GraphiteReporterServiceTest.java @@ -1,5 +1,5 @@ /* - Copyright (C) 2013-2018 Expedia Inc. + Copyright (C) 2013-2019 Expedia Inc. Licensed under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with the License. @@ -71,7 +71,7 @@ public void stop() { public void logsWhenServiceStarts() { try { StyxFutures.await(service.start()); - assertThat(log.lastMessage(), is(loggingEvent(INFO, "Graphite service started, service name=\"Graphite\\-Reporter\\-test\""))); + assertThat(log.lastMessage(), is(loggingEvent(INFO, "Started serviceName=Graphite\\-Reporter\\-test"))); } finally { StyxFutures.await(service.stop()); } diff --git a/components/proxy/src/test/java/com/hotels/styx/startup/PluginStartupServiceTest.java b/components/proxy/src/test/java/com/hotels/styx/startup/PluginStartupServiceTest.java new file mode 100644 index 0000000000..e785be51c4 --- /dev/null +++ b/components/proxy/src/test/java/com/hotels/styx/startup/PluginStartupServiceTest.java @@ -0,0 +1,165 @@ +/* + Copyright (C) 2013-2019 Expedia Inc. + + 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.hotels.styx.startup; + +import com.google.common.collect.ImmutableMap; +import com.hotels.styx.StyxConfig; +import com.hotels.styx.api.Eventual; +import com.hotels.styx.api.extension.service.spi.ServiceFailureException; +import com.hotels.styx.api.plugins.spi.Plugin; +import com.hotels.styx.configstore.ConfigStore; +import com.hotels.styx.proxy.plugin.NamedPlugin; +import com.hotels.styx.startup.extensions.PluginStatusNotifications.PluginPipelineStatus; +import com.hotels.styx.startup.extensions.PluginStatusNotifications.PluginStatus; +import com.hotels.styx.support.matchers.LoggingTestSupport; +import org.testng.annotations.AfterMethod; +import org.testng.annotations.BeforeMethod; +import org.testng.annotations.Test; + +import java.util.Map; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.TimeoutException; + +import static com.hotels.styx.api.LiveHttpResponse.response; +import static com.hotels.styx.startup.extensions.PluginStatusNotifications.PluginPipelineStatus.ALL_PLUGINS_COMPLETE; +import static com.hotels.styx.startup.extensions.PluginStatusNotifications.PluginStatus.COMPLETE; +import static com.hotels.styx.startup.extensions.PluginStatusNotifications.PluginStatus.FAILED_WHILE_LIFECYCLE_STARTING; +import static com.hotels.styx.support.matchers.IsOptional.isValue; +import static java.util.concurrent.TimeUnit.SECONDS; +import static java.util.stream.Collectors.toMap; +import static java.util.stream.IntStream.range; +import static org.hamcrest.MatcherAssert.assertThat; +import static org.hamcrest.Matchers.containsString; +import static org.mockito.Mockito.doThrow; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.verify; +import static org.testng.Assert.fail; + +public class PluginStartupServiceTest { + private LoggingTestSupport log; + + @BeforeMethod + public void startRecordingLogs() { + log = new LoggingTestSupport(PluginStartupService.class); + } + + @AfterMethod + public void stopRecordingLogs() { + log.stop(); + } + + @Test + public void startsPlugins() throws InterruptedException, ExecutionException, TimeoutException { + Plugin plugin1 = (request, chain) -> Eventual.of(response().build()); + Plugin plugin2 = (request, chain) -> Eventual.of(response().build()); + + StyxServerComponents components = new StyxServerComponents.Builder() + .styxConfig(new StyxConfig()) + .plugins(ImmutableMap.of( + "foo", plugin1, + "bar", plugin2)) + .build(); + + new PluginStartupService(components).start().get(1, SECONDS); + + ConfigStore configStore = components.environment().configStore(); + + assertThat(configStore.get("startup.plugin-pipeline", PluginPipelineStatus.class), isValue(ALL_PLUGINS_COMPLETE)); + + assertThat(configStore.get("plugins.foo", NamedPlugin.class).map(NamedPlugin::originalPlugin), isValue(plugin1)); + assertThat(configStore.get("plugins.bar", NamedPlugin.class).map(NamedPlugin::originalPlugin), isValue(plugin2)); + + assertThat(configStore.get("startup.plugins.foo", PluginStatus.class), isValue(COMPLETE)); + assertThat(configStore.get("startup.plugins.bar", PluginStatus.class), isValue(COMPLETE)); + } + + @Test + public void logsPluginStartupFailures() throws InterruptedException, TimeoutException { + Plugin plugin2 = mock(Plugin.class); + Plugin plugin4 = mock(Plugin.class); + + doThrow(new IllegalStateException("Dummy")).when(plugin2).styxStarting(); + doThrow(new IllegalArgumentException("Dummy")).when(plugin4).styxStarting(); + + StyxServerComponents components = new StyxServerComponents.Builder() + .styxConfig(new StyxConfig()) + .plugins(nameByIndex(mock(Plugin.class), plugin2, mock(Plugin.class), plugin4)) + .build(); + + try { + new PluginStartupService(components).start().get(1, SECONDS); + fail("No exception thrown"); + } catch (ExecutionException e) { + if(e.getCause() instanceof ServiceFailureException) { + assertThat(log.toString(), containsString("Error starting plugin 'plugin2'")); + assertThat(log.toString(), containsString("Error starting plugin 'plugin4'")); + } else { + fail("Wrong exception type", e.getCause()); + } + } + + assertThat(log.toString(), containsString("Error starting plugin 'plugin2'")); + assertThat(log.toString(), containsString("Error starting plugin 'plugin4'")); + + ConfigStore configStore = components.environment().configStore(); + + assertThat(configStore.get("startup.plugins.plugin1", PluginStatus.class), isValue(COMPLETE)); + assertThat(configStore.get("startup.plugins.plugin2", PluginStatus.class), isValue(FAILED_WHILE_LIFECYCLE_STARTING)); + assertThat(configStore.get("startup.plugins.plugin3", PluginStatus.class), isValue(COMPLETE)); + assertThat(configStore.get("startup.plugins.plugin4", PluginStatus.class), isValue(FAILED_WHILE_LIFECYCLE_STARTING)); + } + + @Test + public void attemptsToStartAllPluginsBeforeFailing() throws TimeoutException, InterruptedException { + Plugin plugin1 = mock(Plugin.class); + Plugin plugin2 = mock(Plugin.class); + Plugin plugin3 = mock(Plugin.class); + Plugin plugin4 = mock(Plugin.class); + + doThrow(new IllegalStateException("Dummy")).when(plugin2).styxStarting(); + + StyxServerComponents components = new StyxServerComponents.Builder() + .styxConfig(new StyxConfig()) + .plugins(nameByIndex(plugin1, plugin2, plugin3, plugin4)) + .build(); + + try { + new PluginStartupService(components).start().get(1, SECONDS); + fail("No exception thrown"); + } catch (ExecutionException e) { + verify(plugin1).styxStarting(); + verify(plugin2).styxStarting(); + verify(plugin3).styxStarting(); + verify(plugin4).styxStarting(); + } + + ConfigStore configStore = components.environment().configStore(); + + assertThat(configStore.get("startup.plugins.plugin1", PluginStatus.class), isValue(COMPLETE)); + assertThat(configStore.get("startup.plugins.plugin2", PluginStatus.class), isValue(FAILED_WHILE_LIFECYCLE_STARTING)); + assertThat(configStore.get("startup.plugins.plugin3", PluginStatus.class), isValue(COMPLETE)); + assertThat(configStore.get("startup.plugins.plugin4", PluginStatus.class), isValue(COMPLETE)); + } + + private static Map nameByIndex(Plugin... plugins) { + return range(0, plugins.length) + .boxed() + .collect(toMap( + index -> "plugin" + (index + 1), + index -> plugins[index] + )); + } +} \ No newline at end of file diff --git a/components/proxy/src/test/java/com/hotels/styx/startup/ProxyServerSetUpTest.java b/components/proxy/src/test/java/com/hotels/styx/startup/ProxyServerSetUpTest.java index 76fd657e3b..1dd9eaaf9e 100644 --- a/components/proxy/src/test/java/com/hotels/styx/startup/ProxyServerSetUpTest.java +++ b/components/proxy/src/test/java/com/hotels/styx/startup/ProxyServerSetUpTest.java @@ -21,6 +21,7 @@ import com.hotels.styx.api.configuration.Configuration.MapBackedConfiguration; import com.hotels.styx.api.plugins.spi.Plugin; import com.hotels.styx.proxy.ProxyServerConfig; +import com.hotels.styx.proxy.plugin.NamedPlugin; import com.hotels.styx.server.HttpConnectorConfig; import com.hotels.styx.server.HttpServer; import com.hotels.styx.server.HttpsConnectorConfig; @@ -29,7 +30,9 @@ import org.testng.annotations.BeforeMethod; import org.testng.annotations.Test; +import java.util.List; import java.util.Map; +import java.util.concurrent.ExecutionException; import java.util.concurrent.TimeoutException; import static java.util.concurrent.TimeUnit.SECONDS; @@ -82,10 +85,11 @@ public void stopServer() throws TimeoutException { } @Test - public void createsServer() throws TimeoutException { + public void createsServer() throws TimeoutException, InterruptedException { StyxServerComponents components = new StyxServerComponents.Builder() .styxConfig(new StyxConfig(config)) .build(); + signalPluginsLoaded(components); PipelineFactory pipelineFactory = mockPipelineFactory(components); @@ -98,7 +102,7 @@ public void createsServer() throws TimeoutException { } @Test - public void notifiesPluginsOnStart() throws TimeoutException { + public void notifiesPluginsOnStart() throws TimeoutException, InterruptedException { Plugin plugin1 = mock(Plugin.class); Plugin plugin2 = mock(Plugin.class); @@ -106,6 +110,7 @@ public void notifiesPluginsOnStart() throws TimeoutException { .styxConfig(new StyxConfig(config)) .plugins(nameByIndex(plugin1, plugin2)) .build(); + signalPluginsLoaded(components); PipelineFactory pipelineFactory = mockPipelineFactory(components); @@ -118,7 +123,7 @@ public void notifiesPluginsOnStart() throws TimeoutException { } @Test - public void notifiesPluginsOnStop() throws TimeoutException { + public void notifiesPluginsOnStop() throws TimeoutException, InterruptedException { Plugin plugin1 = mock(Plugin.class); Plugin plugin2 = mock(Plugin.class); @@ -126,6 +131,7 @@ public void notifiesPluginsOnStop() throws TimeoutException { .styxConfig(new StyxConfig(config)) .plugins(nameByIndex(plugin1, plugin2)) .build(); + signalPluginsLoaded(components); PipelineFactory pipelineFactory = mockPipelineFactory(components); @@ -139,56 +145,7 @@ public void notifiesPluginsOnStop() throws TimeoutException { } @Test - public void logsPluginStartupFailures() { - Plugin plugin2 = mock(Plugin.class); - Plugin plugin4 = mock(Plugin.class); - - doThrow(new IllegalStateException("Dummy")).when(plugin2).styxStarting(); - doThrow(new IllegalArgumentException("Dummy")).when(plugin4).styxStarting(); - - StyxServerComponents components = new StyxServerComponents.Builder() - .styxConfig(new StyxConfig(config)) - .plugins(nameByIndex(mock(Plugin.class), plugin2, mock(Plugin.class), plugin4)) - .build(); - - PipelineFactory pipelineFactory = mockPipelineFactory(components); - - server = new ProxyServerSetUp(pipelineFactory).createProxyServer(components); - - expect(() -> server.startAsync().awaitRunning(3, SECONDS), IllegalStateException.class); - - assertThat(log.toString(), containsString("Error starting plugin 'plugin2'")); - assertThat(log.toString(), containsString("Error starting plugin 'plugin4'")); - } - - @Test - public void attemptsToStartAllPluginsBeforeFailing() { - Plugin plugin1 = mock(Plugin.class); - Plugin plugin2 = mock(Plugin.class); - Plugin plugin3 = mock(Plugin.class); - Plugin plugin4 = mock(Plugin.class); - - doThrow(new IllegalStateException("Dummy")).when(plugin2).styxStarting(); - - StyxServerComponents components = new StyxServerComponents.Builder() - .styxConfig(new StyxConfig(config)) - .plugins(nameByIndex(plugin1, plugin2, plugin3, plugin4)) - .build(); - - PipelineFactory pipelineFactory = mockPipelineFactory(components); - - server = new ProxyServerSetUp(pipelineFactory).createProxyServer(components); - - expect(() -> server.startAsync().awaitRunning(3, SECONDS), IllegalStateException.class); - - verify(plugin1).styxStarting(); - verify(plugin2).styxStarting(); - verify(plugin3).styxStarting(); - verify(plugin4).styxStarting(); - } - - @Test - public void logsPluginShutdownFailures() throws TimeoutException { + public void logsPluginShutdownFailures() throws TimeoutException, InterruptedException { Plugin plugin2 = mock(Plugin.class); Plugin plugin4 = mock(Plugin.class); @@ -199,6 +156,7 @@ public void logsPluginShutdownFailures() throws TimeoutException { .styxConfig(new StyxConfig(config)) .plugins(nameByIndex(mock(Plugin.class), plugin2, mock(Plugin.class), plugin4)) .build(); + signalPluginsLoaded(components); PipelineFactory pipelineFactory = mockPipelineFactory(components); @@ -212,10 +170,19 @@ public void logsPluginShutdownFailures() throws TimeoutException { assertThat(log.toString(), containsString("Error stopping plugin 'plugin4'")); } + private static void signalPluginsLoaded(StyxServerComponents components) { + try { + new PluginStartupService(components).start().get(20, SECONDS); + } catch (InterruptedException | ExecutionException | TimeoutException e) { + throw new RuntimeException(e); + } + } + private static PipelineFactory mockPipelineFactory(StyxServerComponents components) { PipelineFactory pipelineFactory = mock(PipelineFactory.class); HttpHandler pipeline = mock(HttpHandler.class); - when(pipelineFactory.create(components)).thenReturn(pipeline); + List plugins = components.environment().configStore().valuesStartingWith("plugins", NamedPlugin.class); + when(pipelineFactory.create(components, plugins)).thenReturn(pipeline); return pipelineFactory; } diff --git a/components/proxy/src/test/java/com/hotels/styx/startup/ServerServiceTest.java b/components/proxy/src/test/java/com/hotels/styx/startup/ServerServiceTest.java new file mode 100644 index 0000000000..58e8a985f7 --- /dev/null +++ b/components/proxy/src/test/java/com/hotels/styx/startup/ServerServiceTest.java @@ -0,0 +1,105 @@ +/* + Copyright (C) 2013-2019 Expedia Inc. + + 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.hotels.styx.startup; + +import com.google.common.util.concurrent.Service.Listener; +import com.hotels.styx.server.HttpServer; +import org.testng.annotations.BeforeMethod; +import org.testng.annotations.Test; + +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.Executor; +import java.util.concurrent.TimeoutException; +import java.util.function.Supplier; + +import static com.google.common.util.concurrent.Service.State.STOPPING; +import static com.hotels.styx.api.extension.service.spi.StyxServiceStatus.CREATED; +import static com.hotels.styx.api.extension.service.spi.StyxServiceStatus.RUNNING; +import static com.hotels.styx.api.extension.service.spi.StyxServiceStatus.STOPPED; +import static java.util.concurrent.TimeUnit.SECONDS; +import static org.hamcrest.MatcherAssert.assertThat; +import static org.hamcrest.Matchers.is; +import static org.mockito.Matchers.any; +import static org.mockito.Mockito.doAnswer; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.verifyNoMoreInteractions; +import static org.mockito.Mockito.verifyZeroInteractions; +import static org.mockito.Mockito.when; + +public class ServerServiceTest { + private Supplier serverCreator; + private volatile Listener serviceListener; + + @BeforeMethod + public void setUp() { + HttpServer server = mock(HttpServer.class); + when(server.startAsync()).thenReturn(server); + when(server.stopAsync()).thenReturn(server); + + doAnswer(invocation -> { + serviceListener = (Listener) invocation.getArguments()[0]; + return null; + }).when(server).addListener(any(Listener.class), any(Executor.class)); + + serverCreator = mock(Supplier.class); + when(serverCreator.get()).thenReturn(server); + } + + @Test + public void startsAndStops() throws InterruptedException, ExecutionException, TimeoutException { + ServerService service = new ServerService("foo", serverCreator::get); + + assertThat(service.serviceName(), is("foo")); + assertThat(service.status(), is(CREATED)); + + // START + CompletableFuture startFuture = service.start(); + serviceListener.running(); + startFuture.get(1, SECONDS); + + assertThat(service.status(), is(RUNNING)); + + // STOP + CompletableFuture stopFuture = service.stop(); + serviceListener.terminated(STOPPING); + stopFuture.get(1, SECONDS); + + assertThat(service.status(), is(STOPPED)); + } + + @Test + public void onlyUsesServerCreatorOnce() throws InterruptedException, ExecutionException, TimeoutException { + ServerService service = new ServerService("foo", serverCreator::get); + + verifyZeroInteractions(serverCreator); + + // START + CompletableFuture startFuture = service.start(); + serviceListener.running(); + startFuture.get(1, SECONDS); + + verify(serverCreator).get(); + + // STOP + CompletableFuture stopFuture = service.stop(); + serviceListener.terminated(STOPPING); + stopFuture.get(1, SECONDS); + + verifyNoMoreInteractions(serverCreator); + } +} \ No newline at end of file diff --git a/components/proxy/src/test/java/com/hotels/styx/startup/StyxServerComponentsTest.java b/components/proxy/src/test/java/com/hotels/styx/startup/StyxServerComponentsTest.java index f31bca6b50..5454b77749 100644 --- a/components/proxy/src/test/java/com/hotels/styx/startup/StyxServerComponentsTest.java +++ b/components/proxy/src/test/java/com/hotels/styx/startup/StyxServerComponentsTest.java @@ -57,22 +57,6 @@ public void setsUpLoggingOnBuild() { verify(loggingSetUp).setUp(any(Environment.class)); } - @Test - public void loadsPlugins() { - ConfiguredPluginFactory f1 = new ConfiguredPluginFactory("plugin1", any -> stubPlugin("MyResponse1")); - ConfiguredPluginFactory f2 = new ConfiguredPluginFactory("plugin2", any -> stubPlugin("MyResponse2")); - - StyxServerComponents components = new StyxServerComponents.Builder() - .styxConfig(new StyxConfig()) - .pluginFactories(ImmutableList.of(f1, f2)) - .build(); - - List plugins = components.plugins(); - List names = plugins.stream().map(NamedPlugin::name).collect(toList()); - - assertThat(names, contains("plugin1", "plugin2")); - } - @Test public void loadsServices() { StyxServerComponents components = new StyxServerComponents.Builder() diff --git a/components/proxy/src/test/java/com/hotels/styx/startup/extensions/PluginLoadingForStartupTest.java b/components/proxy/src/test/java/com/hotels/styx/startup/extensions/PluginLoadingForStartupTest.java index facb56e59d..2a2d8cf97c 100644 --- a/components/proxy/src/test/java/com/hotels/styx/startup/extensions/PluginLoadingForStartupTest.java +++ b/components/proxy/src/test/java/com/hotels/styx/startup/extensions/PluginLoadingForStartupTest.java @@ -16,6 +16,7 @@ package com.hotels.styx.startup.extensions; import com.fasterxml.jackson.annotation.JsonProperty; +import com.hotels.styx.Environment; import com.hotels.styx.StyxConfig; import com.hotels.styx.api.Eventual; import com.hotels.styx.api.LiveHttpRequest; @@ -27,6 +28,7 @@ import com.hotels.styx.api.plugins.spi.PluginFactory; import com.hotels.styx.infrastructure.configuration.yaml.YamlConfig; import com.hotels.styx.proxy.plugin.NamedPlugin; +import com.hotels.styx.startup.extensions.PluginStatusNotifications.PluginStatus; import com.hotels.styx.support.matchers.LoggingTestSupport; import org.testng.annotations.BeforeMethod; import org.testng.annotations.Test; @@ -36,7 +38,9 @@ import java.util.Objects; import static ch.qos.logback.classic.Level.ERROR; +import static com.hotels.styx.startup.extensions.PluginStatusNotifications.PluginStatus.CONSTRUCTED; import static com.hotels.styx.support.ResourcePaths.fixturesHome; +import static com.hotels.styx.support.matchers.IsOptional.isValue; import static com.hotels.styx.support.matchers.LoggingEventMatcher.loggingEvent; import static java.util.stream.Collectors.toList; import static org.hamcrest.MatcherAssert.assertThat; @@ -69,13 +73,16 @@ public void suppliesConfiguredPlugin() { " testConfiguration: test-foo-bar\n"; - List plugins = PluginLoadingForStartup.loadPlugins(environment(yaml)); + Environment environment = environment(yaml); + List plugins = PluginLoadingForStartup.loadPlugins(environment); NamedPlugin plugin = plugins.get(0); assertThat(plugin.originalPlugin(), is(instanceOf(MyPlugin.class))); assertThat(plugin.name(), is("myPlugin")); assertThat(((MyPlugin) plugin.originalPlugin()).myPluginConfig, is(new MyPluginConfig("test-foo-bar"))); + + assertThat(environment.configStore().get("startup.plugins.myPlugin", PluginStatus.class), isValue(CONSTRUCTED)); } @Test @@ -184,12 +191,12 @@ public void throwsExceptionIfFactoryFailsToLoadPlugin() { } @Test(expectedExceptions = RuntimeException.class, expectedExceptionsMessageRegExp = - "3 plugin\\(s\\) could not be loaded: failedPlugins=\\[myPlugin1, myPlugin2, myPlugin3\\]; failureCauses=\\[" + - "myPlugin1: com.hotels.styx.api.configuration.ConfigurationException: Could not load a plugin factory.*, " + - "myPlugin2: com.hotels.styx.api.configuration.ConfigurationException: Could not load a plugin factory.*, " + - "myPlugin3: com.hotels.styx.api.configuration.ConfigurationException: Could not load a plugin factory.*\\]") + "3 plugin\\(s\\) could not be loaded: failedPlugins=\\[myPlugin[123], myPlugin[123], myPlugin[123]\\]; failureCauses=\\[" + + "myPlugin[123]: com.hotels.styx.api.configuration.ConfigurationException: Could not load a plugin factory.*, " + + "myPlugin[123]: com.hotels.styx.api.configuration.ConfigurationException: Could not load a plugin factory.*, " + + "myPlugin[123]: com.hotels.styx.api.configuration.ConfigurationException: Could not load a plugin factory.*\\]") public void attemptsToLoadAllPluginsEvenIfSomePluginFactoriesCannotBeLoaded() { - LoggingTestSupport log = new LoggingTestSupport(FailureHandling.class); + LoggingTestSupport log = new LoggingTestSupport(PluginLoadingForStartup.class); try { String yaml = "" + @@ -219,12 +226,12 @@ public void attemptsToLoadAllPluginsEvenIfSomePluginFactoriesCannotBeLoaded() { } @Test(expectedExceptions = RuntimeException.class, expectedExceptionsMessageRegExp = - "3 plugin\\(s\\) could not be loaded: failedPlugins=\\[myPlugin1, myPlugin2, myPlugin3\\]; failureCauses=\\[" + - "myPlugin1: java.lang.RuntimeException: plugin factory error, " + - "myPlugin2: java.lang.RuntimeException: plugin factory error, " + - "myPlugin3: java.lang.RuntimeException: plugin factory error\\]") + "3 plugin\\(s\\) could not be loaded: failedPlugins=\\[myPlugin[123], myPlugin[123], myPlugin[123]\\]; failureCauses=\\[" + + "myPlugin[123]: java.lang.RuntimeException: plugin factory error, " + + "myPlugin[123]: java.lang.RuntimeException: plugin factory error, " + + "myPlugin[123]: java.lang.RuntimeException: plugin factory error\\]") public void attemptsToLoadAllPluginsEvenIfSomePluginFactoriesFailDuringExecution() { - LoggingTestSupport log = new LoggingTestSupport(FailureHandling.class); + LoggingTestSupport log = new LoggingTestSupport(PluginLoadingForStartup.class); try { String yaml = "" + diff --git a/components/server/src/main/java/com/hotels/styx/server/netty/NettyServer.java b/components/server/src/main/java/com/hotels/styx/server/netty/NettyServer.java index 24f7be5418..2c1750bb64 100644 --- a/components/server/src/main/java/com/hotels/styx/server/netty/NettyServer.java +++ b/components/server/src/main/java/com/hotels/styx/server/netty/NettyServer.java @@ -1,5 +1,5 @@ /* - Copyright (C) 2013-2018 Expedia Inc. + Copyright (C) 2013-2019 Expedia Inc. Licensed under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with the License. @@ -65,7 +65,6 @@ final class NettyServer extends AbstractService implements HttpServer { private final Optional httpConnector; private final Optional httpsConnector; - private final Iterable startupActions; private final HttpHandler httpHandler; private final ServerSocketBinder httpServerSocketBinder; private final ServerSocketBinder httpsServerSocketBinder; @@ -83,8 +82,6 @@ final class NettyServer extends AbstractService implements HttpServer { this.httpServerSocketBinder = httpConnector.map(ServerSocketBinder::new).orElse(null); this.httpsServerSocketBinder = httpsConnector.map(ServerSocketBinder::new).orElse(null); - - this.startupActions = nettyServerBuilder.startupActions(); } @Override @@ -109,15 +106,6 @@ public InetSocketAddress httpsAddress() { protected void doStart() { LOGGER.info("starting services"); - for (Runnable action : startupActions) { - try { - action.run(); - } catch (Exception e) { - notifyFailed(e); - return; - } - } - ServiceManager serviceManager = new ServiceManager( Stream.of(httpServerSocketBinder, httpsServerSocketBinder) .filter(Objects::nonNull) @@ -194,7 +182,7 @@ protected void doStart() { .childOption(ALLOCATOR, PooledByteBufAllocator.DEFAULT) .childHandler(new ChannelInitializer() { @Override - protected void initChannel(Channel ch) throws Exception { + protected void initChannel(Channel ch) { serverConnector.configure(ch, httpHandler); } }); @@ -208,7 +196,7 @@ protected void initChannel(Channel ch) throws Exception { Channel channel = future.channel(); channelGroup.add(channel); address = (InetSocketAddress) channel.localAddress(); - LOGGER.info("server connector {} bound successfully on port {} socket port {}", new Object[] {serverConnector.getClass(), port, address}); + LOGGER.info("server connector {} bound successfully on port {}", new Object[] {serverConnector.getClass(), address}); connectorStopper = new Stopper(bossGroup, workerGroup); notifyStarted(); } else { @@ -249,7 +237,7 @@ public Stopper(EventLoopGroup bossGroup, EventLoopGroup workerGroup) { } @Override - public Void call() throws Exception { + public Void call() { channelGroup.close().awaitUninterruptibly(); shutdownEventExecutorGroup(bossGroup); shutdownEventExecutorGroup(workerGroup); diff --git a/components/server/src/main/java/com/hotels/styx/server/netty/NettyServerBuilder.java b/components/server/src/main/java/com/hotels/styx/server/netty/NettyServerBuilder.java index a50b6165c4..5339accf85 100644 --- a/components/server/src/main/java/com/hotels/styx/server/netty/NettyServerBuilder.java +++ b/components/server/src/main/java/com/hotels/styx/server/netty/NettyServerBuilder.java @@ -1,5 +1,5 @@ /* - Copyright (C) 2013-2018 Expedia Inc. + Copyright (C) 2013-2019 Expedia Inc. Licensed under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with the License. @@ -15,10 +15,10 @@ */ package com.hotels.styx.server.netty; +import com.hotels.styx.api.Eventual; import com.hotels.styx.api.HttpHandler; import com.hotels.styx.api.LiveHttpResponse; import com.hotels.styx.api.MetricRegistry; -import com.hotels.styx.api.Eventual; import com.hotels.styx.server.HttpServer; import com.hotels.styx.server.ServerEventLoopFactory; import com.hotels.styx.server.netty.eventloop.PlatformAwareServerEventLoopFactory; @@ -26,15 +26,12 @@ import io.netty.channel.group.DefaultChannelGroup; import io.netty.util.concurrent.ImmediateEventExecutor; -import java.util.List; import java.util.Optional; import static com.google.common.base.Objects.firstNonNull; import static com.google.common.base.Preconditions.checkArgument; -import static com.google.common.collect.Lists.newCopyOnWriteArrayList; import static com.hotels.styx.api.HttpResponseStatus.NOT_FOUND; import static com.hotels.styx.server.netty.eventloop.ServerEventLoopFactories.memoize; -import static java.util.Arrays.asList; /** * A builder of {@link NettyServer} instances. @@ -48,7 +45,6 @@ public final class NettyServerBuilder { private String name = "styx"; private Optional httpConnector = Optional.empty(); private Optional httpsConnector = Optional.empty(); - private final List startupActions = newCopyOnWriteArrayList(); private HttpHandler httpHandler = (request, context) -> Eventual.of(LiveHttpResponse.response(NOT_FOUND).build()); public static NettyServerBuilder newBuilder() { @@ -118,15 +114,6 @@ Optional httpsConnector() { return httpsConnector; } - public NettyServerBuilder doOnStartUp(Runnable... startupActions) { - this.startupActions.addAll(asList(startupActions)); - return this; - } - - Iterable startupActions() { - return startupActions; - } - public HttpServer build() { checkArgument(httpConnector.isPresent() || httpsConnector.isPresent(), "Must configure at least one connector"); diff --git a/distribution/conf/default.yml b/distribution/conf/default.yml index bb30a9eb28..d1503b7600 100644 --- a/distribution/conf/default.yml +++ b/distribution/conf/default.yml @@ -86,17 +86,19 @@ url: encoding: unwiseCharactersToEncode: +startup: + timeoutSeconds: 600 + #originRestrictionCookie: yourCookieNameHere -# TODO comment out before raising PR -plugins: - active: demo - all: - demo: - factory: - class: "com.hotels.styx.startup.extensions.DemoPlugin$Factory" - classPath: "${STYX_HOME:}" - config: - adminText: "this is the admin page" - responseHeaderValue: "this is the response header value" +#plugins: +# active: demo +# all: +# demo: +# factory: +# class: "com.hotels.styx.startup.extensions.DemoPlugin$Factory" +# classPath: "${STYX_HOME:}" +# config: +# adminText: "this is the admin page" +# responseHeaderValue: "this is the response header value" diff --git a/support/test-api/src/main/java/com/hotels/styx/testapi/StyxServer.java b/support/test-api/src/main/java/com/hotels/styx/testapi/StyxServer.java index b9eb779ff7..b1d649ad7a 100644 --- a/support/test-api/src/main/java/com/hotels/styx/testapi/StyxServer.java +++ b/support/test-api/src/main/java/com/hotels/styx/testapi/StyxServer.java @@ -17,6 +17,7 @@ import com.google.common.collect.ImmutableMap; import com.google.common.collect.ImmutableSet; +import com.google.common.util.concurrent.Service; import com.hotels.styx.StyxConfig; import com.hotels.styx.admin.AdminServerConfig; import com.hotels.styx.api.MetricRegistry; @@ -80,6 +81,11 @@ private StyxServer start() { return this; } + private StyxServer startAsync() { + server.startAsync(); + return this; + } + private static StyxConfig styxConfig(Builder builder) { return new StyxConfig(new MapBackedConfiguration() .set("proxy", proxyServerConfig(builder)) @@ -257,5 +263,14 @@ private Builder addRoute(String pathPrefix, Set origins) { public StyxServer start() { return new StyxServer(this).start(); } + + /** + * Creates and starts the test server, but does not wait for startup to complete. + * + * @return new test server + */ + public StyxServer startAsync() { + return new StyxServer(this).startAsync(); + } } } diff --git a/system-tests/e2e-suite/src/test/java/com/hotels/styx/startup/StartupTest.java b/system-tests/e2e-suite/src/test/java/com/hotels/styx/startup/StartupTest.java new file mode 100644 index 0000000000..7d27869aab --- /dev/null +++ b/system-tests/e2e-suite/src/test/java/com/hotels/styx/startup/StartupTest.java @@ -0,0 +1,151 @@ +/* + Copyright (C) 2013-2019 Expedia Inc. + + 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.hotels.styx.startup; + +import com.hotels.styx.api.Eventual; +import com.hotels.styx.api.HttpResponse; +import com.hotels.styx.api.LiveHttpRequest; +import com.hotels.styx.api.LiveHttpResponse; +import com.hotels.styx.api.plugins.spi.Plugin; +import com.hotels.styx.api.plugins.spi.PluginFactory; +import com.hotels.styx.client.StyxHttpClient; +import com.hotels.styx.testapi.StyxServer; +import org.slf4j.Logger; +import org.testng.annotations.Test; + +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.TimeoutException; + +import static com.hotels.styx.api.HttpRequest.get; +import static com.hotels.styx.api.HttpResponseStatus.OK; +import static java.lang.System.currentTimeMillis; +import static java.lang.Thread.sleep; +import static java.nio.charset.StandardCharsets.UTF_8; +import static java.util.concurrent.TimeUnit.SECONDS; +import static org.hamcrest.MatcherAssert.assertThat; +import static org.hamcrest.Matchers.containsString; +import static org.hamcrest.Matchers.is; +import static org.slf4j.LoggerFactory.getLogger; +import static org.testng.Assert.fail; + +public class StartupTest { + private static final Logger LOGGER = getLogger(StartupTest.class); + + private StyxServer server; + + @Test + public void adminServerIsAvailableBeforeProxyServerCompletes() throws InterruptedException, TimeoutException, ExecutionException { + CountDownLatch latch = new CountDownLatch(1); + + server = new StyxServer.Builder() + .addPluginFactory("plug", new SlowToCreatePlugin.Factory(latch), null) + .startAsync(); + + int adminPort = waitForAdminPort(); + + StyxHttpClient client = new StyxHttpClient.Builder() + .build(); + + HttpResponse livenessResponse = client.send(get("http://localhost:" + adminPort + "/admin/ping").build()).get(1, SECONDS); + assertThat(livenessResponse.status(), is(OK)); + assertThat(livenessResponse.bodyAs(UTF_8), is("pong")); + + HttpResponse readinessResponse = client.send(get("http://localhost:" + adminPort + "/admin/startup").build()).get(1, SECONDS); + assertThat(readinessResponse.status(), is(OK)); + assertThat(readinessResponse.bodyAs(UTF_8), containsString("\"http\" : \"incomplete\"")); + + latch.countDown(); + + eventually(3, SECONDS, () -> { + HttpResponse readinessResponse2 = client.send(get("http://localhost:" + adminPort + "/admin/startup").build()).get(1, SECONDS); + assertThat(readinessResponse2.status(), is(OK)); + assertThat(readinessResponse2.bodyAs(UTF_8), containsString("\"http\" : \"complete\"")); + }); + } + + private int waitForAdminPort() throws InterruptedException { + while (throwsException(() -> server.adminPort())) { + sleep(100); + } + + return server.adminPort(); + } + + private static boolean throwsException(Runnable2 runnable2) { + try { + runnable2.run(); + return false; + } catch (Exception e) { + return true; + } + } + + private static void await(CountDownLatch latch) { + try { + boolean completed = latch.await(10, SECONDS); + + if (!completed) { + fail("Latch timeout"); + } + } catch (InterruptedException e) { + throw new RuntimeException(e); + } + } + + private static void eventually(long timeout, TimeUnit timeUnit, Runnable2 block) { + long timeoutMillis = timeUnit.toMillis(timeout); + long startTime = currentTimeMillis(); + Throwable err = null; + while (currentTimeMillis() - startTime < timeoutMillis) { + try { + block.run(); + return; + } catch (Throwable e) { + err = e; + } + } + fail("Eventually block did not complete in " + timeout + " " + timeUnit, err); + } + + private interface Runnable2 { + void run() throws Exception; + } + + private static final class SlowToCreatePlugin implements Plugin { + @Override + public Eventual intercept(LiveHttpRequest request, Chain chain) { + return chain.proceed(request); + } + + public static class Factory implements PluginFactory { + private final CountDownLatch latch; + + Factory(CountDownLatch latch) { + this.latch = latch; + } + + @Override + public Plugin create(Environment environment) { + LOGGER.info("Waiting for latch to release..."); + await(latch); + LOGGER.info("Latch released"); + return new SlowToCreatePlugin(); + } + } + } +} diff --git a/system-tests/e2e-suite/src/test/scala/com/hotels/styx/plugins/PluginAdminInterfaceSpec.scala b/system-tests/e2e-suite/src/test/scala/com/hotels/styx/plugins/PluginAdminInterfaceSpec.scala index f735a82f66..95af24a025 100644 --- a/system-tests/e2e-suite/src/test/scala/com/hotels/styx/plugins/PluginAdminInterfaceSpec.scala +++ b/system-tests/e2e-suite/src/test/scala/com/hotels/styx/plugins/PluginAdminInterfaceSpec.scala @@ -97,6 +97,23 @@ class PluginAdminInterfaceSpec extends FunSpec with StyxProxySpec with StyxClien response.bodyAs(UTF_8) should include("This plugin (plugw) does not expose any admin interfaces") } + + it("Exposes startup status") { + val response = get(styxServer.adminURL("/admin/startup")) + + response.bodyAs(UTF_8) should include regex """\{ + | "connectors" : \{ + | "http" : "complete", + | "https" : "disabled" + | \}, + | "plugins" : \{ + | "plug[w-z]" : "complete", + | "plug[w-z]" : "complete", + | "plug[w-z]" : "complete", + | "plug[w-z]" : "complete" + | \} + |\}""".stripMargin + } } private def get(url: String) = { From 36c8c13a190ef8f124e04b41bbb8804f550e5ca3 Mon Sep 17 00:00:00 2001 From: Kyle Vosper Date: Tue, 26 Mar 2019 10:55:08 +0000 Subject: [PATCH 2/7] Fix checkstyle stuff --- codequality/checkstyle_rules.xml | 38 +++++++++---------- codequality/checkstyle_suppressions.xml | 2 +- .../strategies/PowerOfTwoStrategy.java | 2 +- .../hotels/styx/startup/ServerService.java | 1 - docker/styx-env.sh | 16 ++++++++ 5 files changed, 37 insertions(+), 22 deletions(-) diff --git a/codequality/checkstyle_rules.xml b/codequality/checkstyle_rules.xml index ec43caffe5..7e2c52059e 100644 --- a/codequality/checkstyle_rules.xml +++ b/codequality/checkstyle_rules.xml @@ -68,7 +68,7 @@ - + @@ -76,21 +76,21 @@ - + - + - + - + @@ -179,7 +179,7 @@ - + @@ -188,7 +188,7 @@ - + @@ -196,7 +196,7 @@ - + @@ -292,7 +292,7 @@ - + @@ -304,7 +304,7 @@ - + @@ -321,17 +321,17 @@ - + - + - + @@ -354,39 +354,39 @@ - + - + - + - + - + - + diff --git a/codequality/checkstyle_suppressions.xml b/codequality/checkstyle_suppressions.xml index 7d698c2352..06a6eb64af 100644 --- a/codequality/checkstyle_suppressions.xml +++ b/codequality/checkstyle_suppressions.xml @@ -37,7 +37,7 @@ - + diff --git a/components/client/src/main/java/com/hotels/styx/client/loadbalancing/strategies/PowerOfTwoStrategy.java b/components/client/src/main/java/com/hotels/styx/client/loadbalancing/strategies/PowerOfTwoStrategy.java index 71a5d41ced..2f3e2b4f8b 100644 --- a/components/client/src/main/java/com/hotels/styx/client/loadbalancing/strategies/PowerOfTwoStrategy.java +++ b/components/client/src/main/java/com/hotels/styx/client/loadbalancing/strategies/PowerOfTwoStrategy.java @@ -1,5 +1,5 @@ /* - Copyright (C) 2013-2018 Expedia Inc. + Copyright (C) 2013-2019 Expedia Inc. Licensed under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with the License. diff --git a/components/proxy/src/main/java/com/hotels/styx/startup/ServerService.java b/components/proxy/src/main/java/com/hotels/styx/startup/ServerService.java index 0e2604900a..602c1a043a 100644 --- a/components/proxy/src/main/java/com/hotels/styx/startup/ServerService.java +++ b/components/proxy/src/main/java/com/hotels/styx/startup/ServerService.java @@ -58,7 +58,6 @@ public void running() { @Override public void failed(Service.State from, Throwable failure) { - failure.printStackTrace(); future.completeExceptionally(failure); } }, executor); diff --git a/docker/styx-env.sh b/docker/styx-env.sh index 86f360060e..94d31dbf37 100644 --- a/docker/styx-env.sh +++ b/docker/styx-env.sh @@ -1,4 +1,20 @@ #!/usr/bin/env bash +# +# Copyright (C) 2013-2019 Expedia Inc. +# +# 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. +# + # Styx JVM startup parameters # From b8a6c9cd81f4e1d22559b9f55626d9844632c31b Mon Sep 17 00:00:00 2001 From: Kyle Vosper Date: Tue, 26 Mar 2019 14:36:20 +0000 Subject: [PATCH 3/7] Remove commented bits from checkstyle rules --- codequality/checkstyle_rules.xml | 18 ------------------ 1 file changed, 18 deletions(-) diff --git a/codequality/checkstyle_rules.xml b/codequality/checkstyle_rules.xml index 7e2c52059e..616dcfcd59 100644 --- a/codequality/checkstyle_rules.xml +++ b/codequality/checkstyle_rules.xml @@ -179,7 +179,6 @@ - @@ -188,7 +187,6 @@ - @@ -196,7 +194,6 @@ - @@ -228,10 +225,6 @@ - - - - @@ -292,7 +285,6 @@ - @@ -304,7 +296,6 @@ - @@ -321,17 +312,14 @@ - - - @@ -354,39 +342,33 @@ - - - - - - From 7d7e2dcb104bf7113283215027e4d773193638ba Mon Sep 17 00:00:00 2001 From: Kyle Vosper Date: Tue, 26 Mar 2019 15:37:14 +0000 Subject: [PATCH 4/7] Address some comments --- .../service/spi/AbstractStyxService.java | 5 +- .../service/spi/AbstractStyxServiceTest.java | 22 ------ .../java/com/hotels/styx/api/Services.java | 74 +++++++++++++++++++ .../main/java/com/hotels/styx/StyxServer.java | 7 +- .../styx/startup/PluginStartupService.java | 8 +- .../extensions/PluginStatusNotifications.java | 4 +- .../startup/PluginStartupServiceTest.java | 8 +- 7 files changed, 91 insertions(+), 37 deletions(-) create mode 100644 components/common/src/main/java/com/hotels/styx/api/Services.java diff --git a/components/api/src/main/java/com/hotels/styx/api/extension/service/spi/AbstractStyxService.java b/components/api/src/main/java/com/hotels/styx/api/extension/service/spi/AbstractStyxService.java index 0f83d83613..55627614cc 100644 --- a/components/api/src/main/java/com/hotels/styx/api/extension/service/spi/AbstractStyxService.java +++ b/components/api/src/main/java/com/hotels/styx/api/extension/service/spi/AbstractStyxService.java @@ -29,6 +29,7 @@ import static com.hotels.styx.api.HttpResponse.response; import static com.hotels.styx.api.HttpResponseStatus.OK; import static com.hotels.styx.api.extension.service.spi.StyxServiceStatus.CREATED; +import static com.hotels.styx.api.extension.service.spi.StyxServiceStatus.FAILED; import static com.hotels.styx.api.extension.service.spi.StyxServiceStatus.RUNNING; import static com.hotels.styx.api.extension.service.spi.StyxServiceStatus.STARTING; import static com.hotels.styx.api.extension.service.spi.StyxServiceStatus.STOPPED; @@ -73,7 +74,7 @@ public CompletableFuture start() { logger.info("Starting serviceName={}...", serviceName()); return startService() .exceptionally(cause -> { - status.set(StyxServiceStatus.FAILED); + status.set(FAILED); logger.error("Failed to start serviceName=" + serviceName(), cause); throw new ServiceFailureException("Service failed to start.", cause); }) @@ -93,7 +94,7 @@ public CompletableFuture stop() { if (changed) { return stopService() .exceptionally(cause -> { - status.set(StyxServiceStatus.FAILED); + status.set(FAILED); logger.error("Failed to stop serviceName=" + serviceName(), cause); throw new ServiceFailureException("Service failed to stop.", cause); }) diff --git a/components/api/src/test/java/com/hotels/styx/api/extension/service/spi/AbstractStyxServiceTest.java b/components/api/src/test/java/com/hotels/styx/api/extension/service/spi/AbstractStyxServiceTest.java index f6453fcc28..f46b27764a 100644 --- a/components/api/src/test/java/com/hotels/styx/api/extension/service/spi/AbstractStyxServiceTest.java +++ b/components/api/src/test/java/com/hotels/styx/api/extension/service/spi/AbstractStyxServiceTest.java @@ -22,7 +22,6 @@ import java.util.concurrent.CompletableFuture; import java.util.concurrent.ExecutionException; -import java.util.concurrent.atomic.AtomicReference; import static com.hotels.styx.api.extension.service.spi.MockContext.MOCK_CONTEXT; import static com.hotels.styx.api.extension.service.spi.StyxServiceStatus.FAILED; @@ -34,9 +33,7 @@ import static java.util.concurrent.CompletableFuture.completedFuture; import static java.util.concurrent.TimeUnit.SECONDS; import static org.hamcrest.MatcherAssert.assertThat; -import static org.hamcrest.Matchers.instanceOf; import static org.hamcrest.Matchers.is; -import static org.testng.Assert.fail; public class AbstractStyxServiceTest { private final LiveHttpRequest get = LiveHttpRequest.get("/").build(); @@ -196,24 +193,6 @@ public void propagatesExceptions() throws Throwable { } } - @Test - public void canAttachDoOnErrorBehaviour() throws Throwable { - CompletableFuture future = new CompletableFuture<>(); - DerivedStyxService derivedStyxService = new DerivedStyxService("foo", future); - future.completeExceptionally(new RuntimeException("This is just a test")); - - AtomicReference caught = new AtomicReference<>(); - - try { - derivedStyxService - .doOnError(caught::set) - .start().get(1, SECONDS); - fail("Expected exception"); - } catch (ExecutionException e) { - assertThat(caught.get(), is(instanceOf(RuntimeException.class))); - } - } - static class DerivedStyxService extends AbstractStyxService { private final CompletableFuture startFuture; private final CompletableFuture stopFuture; @@ -238,5 +217,4 @@ protected CompletableFuture stopService() { return stopFuture; } } - } \ No newline at end of file diff --git a/components/common/src/main/java/com/hotels/styx/api/Services.java b/components/common/src/main/java/com/hotels/styx/api/Services.java new file mode 100644 index 0000000000..94ce0115b8 --- /dev/null +++ b/components/common/src/main/java/com/hotels/styx/api/Services.java @@ -0,0 +1,74 @@ +/* + Copyright (C) 2013-2019 Expedia Inc. + + 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.hotels.styx.api; + +import com.hotels.styx.api.extension.service.spi.StyxService; + +import java.util.Map; +import java.util.concurrent.CompletableFuture; +import java.util.function.Consumer; + +/** + * Provides utility methods for services. + */ +public final class Services { + private Services() { + } + + /** + * Derives a new service interface with added side-effects for errors. + * This could be used for logging, metrics, etc. + * + * @param service service + * @param consumer error consumer + * @return a new service interface + */ + public static StyxService doOnStartFailure(StyxService service, Consumer consumer) { + return new StyxService() { + @Override + public CompletableFuture start() { + return service.start().exceptionally(throwable -> { + consumer.accept(throwable); + + if (throwable instanceof RuntimeException) { + throw (RuntimeException) throwable; + } + + if (throwable instanceof Error) { + throw (Error) throwable; + } + + throw new RuntimeException(throwable); + }); + } + + @Override + public CompletableFuture stop() { + return service.stop(); + } + + @Override + public Map adminInterfaceHandlers() { + return service.adminInterfaceHandlers(); + } + + @Override + public String toString() { + return service.toString(); + } + }; + } +} diff --git a/components/proxy/src/main/java/com/hotels/styx/StyxServer.java b/components/proxy/src/main/java/com/hotels/styx/StyxServer.java index eb002aa8dc..db423eab04 100644 --- a/components/proxy/src/main/java/com/hotels/styx/StyxServer.java +++ b/components/proxy/src/main/java/com/hotels/styx/StyxServer.java @@ -46,6 +46,7 @@ import java.util.concurrent.atomic.AtomicReference; import static com.hotels.styx.ServerConfigSchema.validateServerConfiguration; +import static com.hotels.styx.api.Services.doOnStartFailure; import static com.hotels.styx.infrastructure.configuration.ConfigurationSource.configSource; import static com.hotels.styx.infrastructure.configuration.yaml.YamlConfigurationFormat.YAML; import static com.hotels.styx.infrastructure.logging.LOGBackConfigurer.shutdownLogging; @@ -174,9 +175,9 @@ public StyxServer(StyxServerComponents components, Stopwatch stopwatch) { StyxService adminServerService = new ServerService("adminServer", () -> this.adminServer.store(createAdminServer(components))); - StyxService proxyServerService = new ServerService("proxyServer", () -> - this.proxyServer.store(createProxyServer(components))) - .doOnError(err -> notifyProxyFailed(configStore)); + StyxService proxyServerService = doOnStartFailure(new ServerService("proxyServer", () -> + this.proxyServer.store(createProxyServer(components))), + err -> notifyProxyFailed(configStore)); this.serviceManager = new ServiceManager(new ArrayList() { { diff --git a/components/proxy/src/main/java/com/hotels/styx/startup/PluginStartupService.java b/components/proxy/src/main/java/com/hotels/styx/startup/PluginStartupService.java index a45a847bd8..4033924707 100644 --- a/components/proxy/src/main/java/com/hotels/styx/startup/PluginStartupService.java +++ b/components/proxy/src/main/java/com/hotels/styx/startup/PluginStartupService.java @@ -31,8 +31,8 @@ import static com.hotels.styx.startup.extensions.PluginStatusNotifications.PluginPipelineStatus.AT_LEAST_ONE_PLUGIN_FAILED; import static com.hotels.styx.startup.extensions.PluginStatusNotifications.PluginPipelineStatus.INCOMPLETE; import static com.hotels.styx.startup.extensions.PluginStatusNotifications.PluginStatus.COMPLETE; -import static com.hotels.styx.startup.extensions.PluginStatusNotifications.PluginStatus.FAILED_WHILE_LIFECYCLE_STARTING; -import static com.hotels.styx.startup.extensions.PluginStatusNotifications.PluginStatus.LIFECYCLE_STARTING; +import static com.hotels.styx.startup.extensions.PluginStatusNotifications.PluginStatus.FAILED_WHILE_STARTING; +import static com.hotels.styx.startup.extensions.PluginStatusNotifications.PluginStatus.STARTING; import static java.lang.String.format; import static org.slf4j.LoggerFactory.getLogger; @@ -72,7 +72,7 @@ protected CompletableFuture startService() { processSequence(plugins) .map(plugin -> { - notifications.notifyPluginStatus(plugin.name(), LIFECYCLE_STARTING); + notifications.notifyPluginStatus(plugin.name(), STARTING); plugin.styxStarting(); return null; }) @@ -83,7 +83,7 @@ protected CompletableFuture startService() { }) .onEachFailure((plugin, err) -> { - notifications.notifyPluginStatus(plugin.name(), FAILED_WHILE_LIFECYCLE_STARTING); + notifications.notifyPluginStatus(plugin.name(), FAILED_WHILE_STARTING); LOGGER.error("Error starting plugin '{}'", plugin.name(), err); }) diff --git a/components/proxy/src/main/java/com/hotels/styx/startup/extensions/PluginStatusNotifications.java b/components/proxy/src/main/java/com/hotels/styx/startup/extensions/PluginStatusNotifications.java index d4218348e3..47ee84f0d7 100644 --- a/components/proxy/src/main/java/com/hotels/styx/startup/extensions/PluginStatusNotifications.java +++ b/components/proxy/src/main/java/com/hotels/styx/startup/extensions/PluginStatusNotifications.java @@ -73,11 +73,11 @@ public enum PluginStatus { LOADED_CLASSES("incomplete:loaded-classes"), CONSTRUCTING("incomplete:constructing"), CONSTRUCTED("incomplete:constructed"), - LIFECYCLE_STARTING("incomplete:lifecycle-starting"), + STARTING("incomplete:starting"), COMPLETE("complete"), FAILED_WHILE_LOADING_CLASSES("failed:loading-classes"), FAILED_WHILE_CONSTRUCTING("failed:constructing"), - FAILED_WHILE_LIFECYCLE_STARTING("failed:lifecycle-starting"); + FAILED_WHILE_STARTING("failed:starting"); private final String description; diff --git a/components/proxy/src/test/java/com/hotels/styx/startup/PluginStartupServiceTest.java b/components/proxy/src/test/java/com/hotels/styx/startup/PluginStartupServiceTest.java index e785be51c4..6556c45c28 100644 --- a/components/proxy/src/test/java/com/hotels/styx/startup/PluginStartupServiceTest.java +++ b/components/proxy/src/test/java/com/hotels/styx/startup/PluginStartupServiceTest.java @@ -36,7 +36,7 @@ import static com.hotels.styx.api.LiveHttpResponse.response; import static com.hotels.styx.startup.extensions.PluginStatusNotifications.PluginPipelineStatus.ALL_PLUGINS_COMPLETE; import static com.hotels.styx.startup.extensions.PluginStatusNotifications.PluginStatus.COMPLETE; -import static com.hotels.styx.startup.extensions.PluginStatusNotifications.PluginStatus.FAILED_WHILE_LIFECYCLE_STARTING; +import static com.hotels.styx.startup.extensions.PluginStatusNotifications.PluginStatus.FAILED_WHILE_STARTING; import static com.hotels.styx.support.matchers.IsOptional.isValue; import static java.util.concurrent.TimeUnit.SECONDS; import static java.util.stream.Collectors.toMap; @@ -117,9 +117,9 @@ public void logsPluginStartupFailures() throws InterruptedException, TimeoutExce ConfigStore configStore = components.environment().configStore(); assertThat(configStore.get("startup.plugins.plugin1", PluginStatus.class), isValue(COMPLETE)); - assertThat(configStore.get("startup.plugins.plugin2", PluginStatus.class), isValue(FAILED_WHILE_LIFECYCLE_STARTING)); + assertThat(configStore.get("startup.plugins.plugin2", PluginStatus.class), isValue(FAILED_WHILE_STARTING)); assertThat(configStore.get("startup.plugins.plugin3", PluginStatus.class), isValue(COMPLETE)); - assertThat(configStore.get("startup.plugins.plugin4", PluginStatus.class), isValue(FAILED_WHILE_LIFECYCLE_STARTING)); + assertThat(configStore.get("startup.plugins.plugin4", PluginStatus.class), isValue(FAILED_WHILE_STARTING)); } @Test @@ -149,7 +149,7 @@ public void attemptsToStartAllPluginsBeforeFailing() throws TimeoutException, In ConfigStore configStore = components.environment().configStore(); assertThat(configStore.get("startup.plugins.plugin1", PluginStatus.class), isValue(COMPLETE)); - assertThat(configStore.get("startup.plugins.plugin2", PluginStatus.class), isValue(FAILED_WHILE_LIFECYCLE_STARTING)); + assertThat(configStore.get("startup.plugins.plugin2", PluginStatus.class), isValue(FAILED_WHILE_STARTING)); assertThat(configStore.get("startup.plugins.plugin3", PluginStatus.class), isValue(COMPLETE)); assertThat(configStore.get("startup.plugins.plugin4", PluginStatus.class), isValue(COMPLETE)); } From 0de4f33b2881f8dcee4d79fd1e1b894c6663caa8 Mon Sep 17 00:00:00 2001 From: Kyle Vosper Date: Tue, 26 Mar 2019 16:05:57 +0000 Subject: [PATCH 5/7] Remove no-longer-used method --- .../extension/service/spi/StyxService.java | 47 ------------------- 1 file changed, 47 deletions(-) diff --git a/components/api/src/main/java/com/hotels/styx/api/extension/service/spi/StyxService.java b/components/api/src/main/java/com/hotels/styx/api/extension/service/spi/StyxService.java index e3084df4a8..3f6e31b562 100644 --- a/components/api/src/main/java/com/hotels/styx/api/extension/service/spi/StyxService.java +++ b/components/api/src/main/java/com/hotels/styx/api/extension/service/spi/StyxService.java @@ -20,7 +20,6 @@ import java.util.Map; import java.util.concurrent.CompletableFuture; -import java.util.function.Consumer; import static java.util.Collections.emptyMap; @@ -75,50 +74,4 @@ public interface StyxService extends StyxLifecycleListener { default Map adminInterfaceHandlers() { return emptyMap(); } - - /** - * Derives a new service interface with added side-effects for errors. - * This could be used for logging, metrics, etc. - * - * @param consumer error consumer - * @return a new service interface - */ - default StyxService doOnError(Consumer consumer) { - StyxService parent = this; - - return new StyxService() { - @Override - public CompletableFuture start() { - return parent.start().exceptionally(throwable -> { - consumer.accept(throwable); - - if (throwable instanceof RuntimeException) { - throw (RuntimeException) throwable; - } - - if (throwable instanceof Error) { - throw (Error) throwable; - } - - throw new RuntimeException(throwable); - }); - } - - @Override - public CompletableFuture stop() { - return parent.stop(); - } - - @Override - public Map adminInterfaceHandlers() { - return parent.adminInterfaceHandlers(); - } - - @Override - public String toString() { - return parent.toString(); - } - - }; - } } From 04f08f0aa46bd1bf89ac7e3643d3459f4b2e2d2a Mon Sep 17 00:00:00 2001 From: Kyle Vosper Date: Tue, 26 Mar 2019 17:36:18 +0000 Subject: [PATCH 6/7] Remove SequenceProcessor --- .../hotels/styx/common/SequenceProcessor.java | 179 ------------------ .../styx/common/SequenceProcessorTest.java | 76 -------- .../styx/startup/PluginStartupService.java | 43 ++--- .../extensions/PluginLoadingForStartup.java | 73 +++---- 4 files changed, 61 insertions(+), 310 deletions(-) delete mode 100644 components/common/src/main/java/com/hotels/styx/common/SequenceProcessor.java delete mode 100644 components/common/src/test/java/com/hotels/styx/common/SequenceProcessorTest.java diff --git a/components/common/src/main/java/com/hotels/styx/common/SequenceProcessor.java b/components/common/src/main/java/com/hotels/styx/common/SequenceProcessor.java deleted file mode 100644 index 905d8d9e01..0000000000 --- a/components/common/src/main/java/com/hotels/styx/common/SequenceProcessor.java +++ /dev/null @@ -1,179 +0,0 @@ -/* - Copyright (C) 2013-2019 Expedia Inc. - - 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.hotels.styx.common; - -import reactor.core.publisher.Flux; - -import java.util.List; -import java.util.Map; -import java.util.function.BiConsumer; -import java.util.function.Consumer; -import java.util.function.Function; - -import static java.util.Objects.requireNonNull; -import static java.util.stream.Collectors.toList; -import static java.util.stream.Collectors.toMap; - -/** - * Provides the ability to easily apply side effects and failure handling to the processing of a sequence of inputs - * without cluttering code with nested loops, try-catch and if-else statements. - * - * @param input type - * @param output type - */ -public class SequenceProcessor { - private final Flux> flux; - private final Consumer> failuresPostProcessing; - - private SequenceProcessor(Flux> flux, Consumer> failuresPostProcessing) { - this.flux = requireNonNull(flux); - this.failuresPostProcessing = requireNonNull(failuresPostProcessing); - } - - /** - * Create a SequenceProcessor, handling a list of inputs. - * If no mapping is added, all inputs will be returned as outputs with no failures. - * - * @param inputs inputs - * @param input type - * @return a new SequenceProcessor - */ - public static SequenceProcessor processSequence(List inputs) { - return new SequenceProcessor<>( - Flux.fromIterable(inputs).map(element -> new Attempt<>(element, element)), - any -> { - // do nothing - }); - } - - /** - * Maps a function onto the SequenceProcessor so that it will be performed on every input that goes through. - * It will run on every input even if the function throws exceptions - these can be handled later with the - * {@link #onEachFailure(BiConsumer)} and {@link #failuresPostProcessing(Consumer)} methods. - * - * @param function function - * @param output type - * @return transformed SequenceProcessor - */ - public SequenceProcessor map(Function function) { - Flux> mapped = this.flux.map(item -> { - if (item.failed()) { - return (Attempt) item; - } - - try { - return new Attempt<>(item.input, function.apply(item.output)); - } catch (Exception e) { - return new Attempt<>(item.input, e); - } - }); - - return new SequenceProcessor<>(mapped, failuresPostProcessing); - } - - /** - * Adds a side-effect to each successful input and its output. - * - * @param consumer side-effect - * @return transformed SequenceProcessor - */ - public SequenceProcessor onEachSuccess(BiConsumer consumer) { - return new SequenceProcessor<>(flux.doOnNext(attempt -> { - if (attempt.success()) { - consumer.accept(attempt.input, attempt.output); - } - }), failuresPostProcessing); - } - - /** - * Adds a side-effect to each failed input and its exception. - * This can be used to log, collect metrics, or fail-fast by rethrowing the exception (or throwing a new exception). - * - * @param consumer side-effect - * @return transformed SequenceProcessor - */ - public SequenceProcessor onEachFailure(BiConsumer consumer) { - return new SequenceProcessor<>(flux.doOnNext(attempt -> { - if (attempt.failed()) { - consumer.accept(attempt.input, attempt.exception); - } - }), failuresPostProcessing); - } - - /** - * Adds behaviour to execute on failed input and its exception, after all processing is finished. - * It is only executed if at least one input resulted in an exception. - *

- * This is useful for when you choose not to fail-fast on individual inputs. - * By throwing an exception in the provided lambda, you can fail instead of allowing the outputs to be collected. - * - * @param failuresPostProcessing behaviour to execute - * @return transformed SequenceProcessor - */ - public SequenceProcessor failuresPostProcessing(Consumer> failuresPostProcessing) { - return new SequenceProcessor<>(flux, failuresPostProcessing); - } - - /** - * Collect up all outputs from successful inputs. - * - * @return list of outputs - */ - public List collect() { - List> attempts = flux.collectList().block(); - - Map failures = attempts.stream() - .filter(Attempt::failed) - .collect(toMap( - attempt -> attempt.input, - attempt -> attempt.exception - )); - - if (!failures.isEmpty()) { - failuresPostProcessing.accept(failures); - } - - return attempts.stream().filter(Attempt::success) - .map(attempt -> attempt.output) - .collect(toList()); - } - - private static final class Attempt { - private final T input; - private final E output; - private final Exception exception; - - private Attempt(T input, E output) { - this.input = requireNonNull(input); - this.output = output; - this.exception = null; - } - - private Attempt(T input, Exception exception) { - this.input = requireNonNull(input); - this.output = null; - this.exception = requireNonNull(exception); - } - - private boolean failed() { - return exception != null; - } - - private boolean success() { - return !failed(); - } - } -} diff --git a/components/common/src/test/java/com/hotels/styx/common/SequenceProcessorTest.java b/components/common/src/test/java/com/hotels/styx/common/SequenceProcessorTest.java deleted file mode 100644 index 1f392e805c..0000000000 --- a/components/common/src/test/java/com/hotels/styx/common/SequenceProcessorTest.java +++ /dev/null @@ -1,76 +0,0 @@ -/* - Copyright (C) 2013-2019 Expedia Inc. - - 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.hotels.styx.common; - -import org.testng.annotations.Test; - -import java.util.List; -import java.util.Map; -import java.util.concurrent.atomic.AtomicReference; -import java.util.function.BiConsumer; - -import static java.util.Arrays.asList; -import static org.hamcrest.MatcherAssert.assertThat; -import static org.hamcrest.Matchers.contains; -import static org.hamcrest.Matchers.containsInAnyOrder; -import static org.hamcrest.Matchers.is; -import static org.mockito.Matchers.any; -import static org.mockito.Matchers.eq; -import static org.mockito.Mockito.mock; -import static org.mockito.Mockito.verify; -import static org.mockito.Mockito.verifyNoMoreInteractions; - -public class SequenceProcessorTest { - @Test - public void countsSuccessesAndFailures() { - BiConsumer onSuccess = mock(BiConsumer.class); - BiConsumer onFailure = mock(BiConsumer.class); - - AtomicReference> collectedFailures = new AtomicReference<>(); - - List inputs = asList(1, 2, 3, 4, 5, 6, 7); - - List outputs = SequenceProcessor.processSequence(inputs).map(number -> { - // throw exception if even - - if (number % 2 == 0) { - throw new IllegalStateException(); - } - - return String.valueOf(number); - }) - .onEachSuccess(onSuccess) - .onEachFailure(onFailure) - .failuresPostProcessing(collectedFailures::set) - .collect(); - - assertThat(outputs, contains("1", "3", "5", "7")); - - verify(onSuccess).accept(1, "1"); - verify(onSuccess).accept(3, "3"); - verify(onSuccess).accept(5, "5"); - verify(onSuccess).accept(7, "7"); - verifyNoMoreInteractions(onSuccess); - - verify(onFailure).accept(eq(2), any(IllegalStateException.class)); - verify(onFailure).accept(eq(4), any(IllegalStateException.class)); - verify(onFailure).accept(eq(6), any(IllegalStateException.class)); - verifyNoMoreInteractions(onFailure); - - assertThat(collectedFailures.get().keySet(), containsInAnyOrder(2, 4, 6)); - assertThat(collectedFailures.get().values().stream().allMatch(err -> err instanceof IllegalStateException), is(true)); - } -} \ No newline at end of file diff --git a/components/proxy/src/main/java/com/hotels/styx/startup/PluginStartupService.java b/components/proxy/src/main/java/com/hotels/styx/startup/PluginStartupService.java index 4033924707..898a2cec21 100644 --- a/components/proxy/src/main/java/com/hotels/styx/startup/PluginStartupService.java +++ b/components/proxy/src/main/java/com/hotels/styx/startup/PluginStartupService.java @@ -25,7 +25,6 @@ import java.util.List; import java.util.concurrent.CompletableFuture; -import static com.hotels.styx.common.SequenceProcessor.processSequence; import static com.hotels.styx.startup.extensions.PluginLoadingForStartup.loadPlugins; import static com.hotels.styx.startup.extensions.PluginStatusNotifications.PluginPipelineStatus.ALL_PLUGINS_COMPLETE; import static com.hotels.styx.startup.extensions.PluginStatusNotifications.PluginPipelineStatus.AT_LEAST_ONE_PLUGIN_FAILED; @@ -40,7 +39,6 @@ * Service to start up plugins and add them to the config store. */ public class PluginStartupService extends AbstractStyxService { - public static final String PLUGIN_STATUS_KEY_FORMAT = "startup.plugins.%s"; private static final String PLUGIN_KEY_FORMAT = "plugins.%s"; private static final Logger LOGGER = getLogger(PluginStartupService.class); @@ -70,32 +68,31 @@ protected CompletableFuture startService() { PluginStatusNotifications notifications = new PluginStatusNotifications(environment.configStore()); notifications.notifyPluginPipelineStatus(INCOMPLETE); - processSequence(plugins) - .map(plugin -> { - notifications.notifyPluginStatus(plugin.name(), STARTING); - plugin.styxStarting(); - return null; - }) - - .onEachSuccess((plugin, ignore) -> { - registerPlugin(plugin); - notifications.notifyPluginStatus(plugin.name(), COMPLETE); - }) - - .onEachFailure((plugin, err) -> { - notifications.notifyPluginStatus(plugin.name(), FAILED_WHILE_STARTING); - LOGGER.error("Error starting plugin '{}'", plugin.name(), err); - }) - - .failuresPostProcessing(failures -> { - notifications.notifyPluginPipelineStatus(AT_LEAST_ONE_PLUGIN_FAILED); - throw new StartupException(format("%s plugins failed to start", failures.size())); - }).collect(); + int failureCount = 0; + for (NamedPlugin plugin : plugins) { + try { + notifications.notifyPluginStatus(plugin.name(), STARTING); + plugin.styxStarting(); + + registerPlugin(plugin); + notifications.notifyPluginStatus(plugin.name(), COMPLETE); + } catch (Exception e) { + notifications.notifyPluginStatus(plugin.name(), FAILED_WHILE_STARTING); + LOGGER.error("Error starting plugin '{}'", plugin.name(), e); + failureCount++; + } + } + + if (failureCount > 0) { + notifications.notifyPluginPipelineStatus(AT_LEAST_ONE_PLUGIN_FAILED); + throw new StartupException(format("%s plugins failed to start", failureCount)); + } notifications.notifyPluginPipelineStatus(ALL_PLUGINS_COMPLETE); }); } + private void registerPlugin(NamedPlugin plugin) { environment.configStore().set(format(PLUGIN_KEY_FORMAT, plugin.name()), plugin); } diff --git a/components/proxy/src/main/java/com/hotels/styx/startup/extensions/PluginLoadingForStartup.java b/components/proxy/src/main/java/com/hotels/styx/startup/extensions/PluginLoadingForStartup.java index eebf61651c..61cc16689e 100644 --- a/components/proxy/src/main/java/com/hotels/styx/startup/extensions/PluginLoadingForStartup.java +++ b/components/proxy/src/main/java/com/hotels/styx/startup/extensions/PluginLoadingForStartup.java @@ -28,13 +28,14 @@ import com.hotels.styx.spi.config.SpiExtension; import org.slf4j.Logger; +import java.util.ArrayList; +import java.util.HashMap; import java.util.List; import java.util.Map; import java.util.function.BiFunction; import java.util.function.Function; import static com.hotels.styx.common.MapStream.stream; -import static com.hotels.styx.common.SequenceProcessor.processSequence; import static com.hotels.styx.proxy.plugin.NamedPlugin.namedPlugin; import static com.hotels.styx.startup.extensions.PluginStatusNotifications.PluginPipelineStatus.AT_LEAST_ONE_PLUGIN_FAILED; import static com.hotels.styx.startup.extensions.PluginStatusNotifications.PluginStatus.CONSTRUCTED; @@ -83,24 +84,28 @@ private static List loadFactoriesFromConfig(Environment PluginStatusNotifications notifications = new PluginStatusNotifications(environment.configStore()); - return processSequence(configList) - .map(config -> { - notifications.notifyPluginStatus(config.key(), LOADING_CLASSES); - return loadPluginFactory(config); - }) - - .onEachSuccess((config, factory) -> notifications.notifyPluginStatus(config.key(), LOADED_CLASSES)) + List factories = new ArrayList<>(); + Map, Exception> failures = new HashMap<>(); + + for (Pair config : configList) { + try { + notifications.notifyPluginStatus(config.key(), LOADING_CLASSES); + factories.add(loadPluginFactory(config)); + notifications.notifyPluginStatus(config.key(), LOADED_CLASSES); + } catch (Exception e) { + notifications.notifyPluginStatus(config.key(), FAILED_WHILE_LOADING_CLASSES); + environment.configStore().set("startup.plugins." + config.key(), "failed-while-loading"); + LOGGER.error(format("Could not load plugin: pluginName=%s; factoryClass=%s", config.key(), config.value().factory().factoryClass()), e); + failures.put(config, e); + } + } - .onEachFailure((config, err) -> { - notifications.notifyPluginStatus(config.key(), FAILED_WHILE_LOADING_CLASSES); - environment.configStore().set("startup.plugins." + config.key(), "failed-while-loading"); - LOGGER.error(format("Could not load plugin: pluginName=%s; factoryClass=%s", config.key(), config.value().factory().factoryClass()), err); - }) + if (!failures.isEmpty()) { + notifications.notifyPluginPipelineStatus(AT_LEAST_ONE_PLUGIN_FAILED); + throw new PluginStartupException(afterFailuresErrorMessage(failures, Pair::key)); + } - .failuresPostProcessing(failures -> { - notifications.notifyPluginPipelineStatus(AT_LEAST_ONE_PLUGIN_FAILED); - throw new PluginStartupException(afterFailuresErrorMessage(failures, Pair::key)); - }).collect(); + return factories; } private static ConfiguredPluginFactory loadPluginFactory(Pair pair) { @@ -115,23 +120,27 @@ private static ConfiguredPluginFactory loadPluginFactory(Pair loadPluginsFromFactories(Environment environment, List factories) { PluginStatusNotifications notifications = new PluginStatusNotifications(environment.configStore()); - return processSequence(factories) - .map(factory -> { - notifications.notifyPluginStatus(factory.name(), CONSTRUCTING); - return loadPlugin(environment, factory); - }) - - .onEachSuccess((factory, plugin) -> notifications.notifyPluginStatus(factory.name(), CONSTRUCTED)) + List plugins = new ArrayList<>(); + Map failures = new HashMap<>(); + + for (ConfiguredPluginFactory factory : factories) { + try { + notifications.notifyPluginStatus(factory.name(), CONSTRUCTING); + plugins.add(loadPlugin(environment, factory)); + notifications.notifyPluginStatus(factory.name(), CONSTRUCTED); + } catch (Exception e) { + notifications.notifyPluginStatus(factory.name(), FAILED_WHILE_CONSTRUCTING); + LOGGER.error(format("Could not load plugin: pluginName=%s; factoryClass=%s", factory.name(), factory.pluginFactory().getClass().getName()), e); + failures.put(factory, e); + } + } - .onEachFailure((factory, err) -> { - notifications.notifyPluginStatus(factory.name(), FAILED_WHILE_CONSTRUCTING); - LOGGER.error(format("Could not load plugin: pluginName=%s; factoryClass=%s", factory.name(), factory.pluginFactory().getClass().getName()), err); - }) + if (!failures.isEmpty()) { + notifications.notifyPluginPipelineStatus(AT_LEAST_ONE_PLUGIN_FAILED); + throw new PluginStartupException(afterFailuresErrorMessage(failures, ConfiguredPluginFactory::name)); + } - .failuresPostProcessing(failures -> { - notifications.notifyPluginPipelineStatus(AT_LEAST_ONE_PLUGIN_FAILED); - throw new PluginStartupException(afterFailuresErrorMessage(failures, ConfiguredPluginFactory::name)); - }).collect(); + return plugins; } private static NamedPlugin loadPlugin(Environment environment, ConfiguredPluginFactory factory) { From 46807a4f81a754e8782cd110c33771272fb5b498 Mon Sep 17 00:00:00 2001 From: Kyle Vosper Date: Wed, 27 Mar 2019 12:09:02 +0000 Subject: [PATCH 7/7] Addressed comments --- Makefile | 1 - .../styx/api/extension/service/spi/AbstractStyxService.java | 2 +- .../main/java/com/hotels/styx/admin/AdminServerBuilder.java | 3 ++- 3 files changed, 3 insertions(+), 3 deletions(-) diff --git a/Makefile b/Makefile index 906b65406c..618dec66d6 100644 --- a/Makefile +++ b/Makefile @@ -99,7 +99,6 @@ CONFIG_ROOT := $(STYX_HOME)/conf/env-$(STACK) ## Compile and create styx.zip then unzip into a directory defined by STYX_HOME release-styx: release-no-tests unzip -oq ${STYX_BUILD_ARTIFACT} -d $(dir ${STYX_HOME}) - mv `find $(CURRENT_DIR)/distribution/target/styx -type d -name "styx-*"` ${STYX_HOME} ## Stops running netty-based origins (i.e. the origins started by start-origins) stop-origins: diff --git a/components/api/src/main/java/com/hotels/styx/api/extension/service/spi/AbstractStyxService.java b/components/api/src/main/java/com/hotels/styx/api/extension/service/spi/AbstractStyxService.java index 55627614cc..84c37c152d 100644 --- a/components/api/src/main/java/com/hotels/styx/api/extension/service/spi/AbstractStyxService.java +++ b/components/api/src/main/java/com/hotels/styx/api/extension/service/spi/AbstractStyxService.java @@ -120,6 +120,6 @@ public String serviceName() { @Override public String toString() { - return "service:" + serviceName(); + return getClass().getName() + "{name=" + serviceName() + ",status=" + status.get() + "}"; } } diff --git a/components/proxy/src/main/java/com/hotels/styx/admin/AdminServerBuilder.java b/components/proxy/src/main/java/com/hotels/styx/admin/AdminServerBuilder.java index 8cbef8ce91..8a44a6b83a 100644 --- a/components/proxy/src/main/java/com/hotels/styx/admin/AdminServerBuilder.java +++ b/components/proxy/src/main/java/com/hotels/styx/admin/AdminServerBuilder.java @@ -165,7 +165,8 @@ private static Iterable indexLinkPaths() { link("JVM", "/admin/jvm?pretty"), link("Origins Status", "/admin/origins/status?pretty"), link("Dashboard", "/admin/dashboard/index.html"), - link("Plugins", "/admin/plugins")); + link("Plugins", "/admin/plugins"), + link("Startup Status", "/admin/startup")); } private static List routesForPlugin(NamedPlugin namedPlugin) {