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/08/17 08:12:14 UTC

[GitHub] [druid] abhishekagarwal87 commented on a change in pull request #11601: Support custom coordinator duties

abhishekagarwal87 commented on a change in pull request #11601:
URL: https://github.com/apache/druid/pull/11601#discussion_r690122390



##########
File path: docs/development/modules.md
##########
@@ -324,6 +324,28 @@ public class MyTransformModule implements DruidModule {
 }
 ```
 
+### Adding your own custom pluggable Coordinator Duty
+
+The coordinator periodically runs jobs, so-called `CoordinatorDuty` which include loading new segments, segment balancing, etc. 
+Druid users can add custom pluggable coordinator duties, which are not part of Core Druid, without modifying any Core Druid classes.
+Users can do this by writing their own custom coordinator duty implementing the interface `CoordinatorCustomDuty` and setting the `JsonTypeName`.
+Next, users will need to register their custom coordinator as subtypes in their Module's `DruidModule#getJacksonModules()`.
+Once these steps are done, user will be able to load their custom coordinator duty using the following properties:
+```
+druid.coordinator.dutyGroups=[<GROUP_NAME_1>, <GROUP_NAME_2>, ...]
+druid.coordinator.<GROUP_NAME_1>.duties=[<DUTY_NAME_MATCHING_JSON_TYPE_NAME_1>, <DUTY_NAME_MATCHING_JSON_TYPE_NAME_2>, ...]
+druid.coordinator.<GROUP_NAME_1>.duty.<DUTY_NAME_MATCHING_JSON_TYPE_NAME_1>.<SOME_CONFIG_1_KEY>=<SOME_CONFIG_1_VALUE>
+druid.coordinator.<GROUP_NAME_1>.duty.<DUTY_NAME_MATCHING_JSON_TYPE_NAME_1>.<SOME_CONFIG_2_KEY>=<SOME_CONFIG_2_VALUE>
+druid.coordinator.<GROUP_NAME_1>.period=<GROUP_NAME_1_RUN_PERIOD>

Review comment:
       ```suggestion
   druid.coordinator.dutyGroups=[<GROUP_NAME_1>, <GROUP_NAME_2>, ...]
   druid.coordinator.<GROUP_NAME_1>.duties=[<DUTY_NAME_MATCHING_JSON_TYPE_NAME_1>, <DUTY_NAME_MATCHING_JSON_TYPE_NAME_2>, ...]
   druid.coordinator.<GROUP_NAME_1>.period=<GROUP_NAME_1_RUN_PERIOD>
   
   druid.coordinator.<GROUP_NAME_1>.duty.<DUTY_NAME_MATCHING_JSON_TYPE_NAME_1>.<SOME_CONFIG_1_KEY>=<SOME_CONFIG_1_VALUE>
   druid.coordinator.<GROUP_NAME_1>.duty.<DUTY_NAME_MATCHING_JSON_TYPE_NAME_1>.<SOME_CONFIG_2_KEY>=<SOME_CONFIG_2_VALUE>
   ```

##########
File path: docs/development/modules.md
##########
@@ -324,6 +324,28 @@ public class MyTransformModule implements DruidModule {
 }
 ```
 
