You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@druid.apache.org by GitBox <gi...@apache.org> on 2021/12/01 08:22:38 UTC

[GitHub] [druid] jihoonson opened a new pull request #12012: Make NodeRole available during binding; add support for dynamic registration of DruidService

jihoonson opened a new pull request #12012:
URL: https://github.com/apache/druid/pull/12012


   ### Description
   
   This PR has 2 changes as noted in the title.
   
   #### Making `NodeRole` available during binding
   
   Sometimes we want to load some modules or even bind some objects on only certain node types. For example, deep storage extension modules don't have to be loaded in the coordinator, it isn't harm to do though. We are using `NodeRole`s to represent node types which are injected through our regular injection process. As a result, nodeRoles are only available after binding is done but is not available during binding. This PR fills in this gap by adding a new phase between the startup phase and the binding phase that binds nodeRoles first before others and provide them for binding rest of modules. There are 2 ways to use the nodeRole information.
   
   1) Using `LoadOn` annotation. This example shows how to load a module only in brokers and historicals using this annotation.
   
   ```java
   @LoadOn({"broker", "historical"})
   public class MyModule implements Module
   ...
   ```
   
   2) Injecting nodeRoles in the module. This example shows how to bind objects only in brokers.
   
   ```java
   public class MyModule implements Module
   {
       private Set<NodeRole> nodeRoles;
   
       @Inject
       public void init(@Self Set<NodeRole> nodeRoles)
       {
         this.nodeRoles = nodeRoles;
       }
   
       @Override
       public void configure(Binder binder)
       {
         if (nodeRoles.contains(NodeRole.BROKER) {
           // do something
         }
       }
   }
   ```
   
   #### Dynamic registration of `DruidServices`
   
   `DruidService` represents the service type served by each node and is announced along with node information together. Currently, each node has a fixed set of `DruidServices` which are hard-coded in each `Cli*` class. This PR makes it extensible, so that extensions can register their own custom `DruidServices` when they are loaded. Here is an example for how to do it.
   
   ```java
   public class MyDruidServiceModule implements DruidModule
   {
     @Override
     public void configure(Binder binder)
     {
       binder.install(MultibindingsScanner.asModule());
     }
   
     @ProvidesIntoSet
     @Named("broker")
     public Class<? extends DruidService> getMyService()
     {
       return MyService.class;
     }
   }
   ```
   
   In this example, `MyService` will be announced as one of the broker services if the extension is loaded.
   
   Integration tests are not added in this PR. I could add some as a follow-up.
   
   <hr>
   
   ##### Key changed/added classes in this PR
    * `GuiceRunnable`
    * `ServerRunnable`
    * `Initialization`
   
   <hr>
   
   This PR has:
   - [x] been self-reviewed.
      - [ ] using the [concurrency checklist](https://github.com/apache/druid/blob/master/dev/code-review/concurrency.md) (Remove this item if the PR doesn't have any relation to concurrency.)
   - [ ] added documentation for new or modified features or behaviors.
   - [x] added Javadocs for most classes and all non-trivial methods. Linked related entities via Javadoc links.
   - [ ] added or updated version, license, or notice information in [licenses.yaml](https://github.com/apache/druid/blob/master/dev/license.md)
   - [x] added comments explaining the "why" and the intent of the code wherever would not be obvious for an unfamiliar reader.
   - [x] added unit tests or modified existing tests to cover new code paths, ensuring the threshold for [code coverage](https://github.com/apache/druid/blob/master/dev/code-review/code-coverage.md) is met.
   - [ ] added integration tests.
   - [x] been tested in a test Druid cluster.
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] jihoonson commented on pull request #12012: Make NodeRole available during binding; add support for dynamic registration of DruidService

Posted by GitBox <gi...@apache.org>.
jihoonson commented on pull request #12012:
URL: https://github.com/apache/druid/pull/12012#issuecomment-985790386


   @paul-rogers @clintropolis thank you for the review!


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] jihoonson merged pull request #12012: Make NodeRole available during binding; add support for dynamic registration of DruidService

Posted by GitBox <gi...@apache.org>.
jihoonson merged pull request #12012:
URL: https://github.com/apache/druid/pull/12012


   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] jihoonson commented on a change in pull request #12012: Make NodeRole available during binding; add support for dynamic registration of DruidService

