kfaraz commented on code in PR #18745:
URL: https://github.com/apache/druid/pull/18745#discussion_r2610496869


##########
indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisor.java:
##########
@@ -556,43 +566,51 @@ public String getType()
   private boolean changeTaskCount(int desiredActiveTaskCount)
       throws InterruptedException, ExecutionException
   {
+    if (autoScalerConfig == null) {
+      log.warn("autoScalerConfig is 'null' but dynamic allocation notice is 
submitted, how can it be ?");
+      return false;
+    }
     int currentActiveTaskCount;
     Collection<TaskGroup> activeTaskGroups = 
activelyReadingTaskGroups.values();
     currentActiveTaskCount = activeTaskGroups.size();
 
     if (desiredActiveTaskCount < 0 || desiredActiveTaskCount == 
currentActiveTaskCount) {
       return false;
-    } else {
-      log.info(
-          "Starting scale action, current active task count is [%d] and 
desired task count is [%d] for supervisor[%s] for dataSource[%s].",
-          currentActiveTaskCount,
-          desiredActiveTaskCount,
-          supervisorId,
-          dataSource
-      );
-      final Stopwatch scaleActionStopwatch = Stopwatch.createStarted();
-      gracefulShutdownInternal();
-      changeTaskCountInIOConfig(desiredActiveTaskCount);
-      clearAllocationInfo();
-      emitter.emit(ServiceMetricEvent.builder()
-                                     .setDimension(DruidMetrics.SUPERVISOR_ID, 
supervisorId)
-                                     .setDimension(DruidMetrics.DATASOURCE, 
dataSource)
-                                     .setDimension(DruidMetrics.STREAM, 
getIoConfig().getStream())
-                                     .setDimensionIfNotNull(
-                                         DruidMetrics.TAGS,
-                                         
spec.getContextValue(DruidMetrics.TAGS)
-                                     )
-                                     .setMetric(
-                                         AUTOSCALER_SCALING_TIME_METRIC,
-                                         scaleActionStopwatch.millisElapsed()
-                                     ));
-      log.info("Changed taskCount to [%s] for supervisor[%s] for 
dataSource[%s].", desiredActiveTaskCount, supervisorId, dataSource);
-      return true;
     }
+    log.info(
+        "Starting scale action, current active task count is [%d] and desired 
task count is [%d] for supervisor[%s] for dataSource[%s].",
+        currentActiveTaskCount,
+        desiredActiveTaskCount,
+        supervisorId,
+        dataSource
+    );
+    final Stopwatch scaleActionStopwatch = Stopwatch.createStarted();
+    gracefulShutdownInternal();
+    changeTaskCountInAutoScalerConfig(desiredActiveTaskCount);
+    clearAllocationInfo();
+    emitter.emit(ServiceMetricEvent.builder()
+                                 .setDimension(DruidMetrics.SUPERVISOR_ID, 
supervisorId)
+                                 .setDimension(DruidMetrics.DATASOURCE, 
dataSource)
+                                 .setDimension(DruidMetrics.STREAM, 
getIoConfig().getStream())
+                                 .setDimensionIfNotNull(
+                                     DruidMetrics.TAGS,
+                                     spec.getContextValue(DruidMetrics.TAGS)
+                                 )
+                                 .setMetric(
+                                     AUTOSCALER_SCALING_TIME_METRIC,
+                                     scaleActionStopwatch.millisElapsed()
+                                 ));
+    log.info("Changed taskCount to [%s] for supervisor[%s] for 
dataSource[%s].", desiredActiveTaskCount, supervisorId, dataSource);
+    return true;
   }
 
-  private void changeTaskCountInIOConfig(int desiredActiveTaskCount)
+  private void changeTaskCountInAutoScalerConfig(int desiredActiveTaskCount)
   {
+    // Sanity check.
+    if (autoScalerConfig == null) {
+      log.warn("autoScalerConfig is null but scale action is submitted, how 
can it be ?");
+      return;
+    }

Review Comment:
   Please revert this change as well since the old method name is still 
relevant.



##########
indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisorSpec.java:
##########
@@ -255,6 +261,33 @@ public void validateSpecUpdateTo(SupervisorSpec 
proposedSpec) throws DruidExcept
     }
   }
 
+  @Override
+  public void merge(@NotNull SupervisorSpec existingSpec)
+  {
+    AutoScalerConfig thisAutoScalerConfig = 
this.getIoConfig().getAutoScalerConfig();
+    // Either if autoscaler is absent or taskCountStart is specified - just 
return.
+    if (thisAutoScalerConfig == null || 
thisAutoScalerConfig.getTaskCountStart() != null) {
+      return;
+    }

Review Comment:
   Should we also return early if `this.ioConfig.getTaskCount()` is specified?



##########
extensions-core/kafka-indexing-service/src/test/java/org/apache/druid/indexing/kafka/supervisor/KafkaSupervisorTest.java:
##########
@@ -389,7 +389,7 @@ public SeekableStreamIndexTaskClient<KafkaTopicPartition, 
Long> build(
 
     supervisor.start();
     int taskCountBeforeScale = supervisor.getIoConfig().getTaskCount();
-    Assert.assertEquals(1, taskCountBeforeScale);
+    
Assert.assertNull(supervisor.getIoConfig().getAutoScalerConfig().getTaskCountStart());

Review Comment:
   Please revert the changes to `KafkaSupervisorTest` and 
`KinesisSupervisorTest`.
   I don't think they are needed anymore.



##########
indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/SupervisorResourceConfigMergeTest.java:
##########
@@ -0,0 +1,533 @@
+/*
+ * 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.indexing.seekablestream;
+
+import com.fasterxml.jackson.annotation.JsonIgnore;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.google.common.base.Optional;
+import org.apache.druid.audit.AuditManager;
+import org.apache.druid.indexing.overlord.IndexerMetadataStorageCoordinator;
+import org.apache.druid.indexing.overlord.TaskMaster;
+import org.apache.druid.indexing.overlord.TaskStorage;
+import org.apache.druid.indexing.overlord.supervisor.Supervisor;
+import org.apache.druid.indexing.overlord.supervisor.SupervisorManager;
+import org.apache.druid.indexing.overlord.supervisor.SupervisorResource;
+import org.apache.druid.indexing.overlord.supervisor.SupervisorSpec;
+import 
org.apache.druid.indexing.overlord.supervisor.SupervisorStateManagerConfig;
+import 
org.apache.druid.indexing.seekablestream.supervisor.SeekableStreamSupervisorIOConfig;
+import 
org.apache.druid.indexing.seekablestream.supervisor.SeekableStreamSupervisorIngestionSpec;
+import 
org.apache.druid.indexing.seekablestream.supervisor.SeekableStreamSupervisorSpec;
+import 
org.apache.druid.indexing.seekablestream.supervisor.autoscaler.AutoScalerConfig;
+import org.apache.druid.java.util.emitter.service.ServiceEmitter;
+import org.apache.druid.java.util.metrics.DruidMonitorSchedulerConfig;
+import org.apache.druid.segment.TestHelper;
+import org.apache.druid.segment.incremental.RowIngestionMetersFactory;
+import org.apache.druid.segment.indexing.DataSchema;
+import org.apache.druid.server.security.Access;
+import org.apache.druid.server.security.Action;
+import org.apache.druid.server.security.AuthConfig;
+import org.apache.druid.server.security.AuthenticationResult;
+import org.apache.druid.server.security.Authorizer;
+import org.apache.druid.server.security.AuthorizerMapper;
+import org.apache.druid.server.security.Resource;
+import org.apache.druid.server.security.ResourceAction;
+import org.apache.druid.server.security.ResourceType;
+import org.easymock.Capture;
+import org.easymock.EasyMock;
+import org.easymock.EasyMockRunner;
+import org.easymock.EasyMockSupport;
+import org.easymock.Mock;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+
+import javax.annotation.Nonnull;
+import javax.annotation.Nullable;
+import javax.servlet.http.HttpServletRequest;
+import javax.ws.rs.core.Response;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Set;
+
+@RunWith(EasyMockRunner.class)
+public class SupervisorResourceConfigMergeTest extends EasyMockSupport
+{
+  private static final ObjectMapper OBJECT_MAPPER = 
TestHelper.makeJsonMapper();
+  @Mock
+  private TaskMaster taskMaster;
+
+  @Mock
+  private SupervisorManager supervisorManager;
+
+  @Mock
+  private HttpServletRequest request;
+
+  @Mock
+  private AuthConfig authConfig;
+
+  @Mock
+  private AuditManager auditManager;
+
+  private SupervisorResource supervisorResource;
+
+  @Before
+  public void setUp()
+  {
+    supervisorResource = new SupervisorResource(
+        taskMaster,
+        new AuthorizerMapper(null)
+        {
+          @Override
+          public Authorizer getAuthorizer(String name)
+          {
+            return (authenticationResult, resource, action) -> {
+              if (authenticationResult.getIdentity().equals("druid")) {
+                return Access.OK;
+              } else {
+                if (resource.getType().equals(ResourceType.DATASOURCE)) {
+                  if (resource.getName().equals("datasource2")) {
+                    return Access.deny("not authorized.");
+                  } else {
+                    return Access.OK;
+                  }
+                } else if (resource.getType().equals(ResourceType.EXTERNAL)) {
+                  if (resource.getName().equals("test")) {
+                    return Access.deny("not authorized.");
+                  } else {
+                    return Access.OK;
+                  }
+                }
+                return Access.OK;
+              }
+            };
+          }
+        },
+        OBJECT_MAPPER,
+        authConfig,
+        auditManager
+    );
+  }
+
+  /**
+   * Tests that when a new spec is submitted without taskCountStart, the merge 
function
+   * follows the priority: provided taskCountStart > provided taskCount > 
existing taskCount > provided taskCountMin.
+   * In this test, existing spec has taskCount=5, new spec has no taskCount 
set,
+   * so merge should use existing taskCount (5).
+   */

Review Comment:
   Can we remove the javadocs and update the method name to include the 
expectation/condition?



##########
indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisorSpec.java:
##########
@@ -183,15 +185,16 @@ public DruidMonitorSchedulerConfig 
getMonitorSchedulerConfig()
 
   /**
    * An autoScaler instance will be returned depending on the 
autoScalerConfig. In case autoScalerConfig is null or autoScaler is disabled 
then NoopTaskAutoScaler will be returned.
+   *
    * @param supervisor
    * @return autoScaler
    */
   @Override
   public SupervisorTaskAutoScaler createAutoscaler(Supervisor supervisor)
   {
-    AutoScalerConfig autoScalerConfig = 
ingestionSchema.getIOConfig().getAutoScalerConfig();
-    if (autoScalerConfig != null && autoScalerConfig.getEnableTaskAutoScaler() 
&& supervisor instanceof SeekableStreamSupervisor) {
-      return autoScalerConfig.createAutoScaler(supervisor, this, emitter);
+    AutoScalerConfig autoScalerCfg = 
ingestionSchema.getIOConfig().getAutoScalerConfig();

Review Comment:
   Please revert this rename as it does not seem required.



##########
indexing-service/src/test/java/org/apache/druid/indexing/seekablestream/SupervisorResourceConfigMergeTest.java:
##########
@@ -0,0 +1,533 @@
+/*
+ * 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.indexing.seekablestream;
+
+import com.fasterxml.jackson.annotation.JsonIgnore;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.google.common.base.Optional;
+import org.apache.druid.audit.AuditManager;
+import org.apache.druid.indexing.overlord.IndexerMetadataStorageCoordinator;
+import org.apache.druid.indexing.overlord.TaskMaster;
+import org.apache.druid.indexing.overlord.TaskStorage;
+import org.apache.druid.indexing.overlord.supervisor.Supervisor;
+import org.apache.druid.indexing.overlord.supervisor.SupervisorManager;
+import org.apache.druid.indexing.overlord.supervisor.SupervisorResource;
+import org.apache.druid.indexing.overlord.supervisor.SupervisorSpec;
+import 
org.apache.druid.indexing.overlord.supervisor.SupervisorStateManagerConfig;
+import 
org.apache.druid.indexing.seekablestream.supervisor.SeekableStreamSupervisorIOConfig;
+import 
org.apache.druid.indexing.seekablestream.supervisor.SeekableStreamSupervisorIngestionSpec;
+import 
org.apache.druid.indexing.seekablestream.supervisor.SeekableStreamSupervisorSpec;
+import 
org.apache.druid.indexing.seekablestream.supervisor.autoscaler.AutoScalerConfig;
+import org.apache.druid.java.util.emitter.service.ServiceEmitter;
+import org.apache.druid.java.util.metrics.DruidMonitorSchedulerConfig;
+import org.apache.druid.segment.TestHelper;
+import org.apache.druid.segment.incremental.RowIngestionMetersFactory;
+import org.apache.druid.segment.indexing.DataSchema;
+import org.apache.druid.server.security.Access;
+import org.apache.druid.server.security.Action;
+import org.apache.druid.server.security.AuthConfig;
+import org.apache.druid.server.security.AuthenticationResult;
+import org.apache.druid.server.security.Authorizer;
+import org.apache.druid.server.security.AuthorizerMapper;
+import org.apache.druid.server.security.Resource;
+import org.apache.druid.server.security.ResourceAction;
+import org.apache.druid.server.security.ResourceType;
+import org.easymock.Capture;
+import org.easymock.EasyMock;
+import org.easymock.EasyMockRunner;
+import org.easymock.EasyMockSupport;
+import org.easymock.Mock;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+
+import javax.annotation.Nonnull;
+import javax.annotation.Nullable;
+import javax.servlet.http.HttpServletRequest;
+import javax.ws.rs.core.Response;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Set;
+
+@RunWith(EasyMockRunner.class)
+public class SupervisorResourceConfigMergeTest extends EasyMockSupport

Review Comment:
   Please move the test methods in this class to `SupervisorResourceTest` 
itself.



##########
indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisor.java:
##########
@@ -556,44 +566,52 @@ public String getType()
   private boolean changeTaskCount(int desiredActiveTaskCount)
       throws InterruptedException, ExecutionException
   {
+    if (autoScalerConfig == null) {
+      log.warn("autoScalerConfig is 'null' but dynamic allocation notice is 
submitted, how can it be ?");
+      return false;
+    }
     int currentActiveTaskCount;
     Collection<TaskGroup> activeTaskGroups = 
activelyReadingTaskGroups.values();
     currentActiveTaskCount = activeTaskGroups.size();
 
     if (desiredActiveTaskCount < 0 || desiredActiveTaskCount == 
currentActiveTaskCount) {
       return false;
-    } else {
-      log.info(
-          "Starting scale action, current active task count is [%d] and 
desired task count is [%d] for supervisor[%s] for dataSource[%s].",
-          currentActiveTaskCount,
-          desiredActiveTaskCount,
-          supervisorId,
-          dataSource
-      );
-      final Stopwatch scaleActionStopwatch = Stopwatch.createStarted();
-      gracefulShutdownInternal();
-      changeTaskCountInIOConfig(desiredActiveTaskCount);
-      clearAllocationInfo();
-      emitter.emit(ServiceMetricEvent.builder()
-                                     .setDimension(DruidMetrics.SUPERVISOR_ID, 
supervisorId)
-                                     .setDimension(DruidMetrics.DATASOURCE, 
dataSource)
-                                     .setDimension(DruidMetrics.STREAM, 
getIoConfig().getStream())
-                                     .setDimensionIfNotNull(
-                                         DruidMetrics.TAGS,
-                                         
spec.getContextValue(DruidMetrics.TAGS)
-                                     )
-                                     .setMetric(
-                                         AUTOSCALER_SCALING_TIME_METRIC,
-                                         scaleActionStopwatch.millisElapsed()
-                                     ));
-      log.info("Changed taskCount to [%s] for supervisor[%s] for 
dataSource[%s].", desiredActiveTaskCount, supervisorId, dataSource);
-      return true;
     }
+    log.info(
+        "Starting scale action, current active task count is [%d] and desired 
task count is [%d] for supervisor[%s] for dataSource[%s].",
+        currentActiveTaskCount,
+        desiredActiveTaskCount,
+        supervisorId,
+        dataSource
+    );
+    final Stopwatch scaleActionStopwatch = Stopwatch.createStarted();
+    gracefulShutdownInternal();
+    changeTaskCountInAutoScalerConfig(desiredActiveTaskCount);
+    clearAllocationInfo();
+    emitter.emit(ServiceMetricEvent.builder()
+                                 .setDimension(DruidMetrics.SUPERVISOR_ID, 
supervisorId)
+                                 .setDimension(DruidMetrics.DATASOURCE, 
dataSource)
+                                 .setDimension(DruidMetrics.STREAM, 
getIoConfig().getStream())
+                                 .setDimensionIfNotNull(
+                                     DruidMetrics.TAGS,
+                                     spec.getContextValue(DruidMetrics.TAGS)
+                                 )
+                                 .setMetric(
+                                     AUTOSCALER_SCALING_TIME_METRIC,
+                                     scaleActionStopwatch.millisElapsed()
+                                 ));
+    log.info("Changed taskCount to [%s] for supervisor[%s] for 
dataSource[%s].", desiredActiveTaskCount, supervisorId, dataSource);
+    return true;

Review Comment:
   Please revert this change to avoid bloating the diff.



-- 
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: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]


---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]

Reply via email to