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


##########
extensions-core/kinesis-indexing-service/src/main/java/org/apache/druid/indexing/kinesis/supervisor/KinesisSupervisorIOConfig.java:
##########
@@ -91,7 +92,7 @@ public KinesisSupervisorIOConfig(
         earlyMessageRejectionPeriod,
         autoScalerConfig,
         lateMessageRejectionStartDateTime,
-        null
+        new IdleConfig(false, null)

Review Comment:
   We should either pass both as null or both as the default values (`false` 
and `600_000`). I prefer both null as they are handled inside the `IdleConfig` 
anyway.



##########
server/src/main/java/org/apache/druid/indexing/overlord/supervisor/SupervisorStateManagerConfig.java:
##########
@@ -46,6 +46,12 @@
   @JsonProperty
   private int maxStoredExceptionEvents = Math.max(unhealthinessThreshold, 
healthinessThreshold);
 
+  @JsonProperty("idleConfig.enabled")
+  private boolean idleConfigEnabled = false;

Review Comment:
   Maybe instead of flattening the spec, keep an `idleConfig` itself here and 
handle it correctly inside the `SeekableStreamSupervisor`.



##########
indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisor.java:
##########
@@ -3290,8 +3291,17 @@ private void checkCurrentTaskState() throws 
ExecutionException, InterruptedExcep
 
   private void checkIfStreamInactiveAndTurnSupervisorIdle()
   {
-    IdleConfig idleConfig = spec.getIoConfig().getIdleConfig();
-    if ((idleConfig == null || !idleConfig.isEnabled()) || spec.isSuspended()) 
{
+    if (idleConfig == null) {

Review Comment:
   This initialization should be done in the constructor itself.



##########
docs/configuration/index.md:
##########
@@ -1152,7 +1152,10 @@ There are additional configs for autoscaling (if it is 
enabled):
 |`druid.supervisor.taskUnhealthinessThreshold`|The number of consecutive task 
failures before the supervisor is considered unhealthy.|3|
 |`druid.supervisor.storeStackTrace`|Whether full stack traces of supervisor 
exceptions should be stored and returned by the supervisor `/status` 
endpoint.|false|
 |`druid.supervisor.maxStoredExceptionEvents`|The maximum number of exception 
events that can be returned through the supervisor `/status` 
endpoint.|`max(healthinessThreshold, unhealthinessThreshold)`|
+|`druid.supervisor.idleConfig.enabled`|If `true`, supervisor can become idle 
if there is no data on input stream/topic for some time.|false|
+|`druid.supervisor.idleConfig.inactiveAfterMillis`|Supervisor is marked as 
idle if all existing data has been read from input topic and no new data has 
been published for `inactiveAfterMillis` milliseconds.|`600_000`|
 
+idleConfig here becomes the default behaviour for the entire cluster. See 
[Idle Configuration in 
KafkaSupervisorIOConfig](../development/extensions-core/kafka-supervisor-reference.md/#KafkaSupervisorIOConfig)
 to override it on individual basis.

Review Comment:
   ```suggestion
   The `idleConfig` specified in the runtime properties of the overlord defines 
the default behaviour for the entire cluster. See [Idle Configuration in 
KafkaSupervisorIOConfig](../development/extensions-core/kafka-supervisor-reference.md/#KafkaSupervisorIOConfig)
 to override it for an individual supervisor.
   ```



##########
indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisor.java:
##########
@@ -3290,8 +3291,17 @@ private void checkCurrentTaskState() throws 
ExecutionException, InterruptedExcep
 
   private void checkIfStreamInactiveAndTurnSupervisorIdle()
   {
-    IdleConfig idleConfig = spec.getIoConfig().getIdleConfig();
-    if ((idleConfig == null || !idleConfig.isEnabled()) || spec.isSuspended()) 
{
+    if (idleConfig == null) {
+      idleConfig = 
java.util.Optional.ofNullable(spec.getIoConfig().getIdleConfig())

Review Comment:
   Please simplify this to a plain if-else.



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