Posted by GitBox <gi...@apache.org>.
jihoonson commented on a change in pull request #12012:
URL: https://github.com/apache/druid/pull/12012#discussion_r761551705



##########
File path: extensions-contrib/materialized-view-maintenance/src/main/java/org/apache/druid/indexing/materializedview/MaterializedViewMaintenanceDruidModule.java
##########
@@ -25,10 +25,12 @@
 import com.google.common.collect.ImmutableList;
 import com.google.inject.Binder;
 import org.apache.druid.guice.JsonConfigProvider;
+import org.apache.druid.guice.annotations.LoadOn;
 import org.apache.druid.initialization.DruidModule;
 
 import java.util.List;
 
+@LoadOn(roles = "overlord")

Review comment:
       I renamed the annotation to `LoadScope`. Let me know if that makes more sense. For people who have fat-fingers like myself, I added constants in `NodeRole` that can be used here. 




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] paul-rogers commented on a change in pull request #12012: Make NodeRole available during binding; add support for dynamic registration of DruidService

Posted by GitBox <gi...@apache.org>.
paul-rogers commented on a change in pull request #12012:
URL: https://github.com/apache/druid/pull/12012#discussion_r760651944



##########
File path: server/src/main/java/org/apache/druid/initialization/Initialization.java
##########
@@ -486,6 +507,19 @@ public void addModule(Object input)
       }
     }
 