+### Adding your own custom pluggable Coordinator Duty
+
+The coordinator periodically runs jobs, so-called `CoordinatorDuty` which include loading new segments, segment balancing, etc. 
+Druid users can add custom pluggable coordinator duties, which are not part of Core Druid, without modifying any Core Druid classes.
+Users can do this by writing their own custom coordinator duty implementing the interface `CoordinatorCustomDuty` and setting the `JsonTypeName`.
+Next, users will need to register their custom coordinator as subtypes in their Module's `DruidModule#getJacksonModules()`.
+Once these steps are done, user will be able to load their custom coordinator duty using the following properties:
+```
+druid.coordinator.dutyGroups=[<GROUP_NAME_1>, <GROUP_NAME_2>, ...]
+druid.coordinator.<GROUP_NAME_1>.duties=[<DUTY_NAME_MATCHING_JSON_TYPE_NAME_1>, <DUTY_NAME_MATCHING_JSON_TYPE_NAME_2>, ...]
+druid.coordinator.<GROUP_NAME_1>.duty.<DUTY_NAME_MATCHING_JSON_TYPE_NAME_1>.<SOME_CONFIG_1_KEY>=<SOME_CONFIG_1_VALUE>
+druid.coordinator.<GROUP_NAME_1>.duty.<DUTY_NAME_MATCHING_JSON_TYPE_NAME_1>.<SOME_CONFIG_2_KEY>=<SOME_CONFIG_2_VALUE>
+druid.coordinator.<GROUP_NAME_1>.period=<GROUP_NAME_1_RUN_PERIOD>
+```
+In the new system for pluggable Coordinator duties, similar to what coordinator already does today, the duties can be grouped together.
+The duties will be grouped into multiple groups as per the elements in list `druid.coordinator.dutyGroups`. 
+All duties in the same group will have the same run period configured by `druid.coordinator.<GROUP_NAME>.period`.
+Currently, there is a single thread running the duties sequentially for each group. 
+
+For example, see `KillSupervisorsCustomDuty` for a custom coordinator duty implementation and `common-custom-coordinator-duties`

Review comment:
       can you add an example config here? 

##########
File path: server/src/main/java/org/apache/druid/server/coordinator/DruidCoordinator.java
##########
@@ -679,6 +686,20 @@ private void becomeLeader()
           )
       );
 
