vvivekiyer commented on code in PR #13847:
URL: https://github.com/apache/pinot/pull/13847#discussion_r1724040928


##########
pinot-core/src/main/java/org/apache/pinot/core/query/scheduler/BinaryWorkloadScheduler.java:
##########
@@ -0,0 +1,212 @@
+/**
+ * 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.pinot.core.query.scheduler;
+
+import com.google.common.util.concurrent.ListenableFuture;
+import com.google.common.util.concurrent.ListenableFutureTask;
+import com.google.common.util.concurrent.MoreExecutors;
+import java.util.List;
+import java.util.concurrent.Semaphore;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.LongAccumulator;
+import org.apache.pinot.common.exception.QueryException;
+import org.apache.pinot.common.metrics.ServerMeter;
+import org.apache.pinot.common.metrics.ServerMetrics;
+import org.apache.pinot.common.metrics.ServerQueryPhase;
+import org.apache.pinot.common.metrics.ServerTimer;
+import org.apache.pinot.common.utils.config.QueryOptionsUtils;
+import org.apache.pinot.core.query.executor.QueryExecutor;
+import org.apache.pinot.core.query.request.ServerQueryRequest;
+import 
org.apache.pinot.core.query.scheduler.resources.BinaryWorkloadResourceManager;
+import org.apache.pinot.core.query.scheduler.resources.QueryExecutorService;
+import org.apache.pinot.spi.env.PinotConfiguration;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/**
+ * This scheduler is designed to deal with two types of workloads
+ * 1. Primary Workloads -> regular queries from the application
+ * 2. Secondary Workloads -> adhoc queries fired from tools, testing, etc
+ *
+ *
+ * Primary Workload Queries
+ * Primary workloads queries are executed with priority and submitted to the 
Runner threads as and when they arrive.
+ * The resources used by a primary workload query is not capped.
+ *
+ * Secondary Workload Queries
+ *   - Secondary workload queries are identified using a query option -> "SET 
isSecondaryWorkload=true"
+ *   - Secondary workload queries are contained as follows:
+ *       - Restrictions on number of runner threads available to process 
secondary queries
+ *       - Restrictions on total number of worker threads available to process 
a single secondary query
+ *       - Restrictions on total number of worker threads available to process 
all in-progress secondary queries
+ */
+public class BinaryWorkloadScheduler extends QueryScheduler {
+  private static final Logger LOGGER = 
LoggerFactory.getLogger(BinaryWorkloadScheduler.class);
+
+  public static final String MAX_SECONDARY_QUERIES = 
"binarywlm.maxSecondaryRunnerThreads";
+  public static final int DEFAULT_MAX_SECONDARY_QUERIES = 5;
+
+  // Secondary Workload Runners.
+  private final int _numSecondaryRunners;
+  private final Semaphore _secondaryRunnerSemaphore;
+
+  private final SecondaryWorkloadQueue _secondaryQueryQ;
+
+  Thread _scheduler;
+
+  public BinaryWorkloadScheduler(PinotConfiguration config, QueryExecutor 
queryExecutor, ServerMetrics metrics,
+      LongAccumulator latestQueryTime) {
+    super(config, queryExecutor, new BinaryWorkloadResourceManager(config), 
metrics, latestQueryTime);
+
+    _secondaryQueryQ = new SecondaryWorkloadQueue(config, _resourceManager);
+    _numSecondaryRunners = config.getProperty(MAX_SECONDARY_QUERIES, 
DEFAULT_MAX_SECONDARY_QUERIES);
+    LOGGER.info("numSecondaryRunners={}", _numSecondaryRunners);
+    _secondaryRunnerSemaphore = new Semaphore(_numSecondaryRunners);
+  }
+
+  @Override
+  public String name() {
+    return "BinaryWorkloadScheduler";
+  }
+
+  @Override
+  public ListenableFuture<byte[]> submit(ServerQueryRequest queryRequest) {
+    if (!_isRunning) {
+      return immediateErrorResponse(queryRequest, 
QueryException.SERVER_SCHEDULER_DOWN_ERROR);
+    }
+
+    
queryRequest.getTimerContext().startNewPhaseTimer(ServerQueryPhase.SCHEDULER_WAIT);
+    if 
(!QueryOptionsUtils.isSecondaryWorkload(queryRequest.getQueryContext().getQueryOptions()))
 {
+      QueryExecutorService queryExecutorService = 
_resourceManager.getExecutorService(queryRequest, null);

Review Comment:
   For Primary workloads, we wanted to retain existing default behavior without 
any disruption (as they correspond to all our production queries) - hence the 
unbounded FCFS.  Using an unbounded FCFS for primary ensures that Primary 
workloads can always eat into the resources of secondary workloads but not 
vice-versa. 



-- 
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-unsubscr...@pinot.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscr...@pinot.apache.org
For additional commands, e-mail: commits-h...@pinot.apache.org

Reply via email to