+    private boolean shouldLoadOnCurrentNodeType(Object object)
+    {
+      LoadOn loadOn = object.getClass().getAnnotation(LoadOn.class);
+      if (loadOn != null) {
+        Set<NodeRole> rolesPredicate = Arrays.stream(loadOn.roles())
+                                             .map(NodeRole::fromJsonName)
+                                             .collect(Collectors.toSet());
+        return rolesPredicate.stream().anyMatch(nodeRoles::contains);
+      }
+      // always load if annotation is not specified
+      return true;

Review comment:
       Nit: reverse "polarity":
   
   ```java
       if (loadOn == null) {
         return true;
       }
       ...
   ```

##########
File path: extensions-contrib/materialized-view-maintenance/src/main/java/org/apache/druid/indexing/materializedview/MaterializedViewMaintenanceDruidModule.java
##########
@@ -25,10 +25,12 @@
 import com.google.common.collect.ImmutableList;
 import com.google.inject.Binder;
 import org.apache.druid.guice.JsonConfigProvider;
+import org.apache.druid.guice.annotations.LoadOn;
 import org.apache.druid.initialization.DruidModule;
 
 import java.util.List;
 
+@LoadOn(roles = "overlord")

Review comment:
       Two nits. First, can we make the name a bit clearer? Maybe `DruidRole` or `RoleScope` or some such that gives a bit more of a hit of the purpose?
   
   Second, can we provide constants for the allowed loads? Will allow the compiler to catch fat-finger errors rather than trying to debug them at runtime: `"borker"` instead of `"broker"`, say.

##########
File path: services/src/main/java/org/apache/druid/cli/ServerRunnable.java
##########
@@ -59,7 +62,7 @@ public ServerRunnable(Logger log)
   @Override
   public void run()
   {
-    final Injector injector = makeInjector();
+    final Injector injector = makeInjector(getNodeRoles(getProperties()));

Review comment:
       Again, no need to pass the node roles into a method on this same class.

##########
File path: services/src/main/java/org/apache/druid/cli/CliRouter.java
##########
@@ -75,6 +80,12 @@ public CliRouter()
     super(log);
   }
 
+  @Override
+  protected Set<NodeRole> getNodeRoles(Properties properties)
+  {
+    return ImmutableSet.of(NodeRole.ROUTER);
+  }

Review comment:
       I wonder about the timing of this call. (See comments below.) The `configure()` method now takes `Properties`: that would seem to be a fine time to define the roles and store them in a member variable. A base class `getNodeRoles()` method (without arguments) could return those roles.
   
   The key question is timing of when the `configure()` happens relative to this method.

##########
File path: server/src/main/java/org/apache/druid/discovery/NoopService.java
##########
@@ -0,0 +1,43 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.druid.discovery;
+
+/**
+ * No-op DruidService. Used as a default DruidService for binding services.
+ * See {@code ServerRunnable#bindDruidServiceType}.
+ *
+ * This is not discoverable and thus is not JsonSubType of DruidService.

Review comment:
       I wonder, why do we need a dummy service? If services are things that can be discovered, then why do we need a dummy one that can't be discovered? Does the `DruidService` actually perform two tasks, only one of which is discovery? This no-op performs the other task (whatever it is?)

##########
File path: server/src/main/java/org/apache/druid/guice/annotations/LoadOn.java
##########
@@ -0,0 +1,38 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.druid.guice.annotations;
+
+import java.lang.annotation.ElementType;
+import java.lang.annotation.Retention;
+import java.lang.annotation.RetentionPolicy;
+import java.lang.annotation.Target;
+
+/**
+ * An annotation to specify node types that each {@link com.google.inject.Module} can be loaded on.
+ * In other words, you can specify particular node types for each module using this annotation,
+ * so that those modeuls can be loaded on only those particular node types.

Review comment:
       modeuls -> modules

##########
File path: services/src/main/java/org/apache/druid/cli/GuiceRunnable.java
##########
@@ -56,23 +63,57 @@ public GuiceRunnable(Logger log)
   public abstract void run();
 
   @Inject
-  public void configure(Injector injector)
+  public void configure(Properties properties, Injector injector)

Review comment:
       `Properties` are passed in here and stored in a member variable. Yet, in `getNodeRoles()`, we pass in the `Properties` again. Are these the same properties? Is the timing such that this method has not been called when the node roles method is called?

##########
File path: services/src/main/java/org/apache/druid/cli/GuiceRunnable.java
##########
@@ -56,23 +63,57 @@ public GuiceRunnable(Logger log)
   public abstract void run();
 
   @Inject
-  public void configure(Injector injector)
+  public void configure(Properties properties, Injector injector)
   {
+    this.properties = properties;
     this.baseInjector = injector;
   }
 
+  protected Properties getProperties()
+  {
+    return properties;
+  }
+
   protected abstract List<? extends Module> getModules();
 
   public Injector makeInjector()
   {
+    // Pass an empty set of nodeRoles for non-ServerRunnables.
+    // They will still load all modules except for the ones annotated with `LoadOn`.
+    return makeInjector(ImmutableSet.of());
+  }
+
+  public Injector makeInjector(Set<NodeRole> nodeRoles)
+  {
+    Module registerNodeRoleModule = registerNodeRoleModule(nodeRoles);
     try {
-      return Initialization.makeInjectorWithModules(baseInjector, getModules());
+      return Initialization.makeInjectorWithModules(
+          nodeRoles,
+          baseInjector.createChildInjector(registerNodeRoleModule),
+          Iterables.concat(
+              // bind nodeRoles for the new injector as well
+              ImmutableList.of(registerNodeRoleModule),
+              getModules()
+          )
+      );
     }
     catch (Exception e) {
       throw new RuntimeException(e);
     }
   }
 
+  static Module registerNodeRoleModule(Set<NodeRole> nodeRoles)

Review comment:
       Again, this is a method on the class that defines the node roles: no need to pass them in.

##########
File path: services/src/main/java/org/apache/druid/cli/GuiceRunnable.java
##########
@@ -56,23 +63,57 @@ public GuiceRunnable(Logger log)
   public abstract void run();
 
   @Inject
-  public void configure(Injector injector)
+  public void configure(Properties properties, Injector injector)
   {
+    this.properties = properties;
     this.baseInjector = injector;
   }
 
+  protected Properties getProperties()
+  {
+    return properties;
+  }
+
   protected abstract List<? extends Module> getModules();
 
   public Injector makeInjector()
   {
+    // Pass an empty set of nodeRoles for non-ServerRunnables.
+    // They will still load all modules except for the ones annotated with `LoadOn`.
+    return makeInjector(ImmutableSet.of());
+  }
+
+  public Injector makeInjector(Set<NodeRole> nodeRoles)

Review comment:
       We pass the node roles into this method. And, yet, this class certainly knows about its own roles: it is the one that defined them.

##########
File path: server/src/main/java/org/apache/druid/initialization/Initialization.java
##########
@@ -486,6 +507,19 @@ public void addModule(Object input)
       }
     }
 
+    private boolean shouldLoadOnCurrentNodeType(Object object)
+    {
+      LoadOn loadOn = object.getClass().getAnnotation(LoadOn.class);
+      if (loadOn != null) {
+        Set<NodeRole> rolesPredicate = Arrays.stream(loadOn.roles())
+                                             .map(NodeRole::fromJsonName)
+                                             .collect(Collectors.toSet());
+        return rolesPredicate.stream().anyMatch(nodeRoles::contains);

Review comment:
       Can we just use the same enum for both cases and just use the `Set` methods directly?

##########
File path: services/src/main/java/org/apache/druid/guice/AbstractDruidServiceModule.java
##########
@@ -0,0 +1,54 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.druid.guice;
+
+import com.google.inject.Binder;
+import com.google.inject.Module;
+import com.google.inject.multibindings.MultibindingsScanner;
+
+/**
+ * An abstract module for dynamic registration of {@link org.apache.druid.discovery.DruidService}.
+ * DruidServices are bound to a set which is mapped to a certain {@link org.apache.druid.discovery.NodeRole}.
+ * See {@link org.apache.druid.cli.ServerRunnable#bindDruidServiceType} for how the map is bound.
+ *
+ * To register a DruidService, create a class something like below:
+ *
+ * <pre>
+ *   public class MyModule extends AbstractDruidServiceModule
+ *   {
+ *     @ProvidesIntoSet
+ *     @Named("myNodeTypeKey")
+ *     public Class<? extends DruidService> getDataNodeService()
+ *     {
+ *       return DataNodeService.class;
+ *     }
+ *   }
+ * </pre>
+ *
+ * and add it in {@link org.apache.druid.cli.ServerRunnable#getModules}.
+ */

Review comment:
       What is the relationship between `myNodeTypeKey` and the node roles? Does "nodeType" here mean "node role" or something else?

##########
File path: services/src/main/java/org/apache/druid/guice/CoordinatorServiceModule.java
##########
@@ -0,0 +1,37 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.druid.guice;
+
+import com.google.inject.multibindings.ProvidesIntoSet;
+import com.google.inject.name.Named;
+import org.apache.druid.discovery.DruidService;
+import org.apache.druid.discovery.NoopService;
+
+public class CoordinatorServiceModule extends AbstractDruidServiceModule
+{
+  public static final String COORDINATOR_SERVICE_KEY = "coordinatorService";
+
+  @ProvidesIntoSet
+  @Named(COORDINATOR_SERVICE_KEY)
+  public Class<? extends DruidService> getDefaultNodeService()
+  {
+    return NoopService.class;
+  }

Review comment:
       I wonder, why do we need to provide a no-op for the node service, but not, say, the lookup service? Would an extension need to provide its on no-op, or is this one enough?
   
   Is the need for the no-op somehow unique to this service? The others can handle an empty set, but this one can't?




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] jihoonson commented on a change in pull request #12012: Make NodeRole available during binding; add support for dynamic registration of DruidService

Posted by GitBox <gi...@apache.org>.
jihoonson commented on a change in pull request #12012:
URL: https://github.com/apache/druid/pull/12012#discussion_r761551860



##########
File path: services/src/main/java/org/apache/druid/cli/CliRouter.java
##########
@@ -75,6 +80,12 @@ public CliRouter()
     super(log);
   }
 
