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


##########
indexing-service/src/main/java/org/apache/druid/guice/PeonProcessingModule.java:
##########
@@ -0,0 +1,140 @@
+/*
+ * 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.fasterxml.jackson.databind.ObjectMapper;
+import com.google.inject.Binder;
+import com.google.inject.Module;
+import com.google.inject.Provides;
+import org.apache.druid.client.cache.CacheConfig;
+import org.apache.druid.client.cache.CachePopulator;
+import org.apache.druid.client.cache.CachePopulatorStats;
+import org.apache.druid.collections.BlockingPool;
+import org.apache.druid.collections.DummyBlockingPool;
+import org.apache.druid.collections.DummyNonBlockingPool;
+import org.apache.druid.collections.NonBlockingPool;
+import org.apache.druid.guice.annotations.Global;
+import org.apache.druid.guice.annotations.Merging;
+import org.apache.druid.guice.annotations.Smile;
+import org.apache.druid.indexing.common.task.Task;
+import org.apache.druid.java.util.common.lifecycle.Lifecycle;
+import org.apache.druid.java.util.common.logger.Logger;
+import org.apache.druid.query.DruidProcessingConfig;
+import org.apache.druid.query.ExecutorServiceMonitor;
+import org.apache.druid.query.NoopQueryProcessingPool;
+import org.apache.druid.query.QueryProcessingPool;
+import org.apache.druid.query.groupby.GroupByQueryConfig;
+import org.apache.druid.query.groupby.GroupByResourcesReservationPool;
+
+import java.nio.ByteBuffer;
+
+/**
+ * This module fulfills the dependency injection of query processing and 
caching resources: buffer pools and
+ * thread pools on Peon selectively. Only the peons for the tasks supporting 
queries need to allocate direct buffers
+ * and thread pools. Thus, this is separate from the {@link 
DruidProcessingModule} to separate the needs of the peons and
+ * the historicals
+ *
+ * @see DruidProcessingModule
+ */
+public class PeonProcessingModule implements Module
+{
+  private static final Logger log = new Logger(PeonProcessingModule.class);
+
+  @Override
+  public void configure(Binder binder)
+  {
+    DruidProcessingModule.registerConfigsAndMonitor(binder);
+  }
+
+  @Provides
+  @LazySingleton
+  public CachePopulator getCachePopulator(
+      @Smile ObjectMapper smileMapper,
+      CachePopulatorStats cachePopulatorStats,
+      CacheConfig cacheConfig
+  )
+  {
+    return DruidProcessingModule.createCachePopulator(smileMapper, 
cachePopulatorStats, cacheConfig);
+  }
+
+  @Provides
+  @ManageLifecycle
+  public QueryProcessingPool getProcessingExecutorPool(
+      Task task,
+      DruidProcessingConfig config,
+      ExecutorServiceMonitor executorServiceMonitor,
+      Lifecycle lifecycle
+  )
+  {
+    if (!task.supportsQueries()) {
+      if (config.isNumThreadsConfigured()) {
+        log.warn(
+            "Ignoring the configured numThreads[%d] because task[%s] of 
type[%s] does not support queries",
+            config.getNumThreads(),
+            task.getId(),
+            task.getType()
+        );
+      }
+      return NoopQueryProcessingPool.instance();
+    }
+    return DruidProcessingModule.createProcessingExecutorPool(config, 
executorServiceMonitor, lifecycle);
+  }
+
+  @Provides
+  @LazySingleton
+  @Global
+  public NonBlockingPool<ByteBuffer> getIntermediateResultsPool(Task task, 
DruidProcessingConfig config)
+  {
+    if (!task.supportsQueries()) {
+      return DummyNonBlockingPool.instance();
+    }
+    return DruidProcessingModule.createIntermediateResultsPool(config);

Review Comment:
   Maybe invert the condition for readability:
   
   ```suggestion
       if (task.supportsQueries()) {
         return DruidProcessingModule.createIntermediateResultsPool(config);
       } else {
         return DummyNonBlockingPool.instance();
       }
   ```



##########
processing/src/main/java/org/apache/druid/query/NoopQueryProcessingPool.java:
##########
@@ -0,0 +1,131 @@
+/*
+ * 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.query;
+
+import com.google.common.util.concurrent.ListenableFuture;
+import org.apache.druid.error.DruidException;
+
+import java.util.Collection;
+import java.util.Collections;
+import java.util.List;
+import java.util.concurrent.Callable;
+import java.util.concurrent.Future;
+import java.util.concurrent.TimeUnit;
+
+/**
+ * Implementation of {@link QueryProcessingPool} that throws when any query 
execution task unit is submitted to it. It is
+ * semantically shutdown from the moment it is created, and since the shutdown 
methods are supposed to be idempotent,
+ * they do not throw like the execution methods
+ */
+public class NoopQueryProcessingPool implements QueryProcessingPool
+{
+  private static final QueryProcessingPool INSTANCE = new 
NoopQueryProcessingPool();
+  private static final DruidException UNSUPPORTED_EXCEPTION =

Review Comment:
   I am not sure if keeping an exception constant is desirable. You can keep 
the exception message as a constant but throw a fresh exception wherever needed.



##########
processing/src/main/java/org/apache/druid/query/NoopQueryProcessingPool.java:
##########
@@ -0,0 +1,131 @@
+/*
+ * 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.query;
+
+import com.google.common.util.concurrent.ListenableFuture;
+import org.apache.druid.error.DruidException;
+
+import java.util.Collection;
+import java.util.Collections;
+import java.util.List;
+import java.util.concurrent.Callable;
+import java.util.concurrent.Future;
+import java.util.concurrent.TimeUnit;
+
+/**
+ * Implementation of {@link QueryProcessingPool} that throws when any query 
execution task unit is submitted to it. It is
+ * semantically shutdown from the moment it is created, and since the shutdown 
methods are supposed to be idempotent,
+ * they do not throw like the execution methods
+ */
+public class NoopQueryProcessingPool implements QueryProcessingPool
+{
+  private static final QueryProcessingPool INSTANCE = new 
NoopQueryProcessingPool();

Review Comment:
   ```suggestion
     private static final NoopQueryProcessingPool INSTANCE = new 
NoopQueryProcessingPool();
   ```



##########
processing/src/main/java/org/apache/druid/query/NoopQueryProcessingPool.java:
##########
@@ -0,0 +1,131 @@
+/*
+ * 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.query;
+
+import com.google.common.util.concurrent.ListenableFuture;
+import org.apache.druid.error.DruidException;
+
+import java.util.Collection;
+import java.util.Collections;
+import java.util.List;
+import java.util.concurrent.Callable;
+import java.util.concurrent.Future;
+import java.util.concurrent.TimeUnit;
+
+/**
+ * Implementation of {@link QueryProcessingPool} that throws when any query 
execution task unit is submitted to it. It is
+ * semantically shutdown from the moment it is created, and since the shutdown 
methods are supposed to be idempotent,
+ * they do not throw like the execution methods
+ */
+public class NoopQueryProcessingPool implements QueryProcessingPool
+{
+  private static final QueryProcessingPool INSTANCE = new 
NoopQueryProcessingPool();
+  private static final DruidException UNSUPPORTED_EXCEPTION =
+      DruidException.defensive("Unexpected call made to 
NoopQueryProcessingPool");
+
+  public static QueryProcessingPool instance()

Review Comment:
   ```suggestion
     public static NoopQueryProcessingPool instance()
   ```



##########
indexing-service/src/main/java/org/apache/druid/guice/PeonProcessingModule.java:
##########
@@ -0,0 +1,140 @@
+/*
+ * 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.fasterxml.jackson.databind.ObjectMapper;
+import com.google.inject.Binder;
+import com.google.inject.Module;
+import com.google.inject.Provides;
+import org.apache.druid.client.cache.CacheConfig;
+import org.apache.druid.client.cache.CachePopulator;
+import org.apache.druid.client.cache.CachePopulatorStats;
+import org.apache.druid.collections.BlockingPool;
+import org.apache.druid.collections.DummyBlockingPool;
+import org.apache.druid.collections.DummyNonBlockingPool;
+import org.apache.druid.collections.NonBlockingPool;
+import org.apache.druid.guice.annotations.Global;
+import org.apache.druid.guice.annotations.Merging;
+import org.apache.druid.guice.annotations.Smile;
+import org.apache.druid.indexing.common.task.Task;
+import org.apache.druid.java.util.common.lifecycle.Lifecycle;
+import org.apache.druid.java.util.common.logger.Logger;
+import org.apache.druid.query.DruidProcessingConfig;
+import org.apache.druid.query.ExecutorServiceMonitor;
+import org.apache.druid.query.NoopQueryProcessingPool;
+import org.apache.druid.query.QueryProcessingPool;
+import org.apache.druid.query.groupby.GroupByQueryConfig;
+import org.apache.druid.query.groupby.GroupByResourcesReservationPool;
+
+import java.nio.ByteBuffer;
+
+/**
+ * This module fulfills the dependency injection of query processing and 
caching resources: buffer pools and
+ * thread pools on Peon selectively. Only the peons for the tasks supporting 
queries need to allocate direct buffers
+ * and thread pools. Thus, this is separate from the {@link 
DruidProcessingModule} to separate the needs of the peons and
+ * the historicals
+ *
+ * @see DruidProcessingModule
+ */
+public class PeonProcessingModule implements Module
+{
+  private static final Logger log = new Logger(PeonProcessingModule.class);
+
+  @Override
+  public void configure(Binder binder)
+  {
+    DruidProcessingModule.registerConfigsAndMonitor(binder);
+  }
+
+  @Provides
+  @LazySingleton
+  public CachePopulator getCachePopulator(
+      @Smile ObjectMapper smileMapper,
+      CachePopulatorStats cachePopulatorStats,
+      CacheConfig cacheConfig
+  )
+  {
+    return DruidProcessingModule.createCachePopulator(smileMapper, 
cachePopulatorStats, cacheConfig);
+  }
+
+  @Provides
+  @ManageLifecycle
+  public QueryProcessingPool getProcessingExecutorPool(
+      Task task,
+      DruidProcessingConfig config,
+      ExecutorServiceMonitor executorServiceMonitor,
+      Lifecycle lifecycle
+  )
+  {
+    if (!task.supportsQueries()) {
+      if (config.isNumThreadsConfigured()) {
+        log.warn(
+            "Ignoring the configured numThreads[%d] because task[%s] of 
type[%s] does not support queries",
+            config.getNumThreads(),
+            task.getId(),
+            task.getType()
+        );
+      }
+      return NoopQueryProcessingPool.instance();
+    }
+    return DruidProcessingModule.createProcessingExecutorPool(config, 
executorServiceMonitor, lifecycle);
+  }
+
+  @Provides
+  @LazySingleton
+  @Global
+  public NonBlockingPool<ByteBuffer> getIntermediateResultsPool(Task task, 
DruidProcessingConfig config)
+  {
+    if (!task.supportsQueries()) {
+      return DummyNonBlockingPool.instance();
+    }
+    return DruidProcessingModule.createIntermediateResultsPool(config);

Review Comment:
   Same comment in other methods.



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