From 2ee2016d15762391981441680c562757834e5772 Mon Sep 17 00:00:00 2001 From: Himanshu Gupta Date: Tue, 1 Aug 2017 10:32:55 -0500 Subject: [PATCH 1/9] internal-discovery: interfaces for announcement/discovery, curator impls --- .../druid/indexing/kafka/KafkaIndexTask.java | 13 + .../indexing/kafka/KafkaIndexTaskTest.java | 6 +- .../io/druid/indexing/common/TaskToolbox.java | 39 ++- .../indexing/common/TaskToolboxFactory.java | 25 +- .../common/task/RealtimeIndexTask.java | 14 + .../indexing/common/TaskToolboxTest.java | 6 +- .../indexing/common/task/IndexTaskTest.java | 2 +- .../common/task/RealtimeIndexTaskTest.java | 11 +- .../task/SameIntervalMergeTaskTest.java | 3 +- .../IngestSegmentFirehoseFactoryTest.java | 6 +- ...estSegmentFirehoseFactoryTimelineTest.java | 6 +- .../indexing/overlord/TaskLifecycleTest.java | 10 +- .../worker/WorkerTaskMonitorTest.java | 6 +- .../io/druid/client/DruidServerDiscovery.java | 174 ---------- ...lteredHttpServerInventoryViewProvider.java | 11 +- .../druid/client/HttpServerInventoryView.java | 53 ++- .../HttpServerInventoryViewProvider.java | 11 +- .../discovery/CuratorDruidNodeAnnouncer.java | 93 +++++ .../CuratorDruidNodeDiscoveryProvider.java | 327 ++++++++++++++++++ .../curator/discovery/DiscoveryModule.java | 19 +- .../io/druid/discovery/DataNodeService.java | 117 +++++++ .../druid/discovery/DiscoveryDruidNode.java | 108 ++++++ .../druid/discovery/DruidDiscoveryModule.java | 49 +++ .../druid/discovery/DruidNodeAnnouncer.java | 29 ++ .../druid/discovery/DruidNodeDiscovery.java | 37 ++ .../discovery/DruidNodeDiscoveryProvider.java | 142 ++++++++ .../java/io/druid/discovery/DruidService.java | 37 ++ .../io/druid/discovery/LookupNodeService.java | 80 +++++ .../io/druid/discovery/WorkerNodeService.java | 102 ++++++ .../io/druid/guice/StorageNodeModule.java | 20 ++ .../druid/initialization/Initialization.java | 2 + .../io/druid/query/lookup/LookupModule.java | 10 + .../io/druid/server/http/ClusterResource.java | 89 +++++ .../CuratorDiscoveryConfig.java | 8 + .../src/main/java/io/druid/cli/CliBroker.java | 52 +++ .../java/io/druid/cli/CliCoordinator.java | 51 ++- .../main/java/io/druid/cli/CliHistorical.java | 54 +++ .../java/io/druid/cli/CliMiddleManager.java | 59 +++- .../main/java/io/druid/cli/CliOverlord.java | 49 +++ .../src/main/java/io/druid/cli/CliRouter.java | 48 +++ 40 files changed, 1745 insertions(+), 233 deletions(-) delete mode 100644 server/src/main/java/io/druid/client/DruidServerDiscovery.java create mode 100644 server/src/main/java/io/druid/curator/discovery/CuratorDruidNodeAnnouncer.java create mode 100644 server/src/main/java/io/druid/curator/discovery/CuratorDruidNodeDiscoveryProvider.java create mode 100644 server/src/main/java/io/druid/discovery/DataNodeService.java create mode 100644 server/src/main/java/io/druid/discovery/DiscoveryDruidNode.java create mode 100644 server/src/main/java/io/druid/discovery/DruidDiscoveryModule.java create mode 100644 server/src/main/java/io/druid/discovery/DruidNodeAnnouncer.java create mode 100644 server/src/main/java/io/druid/discovery/DruidNodeDiscovery.java create mode 100644 server/src/main/java/io/druid/discovery/DruidNodeDiscoveryProvider.java create mode 100644 server/src/main/java/io/druid/discovery/DruidService.java create mode 100644 server/src/main/java/io/druid/discovery/LookupNodeService.java create mode 100644 server/src/main/java/io/druid/discovery/WorkerNodeService.java create mode 100644 server/src/main/java/io/druid/server/http/ClusterResource.java diff --git a/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/KafkaIndexTask.java b/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/KafkaIndexTask.java index 0a7a57f3dc74..869e89e10029 100644 --- a/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/KafkaIndexTask.java +++ b/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/KafkaIndexTask.java @@ -42,6 +42,8 @@ import io.druid.data.input.Committer; import io.druid.data.input.InputRow; import io.druid.data.input.impl.InputRowParser; +import io.druid.discovery.DiscoveryDruidNode; +import io.druid.discovery.DruidNodeDiscoveryProvider; import io.druid.indexing.appenderator.ActionBasedSegmentAllocator; import io.druid.indexing.appenderator.ActionBasedUsedSegmentChecker; import io.druid.indexing.common.TaskStatus; @@ -289,12 +291,22 @@ public TaskStatus run(final TaskToolbox toolbox) throws Exception ) ); + DiscoveryDruidNode discoveryDruidNode = new DiscoveryDruidNode( + toolbox.getDruidNode(), + DruidNodeDiscoveryProvider.NODE_TYPE_PEON, + ImmutableMap.of( + toolbox.getDataNodeService().getName(), toolbox.getDataNodeService(), + toolbox.getLookupNodeService().getName(), toolbox.getLookupNodeService() + ) + ); + try ( final Appenderator appenderator0 = newAppenderator(fireDepartmentMetrics, toolbox); final AppenderatorDriver driver = newDriver(appenderator0, toolbox, fireDepartmentMetrics); final KafkaConsumer consumer = newConsumer() ) { toolbox.getDataSegmentServerAnnouncer().announce(); + toolbox.getDruidNodeAnnouncer().announce(discoveryDruidNode); appenderator = appenderator0; @@ -597,6 +609,7 @@ public String apply(DataSegment input) } } + toolbox.getDruidNodeAnnouncer().unannounce(discoveryDruidNode); toolbox.getDataSegmentServerAnnouncer().unannounce(); return success(); diff --git a/extensions-core/kafka-indexing-service/src/test/java/io/druid/indexing/kafka/KafkaIndexTaskTest.java b/extensions-core/kafka-indexing-service/src/test/java/io/druid/indexing/kafka/KafkaIndexTaskTest.java index a6bc82afe8ae..6ae96f209e18 100644 --- a/extensions-core/kafka-indexing-service/src/test/java/io/druid/indexing/kafka/KafkaIndexTaskTest.java +++ b/extensions-core/kafka-indexing-service/src/test/java/io/druid/indexing/kafka/KafkaIndexTaskTest.java @@ -1637,7 +1637,11 @@ public List getLocations() testUtils.getTestIndexIO(), MapCache.create(1024), new CacheConfig(), - testUtils.getTestIndexMergerV9() + testUtils.getTestIndexMergerV9(), + null, + null, + null, + null ); } diff --git a/indexing-service/src/main/java/io/druid/indexing/common/TaskToolbox.java b/indexing-service/src/main/java/io/druid/indexing/common/TaskToolbox.java index 2d0ce1ac9b66..16c6729145fa 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/TaskToolbox.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/TaskToolbox.java @@ -31,6 +31,9 @@ import com.metamx.metrics.MonitorScheduler; import io.druid.client.cache.Cache; import io.druid.client.cache.CacheConfig; +import io.druid.discovery.DataNodeService; +import io.druid.discovery.DruidNodeAnnouncer; +import io.druid.discovery.LookupNodeService; import io.druid.indexing.common.actions.SegmentInsertAction; import io.druid.indexing.common.actions.TaskActionClient; import io.druid.indexing.common.config.TaskConfig; @@ -44,6 +47,7 @@ import io.druid.segment.loading.SegmentLoader; import io.druid.segment.loading.SegmentLoadingException; import io.druid.segment.realtime.plumber.SegmentHandoffNotifierFactory; +import io.druid.server.DruidNode; import io.druid.server.coordination.DataSegmentAnnouncer; import io.druid.server.coordination.DataSegmentServerAnnouncer; import io.druid.timeline.DataSegment; @@ -87,6 +91,11 @@ public class TaskToolbox private final CacheConfig cacheConfig; private final IndexMergerV9 indexMergerV9; + private final DruidNodeAnnouncer druidNodeAnnouncer; + private final DruidNode druidNode; + private final LookupNodeService lookupNodeService; + private final DataNodeService dataNodeService; + public TaskToolbox( TaskConfig config, TaskActionClient taskActionClient, @@ -107,7 +116,11 @@ public TaskToolbox( IndexIO indexIO, Cache cache, CacheConfig cacheConfig, - IndexMergerV9 indexMergerV9 + IndexMergerV9 indexMergerV9, + DruidNodeAnnouncer druidNodeAnnouncer, + DruidNode druidNode, + LookupNodeService lookupNodeService, + DataNodeService dataNodeService ) { this.config = config; @@ -130,6 +143,10 @@ public TaskToolbox( this.cache = cache; this.cacheConfig = cacheConfig; this.indexMergerV9 = Preconditions.checkNotNull(indexMergerV9, "Null IndexMergerV9"); + this.druidNodeAnnouncer = druidNodeAnnouncer; + this.druidNode = druidNode; + this.lookupNodeService = lookupNodeService; + this.dataNodeService = dataNodeService; } public TaskConfig getConfig() @@ -271,4 +288,24 @@ public File getPersistDir() { return new File(taskWorkDir, "persist"); } + + public DruidNodeAnnouncer getDruidNodeAnnouncer() + { + return druidNodeAnnouncer; + } + + public LookupNodeService getLookupNodeService() + { + return lookupNodeService; + } + + public DataNodeService getDataNodeService() + { + return dataNodeService; + } + + public DruidNode getDruidNode() + { + return druidNode; + } } diff --git a/indexing-service/src/main/java/io/druid/indexing/common/TaskToolboxFactory.java b/indexing-service/src/main/java/io/druid/indexing/common/TaskToolboxFactory.java index ab0b5bf06d1b..43d2abea594c 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/TaskToolboxFactory.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/TaskToolboxFactory.java @@ -27,7 +27,11 @@ import com.metamx.metrics.MonitorScheduler; import io.druid.client.cache.Cache; import io.druid.client.cache.CacheConfig; +import io.druid.discovery.DataNodeService; +import io.druid.discovery.DruidNodeAnnouncer; +import io.druid.discovery.LookupNodeService; import io.druid.guice.annotations.Processing; +import io.druid.guice.annotations.RemoteChatHandler; import io.druid.indexing.common.actions.TaskActionClientFactory; import io.druid.indexing.common.config.TaskConfig; import io.druid.indexing.common.task.Task; @@ -39,6 +43,7 @@ import io.druid.segment.loading.DataSegmentMover; import io.druid.segment.loading.DataSegmentPusher; import io.druid.segment.realtime.plumber.SegmentHandoffNotifierFactory; +import io.druid.server.DruidNode; import io.druid.server.coordination.DataSegmentAnnouncer; import io.druid.server.coordination.DataSegmentServerAnnouncer; @@ -69,6 +74,10 @@ public class TaskToolboxFactory private final Cache cache; private final CacheConfig cacheConfig; private final IndexMergerV9 indexMergerV9; + private final DruidNodeAnnouncer druidNodeAnnouncer; + private final DruidNode druidNode; + private final LookupNodeService lookupNodeService; + private final DataNodeService dataNodeService; @Inject public TaskToolboxFactory( @@ -90,7 +99,11 @@ public TaskToolboxFactory( IndexIO indexIO, Cache cache, CacheConfig cacheConfig, - IndexMergerV9 indexMergerV9 + IndexMergerV9 indexMergerV9, + DruidNodeAnnouncer druidNodeAnnouncer, + @RemoteChatHandler DruidNode druidNode, + LookupNodeService lookupNodeService, + DataNodeService dataNodeService ) { this.config = config; @@ -112,6 +125,10 @@ public TaskToolboxFactory( this.cache = cache; this.cacheConfig = cacheConfig; this.indexMergerV9 = indexMergerV9; + this.druidNodeAnnouncer = druidNodeAnnouncer; + this.druidNode = druidNode; + this.lookupNodeService = lookupNodeService; + this.dataNodeService = dataNodeService; } public TaskToolbox build(Task task) @@ -137,7 +154,11 @@ public TaskToolbox build(Task task) indexIO, cache, cacheConfig, - indexMergerV9 + indexMergerV9, + druidNodeAnnouncer, + druidNode, + lookupNodeService, + dataNodeService ); } } diff --git a/indexing-service/src/main/java/io/druid/indexing/common/task/RealtimeIndexTask.java b/indexing-service/src/main/java/io/druid/indexing/common/task/RealtimeIndexTask.java index 9595a28cb2f2..f486efe6b104 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/task/RealtimeIndexTask.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/task/RealtimeIndexTask.java @@ -31,6 +31,8 @@ import io.druid.data.input.Committer; import io.druid.data.input.Firehose; import io.druid.data.input.FirehoseFactory; +import io.druid.discovery.DiscoveryDruidNode; +import io.druid.discovery.DruidNodeDiscoveryProvider; import io.druid.indexing.common.TaskLock; import io.druid.indexing.common.TaskStatus; import io.druid.indexing.common.TaskToolbox; @@ -324,8 +326,19 @@ public String getVersion(final Interval interval) Supplier committerSupplier = null; final File firehoseTempDir = toolbox.getFirehoseTemporaryDir(); + DiscoveryDruidNode discoveryDruidNode = new DiscoveryDruidNode( + toolbox.getDruidNode(), + DruidNodeDiscoveryProvider.NODE_TYPE_PEON, + ImmutableMap.of( + toolbox.getDataNodeService().getName(), toolbox.getDataNodeService(), + toolbox.getLookupNodeService().getName(), toolbox.getLookupNodeService() + ) + ); + try { toolbox.getDataSegmentServerAnnouncer().announce(); + toolbox.getDruidNodeAnnouncer().announce(discoveryDruidNode); + plumber.startJob(); @@ -431,6 +444,7 @@ public void run() } toolbox.getDataSegmentServerAnnouncer().unannounce(); + toolbox.getDruidNodeAnnouncer().unannounce(discoveryDruidNode); } log.info("Job done!"); diff --git a/indexing-service/src/test/java/io/druid/indexing/common/TaskToolboxTest.java b/indexing-service/src/test/java/io/druid/indexing/common/TaskToolboxTest.java index 5430938f96c6..da3b66f32800 100644 --- a/indexing-service/src/test/java/io/druid/indexing/common/TaskToolboxTest.java +++ b/indexing-service/src/test/java/io/druid/indexing/common/TaskToolboxTest.java @@ -110,7 +110,11 @@ public void setUp() throws IOException mockIndexIO, mockCache, mockCacheConfig, - mockIndexMergerV9 + mockIndexMergerV9, + null, + null, + null, + null ); } diff --git a/indexing-service/src/test/java/io/druid/indexing/common/task/IndexTaskTest.java b/indexing-service/src/test/java/io/druid/indexing/common/task/IndexTaskTest.java index 8f93db29ebfa..fd2adc740a5c 100644 --- a/indexing-service/src/test/java/io/druid/indexing/common/task/IndexTaskTest.java +++ b/indexing-service/src/test/java/io/druid/indexing/common/task/IndexTaskTest.java @@ -950,7 +950,7 @@ public Map makeLoadSpec(URI uri) throw new UnsupportedOperationException(); } }, null, null, null, null, null, null, null, null, null, null, jsonMapper, temporaryFolder.newFolder(), - indexIO, null, null, indexMergerV9 + indexIO, null, null, indexMergerV9, null, null, null, null ) ); diff --git a/indexing-service/src/test/java/io/druid/indexing/common/task/RealtimeIndexTaskTest.java b/indexing-service/src/test/java/io/druid/indexing/common/task/RealtimeIndexTaskTest.java index ce6510fa316d..c6d954420fc2 100644 --- a/indexing-service/src/test/java/io/druid/indexing/common/task/RealtimeIndexTaskTest.java +++ b/indexing-service/src/test/java/io/druid/indexing/common/task/RealtimeIndexTaskTest.java @@ -46,6 +46,9 @@ import io.druid.data.input.InputRow; import io.druid.data.input.MapBasedInputRow; import io.druid.data.input.impl.InputRowParser; +import io.druid.discovery.DataNodeService; +import io.druid.discovery.DruidNodeAnnouncer; +import io.druid.discovery.LookupNodeService; import io.druid.indexing.common.SegmentLoaderFactory; import io.druid.indexing.common.TaskStatus; import io.druid.indexing.common.TaskToolbox; @@ -104,7 +107,9 @@ import io.druid.segment.realtime.plumber.SegmentHandoffNotifier; import io.druid.segment.realtime.plumber.SegmentHandoffNotifierFactory; import io.druid.segment.realtime.plumber.ServerTimeRejectionPolicyFactory; +import io.druid.server.DruidNode; import io.druid.server.coordination.DataSegmentServerAnnouncer; +import io.druid.server.coordination.ServerType; import io.druid.timeline.DataSegment; import org.easymock.EasyMock; import org.hamcrest.CoreMatchers; @@ -1048,7 +1053,11 @@ public List getLocations() testUtils.getTestIndexIO(), MapCache.create(1024), new CacheConfig(), - testUtils.getTestIndexMergerV9() + testUtils.getTestIndexMergerV9(), + EasyMock.createNiceMock(DruidNodeAnnouncer.class), + EasyMock.createNiceMock(DruidNode.class), + new LookupNodeService("tier"), + new DataNodeService("tier", 1000, ServerType.INDEXER_EXECUTOR, 0) ); return toolboxFactory.build(task); diff --git a/indexing-service/src/test/java/io/druid/indexing/common/task/SameIntervalMergeTaskTest.java b/indexing-service/src/test/java/io/druid/indexing/common/task/SameIntervalMergeTaskTest.java index 4dfb87641576..265e66a177cd 100644 --- a/indexing-service/src/test/java/io/druid/indexing/common/task/SameIntervalMergeTaskTest.java +++ b/indexing-service/src/test/java/io/druid/indexing/common/task/SameIntervalMergeTaskTest.java @@ -233,7 +233,8 @@ public void cleanup(DataSegment segment) throws SegmentLoadingException { } }, jsonMapper, temporaryFolder.newFolder(), - indexIO, null, null, EasyMock.createMock(IndexMergerV9.class) + indexIO, null, null, EasyMock.createMock(IndexMergerV9.class), + null, null, null, null ) ); diff --git a/indexing-service/src/test/java/io/druid/indexing/firehose/IngestSegmentFirehoseFactoryTest.java b/indexing-service/src/test/java/io/druid/indexing/firehose/IngestSegmentFirehoseFactoryTest.java index 067ec8b4cd41..7b8b0c33c659 100644 --- a/indexing-service/src/test/java/io/druid/indexing/firehose/IngestSegmentFirehoseFactoryTest.java +++ b/indexing-service/src/test/java/io/druid/indexing/firehose/IngestSegmentFirehoseFactoryTest.java @@ -292,7 +292,11 @@ public List getLocations() INDEX_IO, null, null, - INDEX_MERGER_V9 + INDEX_MERGER_V9, + null, + null, + null, + null ); Collection values = new LinkedList<>(); for (InputRowParser parser : Arrays.asList( diff --git a/indexing-service/src/test/java/io/druid/indexing/firehose/IngestSegmentFirehoseFactoryTimelineTest.java b/indexing-service/src/test/java/io/druid/indexing/firehose/IngestSegmentFirehoseFactoryTimelineTest.java index ffcddf29e95b..668564a05790 100644 --- a/indexing-service/src/test/java/io/druid/indexing/firehose/IngestSegmentFirehoseFactoryTimelineTest.java +++ b/indexing-service/src/test/java/io/druid/indexing/firehose/IngestSegmentFirehoseFactoryTimelineTest.java @@ -333,7 +333,11 @@ public List getLocations() INDEX_IO, null, null, - INDEX_MERGER_V9 + INDEX_MERGER_V9, + null, + null, + null, + null ); final Injector injector = Guice.createInjector( new Module() diff --git a/indexing-service/src/test/java/io/druid/indexing/overlord/TaskLifecycleTest.java b/indexing-service/src/test/java/io/druid/indexing/overlord/TaskLifecycleTest.java index 569590cbbcd9..1aa8a3f51ba4 100644 --- a/indexing-service/src/test/java/io/druid/indexing/overlord/TaskLifecycleTest.java +++ b/indexing-service/src/test/java/io/druid/indexing/overlord/TaskLifecycleTest.java @@ -45,6 +45,9 @@ import io.druid.data.input.InputRow; import io.druid.data.input.MapBasedInputRow; import io.druid.data.input.impl.InputRowParser; +import io.druid.discovery.DataNodeService; +import io.druid.discovery.DruidNodeAnnouncer; +import io.druid.discovery.LookupNodeService; import io.druid.indexing.common.SegmentLoaderFactory; import io.druid.indexing.common.TaskLock; import io.druid.indexing.common.TaskStatus; @@ -107,6 +110,7 @@ import io.druid.server.DruidNode; import io.druid.server.coordination.DataSegmentAnnouncer; import io.druid.server.coordination.DataSegmentServerAnnouncer; +import io.druid.server.coordination.ServerType; import io.druid.server.initialization.ServerConfig; import io.druid.server.metrics.NoopServiceEmitter; import io.druid.timeline.DataSegment; @@ -600,7 +604,11 @@ public List getLocations() INDEX_IO, MapCache.create(0), FireDepartmentTest.NO_CACHE_CONFIG, - INDEX_MERGER_V9 + INDEX_MERGER_V9, + EasyMock.createNiceMock(DruidNodeAnnouncer.class), + EasyMock.createNiceMock(DruidNode.class), + new LookupNodeService("tier"), + new DataNodeService("tier", 1000, ServerType.INDEXER_EXECUTOR, 0) ); } diff --git a/indexing-service/src/test/java/io/druid/indexing/worker/WorkerTaskMonitorTest.java b/indexing-service/src/test/java/io/druid/indexing/worker/WorkerTaskMonitorTest.java index ef4bdba34737..82cacf88dcde 100644 --- a/indexing-service/src/test/java/io/druid/indexing/worker/WorkerTaskMonitorTest.java +++ b/indexing-service/src/test/java/io/druid/indexing/worker/WorkerTaskMonitorTest.java @@ -187,7 +187,11 @@ public List getLocations() indexIO, null, null, - indexMergerV9 + indexMergerV9, + null, + null, + null, + null ), taskConfig, new NoopServiceEmitter(), diff --git a/server/src/main/java/io/druid/client/DruidServerDiscovery.java b/server/src/main/java/io/druid/client/DruidServerDiscovery.java deleted file mode 100644 index 7b28ef6850ae..000000000000 --- a/server/src/main/java/io/druid/client/DruidServerDiscovery.java +++ /dev/null @@ -1,174 +0,0 @@ -/* - * Licensed to Metamarkets Group Inc. (Metamarkets) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. Metamarkets 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 io.druid.client; - -import com.fasterxml.jackson.databind.ObjectMapper; -import com.google.common.base.Preconditions; -import com.google.common.base.Throwables; -import com.metamx.emitter.EmittingLogger; -import io.druid.concurrent.Execs; -import io.druid.curator.inventory.CuratorInventoryManager; -import io.druid.curator.inventory.CuratorInventoryManagerStrategy; -import io.druid.curator.inventory.InventoryManagerConfig; -import io.druid.java.util.common.ISE; -import org.apache.curator.framework.CuratorFramework; - -import java.io.IOException; - -/** - * Discovers DruidServer instances that serve segments using CuratorInventoryManager. - */ -public class DruidServerDiscovery -{ - private final EmittingLogger log = new EmittingLogger(DruidServerDiscovery.class); - private final CuratorInventoryManager curatorInventoryManager; - private volatile Listener listener; - - DruidServerDiscovery( - final CuratorFramework curatorFramework, - final String announcementsPath, - final ObjectMapper jsonMapper - ) - { - curatorInventoryManager = initCuratorInventoryManager(curatorFramework, announcementsPath, jsonMapper); - } - - public void start() throws Exception - { - Preconditions.checkNotNull(listener, "listener is not configured yet"); - curatorInventoryManager.start(); - } - - public void stop() throws IOException - { - curatorInventoryManager.stop(); - } - - private CuratorInventoryManager initCuratorInventoryManager( - final CuratorFramework curator, - final String announcementsPath, - final ObjectMapper jsonMapper - ) - { - return new CuratorInventoryManager<>( - curator, - new InventoryManagerConfig() - { - @Override - public String getContainerPath() - { - return announcementsPath; - } - - @Override - public String getInventoryPath() - { - return "/NON_EXISTENT_DUMMY_INVENTORY_PATH"; - } - }, - Execs.singleThreaded("CuratorInventoryManagerBasedServerDiscovery-%s"), - new CuratorInventoryManagerStrategy() - { - @Override - public DruidServer deserializeContainer(byte[] bytes) - { - try { - return jsonMapper.readValue(bytes, DruidServer.class); - } - catch (IOException e) { - throw Throwables.propagate(e); - } - } - - @Override - public void newContainer(DruidServer container) - { - log.info("New Server[%s]", container.getName()); - listener.serverAdded(container); - } - - @Override - public void deadContainer(DruidServer container) - { - log.info("Server Disappeared[%s]", container.getName()); - listener.serverRemoved(container); - } - - @Override - public DruidServer updateContainer(DruidServer oldContainer, DruidServer newContainer) - { - log.info("Server updated[%s]", oldContainer.getName()); - return listener.serverUpdated(oldContainer, newContainer); - } - - @Override - public Object deserializeInventory(byte[] bytes) - { - throw new ISE("no inventory should exist."); - } - - @Override - public DruidServer addInventory( - final DruidServer container, - String inventoryKey, - final Object inventory - ) - { - throw new ISE("no inventory should exist."); - } - - @Override - public DruidServer updateInventory( - DruidServer container, String inventoryKey, Object inventory - ) - { - throw new ISE("no inventory should exist."); - } - - @Override - public DruidServer removeInventory(final DruidServer container, String inventoryKey) - { - throw new ISE("no inventory should exist."); - } - - @Override - public void inventoryInitialized() - { - log.info("Server inventory initialized."); - listener.initialized(); - } - } - ); - } - - public void registerListener(Listener listener) - { - Preconditions.checkArgument(this.listener == null, "listener registered already."); - this.listener = listener; - } - - public interface Listener - { - void serverAdded(DruidServer server); - DruidServer serverUpdated(DruidServer oldServer, DruidServer newServer); - void serverRemoved(DruidServer server); - void initialized(); - } -} diff --git a/server/src/main/java/io/druid/client/FilteredHttpServerInventoryViewProvider.java b/server/src/main/java/io/druid/client/FilteredHttpServerInventoryViewProvider.java index 6dbb7cf4bcb6..2741739ce429 100644 --- a/server/src/main/java/io/druid/client/FilteredHttpServerInventoryViewProvider.java +++ b/server/src/main/java/io/druid/client/FilteredHttpServerInventoryViewProvider.java @@ -23,14 +23,13 @@ import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.base.Predicates; import com.metamx.http.client.HttpClient; +import io.druid.discovery.DruidNodeDiscoveryProvider; import io.druid.guice.annotations.Client; import io.druid.guice.annotations.Json; import io.druid.guice.annotations.Smile; import io.druid.java.util.common.Pair; import io.druid.server.coordination.DruidServerMetadata; -import io.druid.server.initialization.ZkPathsConfig; import io.druid.timeline.DataSegment; -import org.apache.curator.framework.CuratorFramework; import javax.validation.constraints.NotNull; @@ -59,18 +58,14 @@ public class FilteredHttpServerInventoryViewProvider implements FilteredServerIn @JacksonInject @NotNull - private ZkPathsConfig zkPaths = null; - - @JacksonInject - @NotNull - private CuratorFramework curator = null; + private DruidNodeDiscoveryProvider druidNodeDiscoveryProvider = null; @Override public HttpServerInventoryView get() { return new HttpServerInventoryView( jsonMapper, smileMapper, httpClient, - new DruidServerDiscovery(curator, zkPaths.getAnnouncementsPath(), jsonMapper), + druidNodeDiscoveryProvider, Predicates.>alwaysTrue(), config ); diff --git a/server/src/main/java/io/druid/client/HttpServerInventoryView.java b/server/src/main/java/io/druid/client/HttpServerInventoryView.java index 03390f982eaa..201276ed473e 100644 --- a/server/src/main/java/io/druid/client/HttpServerInventoryView.java +++ b/server/src/main/java/io/druid/client/HttpServerInventoryView.java @@ -40,6 +40,10 @@ import com.metamx.http.client.response.ClientResponse; import com.metamx.http.client.response.InputStreamResponseHandler; import io.druid.concurrent.LifecycleLock; +import io.druid.discovery.DataNodeService; +import io.druid.discovery.DiscoveryDruidNode; +import io.druid.discovery.DruidNodeDiscovery; +import io.druid.discovery.DruidNodeDiscoveryProvider; import io.druid.guice.annotations.Global; import io.druid.guice.annotations.Json; import io.druid.guice.annotations.Smile; @@ -83,7 +87,7 @@ public class HttpServerInventoryView implements ServerInventoryView, FilteredServerInventoryView { private final EmittingLogger log = new EmittingLogger(HttpServerInventoryView.class); - private final DruidServerDiscovery serverDiscovery; + private final DruidNodeDiscoveryProvider druidNodeDiscoveryProvider; private final LifecycleLock lifecycleLock = new LifecycleLock(); @@ -105,8 +109,6 @@ public class HttpServerInventoryView implements ServerInventoryView, FilteredSer // to this queue again for next update. private final BlockingQueue queue = new LinkedBlockingDeque<>(); - - private final HttpClient httpClient; private final ObjectMapper smileMapper; private final HttpServerInventoryViewConfig config; @@ -116,14 +118,14 @@ public HttpServerInventoryView( final @Json ObjectMapper jsonMapper, final @Smile ObjectMapper smileMapper, final @Global HttpClient httpClient, - final DruidServerDiscovery serverDiscovery, + final DruidNodeDiscoveryProvider druidNodeDiscoveryProvider, final Predicate> defaultFilter, final HttpServerInventoryViewConfig config ) { this.httpClient = httpClient; this.smileMapper = smileMapper; - this.serverDiscovery = serverDiscovery; + this.druidNodeDiscoveryProvider = druidNodeDiscoveryProvider; this.defaultFilter = defaultFilter; this.finalPredicate = defaultFilter; this.config = config; @@ -178,36 +180,38 @@ public void run() } ); - serverDiscovery.registerListener( - new DruidServerDiscovery.Listener() + DruidNodeDiscovery druidNodeDiscovery = druidNodeDiscoveryProvider.getForService(DataNodeService.DISCOVERY_SERVICE_KEY); + druidNodeDiscovery.registerListener( + new DruidNodeDiscovery.Listener() { - @Override - public void serverAdded(DruidServer server) - { - serverAddedOrUpdated(server); - } @Override - public DruidServer serverUpdated(DruidServer oldServer, DruidServer newServer) + public void nodeAdded(DiscoveryDruidNode node) { - return serverAddedOrUpdated(newServer); + serverAddedOrUpdated(toDruidServer(node)); } @Override - public void serverRemoved(DruidServer server) + public void nodeRemoved(DiscoveryDruidNode node) { - HttpServerInventoryView.this.serverRemoved(server); - runServerCallbacks(server); + serverRemoved(toDruidServer(node)); } - @Override - public void initialized() + private DruidServer toDruidServer(DiscoveryDruidNode node) { - serverInventoryInitialized(); + + return new DruidServer( + node.getDruidNode().getHostAndPortToUse(), + node.getDruidNode().getHostAndPort(), + node.getDruidNode().getHostAndTlsPort(), + ((DataNodeService) node.getServices().get(DataNodeService.DISCOVERY_SERVICE_KEY)).getMaxSize(), + ((DataNodeService) node.getServices().get(DataNodeService.DISCOVERY_SERVICE_KEY)).getType(), + ((DataNodeService) node.getServices().get(DataNodeService.DISCOVERY_SERVICE_KEY)).getTier(), + ((DataNodeService) node.getServices().get(DataNodeService.DISCOVERY_SERVICE_KEY)).getPriority() + ); } } ); - serverDiscovery.start(); log.info("Started HttpServerInventoryView."); lifecycleLock.started(); @@ -228,8 +232,6 @@ public void stop() throws IOException log.info("Stopping HttpServerInventoryView."); - serverDiscovery.stop(); - if (executor != null) { executor.shutdownNow(); executor = null; @@ -373,11 +375,6 @@ private void serverRemoved(DruidServer server) servers.remove(server.getName()); } - public DruidServer serverUpdated(DruidServer oldServer, DruidServer newServer) - { - return serverAddedOrUpdated(newServer); - } - @Override public boolean isStarted() { diff --git a/server/src/main/java/io/druid/client/HttpServerInventoryViewProvider.java b/server/src/main/java/io/druid/client/HttpServerInventoryViewProvider.java index efc109542ec2..13a71f0928e7 100644 --- a/server/src/main/java/io/druid/client/HttpServerInventoryViewProvider.java +++ b/server/src/main/java/io/druid/client/HttpServerInventoryViewProvider.java @@ -23,14 +23,13 @@ import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.base.Predicates; import com.metamx.http.client.HttpClient; +import io.druid.discovery.DruidNodeDiscoveryProvider; import io.druid.guice.annotations.Client; import io.druid.guice.annotations.Json; import io.druid.guice.annotations.Smile; import io.druid.java.util.common.Pair; import io.druid.server.coordination.DruidServerMetadata; -import io.druid.server.initialization.ZkPathsConfig; import io.druid.timeline.DataSegment; -import org.apache.curator.framework.CuratorFramework; import javax.validation.constraints.NotNull; @@ -59,11 +58,7 @@ public class HttpServerInventoryViewProvider implements ServerInventoryViewProvi @JacksonInject @NotNull - private ZkPathsConfig zkPaths = null; - - @JacksonInject - @NotNull - private CuratorFramework curator = null; + private DruidNodeDiscoveryProvider druidNodeDiscoveryProvider = null; @Override public HttpServerInventoryView get() @@ -72,7 +67,7 @@ public HttpServerInventoryView get() jsonMapper, smileMapper, httpClient, - new DruidServerDiscovery(curator, zkPaths.getAnnouncementsPath(), jsonMapper), + druidNodeDiscoveryProvider, Predicates.>alwaysTrue(), config ); diff --git a/server/src/main/java/io/druid/curator/discovery/CuratorDruidNodeAnnouncer.java b/server/src/main/java/io/druid/curator/discovery/CuratorDruidNodeAnnouncer.java new file mode 100644 index 000000000000..f8c6f1f48ca0 --- /dev/null +++ b/server/src/main/java/io/druid/curator/discovery/CuratorDruidNodeAnnouncer.java @@ -0,0 +1,93 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets 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 io.druid.curator.discovery; + +import com.fasterxml.jackson.core.JsonProcessingException; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.base.Throwables; +import com.google.inject.Inject; +import io.druid.curator.announcement.Announcer; +import io.druid.discovery.DiscoveryDruidNode; +import io.druid.discovery.DruidNodeAnnouncer; +import io.druid.guice.annotations.Json; +import io.druid.java.util.common.logger.Logger; +import io.druid.server.initialization.CuratorDiscoveryConfig; +import org.apache.curator.utils.ZKPaths; + +/** + */ +public class CuratorDruidNodeAnnouncer implements DruidNodeAnnouncer +{ + private static final Logger log = new Logger(CuratorDruidNodeAnnouncer.class); + + private final Announcer announcer; + private final CuratorDiscoveryConfig config; + private final ObjectMapper jsonMapper; + + @Inject + public CuratorDruidNodeAnnouncer( + Announcer announcer, + CuratorDiscoveryConfig config, + @Json ObjectMapper jsonMapper + ) + { + this.announcer = announcer; + this.config = config; + this.jsonMapper = jsonMapper; + } + + @Override + public synchronized void announce(DiscoveryDruidNode discoveryDruidNode) + { + try { + log.info("Announcing [%s].", discoveryDruidNode); + + announcer.announce( + ZKPaths.makePath( + config.getInternalDiscoveryPath(), + discoveryDruidNode.getNodeType(), + discoveryDruidNode.getDruidNode().getHostAndPortToUse() + ), + jsonMapper.writeValueAsBytes(discoveryDruidNode) + ); + + log.info("Announced [%s].", discoveryDruidNode); + } + catch (JsonProcessingException e) { + throw Throwables.propagate(e); + } + } + + @Override + public synchronized void unannounce(DiscoveryDruidNode discoveryDruidNode) + { + log.info("Unannouncing [%s].", discoveryDruidNode); + + announcer.unannounce( + ZKPaths.makePath( + config.getInternalDiscoveryPath(), + discoveryDruidNode.getNodeType(), + discoveryDruidNode.getDruidNode().getHostAndPortToUse() + ) + ); + + log.info("Unannounced [%s].", discoveryDruidNode); + } +} diff --git a/server/src/main/java/io/druid/curator/discovery/CuratorDruidNodeDiscoveryProvider.java b/server/src/main/java/io/druid/curator/discovery/CuratorDruidNodeDiscoveryProvider.java new file mode 100644 index 000000000000..b7143a465cd2 --- /dev/null +++ b/server/src/main/java/io/druid/curator/discovery/CuratorDruidNodeDiscoveryProvider.java @@ -0,0 +1,327 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets 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 io.druid.curator.discovery; + +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.base.Throwables; +import com.google.common.collect.ImmutableSet; +import com.google.inject.Inject; +import io.druid.concurrent.Execs; +import io.druid.discovery.DiscoveryDruidNode; +import io.druid.discovery.DruidNodeDiscovery; +import io.druid.discovery.DruidNodeDiscoveryProvider; +import io.druid.guice.ManageLifecycle; +import io.druid.guice.annotations.Json; +import io.druid.java.util.common.ISE; +import io.druid.java.util.common.lifecycle.LifecycleStart; +import io.druid.java.util.common.lifecycle.LifecycleStop; +import io.druid.java.util.common.logger.Logger; +import io.druid.server.initialization.CuratorDiscoveryConfig; +import org.apache.curator.framework.CuratorFramework; +import org.apache.curator.framework.recipes.cache.PathChildrenCache; +import org.apache.curator.framework.recipes.cache.PathChildrenCacheEvent; +import org.apache.curator.framework.recipes.cache.PathChildrenCacheListener; +import org.apache.curator.utils.ZKPaths; + +import java.util.ArrayList; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ExecutorService; + +/** + */ +@ManageLifecycle +public class CuratorDruidNodeDiscoveryProvider extends DruidNodeDiscoveryProvider +{ + private static final Logger log = new Logger(CuratorDruidNodeDiscoveryProvider.class); + + private final CuratorFramework curatorFramework; + private final CuratorDiscoveryConfig config; + private final ObjectMapper jsonMapper; + + private final ExecutorService listenerExecutor; + + private final Map nodeTypeWatchers = new ConcurrentHashMap<>(); + + private boolean stopped = false; + + @Inject + public CuratorDruidNodeDiscoveryProvider( + CuratorFramework curatorFramework, + CuratorDiscoveryConfig config, + @Json ObjectMapper jsonMapper + ) + { + // This is single-threaded to ensure that all listener calls are executed precisesly in the oder of add/remove + // event occurences. + this.listenerExecutor = Execs.singleThreaded("CuratorDruidNodeDiscoveryProvider-ListenerExecutor"); + this.curatorFramework = curatorFramework; + this.config = config; + this.jsonMapper = jsonMapper; + } + + + + @Override + public synchronized DruidNodeDiscovery getForNodeType(String nodeType) + { + if (stopped) { + throw new ISE("Provider has been stopped."); + } + + NodeTypeWatcher nodeTypeWatcher = nodeTypeWatchers.get(nodeType); + if (nodeTypeWatcher == null) { + log.info("Creating/Starting NodeTypeWatcher for nodeType [%s].", nodeType); + nodeTypeWatcher = new NodeTypeWatcher( + listenerExecutor, + curatorFramework, + config.getInternalDiscoveryPath(), + jsonMapper, + nodeType + ); + nodeTypeWatcher.start(); + nodeTypeWatchers.put(nodeType, nodeTypeWatcher); + log.info("Created/Started NodeTypeWatcher for nodeType [%s].", nodeType); + } + + return nodeTypeWatchers.get(nodeType); + } + + @LifecycleStart + public synchronized void start() + { + log.info("started"); + } + + @LifecycleStop + public synchronized void stop() + { + if (stopped) { + return; + } + + log.info("stopping"); + + stopped = true; + + for (NodeTypeWatcher watcher : nodeTypeWatchers.values()) { + watcher.stop(); + } + listenerExecutor.shutdownNow(); + + log.info("stopped"); + } + + private static class NodeTypeWatcher implements DruidNodeDiscovery, PathChildrenCacheListener + { + private static final Logger log = new Logger(NodeTypeWatcher.class); + + private final CuratorFramework curatorFramework; + + private final String nodeType; + private final ObjectMapper jsonMapper; + + // hostAndPort -> DiscoveryDruidNode + private final Map nodes = new ConcurrentHashMap<>(); + + private final PathChildrenCache cache; + private final ExecutorService cacheExecutor; + + private final ExecutorService listenerExecutor; + + private final List nodeListeners = new ArrayList(); + + NodeTypeWatcher( + ExecutorService listenerExecutor, + CuratorFramework curatorFramework, + String basePath, + ObjectMapper jsonMapper, + String nodeType + ) + { + this.listenerExecutor = listenerExecutor; + this.curatorFramework = curatorFramework; + this.nodeType = nodeType; + this.jsonMapper = jsonMapper; + + // This is required to be single threaded from Docs in PathChildrenCache; + this.cacheExecutor = Execs.singleThreaded(String.format("NodeTypeWatcher[%s]-[%%d]", nodeType)); + this.cache = new PathChildrenCache( + curatorFramework, + ZKPaths.makePath(basePath, nodeType), + true, + true, + cacheExecutor + ); + } + + @Override + public Set getAllNodes() + { + return new ImmutableSet.Builder().addAll(nodes.values()).build(); + } + + @Override + public synchronized void registerListener(DruidNodeDiscovery.Listener listener) + { + for (DiscoveryDruidNode node : nodes.values()) { + listenerExecutor.submit(() -> { + try { + listener.nodeAdded(node); + } + catch (Exception ex) { + log.error(ex, "Exception occured in DiscoveryDruidNode.nodeAdded(node=[%s]) in listener [%s].", node, listener); + } + }); + } + + nodeListeners.add(listener); + } + + @Override + public synchronized void childEvent(CuratorFramework client, PathChildrenCacheEvent event) throws Exception + { + 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.error( + "Node[%s:%s] add is discovered by node watcher of nodeType [%s]. Ignored.", + druidNode.getNodeType(), + druidNode, + nodeType + ); + return; + } + + log.info("Received event [%s] for Node[%s:%s].", event.getType(), druidNode.getNodeType(), druidNode); + + addNode(druidNode); + + break; + } + case CHILD_REMOVED: { + DiscoveryDruidNode druidNode = jsonMapper.readValue(event.getData().getData(), DiscoveryDruidNode.class); + + if (!nodeType.equals(druidNode.getNodeType())) { + log.error( + "Node[%s:%s] removal is discovered by node watcher of nodeType [%s]. Ignored.", + druidNode.getNodeType(), + druidNode, + nodeType + ); + return; + } + + log.info("Node[%s:%s] disappeared.", druidNode.getNodeType(), druidNode); + + removeNode(druidNode); + + break; + } + default: { + log.error("Ignored event type [%s] for nodeType [%s] watcher.", event.getType(), nodeType); + } + } + } + catch (Exception ex) { + log.error(ex, "unknown error in node watcher for type [%s].", nodeType); + } + } + + private synchronized void addNode(DiscoveryDruidNode druidNode) + { + DiscoveryDruidNode prev = nodes.put(druidNode.getDruidNode().getHostAndPortToUse(), druidNode); + if (prev == null) { + for (DruidNodeDiscovery.Listener l : nodeListeners) { + listenerExecutor.submit(() -> { + try { + l.nodeAdded(druidNode); + } + catch (Exception ex) { + log.error(ex, "Exception occured in DiscoveryDruidNode.nodeAdded(node=[%s]) in listener [%s].", druidNode, l); + } + }); + } + } else { + log.warn("Node[%s] discovered but existed already [%s].", druidNode, prev); + } + } + + private synchronized void removeNode(DiscoveryDruidNode druidNode) + { + DiscoveryDruidNode prev = nodes.remove(druidNode.getDruidNode().getHostAndPortToUse()); + + if (prev == null) { + log.warn("Noticed disappearance of unknown druid node [%s:%s].", druidNode.getNodeType(), druidNode); + return; + } + + for (DruidNodeDiscovery.Listener l : nodeListeners) { + listenerExecutor.submit(() -> { + try { + l.nodeRemoved(druidNode); + } + catch (Exception ex) { + log.error(ex, "Exception occured in DiscoveryDruidNode.nodeRemoved(node=[%s]) in listener [%s].", druidNode, l); + } + }); + } + } + + public void start() + { + try { + cache.getListenable().addListener(this); + cache.start(); + } + catch (Exception ex) { + throw Throwables.propagate(ex); + } + } + + public void stop() + { + try { + cache.close(); + cacheExecutor.shutdownNow(); + } + catch (Exception ex) { + log.error(ex, "Failed to stop node watcher for type [%s].", nodeType); + } + } + } +} diff --git a/server/src/main/java/io/druid/curator/discovery/DiscoveryModule.java b/server/src/main/java/io/druid/curator/discovery/DiscoveryModule.java index 50eb29f8e875..ee1a6dc86628 100644 --- a/server/src/main/java/io/druid/curator/discovery/DiscoveryModule.java +++ b/server/src/main/java/io/druid/curator/discovery/DiscoveryModule.java @@ -30,12 +30,15 @@ import com.google.inject.TypeLiteral; import com.google.inject.name.Named; import com.google.inject.name.Names; - +import io.druid.discovery.DruidDiscoveryModule; +import io.druid.discovery.DruidNodeDiscoveryProvider; +import io.druid.discovery.DruidNodeAnnouncer; import io.druid.guice.DruidBinders; import io.druid.guice.JsonConfigProvider; import io.druid.guice.KeyHolder; import io.druid.guice.LazySingleton; import io.druid.guice.LifecycleModule; +import io.druid.guice.PolyBind; import io.druid.java.util.common.lifecycle.Lifecycle; import io.druid.server.DruidNode; import io.druid.server.initialization.CuratorDiscoveryConfig; @@ -73,6 +76,7 @@ public class DiscoveryModule implements Module { private static final String NAME = "DiscoveryModule:internal"; + private static final String CURATOR_KEY = "curator"; /** * Requests that the un-annotated DruidNode instance be injected and published as part of the lifecycle. @@ -146,6 +150,19 @@ public void configure(Binder binder) binder.bind(ServiceAnnouncer.class) .to(Key.get(CuratorServiceAnnouncer.class, Names.named(NAME))) .in(LazySingleton.class); + + // internal discovery bindings. + DruidDiscoveryModule.createBindingChoices(binder, CURATOR_KEY); + + PolyBind.optionBinder(binder, Key.get(DruidNodeDiscoveryProvider.class)) + .addBinding(CURATOR_KEY) + .to(CuratorDruidNodeDiscoveryProvider.class) + .in(LazySingleton.class); + + PolyBind.optionBinder(binder, Key.get(DruidNodeAnnouncer.class)) + .addBinding(CURATOR_KEY) + .to(CuratorDruidNodeAnnouncer.class) + .in(LazySingleton.class); } @Provides diff --git a/server/src/main/java/io/druid/discovery/DataNodeService.java b/server/src/main/java/io/druid/discovery/DataNodeService.java new file mode 100644 index 000000000000..cd6033d02278 --- /dev/null +++ b/server/src/main/java/io/druid/discovery/DataNodeService.java @@ -0,0 +1,117 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets 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 io.druid.discovery; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import io.druid.server.coordination.ServerType; + +import java.util.Objects; + +/** + * Metadata announced by any node that serves segments. + */ +public class DataNodeService extends DruidService +{ + public static final String DISCOVERY_SERVICE_KEY = "dataNodeService"; + + private final String tier; + private final long maxSize; + private final ServerType type; + private final int priority; + + @JsonCreator + public DataNodeService( + @JsonProperty("tier") String tier, + @JsonProperty("maxSize") long maxSize, + @JsonProperty("type") ServerType type, + @JsonProperty("priority") int priority + ) + { + this.tier = tier; + this.maxSize = maxSize; + this.type = type; + this.priority = priority; + } + + @Override + public String getName() + { + return DISCOVERY_SERVICE_KEY; + } + + @JsonProperty + public String getTier() + { + return tier; + } + + @JsonProperty + public long getMaxSize() + { + return maxSize; + } + + @JsonProperty + public ServerType getType() + { + return type; + } + + @JsonProperty + public int getPriority() + { + return priority; + } + + @Override + public boolean equals(Object o) + { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + DataNodeService that = (DataNodeService) o; + return maxSize == that.maxSize && + priority == that.priority && + Objects.equals(tier, that.tier) && + type == that.type; + } + + @Override + public int hashCode() + { + return Objects.hash(tier, maxSize, type, priority); + } + + @Override + public String toString() + { + return "DataNodeService{" + + "tier='" + tier + '\'' + + ", maxSize=" + maxSize + + ", type='" + type + '\'' + + ", priority=" + priority + + ", " + super.toString() + + '}'; + } +} diff --git a/server/src/main/java/io/druid/discovery/DiscoveryDruidNode.java b/server/src/main/java/io/druid/discovery/DiscoveryDruidNode.java new file mode 100644 index 000000000000..c6ab1fcbf6f6 --- /dev/null +++ b/server/src/main/java/io/druid/discovery/DiscoveryDruidNode.java @@ -0,0 +1,108 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets 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 io.druid.discovery; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import io.druid.server.DruidNode; + +import java.util.HashMap; +import java.util.Map; +import java.util.Objects; + +/** + * 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. + */ +public class DiscoveryDruidNode +{ + private final DruidNode druidNode; + private final String nodeType; + + // Other metadata associated with the node e.g. + // if its a historical node then lookup information, segment loading capacity etc. + private final Map services = new HashMap<>(); + + @JsonCreator + public DiscoveryDruidNode( + @JsonProperty("druidNode") DruidNode druidNode, + @JsonProperty("nodeType") String nodeType, + @JsonProperty("services") Map services + ) + { + this.druidNode = druidNode; + this.nodeType = nodeType; + + if (services != null && !services.isEmpty()) { + this.services.putAll(services); + } + } + + @JsonProperty + public Map getServices() + { + return services; + } + + @JsonProperty + public String getNodeType() + { + return nodeType; + } + + @JsonProperty + public DruidNode getDruidNode() + { + return druidNode; + } + + @Override + public boolean equals(Object o) + { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + DiscoveryDruidNode that = (DiscoveryDruidNode) o; + return Objects.equals(druidNode, that.druidNode) && + Objects.equals(nodeType, that.nodeType) && + Objects.equals(services, that.services); + } + + @Override + public int hashCode() + { + return Objects.hash(druidNode, nodeType, services); + } + + @Override + public String toString() + { + return "DiscoveryDruidNode{" + + "druidNode=" + druidNode + + ", nodeType='" + nodeType + '\'' + + ", services=" + services + + '}'; + } +} diff --git a/server/src/main/java/io/druid/discovery/DruidDiscoveryModule.java b/server/src/main/java/io/druid/discovery/DruidDiscoveryModule.java new file mode 100644 index 000000000000..01a71531fcf5 --- /dev/null +++ b/server/src/main/java/io/druid/discovery/DruidDiscoveryModule.java @@ -0,0 +1,49 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets 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 io.druid.discovery; + +import com.google.inject.Binder; +import com.google.inject.Key; +import com.google.inject.Module; +import io.druid.guice.PolyBind; + +/** + */ +public class DruidDiscoveryModule implements Module +{ + public static final String PROPERTY = "druid.discovery.type"; + + @Override + public void configure(Binder binder) + { + + } + + public static void createBindingChoices(Binder binder, String defaultPropertyValue) + { + PolyBind.createChoiceWithDefault( + binder, PROPERTY, Key.get(DruidNodeAnnouncer.class), defaultPropertyValue + ); + + PolyBind.createChoiceWithDefault( + binder, PROPERTY, Key.get(DruidNodeDiscoveryProvider.class), defaultPropertyValue + ); + } +} diff --git a/server/src/main/java/io/druid/discovery/DruidNodeAnnouncer.java b/server/src/main/java/io/druid/discovery/DruidNodeAnnouncer.java new file mode 100644 index 000000000000..78892ba51d59 --- /dev/null +++ b/server/src/main/java/io/druid/discovery/DruidNodeAnnouncer.java @@ -0,0 +1,29 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets 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 io.druid.discovery; + +/** + * DiscoveryDruidNode announcer for internal discovery. + */ +public interface DruidNodeAnnouncer +{ + void announce(DiscoveryDruidNode discoveryDruidNode); + void unannounce(DiscoveryDruidNode discoveryDruidNode); +} diff --git a/server/src/main/java/io/druid/discovery/DruidNodeDiscovery.java b/server/src/main/java/io/druid/discovery/DruidNodeDiscovery.java new file mode 100644 index 000000000000..d057c82cee83 --- /dev/null +++ b/server/src/main/java/io/druid/discovery/DruidNodeDiscovery.java @@ -0,0 +1,37 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets 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 io.druid.discovery; + +import java.util.Set; + +/** + * Interface for discovering Druid Nodes announced by DruidNodeAnnouncer. + */ +public interface DruidNodeDiscovery +{ + Set getAllNodes(); + void registerListener(Listener listener); + + interface Listener + { + void nodeAdded(DiscoveryDruidNode node); + void nodeRemoved(DiscoveryDruidNode node); + } +} diff --git a/server/src/main/java/io/druid/discovery/DruidNodeDiscoveryProvider.java b/server/src/main/java/io/druid/discovery/DruidNodeDiscoveryProvider.java new file mode 100644 index 000000000000..26c4545ffada --- /dev/null +++ b/server/src/main/java/io/druid/discovery/DruidNodeDiscoveryProvider.java @@ -0,0 +1,142 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets 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 io.druid.discovery; + +import com.google.common.collect.ImmutableMap; +import com.google.common.collect.ImmutableSet; +import io.druid.java.util.common.IAE; +import io.druid.java.util.common.logger.Logger; + +import java.util.ArrayList; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.concurrent.ConcurrentHashMap; + +/** + * Provider of DruidNodeDiscovery instances. + */ +public abstract class DruidNodeDiscoveryProvider +{ + private static final Logger log = new Logger(DruidNodeDiscoveryProvider.class); + + public static final String NODE_TYPE_COORDINATOR = "coordinator"; + public static final String NODE_TYPE_HISTORICAL = "historical"; + public static final String NODE_TYPE_BROKER = "broker"; + public static final String NODE_TYPE_OVERLORD = "overlord"; + public static final String NODE_TYPE_PEON = "peon"; + public static final String NODE_TYPE_ROUTER = "router"; + public static final String NODE_TYPE_MM = "middleManager"; + + public static final Map> SERVICE_TO_NODE_TYPES = ImmutableMap.of( + LookupNodeService.DISCOVERY_SERVICE_KEY, ImmutableSet.of(NODE_TYPE_BROKER, NODE_TYPE_HISTORICAL, NODE_TYPE_PEON), + DataNodeService.DISCOVERY_SERVICE_KEY, ImmutableSet.of(NODE_TYPE_HISTORICAL, NODE_TYPE_PEON), + WorkerNodeService.DISCOVERY_SERVICE_KEY, ImmutableSet.of(NODE_TYPE_MM) + ); + + private Map serviceDiscoveryMap = new ConcurrentHashMap<>(SERVICE_TO_NODE_TYPES.size()); + + /** + * Get DruidNodeDiscovery instance to discover nodes of given nodeType. + */ + public abstract DruidNodeDiscovery getForNodeType(String nodeType); + + /** + * Get DruidNodeDiscovery instance to discover nodes that announce given service in its metadata. + */ + public synchronized DruidNodeDiscovery getForService(String serviceName) + { + ServiceListener nodeDiscovery = serviceDiscoveryMap.get(serviceName); + + if (nodeDiscovery == null) { + Set nodeTypesToWatch = DruidNodeDiscoveryProvider.SERVICE_TO_NODE_TYPES.get(serviceName); + if (nodeTypesToWatch == null) { + throw new IAE("Unknown service [%s].", serviceName); + } + + nodeDiscovery = new ServiceListener(serviceName); + for (String nodeType : nodeTypesToWatch) { + getForNodeType(nodeType).registerListener(nodeDiscovery); + } + serviceDiscoveryMap.put(serviceName, nodeDiscovery); + } + + return nodeDiscovery; + } + + private static class ServiceListener implements DruidNodeDiscovery, DruidNodeDiscovery.Listener + { + private final String service; + private final Map nodes = new ConcurrentHashMap<>(); + + private final List listeners = new ArrayList(); + + ServiceListener(String service) + { + this.service = service; + } + + @Override + public synchronized void nodeAdded(DiscoveryDruidNode node) + { + if (node.getServices().containsKey(service)) { + DiscoveryDruidNode prev = nodes.putIfAbsent(node.getDruidNode().getHostAndPortToUse(), node); + + if (prev == null) { + for (Listener listener : listeners) { + listener.nodeAdded(node); + } + } else { + log.warn("Node[%s] discovered but already exists [%s].", node, prev); + } + } else { + log.warn("Node[%s] discovered but doesn't have service[%s]. Ignored.", node, service); + } + } + + @Override + public synchronized void nodeRemoved(DiscoveryDruidNode node) + { + DiscoveryDruidNode prev = nodes.remove(node.getDruidNode().getHostAndPortToUse()); + if (prev != null) { + for (Listener listener : listeners) { + listener.nodeRemoved(node); + } + } else { + log.warn("Node[%s] disappeared but was unknown for service listener [%s].", node, service); + } + } + + @Override + public Set getAllNodes() + { + return ImmutableSet.builder().addAll(nodes.values()).build(); + } + + @Override + public synchronized void registerListener(Listener listener) + { + for (DiscoveryDruidNode node : nodes.values()) { + listener.nodeAdded(node); + } + listeners.add(listener); + } + } +} diff --git a/server/src/main/java/io/druid/discovery/DruidService.java b/server/src/main/java/io/druid/discovery/DruidService.java new file mode 100644 index 000000000000..6036864966a1 --- /dev/null +++ b/server/src/main/java/io/druid/discovery/DruidService.java @@ -0,0 +1,37 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets 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 io.druid.discovery; + +import com.fasterxml.jackson.annotation.JsonSubTypes; +import com.fasterxml.jackson.annotation.JsonTypeInfo; + +/** + * Metadata of a service announced by node. See DataNodeService and LookupNodeService for examples. + */ +@JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "type") +@JsonSubTypes(value = { + @JsonSubTypes.Type(name = DataNodeService.DISCOVERY_SERVICE_KEY, value = DataNodeService.class), + @JsonSubTypes.Type(name = LookupNodeService.DISCOVERY_SERVICE_KEY, value = LookupNodeService.class), + @JsonSubTypes.Type(name = WorkerNodeService.DISCOVERY_SERVICE_KEY, value = WorkerNodeService.class) +}) +public abstract class DruidService +{ + public abstract String getName(); +} diff --git a/server/src/main/java/io/druid/discovery/LookupNodeService.java b/server/src/main/java/io/druid/discovery/LookupNodeService.java new file mode 100644 index 000000000000..7560795bbad0 --- /dev/null +++ b/server/src/main/java/io/druid/discovery/LookupNodeService.java @@ -0,0 +1,80 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets 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 io.druid.discovery; + +import com.fasterxml.jackson.annotation.JsonProperty; + +import java.util.Objects; + +/** + * Metadata announced by any node that serves queries and hence applies lookups. + */ +public class LookupNodeService extends DruidService +{ + public static final String DISCOVERY_SERVICE_KEY = "lookupNodeService"; + + private final String lookupTier; + + public LookupNodeService( + @JsonProperty("lookupTier") String lookupTier + ) + { + this.lookupTier = lookupTier; + } + + @Override + public String getName() + { + return DISCOVERY_SERVICE_KEY; + } + + @JsonProperty + public String getLookupTier() + { + return lookupTier; + } + + @Override + public boolean equals(Object o) + { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + LookupNodeService that = (LookupNodeService) o; + return Objects.equals(lookupTier, that.lookupTier); + } + + @Override + public int hashCode() + { + return Objects.hash(lookupTier); + } + + @Override + public String toString() + { + return "LookupNodeService{" + + "lookupTier='" + lookupTier + '\'' + + '}'; + } +} diff --git a/server/src/main/java/io/druid/discovery/WorkerNodeService.java b/server/src/main/java/io/druid/discovery/WorkerNodeService.java new file mode 100644 index 000000000000..070f54b22d50 --- /dev/null +++ b/server/src/main/java/io/druid/discovery/WorkerNodeService.java @@ -0,0 +1,102 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets 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 io.druid.discovery; + +import com.fasterxml.jackson.annotation.JsonProperty; + +import java.util.Objects; + +/** + * Worker metadata announced by Middle Manager. + */ +public class WorkerNodeService extends DruidService +{ + public static final String DISCOVERY_SERVICE_KEY = "workerNodeService"; + + private final String ip; + private final int capacity; + private final String version; + + public WorkerNodeService( + @JsonProperty("ip") String ip, + @JsonProperty("capacity") int capacity, + @JsonProperty("version") String version + ) + { + this.ip = ip; + this.capacity = capacity; + this.version = version; + } + + @Override + public String getName() + { + return DISCOVERY_SERVICE_KEY; + } + + @JsonProperty + public String getIp() + { + return ip; + } + + @JsonProperty + public int getCapacity() + { + return capacity; + } + + @JsonProperty + public String getVersion() + { + return version; + } + + @Override + public boolean equals(Object o) + { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + WorkerNodeService that = (WorkerNodeService) o; + return capacity == that.capacity && + Objects.equals(ip, that.ip) && + Objects.equals(version, that.version); + } + + @Override + public int hashCode() + { + return Objects.hash(ip, capacity, version); + } + + @Override + public String toString() + { + return "WorkerNodeService{" + + "ip='" + ip + '\'' + + ", capacity=" + capacity + + ", version='" + version + '\'' + + '}'; + } +} diff --git a/server/src/main/java/io/druid/guice/StorageNodeModule.java b/server/src/main/java/io/druid/guice/StorageNodeModule.java index 2a1cee02b065..1bff1220ab37 100644 --- a/server/src/main/java/io/druid/guice/StorageNodeModule.java +++ b/server/src/main/java/io/druid/guice/StorageNodeModule.java @@ -25,6 +25,7 @@ import com.google.inject.ProvisionException; import com.google.inject.util.Providers; import io.druid.client.DruidServerConfig; +import io.druid.discovery.DataNodeService; import io.druid.guice.annotations.Self; import io.druid.query.DruidProcessingConfig; import io.druid.segment.column.ColumnConfig; @@ -66,4 +67,23 @@ public DruidServerMetadata getMetadata(@Self DruidNode node, @Nullable NodeTypeC config.getPriority() ); } + + @Provides + @LazySingleton + public DataNodeService getDataNodeService( + @Nullable NodeTypeConfig nodeType, + DruidServerConfig config + ) + { + if (nodeType == null) { + throw new ProvisionException("Must override the binding for NodeTypeConfig if you want a DruidServerMetadata."); + } + + return new DataNodeService( + config.getTier(), + config.getMaxSize(), + nodeType.getNodeType(), + config.getPriority() + ); + } } diff --git a/server/src/main/java/io/druid/initialization/Initialization.java b/server/src/main/java/io/druid/initialization/Initialization.java index 53b7e2110af8..cfeea5023de6 100644 --- a/server/src/main/java/io/druid/initialization/Initialization.java +++ b/server/src/main/java/io/druid/initialization/Initialization.java @@ -32,6 +32,7 @@ import com.google.inject.util.Modules; import io.druid.curator.CuratorModule; import io.druid.curator.discovery.DiscoveryModule; +import io.druid.discovery.DruidDiscoveryModule; import io.druid.guice.AWSModule; import io.druid.guice.AnnouncerModule; import io.druid.guice.CoordinatorDiscoveryModule; @@ -358,6 +359,7 @@ public static Injector makeInjectorWithModules(final Injector baseInjector, Iter new JettyServerModule(), new ExpressionModule(), new DiscoveryModule(), + new DruidDiscoveryModule(), new ServerViewModule(), new MetadataConfigModule(), new DerbyMetadataStorageDruidModule(), diff --git a/server/src/main/java/io/druid/query/lookup/LookupModule.java b/server/src/main/java/io/druid/query/lookup/LookupModule.java index 51b7ba9b36ae..151c65fbbedc 100644 --- a/server/src/main/java/io/druid/query/lookup/LookupModule.java +++ b/server/src/main/java/io/druid/query/lookup/LookupModule.java @@ -32,11 +32,14 @@ import com.google.common.collect.ImmutableMap; import com.google.inject.Binder; import com.google.inject.Inject; +import com.google.inject.Provides; import io.druid.common.utils.ServletResourceUtils; import io.druid.curator.announcement.Announcer; import io.druid.guice.ExpressionModule; +import io.druid.discovery.LookupNodeService; import io.druid.guice.Jerseys; import io.druid.guice.JsonConfigProvider; +import io.druid.guice.LazySingleton; import io.druid.guice.LifecycleModule; import io.druid.guice.ManageLifecycle; import io.druid.guice.annotations.Json; @@ -101,6 +104,13 @@ public void configure(Binder binder) 2 // 1 for "normal" operation and 1 for "emergency" or other ); } + + @Provides + @LazySingleton + public LookupNodeService getLookupNodeService(LookupListeningAnnouncerConfig lookupListeningAnnouncerConfig) + { + return new LookupNodeService(lookupListeningAnnouncerConfig.getLookupTier()); + } } @Path(ListenerResource.BASE_PATH + "/" + LookupCoordinatorManager.LOOKUP_LISTEN_ANNOUNCE_KEY) diff --git a/server/src/main/java/io/druid/server/http/ClusterResource.java b/server/src/main/java/io/druid/server/http/ClusterResource.java new file mode 100644 index 000000000000..074c04d3e135 --- /dev/null +++ b/server/src/main/java/io/druid/server/http/ClusterResource.java @@ -0,0 +1,89 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets 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 io.druid.server.http; + +import com.google.common.collect.ImmutableMap; +import com.google.inject.Inject; +import com.sun.jersey.spi.container.ResourceFilters; +import io.druid.discovery.DiscoveryDruidNode; +import io.druid.discovery.DruidNodeDiscoveryProvider; +import io.druid.guice.LazySingleton; +import io.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.Set; + +/** + */ +@Path("/druid/coordinator/v1/cluster") +@LazySingleton +@ResourceFilters(StateResourceFilter.class) +public class ClusterResource +{ + private final DruidNodeDiscoveryProvider druidNodeDiscoveryProvider; + + @Inject + public ClusterResource(DruidNodeDiscoveryProvider discoveryProvider) + { + this.druidNodeDiscoveryProvider = discoveryProvider; + } + + @GET + @Produces(MediaType.APPLICATION_JSON) + public Response getClusterServers() + { + ImmutableMap.Builder entityBuilder = new ImmutableMap.Builder<>(); + + entityBuilder.put(DruidNodeDiscoveryProvider.NODE_TYPE_COORDINATOR, + druidNodeDiscoveryProvider.getForNodeType(DruidNodeDiscoveryProvider.NODE_TYPE_COORDINATOR) + .getAllNodes() + ); + entityBuilder.put(DruidNodeDiscoveryProvider.NODE_TYPE_OVERLORD, + druidNodeDiscoveryProvider.getForNodeType(DruidNodeDiscoveryProvider.NODE_TYPE_OVERLORD) + .getAllNodes() + ); + entityBuilder.put(DruidNodeDiscoveryProvider.NODE_TYPE_BROKER, + druidNodeDiscoveryProvider.getForNodeType(DruidNodeDiscoveryProvider.NODE_TYPE_BROKER) + .getAllNodes() + ); + entityBuilder.put(DruidNodeDiscoveryProvider.NODE_TYPE_HISTORICAL, + druidNodeDiscoveryProvider.getForNodeType(DruidNodeDiscoveryProvider.NODE_TYPE_HISTORICAL) + .getAllNodes() + ); + + Set mmNodes = druidNodeDiscoveryProvider.getForNodeType(DruidNodeDiscoveryProvider.NODE_TYPE_MM) + .getAllNodes(); + if (!mmNodes.isEmpty()) { + entityBuilder.put(DruidNodeDiscoveryProvider.NODE_TYPE_MM, mmNodes); + } + + Set routerNodes = druidNodeDiscoveryProvider.getForNodeType(DruidNodeDiscoveryProvider.NODE_TYPE_ROUTER) + .getAllNodes(); + if (!routerNodes.isEmpty()) { + entityBuilder.put(DruidNodeDiscoveryProvider.NODE_TYPE_ROUTER, routerNodes); + } + + return Response.status(Response.Status.OK).entity(entityBuilder.build()).build(); + } +} diff --git a/server/src/main/java/io/druid/server/initialization/CuratorDiscoveryConfig.java b/server/src/main/java/io/druid/server/initialization/CuratorDiscoveryConfig.java index b05c46dfe2da..1de970c6fa67 100644 --- a/server/src/main/java/io/druid/server/initialization/CuratorDiscoveryConfig.java +++ b/server/src/main/java/io/druid/server/initialization/CuratorDiscoveryConfig.java @@ -28,6 +28,14 @@ public class CuratorDiscoveryConfig @JsonProperty private String path = "/druid/discovery"; + @JsonProperty + private String internalDiscoveryPath = "/druid/internal-discovery"; + + public String getInternalDiscoveryPath() + { + return internalDiscoveryPath; + } + public String getPath() { return path; diff --git a/services/src/main/java/io/druid/cli/CliBroker.java b/services/src/main/java/io/druid/cli/CliBroker.java index 95586e7af29f..b1e142e1a3aa 100644 --- a/services/src/main/java/io/druid/cli/CliBroker.java +++ b/services/src/main/java/io/druid/cli/CliBroker.java @@ -20,8 +20,11 @@ package io.druid.cli; import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; import com.google.inject.Binder; +import com.google.inject.Inject; import com.google.inject.Module; +import com.google.inject.Provider; import com.google.inject.name.Names; import io.airlift.airline.Command; import io.druid.client.BrokerSegmentWatcherConfig; @@ -33,6 +36,10 @@ import io.druid.client.selector.CustomTierSelectorStrategyConfig; import io.druid.client.selector.ServerSelectorStrategy; import io.druid.client.selector.TierSelectorStrategy; +import io.druid.discovery.DiscoveryDruidNode; +import io.druid.discovery.DruidNodeAnnouncer; +import io.druid.discovery.DruidNodeDiscoveryProvider; +import io.druid.discovery.LookupNodeService; import io.druid.guice.CacheModule; import io.druid.guice.DruidProcessingModule; import io.druid.guice.Jerseys; @@ -41,6 +48,8 @@ import io.druid.guice.LifecycleModule; import io.druid.guice.QueryRunnerFactoryModule; import io.druid.guice.QueryableModule; +import io.druid.guice.annotations.Self; +import io.druid.java.util.common.lifecycle.Lifecycle; import io.druid.java.util.common.logger.Logger; import io.druid.query.QuerySegmentWalker; import io.druid.query.RetryQueryRunnerConfig; @@ -48,6 +57,7 @@ import io.druid.server.BrokerQueryResource; import io.druid.server.ClientInfoResource; import io.druid.server.ClientQuerySegmentWalker; +import io.druid.server.DruidNode; import io.druid.server.coordination.broker.DruidBroker; import io.druid.server.http.BrokerResource; import io.druid.server.initialization.jetty.JettyServerInitializer; @@ -120,10 +130,52 @@ public void configure(Binder binder) MetricsModule.register(binder, CacheMonitor.class); LifecycleModule.register(binder, Server.class); + + binder.bind(ForSideEffectsOnlyProvider.Child.class).toProvider(ForSideEffectsOnlyProvider.class).asEagerSingleton(); } }, new LookupModule(), new SqlModule() ); } + + private static class ForSideEffectsOnlyProvider implements Provider + { + final static class Child {}; + + @Inject + public ForSideEffectsOnlyProvider(DruidNodeAnnouncer announcer, @Self DruidNode druidNode, + LookupNodeService lookupNodeService, Lifecycle lifecycle) + { + DiscoveryDruidNode discoveryDruidNode = new DiscoveryDruidNode(druidNode, + DruidNodeDiscoveryProvider.NODE_TYPE_BROKER, + ImmutableMap.of( + lookupNodeService.getName(), lookupNodeService + )); + + lifecycle.addHandler( + new Lifecycle.Handler() + { + @Override + public void start() throws Exception + { + announcer.announce(discoveryDruidNode); + } + + @Override + public void stop() + { + announcer.unannounce(discoveryDruidNode); + } + }, + Lifecycle.Stage.LAST + ); + } + + @Override + public ForSideEffectsOnlyProvider.Child get() + { + return new Child(); + } + } } diff --git a/services/src/main/java/io/druid/cli/CliCoordinator.java b/services/src/main/java/io/druid/cli/CliCoordinator.java index 87fba0a3d145..5c203f0e4879 100644 --- a/services/src/main/java/io/druid/cli/CliCoordinator.java +++ b/services/src/main/java/io/druid/cli/CliCoordinator.java @@ -21,16 +21,20 @@ import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.base.Predicates; +import com.google.common.collect.ImmutableMap; import com.google.inject.Binder; import com.google.inject.Inject; import com.google.inject.Module; +import com.google.inject.Provider; import com.google.inject.Provides; import com.google.inject.name.Names; - import io.airlift.airline.Command; import io.druid.audit.AuditManager; import io.druid.client.CoordinatorServerView; import io.druid.client.indexing.IndexingServiceClient; +import io.druid.discovery.DiscoveryDruidNode; +import io.druid.discovery.DruidNodeAnnouncer; +import io.druid.discovery.DruidNodeDiscoveryProvider; import io.druid.guice.ConditionalMultibind; import io.druid.guice.ConfigProvider; import io.druid.guice.Jerseys; @@ -39,7 +43,9 @@ import io.druid.guice.LifecycleModule; import io.druid.guice.ManageLifecycle; import io.druid.guice.annotations.CoordinatorIndexingServiceHelper; +import io.druid.guice.annotations.Self; import io.druid.java.util.common.concurrent.ScheduledExecutorFactory; +import io.druid.java.util.common.lifecycle.Lifecycle; import io.druid.java.util.common.logger.Logger; import io.druid.metadata.MetadataRuleManager; import io.druid.metadata.MetadataRuleManagerConfig; @@ -49,6 +55,7 @@ import io.druid.metadata.MetadataSegmentManagerProvider; import io.druid.metadata.MetadataStorage; import io.druid.metadata.MetadataStorageProvider; +import io.druid.server.DruidNode; import io.druid.server.audit.AuditManagerProvider; import io.druid.server.coordinator.BalancerStrategyFactory; import io.druid.server.coordinator.DruidCoordinator; @@ -58,6 +65,7 @@ import io.druid.server.coordinator.helper.DruidCoordinatorSegmentKiller; import io.druid.server.coordinator.helper.DruidCoordinatorSegmentMerger; import io.druid.server.coordinator.helper.DruidCoordinatorVersionConverter; +import io.druid.server.http.ClusterResource; import io.druid.server.http.CoordinatorDynamicConfigsResource; import io.druid.server.http.CoordinatorRedirectInfo; import io.druid.server.http.CoordinatorResource; @@ -182,6 +190,7 @@ public void configure(Binder binder) Jerseys.addResource(binder, MetadataResource.class); Jerseys.addResource(binder, IntervalsResource.class); Jerseys.addResource(binder, LookupCoordinatorResource.class); + Jerseys.addResource(binder, ClusterResource.class); LifecycleModule.register(binder, Server.class); LifecycleModule.register(binder, DatasourcesResource.class); @@ -204,6 +213,8 @@ public void configure(Binder binder) Predicates.equalTo("true"), DruidCoordinatorSegmentKiller.class ); + + binder.bind(ForSideEffectsOnlyProvider.Child.class).toProvider(ForSideEffectsOnlyProvider.class).asEagerSingleton(); } @Provides @@ -234,4 +245,42 @@ public static boolean isOverlord(Properties properties) { return Boolean.valueOf(properties.getProperty("druid.coordinator.asOverlord.enabled")).booleanValue(); } + + private static class ForSideEffectsOnlyProvider implements Provider + { + final static class Child {}; + + @Inject + public ForSideEffectsOnlyProvider(DruidNodeAnnouncer announcer, @Self DruidNode druidNode, Lifecycle lifecycle) + { + DiscoveryDruidNode discoveryDruidNode = new DiscoveryDruidNode(druidNode, + DruidNodeDiscoveryProvider.NODE_TYPE_COORDINATOR, + ImmutableMap.of() + ); + + lifecycle.addHandler( + new Lifecycle.Handler() + { + @Override + public void start() throws Exception + { + announcer.announce(discoveryDruidNode); + } + + @Override + public void stop() + { + announcer.unannounce(discoveryDruidNode); + } + }, + Lifecycle.Stage.LAST + ); + } + + @Override + public ForSideEffectsOnlyProvider.Child get() + { + return new Child(); + } + } } diff --git a/services/src/main/java/io/druid/cli/CliHistorical.java b/services/src/main/java/io/druid/cli/CliHistorical.java index d23364b81b92..70550843ff35 100644 --- a/services/src/main/java/io/druid/cli/CliHistorical.java +++ b/services/src/main/java/io/druid/cli/CliHistorical.java @@ -20,12 +20,20 @@ package io.druid.cli; import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; import com.google.inject.Binder; +import com.google.inject.Inject; import com.google.inject.Module; +import com.google.inject.Provider; import com.google.inject.name.Names; import io.airlift.airline.Command; import io.druid.client.cache.CacheConfig; import io.druid.client.cache.CacheMonitor; +import io.druid.discovery.DataNodeService; +import io.druid.discovery.DiscoveryDruidNode; +import io.druid.discovery.DruidNodeAnnouncer; +import io.druid.discovery.DruidNodeDiscoveryProvider; +import io.druid.discovery.LookupNodeService; import io.druid.guice.CacheModule; import io.druid.guice.DruidProcessingModule; import io.druid.guice.Jerseys; @@ -36,9 +44,12 @@ import io.druid.guice.NodeTypeConfig; import io.druid.guice.QueryRunnerFactoryModule; import io.druid.guice.QueryableModule; +import io.druid.guice.annotations.Self; +import io.druid.java.util.common.lifecycle.Lifecycle; import io.druid.java.util.common.logger.Logger; import io.druid.query.QuerySegmentWalker; import io.druid.query.lookup.LookupModule; +import io.druid.server.DruidNode; import io.druid.server.QueryResource; import io.druid.server.SegmentManager; import io.druid.server.coordination.ServerManager; @@ -103,9 +114,52 @@ public void configure(Binder binder) JsonConfigProvider.bind(binder, "druid.historical.cache", CacheConfig.class); binder.install(new CacheModule()); MetricsModule.register(binder, CacheMonitor.class); + + binder.bind(ForSideEffectsOnlyProvider.Child.class).toProvider(ForSideEffectsOnlyProvider.class).asEagerSingleton(); } }, new LookupModule() ); } + + private static class ForSideEffectsOnlyProvider implements Provider + { + final static class Child {}; + + @Inject + public ForSideEffectsOnlyProvider(DruidNodeAnnouncer announcer, @Self DruidNode druidNode, DataNodeService dataNodeService, + LookupNodeService lookupNodeService, Lifecycle lifecycle) + { + DiscoveryDruidNode discoveryDruidNode = new DiscoveryDruidNode(druidNode, + DruidNodeDiscoveryProvider.NODE_TYPE_HISTORICAL, + ImmutableMap.of( + dataNodeService.getName(), dataNodeService, + lookupNodeService.getName(), lookupNodeService + )); + + lifecycle.addHandler( + new Lifecycle.Handler() + { + @Override + public void start() throws Exception + { + announcer.announce(discoveryDruidNode); + } + + @Override + public void stop() + { + announcer.unannounce(discoveryDruidNode); + } + }, + Lifecycle.Stage.LAST + ); + } + + @Override + public ForSideEffectsOnlyProvider.Child get() + { + return new Child(); + } + } } diff --git a/services/src/main/java/io/druid/cli/CliMiddleManager.java b/services/src/main/java/io/druid/cli/CliMiddleManager.java index 764c7cb9e025..6f2799892c1a 100644 --- a/services/src/main/java/io/druid/cli/CliMiddleManager.java +++ b/services/src/main/java/io/druid/cli/CliMiddleManager.java @@ -20,13 +20,19 @@ package io.druid.cli; import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; import com.google.inject.Binder; +import com.google.inject.Inject; import com.google.inject.Module; +import com.google.inject.Provider; import com.google.inject.Provides; import com.google.inject.name.Names; import com.google.inject.util.Providers; - import io.airlift.airline.Command; +import io.druid.discovery.DiscoveryDruidNode; +import io.druid.discovery.DruidNodeAnnouncer; +import io.druid.discovery.DruidNodeDiscoveryProvider; +import io.druid.discovery.WorkerNodeService; import io.druid.guice.IndexingServiceFirehoseModule; import io.druid.guice.IndexingServiceModuleHelper; import io.druid.guice.IndexingServiceTaskLogsModule; @@ -44,6 +50,7 @@ import io.druid.indexing.worker.WorkerTaskMonitor; import io.druid.indexing.worker.config.WorkerConfig; import io.druid.indexing.worker.http.WorkerResource; +import io.druid.java.util.common.lifecycle.Lifecycle; import io.druid.java.util.common.logger.Logger; import io.druid.segment.realtime.firehose.ChatHandlerProvider; import io.druid.server.DruidNode; @@ -98,6 +105,7 @@ public void configure(Binder binder) Jerseys.addResource(binder, WorkerResource.class); LifecycleModule.register(binder, Server.class); + binder.bind(ForSideEffectsOnlyProvider.Child.class).toProvider(ForSideEffectsOnlyProvider.class).asEagerSingleton(); } @Provides @@ -117,4 +125,53 @@ public Worker getWorker(@Self DruidNode node, WorkerConfig config) new IndexingServiceTaskLogsModule() ); } + + private static class ForSideEffectsOnlyProvider implements Provider + { + final static class Child {}; + + @Inject + public ForSideEffectsOnlyProvider( + DruidNodeAnnouncer announcer, + @Self DruidNode druidNode, + WorkerConfig workerConfig, + Lifecycle lifecycle + ) + { + WorkerNodeService workerNodeService = new WorkerNodeService( + workerConfig.getIp(), + workerConfig.getCapacity(), + workerConfig.getVersion() + ); + DiscoveryDruidNode discoveryDruidNode = new DiscoveryDruidNode(druidNode, + DruidNodeDiscoveryProvider.NODE_TYPE_MM, + ImmutableMap.of( + workerNodeService.getName(), workerNodeService + )); + + lifecycle.addHandler( + new Lifecycle.Handler() + { + @Override + public void start() throws Exception + { + announcer.announce(discoveryDruidNode); + } + + @Override + public void stop() + { + announcer.unannounce(discoveryDruidNode); + } + }, + Lifecycle.Stage.LAST + ); + } + + @Override + public ForSideEffectsOnlyProvider.Child get() + { + return new Child(); + } + } } diff --git a/services/src/main/java/io/druid/cli/CliOverlord.java b/services/src/main/java/io/druid/cli/CliOverlord.java index 9ae4b5233309..ff00d0f86ebc 100644 --- a/services/src/main/java/io/druid/cli/CliOverlord.java +++ b/services/src/main/java/io/druid/cli/CliOverlord.java @@ -20,10 +20,13 @@ package io.druid.cli; import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; import com.google.inject.Binder; +import com.google.inject.Inject; import com.google.inject.Injector; import com.google.inject.Key; import com.google.inject.Module; +import com.google.inject.Provider; import com.google.inject.TypeLiteral; import com.google.inject.multibindings.MapBinder; import com.google.inject.multibindings.Multibinder; @@ -33,6 +36,9 @@ import io.airlift.airline.Command; import io.druid.audit.AuditManager; import io.druid.client.indexing.IndexingServiceSelectorConfig; +import io.druid.discovery.DiscoveryDruidNode; +import io.druid.discovery.DruidNodeAnnouncer; +import io.druid.discovery.DruidNodeDiscoveryProvider; import io.druid.guice.IndexingServiceFirehoseModule; import io.druid.guice.IndexingServiceModuleHelper; import io.druid.guice.IndexingServiceTaskLogsModule; @@ -44,6 +50,7 @@ import io.druid.guice.ListProvider; import io.druid.guice.ManageLifecycle; import io.druid.guice.PolyBind; +import io.druid.guice.annotations.Self; import io.druid.indexing.common.actions.LocalTaskActionClientFactory; import io.druid.indexing.common.actions.TaskActionClientFactory; import io.druid.indexing.common.actions.TaskActionToolbox; @@ -76,8 +83,10 @@ import io.druid.indexing.overlord.supervisor.SupervisorManager; import io.druid.indexing.overlord.supervisor.SupervisorResource; import io.druid.indexing.worker.config.WorkerConfig; +import io.druid.java.util.common.lifecycle.Lifecycle; import io.druid.java.util.common.logger.Logger; import io.druid.segment.realtime.firehose.ChatHandlerProvider; +import io.druid.server.DruidNode; import io.druid.server.audit.AuditManagerProvider; import io.druid.server.coordinator.CoordinatorOverlordServiceConfig; import io.druid.server.http.RedirectFilter; @@ -182,6 +191,8 @@ public void configure(Binder binder) if (standalone) { LifecycleModule.register(binder, Server.class); } + + binder.bind(ForSideEffectsOnlyProvider.Child.class).toProvider(ForSideEffectsOnlyProvider.class).asEagerSingleton(); } private void configureTaskStorage(Binder binder) @@ -266,6 +277,44 @@ private void configureOverlordHelpers(Binder binder) ); } + private static class ForSideEffectsOnlyProvider implements Provider + { + final static class Child {}; + + @Inject + public ForSideEffectsOnlyProvider(DruidNodeAnnouncer announcer, @Self DruidNode druidNode, Lifecycle lifecycle) + { + DiscoveryDruidNode discoveryDruidNode = new DiscoveryDruidNode(druidNode, + DruidNodeDiscoveryProvider.NODE_TYPE_OVERLORD, + ImmutableMap.of() + ); + + lifecycle.addHandler( + new Lifecycle.Handler() + { + @Override + public void start() throws Exception + { + announcer.announce(discoveryDruidNode); + } + + @Override + public void stop() + { + announcer.unannounce(discoveryDruidNode); + } + }, + Lifecycle.Stage.LAST + ); + } + + @Override + public ForSideEffectsOnlyProvider.Child get() + { + return new Child(); + } + } + /** */ private static class OverlordJettyServerInitializer implements JettyServerInitializer diff --git a/services/src/main/java/io/druid/cli/CliRouter.java b/services/src/main/java/io/druid/cli/CliRouter.java index a84ac258b66b..c31ce80f945f 100644 --- a/services/src/main/java/io/druid/cli/CliRouter.java +++ b/services/src/main/java/io/druid/cli/CliRouter.java @@ -20,8 +20,11 @@ package io.druid.cli; import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; import com.google.inject.Binder; +import com.google.inject.Inject; import com.google.inject.Module; +import com.google.inject.Provider; import com.google.inject.Provides; import com.google.inject.TypeLiteral; import com.google.inject.name.Names; @@ -29,6 +32,9 @@ import io.druid.curator.discovery.DiscoveryModule; import io.druid.curator.discovery.ServerDiscoveryFactory; import io.druid.curator.discovery.ServerDiscoverySelector; +import io.druid.discovery.DiscoveryDruidNode; +import io.druid.discovery.DruidNodeAnnouncer; +import io.druid.discovery.DruidNodeDiscoveryProvider; import io.druid.guice.Jerseys; import io.druid.guice.JsonConfigProvider; import io.druid.guice.LazySingleton; @@ -39,9 +45,11 @@ import io.druid.guice.RouterProcessingModule; import io.druid.guice.annotations.Self; import io.druid.guice.http.JettyHttpClientModule; +import io.druid.java.util.common.lifecycle.Lifecycle; import io.druid.java.util.common.logger.Logger; import io.druid.query.lookup.LookupModule; import io.druid.server.AsyncQueryForwardingServlet; +import io.druid.server.DruidNode; import io.druid.server.http.RouterResource; import io.druid.server.initialization.jetty.JettyServerInitializer; import io.druid.server.metrics.QueryCountStatsProvider; @@ -109,6 +117,8 @@ public void configure(Binder binder) LifecycleModule.register(binder, RouterResource.class); LifecycleModule.register(binder, Server.class); DiscoveryModule.register(binder, Self.class); + + binder.bind(ForSideEffectsOnlyProvider.Child.class).toProvider(ForSideEffectsOnlyProvider.class).asEagerSingleton(); } @Provides @@ -125,4 +135,42 @@ public ServerDiscoverySelector getCoordinatorServerDiscoverySelector( new LookupModule() ); } + + private static class ForSideEffectsOnlyProvider implements Provider + { + final static class Child {}; + + @Inject + public ForSideEffectsOnlyProvider(DruidNodeAnnouncer announcer, @Self DruidNode druidNode, Lifecycle lifecycle) + { + DiscoveryDruidNode discoveryDruidNode = new DiscoveryDruidNode(druidNode, + DruidNodeDiscoveryProvider.NODE_TYPE_ROUTER, + ImmutableMap.of() + ); + + lifecycle.addHandler( + new Lifecycle.Handler() + { + @Override + public void start() throws Exception + { + announcer.announce(discoveryDruidNode); + } + + @Override + public void stop() + { + announcer.unannounce(discoveryDruidNode); + } + }, + Lifecycle.Stage.LAST + ); + } + + @Override + public ForSideEffectsOnlyProvider.Child get() + { + return new Child(); + } + } } From 5d8b038fc71e6c60f247a4731f04d633d19da9d5 Mon Sep 17 00:00:00 2001 From: Himanshu Gupta Date: Mon, 7 Aug 2017 14:01:49 -0500 Subject: [PATCH 2/9] more tests --- .../server/initialization/ServerConfig.java | 25 ++ ...torDruidNodeAnnouncerAndDiscoveryTest.java | 231 ++++++++++++++++++ .../druid/discovery/DataNodeServiceTest.java | 50 ++++ .../DruidNodeDiscoveryProviderTest.java | 217 ++++++++++++++++ .../discovery/LookupNodeServiceTest.java | 46 ++++ .../discovery/WorkerNodeServiceTest.java | 48 ++++ 6 files changed, 617 insertions(+) create mode 100644 server/src/test/java/io/druid/curator/discovery/CuratorDruidNodeAnnouncerAndDiscoveryTest.java create mode 100644 server/src/test/java/io/druid/discovery/DataNodeServiceTest.java create mode 100644 server/src/test/java/io/druid/discovery/DruidNodeDiscoveryProviderTest.java create mode 100644 server/src/test/java/io/druid/discovery/LookupNodeServiceTest.java create mode 100644 server/src/test/java/io/druid/discovery/WorkerNodeServiceTest.java diff --git a/server/src/main/java/io/druid/server/initialization/ServerConfig.java b/server/src/main/java/io/druid/server/initialization/ServerConfig.java index b299908bca4d..82e85318ee58 100644 --- a/server/src/main/java/io/druid/server/initialization/ServerConfig.java +++ b/server/src/main/java/io/druid/server/initialization/ServerConfig.java @@ -24,6 +24,7 @@ import javax.validation.constraints.Min; import javax.validation.constraints.NotNull; +import java.util.Objects; /** */ @@ -81,6 +82,30 @@ public boolean isTls() return tls; } + @Override + public boolean equals(Object o) + { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + ServerConfig that = (ServerConfig) o; + return numThreads == that.numThreads && + defaultQueryTimeout == that.defaultQueryTimeout && + maxScatterGatherBytes == that.maxScatterGatherBytes && + plaintext == that.plaintext && + tls == that.tls && + Objects.equals(maxIdleTime, that.maxIdleTime); + } + + @Override + public int hashCode() + { + return Objects.hash(numThreads, maxIdleTime, defaultQueryTimeout, maxScatterGatherBytes, plaintext, tls); + } + @Override public String toString() { diff --git a/server/src/test/java/io/druid/curator/discovery/CuratorDruidNodeAnnouncerAndDiscoveryTest.java b/server/src/test/java/io/druid/curator/discovery/CuratorDruidNodeAnnouncerAndDiscoveryTest.java new file mode 100644 index 000000000000..6ed4440e57c6 --- /dev/null +++ b/server/src/test/java/io/druid/curator/discovery/CuratorDruidNodeAnnouncerAndDiscoveryTest.java @@ -0,0 +1,231 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets 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 io.druid.curator.discovery; + +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; +import com.google.common.collect.ImmutableSet; +import com.google.common.util.concurrent.MoreExecutors; +import com.google.inject.Binder; +import com.google.inject.Injector; +import com.google.inject.Key; +import com.google.inject.Module; +import com.google.inject.name.Names; +import io.druid.curator.CuratorTestBase; +import io.druid.curator.announcement.Announcer; +import io.druid.discovery.DiscoveryDruidNode; +import io.druid.discovery.DruidNodeDiscovery; +import io.druid.discovery.DruidNodeDiscoveryProvider; +import io.druid.guice.GuiceInjectors; +import io.druid.initialization.Initialization; +import io.druid.server.DruidNode; +import io.druid.server.initialization.CuratorDiscoveryConfig; +import io.druid.server.initialization.ServerConfig; +import org.junit.After; +import org.junit.Before; +import org.junit.Test; + +import java.util.HashSet; + +/** + */ +public class CuratorDruidNodeAnnouncerAndDiscoveryTest extends CuratorTestBase +{ + @Before + public void setUp() throws Exception + { + setupServerAndCurator(); + } + + @Test(timeout = 10000) + public void testAnnouncementAndDiscovery() throws Exception + { + Injector injector = Initialization.makeInjectorWithModules( + GuiceInjectors.makeStartupInjector(), + ImmutableList.of( + new Module() + { + @Override + public void configure(Binder binder) + { + binder.bind(Key.get(String.class, Names.named("serviceName"))).toInstance("some service"); + binder.bind(Key.get(Integer.class, Names.named("servicePort"))).toInstance(0); + binder.bind(Key.get(Integer.class, Names.named("tlsServicePort"))).toInstance(-1); + } + } + ) + ); + ObjectMapper objectMapper = injector.getInstance(ObjectMapper.class); + + curator.start(); + curator.blockUntilConnected(); + + Announcer announcer = new Announcer( + curator, + MoreExecutors.sameThreadExecutor() + ); + announcer.start(); + + CuratorDruidNodeAnnouncer druidNodeAnnouncer = new CuratorDruidNodeAnnouncer( + announcer, + new CuratorDiscoveryConfig(), + objectMapper + ); + + DiscoveryDruidNode node1 = new DiscoveryDruidNode( + new DruidNode("s1", "h1", 8080, null, new ServerConfig()), + DruidNodeDiscoveryProvider.NODE_TYPE_COORDINATOR, + ImmutableMap.of() + ); + + DiscoveryDruidNode node2 = new DiscoveryDruidNode( + new DruidNode("s2", "h2", 8080, null, new ServerConfig()), + DruidNodeDiscoveryProvider.NODE_TYPE_COORDINATOR, + ImmutableMap.of() + ); + + DiscoveryDruidNode node3 = new DiscoveryDruidNode( + new DruidNode("s3", "h3", 8080, null, new ServerConfig()), + DruidNodeDiscoveryProvider.NODE_TYPE_OVERLORD, + ImmutableMap.of() + ); + + DiscoveryDruidNode node4 = new DiscoveryDruidNode( + new DruidNode("s4", "h4", 8080, null, new ServerConfig()), + DruidNodeDiscoveryProvider.NODE_TYPE_OVERLORD, + ImmutableMap.of() + ); + + druidNodeAnnouncer.announce(node1); + druidNodeAnnouncer.announce(node3); + + CuratorDruidNodeDiscoveryProvider druidNodeDiscoveryProvider = new CuratorDruidNodeDiscoveryProvider( + curator, + new CuratorDiscoveryConfig(), + objectMapper + ); + druidNodeDiscoveryProvider.start(); + + DruidNodeDiscovery coordDiscovery = druidNodeDiscoveryProvider.getForNodeType(DruidNodeDiscoveryProvider.NODE_TYPE_COORDINATOR); + DruidNodeDiscovery overlordDiscovery = druidNodeDiscoveryProvider.getForNodeType(DruidNodeDiscoveryProvider.NODE_TYPE_OVERLORD); + + while (!ImmutableSet.of(node1).equals(coordDiscovery.getAllNodes())) { + Thread.sleep(100); + } + + while (!ImmutableSet.of(node3).equals(overlordDiscovery.getAllNodes())) { + Thread.sleep(100); + } + + HashSet coordNodes = new HashSet<>(); + coordDiscovery.registerListener( + new DruidNodeDiscovery.Listener() + { + @Override + public void nodeAdded(DiscoveryDruidNode node) + { + coordNodes.add(node); + } + + @Override + public void nodeRemoved(DiscoveryDruidNode node) + { + coordNodes.remove(node); + } + } + ); + + HashSet overlordNodes = new HashSet<>(); + overlordDiscovery.registerListener( + new DruidNodeDiscovery.Listener() + { + @Override + public void nodeAdded(DiscoveryDruidNode node) + { + overlordNodes.add(node); + } + + @Override + public void nodeRemoved(DiscoveryDruidNode node) + { + overlordNodes.remove(node); + } + } + ); + + while (!ImmutableSet.of(node1).equals(coordNodes)) { + Thread.sleep(100); + } + + while (!ImmutableSet.of(node3).equals(overlordNodes)) { + Thread.sleep(100); + } + + druidNodeAnnouncer.announce(node2); + druidNodeAnnouncer.announce(node4); + + while (!ImmutableSet.of(node1, node2).equals(coordDiscovery.getAllNodes())) { + Thread.sleep(100); + } + + while (!ImmutableSet.of(node3, node4).equals(overlordDiscovery.getAllNodes())) { + Thread.sleep(100); + } + + while (!ImmutableSet.of(node1, node2).equals(coordNodes)) { + Thread.sleep(100); + } + + while (!ImmutableSet.of(node3, node4).equals(overlordNodes)) { + Thread.sleep(100); + } + + druidNodeAnnouncer.unannounce(node1); + druidNodeAnnouncer.unannounce(node2); + druidNodeAnnouncer.unannounce(node3); + druidNodeAnnouncer.unannounce(node4); + + while (!ImmutableSet.of().equals(coordDiscovery.getAllNodes())) { + Thread.sleep(100); + } + + while (!ImmutableSet.of().equals(overlordDiscovery.getAllNodes())) { + Thread.sleep(100); + } + + while (!ImmutableSet.of().equals(coordNodes)) { + Thread.sleep(100); + } + + while (!ImmutableSet.of().equals(overlordNodes)) { + Thread.sleep(100); + } + + druidNodeDiscoveryProvider.stop(); + announcer.stop(); + } + + @After + public void tearDown() + { + tearDownServerAndCurator(); + } +} diff --git a/server/src/test/java/io/druid/discovery/DataNodeServiceTest.java b/server/src/test/java/io/druid/discovery/DataNodeServiceTest.java new file mode 100644 index 000000000000..f2f13c41adf0 --- /dev/null +++ b/server/src/test/java/io/druid/discovery/DataNodeServiceTest.java @@ -0,0 +1,50 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets 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 io.druid.discovery; + +import com.fasterxml.jackson.databind.ObjectMapper; +import io.druid.segment.TestHelper; +import io.druid.server.coordination.ServerType; +import org.junit.Assert; +import org.junit.Test; + +/** + */ +public class DataNodeServiceTest +{ + @Test + public void testSerde() throws Exception + { + DruidService expected = new DataNodeService( + "tier", + 100, + ServerType.HISTORICAL, + 1 + ); + + ObjectMapper mapper = TestHelper.getJsonMapper(); + DruidService actual = mapper.readValue( + mapper.writeValueAsString(expected), + DruidService.class + ); + + Assert.assertEquals(expected, actual); + } +} diff --git a/server/src/test/java/io/druid/discovery/DruidNodeDiscoveryProviderTest.java b/server/src/test/java/io/druid/discovery/DruidNodeDiscoveryProviderTest.java new file mode 100644 index 000000000000..d6f991443b02 --- /dev/null +++ b/server/src/test/java/io/druid/discovery/DruidNodeDiscoveryProviderTest.java @@ -0,0 +1,217 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets 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 io.druid.discovery; + +import com.google.common.collect.ImmutableMap; +import com.google.common.collect.ImmutableSet; +import io.druid.server.DruidNode; +import io.druid.server.coordination.ServerType; +import io.druid.server.initialization.ServerConfig; +import org.junit.Assert; +import org.junit.Test; + +import java.util.ArrayList; +import java.util.HashSet; +import java.util.List; +import java.util.Set; + +/** + */ +public class DruidNodeDiscoveryProviderTest +{ + @Test + public void testGetForService() + { + TestDruidNodeDiscoveryProvider provider = new TestDruidNodeDiscoveryProvider(); + + DruidNodeDiscovery dataNodeDiscovery = provider.getForService(DataNodeService.DISCOVERY_SERVICE_KEY); + Set dataNodes = new HashSet<>(); + dataNodeDiscovery.registerListener( + new DruidNodeDiscovery.Listener() + { + @Override + public void nodeAdded(DiscoveryDruidNode node) + { + dataNodes.add(node); + } + + @Override + public void nodeRemoved(DiscoveryDruidNode node) + { + dataNodes.remove(node); + } + } + ); + + DruidNodeDiscovery lookupNodeDiscovery = provider.getForService(LookupNodeService.DISCOVERY_SERVICE_KEY); + Set lookupNodes = new HashSet<>(); + lookupNodeDiscovery.registerListener( + new DruidNodeDiscovery.Listener() + { + @Override + public void nodeAdded(DiscoveryDruidNode node) + { + lookupNodes.add(node); + } + + @Override + public void nodeRemoved(DiscoveryDruidNode node) + { + lookupNodes.remove(node); + } + } + ); + + Assert.assertEquals(ImmutableSet.of(), dataNodes); + Assert.assertEquals(ImmutableSet.of(), dataNodeDiscovery.getAllNodes()); + Assert.assertEquals(ImmutableSet.of(), lookupNodes); + Assert.assertEquals(ImmutableSet.of(), lookupNodeDiscovery.getAllNodes()); + + DiscoveryDruidNode node1 = new DiscoveryDruidNode( + new DruidNode("s1", "h1", 8080, null, new ServerConfig()), + DruidNodeDiscoveryProvider.NODE_TYPE_HISTORICAL, + ImmutableMap.of( + DataNodeService.DISCOVERY_SERVICE_KEY, new DataNodeService("tier", 1000, ServerType.HISTORICAL, 0), + LookupNodeService.DISCOVERY_SERVICE_KEY, new LookupNodeService("tier")) + ); + + DiscoveryDruidNode node2 = new DiscoveryDruidNode( + new DruidNode("s2", "h2", 8080, null, new ServerConfig()), + DruidNodeDiscoveryProvider.NODE_TYPE_HISTORICAL, + ImmutableMap.of( + DataNodeService.DISCOVERY_SERVICE_KEY, new DataNodeService("tier", 1000, ServerType.HISTORICAL, 0)) + ); + + DiscoveryDruidNode node3 = new DiscoveryDruidNode( + new DruidNode("s3", "h3", 8080, null, new ServerConfig()), + DruidNodeDiscoveryProvider.NODE_TYPE_HISTORICAL, + ImmutableMap.of( + LookupNodeService.DISCOVERY_SERVICE_KEY, new LookupNodeService("tier")) + ); + + DiscoveryDruidNode node4 = new DiscoveryDruidNode( + new DruidNode("s4", "h4", 8080, null, new ServerConfig()), + DruidNodeDiscoveryProvider.NODE_TYPE_PEON, + ImmutableMap.of( + DataNodeService.DISCOVERY_SERVICE_KEY, new DataNodeService("tier", 1000, ServerType.HISTORICAL, 0), + LookupNodeService.DISCOVERY_SERVICE_KEY, new LookupNodeService("tier")) + ); + + DiscoveryDruidNode node5 = new DiscoveryDruidNode( + new DruidNode("s5", "h5", 8080, null, new ServerConfig()), + DruidNodeDiscoveryProvider.NODE_TYPE_PEON, + ImmutableMap.of( + DataNodeService.DISCOVERY_SERVICE_KEY, new DataNodeService("tier", 1000, ServerType.HISTORICAL, 0)) + ); + + DiscoveryDruidNode node6 = new DiscoveryDruidNode( + new DruidNode("s6", "h6", 8080, null, new ServerConfig()), + DruidNodeDiscoveryProvider.NODE_TYPE_PEON, + ImmutableMap.of( + LookupNodeService.DISCOVERY_SERVICE_KEY, new LookupNodeService("tier")) + ); + + DiscoveryDruidNode node7 = new DiscoveryDruidNode( + new DruidNode("s7", "h7", 8080, null, new ServerConfig()), + DruidNodeDiscoveryProvider.NODE_TYPE_BROKER, + ImmutableMap.of( + LookupNodeService.DISCOVERY_SERVICE_KEY, new LookupNodeService("tier")) + ); + + DiscoveryDruidNode node7Clone = new DiscoveryDruidNode( + new DruidNode("s7", "h7", 8080, null, new ServerConfig()), + DruidNodeDiscoveryProvider.NODE_TYPE_BROKER, + ImmutableMap.of( + LookupNodeService.DISCOVERY_SERVICE_KEY, new LookupNodeService("tier")) + ); + + DiscoveryDruidNode node8 = new DiscoveryDruidNode( + new DruidNode("s8", "h8", 8080, null, new ServerConfig()), + DruidNodeDiscoveryProvider.NODE_TYPE_COORDINATOR, + ImmutableMap.of() + ); + + provider.add(node1); + provider.add(node2); + provider.add(node3); + provider.add(node4); + provider.add(node5); + provider.add(node6); + provider.add(node7); + provider.add(node7Clone); + provider.add(node8); + + Assert.assertEquals(ImmutableSet.of(node1, node2, node4, node5), dataNodeDiscovery.getAllNodes()); + Assert.assertEquals(ImmutableSet.of(node1, node2, node4, node5), dataNodes); + + Assert.assertEquals(ImmutableSet.of(node1, node3, node4, node6, node7), lookupNodeDiscovery.getAllNodes()); + Assert.assertEquals(ImmutableSet.of(node1, node3, node4, node6, node7), lookupNodes); + + provider.remove(node8); + provider.remove(node7Clone); + provider.remove(node6); + provider.remove(node5); + provider.remove(node4); + + Assert.assertEquals(ImmutableSet.of(node1, node2), dataNodeDiscovery.getAllNodes()); + Assert.assertEquals(ImmutableSet.of(node1, node2), dataNodes); + + Assert.assertEquals(ImmutableSet.of(node1, node3), lookupNodeDiscovery.getAllNodes()); + Assert.assertEquals(ImmutableSet.of(node1, node3), lookupNodes); + } + + private static class TestDruidNodeDiscoveryProvider extends DruidNodeDiscoveryProvider + { + private List listeners = new ArrayList<>(); + + @Override + public DruidNodeDiscovery getForNodeType(String nodeType) + { + return new DruidNodeDiscovery() + { + @Override + public Set getAllNodes() + { + throw new UnsupportedOperationException(); + } + + @Override + public void registerListener(Listener listener) + { + TestDruidNodeDiscoveryProvider.this.listeners.add(listener); + } + }; + } + + void add(DiscoveryDruidNode node) + { + for (DruidNodeDiscovery.Listener listener : listeners) { + listener.nodeAdded(node); + } + } + + void remove(DiscoveryDruidNode node) + { + for (DruidNodeDiscovery.Listener listener : listeners) { + listener.nodeRemoved(node); + } + } + } +} diff --git a/server/src/test/java/io/druid/discovery/LookupNodeServiceTest.java b/server/src/test/java/io/druid/discovery/LookupNodeServiceTest.java new file mode 100644 index 000000000000..69e5ae6930e2 --- /dev/null +++ b/server/src/test/java/io/druid/discovery/LookupNodeServiceTest.java @@ -0,0 +1,46 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets 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 io.druid.discovery; + +import com.fasterxml.jackson.databind.ObjectMapper; +import io.druid.segment.TestHelper; +import org.junit.Assert; +import org.junit.Test; + +/** + */ +public class LookupNodeServiceTest +{ + @Test + public void testSerde() throws Exception + { + DruidService expected = new LookupNodeService( + "tier" + ); + + ObjectMapper mapper = TestHelper.getJsonMapper(); + DruidService actual = mapper.readValue( + mapper.writeValueAsString(expected), + DruidService.class + ); + + Assert.assertEquals(expected, actual); + } +} diff --git a/server/src/test/java/io/druid/discovery/WorkerNodeServiceTest.java b/server/src/test/java/io/druid/discovery/WorkerNodeServiceTest.java new file mode 100644 index 000000000000..fd860cb568a0 --- /dev/null +++ b/server/src/test/java/io/druid/discovery/WorkerNodeServiceTest.java @@ -0,0 +1,48 @@ +/* + * Licensed to Metamarkets Group Inc. (Metamarkets) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. Metamarkets 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 io.druid.discovery; + +import com.fasterxml.jackson.databind.ObjectMapper; +import io.druid.segment.TestHelper; +import org.junit.Assert; +import org.junit.Test; + +/** + */ +public class WorkerNodeServiceTest +{ + @Test + public void testSerde() throws Exception + { + DruidService expected = new WorkerNodeService( + "1.1.1.1", + 100, + "v1" + ); + + ObjectMapper mapper = TestHelper.getJsonMapper(); + DruidService actual = mapper.readValue( + mapper.writeValueAsString(expected), + DruidService.class + ); + + Assert.assertEquals(expected, actual); + } +} From e47b63ecd03b38d97361b383d751cc763f8f3416 Mon Sep 17 00:00:00 2001 From: Himanshu Gupta Date: Wed, 9 Aug 2017 16:38:09 -0500 Subject: [PATCH 3/9] address some review comments --- .../CuratorDruidNodeDiscoveryProvider.java | 95 +++++++++++-------- .../curator/discovery/DiscoveryModule.java | 13 ++- .../druid/discovery/DruidDiscoveryModule.java | 49 ---------- .../druid/discovery/DruidNodeDiscovery.java | 4 +- .../discovery/DruidNodeDiscoveryProvider.java | 18 +++- .../druid/initialization/Initialization.java | 2 - .../io/druid/server/http/ClusterResource.java | 32 ++++++- ...torDruidNodeAnnouncerAndDiscoveryTest.java | 19 ++-- .../DruidNodeDiscoveryProviderTest.java | 17 ++-- .../src/main/java/io/druid/cli/CliBroker.java | 57 ++--------- .../java/io/druid/cli/CliCoordinator.java | 56 ++--------- .../main/java/io/druid/cli/CliHistorical.java | 58 ++--------- .../java/io/druid/cli/CliMiddleManager.java | 76 ++++----------- .../main/java/io/druid/cli/CliOverlord.java | 55 ++--------- .../src/main/java/io/druid/cli/CliRouter.java | 54 ++--------- .../java/io/druid/cli/ServerRunnable.java | 76 ++++++++++++++- 16 files changed, 269 insertions(+), 412 deletions(-) delete mode 100644 server/src/main/java/io/druid/discovery/DruidDiscoveryModule.java diff --git a/server/src/main/java/io/druid/curator/discovery/CuratorDruidNodeDiscoveryProvider.java b/server/src/main/java/io/druid/curator/discovery/CuratorDruidNodeDiscoveryProvider.java index b7143a465cd2..28b33c1bad7d 100644 --- a/server/src/main/java/io/druid/curator/discovery/CuratorDruidNodeDiscoveryProvider.java +++ b/server/src/main/java/io/druid/curator/discovery/CuratorDruidNodeDiscoveryProvider.java @@ -20,10 +20,11 @@ package io.druid.curator.discovery; import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.base.Preconditions; import com.google.common.base.Throwables; -import com.google.common.collect.ImmutableSet; import com.google.inject.Inject; import io.druid.concurrent.Execs; +import io.druid.concurrent.LifecycleLock; import io.druid.discovery.DiscoveryDruidNode; import io.druid.discovery.DruidNodeDiscovery; import io.druid.discovery.DruidNodeDiscoveryProvider; @@ -41,11 +42,13 @@ import org.apache.curator.utils.ZKPaths; import java.util.ArrayList; +import java.util.Collection; +import java.util.Collections; import java.util.List; import java.util.Map; -import java.util.Set; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ExecutorService; +import java.util.concurrent.TimeUnit; /** */ @@ -58,11 +61,11 @@ public class CuratorDruidNodeDiscoveryProvider extends DruidNodeDiscoveryProvide private final CuratorDiscoveryConfig config; private final ObjectMapper jsonMapper; - private final ExecutorService listenerExecutor; + private ExecutorService listenerExecutor; private final Map nodeTypeWatchers = new ConcurrentHashMap<>(); - private boolean stopped = false; + private final LifecycleLock lifecycleLock = new LifecycleLock(); @Inject public CuratorDruidNodeDiscoveryProvider( @@ -71,58 +74,70 @@ public CuratorDruidNodeDiscoveryProvider( @Json ObjectMapper jsonMapper ) { - // This is single-threaded to ensure that all listener calls are executed precisesly in the oder of add/remove - // event occurences. - this.listenerExecutor = Execs.singleThreaded("CuratorDruidNodeDiscoveryProvider-ListenerExecutor"); this.curatorFramework = curatorFramework; this.config = config; this.jsonMapper = jsonMapper; } - - @Override - public synchronized DruidNodeDiscovery getForNodeType(String nodeType) + public DruidNodeDiscovery getForNodeType(String nodeType) { - if (stopped) { - throw new ISE("Provider has been stopped."); - } + Preconditions.checkState(lifecycleLock.awaitStarted(1, TimeUnit.MILLISECONDS)); - NodeTypeWatcher nodeTypeWatcher = nodeTypeWatchers.get(nodeType); - if (nodeTypeWatcher == null) { - log.info("Creating/Starting NodeTypeWatcher for nodeType [%s].", nodeType); - nodeTypeWatcher = new NodeTypeWatcher( - listenerExecutor, - curatorFramework, - config.getInternalDiscoveryPath(), - jsonMapper, - nodeType - ); - nodeTypeWatcher.start(); - nodeTypeWatchers.put(nodeType, nodeTypeWatcher); - log.info("Created/Started NodeTypeWatcher for nodeType [%s].", nodeType); - } + return nodeTypeWatchers.compute( + nodeType, + (k, v) -> { + if (v != null) { + return v; + } - return nodeTypeWatchers.get(nodeType); + log.info("Creating NodeTypeWatcher for nodeType [%s].", nodeType); + NodeTypeWatcher nodeTypeWatcher = new NodeTypeWatcher( + listenerExecutor, + curatorFramework, + config.getInternalDiscoveryPath(), + jsonMapper, + nodeType + ); + nodeTypeWatcher.start(); + log.info("Created NodeTypeWatcher for nodeType [%s].", nodeType); + return nodeTypeWatcher; + } + ); } @LifecycleStart - public synchronized void start() + public void start() { - log.info("started"); + if (!lifecycleLock.canStart()) { + throw new ISE("can't start."); + } + + try { + log.info("starting"); + + // This is single-threaded to ensure that all listener calls are executed precisely in the oder of add/remove + // event occurences. + listenerExecutor = Execs.singleThreaded("CuratorDruidNodeDiscoveryProvider-ListenerExecutor"); + + log.info("started"); + + lifecycleLock.started(); + } + finally { + lifecycleLock.exitStart(); + } } @LifecycleStop - public synchronized void stop() + public void stop() { - if (stopped) { - return; + if (!lifecycleLock.canStop()) { + throw new ISE("can't stop."); } log.info("stopping"); - stopped = true; - for (NodeTypeWatcher watcher : nodeTypeWatchers.values()) { watcher.stop(); } @@ -175,9 +190,9 @@ private static class NodeTypeWatcher implements DruidNodeDiscovery, PathChildren } @Override - public Set getAllNodes() + public Collection getAllNodes() { - return new ImmutableSet.Builder().addAll(nodes.values()).build(); + return Collections.unmodifiableCollection(nodes.values()); } @Override @@ -218,7 +233,7 @@ public synchronized void childEvent(CuratorFramework client, PathChildrenCacheEv ); if (!nodeType.equals(druidNode.getNodeType())) { - log.error( + log.warn( "Node[%s:%s] add is discovered by node watcher of nodeType [%s]. Ignored.", druidNode.getNodeType(), druidNode, @@ -237,7 +252,7 @@ public synchronized void childEvent(CuratorFramework client, PathChildrenCacheEv DiscoveryDruidNode druidNode = jsonMapper.readValue(event.getData().getData(), DiscoveryDruidNode.class); if (!nodeType.equals(druidNode.getNodeType())) { - log.error( + log.warn( "Node[%s:%s] removal is discovered by node watcher of nodeType [%s]. Ignored.", druidNode.getNodeType(), druidNode, @@ -264,7 +279,7 @@ public synchronized void childEvent(CuratorFramework client, PathChildrenCacheEv private synchronized void addNode(DiscoveryDruidNode druidNode) { - DiscoveryDruidNode prev = nodes.put(druidNode.getDruidNode().getHostAndPortToUse(), druidNode); + DiscoveryDruidNode prev = nodes.putIfAbsent(druidNode.getDruidNode().getHostAndPortToUse(), druidNode); if (prev == null) { for (DruidNodeDiscovery.Listener l : nodeListeners) { listenerExecutor.submit(() -> { diff --git a/server/src/main/java/io/druid/curator/discovery/DiscoveryModule.java b/server/src/main/java/io/druid/curator/discovery/DiscoveryModule.java index ee1a6dc86628..f30a2060e420 100644 --- a/server/src/main/java/io/druid/curator/discovery/DiscoveryModule.java +++ b/server/src/main/java/io/druid/curator/discovery/DiscoveryModule.java @@ -30,9 +30,8 @@ import com.google.inject.TypeLiteral; import com.google.inject.name.Named; import com.google.inject.name.Names; -import io.druid.discovery.DruidDiscoveryModule; -import io.druid.discovery.DruidNodeDiscoveryProvider; import io.druid.discovery.DruidNodeAnnouncer; +import io.druid.discovery.DruidNodeDiscoveryProvider; import io.druid.guice.DruidBinders; import io.druid.guice.JsonConfigProvider; import io.druid.guice.KeyHolder; @@ -76,6 +75,8 @@ public class DiscoveryModule implements Module { private static final String NAME = "DiscoveryModule:internal"; + + private static final String INTERNAL_DISCOVERY_PROP = "druid.discovery.type"; private static final String CURATOR_KEY = "curator"; /** @@ -152,7 +153,13 @@ public void configure(Binder binder) .in(LazySingleton.class); // internal discovery bindings. - DruidDiscoveryModule.createBindingChoices(binder, CURATOR_KEY); + PolyBind.createChoiceWithDefault( + binder, INTERNAL_DISCOVERY_PROP, Key.get(DruidNodeAnnouncer.class), CURATOR_KEY + ); + + PolyBind.createChoiceWithDefault( + binder, INTERNAL_DISCOVERY_PROP, Key.get(DruidNodeDiscoveryProvider.class), CURATOR_KEY + ); PolyBind.optionBinder(binder, Key.get(DruidNodeDiscoveryProvider.class)) .addBinding(CURATOR_KEY) diff --git a/server/src/main/java/io/druid/discovery/DruidDiscoveryModule.java b/server/src/main/java/io/druid/discovery/DruidDiscoveryModule.java deleted file mode 100644 index 01a71531fcf5..000000000000 --- a/server/src/main/java/io/druid/discovery/DruidDiscoveryModule.java +++ /dev/null @@ -1,49 +0,0 @@ -/* - * Licensed to Metamarkets Group Inc. (Metamarkets) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. Metamarkets 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 io.druid.discovery; - -import com.google.inject.Binder; -import com.google.inject.Key; -import com.google.inject.Module; -import io.druid.guice.PolyBind; - -/** - */ -public class DruidDiscoveryModule implements Module -{ - public static final String PROPERTY = "druid.discovery.type"; - - @Override - public void configure(Binder binder) - { - - } - - public static void createBindingChoices(Binder binder, String defaultPropertyValue) - { - PolyBind.createChoiceWithDefault( - binder, PROPERTY, Key.get(DruidNodeAnnouncer.class), defaultPropertyValue - ); - - PolyBind.createChoiceWithDefault( - binder, PROPERTY, Key.get(DruidNodeDiscoveryProvider.class), defaultPropertyValue - ); - } -} diff --git a/server/src/main/java/io/druid/discovery/DruidNodeDiscovery.java b/server/src/main/java/io/druid/discovery/DruidNodeDiscovery.java index d057c82cee83..7b051ccbf00b 100644 --- a/server/src/main/java/io/druid/discovery/DruidNodeDiscovery.java +++ b/server/src/main/java/io/druid/discovery/DruidNodeDiscovery.java @@ -19,14 +19,14 @@ package io.druid.discovery; -import java.util.Set; +import java.util.Collection; /** * Interface for discovering Druid Nodes announced by DruidNodeAnnouncer. */ public interface DruidNodeDiscovery { - Set getAllNodes(); + Collection getAllNodes(); void registerListener(Listener listener); interface Listener diff --git a/server/src/main/java/io/druid/discovery/DruidNodeDiscoveryProvider.java b/server/src/main/java/io/druid/discovery/DruidNodeDiscoveryProvider.java index 26c4545ffada..b959827ed63f 100644 --- a/server/src/main/java/io/druid/discovery/DruidNodeDiscoveryProvider.java +++ b/server/src/main/java/io/druid/discovery/DruidNodeDiscoveryProvider.java @@ -25,6 +25,8 @@ import io.druid.java.util.common.logger.Logger; import java.util.ArrayList; +import java.util.Collection; +import java.util.Collections; import java.util.List; import java.util.Map; import java.util.Set; @@ -45,13 +47,23 @@ public abstract class DruidNodeDiscoveryProvider public static final String NODE_TYPE_ROUTER = "router"; public static final String NODE_TYPE_MM = "middleManager"; + public static final Set ALL_NODE_TYPES = ImmutableSet.of( + NODE_TYPE_COORDINATOR, + NODE_TYPE_HISTORICAL, + NODE_TYPE_BROKER, + NODE_TYPE_OVERLORD, + NODE_TYPE_PEON, + NODE_TYPE_ROUTER, + NODE_TYPE_MM + ); + public static final Map> SERVICE_TO_NODE_TYPES = ImmutableMap.of( LookupNodeService.DISCOVERY_SERVICE_KEY, ImmutableSet.of(NODE_TYPE_BROKER, NODE_TYPE_HISTORICAL, NODE_TYPE_PEON), DataNodeService.DISCOVERY_SERVICE_KEY, ImmutableSet.of(NODE_TYPE_HISTORICAL, NODE_TYPE_PEON), WorkerNodeService.DISCOVERY_SERVICE_KEY, ImmutableSet.of(NODE_TYPE_MM) ); - private Map serviceDiscoveryMap = new ConcurrentHashMap<>(SERVICE_TO_NODE_TYPES.size()); + private final Map serviceDiscoveryMap = new ConcurrentHashMap<>(SERVICE_TO_NODE_TYPES.size()); /** * Get DruidNodeDiscovery instance to discover nodes of given nodeType. @@ -125,9 +137,9 @@ public synchronized void nodeRemoved(DiscoveryDruidNode node) } @Override - public Set getAllNodes() + public Collection getAllNodes() { - return ImmutableSet.builder().addAll(nodes.values()).build(); + return Collections.unmodifiableCollection(nodes.values()); } @Override diff --git a/server/src/main/java/io/druid/initialization/Initialization.java b/server/src/main/java/io/druid/initialization/Initialization.java index cfeea5023de6..53b7e2110af8 100644 --- a/server/src/main/java/io/druid/initialization/Initialization.java +++ b/server/src/main/java/io/druid/initialization/Initialization.java @@ -32,7 +32,6 @@ import com.google.inject.util.Modules; import io.druid.curator.CuratorModule; import io.druid.curator.discovery.DiscoveryModule; -import io.druid.discovery.DruidDiscoveryModule; import io.druid.guice.AWSModule; import io.druid.guice.AnnouncerModule; import io.druid.guice.CoordinatorDiscoveryModule; @@ -359,7 +358,6 @@ public static Injector makeInjectorWithModules(final Injector baseInjector, Iter new JettyServerModule(), new ExpressionModule(), new DiscoveryModule(), - new DruidDiscoveryModule(), new ServerViewModule(), new MetadataConfigModule(), new DerbyMetadataStorageDruidModule(), diff --git a/server/src/main/java/io/druid/server/http/ClusterResource.java b/server/src/main/java/io/druid/server/http/ClusterResource.java index 074c04d3e135..4092ed34a38f 100644 --- a/server/src/main/java/io/druid/server/http/ClusterResource.java +++ b/server/src/main/java/io/druid/server/http/ClusterResource.java @@ -29,10 +29,11 @@ import javax.ws.rs.GET; import javax.ws.rs.Path; +import javax.ws.rs.PathParam; import javax.ws.rs.Produces; import javax.ws.rs.core.MediaType; import javax.ws.rs.core.Response; -import java.util.Set; +import java.util.Collection; /** */ @@ -72,13 +73,13 @@ public Response getClusterServers() .getAllNodes() ); - Set mmNodes = druidNodeDiscoveryProvider.getForNodeType(DruidNodeDiscoveryProvider.NODE_TYPE_MM) - .getAllNodes(); + Collection mmNodes = druidNodeDiscoveryProvider.getForNodeType(DruidNodeDiscoveryProvider.NODE_TYPE_MM) + .getAllNodes(); if (!mmNodes.isEmpty()) { entityBuilder.put(DruidNodeDiscoveryProvider.NODE_TYPE_MM, mmNodes); } - Set routerNodes = druidNodeDiscoveryProvider.getForNodeType(DruidNodeDiscoveryProvider.NODE_TYPE_ROUTER) + Collection routerNodes = druidNodeDiscoveryProvider.getForNodeType(DruidNodeDiscoveryProvider.NODE_TYPE_ROUTER) .getAllNodes(); if (!routerNodes.isEmpty()) { entityBuilder.put(DruidNodeDiscoveryProvider.NODE_TYPE_ROUTER, routerNodes); @@ -86,4 +87,27 @@ public Response getClusterServers() return Response.status(Response.Status.OK).entity(entityBuilder.build()).build(); } + + @GET + @Produces({MediaType.APPLICATION_JSON}) + @Path("/{nodeType}") + public Response getClusterServers( + @PathParam("nodeType") String nodeType + ) + { + if (nodeType == null || !DruidNodeDiscoveryProvider.ALL_NODE_TYPES.contains(nodeType)) { + return Response.serverError() + .status(Response.Status.BAD_REQUEST) + .entity(String.format( + "Invalid nodeType [%s]. Valid node types are %s .", + nodeType, + DruidNodeDiscoveryProvider.ALL_NODE_TYPES + )) + .build(); + } else { + return Response.status(Response.Status.OK).entity( + druidNodeDiscoveryProvider.getForNodeType(nodeType).getAllNodes() + ).build(); + } + } } diff --git a/server/src/test/java/io/druid/curator/discovery/CuratorDruidNodeAnnouncerAndDiscoveryTest.java b/server/src/test/java/io/druid/curator/discovery/CuratorDruidNodeAnnouncerAndDiscoveryTest.java index 6ed4440e57c6..d75e2bc632c5 100644 --- a/server/src/test/java/io/druid/curator/discovery/CuratorDruidNodeAnnouncerAndDiscoveryTest.java +++ b/server/src/test/java/io/druid/curator/discovery/CuratorDruidNodeAnnouncerAndDiscoveryTest.java @@ -43,7 +43,9 @@ import org.junit.Before; import org.junit.Test; +import java.util.Collection; import java.util.HashSet; +import java.util.Set; /** */ @@ -127,11 +129,11 @@ public void configure(Binder binder) DruidNodeDiscovery coordDiscovery = druidNodeDiscoveryProvider.getForNodeType(DruidNodeDiscoveryProvider.NODE_TYPE_COORDINATOR); DruidNodeDiscovery overlordDiscovery = druidNodeDiscoveryProvider.getForNodeType(DruidNodeDiscoveryProvider.NODE_TYPE_OVERLORD); - while (!ImmutableSet.of(node1).equals(coordDiscovery.getAllNodes())) { + while (!checkNodes(ImmutableSet.of(node1), (coordDiscovery.getAllNodes()))) { Thread.sleep(100); } - while (!ImmutableSet.of(node3).equals(overlordDiscovery.getAllNodes())) { + while (!checkNodes(ImmutableSet.of(node3), overlordDiscovery.getAllNodes())) { Thread.sleep(100); } @@ -182,11 +184,11 @@ public void nodeRemoved(DiscoveryDruidNode node) druidNodeAnnouncer.announce(node2); druidNodeAnnouncer.announce(node4); - while (!ImmutableSet.of(node1, node2).equals(coordDiscovery.getAllNodes())) { + while (!checkNodes(ImmutableSet.of(node1, node2), coordDiscovery.getAllNodes())) { Thread.sleep(100); } - while (!ImmutableSet.of(node3, node4).equals(overlordDiscovery.getAllNodes())) { + while (!checkNodes(ImmutableSet.of(node3, node4), overlordDiscovery.getAllNodes())) { Thread.sleep(100); } @@ -203,11 +205,11 @@ public void nodeRemoved(DiscoveryDruidNode node) druidNodeAnnouncer.unannounce(node3); druidNodeAnnouncer.unannounce(node4); - while (!ImmutableSet.of().equals(coordDiscovery.getAllNodes())) { + while (!checkNodes(ImmutableSet.of(), coordDiscovery.getAllNodes())) { Thread.sleep(100); } - while (!ImmutableSet.of().equals(overlordDiscovery.getAllNodes())) { + while (!checkNodes(ImmutableSet.of(), overlordDiscovery.getAllNodes())) { Thread.sleep(100); } @@ -223,6 +225,11 @@ public void nodeRemoved(DiscoveryDruidNode node) announcer.stop(); } + private boolean checkNodes(Set expected, Collection actual) + { + return expected.equals(ImmutableSet.copyOf(actual)); + } + @After public void tearDown() { diff --git a/server/src/test/java/io/druid/discovery/DruidNodeDiscoveryProviderTest.java b/server/src/test/java/io/druid/discovery/DruidNodeDiscoveryProviderTest.java index d6f991443b02..e9e5b0e689fd 100644 --- a/server/src/test/java/io/druid/discovery/DruidNodeDiscoveryProviderTest.java +++ b/server/src/test/java/io/druid/discovery/DruidNodeDiscoveryProviderTest.java @@ -79,10 +79,11 @@ public void nodeRemoved(DiscoveryDruidNode node) } ); - Assert.assertEquals(ImmutableSet.of(), dataNodes); - Assert.assertEquals(ImmutableSet.of(), dataNodeDiscovery.getAllNodes()); - Assert.assertEquals(ImmutableSet.of(), lookupNodes); - Assert.assertEquals(ImmutableSet.of(), lookupNodeDiscovery.getAllNodes()); + Assert.assertTrue(dataNodes.isEmpty()); + Assert.assertTrue(dataNodes.isEmpty()); + Assert.assertTrue(dataNodeDiscovery.getAllNodes().isEmpty()); + Assert.assertTrue(lookupNodes.isEmpty()); + Assert.assertTrue(lookupNodeDiscovery.getAllNodes().isEmpty()); DiscoveryDruidNode node1 = new DiscoveryDruidNode( new DruidNode("s1", "h1", 8080, null, new ServerConfig()), @@ -158,10 +159,10 @@ LookupNodeService.DISCOVERY_SERVICE_KEY, new LookupNodeService("tier")) provider.add(node7Clone); provider.add(node8); - Assert.assertEquals(ImmutableSet.of(node1, node2, node4, node5), dataNodeDiscovery.getAllNodes()); + Assert.assertEquals(ImmutableSet.of(node1, node2, node4, node5), ImmutableSet.copyOf(dataNodeDiscovery.getAllNodes())); Assert.assertEquals(ImmutableSet.of(node1, node2, node4, node5), dataNodes); - Assert.assertEquals(ImmutableSet.of(node1, node3, node4, node6, node7), lookupNodeDiscovery.getAllNodes()); + Assert.assertEquals(ImmutableSet.of(node1, node3, node4, node6, node7), ImmutableSet.copyOf(lookupNodeDiscovery.getAllNodes())); Assert.assertEquals(ImmutableSet.of(node1, node3, node4, node6, node7), lookupNodes); provider.remove(node8); @@ -170,10 +171,10 @@ LookupNodeService.DISCOVERY_SERVICE_KEY, new LookupNodeService("tier")) provider.remove(node5); provider.remove(node4); - Assert.assertEquals(ImmutableSet.of(node1, node2), dataNodeDiscovery.getAllNodes()); + Assert.assertEquals(ImmutableSet.of(node1, node2), ImmutableSet.copyOf(dataNodeDiscovery.getAllNodes())); Assert.assertEquals(ImmutableSet.of(node1, node2), dataNodes); - Assert.assertEquals(ImmutableSet.of(node1, node3), lookupNodeDiscovery.getAllNodes()); + Assert.assertEquals(ImmutableSet.of(node1, node3), ImmutableSet.copyOf(lookupNodeDiscovery.getAllNodes())); Assert.assertEquals(ImmutableSet.of(node1, node3), lookupNodes); } diff --git a/services/src/main/java/io/druid/cli/CliBroker.java b/services/src/main/java/io/druid/cli/CliBroker.java index b1e142e1a3aa..db1961c30921 100644 --- a/services/src/main/java/io/druid/cli/CliBroker.java +++ b/services/src/main/java/io/druid/cli/CliBroker.java @@ -20,11 +20,9 @@ package io.druid.cli; import com.google.common.collect.ImmutableList; -import com.google.common.collect.ImmutableMap; import com.google.inject.Binder; -import com.google.inject.Inject; +import com.google.inject.Key; import com.google.inject.Module; -import com.google.inject.Provider; import com.google.inject.name.Names; import io.airlift.airline.Command; import io.druid.client.BrokerSegmentWatcherConfig; @@ -36,8 +34,6 @@ import io.druid.client.selector.CustomTierSelectorStrategyConfig; import io.druid.client.selector.ServerSelectorStrategy; import io.druid.client.selector.TierSelectorStrategy; -import io.druid.discovery.DiscoveryDruidNode; -import io.druid.discovery.DruidNodeAnnouncer; import io.druid.discovery.DruidNodeDiscoveryProvider; import io.druid.discovery.LookupNodeService; import io.druid.guice.CacheModule; @@ -48,8 +44,6 @@ import io.druid.guice.LifecycleModule; import io.druid.guice.QueryRunnerFactoryModule; import io.druid.guice.QueryableModule; -import io.druid.guice.annotations.Self; -import io.druid.java.util.common.lifecycle.Lifecycle; import io.druid.java.util.common.logger.Logger; import io.druid.query.QuerySegmentWalker; import io.druid.query.RetryQueryRunnerConfig; @@ -57,7 +51,6 @@ import io.druid.server.BrokerQueryResource; import io.druid.server.ClientInfoResource; import io.druid.server.ClientQuerySegmentWalker; -import io.druid.server.DruidNode; import io.druid.server.coordination.broker.DruidBroker; import io.druid.server.http.BrokerResource; import io.druid.server.initialization.jetty.JettyServerInitializer; @@ -131,51 +124,17 @@ public void configure(Binder binder) LifecycleModule.register(binder, Server.class); - binder.bind(ForSideEffectsOnlyProvider.Child.class).toProvider(ForSideEffectsOnlyProvider.class).asEagerSingleton(); + binder.bind(SideEffectsProvider.Child.class).toProvider( + new SideEffectsProvider( + DruidNodeDiscoveryProvider.NODE_TYPE_BROKER, + ImmutableList.of(LookupNodeService.class) + ) + ).in(LazySingleton.class); + LifecycleModule.registerKey(binder, Key.get(SideEffectsProvider.Child.class)); } }, new LookupModule(), new SqlModule() ); } - - private static class ForSideEffectsOnlyProvider implements Provider - { - final static class Child {}; - - @Inject - public ForSideEffectsOnlyProvider(DruidNodeAnnouncer announcer, @Self DruidNode druidNode, - LookupNodeService lookupNodeService, Lifecycle lifecycle) - { - DiscoveryDruidNode discoveryDruidNode = new DiscoveryDruidNode(druidNode, - DruidNodeDiscoveryProvider.NODE_TYPE_BROKER, - ImmutableMap.of( - lookupNodeService.getName(), lookupNodeService - )); - - lifecycle.addHandler( - new Lifecycle.Handler() - { - @Override - public void start() throws Exception - { - announcer.announce(discoveryDruidNode); - } - - @Override - public void stop() - { - announcer.unannounce(discoveryDruidNode); - } - }, - Lifecycle.Stage.LAST - ); - } - - @Override - public ForSideEffectsOnlyProvider.Child get() - { - return new Child(); - } - } } diff --git a/services/src/main/java/io/druid/cli/CliCoordinator.java b/services/src/main/java/io/druid/cli/CliCoordinator.java index 5c203f0e4879..234f40256530 100644 --- a/services/src/main/java/io/druid/cli/CliCoordinator.java +++ b/services/src/main/java/io/druid/cli/CliCoordinator.java @@ -21,19 +21,18 @@ import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.base.Predicates; -import com.google.common.collect.ImmutableMap; +import com.google.common.collect.ImmutableList; import com.google.inject.Binder; import com.google.inject.Inject; +import com.google.inject.Key; import com.google.inject.Module; -import com.google.inject.Provider; import com.google.inject.Provides; import com.google.inject.name.Names; import io.airlift.airline.Command; import io.druid.audit.AuditManager; import io.druid.client.CoordinatorServerView; +import io.druid.client.coordinator.Coordinator; import io.druid.client.indexing.IndexingServiceClient; -import io.druid.discovery.DiscoveryDruidNode; -import io.druid.discovery.DruidNodeAnnouncer; import io.druid.discovery.DruidNodeDiscoveryProvider; import io.druid.guice.ConditionalMultibind; import io.druid.guice.ConfigProvider; @@ -43,9 +42,7 @@ import io.druid.guice.LifecycleModule; import io.druid.guice.ManageLifecycle; import io.druid.guice.annotations.CoordinatorIndexingServiceHelper; -import io.druid.guice.annotations.Self; import io.druid.java.util.common.concurrent.ScheduledExecutorFactory; -import io.druid.java.util.common.lifecycle.Lifecycle; import io.druid.java.util.common.logger.Logger; import io.druid.metadata.MetadataRuleManager; import io.druid.metadata.MetadataRuleManagerConfig; @@ -55,7 +52,6 @@ import io.druid.metadata.MetadataSegmentManagerProvider; import io.druid.metadata.MetadataStorage; import io.druid.metadata.MetadataStorageProvider; -import io.druid.server.DruidNode; import io.druid.server.audit.AuditManagerProvider; import io.druid.server.coordinator.BalancerStrategyFactory; import io.druid.server.coordinator.DruidCoordinator; @@ -214,7 +210,13 @@ public void configure(Binder binder) DruidCoordinatorSegmentKiller.class ); - binder.bind(ForSideEffectsOnlyProvider.Child.class).toProvider(ForSideEffectsOnlyProvider.class).asEagerSingleton(); + binder.bind(SideEffectsProvider.Child.class).annotatedWith(Coordinator.class).toProvider( + new SideEffectsProvider( + DruidNodeDiscoveryProvider.NODE_TYPE_COORDINATOR, + ImmutableList.of() + ) + ).in(LazySingleton.class); + LifecycleModule.registerKey(binder, Key.get(SideEffectsProvider.Child.class, Coordinator.class)); } @Provides @@ -245,42 +247,4 @@ public static boolean isOverlord(Properties properties) { return Boolean.valueOf(properties.getProperty("druid.coordinator.asOverlord.enabled")).booleanValue(); } - - private static class ForSideEffectsOnlyProvider implements Provider - { - final static class Child {}; - - @Inject - public ForSideEffectsOnlyProvider(DruidNodeAnnouncer announcer, @Self DruidNode druidNode, Lifecycle lifecycle) - { - DiscoveryDruidNode discoveryDruidNode = new DiscoveryDruidNode(druidNode, - DruidNodeDiscoveryProvider.NODE_TYPE_COORDINATOR, - ImmutableMap.of() - ); - - lifecycle.addHandler( - new Lifecycle.Handler() - { - @Override - public void start() throws Exception - { - announcer.announce(discoveryDruidNode); - } - - @Override - public void stop() - { - announcer.unannounce(discoveryDruidNode); - } - }, - Lifecycle.Stage.LAST - ); - } - - @Override - public ForSideEffectsOnlyProvider.Child get() - { - return new Child(); - } - } } diff --git a/services/src/main/java/io/druid/cli/CliHistorical.java b/services/src/main/java/io/druid/cli/CliHistorical.java index 70550843ff35..f7a13b63a008 100644 --- a/services/src/main/java/io/druid/cli/CliHistorical.java +++ b/services/src/main/java/io/druid/cli/CliHistorical.java @@ -20,18 +20,14 @@ package io.druid.cli; import com.google.common.collect.ImmutableList; -import com.google.common.collect.ImmutableMap; import com.google.inject.Binder; -import com.google.inject.Inject; +import com.google.inject.Key; import com.google.inject.Module; -import com.google.inject.Provider; import com.google.inject.name.Names; import io.airlift.airline.Command; import io.druid.client.cache.CacheConfig; import io.druid.client.cache.CacheMonitor; import io.druid.discovery.DataNodeService; -import io.druid.discovery.DiscoveryDruidNode; -import io.druid.discovery.DruidNodeAnnouncer; import io.druid.discovery.DruidNodeDiscoveryProvider; import io.druid.discovery.LookupNodeService; import io.druid.guice.CacheModule; @@ -44,12 +40,9 @@ import io.druid.guice.NodeTypeConfig; import io.druid.guice.QueryRunnerFactoryModule; import io.druid.guice.QueryableModule; -import io.druid.guice.annotations.Self; -import io.druid.java.util.common.lifecycle.Lifecycle; import io.druid.java.util.common.logger.Logger; import io.druid.query.QuerySegmentWalker; import io.druid.query.lookup.LookupModule; -import io.druid.server.DruidNode; import io.druid.server.QueryResource; import io.druid.server.SegmentManager; import io.druid.server.coordination.ServerManager; @@ -115,51 +108,16 @@ public void configure(Binder binder) binder.install(new CacheModule()); MetricsModule.register(binder, CacheMonitor.class); - binder.bind(ForSideEffectsOnlyProvider.Child.class).toProvider(ForSideEffectsOnlyProvider.class).asEagerSingleton(); + binder.bind(SideEffectsProvider.Child.class).toProvider( + new SideEffectsProvider( + DruidNodeDiscoveryProvider.NODE_TYPE_HISTORICAL, + ImmutableList.of(DataNodeService.class, LookupNodeService.class) + ) + ).in(LazySingleton.class); + LifecycleModule.registerKey(binder, Key.get(SideEffectsProvider.Child.class)); } }, new LookupModule() ); } - - private static class ForSideEffectsOnlyProvider implements Provider - { - final static class Child {}; - - @Inject - public ForSideEffectsOnlyProvider(DruidNodeAnnouncer announcer, @Self DruidNode druidNode, DataNodeService dataNodeService, - LookupNodeService lookupNodeService, Lifecycle lifecycle) - { - DiscoveryDruidNode discoveryDruidNode = new DiscoveryDruidNode(druidNode, - DruidNodeDiscoveryProvider.NODE_TYPE_HISTORICAL, - ImmutableMap.of( - dataNodeService.getName(), dataNodeService, - lookupNodeService.getName(), lookupNodeService - )); - - lifecycle.addHandler( - new Lifecycle.Handler() - { - @Override - public void start() throws Exception - { - announcer.announce(discoveryDruidNode); - } - - @Override - public void stop() - { - announcer.unannounce(discoveryDruidNode); - } - }, - Lifecycle.Stage.LAST - ); - } - - @Override - public ForSideEffectsOnlyProvider.Child get() - { - return new Child(); - } - } } diff --git a/services/src/main/java/io/druid/cli/CliMiddleManager.java b/services/src/main/java/io/druid/cli/CliMiddleManager.java index 6f2799892c1a..67c58c229727 100644 --- a/services/src/main/java/io/druid/cli/CliMiddleManager.java +++ b/services/src/main/java/io/druid/cli/CliMiddleManager.java @@ -20,17 +20,13 @@ package io.druid.cli; import com.google.common.collect.ImmutableList; -import com.google.common.collect.ImmutableMap; import com.google.inject.Binder; -import com.google.inject.Inject; +import com.google.inject.Key; import com.google.inject.Module; -import com.google.inject.Provider; import com.google.inject.Provides; import com.google.inject.name.Names; import com.google.inject.util.Providers; import io.airlift.airline.Command; -import io.druid.discovery.DiscoveryDruidNode; -import io.druid.discovery.DruidNodeAnnouncer; import io.druid.discovery.DruidNodeDiscoveryProvider; import io.druid.discovery.WorkerNodeService; import io.druid.guice.IndexingServiceFirehoseModule; @@ -50,7 +46,6 @@ import io.druid.indexing.worker.WorkerTaskMonitor; import io.druid.indexing.worker.config.WorkerConfig; import io.druid.indexing.worker.http.WorkerResource; -import io.druid.java.util.common.lifecycle.Lifecycle; import io.druid.java.util.common.logger.Logger; import io.druid.segment.realtime.firehose.ChatHandlerProvider; import io.druid.server.DruidNode; @@ -105,7 +100,14 @@ public void configure(Binder binder) Jerseys.addResource(binder, WorkerResource.class); LifecycleModule.register(binder, Server.class); - binder.bind(ForSideEffectsOnlyProvider.Child.class).toProvider(ForSideEffectsOnlyProvider.class).asEagerSingleton(); + + binder.bind(SideEffectsProvider.Child.class).toProvider( + new SideEffectsProvider( + DruidNodeDiscoveryProvider.NODE_TYPE_MM, + ImmutableList.of(WorkerNodeService.class) + ) + ).in(LazySingleton.class); + LifecycleModule.registerKey(binder, Key.get(SideEffectsProvider.Child.class)); } @Provides @@ -120,58 +122,20 @@ public Worker getWorker(@Self DruidNode node, WorkerConfig config) config.getVersion() ); } + + @Provides + @LazySingleton + public WorkerNodeService getWorkerNodeService(WorkerConfig workerConfig) + { + return new WorkerNodeService( + workerConfig.getIp(), + workerConfig.getCapacity(), + workerConfig.getVersion() + ); + } }, new IndexingServiceFirehoseModule(), new IndexingServiceTaskLogsModule() ); } - - private static class ForSideEffectsOnlyProvider implements Provider - { - final static class Child {}; - - @Inject - public ForSideEffectsOnlyProvider( - DruidNodeAnnouncer announcer, - @Self DruidNode druidNode, - WorkerConfig workerConfig, - Lifecycle lifecycle - ) - { - WorkerNodeService workerNodeService = new WorkerNodeService( - workerConfig.getIp(), - workerConfig.getCapacity(), - workerConfig.getVersion() - ); - DiscoveryDruidNode discoveryDruidNode = new DiscoveryDruidNode(druidNode, - DruidNodeDiscoveryProvider.NODE_TYPE_MM, - ImmutableMap.of( - workerNodeService.getName(), workerNodeService - )); - - lifecycle.addHandler( - new Lifecycle.Handler() - { - @Override - public void start() throws Exception - { - announcer.announce(discoveryDruidNode); - } - - @Override - public void stop() - { - announcer.unannounce(discoveryDruidNode); - } - }, - Lifecycle.Stage.LAST - ); - } - - @Override - public ForSideEffectsOnlyProvider.Child get() - { - return new Child(); - } - } } diff --git a/services/src/main/java/io/druid/cli/CliOverlord.java b/services/src/main/java/io/druid/cli/CliOverlord.java index ff00d0f86ebc..01ed9ba7672f 100644 --- a/services/src/main/java/io/druid/cli/CliOverlord.java +++ b/services/src/main/java/io/druid/cli/CliOverlord.java @@ -20,13 +20,10 @@ package io.druid.cli; import com.google.common.collect.ImmutableList; -import com.google.common.collect.ImmutableMap; import com.google.inject.Binder; -import com.google.inject.Inject; import com.google.inject.Injector; import com.google.inject.Key; import com.google.inject.Module; -import com.google.inject.Provider; import com.google.inject.TypeLiteral; import com.google.inject.multibindings.MapBinder; import com.google.inject.multibindings.Multibinder; @@ -35,9 +32,8 @@ import com.google.inject.util.Providers; import io.airlift.airline.Command; import io.druid.audit.AuditManager; +import io.druid.client.indexing.IndexingService; import io.druid.client.indexing.IndexingServiceSelectorConfig; -import io.druid.discovery.DiscoveryDruidNode; -import io.druid.discovery.DruidNodeAnnouncer; import io.druid.discovery.DruidNodeDiscoveryProvider; import io.druid.guice.IndexingServiceFirehoseModule; import io.druid.guice.IndexingServiceModuleHelper; @@ -50,7 +46,6 @@ import io.druid.guice.ListProvider; import io.druid.guice.ManageLifecycle; import io.druid.guice.PolyBind; -import io.druid.guice.annotations.Self; import io.druid.indexing.common.actions.LocalTaskActionClientFactory; import io.druid.indexing.common.actions.TaskActionClientFactory; import io.druid.indexing.common.actions.TaskActionToolbox; @@ -83,10 +78,8 @@ import io.druid.indexing.overlord.supervisor.SupervisorManager; import io.druid.indexing.overlord.supervisor.SupervisorResource; import io.druid.indexing.worker.config.WorkerConfig; -import io.druid.java.util.common.lifecycle.Lifecycle; import io.druid.java.util.common.logger.Logger; import io.druid.segment.realtime.firehose.ChatHandlerProvider; -import io.druid.server.DruidNode; import io.druid.server.audit.AuditManagerProvider; import io.druid.server.coordinator.CoordinatorOverlordServiceConfig; import io.druid.server.http.RedirectFilter; @@ -192,7 +185,13 @@ public void configure(Binder binder) LifecycleModule.register(binder, Server.class); } - binder.bind(ForSideEffectsOnlyProvider.Child.class).toProvider(ForSideEffectsOnlyProvider.class).asEagerSingleton(); + binder.bind(SideEffectsProvider.Child.class).annotatedWith(IndexingService.class).toProvider( + new SideEffectsProvider( + DruidNodeDiscoveryProvider.NODE_TYPE_OVERLORD, + ImmutableList.of() + ) + ).in(LazySingleton.class); + LifecycleModule.registerKey(binder, Key.get(SideEffectsProvider.Child.class, IndexingService.class)); } private void configureTaskStorage(Binder binder) @@ -277,44 +276,6 @@ private void configureOverlordHelpers(Binder binder) ); } - private static class ForSideEffectsOnlyProvider implements Provider - { - final static class Child {}; - - @Inject - public ForSideEffectsOnlyProvider(DruidNodeAnnouncer announcer, @Self DruidNode druidNode, Lifecycle lifecycle) - { - DiscoveryDruidNode discoveryDruidNode = new DiscoveryDruidNode(druidNode, - DruidNodeDiscoveryProvider.NODE_TYPE_OVERLORD, - ImmutableMap.of() - ); - - lifecycle.addHandler( - new Lifecycle.Handler() - { - @Override - public void start() throws Exception - { - announcer.announce(discoveryDruidNode); - } - - @Override - public void stop() - { - announcer.unannounce(discoveryDruidNode); - } - }, - Lifecycle.Stage.LAST - ); - } - - @Override - public ForSideEffectsOnlyProvider.Child get() - { - return new Child(); - } - } - /** */ private static class OverlordJettyServerInitializer implements JettyServerInitializer diff --git a/services/src/main/java/io/druid/cli/CliRouter.java b/services/src/main/java/io/druid/cli/CliRouter.java index c31ce80f945f..adcb1eacb651 100644 --- a/services/src/main/java/io/druid/cli/CliRouter.java +++ b/services/src/main/java/io/druid/cli/CliRouter.java @@ -20,11 +20,9 @@ package io.druid.cli; import com.google.common.collect.ImmutableList; -import com.google.common.collect.ImmutableMap; import com.google.inject.Binder; -import com.google.inject.Inject; +import com.google.inject.Key; import com.google.inject.Module; -import com.google.inject.Provider; import com.google.inject.Provides; import com.google.inject.TypeLiteral; import com.google.inject.name.Names; @@ -32,8 +30,6 @@ import io.druid.curator.discovery.DiscoveryModule; import io.druid.curator.discovery.ServerDiscoveryFactory; import io.druid.curator.discovery.ServerDiscoverySelector; -import io.druid.discovery.DiscoveryDruidNode; -import io.druid.discovery.DruidNodeAnnouncer; import io.druid.discovery.DruidNodeDiscoveryProvider; import io.druid.guice.Jerseys; import io.druid.guice.JsonConfigProvider; @@ -45,11 +41,9 @@ import io.druid.guice.RouterProcessingModule; import io.druid.guice.annotations.Self; import io.druid.guice.http.JettyHttpClientModule; -import io.druid.java.util.common.lifecycle.Lifecycle; import io.druid.java.util.common.logger.Logger; import io.druid.query.lookup.LookupModule; import io.druid.server.AsyncQueryForwardingServlet; -import io.druid.server.DruidNode; import io.druid.server.http.RouterResource; import io.druid.server.initialization.jetty.JettyServerInitializer; import io.druid.server.metrics.QueryCountStatsProvider; @@ -118,7 +112,13 @@ public void configure(Binder binder) LifecycleModule.register(binder, Server.class); DiscoveryModule.register(binder, Self.class); - binder.bind(ForSideEffectsOnlyProvider.Child.class).toProvider(ForSideEffectsOnlyProvider.class).asEagerSingleton(); + binder.bind(SideEffectsProvider.Child.class).toProvider( + new SideEffectsProvider( + DruidNodeDiscoveryProvider.NODE_TYPE_ROUTER, + ImmutableList.of() + ) + ).in(LazySingleton.class); + LifecycleModule.registerKey(binder, Key.get(SideEffectsProvider.Child.class)); } @Provides @@ -135,42 +135,4 @@ public ServerDiscoverySelector getCoordinatorServerDiscoverySelector( new LookupModule() ); } - - private static class ForSideEffectsOnlyProvider implements Provider - { - final static class Child {}; - - @Inject - public ForSideEffectsOnlyProvider(DruidNodeAnnouncer announcer, @Self DruidNode druidNode, Lifecycle lifecycle) - { - DiscoveryDruidNode discoveryDruidNode = new DiscoveryDruidNode(druidNode, - DruidNodeDiscoveryProvider.NODE_TYPE_ROUTER, - ImmutableMap.of() - ); - - lifecycle.addHandler( - new Lifecycle.Handler() - { - @Override - public void start() throws Exception - { - announcer.announce(discoveryDruidNode); - } - - @Override - public void stop() - { - announcer.unannounce(discoveryDruidNode); - } - }, - Lifecycle.Stage.LAST - ); - } - - @Override - public ForSideEffectsOnlyProvider.Child get() - { - return new Child(); - } - } } diff --git a/services/src/main/java/io/druid/cli/ServerRunnable.java b/services/src/main/java/io/druid/cli/ServerRunnable.java index 5e2c63816241..9dec4ee60d47 100644 --- a/services/src/main/java/io/druid/cli/ServerRunnable.java +++ b/services/src/main/java/io/druid/cli/ServerRunnable.java @@ -20,10 +20,19 @@ package io.druid.cli; import com.google.common.base.Throwables; +import com.google.common.collect.ImmutableMap; +import com.google.inject.Inject; import com.google.inject.Injector; - +import com.google.inject.Provider; +import io.druid.discovery.DiscoveryDruidNode; +import io.druid.discovery.DruidNodeAnnouncer; +import io.druid.discovery.DruidService; +import io.druid.guice.annotations.Self; import io.druid.java.util.common.lifecycle.Lifecycle; import io.druid.java.util.common.logger.Logger; +import io.druid.server.DruidNode; + +import java.util.List; /** */ @@ -47,4 +56,69 @@ public void run() throw Throwables.propagate(e); } } + + /** + * This is a helper class used by CliXXX classes to announce DiscoveryDruidNode + * as part of lifecycle Stage.LAST . + */ + protected static class SideEffectsProvider implements Provider + { + public static class Child {} + + @Inject @Self + private DruidNode druidNode; + + @Inject + private DruidNodeAnnouncer announcer; + + @Inject + private Lifecycle lifecycle; + + @Inject + private Injector injector; + + private final String nodeType; + private final List> serviceClasses; + + public SideEffectsProvider(String nodeType, List> serviceClasses) + { + this.nodeType = nodeType; + this.serviceClasses = serviceClasses; + } + + @Override + public Child get() + { + ImmutableMap.Builder builder = new ImmutableMap.Builder<>(); + for (Class clazz : serviceClasses) { + DruidService service = injector.getInstance(clazz); + builder.put(service.getName(), service); + } + + DiscoveryDruidNode discoveryDruidNode = new DiscoveryDruidNode(druidNode, + nodeType, + builder.build() + ); + + lifecycle.addHandler( + new Lifecycle.Handler() + { + @Override + public void start() throws Exception + { + announcer.announce(discoveryDruidNode); + } + + @Override + public void stop() + { + announcer.unannounce(discoveryDruidNode); + } + }, + Lifecycle.Stage.LAST + ); + + return new Child(); + } + } } From 013a0bc10c8d5041b081063a07abfdca51b92c92 Mon Sep 17 00:00:00 2001 From: Himanshu Gupta Date: Thu, 10 Aug 2017 10:25:13 -0500 Subject: [PATCH 4/9] more fixes --- .../curator/discovery/CuratorDruidNodeDiscoveryProvider.java | 2 +- .../discovery/CuratorDruidNodeAnnouncerAndDiscoveryTest.java | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/server/src/main/java/io/druid/curator/discovery/CuratorDruidNodeDiscoveryProvider.java b/server/src/main/java/io/druid/curator/discovery/CuratorDruidNodeDiscoveryProvider.java index 28b33c1bad7d..32c64b1a9e94 100644 --- a/server/src/main/java/io/druid/curator/discovery/CuratorDruidNodeDiscoveryProvider.java +++ b/server/src/main/java/io/druid/curator/discovery/CuratorDruidNodeDiscoveryProvider.java @@ -179,7 +179,7 @@ private static class NodeTypeWatcher implements DruidNodeDiscovery, PathChildren this.jsonMapper = jsonMapper; // This is required to be single threaded from Docs in PathChildrenCache; - this.cacheExecutor = Execs.singleThreaded(String.format("NodeTypeWatcher[%s]-[%%d]", nodeType)); + this.cacheExecutor = Execs.singleThreaded(String.format("NodeTypeWatcher[%s]", nodeType)); this.cache = new PathChildrenCache( curatorFramework, ZKPaths.makePath(basePath, nodeType), diff --git a/server/src/test/java/io/druid/curator/discovery/CuratorDruidNodeAnnouncerAndDiscoveryTest.java b/server/src/test/java/io/druid/curator/discovery/CuratorDruidNodeAnnouncerAndDiscoveryTest.java index d75e2bc632c5..c003c69524c3 100644 --- a/server/src/test/java/io/druid/curator/discovery/CuratorDruidNodeAnnouncerAndDiscoveryTest.java +++ b/server/src/test/java/io/druid/curator/discovery/CuratorDruidNodeAnnouncerAndDiscoveryTest.java @@ -129,7 +129,7 @@ public void configure(Binder binder) DruidNodeDiscovery coordDiscovery = druidNodeDiscoveryProvider.getForNodeType(DruidNodeDiscoveryProvider.NODE_TYPE_COORDINATOR); DruidNodeDiscovery overlordDiscovery = druidNodeDiscoveryProvider.getForNodeType(DruidNodeDiscoveryProvider.NODE_TYPE_OVERLORD); - while (!checkNodes(ImmutableSet.of(node1), (coordDiscovery.getAllNodes()))) { + while (!checkNodes(ImmutableSet.of(node1), coordDiscovery.getAllNodes())) { Thread.sleep(100); } From 3eb23b3b23351c5ef4c8954099ad1ed2655df051 Mon Sep 17 00:00:00 2001 From: Himanshu Gupta Date: Mon, 14 Aug 2017 14:16:13 -0500 Subject: [PATCH 5/9] address more review comments --- .../discovery/CuratorDruidNodeAnnouncer.java | 10 +- .../CuratorDruidNodeDiscoveryProvider.java | 217 ++++++++++-------- .../CuratorDiscoveryConfig.java | 8 - .../server/initialization/ZkPathsConfig.java | 5 + ...torDruidNodeAnnouncerAndDiscoveryTest.java | 6 +- .../src/main/java/io/druid/cli/CliBroker.java | 6 +- .../java/io/druid/cli/CliCoordinator.java | 6 +- .../main/java/io/druid/cli/CliHistorical.java | 6 +- .../java/io/druid/cli/CliMiddleManager.java | 6 +- .../main/java/io/druid/cli/CliOverlord.java | 6 +- .../src/main/java/io/druid/cli/CliRouter.java | 6 +- .../java/io/druid/cli/ServerRunnable.java | 4 +- 12 files changed, 156 insertions(+), 130 deletions(-) diff --git a/server/src/main/java/io/druid/curator/discovery/CuratorDruidNodeAnnouncer.java b/server/src/main/java/io/druid/curator/discovery/CuratorDruidNodeAnnouncer.java index f8c6f1f48ca0..f2ea115ceb9e 100644 --- a/server/src/main/java/io/druid/curator/discovery/CuratorDruidNodeAnnouncer.java +++ b/server/src/main/java/io/druid/curator/discovery/CuratorDruidNodeAnnouncer.java @@ -28,7 +28,7 @@ import io.druid.discovery.DruidNodeAnnouncer; import io.druid.guice.annotations.Json; import io.druid.java.util.common.logger.Logger; -import io.druid.server.initialization.CuratorDiscoveryConfig; +import io.druid.server.initialization.ZkPathsConfig; import org.apache.curator.utils.ZKPaths; /** @@ -38,13 +38,13 @@ public class CuratorDruidNodeAnnouncer implements DruidNodeAnnouncer private static final Logger log = new Logger(CuratorDruidNodeAnnouncer.class); private final Announcer announcer; - private final CuratorDiscoveryConfig config; + private final ZkPathsConfig config; private final ObjectMapper jsonMapper; @Inject public CuratorDruidNodeAnnouncer( Announcer announcer, - CuratorDiscoveryConfig config, + ZkPathsConfig config, @Json ObjectMapper jsonMapper ) { @@ -54,7 +54,7 @@ public CuratorDruidNodeAnnouncer( } @Override - public synchronized void announce(DiscoveryDruidNode discoveryDruidNode) + public void announce(DiscoveryDruidNode discoveryDruidNode) { try { log.info("Announcing [%s].", discoveryDruidNode); @@ -76,7 +76,7 @@ public synchronized void announce(DiscoveryDruidNode discoveryDruidNode) } @Override - public synchronized void unannounce(DiscoveryDruidNode discoveryDruidNode) + public void unannounce(DiscoveryDruidNode discoveryDruidNode) { log.info("Unannouncing [%s].", discoveryDruidNode); diff --git a/server/src/main/java/io/druid/curator/discovery/CuratorDruidNodeDiscoveryProvider.java b/server/src/main/java/io/druid/curator/discovery/CuratorDruidNodeDiscoveryProvider.java index 32c64b1a9e94..5996f500adb7 100644 --- a/server/src/main/java/io/druid/curator/discovery/CuratorDruidNodeDiscoveryProvider.java +++ b/server/src/main/java/io/druid/curator/discovery/CuratorDruidNodeDiscoveryProvider.java @@ -34,11 +34,10 @@ import io.druid.java.util.common.lifecycle.LifecycleStart; import io.druid.java.util.common.lifecycle.LifecycleStop; import io.druid.java.util.common.logger.Logger; -import io.druid.server.initialization.CuratorDiscoveryConfig; +import io.druid.server.initialization.ZkPathsConfig; import org.apache.curator.framework.CuratorFramework; import org.apache.curator.framework.recipes.cache.PathChildrenCache; import org.apache.curator.framework.recipes.cache.PathChildrenCacheEvent; -import org.apache.curator.framework.recipes.cache.PathChildrenCacheListener; import org.apache.curator.utils.ZKPaths; import java.util.ArrayList; @@ -58,7 +57,7 @@ public class CuratorDruidNodeDiscoveryProvider extends DruidNodeDiscoveryProvide private static final Logger log = new Logger(CuratorDruidNodeDiscoveryProvider.class); private final CuratorFramework curatorFramework; - private final CuratorDiscoveryConfig config; + private final ZkPathsConfig config; private final ObjectMapper jsonMapper; private ExecutorService listenerExecutor; @@ -70,7 +69,7 @@ public class CuratorDruidNodeDiscoveryProvider extends DruidNodeDiscoveryProvide @Inject public CuratorDruidNodeDiscoveryProvider( CuratorFramework curatorFramework, - CuratorDiscoveryConfig config, + ZkPathsConfig config, @Json ObjectMapper jsonMapper ) { @@ -146,7 +145,7 @@ public void stop() log.info("stopped"); } - private static class NodeTypeWatcher implements DruidNodeDiscovery, PathChildrenCacheListener + private static class NodeTypeWatcher implements DruidNodeDiscovery { private static final Logger log = new Logger(NodeTypeWatcher.class); @@ -165,6 +164,8 @@ private static class NodeTypeWatcher implements DruidNodeDiscovery, PathChildren private final List nodeListeners = new ArrayList(); + private final Object lock = new Object(); + NodeTypeWatcher( ExecutorService listenerExecutor, CuratorFramework curatorFramework, @@ -196,131 +197,159 @@ public Collection getAllNodes() } @Override - public synchronized void registerListener(DruidNodeDiscovery.Listener listener) - { - for (DiscoveryDruidNode node : nodes.values()) { - listenerExecutor.submit(() -> { - try { - listener.nodeAdded(node); - } - catch (Exception ex) { - log.error(ex, "Exception occured in DiscoveryDruidNode.nodeAdded(node=[%s]) in listener [%s].", node, listener); - } - }); - } - - nodeListeners.add(listener); - } - - @Override - public synchronized void childEvent(CuratorFramework client, PathChildrenCacheEvent event) throws Exception + public void registerListener(DruidNodeDiscovery.Listener listener) { - try { - switch (event.getType()) { - case CHILD_ADDED: { - final byte[] data; + synchronized (lock) { + for (DiscoveryDruidNode node : nodes.values()) { + listenerExecutor.submit(() -> { try { - data = curatorFramework.getData().decompressed().forPath(event.getData().getPath()); + listener.nodeAdded(node); } catch (Exception ex) { - log.error(ex, "Failed to get data for path [%s]. Ignoring event [%s].", event.getData().getPath(), event.getType()); - return; + log.error( + ex, + "Exception occured in DiscoveryDruidNode.nodeAdded(node=[%s]) in listener [%s].", + node, + listener + ); } + }); + } + nodeListeners.add(listener); + } + } - DiscoveryDruidNode druidNode = jsonMapper.readValue( - data, - DiscoveryDruidNode.class - ); - - if (!nodeType.equals(druidNode.getNodeType())) { - log.warn( - "Node[%s:%s] add is discovered by node watcher of nodeType [%s]. Ignored.", - druidNode.getNodeType(), - druidNode, - nodeType + public void handleChildEvent(CuratorFramework client, PathChildrenCacheEvent event) throws Exception + { + synchronized (lock) { + 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 ); - return; - } - log.info("Received event [%s] for Node[%s:%s].", event.getType(), druidNode.getNodeType(), druidNode); + if (!nodeType.equals(druidNode.getNodeType())) { + log.warn( + "Node[%s:%s] add is discovered by node watcher of nodeType [%s]. Ignored.", + druidNode.getNodeType(), + druidNode, + nodeType + ); + return; + } - addNode(druidNode); + log.info("Received event [%s] for Node[%s:%s].", event.getType(), druidNode.getNodeType(), druidNode); - break; - } - case CHILD_REMOVED: { - DiscoveryDruidNode druidNode = jsonMapper.readValue(event.getData().getData(), DiscoveryDruidNode.class); + addNode(druidNode); - if (!nodeType.equals(druidNode.getNodeType())) { - log.warn( - "Node[%s:%s] removal is discovered by node watcher of nodeType [%s]. Ignored.", - druidNode.getNodeType(), - druidNode, - nodeType - ); - return; + break; } + case CHILD_REMOVED: { + DiscoveryDruidNode druidNode = jsonMapper.readValue(event.getData().getData(), DiscoveryDruidNode.class); - log.info("Node[%s:%s] disappeared.", druidNode.getNodeType(), druidNode); + if (!nodeType.equals(druidNode.getNodeType())) { + log.warn( + "Node[%s:%s] removal is discovered by node watcher of nodeType [%s]. Ignored.", + druidNode.getNodeType(), + druidNode, + nodeType + ); + return; + } - removeNode(druidNode); + log.info("Node[%s:%s] disappeared.", druidNode.getNodeType(), druidNode); - break; - } - default: { - log.error("Ignored event type [%s] for nodeType [%s] watcher.", event.getType(), nodeType); + removeNode(druidNode); + + break; + } + default: { + log.error("Ignored event type [%s] for nodeType [%s] watcher.", event.getType(), nodeType); + } } } + catch (Exception ex) { + log.error(ex, "unknown error in node watcher for type [%s].", nodeType); + } } - catch (Exception ex) { - log.error(ex, "unknown error in node watcher for type [%s].", nodeType); + } + + private void addNode(DiscoveryDruidNode druidNode) + { + synchronized (lock) { + DiscoveryDruidNode prev = nodes.putIfAbsent(druidNode.getDruidNode().getHostAndPortToUse(), druidNode); + if (prev == null) { + for (DruidNodeDiscovery.Listener l : nodeListeners) { + listenerExecutor.submit(() -> { + try { + l.nodeAdded(druidNode); + } + catch (Exception ex) { + log.error( + ex, + "Exception occured in DiscoveryDruidNode.nodeAdded(node=[%s]) in listener [%s].", + druidNode, + l + ); + } + }); + } + } else { + log.warn("Node[%s] discovered but existed already [%s].", druidNode, prev); + } } } - private synchronized void addNode(DiscoveryDruidNode druidNode) + private void removeNode(DiscoveryDruidNode druidNode) { - DiscoveryDruidNode prev = nodes.putIfAbsent(druidNode.getDruidNode().getHostAndPortToUse(), druidNode); - if (prev == null) { + synchronized (lock) { + DiscoveryDruidNode prev = nodes.remove(druidNode.getDruidNode().getHostAndPortToUse()); + + if (prev == null) { + log.warn("Noticed disappearance of unknown druid node [%s:%s].", druidNode.getNodeType(), druidNode); + return; + } + for (DruidNodeDiscovery.Listener l : nodeListeners) { listenerExecutor.submit(() -> { try { - l.nodeAdded(druidNode); + l.nodeRemoved(druidNode); } catch (Exception ex) { - log.error(ex, "Exception occured in DiscoveryDruidNode.nodeAdded(node=[%s]) in listener [%s].", druidNode, l); + log.error( + ex, + "Exception occured in DiscoveryDruidNode.nodeRemoved(node=[%s]) in listener [%s].", + druidNode, + l + ); } }); } - } else { - log.warn("Node[%s] discovered but existed already [%s].", druidNode, prev); - } - } - - private synchronized void removeNode(DiscoveryDruidNode druidNode) - { - DiscoveryDruidNode prev = nodes.remove(druidNode.getDruidNode().getHostAndPortToUse()); - - if (prev == null) { - log.warn("Noticed disappearance of unknown druid node [%s:%s].", druidNode.getNodeType(), druidNode); - return; - } - - for (DruidNodeDiscovery.Listener l : nodeListeners) { - listenerExecutor.submit(() -> { - try { - l.nodeRemoved(druidNode); - } - catch (Exception ex) { - log.error(ex, "Exception occured in DiscoveryDruidNode.nodeRemoved(node=[%s]) in listener [%s].", druidNode, l); - } - }); } } public void start() { try { - cache.getListenable().addListener(this); + cache.getListenable().addListener( + (client, event) -> handleChildEvent(client, event) + ); cache.start(); } catch (Exception ex) { diff --git a/server/src/main/java/io/druid/server/initialization/CuratorDiscoveryConfig.java b/server/src/main/java/io/druid/server/initialization/CuratorDiscoveryConfig.java index 1de970c6fa67..b05c46dfe2da 100644 --- a/server/src/main/java/io/druid/server/initialization/CuratorDiscoveryConfig.java +++ b/server/src/main/java/io/druid/server/initialization/CuratorDiscoveryConfig.java @@ -28,14 +28,6 @@ public class CuratorDiscoveryConfig @JsonProperty private String path = "/druid/discovery"; - @JsonProperty - private String internalDiscoveryPath = "/druid/internal-discovery"; - - public String getInternalDiscoveryPath() - { - return internalDiscoveryPath; - } - public String getPath() { return path; diff --git a/server/src/main/java/io/druid/server/initialization/ZkPathsConfig.java b/server/src/main/java/io/druid/server/initialization/ZkPathsConfig.java index 96e3532eb86d..3e8006a029d4 100644 --- a/server/src/main/java/io/druid/server/initialization/ZkPathsConfig.java +++ b/server/src/main/java/io/druid/server/initialization/ZkPathsConfig.java @@ -90,6 +90,11 @@ public String getConnectorPath() return (null == connectorPath) ? defaultPath("connector") : connectorPath; } + public String getInternalDiscoveryPath() + { + return defaultPath("internal-discovery"); + } + public String defaultPath(final String subPath) { return ZKPaths.makePath(getBase(), subPath); diff --git a/server/src/test/java/io/druid/curator/discovery/CuratorDruidNodeAnnouncerAndDiscoveryTest.java b/server/src/test/java/io/druid/curator/discovery/CuratorDruidNodeAnnouncerAndDiscoveryTest.java index c003c69524c3..d2ac80bc919c 100644 --- a/server/src/test/java/io/druid/curator/discovery/CuratorDruidNodeAnnouncerAndDiscoveryTest.java +++ b/server/src/test/java/io/druid/curator/discovery/CuratorDruidNodeAnnouncerAndDiscoveryTest.java @@ -37,8 +37,8 @@ import io.druid.guice.GuiceInjectors; import io.druid.initialization.Initialization; import io.druid.server.DruidNode; -import io.druid.server.initialization.CuratorDiscoveryConfig; import io.druid.server.initialization.ServerConfig; +import io.druid.server.initialization.ZkPathsConfig; import org.junit.After; import org.junit.Before; import org.junit.Test; @@ -88,7 +88,7 @@ public void configure(Binder binder) CuratorDruidNodeAnnouncer druidNodeAnnouncer = new CuratorDruidNodeAnnouncer( announcer, - new CuratorDiscoveryConfig(), + new ZkPathsConfig(), objectMapper ); @@ -121,7 +121,7 @@ public void configure(Binder binder) CuratorDruidNodeDiscoveryProvider druidNodeDiscoveryProvider = new CuratorDruidNodeDiscoveryProvider( curator, - new CuratorDiscoveryConfig(), + new ZkPathsConfig(), objectMapper ); druidNodeDiscoveryProvider.start(); diff --git a/services/src/main/java/io/druid/cli/CliBroker.java b/services/src/main/java/io/druid/cli/CliBroker.java index db1961c30921..3bade6247bf3 100644 --- a/services/src/main/java/io/druid/cli/CliBroker.java +++ b/services/src/main/java/io/druid/cli/CliBroker.java @@ -124,13 +124,13 @@ public void configure(Binder binder) LifecycleModule.register(binder, Server.class); - binder.bind(SideEffectsProvider.Child.class).toProvider( - new SideEffectsProvider( + binder.bind(DiscoverySideEffectsProvider.Child.class).toProvider( + new DiscoverySideEffectsProvider( DruidNodeDiscoveryProvider.NODE_TYPE_BROKER, ImmutableList.of(LookupNodeService.class) ) ).in(LazySingleton.class); - LifecycleModule.registerKey(binder, Key.get(SideEffectsProvider.Child.class)); + LifecycleModule.registerKey(binder, Key.get(DiscoverySideEffectsProvider.Child.class)); } }, new LookupModule(), diff --git a/services/src/main/java/io/druid/cli/CliCoordinator.java b/services/src/main/java/io/druid/cli/CliCoordinator.java index 234f40256530..939ddeaceb59 100644 --- a/services/src/main/java/io/druid/cli/CliCoordinator.java +++ b/services/src/main/java/io/druid/cli/CliCoordinator.java @@ -210,13 +210,13 @@ public void configure(Binder binder) DruidCoordinatorSegmentKiller.class ); - binder.bind(SideEffectsProvider.Child.class).annotatedWith(Coordinator.class).toProvider( - new SideEffectsProvider( + binder.bind(DiscoverySideEffectsProvider.Child.class).annotatedWith(Coordinator.class).toProvider( + new DiscoverySideEffectsProvider( DruidNodeDiscoveryProvider.NODE_TYPE_COORDINATOR, ImmutableList.of() ) ).in(LazySingleton.class); - LifecycleModule.registerKey(binder, Key.get(SideEffectsProvider.Child.class, Coordinator.class)); + LifecycleModule.registerKey(binder, Key.get(DiscoverySideEffectsProvider.Child.class, Coordinator.class)); } @Provides diff --git a/services/src/main/java/io/druid/cli/CliHistorical.java b/services/src/main/java/io/druid/cli/CliHistorical.java index f7a13b63a008..7f761049fef2 100644 --- a/services/src/main/java/io/druid/cli/CliHistorical.java +++ b/services/src/main/java/io/druid/cli/CliHistorical.java @@ -108,13 +108,13 @@ public void configure(Binder binder) binder.install(new CacheModule()); MetricsModule.register(binder, CacheMonitor.class); - binder.bind(SideEffectsProvider.Child.class).toProvider( - new SideEffectsProvider( + binder.bind(DiscoverySideEffectsProvider.Child.class).toProvider( + new DiscoverySideEffectsProvider( DruidNodeDiscoveryProvider.NODE_TYPE_HISTORICAL, ImmutableList.of(DataNodeService.class, LookupNodeService.class) ) ).in(LazySingleton.class); - LifecycleModule.registerKey(binder, Key.get(SideEffectsProvider.Child.class)); + LifecycleModule.registerKey(binder, Key.get(DiscoverySideEffectsProvider.Child.class)); } }, new LookupModule() diff --git a/services/src/main/java/io/druid/cli/CliMiddleManager.java b/services/src/main/java/io/druid/cli/CliMiddleManager.java index 67c58c229727..7be166cb40a5 100644 --- a/services/src/main/java/io/druid/cli/CliMiddleManager.java +++ b/services/src/main/java/io/druid/cli/CliMiddleManager.java @@ -101,13 +101,13 @@ public void configure(Binder binder) LifecycleModule.register(binder, Server.class); - binder.bind(SideEffectsProvider.Child.class).toProvider( - new SideEffectsProvider( + binder.bind(DiscoverySideEffectsProvider.Child.class).toProvider( + new DiscoverySideEffectsProvider( DruidNodeDiscoveryProvider.NODE_TYPE_MM, ImmutableList.of(WorkerNodeService.class) ) ).in(LazySingleton.class); - LifecycleModule.registerKey(binder, Key.get(SideEffectsProvider.Child.class)); + LifecycleModule.registerKey(binder, Key.get(DiscoverySideEffectsProvider.Child.class)); } @Provides diff --git a/services/src/main/java/io/druid/cli/CliOverlord.java b/services/src/main/java/io/druid/cli/CliOverlord.java index 01ed9ba7672f..c3db181df839 100644 --- a/services/src/main/java/io/druid/cli/CliOverlord.java +++ b/services/src/main/java/io/druid/cli/CliOverlord.java @@ -185,13 +185,13 @@ public void configure(Binder binder) LifecycleModule.register(binder, Server.class); } - binder.bind(SideEffectsProvider.Child.class).annotatedWith(IndexingService.class).toProvider( - new SideEffectsProvider( + binder.bind(DiscoverySideEffectsProvider.Child.class).annotatedWith(IndexingService.class).toProvider( + new DiscoverySideEffectsProvider( DruidNodeDiscoveryProvider.NODE_TYPE_OVERLORD, ImmutableList.of() ) ).in(LazySingleton.class); - LifecycleModule.registerKey(binder, Key.get(SideEffectsProvider.Child.class, IndexingService.class)); + LifecycleModule.registerKey(binder, Key.get(DiscoverySideEffectsProvider.Child.class, IndexingService.class)); } private void configureTaskStorage(Binder binder) diff --git a/services/src/main/java/io/druid/cli/CliRouter.java b/services/src/main/java/io/druid/cli/CliRouter.java index adcb1eacb651..aa28951c836d 100644 --- a/services/src/main/java/io/druid/cli/CliRouter.java +++ b/services/src/main/java/io/druid/cli/CliRouter.java @@ -112,13 +112,13 @@ public void configure(Binder binder) LifecycleModule.register(binder, Server.class); DiscoveryModule.register(binder, Self.class); - binder.bind(SideEffectsProvider.Child.class).toProvider( - new SideEffectsProvider( + binder.bind(DiscoverySideEffectsProvider.Child.class).toProvider( + new DiscoverySideEffectsProvider( DruidNodeDiscoveryProvider.NODE_TYPE_ROUTER, ImmutableList.of() ) ).in(LazySingleton.class); - LifecycleModule.registerKey(binder, Key.get(SideEffectsProvider.Child.class)); + LifecycleModule.registerKey(binder, Key.get(DiscoverySideEffectsProvider.Child.class)); } @Provides diff --git a/services/src/main/java/io/druid/cli/ServerRunnable.java b/services/src/main/java/io/druid/cli/ServerRunnable.java index 9dec4ee60d47..a2f741b0f3f5 100644 --- a/services/src/main/java/io/druid/cli/ServerRunnable.java +++ b/services/src/main/java/io/druid/cli/ServerRunnable.java @@ -61,7 +61,7 @@ public void run() * This is a helper class used by CliXXX classes to announce DiscoveryDruidNode * as part of lifecycle Stage.LAST . */ - protected static class SideEffectsProvider implements Provider + protected static class DiscoverySideEffectsProvider implements Provider { public static class Child {} @@ -80,7 +80,7 @@ public static class Child {} private final String nodeType; private final List> serviceClasses; - public SideEffectsProvider(String nodeType, List> serviceClasses) + public DiscoverySideEffectsProvider(String nodeType, List> serviceClasses) { this.nodeType = nodeType; this.serviceClasses = serviceClasses; From f928a46597cadb6456e7b2dc48c89ac37fc81f77 Mon Sep 17 00:00:00 2001 From: Himanshu Gupta Date: Mon, 14 Aug 2017 15:46:53 -0500 Subject: [PATCH 6/9] simplify ObjectMapper setup in CuratorDruidNodeAnnouncerAndDiscoveryTest --- ...torDruidNodeAnnouncerAndDiscoveryTest.java | 34 ++++++------------- 1 file changed, 10 insertions(+), 24 deletions(-) diff --git a/server/src/test/java/io/druid/curator/discovery/CuratorDruidNodeAnnouncerAndDiscoveryTest.java b/server/src/test/java/io/druid/curator/discovery/CuratorDruidNodeAnnouncerAndDiscoveryTest.java index d2ac80bc919c..69b2340620d9 100644 --- a/server/src/test/java/io/druid/curator/discovery/CuratorDruidNodeAnnouncerAndDiscoveryTest.java +++ b/server/src/test/java/io/druid/curator/discovery/CuratorDruidNodeAnnouncerAndDiscoveryTest.java @@ -19,23 +19,17 @@ package io.druid.curator.discovery; +import com.fasterxml.jackson.databind.InjectableValues; import com.fasterxml.jackson.databind.ObjectMapper; -import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; import com.google.common.collect.ImmutableSet; import com.google.common.util.concurrent.MoreExecutors; -import com.google.inject.Binder; -import com.google.inject.Injector; -import com.google.inject.Key; -import com.google.inject.Module; -import com.google.inject.name.Names; import io.druid.curator.CuratorTestBase; import io.druid.curator.announcement.Announcer; import io.druid.discovery.DiscoveryDruidNode; import io.druid.discovery.DruidNodeDiscovery; import io.druid.discovery.DruidNodeDiscoveryProvider; -import io.druid.guice.GuiceInjectors; -import io.druid.initialization.Initialization; +import io.druid.jackson.DefaultObjectMapper; import io.druid.server.DruidNode; import io.druid.server.initialization.ServerConfig; import io.druid.server.initialization.ZkPathsConfig; @@ -57,25 +51,17 @@ public void setUp() throws Exception setupServerAndCurator(); } - @Test(timeout = 10000) + @Test(timeout = 5000) public void testAnnouncementAndDiscovery() throws Exception { - Injector injector = Initialization.makeInjectorWithModules( - GuiceInjectors.makeStartupInjector(), - ImmutableList.of( - new Module() - { - @Override - public void configure(Binder binder) - { - binder.bind(Key.get(String.class, Names.named("serviceName"))).toInstance("some service"); - binder.bind(Key.get(Integer.class, Names.named("servicePort"))).toInstance(0); - binder.bind(Key.get(Integer.class, Names.named("tlsServicePort"))).toInstance(-1); - } - } - ) + 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 objectMapper = injector.getInstance(ObjectMapper.class); curator.start(); curator.blockUntilConnected(); From 505ba70d7310c0134bcefec7fb714650fe4025f9 Mon Sep 17 00:00:00 2001 From: Himanshu Gupta Date: Wed, 16 Aug 2017 09:53:42 -0500 Subject: [PATCH 7/9] fix KafkaIndexTaskTest --- .../io/druid/indexing/kafka/KafkaIndexTaskTest.java | 13 +++++++++---- 1 file changed, 9 insertions(+), 4 deletions(-) diff --git a/extensions-core/kafka-indexing-service/src/test/java/io/druid/indexing/kafka/KafkaIndexTaskTest.java b/extensions-core/kafka-indexing-service/src/test/java/io/druid/indexing/kafka/KafkaIndexTaskTest.java index 6ae96f209e18..cbcaca7ff81c 100644 --- a/extensions-core/kafka-indexing-service/src/test/java/io/druid/indexing/kafka/KafkaIndexTaskTest.java +++ b/extensions-core/kafka-indexing-service/src/test/java/io/druid/indexing/kafka/KafkaIndexTaskTest.java @@ -53,6 +53,9 @@ import io.druid.data.input.impl.JSONPathSpec; import io.druid.data.input.impl.StringInputRowParser; import io.druid.data.input.impl.TimestampSpec; +import io.druid.discovery.DataNodeService; +import io.druid.discovery.DruidNodeAnnouncer; +import io.druid.discovery.LookupNodeService; import io.druid.indexing.common.SegmentLoaderFactory; import io.druid.indexing.common.TaskLock; import io.druid.indexing.common.TaskStatus; @@ -117,7 +120,9 @@ import io.druid.segment.loading.StorageLocationConfig; import io.druid.segment.realtime.plumber.SegmentHandoffNotifier; import io.druid.segment.realtime.plumber.SegmentHandoffNotifierFactory; +import io.druid.server.DruidNode; import io.druid.server.coordination.DataSegmentServerAnnouncer; +import io.druid.server.coordination.ServerType; import io.druid.timeline.DataSegment; import org.apache.curator.test.TestingCluster; import org.apache.kafka.clients.producer.KafkaProducer; @@ -1638,10 +1643,10 @@ public List getLocations() MapCache.create(1024), new CacheConfig(), testUtils.getTestIndexMergerV9(), - null, - null, - null, - null + EasyMock.createNiceMock(DruidNodeAnnouncer.class), + EasyMock.createNiceMock(DruidNode.class), + new LookupNodeService("tier"), + new DataNodeService("tier", 1, ServerType.INDEXER_EXECUTOR, 0) ); } From 76bfb5e0033b81ea76f9ee1d1a7394b8a0447633 Mon Sep 17 00:00:00 2001 From: Himanshu Gupta Date: Wed, 16 Aug 2017 09:54:12 -0500 Subject: [PATCH 8/9] make lookupTier overridable via RealtimeIndexTask and KafkaIndexTask context --- .../main/java/io/druid/indexing/kafka/KafkaIndexTask.java | 7 ++++++- .../io/druid/indexing/common/task/RealtimeIndexTask.java | 8 +++++++- 2 files changed, 13 insertions(+), 2 deletions(-) diff --git a/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/KafkaIndexTask.java b/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/KafkaIndexTask.java index 869e89e10029..68450cc5066c 100644 --- a/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/KafkaIndexTask.java +++ b/extensions-core/kafka-indexing-service/src/main/java/io/druid/indexing/kafka/KafkaIndexTask.java @@ -44,6 +44,7 @@ import io.druid.data.input.impl.InputRowParser; import io.druid.discovery.DiscoveryDruidNode; import io.druid.discovery.DruidNodeDiscoveryProvider; +import io.druid.discovery.LookupNodeService; import io.druid.indexing.appenderator.ActionBasedSegmentAllocator; import io.druid.indexing.appenderator.ActionBasedUsedSegmentChecker; import io.druid.indexing.common.TaskStatus; @@ -52,6 +53,7 @@ import io.druid.indexing.common.actions.SegmentTransactionalInsertAction; import io.druid.indexing.common.actions.TaskActionClient; import io.druid.indexing.common.task.AbstractTask; +import io.druid.indexing.common.task.RealtimeIndexTask; import io.druid.indexing.common.task.TaskResource; import io.druid.java.util.common.ISE; import io.druid.java.util.common.StringUtils; @@ -291,12 +293,15 @@ public TaskStatus run(final TaskToolbox toolbox) throws Exception ) ); + LookupNodeService lookupNodeService = getContextValue(RealtimeIndexTask.CTX_KEY_LOOKUP_TIER) == null ? + toolbox.getLookupNodeService() : + new LookupNodeService((String) getContextValue(RealtimeIndexTask.CTX_KEY_LOOKUP_TIER)); DiscoveryDruidNode discoveryDruidNode = new DiscoveryDruidNode( toolbox.getDruidNode(), DruidNodeDiscoveryProvider.NODE_TYPE_PEON, ImmutableMap.of( toolbox.getDataNodeService().getName(), toolbox.getDataNodeService(), - toolbox.getLookupNodeService().getName(), toolbox.getLookupNodeService() + lookupNodeService.getName(), lookupNodeService ) ); diff --git a/indexing-service/src/main/java/io/druid/indexing/common/task/RealtimeIndexTask.java b/indexing-service/src/main/java/io/druid/indexing/common/task/RealtimeIndexTask.java index f486efe6b104..7e641da3f84b 100644 --- a/indexing-service/src/main/java/io/druid/indexing/common/task/RealtimeIndexTask.java +++ b/indexing-service/src/main/java/io/druid/indexing/common/task/RealtimeIndexTask.java @@ -33,6 +33,7 @@ import io.druid.data.input.FirehoseFactory; import io.druid.discovery.DiscoveryDruidNode; import io.druid.discovery.DruidNodeDiscoveryProvider; +import io.druid.discovery.LookupNodeService; import io.druid.indexing.common.TaskLock; import io.druid.indexing.common.TaskStatus; import io.druid.indexing.common.TaskToolbox; @@ -80,6 +81,8 @@ public class RealtimeIndexTask extends AbstractTask { + public static final String CTX_KEY_LOOKUP_TIER = "lookupTier"; + private static final EmittingLogger log = new EmittingLogger(RealtimeIndexTask.class); private final static Random random = new Random(); @@ -326,12 +329,15 @@ public String getVersion(final Interval interval) Supplier committerSupplier = null; final File firehoseTempDir = toolbox.getFirehoseTemporaryDir(); + LookupNodeService lookupNodeService = getContextValue(CTX_KEY_LOOKUP_TIER) == null ? + toolbox.getLookupNodeService() : + new LookupNodeService((String) getContextValue(CTX_KEY_LOOKUP_TIER)); DiscoveryDruidNode discoveryDruidNode = new DiscoveryDruidNode( toolbox.getDruidNode(), DruidNodeDiscoveryProvider.NODE_TYPE_PEON, ImmutableMap.of( toolbox.getDataNodeService().getName(), toolbox.getDataNodeService(), - toolbox.getLookupNodeService().getName(), toolbox.getLookupNodeService() + lookupNodeService.getName(), lookupNodeService ) ); From 8c46cccb320dd19786e587e341910148ed15846c Mon Sep 17 00:00:00 2001 From: Himanshu Gupta Date: Wed, 16 Aug 2017 13:00:01 -0500 Subject: [PATCH 9/9] make teamcity build happy --- .../java/io/druid/discovery/DataNodeService.java | 3 +-- .../druid/discovery/DruidNodeDiscoveryProvider.java | 4 ++-- .../CuratorDruidNodeAnnouncerAndDiscoveryTest.java | 12 ++++++------ 3 files changed, 9 insertions(+), 10 deletions(-) diff --git a/server/src/main/java/io/druid/discovery/DataNodeService.java b/server/src/main/java/io/druid/discovery/DataNodeService.java index cd6033d02278..250449b57ec8 100644 --- a/server/src/main/java/io/druid/discovery/DataNodeService.java +++ b/server/src/main/java/io/druid/discovery/DataNodeService.java @@ -109,9 +109,8 @@ public String toString() return "DataNodeService{" + "tier='" + tier + '\'' + ", maxSize=" + maxSize + - ", type='" + type + '\'' + + ", type=" + type + ", priority=" + priority + - ", " + super.toString() + '}'; } } diff --git a/server/src/main/java/io/druid/discovery/DruidNodeDiscoveryProvider.java b/server/src/main/java/io/druid/discovery/DruidNodeDiscoveryProvider.java index b959827ed63f..035840958369 100644 --- a/server/src/main/java/io/druid/discovery/DruidNodeDiscoveryProvider.java +++ b/server/src/main/java/io/druid/discovery/DruidNodeDiscoveryProvider.java @@ -57,7 +57,7 @@ public abstract class DruidNodeDiscoveryProvider NODE_TYPE_MM ); - public static final Map> SERVICE_TO_NODE_TYPES = ImmutableMap.of( + private static final Map> SERVICE_TO_NODE_TYPES = ImmutableMap.of( LookupNodeService.DISCOVERY_SERVICE_KEY, ImmutableSet.of(NODE_TYPE_BROKER, NODE_TYPE_HISTORICAL, NODE_TYPE_PEON), DataNodeService.DISCOVERY_SERVICE_KEY, ImmutableSet.of(NODE_TYPE_HISTORICAL, NODE_TYPE_PEON), WorkerNodeService.DISCOVERY_SERVICE_KEY, ImmutableSet.of(NODE_TYPE_MM) @@ -98,7 +98,7 @@ private static class ServiceListener implements DruidNodeDiscovery, DruidNodeDis private final String service; private final Map nodes = new ConcurrentHashMap<>(); - private final List listeners = new ArrayList(); + private final List listeners = new ArrayList<>(); ServiceListener(String service) { diff --git a/server/src/test/java/io/druid/curator/discovery/CuratorDruidNodeAnnouncerAndDiscoveryTest.java b/server/src/test/java/io/druid/curator/discovery/CuratorDruidNodeAnnouncerAndDiscoveryTest.java index 69b2340620d9..2ddce0843d94 100644 --- a/server/src/test/java/io/druid/curator/discovery/CuratorDruidNodeAnnouncerAndDiscoveryTest.java +++ b/server/src/test/java/io/druid/curator/discovery/CuratorDruidNodeAnnouncerAndDiscoveryTest.java @@ -159,11 +159,11 @@ public void nodeRemoved(DiscoveryDruidNode node) } ); - while (!ImmutableSet.of(node1).equals(coordNodes)) { + while (!checkNodes(ImmutableSet.of(node1), coordNodes)) { Thread.sleep(100); } - while (!ImmutableSet.of(node3).equals(overlordNodes)) { + while (!checkNodes(ImmutableSet.of(node3), overlordNodes)) { Thread.sleep(100); } @@ -178,11 +178,11 @@ public void nodeRemoved(DiscoveryDruidNode node) Thread.sleep(100); } - while (!ImmutableSet.of(node1, node2).equals(coordNodes)) { + while (!checkNodes(ImmutableSet.of(node1, node2), coordNodes)) { Thread.sleep(100); } - while (!ImmutableSet.of(node3, node4).equals(overlordNodes)) { + while (!checkNodes(ImmutableSet.of(node3, node4), overlordNodes)) { Thread.sleep(100); } @@ -199,11 +199,11 @@ public void nodeRemoved(DiscoveryDruidNode node) Thread.sleep(100); } - while (!ImmutableSet.of().equals(coordNodes)) { + while (!coordNodes.isEmpty()) { Thread.sleep(100); } - while (!ImmutableSet.of().equals(overlordNodes)) { + while (!overlordNodes.isEmpty()) { Thread.sleep(100); }