From d1b12d7089a62231623a6f6d7bfea0cd95d2a48f Mon Sep 17 00:00:00 2001 From: leventov Date: Tue, 25 Sep 2018 16:39:54 +0300 Subject: [PATCH 01/14] Add SelfDiscoveryResource --- docs/content/operations/api-reference.md | 7 ++ .../server/http/SelfDiscoveryResource.java | 117 ++++++++++++++++++ .../http/security/StateResourceFilter.java | 4 +- .../security/SecurityResourceFilterTest.java | 5 +- .../java/org/apache/druid/cli/CliBroker.java | 7 ++ .../org/apache/druid/cli/CliCoordinator.java | 7 ++ .../org/apache/druid/cli/CliHistorical.java | 7 ++ .../apache/druid/cli/CliMiddleManager.java | 6 + .../org/apache/druid/cli/CliOverlord.java | 10 +- .../java/org/apache/druid/cli/CliPeon.java | 14 +-- .../java/org/apache/druid/cli/CliRouter.java | 6 + 11 files changed, 176 insertions(+), 14 deletions(-) create mode 100644 server/src/main/java/org/apache/druid/server/http/SelfDiscoveryResource.java diff --git a/docs/content/operations/api-reference.md b/docs/content/operations/api-reference.md index 5d3fda344584..374fd1ea0832 100644 --- a/docs/content/operations/api-reference.md +++ b/docs/content/operations/api-reference.md @@ -54,6 +54,13 @@ An endpoint that always returns a boolean "true" value with a 200 OK response, u Returns the current configuration properties of the node. +* `/selfDiscovered` + +Returns a JSON map of the form `{"selfDiscovered": true/false}`, indicating whether the node has recieved a confirmation +from the central node discovery mechanism (currently ZooKeeper) of the Druid cluster that the node has been added to the +cluster. It is recommended to not consider a Druid node "healthy" or "ready" in automated deployment/container +management systems until it returns `{"selfDiscovered": true}` from this endpoint. + ## Coordinator ### Leadership diff --git a/server/src/main/java/org/apache/druid/server/http/SelfDiscoveryResource.java b/server/src/main/java/org/apache/druid/server/http/SelfDiscoveryResource.java new file mode 100644 index 000000000000..0beb3b249866 --- /dev/null +++ b/server/src/main/java/org/apache/druid/server/http/SelfDiscoveryResource.java @@ -0,0 +1,117 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.server.http; + +import com.google.inject.Inject; +import com.google.inject.Singleton; +import com.sun.jersey.spi.container.ResourceFilters; +import org.apache.druid.discovery.DiscoveryDruidNode; +import org.apache.druid.discovery.DruidNodeDiscovery; +import org.apache.druid.discovery.DruidNodeDiscoveryProvider; +import org.apache.druid.discovery.NodeType; +import org.apache.druid.guice.annotations.Self; +import org.apache.druid.java.util.common.lifecycle.Lifecycle; +import org.apache.druid.server.DruidNode; +import org.apache.druid.server.http.security.StateResourceFilter; + +import javax.ws.rs.GET; +import javax.ws.rs.Path; +import javax.ws.rs.Produces; +import javax.ws.rs.core.MediaType; +import javax.ws.rs.core.Response; +import java.util.Collection; +import java.util.Collections; + +/** + * This class is annotated {@link Singleton} rather than {@link org.apache.druid.guice.LazySingleton}, because it adds + * a lifecycle handler in the constructor, that should happen before the lifecycle is started, i. e. eagerly during the + * DI configuration phase. + */ +@Singleton +@Path("/selfDiscovered") +@ResourceFilters(StateResourceFilter.class) +public class SelfDiscoveryResource +{ + private boolean selfDiscovered = false; + + @Inject + public SelfDiscoveryResource( + @Self DruidNode thisDruidNode, + @Self NodeType thisNodeType, + DruidNodeDiscoveryProvider nodeDiscoveryProvider, + Lifecycle lifecycle + ) + { + Lifecycle.Handler selfDiscoveryListenerRegistrator = new Lifecycle.Handler() + { + @Override + public void start() + { + registerSelfDiscoveryListener(thisDruidNode, thisNodeType, nodeDiscoveryProvider); + } + + @Override + public void stop() + { + // do nothing + } + }; + // Using Lifecycle.Stage.LAST because DruidNodeDiscoveryProvider should be already started when + // registerSelfDiscoveryListener() is called. + lifecycle.addHandler(selfDiscoveryListenerRegistrator, Lifecycle.Stage.LAST); + } + + private void registerSelfDiscoveryListener( + DruidNode thisDruidNode, + NodeType thisNodeType, + DruidNodeDiscoveryProvider nodeDiscoveryProvider + ) + { + nodeDiscoveryProvider.getForNodeType(thisNodeType).registerListener(new DruidNodeDiscovery.Listener() + { + @Override + public void nodesAdded(Collection nodes) + { + if (selfDiscovered) { + return; + } + for (DiscoveryDruidNode node : nodes) { + if (node.getDruidNode().equals(thisDruidNode)) { + selfDiscovered = true; + break; + } + } + } + + @Override + public void nodesRemoved(Collection nodes) + { + // do nothing + } + }); + } + + @GET + @Produces(MediaType.APPLICATION_JSON) + public Response getSelfDiscovered() + { + return Response.ok(Collections.singletonMap("selfDiscovered", selfDiscovered)).build(); + } +} diff --git a/server/src/main/java/org/apache/druid/server/http/security/StateResourceFilter.java b/server/src/main/java/org/apache/druid/server/http/security/StateResourceFilter.java index b3231dc2b373..8499f3cc4280 100644 --- a/server/src/main/java/org/apache/druid/server/http/security/StateResourceFilter.java +++ b/server/src/main/java/org/apache/druid/server/http/security/StateResourceFilter.java @@ -47,9 +47,7 @@ public class StateResourceFilter extends AbstractResourceFilter { @Inject - public StateResourceFilter( - AuthorizerMapper authorizerMapper - ) + public StateResourceFilter(AuthorizerMapper authorizerMapper) { super(authorizerMapper); } diff --git a/server/src/test/java/org/apache/druid/server/http/security/SecurityResourceFilterTest.java b/server/src/test/java/org/apache/druid/server/http/security/SecurityResourceFilterTest.java index 0d3be7fce0bd..01889c2c203f 100644 --- a/server/src/test/java/org/apache/druid/server/http/security/SecurityResourceFilterTest.java +++ b/server/src/test/java/org/apache/druid/server/http/security/SecurityResourceFilterTest.java @@ -35,6 +35,7 @@ import org.apache.druid.server.http.IntervalsResource; import org.apache.druid.server.http.MetadataResource; import org.apache.druid.server.http.RulesResource; +import org.apache.druid.server.http.SelfDiscoveryResource; import org.apache.druid.server.http.ServersResource; import org.apache.druid.server.http.TiersResource; import org.apache.druid.server.security.ForbiddenException; @@ -46,13 +47,10 @@ import org.junit.runners.Parameterized; import java.util.Collection; -import java.util.regex.Pattern; @RunWith(Parameterized.class) public class SecurityResourceFilterTest extends ResourceFilterTestHelper { - private static final Pattern WORD = Pattern.compile("\\w+"); - @Parameterized.Parameters public static Collection data() { @@ -71,6 +69,7 @@ public static Collection data() getRequestPathsWithAuthorizer(CoordinatorDynamicConfigsResource.class), getRequestPathsWithAuthorizer(QueryResource.class), getRequestPathsWithAuthorizer(StatusResource.class), + getRequestPathsWithAuthorizer(SelfDiscoveryResource.class), getRequestPathsWithAuthorizer(BrokerQueryResource.class) ) ); diff --git a/services/src/main/java/org/apache/druid/cli/CliBroker.java b/services/src/main/java/org/apache/druid/cli/CliBroker.java index 009a6fedbc54..06eb8e4e05e7 100644 --- a/services/src/main/java/org/apache/druid/cli/CliBroker.java +++ b/services/src/main/java/org/apache/druid/cli/CliBroker.java @@ -44,6 +44,7 @@ import org.apache.druid.guice.LifecycleModule; import org.apache.druid.guice.QueryRunnerFactoryModule; import org.apache.druid.guice.QueryableModule; +import org.apache.druid.guice.annotations.Self; import org.apache.druid.java.util.common.logger.Logger; import org.apache.druid.query.QuerySegmentWalker; import org.apache.druid.query.RetryQueryRunnerConfig; @@ -53,6 +54,7 @@ import org.apache.druid.server.ClientQuerySegmentWalker; import org.apache.druid.server.coordination.broker.DruidBroker; import org.apache.druid.server.http.BrokerResource; +import org.apache.druid.server.http.SelfDiscoveryResource; import org.apache.druid.server.initialization.jetty.JettyServerInitializer; import org.apache.druid.server.metrics.MetricsModule; import org.apache.druid.server.metrics.QueryCountStatsProvider; @@ -125,11 +127,16 @@ protected List getModules() LifecycleModule.register(binder, Server.class); + binder.bind(NodeType.class).annotatedWith(Self.class).toInstance(NodeType.BROKER); + binder .bind(DiscoverySideEffectsProvider.Child.class) .toProvider(new DiscoverySideEffectsProvider(NodeType.BROKER, ImmutableList.of(LookupNodeService.class))) .in(LazySingleton.class); LifecycleModule.registerKey(binder, Key.get(DiscoverySideEffectsProvider.Child.class)); + + Jerseys.addResource(binder, SelfDiscoveryResource.class); + LifecycleModule.registerKey(binder, Key.get(SelfDiscoveryResource.class)); }, new LookupModule(), new SqlModule() diff --git a/services/src/main/java/org/apache/druid/cli/CliCoordinator.java b/services/src/main/java/org/apache/druid/cli/CliCoordinator.java index dd4aa53caa0a..92f4de595e21 100644 --- a/services/src/main/java/org/apache/druid/cli/CliCoordinator.java +++ b/services/src/main/java/org/apache/druid/cli/CliCoordinator.java @@ -46,6 +46,7 @@ import org.apache.druid.guice.ManageLifecycle; import org.apache.druid.guice.annotations.CoordinatorIndexingServiceHelper; import org.apache.druid.guice.annotations.EscalatedGlobal; +import org.apache.druid.guice.annotations.Self; import org.apache.druid.guice.http.JettyHttpClientModule; import org.apache.druid.java.util.common.concurrent.ScheduledExecutorFactory; import org.apache.druid.java.util.common.logger.Logger; @@ -79,6 +80,7 @@ import org.apache.druid.server.http.RedirectFilter; import org.apache.druid.server.http.RedirectInfo; import org.apache.druid.server.http.RulesResource; +import org.apache.druid.server.http.SelfDiscoveryResource; import org.apache.druid.server.http.ServersResource; import org.apache.druid.server.http.TiersResource; import org.apache.druid.server.initialization.ZkPathsConfig; @@ -217,12 +219,17 @@ public void configure(Binder binder) DruidCoordinatorCleanupPendingSegments.class ); + binder.bind(NodeType.class).annotatedWith(Self.class).toInstance(NodeType.COORDINATOR); + binder .bind(DiscoverySideEffectsProvider.Child.class) .annotatedWith(Coordinator.class) .toProvider(new DiscoverySideEffectsProvider(NodeType.COORDINATOR, ImmutableList.of())) .in(LazySingleton.class); LifecycleModule.registerKey(binder, Key.get(DiscoverySideEffectsProvider.Child.class, Coordinator.class)); + + Jerseys.addResource(binder, SelfDiscoveryResource.class); + LifecycleModule.registerKey(binder, Key.get(SelfDiscoveryResource.class)); } @Provides diff --git a/services/src/main/java/org/apache/druid/cli/CliHistorical.java b/services/src/main/java/org/apache/druid/cli/CliHistorical.java index 7d3c7e8767b3..4443ff99b25c 100644 --- a/services/src/main/java/org/apache/druid/cli/CliHistorical.java +++ b/services/src/main/java/org/apache/druid/cli/CliHistorical.java @@ -39,6 +39,7 @@ import org.apache.druid.guice.NodeTypeConfig; import org.apache.druid.guice.QueryRunnerFactoryModule; import org.apache.druid.guice.QueryableModule; +import org.apache.druid.guice.annotations.Self; import org.apache.druid.java.util.common.logger.Logger; import org.apache.druid.query.QuerySegmentWalker; import org.apache.druid.query.lookup.LookupModule; @@ -49,6 +50,7 @@ import org.apache.druid.server.coordination.ZkCoordinator; import org.apache.druid.server.http.HistoricalResource; import org.apache.druid.server.http.SegmentListerResource; +import org.apache.druid.server.http.SelfDiscoveryResource; import org.apache.druid.server.initialization.jetty.JettyServerInitializer; import org.apache.druid.server.metrics.MetricsModule; import org.apache.druid.server.metrics.QueryCountStatsProvider; @@ -103,6 +105,8 @@ protected List getModules() binder.install(new CacheModule()); MetricsModule.register(binder, CacheMonitor.class); + binder.bind(NodeType.class).annotatedWith(Self.class).toInstance(NodeType.HISTORICAL); + binder .bind(DiscoverySideEffectsProvider.Child.class) .toProvider( @@ -113,6 +117,9 @@ protected List getModules() ) .in(LazySingleton.class); LifecycleModule.registerKey(binder, Key.get(DiscoverySideEffectsProvider.Child.class)); + + Jerseys.addResource(binder, SelfDiscoveryResource.class); + LifecycleModule.registerKey(binder, Key.get(SelfDiscoveryResource.class)); }, new LookupModule() ); diff --git a/services/src/main/java/org/apache/druid/cli/CliMiddleManager.java b/services/src/main/java/org/apache/druid/cli/CliMiddleManager.java index e266233f8264..b380717e4870 100644 --- a/services/src/main/java/org/apache/druid/cli/CliMiddleManager.java +++ b/services/src/main/java/org/apache/druid/cli/CliMiddleManager.java @@ -58,6 +58,7 @@ import org.apache.druid.java.util.common.logger.Logger; import org.apache.druid.segment.realtime.firehose.ChatHandlerProvider; import org.apache.druid.server.DruidNode; +import org.apache.druid.server.http.SelfDiscoveryResource; import org.apache.druid.server.initialization.jetty.JettyServerInitializer; import org.eclipse.jetty.server.Server; @@ -130,6 +131,8 @@ public void configure(Binder binder) LifecycleModule.register(binder, Server.class); + binder.bind(NodeType.class).annotatedWith(Self.class).toInstance(NodeType.MIDDLE_MANAGER); + binder .bind(DiscoverySideEffectsProvider.Child.class) .toProvider( @@ -137,6 +140,9 @@ public void configure(Binder binder) ) .in(LazySingleton.class); LifecycleModule.registerKey(binder, Key.get(DiscoverySideEffectsProvider.Child.class)); + + Jerseys.addResource(binder, SelfDiscoveryResource.class); + LifecycleModule.registerKey(binder, Key.get(SelfDiscoveryResource.class)); } @Provides diff --git a/services/src/main/java/org/apache/druid/cli/CliOverlord.java b/services/src/main/java/org/apache/druid/cli/CliOverlord.java index 038ad837c4d6..b40f7bacc0b9 100644 --- a/services/src/main/java/org/apache/druid/cli/CliOverlord.java +++ b/services/src/main/java/org/apache/druid/cli/CliOverlord.java @@ -51,6 +51,7 @@ import org.apache.druid.guice.ManageLifecycle; import org.apache.druid.guice.PolyBind; import org.apache.druid.guice.annotations.Json; +import org.apache.druid.guice.annotations.Self; import org.apache.druid.indexing.common.actions.LocalTaskActionClientFactory; import org.apache.druid.indexing.common.actions.TaskActionClientFactory; import org.apache.druid.indexing.common.actions.TaskActionToolbox; @@ -97,6 +98,7 @@ import org.apache.druid.server.coordinator.CoordinatorOverlordServiceConfig; import org.apache.druid.server.http.RedirectFilter; import org.apache.druid.server.http.RedirectInfo; +import org.apache.druid.server.http.SelfDiscoveryResource; import org.apache.druid.server.initialization.ServerConfig; import org.apache.druid.server.initialization.jetty.JettyServerInitUtils; import org.apache.druid.server.initialization.jetty.JettyServerInitializer; @@ -235,12 +237,18 @@ public void configure(Binder binder) LifecycleModule.register(binder, Server.class); } + binder.bind(NodeType.class).annotatedWith(Self.class).toInstance(NodeType.OVERLORD); + binder .bind(DiscoverySideEffectsProvider.Child.class) .annotatedWith(IndexingService.class) .toProvider(new DiscoverySideEffectsProvider(NodeType.OVERLORD, ImmutableList.of())) .in(LazySingleton.class); - LifecycleModule.registerKey(binder, Key.get(DiscoverySideEffectsProvider.Child.class, IndexingService.class)); + LifecycleModule + .registerKey(binder, Key.get(DiscoverySideEffectsProvider.Child.class, IndexingService.class)); + + Jerseys.addResource(binder, SelfDiscoveryResource.class); + LifecycleModule.registerKey(binder, Key.get(SelfDiscoveryResource.class)); } private void configureTaskStorage(Binder binder) diff --git a/services/src/main/java/org/apache/druid/cli/CliPeon.java b/services/src/main/java/org/apache/druid/cli/CliPeon.java index 991429bc57b0..e317920bf940 100644 --- a/services/src/main/java/org/apache/druid/cli/CliPeon.java +++ b/services/src/main/java/org/apache/druid/cli/CliPeon.java @@ -39,6 +39,7 @@ import org.apache.druid.client.coordinator.CoordinatorClient; import org.apache.druid.client.indexing.HttpIndexingServiceClient; import org.apache.druid.client.indexing.IndexingServiceClient; +import org.apache.druid.discovery.NodeType; import org.apache.druid.guice.Binders; import org.apache.druid.guice.CacheModule; import org.apache.druid.guice.DruidProcessingModule; @@ -54,6 +55,7 @@ import org.apache.druid.guice.QueryableModule; import org.apache.druid.guice.QueryablePeonModule; import org.apache.druid.guice.annotations.Json; +import org.apache.druid.guice.annotations.Self; import org.apache.druid.guice.annotations.Smile; import org.apache.druid.indexing.common.RetryPolicyConfig; import org.apache.druid.indexing.common.RetryPolicyFactory; @@ -118,7 +120,8 @@ @Command( name = "peon", description = "Runs a Peon, this is an individual forked \"task\" used as part of the indexing service. " - + "This should rarely, if ever, be used directly. See http://druid.io/docs/latest/design/peons.html for a description" + + "This should rarely, if ever, be used directly. See http://druid.io/docs/latest/design/peons.html" + + " for a description" ) public class CliPeon extends GuiceRunnable { @@ -293,6 +296,8 @@ private void configureTaskActionClient(Binder binder) .addBinding("remote") .to(RemoteTaskActionClientFactory.class) .in(LazySingleton.class); + + binder.bind(NodeType.class).annotatedWith(Self.class).toInstance(NodeType.PEON); } @Provides @@ -330,12 +335,7 @@ public SegmentListerResource getSegmentListerResource( @Nullable BatchDataSegmentAnnouncer announcer ) { - return new SegmentListerResource( - jsonMapper, - smileMapper, - announcer, - null - ); + return new SegmentListerResource(jsonMapper, smileMapper, announcer, null); } }, new QueryablePeonModule(), diff --git a/services/src/main/java/org/apache/druid/cli/CliRouter.java b/services/src/main/java/org/apache/druid/cli/CliRouter.java index 8c0d83e4d938..ad2041e9b56a 100644 --- a/services/src/main/java/org/apache/druid/cli/CliRouter.java +++ b/services/src/main/java/org/apache/druid/cli/CliRouter.java @@ -49,6 +49,7 @@ import org.apache.druid.query.lookup.LookupModule; import org.apache.druid.server.AsyncQueryForwardingServlet; import org.apache.druid.server.http.RouterResource; +import org.apache.druid.server.http.SelfDiscoveryResource; import org.apache.druid.server.initialization.jetty.JettyServerInitializer; import org.apache.druid.server.metrics.QueryCountStatsProvider; import org.apache.druid.server.router.AvaticaConnectionBalancer; @@ -121,11 +122,16 @@ public void configure(Binder binder) LifecycleModule.register(binder, Server.class); DiscoveryModule.register(binder, Self.class); + binder.bind(NodeType.class).annotatedWith(Self.class).toInstance(NodeType.ROUTER); + binder .bind(DiscoverySideEffectsProvider.Child.class) .toProvider(new DiscoverySideEffectsProvider(NodeType.ROUTER, ImmutableList.of())) .in(LazySingleton.class); LifecycleModule.registerKey(binder, Key.get(DiscoverySideEffectsProvider.Child.class)); + + Jerseys.addResource(binder, SelfDiscoveryResource.class); + LifecycleModule.registerKey(binder, Key.get(SelfDiscoveryResource.class)); } @Provides From 2239684b5e1ce2301e833e6b84012d0cee7b5fd2 Mon Sep 17 00:00:00 2001 From: Roman Leventov Date: Mon, 15 Apr 2019 15:09:12 +0200 Subject: [PATCH 02/14] Rename org.apache.druid.discovery.NodeType to NodeRole. Refactor CuratorDruidNodeDiscoveryProvider. Make SelfDiscoveryResource to listen to updates only about a single node (itself). --- .../apache/druid/guice/LifecycleModule.java | 13 +- .../apache/druid/guice/ManageLifecycle.java | 6 +- .../guice/ManageLifecycleAnnouncements.java | 6 +- .../druid/guice/ManageLifecycleInit.java | 6 +- .../druid/guice/ManageLifecycleServer.java | 6 +- .../druid/java/util/common/io/Closer.java | 8 + .../java/util/common/lifecycle/Lifecycle.java | 46 ++- .../security/basic/CommonCacheNotifier.java | 22 +- .../kafka/LegacyKafkaIndexTaskRunner.java | 4 +- .../AppenderatorDriverRealtimeIndexTask.java | 4 +- .../common/task/RealtimeIndexTask.java | 4 +- .../overlord/hrtr/HttpRemoteTaskRunner.java | 4 +- .../SeekableStreamIndexTaskRunner.java | 4 +- .../hrtr/HttpRemoteTaskRunnerTest.java | 36 +- .../discovery/CuratorDruidNodeAnnouncer.java | 21 +- .../CuratorDruidNodeDiscoveryProvider.java | 364 ++++++++++++------ .../curator/discovery/DiscoveryModule.java | 8 +- .../inventory/CuratorInventoryManager.java | 36 +- .../druid/discovery/DiscoveryDruidNode.java | 42 +- .../druid/discovery/DruidLeaderClient.java | 8 +- .../druid/discovery/DruidNodeDiscovery.java | 8 +- .../discovery/DruidNodeDiscoveryProvider.java | 46 +-- .../{NodeType.java => NodeRole.java} | 15 +- .../guice/CoordinatorDiscoveryModule.java | 4 +- .../guice/IndexingServiceDiscoveryModule.java | 4 +- .../druid/server/coordination/ServerType.java | 6 +- .../druid/server/http/ClusterResource.java | 34 +- .../server/http/SelfDiscoveryResource.java | 54 +-- .../router/TieredBrokerHostSelector.java | 4 +- .../client/HttpServerInventoryViewTest.java | 4 +- ...torDruidNodeAnnouncerAndDiscoveryTest.java | 123 +++--- .../discovery/DruidLeaderClientTest.java | 24 +- .../DruidNodeDiscoveryProviderTest.java | 27 +- .../lookup/cache/LookupNodeDiscoveryTest.java | 8 +- .../router/TieredBrokerHostSelectorTest.java | 10 +- .../java/org/apache/druid/cli/CliBroker.java | 6 +- .../org/apache/druid/cli/CliCoordinator.java | 6 +- .../org/apache/druid/cli/CliHistorical.java | 6 +- .../apache/druid/cli/CliMiddleManager.java | 6 +- .../org/apache/druid/cli/CliOverlord.java | 6 +- .../java/org/apache/druid/cli/CliPeon.java | 4 +- .../java/org/apache/druid/cli/CliRouter.java | 6 +- .../org/apache/druid/cli/ServerRunnable.java | 22 +- .../druid/sql/calcite/util/CalciteTests.java | 4 +- 44 files changed, 598 insertions(+), 487 deletions(-) rename server/src/main/java/org/apache/druid/discovery/{NodeType.java => NodeRole.java} (73%) diff --git a/core/src/main/java/org/apache/druid/guice/LifecycleModule.java b/core/src/main/java/org/apache/druid/guice/LifecycleModule.java index b2ec09bd48af..705ba3fec243 100644 --- a/core/src/main/java/org/apache/druid/guice/LifecycleModule.java +++ b/core/src/main/java/org/apache/druid/guice/LifecycleModule.java @@ -37,10 +37,13 @@ */ public class LifecycleModule implements Module { - // this scope includes final logging shutdown, so all other handlers in this lifecycle scope should avoid logging in - // the 'stop' method, either failing silently or failing violently and throwing an exception causing an ungraceful exit + /** + * This scope includes final logging shutdown, so all other handlers in this lifecycle scope should avoid logging in + * their stop() method, either failing silently or failing violently and throwing an exception causing an ungraceful + * exit. + */ private final LifecycleScope initScope = new LifecycleScope(Lifecycle.Stage.INIT); - private final LifecycleScope scope = new LifecycleScope(Lifecycle.Stage.NORMAL); + private final LifecycleScope normalScope = new LifecycleScope(Lifecycle.Stage.NORMAL); private final LifecycleScope serverScope = new LifecycleScope(Lifecycle.Stage.SERVER); private final LifecycleScope annoucementsScope = new LifecycleScope(Lifecycle.Stage.ANNOUNCEMENTS); @@ -118,7 +121,7 @@ public void configure(Binder binder) getEagerBinder(binder); // Load up the eager binder so that it will inject the empty set at a minimum. binder.bindScope(ManageLifecycleInit.class, initScope); - binder.bindScope(ManageLifecycle.class, scope); + binder.bindScope(ManageLifecycle.class, normalScope); binder.bindScope(ManageLifecycleServer.class, serverScope); binder.bindScope(ManageLifecycleAnnouncements.class, annoucementsScope); } @@ -141,7 +144,7 @@ public void start() throws Exception } }; initScope.setLifecycle(lifecycle); - scope.setLifecycle(lifecycle); + normalScope.setLifecycle(lifecycle); serverScope.setLifecycle(lifecycle); annoucementsScope.setLifecycle(lifecycle); diff --git a/core/src/main/java/org/apache/druid/guice/ManageLifecycle.java b/core/src/main/java/org/apache/druid/guice/ManageLifecycle.java index 0e6790d37d76..2e7fd7e82f5b 100644 --- a/core/src/main/java/org/apache/druid/guice/ManageLifecycle.java +++ b/core/src/main/java/org/apache/druid/guice/ManageLifecycle.java @@ -28,9 +28,9 @@ import java.lang.annotation.Target; /** - * Marks the object to be managed by {@link org.apache.druid.java.util.common.lifecycle.Lifecycle} - * - * This Scope gets defined by {@link LifecycleModule} + * Marks the object to be managed by {@link org.apache.druid.java.util.common.lifecycle.Lifecycle} and set to be on + * {@link org.apache.druid.java.util.common.lifecycle.Lifecycle.Stage#NORMAL} stage. This stage gets defined by {@link + * LifecycleModule}. */ @Target({ ElementType.TYPE, ElementType.METHOD }) @Retention(RetentionPolicy.RUNTIME) diff --git a/core/src/main/java/org/apache/druid/guice/ManageLifecycleAnnouncements.java b/core/src/main/java/org/apache/druid/guice/ManageLifecycleAnnouncements.java index f9537bc27de2..0f193d10f5b5 100644 --- a/core/src/main/java/org/apache/druid/guice/ManageLifecycleAnnouncements.java +++ b/core/src/main/java/org/apache/druid/guice/ManageLifecycleAnnouncements.java @@ -28,9 +28,9 @@ import java.lang.annotation.Target; /** - * Marks the object to be managed by {@link org.apache.druid.java.util.common.lifecycle.Lifecycle} and set to be on Stage.ANNOUNCEMENTS - * - * This Scope gets defined by {@link LifecycleModule} + * Marks the object to be managed by {@link org.apache.druid.java.util.common.lifecycle.Lifecycle} and set to be on + * {@link org.apache.druid.java.util.common.lifecycle.Lifecycle.Stage#ANNOUNCEMENTS} stage. This stage gets defined by + * {@link LifecycleModule}. */ @Target({ ElementType.TYPE, ElementType.METHOD }) @Retention(RetentionPolicy.RUNTIME) diff --git a/core/src/main/java/org/apache/druid/guice/ManageLifecycleInit.java b/core/src/main/java/org/apache/druid/guice/ManageLifecycleInit.java index f8a3750207db..c3c098a53bf4 100644 --- a/core/src/main/java/org/apache/druid/guice/ManageLifecycleInit.java +++ b/core/src/main/java/org/apache/druid/guice/ManageLifecycleInit.java @@ -28,9 +28,9 @@ import java.lang.annotation.Target; /** - * Marks the object to be managed by {@link org.apache.druid.java.util.common.lifecycle.Lifecycle} and set to be on Stage.INIT - * - * This Scope gets defined by {@link LifecycleModule} + * Marks the object to be managed by {@link org.apache.druid.java.util.common.lifecycle.Lifecycle} and set to be on + * {@link org.apache.druid.java.util.common.lifecycle.Lifecycle.Stage#INIT} stage. This stage gets defined by {@link + * LifecycleModule}. */ @Target({ ElementType.TYPE, ElementType.METHOD }) @Retention(RetentionPolicy.RUNTIME) diff --git a/core/src/main/java/org/apache/druid/guice/ManageLifecycleServer.java b/core/src/main/java/org/apache/druid/guice/ManageLifecycleServer.java index f17b49ac4b8a..7f9c93ff205b 100644 --- a/core/src/main/java/org/apache/druid/guice/ManageLifecycleServer.java +++ b/core/src/main/java/org/apache/druid/guice/ManageLifecycleServer.java @@ -28,9 +28,9 @@ import java.lang.annotation.Target; /** - * Marks the object to be managed by {@link org.apache.druid.java.util.common.lifecycle.Lifecycle} and set to be on Stage.SERVER - * - * This Scope gets defined by {@link LifecycleModule} + * Marks the object to be managed by {@link org.apache.druid.java.util.common.lifecycle.Lifecycle} and set to be on + * {@link org.apache.druid.java.util.common.lifecycle.Lifecycle.Stage#SERVER} stage. This stage gets defined by {@link + * LifecycleModule}. */ @Target({ElementType.TYPE, ElementType.METHOD }) @Retention(RetentionPolicy.RUNTIME) diff --git a/core/src/main/java/org/apache/druid/java/util/common/io/Closer.java b/core/src/main/java/org/apache/druid/java/util/common/io/Closer.java index 5bba0dbd1877..4ac567e8a8b2 100644 --- a/core/src/main/java/org/apache/druid/java/util/common/io/Closer.java +++ b/core/src/main/java/org/apache/druid/java/util/common/io/Closer.java @@ -26,6 +26,7 @@ import java.io.Closeable; import java.io.IOException; import java.util.ArrayDeque; +import java.util.Collection; import java.util.Deque; /** @@ -108,6 +109,13 @@ private Closer() { } + public void registerAll(Collection closeables) + { + for (C closeable : closeables) { + register(closeable); + } + } + /** * Registers the given {@code closeable} to be closed when this {@code Closer} is * {@linkplain #close closed}. diff --git a/core/src/main/java/org/apache/druid/java/util/common/lifecycle/Lifecycle.java b/core/src/main/java/org/apache/druid/java/util/common/lifecycle/Lifecycle.java index b9e5b41c6c2e..867bddf84703 100644 --- a/core/src/main/java/org/apache/druid/java/util/common/lifecycle/Lifecycle.java +++ b/core/src/main/java/org/apache/druid/java/util/common/lifecycle/Lifecycle.java @@ -42,27 +42,29 @@ * A manager of object Lifecycles. * * This object has methods for registering objects that should be started and stopped. The Lifecycle allows for - * four stages: Stage.INIT, Stage.NORMAL, Stage.SERVER, and Stage.ANNOUNCEMENTS. + * four stages: {@link Stage#INIT}, {@link Stage#NORMAL}, {@link Stage#SERVER}, and {@link Stage#ANNOUNCEMENTS}. * - * Things added at Stage.INIT will be started first (in the order that they are added to the Lifecycle instance) and - * then things added at Stage.NORMAL, then Stage.SERVER, and finally, Stage.ANNOUNCEMENTS will be started. + * Things added at {@link Stage#INIT} will be started first (in the order that they are added to the Lifecycle instance) + * and then things added at {@link Stage#NORMAL}, then {@link Stage#SERVER}, and finally, {@link Stage#ANNOUNCEMENTS} + * will be started. * - * The close operation goes in reverse order, starting with the last thing added at Stage.ANNOUNCEMENTS and working - * backwards. + * The close operation goes in reverse order, starting with the last thing added at {@link Stage#ANNOUNCEMENTS} and + * working backwards. * * Conceptually, the stages have the following purposes: - * - Stage.INIT: Currently, this stage is used exclusively for log4j initialization, since almost everything needs - * logging and it should be the last thing to shutdown. Any sort of bootstrapping object that provides something that - * should be initialized before nearly all other Lifecycle objects could also belong here (if it doesn't need - * logging during start or stop). - * - Stage.NORMAL: This is the default stage. Most objects will probably make the most sense to be registered at - * this level, with the exception of any form of server or service announcements - * - Stage.SERVER: This lifecycle stage is intended for all 'server' objects, and currently only contains the Jetty - * module, but any sort of 'server' that expects most Lifecycle objects to be initialized by the time it starts, and - * still available at the time it stops can logically live in this stage. - * - Stage.ANNOUNCENTS: Any object which announces to a cluster this servers location belongs in this stage. By being - * last, we can be sure that all servers are initialized before we advertise the endpoint locations, and also can be - * sure that we un-announce these advertisements prior to the Stage.SERVER objects stop. + * - {@link Stage#INIT}: Currently, this stage is used exclusively for log4j initialization, since almost everything + * needs logging and it should be the last thing to shutdown. Any sort of bootstrapping object that provides + * something that should be initialized before nearly all other Lifecycle objects could also belong here (if it + * doesn't need logging during start or stop). + * - {@link Stage#NORMAL}: This is the default stage. Most objects will probably make the most sense to be registered + * at this level, with the exception of any form of server or service announcements + * - {@link Stage#SERVER}: This lifecycle stage is intended for all 'server' objects, for example, {@link + * org.apache.druid.server.initialization.jetty.JettyServerModule}, but any sort of 'server' that expects most (or + * some specific) Lifecycle objects to be initialized by the time it starts, and still available at the time it stops + * can logically live in this stage. + * - {@link Stage#ANNOUNCEMENTS}: Any object which announces to a cluster this servers location belongs in this stage. + * By being last, we can be sure that all servers are initialized before we advertise the endpoint locations, and + * also can be sure that we un-announce these advertisements prior to the Stage.SERVER objects stop. * * There are two sets of methods to add things to the Lifecycle. One set that will just add instances and enforce that * start() has not been called yet. The other set will add instances and, if the lifecycle is already started, start @@ -357,7 +359,7 @@ public void stop() } startStopLock.lock(); try { - RuntimeException thrown = null; + Exception thrown = null; for (Stage s : handlers.navigableKeySet().descendingSet()) { log.info("Stopping lifecycle [%s] stage [%s]", name, s.name()); @@ -365,17 +367,19 @@ public void stop() try { handler.stop(); } - catch (RuntimeException e) { + catch (Exception e) { log.warn(e, "Lifecycle [%s] encountered exception while stopping %s", name, handler); if (thrown == null) { thrown = e; + } else { + thrown.addSuppressed(e); } } } } if (thrown != null) { - throw thrown; + throw new RuntimeException(thrown); } } finally { @@ -412,7 +416,7 @@ public interface Handler { void start() throws Exception; - void stop(); + void stop() throws Exception; } private static class AnnotationBasedHandler implements Handler diff --git a/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/CommonCacheNotifier.java b/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/CommonCacheNotifier.java index f1e61ebfba75..18ca09652a22 100644 --- a/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/CommonCacheNotifier.java +++ b/extensions-core/druid-basic-security/src/main/java/org/apache/druid/security/basic/CommonCacheNotifier.java @@ -24,7 +24,7 @@ import org.apache.druid.discovery.DiscoveryDruidNode; import org.apache.druid.discovery.DruidNodeDiscovery; import org.apache.druid.discovery.DruidNodeDiscoveryProvider; -import org.apache.druid.discovery.NodeType; +import org.apache.druid.discovery.NodeRole; import org.apache.druid.java.util.common.Pair; import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.common.concurrent.Execs; @@ -59,16 +59,16 @@ public class CommonCacheNotifier private static final EmittingLogger LOG = new EmittingLogger(CommonCacheNotifier.class); /** - * {@link NodeType#COORDINATOR} is intentionally omitted because it gets its information about the auth state directly + * {@link NodeRole#COORDINATOR} is intentionally omitted because it gets its information about the auth state directly * from metadata storage. */ - private static final List NODE_TYPES = Arrays.asList( - NodeType.BROKER, - NodeType.OVERLORD, - NodeType.HISTORICAL, - NodeType.PEON, - NodeType.ROUTER, - NodeType.MIDDLE_MANAGER + private static final List NODE_TYPES = Arrays.asList( + NodeRole.BROKER, + NodeRole.OVERLORD, + NodeRole.HISTORICAL, + NodeRole.PEON, + NodeRole.ROUTER, + NodeRole.MIDDLE_MANAGER ); private final DruidNodeDiscoveryProvider discoveryProvider; @@ -159,8 +159,8 @@ public void addUpdate(String updatedItemName, byte[] updatedItemData) private List> sendUpdate(String updatedAuthorizerPrefix, byte[] serializedUserMap) { List> futures = new ArrayList<>(); - for (NodeType nodeType : NODE_TYPES) { - DruidNodeDiscovery nodeDiscovery = discoveryProvider.getForNodeType(nodeType); + for (NodeRole nodeRole : NODE_TYPES) { + DruidNodeDiscovery nodeDiscovery = discoveryProvider.getForNodeRole(nodeRole); Collection nodes = nodeDiscovery.getAllNodes(); for (DiscoveryDruidNode node : nodes) { URL listenerURL = getListenerURL(node.getDruidNode(), baseUrl, updatedAuthorizerPrefix); diff --git a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/LegacyKafkaIndexTaskRunner.java b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/LegacyKafkaIndexTaskRunner.java index 5b186c44ca0c..bd173e0a6ccc 100644 --- a/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/LegacyKafkaIndexTaskRunner.java +++ b/extensions-core/kafka-indexing-service/src/main/java/org/apache/druid/indexing/kafka/LegacyKafkaIndexTaskRunner.java @@ -33,7 +33,7 @@ import org.apache.druid.data.input.impl.InputRowParser; import org.apache.druid.discovery.DiscoveryDruidNode; import org.apache.druid.discovery.LookupNodeService; -import org.apache.druid.discovery.NodeType; +import org.apache.druid.discovery.NodeRole; import org.apache.druid.indexer.IngestionState; import org.apache.druid.indexer.TaskStatus; import org.apache.druid.indexing.common.IngestionStatsAndErrorsTaskReport; @@ -277,7 +277,7 @@ private TaskStatus runInternal(TaskToolbox toolbox) throws Exception new LookupNodeService(lookupTier); DiscoveryDruidNode discoveryDruidNode = new DiscoveryDruidNode( toolbox.getDruidNode(), - NodeType.PEON, + NodeRole.PEON, ImmutableMap.of( toolbox.getDataNodeService().getName(), toolbox.getDataNodeService(), lookupNodeService.getName(), lookupNodeService diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/AppenderatorDriverRealtimeIndexTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/AppenderatorDriverRealtimeIndexTask.java index 19a256883edd..7625f523d364 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/AppenderatorDriverRealtimeIndexTask.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/AppenderatorDriverRealtimeIndexTask.java @@ -37,7 +37,7 @@ import org.apache.druid.data.input.InputRow; import org.apache.druid.discovery.DiscoveryDruidNode; import org.apache.druid.discovery.LookupNodeService; -import org.apache.druid.discovery.NodeType; +import org.apache.druid.discovery.NodeRole; import org.apache.druid.indexer.IngestionState; import org.apache.druid.indexer.TaskStatus; import org.apache.druid.indexing.appenderator.ActionBasedSegmentAllocator; @@ -674,7 +674,7 @@ private DiscoveryDruidNode createDiscoveryDruidNode(TaskToolbox toolbox) new LookupNodeService(getContextValue(CTX_KEY_LOOKUP_TIER)); return new DiscoveryDruidNode( toolbox.getDruidNode(), - NodeType.PEON, + NodeRole.PEON, ImmutableMap.of( toolbox.getDataNodeService().getName(), toolbox.getDataNodeService(), lookupNodeService.getName(), lookupNodeService diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/RealtimeIndexTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/RealtimeIndexTask.java index 2786805b9f4c..e2be9f084bb5 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/RealtimeIndexTask.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/RealtimeIndexTask.java @@ -32,7 +32,7 @@ import org.apache.druid.data.input.FirehoseFactory; import org.apache.druid.discovery.DiscoveryDruidNode; import org.apache.druid.discovery.LookupNodeService; -import org.apache.druid.discovery.NodeType; +import org.apache.druid.discovery.NodeRole; import org.apache.druid.indexer.TaskStatus; import org.apache.druid.indexing.common.TaskLock; import org.apache.druid.indexing.common.TaskLockType; @@ -363,7 +363,7 @@ public String getVersion(final Interval interval) new LookupNodeService((String) getContextValue(CTX_KEY_LOOKUP_TIER)); DiscoveryDruidNode discoveryDruidNode = new DiscoveryDruidNode( toolbox.getDruidNode(), - NodeType.PEON, + NodeRole.PEON, ImmutableMap.of( toolbox.getDataNodeService().getName(), toolbox.getDataNodeService(), lookupNodeService.getName(), lookupNodeService diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/hrtr/HttpRemoteTaskRunner.java b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/hrtr/HttpRemoteTaskRunner.java index 532b2fc91548..e187494b9212 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/hrtr/HttpRemoteTaskRunner.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/hrtr/HttpRemoteTaskRunner.java @@ -41,7 +41,7 @@ import org.apache.druid.discovery.DiscoveryDruidNode; import org.apache.druid.discovery.DruidNodeDiscovery; import org.apache.druid.discovery.DruidNodeDiscoveryProvider; -import org.apache.druid.discovery.NodeType; +import org.apache.druid.discovery.NodeRole; import org.apache.druid.discovery.WorkerNodeService; import org.apache.druid.indexer.RunnerTaskState; import org.apache.druid.indexer.TaskLocation; @@ -438,7 +438,7 @@ private void taskComplete( private void startWorkersHandling() throws InterruptedException { final CountDownLatch workerViewInitialized = new CountDownLatch(1); - DruidNodeDiscovery druidNodeDiscovery = druidNodeDiscoveryProvider.getForNodeType(NodeType.MIDDLE_MANAGER); + DruidNodeDiscovery druidNodeDiscovery = druidNodeDiscoveryProvider.getForNodeRole(NodeRole.MIDDLE_MANAGER); druidNodeDiscovery.registerListener( new DruidNodeDiscovery.Listener() { diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskRunner.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskRunner.java index c07118a99fd1..4659469e5b24 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskRunner.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/SeekableStreamIndexTaskRunner.java @@ -41,7 +41,7 @@ import org.apache.druid.data.input.impl.InputRowParser; import org.apache.druid.discovery.DiscoveryDruidNode; import org.apache.druid.discovery.LookupNodeService; -import org.apache.druid.discovery.NodeType; +import org.apache.druid.discovery.NodeRole; import org.apache.druid.indexer.IngestionState; import org.apache.druid.indexer.TaskStatus; import org.apache.druid.indexing.common.IngestionStatsAndErrorsTaskReport; @@ -367,7 +367,7 @@ private TaskStatus runInternal(TaskToolbox toolbox) throws Exception final DiscoveryDruidNode discoveryDruidNode = new DiscoveryDruidNode( toolbox.getDruidNode(), - NodeType.PEON, + NodeRole.PEON, ImmutableMap.of( toolbox.getDataNodeService().getName(), toolbox.getDataNodeService(), lookupNodeService.getName(), lookupNodeService diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/hrtr/HttpRemoteTaskRunnerTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/hrtr/HttpRemoteTaskRunnerTest.java index b62a536ad1f0..e53437297a75 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/hrtr/HttpRemoteTaskRunnerTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/hrtr/HttpRemoteTaskRunnerTest.java @@ -31,7 +31,7 @@ import org.apache.druid.discovery.DiscoveryDruidNode; import org.apache.druid.discovery.DruidNodeDiscovery; import org.apache.druid.discovery.DruidNodeDiscoveryProvider; -import org.apache.druid.discovery.NodeType; +import org.apache.druid.discovery.NodeRole; import org.apache.druid.discovery.WorkerNodeService; import org.apache.druid.indexer.TaskLocation; import org.apache.druid.indexer.TaskState; @@ -86,7 +86,7 @@ public void testFreshStart() throws Exception { TestDruidNodeDiscovery druidNodeDiscovery = new TestDruidNodeDiscovery(); DruidNodeDiscoveryProvider druidNodeDiscoveryProvider = EasyMock.createMock(DruidNodeDiscoveryProvider.class); - EasyMock.expect(druidNodeDiscoveryProvider.getForNodeType(NodeType.MIDDLE_MANAGER)) + EasyMock.expect(druidNodeDiscoveryProvider.getForNodeRole(NodeRole.MIDDLE_MANAGER)) .andReturn(druidNodeDiscovery); EasyMock.replay(druidNodeDiscoveryProvider); @@ -138,7 +138,7 @@ protected WorkerHolder createWorkerHolder( DiscoveryDruidNode druidNode1 = new DiscoveryDruidNode( new DruidNode("service", "host1", false, 8080, null, true, false), - NodeType.MIDDLE_MANAGER, + NodeRole.MIDDLE_MANAGER, ImmutableMap.of( WorkerNodeService.DISCOVERY_SERVICE_KEY, new WorkerNodeService("ip1", 2, "0") ) @@ -146,7 +146,7 @@ WorkerNodeService.DISCOVERY_SERVICE_KEY, new WorkerNodeService("ip1", 2, "0") DiscoveryDruidNode druidNode2 = new DiscoveryDruidNode( new DruidNode("service", "host2", false, 8080, null, true, false), - NodeType.MIDDLE_MANAGER, + NodeRole.MIDDLE_MANAGER, ImmutableMap.of( WorkerNodeService.DISCOVERY_SERVICE_KEY, new WorkerNodeService("ip2", 2, "0") ) @@ -177,7 +177,7 @@ public void testOneStuckTaskAssignmentDoesntBlockOthers() throws Exception { TestDruidNodeDiscovery druidNodeDiscovery = new TestDruidNodeDiscovery(); DruidNodeDiscoveryProvider druidNodeDiscoveryProvider = EasyMock.createMock(DruidNodeDiscoveryProvider.class); - EasyMock.expect(druidNodeDiscoveryProvider.getForNodeType(NodeType.MIDDLE_MANAGER)) + EasyMock.expect(druidNodeDiscoveryProvider.getForNodeRole(NodeRole.MIDDLE_MANAGER)) .andReturn(druidNodeDiscovery); EasyMock.replay(druidNodeDiscoveryProvider); @@ -233,7 +233,7 @@ protected WorkerHolder createWorkerHolder( DiscoveryDruidNode druidNode1 = new DiscoveryDruidNode( new DruidNode("service", "host1", false, 8080, null, true, false), - NodeType.MIDDLE_MANAGER, + NodeRole.MIDDLE_MANAGER, ImmutableMap.of( WorkerNodeService.DISCOVERY_SERVICE_KEY, new WorkerNodeService("ip1", 2, "0") ) @@ -241,7 +241,7 @@ WorkerNodeService.DISCOVERY_SERVICE_KEY, new WorkerNodeService("ip1", 2, "0") DiscoveryDruidNode druidNode2 = new DiscoveryDruidNode( new DruidNode("service", "host2", false, 8080, null, true, false), - NodeType.MIDDLE_MANAGER, + NodeRole.MIDDLE_MANAGER, ImmutableMap.of( WorkerNodeService.DISCOVERY_SERVICE_KEY, new WorkerNodeService("ip2", 2, "0") ) @@ -267,7 +267,7 @@ public void testTaskRunnerRestart() throws Exception { TestDruidNodeDiscovery druidNodeDiscovery = new TestDruidNodeDiscovery(); DruidNodeDiscoveryProvider druidNodeDiscoveryProvider = EasyMock.createMock(DruidNodeDiscoveryProvider.class); - EasyMock.expect(druidNodeDiscoveryProvider.getForNodeType(NodeType.MIDDLE_MANAGER)) + EasyMock.expect(druidNodeDiscoveryProvider.getForNodeRole(NodeRole.MIDDLE_MANAGER)) .andReturn(druidNodeDiscovery); EasyMock.replay(druidNodeDiscoveryProvider); @@ -335,7 +335,7 @@ protected WorkerHolder createWorkerHolder( DiscoveryDruidNode druidNode = new DiscoveryDruidNode( new DruidNode("service", "host", false, 1234, null, true, false), - NodeType.MIDDLE_MANAGER, + NodeRole.MIDDLE_MANAGER, ImmutableMap.of( WorkerNodeService.DISCOVERY_SERVICE_KEY, new WorkerNodeService("ip1", 2, "0") ) @@ -420,7 +420,7 @@ public void testWorkerDisapperAndReappearBeforeItsCleanup() throws Exception { TestDruidNodeDiscovery druidNodeDiscovery = new TestDruidNodeDiscovery(); DruidNodeDiscoveryProvider druidNodeDiscoveryProvider = EasyMock.createMock(DruidNodeDiscoveryProvider.class); - EasyMock.expect(druidNodeDiscoveryProvider.getForNodeType(NodeType.MIDDLE_MANAGER)) + EasyMock.expect(druidNodeDiscoveryProvider.getForNodeRole(NodeRole.MIDDLE_MANAGER)) .andReturn(druidNodeDiscovery); EasyMock.replay(druidNodeDiscoveryProvider); @@ -477,7 +477,7 @@ protected WorkerHolder createWorkerHolder( DiscoveryDruidNode druidNode = new DiscoveryDruidNode( new DruidNode("service", "host", false, 1234, null, true, false), - NodeType.MIDDLE_MANAGER, + NodeRole.MIDDLE_MANAGER, ImmutableMap.of( WorkerNodeService.DISCOVERY_SERVICE_KEY, new WorkerNodeService("ip1", 2, "0") ) @@ -596,7 +596,7 @@ public void testWorkerDisapperAndReappearAfterItsCleanup() throws Exception { TestDruidNodeDiscovery druidNodeDiscovery = new TestDruidNodeDiscovery(); DruidNodeDiscoveryProvider druidNodeDiscoveryProvider = EasyMock.createMock(DruidNodeDiscoveryProvider.class); - EasyMock.expect(druidNodeDiscoveryProvider.getForNodeType(NodeType.MIDDLE_MANAGER)) + EasyMock.expect(druidNodeDiscoveryProvider.getForNodeRole(NodeRole.MIDDLE_MANAGER)) .andReturn(druidNodeDiscovery); EasyMock.replay(druidNodeDiscoveryProvider); @@ -653,7 +653,7 @@ protected WorkerHolder createWorkerHolder( DiscoveryDruidNode druidNode = new DiscoveryDruidNode( new DruidNode("service", "host", false, 1234, null, true, false), - NodeType.MIDDLE_MANAGER, + NodeRole.MIDDLE_MANAGER, ImmutableMap.of(WorkerNodeService.DISCOVERY_SERVICE_KEY, new WorkerNodeService("ip1", 2, "0")) ); @@ -770,7 +770,7 @@ public void testMarkWorkersLazy() throws Exception { TestDruidNodeDiscovery druidNodeDiscovery = new TestDruidNodeDiscovery(); DruidNodeDiscoveryProvider druidNodeDiscoveryProvider = EasyMock.createMock(DruidNodeDiscoveryProvider.class); - EasyMock.expect(druidNodeDiscoveryProvider.getForNodeType(NodeType.MIDDLE_MANAGER)) + EasyMock.expect(druidNodeDiscoveryProvider.getForNodeRole(NodeRole.MIDDLE_MANAGER)) .andReturn(druidNodeDiscovery); EasyMock.replay(druidNodeDiscoveryProvider); @@ -829,7 +829,7 @@ protected WorkerHolder createWorkerHolder( DiscoveryDruidNode druidNode1 = new DiscoveryDruidNode( new DruidNode("service", "host1", false, 8080, null, true, false), - NodeType.MIDDLE_MANAGER, + NodeRole.MIDDLE_MANAGER, ImmutableMap.of( WorkerNodeService.DISCOVERY_SERVICE_KEY, new WorkerNodeService("ip1", 1, "0") ) @@ -874,7 +874,7 @@ WorkerNodeService.DISCOVERY_SERVICE_KEY, new WorkerNodeService("ip1", 1, "0") DiscoveryDruidNode druidNode2 = new DiscoveryDruidNode( new DruidNode("service", "host2", false, 8080, null, true, false), - NodeType.MIDDLE_MANAGER, + NodeRole.MIDDLE_MANAGER, ImmutableMap.of(WorkerNodeService.DISCOVERY_SERVICE_KEY, new WorkerNodeService("ip2", 1, "0")) ); @@ -904,7 +904,7 @@ WorkerNodeService.DISCOVERY_SERVICE_KEY, new WorkerNodeService("ip1", 1, "0") DiscoveryDruidNode druidNode3 = new DiscoveryDruidNode( new DruidNode("service", "host3", false, 8080, null, true, false), - NodeType.MIDDLE_MANAGER, + NodeRole.MIDDLE_MANAGER, ImmutableMap.of(WorkerNodeService.DISCOVERY_SERVICE_KEY, new WorkerNodeService("ip2", 1, "0")) ); @@ -1205,7 +1205,7 @@ private HttpRemoteTaskRunner createTaskRunnerForTestTaskAddedOrUpdated( { TestDruidNodeDiscovery druidNodeDiscovery = new TestDruidNodeDiscovery(); DruidNodeDiscoveryProvider druidNodeDiscoveryProvider = EasyMock.createMock(DruidNodeDiscoveryProvider.class); - EasyMock.expect(druidNodeDiscoveryProvider.getForNodeType(NodeType.MIDDLE_MANAGER)) + EasyMock.expect(druidNodeDiscoveryProvider.getForNodeRole(NodeRole.MIDDLE_MANAGER)) .andReturn(druidNodeDiscovery); EasyMock.replay(druidNodeDiscoveryProvider); diff --git a/server/src/main/java/org/apache/druid/curator/discovery/CuratorDruidNodeAnnouncer.java b/server/src/main/java/org/apache/druid/curator/discovery/CuratorDruidNodeAnnouncer.java index ab27916907d3..6f0bf629c79f 100644 --- a/server/src/main/java/org/apache/druid/curator/discovery/CuratorDruidNodeAnnouncer.java +++ b/server/src/main/java/org/apache/druid/curator/discovery/CuratorDruidNodeAnnouncer.java @@ -26,14 +26,21 @@ import org.apache.druid.curator.announcement.Announcer; import org.apache.druid.discovery.DiscoveryDruidNode; import org.apache.druid.discovery.DruidNodeAnnouncer; +import org.apache.druid.discovery.NodeRole; import org.apache.druid.guice.annotations.Json; import org.apache.druid.java.util.common.logger.Logger; +import org.apache.druid.server.DruidNode; import org.apache.druid.server.initialization.ZkPathsConfig; /** */ public class CuratorDruidNodeAnnouncer implements DruidNodeAnnouncer { + public static String makeNodeAnnouncementPath(ZkPathsConfig config, NodeRole nodeRole, DruidNode node) + { + return ZKPaths.makePath(config.getInternalDiscoveryPath(), nodeRole.toString(), node.getHostAndPortToUse()); + } + private static final Logger log = new Logger(CuratorDruidNodeAnnouncer.class); private final Announcer announcer; @@ -54,11 +61,8 @@ public void announce(DiscoveryDruidNode discoveryDruidNode) try { log.info("Announcing [%s].", discoveryDruidNode); - String path = ZKPaths.makePath( - config.getInternalDiscoveryPath(), - discoveryDruidNode.getNodeType().toString(), - discoveryDruidNode.getDruidNode().getHostAndPortToUse() - ); + String path = + makeNodeAnnouncementPath(config, discoveryDruidNode.getNodeRole(), discoveryDruidNode.getDruidNode()); announcer.announce(path, jsonMapper.writeValueAsBytes(discoveryDruidNode)); log.info("Announced [%s].", discoveryDruidNode); @@ -73,11 +77,8 @@ public void unannounce(DiscoveryDruidNode discoveryDruidNode) { log.info("Unannouncing [%s].", discoveryDruidNode); - String path = ZKPaths.makePath( - config.getInternalDiscoveryPath(), - discoveryDruidNode.getNodeType().toString(), - discoveryDruidNode.getDruidNode().getHostAndPortToUse() - ); + String path = + makeNodeAnnouncementPath(config, discoveryDruidNode.getNodeRole(), discoveryDruidNode.getDruidNode()); announcer.unannounce(path); log.info("Unannounced [%s].", discoveryDruidNode); diff --git a/server/src/main/java/org/apache/druid/curator/discovery/CuratorDruidNodeDiscoveryProvider.java b/server/src/main/java/org/apache/druid/curator/discovery/CuratorDruidNodeDiscoveryProvider.java index fbe5e61a0972..40ce93c55b57 100644 --- a/server/src/main/java/org/apache/druid/curator/discovery/CuratorDruidNodeDiscoveryProvider.java +++ b/server/src/main/java/org/apache/druid/curator/discovery/CuratorDruidNodeDiscoveryProvider.java @@ -22,36 +22,47 @@ import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.base.Preconditions; import com.google.common.collect.ImmutableList; +import com.google.errorprone.annotations.concurrent.GuardedBy; import com.google.inject.Inject; import org.apache.curator.framework.CuratorFramework; +import org.apache.curator.framework.recipes.cache.ChildData; +import org.apache.curator.framework.recipes.cache.NodeCache; import org.apache.curator.framework.recipes.cache.PathChildrenCache; import org.apache.curator.framework.recipes.cache.PathChildrenCacheEvent; import org.apache.curator.utils.ZKPaths; import org.apache.druid.concurrent.LifecycleLock; +import org.apache.druid.curator.cache.PathChildrenCacheFactory; import org.apache.druid.discovery.DiscoveryDruidNode; import org.apache.druid.discovery.DruidNodeDiscovery; import org.apache.druid.discovery.DruidNodeDiscoveryProvider; -import org.apache.druid.discovery.NodeType; +import org.apache.druid.discovery.NodeRole; import org.apache.druid.guice.ManageLifecycle; import org.apache.druid.guice.annotations.Json; import org.apache.druid.java.util.common.ISE; import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.common.concurrent.Execs; +import org.apache.druid.java.util.common.io.Closer; import org.apache.druid.java.util.common.lifecycle.LifecycleStart; import org.apache.druid.java.util.common.lifecycle.LifecycleStop; import org.apache.druid.java.util.common.logger.Logger; +import org.apache.druid.server.DruidNode; import org.apache.druid.server.initialization.ZkPathsConfig; +import org.apache.druid.utils.CloseableUtils; -import javax.annotation.concurrent.GuardedBy; +import javax.annotation.Nullable; +import java.io.Closeable; +import java.io.IOException; import java.util.ArrayList; import java.util.Collection; import java.util.Collections; import java.util.List; import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ConcurrentLinkedQueue; import java.util.concurrent.ConcurrentMap; import java.util.concurrent.CountDownLatch; import java.util.concurrent.ExecutorService; import java.util.concurrent.TimeUnit; +import java.util.function.BooleanSupplier; /** */ @@ -66,7 +77,8 @@ public class CuratorDruidNodeDiscoveryProvider extends DruidNodeDiscoveryProvide private ExecutorService listenerExecutor; - private final ConcurrentHashMap nodeTypeWatchers = new ConcurrentHashMap<>(); + private final ConcurrentHashMap nodeRoleWatchers = new ConcurrentHashMap<>(); + private final ConcurrentLinkedQueue nodeDiscoverers = new ConcurrentLinkedQueue<>(); private final LifecycleLock lifecycleLock = new LifecycleLock(); @@ -83,24 +95,33 @@ public CuratorDruidNodeDiscoveryProvider( } @Override - public DruidNodeDiscovery getForNodeType(NodeType nodeType) + public BooleanSupplier getForNode(DruidNode node, NodeRole nodeRole) { - Preconditions.checkState(lifecycleLock.awaitStarted(1, TimeUnit.MILLISECONDS)); + Preconditions.checkState(lifecycleLock.isStarted()); + log.info("Creating a NodeDiscoverer for node [%s] and role [%s]", node, nodeRole); + NodeDiscoverer nodeDiscoverer = new NodeDiscoverer(config, jsonMapper, curatorFramework, node, nodeRole); + nodeDiscoverers.add(nodeDiscoverer); + return nodeDiscoverer::nodeDiscovered; + } + + @Override + public DruidNodeDiscovery getForNodeRole(NodeRole nodeRole) + { + Preconditions.checkState(lifecycleLock.isStarted()); - return nodeTypeWatchers.computeIfAbsent( - nodeType, - nType -> { - log.info("Creating NodeTypeWatcher for nodeType [%s].", nType); - NodeTypeWatcher nodeTypeWatcher = new NodeTypeWatcher( + return nodeRoleWatchers.computeIfAbsent( + nodeRole, + role -> { + log.info("Creating NodeRoleWatcher for nodeRole [%s].", role); + NodeRoleWatcher nodeRoleWatcher = new NodeRoleWatcher( listenerExecutor, curatorFramework, config.getInternalDiscoveryPath(), jsonMapper, - nType + role ); - nodeTypeWatcher.start(); - log.info("Created NodeTypeWatcher for nodeType [%s].", nType); - return nodeTypeWatcher; + log.info("Created NodeRoleWatcher for nodeRole [%s].", role); + return nodeRoleWatcher; } ); } @@ -129,7 +150,7 @@ public void start() } @LifecycleStop - public void stop() + public void stop() throws IOException { if (!lifecycleLock.canStop()) { throw new ISE("can't stop."); @@ -137,21 +158,22 @@ public void stop() log.info("stopping"); - for (NodeTypeWatcher watcher : nodeTypeWatchers.values()) { - watcher.stop(); - } - listenerExecutor.shutdownNow(); + Closer closer = Closer.create(); + closer.registerAll(nodeRoleWatchers.values()); + closer.registerAll(nodeDiscoverers); + + CloseableUtils.closeBoth(closer, listenerExecutor::shutdownNow); log.info("stopped"); } - private static class NodeTypeWatcher implements DruidNodeDiscovery + private static class NodeRoleWatcher implements DruidNodeDiscovery, Closeable { - private static final Logger log = new Logger(NodeTypeWatcher.class); + private static final Logger log = new Logger(NodeRoleWatcher.class); private final CuratorFramework curatorFramework; - private final NodeType nodeType; + private final NodeRole nodeRole; private final ObjectMapper jsonMapper; /** hostAndPort -> DiscoveryDruidNode */ @@ -169,28 +191,45 @@ private static class NodeTypeWatcher implements DruidNodeDiscovery private final CountDownLatch cacheInitialized = new CountDownLatch(1); - NodeTypeWatcher( + NodeRoleWatcher( ExecutorService listenerExecutor, CuratorFramework curatorFramework, String basePath, ObjectMapper jsonMapper, - NodeType nodeType + NodeRole nodeRole ) { this.listenerExecutor = listenerExecutor; this.curatorFramework = curatorFramework; - this.nodeType = nodeType; + this.nodeRole = nodeRole; this.jsonMapper = jsonMapper; - // This is required to be single threaded from Docs in PathChildrenCache; - this.cacheExecutor = Execs.singleThreaded(StringUtils.format("NodeTypeWatcher[%s]", nodeType)); - this.cache = new PathChildrenCache( - curatorFramework, - ZKPaths.makePath(basePath, nodeType.toString()), - true, - true, - cacheExecutor - ); + // This is required to be single threaded from docs in PathChildrenCache. + this.cacheExecutor = Execs.singleThreaded(StringUtils.format("NodeRoleWatcher[%s]", nodeRole)); + cache = new PathChildrenCacheFactory.Builder() + //NOTE: cacheData is temporarily set to false and we get data directly from ZK on each event. + //this is a workaround to solve curator's out-of-order events problem + //https://issues.apache.org/jira/browse/CURATOR-191 + // This is also done in CuratorInventoryManager. + .withCacheData(true) + .withCompressed(true) + .withExecutorService(cacheExecutor) + .build() + .make(curatorFramework, ZKPaths.makePath(basePath, nodeRole.toString())); + + try { + cache.getListenable().addListener((client, event) -> handleChildEvent(event)); + cache.start(PathChildrenCache.StartMode.POST_INITIALIZED_EVENT); + } + catch (Exception ex) { + throw new RuntimeException(ex); + } + } + + @Override + public void close() throws IOException + { + CloseableUtils.closeBoth(cache, cacheExecutor::shutdownNow); } @Override @@ -234,86 +273,19 @@ void handleChildEvent(PathChildrenCacheEvent event) try { switch (event.getType()) { case CHILD_ADDED: { - final byte[] data; - try { - data = curatorFramework.getData().decompressed().forPath(event.getData().getPath()); - } - catch (Exception ex) { - log.error( - ex, - "Failed to get data for path [%s]. Ignoring event [%s].", - event.getData().getPath(), - event.getType() - ); - return; - } - - DiscoveryDruidNode druidNode = jsonMapper.readValue(data, DiscoveryDruidNode.class); - - if (!nodeType.equals(druidNode.getNodeType())) { - log.warn( - "Node[%s:%s] add is discovered by node watcher of different node type. Ignored.", - druidNode.getDruidNode().getHostAndPortToUse(), - druidNode - ); - return; - } - - log.info("Node[%s:%s] appeared.", druidNode.getDruidNode().getHostAndPortToUse(), druidNode); - - addNode(druidNode); - + childAdded(event); break; } case CHILD_REMOVED: { - DiscoveryDruidNode druidNode = jsonMapper.readValue(event.getData().getData(), DiscoveryDruidNode.class); - - if (!nodeType.equals(druidNode.getNodeType())) { - log.warn( - "Node[%s:%s] removal is discovered by node watcher of different type. Ignored.", - druidNode.getDruidNode().getHostAndPortToUse(), - druidNode - ); - return; - } - - log.info( - "Node[%s:%s] disappeared.", - druidNode.getDruidNode().getHostAndPortToUse(), - druidNode - ); - - removeNode(druidNode); - + childRemoved(event); break; } case INITIALIZED: { - // No need to wait on CountDownLatch, because we are holding the lock under which it could only be - // counted down. - if (cacheInitialized.getCount() == 0) { - log.warn("cache is already initialized. ignoring [%s] event.", event.getType()); - return; - } - - log.info("Received INITIALIZED in node watcher."); - - for (Listener listener : nodeListeners) { - safeSchedule( - () -> { - listener.nodesAdded(unmodifiableNodes); - listener.nodeViewInitialized(); - }, - "Exception occured in nodesAdded([%s]) in listener [%s].", - unmodifiableNodes, - listener - ); - } - - cacheInitialized.countDown(); + cacheInitialized(); break; } default: { - log.info("Ignored event type [%s] for nodeType watcher.", event.getType()); + log.info("Ignored event type [%s] for nodeRole watcher.", event.getType()); } } } @@ -323,16 +295,32 @@ void handleChildEvent(PathChildrenCacheEvent event) } } - private void safeSchedule(Runnable runnable, String errMsgFormat, Object... args) + @GuardedBy("lock") + void childAdded(PathChildrenCacheEvent event) throws IOException { - listenerExecutor.submit(() -> { - try { - runnable.run(); - } - catch (Exception ex) { - log.error(errMsgFormat, args); - } - }); + final byte[] data = getZkDataForNode(event.getData()); + if (data == null) { + log.error( + "Failed to get data for path [%s]. Ignoring a child addition event.", + event.getData().getPath() + ); + return; + } + + DiscoveryDruidNode druidNode = jsonMapper.readValue(data, DiscoveryDruidNode.class); + + if (!nodeRole.equals(druidNode.getNodeRole())) { + log.error( + "Node[%s:%s] add is discovered by node watcher of different node role. Ignored.", + druidNode.getDruidNode().getHostAndPortToUse(), + druidNode + ); + return; + } + + log.info("Node[%s:%s] appeared.", druidNode.getDruidNode().getHostAndPortToUse(), druidNode); + + addNode(druidNode); } @GuardedBy("lock") @@ -353,7 +341,7 @@ private void addNode(DiscoveryDruidNode druidNode) } } } else { - log.warn( + log.error( "Node[%s:%s] discovered but existed already [%s].", druidNode.getDruidNode().getHostAndPortToUse(), druidNode, @@ -362,13 +350,41 @@ private void addNode(DiscoveryDruidNode druidNode) } } + @GuardedBy("lock") + private void childRemoved(PathChildrenCacheEvent event) throws IOException + { + final byte[] data = event.getData().getData(); + if (data == null) { + log.error( + "Failed to get data for path [%s]. Ignoring a child removal event.", + event.getData().getPath() + ); + return; + } + + DiscoveryDruidNode druidNode = jsonMapper.readValue(data, DiscoveryDruidNode.class); + + if (!nodeRole.equals(druidNode.getNodeRole())) { + log.error( + "Node[%s:%s] removal is discovered by node watcher of different role. Ignored.", + druidNode.getDruidNode().getHostAndPortToUse(), + druidNode + ); + return; + } + + log.info("Node[%s:%s] disappeared.", druidNode.getDruidNode().getHostAndPortToUse(), druidNode); + + removeNode(druidNode); + } + @GuardedBy("lock") private void removeNode(DiscoveryDruidNode druidNode) { DiscoveryDruidNode prev = nodes.remove(druidNode.getDruidNode().getHostAndPortToUse()); if (prev == null) { - log.warn( + log.error( "Noticed disappearance of unknown druid node [%s:%s].", druidNode.getDruidNode().getHostAndPortToUse(), druidNode @@ -390,26 +406,124 @@ private void removeNode(DiscoveryDruidNode druidNode) } } - public void start() + /** + * Doing this instead of a simple call to {@link ChildData#getData()} because data cache is turned off, see a + * comment in {@link #NodeRoleWatcher}. + */ + @Nullable + private byte[] getZkDataForNode(ChildData child) { try { - cache.getListenable().addListener((client, event) -> handleChildEvent(event)); - cache.start(PathChildrenCache.StartMode.POST_INITIALIZED_EVENT); + return curatorFramework.getData().decompressed().forPath(child.getPath()); } catch (Exception ex) { - throw new RuntimeException(ex); + log.error(ex, "Exception while getting data for node %s", child.getPath()); + return null; } } - public void stop() + @GuardedBy("lock") + private void cacheInitialized() { + // No need to wait on CountDownLatch, because we are holding the lock under which it could only be + // counted down. + if (cacheInitialized.getCount() == 0) { + log.error("cache is already initialized. ignoring cache initialization event."); + return; + } + + log.info("Received INITIALIZED in node watcher."); + + for (Listener listener : nodeListeners) { + safeSchedule( + () -> { + listener.nodesAdded(unmodifiableNodes); + listener.nodeViewInitialized(); + }, + "Exception occured in nodesAdded([%s]) in listener [%s].", + unmodifiableNodes, + listener + ); + } + + cacheInitialized.countDown(); + } + + private void safeSchedule(Runnable runnable, String errMsgFormat, Object... args) { + listenerExecutor.submit(() -> { + try { + runnable.run(); + } + catch (Exception ex) { + log.error(errMsgFormat, args); + } + }); + } + } + + private static class NodeDiscoverer implements Closeable + { + private final ObjectMapper jsonMapper; + private final NodeCache nodeCache; + private final NodeRole nodeRole; + + private NodeDiscoverer( + ZkPathsConfig config, + ObjectMapper jsonMapper, + CuratorFramework curatorFramework, + DruidNode node, + NodeRole nodeRole + ) + { + this.jsonMapper = jsonMapper; + String path = CuratorDruidNodeAnnouncer.makeNodeAnnouncementPath(config, nodeRole, node); + nodeCache = new NodeCache(curatorFramework, path, true); + this.nodeRole = nodeRole; + try { - cache.close(); - cacheExecutor.shutdownNow(); + nodeCache.start(true /* buildInitial */); } - catch (Exception ex) { - log.error(ex, "Failed to stop node watcher for type [%s].", nodeType); + catch (Exception e) { + throw new RuntimeException(e); + } + } + + private boolean nodeDiscovered() + { + @Nullable ChildData currentChild = nodeCache.getCurrentData(); + if (currentChild == null) { + // Not discovered yet. + return false; + } + + final byte[] data = currentChild.getData(); + + DiscoveryDruidNode druidNode; + try { + druidNode = jsonMapper.readValue(data, DiscoveryDruidNode.class); + } + catch (IOException e) { + log.error(e, "Exception occurred when reading node's value"); + return false; + } + + if (!nodeRole.equals(druidNode.getNodeRole())) { + log.error( + "Node[%s:%s] add is discovered by node watcher of different node role. Ignored.", + druidNode.getDruidNode().getHostAndPortToUse(), + druidNode + ); + return false; } + + log.info("Node[%s:%s] appeared.", druidNode.getDruidNode().getHostAndPortToUse(), druidNode); + return true; + } + + @Override + public void close() throws IOException + { + nodeCache.close(); } } } diff --git a/server/src/main/java/org/apache/druid/curator/discovery/DiscoveryModule.java b/server/src/main/java/org/apache/druid/curator/discovery/DiscoveryModule.java index 5dd53dad86c7..fd899640c67f 100644 --- a/server/src/main/java/org/apache/druid/curator/discovery/DiscoveryModule.java +++ b/server/src/main/java/org/apache/druid/curator/discovery/DiscoveryModule.java @@ -195,8 +195,10 @@ public void configure(Binder binder) PolyBind.optionBinder(binder, Key.get(DruidLeaderSelector.class, Coordinator.class)) .addBinding(CURATOR_KEY) - .toProvider(new DruidLeaderSelectorProvider( - (zkPathsConfig) -> ZKPaths.makePath(zkPathsConfig.getCoordinatorPath(), "_COORDINATOR")) + .toProvider( + new DruidLeaderSelectorProvider( + zkPathsConfig -> ZKPaths.makePath(zkPathsConfig.getCoordinatorPath(), "_COORDINATOR") + ) ) .in(LazySingleton.class); @@ -204,7 +206,7 @@ public void configure(Binder binder) .addBinding(CURATOR_KEY) .toProvider( new DruidLeaderSelectorProvider( - (zkPathsConfig) -> ZKPaths.makePath(zkPathsConfig.getOverlordPath(), "_OVERLORD") + zkPathsConfig -> ZKPaths.makePath(zkPathsConfig.getOverlordPath(), "_OVERLORD") ) ) .in(LazySingleton.class); diff --git a/server/src/main/java/org/apache/druid/curator/inventory/CuratorInventoryManager.java b/server/src/main/java/org/apache/druid/curator/inventory/CuratorInventoryManager.java index bf6c7c670006..9f9b1bd6f9e7 100644 --- a/server/src/main/java/org/apache/druid/curator/inventory/CuratorInventoryManager.java +++ b/server/src/main/java/org/apache/druid/curator/inventory/CuratorInventoryManager.java @@ -33,6 +33,7 @@ import org.apache.druid.java.util.common.lifecycle.LifecycleStop; import org.apache.druid.java.util.common.logger.Logger; +import javax.annotation.Nullable; import java.io.IOException; import java.util.Collection; import java.util.Set; @@ -70,7 +71,7 @@ public class CuratorInventoryManager private final PathChildrenCacheFactory cacheFactory; private final ExecutorService pathChildrenCacheExecutor; - private volatile PathChildrenCache childrenCache; + private volatile @Nullable PathChildrenCache childrenCache; public CuratorInventoryManager( CuratorFramework curatorFramework, @@ -91,6 +92,7 @@ public CuratorInventoryManager( //NOTE: cacheData is temporarily set to false and we get data directly from ZK on each event. //this is a workaround to solve curator's out-of-order events problem //https://issues.apache.org/jira/browse/CURATOR-191 + // This is also done in CuratorDruidNodeDiscoveryProvider. .withCacheData(false) .withCompressed(true) .withExecutorService(pathChildrenCacheExecutor) @@ -101,12 +103,14 @@ public CuratorInventoryManager( @LifecycleStart public void start() throws Exception { + PathChildrenCache childrenCache; synchronized (lock) { + childrenCache = this.childrenCache; if (childrenCache != null) { return; } - childrenCache = cacheFactory.make(curatorFramework, config.getContainerPath()); + this.childrenCache = childrenCache = cacheFactory.make(curatorFramework, config.getContainerPath()); } childrenCache.getListenable().addListener(new ContainerCacheListener()); @@ -131,13 +135,14 @@ public void start() throws Exception public void stop() throws IOException { synchronized (lock) { + PathChildrenCache childrenCache = this.childrenCache; if (childrenCache == null) { return; } // This close() call actually calls shutdownNow() on the executor registered with the Cache object... childrenCache.close(); - childrenCache = null; + this.childrenCache = null; } Closer closer = Closer.create(); @@ -157,6 +162,7 @@ public InventoryManagerConfig getConfig() return config; } + @Nullable public ContainerClass getInventoryValue(String containerKey) { final ContainerHolder containerHolder = containers.get(containerKey); @@ -171,13 +177,18 @@ public Collection getInventory() .collect(Collectors.toList()); } - private byte[] getZkDataForNode(String path) + /** + * Doing this instead of a simple call to {@link ChildData#getData()} because data cache is turned off, see a comment + * in {@link #CuratorInventoryManager}. + */ + @Nullable + private byte[] getZkDataForNode(ChildData child) { try { - return curatorFramework.getData().decompressed().forPath(path); + return curatorFramework.getData().decompressed().forPath(child.getPath()); } catch (Exception ex) { - log.warn(ex, "Exception while getting data for node %s", path); + log.warn(ex, "Exception while getting data for node %s", child.getPath()); return null; } } @@ -193,7 +204,7 @@ private class ContainerHolder PathChildrenCache cache ) { - this.container = new AtomicReference(container); + this.container = new AtomicReference<>(container); this.cache = cache; } @@ -226,7 +237,7 @@ public void childEvent(CuratorFramework client, PathChildrenCacheEvent event) th synchronized (lock) { final ChildData child = event.getData(); - byte[] data = getZkDataForNode(child.getPath()); + byte[] data = getZkDataForNode(child); if (data == null) { log.info("Ignoring event: Type - %s , Path - %s , Version - %s", event.getType(), @@ -285,7 +296,7 @@ public void childEvent(CuratorFramework client, PathChildrenCacheEvent event) th synchronized (lock) { final ChildData child = event.getData(); - byte[] data = getZkDataForNode(child.getPath()); + byte[] data = getZkDataForNode(child); if (data == null) { log.info( "Ignoring event: Type - %s , Path - %s , Version - %s", @@ -356,13 +367,10 @@ private void maybeDoneInitializing() private class InventoryCacheListener implements PathChildrenCacheListener { private final String containerKey; - private final String inventoryPath; public InventoryCacheListener(String containerKey, String inventoryPath) { this.containerKey = containerKey; - this.inventoryPath = inventoryPath; - log.info("Created new InventoryCacheListener for %s", inventoryPath); } @@ -378,7 +386,7 @@ public void childEvent(CuratorFramework client, PathChildrenCacheEvent event) case CHILD_ADDED: { final ChildData child = event.getData(); - byte[] data = getZkDataForNode(child.getPath()); + byte[] data = getZkDataForNode(child); if (data == null) { log.info("Ignoring event: Type - %s , Path - %s , Version - %s", event.getType(), @@ -401,7 +409,7 @@ public void childEvent(CuratorFramework client, PathChildrenCacheEvent event) case CHILD_UPDATED: { final ChildData child = event.getData(); - byte[] data = getZkDataForNode(child.getPath()); + byte[] data = getZkDataForNode(child); if (data == null) { log.info("Ignoring event: Type - %s , Path - %s , Version - %s", event.getType(), diff --git a/server/src/main/java/org/apache/druid/discovery/DiscoveryDruidNode.java b/server/src/main/java/org/apache/druid/discovery/DiscoveryDruidNode.java index a8d2d140e396..813ef8d2956d 100644 --- a/server/src/main/java/org/apache/druid/discovery/DiscoveryDruidNode.java +++ b/server/src/main/java/org/apache/druid/discovery/DiscoveryDruidNode.java @@ -21,7 +21,6 @@ import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; -import org.apache.druid.client.DruidServer; import org.apache.druid.server.DruidNode; import java.util.HashMap; @@ -30,14 +29,14 @@ /** * Representation of all information related to discovery of a node and all the other metadata associated with - * the node per nodeType such as broker, historical etc. - * Note that one Druid process might announce multiple DiscoveryDruidNode if it acts as multiple nodeTypes e.g. - * coordinator would announce DiscoveryDruidNode for overlord nodeType as well when acting as overlord. + * the node per nodeRole such as broker, historical etc. + * Note that one Druid process might announce multiple DiscoveryDruidNode if it acts in multiple {@link NodeRole}s e. g. + * Coordinator would announce DiscoveryDruidNode for {@link NodeRole#OVERLORD} as well when acting as Overlord. */ public class DiscoveryDruidNode { private final DruidNode druidNode; - private final NodeType nodeType; + private final NodeRole nodeRole; // Other metadata associated with the node e.g. // if its a historical node then lookup information, segment loading capacity etc. @@ -46,12 +45,12 @@ public class DiscoveryDruidNode @JsonCreator public DiscoveryDruidNode( @JsonProperty("druidNode") DruidNode druidNode, - @JsonProperty("nodeType") NodeType nodeType, + @JsonProperty("nodeType") NodeRole nodeRole, @JsonProperty("services") Map services ) { this.druidNode = druidNode; - this.nodeType = nodeType; + this.nodeRole = nodeRole; if (services != null && !services.isEmpty()) { this.services.putAll(services); @@ -64,10 +63,14 @@ public Map getServices() return services; } - @JsonProperty - public NodeType getNodeType() + /** + * Keeping the legacy name 'nodeType' property name for backward compatibility. When the project is updated to + * Jackson 2.9 it could be changed, see https://github.com/apache/incubator-druid/issues/7152. + */ + @JsonProperty("nodeType") + public NodeRole getNodeRole() { - return nodeType; + return nodeRole; } @JsonProperty @@ -76,19 +79,6 @@ public DruidNode getDruidNode() return druidNode; } - public DruidServer toDruidServer() - { - return new DruidServer( - getDruidNode().getHostAndPortToUse(), - getDruidNode().getHostAndPort(), - getDruidNode().getHostAndTlsPort(), - ((DataNodeService) getServices().get(DataNodeService.DISCOVERY_SERVICE_KEY)).getMaxSize(), - ((DataNodeService) getServices().get(DataNodeService.DISCOVERY_SERVICE_KEY)).getType(), - ((DataNodeService) getServices().get(DataNodeService.DISCOVERY_SERVICE_KEY)).getTier(), - ((DataNodeService) getServices().get(DataNodeService.DISCOVERY_SERVICE_KEY)).getPriority() - ); - } - @Override public boolean equals(Object o) { @@ -100,14 +90,14 @@ public boolean equals(Object o) } DiscoveryDruidNode that = (DiscoveryDruidNode) o; return Objects.equals(druidNode, that.druidNode) && - Objects.equals(nodeType, that.nodeType) && + Objects.equals(nodeRole, that.nodeRole) && Objects.equals(services, that.services); } @Override public int hashCode() { - return Objects.hash(druidNode, nodeType, services); + return Objects.hash(druidNode, nodeRole, services); } @Override @@ -115,7 +105,7 @@ public String toString() { return "DiscoveryDruidNode{" + "druidNode=" + druidNode + - ", nodeType='" + nodeType + '\'' + + ", nodeRole='" + nodeRole + '\'' + ", services=" + services + '}'; } diff --git a/server/src/main/java/org/apache/druid/discovery/DruidLeaderClient.java b/server/src/main/java/org/apache/druid/discovery/DruidLeaderClient.java index 79d5a69917af..e496bbd77a61 100644 --- a/server/src/main/java/org/apache/druid/discovery/DruidLeaderClient.java +++ b/server/src/main/java/org/apache/druid/discovery/DruidLeaderClient.java @@ -67,7 +67,7 @@ public class DruidLeaderClient private final HttpClient httpClient; private final DruidNodeDiscoveryProvider druidNodeDiscoveryProvider; - private final NodeType nodeTypeToWatch; + private final NodeRole nodeRoleToWatch; private final String leaderRequestPath; @@ -81,14 +81,14 @@ public class DruidLeaderClient public DruidLeaderClient( HttpClient httpClient, DruidNodeDiscoveryProvider druidNodeDiscoveryProvider, - NodeType nodeTypeToWatch, + NodeRole nodeRoleToWatch, String leaderRequestPath, ServerDiscoverySelector serverDiscoverySelector ) { this.httpClient = httpClient; this.druidNodeDiscoveryProvider = druidNodeDiscoveryProvider; - this.nodeTypeToWatch = nodeTypeToWatch; + this.nodeRoleToWatch = nodeRoleToWatch; this.leaderRequestPath = leaderRequestPath; this.serverDiscoverySelector = serverDiscoverySelector; } @@ -101,7 +101,7 @@ public void start() } try { - druidNodeDiscovery = druidNodeDiscoveryProvider.getForNodeType(nodeTypeToWatch); + druidNodeDiscovery = druidNodeDiscoveryProvider.getForNodeRole(nodeRoleToWatch); lifecycleLock.started(); log.info("Started."); } diff --git a/server/src/main/java/org/apache/druid/discovery/DruidNodeDiscovery.java b/server/src/main/java/org/apache/druid/discovery/DruidNodeDiscovery.java index d9148c34f8bb..a1e65af0e1dc 100644 --- a/server/src/main/java/org/apache/druid/discovery/DruidNodeDiscovery.java +++ b/server/src/main/java/org/apache/druid/discovery/DruidNodeDiscovery.java @@ -22,7 +22,7 @@ import java.util.Collection; /** - * Interface for discovering Druid Nodes announced by DruidNodeAnnouncer. + * Interface for discovering Druid nodes announced by {@link DruidNodeAnnouncer}. */ public interface DruidNodeDiscovery { @@ -30,9 +30,9 @@ public interface DruidNodeDiscovery void registerListener(Listener listener); /** - * Listener for watching nodes in a DruidNodeDiscovery instance obtained via DruidNodeDiscoveryProvider.getXXX(). - * DruidNodeDiscovery implementation should assume that Listener is not threadsafe and never call methods in - * Listener concurrently. + * Listener for watching nodes in a DruidNodeDiscovery instance obtained via {@link + * DruidNodeDiscoveryProvider}.getXXX(). DruidNodeDiscovery implementation should assume that Listener is not + * thread-safe and never call methods in Listener concurrently. * * Implementation of Listener must ensure to not do any time consuming work or block in any of the methods. */ diff --git a/server/src/main/java/org/apache/druid/discovery/DruidNodeDiscoveryProvider.java b/server/src/main/java/org/apache/druid/discovery/DruidNodeDiscoveryProvider.java index 4dcb77759a86..f5127189de3c 100644 --- a/server/src/main/java/org/apache/druid/discovery/DruidNodeDiscoveryProvider.java +++ b/server/src/main/java/org/apache/druid/discovery/DruidNodeDiscoveryProvider.java @@ -24,6 +24,7 @@ import com.google.common.collect.ImmutableSet; import org.apache.druid.java.util.common.IAE; import org.apache.druid.java.util.common.logger.Logger; +import org.apache.druid.server.DruidNode; import java.util.ArrayList; import java.util.Collection; @@ -33,25 +34,26 @@ import java.util.Set; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ConcurrentMap; +import java.util.function.BooleanSupplier; /** * Provider of {@link DruidNodeDiscovery} instances. */ public abstract class DruidNodeDiscoveryProvider { - private static final Map> SERVICE_TO_NODE_TYPES = ImmutableMap.of( - LookupNodeService.DISCOVERY_SERVICE_KEY, ImmutableSet.of(NodeType.BROKER, NodeType.HISTORICAL, NodeType.PEON), - DataNodeService.DISCOVERY_SERVICE_KEY, ImmutableSet.of(NodeType.HISTORICAL, NodeType.PEON), - WorkerNodeService.DISCOVERY_SERVICE_KEY, ImmutableSet.of(NodeType.PEON) + private static final Map> SERVICE_TO_NODE_TYPES = ImmutableMap.of( + LookupNodeService.DISCOVERY_SERVICE_KEY, ImmutableSet.of(NodeRole.BROKER, NodeRole.HISTORICAL, NodeRole.PEON), + DataNodeService.DISCOVERY_SERVICE_KEY, ImmutableSet.of(NodeRole.HISTORICAL, NodeRole.PEON), + WorkerNodeService.DISCOVERY_SERVICE_KEY, ImmutableSet.of(NodeRole.PEON) ); private final ConcurrentHashMap serviceDiscoveryMap = new ConcurrentHashMap<>(SERVICE_TO_NODE_TYPES.size()); - /** - * Get DruidNodeDiscovery instance to discover nodes of given nodeType. - */ - public abstract DruidNodeDiscovery getForNodeType(NodeType nodeType); + public abstract BooleanSupplier getForNode(DruidNode node, NodeRole nodeRole); + + /** Get a {@link DruidNodeDiscovery} instance to discover nodes of the given node role. */ + public abstract DruidNodeDiscovery getForNodeRole(NodeRole nodeRole); /** * Get DruidNodeDiscovery instance to discover nodes that announce given service in its metadata. @@ -62,15 +64,15 @@ public DruidNodeDiscovery getForService(String serviceName) serviceName, service -> { - Set nodeTypesToWatch = DruidNodeDiscoveryProvider.SERVICE_TO_NODE_TYPES.get(service); - if (nodeTypesToWatch == null) { + Set nodeRolesToWatch = DruidNodeDiscoveryProvider.SERVICE_TO_NODE_TYPES.get(service); + if (nodeRolesToWatch == null) { throw new IAE("Unknown service [%s].", service); } - ServiceDruidNodeDiscovery serviceDiscovery = new ServiceDruidNodeDiscovery(service, nodeTypesToWatch.size()); + ServiceDruidNodeDiscovery serviceDiscovery = new ServiceDruidNodeDiscovery(service, nodeRolesToWatch.size()); DruidNodeDiscovery.Listener filteringGatheringUpstreamListener = serviceDiscovery.filteringUpstreamListener(); - for (NodeType nodeType : nodeTypesToWatch) { - getForNodeType(nodeType).registerListener(filteringGatheringUpstreamListener); + for (NodeRole nodeRole : nodeRolesToWatch) { + getForNodeRole(nodeRole).registerListener(filteringGatheringUpstreamListener); } return serviceDiscovery; } @@ -89,13 +91,13 @@ private static class ServiceDruidNodeDiscovery implements DruidNodeDiscovery private final Object lock = new Object(); - private int uninitializedNodeTypes; + private int uninitializedNodeRoles; - ServiceDruidNodeDiscovery(String service, int watchedNodeTypes) + ServiceDruidNodeDiscovery(String service, int watchedNodeRoles) { - Preconditions.checkArgument(watchedNodeTypes > 0); + Preconditions.checkArgument(watchedNodeRoles > 0); this.service = service; - this.uninitializedNodeTypes = watchedNodeTypes; + this.uninitializedNodeRoles = watchedNodeRoles; } @Override @@ -114,7 +116,7 @@ public void registerListener(Listener listener) if (!unmodifiableNodes.isEmpty()) { listener.nodesAdded(unmodifiableNodes); } - if (uninitializedNodeTypes == 0) { + if (uninitializedNodeRoles == 0) { listener.nodeViewInitialized(); } listeners.add(listener); @@ -128,7 +130,7 @@ DruidNodeDiscovery.Listener filteringUpstreamListener() /** * Listens for all node updates and filters them based on {@link #service}. Note: this listener is registered with - * the objects returned from {@link #getForNodeType(NodeType)}, NOT with {@link ServiceDruidNodeDiscovery} itself. + * the objects returned from {@link #getForNodeRole(NodeRole)}, NOT with {@link ServiceDruidNodeDiscovery} itself. */ class FilteringUpstreamListener implements DruidNodeDiscovery.Listener { @@ -203,12 +205,12 @@ public void nodesRemoved(Collection nodesDisappeared) public void nodeViewInitialized() { synchronized (lock) { - if (uninitializedNodeTypes == 0) { + if (uninitializedNodeRoles == 0) { log.error("Unexpected call of nodeViewInitialized()"); return; } - uninitializedNodeTypes--; - if (uninitializedNodeTypes == 0) { + uninitializedNodeRoles--; + if (uninitializedNodeRoles == 0) { for (Listener listener : listeners) { try { listener.nodeViewInitialized(); diff --git a/server/src/main/java/org/apache/druid/discovery/NodeType.java b/server/src/main/java/org/apache/druid/discovery/NodeRole.java similarity index 73% rename from server/src/main/java/org/apache/druid/discovery/NodeType.java rename to server/src/main/java/org/apache/druid/discovery/NodeRole.java index 841656bf1fc9..e80e0bb0275c 100644 --- a/server/src/main/java/org/apache/druid/discovery/NodeType.java +++ b/server/src/main/java/org/apache/druid/discovery/NodeRole.java @@ -22,13 +22,16 @@ import com.fasterxml.jackson.annotation.JsonValue; /** - * * This is a historical occasion that this enum is different from {@link - * org.apache.druid.server.coordination.ServerType} because they are essentially the same abstraction, but merging them - * could only increase the complexity and drop the code safety, because they name the same types differently ("peon" - - * "indexer-executor" and "middleManager" - "realtime") and both expose them via JSON APIs. + * org.apache.druid.server.coordination.ServerType} (also called "node type" in various places) because they are + * essentially the same abstraction, but merging them could only increase the complexity and drop the code safety, + * because they name the same types differently ("peon" - "indexer-executor" and "middleManager" - "realtime") and both + * expose them via JSON APIs. + * + * These abstractions can probably be merged when Druid updates to Jackson 2.9 that supports JsonAliases, see + * see https://github.com/apache/incubator-druid/issues/7152. */ -public enum NodeType +public enum NodeRole { COORDINATOR("coordinator"), HISTORICAL("historical"), @@ -40,7 +43,7 @@ public enum NodeType private final String jsonName; - NodeType(String jsonName) + NodeRole(String jsonName) { this.jsonName = jsonName; } diff --git a/server/src/main/java/org/apache/druid/guice/CoordinatorDiscoveryModule.java b/server/src/main/java/org/apache/druid/guice/CoordinatorDiscoveryModule.java index d4ac2c454a04..b90a9b5e76ff 100644 --- a/server/src/main/java/org/apache/druid/guice/CoordinatorDiscoveryModule.java +++ b/server/src/main/java/org/apache/druid/guice/CoordinatorDiscoveryModule.java @@ -28,7 +28,7 @@ import org.apache.druid.curator.discovery.ServerDiscoverySelector; import org.apache.druid.discovery.DruidLeaderClient; import org.apache.druid.discovery.DruidNodeDiscoveryProvider; -import org.apache.druid.discovery.NodeType; +import org.apache.druid.discovery.NodeRole; import org.apache.druid.guice.annotations.EscalatedGlobal; import org.apache.druid.java.util.http.client.HttpClient; @@ -65,7 +65,7 @@ public DruidLeaderClient getLeaderHttpClient( return new DruidLeaderClient( httpClient, druidNodeDiscoveryProvider, - NodeType.COORDINATOR, + NodeRole.COORDINATOR, "/druid/coordinator/v1/leader", serverDiscoverySelector ); diff --git a/server/src/main/java/org/apache/druid/guice/IndexingServiceDiscoveryModule.java b/server/src/main/java/org/apache/druid/guice/IndexingServiceDiscoveryModule.java index 05b76ed2294f..3c4f63c5404f 100644 --- a/server/src/main/java/org/apache/druid/guice/IndexingServiceDiscoveryModule.java +++ b/server/src/main/java/org/apache/druid/guice/IndexingServiceDiscoveryModule.java @@ -28,7 +28,7 @@ import org.apache.druid.curator.discovery.ServerDiscoverySelector; import org.apache.druid.discovery.DruidLeaderClient; import org.apache.druid.discovery.DruidNodeDiscoveryProvider; -import org.apache.druid.discovery.NodeType; +import org.apache.druid.discovery.NodeRole; import org.apache.druid.guice.annotations.EscalatedGlobal; import org.apache.druid.java.util.http.client.HttpClient; @@ -65,7 +65,7 @@ public DruidLeaderClient getLeaderHttpClient( return new DruidLeaderClient( httpClient, druidNodeDiscoveryProvider, - NodeType.OVERLORD, + NodeRole.OVERLORD, "/druid/indexer/v1/leader", serverDiscoverySelector ); diff --git a/server/src/main/java/org/apache/druid/server/coordination/ServerType.java b/server/src/main/java/org/apache/druid/server/coordination/ServerType.java index d1b7fd1204fb..df25fb3aa08b 100644 --- a/server/src/main/java/org/apache/druid/server/coordination/ServerType.java +++ b/server/src/main/java/org/apache/druid/server/coordination/ServerType.java @@ -21,6 +21,7 @@ import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonValue; +import org.apache.druid.discovery.NodeRole; import org.apache.druid.java.util.common.StringUtils; /** @@ -36,10 +37,13 @@ * which is the format expected for the server type string prior to the patch that introduced ServerType: * https://github.com/apache/incubator-druid/pull/4148 * - * This is a historical occasion that this enum is different from {@link org.apache.druid.discovery.NodeType} because + * This is a historical occasion that this enum is different from {@link NodeRole} because * they are essentially the same abstraction, but merging them could only increase the complexity and drop the code * safety, because they name the same types differently ("indexer-executor" - "peon" and "realtime" - "middleManager") * and both expose them via JSON APIs. + * + * These abstractions can probably be merged when Druid updates to Jackson 2.9 that supports JsonAliases, see + * see https://github.com/apache/incubator-druid/issues/7152. */ public enum ServerType { diff --git a/server/src/main/java/org/apache/druid/server/http/ClusterResource.java b/server/src/main/java/org/apache/druid/server/http/ClusterResource.java index 0bc871386c1e..939286a420ca 100644 --- a/server/src/main/java/org/apache/druid/server/http/ClusterResource.java +++ b/server/src/main/java/org/apache/druid/server/http/ClusterResource.java @@ -28,7 +28,7 @@ import com.sun.jersey.spi.container.ResourceFilters; import org.apache.druid.discovery.DiscoveryDruidNode; import org.apache.druid.discovery.DruidNodeDiscoveryProvider; -import org.apache.druid.discovery.NodeType; +import org.apache.druid.discovery.NodeRole; import org.apache.druid.guice.LazySingleton; import org.apache.druid.server.DruidNode; import org.apache.druid.server.http.security.StateResourceFilter; @@ -62,21 +62,21 @@ public ClusterResource(DruidNodeDiscoveryProvider discoveryProvider) @Produces(MediaType.APPLICATION_JSON) public Response getClusterServers(@QueryParam("full") boolean full) { - ImmutableMap.Builder entityBuilder = new ImmutableMap.Builder<>(); + ImmutableMap.Builder entityBuilder = new ImmutableMap.Builder<>(); - entityBuilder.put(NodeType.COORDINATOR, getNodes(NodeType.COORDINATOR, full)); - entityBuilder.put(NodeType.OVERLORD, getNodes(NodeType.OVERLORD, full)); - entityBuilder.put(NodeType.BROKER, getNodes(NodeType.BROKER, full)); - entityBuilder.put(NodeType.HISTORICAL, getNodes(NodeType.HISTORICAL, full)); + entityBuilder.put(NodeRole.COORDINATOR, getNodes(NodeRole.COORDINATOR, full)); + entityBuilder.put(NodeRole.OVERLORD, getNodes(NodeRole.OVERLORD, full)); + entityBuilder.put(NodeRole.BROKER, getNodes(NodeRole.BROKER, full)); + entityBuilder.put(NodeRole.HISTORICAL, getNodes(NodeRole.HISTORICAL, full)); - Collection mmNodes = getNodes(NodeType.MIDDLE_MANAGER, full); + Collection mmNodes = getNodes(NodeRole.MIDDLE_MANAGER, full); if (!mmNodes.isEmpty()) { - entityBuilder.put(NodeType.MIDDLE_MANAGER, mmNodes); + entityBuilder.put(NodeRole.MIDDLE_MANAGER, mmNodes); } - Collection routerNodes = getNodes(NodeType.ROUTER, full); + Collection routerNodes = getNodes(NodeRole.ROUTER, full); if (!routerNodes.isEmpty()) { - entityBuilder.put(NodeType.ROUTER, routerNodes); + entityBuilder.put(NodeRole.ROUTER, routerNodes); } return Response.status(Response.Status.OK).entity(entityBuilder.build()).build(); @@ -84,22 +84,22 @@ public Response getClusterServers(@QueryParam("full") boolean full) @GET @Produces({MediaType.APPLICATION_JSON}) - @Path("/{nodeType}") - public Response getClusterServers(@PathParam("nodeType") NodeType nodeType, @QueryParam("full") boolean full) + @Path("/{nodeRole}") + public Response getClusterServers(@PathParam("nodeRole") NodeRole nodeRole, @QueryParam("full") boolean full) { - if (nodeType == null) { + if (nodeRole == null) { return Response.serverError() .status(Response.Status.BAD_REQUEST) - .entity("Invalid nodeType of null. Valid node types are " + Arrays.toString(NodeType.values())) + .entity("Invalid nodeRole of null. Valid node roles are " + Arrays.toString(NodeRole.values())) .build(); } else { - return Response.status(Response.Status.OK).entity(getNodes(nodeType, full)).build(); + return Response.status(Response.Status.OK).entity(getNodes(nodeRole, full)).build(); } } - private Collection getNodes(NodeType nodeType, boolean full) + private Collection getNodes(NodeRole nodeRole, boolean full) { - Collection discoveryDruidNodes = druidNodeDiscoveryProvider.getForNodeType(nodeType) + Collection discoveryDruidNodes = druidNodeDiscoveryProvider.getForNodeRole(nodeRole) .getAllNodes(); if (full) { return (Collection) discoveryDruidNodes; diff --git a/server/src/main/java/org/apache/druid/server/http/SelfDiscoveryResource.java b/server/src/main/java/org/apache/druid/server/http/SelfDiscoveryResource.java index 0beb3b249866..6fab65905c95 100644 --- a/server/src/main/java/org/apache/druid/server/http/SelfDiscoveryResource.java +++ b/server/src/main/java/org/apache/druid/server/http/SelfDiscoveryResource.java @@ -22,10 +22,8 @@ import com.google.inject.Inject; import com.google.inject.Singleton; import com.sun.jersey.spi.container.ResourceFilters; -import org.apache.druid.discovery.DiscoveryDruidNode; -import org.apache.druid.discovery.DruidNodeDiscovery; import org.apache.druid.discovery.DruidNodeDiscoveryProvider; -import org.apache.druid.discovery.NodeType; +import org.apache.druid.discovery.NodeRole; import org.apache.druid.guice.annotations.Self; import org.apache.druid.java.util.common.lifecycle.Lifecycle; import org.apache.druid.server.DruidNode; @@ -36,12 +34,12 @@ import javax.ws.rs.Produces; import javax.ws.rs.core.MediaType; import javax.ws.rs.core.Response; -import java.util.Collection; import java.util.Collections; +import java.util.function.BooleanSupplier; /** - * This class is annotated {@link Singleton} rather than {@link org.apache.druid.guice.LazySingleton}, because it adds - * a lifecycle handler in the constructor, that should happen before the lifecycle is started, i. e. eagerly during the + * This class is annotated {@link Singleton} rather than {@link org.apache.druid.guice.LazySingleton} because it adds + * a lifecycle handler in the constructor. That should happen before the lifecycle is started, i. e. eagerly during the * DI configuration phase. */ @Singleton @@ -49,12 +47,12 @@ @ResourceFilters(StateResourceFilter.class) public class SelfDiscoveryResource { - private boolean selfDiscovered = false; + private BooleanSupplier selfDiscovered; @Inject public SelfDiscoveryResource( @Self DruidNode thisDruidNode, - @Self NodeType thisNodeType, + @Self NodeRole thisNodeRole, DruidNodeDiscoveryProvider nodeDiscoveryProvider, Lifecycle lifecycle ) @@ -64,7 +62,7 @@ public SelfDiscoveryResource( @Override public void start() { - registerSelfDiscoveryListener(thisDruidNode, thisNodeType, nodeDiscoveryProvider); + selfDiscovered = nodeDiscoveryProvider.getForNode(thisDruidNode, thisNodeRole); } @Override @@ -73,45 +71,15 @@ public void stop() // do nothing } }; - // Using Lifecycle.Stage.LAST because DruidNodeDiscoveryProvider should be already started when - // registerSelfDiscoveryListener() is called. - lifecycle.addHandler(selfDiscoveryListenerRegistrator, Lifecycle.Stage.LAST); - } - - private void registerSelfDiscoveryListener( - DruidNode thisDruidNode, - NodeType thisNodeType, - DruidNodeDiscoveryProvider nodeDiscoveryProvider - ) - { - nodeDiscoveryProvider.getForNodeType(thisNodeType).registerListener(new DruidNodeDiscovery.Listener() - { - @Override - public void nodesAdded(Collection nodes) - { - if (selfDiscovered) { - return; - } - for (DiscoveryDruidNode node : nodes) { - if (node.getDruidNode().equals(thisDruidNode)) { - selfDiscovered = true; - break; - } - } - } - - @Override - public void nodesRemoved(Collection nodes) - { - // do nothing - } - }); + // Using Lifecycle.Stage.SERVER because DruidNodeDiscoveryProvider should be already started when + // selfDiscoveryListenerRegistrator.start() is called. + lifecycle.addHandler(selfDiscoveryListenerRegistrator, Lifecycle.Stage.SERVER); } @GET @Produces(MediaType.APPLICATION_JSON) public Response getSelfDiscovered() { - return Response.ok(Collections.singletonMap("selfDiscovered", selfDiscovered)).build(); + return Response.ok(Collections.singletonMap("selfDiscovered", selfDiscovered.getAsBoolean())).build(); } } diff --git a/server/src/main/java/org/apache/druid/server/router/TieredBrokerHostSelector.java b/server/src/main/java/org/apache/druid/server/router/TieredBrokerHostSelector.java index 9263b351b483..96473e1614e2 100644 --- a/server/src/main/java/org/apache/druid/server/router/TieredBrokerHostSelector.java +++ b/server/src/main/java/org/apache/druid/server/router/TieredBrokerHostSelector.java @@ -29,7 +29,7 @@ import org.apache.druid.discovery.DiscoveryDruidNode; import org.apache.druid.discovery.DruidNodeDiscovery; import org.apache.druid.discovery.DruidNodeDiscoveryProvider; -import org.apache.druid.discovery.NodeType; +import org.apache.druid.discovery.NodeRole; import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.java.util.common.Pair; import org.apache.druid.java.util.common.lifecycle.LifecycleStart; @@ -127,7 +127,7 @@ public void start() servers.put(entry.getValue(), new NodesHolder()); } - DruidNodeDiscovery druidNodeDiscovery = druidNodeDiscoveryProvider.getForNodeType(NodeType.BROKER); + DruidNodeDiscovery druidNodeDiscovery = druidNodeDiscoveryProvider.getForNodeRole(NodeRole.BROKER); druidNodeDiscovery.registerListener( new DruidNodeDiscovery.Listener() { diff --git a/server/src/test/java/org/apache/druid/client/HttpServerInventoryViewTest.java b/server/src/test/java/org/apache/druid/client/HttpServerInventoryViewTest.java index 5e07b7fb71bf..f0fd54afddfa 100644 --- a/server/src/test/java/org/apache/druid/client/HttpServerInventoryViewTest.java +++ b/server/src/test/java/org/apache/druid/client/HttpServerInventoryViewTest.java @@ -29,7 +29,7 @@ import org.apache.druid.discovery.DiscoveryDruidNode; import org.apache.druid.discovery.DruidNodeDiscovery; import org.apache.druid.discovery.DruidNodeDiscoveryProvider; -import org.apache.druid.discovery.NodeType; +import org.apache.druid.discovery.NodeRole; import org.apache.druid.java.util.common.Intervals; import org.apache.druid.java.util.common.RE; import org.apache.druid.java.util.common.concurrent.Execs; @@ -171,7 +171,7 @@ public void testSimple() throws Exception DiscoveryDruidNode druidNode = new DiscoveryDruidNode( new DruidNode("service", "host", false, 8080, null, true, false), - NodeType.HISTORICAL, + NodeRole.HISTORICAL, ImmutableMap.of( DataNodeService.DISCOVERY_SERVICE_KEY, new DataNodeService("tier", 1000, ServerType.HISTORICAL, 0) ) diff --git a/server/src/test/java/org/apache/druid/curator/discovery/CuratorDruidNodeAnnouncerAndDiscoveryTest.java b/server/src/test/java/org/apache/druid/curator/discovery/CuratorDruidNodeAnnouncerAndDiscoveryTest.java index c2121a74888d..1851cc749614 100644 --- a/server/src/test/java/org/apache/druid/curator/discovery/CuratorDruidNodeAnnouncerAndDiscoveryTest.java +++ b/server/src/test/java/org/apache/druid/curator/discovery/CuratorDruidNodeAnnouncerAndDiscoveryTest.java @@ -27,7 +27,7 @@ import org.apache.druid.curator.announcement.Announcer; import org.apache.druid.discovery.DiscoveryDruidNode; import org.apache.druid.discovery.DruidNodeDiscovery; -import org.apache.druid.discovery.NodeType; +import org.apache.druid.discovery.NodeRole; import org.apache.druid.jackson.DefaultObjectMapper; import org.apache.druid.java.util.common.concurrent.Execs; import org.apache.druid.server.DruidNode; @@ -40,6 +40,7 @@ import java.util.Collection; import java.util.HashSet; import java.util.Set; +import java.util.function.BooleanSupplier; /** * @@ -58,10 +59,11 @@ public void testAnnouncementAndDiscovery() throws Exception ObjectMapper objectMapper = new DefaultObjectMapper(); //additional setup to serde DruidNode - objectMapper.setInjectableValues(new InjectableValues.Std() - .addValue(ServerConfig.class, new ServerConfig()) - .addValue("java.lang.String", "dummy") - .addValue("java.lang.Integer", 1234) + objectMapper.setInjectableValues( + new InjectableValues.Std() + .addValue(ServerConfig.class, new ServerConfig()) + .addValue("java.lang.String", "dummy") + .addValue("java.lang.Integer", 1234) ); curator.start(); @@ -79,32 +81,32 @@ public void testAnnouncementAndDiscovery() throws Exception objectMapper ); - DiscoveryDruidNode node1 = new DiscoveryDruidNode( + DiscoveryDruidNode coordinatorNode1 = new DiscoveryDruidNode( new DruidNode("s1", "h1", false, 8080, null, true, false), - NodeType.COORDINATOR, + NodeRole.COORDINATOR, ImmutableMap.of() ); - DiscoveryDruidNode node2 = new DiscoveryDruidNode( + DiscoveryDruidNode coordinatorNode2 = new DiscoveryDruidNode( new DruidNode("s2", "h2", false, 8080, null, true, false), - NodeType.COORDINATOR, + NodeRole.COORDINATOR, ImmutableMap.of() ); - DiscoveryDruidNode node3 = new DiscoveryDruidNode( + DiscoveryDruidNode overlordNode1 = new DiscoveryDruidNode( new DruidNode("s3", "h3", false, 8080, null, true, false), - NodeType.OVERLORD, + NodeRole.OVERLORD, ImmutableMap.of() ); - DiscoveryDruidNode node4 = new DiscoveryDruidNode( + DiscoveryDruidNode overlordNode2 = new DiscoveryDruidNode( new DruidNode("s4", "h4", false, 8080, null, true, false), - NodeType.OVERLORD, + NodeRole.OVERLORD, ImmutableMap.of() ); - druidNodeAnnouncer.announce(node1); - druidNodeAnnouncer.announce(node3); + druidNodeAnnouncer.announce(coordinatorNode1); + druidNodeAnnouncer.announce(overlordNode1); CuratorDruidNodeDiscoveryProvider druidNodeDiscoveryProvider = new CuratorDruidNodeDiscoveryProvider( curator, @@ -113,84 +115,61 @@ public void testAnnouncementAndDiscovery() throws Exception ); druidNodeDiscoveryProvider.start(); - DruidNodeDiscovery coordDiscovery = druidNodeDiscoveryProvider.getForNodeType(NodeType.COORDINATOR); - DruidNodeDiscovery overlordDiscovery = druidNodeDiscoveryProvider.getForNodeType(NodeType.OVERLORD); + DruidNodeDiscovery coordDiscovery = druidNodeDiscoveryProvider.getForNodeRole(NodeRole.COORDINATOR); + BooleanSupplier coord1NodeDiscovery = + druidNodeDiscoveryProvider.getForNode(coordinatorNode1.getDruidNode(), NodeRole.COORDINATOR); - while (!checkNodes(ImmutableSet.of(node1), coordDiscovery.getAllNodes())) { + DruidNodeDiscovery overlordDiscovery = druidNodeDiscoveryProvider.getForNodeRole(NodeRole.OVERLORD); + BooleanSupplier overlord1NodeDiscovery = + druidNodeDiscoveryProvider.getForNode(overlordNode1.getDruidNode(), NodeRole.OVERLORD); + + while (!checkNodes(ImmutableSet.of(coordinatorNode1), coordDiscovery.getAllNodes()) && + !coord1NodeDiscovery.getAsBoolean()) { Thread.sleep(100); } - while (!checkNodes(ImmutableSet.of(node3), overlordDiscovery.getAllNodes())) { + while (!checkNodes(ImmutableSet.of(overlordNode1), overlordDiscovery.getAllNodes()) && + !overlord1NodeDiscovery.getAsBoolean()) { Thread.sleep(100); } HashSet coordNodes = new HashSet<>(); - coordDiscovery.registerListener( - new DruidNodeDiscovery.Listener() - { - @Override - public void nodesAdded(Collection nodes) - { - coordNodes.addAll(nodes); - } - - @Override - public void nodesRemoved(Collection nodes) - { - coordNodes.removeAll(nodes); - } - } - ); + coordDiscovery.registerListener(createSetAggregatingListener(coordNodes)); HashSet overlordNodes = new HashSet<>(); - overlordDiscovery.registerListener( - new DruidNodeDiscovery.Listener() - { - @Override - public void nodesAdded(Collection nodes) - { - overlordNodes.addAll(nodes); - } - - @Override - public void nodesRemoved(Collection nodes) - { - overlordNodes.removeAll(nodes); - } - } - ); + overlordDiscovery.registerListener(createSetAggregatingListener(overlordNodes)); - while (!checkNodes(ImmutableSet.of(node1), coordNodes)) { + while (!checkNodes(ImmutableSet.of(coordinatorNode1), coordNodes)) { Thread.sleep(100); } - while (!checkNodes(ImmutableSet.of(node3), overlordNodes)) { + while (!checkNodes(ImmutableSet.of(overlordNode1), overlordNodes)) { Thread.sleep(100); } - druidNodeAnnouncer.announce(node2); - druidNodeAnnouncer.announce(node4); + druidNodeAnnouncer.announce(coordinatorNode2); + druidNodeAnnouncer.announce(overlordNode2); - while (!checkNodes(ImmutableSet.of(node1, node2), coordDiscovery.getAllNodes())) { + while (!checkNodes(ImmutableSet.of(coordinatorNode1, coordinatorNode2), coordDiscovery.getAllNodes())) { Thread.sleep(100); } - while (!checkNodes(ImmutableSet.of(node3, node4), overlordDiscovery.getAllNodes())) { + while (!checkNodes(ImmutableSet.of(overlordNode1, overlordNode2), overlordDiscovery.getAllNodes())) { Thread.sleep(100); } - while (!checkNodes(ImmutableSet.of(node1, node2), coordNodes)) { + while (!checkNodes(ImmutableSet.of(coordinatorNode1, coordinatorNode2), coordNodes)) { Thread.sleep(100); } - while (!checkNodes(ImmutableSet.of(node3, node4), overlordNodes)) { + while (!checkNodes(ImmutableSet.of(overlordNode1, overlordNode2), overlordNodes)) { Thread.sleep(100); } - druidNodeAnnouncer.unannounce(node1); - druidNodeAnnouncer.unannounce(node2); - druidNodeAnnouncer.unannounce(node3); - druidNodeAnnouncer.unannounce(node4); + druidNodeAnnouncer.unannounce(coordinatorNode1); + druidNodeAnnouncer.unannounce(coordinatorNode2); + druidNodeAnnouncer.unannounce(overlordNode1); + druidNodeAnnouncer.unannounce(overlordNode2); while (!checkNodes(ImmutableSet.of(), coordDiscovery.getAllNodes())) { Thread.sleep(100); @@ -212,6 +191,24 @@ public void nodesRemoved(Collection nodes) announcer.stop(); } + private static DruidNodeDiscovery.Listener createSetAggregatingListener(Set set) + { + return new DruidNodeDiscovery.Listener() + { + @Override + public void nodesAdded(Collection nodes) + { + set.addAll(nodes); + } + + @Override + public void nodesRemoved(Collection nodes) + { + set.removeAll(nodes); + } + }; + } + private boolean checkNodes(Set expected, Collection actual) { return expected.equals(ImmutableSet.copyOf(actual)); diff --git a/server/src/test/java/org/apache/druid/discovery/DruidLeaderClientTest.java b/server/src/test/java/org/apache/druid/discovery/DruidLeaderClientTest.java index 40576e211f8d..7c376ea20b68 100644 --- a/server/src/test/java/org/apache/druid/discovery/DruidLeaderClientTest.java +++ b/server/src/test/java/org/apache/druid/discovery/DruidLeaderClientTest.java @@ -80,7 +80,7 @@ public class DruidLeaderClientTest extends BaseJettyTest protected Injector setupInjector() { final DruidNode node = new DruidNode("test", "localhost", false, null, null, true, false); - discoveryDruidNode = new DiscoveryDruidNode(node, NodeType.PEON, ImmutableMap.of()); + discoveryDruidNode = new DiscoveryDruidNode(node, NodeRole.PEON, ImmutableMap.of()); Injector injector = Initialization.makeInjectorWithModules( GuiceInjectors.makeStartupInjector(), ImmutableList.of( @@ -115,14 +115,14 @@ public void testSimple() throws Exception ); DruidNodeDiscoveryProvider druidNodeDiscoveryProvider = EasyMock.createMock(DruidNodeDiscoveryProvider.class); - EasyMock.expect(druidNodeDiscoveryProvider.getForNodeType(NodeType.PEON)).andReturn(druidNodeDiscovery); + EasyMock.expect(druidNodeDiscoveryProvider.getForNodeRole(NodeRole.PEON)).andReturn(druidNodeDiscovery); EasyMock.replay(druidNodeDiscovery, druidNodeDiscoveryProvider); DruidLeaderClient druidLeaderClient = new DruidLeaderClient( httpClient, druidNodeDiscoveryProvider, - NodeType.PEON, + NodeRole.PEON, "/simple/leader", EasyMock.createNiceMock(ServerDiscoverySelector.class) ); @@ -140,14 +140,14 @@ public void testNoLeaderFound() throws Exception EasyMock.expect(druidNodeDiscovery.getAllNodes()).andReturn(ImmutableList.of()); DruidNodeDiscoveryProvider druidNodeDiscoveryProvider = EasyMock.createMock(DruidNodeDiscoveryProvider.class); - EasyMock.expect(druidNodeDiscoveryProvider.getForNodeType(NodeType.PEON)).andReturn(druidNodeDiscovery); + EasyMock.expect(druidNodeDiscoveryProvider.getForNodeRole(NodeRole.PEON)).andReturn(druidNodeDiscovery); EasyMock.replay(druidNodeDiscovery, druidNodeDiscoveryProvider); DruidLeaderClient druidLeaderClient = new DruidLeaderClient( httpClient, druidNodeDiscoveryProvider, - NodeType.PEON, + NodeRole.PEON, "/simple/leader", EasyMock.createNiceMock(ServerDiscoverySelector.class) ); @@ -167,14 +167,14 @@ public void testRedirection() throws Exception ); DruidNodeDiscoveryProvider druidNodeDiscoveryProvider = EasyMock.createMock(DruidNodeDiscoveryProvider.class); - EasyMock.expect(druidNodeDiscoveryProvider.getForNodeType(NodeType.PEON)).andReturn(druidNodeDiscovery); + EasyMock.expect(druidNodeDiscoveryProvider.getForNodeRole(NodeRole.PEON)).andReturn(druidNodeDiscovery); EasyMock.replay(druidNodeDiscovery, druidNodeDiscoveryProvider); DruidLeaderClient druidLeaderClient = new DruidLeaderClient( httpClient, druidNodeDiscoveryProvider, - NodeType.PEON, + NodeRole.PEON, "/simple/leader", EasyMock.createNiceMock(ServerDiscoverySelector.class) ); @@ -194,21 +194,21 @@ public void testServerFailureAndRedirect() throws Exception DruidNodeDiscovery druidNodeDiscovery = EasyMock.createMock(DruidNodeDiscovery.class); DiscoveryDruidNode dummyNode = new DiscoveryDruidNode( new DruidNode("test", "dummyhost", false, 64231, null, true, false), - NodeType.PEON, + NodeRole.PEON, ImmutableMap.of() ); EasyMock.expect(druidNodeDiscovery.getAllNodes()).andReturn(ImmutableList.of(dummyNode)); EasyMock.expect(druidNodeDiscovery.getAllNodes()).andReturn(ImmutableList.of(discoveryDruidNode)); DruidNodeDiscoveryProvider druidNodeDiscoveryProvider = EasyMock.createMock(DruidNodeDiscoveryProvider.class); - EasyMock.expect(druidNodeDiscoveryProvider.getForNodeType(NodeType.PEON)).andReturn(druidNodeDiscovery).anyTimes(); + EasyMock.expect(druidNodeDiscoveryProvider.getForNodeRole(NodeRole.PEON)).andReturn(druidNodeDiscovery).anyTimes(); EasyMock.replay(serverDiscoverySelector, druidNodeDiscovery, druidNodeDiscoveryProvider); DruidLeaderClient druidLeaderClient = new DruidLeaderClient( httpClient, druidNodeDiscoveryProvider, - NodeType.PEON, + NodeRole.PEON, "/simple/leader", serverDiscoverySelector ); @@ -228,14 +228,14 @@ public void testFindCurrentLeader() ); DruidNodeDiscoveryProvider druidNodeDiscoveryProvider = EasyMock.createMock(DruidNodeDiscoveryProvider.class); - EasyMock.expect(druidNodeDiscoveryProvider.getForNodeType(NodeType.PEON)).andReturn(druidNodeDiscovery); + EasyMock.expect(druidNodeDiscoveryProvider.getForNodeRole(NodeRole.PEON)).andReturn(druidNodeDiscovery); EasyMock.replay(druidNodeDiscovery, druidNodeDiscoveryProvider); DruidLeaderClient druidLeaderClient = new DruidLeaderClient( httpClient, druidNodeDiscoveryProvider, - NodeType.PEON, + NodeRole.PEON, "/simple/leader", EasyMock.createNiceMock(ServerDiscoverySelector.class) ); diff --git a/server/src/test/java/org/apache/druid/discovery/DruidNodeDiscoveryProviderTest.java b/server/src/test/java/org/apache/druid/discovery/DruidNodeDiscoveryProviderTest.java index 2b5722d67427..64cc54fb517b 100644 --- a/server/src/test/java/org/apache/druid/discovery/DruidNodeDiscoveryProviderTest.java +++ b/server/src/test/java/org/apache/druid/discovery/DruidNodeDiscoveryProviderTest.java @@ -32,6 +32,7 @@ import java.util.HashSet; import java.util.List; import java.util.Set; +import java.util.function.BooleanSupplier; /** */ @@ -88,7 +89,7 @@ public void nodesRemoved(Collection nodes) DiscoveryDruidNode node1 = new DiscoveryDruidNode( new DruidNode("s1", "h1", false, 8080, null, true, false), - NodeType.HISTORICAL, + NodeRole.HISTORICAL, ImmutableMap.of( DataNodeService.DISCOVERY_SERVICE_KEY, new DataNodeService("tier", 1000, ServerType.HISTORICAL, 0), LookupNodeService.DISCOVERY_SERVICE_KEY, new LookupNodeService("tier")) @@ -96,21 +97,21 @@ LookupNodeService.DISCOVERY_SERVICE_KEY, new LookupNodeService("tier")) DiscoveryDruidNode node2 = new DiscoveryDruidNode( new DruidNode("s2", "h2", false, 8080, null, true, false), - NodeType.HISTORICAL, + NodeRole.HISTORICAL, ImmutableMap.of( DataNodeService.DISCOVERY_SERVICE_KEY, new DataNodeService("tier", 1000, ServerType.HISTORICAL, 0)) ); DiscoveryDruidNode node3 = new DiscoveryDruidNode( new DruidNode("s3", "h3", false, 8080, null, true, false), - NodeType.HISTORICAL, + NodeRole.HISTORICAL, ImmutableMap.of( LookupNodeService.DISCOVERY_SERVICE_KEY, new LookupNodeService("tier")) ); DiscoveryDruidNode node4 = new DiscoveryDruidNode( new DruidNode("s4", "h4", false, 8080, null, true, false), - NodeType.PEON, + NodeRole.PEON, ImmutableMap.of( DataNodeService.DISCOVERY_SERVICE_KEY, new DataNodeService("tier", 1000, ServerType.HISTORICAL, 0), LookupNodeService.DISCOVERY_SERVICE_KEY, new LookupNodeService("tier")) @@ -118,35 +119,35 @@ LookupNodeService.DISCOVERY_SERVICE_KEY, new LookupNodeService("tier")) DiscoveryDruidNode node5 = new DiscoveryDruidNode( new DruidNode("s5", "h5", false, 8080, null, true, false), - NodeType.PEON, + NodeRole.PEON, ImmutableMap.of( DataNodeService.DISCOVERY_SERVICE_KEY, new DataNodeService("tier", 1000, ServerType.HISTORICAL, 0)) ); DiscoveryDruidNode node6 = new DiscoveryDruidNode( new DruidNode("s6", "h6", false, 8080, null, true, false), - NodeType.PEON, + NodeRole.PEON, ImmutableMap.of( LookupNodeService.DISCOVERY_SERVICE_KEY, new LookupNodeService("tier")) ); DiscoveryDruidNode node7 = new DiscoveryDruidNode( new DruidNode("s7", "h7", false, 8080, null, true, false), - NodeType.BROKER, + NodeRole.BROKER, ImmutableMap.of( LookupNodeService.DISCOVERY_SERVICE_KEY, new LookupNodeService("tier")) ); DiscoveryDruidNode node7Clone = new DiscoveryDruidNode( new DruidNode("s7", "h7", false, 8080, null, true, false), - NodeType.BROKER, + NodeRole.BROKER, ImmutableMap.of( LookupNodeService.DISCOVERY_SERVICE_KEY, new LookupNodeService("tier")) ); DiscoveryDruidNode node8 = new DiscoveryDruidNode( new DruidNode("s8", "h8", false, 8080, null, true, false), - NodeType.COORDINATOR, + NodeRole.COORDINATOR, ImmutableMap.of() ); @@ -184,7 +185,13 @@ private static class TestDruidNodeDiscoveryProvider extends DruidNodeDiscoveryPr private List listeners = new ArrayList<>(); @Override - public DruidNodeDiscovery getForNodeType(NodeType nodeType) + public BooleanSupplier getForNode(DruidNode node, NodeRole nodeRole) + { + throw new UnsupportedOperationException(); + } + + @Override + public DruidNodeDiscovery getForNodeRole(NodeRole nodeRole) { return new DruidNodeDiscovery() { diff --git a/server/src/test/java/org/apache/druid/server/lookup/cache/LookupNodeDiscoveryTest.java b/server/src/test/java/org/apache/druid/server/lookup/cache/LookupNodeDiscoveryTest.java index be6d607f42e4..66c2493f2c8c 100644 --- a/server/src/test/java/org/apache/druid/server/lookup/cache/LookupNodeDiscoveryTest.java +++ b/server/src/test/java/org/apache/druid/server/lookup/cache/LookupNodeDiscoveryTest.java @@ -26,7 +26,7 @@ import org.apache.druid.discovery.DruidNodeDiscovery; import org.apache.druid.discovery.DruidNodeDiscoveryProvider; import org.apache.druid.discovery.LookupNodeService; -import org.apache.druid.discovery.NodeType; +import org.apache.druid.discovery.NodeRole; import org.apache.druid.server.DruidNode; import org.apache.druid.server.http.HostAndPortWithScheme; import org.easymock.EasyMock; @@ -54,21 +54,21 @@ public void setup() DiscoveryDruidNode node1 = new DiscoveryDruidNode( new DruidNode("s1", "h1", false, 8080, null, true, false), - NodeType.HISTORICAL, + NodeRole.HISTORICAL, ImmutableMap.of( LookupNodeService.DISCOVERY_SERVICE_KEY, new LookupNodeService("tier1")) ); DiscoveryDruidNode node2 = new DiscoveryDruidNode( new DruidNode("s2", "h2", false, 8080, null, true, false), - NodeType.PEON, + NodeRole.PEON, ImmutableMap.of( LookupNodeService.DISCOVERY_SERVICE_KEY, new LookupNodeService("tier1")) ); DiscoveryDruidNode node3 = new DiscoveryDruidNode( new DruidNode("s3", "h3", false, 8080, null, true, false), - NodeType.PEON, + NodeRole.PEON, ImmutableMap.of( LookupNodeService.DISCOVERY_SERVICE_KEY, new LookupNodeService("tier2")) ); diff --git a/server/src/test/java/org/apache/druid/server/router/TieredBrokerHostSelectorTest.java b/server/src/test/java/org/apache/druid/server/router/TieredBrokerHostSelectorTest.java index 81f2c1030326..1ea675fdc3cb 100644 --- a/server/src/test/java/org/apache/druid/server/router/TieredBrokerHostSelectorTest.java +++ b/server/src/test/java/org/apache/druid/server/router/TieredBrokerHostSelectorTest.java @@ -32,7 +32,7 @@ import org.apache.druid.discovery.DiscoveryDruidNode; import org.apache.druid.discovery.DruidNodeDiscovery; import org.apache.druid.discovery.DruidNodeDiscoveryProvider; -import org.apache.druid.discovery.NodeType; +import org.apache.druid.discovery.NodeRole; import org.apache.druid.guice.annotations.Json; import org.apache.druid.java.util.common.Intervals; import org.apache.druid.java.util.common.Pair; @@ -75,19 +75,19 @@ public void setUp() node1 = new DiscoveryDruidNode( new DruidNode("hotBroker", "hotHost", false, 8080, null, true, false), - NodeType.BROKER, + NodeRole.BROKER, ImmutableMap.of() ); node2 = new DiscoveryDruidNode( new DruidNode("coldBroker", "coldHost1", false, 8080, null, true, false), - NodeType.BROKER, + NodeRole.BROKER, ImmutableMap.of() ); node3 = new DiscoveryDruidNode( new DruidNode("coldBroker", "coldHost2", false, 8080, null, true, false), - NodeType.BROKER, + NodeRole.BROKER, ImmutableMap.of() ); @@ -107,7 +107,7 @@ public void registerListener(Listener listener) } }; - EasyMock.expect(druidNodeDiscoveryProvider.getForNodeType(NodeType.BROKER)) + EasyMock.expect(druidNodeDiscoveryProvider.getForNodeRole(NodeRole.BROKER)) .andReturn(druidNodeDiscovery); EasyMock.replay(druidNodeDiscoveryProvider); diff --git a/services/src/main/java/org/apache/druid/cli/CliBroker.java b/services/src/main/java/org/apache/druid/cli/CliBroker.java index 62c03a998d84..c72efc2adb37 100644 --- a/services/src/main/java/org/apache/druid/cli/CliBroker.java +++ b/services/src/main/java/org/apache/druid/cli/CliBroker.java @@ -35,7 +35,7 @@ import org.apache.druid.client.selector.ServerSelectorStrategy; import org.apache.druid.client.selector.TierSelectorStrategy; import org.apache.druid.discovery.LookupNodeService; -import org.apache.druid.discovery.NodeType; +import org.apache.druid.discovery.NodeRole; import org.apache.druid.guice.CacheModule; import org.apache.druid.guice.DruidProcessingModule; import org.apache.druid.guice.Jerseys; @@ -125,12 +125,12 @@ protected List getModules() LifecycleModule.register(binder, Server.class); - binder.bind(NodeType.class).annotatedWith(Self.class).toInstance(NodeType.BROKER); + binder.bind(NodeRole.class).annotatedWith(Self.class).toInstance(NodeRole.BROKER); bindAnnouncer( binder, DiscoverySideEffectsProvider - .builder(NodeType.BROKER) + .builder(NodeRole.BROKER) .serviceClasses(ImmutableList.of(LookupNodeService.class)) .useLegacyAnnouncer(true) .build() diff --git a/services/src/main/java/org/apache/druid/cli/CliCoordinator.java b/services/src/main/java/org/apache/druid/cli/CliCoordinator.java index 5da2555c1619..b224e7899940 100644 --- a/services/src/main/java/org/apache/druid/cli/CliCoordinator.java +++ b/services/src/main/java/org/apache/druid/cli/CliCoordinator.java @@ -36,7 +36,7 @@ import org.apache.druid.client.coordinator.Coordinator; import org.apache.druid.client.indexing.HttpIndexingServiceClient; import org.apache.druid.client.indexing.IndexingServiceClient; -import org.apache.druid.discovery.NodeType; +import org.apache.druid.discovery.NodeRole; import org.apache.druid.guice.ConditionalMultibind; import org.apache.druid.guice.ConfigProvider; import org.apache.druid.guice.Jerseys; @@ -234,12 +234,12 @@ public void configure(Binder binder) DruidCoordinatorCleanupPendingSegments.class ); - binder.bind(NodeType.class).annotatedWith(Self.class).toInstance(NodeType.COORDINATOR); + binder.bind(NodeRole.class).annotatedWith(Self.class).toInstance(NodeRole.COORDINATOR); bindAnnouncer( binder, Coordinator.class, - DiscoverySideEffectsProvider.builder(NodeType.COORDINATOR).build() + DiscoverySideEffectsProvider.builder(NodeRole.COORDINATOR).build() ); Jerseys.addResource(binder, SelfDiscoveryResource.class); diff --git a/services/src/main/java/org/apache/druid/cli/CliHistorical.java b/services/src/main/java/org/apache/druid/cli/CliHistorical.java index 38a0601c6160..5ebee14b7383 100644 --- a/services/src/main/java/org/apache/druid/cli/CliHistorical.java +++ b/services/src/main/java/org/apache/druid/cli/CliHistorical.java @@ -28,7 +28,7 @@ import org.apache.druid.client.cache.CacheMonitor; import org.apache.druid.discovery.DataNodeService; import org.apache.druid.discovery.LookupNodeService; -import org.apache.druid.discovery.NodeType; +import org.apache.druid.discovery.NodeRole; import org.apache.druid.guice.CacheModule; import org.apache.druid.guice.DruidProcessingModule; import org.apache.druid.guice.Jerseys; @@ -105,12 +105,12 @@ protected List getModules() binder.install(new CacheModule()); MetricsModule.register(binder, CacheMonitor.class); - binder.bind(NodeType.class).annotatedWith(Self.class).toInstance(NodeType.HISTORICAL); + binder.bind(NodeRole.class).annotatedWith(Self.class).toInstance(NodeRole.HISTORICAL); bindAnnouncer( binder, DiscoverySideEffectsProvider - .builder(NodeType.HISTORICAL) + .builder(NodeRole.HISTORICAL) .serviceClasses(ImmutableList.of(DataNodeService.class, LookupNodeService.class)) .build() ); diff --git a/services/src/main/java/org/apache/druid/cli/CliMiddleManager.java b/services/src/main/java/org/apache/druid/cli/CliMiddleManager.java index 9d3a4c850247..622f15beb055 100644 --- a/services/src/main/java/org/apache/druid/cli/CliMiddleManager.java +++ b/services/src/main/java/org/apache/druid/cli/CliMiddleManager.java @@ -30,7 +30,7 @@ import com.google.inject.util.Providers; import io.airlift.airline.Command; import org.apache.druid.client.indexing.IndexingServiceClient; -import org.apache.druid.discovery.NodeType; +import org.apache.druid.discovery.NodeRole; import org.apache.druid.discovery.WorkerNodeService; import org.apache.druid.guice.IndexingServiceFirehoseModule; import org.apache.druid.guice.IndexingServiceModuleHelper; @@ -131,12 +131,12 @@ public void configure(Binder binder) LifecycleModule.register(binder, Server.class); - binder.bind(NodeType.class).annotatedWith(Self.class).toInstance(NodeType.MIDDLE_MANAGER); + binder.bind(NodeRole.class).annotatedWith(Self.class).toInstance(NodeRole.MIDDLE_MANAGER); bindAnnouncer( binder, DiscoverySideEffectsProvider - .builder(NodeType.MIDDLE_MANAGER) + .builder(NodeRole.MIDDLE_MANAGER) .serviceClasses(ImmutableList.of(WorkerNodeService.class)) .build() ); diff --git a/services/src/main/java/org/apache/druid/cli/CliOverlord.java b/services/src/main/java/org/apache/druid/cli/CliOverlord.java index 1aa79bf0a362..7b1972f6411a 100644 --- a/services/src/main/java/org/apache/druid/cli/CliOverlord.java +++ b/services/src/main/java/org/apache/druid/cli/CliOverlord.java @@ -38,7 +38,7 @@ import org.apache.druid.client.indexing.IndexingService; import org.apache.druid.client.indexing.IndexingServiceClient; import org.apache.druid.client.indexing.IndexingServiceSelectorConfig; -import org.apache.druid.discovery.NodeType; +import org.apache.druid.discovery.NodeRole; import org.apache.druid.guice.IndexingServiceFirehoseModule; import org.apache.druid.guice.IndexingServiceModuleHelper; import org.apache.druid.guice.IndexingServiceTaskLogsModule; @@ -241,12 +241,12 @@ public void configure(Binder binder) LifecycleModule.register(binder, Server.class); } - binder.bind(NodeType.class).annotatedWith(Self.class).toInstance(NodeType.OVERLORD); + binder.bind(NodeRole.class).annotatedWith(Self.class).toInstance(NodeRole.OVERLORD); bindAnnouncer( binder, IndexingService.class, - DiscoverySideEffectsProvider.builder(NodeType.OVERLORD).build() + DiscoverySideEffectsProvider.builder(NodeRole.OVERLORD).build() ); Jerseys.addResource(binder, SelfDiscoveryResource.class); diff --git a/services/src/main/java/org/apache/druid/cli/CliPeon.java b/services/src/main/java/org/apache/druid/cli/CliPeon.java index 8500ebf347a6..d6ea9e1ce53d 100644 --- a/services/src/main/java/org/apache/druid/cli/CliPeon.java +++ b/services/src/main/java/org/apache/druid/cli/CliPeon.java @@ -40,7 +40,7 @@ import org.apache.druid.client.coordinator.CoordinatorClient; import org.apache.druid.client.indexing.HttpIndexingServiceClient; import org.apache.druid.client.indexing.IndexingServiceClient; -import org.apache.druid.discovery.NodeType; +import org.apache.druid.discovery.NodeRole; import org.apache.druid.guice.Binders; import org.apache.druid.guice.CacheModule; import org.apache.druid.guice.DruidProcessingModule; @@ -290,7 +290,7 @@ private void configureTaskActionClient(Binder binder) .to(RemoteTaskActionClientFactory.class) .in(LazySingleton.class); - binder.bind(NodeType.class).annotatedWith(Self.class).toInstance(NodeType.PEON); + binder.bind(NodeRole.class).annotatedWith(Self.class).toInstance(NodeRole.PEON); } @Provides diff --git a/services/src/main/java/org/apache/druid/cli/CliRouter.java b/services/src/main/java/org/apache/druid/cli/CliRouter.java index 2ad76c23ab24..40cd46b691ca 100644 --- a/services/src/main/java/org/apache/druid/cli/CliRouter.java +++ b/services/src/main/java/org/apache/druid/cli/CliRouter.java @@ -27,7 +27,7 @@ import com.google.inject.name.Names; import io.airlift.airline.Command; import org.apache.druid.curator.discovery.DiscoveryModule; -import org.apache.druid.discovery.NodeType; +import org.apache.druid.discovery.NodeRole; import org.apache.druid.guice.Jerseys; import org.apache.druid.guice.JsonConfigProvider; import org.apache.druid.guice.LazySingleton; @@ -114,11 +114,11 @@ public void configure(Binder binder) LifecycleModule.register(binder, Server.class); DiscoveryModule.register(binder, Self.class); - binder.bind(NodeType.class).annotatedWith(Self.class).toInstance(NodeType.ROUTER); + binder.bind(NodeRole.class).annotatedWith(Self.class).toInstance(NodeRole.ROUTER); bindAnnouncer( binder, - DiscoverySideEffectsProvider.builder(NodeType.ROUTER).build() + DiscoverySideEffectsProvider.builder(NodeRole.ROUTER).build() ); Jerseys.addResource(binder, SelfDiscoveryResource.class); diff --git a/services/src/main/java/org/apache/druid/cli/ServerRunnable.java b/services/src/main/java/org/apache/druid/cli/ServerRunnable.java index 9aa0e2b32cf1..4fda45b9902c 100644 --- a/services/src/main/java/org/apache/druid/cli/ServerRunnable.java +++ b/services/src/main/java/org/apache/druid/cli/ServerRunnable.java @@ -30,7 +30,7 @@ import org.apache.druid.discovery.DiscoveryDruidNode; import org.apache.druid.discovery.DruidNodeAnnouncer; import org.apache.druid.discovery.DruidService; -import org.apache.druid.discovery.NodeType; +import org.apache.druid.discovery.NodeRole; import org.apache.druid.guice.LazySingleton; import org.apache.druid.guice.LifecycleModule; import org.apache.druid.guice.annotations.Self; @@ -100,13 +100,13 @@ public static class Child {} public static class Builder { - private NodeType nodeType; + private NodeRole nodeRole; private List> serviceClasses = ImmutableList.of(); private boolean useLegacyAnnouncer; - public Builder(final NodeType nodeType) + public Builder(final NodeRole nodeRole) { - this.nodeType = nodeType; + this.nodeRole = nodeRole; } public Builder serviceClasses(final List> serviceClasses) @@ -123,13 +123,13 @@ public Builder useLegacyAnnouncer(final boolean useLegacyAnnouncer) public DiscoverySideEffectsProvider build() { - return new DiscoverySideEffectsProvider(nodeType, serviceClasses, useLegacyAnnouncer); + return new DiscoverySideEffectsProvider(nodeRole, serviceClasses, useLegacyAnnouncer); } } - public static Builder builder(final NodeType nodeType) + public static Builder builder(final NodeRole nodeRole) { - return new Builder(nodeType); + return new Builder(nodeRole); } @Inject @@ -148,17 +148,17 @@ public static Builder builder(final NodeType nodeType) @Inject private Injector injector; - private final NodeType nodeType; + private final NodeRole nodeRole; private final List> serviceClasses; private final boolean useLegacyAnnouncer; private DiscoverySideEffectsProvider( - final NodeType nodeType, + final NodeRole nodeRole, final List> serviceClasses, final boolean useLegacyAnnouncer ) { - this.nodeType = nodeType; + this.nodeRole = nodeRole; this.serviceClasses = serviceClasses; this.useLegacyAnnouncer = useLegacyAnnouncer; } @@ -172,7 +172,7 @@ public Child get() builder.put(service.getName(), service); } - DiscoveryDruidNode discoveryDruidNode = new DiscoveryDruidNode(druidNode, nodeType, builder.build()); + DiscoveryDruidNode discoveryDruidNode = new DiscoveryDruidNode(druidNode, nodeRole, builder.build()); lifecycle.addHandler( new Lifecycle.Handler() diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/util/CalciteTests.java b/sql/src/test/java/org/apache/druid/sql/calcite/util/CalciteTests.java index d692d1b9aadd..29df845cc85c 100644 --- a/sql/src/test/java/org/apache/druid/sql/calcite/util/CalciteTests.java +++ b/sql/src/test/java/org/apache/druid/sql/calcite/util/CalciteTests.java @@ -45,7 +45,7 @@ import org.apache.druid.data.input.impl.TimestampSpec; import org.apache.druid.discovery.DruidLeaderClient; import org.apache.druid.discovery.DruidNodeDiscoveryProvider; -import org.apache.druid.discovery.NodeType; +import org.apache.druid.discovery.NodeRole; import org.apache.druid.guice.ExpressionModule; import org.apache.druid.guice.annotations.Json; import org.apache.druid.java.util.common.Pair; @@ -747,7 +747,7 @@ public static SystemSchema createMockSystemSchema( final DruidLeaderClient druidLeaderClient = new DruidLeaderClient( EasyMock.createMock(HttpClient.class), EasyMock.createMock(DruidNodeDiscoveryProvider.class), - NodeType.COORDINATOR, + NodeRole.COORDINATOR, "/simple/leader", new ServerDiscoverySelector(EasyMock.createMock(ServiceProvider.class), "test") ) From 7cb1048590162db2c796591b03b73bbaa6c42d11 Mon Sep 17 00:00:00 2001 From: Roman Leventov Date: Mon, 15 Apr 2019 15:30:05 +0200 Subject: [PATCH 03/14] Extended docs --- docs/content/operations/api-reference.md | 8 +++++++- 1 file changed, 7 insertions(+), 1 deletion(-) diff --git a/docs/content/operations/api-reference.md b/docs/content/operations/api-reference.md index 728da9478c43..ab5bfca53f94 100644 --- a/docs/content/operations/api-reference.md +++ b/docs/content/operations/api-reference.md @@ -63,7 +63,13 @@ Returns the current configuration properties of the process. Returns a JSON map of the form `{"selfDiscovered": true/false}`, indicating whether the node has recieved a confirmation from the central node discovery mechanism (currently ZooKeeper) of the Druid cluster that the node has been added to the cluster. It is recommended to not consider a Druid node "healthy" or "ready" in automated deployment/container -management systems until it returns `{"selfDiscovered": true}` from this endpoint. +management systems until it returns `{"selfDiscovered": true}` from this endpoint. This is because a node may be +isolated from the rest of the cluster due to network issues and it doesn't make sense to consider nodes "healthy" in +this case. Also, when nodes such as Brokers use ZooKeeper segment discovery for building their view of the Druid cluster +(as opposed to HTTP segment discovery), they may be unusable until the ZooKeeper client is fully initialized and starts +to receive data from the ZooKeeper cluster. `{"selfDiscovered": true}` is a proxy event indicating that the ZooKeeper +client on the node has started to receive data from the ZooKeeper cluster and it's expected that all segments and other +nodes will be discovered by this node timely from this point. ## Master Server From 679a303e2495a11dc245388abe5ea0b985cd783b Mon Sep 17 00:00:00 2001 From: Roman Leventov Date: Wed, 17 Apr 2019 15:30:02 +0200 Subject: [PATCH 04/14] Fix brace --- .../curator/discovery/CuratorDruidNodeDiscoveryProvider.java | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/server/src/main/java/org/apache/druid/curator/discovery/CuratorDruidNodeDiscoveryProvider.java b/server/src/main/java/org/apache/druid/curator/discovery/CuratorDruidNodeDiscoveryProvider.java index 40ce93c55b57..45e87e6e4607 100644 --- a/server/src/main/java/org/apache/druid/curator/discovery/CuratorDruidNodeDiscoveryProvider.java +++ b/server/src/main/java/org/apache/druid/curator/discovery/CuratorDruidNodeDiscoveryProvider.java @@ -423,7 +423,8 @@ private byte[] getZkDataForNode(ChildData child) } @GuardedBy("lock") - private void cacheInitialized() { + private void cacheInitialized() + { // No need to wait on CountDownLatch, because we are holding the lock under which it could only be // counted down. if (cacheInitialized.getCount() == 0) { From 72da2286157a7e5f36b1645c070a3dd2bd0a536a Mon Sep 17 00:00:00 2001 From: Roman Leventov Date: Wed, 17 Apr 2019 15:38:27 +0200 Subject: [PATCH 05/14] Remove redundant throws in Lifecycle.Handler.stop() --- .../org/apache/druid/java/util/common/lifecycle/Lifecycle.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/java/org/apache/druid/java/util/common/lifecycle/Lifecycle.java b/core/src/main/java/org/apache/druid/java/util/common/lifecycle/Lifecycle.java index 867bddf84703..b168fbf5bc14 100644 --- a/core/src/main/java/org/apache/druid/java/util/common/lifecycle/Lifecycle.java +++ b/core/src/main/java/org/apache/druid/java/util/common/lifecycle/Lifecycle.java @@ -416,7 +416,7 @@ public interface Handler { void start() throws Exception; - void stop() throws Exception; + void stop(); } private static class AnnotationBasedHandler implements Handler From 7f07dca911b7486e10a69b2e96749cc9ef124fdc Mon Sep 17 00:00:00 2001 From: Roman Leventov Date: Tue, 28 May 2019 15:36:01 +0200 Subject: [PATCH 06/14] Import order --- services/src/main/java/org/apache/druid/cli/CliHistorical.java | 2 +- services/src/main/java/org/apache/druid/cli/CliPeon.java | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/services/src/main/java/org/apache/druid/cli/CliHistorical.java b/services/src/main/java/org/apache/druid/cli/CliHistorical.java index 922717d1a13a..fff0d17cbd91 100644 --- a/services/src/main/java/org/apache/druid/cli/CliHistorical.java +++ b/services/src/main/java/org/apache/druid/cli/CliHistorical.java @@ -36,9 +36,9 @@ import org.apache.druid.guice.LazySingleton; import org.apache.druid.guice.LifecycleModule; import org.apache.druid.guice.ManageLifecycle; -import org.apache.druid.guice.ServerTypeConfig; import org.apache.druid.guice.QueryRunnerFactoryModule; import org.apache.druid.guice.QueryableModule; +import org.apache.druid.guice.ServerTypeConfig; import org.apache.druid.guice.annotations.Self; import org.apache.druid.java.util.common.logger.Logger; import org.apache.druid.query.QuerySegmentWalker; diff --git a/services/src/main/java/org/apache/druid/cli/CliPeon.java b/services/src/main/java/org/apache/druid/cli/CliPeon.java index a87cee0a11a0..b90faca94784 100644 --- a/services/src/main/java/org/apache/druid/cli/CliPeon.java +++ b/services/src/main/java/org/apache/druid/cli/CliPeon.java @@ -50,11 +50,11 @@ import org.apache.druid.guice.LazySingleton; import org.apache.druid.guice.LifecycleModule; import org.apache.druid.guice.ManageLifecycle; -import org.apache.druid.guice.ServerTypeConfig; import org.apache.druid.guice.PolyBind; import org.apache.druid.guice.QueryRunnerFactoryModule; import org.apache.druid.guice.QueryableModule; import org.apache.druid.guice.QueryablePeonModule; +import org.apache.druid.guice.ServerTypeConfig; import org.apache.druid.guice.annotations.Json; import org.apache.druid.guice.annotations.Self; import org.apache.druid.guice.annotations.Smile; From 4eed8d80f258eec134b7786983856f08d58d83ac Mon Sep 17 00:00:00 2001 From: Roman Leventov Date: Wed, 29 May 2019 08:47:49 +0200 Subject: [PATCH 07/14] Remove unresolvable link --- .../apache/druid/java/util/common/lifecycle/Lifecycle.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/core/src/main/java/org/apache/druid/java/util/common/lifecycle/Lifecycle.java b/core/src/main/java/org/apache/druid/java/util/common/lifecycle/Lifecycle.java index b168fbf5bc14..74b74e1fb8d7 100644 --- a/core/src/main/java/org/apache/druid/java/util/common/lifecycle/Lifecycle.java +++ b/core/src/main/java/org/apache/druid/java/util/common/lifecycle/Lifecycle.java @@ -58,8 +58,8 @@ * doesn't need logging during start or stop). * - {@link Stage#NORMAL}: This is the default stage. Most objects will probably make the most sense to be registered * at this level, with the exception of any form of server or service announcements - * - {@link Stage#SERVER}: This lifecycle stage is intended for all 'server' objects, for example, {@link - * org.apache.druid.server.initialization.jetty.JettyServerModule}, but any sort of 'server' that expects most (or + * - {@link Stage#SERVER}: This lifecycle stage is intended for all 'server' objects, for example, + * org.apache.druid.server.initialization.jetty.JettyServerModule, but any sort of 'server' that expects most (or * some specific) Lifecycle objects to be initialized by the time it starts, and still available at the time it stops * can logically live in this stage. * - {@link Stage#ANNOUNCEMENTS}: Any object which announces to a cluster this servers location belongs in this stage. From bd01a6bc4f5b46221c809c4ff91f4699bc9d1249 Mon Sep 17 00:00:00 2001 From: Roman Leventov Date: Thu, 30 May 2019 18:45:05 +0200 Subject: [PATCH 08/14] Address comments --- docs/content/operations/api-reference.md | 8 +++++++- .../server/http/SelfDiscoveryResource.java | 19 +++++++++++++++++-- 2 files changed, 24 insertions(+), 3 deletions(-) diff --git a/docs/content/operations/api-reference.md b/docs/content/operations/api-reference.md index 7d2e540e8c2f..865b5daa8677 100644 --- a/docs/content/operations/api-reference.md +++ b/docs/content/operations/api-reference.md @@ -58,7 +58,7 @@ An endpoint that always returns a boolean "true" value with a 200 OK response, u Returns the current configuration properties of the process. -* `/selfDiscovered` +* `/status/selfDiscoveredStatus` Returns a JSON map of the form `{"selfDiscovered": true/false}`, indicating whether the node has recieved a confirmation from the central node discovery mechanism (currently ZooKeeper) of the Druid cluster that the node has been added to the @@ -71,6 +71,12 @@ to receive data from the ZooKeeper cluster. `{"selfDiscovered": true}` is a prox client on the node has started to receive data from the ZooKeeper cluster and it's expected that all segments and other nodes will be discovered by this node timely from this point. +* `/status/selfDiscovered` + +Similar to `/status/selfDiscoveredStatus`, but returns 200 OK response with empty body if the node has discovered itself +and 503 SERVICE UNAVAILABLE if the node hasn't discovered itself yet. This endpoint might be useful because some +monitoring checks such as AWS load balancer health checks are not able to look at the response body. + ## Master Server This section documents the API endpoints for the processes that reside on Master servers (Coordinators and Overlords) diff --git a/server/src/main/java/org/apache/druid/server/http/SelfDiscoveryResource.java b/server/src/main/java/org/apache/druid/server/http/SelfDiscoveryResource.java index 6fab65905c95..ff58150434f6 100644 --- a/server/src/main/java/org/apache/druid/server/http/SelfDiscoveryResource.java +++ b/server/src/main/java/org/apache/druid/server/http/SelfDiscoveryResource.java @@ -28,6 +28,7 @@ import org.apache.druid.java.util.common.lifecycle.Lifecycle; import org.apache.druid.server.DruidNode; import org.apache.druid.server.http.security.StateResourceFilter; +import org.eclipse.jetty.http.HttpStatus; import javax.ws.rs.GET; import javax.ws.rs.Path; @@ -43,7 +44,6 @@ * DI configuration phase. */ @Singleton -@Path("/selfDiscovered") @ResourceFilters(StateResourceFilter.class) public class SelfDiscoveryResource { @@ -76,10 +76,25 @@ public void stop() lifecycle.addHandler(selfDiscoveryListenerRegistrator, Lifecycle.Stage.SERVER); } + /** See the description of this endpoint in api-reference.md. */ @GET + @Path("/status/selfDiscoveredStatus") @Produces(MediaType.APPLICATION_JSON) - public Response getSelfDiscovered() + public Response getSelfDiscoveredStatus() { return Response.ok(Collections.singletonMap("selfDiscovered", selfDiscovered.getAsBoolean())).build(); } + + /** See the description of this endpoint in api-reference.md. */ + @GET + @Path("/status/selfDiscovered") + @Produces(MediaType.APPLICATION_JSON) + public Response getSelfDiscovered() + { + if (selfDiscovered.getAsBoolean()) { + return Response.ok().build(); + } else { + return Response.status(HttpStatus.SERVICE_UNAVAILABLE_503).build(); + } + } } From f0a3c550020d86791c2e2c7c95be846cf5380dc1 Mon Sep 17 00:00:00 2001 From: Roman Leventov Date: Thu, 6 Jun 2019 15:48:31 +0200 Subject: [PATCH 09/14] tmp --- .../util/common/jackson/JacksonUtils.java | 4 + .../response/StatusResponseHandler.java | 1 + .../AbstractQueryResourceTestClient.java | 70 ++++++++++++++- .../apache/druid/tests/SelfDiscoveryTest.java | 80 +++++++++++++++++ .../ITBasicAuthConfigurationTest.java | 86 ++----------------- .../server/http/SelfDiscoveryResource.java | 6 +- 6 files changed, 163 insertions(+), 84 deletions(-) create mode 100644 integration-tests/src/test/java/org/apache/druid/tests/SelfDiscoveryTest.java diff --git a/core/src/main/java/org/apache/druid/java/util/common/jackson/JacksonUtils.java b/core/src/main/java/org/apache/druid/java/util/common/jackson/JacksonUtils.java index ee8bf660bd27..4798b8a2177d 100644 --- a/core/src/main/java/org/apache/druid/java/util/common/jackson/JacksonUtils.java +++ b/core/src/main/java/org/apache/druid/java/util/common/jackson/JacksonUtils.java @@ -31,4 +31,8 @@ public class JacksonUtils public static final TypeReference> TYPE_REFERENCE_MAP_STRING_STRING = new TypeReference>() { }; + + public static final TypeReference TYPE_REFERENCE_MAP_STRING_BOOLEAN = new TypeReference>() + { + }; } diff --git a/core/src/main/java/org/apache/druid/java/util/http/client/response/StatusResponseHandler.java b/core/src/main/java/org/apache/druid/java/util/http/client/response/StatusResponseHandler.java index 594acd22e057..21291a2dcfd6 100644 --- a/core/src/main/java/org/apache/druid/java/util/http/client/response/StatusResponseHandler.java +++ b/core/src/main/java/org/apache/druid/java/util/http/client/response/StatusResponseHandler.java @@ -25,6 +25,7 @@ import java.nio.charset.Charset; /** + * Can be singleton: https://github.com/apache/incubator-druid/issues/7833 */ public class StatusResponseHandler implements HttpResponseHandler { diff --git a/integration-tests/src/main/java/org/apache/druid/testing/clients/AbstractQueryResourceTestClient.java b/integration-tests/src/main/java/org/apache/druid/testing/clients/AbstractQueryResourceTestClient.java index c1202445f671..4383f186dbf4 100644 --- a/integration-tests/src/main/java/org/apache/druid/testing/clients/AbstractQueryResourceTestClient.java +++ b/integration-tests/src/main/java/org/apache/druid/testing/clients/AbstractQueryResourceTestClient.java @@ -23,6 +23,8 @@ import com.fasterxml.jackson.databind.ObjectMapper; import com.google.inject.Inject; import org.apache.druid.java.util.common.ISE; +import org.apache.druid.java.util.common.StringUtils; +import org.apache.druid.java.util.common.logger.Logger; import org.apache.druid.java.util.http.client.HttpClient; import org.apache.druid.java.util.http.client.Request; import org.apache.druid.java.util.http.client.response.StatusResponseHandler; @@ -32,6 +34,8 @@ import org.jboss.netty.handler.codec.http.HttpMethod; import org.jboss.netty.handler.codec.http.HttpResponseStatus; +import javax.annotation.Nullable; +import javax.ws.rs.core.MediaType; import java.net.URL; import java.nio.charset.StandardCharsets; import java.util.List; @@ -39,10 +43,12 @@ public abstract class AbstractQueryResourceTestClient { + private static final Logger LOG = new Logger(AbstractQueryResourceTestClient.class); + private static final StatusResponseHandler RESPONSE_HANDLER = new StatusResponseHandler(StandardCharsets.UTF_8); + private final ObjectMapper jsonMapper; private final HttpClient httpClient; - protected final String routerUrl; - private final StatusResponseHandler responseHandler; + final String routerUrl; @Inject AbstractQueryResourceTestClient( @@ -54,7 +60,6 @@ public abstract class AbstractQueryResourceTestClient this.jsonMapper = jsonMapper; this.httpClient = httpClient; this.routerUrl = config.getRouterUrl(); - this.responseHandler = new StatusResponseHandler(StandardCharsets.UTF_8); } public abstract String getBrokerURL(); @@ -66,7 +71,7 @@ public List> query(String url, QueryType query) new Request(HttpMethod.POST, new URL(url)).setContent( "application/json", jsonMapper.writeValueAsBytes(query) - ), responseHandler + ), RESPONSE_HANDLER ).get(); @@ -90,4 +95,61 @@ public List> query(String url, QueryType query) } } + public static StatusResponseHolder makeRequest(HttpClient httpClient, HttpMethod method, String url, byte[] content) + { + return makeRequestWithExpectedStatus( + httpClient, + method, + url, + content, + HttpResponseStatus.OK + ); + } + + public static StatusResponseHolder makeRequestWithExpectedStatus( + HttpClient httpClient, + HttpMethod method, + String url, + @Nullable byte[] content, + HttpResponseStatus expectedStatus + ) + { + try { + Request request = new Request(method, new URL(url)); + if (content != null) { + request.setContent(MediaType.APPLICATION_JSON, content); + } + int retryCount = 0; + + StatusResponseHolder response; + + while (true) { + response = httpClient.go(request, RESPONSE_HANDLER).get(); + + if (!response.getStatus().equals(expectedStatus)) { + String errMsg = StringUtils.format( + "Error while making request to url[%s] status[%s] content[%s]", + url, + response.getStatus(), + response.getContent() + ); + // it can take time for the auth config to propagate, so we retry + if (retryCount > 10) { + throw new ISE(errMsg); + } else { + LOG.error(errMsg); + LOG.error("retrying in 3000ms, retryCount: " + retryCount); + retryCount++; + Thread.sleep(3000); + } + } else { + break; + } + } + return response; + } + catch (Exception e) { + throw new RuntimeException(e); + } + } } diff --git a/integration-tests/src/test/java/org/apache/druid/tests/SelfDiscoveryTest.java b/integration-tests/src/test/java/org/apache/druid/tests/SelfDiscoveryTest.java new file mode 100644 index 000000000000..86dcf2ef6a98 --- /dev/null +++ b/integration-tests/src/test/java/org/apache/druid/tests/SelfDiscoveryTest.java @@ -0,0 +1,80 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.tests; + +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.inject.Inject; +import org.apache.druid.guice.annotations.Client; +import org.apache.druid.java.util.common.jackson.JacksonUtils; +import org.apache.druid.java.util.common.logger.Logger; +import org.apache.druid.java.util.http.client.HttpClient; +import org.apache.druid.java.util.http.client.response.StatusResponseHolder; +import org.apache.druid.testing.IntegrationTestingConfig; +import org.apache.druid.testing.clients.AbstractQueryResourceTestClient; +import org.apache.druid.testing.guice.DruidTestModuleFactory; +import org.apache.druid.testing.utils.RetryUtil; +import org.jboss.netty.handler.codec.http.HttpMethod; +import org.jboss.netty.handler.codec.http.HttpResponseStatus; +import org.testng.annotations.Guice; +import org.testng.annotations.Test; + +import java.util.Map; + +@Guice(moduleFactory = DruidTestModuleFactory.class) +public class SelfDiscoveryTest +{ + private static final Logger LOG = new Logger(SelfDiscoveryTest.class); + + @Inject + IntegrationTestingConfig config; + + @Inject + ObjectMapper jsonMapper; + + @Inject + @Client + HttpClient httpClient; + + @Test + public void testHistorical() + { + RetryUtil.retryUntilTrue(() -> selfDiscovered(config.getHistoricalUrl()), "Historical self-discovered"); + } + + private boolean selfDiscovered(String nodeUrl) + { + StatusResponseHolder response = AbstractQueryResourceTestClient.makeRequestWithExpectedStatus( + httpClient, + HttpMethod.GET, + nodeUrl + "/status/selfDiscoveredStatus", + null, + HttpResponseStatus.OK + ); + try { + Map selfDiscoveredStatus = + jsonMapper.readValue(response.getContent(), JacksonUtils.TYPE_REFERENCE_MAP_STRING_BOOLEAN); + return selfDiscoveredStatus.get("selfDiscovered"); + } + catch (Exception e) { + LOG.warn(e, "Failed to probe selfDiscoveryStatus on %s", nodeUrl); + return false; + } + } +} diff --git a/integration-tests/src/test/java/org/apache/druid/tests/security/ITBasicAuthConfigurationTest.java b/integration-tests/src/test/java/org/apache/druid/tests/security/ITBasicAuthConfigurationTest.java index 5913d7c70adc..1f5c11425de1 100644 --- a/integration-tests/src/test/java/org/apache/druid/tests/security/ITBasicAuthConfigurationTest.java +++ b/integration-tests/src/test/java/org/apache/druid/tests/security/ITBasicAuthConfigurationTest.java @@ -27,14 +27,12 @@ import com.google.inject.Inject; import org.apache.calcite.avatica.AvaticaSqlException; import org.apache.druid.guice.annotations.Client; -import org.apache.druid.java.util.common.ISE; import org.apache.druid.java.util.common.StringUtils; +import org.apache.druid.java.util.common.jackson.JacksonUtils; import org.apache.druid.java.util.common.logger.Logger; import org.apache.druid.java.util.http.client.CredentialedHttpClient; import org.apache.druid.java.util.http.client.HttpClient; -import org.apache.druid.java.util.http.client.Request; import org.apache.druid.java.util.http.client.auth.BasicCredentials; -import org.apache.druid.java.util.http.client.response.StatusResponseHandler; import org.apache.druid.java.util.http.client.response.StatusResponseHolder; import org.apache.druid.security.basic.authentication.entity.BasicAuthenticatorCredentialUpdate; import org.apache.druid.server.security.Action; @@ -54,9 +52,6 @@ import org.testng.annotations.Guice; import org.testng.annotations.Test; -import javax.ws.rs.core.MediaType; -import java.net.URL; -import java.nio.charset.StandardCharsets; import java.sql.Connection; import java.sql.DriverManager; import java.sql.ResultSet; @@ -68,16 +63,14 @@ import java.util.Properties; import java.util.stream.Collectors; +import static org.apache.druid.testing.clients.AbstractQueryResourceTestClient.makeRequest; +import static org.apache.druid.testing.clients.AbstractQueryResourceTestClient.makeRequestWithExpectedStatus; + @Guice(moduleFactory = DruidTestModuleFactory.class) public class ITBasicAuthConfigurationTest { private static final Logger LOG = new Logger(ITBasicAuthConfigurationTest.class); - private static final TypeReference LOAD_STATUS_TYPE_REFERENCE = - new TypeReference>() - { - }; - private static final TypeReference SYS_SCHEMA_RESULTS_TYPE_REFERENCE = new TypeReference>>() { @@ -114,8 +107,6 @@ public class ITBasicAuthConfigurationTest @Client HttpClient httpClient; - StatusResponseHandler responseHandler = new StatusResponseHandler(StandardCharsets.UTF_8); - @Inject private CoordinatorResourceTestClient coordinatorClient; @@ -525,8 +516,8 @@ private void testAvaticaAuthFailure(String url) throws Exception LOG.info("URL: " + url); try { Properties connectionProperties = new Properties(); - connectionProperties.put("user", "admin"); - connectionProperties.put("password", "wrongpassword"); + connectionProperties.setProperty("user", "admin"); + connectionProperties.setProperty("password", "wrongpassword"); Connection connection = DriverManager.getConnection(url, connectionProperties); Statement statement = connection.createStatement(); statement.setMaxRows(450); @@ -571,7 +562,7 @@ private void checkLoadStatusSingle(HttpClient httpClient, String baseUrl) throws null ); String content = holder.getContent(); - Map loadStatus = jsonMapper.readValue(content, LOAD_STATUS_TYPE_REFERENCE); + Map loadStatus = jsonMapper.readValue(content, JacksonUtils.TYPE_REFERENCE_MAP_STRING_BOOLEAN); Assert.assertNotNull(loadStatus.get("basic")); Assert.assertTrue(loadStatus.get("basic")); @@ -583,73 +574,12 @@ private void checkLoadStatusSingle(HttpClient httpClient, String baseUrl) throws null ); content = holder.getContent(); - loadStatus = jsonMapper.readValue(content, LOAD_STATUS_TYPE_REFERENCE); + loadStatus = jsonMapper.readValue(content, JacksonUtils.TYPE_REFERENCE_MAP_STRING_BOOLEAN); Assert.assertNotNull(loadStatus.get("basic")); Assert.assertTrue(loadStatus.get("basic")); } - private StatusResponseHolder makeRequest(HttpClient httpClient, HttpMethod method, String url, byte[] content) - { - return makeRequestWithExpectedStatus( - httpClient, - method, - url, - content, - HttpResponseStatus.OK - ); - } - - private StatusResponseHolder makeRequestWithExpectedStatus( - HttpClient httpClient, - HttpMethod method, - String url, - byte[] content, - HttpResponseStatus expectedStatus - ) - { - try { - Request request = new Request(method, new URL(url)); - if (content != null) { - request.setContent(MediaType.APPLICATION_JSON, content); - } - int retryCount = 0; - - StatusResponseHolder response; - - while (true) { - response = httpClient.go( - request, - responseHandler - ).get(); - - if (!response.getStatus().equals(expectedStatus)) { - String errMsg = StringUtils.format( - "Error while making request to url[%s] status[%s] content[%s]", - url, - response.getStatus(), - response.getContent() - ); - // it can take time for the auth config to propagate, so we retry - if (retryCount > 10) { - throw new ISE(errMsg); - } else { - LOG.error(errMsg); - LOG.error("retrying in 3000ms, retryCount: " + retryCount); - retryCount++; - Thread.sleep(3000); - } - } else { - break; - } - } - return response; - } - catch (Exception e) { - throw new RuntimeException(e); - } - } - private void createUserAndRoleWithPermissions( HttpClient adminClient, String user, diff --git a/server/src/main/java/org/apache/druid/server/http/SelfDiscoveryResource.java b/server/src/main/java/org/apache/druid/server/http/SelfDiscoveryResource.java index ff58150434f6..762004694a6b 100644 --- a/server/src/main/java/org/apache/druid/server/http/SelfDiscoveryResource.java +++ b/server/src/main/java/org/apache/druid/server/http/SelfDiscoveryResource.java @@ -44,6 +44,7 @@ * DI configuration phase. */ @Singleton +@Path("/status/") @ResourceFilters(StateResourceFilter.class) public class SelfDiscoveryResource { @@ -78,7 +79,7 @@ public void stop() /** See the description of this endpoint in api-reference.md. */ @GET - @Path("/status/selfDiscoveredStatus") + @Path("/selfDiscoveredStatus") @Produces(MediaType.APPLICATION_JSON) public Response getSelfDiscoveredStatus() { @@ -87,8 +88,9 @@ public Response getSelfDiscoveredStatus() /** See the description of this endpoint in api-reference.md. */ @GET - @Path("/status/selfDiscovered") + @Path("/selfDiscovered") @Produces(MediaType.APPLICATION_JSON) + @ResourceFilters(StateResourceFilter.class) public Response getSelfDiscovered() { if (selfDiscovered.getAsBoolean()) { From 8d0291297837ffe59c1e0dbf06e4e524ddb0d92c Mon Sep 17 00:00:00 2001 From: Roman Leventov Date: Tue, 23 Jul 2019 16:19:11 +0300 Subject: [PATCH 10/14] tmp --- integration-tests/docker/Dockerfile | 3 +- integration-tests/docker/docker-entrypoint.sh | 27 +++++ .../tls/generate-expired-client-cert.sh | 7 +- .../docker/tls/generate-good-client-cert.sh | 2 +- ...generate-incorrect-hostname-client-cert.sh | 3 +- ...nerate-invalid-intermediate-client-cert.sh | 2 +- .../tls/generate-to-be-revoked-client-cert.sh | 2 +- .../generate-untrusted-root-client-cert.sh | 2 +- ...generate-valid-intermediate-client-cert.sh | 2 +- .../docker/tls/set-docker-host-ip.sh | 23 ++++ .../apache/druid/testing/utils/HttpUtil.java | 103 ++++++++++++++++++ 11 files changed, 168 insertions(+), 8 deletions(-) create mode 100755 integration-tests/docker/docker-entrypoint.sh create mode 100755 integration-tests/docker/tls/set-docker-host-ip.sh create mode 100644 integration-tests/src/main/java/org/apache/druid/testing/utils/HttpUtil.java diff --git a/integration-tests/docker/Dockerfile b/integration-tests/docker/Dockerfile index 6c5094cf7d11..a13042b3aa15 100644 --- a/integration-tests/docker/Dockerfile +++ b/integration-tests/docker/Dockerfile @@ -86,4 +86,5 @@ EXPOSE 8300 8301 8302 8303 8304 8305 EXPOSE 9092 9093 WORKDIR /var/lib/druid -ENTRYPOINT export HOST_IP="$(resolveip -s $HOSTNAME)" && /tls/generate-server-certs-and-keystores.sh && exec /usr/bin/supervisord -c /etc/supervisor/conf.d/supervisord.conf +COPY ./docker-entrypoint.sh /var/lib/druid/docker-entrypoint.sh +ENTRYPOINT ["/var/lib/druid/docker-entrypoint.sh"] diff --git a/integration-tests/docker/docker-entrypoint.sh b/integration-tests/docker/docker-entrypoint.sh new file mode 100755 index 000000000000..d04465a01127 --- /dev/null +++ b/integration-tests/docker/docker-entrypoint.sh @@ -0,0 +1,27 @@ +#!/bin/bash -eu + +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +if [[ "$OSTYPE" == "darwin"* ]]; then + # On Mac OS X resolveip may not be available + export HOST_IP=$(dig +short $HOSTNAME | awk '{ print ; exit }') +else + export HOST_IP=$(resolveip -s $HOSTNAME) +fi + +/tls/generate-server-certs-and-keystores.sh + +exec /usr/bin/supervisord -c /etc/supervisor/conf.d/supervisord.conf \ No newline at end of file diff --git a/integration-tests/docker/tls/generate-expired-client-cert.sh b/integration-tests/docker/tls/generate-expired-client-cert.sh index 71fb8b7f633b..a50df5cd88d0 100755 --- a/integration-tests/docker/tls/generate-expired-client-cert.sh +++ b/integration-tests/docker/tls/generate-expired-client-cert.sh @@ -15,7 +15,12 @@ # See the License for the specific language governing permissions and # limitations under the License. -export DOCKER_HOST_IP=$(resolveip -s $HOSTNAME) +if [[ "$OSTYPE" == "darwin"* ]]; then + # On Mac OS X resolveip may not be available + export DOCKER_HOST_IP=$(dig +short $HOSTNAME | awk '{ print ; exit }') +else + export DOCKER_HOST_IP=$(resolveip -s $HOSTNAME) +fi cat < expired_csr.conf [req] diff --git a/integration-tests/docker/tls/generate-good-client-cert.sh b/integration-tests/docker/tls/generate-good-client-cert.sh index e166d0908603..65046fa66841 100755 --- a/integration-tests/docker/tls/generate-good-client-cert.sh +++ b/integration-tests/docker/tls/generate-good-client-cert.sh @@ -15,7 +15,7 @@ # See the License for the specific language governing permissions and # limitations under the License. -export DOCKER_HOST_IP=$(resolveip -s $HOSTNAME) +./set-docker-host-ip.sh cat < csr.conf [req] diff --git a/integration-tests/docker/tls/generate-incorrect-hostname-client-cert.sh b/integration-tests/docker/tls/generate-incorrect-hostname-client-cert.sh index b778aa2aa3ba..f3f7ed240876 100755 --- a/integration-tests/docker/tls/generate-incorrect-hostname-client-cert.sh +++ b/integration-tests/docker/tls/generate-incorrect-hostname-client-cert.sh @@ -15,7 +15,7 @@ # See the License for the specific language governing permissions and # limitations under the License. -export DOCKER_HOST_IP=$(resolveip -s $HOSTNAME) +./set-docker-host-ip.sh # Generate a client cert with an incorrect hostname for testing cat < invalid_hostname_csr.conf @@ -41,6 +41,7 @@ basicConstraints=CA:FALSE,pathlen:0 [ alt_names ] DNS.1 = thisisprobablywrongtoo +email = xyz@example.com EOT diff --git a/integration-tests/docker/tls/generate-invalid-intermediate-client-cert.sh b/integration-tests/docker/tls/generate-invalid-intermediate-client-cert.sh index fc7771683250..8fcd0e98eab5 100755 --- a/integration-tests/docker/tls/generate-invalid-intermediate-client-cert.sh +++ b/integration-tests/docker/tls/generate-invalid-intermediate-client-cert.sh @@ -15,7 +15,7 @@ # See the License for the specific language governing permissions and # limitations under the License. -export DOCKER_HOST_IP=$(resolveip -s $HOSTNAME) +./set-docker-host-ip.sh cat < invalid_ca_intermediate.conf [req] diff --git a/integration-tests/docker/tls/generate-to-be-revoked-client-cert.sh b/integration-tests/docker/tls/generate-to-be-revoked-client-cert.sh index effcad5e3099..c3c7df499847 100755 --- a/integration-tests/docker/tls/generate-to-be-revoked-client-cert.sh +++ b/integration-tests/docker/tls/generate-to-be-revoked-client-cert.sh @@ -15,7 +15,7 @@ # See the License for the specific language governing permissions and # limitations under the License. -export DOCKER_HOST_IP=$(resolveip -s $HOSTNAME) +./set-docker-host-ip.sh # Generate a client cert that will be revoked cat < revoked_csr.conf diff --git a/integration-tests/docker/tls/generate-untrusted-root-client-cert.sh b/integration-tests/docker/tls/generate-untrusted-root-client-cert.sh index c133a5df1eab..1c6cbc6275b7 100755 --- a/integration-tests/docker/tls/generate-untrusted-root-client-cert.sh +++ b/integration-tests/docker/tls/generate-untrusted-root-client-cert.sh @@ -15,7 +15,7 @@ # See the License for the specific language governing permissions and # limitations under the License. -export DOCKER_HOST_IP=$(resolveip -s $HOSTNAME) +./set-docker-host-ip.sh cat < csr_another_root.conf [req] diff --git a/integration-tests/docker/tls/generate-valid-intermediate-client-cert.sh b/integration-tests/docker/tls/generate-valid-intermediate-client-cert.sh index 914a7031100f..abc05425256f 100755 --- a/integration-tests/docker/tls/generate-valid-intermediate-client-cert.sh +++ b/integration-tests/docker/tls/generate-valid-intermediate-client-cert.sh @@ -15,7 +15,7 @@ # See the License for the specific language governing permissions and # limitations under the License. -export DOCKER_HOST_IP=$(resolveip -s $HOSTNAME) +./set-docker-host-ip.sh cat < ca_intermediate.conf [req] diff --git a/integration-tests/docker/tls/set-docker-host-ip.sh b/integration-tests/docker/tls/set-docker-host-ip.sh new file mode 100755 index 000000000000..b6b00a1963f4 --- /dev/null +++ b/integration-tests/docker/tls/set-docker-host-ip.sh @@ -0,0 +1,23 @@ +#!/bin/bash -eu + +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +if [[ "$OSTYPE" == "darwin"* ]]; then + # On Mac OS X resolveip may not be available + export DOCKER_HOST_IP=$(dig +short $HOSTNAME | awk '{ print ; exit }') +else + export DOCKER_HOST_IP=$(resolveip -s $HOSTNAME) +fi \ No newline at end of file diff --git a/integration-tests/src/main/java/org/apache/druid/testing/utils/HttpUtil.java b/integration-tests/src/main/java/org/apache/druid/testing/utils/HttpUtil.java new file mode 100644 index 000000000000..5e011264c478 --- /dev/null +++ b/integration-tests/src/main/java/org/apache/druid/testing/utils/HttpUtil.java @@ -0,0 +1,103 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.druid.testing.utils; + +import org.apache.druid.java.util.common.ISE; +import org.apache.druid.java.util.common.StringUtils; +import org.apache.druid.java.util.common.logger.Logger; +import org.apache.druid.java.util.http.client.HttpClient; +import org.apache.druid.java.util.http.client.Request; +import org.apache.druid.java.util.http.client.response.StatusResponseHandler; +import org.apache.druid.java.util.http.client.response.StatusResponseHolder; +import org.apache.druid.testing.clients.AbstractQueryResourceTestClient; +import org.jboss.netty.handler.codec.http.HttpMethod; +import org.jboss.netty.handler.codec.http.HttpResponseStatus; + +import javax.annotation.Nullable; +import javax.ws.rs.core.MediaType; +import java.net.URL; + +public class HttpUtil +{ + private static final Logger LOG = new Logger(AbstractQueryResourceTestClient.class); + private static final StatusResponseHandler RESPONSE_HANDLER = StatusResponseHandler.getInstance(); + + public static StatusResponseHolder makeRequest(HttpClient httpClient, HttpMethod method, String url, byte[] content) + { + return makeRequestWithExpectedStatus( + httpClient, + method, + url, + content, + HttpResponseStatus.OK + ); + } + + public static StatusResponseHolder makeRequestWithExpectedStatus( + HttpClient httpClient, + HttpMethod method, + String url, + @Nullable byte[] content, + HttpResponseStatus expectedStatus + ) + { + try { + Request request = new Request(method, new URL(url)); + if (content != null) { + request.setContent(MediaType.APPLICATION_JSON, content); + } + int retryCount = 0; + + StatusResponseHolder response; + + while (true) { + response = httpClient.go(request, RESPONSE_HANDLER).get(); + + if (!response.getStatus().equals(expectedStatus)) { + String errMsg = StringUtils.format( + "Error while making request to url[%s] status[%s] content[%s]", + url, + response.getStatus(), + response.getContent() + ); + // it can take time for the auth config to propagate, so we retry + if (retryCount > 10) { + throw new ISE(errMsg); + } else { + LOG.error(errMsg); + LOG.error("retrying in 3000ms, retryCount: " + retryCount); + retryCount++; + Thread.sleep(3000); + } + } else { + break; + } + } + return response; + } + catch (Exception e) { + throw new RuntimeException(e); + } + } + + private HttpUtil() + { + } +} From df813e1072b6c55a1ee661356d34bd51c5f145f5 Mon Sep 17 00:00:00 2001 From: Roman Leventov Date: Sat, 23 Nov 2019 13:01:46 +0300 Subject: [PATCH 11/14] Rollback docker changes --- docs/operations/api-reference.md | 2 +- integration-tests/docker/Dockerfile | 3 +-- .../docker/tls/generate-expired-client-cert.sh | 7 +------ integration-tests/docker/tls/generate-good-client-cert.sh | 2 +- .../docker/tls/generate-incorrect-hostname-client-cert.sh | 3 +-- .../tls/generate-invalid-intermediate-client-cert.sh | 2 +- .../docker/tls/generate-to-be-revoked-client-cert.sh | 2 +- .../docker/tls/generate-untrusted-root-client-cert.sh | 2 +- .../docker/tls/generate-valid-intermediate-client-cert.sh | 2 +- integration-tests/pom.xml | 4 ++++ 10 files changed, 13 insertions(+), 16 deletions(-) diff --git a/docs/operations/api-reference.md b/docs/operations/api-reference.md index feefa0c0a103..c184e8daf971 100644 --- a/docs/operations/api-reference.md +++ b/docs/operations/api-reference.md @@ -47,7 +47,7 @@ Returns the current configuration properties of the process. * `/status/selfDiscoveredStatus` -Returns a JSON map of the form `{"selfDiscovered": true/false}`, indicating whether the node has recieved a confirmation +Returns a JSON map of the form `{"selfDiscovered": true/false}`, indicating whether the node has received a confirmation from the central node discovery mechanism (currently ZooKeeper) of the Druid cluster that the node has been added to the cluster. It is recommended to not consider a Druid node "healthy" or "ready" in automated deployment/container management systems until it returns `{"selfDiscovered": true}` from this endpoint. This is because a node may be diff --git a/integration-tests/docker/Dockerfile b/integration-tests/docker/Dockerfile index a13042b3aa15..6c5094cf7d11 100644 --- a/integration-tests/docker/Dockerfile +++ b/integration-tests/docker/Dockerfile @@ -86,5 +86,4 @@ EXPOSE 8300 8301 8302 8303 8304 8305 EXPOSE 9092 9093 WORKDIR /var/lib/druid -COPY ./docker-entrypoint.sh /var/lib/druid/docker-entrypoint.sh -ENTRYPOINT ["/var/lib/druid/docker-entrypoint.sh"] +ENTRYPOINT export HOST_IP="$(resolveip -s $HOSTNAME)" && /tls/generate-server-certs-and-keystores.sh && exec /usr/bin/supervisord -c /etc/supervisor/conf.d/supervisord.conf diff --git a/integration-tests/docker/tls/generate-expired-client-cert.sh b/integration-tests/docker/tls/generate-expired-client-cert.sh index a50df5cd88d0..71fb8b7f633b 100755 --- a/integration-tests/docker/tls/generate-expired-client-cert.sh +++ b/integration-tests/docker/tls/generate-expired-client-cert.sh @@ -15,12 +15,7 @@ # See the License for the specific language governing permissions and # limitations under the License. -if [[ "$OSTYPE" == "darwin"* ]]; then - # On Mac OS X resolveip may not be available - export DOCKER_HOST_IP=$(dig +short $HOSTNAME | awk '{ print ; exit }') -else - export DOCKER_HOST_IP=$(resolveip -s $HOSTNAME) -fi +export DOCKER_HOST_IP=$(resolveip -s $HOSTNAME) cat < expired_csr.conf [req] diff --git a/integration-tests/docker/tls/generate-good-client-cert.sh b/integration-tests/docker/tls/generate-good-client-cert.sh index 65046fa66841..e166d0908603 100755 --- a/integration-tests/docker/tls/generate-good-client-cert.sh +++ b/integration-tests/docker/tls/generate-good-client-cert.sh @@ -15,7 +15,7 @@ # See the License for the specific language governing permissions and # limitations under the License. -./set-docker-host-ip.sh +export DOCKER_HOST_IP=$(resolveip -s $HOSTNAME) cat < csr.conf [req] diff --git a/integration-tests/docker/tls/generate-incorrect-hostname-client-cert.sh b/integration-tests/docker/tls/generate-incorrect-hostname-client-cert.sh index f3f7ed240876..b778aa2aa3ba 100755 --- a/integration-tests/docker/tls/generate-incorrect-hostname-client-cert.sh +++ b/integration-tests/docker/tls/generate-incorrect-hostname-client-cert.sh @@ -15,7 +15,7 @@ # See the License for the specific language governing permissions and # limitations under the License. -./set-docker-host-ip.sh +export DOCKER_HOST_IP=$(resolveip -s $HOSTNAME) # Generate a client cert with an incorrect hostname for testing cat < invalid_hostname_csr.conf @@ -41,7 +41,6 @@ basicConstraints=CA:FALSE,pathlen:0 [ alt_names ] DNS.1 = thisisprobablywrongtoo -email = xyz@example.com EOT diff --git a/integration-tests/docker/tls/generate-invalid-intermediate-client-cert.sh b/integration-tests/docker/tls/generate-invalid-intermediate-client-cert.sh index 8fcd0e98eab5..fc7771683250 100755 --- a/integration-tests/docker/tls/generate-invalid-intermediate-client-cert.sh +++ b/integration-tests/docker/tls/generate-invalid-intermediate-client-cert.sh @@ -15,7 +15,7 @@ # See the License for the specific language governing permissions and # limitations under the License. -./set-docker-host-ip.sh +export DOCKER_HOST_IP=$(resolveip -s $HOSTNAME) cat < invalid_ca_intermediate.conf [req] diff --git a/integration-tests/docker/tls/generate-to-be-revoked-client-cert.sh b/integration-tests/docker/tls/generate-to-be-revoked-client-cert.sh index c3c7df499847..effcad5e3099 100755 --- a/integration-tests/docker/tls/generate-to-be-revoked-client-cert.sh +++ b/integration-tests/docker/tls/generate-to-be-revoked-client-cert.sh @@ -15,7 +15,7 @@ # See the License for the specific language governing permissions and # limitations under the License. -./set-docker-host-ip.sh +export DOCKER_HOST_IP=$(resolveip -s $HOSTNAME) # Generate a client cert that will be revoked cat < revoked_csr.conf diff --git a/integration-tests/docker/tls/generate-untrusted-root-client-cert.sh b/integration-tests/docker/tls/generate-untrusted-root-client-cert.sh index 1c6cbc6275b7..c133a5df1eab 100755 --- a/integration-tests/docker/tls/generate-untrusted-root-client-cert.sh +++ b/integration-tests/docker/tls/generate-untrusted-root-client-cert.sh @@ -15,7 +15,7 @@ # See the License for the specific language governing permissions and # limitations under the License. -./set-docker-host-ip.sh +export DOCKER_HOST_IP=$(resolveip -s $HOSTNAME) cat < csr_another_root.conf [req] diff --git a/integration-tests/docker/tls/generate-valid-intermediate-client-cert.sh b/integration-tests/docker/tls/generate-valid-intermediate-client-cert.sh index abc05425256f..914a7031100f 100755 --- a/integration-tests/docker/tls/generate-valid-intermediate-client-cert.sh +++ b/integration-tests/docker/tls/generate-valid-intermediate-client-cert.sh @@ -15,7 +15,7 @@ # See the License for the specific language governing permissions and # limitations under the License. -./set-docker-host-ip.sh +export DOCKER_HOST_IP=$(resolveip -s $HOSTNAME) cat < ca_intermediate.conf [req] diff --git a/integration-tests/pom.xml b/integration-tests/pom.xml index 79e8c2c8c65d..6e286021a236 100644 --- a/integration-tests/pom.xml +++ b/integration-tests/pom.xml @@ -183,6 +183,10 @@ log4j-slf4j-impl runtime + + com.google.code.findbugs + jsr305 + From 81e36f0c13477eb62c2f233a76f00cb8336e72a2 Mon Sep 17 00:00:00 2001 From: Roman Leventov Date: Sat, 23 Nov 2019 13:43:57 +0300 Subject: [PATCH 12/14] Remove extra .sh files --- .../response/StatusResponseHandler.java | 3 --- integration-tests/docker/docker-entrypoint.sh | 27 ------------------- .../docker/tls/set-docker-host-ip.sh | 23 ---------------- 3 files changed, 53 deletions(-) delete mode 100755 integration-tests/docker/docker-entrypoint.sh delete mode 100755 integration-tests/docker/tls/set-docker-host-ip.sh diff --git a/core/src/main/java/org/apache/druid/java/util/http/client/response/StatusResponseHandler.java b/core/src/main/java/org/apache/druid/java/util/http/client/response/StatusResponseHandler.java index c1b3143af953..07743524f2e4 100644 --- a/core/src/main/java/org/apache/druid/java/util/http/client/response/StatusResponseHandler.java +++ b/core/src/main/java/org/apache/druid/java/util/http/client/response/StatusResponseHandler.java @@ -24,9 +24,6 @@ import java.nio.charset.StandardCharsets; -/** - * Can be singleton: https://github.com/apache/incubator-druid/issues/7833 - */ public class StatusResponseHandler implements HttpResponseHandler { diff --git a/integration-tests/docker/docker-entrypoint.sh b/integration-tests/docker/docker-entrypoint.sh deleted file mode 100755 index d04465a01127..000000000000 --- a/integration-tests/docker/docker-entrypoint.sh +++ /dev/null @@ -1,27 +0,0 @@ -#!/bin/bash -eu - -# Licensed to the Apache Software Foundation (ASF) under one or more -# contributor license agreements. See the NOTICE file distributed with -# this work for additional information regarding copyright ownership. -# The ASF licenses this file to You under the Apache License, Version 2.0 -# (the "License"); you may not use this file except in compliance with -# the License. You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, software -# distributed under the License is distributed on an "AS IS" BASIS, -# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -# See the License for the specific language governing permissions and -# limitations under the License. - -if [[ "$OSTYPE" == "darwin"* ]]; then - # On Mac OS X resolveip may not be available - export HOST_IP=$(dig +short $HOSTNAME | awk '{ print ; exit }') -else - export HOST_IP=$(resolveip -s $HOSTNAME) -fi - -/tls/generate-server-certs-and-keystores.sh - -exec /usr/bin/supervisord -c /etc/supervisor/conf.d/supervisord.conf \ No newline at end of file diff --git a/integration-tests/docker/tls/set-docker-host-ip.sh b/integration-tests/docker/tls/set-docker-host-ip.sh deleted file mode 100755 index b6b00a1963f4..000000000000 --- a/integration-tests/docker/tls/set-docker-host-ip.sh +++ /dev/null @@ -1,23 +0,0 @@ -#!/bin/bash -eu - -# Licensed to the Apache Software Foundation (ASF) under one or more -# contributor license agreements. See the NOTICE file distributed with -# this work for additional information regarding copyright ownership. -# The ASF licenses this file to You under the Apache License, Version 2.0 -# (the "License"); you may not use this file except in compliance with -# the License. You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, software -# distributed under the License is distributed on an "AS IS" BASIS, -# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -# See the License for the specific language governing permissions and -# limitations under the License. - -if [[ "$OSTYPE" == "darwin"* ]]; then - # On Mac OS X resolveip may not be available - export DOCKER_HOST_IP=$(dig +short $HOSTNAME | awk '{ print ; exit }') -else - export DOCKER_HOST_IP=$(resolveip -s $HOSTNAME) -fi \ No newline at end of file From 39266697848896a9d0e3b7dd07b6c70b645ba350 Mon Sep 17 00:00:00 2001 From: Roman Leventov Date: Sun, 8 Dec 2019 15:04:53 +0300 Subject: [PATCH 13/14] Move filter --- .../org/apache/druid/server/http/SelfDiscoveryResource.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/server/src/main/java/org/apache/druid/server/http/SelfDiscoveryResource.java b/server/src/main/java/org/apache/druid/server/http/SelfDiscoveryResource.java index b61b4b62dabd..46d04a9006f7 100644 --- a/server/src/main/java/org/apache/druid/server/http/SelfDiscoveryResource.java +++ b/server/src/main/java/org/apache/druid/server/http/SelfDiscoveryResource.java @@ -44,7 +44,6 @@ * DI configuration phase. */ @Singleton -@ResourceFilters(StateResourceFilter.class) public class SelfDiscoveryResource { private BooleanSupplier selfDiscovered; @@ -80,6 +79,7 @@ public void stop() @GET @Path("/status/selfDiscoveredStatus") @Produces(MediaType.APPLICATION_JSON) + @ResourceFilters(StateResourceFilter.class) public Response getSelfDiscoveredStatus() { return Response.ok(Collections.singletonMap("selfDiscovered", selfDiscovered.getAsBoolean())).build(); From 3737cb5adea9c3bff970ed4a06c0eace529bbbc5 Mon Sep 17 00:00:00 2001 From: Roman Leventov Date: Sun, 8 Dec 2019 15:58:58 +0300 Subject: [PATCH 14/14] Fix SecurityResourceFilterTest --- .../security/ResourceFilterTestHelper.java | 30 ++++++++++--------- .../security/SecurityResourceFilterTest.java | 5 ++-- 2 files changed, 19 insertions(+), 16 deletions(-) diff --git a/server/src/test/java/org/apache/druid/server/http/security/ResourceFilterTestHelper.java b/server/src/test/java/org/apache/druid/server/http/security/ResourceFilterTestHelper.java index cfaa61ebd0e4..95a0cc29e31a 100644 --- a/server/src/test/java/org/apache/druid/server/http/security/ResourceFilterTestHelper.java +++ b/server/src/test/java/org/apache/druid/server/http/security/ResourceFilterTestHelper.java @@ -49,6 +49,7 @@ import javax.ws.rs.Path; import javax.ws.rs.core.MultivaluedMap; import javax.ws.rs.core.PathSegment; +import java.lang.reflect.AnnotatedElement; import java.lang.reflect.Method; import java.util.Arrays; import java.util.Collection; @@ -133,14 +134,9 @@ public Access authorize(AuthenticationResult authenticationResult1, Resource res ).atLeastOnce(); } - public static Collection getRequestPaths(final Class clazz) + public static Collection getRequestPathsWithAuthorizer(final AnnotatedElement classOrMethod) { - return getRequestPaths(clazz, ImmutableList.of(), ImmutableList.of()); - } - - public static Collection getRequestPathsWithAuthorizer(final Class clazz) - { - return getRequestPaths(clazz, ImmutableList.of(AuthorizerMapper.class), ImmutableList.of()); + return getRequestPaths(classOrMethod, ImmutableList.of(AuthorizerMapper.class), ImmutableList.of()); } public static Collection getRequestPaths( @@ -152,17 +148,17 @@ public static Collection getRequestPaths( } public static Collection getRequestPaths( - final Class clazz, + final AnnotatedElement classOrMethod, final Iterable> mockableInjections, final Iterable> mockableKeys ) { - return getRequestPaths(clazz, mockableInjections, mockableKeys, ImmutableList.of()); + return getRequestPaths(classOrMethod, mockableInjections, mockableKeys, ImmutableList.of()); } // Feeds in an array of [ PathName, MethodName, ResourceFilter , Injector] public static Collection getRequestPaths( - final Class clazz, + final AnnotatedElement classOrMethod, final Iterable> mockableInjections, final Iterable> mockableKeys, final Iterable injectedObjs @@ -187,11 +183,17 @@ public void configure(Binder binder) } } ); - final String basepath = ((Path) clazz.getAnnotation(Path.class)).value().substring(1); //Ignore the first "/" + final String basepath = classOrMethod.getAnnotation(Path.class).value().substring(1); //Ignore the first "/" final List> baseResourceFilters = - clazz.getAnnotation(ResourceFilters.class) == null ? Collections.emptyList() : - ImmutableList.copyOf(((ResourceFilters) clazz.getAnnotation(ResourceFilters.class)).value()); + classOrMethod.getAnnotation(ResourceFilters.class) == null ? Collections.emptyList() : + ImmutableList.copyOf(classOrMethod.getAnnotation(ResourceFilters.class).value()); + List methods; + if (classOrMethod instanceof Class) { + methods = ImmutableList.copyOf(((Class) classOrMethod).getDeclaredMethods()); + } else { + methods = Collections.singletonList((Method) classOrMethod); + } return ImmutableList.copyOf( Iterables.concat( // Step 3 - Merge all the Objects arrays for each endpoints @@ -206,7 +208,7 @@ public void configure(Binder binder) // Filter out non resource endpoint methods // and also the endpoints that does not have any // ResourceFilters applied to them - ImmutableList.copyOf(clazz.getDeclaredMethods()), + methods, new Predicate() { @Override diff --git a/server/src/test/java/org/apache/druid/server/http/security/SecurityResourceFilterTest.java b/server/src/test/java/org/apache/druid/server/http/security/SecurityResourceFilterTest.java index e0a5ba13334d..344396c98b3b 100644 --- a/server/src/test/java/org/apache/druid/server/http/security/SecurityResourceFilterTest.java +++ b/server/src/test/java/org/apache/druid/server/http/security/SecurityResourceFilterTest.java @@ -53,7 +53,7 @@ public class SecurityResourceFilterTest extends ResourceFilterTestHelper { @Parameterized.Parameters(name = "{index}: requestPath={0}, requestMethod={1}, resourceFilter={2}") - public static Collection data() + public static Collection data() throws NoSuchMethodException { return ImmutableList.copyOf( Iterables.concat( @@ -70,7 +70,8 @@ public static Collection data() getRequestPathsWithAuthorizer(CoordinatorDynamicConfigsResource.class), getRequestPathsWithAuthorizer(QueryResource.class), getRequestPathsWithAuthorizer(StatusResource.class), - getRequestPathsWithAuthorizer(SelfDiscoveryResource.class), + getRequestPathsWithAuthorizer(SelfDiscoveryResource.class.getDeclaredMethod("getSelfDiscoveredStatus")), + getRequestPathsWithAuthorizer(SelfDiscoveryResource.class.getDeclaredMethod("getSelfDiscovered")), getRequestPathsWithAuthorizer(BrokerQueryResource.class), getRequestPathsWithAuthorizer(RouterResource.class) )