+  @Override
+  protected Set<NodeRole> getNodeRoles(Properties properties)
+  {
+    return ImmutableSet.of(NodeRole.ROUTER);
+  }

Review comment:
       I suppose you mean `GuiceRunnable.configure()`? That method is called at https://github.com/apache/druid/blob/master/services/src/main/java/org/apache/druid/cli/Main.java#L111 which is always called before `GuiceRunnable.run()`.

##########
File path: services/src/main/java/org/apache/druid/guice/AbstractDruidServiceModule.java
##########
@@ -0,0 +1,54 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.druid.guice;
+
+import com.google.inject.Binder;
+import com.google.inject.Module;
+import com.google.inject.multibindings.MultibindingsScanner;
+
+/**
+ * An abstract module for dynamic registration of {@link org.apache.druid.discovery.DruidService}.
+ * DruidServices are bound to a set which is mapped to a certain {@link org.apache.druid.discovery.NodeRole}.
+ * See {@link org.apache.druid.cli.ServerRunnable#bindDruidServiceType} for how the map is bound.
+ *
+ * To register a DruidService, create a class something like below:
+ *
+ * <pre>
+ *   public class MyModule extends AbstractDruidServiceModule
+ *   {
+ *     @ProvidesIntoSet
+ *     @Named("myNodeTypeKey")
+ *     public Class<? extends DruidService> getDataNodeService()
+ *     {
+ *       return DataNodeService.class;
+ *     }
+ *   }
+ * </pre>
+ *
+ * and add it in {@link org.apache.druid.cli.ServerRunnable#getModules}.
+ */