+      for (CoordinatorCustomDutyGroup customDutyGroup : customDutyGroups.getCoordinatorCustomDutyGroups()) {
+        dutiesRunnables.add(
+            Pair.of(
+                new DutiesRunnable(customDutyGroup.getCustomDutyList(), startingLeaderCounter, customDutyGroup.getName()),
+                customDutyGroup.getPeriod()
+            )
+        );
+        log.debug(

Review comment:
       this could be an info IMO. 

##########
File path: server/src/main/java/org/apache/druid/server/coordinator/duty/CoordinatorCustomDuty.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.server.coordinator.duty;
+
+import com.fasterxml.jackson.annotation.JsonSubTypes;
+import com.fasterxml.jackson.annotation.JsonTypeInfo;
+import com.fasterxml.jackson.annotation.JsonTypeName;
+import org.apache.druid.guice.annotations.ExtensionPoint;
+import org.apache.druid.initialization.DruidModule;
+
+/**
+ * This {@link ExtensionPoint} allows for coordinator duty to be pluggable
+ * so that users can register their own duties without modifying Core Druid classes.
+ *
+ * Users can write their own custom coordinator duty implemnting this interface and setting the {@link JsonTypeName}.
+ * Next, users will need to register their custom coordinator as subtypes in their
+ * Module's {@link DruidModule#getJacksonModules()}. Once these steps are done, user will be able to load their
+ * custom coordinator duty using the following properties:
+ * druid.coordinator.dutyGroups=[<GROUP_NAME>]
+ * druid.coordinator.<GROUP_NAME>.duties=[<DUTY_NAME_MATCHING_JSON_TYPE_NAME>]
+ * druid.coordinator.<GROUP_NAME>.duty.<DUTY_NAME_MATCHING_JSON_TYPE_NAME>.<SOME_CONFIG_1>=100
+ * druid.coordinator.<GROUP_NAME>.duty.<DUTY_NAME_MATCHING_JSON_TYPE_NAME>.<SOME_CONFIG_2>=200
+ * druid.coordinator.<GROUP_NAME>.period="P1D"
+ *
+ * The duties can be grouped into multiple groups as per the elements in list druid.coordinator.dutyGroups.
+ * All duties in the same group will have the same run period configured by druid.coordinator.<GROUP_NAME>.period.
+ * There will be a single thread running the duties sequentially for each group.
+ */
+@JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "type")
+@JsonSubTypes({
+    @JsonSubTypes.Type(name = "killSupervisors", value = KillSupervisorsCustomDuty.class),
+})
+@ExtensionPoint
+public interface CoordinatorCustomDuty extends CoordinatorDuty

Review comment:
       what is the reason for not implementing `CoordinatorDuty` directly in extensions? is it because existing implementations don't have  associated JSON type? If so, can you document that reasoning. 
   
   should you also add a method for getting the type of the duty? 

##########
File path: server/src/main/java/org/apache/druid/server/coordinator/duty/CoordinatorCustomDutyGroup.java
##########
@@ -0,0 +1,91 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.druid.server.coordinator.duty;
+
+import org.joda.time.Duration;
+
+import java.util.List;
+import java.util.Objects;
+
+public class CoordinatorCustomDutyGroup
+{
+  private final String name;
+
+  private final Duration period;
+
+  private final List<CoordinatorCustomDuty> customDutyList;
+
+  public CoordinatorCustomDutyGroup(
+      String name,
+      Duration period,
+      List<CoordinatorCustomDuty> customDutyList
+  )
+  {
+    this.name = name;
+    this.period = period;
+    this.customDutyList = customDutyList;
+  }
+
+  public String getName()
+  {
+    return name;
+  }
+
+  public Duration getPeriod()
+  {
+    return period;
+  }
+
+  public List<CoordinatorCustomDuty> getCustomDutyList()
+  {
+    return customDutyList;
+  }
+
+  @Override
+  public boolean equals(Object o)

Review comment:
       It is not guaranteed that any implementation of `CoordinatorCustomDuty` will implement equalsAndHashCode correctly. Since we can't have two objects of the same custom duty type, you can use the type of custom duty in your equalsAndHashCode implementation? 

##########
File path: server/src/main/java/org/apache/druid/server/coordinator/duty/CoordinatorCustomDutyGroups.java
##########
@@ -0,0 +1,67 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.druid.server.coordinator.duty;
+
+import java.util.Objects;
+import java.util.Set;
+
+public class CoordinatorCustomDutyGroups
+{
+  private final Set<CoordinatorCustomDutyGroup> coordinatorCustomDutyGroups;
+
+  public CoordinatorCustomDutyGroups(
+      Set<CoordinatorCustomDutyGroup> coordinatorCustomDutyGroups
+  )
+  {
+    this.coordinatorCustomDutyGroups = coordinatorCustomDutyGroups;
+  }
+
+  public Set<CoordinatorCustomDutyGroup> getCoordinatorCustomDutyGroups()
+  {
+    return coordinatorCustomDutyGroups;
+  }
+
+  @Override
+  public String toString()
+  {
+    return "CoordinatorCustomDutyGroups{" +
+           "coordinatorCustomDutyGroups=" + coordinatorCustomDutyGroups +
+           '}';
+  }
+
+  @Override
+  public boolean equals(Object o)
+  {
+    if (this == o) {
+      return true;
+    }
+    if (o == null || getClass() != o.getClass()) {
+      return false;
+    }
+    CoordinatorCustomDutyGroups that = (CoordinatorCustomDutyGroups) o;
+    return Objects.equals(coordinatorCustomDutyGroups, that.coordinatorCustomDutyGroups);
+  }
+
+  @Override
+  public int hashCode()

Review comment:
       is this method required in this class? 

##########
File path: services/src/main/java/org/apache/druid/cli/CliCoordinator.java
##########
@@ -353,4 +370,76 @@ public static boolean isOverlord(Properties properties)
   {
     return Boolean.parseBoolean(properties.getProperty(AS_OVERLORD_PROPERTY));
   }
+
+  private static class CoordinatorCustomDutyGroupsProvider implements Provider<CoordinatorCustomDutyGroups>
+  {
+    private Properties props;
+    private JsonConfigurator configurator;
+    private ObjectMapper jsonMapper;
+
+    @Inject
+    public void inject(Properties props, JsonConfigurator configurator, ObjectMapper jsonMapper)
+    {
+      this.props = props;
+      this.configurator = configurator;
+      this.jsonMapper = jsonMapper;
+    }
+
+    @Override
+    public CoordinatorCustomDutyGroups get()

Review comment:
       is there a test for this deserialization? 

##########
File path: services/src/main/java/org/apache/druid/cli/CliCoordinator.java
##########
@@ -353,4 +370,76 @@ public static boolean isOverlord(Properties properties)
   {
     return Boolean.parseBoolean(properties.getProperty(AS_OVERLORD_PROPERTY));
   }
+
+  private static class CoordinatorCustomDutyGroupsProvider implements Provider<CoordinatorCustomDutyGroups>
+  {
+    private Properties props;
+    private JsonConfigurator configurator;
+    private ObjectMapper jsonMapper;
+
+    @Inject
+    public void inject(Properties props, JsonConfigurator configurator, ObjectMapper jsonMapper)
+    {
+      this.props = props;
+      this.configurator = configurator;
+      this.jsonMapper = jsonMapper;
+    }
+
+    @Override
+    public CoordinatorCustomDutyGroups get()
+    {
+      try {
+        Set<CoordinatorCustomDutyGroup> coordinatorCustomDutyGroups = new HashSet<>();
+        if (Strings.isNullOrEmpty(props.getProperty("druid.coordinator.dutyGroups"))) {
+          return new CoordinatorCustomDutyGroups(coordinatorCustomDutyGroups);
+        }
+        List<String> coordinatorCustomDutyGroupNames = jsonMapper.readValue(props.getProperty(
+            "druid.coordinator.dutyGroups"), new TypeReference<List<String>>() {});
+        for (String coordinatorCustomDutyGroupName : coordinatorCustomDutyGroupNames) {
+          String dutyListProperty = StringUtils.format("druid.coordinator.%s.duties", coordinatorCustomDutyGroupName);
+          if (Strings.isNullOrEmpty(props.getProperty(dutyListProperty))) {
+            throw new IAE("Coordinator custom duty group given without any duty for group %s", coordinatorCustomDutyGroupName);
+          }
+          List<String> dutyForGroup = jsonMapper.readValue(props.getProperty(dutyListProperty), new TypeReference<List<String>>() {});
+          List<CoordinatorCustomDuty> coordinatorCustomDuties = new ArrayList<>();
+          for (String dutyName : dutyForGroup) {
+            final String dutyPropertyBase = StringUtils.format(
+                "druid.coordinator.%s.duty.%s",
+                coordinatorCustomDutyGroupName,
+                dutyName
+            );
+            final JsonConfigProvider<CoordinatorCustomDuty> coordinatorCustomDutyProvider = JsonConfigProvider.of(
+                dutyPropertyBase,
+                CoordinatorCustomDuty.class
+            );
+
+            String typeProperty = StringUtils.format("%s.type", dutyPropertyBase);
+            Properties adjustedProps = new Properties(props);
+            if (adjustedProps.containsKey(typeProperty)) {
+              throw new IAE("'type' property [%s] is reserved.", typeProperty);
+            } else {
+              adjustedProps.put(typeProperty, dutyName);
+            }
+            coordinatorCustomDutyProvider.inject(adjustedProps, configurator);
+            Supplier<CoordinatorCustomDuty> coordinatorCustomDutySupplier = coordinatorCustomDutyProvider.get();
+            if (coordinatorCustomDutySupplier == null) {
+              throw new ISE("Could not create CoordinatorCustomDuty with name: %s for group: %s", dutyName, coordinatorCustomDutyGroupName);
+            }
+            CoordinatorCustomDuty coordinatorCustomDuty = coordinatorCustomDutySupplier.get();

Review comment:
       can this be null as well? 




-- 
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