Review comment:
       To make it more obvious, I added `NodeRole.getDruidServiceInjectName()` as below:
   
   ```java
     public Named getDruidServiceInjectName()
     {
       return Names.named(jsonName);
     }
   ```
   
   Custom modules now can use the JSON name of node roles as the key.
   ```
   public class BrokerServiceModule extends AbstractDruidServiceModule
   {
     @ProvidesIntoSet
     @Named(NodeRole.BROKER_JSON_NAME)
     public Class<? extends DruidService> getDataNodeService()
     {
       return DataNodeService.class;
     }
   }
   ```




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] jihoonson commented on a change in pull request #12012: Make NodeRole available during binding; add support for dynamic registration of DruidService

Posted by GitBox <gi...@apache.org>.
jihoonson commented on a change in pull request #12012:
URL: https://github.com/apache/druid/pull/12012#discussion_r761552250



##########
File path: services/src/main/java/org/apache/druid/guice/CoordinatorServiceModule.java
##########
@@ -0,0 +1,37 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.druid.guice;
+
+import com.google.inject.multibindings.ProvidesIntoSet;
+import com.google.inject.name.Named;
+import org.apache.druid.discovery.DruidService;
+import org.apache.druid.discovery.NoopService;
+
+public class CoordinatorServiceModule extends AbstractDruidServiceModule
+{
+  public static final String COORDINATOR_SERVICE_KEY = "coordinatorService";
+
+  @ProvidesIntoSet
+  @Named(COORDINATOR_SERVICE_KEY)
+  public Class<? extends DruidService> getDefaultNodeService()
+  {
+    return NoopService.class;
+  }

Review comment:
       I was not sure how to bind an empty set in a map binder. So, this default dummy service was for always providing something so that the set is not empty. But it turns it's easy to bind an empty set in a map binder without this dummy service. I have removed it now. 




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] jihoonson commented on a change in pull request #12012: Make NodeRole available during binding; add support for dynamic registration of DruidService

Posted by GitBox <gi...@apache.org>.
jihoonson commented on a change in pull request #12012:
URL: https://github.com/apache/druid/pull/12012#discussion_r761551812



##########
File path: services/src/main/java/org/apache/druid/cli/GuiceRunnable.java
##########
@@ -56,23 +63,57 @@ public GuiceRunnable(Logger log)
   public abstract void run();
 
   @Inject
-  public void configure(Injector injector)
+  public void configure(Properties properties, Injector injector)

Review comment:
       `getNodeRoles` is defined in `ServerRunnable` which is a child class of `GuiceRunnable`. I think it makes more sense to have it there as `ServerRunnable` specifically represents servers while `GuiceRunnable` can be any runnable that uses Guice injection. 




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org