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 2020/02/26 12:21:23 UTC

[GitHub] [druid] clintropolis opened a new pull request #9407: query laning and load shedding

clintropolis opened a new pull request #9407: query laning and load shedding
URL: https://github.com/apache/druid/pull/9407
 
 
   ### Description
   Initial piece of #6993, adding basic query laning facilities and capacity enforcement. Does not include automatic prioritization, this will be done in a follow-up PR that builds on top of this branch.
   
   Work in progress, actual description and design discussion TBD.
   
   <hr>
   
   This PR has:
   - [ ] been self-reviewed.
      - [ ] using the [concurrency checklist](https://github.com/apache/druid/blob/master/dev/code-review/concurrency.md) (Remove this item if the PR doesn't have any relation to concurrency.)
   - [x] added documentation for new or modified features or behaviors.
   - [ ] added Javadocs for most classes and all non-trivial methods. Linked related entities via Javadoc links.
   - [ ] added comments explaining the "why" and the intent of the code wherever would not be obvious for an unfamiliar reader.
   - [x] added unit tests or modified existing tests to cover new code paths.
   - [ ] added integration tests.
   - [x] been tested in a test Druid cluster.
   
   <!-- Check the items by putting "x" in the brackets for the done things. Not all of these items apply to every PR. Remove the items which are not done or not relevant to the PR. None of the items from the checklist above are strictly necessary, but it would be very helpful if you at least self-review the PR. -->
   
   <hr>
   
   ##### Key changed/added classes in this PR
    * `QueryManager` -> `QueryScheduler`
    * other stuff
   

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] clintropolis commented on a change in pull request #9407: query laning and load shedding

Posted by GitBox <gi...@apache.org>.
clintropolis commented on a change in pull request #9407: query laning and load shedding
URL: https://github.com/apache/druid/pull/9407#discussion_r389162250
 
 

 ##########
 File path: server/src/main/java/org/apache/druid/server/QueryScheduler.java
 ##########
 @@ -0,0 +1,241 @@
+/*
+ * 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;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.collect.HashMultimap;
+import com.google.common.collect.Multimaps;
+import com.google.common.collect.SetMultimap;
+import com.google.common.util.concurrent.ListenableFuture;
+import io.github.resilience4j.bulkhead.Bulkhead;
+import io.github.resilience4j.bulkhead.BulkheadConfig;
+import io.github.resilience4j.bulkhead.BulkheadRegistry;
+import it.unimi.dsi.fastutil.objects.Object2IntMap;
+import org.apache.druid.client.SegmentServerSelector;
+import org.apache.druid.java.util.common.concurrent.Execs;
+import org.apache.druid.java.util.common.guava.Sequence;
+import org.apache.druid.query.Query;
+import org.apache.druid.query.QueryContexts;
+import org.apache.druid.query.QueryPlus;
+import org.apache.druid.query.QueryWatcher;
+import org.apache.druid.server.initialization.ServerConfig;
+
+import java.time.Duration;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+
+/**
+ * QueryScheduler (potentially) assigns any {@link Query} that is to be executed to a 'query lane' using the
+ * {@link QueryLaningStrategy} that is defined in {@link QuerySchedulerConfig}.
+ *
+ * As a {@link QueryWatcher}, it also provides cancellation facilities.
+ *
+ * This class is shared by all requests on the Jetty HTTP theadpool and must be thread safe.
+ */
+public class QueryScheduler implements QueryWatcher
+{
+  static final String TOTAL = "default";
+  private final int totalCapacity;
+  private final QueryLaningStrategy laningStrategy;
+  private final BulkheadRegistry laneRegistry;
+  private final SetMultimap<String, ListenableFuture<?>> queryFutures;
+  private final SetMultimap<String, String> queryDatasources;
+
+  public QueryScheduler(int totalNumThreads, QueryLaningStrategy laningStrategy, ServerConfig serverConfig)
+  {
+    this.laningStrategy = laningStrategy;
+    this.queryFutures = Multimaps.synchronizedSetMultimap(HashMultimap.create());
+    this.queryDatasources = Multimaps.synchronizedSetMultimap(HashMultimap.create());
+    // if totalNumThreads is above 0 and less than druid.server.http.numThreads, enforce total limit
+    final boolean limitTotal;
+    if (totalNumThreads > 0 && totalNumThreads < serverConfig.getNumThreads()) {
+      limitTotal = true;
+      this.totalCapacity = totalNumThreads;
+    } else {
+      limitTotal = false;
+      this.totalCapacity = serverConfig.getNumThreads();
+    }
+    this.laneRegistry = BulkheadRegistry.of(getLaneConfigs(limitTotal));
+  }
+
+  @Override
+  public void registerQueryFuture(Query<?> query, ListenableFuture<?> future)
+  {
+    final String id = query.getId();
+    final Set<String> datasources = query.getDataSource().getTableNames();
+    queryFutures.put(id, future);
+    queryDatasources.putAll(id, datasources);
+    future.addListener(
+        () -> {
+          queryFutures.remove(id, future);
+          for (String datasource : datasources) {
+            queryDatasources.remove(id, datasource);
+          }
+        },
+        Execs.directExecutor()
+    );
+  }
+
+  /**
+   * Assign a query a lane (if not set)
+   */
+  public <T> Query<T> laneQuery(QueryPlus<T> queryPlus, Set<SegmentServerSelector> segments)
+  {
+    Query<T> query = queryPlus.getQuery();
+    // man wins over machine.. for now.
+    if (QueryContexts.getLane(query) != null) {
+      return query;
+    }
+    Optional<String> lane = laningStrategy.computeLane(queryPlus, segments);
+    return lane.map(query::withLane).orElse(query);
+  }
+
+  /**
+   * Run a query with the scheduler, attempting to acquire a semaphore from the total and lane specific query capacities
+   *
+   * Note that {@link #cancelQuery} should not interrupt the thread that calls run, in all current usages it only
+   * cancels any {@link ListenableFuture} created downstream. If this ever commonly changes, we should add
+   * synchronization between {@link #cancelQuery} and the acquisition of the {@link Bulkhead} to continue to ensure that
+   * anything acquired is also released.
+   *
+   * In the meantime, if a {@link ListenableFuture} is registered for the query that calls this method, it MUST handle
+   * this synchronization itself to ensure that no {@link Bulkhead} is acquired without releasing it.
+   */
+  public <T> Sequence<T> run(Query<?> query, Sequence<T> resultSequence)
+  {
+    List<Bulkhead> bulkheads = acquireLanes(query);
+    return resultSequence.withBaggage(() -> releaseLanes(bulkheads));
+  }
+
+  /**
+   * Forcibly cancel all futures that have been registered to a specific query id
+   */
+  public boolean cancelQuery(String id)
+  {
+    // if you re-use queryId and cancel queries... you are going to have a bad time
 
 Review comment:
   Ah, the comment was referring to the fact that query id can be manually set, like if you and I both send queries with the same id, one of us cancelling would cancel both I think.

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] jihoonson commented on a change in pull request #9407: query laning and load shedding

Posted by GitBox <gi...@apache.org>.
jihoonson commented on a change in pull request #9407: query laning and load shedding
URL: https://github.com/apache/druid/pull/9407#discussion_r389064900
 
 

 ##########
 File path: server/src/main/java/org/apache/druid/server/QueryLaningStrategy.java
 ##########
 @@ -0,0 +1,51 @@
+/*
+ * 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;
+
+import com.fasterxml.jackson.annotation.JsonSubTypes;
+import com.fasterxml.jackson.annotation.JsonTypeInfo;
+import it.unimi.dsi.fastutil.objects.Object2IntMap;
+import org.apache.druid.client.SegmentServerSelector;
+import org.apache.druid.query.QueryPlus;
+import org.apache.druid.server.scheduling.HiLoQueryLaningStrategy;
+import org.apache.druid.server.scheduling.NoQueryLaningStrategy;
+
+import java.util.Optional;
+import java.util.Set;
+
+
+@JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "strategy", defaultImpl = NoQueryLaningStrategy.class)
+@JsonSubTypes(value = {
+    @JsonSubTypes.Type(name = "none", value = NoQueryLaningStrategy.class),
+    @JsonSubTypes.Type(name = "hilo", value = HiLoQueryLaningStrategy.class)
+})
+public interface QueryLaningStrategy
+{
+  /**
+   * Provide map of lane names to the limit on the number of concurrent queries for that lane
+   * @param totalLimit
+   */
+  Object2IntMap<String> getLaneLimits(int totalLimit);
+
+  /**
+   * For a given {@link QueryPlus} and set of {@link SegmentServerSelector}, compute if a query belongs to a lane
+   */
+  <T> Optional<String> computeLane(QueryPlus<T> query, Set<SegmentServerSelector> segments);
 
 Review comment:
   Probably the javadoc of this method (or this class) should say that this class is used in `QueryScheduler` and should be thread-safe.

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] clintropolis commented on a change in pull request #9407: query laning and load shedding

Posted by GitBox <gi...@apache.org>.
clintropolis commented on a change in pull request #9407: query laning and load shedding
URL: https://github.com/apache/druid/pull/9407#discussion_r389218850
 
 

 ##########
 File path: server/src/main/java/org/apache/druid/server/QueryCapacityExceededException.java
 ##########
 @@ -0,0 +1,53 @@
+/*
+ * 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;
+
+import com.fasterxml.jackson.annotation.JsonCreator;
+import com.fasterxml.jackson.annotation.JsonProperty;
+import org.apache.druid.java.util.common.StringUtils;
+import org.apache.druid.query.QueryException;
+
+public class QueryCapacityExceededException extends QueryException
+{
+  private static final String ERROR_CLASS = QueryCapacityExceededException.class.getName();
+  public static final String ERROR_CODE = "Query capacity exceeded";
+  public static final String ERROR_MESSAGE = "Total query capacity exceeded";
+  public static final String ERROR_MESSAGE_TEMPLATE = "Query capacity exceeded for lane %s";
 
 Review comment:
   Added docs to `querying.md` with the other error response documentation

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] clintropolis commented on a change in pull request #9407: query laning and load shedding

Posted by GitBox <gi...@apache.org>.
clintropolis commented on a change in pull request #9407: query laning and load shedding
URL: https://github.com/apache/druid/pull/9407#discussion_r390201171
 
 

 ##########
 File path: server/src/main/java/org/apache/druid/server/QueryLaningStrategy.java
 ##########
 @@ -0,0 +1,53 @@
+/*
+ * 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;
+
+import com.fasterxml.jackson.annotation.JsonSubTypes;
+import com.fasterxml.jackson.annotation.JsonTypeInfo;
+import it.unimi.dsi.fastutil.objects.Object2IntMap;
+import org.apache.druid.client.SegmentServerSelector;
+import org.apache.druid.query.QueryPlus;
+import org.apache.druid.server.scheduling.HiLoQueryLaningStrategy;
+import org.apache.druid.server.scheduling.NoQueryLaningStrategy;
+
+import java.util.Optional;
+import java.util.Set;
+
+
+@JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "strategy", defaultImpl = NoQueryLaningStrategy.class)
+@JsonSubTypes(value = {
+    @JsonSubTypes.Type(name = "none", value = NoQueryLaningStrategy.class),
+    @JsonSubTypes.Type(name = "hilo", value = HiLoQueryLaningStrategy.class)
+})
+public interface QueryLaningStrategy
+{
+  /**
+   * Provide a map of lane names to the limit on the number of concurrent queries for that lane
+   * @param totalLimit
+   */
+  Object2IntMap<String> getLaneLimits(int totalLimit);
 
 Review comment:
   >is it expected that sum(returned_map.values) < totalLimit ?
   
   Right now I am not requiring this needs to be true since the limits are not guaranteed capacity, but rather maximums. It seemed more flexible to leave it to individual `QueryLaningStrategy` implementations to enforce that if they wish.
   
   >nit: Also wonder why limits returned here were not percentages but absolute counts given that each implementation would probably end up doing that calculation.
   
   Hmm, I think that is definitely worth considering, though @jihoonson was asking for absolute limits in this comment https://github.com/apache/druid/pull/9407#discussion_r389173425, so in the very least I will add a utility method to the `QueryLaningStrategy` interface for doing this conversion.
   
   

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] jihoonson commented on a change in pull request #9407: query laning and load shedding

Posted by GitBox <gi...@apache.org>.
jihoonson commented on a change in pull request #9407: query laning and load shedding
URL: https://github.com/apache/druid/pull/9407#discussion_r389160387
 
 

 ##########
 File path: server/src/main/java/org/apache/druid/server/scheduling/HiLoQueryLaningStrategy.java
 ##########
 @@ -0,0 +1,79 @@
+/*
+ * 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.scheduling;
+
+import com.fasterxml.jackson.annotation.JsonCreator;
+import com.fasterxml.jackson.annotation.JsonProperty;
+import com.google.common.base.Preconditions;
+import com.google.common.primitives.Ints;
+import it.unimi.dsi.fastutil.objects.Object2IntArrayMap;
+import it.unimi.dsi.fastutil.objects.Object2IntMap;
+import org.apache.druid.client.SegmentServerSelector;
+import org.apache.druid.query.Query;
+import org.apache.druid.query.QueryContexts;
+import org.apache.druid.query.QueryPlus;
+import org.apache.druid.server.QueryLaningStrategy;
+
+import java.util.Optional;
+import java.util.Set;
+
+/**
+ * Query laning strategy which associates all {@link Query} with priority lower than 0 into a 'low' lane
+ */
+public class HiLoQueryLaningStrategy implements QueryLaningStrategy
+{
+  public static final String LOW = "low";
+
+  @JsonProperty
+  private final int maxLowPercent;
 
 Review comment:
   Does it make sense to add `maxCapacity` 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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] sthetland commented on a change in pull request #9407: query laning and load shedding

Posted by GitBox <gi...@apache.org>.
sthetland commented on a change in pull request #9407: query laning and load shedding
URL: https://github.com/apache/druid/pull/9407#discussion_r389988957
 
 

 ##########
 File path: docs/configuration/index.md
 ##########
 @@ -1476,9 +1476,35 @@ These Broker configurations can be defined in the `broker/runtime.properties` fi
 |`druid.broker.select.tier`|`highestPriority`, `lowestPriority`, `custom`|If segments are cross-replicated across tiers in a cluster, you can tell the broker to prefer to select segments in a tier with a certain priority.|`highestPriority`|
 |`druid.broker.select.tier.custom.priorities`|`An array of integer priorities.`|Select servers in tiers with a custom priority list.|None|
 
+##### Query laning
+
+The Broker provides facilities to aid in query capacity reservation for heterogeneous query workloads in the form of 'laning' strategies, which provide a variety of mechanisms to examine and classify a query, assigning it to a 'lane'. Lanes are defined with capacity limits which the broker will enforce, causing requests in excess of the capacity to be discarded with an HTTP 429 status code, reserving resources for other lanes or for interactive queries (with no lane).
+
+|Property|Description|Default|
+|--------|-----------|-------|
+|`druid.query.scheduler.numThreads`|Maximum number of HTTP threads to dedicate to query processing. To save HTTP thread capacity, this should be lower than `druid.server.http.numThreads`.|Unbounded|
+|`druid.query.scheduler.laning.strategy`|Query laning strategy to use to assign queries to a lane in order to control capacities for certain classes of queries.|`none`|
+
+##### Laning strategies
+
+###### No laning strategy
+
+In this mode, queries are never assigned lane, and concurrent query count will only limited by `druid.server.http.numThreads` or `druid.query.scheduler.numThreads`, if set. This is the default Druid query scheduler operating mode. This strategy can also be explicitly enabled by setting `druid.query.scheduler.laning.strategy` to `none`.
 
 Review comment:
   ```suggestion
   In this mode, queries are never assigned a lane, and the concurrent query count will only be limited by `druid.server.http.numThreads` or `druid.query.scheduler.numThreads`, if set. This is the default Druid query scheduler operating mode. Enable this strategy explicitly by setting `druid.query.scheduler.laning.strategy` to `none`.
   ```

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] clintropolis commented on a change in pull request #9407: query laning and load shedding

Posted by GitBox <gi...@apache.org>.
clintropolis commented on a change in pull request #9407: query laning and load shedding
URL: https://github.com/apache/druid/pull/9407#discussion_r389166112
 
 

 ##########
 File path: docs/configuration/index.md
 ##########
 @@ -1476,9 +1476,31 @@ These Broker configurations can be defined in the `broker/runtime.properties` fi
 |`druid.broker.select.tier`|`highestPriority`, `lowestPriority`, `custom`|If segments are cross-replicated across tiers in a cluster, you can tell the broker to prefer to select segments in a tier with a certain priority.|`highestPriority`|
 |`druid.broker.select.tier.custom.priorities`|`An array of integer priorities.`|Select servers in tiers with a custom priority list.|None|
 
+##### Query laning
+
+The Broker provides facilities to aid in query capacity reservation for heterogeneous query workloads in the form of 'laning' strategies, which provide a variety of mechanisms to examine and classify a query, assigning it to a 'lane'. Lanes are defined with capacity limits which the broker will enforce, causing requests in excess of the capacity to be discarded with an HTTP 429 status code, reserving resources for other lanes or for interactive queries (with no lane).
+
+|Property|Description|Default|
+|--------|-----------|-------|
+|`druid.query.scheduler.numThreads`|Maximum number of HTTP threads to dedicate to query processing. To save HTTP thread capacity, this should be lower than `druid.server.http.numThreads`.|Unbounded|
+|`druid.query.scheduler.laning.type`|Query laning strategy to use to assign queries to a lane in order to control capacities for certain classes of queries.|`none`|
 
 Review comment:
   With regards to documentation flow, I had made the suggested change in a branch I intend to follow-up with: https://github.com/clintropolis/druid/compare/query-laning-and-load-shedding...clintropolis:query-auto-prioritization
   
   I wasn't sure if it made sense to break down yet in this branch because only the laning strategy exists, where as in that branch the scheduler also now has a prioritization strategy. I can go ahead and pull that part back of the doc change back into this PR though.

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] ccaominh commented on a change in pull request #9407: query laning and load shedding

Posted by GitBox <gi...@apache.org>.
ccaominh commented on a change in pull request #9407: query laning and load shedding
URL: https://github.com/apache/druid/pull/9407#discussion_r388659723
 
 

 ##########
 File path: server/src/main/java/org/apache/druid/server/QueryScheduler.java
 ##########
 @@ -0,0 +1,226 @@
+/*
+ * 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;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.collect.HashMultimap;
+import com.google.common.collect.Multimaps;
+import com.google.common.collect.SetMultimap;
+import com.google.common.util.concurrent.ListenableFuture;
+import io.github.resilience4j.bulkhead.Bulkhead;
+import io.github.resilience4j.bulkhead.BulkheadConfig;
+import io.github.resilience4j.bulkhead.BulkheadRegistry;
+import it.unimi.dsi.fastutil.objects.Object2IntMap;
+import org.apache.druid.client.SegmentServerSelector;
+import org.apache.druid.java.util.common.concurrent.Execs;
+import org.apache.druid.java.util.common.guava.Sequence;
+import org.apache.druid.query.Query;
+import org.apache.druid.query.QueryContexts;
+import org.apache.druid.query.QueryPlus;
+import org.apache.druid.query.QueryWatcher;
+
+import java.time.Duration;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+
+/**
+ * QueryScheduler (potentially) assigns any {@link Query} that is to be executed to a 'query lane' using the
+ * {@link QueryLaningStrategy} that is defined in {@link QuerySchedulerConfig}.
+ *
+ * As a {@link QueryWatcher}, it also provides cancellation facilities.
+ *
+ * This class is shared by all requests on the Jetty HTTP theadpool and must be thread safe.
+ */
+public class QueryScheduler implements QueryWatcher
+{
+  static final String TOTAL = "default";
+  private final QueryLaningStrategy laningStrategy;
+  private final BulkheadRegistry laneRegistry;
+  private final SetMultimap<String, ListenableFuture<?>> queryFutures;
+  private final SetMultimap<String, String> queryDatasources;
+
+  public QueryScheduler(int totalNumThreads, QueryLaningStrategy laningStrategy)
+  {
+    this.laningStrategy = laningStrategy;
+    this.laneRegistry = BulkheadRegistry.of(getLaneConfigs(totalNumThreads));
+    this.queryFutures = Multimaps.synchronizedSetMultimap(HashMultimap.create());
+    this.queryDatasources = Multimaps.synchronizedSetMultimap(HashMultimap.create());
+  }
+
+  @Override
+  public void registerQueryFuture(Query<?> query, ListenableFuture<?> future)
+  {
+    final String id = query.getId();
+    final Set<String> datasources = query.getDataSource().getTableNames();
+    queryFutures.put(id, future);
+    queryDatasources.putAll(id, datasources);
+    future.addListener(
+        () -> {
+          queryFutures.remove(id, future);
+          for (String datasource : datasources) {
+            queryDatasources.remove(id, datasource);
+          }
+        },
+        Execs.directExecutor()
+    );
+  }
+
+  /**
+   * Assign a query a lane (if not set)
+   */
+  public <T> Query<T> laneQuery(QueryPlus<T> queryPlus, Set<SegmentServerSelector> segments)
+  {
+    Query<T> query = queryPlus.getQuery();
+    // man wins over machine.. for now.
+    if (QueryContexts.getLane(query) != null) {
+      return query;
+    }
+    Optional<String> lane = laningStrategy.computeLane(queryPlus, segments);
+    return lane.map(query::withLane).orElse(query);
+  }
+
+  /**
+   * Run a query with the scheduler, attempting to acquire a semaphore from the total and lane specific query capacities
+   *
+   * Note that {@link #cancelQuery} should not interrupt the thread that calls run, in all current usages it only
+   * cancels any {@link ListenableFuture} created downstream. If this ever commonly changes, we should add
+   * synchronization between {@link #cancelQuery} and the acquisition of the {@link Bulkhead} to continue to ensure that
+   * anything acquired is also released.
+   *
+   * In the meantime, if a {@link ListenableFuture} is registered for the query that calls this method, it MUST handle
+   * this synchronization itself to ensure that no {@link Bulkhead} is acquired without releasing it.
+   */
+  public <T> Sequence<T> run(Query<?> query, Sequence<T> resultSequence)
+  {
+    List<Bulkhead> bulkheads = acquireLanes(query);
+    return resultSequence.withBaggage(() -> releaseLanes(bulkheads));
+  }
+
+  /**
+   * Forcibly cancel all futures that have been registered to a specific query id
+   */
+  public boolean cancelQuery(String id)
+  {
+    // if you re-use queryId and cancel queries... you are going to have a bad time
+    queryDatasources.removeAll(id);
+    Set<ListenableFuture<?>> futures = queryFutures.removeAll(id);
+    boolean success = true;
+    for (ListenableFuture<?> future : futures) {
+      success = success && future.cancel(true);
+    }
+    return success;
+  }
+
+  /**
+   * Get a {@link Set} of datasource names for a {@link Query} id, used by {@link QueryResource#cancelQuery} to
+   * authorize that a user may call {@link #cancelQuery} for the given id and datasources
+   */
+  public Set<String> getQueryDatasources(final String queryId)
+  {
+    return queryDatasources.get(queryId);
+  }
+
+  /**
+   * Get the maximum number of concurrent queries that {@link #run} can support
+   */
+  public int getTotalAvailableCapacity()
+  {
+    return laneRegistry.getConfiguration(TOTAL)
+                       .map(config -> laneRegistry.bulkhead(TOTAL, config).getMetrics().getAvailableConcurrentCalls())
+                       .orElse(-1);
+  }
+
+  /**
+   * Get the maximum number of concurrent queries that {@link #run} can support for a given lane
+   */
+  public int getLaneAvailableCapacity(String lane)
+  {
+    return laneRegistry.getConfiguration(lane)
+                       .map(config -> laneRegistry.bulkhead(lane, config).getMetrics().getAvailableConcurrentCalls())
+                       .orElse(-1);
+  }
+
+  /**
+   * Acquire a semaphore for both the 'total' and a lane, if any is associated with a query
+   */
+  @VisibleForTesting
+  List<Bulkhead> acquireLanes(Query<?> query)
+  {
+    final String lane = QueryContexts.getLane(query);
+    final Optional<BulkheadConfig> laneConfig = lane == null ? Optional.empty() : laneRegistry.getConfiguration(lane);
+    List<Bulkhead> hallPasses = new ArrayList<>(2);
+    final Optional<BulkheadConfig> totalConfig = laneRegistry.getConfiguration(TOTAL);
+    // if we have a lane, get it first
+    laneConfig.ifPresent(config -> {
+      Bulkhead laneLimiter = laneRegistry.bulkhead(lane, config);
+      if (!laneLimiter.tryAcquirePermission()) {
+        throw new QueryCapacityExceededException(lane);
+      }
+      hallPasses.add(laneLimiter);
+    });
+
+    // everyone needs to take one from the total lane; to ensure we don't acquire a lane and never release it, we want
+    // to check for total capacity exceeded and release the lane (if present) before throwing capacity exceeded
+    totalConfig.ifPresent(config -> {
+      Bulkhead totalLimiter = laneRegistry.bulkhead(TOTAL, config);
+      if (!totalLimiter.tryAcquirePermission()) {
+        releaseLanes(hallPasses);
 
 Review comment:
   May be better to do this in a exception handler in case there's an unexpected error between the two acquisitions. I don't think there's a test for this scenario?

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] clintropolis commented on a change in pull request #9407: query laning and load shedding

Posted by GitBox <gi...@apache.org>.
clintropolis commented on a change in pull request #9407: query laning and load shedding
URL: https://github.com/apache/druid/pull/9407#discussion_r391338145
 
 

 ##########
 File path: docs/configuration/index.md
 ##########
 @@ -1481,9 +1481,35 @@ These Broker configurations can be defined in the `broker/runtime.properties` fi
 |`druid.broker.select.tier`|`highestPriority`, `lowestPriority`, `custom`|If segments are cross-replicated across tiers in a cluster, you can tell the broker to prefer to select segments in a tier with a certain priority.|`highestPriority`|
 |`druid.broker.select.tier.custom.priorities`|`An array of integer priorities.`|Select servers in tiers with a custom priority list.|None|
 
+##### Query laning
+
+*Laning strategies* allow you to control capacity utilization for heterogeneous query workloads. With laning, the broker examines and classifies a query for the purpose of assigning it to a 'lane'. Lanes have capacity limits, enforced by the broker, that can be used to ensure sufficient resources are available for other lanes or for interactive queries (with no lane), or to limit overall throughput for queries within the lane. Requests in excess of the capacity are discarded with an HTTP 429 status code.
+
+|Property|Description|Default|
+|--------|-----------|-------|
+|`druid.query.scheduler.numThreads`|Maximum number of HTTP threads to dedicate to query processing. To save HTTP thread capacity, this should be lower than `druid.server.http.numThreads`.|Unbounded|
 
 Review comment:
   I wonder if it would make sense to instead move towards automatically computing `druid.server.http.numThreads`, since maybe it is easier for operators to only have to think about the number of concurrent queries to serve and just set `druid.query.scheduler.numThreads`? Druid could probably automatically figure out how many more http threads it needs based on configuration.
   
   >Major behavior change with lane usage is really losing the queuing of requests to handle spikes and instead sending 429s immediately. In future, we could introduce mechanism to maintain statically/dynamically sized [per lane] waiting queue ourselves as well along with concurrency limits in lane strategy.
   
   Yeah the current behavior is definitely a hard stop if you are over the line. I agree it would make sense to allow some sort of timed out queuing behavior, which is what jetty QoS filter can sort of provide, which is a large part of [why I am still wondering](https://github.com/apache/druid/pull/9407#discussion_r389159233) if `druid.query.scheduler.numThreads` should be a QoS filter instead of enforced as an actual lane like it is currently.

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] jihoonson commented on a change in pull request #9407: query laning and load shedding

Posted by GitBox <gi...@apache.org>.
jihoonson commented on a change in pull request #9407: query laning and load shedding
URL: https://github.com/apache/druid/pull/9407#discussion_r389176225
 
 

 ##########
 File path: server/src/main/java/org/apache/druid/server/QueryCapacityExceededException.java
 ##########
 @@ -0,0 +1,53 @@
+/*
+ * 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;
+
+import com.fasterxml.jackson.annotation.JsonCreator;
+import com.fasterxml.jackson.annotation.JsonProperty;
+import org.apache.druid.java.util.common.StringUtils;
+import org.apache.druid.query.QueryException;
+
+public class QueryCapacityExceededException extends QueryException
+{
+  private static final String ERROR_CLASS = QueryCapacityExceededException.class.getName();
+  public static final String ERROR_CODE = "Query capacity exceeded";
+  public static final String ERROR_MESSAGE = "Total query capacity exceeded";
+  public static final String ERROR_MESSAGE_TEMPLATE = "Query capacity exceeded for lane %s";
 
 Review comment:
   Maybe worth to say what configuration is related to this error. See https://github.com/apache/druid/blob/master/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/Groupers.java#L39 as an example.

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] clintropolis commented on issue #9407: query laning and load shedding

Posted by GitBox <gi...@apache.org>.
clintropolis commented on issue #9407: query laning and load shedding
URL: https://github.com/apache/druid/pull/9407#issuecomment-596037122
 
 
   >Are there any useful diagnostics (e.g., log messages) that can be added?
   
   These will be caught and logged as having errors in the request logger, which maybe seems sufficient for a first pass, but am willing to add additional information if we think it would be useful. I suppose periodic metrics on the amount of queries that get rejected could be potentially interesting in helping operators know when to either scale up or tell the users of certain lanes to chill out a bit maybe? I could look into that as a follow-up PR if we think that or something else would be useful.

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] jihoonson commented on a change in pull request #9407: query laning and load shedding

Posted by GitBox <gi...@apache.org>.
jihoonson commented on a change in pull request #9407: query laning and load shedding
URL: https://github.com/apache/druid/pull/9407#discussion_r389215278
 
 

 ##########
 File path: server/src/main/java/org/apache/druid/server/QueryCapacityExceededException.java
 ##########
 @@ -0,0 +1,53 @@
+/*
+ * 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;
+
+import com.fasterxml.jackson.annotation.JsonCreator;
+import com.fasterxml.jackson.annotation.JsonProperty;
+import org.apache.druid.java.util.common.StringUtils;
+import org.apache.druid.query.QueryException;
+
+public class QueryCapacityExceededException extends QueryException
+{
+  private static final String ERROR_CLASS = QueryCapacityExceededException.class.getName();
+  public static final String ERROR_CODE = "Query capacity exceeded";
+  public static final String ERROR_MESSAGE = "Total query capacity exceeded";
+  public static final String ERROR_MESSAGE_TEMPLATE = "Query capacity exceeded for lane %s";
 
 Review comment:
   That sounds good to me.

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] himanshug commented on a change in pull request #9407: query laning and load shedding

Posted by GitBox <gi...@apache.org>.
himanshug commented on a change in pull request #9407: query laning and load shedding
URL: https://github.com/apache/druid/pull/9407#discussion_r391343403
 
 

 ##########
 File path: docs/configuration/index.md
 ##########
 @@ -1481,9 +1481,35 @@ These Broker configurations can be defined in the `broker/runtime.properties` fi
 |`druid.broker.select.tier`|`highestPriority`, `lowestPriority`, `custom`|If segments are cross-replicated across tiers in a cluster, you can tell the broker to prefer to select segments in a tier with a certain priority.|`highestPriority`|
 |`druid.broker.select.tier.custom.priorities`|`An array of integer priorities.`|Select servers in tiers with a custom priority list.|None|
 
+##### Query laning
+
+*Laning strategies* allow you to control capacity utilization for heterogeneous query workloads. With laning, the broker examines and classifies a query for the purpose of assigning it to a 'lane'. Lanes have capacity limits, enforced by the broker, that can be used to ensure sufficient resources are available for other lanes or for interactive queries (with no lane), or to limit overall throughput for queries within the lane. Requests in excess of the capacity are discarded with an HTTP 429 status code.
+
+|Property|Description|Default|
+|--------|-----------|-------|
+|`druid.query.scheduler.numThreads`|Maximum number of HTTP threads to dedicate to query processing. To save HTTP thread capacity, this should be lower than `druid.server.http.numThreads`.|Unbounded|
 
 Review comment:
   It is fine to let user provide `druid.query.scheduler.numThreads` and compute `druid.server.http.numThreads` , just that one of those should not be touched by user in most cases.
   
   There are few advantages in maintaining the queues ourselves and not letting jetty do it.
   - we have no control over jetty queue, if a request is dropped then end  user sees that as a TCP connection close and not a HTTP 429. So, to client, it is not clear whether to retry or backoff.
   - we don't know how much time request waited  in jetty queue, consequently request time metrics don't account for that.
   - jetty queue is [probably] static in size, if we managed it ourselves then we have the option of keeping dynamically sized queues and do potentially other cool things.

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] clintropolis commented on a change in pull request #9407: query laning and load shedding

Posted by GitBox <gi...@apache.org>.
clintropolis commented on a change in pull request #9407: query laning and load shedding
URL: https://github.com/apache/druid/pull/9407#discussion_r389161093
 
 

 ##########
 File path: server/src/test/java/org/apache/druid/server/scheduling/HiLoQueryLaningStrategyTest.java
 ##########
 @@ -0,0 +1,108 @@
+/*
+ * 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.scheduling;
+
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.ImmutableMap;
+import com.google.common.collect.ImmutableSet;
+import it.unimi.dsi.fastutil.objects.Object2IntMap;
+import org.apache.druid.java.util.common.Intervals;
+import org.apache.druid.java.util.common.granularity.Granularities;
+import org.apache.druid.query.Druids;
+import org.apache.druid.query.QueryContexts;
+import org.apache.druid.query.QueryPlus;
+import org.apache.druid.query.aggregation.CountAggregatorFactory;
+import org.apache.druid.query.timeseries.TimeseriesQuery;
+import org.apache.druid.server.QueryLaningStrategy;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.ExpectedException;
+
+public class HiLoQueryLaningStrategyTest
+{
+  @Rule
+  public ExpectedException expectedException = ExpectedException.none();
+
+  private Druids.TimeseriesQueryBuilder queryBuilder;
+  private HiLoQueryLaningStrategy strategy;
+
+  @Before
+  public void setup()
+  {
+    this.queryBuilder = Druids.newTimeseriesQueryBuilder()
+                              .dataSource("test")
+                              .intervals(ImmutableList.of(Intervals.ETERNITY))
+                              .granularity(Granularities.DAY)
+                              .aggregators(new CountAggregatorFactory("count"));
+
+    this.strategy = new HiLoQueryLaningStrategy(10);
+  }
+
+  @Test
+  public void testMaxLowThreadsRequired()
+  {
+    expectedException.expect(NullPointerException.class);
+    expectedException.expectMessage("maxLowThreads must be set");
+    QueryLaningStrategy strategy = new HiLoQueryLaningStrategy(null);
+  }
+
+  @Test
+  public void testLaneLimits()
+  {
+    Object2IntMap<String> laneConfig = strategy.getLaneLimits();
+    Assert.assertEquals(1, laneConfig.size());
+    Assert.assertTrue(laneConfig.containsKey(HiLoQueryLaningStrategy.LOW));
+    Assert.assertEquals(10, laneConfig.getInt(HiLoQueryLaningStrategy.LOW));
+  }
+
+  @Test
+  public void testLaningNoPriority()
+  {
+    TimeseriesQuery query = queryBuilder.build();
+    Assert.assertFalse(strategy.computeLane(QueryPlus.wrap(query), ImmutableSet.of()).isPresent());
+  }
+
+  @Test
+  public void testLaningZeroPriority()
+  {
+    TimeseriesQuery query = queryBuilder.context(ImmutableMap.of(QueryContexts.PRIORITY_KEY, 0)).build();
+    Assert.assertFalse(strategy.computeLane(QueryPlus.wrap(query), ImmutableSet.of()).isPresent());
+  }
+
+  @Test
+  public void testLaningInteractivePriority()
+  {
+    TimeseriesQuery query = queryBuilder.context(ImmutableMap.of(QueryContexts.PRIORITY_KEY, 100)).build();
+    Assert.assertFalse(strategy.computeLane(QueryPlus.wrap(query), ImmutableSet.of()).isPresent());
+  }
+
+  @Test
+  public void testLaningLowPriority()
+  {
+    TimeseriesQuery query = queryBuilder.context(ImmutableMap.of(QueryContexts.PRIORITY_KEY, -1)).build();
+    Assert.assertTrue(strategy.computeLane(QueryPlus.wrap(query), ImmutableSet.of()).isPresent());
+    Assert.assertEquals(
+        HiLoQueryLaningStrategy.LOW,
+        strategy.computeLane(QueryPlus.wrap(query), ImmutableSet.of()).get()
+    );
+  }
+}
 
 Review comment:
   I think there are a few reasons to put it on the query context. For one to propagate the value downstream to historical and realtime tasks so they can utilize that information in the future to also make load management decisions. 
   
   But also, I'm not sure that lane necessarily _has_ to be related to priority just because the current implementation I have provided is. It is really just a label used to enforce limits. 
   
   In my list of potential follow-up work, I wanted to support a manual laning strategy that only supports explicit use defined lanes on the context [both to make integration tests easier](https://github.com/clintropolis/druid/compare/query-laning-and-load-shedding...clintropolis:manual-query-laning) but also just to account for scenarios where an external application drives these decisions and still wishes to enforce limits on classes of queries. I also suggested a tier based laning strategy, which would not be related to priority, but rather the set of servers it is going to be querying.

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] ccaominh commented on a change in pull request #9407: query laning and load shedding

Posted by GitBox <gi...@apache.org>.
ccaominh commented on a change in pull request #9407: query laning and load shedding
URL: https://github.com/apache/druid/pull/9407#discussion_r389072849
 
 

 ##########
 File path: server/src/test/java/org/apache/druid/server/scheduling/HiLoQueryLaningStrategyTest.java
 ##########
 @@ -0,0 +1,108 @@
+/*
+ * 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.scheduling;
+
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.ImmutableMap;
+import com.google.common.collect.ImmutableSet;
+import it.unimi.dsi.fastutil.objects.Object2IntMap;
+import org.apache.druid.java.util.common.Intervals;
+import org.apache.druid.java.util.common.granularity.Granularities;
+import org.apache.druid.query.Druids;
+import org.apache.druid.query.QueryContexts;
+import org.apache.druid.query.QueryPlus;
+import org.apache.druid.query.aggregation.CountAggregatorFactory;
+import org.apache.druid.query.timeseries.TimeseriesQuery;
+import org.apache.druid.server.QueryLaningStrategy;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.ExpectedException;
+
+public class HiLoQueryLaningStrategyTest
+{
+  @Rule
+  public ExpectedException expectedException = ExpectedException.none();
+
+  private Druids.TimeseriesQueryBuilder queryBuilder;
+  private HiLoQueryLaningStrategy strategy;
+
+  @Before
+  public void setup()
+  {
+    this.queryBuilder = Druids.newTimeseriesQueryBuilder()
+                              .dataSource("test")
+                              .intervals(ImmutableList.of(Intervals.ETERNITY))
+                              .granularity(Granularities.DAY)
+                              .aggregators(new CountAggregatorFactory("count"));
+
+    this.strategy = new HiLoQueryLaningStrategy(10);
+  }
+
+  @Test
+  public void testMaxLowThreadsRequired()
+  {
+    expectedException.expect(NullPointerException.class);
+    expectedException.expectMessage("maxLowThreads must be set");
+    QueryLaningStrategy strategy = new HiLoQueryLaningStrategy(null);
+  }
+
+  @Test
+  public void testLaneLimits()
+  {
+    Object2IntMap<String> laneConfig = strategy.getLaneLimits();
+    Assert.assertEquals(1, laneConfig.size());
+    Assert.assertTrue(laneConfig.containsKey(HiLoQueryLaningStrategy.LOW));
+    Assert.assertEquals(10, laneConfig.getInt(HiLoQueryLaningStrategy.LOW));
+  }
+
+  @Test
+  public void testLaningNoPriority()
+  {
+    TimeseriesQuery query = queryBuilder.build();
+    Assert.assertFalse(strategy.computeLane(QueryPlus.wrap(query), ImmutableSet.of()).isPresent());
+  }
+
+  @Test
+  public void testLaningZeroPriority()
+  {
+    TimeseriesQuery query = queryBuilder.context(ImmutableMap.of(QueryContexts.PRIORITY_KEY, 0)).build();
+    Assert.assertFalse(strategy.computeLane(QueryPlus.wrap(query), ImmutableSet.of()).isPresent());
+  }
+
+  @Test
+  public void testLaningInteractivePriority()
+  {
+    TimeseriesQuery query = queryBuilder.context(ImmutableMap.of(QueryContexts.PRIORITY_KEY, 100)).build();
+    Assert.assertFalse(strategy.computeLane(QueryPlus.wrap(query), ImmutableSet.of()).isPresent());
+  }
+
+  @Test
+  public void testLaningLowPriority()
+  {
+    TimeseriesQuery query = queryBuilder.context(ImmutableMap.of(QueryContexts.PRIORITY_KEY, -1)).build();
+    Assert.assertTrue(strategy.computeLane(QueryPlus.wrap(query), ImmutableSet.of()).isPresent());
+    Assert.assertEquals(
+        HiLoQueryLaningStrategy.LOW,
+        strategy.computeLane(QueryPlus.wrap(query), ImmutableSet.of()).get()
+    );
+  }
+}
 
 Review comment:
   Since the user can select a lane by setting the priority in the query context, I'm not sure of the benefit of also having a lane in the query context. Later, when there's automatic setting of the lane/priority, I think it should first respect the value in the query context if there's one before applying the automatic behavior.

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] ccaominh commented on a change in pull request #9407: query laning and load shedding

Posted by GitBox <gi...@apache.org>.
ccaominh commented on a change in pull request #9407: query laning and load shedding
URL: https://github.com/apache/druid/pull/9407#discussion_r387999148
 
 

 ##########
 File path: server/src/main/java/org/apache/druid/server/QueryCapacityExceededException.java
 ##########
 @@ -0,0 +1,53 @@
+/*
+ * 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;
+
+import com.fasterxml.jackson.annotation.JsonCreator;
+import com.fasterxml.jackson.annotation.JsonProperty;
+import org.apache.druid.java.util.common.StringUtils;
+import org.apache.druid.query.QueryException;
+
+public class QueryCapacityExceededException extends QueryException
 
 Review comment:
   There's some similarity between this and `QueryInterruptedException` and `ResourceLimitExceededException` and at first glance it may be unclear when to use which (perhaps mitigated by adding javadocs to this class). Is there a way to incorporate this into the same pattern in the `QueryInterruptedException` implementation?

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] clintropolis commented on a change in pull request #9407: query laning and load shedding

Posted by GitBox <gi...@apache.org>.
clintropolis commented on a change in pull request #9407: query laning and load shedding
URL: https://github.com/apache/druid/pull/9407#discussion_r390037318
 
 

 ##########
 File path: docs/configuration/index.md
 ##########
 @@ -1476,9 +1476,35 @@ These Broker configurations can be defined in the `broker/runtime.properties` fi
 |`druid.broker.select.tier`|`highestPriority`, `lowestPriority`, `custom`|If segments are cross-replicated across tiers in a cluster, you can tell the broker to prefer to select segments in a tier with a certain priority.|`highestPriority`|
 |`druid.broker.select.tier.custom.priorities`|`An array of integer priorities.`|Select servers in tiers with a custom priority list.|None|
 
+##### Query laning
+
+The Broker provides facilities to aid in query capacity reservation for heterogeneous query workloads in the form of 'laning' strategies, which provide a variety of mechanisms to examine and classify a query, assigning it to a 'lane'. Lanes are defined with capacity limits which the broker will enforce, causing requests in excess of the capacity to be discarded with an HTTP 429 status code, reserving resources for other lanes or for interactive queries (with no lane).
 
 Review comment:
   thanks, applied with some minor adjustment

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] clintropolis commented on a change in pull request #9407: query laning and load shedding

Posted by GitBox <gi...@apache.org>.
clintropolis commented on a change in pull request #9407: query laning and load shedding
URL: https://github.com/apache/druid/pull/9407#discussion_r388761914
 
 

 ##########
 File path: server/src/test/java/org/apache/druid/server/QuerySchedulerTest.java
 ##########
 @@ -0,0 +1,566 @@
+/*
+ * 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;
+
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.ImmutableMap;
+import com.google.common.collect.ImmutableSet;
+import com.google.common.util.concurrent.ListenableFuture;
+import com.google.common.util.concurrent.ListeningExecutorService;
+import com.google.common.util.concurrent.MoreExecutors;
+import com.google.inject.Injector;
+import io.github.resilience4j.bulkhead.Bulkhead;
+import org.apache.druid.guice.GuiceInjectors;
+import org.apache.druid.guice.JsonConfigProvider;
+import org.apache.druid.guice.JsonConfigurator;
+import org.apache.druid.guice.annotations.Global;
+import org.apache.druid.java.util.common.StringUtils;
+import org.apache.druid.java.util.common.concurrent.Execs;
+import org.apache.druid.java.util.common.guava.BaseSequence;
+import org.apache.druid.java.util.common.guava.LazySequence;
+import org.apache.druid.java.util.common.guava.Sequence;
+import org.apache.druid.java.util.common.guava.SequenceWrapper;
+import org.apache.druid.java.util.common.guava.Sequences;
+import org.apache.druid.java.util.common.guava.Yielder;
+import org.apache.druid.java.util.common.guava.Yielders;
+import org.apache.druid.query.Query;
+import org.apache.druid.query.QueryContexts;
+import org.apache.druid.query.QueryPlus;
+import org.apache.druid.query.aggregation.CountAggregatorFactory;
+import org.apache.druid.query.topn.TopNQuery;
+import org.apache.druid.query.topn.TopNQueryBuilder;
+import org.apache.druid.server.scheduling.HiLoQueryLaningStrategy;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.ExpectedException;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Properties;
+import java.util.UUID;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.Future;
+import java.util.concurrent.ThreadLocalRandom;
+import java.util.concurrent.atomic.AtomicLong;
+
+public class QuerySchedulerTest
+{
+  private static final int NUM_CONCURRENT_QUERIES = 10000;
+  private static final int NUM_ROWS = 10000;
+
+  @Rule
+  public ExpectedException expected = ExpectedException.none();
+
+  private ListeningExecutorService executorService;
+  private QueryScheduler scheduler;
+
+  private AtomicLong totalAcquired;
+  private AtomicLong totalReleased;
+  private AtomicLong laneAcquired;
+  private AtomicLong laneNotAcquired;
+  private AtomicLong laneReleased;
+
+  @Before
+  public void setup()
+  {
+    executorService = MoreExecutors.listeningDecorator(
+        Execs.multiThreaded(8, "test_query_scheduler_%s")
+    );
+    totalAcquired = new AtomicLong();
+    totalReleased = new AtomicLong();
+    laneAcquired = new AtomicLong();
+    laneNotAcquired = new AtomicLong();
+    laneReleased = new AtomicLong();
+    scheduler = new QueryScheduler(5, new HiLoQueryLaningStrategy(2)) {
+      @Override
+      List<Bulkhead> acquireLanes(Query<?> query)
+      {
+        List<Bulkhead> bulkheads = super.acquireLanes(query);
+        if (bulkheads.stream().anyMatch(b -> b.getName().equals(QueryScheduler.TOTAL))) {
+          totalAcquired.incrementAndGet();
+        }
+        if (bulkheads.stream().anyMatch(b -> !b.getName().equals(QueryScheduler.TOTAL))) {
+          laneAcquired.incrementAndGet();
+        }
+
+        return bulkheads;
+      }
+
+      @Override
+      void releaseLanes(List<Bulkhead> bulkheads)
+      {
+        super.releaseLanes(bulkheads);
+        if (bulkheads.stream().anyMatch(b -> b.getName().equals(QueryScheduler.TOTAL))) {
+          totalReleased.incrementAndGet();
+        }
+        if (bulkheads.stream().anyMatch(b -> !b.getName().equals(QueryScheduler.TOTAL))) {
+          laneReleased.incrementAndGet();
+          if (bulkheads.size() == 1) {
+            laneNotAcquired.incrementAndGet();
+          }
+        }
+      }
+    };
+  }
+
+  @After
+  public void teardown()
+  {
+    executorService.shutdownNow();
+  }
+
+  @Test
+  public void testHiLoHi() throws ExecutionException, InterruptedException
+  {
+    TopNQuery interactive = makeInteractiveQuery();
+    ListenableFuture<?> future = executorService.submit(() -> {
+      try {
+        Query<?> scheduled = scheduler.laneQuery(QueryPlus.wrap(interactive), ImmutableSet.of());
+
+        Assert.assertNotNull(scheduled);
+
+        Sequence<Integer> underlyingSequence = makeSequence(10);
+        underlyingSequence = Sequences.wrap(underlyingSequence, new SequenceWrapper()
+        {
+          @Override
+          public void before()
+          {
+            Assert.assertEquals(4, scheduler.getTotalAvailableCapacity());
+            Assert.assertEquals(2, scheduler.getLaneAvailableCapacity(HiLoQueryLaningStrategy.LOW));
+          }
+        });
+        Sequence<Integer> results = scheduler.run(scheduled, underlyingSequence);
+        int rowCount = consumeAndCloseSequence(results);
+
+        Assert.assertEquals(10, rowCount);
+      }
+      catch (IOException ex) {
+        throw new RuntimeException(ex);
+      }
+    });
+    future.get();
+    Assert.assertEquals(5, scheduler.getTotalAvailableCapacity());
+  }
+
+  @Test
+  public void testHiLoLo() throws ExecutionException, InterruptedException
+  {
+    TopNQuery report = makeReportQuery();
+    ListenableFuture<?> future = executorService.submit(() -> {
+      try {
+        Query<?> scheduledReport = scheduler.laneQuery(QueryPlus.wrap(report), ImmutableSet.of());
+        Assert.assertNotNull(scheduledReport);
+        Assert.assertEquals(HiLoQueryLaningStrategy.LOW, QueryContexts.getLane(scheduledReport));
+
+        Sequence<Integer> underlyingSequence = makeSequence(10);
+        underlyingSequence = Sequences.wrap(underlyingSequence, new SequenceWrapper()
+        {
+          @Override
+          public void before()
+          {
+            Assert.assertEquals(4, scheduler.getTotalAvailableCapacity());
+            Assert.assertEquals(1, scheduler.getLaneAvailableCapacity(HiLoQueryLaningStrategy.LOW));
+          }
+        });
+        Sequence<Integer> results = scheduler.run(scheduledReport, underlyingSequence);
+
+        int rowCount = consumeAndCloseSequence(results);
+        Assert.assertEquals(10, rowCount);
+      }
+      catch (IOException ex) {
+        throw new RuntimeException(ex);
+      }
+    });
+    future.get();
+    Assert.assertEquals(5, scheduler.getTotalAvailableCapacity());
+    Assert.assertEquals(2, scheduler.getLaneAvailableCapacity(HiLoQueryLaningStrategy.LOW));
+  }
+
+  @Test
+  public void testHiLoReleaseSemaphoreWhenSequenceExplodes() throws Exception
+  {
+    expected.expectMessage("exploded");
+    expected.expect(ExecutionException.class);
+    TopNQuery interactive = makeInteractiveQuery();
+    ListenableFuture<?> future = executorService.submit(() -> {
+      try {
+        Query<?> scheduled = scheduler.laneQuery(QueryPlus.wrap(interactive), ImmutableSet.of());
+
+        Assert.assertNotNull(scheduled);
+
+        Sequence<Integer> underlyingSequence = makeExplodingSequence(10);
+        underlyingSequence = Sequences.wrap(underlyingSequence, new SequenceWrapper()
+        {
+          @Override
+          public void before()
+          {
+            Assert.assertEquals(4, scheduler.getTotalAvailableCapacity());
+          }
+        });
+        Sequence<Integer> results = scheduler.run(scheduled, underlyingSequence);
+
+        consumeAndCloseSequence(results);
+      }
+      catch (IOException ex) {
+        throw new RuntimeException(ex);
+      }
+    });
+    future.get();
+    Assert.assertEquals(5, scheduler.getTotalAvailableCapacity());
 
 Review comment:
   Ah, i think that was a leftover from copying another test to make this one, but I think it does seem useful to assert that the token was released after the exception so will change

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] clintropolis commented on a change in pull request #9407: query laning and load shedding

Posted by GitBox <gi...@apache.org>.
clintropolis commented on a change in pull request #9407: query laning and load shedding
URL: https://github.com/apache/druid/pull/9407#discussion_r389219232
 
 

 ##########
 File path: server/src/main/java/org/apache/druid/server/QueryLaningStrategy.java
 ##########
 @@ -0,0 +1,51 @@
+/*
+ * 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;
+
+import com.fasterxml.jackson.annotation.JsonSubTypes;
+import com.fasterxml.jackson.annotation.JsonTypeInfo;
+import it.unimi.dsi.fastutil.objects.Object2IntMap;
+import org.apache.druid.client.SegmentServerSelector;
+import org.apache.druid.query.QueryPlus;
+import org.apache.druid.server.scheduling.HiLoQueryLaningStrategy;
+import org.apache.druid.server.scheduling.NoQueryLaningStrategy;
+
+import java.util.Optional;
+import java.util.Set;
+
+
+@JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "strategy", defaultImpl = NoQueryLaningStrategy.class)
+@JsonSubTypes(value = {
+    @JsonSubTypes.Type(name = "none", value = NoQueryLaningStrategy.class),
+    @JsonSubTypes.Type(name = "hilo", value = HiLoQueryLaningStrategy.class)
+})
+public interface QueryLaningStrategy
+{
+  /**
+   * Provide map of lane names to the limit on the number of concurrent queries for that lane
+   * @param totalLimit
+   */
+  Object2IntMap<String> getLaneLimits(int totalLimit);
+
+  /**
+   * For a given {@link QueryPlus} and set of {@link SegmentServerSelector}, compute if a query belongs to a lane
+   */
+  <T> Optional<String> computeLane(QueryPlus<T> query, Set<SegmentServerSelector> segments);
 
 Review comment:
   added

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] jihoonson commented on a change in pull request #9407: query laning and load shedding

Posted by GitBox <gi...@apache.org>.
jihoonson commented on a change in pull request #9407: query laning and load shedding
URL: https://github.com/apache/druid/pull/9407#discussion_r389157507
 
 

 ##########
 File path: server/src/main/java/org/apache/druid/server/QueryScheduler.java
 ##########
 @@ -0,0 +1,241 @@
+/*
+ * 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;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.collect.HashMultimap;
+import com.google.common.collect.Multimaps;
+import com.google.common.collect.SetMultimap;
+import com.google.common.util.concurrent.ListenableFuture;
+import io.github.resilience4j.bulkhead.Bulkhead;
+import io.github.resilience4j.bulkhead.BulkheadConfig;
+import io.github.resilience4j.bulkhead.BulkheadRegistry;
+import it.unimi.dsi.fastutil.objects.Object2IntMap;
+import org.apache.druid.client.SegmentServerSelector;
+import org.apache.druid.java.util.common.concurrent.Execs;
+import org.apache.druid.java.util.common.guava.Sequence;
+import org.apache.druid.query.Query;
+import org.apache.druid.query.QueryContexts;
+import org.apache.druid.query.QueryPlus;
+import org.apache.druid.query.QueryWatcher;
+import org.apache.druid.server.initialization.ServerConfig;
+
+import java.time.Duration;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+
+/**
+ * QueryScheduler (potentially) assigns any {@link Query} that is to be executed to a 'query lane' using the
+ * {@link QueryLaningStrategy} that is defined in {@link QuerySchedulerConfig}.
+ *
+ * As a {@link QueryWatcher}, it also provides cancellation facilities.
+ *
+ * This class is shared by all requests on the Jetty HTTP theadpool and must be thread safe.
+ */
+public class QueryScheduler implements QueryWatcher
+{
+  static final String TOTAL = "default";
+  private final int totalCapacity;
+  private final QueryLaningStrategy laningStrategy;
+  private final BulkheadRegistry laneRegistry;
+  private final SetMultimap<String, ListenableFuture<?>> queryFutures;
+  private final SetMultimap<String, String> queryDatasources;
+
+  public QueryScheduler(int totalNumThreads, QueryLaningStrategy laningStrategy, ServerConfig serverConfig)
+  {
+    this.laningStrategy = laningStrategy;
+    this.queryFutures = Multimaps.synchronizedSetMultimap(HashMultimap.create());
+    this.queryDatasources = Multimaps.synchronizedSetMultimap(HashMultimap.create());
+    // if totalNumThreads is above 0 and less than druid.server.http.numThreads, enforce total limit
+    final boolean limitTotal;
+    if (totalNumThreads > 0 && totalNumThreads < serverConfig.getNumThreads()) {
+      limitTotal = true;
+      this.totalCapacity = totalNumThreads;
+    } else {
+      limitTotal = false;
+      this.totalCapacity = serverConfig.getNumThreads();
+    }
+    this.laneRegistry = BulkheadRegistry.of(getLaneConfigs(limitTotal));
+  }
+
+  @Override
+  public void registerQueryFuture(Query<?> query, ListenableFuture<?> future)
+  {
+    final String id = query.getId();
+    final Set<String> datasources = query.getDataSource().getTableNames();
+    queryFutures.put(id, future);
+    queryDatasources.putAll(id, datasources);
+    future.addListener(
+        () -> {
+          queryFutures.remove(id, future);
+          for (String datasource : datasources) {
+            queryDatasources.remove(id, datasource);
+          }
+        },
+        Execs.directExecutor()
+    );
+  }
+
+  /**
+   * Assign a query a lane (if not set)
+   */
+  public <T> Query<T> laneQuery(QueryPlus<T> queryPlus, Set<SegmentServerSelector> segments)
+  {
+    Query<T> query = queryPlus.getQuery();
+    // man wins over machine.. for now.
+    if (QueryContexts.getLane(query) != null) {
+      return query;
+    }
+    Optional<String> lane = laningStrategy.computeLane(queryPlus, segments);
+    return lane.map(query::withLane).orElse(query);
+  }
+
+  /**
+   * Run a query with the scheduler, attempting to acquire a semaphore from the total and lane specific query capacities
+   *
+   * Note that {@link #cancelQuery} should not interrupt the thread that calls run, in all current usages it only
+   * cancels any {@link ListenableFuture} created downstream. If this ever commonly changes, we should add
+   * synchronization between {@link #cancelQuery} and the acquisition of the {@link Bulkhead} to continue to ensure that
+   * anything acquired is also released.
+   *
+   * In the meantime, if a {@link ListenableFuture} is registered for the query that calls this method, it MUST handle
+   * this synchronization itself to ensure that no {@link Bulkhead} is acquired without releasing it.
+   */
+  public <T> Sequence<T> run(Query<?> query, Sequence<T> resultSequence)
+  {
+    List<Bulkhead> bulkheads = acquireLanes(query);
+    return resultSequence.withBaggage(() -> releaseLanes(bulkheads));
+  }
+
+  /**
+   * Forcibly cancel all futures that have been registered to a specific query id
+   */
+  public boolean cancelQuery(String id)
+  {
+    // if you re-use queryId and cancel queries... you are going to have a bad time
 
 Review comment:
   Oh nvm, I thought that the historical can register multiple futures per segment for the same query which is not true. They always register only one future for one query.

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] ccaominh commented on a change in pull request #9407: query laning and load shedding

Posted by GitBox <gi...@apache.org>.
ccaominh commented on a change in pull request #9407: query laning and load shedding
URL: https://github.com/apache/druid/pull/9407#discussion_r389956045
 
 

 ##########
 File path: server/src/main/java/org/apache/druid/server/scheduling/HiLoQueryLaningStrategy.java
 ##########
 @@ -0,0 +1,79 @@
+/*
+ * 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.scheduling;
+
+import com.fasterxml.jackson.annotation.JsonCreator;
+import com.fasterxml.jackson.annotation.JsonProperty;
+import com.google.common.base.Preconditions;
+import com.google.common.primitives.Ints;
+import it.unimi.dsi.fastutil.objects.Object2IntArrayMap;
+import it.unimi.dsi.fastutil.objects.Object2IntMap;
+import org.apache.druid.client.SegmentServerSelector;
+import org.apache.druid.query.Query;
+import org.apache.druid.query.QueryContexts;
+import org.apache.druid.query.QueryPlus;
+import org.apache.druid.server.QueryLaningStrategy;
+
+import java.util.Optional;
+import java.util.Set;
+
+/**
+ * Query laning strategy which associates all {@link Query} with priority lower than 0 into a 'low' lane
+ */
+public class HiLoQueryLaningStrategy implements QueryLaningStrategy
+{
+  public static final String LOW = "low";
+
+  @JsonProperty
+  private final int maxLowPercent;
+
+  @JsonCreator
+  public HiLoQueryLaningStrategy(
+      @JsonProperty("maxLowPercent") Integer maxLowPercent
+  )
+  {
+    this.maxLowPercent = Preconditions.checkNotNull(maxLowPercent, "maxLowPercent must be set");
+    Preconditions.checkArgument(
+        0 < maxLowPercent && maxLowPercent < 100,
+        "maxLowPercent must be between 0 and 100"
+    );
+  }
+
+  @Override
+  public Object2IntMap<String> getLaneLimits(int totalLimit)
+  {
+    Object2IntMap<String> onlyLow = new Object2IntArrayMap<>(1);
+    onlyLow.put(LOW, Ints.checkedCast(Math.round(totalLimit * ((double) maxLowPercent / 100))));
 
 Review comment:
   Behavior here is a bit different from the docs, which say the value is rounded up.
   
   Since this allows a lane limit of zero or the total limit, it is a bit inconsistent with disallowing `maxLowPercent` from being 0 or 100 percent.

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] clintropolis commented on a change in pull request #9407: query laning and load shedding

Posted by GitBox <gi...@apache.org>.
clintropolis commented on a change in pull request #9407: query laning and load shedding
URL: https://github.com/apache/druid/pull/9407#discussion_r388759220
 
 

 ##########
 File path: server/src/main/java/org/apache/druid/server/QueryScheduler.java
 ##########
 @@ -0,0 +1,226 @@
+/*
+ * 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;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.collect.HashMultimap;
+import com.google.common.collect.Multimaps;
+import com.google.common.collect.SetMultimap;
+import com.google.common.util.concurrent.ListenableFuture;
+import io.github.resilience4j.bulkhead.Bulkhead;
+import io.github.resilience4j.bulkhead.BulkheadConfig;
+import io.github.resilience4j.bulkhead.BulkheadRegistry;
+import it.unimi.dsi.fastutil.objects.Object2IntMap;
+import org.apache.druid.client.SegmentServerSelector;
+import org.apache.druid.java.util.common.concurrent.Execs;
+import org.apache.druid.java.util.common.guava.Sequence;
+import org.apache.druid.query.Query;
+import org.apache.druid.query.QueryContexts;
+import org.apache.druid.query.QueryPlus;
+import org.apache.druid.query.QueryWatcher;
+
+import java.time.Duration;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+
+/**
+ * QueryScheduler (potentially) assigns any {@link Query} that is to be executed to a 'query lane' using the
+ * {@link QueryLaningStrategy} that is defined in {@link QuerySchedulerConfig}.
+ *
+ * As a {@link QueryWatcher}, it also provides cancellation facilities.
+ *
+ * This class is shared by all requests on the Jetty HTTP theadpool and must be thread safe.
+ */
+public class QueryScheduler implements QueryWatcher
+{
+  static final String TOTAL = "default";
+  private final QueryLaningStrategy laningStrategy;
+  private final BulkheadRegistry laneRegistry;
+  private final SetMultimap<String, ListenableFuture<?>> queryFutures;
+  private final SetMultimap<String, String> queryDatasources;
+
+  public QueryScheduler(int totalNumThreads, QueryLaningStrategy laningStrategy)
+  {
+    this.laningStrategy = laningStrategy;
+    this.laneRegistry = BulkheadRegistry.of(getLaneConfigs(totalNumThreads));
+    this.queryFutures = Multimaps.synchronizedSetMultimap(HashMultimap.create());
+    this.queryDatasources = Multimaps.synchronizedSetMultimap(HashMultimap.create());
+  }
+
+  @Override
+  public void registerQueryFuture(Query<?> query, ListenableFuture<?> future)
+  {
+    final String id = query.getId();
+    final Set<String> datasources = query.getDataSource().getTableNames();
+    queryFutures.put(id, future);
+    queryDatasources.putAll(id, datasources);
+    future.addListener(
+        () -> {
+          queryFutures.remove(id, future);
+          for (String datasource : datasources) {
+            queryDatasources.remove(id, datasource);
+          }
+        },
+        Execs.directExecutor()
+    );
+  }
+
+  /**
+   * Assign a query a lane (if not set)
+   */
+  public <T> Query<T> laneQuery(QueryPlus<T> queryPlus, Set<SegmentServerSelector> segments)
+  {
+    Query<T> query = queryPlus.getQuery();
+    // man wins over machine.. for now.
+    if (QueryContexts.getLane(query) != null) {
+      return query;
+    }
+    Optional<String> lane = laningStrategy.computeLane(queryPlus, segments);
+    return lane.map(query::withLane).orElse(query);
+  }
+
+  /**
+   * Run a query with the scheduler, attempting to acquire a semaphore from the total and lane specific query capacities
+   *
+   * Note that {@link #cancelQuery} should not interrupt the thread that calls run, in all current usages it only
+   * cancels any {@link ListenableFuture} created downstream. If this ever commonly changes, we should add
+   * synchronization between {@link #cancelQuery} and the acquisition of the {@link Bulkhead} to continue to ensure that
+   * anything acquired is also released.
+   *
+   * In the meantime, if a {@link ListenableFuture} is registered for the query that calls this method, it MUST handle
+   * this synchronization itself to ensure that no {@link Bulkhead} is acquired without releasing it.
+   */
+  public <T> Sequence<T> run(Query<?> query, Sequence<T> resultSequence)
+  {
+    List<Bulkhead> bulkheads = acquireLanes(query);
+    return resultSequence.withBaggage(() -> releaseLanes(bulkheads));
+  }
+
+  /**
+   * Forcibly cancel all futures that have been registered to a specific query id
+   */
+  public boolean cancelQuery(String id)
+  {
+    // if you re-use queryId and cancel queries... you are going to have a bad time
+    queryDatasources.removeAll(id);
+    Set<ListenableFuture<?>> futures = queryFutures.removeAll(id);
+    boolean success = true;
+    for (ListenableFuture<?> future : futures) {
+      success = success && future.cancel(true);
+    }
+    return success;
+  }
+
+  /**
+   * Get a {@link Set} of datasource names for a {@link Query} id, used by {@link QueryResource#cancelQuery} to
+   * authorize that a user may call {@link #cancelQuery} for the given id and datasources
+   */
+  public Set<String> getQueryDatasources(final String queryId)
+  {
+    return queryDatasources.get(queryId);
+  }
+
+  /**
+   * Get the maximum number of concurrent queries that {@link #run} can support
+   */
+  public int getTotalAvailableCapacity()
+  {
+    return laneRegistry.getConfiguration(TOTAL)
+                       .map(config -> laneRegistry.bulkhead(TOTAL, config).getMetrics().getAvailableConcurrentCalls())
+                       .orElse(-1);
+  }
+
+  /**
+   * Get the maximum number of concurrent queries that {@link #run} can support for a given lane
+   */
+  public int getLaneAvailableCapacity(String lane)
+  {
+    return laneRegistry.getConfiguration(lane)
+                       .map(config -> laneRegistry.bulkhead(lane, config).getMetrics().getAvailableConcurrentCalls())
+                       .orElse(-1);
+  }
+
+  /**
+   * Acquire a semaphore for both the 'total' and a lane, if any is associated with a query
+   */
+  @VisibleForTesting
+  List<Bulkhead> acquireLanes(Query<?> query)
+  {
+    final String lane = QueryContexts.getLane(query);
+    final Optional<BulkheadConfig> laneConfig = lane == null ? Optional.empty() : laneRegistry.getConfiguration(lane);
+    List<Bulkhead> hallPasses = new ArrayList<>(2);
+    final Optional<BulkheadConfig> totalConfig = laneRegistry.getConfiguration(TOTAL);
+    // if we have a lane, get it first
+    laneConfig.ifPresent(config -> {
+      Bulkhead laneLimiter = laneRegistry.bulkhead(lane, config);
+      if (!laneLimiter.tryAcquirePermission()) {
+        throw new QueryCapacityExceededException(lane);
+      }
+      hallPasses.add(laneLimiter);
+    });
+
+    // everyone needs to take one from the total lane; to ensure we don't acquire a lane and never release it, we want
+    // to check for total capacity exceeded and release the lane (if present) before throwing capacity exceeded
+    totalConfig.ifPresent(config -> {
+      Bulkhead totalLimiter = laneRegistry.bulkhead(TOTAL, config);
+      if (!totalLimiter.tryAcquirePermission()) {
+        releaseLanes(hallPasses);
 
 Review comment:
   Hmm, I couldn't find an exception that would be thrown in this block, `tryAcquirePermission` eats `InterruptedException` and also the timeout is configured to 0 so the underlying semaphore call takes a code path that shouldn't explode, so I imagined any exception that happens where would be indicative of a catastrophic failure or the jetty server shutting down. 
   
   That said, I guess it doesn't harm much to wrap in a try catch and rethrow after releasing.
   
   This isn't explicitly tested for, but a shadow of it being tested is visible in the concurrency tests which flex the case where a lane was acquired but the total could not be in [this assert that the 'lane released' count is equal to the 'lane acquired' + 'lane not acquired' count](https://github.com/apache/druid/pull/9407/files#diff-3b8f9c560c324b6f177ec84f50c3c856R551) which fails if we don't consider the 'lane not acquired' count.
   

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] jihoonson commented on a change in pull request #9407: query laning and load shedding

Posted by GitBox <gi...@apache.org>.
jihoonson commented on a change in pull request #9407: query laning and load shedding
URL: https://github.com/apache/druid/pull/9407#discussion_r389124897
 
 

 ##########
 File path: server/src/main/java/org/apache/druid/server/QueryScheduler.java
 ##########
 @@ -0,0 +1,241 @@
+/*
+ * 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;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.collect.HashMultimap;
+import com.google.common.collect.Multimaps;
+import com.google.common.collect.SetMultimap;
+import com.google.common.util.concurrent.ListenableFuture;
+import io.github.resilience4j.bulkhead.Bulkhead;
+import io.github.resilience4j.bulkhead.BulkheadConfig;
+import io.github.resilience4j.bulkhead.BulkheadRegistry;
+import it.unimi.dsi.fastutil.objects.Object2IntMap;
+import org.apache.druid.client.SegmentServerSelector;
+import org.apache.druid.java.util.common.concurrent.Execs;
+import org.apache.druid.java.util.common.guava.Sequence;
+import org.apache.druid.query.Query;
+import org.apache.druid.query.QueryContexts;
+import org.apache.druid.query.QueryPlus;
+import org.apache.druid.query.QueryWatcher;
+import org.apache.druid.server.initialization.ServerConfig;
+
+import java.time.Duration;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+
+/**
+ * QueryScheduler (potentially) assigns any {@link Query} that is to be executed to a 'query lane' using the
+ * {@link QueryLaningStrategy} that is defined in {@link QuerySchedulerConfig}.
+ *
+ * As a {@link QueryWatcher}, it also provides cancellation facilities.
+ *
+ * This class is shared by all requests on the Jetty HTTP theadpool and must be thread safe.
+ */
+public class QueryScheduler implements QueryWatcher
+{
+  static final String TOTAL = "default";
+  private final int totalCapacity;
+  private final QueryLaningStrategy laningStrategy;
+  private final BulkheadRegistry laneRegistry;
+  private final SetMultimap<String, ListenableFuture<?>> queryFutures;
+  private final SetMultimap<String, String> queryDatasources;
+
+  public QueryScheduler(int totalNumThreads, QueryLaningStrategy laningStrategy, ServerConfig serverConfig)
+  {
+    this.laningStrategy = laningStrategy;
+    this.queryFutures = Multimaps.synchronizedSetMultimap(HashMultimap.create());
+    this.queryDatasources = Multimaps.synchronizedSetMultimap(HashMultimap.create());
+    // if totalNumThreads is above 0 and less than druid.server.http.numThreads, enforce total limit
+    final boolean limitTotal;
+    if (totalNumThreads > 0 && totalNumThreads < serverConfig.getNumThreads()) {
+      limitTotal = true;
+      this.totalCapacity = totalNumThreads;
+    } else {
+      limitTotal = false;
+      this.totalCapacity = serverConfig.getNumThreads();
+    }
+    this.laneRegistry = BulkheadRegistry.of(getLaneConfigs(limitTotal));
+  }
+
+  @Override
+  public void registerQueryFuture(Query<?> query, ListenableFuture<?> future)
+  {
+    final String id = query.getId();
+    final Set<String> datasources = query.getDataSource().getTableNames();
+    queryFutures.put(id, future);
+    queryDatasources.putAll(id, datasources);
+    future.addListener(
+        () -> {
+          queryFutures.remove(id, future);
+          for (String datasource : datasources) {
+            queryDatasources.remove(id, datasource);
+          }
+        },
+        Execs.directExecutor()
+    );
+  }
+
+  /**
+   * Assign a query a lane (if not set)
+   */
+  public <T> Query<T> laneQuery(QueryPlus<T> queryPlus, Set<SegmentServerSelector> segments)
+  {
+    Query<T> query = queryPlus.getQuery();
+    // man wins over machine.. for now.
+    if (QueryContexts.getLane(query) != null) {
+      return query;
+    }
+    Optional<String> lane = laningStrategy.computeLane(queryPlus, segments);
+    return lane.map(query::withLane).orElse(query);
+  }
+
+  /**
+   * Run a query with the scheduler, attempting to acquire a semaphore from the total and lane specific query capacities
+   *
+   * Note that {@link #cancelQuery} should not interrupt the thread that calls run, in all current usages it only
+   * cancels any {@link ListenableFuture} created downstream. If this ever commonly changes, we should add
+   * synchronization between {@link #cancelQuery} and the acquisition of the {@link Bulkhead} to continue to ensure that
+   * anything acquired is also released.
+   *
+   * In the meantime, if a {@link ListenableFuture} is registered for the query that calls this method, it MUST handle
+   * this synchronization itself to ensure that no {@link Bulkhead} is acquired without releasing it.
+   */
+  public <T> Sequence<T> run(Query<?> query, Sequence<T> resultSequence)
+  {
+    List<Bulkhead> bulkheads = acquireLanes(query);
+    return resultSequence.withBaggage(() -> releaseLanes(bulkheads));
+  }
+
+  /**
+   * Forcibly cancel all futures that have been registered to a specific query id
+   */
+  public boolean cancelQuery(String id)
+  {
+    // if you re-use queryId and cancel queries... you are going to have a bad time
+    queryDatasources.removeAll(id);
+    Set<ListenableFuture<?>> futures = queryFutures.removeAll(id);
+    boolean success = true;
+    for (ListenableFuture<?> future : futures) {
+      success = success && future.cancel(true);
+    }
+    return success;
+  }
+
+  /**
+   * Get a {@link Set} of datasource names for a {@link Query} id, used by {@link QueryResource#cancelQuery} to
+   * authorize that a user may call {@link #cancelQuery} for the given id and datasources
+   */
+  public Set<String> getQueryDatasources(final String queryId)
+  {
+    return queryDatasources.get(queryId);
+  }
+
+  /**
+   * Get the maximum number of concurrent queries that {@link #run} can support
+   */
+  @VisibleForTesting
+  int getTotalAvailableCapacity()
+  {
+    return laneRegistry.getConfiguration(TOTAL)
+                       .map(config -> laneRegistry.bulkhead(TOTAL, config).getMetrics().getAvailableConcurrentCalls())
+                       .orElse(-1);
+  }
+
+  /**
+   * Get the maximum number of concurrent queries that {@link #run} can support for a given lane
+   */
+  @VisibleForTesting
+  int getLaneAvailableCapacity(String lane)
+  {
+    return laneRegistry.getConfiguration(lane)
+                       .map(config -> laneRegistry.bulkhead(lane, config).getMetrics().getAvailableConcurrentCalls())
+                       .orElse(-1);
+  }
+
+  /**
+   * Acquire a semaphore for both the 'total' and a lane, if any is associated with a query
+   */
+  @VisibleForTesting
+  List<Bulkhead> acquireLanes(Query<?> query)
+  {
+    final String lane = QueryContexts.getLane(query);
+    final Optional<BulkheadConfig> laneConfig = lane == null ? Optional.empty() : laneRegistry.getConfiguration(lane);
+    List<Bulkhead> hallPasses = new ArrayList<>(2);
+    final Optional<BulkheadConfig> totalConfig = laneRegistry.getConfiguration(TOTAL);
+    // if we have a lane, get it first
+    laneConfig.ifPresent(config -> {
+      Bulkhead laneLimiter = laneRegistry.bulkhead(lane, config);
+      if (!laneLimiter.tryAcquirePermission()) {
+        throw new QueryCapacityExceededException(lane);
+      }
+      hallPasses.add(laneLimiter);
+    });
+
+    // everyone needs to take one from the total lane; to ensure we don't acquire a lane and never release it, we want
+    // to check for total capacity exceeded and release the lane (if present) before throwing capacity exceeded
+    totalConfig.ifPresent(config -> {
+      Bulkhead totalLimiter = laneRegistry.bulkhead(TOTAL, config);
+      if (!totalLimiter.tryAcquirePermission()) {
+        releaseLanes(hallPasses);
+        throw new QueryCapacityExceededException();
+      }
+      hallPasses.add(totalLimiter);
+    });
+    return hallPasses;
+  }
+
+  /**
+   * Release all {@link Bulkhead} semaphores in the list
+   */
+  @VisibleForTesting
+  void releaseLanes(List<Bulkhead> bulkheads)
+  {
+    bulkheads.forEach(Bulkhead::releasePermission);
 
 Review comment:
   nit: the javadoc of `releasePermission` says:
   
   ```java
       /**
        * Releases a permission and increases the number of available permits by one.
        * <p>
        * Should only be used when a permission was acquired but not used. Otherwise use {@link
        * Bulkhead#onComplete()} to signal a completed call and release a permission.
        */
       void releasePermission();
   ```
   
   `releasePermission()` and `onComplete()` of `SemaphoreBulkhead` are:
   
   ```java
       @Override
       public void releasePermission() {
           semaphore.release();
       }
   
       @Override
       public void onComplete() {
           semaphore.release();
           publishBulkheadEvent(() -> new BulkheadOnCallFinishedEvent(name));
       }
   ```
   
   I think it's safe to call `releasePermission()` for now since we don't use the `EventPublisher`, but it still wouldn't harm to use `onComplete()`. It would also probably be better to avoid potential bugs if we want to use `EventPublisher` in the future.

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] clintropolis commented on a change in pull request #9407: query laning and load shedding

Posted by GitBox <gi...@apache.org>.
clintropolis commented on a change in pull request #9407: query laning and load shedding
URL: https://github.com/apache/druid/pull/9407#discussion_r389175395
 
 

 ##########
 File path: server/src/main/java/org/apache/druid/server/scheduling/HiLoQueryLaningStrategy.java
 ##########
 @@ -0,0 +1,79 @@
+/*
+ * 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.scheduling;
+
+import com.fasterxml.jackson.annotation.JsonCreator;
+import com.fasterxml.jackson.annotation.JsonProperty;
+import com.google.common.base.Preconditions;
+import com.google.common.primitives.Ints;
+import it.unimi.dsi.fastutil.objects.Object2IntArrayMap;
+import it.unimi.dsi.fastutil.objects.Object2IntMap;
+import org.apache.druid.client.SegmentServerSelector;
+import org.apache.druid.query.Query;
+import org.apache.druid.query.QueryContexts;
+import org.apache.druid.query.QueryPlus;
+import org.apache.druid.server.QueryLaningStrategy;
+
+import java.util.Optional;
+import java.util.Set;
+
+/**
+ * Query laning strategy which associates all {@link Query} with priority lower than 0 into a 'low' lane
+ */
+public class HiLoQueryLaningStrategy implements QueryLaningStrategy
+{
+  public static final String LOW = "low";
+
+  @JsonProperty
+  private final int maxLowPercent;
 
 Review comment:
   Ah, so do you mean the lane limit should be the min or (or max?) of `maxCapacity` and `maxLowPercent` of `totalLimit` when computing the lane limits?

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] clintropolis commented on a change in pull request #9407: query laning and load shedding

Posted by GitBox <gi...@apache.org>.
clintropolis commented on a change in pull request #9407: query laning and load shedding
URL: https://github.com/apache/druid/pull/9407#discussion_r389153310
 
 

 ##########
 File path: server/src/main/java/org/apache/druid/server/QueryScheduler.java
 ##########
 @@ -0,0 +1,241 @@
+/*
+ * 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;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.collect.HashMultimap;
+import com.google.common.collect.Multimaps;
+import com.google.common.collect.SetMultimap;
+import com.google.common.util.concurrent.ListenableFuture;
+import io.github.resilience4j.bulkhead.Bulkhead;
+import io.github.resilience4j.bulkhead.BulkheadConfig;
+import io.github.resilience4j.bulkhead.BulkheadRegistry;
+import it.unimi.dsi.fastutil.objects.Object2IntMap;
+import org.apache.druid.client.SegmentServerSelector;
+import org.apache.druid.java.util.common.concurrent.Execs;
+import org.apache.druid.java.util.common.guava.Sequence;
+import org.apache.druid.query.Query;
+import org.apache.druid.query.QueryContexts;
+import org.apache.druid.query.QueryPlus;
+import org.apache.druid.query.QueryWatcher;
+import org.apache.druid.server.initialization.ServerConfig;
+
+import java.time.Duration;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+
+/**
+ * QueryScheduler (potentially) assigns any {@link Query} that is to be executed to a 'query lane' using the
+ * {@link QueryLaningStrategy} that is defined in {@link QuerySchedulerConfig}.
+ *
+ * As a {@link QueryWatcher}, it also provides cancellation facilities.
+ *
+ * This class is shared by all requests on the Jetty HTTP theadpool and must be thread safe.
+ */
+public class QueryScheduler implements QueryWatcher
+{
+  static final String TOTAL = "default";
+  private final int totalCapacity;
+  private final QueryLaningStrategy laningStrategy;
+  private final BulkheadRegistry laneRegistry;
+  private final SetMultimap<String, ListenableFuture<?>> queryFutures;
+  private final SetMultimap<String, String> queryDatasources;
+
+  public QueryScheduler(int totalNumThreads, QueryLaningStrategy laningStrategy, ServerConfig serverConfig)
+  {
+    this.laningStrategy = laningStrategy;
+    this.queryFutures = Multimaps.synchronizedSetMultimap(HashMultimap.create());
+    this.queryDatasources = Multimaps.synchronizedSetMultimap(HashMultimap.create());
+    // if totalNumThreads is above 0 and less than druid.server.http.numThreads, enforce total limit
+    final boolean limitTotal;
+    if (totalNumThreads > 0 && totalNumThreads < serverConfig.getNumThreads()) {
+      limitTotal = true;
+      this.totalCapacity = totalNumThreads;
+    } else {
+      limitTotal = false;
+      this.totalCapacity = serverConfig.getNumThreads();
+    }
+    this.laneRegistry = BulkheadRegistry.of(getLaneConfigs(limitTotal));
+  }
+
+  @Override
+  public void registerQueryFuture(Query<?> query, ListenableFuture<?> future)
+  {
+    final String id = query.getId();
+    final Set<String> datasources = query.getDataSource().getTableNames();
+    queryFutures.put(id, future);
+    queryDatasources.putAll(id, datasources);
+    future.addListener(
+        () -> {
+          queryFutures.remove(id, future);
+          for (String datasource : datasources) {
+            queryDatasources.remove(id, datasource);
+          }
+        },
+        Execs.directExecutor()
+    );
+  }
+
+  /**
+   * Assign a query a lane (if not set)
+   */
+  public <T> Query<T> laneQuery(QueryPlus<T> queryPlus, Set<SegmentServerSelector> segments)
+  {
+    Query<T> query = queryPlus.getQuery();
+    // man wins over machine.. for now.
+    if (QueryContexts.getLane(query) != null) {
+      return query;
+    }
+    Optional<String> lane = laningStrategy.computeLane(queryPlus, segments);
+    return lane.map(query::withLane).orElse(query);
+  }
+
+  /**
+   * Run a query with the scheduler, attempting to acquire a semaphore from the total and lane specific query capacities
+   *
+   * Note that {@link #cancelQuery} should not interrupt the thread that calls run, in all current usages it only
+   * cancels any {@link ListenableFuture} created downstream. If this ever commonly changes, we should add
+   * synchronization between {@link #cancelQuery} and the acquisition of the {@link Bulkhead} to continue to ensure that
+   * anything acquired is also released.
+   *
+   * In the meantime, if a {@link ListenableFuture} is registered for the query that calls this method, it MUST handle
+   * this synchronization itself to ensure that no {@link Bulkhead} is acquired without releasing it.
+   */
+  public <T> Sequence<T> run(Query<?> query, Sequence<T> resultSequence)
+  {
+    List<Bulkhead> bulkheads = acquireLanes(query);
+    return resultSequence.withBaggage(() -> releaseLanes(bulkheads));
+  }
+
+  /**
+   * Forcibly cancel all futures that have been registered to a specific query id
+   */
+  public boolean cancelQuery(String id)
+  {
+    // if you re-use queryId and cancel queries... you are going to have a bad time
+    queryDatasources.removeAll(id);
+    Set<ListenableFuture<?>> futures = queryFutures.removeAll(id);
+    boolean success = true;
+    for (ListenableFuture<?> future : futures) {
+      success = success && future.cancel(true);
+    }
+    return success;
+  }
+
+  /**
+   * Get a {@link Set} of datasource names for a {@link Query} id, used by {@link QueryResource#cancelQuery} to
+   * authorize that a user may call {@link #cancelQuery} for the given id and datasources
+   */
+  public Set<String> getQueryDatasources(final String queryId)
+  {
+    return queryDatasources.get(queryId);
+  }
+
+  /**
+   * Get the maximum number of concurrent queries that {@link #run} can support
+   */
+  @VisibleForTesting
+  int getTotalAvailableCapacity()
+  {
+    return laneRegistry.getConfiguration(TOTAL)
+                       .map(config -> laneRegistry.bulkhead(TOTAL, config).getMetrics().getAvailableConcurrentCalls())
+                       .orElse(-1);
+  }
+
+  /**
+   * Get the maximum number of concurrent queries that {@link #run} can support for a given lane
+   */
+  @VisibleForTesting
+  int getLaneAvailableCapacity(String lane)
+  {
+    return laneRegistry.getConfiguration(lane)
+                       .map(config -> laneRegistry.bulkhead(lane, config).getMetrics().getAvailableConcurrentCalls())
+                       .orElse(-1);
+  }
+
+  /**
+   * Acquire a semaphore for both the 'total' and a lane, if any is associated with a query
+   */
+  @VisibleForTesting
+  List<Bulkhead> acquireLanes(Query<?> query)
+  {
+    final String lane = QueryContexts.getLane(query);
+    final Optional<BulkheadConfig> laneConfig = lane == null ? Optional.empty() : laneRegistry.getConfiguration(lane);
+    List<Bulkhead> hallPasses = new ArrayList<>(2);
+    final Optional<BulkheadConfig> totalConfig = laneRegistry.getConfiguration(TOTAL);
+    // if we have a lane, get it first
+    laneConfig.ifPresent(config -> {
 
 Review comment:
   yeah, it is definitely racy, but didn't seem especially harmful, is it worth the extra lock (not sure what would be most appropriate, striped on lane i guess?) we would have to hold to grab both and have truly fair behavior?

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] clintropolis commented on a change in pull request #9407: query laning and load shedding

Posted by GitBox <gi...@apache.org>.
clintropolis commented on a change in pull request #9407: query laning and load shedding
URL: https://github.com/apache/druid/pull/9407#discussion_r388754187
 
 

 ##########
 File path: server/src/test/java/org/apache/druid/server/scheduling/HiLoQueryLaningStrategyTest.java
 ##########
 @@ -0,0 +1,108 @@
+/*
+ * 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.scheduling;
+
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.ImmutableMap;
+import com.google.common.collect.ImmutableSet;
+import it.unimi.dsi.fastutil.objects.Object2IntMap;
+import org.apache.druid.java.util.common.Intervals;
+import org.apache.druid.java.util.common.granularity.Granularities;
+import org.apache.druid.query.Druids;
+import org.apache.druid.query.QueryContexts;
+import org.apache.druid.query.QueryPlus;
+import org.apache.druid.query.aggregation.CountAggregatorFactory;
+import org.apache.druid.query.timeseries.TimeseriesQuery;
+import org.apache.druid.server.QueryLaningStrategy;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.ExpectedException;
+
+public class HiLoQueryLaningStrategyTest
+{
+  @Rule
+  public ExpectedException expectedException = ExpectedException.none();
+
+  private Druids.TimeseriesQueryBuilder queryBuilder;
+  private HiLoQueryLaningStrategy strategy;
+
+  @Before
+  public void setup()
+  {
+    this.queryBuilder = Druids.newTimeseriesQueryBuilder()
+                              .dataSource("test")
+                              .intervals(ImmutableList.of(Intervals.ETERNITY))
+                              .granularity(Granularities.DAY)
+                              .aggregators(new CountAggregatorFactory("count"));
+
+    this.strategy = new HiLoQueryLaningStrategy(10);
+  }
+
+  @Test
+  public void testMaxLowThreadsRequired()
+  {
+    expectedException.expect(NullPointerException.class);
+    expectedException.expectMessage("maxLowThreads must be set");
+    QueryLaningStrategy strategy = new HiLoQueryLaningStrategy(null);
+  }
+
+  @Test
+  public void testLaneLimits()
+  {
+    Object2IntMap<String> laneConfig = strategy.getLaneLimits();
+    Assert.assertEquals(1, laneConfig.size());
+    Assert.assertTrue(laneConfig.containsKey(HiLoQueryLaningStrategy.LOW));
+    Assert.assertEquals(10, laneConfig.getInt(HiLoQueryLaningStrategy.LOW));
+  }
+
+  @Test
+  public void testLaningNoPriority()
+  {
+    TimeseriesQuery query = queryBuilder.build();
+    Assert.assertFalse(strategy.computeLane(QueryPlus.wrap(query), ImmutableSet.of()).isPresent());
+  }
+
+  @Test
+  public void testLaningZeroPriority()
+  {
+    TimeseriesQuery query = queryBuilder.context(ImmutableMap.of(QueryContexts.PRIORITY_KEY, 0)).build();
+    Assert.assertFalse(strategy.computeLane(QueryPlus.wrap(query), ImmutableSet.of()).isPresent());
+  }
+
+  @Test
+  public void testLaningInteractivePriority()
+  {
+    TimeseriesQuery query = queryBuilder.context(ImmutableMap.of(QueryContexts.PRIORITY_KEY, 100)).build();
+    Assert.assertFalse(strategy.computeLane(QueryPlus.wrap(query), ImmutableSet.of()).isPresent());
+  }
+
+  @Test
+  public void testLaningLowPriority()
+  {
+    TimeseriesQuery query = queryBuilder.context(ImmutableMap.of(QueryContexts.PRIORITY_KEY, -1)).build();
+    Assert.assertTrue(strategy.computeLane(QueryPlus.wrap(query), ImmutableSet.of()).isPresent());
+    Assert.assertEquals(
+        HiLoQueryLaningStrategy.LOW,
+        strategy.computeLane(QueryPlus.wrap(query), ImmutableSet.of()).get()
+    );
+  }
+}
 
 Review comment:
   Hmm, I'm not yet quite sure if not allowing a laning strategy to override the manually set context value is the correct behavior yet, so I didn't add a test. 
   
   Do you have any thoughts on whether or not to replace an explicit, user specified lane be delegated to the laning strategy? The more I think about it the more I think the laning strategy should get to choose, so I think I'm going to change the behavior. If we still think that hilo should honor the user specified lane, then I can add a test for that (or add a test that it is overridden if we decide to go in the other direction).

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] sascha-coenen commented on a change in pull request #9407: query laning and load shedding

Posted by GitBox <gi...@apache.org>.
sascha-coenen commented on a change in pull request #9407: query laning and load shedding
URL: https://github.com/apache/druid/pull/9407#discussion_r385204827
 
 

 ##########
 File path: docs/configuration/index.md
 ##########
 @@ -1476,9 +1476,31 @@ These Broker configurations can be defined in the `broker/runtime.properties` fi
 |`druid.broker.select.tier`|`highestPriority`, `lowestPriority`, `custom`|If segments are cross-replicated across tiers in a cluster, you can tell the broker to prefer to select segments in a tier with a certain priority.|`highestPriority`|
 |`druid.broker.select.tier.custom.priorities`|`An array of integer priorities.`|Select servers in tiers with a custom priority list.|None|
 
+##### Query laning
+
+Druid provides facilities to aid in query capacity reservation for heterogenous query workloads in the form of 'laning' strategies, which provide a variety of mechanisms examine and classify a query at the broker, assigning it to a 'lane'. Lanes are defined with capacity limits which the broker will enforce, causing requests in excess of the capacity to be discarded with an HTTP 429 status code, reserving resources for other lanes or for interactive queries (with no lane). 
 
 Review comment:
   > a variety of mechanisms examine 
   I believe a "to" is missing here ?

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] jihoonson commented on a change in pull request #9407: query laning and load shedding

Posted by GitBox <gi...@apache.org>.
jihoonson commented on a change in pull request #9407: query laning and load shedding
URL: https://github.com/apache/druid/pull/9407#discussion_r389121756
 
 

 ##########
 File path: server/src/main/java/org/apache/druid/server/QueryScheduler.java
 ##########
 @@ -0,0 +1,241 @@
+/*
+ * 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;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.collect.HashMultimap;
+import com.google.common.collect.Multimaps;
+import com.google.common.collect.SetMultimap;
+import com.google.common.util.concurrent.ListenableFuture;
+import io.github.resilience4j.bulkhead.Bulkhead;
+import io.github.resilience4j.bulkhead.BulkheadConfig;
+import io.github.resilience4j.bulkhead.BulkheadRegistry;
+import it.unimi.dsi.fastutil.objects.Object2IntMap;
+import org.apache.druid.client.SegmentServerSelector;
+import org.apache.druid.java.util.common.concurrent.Execs;
+import org.apache.druid.java.util.common.guava.Sequence;
+import org.apache.druid.query.Query;
+import org.apache.druid.query.QueryContexts;
+import org.apache.druid.query.QueryPlus;
+import org.apache.druid.query.QueryWatcher;
+import org.apache.druid.server.initialization.ServerConfig;
+
+import java.time.Duration;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+
+/**
+ * QueryScheduler (potentially) assigns any {@link Query} that is to be executed to a 'query lane' using the
+ * {@link QueryLaningStrategy} that is defined in {@link QuerySchedulerConfig}.
+ *
+ * As a {@link QueryWatcher}, it also provides cancellation facilities.
+ *
+ * This class is shared by all requests on the Jetty HTTP theadpool and must be thread safe.
+ */
+public class QueryScheduler implements QueryWatcher
+{
+  static final String TOTAL = "default";
+  private final int totalCapacity;
+  private final QueryLaningStrategy laningStrategy;
+  private final BulkheadRegistry laneRegistry;
+  private final SetMultimap<String, ListenableFuture<?>> queryFutures;
+  private final SetMultimap<String, String> queryDatasources;
+
+  public QueryScheduler(int totalNumThreads, QueryLaningStrategy laningStrategy, ServerConfig serverConfig)
+  {
+    this.laningStrategy = laningStrategy;
+    this.queryFutures = Multimaps.synchronizedSetMultimap(HashMultimap.create());
+    this.queryDatasources = Multimaps.synchronizedSetMultimap(HashMultimap.create());
+    // if totalNumThreads is above 0 and less than druid.server.http.numThreads, enforce total limit
+    final boolean limitTotal;
+    if (totalNumThreads > 0 && totalNumThreads < serverConfig.getNumThreads()) {
+      limitTotal = true;
+      this.totalCapacity = totalNumThreads;
+    } else {
+      limitTotal = false;
+      this.totalCapacity = serverConfig.getNumThreads();
+    }
+    this.laneRegistry = BulkheadRegistry.of(getLaneConfigs(limitTotal));
+  }
+
+  @Override
+  public void registerQueryFuture(Query<?> query, ListenableFuture<?> future)
+  {
+    final String id = query.getId();
+    final Set<String> datasources = query.getDataSource().getTableNames();
+    queryFutures.put(id, future);
+    queryDatasources.putAll(id, datasources);
+    future.addListener(
+        () -> {
+          queryFutures.remove(id, future);
+          for (String datasource : datasources) {
+            queryDatasources.remove(id, datasource);
+          }
+        },
+        Execs.directExecutor()
+    );
+  }
+
+  /**
+   * Assign a query a lane (if not set)
+   */
+  public <T> Query<T> laneQuery(QueryPlus<T> queryPlus, Set<SegmentServerSelector> segments)
+  {
+    Query<T> query = queryPlus.getQuery();
+    // man wins over machine.. for now.
+    if (QueryContexts.getLane(query) != null) {
+      return query;
+    }
+    Optional<String> lane = laningStrategy.computeLane(queryPlus, segments);
+    return lane.map(query::withLane).orElse(query);
+  }
+
+  /**
+   * Run a query with the scheduler, attempting to acquire a semaphore from the total and lane specific query capacities
+   *
+   * Note that {@link #cancelQuery} should not interrupt the thread that calls run, in all current usages it only
+   * cancels any {@link ListenableFuture} created downstream. If this ever commonly changes, we should add
+   * synchronization between {@link #cancelQuery} and the acquisition of the {@link Bulkhead} to continue to ensure that
+   * anything acquired is also released.
+   *
+   * In the meantime, if a {@link ListenableFuture} is registered for the query that calls this method, it MUST handle
+   * this synchronization itself to ensure that no {@link Bulkhead} is acquired without releasing it.
+   */
+  public <T> Sequence<T> run(Query<?> query, Sequence<T> resultSequence)
+  {
+    List<Bulkhead> bulkheads = acquireLanes(query);
+    return resultSequence.withBaggage(() -> releaseLanes(bulkheads));
+  }
+
+  /**
+   * Forcibly cancel all futures that have been registered to a specific query id
+   */
+  public boolean cancelQuery(String id)
+  {
+    // if you re-use queryId and cancel queries... you are going to have a bad time
+    queryDatasources.removeAll(id);
+    Set<ListenableFuture<?>> futures = queryFutures.removeAll(id);
+    boolean success = true;
+    for (ListenableFuture<?> future : futures) {
+      success = success && future.cancel(true);
+    }
+    return success;
+  }
+
+  /**
+   * Get a {@link Set} of datasource names for a {@link Query} id, used by {@link QueryResource#cancelQuery} to
+   * authorize that a user may call {@link #cancelQuery} for the given id and datasources
+   */
+  public Set<String> getQueryDatasources(final String queryId)
+  {
+    return queryDatasources.get(queryId);
+  }
+
+  /**
+   * Get the maximum number of concurrent queries that {@link #run} can support
+   */
+  @VisibleForTesting
+  int getTotalAvailableCapacity()
+  {
+    return laneRegistry.getConfiguration(TOTAL)
+                       .map(config -> laneRegistry.bulkhead(TOTAL, config).getMetrics().getAvailableConcurrentCalls())
+                       .orElse(-1);
+  }
+
+  /**
+   * Get the maximum number of concurrent queries that {@link #run} can support for a given lane
+   */
+  @VisibleForTesting
+  int getLaneAvailableCapacity(String lane)
+  {
+    return laneRegistry.getConfiguration(lane)
+                       .map(config -> laneRegistry.bulkhead(lane, config).getMetrics().getAvailableConcurrentCalls())
+                       .orElse(-1);
+  }
+
+  /**
+   * Acquire a semaphore for both the 'total' and a lane, if any is associated with a query
+   */
+  @VisibleForTesting
+  List<Bulkhead> acquireLanes(Query<?> query)
+  {
+    final String lane = QueryContexts.getLane(query);
+    final Optional<BulkheadConfig> laneConfig = lane == null ? Optional.empty() : laneRegistry.getConfiguration(lane);
+    List<Bulkhead> hallPasses = new ArrayList<>(2);
+    final Optional<BulkheadConfig> totalConfig = laneRegistry.getConfiguration(TOTAL);
+    // if we have a lane, get it first
+    laneConfig.ifPresent(config -> {
 
 Review comment:
   It seems racy that two queries can compete on a particular lane and the total lane separately. I guess it would be a better behavior if it guarantees the first-come, first-served basis.

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] clintropolis commented on a change in pull request #9407: query laning and load shedding

Posted by GitBox <gi...@apache.org>.
clintropolis commented on a change in pull request #9407: query laning and load shedding
URL: https://github.com/apache/druid/pull/9407#discussion_r389161870
 
 

 ##########
 File path: server/src/main/java/org/apache/druid/server/QueryScheduler.java
 ##########
 @@ -0,0 +1,241 @@
+/*
+ * 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;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.collect.HashMultimap;
+import com.google.common.collect.Multimaps;
+import com.google.common.collect.SetMultimap;
+import com.google.common.util.concurrent.ListenableFuture;
+import io.github.resilience4j.bulkhead.Bulkhead;
+import io.github.resilience4j.bulkhead.BulkheadConfig;
+import io.github.resilience4j.bulkhead.BulkheadRegistry;
+import it.unimi.dsi.fastutil.objects.Object2IntMap;
+import org.apache.druid.client.SegmentServerSelector;
+import org.apache.druid.java.util.common.concurrent.Execs;
+import org.apache.druid.java.util.common.guava.Sequence;
+import org.apache.druid.query.Query;
+import org.apache.druid.query.QueryContexts;
+import org.apache.druid.query.QueryPlus;
+import org.apache.druid.query.QueryWatcher;
+import org.apache.druid.server.initialization.ServerConfig;
+
+import java.time.Duration;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+
+/**
+ * QueryScheduler (potentially) assigns any {@link Query} that is to be executed to a 'query lane' using the
+ * {@link QueryLaningStrategy} that is defined in {@link QuerySchedulerConfig}.
+ *
+ * As a {@link QueryWatcher}, it also provides cancellation facilities.
+ *
+ * This class is shared by all requests on the Jetty HTTP theadpool and must be thread safe.
+ */
+public class QueryScheduler implements QueryWatcher
+{
+  static final String TOTAL = "default";
+  private final int totalCapacity;
+  private final QueryLaningStrategy laningStrategy;
+  private final BulkheadRegistry laneRegistry;
+  private final SetMultimap<String, ListenableFuture<?>> queryFutures;
+  private final SetMultimap<String, String> queryDatasources;
+
+  public QueryScheduler(int totalNumThreads, QueryLaningStrategy laningStrategy, ServerConfig serverConfig)
+  {
+    this.laningStrategy = laningStrategy;
+    this.queryFutures = Multimaps.synchronizedSetMultimap(HashMultimap.create());
+    this.queryDatasources = Multimaps.synchronizedSetMultimap(HashMultimap.create());
+    // if totalNumThreads is above 0 and less than druid.server.http.numThreads, enforce total limit
+    final boolean limitTotal;
+    if (totalNumThreads > 0 && totalNumThreads < serverConfig.getNumThreads()) {
+      limitTotal = true;
+      this.totalCapacity = totalNumThreads;
+    } else {
+      limitTotal = false;
+      this.totalCapacity = serverConfig.getNumThreads();
+    }
+    this.laneRegistry = BulkheadRegistry.of(getLaneConfigs(limitTotal));
+  }
+
+  @Override
+  public void registerQueryFuture(Query<?> query, ListenableFuture<?> future)
+  {
+    final String id = query.getId();
+    final Set<String> datasources = query.getDataSource().getTableNames();
+    queryFutures.put(id, future);
+    queryDatasources.putAll(id, datasources);
+    future.addListener(
+        () -> {
+          queryFutures.remove(id, future);
+          for (String datasource : datasources) {
+            queryDatasources.remove(id, datasource);
+          }
+        },
+        Execs.directExecutor()
+    );
+  }
+
+  /**
+   * Assign a query a lane (if not set)
+   */
+  public <T> Query<T> laneQuery(QueryPlus<T> queryPlus, Set<SegmentServerSelector> segments)
+  {
+    Query<T> query = queryPlus.getQuery();
+    // man wins over machine.. for now.
+    if (QueryContexts.getLane(query) != null) {
+      return query;
+    }
+    Optional<String> lane = laningStrategy.computeLane(queryPlus, segments);
+    return lane.map(query::withLane).orElse(query);
+  }
+
+  /**
+   * Run a query with the scheduler, attempting to acquire a semaphore from the total and lane specific query capacities
+   *
+   * Note that {@link #cancelQuery} should not interrupt the thread that calls run, in all current usages it only
+   * cancels any {@link ListenableFuture} created downstream. If this ever commonly changes, we should add
+   * synchronization between {@link #cancelQuery} and the acquisition of the {@link Bulkhead} to continue to ensure that
+   * anything acquired is also released.
+   *
+   * In the meantime, if a {@link ListenableFuture} is registered for the query that calls this method, it MUST handle
+   * this synchronization itself to ensure that no {@link Bulkhead} is acquired without releasing it.
+   */
+  public <T> Sequence<T> run(Query<?> query, Sequence<T> resultSequence)
+  {
+    List<Bulkhead> bulkheads = acquireLanes(query);
+    return resultSequence.withBaggage(() -> releaseLanes(bulkheads));
+  }
+
+  /**
+   * Forcibly cancel all futures that have been registered to a specific query id
+   */
+  public boolean cancelQuery(String id)
+  {
+    // if you re-use queryId and cancel queries... you are going to have a bad time
+    queryDatasources.removeAll(id);
+    Set<ListenableFuture<?>> futures = queryFutures.removeAll(id);
+    boolean success = true;
+    for (ListenableFuture<?> future : futures) {
+      success = success && future.cancel(true);
+    }
+    return success;
+  }
+
+  /**
+   * Get a {@link Set} of datasource names for a {@link Query} id, used by {@link QueryResource#cancelQuery} to
+   * authorize that a user may call {@link #cancelQuery} for the given id and datasources
+   */
+  public Set<String> getQueryDatasources(final String queryId)
+  {
+    return queryDatasources.get(queryId);
+  }
+
+  /**
+   * Get the maximum number of concurrent queries that {@link #run} can support
+   */
+  @VisibleForTesting
+  int getTotalAvailableCapacity()
+  {
+    return laneRegistry.getConfiguration(TOTAL)
+                       .map(config -> laneRegistry.bulkhead(TOTAL, config).getMetrics().getAvailableConcurrentCalls())
+                       .orElse(-1);
+  }
+
+  /**
+   * Get the maximum number of concurrent queries that {@link #run} can support for a given lane
+   */
+  @VisibleForTesting
+  int getLaneAvailableCapacity(String lane)
+  {
+    return laneRegistry.getConfiguration(lane)
+                       .map(config -> laneRegistry.bulkhead(lane, config).getMetrics().getAvailableConcurrentCalls())
+                       .orElse(-1);
+  }
+
+  /**
+   * Acquire a semaphore for both the 'total' and a lane, if any is associated with a query
+   */
+  @VisibleForTesting
+  List<Bulkhead> acquireLanes(Query<?> query)
+  {
+    final String lane = QueryContexts.getLane(query);
+    final Optional<BulkheadConfig> laneConfig = lane == null ? Optional.empty() : laneRegistry.getConfiguration(lane);
+    List<Bulkhead> hallPasses = new ArrayList<>(2);
+    final Optional<BulkheadConfig> totalConfig = laneRegistry.getConfiguration(TOTAL);
+    // if we have a lane, get it first
+    laneConfig.ifPresent(config -> {
+      Bulkhead laneLimiter = laneRegistry.bulkhead(lane, config);
+      if (!laneLimiter.tryAcquirePermission()) {
+        throw new QueryCapacityExceededException(lane);
+      }
+      hallPasses.add(laneLimiter);
+    });
+
+    // everyone needs to take one from the total lane; to ensure we don't acquire a lane and never release it, we want
+    // to check for total capacity exceeded and release the lane (if present) before throwing capacity exceeded
+    totalConfig.ifPresent(config -> {
+      Bulkhead totalLimiter = laneRegistry.bulkhead(TOTAL, config);
+      if (!totalLimiter.tryAcquirePermission()) {
+        releaseLanes(hallPasses);
+        throw new QueryCapacityExceededException();
+      }
+      hallPasses.add(totalLimiter);
+    });
+    return hallPasses;
+  }
+
+  /**
+   * Release all {@link Bulkhead} semaphores in the list
+   */
+  @VisibleForTesting
+  void releaseLanes(List<Bulkhead> bulkheads)
+  {
+    bulkheads.forEach(Bulkhead::releasePermission);
 
 Review comment:
   good catch

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] jihoonson commented on a change in pull request #9407: query laning and load shedding

Posted by GitBox <gi...@apache.org>.
jihoonson commented on a change in pull request #9407: query laning and load shedding
URL: https://github.com/apache/druid/pull/9407#discussion_r389125436
 
 

 ##########
 File path: server/src/main/java/org/apache/druid/server/QueryScheduler.java
 ##########
 @@ -0,0 +1,241 @@
+/*
+ * 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;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.collect.HashMultimap;
+import com.google.common.collect.Multimaps;
+import com.google.common.collect.SetMultimap;
+import com.google.common.util.concurrent.ListenableFuture;
+import io.github.resilience4j.bulkhead.Bulkhead;
+import io.github.resilience4j.bulkhead.BulkheadConfig;
+import io.github.resilience4j.bulkhead.BulkheadRegistry;
+import it.unimi.dsi.fastutil.objects.Object2IntMap;
+import org.apache.druid.client.SegmentServerSelector;
+import org.apache.druid.java.util.common.concurrent.Execs;
+import org.apache.druid.java.util.common.guava.Sequence;
+import org.apache.druid.query.Query;
+import org.apache.druid.query.QueryContexts;
+import org.apache.druid.query.QueryPlus;
+import org.apache.druid.query.QueryWatcher;
+import org.apache.druid.server.initialization.ServerConfig;
+
+import java.time.Duration;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+
+/**
+ * QueryScheduler (potentially) assigns any {@link Query} that is to be executed to a 'query lane' using the
+ * {@link QueryLaningStrategy} that is defined in {@link QuerySchedulerConfig}.
+ *
+ * As a {@link QueryWatcher}, it also provides cancellation facilities.
+ *
+ * This class is shared by all requests on the Jetty HTTP theadpool and must be thread safe.
+ */
+public class QueryScheduler implements QueryWatcher
+{
+  static final String TOTAL = "default";
+  private final int totalCapacity;
+  private final QueryLaningStrategy laningStrategy;
+  private final BulkheadRegistry laneRegistry;
+  private final SetMultimap<String, ListenableFuture<?>> queryFutures;
+  private final SetMultimap<String, String> queryDatasources;
+
+  public QueryScheduler(int totalNumThreads, QueryLaningStrategy laningStrategy, ServerConfig serverConfig)
+  {
+    this.laningStrategy = laningStrategy;
+    this.queryFutures = Multimaps.synchronizedSetMultimap(HashMultimap.create());
+    this.queryDatasources = Multimaps.synchronizedSetMultimap(HashMultimap.create());
+    // if totalNumThreads is above 0 and less than druid.server.http.numThreads, enforce total limit
+    final boolean limitTotal;
+    if (totalNumThreads > 0 && totalNumThreads < serverConfig.getNumThreads()) {
+      limitTotal = true;
+      this.totalCapacity = totalNumThreads;
+    } else {
+      limitTotal = false;
+      this.totalCapacity = serverConfig.getNumThreads();
+    }
+    this.laneRegistry = BulkheadRegistry.of(getLaneConfigs(limitTotal));
+  }
+
+  @Override
+  public void registerQueryFuture(Query<?> query, ListenableFuture<?> future)
+  {
+    final String id = query.getId();
+    final Set<String> datasources = query.getDataSource().getTableNames();
+    queryFutures.put(id, future);
+    queryDatasources.putAll(id, datasources);
+    future.addListener(
+        () -> {
+          queryFutures.remove(id, future);
+          for (String datasource : datasources) {
+            queryDatasources.remove(id, datasource);
+          }
+        },
+        Execs.directExecutor()
+    );
+  }
+
+  /**
+   * Assign a query a lane (if not set)
+   */
+  public <T> Query<T> laneQuery(QueryPlus<T> queryPlus, Set<SegmentServerSelector> segments)
+  {
+    Query<T> query = queryPlus.getQuery();
+    // man wins over machine.. for now.
+    if (QueryContexts.getLane(query) != null) {
+      return query;
+    }
+    Optional<String> lane = laningStrategy.computeLane(queryPlus, segments);
+    return lane.map(query::withLane).orElse(query);
+  }
+
+  /**
+   * Run a query with the scheduler, attempting to acquire a semaphore from the total and lane specific query capacities
+   *
+   * Note that {@link #cancelQuery} should not interrupt the thread that calls run, in all current usages it only
+   * cancels any {@link ListenableFuture} created downstream. If this ever commonly changes, we should add
+   * synchronization between {@link #cancelQuery} and the acquisition of the {@link Bulkhead} to continue to ensure that
+   * anything acquired is also released.
+   *
+   * In the meantime, if a {@link ListenableFuture} is registered for the query that calls this method, it MUST handle
+   * this synchronization itself to ensure that no {@link Bulkhead} is acquired without releasing it.
+   */
+  public <T> Sequence<T> run(Query<?> query, Sequence<T> resultSequence)
+  {
+    List<Bulkhead> bulkheads = acquireLanes(query);
+    return resultSequence.withBaggage(() -> releaseLanes(bulkheads));
+  }
+
+  /**
+   * Forcibly cancel all futures that have been registered to a specific query id
+   */
+  public boolean cancelQuery(String id)
+  {
+    // if you re-use queryId and cancel queries... you are going to have a bad time
+    queryDatasources.removeAll(id);
+    Set<ListenableFuture<?>> futures = queryFutures.removeAll(id);
+    boolean success = true;
+    for (ListenableFuture<?> future : futures) {
+      success = success && future.cancel(true);
+    }
+    return success;
+  }
+
+  /**
+   * Get a {@link Set} of datasource names for a {@link Query} id, used by {@link QueryResource#cancelQuery} to
+   * authorize that a user may call {@link #cancelQuery} for the given id and datasources
+   */
+  public Set<String> getQueryDatasources(final String queryId)
+  {
+    return queryDatasources.get(queryId);
+  }
+
+  /**
+   * Get the maximum number of concurrent queries that {@link #run} can support
+   */
+  @VisibleForTesting
+  int getTotalAvailableCapacity()
+  {
+    return laneRegistry.getConfiguration(TOTAL)
+                       .map(config -> laneRegistry.bulkhead(TOTAL, config).getMetrics().getAvailableConcurrentCalls())
+                       .orElse(-1);
+  }
+
+  /**
+   * Get the maximum number of concurrent queries that {@link #run} can support for a given lane
+   */
+  @VisibleForTesting
+  int getLaneAvailableCapacity(String lane)
+  {
+    return laneRegistry.getConfiguration(lane)
+                       .map(config -> laneRegistry.bulkhead(lane, config).getMetrics().getAvailableConcurrentCalls())
+                       .orElse(-1);
+  }
+
+  /**
+   * Acquire a semaphore for both the 'total' and a lane, if any is associated with a query
+   */
+  @VisibleForTesting
+  List<Bulkhead> acquireLanes(Query<?> query)
+  {
+    final String lane = QueryContexts.getLane(query);
+    final Optional<BulkheadConfig> laneConfig = lane == null ? Optional.empty() : laneRegistry.getConfiguration(lane);
+    List<Bulkhead> hallPasses = new ArrayList<>(2);
+    final Optional<BulkheadConfig> totalConfig = laneRegistry.getConfiguration(TOTAL);
+    // if we have a lane, get it first
+    laneConfig.ifPresent(config -> {
+      Bulkhead laneLimiter = laneRegistry.bulkhead(lane, config);
+      if (!laneLimiter.tryAcquirePermission()) {
+        throw new QueryCapacityExceededException(lane);
+      }
+      hallPasses.add(laneLimiter);
+    });
+
+    // everyone needs to take one from the total lane; to ensure we don't acquire a lane and never release it, we want
+    // to check for total capacity exceeded and release the lane (if present) before throwing capacity exceeded
+    totalConfig.ifPresent(config -> {
+      Bulkhead totalLimiter = laneRegistry.bulkhead(TOTAL, config);
+      if (!totalLimiter.tryAcquirePermission()) {
+        releaseLanes(hallPasses);
+        throw new QueryCapacityExceededException();
+      }
+      hallPasses.add(totalLimiter);
+    });
+    return hallPasses;
+  }
+
+  /**
+   * Release all {@link Bulkhead} semaphores in the list
+   */
+  @VisibleForTesting
+  void releaseLanes(List<Bulkhead> bulkheads)
+  {
+    bulkheads.forEach(Bulkhead::releasePermission);
+  }
+
+  /**
+   * With a total thread count and {@link QueryLaningStrategy#getLaneLimits}, create a map of lane name to
+   * {@link BulkheadConfig} to be used to create the {@link #laneRegistry}. This accepts the configured value of
+   * numThreads rather than using {@link #totalCapacity} so that we only have a total {@link Bulkhead} if
+   * {@link QuerySchedulerConfig#getNumThreads()} is set
+   */
+  private Map<String, BulkheadConfig> getLaneConfigs(boolean hastotalLimit)
 
 Review comment:
   `hastotalLimit` -> `hasTotalLimit`

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] clintropolis merged pull request #9407: query laning and load shedding

Posted by GitBox <gi...@apache.org>.
clintropolis merged pull request #9407: query laning and load shedding
URL: https://github.com/apache/druid/pull/9407
 
 
   

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] jihoonson commented on a change in pull request #9407: query laning and load shedding

Posted by GitBox <gi...@apache.org>.
jihoonson commented on a change in pull request #9407: query laning and load shedding
URL: https://github.com/apache/druid/pull/9407#discussion_r389088603
 
 

 ##########
 File path: server/src/main/java/org/apache/druid/server/QueryScheduler.java
 ##########
 @@ -0,0 +1,241 @@
+/*
+ * 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;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.collect.HashMultimap;
+import com.google.common.collect.Multimaps;
+import com.google.common.collect.SetMultimap;
+import com.google.common.util.concurrent.ListenableFuture;
+import io.github.resilience4j.bulkhead.Bulkhead;
+import io.github.resilience4j.bulkhead.BulkheadConfig;
+import io.github.resilience4j.bulkhead.BulkheadRegistry;
+import it.unimi.dsi.fastutil.objects.Object2IntMap;
+import org.apache.druid.client.SegmentServerSelector;
+import org.apache.druid.java.util.common.concurrent.Execs;
+import org.apache.druid.java.util.common.guava.Sequence;
+import org.apache.druid.query.Query;
+import org.apache.druid.query.QueryContexts;
+import org.apache.druid.query.QueryPlus;
+import org.apache.druid.query.QueryWatcher;
+import org.apache.druid.server.initialization.ServerConfig;
+
+import java.time.Duration;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+
+/**
+ * QueryScheduler (potentially) assigns any {@link Query} that is to be executed to a 'query lane' using the
+ * {@link QueryLaningStrategy} that is defined in {@link QuerySchedulerConfig}.
+ *
+ * As a {@link QueryWatcher}, it also provides cancellation facilities.
+ *
+ * This class is shared by all requests on the Jetty HTTP theadpool and must be thread safe.
+ */
+public class QueryScheduler implements QueryWatcher
 
 Review comment:
   Please comment that this class is used in everywhere including historicals and tasks.

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] clintropolis commented on a change in pull request #9407: query laning and load shedding

Posted by GitBox <gi...@apache.org>.
clintropolis commented on a change in pull request #9407: query laning and load shedding
URL: https://github.com/apache/druid/pull/9407#discussion_r388754995
 
 

 ##########
 File path: docs/configuration/index.md
 ##########
 @@ -1476,9 +1476,31 @@ These Broker configurations can be defined in the `broker/runtime.properties` fi
 |`druid.broker.select.tier`|`highestPriority`, `lowestPriority`, `custom`|If segments are cross-replicated across tiers in a cluster, you can tell the broker to prefer to select segments in a tier with a certain priority.|`highestPriority`|
 |`druid.broker.select.tier.custom.priorities`|`An array of integer priorities.`|Select servers in tiers with a custom priority list.|None|
 
+##### Query laning
+
+The Broker provides facilities to aid in query capacity reservation for heterogeneous query workloads in the form of 'laning' strategies, which provide a variety of mechanisms to examine and classify a query, assigning it to a 'lane'. Lanes are defined with capacity limits which the broker will enforce, causing requests in excess of the capacity to be discarded with an HTTP 429 status code, reserving resources for other lanes or for interactive queries (with no lane).
+
+|Property|Description|Default|
+|--------|-----------|-------|
+|`druid.query.scheduler.numThreads`|Maximum number of HTTP threads to dedicate to query processing. To save HTTP thread capacity, this should be lower than `druid.server.http.numThreads`.|Unbounded|
+|`druid.query.scheduler.laning.type`|Query laning strategy to use to assign queries to a lane in order to control capacities for certain classes of queries.|`none`|
 
 Review comment:
   `druid.query.scheduler.laning.strategy` seems reasonable, will switch

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] himanshug commented on a change in pull request #9407: query laning and load shedding

Posted by GitBox <gi...@apache.org>.
himanshug commented on a change in pull request #9407: query laning and load shedding
URL: https://github.com/apache/druid/pull/9407#discussion_r390469100
 
 

 ##########
 File path: server/src/main/java/org/apache/druid/server/QueryLaningStrategy.java
 ##########
 @@ -0,0 +1,53 @@
+/*
+ * 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;
+
+import com.fasterxml.jackson.annotation.JsonSubTypes;
+import com.fasterxml.jackson.annotation.JsonTypeInfo;
+import it.unimi.dsi.fastutil.objects.Object2IntMap;
+import org.apache.druid.client.SegmentServerSelector;
+import org.apache.druid.query.QueryPlus;
+import org.apache.druid.server.scheduling.HiLoQueryLaningStrategy;
+import org.apache.druid.server.scheduling.NoQueryLaningStrategy;
+
+import java.util.Optional;
+import java.util.Set;
+
+
+@JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "strategy", defaultImpl = NoQueryLaningStrategy.class)
+@JsonSubTypes(value = {
+    @JsonSubTypes.Type(name = "none", value = NoQueryLaningStrategy.class),
+    @JsonSubTypes.Type(name = "hilo", value = HiLoQueryLaningStrategy.class)
+})
+public interface QueryLaningStrategy
+{
+  /**
+   * Provide a map of lane names to the limit on the number of concurrent queries for that lane
+   * @param totalLimit
+   */
+  Object2IntMap<String> getLaneLimits(int totalLimit);
 
 Review comment:
   Hehe, that happens with multiple reviewers. Yeah, having a utility method is equally good.

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] clintropolis commented on a change in pull request #9407: query laning and load shedding

Posted by GitBox <gi...@apache.org>.
clintropolis commented on a change in pull request #9407: query laning and load shedding
URL: https://github.com/apache/druid/pull/9407#discussion_r389966075
 
 

 ##########
 File path: server/src/main/java/org/apache/druid/server/scheduling/HiLoQueryLaningStrategy.java
 ##########
 @@ -0,0 +1,79 @@
+/*
+ * 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.scheduling;
+
+import com.fasterxml.jackson.annotation.JsonCreator;
+import com.fasterxml.jackson.annotation.JsonProperty;
+import com.google.common.base.Preconditions;
+import com.google.common.primitives.Ints;
+import it.unimi.dsi.fastutil.objects.Object2IntArrayMap;
+import it.unimi.dsi.fastutil.objects.Object2IntMap;
+import org.apache.druid.client.SegmentServerSelector;
+import org.apache.druid.query.Query;
+import org.apache.druid.query.QueryContexts;
+import org.apache.druid.query.QueryPlus;
+import org.apache.druid.server.QueryLaningStrategy;
+
+import java.util.Optional;
+import java.util.Set;
+
+/**
+ * Query laning strategy which associates all {@link Query} with priority lower than 0 into a 'low' lane
+ */
+public class HiLoQueryLaningStrategy implements QueryLaningStrategy
+{
+  public static final String LOW = "low";
+
+  @JsonProperty
+  private final int maxLowPercent;
+
+  @JsonCreator
+  public HiLoQueryLaningStrategy(
+      @JsonProperty("maxLowPercent") Integer maxLowPercent
+  )
+  {
+    this.maxLowPercent = Preconditions.checkNotNull(maxLowPercent, "maxLowPercent must be set");
+    Preconditions.checkArgument(
+        0 < maxLowPercent && maxLowPercent < 100,
+        "maxLowPercent must be between 0 and 100"
+    );
+  }
+
+  @Override
+  public Object2IntMap<String> getLaneLimits(int totalLimit)
+  {
+    Object2IntMap<String> onlyLow = new Object2IntArrayMap<>(1);
+    onlyLow.put(LOW, Ints.checkedCast(Math.round(totalLimit * ((double) maxLowPercent / 100))));
 
 Review comment:
   thanks, good catch. fixed, clarified error and docs, and added some tests

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] clintropolis commented on issue #9407: query laning and load shedding

Posted by GitBox <gi...@apache.org>.
clintropolis commented on issue #9407: query laning and load shedding
URL: https://github.com/apache/druid/pull/9407#issuecomment-596804147
 
 
   Thanks for review @jihoonson and @ccaominh :metal:
   
   @sascha-coenen or @himanshug (or anyone else) have any comments?

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] clintropolis commented on a change in pull request #9407: query laning and load shedding

Posted by GitBox <gi...@apache.org>.
clintropolis commented on a change in pull request #9407: query laning and load shedding
URL: https://github.com/apache/druid/pull/9407#discussion_r388751700
 
 

 ##########
 File path: docs/configuration/index.md
 ##########
 @@ -1476,9 +1476,31 @@ These Broker configurations can be defined in the `broker/runtime.properties` fi
 |`druid.broker.select.tier`|`highestPriority`, `lowestPriority`, `custom`|If segments are cross-replicated across tiers in a cluster, you can tell the broker to prefer to select segments in a tier with a certain priority.|`highestPriority`|
 |`druid.broker.select.tier.custom.priorities`|`An array of integer priorities.`|Select servers in tiers with a custom priority list.|None|
 
+##### Query laning
+
+The Broker provides facilities to aid in query capacity reservation for heterogeneous query workloads in the form of 'laning' strategies, which provide a variety of mechanisms to examine and classify a query, assigning it to a 'lane'. Lanes are defined with capacity limits which the broker will enforce, causing requests in excess of the capacity to be discarded with an HTTP 429 status code, reserving resources for other lanes or for interactive queries (with no lane).
+
+|Property|Description|Default|
+|--------|-----------|-------|
+|`druid.query.scheduler.numThreads`|Maximum number of HTTP threads to dedicate to query processing. To save HTTP thread capacity, this should be lower than `druid.server.http.numThreads`.|Unbounded|
+|`druid.query.scheduler.laning.type`|Query laning strategy to use to assign queries to a lane in order to control capacities for certain classes of queries.|`none`|
+
+###### No laning strategy
+
+In this mode, queries are never assigned a lane and only limited by `druid.server.http.numThreads` or `druid.query.scheduler.numThreads`, if set. This is the default Druid query scheduler operating mode.
+
+###### 'High/Low' laning strategy
+This laning strategy splits queries with a `priority` below zero into a `low` query lane, automatically. The limit on `low` queries can be set to some desired fraction of the total capacity (or HTTP thread pool size), reserving capacity for interactive queries. Queries in the `low` lane are _not_ guaranteed their capacity, which may be consumed by interactive queries, but may use up to this limit if total capacity is available. 
+
+This strategy can be enabled by setting `druid.query.scheduler.laning.type` to `hilo`.
+
+|Property|Description|Default|
+|--------|-----------|-------|
+|`druid.query.scheduler.laning.maxLowThreads`|Maximum number of HTTP threads that can be used by queries with a priority lower than 0.|No default, must be set if using this mode|
 
 Review comment:
   I'm not sure I agree that it is particularly confusing, since the HTTP setting is `druid.server.http.numThreads` and the new scheduler 'total' lane is `druid.query.scheduler.numThreads`. Are you considering the hypothetical of if we ever decided to extend the concept of laning to the processing pool I guess?
   
   Would this setting be better as a percentage, so one property could be applicable to either usage? It doesn't seem like it would be hard to switch, would just need to adjust `QueryLaningStrategy.getLaneLimits` to take the 'total' limit, and make `ServerConfig` available to the `QueryScheduler` so it could fall back to `druid.server.http.numThreads` if `druid.query.scheduler.numThreads` isn't set. `ServerConfig` should probably already be provided to `QueryScheduler` so it can treat the case where `druid.query.scheduler.numThreads` is higher than `druid.server.http.numThreads` the same as not setting the scheduler `numThreads` at all and ignoring total limiter.

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] ccaominh commented on a change in pull request #9407: query laning and load shedding

Posted by GitBox <gi...@apache.org>.
ccaominh commented on a change in pull request #9407: query laning and load shedding
URL: https://github.com/apache/druid/pull/9407#discussion_r387962715
 
 

 ##########
 File path: docs/configuration/index.md
 ##########
 @@ -1476,9 +1476,31 @@ These Broker configurations can be defined in the `broker/runtime.properties` fi
 |`druid.broker.select.tier`|`highestPriority`, `lowestPriority`, `custom`|If segments are cross-replicated across tiers in a cluster, you can tell the broker to prefer to select segments in a tier with a certain priority.|`highestPriority`|
 |`druid.broker.select.tier.custom.priorities`|`An array of integer priorities.`|Select servers in tiers with a custom priority list.|None|
 
+##### Query laning
+
+The Broker provides facilities to aid in query capacity reservation for heterogeneous query workloads in the form of 'laning' strategies, which provide a variety of mechanisms to examine and classify a query, assigning it to a 'lane'. Lanes are defined with capacity limits which the broker will enforce, causing requests in excess of the capacity to be discarded with an HTTP 429 status code, reserving resources for other lanes or for interactive queries (with no lane).
+
+|Property|Description|Default|
+|--------|-----------|-------|
+|`druid.query.scheduler.numThreads`|Maximum number of HTTP threads to dedicate to query processing. To save HTTP thread capacity, this should be lower than `druid.server.http.numThreads`.|Unbounded|
+|`druid.query.scheduler.laning.type`|Query laning strategy to use to assign queries to a lane in order to control capacities for certain classes of queries.|`none`|
 
 Review comment:
   Perhaps the setting name ("type") and the name in the docs ("strategy") should be made consistent.
   
   In terms of documentation flow, it may be helpful to add a section below (e.g. "Laning strategy") and reference it here. "No laning strategy", etc. would be children on this new section.

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] clintropolis commented on a change in pull request #9407: query laning and load shedding

Posted by GitBox <gi...@apache.org>.
clintropolis commented on a change in pull request #9407: query laning and load shedding
URL: https://github.com/apache/druid/pull/9407#discussion_r389219215
 
 

 ##########
 File path: server/src/main/java/org/apache/druid/server/QueryScheduler.java
 ##########
 @@ -0,0 +1,241 @@
+/*
+ * 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;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.collect.HashMultimap;
+import com.google.common.collect.Multimaps;
+import com.google.common.collect.SetMultimap;
+import com.google.common.util.concurrent.ListenableFuture;
+import io.github.resilience4j.bulkhead.Bulkhead;
+import io.github.resilience4j.bulkhead.BulkheadConfig;
+import io.github.resilience4j.bulkhead.BulkheadRegistry;
+import it.unimi.dsi.fastutil.objects.Object2IntMap;
+import org.apache.druid.client.SegmentServerSelector;
+import org.apache.druid.java.util.common.concurrent.Execs;
+import org.apache.druid.java.util.common.guava.Sequence;
+import org.apache.druid.query.Query;
+import org.apache.druid.query.QueryContexts;
+import org.apache.druid.query.QueryPlus;
+import org.apache.druid.query.QueryWatcher;
+import org.apache.druid.server.initialization.ServerConfig;
+
+import java.time.Duration;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+
+/**
+ * QueryScheduler (potentially) assigns any {@link Query} that is to be executed to a 'query lane' using the
+ * {@link QueryLaningStrategy} that is defined in {@link QuerySchedulerConfig}.
+ *
+ * As a {@link QueryWatcher}, it also provides cancellation facilities.
+ *
+ * This class is shared by all requests on the Jetty HTTP theadpool and must be thread safe.
+ */
+public class QueryScheduler implements QueryWatcher
+{
+  static final String TOTAL = "default";
+  private final int totalCapacity;
+  private final QueryLaningStrategy laningStrategy;
+  private final BulkheadRegistry laneRegistry;
+  private final SetMultimap<String, ListenableFuture<?>> queryFutures;
 
 Review comment:
   added javadoc

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] ccaominh commented on a change in pull request #9407: query laning and load shedding

Posted by GitBox <gi...@apache.org>.
ccaominh commented on a change in pull request #9407: query laning and load shedding
URL: https://github.com/apache/druid/pull/9407#discussion_r389073782
 
 

 ##########
 File path: server/src/main/java/org/apache/druid/server/QueryScheduler.java
 ##########
 @@ -0,0 +1,226 @@
+/*
+ * 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;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.collect.HashMultimap;
+import com.google.common.collect.Multimaps;
+import com.google.common.collect.SetMultimap;
+import com.google.common.util.concurrent.ListenableFuture;
+import io.github.resilience4j.bulkhead.Bulkhead;
+import io.github.resilience4j.bulkhead.BulkheadConfig;
+import io.github.resilience4j.bulkhead.BulkheadRegistry;
+import it.unimi.dsi.fastutil.objects.Object2IntMap;
+import org.apache.druid.client.SegmentServerSelector;
+import org.apache.druid.java.util.common.concurrent.Execs;
+import org.apache.druid.java.util.common.guava.Sequence;
+import org.apache.druid.query.Query;
+import org.apache.druid.query.QueryContexts;
+import org.apache.druid.query.QueryPlus;
+import org.apache.druid.query.QueryWatcher;
+
+import java.time.Duration;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+
+/**
+ * QueryScheduler (potentially) assigns any {@link Query} that is to be executed to a 'query lane' using the
+ * {@link QueryLaningStrategy} that is defined in {@link QuerySchedulerConfig}.
+ *
+ * As a {@link QueryWatcher}, it also provides cancellation facilities.
+ *
+ * This class is shared by all requests on the Jetty HTTP theadpool and must be thread safe.
+ */
+public class QueryScheduler implements QueryWatcher
+{
+  static final String TOTAL = "default";
+  private final QueryLaningStrategy laningStrategy;
+  private final BulkheadRegistry laneRegistry;
+  private final SetMultimap<String, ListenableFuture<?>> queryFutures;
+  private final SetMultimap<String, String> queryDatasources;
+
+  public QueryScheduler(int totalNumThreads, QueryLaningStrategy laningStrategy)
+  {
+    this.laningStrategy = laningStrategy;
+    this.laneRegistry = BulkheadRegistry.of(getLaneConfigs(totalNumThreads));
+    this.queryFutures = Multimaps.synchronizedSetMultimap(HashMultimap.create());
+    this.queryDatasources = Multimaps.synchronizedSetMultimap(HashMultimap.create());
+  }
+
+  @Override
+  public void registerQueryFuture(Query<?> query, ListenableFuture<?> future)
+  {
+    final String id = query.getId();
+    final Set<String> datasources = query.getDataSource().getTableNames();
+    queryFutures.put(id, future);
+    queryDatasources.putAll(id, datasources);
+    future.addListener(
+        () -> {
+          queryFutures.remove(id, future);
+          for (String datasource : datasources) {
+            queryDatasources.remove(id, datasource);
+          }
+        },
+        Execs.directExecutor()
+    );
+  }
+
+  /**
+   * Assign a query a lane (if not set)
+   */
+  public <T> Query<T> laneQuery(QueryPlus<T> queryPlus, Set<SegmentServerSelector> segments)
+  {
+    Query<T> query = queryPlus.getQuery();
+    // man wins over machine.. for now.
+    if (QueryContexts.getLane(query) != null) {
+      return query;
+    }
+    Optional<String> lane = laningStrategy.computeLane(queryPlus, segments);
+    return lane.map(query::withLane).orElse(query);
+  }
+
+  /**
+   * Run a query with the scheduler, attempting to acquire a semaphore from the total and lane specific query capacities
+   *
+   * Note that {@link #cancelQuery} should not interrupt the thread that calls run, in all current usages it only
+   * cancels any {@link ListenableFuture} created downstream. If this ever commonly changes, we should add
+   * synchronization between {@link #cancelQuery} and the acquisition of the {@link Bulkhead} to continue to ensure that
+   * anything acquired is also released.
+   *
+   * In the meantime, if a {@link ListenableFuture} is registered for the query that calls this method, it MUST handle
+   * this synchronization itself to ensure that no {@link Bulkhead} is acquired without releasing it.
+   */
+  public <T> Sequence<T> run(Query<?> query, Sequence<T> resultSequence)
+  {
+    List<Bulkhead> bulkheads = acquireLanes(query);
+    return resultSequence.withBaggage(() -> releaseLanes(bulkheads));
+  }
+
+  /**
+   * Forcibly cancel all futures that have been registered to a specific query id
+   */
+  public boolean cancelQuery(String id)
+  {
+    // if you re-use queryId and cancel queries... you are going to have a bad time
+    queryDatasources.removeAll(id);
+    Set<ListenableFuture<?>> futures = queryFutures.removeAll(id);
+    boolean success = true;
+    for (ListenableFuture<?> future : futures) {
+      success = success && future.cancel(true);
+    }
+    return success;
+  }
+
+  /**
+   * Get a {@link Set} of datasource names for a {@link Query} id, used by {@link QueryResource#cancelQuery} to
+   * authorize that a user may call {@link #cancelQuery} for the given id and datasources
+   */
+  public Set<String> getQueryDatasources(final String queryId)
+  {
+    return queryDatasources.get(queryId);
+  }
+
+  /**
+   * Get the maximum number of concurrent queries that {@link #run} can support
+   */
+  public int getTotalAvailableCapacity()
+  {
+    return laneRegistry.getConfiguration(TOTAL)
+                       .map(config -> laneRegistry.bulkhead(TOTAL, config).getMetrics().getAvailableConcurrentCalls())
+                       .orElse(-1);
+  }
+
+  /**
+   * Get the maximum number of concurrent queries that {@link #run} can support for a given lane
+   */
+  public int getLaneAvailableCapacity(String lane)
+  {
+    return laneRegistry.getConfiguration(lane)
+                       .map(config -> laneRegistry.bulkhead(lane, config).getMetrics().getAvailableConcurrentCalls())
+                       .orElse(-1);
+  }
+
+  /**
+   * Acquire a semaphore for both the 'total' and a lane, if any is associated with a query
+   */
+  @VisibleForTesting
+  List<Bulkhead> acquireLanes(Query<?> query)
+  {
+    final String lane = QueryContexts.getLane(query);
+    final Optional<BulkheadConfig> laneConfig = lane == null ? Optional.empty() : laneRegistry.getConfiguration(lane);
+    List<Bulkhead> hallPasses = new ArrayList<>(2);
+    final Optional<BulkheadConfig> totalConfig = laneRegistry.getConfiguration(TOTAL);
+    // if we have a lane, get it first
+    laneConfig.ifPresent(config -> {
+      Bulkhead laneLimiter = laneRegistry.bulkhead(lane, config);
+      if (!laneLimiter.tryAcquirePermission()) {
+        throw new QueryCapacityExceededException(lane);
+      }
+      hallPasses.add(laneLimiter);
+    });
+
+    // everyone needs to take one from the total lane; to ensure we don't acquire a lane and never release it, we want
 
 Review comment:
   I was thinking that if there are only counters per lane, then the code would be simpler as the lane and total counter would not have to be kept consistent.

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] ccaominh commented on a change in pull request #9407: query laning and load shedding

Posted by GitBox <gi...@apache.org>.
ccaominh commented on a change in pull request #9407: query laning and load shedding
URL: https://github.com/apache/druid/pull/9407#discussion_r387991784
 
 

 ##########
 File path: processing/src/main/java/org/apache/druid/query/QueryContexts.java
 ##########
 @@ -35,6 +35,7 @@
 public class QueryContexts
 {
   public static final String PRIORITY_KEY = "priority";
+  public static final String LANE_KEY = "lane";
 
 Review comment:
   Was this added to the docs?

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] clintropolis commented on issue #9407: query laning and load shedding

Posted by GitBox <gi...@apache.org>.
clintropolis commented on issue #9407: query laning and load shedding
URL: https://github.com/apache/druid/pull/9407#issuecomment-596999295
 
 
   Thanks for taking a look @himanshug!

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] clintropolis commented on a change in pull request #9407: query laning and load shedding

Posted by GitBox <gi...@apache.org>.
clintropolis commented on a change in pull request #9407: query laning and load shedding
URL: https://github.com/apache/druid/pull/9407#discussion_r388755534
 
 

 ##########
 File path: processing/src/main/java/org/apache/druid/query/QueryContexts.java
 ##########
 @@ -35,6 +35,7 @@
 public class QueryContexts
 {
   public static final String PRIORITY_KEY = "priority";
+  public static final String LANE_KEY = "lane";
 
 Review comment:
   No, I hadn't documented since I hadn't decided the behavior of whether or not a lane specified in the query context by the user should override the laning strategy, or if it should be laning strategy specific, see other comment about this

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] clintropolis commented on a change in pull request #9407: query laning and load shedding

Posted by GitBox <gi...@apache.org>.
clintropolis commented on a change in pull request #9407: query laning and load shedding
URL: https://github.com/apache/druid/pull/9407#discussion_r390037318
 
 

 ##########
 File path: docs/configuration/index.md
 ##########
 @@ -1476,9 +1476,35 @@ These Broker configurations can be defined in the `broker/runtime.properties` fi
 |`druid.broker.select.tier`|`highestPriority`, `lowestPriority`, `custom`|If segments are cross-replicated across tiers in a cluster, you can tell the broker to prefer to select segments in a tier with a certain priority.|`highestPriority`|
 |`druid.broker.select.tier.custom.priorities`|`An array of integer priorities.`|Select servers in tiers with a custom priority list.|None|
 
+##### Query laning
+
+The Broker provides facilities to aid in query capacity reservation for heterogeneous query workloads in the form of 'laning' strategies, which provide a variety of mechanisms to examine and classify a query, assigning it to a 'lane'. Lanes are defined with capacity limits which the broker will enforce, causing requests in excess of the capacity to be discarded with an HTTP 429 status code, reserving resources for other lanes or for interactive queries (with no lane).
 
 Review comment:
   thanks, applied

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] ccaominh commented on a change in pull request #9407: query laning and load shedding

Posted by GitBox <gi...@apache.org>.
ccaominh commented on a change in pull request #9407: query laning and load shedding
URL: https://github.com/apache/druid/pull/9407#discussion_r388666582
 
 

 ##########
 File path: sql/src/test/java/org/apache/druid/sql/calcite/util/CalciteTests.java
 ##########
 @@ -700,6 +701,14 @@ public static SpecificSegmentsQuerySegmentWalker createMockWalker(
       final QueryRunnerFactoryConglomerate conglomerate,
       final File tmpDir
   )
+  {
+    return createMockWalker(conglomerate, tmpDir, null);
+  }
+  public static SpecificSegmentsQuerySegmentWalker createMockWalker(
 
 Review comment:
   Missing blank line

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] ccaominh commented on a change in pull request #9407: query laning and load shedding

Posted by GitBox <gi...@apache.org>.
ccaominh commented on a change in pull request #9407: query laning and load shedding
URL: https://github.com/apache/druid/pull/9407#discussion_r388632256
 
 

 ##########
 File path: server/src/main/java/org/apache/druid/server/QueryScheduler.java
 ##########
 @@ -0,0 +1,226 @@
+/*
+ * 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;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.collect.HashMultimap;
+import com.google.common.collect.Multimaps;
+import com.google.common.collect.SetMultimap;
+import com.google.common.util.concurrent.ListenableFuture;
+import io.github.resilience4j.bulkhead.Bulkhead;
+import io.github.resilience4j.bulkhead.BulkheadConfig;
+import io.github.resilience4j.bulkhead.BulkheadRegistry;
+import it.unimi.dsi.fastutil.objects.Object2IntMap;
+import org.apache.druid.client.SegmentServerSelector;
+import org.apache.druid.java.util.common.concurrent.Execs;
+import org.apache.druid.java.util.common.guava.Sequence;
+import org.apache.druid.query.Query;
+import org.apache.druid.query.QueryContexts;
+import org.apache.druid.query.QueryPlus;
+import org.apache.druid.query.QueryWatcher;
+
+import java.time.Duration;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+
+/**
+ * QueryScheduler (potentially) assigns any {@link Query} that is to be executed to a 'query lane' using the
+ * {@link QueryLaningStrategy} that is defined in {@link QuerySchedulerConfig}.
+ *
+ * As a {@link QueryWatcher}, it also provides cancellation facilities.
+ *
+ * This class is shared by all requests on the Jetty HTTP theadpool and must be thread safe.
+ */
+public class QueryScheduler implements QueryWatcher
+{
+  static final String TOTAL = "default";
+  private final QueryLaningStrategy laningStrategy;
+  private final BulkheadRegistry laneRegistry;
+  private final SetMultimap<String, ListenableFuture<?>> queryFutures;
+  private final SetMultimap<String, String> queryDatasources;
+
+  public QueryScheduler(int totalNumThreads, QueryLaningStrategy laningStrategy)
+  {
+    this.laningStrategy = laningStrategy;
+    this.laneRegistry = BulkheadRegistry.of(getLaneConfigs(totalNumThreads));
+    this.queryFutures = Multimaps.synchronizedSetMultimap(HashMultimap.create());
+    this.queryDatasources = Multimaps.synchronizedSetMultimap(HashMultimap.create());
+  }
+
+  @Override
+  public void registerQueryFuture(Query<?> query, ListenableFuture<?> future)
+  {
+    final String id = query.getId();
+    final Set<String> datasources = query.getDataSource().getTableNames();
+    queryFutures.put(id, future);
+    queryDatasources.putAll(id, datasources);
+    future.addListener(
+        () -> {
+          queryFutures.remove(id, future);
+          for (String datasource : datasources) {
+            queryDatasources.remove(id, datasource);
+          }
+        },
+        Execs.directExecutor()
+    );
+  }
+
+  /**
+   * Assign a query a lane (if not set)
+   */
+  public <T> Query<T> laneQuery(QueryPlus<T> queryPlus, Set<SegmentServerSelector> segments)
+  {
+    Query<T> query = queryPlus.getQuery();
+    // man wins over machine.. for now.
+    if (QueryContexts.getLane(query) != null) {
+      return query;
+    }
+    Optional<String> lane = laningStrategy.computeLane(queryPlus, segments);
+    return lane.map(query::withLane).orElse(query);
+  }
+
+  /**
+   * Run a query with the scheduler, attempting to acquire a semaphore from the total and lane specific query capacities
+   *
+   * Note that {@link #cancelQuery} should not interrupt the thread that calls run, in all current usages it only
+   * cancels any {@link ListenableFuture} created downstream. If this ever commonly changes, we should add
+   * synchronization between {@link #cancelQuery} and the acquisition of the {@link Bulkhead} to continue to ensure that
+   * anything acquired is also released.
+   *
+   * In the meantime, if a {@link ListenableFuture} is registered for the query that calls this method, it MUST handle
+   * this synchronization itself to ensure that no {@link Bulkhead} is acquired without releasing it.
+   */
+  public <T> Sequence<T> run(Query<?> query, Sequence<T> resultSequence)
+  {
+    List<Bulkhead> bulkheads = acquireLanes(query);
+    return resultSequence.withBaggage(() -> releaseLanes(bulkheads));
+  }
+
+  /**
+   * Forcibly cancel all futures that have been registered to a specific query id
+   */
+  public boolean cancelQuery(String id)
+  {
+    // if you re-use queryId and cancel queries... you are going to have a bad time
+    queryDatasources.removeAll(id);
+    Set<ListenableFuture<?>> futures = queryFutures.removeAll(id);
+    boolean success = true;
+    for (ListenableFuture<?> future : futures) {
+      success = success && future.cancel(true);
+    }
+    return success;
+  }
+
+  /**
+   * Get a {@link Set} of datasource names for a {@link Query} id, used by {@link QueryResource#cancelQuery} to
+   * authorize that a user may call {@link #cancelQuery} for the given id and datasources
+   */
+  public Set<String> getQueryDatasources(final String queryId)
+  {
+    return queryDatasources.get(queryId);
+  }
+
+  /**
+   * Get the maximum number of concurrent queries that {@link #run} can support
+   */
+  public int getTotalAvailableCapacity()
+  {
+    return laneRegistry.getConfiguration(TOTAL)
+                       .map(config -> laneRegistry.bulkhead(TOTAL, config).getMetrics().getAvailableConcurrentCalls())
+                       .orElse(-1);
 
 Review comment:
   Consider adding a named constant for the `-1` (similar for `getLaneAvailableCapacity`)

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] clintropolis commented on a change in pull request #9407: query laning and load shedding

Posted by GitBox <gi...@apache.org>.
clintropolis commented on a change in pull request #9407: query laning and load shedding
URL: https://github.com/apache/druid/pull/9407#discussion_r389214087
 
 

 ##########
 File path: server/src/main/java/org/apache/druid/server/QueryCapacityExceededException.java
 ##########
 @@ -0,0 +1,53 @@
+/*
+ * 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;
+
+import com.fasterxml.jackson.annotation.JsonCreator;
+import com.fasterxml.jackson.annotation.JsonProperty;
+import org.apache.druid.java.util.common.StringUtils;
+import org.apache.druid.query.QueryException;
+
+public class QueryCapacityExceededException extends QueryException
+{
+  private static final String ERROR_CLASS = QueryCapacityExceededException.class.getName();
+  public static final String ERROR_CODE = "Query capacity exceeded";
+  public static final String ERROR_MESSAGE = "Total query capacity exceeded";
+  public static final String ERROR_MESSAGE_TEMPLATE = "Query capacity exceeded for lane %s";
 
 Review comment:
   This seems like maybe a good idea, but I am also afraid it might be a little chatty. I think people would expect to see these error messages if they configure limits, especially since there are no default limits. I think is maybe a bit different than the message you referenced, which I think is more likely something that would be bumped into by accident while trying to query something too big. Maybe if i document the error responses to expect in the configuration documentation that would be useful enough?

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] ccaominh commented on a change in pull request #9407: query laning and load shedding

Posted by GitBox <gi...@apache.org>.
ccaominh commented on a change in pull request #9407: query laning and load shedding
URL: https://github.com/apache/druid/pull/9407#discussion_r388663186
 
 

 ##########
 File path: server/src/test/java/org/apache/druid/server/QuerySchedulerTest.java
 ##########
 @@ -0,0 +1,566 @@
+/*
+ * 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;
+
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.ImmutableMap;
+import com.google.common.collect.ImmutableSet;
+import com.google.common.util.concurrent.ListenableFuture;
+import com.google.common.util.concurrent.ListeningExecutorService;
+import com.google.common.util.concurrent.MoreExecutors;
+import com.google.inject.Injector;
+import io.github.resilience4j.bulkhead.Bulkhead;
+import org.apache.druid.guice.GuiceInjectors;
+import org.apache.druid.guice.JsonConfigProvider;
+import org.apache.druid.guice.JsonConfigurator;
+import org.apache.druid.guice.annotations.Global;
+import org.apache.druid.java.util.common.StringUtils;
+import org.apache.druid.java.util.common.concurrent.Execs;
+import org.apache.druid.java.util.common.guava.BaseSequence;
+import org.apache.druid.java.util.common.guava.LazySequence;
+import org.apache.druid.java.util.common.guava.Sequence;
+import org.apache.druid.java.util.common.guava.SequenceWrapper;
+import org.apache.druid.java.util.common.guava.Sequences;
+import org.apache.druid.java.util.common.guava.Yielder;
+import org.apache.druid.java.util.common.guava.Yielders;
+import org.apache.druid.query.Query;
+import org.apache.druid.query.QueryContexts;
+import org.apache.druid.query.QueryPlus;
+import org.apache.druid.query.aggregation.CountAggregatorFactory;
+import org.apache.druid.query.topn.TopNQuery;
+import org.apache.druid.query.topn.TopNQueryBuilder;
+import org.apache.druid.server.scheduling.HiLoQueryLaningStrategy;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.ExpectedException;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Properties;
+import java.util.UUID;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.Future;
+import java.util.concurrent.ThreadLocalRandom;
+import java.util.concurrent.atomic.AtomicLong;
+
+public class QuerySchedulerTest
+{
+  private static final int NUM_CONCURRENT_QUERIES = 10000;
+  private static final int NUM_ROWS = 10000;
+
+  @Rule
+  public ExpectedException expected = ExpectedException.none();
+
+  private ListeningExecutorService executorService;
+  private QueryScheduler scheduler;
+
+  private AtomicLong totalAcquired;
+  private AtomicLong totalReleased;
+  private AtomicLong laneAcquired;
+  private AtomicLong laneNotAcquired;
+  private AtomicLong laneReleased;
+
+  @Before
+  public void setup()
+  {
+    executorService = MoreExecutors.listeningDecorator(
+        Execs.multiThreaded(8, "test_query_scheduler_%s")
+    );
+    totalAcquired = new AtomicLong();
+    totalReleased = new AtomicLong();
+    laneAcquired = new AtomicLong();
+    laneNotAcquired = new AtomicLong();
+    laneReleased = new AtomicLong();
+    scheduler = new QueryScheduler(5, new HiLoQueryLaningStrategy(2)) {
+      @Override
+      List<Bulkhead> acquireLanes(Query<?> query)
+      {
+        List<Bulkhead> bulkheads = super.acquireLanes(query);
+        if (bulkheads.stream().anyMatch(b -> b.getName().equals(QueryScheduler.TOTAL))) {
+          totalAcquired.incrementAndGet();
+        }
+        if (bulkheads.stream().anyMatch(b -> !b.getName().equals(QueryScheduler.TOTAL))) {
+          laneAcquired.incrementAndGet();
+        }
+
+        return bulkheads;
+      }
+
+      @Override
+      void releaseLanes(List<Bulkhead> bulkheads)
+      {
+        super.releaseLanes(bulkheads);
+        if (bulkheads.stream().anyMatch(b -> b.getName().equals(QueryScheduler.TOTAL))) {
+          totalReleased.incrementAndGet();
+        }
+        if (bulkheads.stream().anyMatch(b -> !b.getName().equals(QueryScheduler.TOTAL))) {
+          laneReleased.incrementAndGet();
+          if (bulkheads.size() == 1) {
+            laneNotAcquired.incrementAndGet();
+          }
+        }
+      }
+    };
+  }
+
+  @After
+  public void teardown()
+  {
+    executorService.shutdownNow();
+  }
+
+  @Test
+  public void testHiLoHi() throws ExecutionException, InterruptedException
+  {
+    TopNQuery interactive = makeInteractiveQuery();
+    ListenableFuture<?> future = executorService.submit(() -> {
+      try {
+        Query<?> scheduled = scheduler.laneQuery(QueryPlus.wrap(interactive), ImmutableSet.of());
+
+        Assert.assertNotNull(scheduled);
+
+        Sequence<Integer> underlyingSequence = makeSequence(10);
+        underlyingSequence = Sequences.wrap(underlyingSequence, new SequenceWrapper()
+        {
+          @Override
+          public void before()
+          {
+            Assert.assertEquals(4, scheduler.getTotalAvailableCapacity());
+            Assert.assertEquals(2, scheduler.getLaneAvailableCapacity(HiLoQueryLaningStrategy.LOW));
+          }
+        });
+        Sequence<Integer> results = scheduler.run(scheduled, underlyingSequence);
+        int rowCount = consumeAndCloseSequence(results);
+
+        Assert.assertEquals(10, rowCount);
+      }
+      catch (IOException ex) {
+        throw new RuntimeException(ex);
+      }
+    });
+    future.get();
+    Assert.assertEquals(5, scheduler.getTotalAvailableCapacity());
+  }
+
+  @Test
+  public void testHiLoLo() throws ExecutionException, InterruptedException
+  {
+    TopNQuery report = makeReportQuery();
+    ListenableFuture<?> future = executorService.submit(() -> {
+      try {
+        Query<?> scheduledReport = scheduler.laneQuery(QueryPlus.wrap(report), ImmutableSet.of());
+        Assert.assertNotNull(scheduledReport);
+        Assert.assertEquals(HiLoQueryLaningStrategy.LOW, QueryContexts.getLane(scheduledReport));
+
+        Sequence<Integer> underlyingSequence = makeSequence(10);
+        underlyingSequence = Sequences.wrap(underlyingSequence, new SequenceWrapper()
+        {
+          @Override
+          public void before()
+          {
+            Assert.assertEquals(4, scheduler.getTotalAvailableCapacity());
+            Assert.assertEquals(1, scheduler.getLaneAvailableCapacity(HiLoQueryLaningStrategy.LOW));
+          }
+        });
+        Sequence<Integer> results = scheduler.run(scheduledReport, underlyingSequence);
+
+        int rowCount = consumeAndCloseSequence(results);
+        Assert.assertEquals(10, rowCount);
+      }
+      catch (IOException ex) {
+        throw new RuntimeException(ex);
+      }
+    });
+    future.get();
+    Assert.assertEquals(5, scheduler.getTotalAvailableCapacity());
+    Assert.assertEquals(2, scheduler.getLaneAvailableCapacity(HiLoQueryLaningStrategy.LOW));
+  }
+
+  @Test
+  public void testHiLoReleaseSemaphoreWhenSequenceExplodes() throws Exception
+  {
+    expected.expectMessage("exploded");
+    expected.expect(ExecutionException.class);
+    TopNQuery interactive = makeInteractiveQuery();
+    ListenableFuture<?> future = executorService.submit(() -> {
+      try {
+        Query<?> scheduled = scheduler.laneQuery(QueryPlus.wrap(interactive), ImmutableSet.of());
+
+        Assert.assertNotNull(scheduled);
+
+        Sequence<Integer> underlyingSequence = makeExplodingSequence(10);
+        underlyingSequence = Sequences.wrap(underlyingSequence, new SequenceWrapper()
+        {
+          @Override
+          public void before()
+          {
+            Assert.assertEquals(4, scheduler.getTotalAvailableCapacity());
+          }
+        });
+        Sequence<Integer> results = scheduler.run(scheduled, underlyingSequence);
+
+        consumeAndCloseSequence(results);
+      }
+      catch (IOException ex) {
+        throw new RuntimeException(ex);
+      }
+    });
+    future.get();
+    Assert.assertEquals(5, scheduler.getTotalAvailableCapacity());
 
 Review comment:
   This assert is not executed. Maybe use an exception handler instead of `expected`?

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] clintropolis commented on a change in pull request #9407: query laning and load shedding

Posted by GitBox <gi...@apache.org>.
clintropolis commented on a change in pull request #9407: query laning and load shedding
URL: https://github.com/apache/druid/pull/9407#discussion_r390198573
 
 

 ##########
 File path: docs/configuration/index.md
 ##########
 @@ -1481,9 +1481,35 @@ These Broker configurations can be defined in the `broker/runtime.properties` fi
 |`druid.broker.select.tier`|`highestPriority`, `lowestPriority`, `custom`|If segments are cross-replicated across tiers in a cluster, you can tell the broker to prefer to select segments in a tier with a certain priority.|`highestPriority`|
 |`druid.broker.select.tier.custom.priorities`|`An array of integer priorities.`|Select servers in tiers with a custom priority list.|None|
 
+##### Query laning
+
+*Laning strategies* allow you to control capacity utilization for heterogeneous query workloads. With laning, the broker examines and classifies a query for the purpose of assigning it to a 'lane'. Lanes have capacity limits, enforced by the broker, that can be used to ensure sufficient resources are available for other lanes or for interactive queries (with no lane), or to limit overall throughput for queries within the lane. Requests in excess of the capacity are discarded with an HTTP 429 status code.
+
+|Property|Description|Default|
+|--------|-----------|-------|
+|`druid.query.scheduler.numThreads`|Maximum number of HTTP threads to dedicate to query processing. To save HTTP thread capacity, this should be lower than `druid.server.http.numThreads`.|Unbounded|
 
 Review comment:
   >in what use case would I ever want to set it something other than druid.server.http.numThreads ?
   
   I actually think we might always want to set it lower than `druid.server.http.numThreads`, but I was too nervous to make this the default and made it opt in behavior instead (since it grabs and releases locks for each query if there is some bug in releasing locks a broker would eventually stop accepting queries entirely). The primary reason I think we want it lower than `druid.server.http.numThreads` is to save some 'slack' space for non-query http connections, like accepting health checks, lookup management, and other such things that can be starved when long running queries start to pile up.
   
   >If the recommendation is to set it lower than `druid.server.http.numThreads` then why the default value is not set to `druid.server.http.numThreads - 1` ?
   
   See my above nervousness, but I think `druid.server.http.numThreads - 1` would probably be a good default. This might want to be adjusted to be even lower depending on how much other non query http traffic the server is receiving (e.g. frequently polled/updated lookups, etc).
   
   >I guess, as a user I don't quite understand the importance of setting this higher/same/lower compared to druid.server.http.numThreads and when I should choose one vs the other.
   
   I would agree the current documentation doesn't quite adequately describe how this stuff might be utilized, in a future PR i want to add a section to [cluster tuning docs](https://github.com/apache/druid/blob/master/docs/operations/basic-cluster-tuning.md) to more properly advise on when and how to set this stuff up.

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] ccaominh commented on a change in pull request #9407: query laning and load shedding

Posted by GitBox <gi...@apache.org>.
ccaominh commented on a change in pull request #9407: query laning and load shedding
URL: https://github.com/apache/druid/pull/9407#discussion_r388016279
 
 

 ##########
 File path: docs/configuration/index.md
 ##########
 @@ -1476,9 +1476,31 @@ These Broker configurations can be defined in the `broker/runtime.properties` fi
 |`druid.broker.select.tier`|`highestPriority`, `lowestPriority`, `custom`|If segments are cross-replicated across tiers in a cluster, you can tell the broker to prefer to select segments in a tier with a certain priority.|`highestPriority`|
 |`druid.broker.select.tier.custom.priorities`|`An array of integer priorities.`|Select servers in tiers with a custom priority list.|None|
 
+##### Query laning
+
+The Broker provides facilities to aid in query capacity reservation for heterogeneous query workloads in the form of 'laning' strategies, which provide a variety of mechanisms to examine and classify a query, assigning it to a 'lane'. Lanes are defined with capacity limits which the broker will enforce, causing requests in excess of the capacity to be discarded with an HTTP 429 status code, reserving resources for other lanes or for interactive queries (with no lane).
+
+|Property|Description|Default|
+|--------|-----------|-------|
+|`druid.query.scheduler.numThreads`|Maximum number of HTTP threads to dedicate to query processing. To save HTTP thread capacity, this should be lower than `druid.server.http.numThreads`.|Unbounded|
+|`druid.query.scheduler.laning.type`|Query laning strategy to use to assign queries to a lane in order to control capacities for certain classes of queries.|`none`|
+
+###### No laning strategy
+
+In this mode, queries are never assigned a lane and only limited by `druid.server.http.numThreads` or `druid.query.scheduler.numThreads`, if set. This is the default Druid query scheduler operating mode.
+
+###### 'High/Low' laning strategy
+This laning strategy splits queries with a `priority` below zero into a `low` query lane, automatically. The limit on `low` queries can be set to some desired fraction of the total capacity (or HTTP thread pool size), reserving capacity for interactive queries. Queries in the `low` lane are _not_ guaranteed their capacity, which may be consumed by interactive queries, but may use up to this limit if total capacity is available. 
+
+This strategy can be enabled by setting `druid.query.scheduler.laning.type` to `hilo`.
+
+|Property|Description|Default|
+|--------|-----------|-------|
+|`druid.query.scheduler.laning.maxLowThreads`|Maximum number of HTTP threads that can be used by queries with a priority lower than 0.|No default, must be set if using this mode|
 
 Review comment:
   Perhaps rename to `maxLowHttpThreads` so that it's not confused with the number of processing threads

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] clintropolis commented on a change in pull request #9407: query laning and load shedding

Posted by GitBox <gi...@apache.org>.
clintropolis commented on a change in pull request #9407: query laning and load shedding
URL: https://github.com/apache/druid/pull/9407#discussion_r389219211
 
 

 ##########
 File path: server/src/main/java/org/apache/druid/server/QueryScheduler.java
 ##########
 @@ -0,0 +1,241 @@
+/*
+ * 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;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.collect.HashMultimap;
+import com.google.common.collect.Multimaps;
+import com.google.common.collect.SetMultimap;
+import com.google.common.util.concurrent.ListenableFuture;
+import io.github.resilience4j.bulkhead.Bulkhead;
+import io.github.resilience4j.bulkhead.BulkheadConfig;
+import io.github.resilience4j.bulkhead.BulkheadRegistry;
+import it.unimi.dsi.fastutil.objects.Object2IntMap;
+import org.apache.druid.client.SegmentServerSelector;
+import org.apache.druid.java.util.common.concurrent.Execs;
+import org.apache.druid.java.util.common.guava.Sequence;
+import org.apache.druid.query.Query;
+import org.apache.druid.query.QueryContexts;
+import org.apache.druid.query.QueryPlus;
+import org.apache.druid.query.QueryWatcher;
+import org.apache.druid.server.initialization.ServerConfig;
+
+import java.time.Duration;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+
+/**
+ * QueryScheduler (potentially) assigns any {@link Query} that is to be executed to a 'query lane' using the
+ * {@link QueryLaningStrategy} that is defined in {@link QuerySchedulerConfig}.
+ *
+ * As a {@link QueryWatcher}, it also provides cancellation facilities.
+ *
+ * This class is shared by all requests on the Jetty HTTP theadpool and must be thread safe.
+ */
+public class QueryScheduler implements QueryWatcher
+{
+  static final String TOTAL = "default";
+  private final int totalCapacity;
+  private final QueryLaningStrategy laningStrategy;
+  private final BulkheadRegistry laneRegistry;
+  private final SetMultimap<String, ListenableFuture<?>> queryFutures;
+  private final SetMultimap<String, String> queryDatasources;
+
+  public QueryScheduler(int totalNumThreads, QueryLaningStrategy laningStrategy, ServerConfig serverConfig)
+  {
+    this.laningStrategy = laningStrategy;
+    this.queryFutures = Multimaps.synchronizedSetMultimap(HashMultimap.create());
+    this.queryDatasources = Multimaps.synchronizedSetMultimap(HashMultimap.create());
+    // if totalNumThreads is above 0 and less than druid.server.http.numThreads, enforce total limit
+    final boolean limitTotal;
+    if (totalNumThreads > 0 && totalNumThreads < serverConfig.getNumThreads()) {
+      limitTotal = true;
+      this.totalCapacity = totalNumThreads;
+    } else {
+      limitTotal = false;
+      this.totalCapacity = serverConfig.getNumThreads();
+    }
+    this.laneRegistry = BulkheadRegistry.of(getLaneConfigs(limitTotal));
+  }
+
+  @Override
+  public void registerQueryFuture(Query<?> query, ListenableFuture<?> future)
+  {
+    final String id = query.getId();
+    final Set<String> datasources = query.getDataSource().getTableNames();
+    queryFutures.put(id, future);
+    queryDatasources.putAll(id, datasources);
+    future.addListener(
+        () -> {
+          queryFutures.remove(id, future);
+          for (String datasource : datasources) {
+            queryDatasources.remove(id, datasource);
+          }
+        },
+        Execs.directExecutor()
+    );
+  }
+
+  /**
+   * Assign a query a lane (if not set)
+   */
+  public <T> Query<T> laneQuery(QueryPlus<T> queryPlus, Set<SegmentServerSelector> segments)
+  {
+    Query<T> query = queryPlus.getQuery();
+    // man wins over machine.. for now.
+    if (QueryContexts.getLane(query) != null) {
+      return query;
+    }
+    Optional<String> lane = laningStrategy.computeLane(queryPlus, segments);
+    return lane.map(query::withLane).orElse(query);
+  }
+
+  /**
+   * Run a query with the scheduler, attempting to acquire a semaphore from the total and lane specific query capacities
+   *
+   * Note that {@link #cancelQuery} should not interrupt the thread that calls run, in all current usages it only
+   * cancels any {@link ListenableFuture} created downstream. If this ever commonly changes, we should add
+   * synchronization between {@link #cancelQuery} and the acquisition of the {@link Bulkhead} to continue to ensure that
+   * anything acquired is also released.
+   *
+   * In the meantime, if a {@link ListenableFuture} is registered for the query that calls this method, it MUST handle
+   * this synchronization itself to ensure that no {@link Bulkhead} is acquired without releasing it.
+   */
+  public <T> Sequence<T> run(Query<?> query, Sequence<T> resultSequence)
+  {
+    List<Bulkhead> bulkheads = acquireLanes(query);
+    return resultSequence.withBaggage(() -> releaseLanes(bulkheads));
+  }
+
+  /**
+   * Forcibly cancel all futures that have been registered to a specific query id
+   */
+  public boolean cancelQuery(String id)
+  {
+    // if you re-use queryId and cancel queries... you are going to have a bad time
 
 Review comment:
   I adjusted the comment to try and clear up what i was talking about

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] ccaominh commented on issue #9407: query laning and load shedding

Posted by GitBox <gi...@apache.org>.
ccaominh commented on issue #9407: query laning and load shedding
URL: https://github.com/apache/druid/pull/9407#issuecomment-595551387
 
 
   Are there any useful diagnostics (e.g., log messages) that can be added?

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] ccaominh commented on a change in pull request #9407: query laning and load shedding

Posted by GitBox <gi...@apache.org>.
ccaominh commented on a change in pull request #9407: query laning and load shedding
URL: https://github.com/apache/druid/pull/9407#discussion_r388662051
 
 

 ##########
 File path: server/src/test/java/org/apache/druid/server/QuerySchedulerTest.java
 ##########
 @@ -0,0 +1,566 @@
+/*
+ * 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;
+
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.ImmutableMap;
+import com.google.common.collect.ImmutableSet;
+import com.google.common.util.concurrent.ListenableFuture;
+import com.google.common.util.concurrent.ListeningExecutorService;
+import com.google.common.util.concurrent.MoreExecutors;
+import com.google.inject.Injector;
+import io.github.resilience4j.bulkhead.Bulkhead;
+import org.apache.druid.guice.GuiceInjectors;
+import org.apache.druid.guice.JsonConfigProvider;
+import org.apache.druid.guice.JsonConfigurator;
+import org.apache.druid.guice.annotations.Global;
+import org.apache.druid.java.util.common.StringUtils;
+import org.apache.druid.java.util.common.concurrent.Execs;
+import org.apache.druid.java.util.common.guava.BaseSequence;
+import org.apache.druid.java.util.common.guava.LazySequence;
+import org.apache.druid.java.util.common.guava.Sequence;
+import org.apache.druid.java.util.common.guava.SequenceWrapper;
+import org.apache.druid.java.util.common.guava.Sequences;
+import org.apache.druid.java.util.common.guava.Yielder;
+import org.apache.druid.java.util.common.guava.Yielders;
+import org.apache.druid.query.Query;
+import org.apache.druid.query.QueryContexts;
+import org.apache.druid.query.QueryPlus;
+import org.apache.druid.query.aggregation.CountAggregatorFactory;
+import org.apache.druid.query.topn.TopNQuery;
+import org.apache.druid.query.topn.TopNQueryBuilder;
+import org.apache.druid.server.scheduling.HiLoQueryLaningStrategy;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.ExpectedException;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Properties;
+import java.util.UUID;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.Future;
+import java.util.concurrent.ThreadLocalRandom;
+import java.util.concurrent.atomic.AtomicLong;
+
+public class QuerySchedulerTest
+{
+  private static final int NUM_CONCURRENT_QUERIES = 10000;
+  private static final int NUM_ROWS = 10000;
+
+  @Rule
+  public ExpectedException expected = ExpectedException.none();
+
+  private ListeningExecutorService executorService;
+  private QueryScheduler scheduler;
+
+  private AtomicLong totalAcquired;
+  private AtomicLong totalReleased;
+  private AtomicLong laneAcquired;
+  private AtomicLong laneNotAcquired;
+  private AtomicLong laneReleased;
+
+  @Before
+  public void setup()
+  {
+    executorService = MoreExecutors.listeningDecorator(
+        Execs.multiThreaded(8, "test_query_scheduler_%s")
+    );
+    totalAcquired = new AtomicLong();
+    totalReleased = new AtomicLong();
+    laneAcquired = new AtomicLong();
+    laneNotAcquired = new AtomicLong();
+    laneReleased = new AtomicLong();
+    scheduler = new QueryScheduler(5, new HiLoQueryLaningStrategy(2)) {
+      @Override
+      List<Bulkhead> acquireLanes(Query<?> query)
+      {
+        List<Bulkhead> bulkheads = super.acquireLanes(query);
+        if (bulkheads.stream().anyMatch(b -> b.getName().equals(QueryScheduler.TOTAL))) {
+          totalAcquired.incrementAndGet();
+        }
+        if (bulkheads.stream().anyMatch(b -> !b.getName().equals(QueryScheduler.TOTAL))) {
+          laneAcquired.incrementAndGet();
+        }
+
+        return bulkheads;
+      }
+
+      @Override
+      void releaseLanes(List<Bulkhead> bulkheads)
+      {
+        super.releaseLanes(bulkheads);
+        if (bulkheads.stream().anyMatch(b -> b.getName().equals(QueryScheduler.TOTAL))) {
+          totalReleased.incrementAndGet();
+        }
+        if (bulkheads.stream().anyMatch(b -> !b.getName().equals(QueryScheduler.TOTAL))) {
+          laneReleased.incrementAndGet();
+          if (bulkheads.size() == 1) {
+            laneNotAcquired.incrementAndGet();
+          }
+        }
+      }
+    };
+  }
+
+  @After
+  public void teardown()
+  {
+    executorService.shutdownNow();
+  }
+
+  @Test
+  public void testHiLoHi() throws ExecutionException, InterruptedException
+  {
+    TopNQuery interactive = makeInteractiveQuery();
+    ListenableFuture<?> future = executorService.submit(() -> {
+      try {
+        Query<?> scheduled = scheduler.laneQuery(QueryPlus.wrap(interactive), ImmutableSet.of());
+
+        Assert.assertNotNull(scheduled);
+
+        Sequence<Integer> underlyingSequence = makeSequence(10);
+        underlyingSequence = Sequences.wrap(underlyingSequence, new SequenceWrapper()
+        {
+          @Override
+          public void before()
+          {
+            Assert.assertEquals(4, scheduler.getTotalAvailableCapacity());
+            Assert.assertEquals(2, scheduler.getLaneAvailableCapacity(HiLoQueryLaningStrategy.LOW));
+          }
+        });
+        Sequence<Integer> results = scheduler.run(scheduled, underlyingSequence);
+        int rowCount = consumeAndCloseSequence(results);
+
+        Assert.assertEquals(10, rowCount);
+      }
+      catch (IOException ex) {
+        throw new RuntimeException(ex);
+      }
+    });
+    future.get();
+    Assert.assertEquals(5, scheduler.getTotalAvailableCapacity());
+  }
+
+  @Test
+  public void testHiLoLo() throws ExecutionException, InterruptedException
+  {
+    TopNQuery report = makeReportQuery();
+    ListenableFuture<?> future = executorService.submit(() -> {
+      try {
+        Query<?> scheduledReport = scheduler.laneQuery(QueryPlus.wrap(report), ImmutableSet.of());
+        Assert.assertNotNull(scheduledReport);
+        Assert.assertEquals(HiLoQueryLaningStrategy.LOW, QueryContexts.getLane(scheduledReport));
+
+        Sequence<Integer> underlyingSequence = makeSequence(10);
+        underlyingSequence = Sequences.wrap(underlyingSequence, new SequenceWrapper()
+        {
+          @Override
+          public void before()
+          {
+            Assert.assertEquals(4, scheduler.getTotalAvailableCapacity());
+            Assert.assertEquals(1, scheduler.getLaneAvailableCapacity(HiLoQueryLaningStrategy.LOW));
+          }
+        });
+        Sequence<Integer> results = scheduler.run(scheduledReport, underlyingSequence);
+
+        int rowCount = consumeAndCloseSequence(results);
+        Assert.assertEquals(10, rowCount);
+      }
+      catch (IOException ex) {
+        throw new RuntimeException(ex);
+      }
+    });
+    future.get();
+    Assert.assertEquals(5, scheduler.getTotalAvailableCapacity());
+    Assert.assertEquals(2, scheduler.getLaneAvailableCapacity(HiLoQueryLaningStrategy.LOW));
+  }
+
+  @Test
+  public void testHiLoReleaseSemaphoreWhenSequenceExplodes() throws Exception
 
 Review comment:
   Maybe rename to "bulkhead"

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] jihoonson commented on a change in pull request #9407: query laning and load shedding

Posted by GitBox <gi...@apache.org>.
jihoonson commented on a change in pull request #9407: query laning and load shedding
URL: https://github.com/apache/druid/pull/9407#discussion_r389077822
 
 

 ##########
 File path: server/src/main/java/org/apache/druid/server/QueryScheduler.java
 ##########
 @@ -0,0 +1,241 @@
+/*
+ * 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;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.collect.HashMultimap;
+import com.google.common.collect.Multimaps;
+import com.google.common.collect.SetMultimap;
+import com.google.common.util.concurrent.ListenableFuture;
+import io.github.resilience4j.bulkhead.Bulkhead;
+import io.github.resilience4j.bulkhead.BulkheadConfig;
+import io.github.resilience4j.bulkhead.BulkheadRegistry;
+import it.unimi.dsi.fastutil.objects.Object2IntMap;
+import org.apache.druid.client.SegmentServerSelector;
+import org.apache.druid.java.util.common.concurrent.Execs;
+import org.apache.druid.java.util.common.guava.Sequence;
+import org.apache.druid.query.Query;
+import org.apache.druid.query.QueryContexts;
+import org.apache.druid.query.QueryPlus;
+import org.apache.druid.query.QueryWatcher;
+import org.apache.druid.server.initialization.ServerConfig;
+
+import java.time.Duration;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+
+/**
+ * QueryScheduler (potentially) assigns any {@link Query} that is to be executed to a 'query lane' using the
+ * {@link QueryLaningStrategy} that is defined in {@link QuerySchedulerConfig}.
+ *
+ * As a {@link QueryWatcher}, it also provides cancellation facilities.
+ *
+ * This class is shared by all requests on the Jetty HTTP theadpool and must be thread safe.
+ */
+public class QueryScheduler implements QueryWatcher
+{
+  static final String TOTAL = "default";
+  private final int totalCapacity;
+  private final QueryLaningStrategy laningStrategy;
+  private final BulkheadRegistry laneRegistry;
+  private final SetMultimap<String, ListenableFuture<?>> queryFutures;
+  private final SetMultimap<String, String> queryDatasources;
+
+  public QueryScheduler(int totalNumThreads, QueryLaningStrategy laningStrategy, ServerConfig serverConfig)
+  {
+    this.laningStrategy = laningStrategy;
+    this.queryFutures = Multimaps.synchronizedSetMultimap(HashMultimap.create());
+    this.queryDatasources = Multimaps.synchronizedSetMultimap(HashMultimap.create());
+    // if totalNumThreads is above 0 and less than druid.server.http.numThreads, enforce total limit
+    final boolean limitTotal;
+    if (totalNumThreads > 0 && totalNumThreads < serverConfig.getNumThreads()) {
+      limitTotal = true;
+      this.totalCapacity = totalNumThreads;
+    } else {
+      limitTotal = false;
+      this.totalCapacity = serverConfig.getNumThreads();
+    }
+    this.laneRegistry = BulkheadRegistry.of(getLaneConfigs(limitTotal));
+  }
+
+  @Override
+  public void registerQueryFuture(Query<?> query, ListenableFuture<?> future)
+  {
+    final String id = query.getId();
+    final Set<String> datasources = query.getDataSource().getTableNames();
+    queryFutures.put(id, future);
+    queryDatasources.putAll(id, datasources);
+    future.addListener(
+        () -> {
+          queryFutures.remove(id, future);
+          for (String datasource : datasources) {
+            queryDatasources.remove(id, datasource);
+          }
+        },
+        Execs.directExecutor()
+    );
+  }
+
+  /**
+   * Assign a query a lane (if not set)
+   */
+  public <T> Query<T> laneQuery(QueryPlus<T> queryPlus, Set<SegmentServerSelector> segments)
+  {
+    Query<T> query = queryPlus.getQuery();
+    // man wins over machine.. for now.
+    if (QueryContexts.getLane(query) != null) {
+      return query;
+    }
+    Optional<String> lane = laningStrategy.computeLane(queryPlus, segments);
+    return lane.map(query::withLane).orElse(query);
+  }
+
+  /**
+   * Run a query with the scheduler, attempting to acquire a semaphore from the total and lane specific query capacities
+   *
+   * Note that {@link #cancelQuery} should not interrupt the thread that calls run, in all current usages it only
+   * cancels any {@link ListenableFuture} created downstream. If this ever commonly changes, we should add
+   * synchronization between {@link #cancelQuery} and the acquisition of the {@link Bulkhead} to continue to ensure that
+   * anything acquired is also released.
+   *
+   * In the meantime, if a {@link ListenableFuture} is registered for the query that calls this method, it MUST handle
+   * this synchronization itself to ensure that no {@link Bulkhead} is acquired without releasing it.
+   */
+  public <T> Sequence<T> run(Query<?> query, Sequence<T> resultSequence)
+  {
+    List<Bulkhead> bulkheads = acquireLanes(query);
+    return resultSequence.withBaggage(() -> releaseLanes(bulkheads));
+  }
+
+  /**
+   * Forcibly cancel all futures that have been registered to a specific query id
+   */
+  public boolean cancelQuery(String id)
+  {
+    // if you re-use queryId and cancel queries... you are going to have a bad time
 
 Review comment:
   I find this comment confusing. There will be multiple futures of the same ID in historicals most of the time unless the query reads only one segment in a historical. Probably better to comment about what could happen in brokers and historicals.

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] himanshug commented on issue #9407: query laning and load shedding

Posted by GitBox <gi...@apache.org>.
himanshug commented on issue #9407: query laning and load shedding
URL: https://github.com/apache/druid/pull/9407#issuecomment-596859406
 
 
   @clintropolis sorry, I will take a look at this tonight. that said, it  is already approved by ppl so good to merge irrespectively.

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] jihoonson commented on a change in pull request #9407: query laning and load shedding

Posted by GitBox <gi...@apache.org>.
jihoonson commented on a change in pull request #9407: query laning and load shedding
URL: https://github.com/apache/druid/pull/9407#discussion_r389176852
 
 

 ##########
 File path: server/src/main/java/org/apache/druid/server/scheduling/HiLoQueryLaningStrategy.java
 ##########
 @@ -0,0 +1,79 @@
+/*
+ * 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.scheduling;
+
+import com.fasterxml.jackson.annotation.JsonCreator;
+import com.fasterxml.jackson.annotation.JsonProperty;
+import com.google.common.base.Preconditions;
+import com.google.common.primitives.Ints;
+import it.unimi.dsi.fastutil.objects.Object2IntArrayMap;
+import it.unimi.dsi.fastutil.objects.Object2IntMap;
+import org.apache.druid.client.SegmentServerSelector;
+import org.apache.druid.query.Query;
+import org.apache.druid.query.QueryContexts;
+import org.apache.druid.query.QueryPlus;
+import org.apache.druid.server.QueryLaningStrategy;
+
+import java.util.Optional;
+import java.util.Set;
+
+/**
+ * Query laning strategy which associates all {@link Query} with priority lower than 0 into a 'low' lane
+ */
+public class HiLoQueryLaningStrategy implements QueryLaningStrategy
+{
+  public static final String LOW = "low";
+
+  @JsonProperty
+  private final int maxLowPercent;
 
 Review comment:
   We could do that. Or we can allow either `maxLowPercent` or `maxCapacity` to be set.

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] jihoonson commented on a change in pull request #9407: query laning and load shedding

Posted by GitBox <gi...@apache.org>.
jihoonson commented on a change in pull request #9407: query laning and load shedding
URL: https://github.com/apache/druid/pull/9407#discussion_r389066184
 
 

 ##########
 File path: server/src/main/java/org/apache/druid/server/QueryScheduler.java
 ##########
 @@ -0,0 +1,241 @@
+/*
+ * 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;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.collect.HashMultimap;
+import com.google.common.collect.Multimaps;
+import com.google.common.collect.SetMultimap;
+import com.google.common.util.concurrent.ListenableFuture;
+import io.github.resilience4j.bulkhead.Bulkhead;
+import io.github.resilience4j.bulkhead.BulkheadConfig;
+import io.github.resilience4j.bulkhead.BulkheadRegistry;
+import it.unimi.dsi.fastutil.objects.Object2IntMap;
+import org.apache.druid.client.SegmentServerSelector;
+import org.apache.druid.java.util.common.concurrent.Execs;
+import org.apache.druid.java.util.common.guava.Sequence;
+import org.apache.druid.query.Query;
+import org.apache.druid.query.QueryContexts;
+import org.apache.druid.query.QueryPlus;
+import org.apache.druid.query.QueryWatcher;
+import org.apache.druid.server.initialization.ServerConfig;
+
+import java.time.Duration;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+
+/**
+ * QueryScheduler (potentially) assigns any {@link Query} that is to be executed to a 'query lane' using the
+ * {@link QueryLaningStrategy} that is defined in {@link QuerySchedulerConfig}.
+ *
+ * As a {@link QueryWatcher}, it also provides cancellation facilities.
+ *
+ * This class is shared by all requests on the Jetty HTTP theadpool and must be thread safe.
+ */
+public class QueryScheduler implements QueryWatcher
+{
+  static final String TOTAL = "default";
+  private final int totalCapacity;
+  private final QueryLaningStrategy laningStrategy;
+  private final BulkheadRegistry laneRegistry;
+  private final SetMultimap<String, ListenableFuture<?>> queryFutures;
 
 Review comment:
   Please add a comment what the key and the value are.

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] clintropolis commented on a change in pull request #9407: query laning and load shedding

Posted by GitBox <gi...@apache.org>.
clintropolis commented on a change in pull request #9407: query laning and load shedding
URL: https://github.com/apache/druid/pull/9407#discussion_r389162250
 
 

 ##########
 File path: server/src/main/java/org/apache/druid/server/QueryScheduler.java
 ##########
 @@ -0,0 +1,241 @@
+/*
+ * 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;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.collect.HashMultimap;
+import com.google.common.collect.Multimaps;
+import com.google.common.collect.SetMultimap;
+import com.google.common.util.concurrent.ListenableFuture;
+import io.github.resilience4j.bulkhead.Bulkhead;
+import io.github.resilience4j.bulkhead.BulkheadConfig;
+import io.github.resilience4j.bulkhead.BulkheadRegistry;
+import it.unimi.dsi.fastutil.objects.Object2IntMap;
+import org.apache.druid.client.SegmentServerSelector;
+import org.apache.druid.java.util.common.concurrent.Execs;
+import org.apache.druid.java.util.common.guava.Sequence;
+import org.apache.druid.query.Query;
+import org.apache.druid.query.QueryContexts;
+import org.apache.druid.query.QueryPlus;
+import org.apache.druid.query.QueryWatcher;
+import org.apache.druid.server.initialization.ServerConfig;
+
+import java.time.Duration;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+
+/**
+ * QueryScheduler (potentially) assigns any {@link Query} that is to be executed to a 'query lane' using the
+ * {@link QueryLaningStrategy} that is defined in {@link QuerySchedulerConfig}.
+ *
+ * As a {@link QueryWatcher}, it also provides cancellation facilities.
+ *
+ * This class is shared by all requests on the Jetty HTTP theadpool and must be thread safe.
+ */
+public class QueryScheduler implements QueryWatcher
+{
+  static final String TOTAL = "default";
+  private final int totalCapacity;
+  private final QueryLaningStrategy laningStrategy;
+  private final BulkheadRegistry laneRegistry;
+  private final SetMultimap<String, ListenableFuture<?>> queryFutures;
+  private final SetMultimap<String, String> queryDatasources;
+
+  public QueryScheduler(int totalNumThreads, QueryLaningStrategy laningStrategy, ServerConfig serverConfig)
+  {
+    this.laningStrategy = laningStrategy;
+    this.queryFutures = Multimaps.synchronizedSetMultimap(HashMultimap.create());
+    this.queryDatasources = Multimaps.synchronizedSetMultimap(HashMultimap.create());
+    // if totalNumThreads is above 0 and less than druid.server.http.numThreads, enforce total limit
+    final boolean limitTotal;
+    if (totalNumThreads > 0 && totalNumThreads < serverConfig.getNumThreads()) {
+      limitTotal = true;
+      this.totalCapacity = totalNumThreads;
+    } else {
+      limitTotal = false;
+      this.totalCapacity = serverConfig.getNumThreads();
+    }
+    this.laneRegistry = BulkheadRegistry.of(getLaneConfigs(limitTotal));
+  }
+
+  @Override
+  public void registerQueryFuture(Query<?> query, ListenableFuture<?> future)
+  {
+    final String id = query.getId();
+    final Set<String> datasources = query.getDataSource().getTableNames();
+    queryFutures.put(id, future);
+    queryDatasources.putAll(id, datasources);
+    future.addListener(
+        () -> {
+          queryFutures.remove(id, future);
+          for (String datasource : datasources) {
+            queryDatasources.remove(id, datasource);
+          }
+        },
+        Execs.directExecutor()
+    );
+  }
+
+  /**
+   * Assign a query a lane (if not set)
+   */
+  public <T> Query<T> laneQuery(QueryPlus<T> queryPlus, Set<SegmentServerSelector> segments)
+  {
+    Query<T> query = queryPlus.getQuery();
+    // man wins over machine.. for now.
+    if (QueryContexts.getLane(query) != null) {
+      return query;
+    }
+    Optional<String> lane = laningStrategy.computeLane(queryPlus, segments);
+    return lane.map(query::withLane).orElse(query);
+  }
+
+  /**
+   * Run a query with the scheduler, attempting to acquire a semaphore from the total and lane specific query capacities
+   *
+   * Note that {@link #cancelQuery} should not interrupt the thread that calls run, in all current usages it only
+   * cancels any {@link ListenableFuture} created downstream. If this ever commonly changes, we should add
+   * synchronization between {@link #cancelQuery} and the acquisition of the {@link Bulkhead} to continue to ensure that
+   * anything acquired is also released.
+   *
+   * In the meantime, if a {@link ListenableFuture} is registered for the query that calls this method, it MUST handle
+   * this synchronization itself to ensure that no {@link Bulkhead} is acquired without releasing it.
+   */
+  public <T> Sequence<T> run(Query<?> query, Sequence<T> resultSequence)
+  {
+    List<Bulkhead> bulkheads = acquireLanes(query);
+    return resultSequence.withBaggage(() -> releaseLanes(bulkheads));
+  }
+
+  /**
+   * Forcibly cancel all futures that have been registered to a specific query id
+   */
+  public boolean cancelQuery(String id)
+  {
+    // if you re-use queryId and cancel queries... you are going to have a bad time
 
 Review comment:
   Ah, the comment was referring to the fact that query id can be manually set, like if you and I both send queries with the same id, one of us would cancel both I think.

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] ccaominh commented on a change in pull request #9407: query laning and load shedding

Posted by GitBox <gi...@apache.org>.
ccaominh commented on a change in pull request #9407: query laning and load shedding
URL: https://github.com/apache/druid/pull/9407#discussion_r387964505
 
 

 ##########
 File path: docs/configuration/index.md
 ##########
 @@ -1476,9 +1476,31 @@ These Broker configurations can be defined in the `broker/runtime.properties` fi
 |`druid.broker.select.tier`|`highestPriority`, `lowestPriority`, `custom`|If segments are cross-replicated across tiers in a cluster, you can tell the broker to prefer to select segments in a tier with a certain priority.|`highestPriority`|
 |`druid.broker.select.tier.custom.priorities`|`An array of integer priorities.`|Select servers in tiers with a custom priority list.|None|
 
+##### Query laning
+
+The Broker provides facilities to aid in query capacity reservation for heterogeneous query workloads in the form of 'laning' strategies, which provide a variety of mechanisms to examine and classify a query, assigning it to a 'lane'. Lanes are defined with capacity limits which the broker will enforce, causing requests in excess of the capacity to be discarded with an HTTP 429 status code, reserving resources for other lanes or for interactive queries (with no lane).
+
+|Property|Description|Default|
+|--------|-----------|-------|
+|`druid.query.scheduler.numThreads`|Maximum number of HTTP threads to dedicate to query processing. To save HTTP thread capacity, this should be lower than `druid.server.http.numThreads`.|Unbounded|
+|`druid.query.scheduler.laning.type`|Query laning strategy to use to assign queries to a lane in order to control capacities for certain classes of queries.|`none`|
+
+###### No laning strategy
+
+In this mode, queries are never assigned a lane and only limited by `druid.server.http.numThreads` or `druid.query.scheduler.numThreads`, if set. This is the default Druid query scheduler operating mode.
+
 
 Review comment:
   Consider adding:
   
   This strategy can be enabled by setting `druid.query.scheduler.laning.type` to `none`.

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] clintropolis commented on a change in pull request #9407: query laning and load shedding

Posted by GitBox <gi...@apache.org>.
clintropolis commented on a change in pull request #9407: query laning and load shedding
URL: https://github.com/apache/druid/pull/9407#discussion_r389153310
 
 

 ##########
 File path: server/src/main/java/org/apache/druid/server/QueryScheduler.java
 ##########
 @@ -0,0 +1,241 @@
+/*
+ * 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;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.collect.HashMultimap;
+import com.google.common.collect.Multimaps;
+import com.google.common.collect.SetMultimap;
+import com.google.common.util.concurrent.ListenableFuture;
+import io.github.resilience4j.bulkhead.Bulkhead;
+import io.github.resilience4j.bulkhead.BulkheadConfig;
+import io.github.resilience4j.bulkhead.BulkheadRegistry;
+import it.unimi.dsi.fastutil.objects.Object2IntMap;
+import org.apache.druid.client.SegmentServerSelector;
+import org.apache.druid.java.util.common.concurrent.Execs;
+import org.apache.druid.java.util.common.guava.Sequence;
+import org.apache.druid.query.Query;
+import org.apache.druid.query.QueryContexts;
+import org.apache.druid.query.QueryPlus;
+import org.apache.druid.query.QueryWatcher;
+import org.apache.druid.server.initialization.ServerConfig;
+
+import java.time.Duration;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+
+/**
+ * QueryScheduler (potentially) assigns any {@link Query} that is to be executed to a 'query lane' using the
+ * {@link QueryLaningStrategy} that is defined in {@link QuerySchedulerConfig}.
+ *
+ * As a {@link QueryWatcher}, it also provides cancellation facilities.
+ *
+ * This class is shared by all requests on the Jetty HTTP theadpool and must be thread safe.
+ */
+public class QueryScheduler implements QueryWatcher
+{
+  static final String TOTAL = "default";
+  private final int totalCapacity;
+  private final QueryLaningStrategy laningStrategy;
+  private final BulkheadRegistry laneRegistry;
+  private final SetMultimap<String, ListenableFuture<?>> queryFutures;
+  private final SetMultimap<String, String> queryDatasources;
+
+  public QueryScheduler(int totalNumThreads, QueryLaningStrategy laningStrategy, ServerConfig serverConfig)
+  {
+    this.laningStrategy = laningStrategy;
+    this.queryFutures = Multimaps.synchronizedSetMultimap(HashMultimap.create());
+    this.queryDatasources = Multimaps.synchronizedSetMultimap(HashMultimap.create());
+    // if totalNumThreads is above 0 and less than druid.server.http.numThreads, enforce total limit
+    final boolean limitTotal;
+    if (totalNumThreads > 0 && totalNumThreads < serverConfig.getNumThreads()) {
+      limitTotal = true;
+      this.totalCapacity = totalNumThreads;
+    } else {
+      limitTotal = false;
+      this.totalCapacity = serverConfig.getNumThreads();
+    }
+    this.laneRegistry = BulkheadRegistry.of(getLaneConfigs(limitTotal));
+  }
+
+  @Override
+  public void registerQueryFuture(Query<?> query, ListenableFuture<?> future)
+  {
+    final String id = query.getId();
+    final Set<String> datasources = query.getDataSource().getTableNames();
+    queryFutures.put(id, future);
+    queryDatasources.putAll(id, datasources);
+    future.addListener(
+        () -> {
+          queryFutures.remove(id, future);
+          for (String datasource : datasources) {
+            queryDatasources.remove(id, datasource);
+          }
+        },
+        Execs.directExecutor()
+    );
+  }
+
+  /**
+   * Assign a query a lane (if not set)
+   */
+  public <T> Query<T> laneQuery(QueryPlus<T> queryPlus, Set<SegmentServerSelector> segments)
+  {
+    Query<T> query = queryPlus.getQuery();
+    // man wins over machine.. for now.
+    if (QueryContexts.getLane(query) != null) {
+      return query;
+    }
+    Optional<String> lane = laningStrategy.computeLane(queryPlus, segments);
+    return lane.map(query::withLane).orElse(query);
+  }
+
+  /**
+   * Run a query with the scheduler, attempting to acquire a semaphore from the total and lane specific query capacities
+   *
+   * Note that {@link #cancelQuery} should not interrupt the thread that calls run, in all current usages it only
+   * cancels any {@link ListenableFuture} created downstream. If this ever commonly changes, we should add
+   * synchronization between {@link #cancelQuery} and the acquisition of the {@link Bulkhead} to continue to ensure that
+   * anything acquired is also released.
+   *
+   * In the meantime, if a {@link ListenableFuture} is registered for the query that calls this method, it MUST handle
+   * this synchronization itself to ensure that no {@link Bulkhead} is acquired without releasing it.
+   */
+  public <T> Sequence<T> run(Query<?> query, Sequence<T> resultSequence)
+  {
+    List<Bulkhead> bulkheads = acquireLanes(query);
+    return resultSequence.withBaggage(() -> releaseLanes(bulkheads));
+  }
+
+  /**
+   * Forcibly cancel all futures that have been registered to a specific query id
+   */
+  public boolean cancelQuery(String id)
+  {
+    // if you re-use queryId and cancel queries... you are going to have a bad time
+    queryDatasources.removeAll(id);
+    Set<ListenableFuture<?>> futures = queryFutures.removeAll(id);
+    boolean success = true;
+    for (ListenableFuture<?> future : futures) {
+      success = success && future.cancel(true);
+    }
+    return success;
+  }
+
+  /**
+   * Get a {@link Set} of datasource names for a {@link Query} id, used by {@link QueryResource#cancelQuery} to
+   * authorize that a user may call {@link #cancelQuery} for the given id and datasources
+   */
+  public Set<String> getQueryDatasources(final String queryId)
+  {
+    return queryDatasources.get(queryId);
+  }
+
+  /**
+   * Get the maximum number of concurrent queries that {@link #run} can support
+   */
+  @VisibleForTesting
+  int getTotalAvailableCapacity()
+  {
+    return laneRegistry.getConfiguration(TOTAL)
+                       .map(config -> laneRegistry.bulkhead(TOTAL, config).getMetrics().getAvailableConcurrentCalls())
+                       .orElse(-1);
+  }
+
+  /**
+   * Get the maximum number of concurrent queries that {@link #run} can support for a given lane
+   */
+  @VisibleForTesting
+  int getLaneAvailableCapacity(String lane)
+  {
+    return laneRegistry.getConfiguration(lane)
+                       .map(config -> laneRegistry.bulkhead(lane, config).getMetrics().getAvailableConcurrentCalls())
+                       .orElse(-1);
+  }
+
+  /**
+   * Acquire a semaphore for both the 'total' and a lane, if any is associated with a query
+   */
+  @VisibleForTesting
+  List<Bulkhead> acquireLanes(Query<?> query)
+  {
+    final String lane = QueryContexts.getLane(query);
+    final Optional<BulkheadConfig> laneConfig = lane == null ? Optional.empty() : laneRegistry.getConfiguration(lane);
+    List<Bulkhead> hallPasses = new ArrayList<>(2);
+    final Optional<BulkheadConfig> totalConfig = laneRegistry.getConfiguration(TOTAL);
+    // if we have a lane, get it first
+    laneConfig.ifPresent(config -> {
 
 Review comment:
   yeah, it is definitely racy, but didn't seem especially harmful, is it worth the extra lock (not sure what would be most appropriate, striped on lane i guess?) we would have to hold to grab both?

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] clintropolis commented on a change in pull request #9407: query laning and load shedding

Posted by GitBox <gi...@apache.org>.
clintropolis commented on a change in pull request #9407: query laning and load shedding
URL: https://github.com/apache/druid/pull/9407#discussion_r389214201
 
 

 ##########
 File path: server/src/main/java/org/apache/druid/server/QueryScheduler.java
 ##########
 @@ -0,0 +1,241 @@
+/*
+ * 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;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.collect.HashMultimap;
+import com.google.common.collect.Multimaps;
+import com.google.common.collect.SetMultimap;
+import com.google.common.util.concurrent.ListenableFuture;
+import io.github.resilience4j.bulkhead.Bulkhead;
+import io.github.resilience4j.bulkhead.BulkheadConfig;
+import io.github.resilience4j.bulkhead.BulkheadRegistry;
+import it.unimi.dsi.fastutil.objects.Object2IntMap;
+import org.apache.druid.client.SegmentServerSelector;
+import org.apache.druid.java.util.common.concurrent.Execs;
+import org.apache.druid.java.util.common.guava.Sequence;
+import org.apache.druid.query.Query;
+import org.apache.druid.query.QueryContexts;
+import org.apache.druid.query.QueryPlus;
+import org.apache.druid.query.QueryWatcher;
+import org.apache.druid.server.initialization.ServerConfig;
+
+import java.time.Duration;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+
+/**
+ * QueryScheduler (potentially) assigns any {@link Query} that is to be executed to a 'query lane' using the
+ * {@link QueryLaningStrategy} that is defined in {@link QuerySchedulerConfig}.
+ *
+ * As a {@link QueryWatcher}, it also provides cancellation facilities.
+ *
+ * This class is shared by all requests on the Jetty HTTP theadpool and must be thread safe.
+ */
+public class QueryScheduler implements QueryWatcher
+{
+  static final String TOTAL = "default";
+  private final int totalCapacity;
+  private final QueryLaningStrategy laningStrategy;
+  private final BulkheadRegistry laneRegistry;
+  private final SetMultimap<String, ListenableFuture<?>> queryFutures;
+  private final SetMultimap<String, String> queryDatasources;
+
+  public QueryScheduler(int totalNumThreads, QueryLaningStrategy laningStrategy, ServerConfig serverConfig)
+  {
+    this.laningStrategy = laningStrategy;
+    this.queryFutures = Multimaps.synchronizedSetMultimap(HashMultimap.create());
+    this.queryDatasources = Multimaps.synchronizedSetMultimap(HashMultimap.create());
+    // if totalNumThreads is above 0 and less than druid.server.http.numThreads, enforce total limit
+    final boolean limitTotal;
+    if (totalNumThreads > 0 && totalNumThreads < serverConfig.getNumThreads()) {
+      limitTotal = true;
+      this.totalCapacity = totalNumThreads;
+    } else {
+      limitTotal = false;
+      this.totalCapacity = serverConfig.getNumThreads();
+    }
+    this.laneRegistry = BulkheadRegistry.of(getLaneConfigs(limitTotal));
+  }
+
+  @Override
+  public void registerQueryFuture(Query<?> query, ListenableFuture<?> future)
+  {
+    final String id = query.getId();
+    final Set<String> datasources = query.getDataSource().getTableNames();
+    queryFutures.put(id, future);
+    queryDatasources.putAll(id, datasources);
+    future.addListener(
+        () -> {
+          queryFutures.remove(id, future);
+          for (String datasource : datasources) {
+            queryDatasources.remove(id, datasource);
+          }
+        },
+        Execs.directExecutor()
+    );
+  }
+
+  /**
+   * Assign a query a lane (if not set)
+   */
+  public <T> Query<T> laneQuery(QueryPlus<T> queryPlus, Set<SegmentServerSelector> segments)
+  {
+    Query<T> query = queryPlus.getQuery();
+    // man wins over machine.. for now.
+    if (QueryContexts.getLane(query) != null) {
+      return query;
+    }
+    Optional<String> lane = laningStrategy.computeLane(queryPlus, segments);
+    return lane.map(query::withLane).orElse(query);
+  }
+
+  /**
+   * Run a query with the scheduler, attempting to acquire a semaphore from the total and lane specific query capacities
+   *
+   * Note that {@link #cancelQuery} should not interrupt the thread that calls run, in all current usages it only
+   * cancels any {@link ListenableFuture} created downstream. If this ever commonly changes, we should add
+   * synchronization between {@link #cancelQuery} and the acquisition of the {@link Bulkhead} to continue to ensure that
+   * anything acquired is also released.
+   *
+   * In the meantime, if a {@link ListenableFuture} is registered for the query that calls this method, it MUST handle
+   * this synchronization itself to ensure that no {@link Bulkhead} is acquired without releasing it.
+   */
+  public <T> Sequence<T> run(Query<?> query, Sequence<T> resultSequence)
+  {
+    List<Bulkhead> bulkheads = acquireLanes(query);
+    return resultSequence.withBaggage(() -> releaseLanes(bulkheads));
+  }
+
+  /**
+   * Forcibly cancel all futures that have been registered to a specific query id
+   */
+  public boolean cancelQuery(String id)
+  {
+    // if you re-use queryId and cancel queries... you are going to have a bad time
+    queryDatasources.removeAll(id);
+    Set<ListenableFuture<?>> futures = queryFutures.removeAll(id);
+    boolean success = true;
+    for (ListenableFuture<?> future : futures) {
+      success = success && future.cancel(true);
+    }
+    return success;
+  }
+
+  /**
+   * Get a {@link Set} of datasource names for a {@link Query} id, used by {@link QueryResource#cancelQuery} to
+   * authorize that a user may call {@link #cancelQuery} for the given id and datasources
+   */
+  public Set<String> getQueryDatasources(final String queryId)
+  {
+    return queryDatasources.get(queryId);
+  }
+
+  /**
+   * Get the maximum number of concurrent queries that {@link #run} can support
+   */
+  @VisibleForTesting
+  int getTotalAvailableCapacity()
+  {
+    return laneRegistry.getConfiguration(TOTAL)
+                       .map(config -> laneRegistry.bulkhead(TOTAL, config).getMetrics().getAvailableConcurrentCalls())
+                       .orElse(-1);
+  }
+
+  /**
+   * Get the maximum number of concurrent queries that {@link #run} can support for a given lane
+   */
+  @VisibleForTesting
+  int getLaneAvailableCapacity(String lane)
+  {
+    return laneRegistry.getConfiguration(lane)
+                       .map(config -> laneRegistry.bulkhead(lane, config).getMetrics().getAvailableConcurrentCalls())
+                       .orElse(-1);
+  }
+
+  /**
+   * Acquire a semaphore for both the 'total' and a lane, if any is associated with a query
+   */
+  @VisibleForTesting
+  List<Bulkhead> acquireLanes(Query<?> query)
+  {
+    final String lane = QueryContexts.getLane(query);
+    final Optional<BulkheadConfig> laneConfig = lane == null ? Optional.empty() : laneRegistry.getConfiguration(lane);
+    List<Bulkhead> hallPasses = new ArrayList<>(2);
+    final Optional<BulkheadConfig> totalConfig = laneRegistry.getConfiguration(TOTAL);
+    // if we have a lane, get it first
+    laneConfig.ifPresent(config -> {
 
 Review comment:
   The lock overhead doesn't seem like it would be too much, but it also doesn't seem like it would change much, especially when queries are not in the same lane. I added some comments that discuss how the lane acquiring is not fair

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] ccaominh commented on a change in pull request #9407: query laning and load shedding

Posted by GitBox <gi...@apache.org>.
ccaominh commented on a change in pull request #9407: query laning and load shedding
URL: https://github.com/apache/druid/pull/9407#discussion_r388012999
 
 

 ##########
 File path: server/src/test/java/org/apache/druid/server/scheduling/HiLoQueryLaningStrategyTest.java
 ##########
 @@ -0,0 +1,108 @@
+/*
+ * 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.scheduling;
+
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.ImmutableMap;
+import com.google.common.collect.ImmutableSet;
+import it.unimi.dsi.fastutil.objects.Object2IntMap;
+import org.apache.druid.java.util.common.Intervals;
+import org.apache.druid.java.util.common.granularity.Granularities;
+import org.apache.druid.query.Druids;
+import org.apache.druid.query.QueryContexts;
+import org.apache.druid.query.QueryPlus;
+import org.apache.druid.query.aggregation.CountAggregatorFactory;
+import org.apache.druid.query.timeseries.TimeseriesQuery;
+import org.apache.druid.server.QueryLaningStrategy;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.ExpectedException;
+
+public class HiLoQueryLaningStrategyTest
+{
+  @Rule
+  public ExpectedException expectedException = ExpectedException.none();
+
+  private Druids.TimeseriesQueryBuilder queryBuilder;
+  private HiLoQueryLaningStrategy strategy;
+
+  @Before
+  public void setup()
+  {
+    this.queryBuilder = Druids.newTimeseriesQueryBuilder()
+                              .dataSource("test")
+                              .intervals(ImmutableList.of(Intervals.ETERNITY))
+                              .granularity(Granularities.DAY)
+                              .aggregators(new CountAggregatorFactory("count"));
+
+    this.strategy = new HiLoQueryLaningStrategy(10);
+  }
+
+  @Test
+  public void testMaxLowThreadsRequired()
+  {
+    expectedException.expect(NullPointerException.class);
+    expectedException.expectMessage("maxLowThreads must be set");
+    QueryLaningStrategy strategy = new HiLoQueryLaningStrategy(null);
+  }
+
+  @Test
+  public void testLaneLimits()
+  {
+    Object2IntMap<String> laneConfig = strategy.getLaneLimits();
+    Assert.assertEquals(1, laneConfig.size());
+    Assert.assertTrue(laneConfig.containsKey(HiLoQueryLaningStrategy.LOW));
+    Assert.assertEquals(10, laneConfig.getInt(HiLoQueryLaningStrategy.LOW));
+  }
+
+  @Test
+  public void testLaningNoPriority()
+  {
+    TimeseriesQuery query = queryBuilder.build();
+    Assert.assertFalse(strategy.computeLane(QueryPlus.wrap(query), ImmutableSet.of()).isPresent());
+  }
+
+  @Test
+  public void testLaningZeroPriority()
+  {
+    TimeseriesQuery query = queryBuilder.context(ImmutableMap.of(QueryContexts.PRIORITY_KEY, 0)).build();
+    Assert.assertFalse(strategy.computeLane(QueryPlus.wrap(query), ImmutableSet.of()).isPresent());
+  }
+
+  @Test
+  public void testLaningInteractivePriority()
+  {
+    TimeseriesQuery query = queryBuilder.context(ImmutableMap.of(QueryContexts.PRIORITY_KEY, 100)).build();
+    Assert.assertFalse(strategy.computeLane(QueryPlus.wrap(query), ImmutableSet.of()).isPresent());
+  }
+
+  @Test
+  public void testLaningLowPriority()
+  {
+    TimeseriesQuery query = queryBuilder.context(ImmutableMap.of(QueryContexts.PRIORITY_KEY, -1)).build();
+    Assert.assertTrue(strategy.computeLane(QueryPlus.wrap(query), ImmutableSet.of()).isPresent());
+    Assert.assertEquals(
+        HiLoQueryLaningStrategy.LOW,
+        strategy.computeLane(QueryPlus.wrap(query), ImmutableSet.of()).get()
+    );
+  }
+}
 
 Review comment:
   Is it useful to test when `QueryContexts.LANE_KEY` is set?

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] ccaominh commented on a change in pull request #9407: query laning and load shedding

Posted by GitBox <gi...@apache.org>.
ccaominh commented on a change in pull request #9407: query laning and load shedding
URL: https://github.com/apache/druid/pull/9407#discussion_r388667087
 
 

 ##########
 File path: sql/src/test/java/org/apache/druid/sql/calcite/util/SpecificSegmentsQuerySegmentWalker.java
 ##########
 @@ -388,13 +410,33 @@ public SegmentDescriptor getDescriptor()
           toolChest
       );
 
+
       // Wrap baseRunner in a runner that rewrites the QuerySegmentSpec to mention the specific segments.
       // This mimics what CachingClusteredClient on the Broker does, and is required for certain queries (like Scan)
       // to function properly.
-      return (theQuery, responseContext) -> baseRunner.run(
-          theQuery.withQuery(Queries.withSpecificSegments(theQuery.getQuery(), ImmutableList.copyOf(specs))),
-          responseContext
-      );
+      return (theQuery, responseContext) -> {
+        if (scheduler != null) {
+          Set<SegmentServerSelector> segments = new HashSet<>();
+          specs.forEach(spec -> segments.add(new SegmentServerSelector(null, spec)));
+          return scheduler.run(
+              scheduler.laneQuery(theQuery, segments),
+              new LazySequence<>(
+                  () -> baseRunner.run(
+                      theQuery.withQuery(Queries.withSpecificSegments(
+                          theQuery.getQuery(),
+                          ImmutableList.copyOf(specs)
+                      )),
+                      responseContext
+                  )
+              )
+          );
 
 Review comment:
   Which tests cover this block?

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] jihoonson commented on a change in pull request #9407: query laning and load shedding

Posted by GitBox <gi...@apache.org>.
jihoonson commented on a change in pull request #9407: query laning and load shedding
URL: https://github.com/apache/druid/pull/9407#discussion_r389219765
 
 

 ##########
 File path: server/src/main/java/org/apache/druid/server/scheduling/HiLoQueryLaningStrategy.java
 ##########
 @@ -0,0 +1,79 @@
+/*
+ * 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.scheduling;
+
+import com.fasterxml.jackson.annotation.JsonCreator;
+import com.fasterxml.jackson.annotation.JsonProperty;
+import com.google.common.base.Preconditions;
+import com.google.common.primitives.Ints;
+import it.unimi.dsi.fastutil.objects.Object2IntArrayMap;
+import it.unimi.dsi.fastutil.objects.Object2IntMap;
+import org.apache.druid.client.SegmentServerSelector;
+import org.apache.druid.query.Query;
+import org.apache.druid.query.QueryContexts;
+import org.apache.druid.query.QueryPlus;
+import org.apache.druid.server.QueryLaningStrategy;
+
+import java.util.Optional;
+import java.util.Set;
+
+/**
+ * Query laning strategy which associates all {@link Query} with priority lower than 0 into a 'low' lane
+ */
+public class HiLoQueryLaningStrategy implements QueryLaningStrategy
+{
+  public static final String LOW = "low";
+
+  @JsonProperty
+  private final int maxLowPercent;
 
 Review comment:
   Sounds good to me.

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] clintropolis commented on a change in pull request #9407: query laning and load shedding

Posted by GitBox <gi...@apache.org>.
clintropolis commented on a change in pull request #9407: query laning and load shedding
URL: https://github.com/apache/druid/pull/9407#discussion_r390194216
 
 

 ##########
 File path: server/src/main/java/org/apache/druid/server/QuerySchedulerConfig.java
 ##########
 @@ -0,0 +1,42 @@
+/*
+ * 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;
+
+import com.fasterxml.jackson.annotation.JsonProperty;
+import org.apache.druid.server.scheduling.NoQueryLaningStrategy;
+
+public class QuerySchedulerConfig
+{
+  @JsonProperty
+  private Integer numThreads = 0;
 
 Review comment:
   Hmm, that is a good point, will change in one of the follow-up PRs.

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] himanshug commented on a change in pull request #9407: query laning and load shedding

Posted by GitBox <gi...@apache.org>.
himanshug commented on a change in pull request #9407: query laning and load shedding
URL: https://github.com/apache/druid/pull/9407#discussion_r390132201
 
 

 ##########
 File path: server/src/main/java/org/apache/druid/server/QueryLaningStrategy.java
 ##########
 @@ -0,0 +1,53 @@
+/*
+ * 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;
+
+import com.fasterxml.jackson.annotation.JsonSubTypes;
+import com.fasterxml.jackson.annotation.JsonTypeInfo;
+import it.unimi.dsi.fastutil.objects.Object2IntMap;
+import org.apache.druid.client.SegmentServerSelector;
+import org.apache.druid.query.QueryPlus;
+import org.apache.druid.server.scheduling.HiLoQueryLaningStrategy;
+import org.apache.druid.server.scheduling.NoQueryLaningStrategy;
+
+import java.util.Optional;
+import java.util.Set;
+
+
+@JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "strategy", defaultImpl = NoQueryLaningStrategy.class)
+@JsonSubTypes(value = {
+    @JsonSubTypes.Type(name = "none", value = NoQueryLaningStrategy.class),
+    @JsonSubTypes.Type(name = "hilo", value = HiLoQueryLaningStrategy.class)
+})
+public interface QueryLaningStrategy
+{
+  /**
+   * Provide a map of lane names to the limit on the number of concurrent queries for that lane
+   * @param totalLimit
+   */
+  Object2IntMap<String> getLaneLimits(int totalLimit);
 
 Review comment:
   is it expected that `sum(returned_map.values) < totalLimit` ? 
   nit:  Also wonder why limits returned here  were not percentages but absolute counts given that each implementation would probably end up doing that calculation.

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] sthetland commented on a change in pull request #9407: query laning and load shedding

Posted by GitBox <gi...@apache.org>.
sthetland commented on a change in pull request #9407: query laning and load shedding
URL: https://github.com/apache/druid/pull/9407#discussion_r389988194
 
 

 ##########
 File path: docs/configuration/index.md
 ##########
 @@ -1476,9 +1476,35 @@ These Broker configurations can be defined in the `broker/runtime.properties` fi
 |`druid.broker.select.tier`|`highestPriority`, `lowestPriority`, `custom`|If segments are cross-replicated across tiers in a cluster, you can tell the broker to prefer to select segments in a tier with a certain priority.|`highestPriority`|
 |`druid.broker.select.tier.custom.priorities`|`An array of integer priorities.`|Select servers in tiers with a custom priority list.|None|
 
+##### Query laning
+
+The Broker provides facilities to aid in query capacity reservation for heterogeneous query workloads in the form of 'laning' strategies, which provide a variety of mechanisms to examine and classify a query, assigning it to a 'lane'. Lanes are defined with capacity limits which the broker will enforce, causing requests in excess of the capacity to be discarded with an HTTP 429 status code, reserving resources for other lanes or for interactive queries (with no lane).
 
 Review comment:
   Not bad as is, but if i were to try to boil it down just a little, it might be something like: 
   
   ```suggestion
   *Laning strategies* allow you to control capacity utilization for heterogeneous query workloads. With laning, the broker examines and classifies a query for the purpose of assigning it to a 'lane'. Lanes have capacity limits, enforced by the broker, that ensure sufficient resources are available for other lanes or for interactive queries (with no lane). Requests in excess of the capacity are discarded with an HTTP 429 status code.
   ```

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] jihoonson commented on a change in pull request #9407: query laning and load shedding

Posted by GitBox <gi...@apache.org>.
jihoonson commented on a change in pull request #9407: query laning and load shedding
URL: https://github.com/apache/druid/pull/9407#discussion_r389177974
 
 

 ##########
 File path: server/src/main/java/org/apache/druid/server/QueryScheduler.java
 ##########
 @@ -0,0 +1,241 @@
+/*
+ * 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;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.collect.HashMultimap;
+import com.google.common.collect.Multimaps;
+import com.google.common.collect.SetMultimap;
+import com.google.common.util.concurrent.ListenableFuture;
+import io.github.resilience4j.bulkhead.Bulkhead;
+import io.github.resilience4j.bulkhead.BulkheadConfig;
+import io.github.resilience4j.bulkhead.BulkheadRegistry;
+import it.unimi.dsi.fastutil.objects.Object2IntMap;
+import org.apache.druid.client.SegmentServerSelector;
+import org.apache.druid.java.util.common.concurrent.Execs;
+import org.apache.druid.java.util.common.guava.Sequence;
+import org.apache.druid.query.Query;
+import org.apache.druid.query.QueryContexts;
+import org.apache.druid.query.QueryPlus;
+import org.apache.druid.query.QueryWatcher;
+import org.apache.druid.server.initialization.ServerConfig;
+
+import java.time.Duration;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+
+/**
+ * QueryScheduler (potentially) assigns any {@link Query} that is to be executed to a 'query lane' using the
+ * {@link QueryLaningStrategy} that is defined in {@link QuerySchedulerConfig}.
+ *
+ * As a {@link QueryWatcher}, it also provides cancellation facilities.
+ *
+ * This class is shared by all requests on the Jetty HTTP theadpool and must be thread safe.
+ */
+public class QueryScheduler implements QueryWatcher
+{
+  static final String TOTAL = "default";
+  private final int totalCapacity;
+  private final QueryLaningStrategy laningStrategy;
+  private final BulkheadRegistry laneRegistry;
+  private final SetMultimap<String, ListenableFuture<?>> queryFutures;
+  private final SetMultimap<String, String> queryDatasources;
+
+  public QueryScheduler(int totalNumThreads, QueryLaningStrategy laningStrategy, ServerConfig serverConfig)
+  {
+    this.laningStrategy = laningStrategy;
+    this.queryFutures = Multimaps.synchronizedSetMultimap(HashMultimap.create());
+    this.queryDatasources = Multimaps.synchronizedSetMultimap(HashMultimap.create());
+    // if totalNumThreads is above 0 and less than druid.server.http.numThreads, enforce total limit
+    final boolean limitTotal;
+    if (totalNumThreads > 0 && totalNumThreads < serverConfig.getNumThreads()) {
+      limitTotal = true;
+      this.totalCapacity = totalNumThreads;
+    } else {
+      limitTotal = false;
+      this.totalCapacity = serverConfig.getNumThreads();
+    }
+    this.laneRegistry = BulkheadRegistry.of(getLaneConfigs(limitTotal));
+  }
+
+  @Override
+  public void registerQueryFuture(Query<?> query, ListenableFuture<?> future)
+  {
+    final String id = query.getId();
+    final Set<String> datasources = query.getDataSource().getTableNames();
+    queryFutures.put(id, future);
+    queryDatasources.putAll(id, datasources);
+    future.addListener(
+        () -> {
+          queryFutures.remove(id, future);
+          for (String datasource : datasources) {
+            queryDatasources.remove(id, datasource);
+          }
+        },
+        Execs.directExecutor()
+    );
+  }
+
+  /**
+   * Assign a query a lane (if not set)
+   */
+  public <T> Query<T> laneQuery(QueryPlus<T> queryPlus, Set<SegmentServerSelector> segments)
+  {
+    Query<T> query = queryPlus.getQuery();
+    // man wins over machine.. for now.
+    if (QueryContexts.getLane(query) != null) {
+      return query;
+    }
+    Optional<String> lane = laningStrategy.computeLane(queryPlus, segments);
+    return lane.map(query::withLane).orElse(query);
+  }
+
+  /**
+   * Run a query with the scheduler, attempting to acquire a semaphore from the total and lane specific query capacities
+   *
+   * Note that {@link #cancelQuery} should not interrupt the thread that calls run, in all current usages it only
+   * cancels any {@link ListenableFuture} created downstream. If this ever commonly changes, we should add
+   * synchronization between {@link #cancelQuery} and the acquisition of the {@link Bulkhead} to continue to ensure that
+   * anything acquired is also released.
+   *
+   * In the meantime, if a {@link ListenableFuture} is registered for the query that calls this method, it MUST handle
+   * this synchronization itself to ensure that no {@link Bulkhead} is acquired without releasing it.
+   */
+  public <T> Sequence<T> run(Query<?> query, Sequence<T> resultSequence)
+  {
+    List<Bulkhead> bulkheads = acquireLanes(query);
+    return resultSequence.withBaggage(() -> releaseLanes(bulkheads));
+  }
+
+  /**
+   * Forcibly cancel all futures that have been registered to a specific query id
+   */
+  public boolean cancelQuery(String id)
+  {
+    // if you re-use queryId and cancel queries... you are going to have a bad time
+    queryDatasources.removeAll(id);
+    Set<ListenableFuture<?>> futures = queryFutures.removeAll(id);
+    boolean success = true;
+    for (ListenableFuture<?> future : futures) {
+      success = success && future.cancel(true);
+    }
+    return success;
+  }
+
+  /**
+   * Get a {@link Set} of datasource names for a {@link Query} id, used by {@link QueryResource#cancelQuery} to
+   * authorize that a user may call {@link #cancelQuery} for the given id and datasources
+   */
+  public Set<String> getQueryDatasources(final String queryId)
+  {
+    return queryDatasources.get(queryId);
+  }
+
+  /**
+   * Get the maximum number of concurrent queries that {@link #run} can support
+   */
+  @VisibleForTesting
+  int getTotalAvailableCapacity()
+  {
+    return laneRegistry.getConfiguration(TOTAL)
+                       .map(config -> laneRegistry.bulkhead(TOTAL, config).getMetrics().getAvailableConcurrentCalls())
+                       .orElse(-1);
+  }
+
+  /**
+   * Get the maximum number of concurrent queries that {@link #run} can support for a given lane
+   */
+  @VisibleForTesting
+  int getLaneAvailableCapacity(String lane)
+  {
+    return laneRegistry.getConfiguration(lane)
+                       .map(config -> laneRegistry.bulkhead(lane, config).getMetrics().getAvailableConcurrentCalls())
+                       .orElse(-1);
+  }
+
+  /**
+   * Acquire a semaphore for both the 'total' and a lane, if any is associated with a query
+   */
+  @VisibleForTesting
+  List<Bulkhead> acquireLanes(Query<?> query)
+  {
+    final String lane = QueryContexts.getLane(query);
+    final Optional<BulkheadConfig> laneConfig = lane == null ? Optional.empty() : laneRegistry.getConfiguration(lane);
+    List<Bulkhead> hallPasses = new ArrayList<>(2);
+    final Optional<BulkheadConfig> totalConfig = laneRegistry.getConfiguration(TOTAL);
+    // if we have a lane, get it first
+    laneConfig.ifPresent(config -> {
 
 Review comment:
   Hmm I can imagine that a user issues a high-priority query immediately followed by a low-priority query. Due to the nature of concurrent execution, even if the high-priority query reached to here first, the low-priority query could execute the code before the high-priority one does. Then the high-priority query could be rejected if the low-priority query took the last available slot in the total lane.
   
   I guess the upside is the more deterministic and thus predictable behavior while the downside is the overhead of the extra lock. How bad is the extra lock? It seems ok?

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] ccaominh commented on a change in pull request #9407: query laning and load shedding

Posted by GitBox <gi...@apache.org>.
ccaominh commented on a change in pull request #9407: query laning and load shedding
URL: https://github.com/apache/druid/pull/9407#discussion_r388631851
 
 

 ##########
 File path: server/src/main/java/org/apache/druid/server/QueryScheduler.java
 ##########
 @@ -0,0 +1,226 @@
+/*
+ * 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;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.collect.HashMultimap;
+import com.google.common.collect.Multimaps;
+import com.google.common.collect.SetMultimap;
+import com.google.common.util.concurrent.ListenableFuture;
+import io.github.resilience4j.bulkhead.Bulkhead;
+import io.github.resilience4j.bulkhead.BulkheadConfig;
+import io.github.resilience4j.bulkhead.BulkheadRegistry;
+import it.unimi.dsi.fastutil.objects.Object2IntMap;
+import org.apache.druid.client.SegmentServerSelector;
+import org.apache.druid.java.util.common.concurrent.Execs;
+import org.apache.druid.java.util.common.guava.Sequence;
+import org.apache.druid.query.Query;
+import org.apache.druid.query.QueryContexts;
+import org.apache.druid.query.QueryPlus;
+import org.apache.druid.query.QueryWatcher;
+
+import java.time.Duration;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+
+/**
+ * QueryScheduler (potentially) assigns any {@link Query} that is to be executed to a 'query lane' using the
+ * {@link QueryLaningStrategy} that is defined in {@link QuerySchedulerConfig}.
+ *
+ * As a {@link QueryWatcher}, it also provides cancellation facilities.
+ *
+ * This class is shared by all requests on the Jetty HTTP theadpool and must be thread safe.
+ */
+public class QueryScheduler implements QueryWatcher
+{
+  static final String TOTAL = "default";
+  private final QueryLaningStrategy laningStrategy;
+  private final BulkheadRegistry laneRegistry;
+  private final SetMultimap<String, ListenableFuture<?>> queryFutures;
+  private final SetMultimap<String, String> queryDatasources;
+
+  public QueryScheduler(int totalNumThreads, QueryLaningStrategy laningStrategy)
+  {
+    this.laningStrategy = laningStrategy;
+    this.laneRegistry = BulkheadRegistry.of(getLaneConfigs(totalNumThreads));
+    this.queryFutures = Multimaps.synchronizedSetMultimap(HashMultimap.create());
+    this.queryDatasources = Multimaps.synchronizedSetMultimap(HashMultimap.create());
+  }
+
+  @Override
+  public void registerQueryFuture(Query<?> query, ListenableFuture<?> future)
+  {
+    final String id = query.getId();
+    final Set<String> datasources = query.getDataSource().getTableNames();
+    queryFutures.put(id, future);
+    queryDatasources.putAll(id, datasources);
+    future.addListener(
+        () -> {
+          queryFutures.remove(id, future);
+          for (String datasource : datasources) {
+            queryDatasources.remove(id, datasource);
+          }
+        },
+        Execs.directExecutor()
+    );
+  }
+
+  /**
+   * Assign a query a lane (if not set)
+   */
+  public <T> Query<T> laneQuery(QueryPlus<T> queryPlus, Set<SegmentServerSelector> segments)
+  {
+    Query<T> query = queryPlus.getQuery();
+    // man wins over machine.. for now.
+    if (QueryContexts.getLane(query) != null) {
+      return query;
+    }
+    Optional<String> lane = laningStrategy.computeLane(queryPlus, segments);
+    return lane.map(query::withLane).orElse(query);
+  }
+
+  /**
+   * Run a query with the scheduler, attempting to acquire a semaphore from the total and lane specific query capacities
+   *
+   * Note that {@link #cancelQuery} should not interrupt the thread that calls run, in all current usages it only
+   * cancels any {@link ListenableFuture} created downstream. If this ever commonly changes, we should add
+   * synchronization between {@link #cancelQuery} and the acquisition of the {@link Bulkhead} to continue to ensure that
+   * anything acquired is also released.
+   *
+   * In the meantime, if a {@link ListenableFuture} is registered for the query that calls this method, it MUST handle
+   * this synchronization itself to ensure that no {@link Bulkhead} is acquired without releasing it.
+   */
+  public <T> Sequence<T> run(Query<?> query, Sequence<T> resultSequence)
+  {
+    List<Bulkhead> bulkheads = acquireLanes(query);
+    return resultSequence.withBaggage(() -> releaseLanes(bulkheads));
+  }
+
+  /**
+   * Forcibly cancel all futures that have been registered to a specific query id
+   */
 
 Review comment:
   Thanks for adding all the javadocs that were missing before!

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] clintropolis commented on a change in pull request #9407: query laning and load shedding

Posted by GitBox <gi...@apache.org>.
clintropolis commented on a change in pull request #9407: query laning and load shedding
URL: https://github.com/apache/druid/pull/9407#discussion_r389219157
 
 

 ##########
 File path: server/src/main/java/org/apache/druid/server/scheduling/HiLoQueryLaningStrategy.java
 ##########
 @@ -0,0 +1,79 @@
+/*
+ * 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.scheduling;
+
+import com.fasterxml.jackson.annotation.JsonCreator;
+import com.fasterxml.jackson.annotation.JsonProperty;
+import com.google.common.base.Preconditions;
+import com.google.common.primitives.Ints;
+import it.unimi.dsi.fastutil.objects.Object2IntArrayMap;
+import it.unimi.dsi.fastutil.objects.Object2IntMap;
+import org.apache.druid.client.SegmentServerSelector;
+import org.apache.druid.query.Query;
+import org.apache.druid.query.QueryContexts;
+import org.apache.druid.query.QueryPlus;
+import org.apache.druid.server.QueryLaningStrategy;
+
+import java.util.Optional;
+import java.util.Set;
+
+/**
+ * Query laning strategy which associates all {@link Query} with priority lower than 0 into a 'low' lane
+ */
+public class HiLoQueryLaningStrategy implements QueryLaningStrategy
+{
+  public static final String LOW = "low";
+
+  @JsonProperty
+  private final int maxLowPercent;
 
 Review comment:
   Would it be ok to do this addition in a follow-up PR? Originally this setting was `maxLowThreads`, but i switched to being percent based to be more versatile. However, it does seem maybe useful to allow one or the other to be set.

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] ccaominh commented on a change in pull request #9407: query laning and load shedding

Posted by GitBox <gi...@apache.org>.
ccaominh commented on a change in pull request #9407: query laning and load shedding
URL: https://github.com/apache/druid/pull/9407#discussion_r388016546
 
 

 ##########
 File path: server/src/main/java/org/apache/druid/server/scheduling/HiLoQueryLaningStrategy.java
 ##########
 @@ -0,0 +1,74 @@
+/*
+ * 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.scheduling;
+
+import com.fasterxml.jackson.annotation.JsonCreator;
+import com.fasterxml.jackson.annotation.JsonProperty;
+import com.google.common.base.Preconditions;
+import it.unimi.dsi.fastutil.objects.Object2IntArrayMap;
+import it.unimi.dsi.fastutil.objects.Object2IntMap;
+import org.apache.druid.client.SegmentServerSelector;
+import org.apache.druid.query.Query;
+import org.apache.druid.query.QueryContexts;
+import org.apache.druid.query.QueryPlus;
+import org.apache.druid.server.QueryLaningStrategy;
+
+import java.util.Optional;
+import java.util.Set;
+
+/**
+ * Query laning strategy which associates all {@link Query} with priority lower than 0 into a 'low' lane
+ */
+public class HiLoQueryLaningStrategy implements QueryLaningStrategy
+{
+  public static final String LOW = "low";
+
+  @JsonProperty
+  private final int maxLowThreads;
+
+  @JsonCreator
+  public HiLoQueryLaningStrategy(
+      @JsonProperty("maxLowThreads") Integer maxLowThreads
 
 Review comment:
   Left a comment in the docs about the property name

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] clintropolis commented on a change in pull request #9407: query laning and load shedding

Posted by GitBox <gi...@apache.org>.
clintropolis commented on a change in pull request #9407: query laning and load shedding
URL: https://github.com/apache/druid/pull/9407#discussion_r391338145
 
 

 ##########
 File path: docs/configuration/index.md
 ##########
 @@ -1481,9 +1481,35 @@ These Broker configurations can be defined in the `broker/runtime.properties` fi
 |`druid.broker.select.tier`|`highestPriority`, `lowestPriority`, `custom`|If segments are cross-replicated across tiers in a cluster, you can tell the broker to prefer to select segments in a tier with a certain priority.|`highestPriority`|
 |`druid.broker.select.tier.custom.priorities`|`An array of integer priorities.`|Select servers in tiers with a custom priority list.|None|
 
+##### Query laning
+
+*Laning strategies* allow you to control capacity utilization for heterogeneous query workloads. With laning, the broker examines and classifies a query for the purpose of assigning it to a 'lane'. Lanes have capacity limits, enforced by the broker, that can be used to ensure sufficient resources are available for other lanes or for interactive queries (with no lane), or to limit overall throughput for queries within the lane. Requests in excess of the capacity are discarded with an HTTP 429 status code.
+
+|Property|Description|Default|
+|--------|-----------|-------|
+|`druid.query.scheduler.numThreads`|Maximum number of HTTP threads to dedicate to query processing. To save HTTP thread capacity, this should be lower than `druid.server.http.numThreads`.|Unbounded|
 
 Review comment:
   I wonder if it would make sense to instead move towards automatically computing `druid.server.http.numThreads`, since maybe it is easier for operators to only have to think about the number of concurrent queries to serve and just set `druid.query.scheduler.numThreads`?
   
   >Major behavior change with lane usage is really losing the queuing of requests to handle spikes and instead sending 429s immediately. In future, we could introduce mechanism to maintain statically/dynamically sized [per lane] waiting queue ourselves as well along with concurrency limits in lane strategy.
   
   Yeah the current behavior is definitely a hard stop if you are over the line. I agree it would make sense to allow some sort of timed out queuing behavior, which is what jetty QoS filter can sort of provide, which is a large part of [why I am still wondering](https://github.com/apache/druid/pull/9407#discussion_r389159233) if `druid.query.scheduler.numThreads` should be a QoS filter instead of enforced as an actual lane like it is currently.

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] clintropolis commented on a change in pull request #9407: query laning and load shedding

Posted by GitBox <gi...@apache.org>.
clintropolis commented on a change in pull request #9407: query laning and load shedding
URL: https://github.com/apache/druid/pull/9407#discussion_r389175395
 
 

 ##########
 File path: server/src/main/java/org/apache/druid/server/scheduling/HiLoQueryLaningStrategy.java
 ##########
 @@ -0,0 +1,79 @@
+/*
+ * 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.scheduling;
+
+import com.fasterxml.jackson.annotation.JsonCreator;
+import com.fasterxml.jackson.annotation.JsonProperty;
+import com.google.common.base.Preconditions;
+import com.google.common.primitives.Ints;
+import it.unimi.dsi.fastutil.objects.Object2IntArrayMap;
+import it.unimi.dsi.fastutil.objects.Object2IntMap;
+import org.apache.druid.client.SegmentServerSelector;
+import org.apache.druid.query.Query;
+import org.apache.druid.query.QueryContexts;
+import org.apache.druid.query.QueryPlus;
+import org.apache.druid.server.QueryLaningStrategy;
+
+import java.util.Optional;
+import java.util.Set;
+
+/**
+ * Query laning strategy which associates all {@link Query} with priority lower than 0 into a 'low' lane
+ */
+public class HiLoQueryLaningStrategy implements QueryLaningStrategy
+{
+  public static final String LOW = "low";
+
+  @JsonProperty
+  private final int maxLowPercent;
 
 Review comment:
   Ah, so do you mean the lane limit should be the min or (or max?) of `maxCapacity` and `maxLowPercent` of `totalLimit`?

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] ccaominh commented on a change in pull request #9407: query laning and load shedding

Posted by GitBox <gi...@apache.org>.
ccaominh commented on a change in pull request #9407: query laning and load shedding
URL: https://github.com/apache/druid/pull/9407#discussion_r388665655
 
 

 ##########
 File path: server/src/main/java/org/apache/druid/server/QueryScheduler.java
 ##########
 @@ -0,0 +1,226 @@
+/*
+ * 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;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.collect.HashMultimap;
+import com.google.common.collect.Multimaps;
+import com.google.common.collect.SetMultimap;
+import com.google.common.util.concurrent.ListenableFuture;
+import io.github.resilience4j.bulkhead.Bulkhead;
+import io.github.resilience4j.bulkhead.BulkheadConfig;
+import io.github.resilience4j.bulkhead.BulkheadRegistry;
+import it.unimi.dsi.fastutil.objects.Object2IntMap;
+import org.apache.druid.client.SegmentServerSelector;
+import org.apache.druid.java.util.common.concurrent.Execs;
+import org.apache.druid.java.util.common.guava.Sequence;
+import org.apache.druid.query.Query;
+import org.apache.druid.query.QueryContexts;
+import org.apache.druid.query.QueryPlus;
+import org.apache.druid.query.QueryWatcher;
+
+import java.time.Duration;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+
+/**
+ * QueryScheduler (potentially) assigns any {@link Query} that is to be executed to a 'query lane' using the
+ * {@link QueryLaningStrategy} that is defined in {@link QuerySchedulerConfig}.
+ *
+ * As a {@link QueryWatcher}, it also provides cancellation facilities.
+ *
+ * This class is shared by all requests on the Jetty HTTP theadpool and must be thread safe.
+ */
+public class QueryScheduler implements QueryWatcher
+{
+  static final String TOTAL = "default";
+  private final QueryLaningStrategy laningStrategy;
+  private final BulkheadRegistry laneRegistry;
+  private final SetMultimap<String, ListenableFuture<?>> queryFutures;
+  private final SetMultimap<String, String> queryDatasources;
+
+  public QueryScheduler(int totalNumThreads, QueryLaningStrategy laningStrategy)
+  {
+    this.laningStrategy = laningStrategy;
+    this.laneRegistry = BulkheadRegistry.of(getLaneConfigs(totalNumThreads));
+    this.queryFutures = Multimaps.synchronizedSetMultimap(HashMultimap.create());
+    this.queryDatasources = Multimaps.synchronizedSetMultimap(HashMultimap.create());
+  }
+
+  @Override
+  public void registerQueryFuture(Query<?> query, ListenableFuture<?> future)
+  {
+    final String id = query.getId();
+    final Set<String> datasources = query.getDataSource().getTableNames();
+    queryFutures.put(id, future);
+    queryDatasources.putAll(id, datasources);
+    future.addListener(
+        () -> {
+          queryFutures.remove(id, future);
+          for (String datasource : datasources) {
+            queryDatasources.remove(id, datasource);
+          }
+        },
+        Execs.directExecutor()
+    );
+  }
+
+  /**
+   * Assign a query a lane (if not set)
+   */
+  public <T> Query<T> laneQuery(QueryPlus<T> queryPlus, Set<SegmentServerSelector> segments)
+  {
+    Query<T> query = queryPlus.getQuery();
+    // man wins over machine.. for now.
+    if (QueryContexts.getLane(query) != null) {
+      return query;
+    }
+    Optional<String> lane = laningStrategy.computeLane(queryPlus, segments);
+    return lane.map(query::withLane).orElse(query);
+  }
+
+  /**
+   * Run a query with the scheduler, attempting to acquire a semaphore from the total and lane specific query capacities
+   *
+   * Note that {@link #cancelQuery} should not interrupt the thread that calls run, in all current usages it only
+   * cancels any {@link ListenableFuture} created downstream. If this ever commonly changes, we should add
+   * synchronization between {@link #cancelQuery} and the acquisition of the {@link Bulkhead} to continue to ensure that
+   * anything acquired is also released.
+   *
+   * In the meantime, if a {@link ListenableFuture} is registered for the query that calls this method, it MUST handle
+   * this synchronization itself to ensure that no {@link Bulkhead} is acquired without releasing it.
+   */
+  public <T> Sequence<T> run(Query<?> query, Sequence<T> resultSequence)
+  {
+    List<Bulkhead> bulkheads = acquireLanes(query);
+    return resultSequence.withBaggage(() -> releaseLanes(bulkheads));
+  }
+
+  /**
+   * Forcibly cancel all futures that have been registered to a specific query id
+   */
+  public boolean cancelQuery(String id)
+  {
+    // if you re-use queryId and cancel queries... you are going to have a bad time
+    queryDatasources.removeAll(id);
+    Set<ListenableFuture<?>> futures = queryFutures.removeAll(id);
+    boolean success = true;
+    for (ListenableFuture<?> future : futures) {
+      success = success && future.cancel(true);
+    }
+    return success;
+  }
+
+  /**
+   * Get a {@link Set} of datasource names for a {@link Query} id, used by {@link QueryResource#cancelQuery} to
+   * authorize that a user may call {@link #cancelQuery} for the given id and datasources
+   */
+  public Set<String> getQueryDatasources(final String queryId)
+  {
+    return queryDatasources.get(queryId);
+  }
+
+  /**
+   * Get the maximum number of concurrent queries that {@link #run} can support
+   */
+  public int getTotalAvailableCapacity()
 
 Review comment:
   This and the method below are public but they appear to just be used in the unit test.

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] clintropolis commented on a change in pull request #9407: query laning and load shedding

Posted by GitBox <gi...@apache.org>.
clintropolis commented on a change in pull request #9407: query laning and load shedding
URL: https://github.com/apache/druid/pull/9407#discussion_r389161719
 
 

 ##########
 File path: server/src/main/java/org/apache/druid/server/scheduling/HiLoQueryLaningStrategy.java
 ##########
 @@ -0,0 +1,79 @@
+/*
+ * 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.scheduling;
+
+import com.fasterxml.jackson.annotation.JsonCreator;
+import com.fasterxml.jackson.annotation.JsonProperty;
+import com.google.common.base.Preconditions;
+import com.google.common.primitives.Ints;
+import it.unimi.dsi.fastutil.objects.Object2IntArrayMap;
+import it.unimi.dsi.fastutil.objects.Object2IntMap;
+import org.apache.druid.client.SegmentServerSelector;
+import org.apache.druid.query.Query;
+import org.apache.druid.query.QueryContexts;
+import org.apache.druid.query.QueryPlus;
+import org.apache.druid.server.QueryLaningStrategy;
+
+import java.util.Optional;
+import java.util.Set;
+
+/**
+ * Query laning strategy which associates all {@link Query} with priority lower than 0 into a 'low' lane
+ */
+public class HiLoQueryLaningStrategy implements QueryLaningStrategy
+{
+  public static final String LOW = "low";
+
+  @JsonProperty
+  private final int maxLowPercent;
 
 Review comment:
   hmm, this is the maximum number of concurrent queries for this lane.
   
   `druid.query.scheduler.numThreads` or `druid.server.http.numThreads` define the maximum number of concurrent running queries, but it's not lane specific.

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] jihoonson commented on a change in pull request #9407: query laning and load shedding

Posted by GitBox <gi...@apache.org>.
jihoonson commented on a change in pull request #9407: query laning and load shedding
URL: https://github.com/apache/druid/pull/9407#discussion_r389215417
 
 

 ##########
 File path: server/src/main/java/org/apache/druid/server/QueryScheduler.java
 ##########
 @@ -0,0 +1,241 @@
+/*
+ * 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;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.collect.HashMultimap;
+import com.google.common.collect.Multimaps;
+import com.google.common.collect.SetMultimap;
+import com.google.common.util.concurrent.ListenableFuture;
+import io.github.resilience4j.bulkhead.Bulkhead;
+import io.github.resilience4j.bulkhead.BulkheadConfig;
+import io.github.resilience4j.bulkhead.BulkheadRegistry;
+import it.unimi.dsi.fastutil.objects.Object2IntMap;
+import org.apache.druid.client.SegmentServerSelector;
+import org.apache.druid.java.util.common.concurrent.Execs;
+import org.apache.druid.java.util.common.guava.Sequence;
+import org.apache.druid.query.Query;
+import org.apache.druid.query.QueryContexts;
+import org.apache.druid.query.QueryPlus;
+import org.apache.druid.query.QueryWatcher;
+import org.apache.druid.server.initialization.ServerConfig;
+
+import java.time.Duration;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+
+/**
+ * QueryScheduler (potentially) assigns any {@link Query} that is to be executed to a 'query lane' using the
+ * {@link QueryLaningStrategy} that is defined in {@link QuerySchedulerConfig}.
+ *
+ * As a {@link QueryWatcher}, it also provides cancellation facilities.
+ *
+ * This class is shared by all requests on the Jetty HTTP theadpool and must be thread safe.
+ */
+public class QueryScheduler implements QueryWatcher
+{
+  static final String TOTAL = "default";
+  private final int totalCapacity;
+  private final QueryLaningStrategy laningStrategy;
+  private final BulkheadRegistry laneRegistry;
+  private final SetMultimap<String, ListenableFuture<?>> queryFutures;
+  private final SetMultimap<String, String> queryDatasources;
+
+  public QueryScheduler(int totalNumThreads, QueryLaningStrategy laningStrategy, ServerConfig serverConfig)
+  {
+    this.laningStrategy = laningStrategy;
+    this.queryFutures = Multimaps.synchronizedSetMultimap(HashMultimap.create());
+    this.queryDatasources = Multimaps.synchronizedSetMultimap(HashMultimap.create());
+    // if totalNumThreads is above 0 and less than druid.server.http.numThreads, enforce total limit
+    final boolean limitTotal;
+    if (totalNumThreads > 0 && totalNumThreads < serverConfig.getNumThreads()) {
+      limitTotal = true;
+      this.totalCapacity = totalNumThreads;
+    } else {
+      limitTotal = false;
+      this.totalCapacity = serverConfig.getNumThreads();
+    }
+    this.laneRegistry = BulkheadRegistry.of(getLaneConfigs(limitTotal));
+  }
+
+  @Override
+  public void registerQueryFuture(Query<?> query, ListenableFuture<?> future)
+  {
+    final String id = query.getId();
+    final Set<String> datasources = query.getDataSource().getTableNames();
+    queryFutures.put(id, future);
+    queryDatasources.putAll(id, datasources);
+    future.addListener(
+        () -> {
+          queryFutures.remove(id, future);
+          for (String datasource : datasources) {
+            queryDatasources.remove(id, datasource);
+          }
+        },
+        Execs.directExecutor()
+    );
+  }
+
+  /**
+   * Assign a query a lane (if not set)
+   */
+  public <T> Query<T> laneQuery(QueryPlus<T> queryPlus, Set<SegmentServerSelector> segments)
+  {
+    Query<T> query = queryPlus.getQuery();
+    // man wins over machine.. for now.
+    if (QueryContexts.getLane(query) != null) {
+      return query;
+    }
+    Optional<String> lane = laningStrategy.computeLane(queryPlus, segments);
+    return lane.map(query::withLane).orElse(query);
+  }
+
+  /**
+   * Run a query with the scheduler, attempting to acquire a semaphore from the total and lane specific query capacities
+   *
+   * Note that {@link #cancelQuery} should not interrupt the thread that calls run, in all current usages it only
+   * cancels any {@link ListenableFuture} created downstream. If this ever commonly changes, we should add
+   * synchronization between {@link #cancelQuery} and the acquisition of the {@link Bulkhead} to continue to ensure that
+   * anything acquired is also released.
+   *
+   * In the meantime, if a {@link ListenableFuture} is registered for the query that calls this method, it MUST handle
+   * this synchronization itself to ensure that no {@link Bulkhead} is acquired without releasing it.
+   */
+  public <T> Sequence<T> run(Query<?> query, Sequence<T> resultSequence)
+  {
+    List<Bulkhead> bulkheads = acquireLanes(query);
+    return resultSequence.withBaggage(() -> releaseLanes(bulkheads));
+  }
+
+  /**
+   * Forcibly cancel all futures that have been registered to a specific query id
+   */
+  public boolean cancelQuery(String id)
+  {
+    // if you re-use queryId and cancel queries... you are going to have a bad time
+    queryDatasources.removeAll(id);
+    Set<ListenableFuture<?>> futures = queryFutures.removeAll(id);
+    boolean success = true;
+    for (ListenableFuture<?> future : futures) {
+      success = success && future.cancel(true);
+    }
+    return success;
+  }
+
+  /**
+   * Get a {@link Set} of datasource names for a {@link Query} id, used by {@link QueryResource#cancelQuery} to
+   * authorize that a user may call {@link #cancelQuery} for the given id and datasources
+   */
+  public Set<String> getQueryDatasources(final String queryId)
+  {
+    return queryDatasources.get(queryId);
+  }
+
+  /**
+   * Get the maximum number of concurrent queries that {@link #run} can support
+   */
+  @VisibleForTesting
+  int getTotalAvailableCapacity()
+  {
+    return laneRegistry.getConfiguration(TOTAL)
+                       .map(config -> laneRegistry.bulkhead(TOTAL, config).getMetrics().getAvailableConcurrentCalls())
+                       .orElse(-1);
+  }
+
+  /**
+   * Get the maximum number of concurrent queries that {@link #run} can support for a given lane
+   */
+  @VisibleForTesting
+  int getLaneAvailableCapacity(String lane)
+  {
+    return laneRegistry.getConfiguration(lane)
+                       .map(config -> laneRegistry.bulkhead(lane, config).getMetrics().getAvailableConcurrentCalls())
+                       .orElse(-1);
+  }
+
+  /**
+   * Acquire a semaphore for both the 'total' and a lane, if any is associated with a query
+   */
+  @VisibleForTesting
+  List<Bulkhead> acquireLanes(Query<?> query)
+  {
+    final String lane = QueryContexts.getLane(query);
+    final Optional<BulkheadConfig> laneConfig = lane == null ? Optional.empty() : laneRegistry.getConfiguration(lane);
+    List<Bulkhead> hallPasses = new ArrayList<>(2);
+    final Optional<BulkheadConfig> totalConfig = laneRegistry.getConfiguration(TOTAL);
+    // if we have a lane, get it first
+    laneConfig.ifPresent(config -> {
 
 Review comment:
   The comment sounds good to me. Thanks.

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] himanshug commented on a change in pull request #9407: query laning and load shedding

Posted by GitBox <gi...@apache.org>.
himanshug commented on a change in pull request #9407: query laning and load shedding
URL: https://github.com/apache/druid/pull/9407#discussion_r390549892
 
 

 ##########
 File path: docs/configuration/index.md
 ##########
 @@ -1481,9 +1481,35 @@ These Broker configurations can be defined in the `broker/runtime.properties` fi
 |`druid.broker.select.tier`|`highestPriority`, `lowestPriority`, `custom`|If segments are cross-replicated across tiers in a cluster, you can tell the broker to prefer to select segments in a tier with a certain priority.|`highestPriority`|
 |`druid.broker.select.tier.custom.priorities`|`An array of integer priorities.`|Select servers in tiers with a custom priority list.|None|
 
+##### Query laning
+
+*Laning strategies* allow you to control capacity utilization for heterogeneous query workloads. With laning, the broker examines and classifies a query for the purpose of assigning it to a 'lane'. Lanes have capacity limits, enforced by the broker, that can be used to ensure sufficient resources are available for other lanes or for interactive queries (with no lane), or to limit overall throughput for queries within the lane. Requests in excess of the capacity are discarded with an HTTP 429 status code.
+
+|Property|Description|Default|
+|--------|-----------|-------|
+|`druid.query.scheduler.numThreads`|Maximum number of HTTP threads to dedicate to query processing. To save HTTP thread capacity, this should be lower than `druid.server.http.numThreads`.|Unbounded|
 
 Review comment:
   thanks, I understand the  reasoning now.
   lookup end points already have a QoS filter to never consume more than two threads from jetty, I wonder if in this world it makes sense to setup QoS filter for non-query endpoints(say hardcoded to 2) so that we can ensure that  they don't end up consuming more jetty threads than intended.
   then default  `druid.query.scheduler.numThreads = druid.server.http.numThreads - numReservedForOthers=4` and users would likely never be expected to touch `druid.query.scheduler.numThreads` .
   
   Major behavior change with lane usage is really losing the queuing  of requests to handle spikes and instead sending 429s immediately.  In future, we  could introduce mechanism to maintain statically/dynamically sized [per lane] waiting  queue ourselves as well along with concurrency limits in lane strategy.

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] ccaominh commented on a change in pull request #9407: query laning and load shedding

Posted by GitBox <gi...@apache.org>.
ccaominh commented on a change in pull request #9407: query laning and load shedding
URL: https://github.com/apache/druid/pull/9407#discussion_r388666358
 
 

 ##########
 File path: server/src/main/java/org/apache/druid/server/QueryScheduler.java
 ##########
 @@ -0,0 +1,226 @@
+/*
+ * 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;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.collect.HashMultimap;
+import com.google.common.collect.Multimaps;
+import com.google.common.collect.SetMultimap;
+import com.google.common.util.concurrent.ListenableFuture;
+import io.github.resilience4j.bulkhead.Bulkhead;
+import io.github.resilience4j.bulkhead.BulkheadConfig;
+import io.github.resilience4j.bulkhead.BulkheadRegistry;
+import it.unimi.dsi.fastutil.objects.Object2IntMap;
+import org.apache.druid.client.SegmentServerSelector;
+import org.apache.druid.java.util.common.concurrent.Execs;
+import org.apache.druid.java.util.common.guava.Sequence;
+import org.apache.druid.query.Query;
+import org.apache.druid.query.QueryContexts;
+import org.apache.druid.query.QueryPlus;
+import org.apache.druid.query.QueryWatcher;
+
+import java.time.Duration;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+
+/**
+ * QueryScheduler (potentially) assigns any {@link Query} that is to be executed to a 'query lane' using the
+ * {@link QueryLaningStrategy} that is defined in {@link QuerySchedulerConfig}.
+ *
+ * As a {@link QueryWatcher}, it also provides cancellation facilities.
+ *
+ * This class is shared by all requests on the Jetty HTTP theadpool and must be thread safe.
+ */
+public class QueryScheduler implements QueryWatcher
+{
+  static final String TOTAL = "default";
+  private final QueryLaningStrategy laningStrategy;
+  private final BulkheadRegistry laneRegistry;
+  private final SetMultimap<String, ListenableFuture<?>> queryFutures;
+  private final SetMultimap<String, String> queryDatasources;
+
+  public QueryScheduler(int totalNumThreads, QueryLaningStrategy laningStrategy)
+  {
+    this.laningStrategy = laningStrategy;
+    this.laneRegistry = BulkheadRegistry.of(getLaneConfigs(totalNumThreads));
+    this.queryFutures = Multimaps.synchronizedSetMultimap(HashMultimap.create());
+    this.queryDatasources = Multimaps.synchronizedSetMultimap(HashMultimap.create());
+  }
+
+  @Override
+  public void registerQueryFuture(Query<?> query, ListenableFuture<?> future)
+  {
+    final String id = query.getId();
+    final Set<String> datasources = query.getDataSource().getTableNames();
+    queryFutures.put(id, future);
+    queryDatasources.putAll(id, datasources);
+    future.addListener(
+        () -> {
+          queryFutures.remove(id, future);
+          for (String datasource : datasources) {
+            queryDatasources.remove(id, datasource);
+          }
+        },
+        Execs.directExecutor()
+    );
+  }
+
+  /**
+   * Assign a query a lane (if not set)
+   */
+  public <T> Query<T> laneQuery(QueryPlus<T> queryPlus, Set<SegmentServerSelector> segments)
+  {
+    Query<T> query = queryPlus.getQuery();
+    // man wins over machine.. for now.
+    if (QueryContexts.getLane(query) != null) {
+      return query;
+    }
+    Optional<String> lane = laningStrategy.computeLane(queryPlus, segments);
+    return lane.map(query::withLane).orElse(query);
+  }
+
+  /**
+   * Run a query with the scheduler, attempting to acquire a semaphore from the total and lane specific query capacities
+   *
+   * Note that {@link #cancelQuery} should not interrupt the thread that calls run, in all current usages it only
+   * cancels any {@link ListenableFuture} created downstream. If this ever commonly changes, we should add
+   * synchronization between {@link #cancelQuery} and the acquisition of the {@link Bulkhead} to continue to ensure that
+   * anything acquired is also released.
+   *
+   * In the meantime, if a {@link ListenableFuture} is registered for the query that calls this method, it MUST handle
+   * this synchronization itself to ensure that no {@link Bulkhead} is acquired without releasing it.
+   */
+  public <T> Sequence<T> run(Query<?> query, Sequence<T> resultSequence)
+  {
+    List<Bulkhead> bulkheads = acquireLanes(query);
+    return resultSequence.withBaggage(() -> releaseLanes(bulkheads));
+  }
+
+  /**
+   * Forcibly cancel all futures that have been registered to a specific query id
+   */
+  public boolean cancelQuery(String id)
+  {
+    // if you re-use queryId and cancel queries... you are going to have a bad time
+    queryDatasources.removeAll(id);
+    Set<ListenableFuture<?>> futures = queryFutures.removeAll(id);
+    boolean success = true;
+    for (ListenableFuture<?> future : futures) {
+      success = success && future.cancel(true);
+    }
+    return success;
+  }
+
+  /**
+   * Get a {@link Set} of datasource names for a {@link Query} id, used by {@link QueryResource#cancelQuery} to
+   * authorize that a user may call {@link #cancelQuery} for the given id and datasources
+   */
+  public Set<String> getQueryDatasources(final String queryId)
+  {
+    return queryDatasources.get(queryId);
+  }
+
+  /**
+   * Get the maximum number of concurrent queries that {@link #run} can support
+   */
+  public int getTotalAvailableCapacity()
+  {
+    return laneRegistry.getConfiguration(TOTAL)
+                       .map(config -> laneRegistry.bulkhead(TOTAL, config).getMetrics().getAvailableConcurrentCalls())
+                       .orElse(-1);
+  }
+
+  /**
+   * Get the maximum number of concurrent queries that {@link #run} can support for a given lane
+   */
+  public int getLaneAvailableCapacity(String lane)
+  {
+    return laneRegistry.getConfiguration(lane)
+                       .map(config -> laneRegistry.bulkhead(lane, config).getMetrics().getAvailableConcurrentCalls())
+                       .orElse(-1);
+  }
+
+  /**
+   * Acquire a semaphore for both the 'total' and a lane, if any is associated with a query
+   */
+  @VisibleForTesting
+  List<Bulkhead> acquireLanes(Query<?> query)
+  {
+    final String lane = QueryContexts.getLane(query);
+    final Optional<BulkheadConfig> laneConfig = lane == null ? Optional.empty() : laneRegistry.getConfiguration(lane);
+    List<Bulkhead> hallPasses = new ArrayList<>(2);
+    final Optional<BulkheadConfig> totalConfig = laneRegistry.getConfiguration(TOTAL);
+    // if we have a lane, get it first
+    laneConfig.ifPresent(config -> {
+      Bulkhead laneLimiter = laneRegistry.bulkhead(lane, config);
+      if (!laneLimiter.tryAcquirePermission()) {
+        throw new QueryCapacityExceededException(lane);
+      }
+      hallPasses.add(laneLimiter);
+    });
+
+    // everyone needs to take one from the total lane; to ensure we don't acquire a lane and never release it, we want
 
 Review comment:
   What's the purpose of the TOTAL bulkhead if each lane has a bulkhead?

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] clintropolis commented on a change in pull request #9407: query laning and load shedding

Posted by GitBox <gi...@apache.org>.
clintropolis commented on a change in pull request #9407: query laning and load shedding
URL: https://github.com/apache/druid/pull/9407#discussion_r389162648
 
 

 ##########
 File path: docs/configuration/index.md
 ##########
 @@ -1476,9 +1476,31 @@ These Broker configurations can be defined in the `broker/runtime.properties` fi
 |`druid.broker.select.tier`|`highestPriority`, `lowestPriority`, `custom`|If segments are cross-replicated across tiers in a cluster, you can tell the broker to prefer to select segments in a tier with a certain priority.|`highestPriority`|
 |`druid.broker.select.tier.custom.priorities`|`An array of integer priorities.`|Select servers in tiers with a custom priority list.|None|
 
+##### Query laning
+
+The Broker provides facilities to aid in query capacity reservation for heterogeneous query workloads in the form of 'laning' strategies, which provide a variety of mechanisms to examine and classify a query, assigning it to a 'lane'. Lanes are defined with capacity limits which the broker will enforce, causing requests in excess of the capacity to be discarded with an HTTP 429 status code, reserving resources for other lanes or for interactive queries (with no lane).
+
+|Property|Description|Default|
+|--------|-----------|-------|
+|`druid.query.scheduler.numThreads`|Maximum number of HTTP threads to dedicate to query processing. To save HTTP thread capacity, this should be lower than `druid.server.http.numThreads`.|Unbounded|
+|`druid.query.scheduler.laning.type`|Query laning strategy to use to assign queries to a lane in order to control capacities for certain classes of queries.|`none`|
+
+###### No laning strategy
+
+In this mode, queries are never assigned a lane and only limited by `druid.server.http.numThreads` or `druid.query.scheduler.numThreads`, if set. This is the default Druid query scheduler operating mode.
+
+###### 'High/Low' laning strategy
+This laning strategy splits queries with a `priority` below zero into a `low` query lane, automatically. The limit on `low` queries can be set to some desired fraction of the total capacity (or HTTP thread pool size), reserving capacity for interactive queries. Queries in the `low` lane are _not_ guaranteed their capacity, which may be consumed by interactive queries, but may use up to this limit if total capacity is available. 
+
+This strategy can be enabled by setting `druid.query.scheduler.laning.type` to `hilo`.
+
+|Property|Description|Default|
+|--------|-----------|-------|
+|`druid.query.scheduler.laning.maxLowThreads`|Maximum number of HTTP threads that can be used by queries with a priority lower than 0.|No default, must be set if using this mode|
 
 Review comment:
   reworked/renamed to `maxLowPercent` to be more flexible

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] clintropolis commented on a change in pull request #9407: query laning and load shedding

Posted by GitBox <gi...@apache.org>.
clintropolis commented on a change in pull request #9407: query laning and load shedding
URL: https://github.com/apache/druid/pull/9407#discussion_r389159233
 
 

 ##########
 File path: server/src/main/java/org/apache/druid/server/QueryScheduler.java
 ##########
 @@ -0,0 +1,226 @@
+/*
+ * 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;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.collect.HashMultimap;
+import com.google.common.collect.Multimaps;
+import com.google.common.collect.SetMultimap;
+import com.google.common.util.concurrent.ListenableFuture;
+import io.github.resilience4j.bulkhead.Bulkhead;
+import io.github.resilience4j.bulkhead.BulkheadConfig;
+import io.github.resilience4j.bulkhead.BulkheadRegistry;
+import it.unimi.dsi.fastutil.objects.Object2IntMap;
+import org.apache.druid.client.SegmentServerSelector;
+import org.apache.druid.java.util.common.concurrent.Execs;
+import org.apache.druid.java.util.common.guava.Sequence;
+import org.apache.druid.query.Query;
+import org.apache.druid.query.QueryContexts;
+import org.apache.druid.query.QueryPlus;
+import org.apache.druid.query.QueryWatcher;
+
+import java.time.Duration;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+
+/**
+ * QueryScheduler (potentially) assigns any {@link Query} that is to be executed to a 'query lane' using the
+ * {@link QueryLaningStrategy} that is defined in {@link QuerySchedulerConfig}.
+ *
+ * As a {@link QueryWatcher}, it also provides cancellation facilities.
+ *
+ * This class is shared by all requests on the Jetty HTTP theadpool and must be thread safe.
+ */
+public class QueryScheduler implements QueryWatcher
+{
+  static final String TOTAL = "default";
+  private final QueryLaningStrategy laningStrategy;
+  private final BulkheadRegistry laneRegistry;
+  private final SetMultimap<String, ListenableFuture<?>> queryFutures;
+  private final SetMultimap<String, String> queryDatasources;
+
+  public QueryScheduler(int totalNumThreads, QueryLaningStrategy laningStrategy)
+  {
+    this.laningStrategy = laningStrategy;
+    this.laneRegistry = BulkheadRegistry.of(getLaneConfigs(totalNumThreads));
+    this.queryFutures = Multimaps.synchronizedSetMultimap(HashMultimap.create());
+    this.queryDatasources = Multimaps.synchronizedSetMultimap(HashMultimap.create());
+  }
+
+  @Override
+  public void registerQueryFuture(Query<?> query, ListenableFuture<?> future)
+  {
+    final String id = query.getId();
+    final Set<String> datasources = query.getDataSource().getTableNames();
+    queryFutures.put(id, future);
+    queryDatasources.putAll(id, datasources);
+    future.addListener(
+        () -> {
+          queryFutures.remove(id, future);
+          for (String datasource : datasources) {
+            queryDatasources.remove(id, datasource);
+          }
+        },
+        Execs.directExecutor()
+    );
+  }
+
+  /**
+   * Assign a query a lane (if not set)
+   */
+  public <T> Query<T> laneQuery(QueryPlus<T> queryPlus, Set<SegmentServerSelector> segments)
+  {
+    Query<T> query = queryPlus.getQuery();
+    // man wins over machine.. for now.
+    if (QueryContexts.getLane(query) != null) {
+      return query;
+    }
+    Optional<String> lane = laningStrategy.computeLane(queryPlus, segments);
+    return lane.map(query::withLane).orElse(query);
+  }
+
+  /**
+   * Run a query with the scheduler, attempting to acquire a semaphore from the total and lane specific query capacities
+   *
+   * Note that {@link #cancelQuery} should not interrupt the thread that calls run, in all current usages it only
+   * cancels any {@link ListenableFuture} created downstream. If this ever commonly changes, we should add
+   * synchronization between {@link #cancelQuery} and the acquisition of the {@link Bulkhead} to continue to ensure that
+   * anything acquired is also released.
+   *
+   * In the meantime, if a {@link ListenableFuture} is registered for the query that calls this method, it MUST handle
+   * this synchronization itself to ensure that no {@link Bulkhead} is acquired without releasing it.
+   */
+  public <T> Sequence<T> run(Query<?> query, Sequence<T> resultSequence)
+  {
+    List<Bulkhead> bulkheads = acquireLanes(query);
+    return resultSequence.withBaggage(() -> releaseLanes(bulkheads));
+  }
+
+  /**
+   * Forcibly cancel all futures that have been registered to a specific query id
+   */
+  public boolean cancelQuery(String id)
+  {
+    // if you re-use queryId and cancel queries... you are going to have a bad time
+    queryDatasources.removeAll(id);
+    Set<ListenableFuture<?>> futures = queryFutures.removeAll(id);
+    boolean success = true;
+    for (ListenableFuture<?> future : futures) {
+      success = success && future.cancel(true);
+    }
+    return success;
+  }
+
+  /**
+   * Get a {@link Set} of datasource names for a {@link Query} id, used by {@link QueryResource#cancelQuery} to
+   * authorize that a user may call {@link #cancelQuery} for the given id and datasources
+   */
+  public Set<String> getQueryDatasources(final String queryId)
+  {
+    return queryDatasources.get(queryId);
+  }
+
+  /**
+   * Get the maximum number of concurrent queries that {@link #run} can support
+   */
+  public int getTotalAvailableCapacity()
+  {
+    return laneRegistry.getConfiguration(TOTAL)
+                       .map(config -> laneRegistry.bulkhead(TOTAL, config).getMetrics().getAvailableConcurrentCalls())
+                       .orElse(-1);
+  }
+
+  /**
+   * Get the maximum number of concurrent queries that {@link #run} can support for a given lane
+   */
+  public int getLaneAvailableCapacity(String lane)
+  {
+    return laneRegistry.getConfiguration(lane)
+                       .map(config -> laneRegistry.bulkhead(lane, config).getMetrics().getAvailableConcurrentCalls())
+                       .orElse(-1);
+  }
+
+  /**
+   * Acquire a semaphore for both the 'total' and a lane, if any is associated with a query
+   */
+  @VisibleForTesting
+  List<Bulkhead> acquireLanes(Query<?> query)
+  {
+    final String lane = QueryContexts.getLane(query);
+    final Optional<BulkheadConfig> laneConfig = lane == null ? Optional.empty() : laneRegistry.getConfiguration(lane);
+    List<Bulkhead> hallPasses = new ArrayList<>(2);
+    final Optional<BulkheadConfig> totalConfig = laneRegistry.getConfiguration(TOTAL);
+    // if we have a lane, get it first
+    laneConfig.ifPresent(config -> {
+      Bulkhead laneLimiter = laneRegistry.bulkhead(lane, config);
+      if (!laneLimiter.tryAcquirePermission()) {
+        throw new QueryCapacityExceededException(lane);
+      }
+      hallPasses.add(laneLimiter);
+    });
+
+    // everyone needs to take one from the total lane; to ensure we don't acquire a lane and never release it, we want
 
 Review comment:
   that is true, but I wanted to dual purpose this to also be able to set total limits to protect the rest of the service from the query processing system, and save room for health checks and the like. 
   
   As I discussed in the description, I considered that the total counter specifically might be suitable as a jetty `QoSFilter`, which also is using a semaphore, however we would need to ensure that we set the request timeout high enough to give the laned queries time to fail and release the semaphore since the total would now be getting acquired before the lane. It seemed easier to me to track here than to precisely determine that interval, however the discussion is worth having I think.

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] clintropolis commented on a change in pull request #9407: query laning and load shedding

Posted by GitBox <gi...@apache.org>.
clintropolis commented on a change in pull request #9407: query laning and load shedding
URL: https://github.com/apache/druid/pull/9407#discussion_r388752431
 
 

 ##########
 File path: server/src/main/java/org/apache/druid/server/QueryCapacityExceededException.java
 ##########
 @@ -0,0 +1,53 @@
+/*
+ * 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;
+
+import com.fasterxml.jackson.annotation.JsonCreator;
+import com.fasterxml.jackson.annotation.JsonProperty;
+import org.apache.druid.java.util.common.StringUtils;
+import org.apache.druid.query.QueryException;
+
+public class QueryCapacityExceededException extends QueryException
 
 Review comment:
   I think I would agree that this area probably needs a refactor, maybe more suitable to be done in a follow-up PR, but will try to at least add some javadocs to clarify that the `QueryExceptions` are intended to be the main exceptions we surface from the API provided by `QueryResource` and `SqlResource`. Currently `QueryInterruptedException` is specifically caught and tied to HTTP 500 errors in these classes, so I split out `QueryCapacityExceededException` so that I could catch it separately and respond with the 429 status code (or 503, whatever we end up going with).

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] clintropolis commented on a change in pull request #9407: query laning and load shedding

Posted by GitBox <gi...@apache.org>.
clintropolis commented on a change in pull request #9407: query laning and load shedding
URL: https://github.com/apache/druid/pull/9407#discussion_r388754569
 
 

 ##########
 File path: server/src/main/java/org/apache/druid/server/QueryScheduler.java
 ##########
 @@ -0,0 +1,226 @@
+/*
+ * 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;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.collect.HashMultimap;
+import com.google.common.collect.Multimaps;
+import com.google.common.collect.SetMultimap;
+import com.google.common.util.concurrent.ListenableFuture;
+import io.github.resilience4j.bulkhead.Bulkhead;
+import io.github.resilience4j.bulkhead.BulkheadConfig;
+import io.github.resilience4j.bulkhead.BulkheadRegistry;
+import it.unimi.dsi.fastutil.objects.Object2IntMap;
+import org.apache.druid.client.SegmentServerSelector;
+import org.apache.druid.java.util.common.concurrent.Execs;
+import org.apache.druid.java.util.common.guava.Sequence;
+import org.apache.druid.query.Query;
+import org.apache.druid.query.QueryContexts;
+import org.apache.druid.query.QueryPlus;
+import org.apache.druid.query.QueryWatcher;
+
+import java.time.Duration;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Set;
+
+/**
+ * QueryScheduler (potentially) assigns any {@link Query} that is to be executed to a 'query lane' using the
+ * {@link QueryLaningStrategy} that is defined in {@link QuerySchedulerConfig}.
+ *
+ * As a {@link QueryWatcher}, it also provides cancellation facilities.
+ *
+ * This class is shared by all requests on the Jetty HTTP theadpool and must be thread safe.
+ */
+public class QueryScheduler implements QueryWatcher
+{
+  static final String TOTAL = "default";
+  private final QueryLaningStrategy laningStrategy;
+  private final BulkheadRegistry laneRegistry;
+  private final SetMultimap<String, ListenableFuture<?>> queryFutures;
+  private final SetMultimap<String, String> queryDatasources;
+
+  public QueryScheduler(int totalNumThreads, QueryLaningStrategy laningStrategy)
+  {
+    this.laningStrategy = laningStrategy;
+    this.laneRegistry = BulkheadRegistry.of(getLaneConfigs(totalNumThreads));
+    this.queryFutures = Multimaps.synchronizedSetMultimap(HashMultimap.create());
+    this.queryDatasources = Multimaps.synchronizedSetMultimap(HashMultimap.create());
+  }
+
+  @Override
+  public void registerQueryFuture(Query<?> query, ListenableFuture<?> future)
+  {
+    final String id = query.getId();
+    final Set<String> datasources = query.getDataSource().getTableNames();
+    queryFutures.put(id, future);
+    queryDatasources.putAll(id, datasources);
+    future.addListener(
+        () -> {
+          queryFutures.remove(id, future);
+          for (String datasource : datasources) {
+            queryDatasources.remove(id, datasource);
+          }
+        },
+        Execs.directExecutor()
+    );
+  }
+
+  /**
+   * Assign a query a lane (if not set)
+   */
+  public <T> Query<T> laneQuery(QueryPlus<T> queryPlus, Set<SegmentServerSelector> segments)
+  {
+    Query<T> query = queryPlus.getQuery();
+    // man wins over machine.. for now.
+    if (QueryContexts.getLane(query) != null) {
+      return query;
+    }
+    Optional<String> lane = laningStrategy.computeLane(queryPlus, segments);
+    return lane.map(query::withLane).orElse(query);
+  }
+
+  /**
+   * Run a query with the scheduler, attempting to acquire a semaphore from the total and lane specific query capacities
+   *
+   * Note that {@link #cancelQuery} should not interrupt the thread that calls run, in all current usages it only
+   * cancels any {@link ListenableFuture} created downstream. If this ever commonly changes, we should add
+   * synchronization between {@link #cancelQuery} and the acquisition of the {@link Bulkhead} to continue to ensure that
+   * anything acquired is also released.
+   *
+   * In the meantime, if a {@link ListenableFuture} is registered for the query that calls this method, it MUST handle
+   * this synchronization itself to ensure that no {@link Bulkhead} is acquired without releasing it.
+   */
+  public <T> Sequence<T> run(Query<?> query, Sequence<T> resultSequence)
+  {
+    List<Bulkhead> bulkheads = acquireLanes(query);
+    return resultSequence.withBaggage(() -> releaseLanes(bulkheads));
+  }
+
+  /**
+   * Forcibly cancel all futures that have been registered to a specific query id
+   */
+  public boolean cancelQuery(String id)
+  {
+    // if you re-use queryId and cancel queries... you are going to have a bad time
+    queryDatasources.removeAll(id);
+    Set<ListenableFuture<?>> futures = queryFutures.removeAll(id);
+    boolean success = true;
+    for (ListenableFuture<?> future : futures) {
+      success = success && future.cancel(true);
+    }
+    return success;
+  }
+
+  /**
+   * Get a {@link Set} of datasource names for a {@link Query} id, used by {@link QueryResource#cancelQuery} to
+   * authorize that a user may call {@link #cancelQuery} for the given id and datasources
+   */
+  public Set<String> getQueryDatasources(final String queryId)
+  {
+    return queryDatasources.get(queryId);
+  }
+
+  /**
+   * Get the maximum number of concurrent queries that {@link #run} can support
+   */
+  public int getTotalAvailableCapacity()
+  {
+    return laneRegistry.getConfiguration(TOTAL)
+                       .map(config -> laneRegistry.bulkhead(TOTAL, config).getMetrics().getAvailableConcurrentCalls())
+                       .orElse(-1);
+  }
+
+  /**
+   * Get the maximum number of concurrent queries that {@link #run} can support for a given lane
+   */
+  public int getLaneAvailableCapacity(String lane)
+  {
+    return laneRegistry.getConfiguration(lane)
+                       .map(config -> laneRegistry.bulkhead(lane, config).getMetrics().getAvailableConcurrentCalls())
+                       .orElse(-1);
+  }
+
+  /**
+   * Acquire a semaphore for both the 'total' and a lane, if any is associated with a query
+   */
+  @VisibleForTesting
+  List<Bulkhead> acquireLanes(Query<?> query)
+  {
+    final String lane = QueryContexts.getLane(query);
+    final Optional<BulkheadConfig> laneConfig = lane == null ? Optional.empty() : laneRegistry.getConfiguration(lane);
+    List<Bulkhead> hallPasses = new ArrayList<>(2);
+    final Optional<BulkheadConfig> totalConfig = laneRegistry.getConfiguration(TOTAL);
+    // if we have a lane, get it first
+    laneConfig.ifPresent(config -> {
+      Bulkhead laneLimiter = laneRegistry.bulkhead(lane, config);
+      if (!laneLimiter.tryAcquirePermission()) {
+        throw new QueryCapacityExceededException(lane);
+      }
+      hallPasses.add(laneLimiter);
+    });
+
+    // everyone needs to take one from the total lane; to ensure we don't acquire a lane and never release it, we want
 
 Review comment:
   the total is to control overall query throughput, and is tied to `druid.query.scheduler.numThreads`. Even queries without a lane are subject to this limit, if configured. I'll try to make this more clear in the javadocs or comments.

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] jihoonson commented on a change in pull request #9407: query laning and load shedding

Posted by GitBox <gi...@apache.org>.
jihoonson commented on a change in pull request #9407: query laning and load shedding
URL: https://github.com/apache/druid/pull/9407#discussion_r389050329
 
 

 ##########
 File path: server/src/main/java/org/apache/druid/server/QueryLaningStrategy.java
 ##########
 @@ -0,0 +1,51 @@
+/*
+ * 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;
+
+import com.fasterxml.jackson.annotation.JsonSubTypes;
+import com.fasterxml.jackson.annotation.JsonTypeInfo;
+import it.unimi.dsi.fastutil.objects.Object2IntMap;
+import org.apache.druid.client.SegmentServerSelector;
+import org.apache.druid.query.QueryPlus;
+import org.apache.druid.server.scheduling.HiLoQueryLaningStrategy;
+import org.apache.druid.server.scheduling.NoQueryLaningStrategy;
+
+import java.util.Optional;
+import java.util.Set;
+
+
+@JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "strategy", defaultImpl = NoQueryLaningStrategy.class)
+@JsonSubTypes(value = {
+    @JsonSubTypes.Type(name = "none", value = NoQueryLaningStrategy.class),
+    @JsonSubTypes.Type(name = "hilo", value = HiLoQueryLaningStrategy.class)
+})
+public interface QueryLaningStrategy
+{
+  /**
+   * Provide map of lane names to the limit on the number of concurrent queries for that lane
 
 Review comment:
   `Provide a map`?

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] clintropolis commented on a change in pull request #9407: query laning and load shedding

Posted by GitBox <gi...@apache.org>.
clintropolis commented on a change in pull request #9407: query laning and load shedding
URL: https://github.com/apache/druid/pull/9407#discussion_r389961130
 
 

 ##########
 File path: server/src/main/java/org/apache/druid/server/scheduling/HiLoQueryLaningStrategy.java
 ##########
 @@ -0,0 +1,79 @@
+/*
+ * 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.scheduling;
+
+import com.fasterxml.jackson.annotation.JsonCreator;
+import com.fasterxml.jackson.annotation.JsonProperty;
+import com.google.common.base.Preconditions;
+import com.google.common.primitives.Ints;
+import it.unimi.dsi.fastutil.objects.Object2IntArrayMap;
+import it.unimi.dsi.fastutil.objects.Object2IntMap;
+import org.apache.druid.client.SegmentServerSelector;
+import org.apache.druid.query.Query;
+import org.apache.druid.query.QueryContexts;
+import org.apache.druid.query.QueryPlus;
+import org.apache.druid.server.QueryLaningStrategy;
+
+import java.util.Optional;
+import java.util.Set;
+
+/**
+ * Query laning strategy which associates all {@link Query} with priority lower than 0 into a 'low' lane
+ */
+public class HiLoQueryLaningStrategy implements QueryLaningStrategy
+{
+  public static final String LOW = "low";
+
+  @JsonProperty
+  private final int maxLowPercent;
+
+  @JsonCreator
+  public HiLoQueryLaningStrategy(
+      @JsonProperty("maxLowPercent") Integer maxLowPercent
+  )
+  {
+    this.maxLowPercent = Preconditions.checkNotNull(maxLowPercent, "maxLowPercent must be set");
+    Preconditions.checkArgument(
+        0 < maxLowPercent && maxLowPercent < 100,
+        "maxLowPercent must be between 0 and 100"
+    );
+  }
+
+  @Override
+  public Object2IntMap<String> getLaneLimits(int totalLimit)
+  {
+    Object2IntMap<String> onlyLow = new Object2IntArrayMap<>(1);
+    onlyLow.put(LOW, Ints.checkedCast(Math.round(totalLimit * ((double) maxLowPercent / 100))));
 
 Review comment:
   Oops, I meant to round up, will adjust

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] jihoonson commented on a change in pull request #9407: query laning and load shedding

Posted by GitBox <gi...@apache.org>.
jihoonson commented on a change in pull request #9407: query laning and load shedding
URL: https://github.com/apache/druid/pull/9407#discussion_r389173425
 
 

 ##########
 File path: server/src/main/java/org/apache/druid/server/scheduling/HiLoQueryLaningStrategy.java
 ##########
 @@ -0,0 +1,79 @@
+/*
+ * 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.scheduling;
+
+import com.fasterxml.jackson.annotation.JsonCreator;
+import com.fasterxml.jackson.annotation.JsonProperty;
+import com.google.common.base.Preconditions;
+import com.google.common.primitives.Ints;
+import it.unimi.dsi.fastutil.objects.Object2IntArrayMap;
+import it.unimi.dsi.fastutil.objects.Object2IntMap;
+import org.apache.druid.client.SegmentServerSelector;
+import org.apache.druid.query.Query;
+import org.apache.druid.query.QueryContexts;
+import org.apache.druid.query.QueryPlus;
+import org.apache.druid.server.QueryLaningStrategy;
+
+import java.util.Optional;
+import java.util.Set;
+
+/**
+ * Query laning strategy which associates all {@link Query} with priority lower than 0 into a 'low' lane
+ */
+public class HiLoQueryLaningStrategy implements QueryLaningStrategy
+{
+  public static final String LOW = "low";
+
+  @JsonProperty
+  private final int maxLowPercent;
 
 Review comment:
   I mean, sometimes you may want to increase the total capacity, but keep it as it is for the low lane. In that case, `maxCapacity` would be more convenient because you don't have to change the configuration.

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] himanshug commented on a change in pull request #9407: query laning and load shedding

Posted by GitBox <gi...@apache.org>.
himanshug commented on a change in pull request #9407: query laning and load shedding
URL: https://github.com/apache/druid/pull/9407#discussion_r390124045
 
 

 ##########
 File path: docs/configuration/index.md
 ##########
 @@ -1481,9 +1481,35 @@ These Broker configurations can be defined in the `broker/runtime.properties` fi
 |`druid.broker.select.tier`|`highestPriority`, `lowestPriority`, `custom`|If segments are cross-replicated across tiers in a cluster, you can tell the broker to prefer to select segments in a tier with a certain priority.|`highestPriority`|
 |`druid.broker.select.tier.custom.priorities`|`An array of integer priorities.`|Select servers in tiers with a custom priority list.|None|
 
+##### Query laning
+
+*Laning strategies* allow you to control capacity utilization for heterogeneous query workloads. With laning, the broker examines and classifies a query for the purpose of assigning it to a 'lane'. Lanes have capacity limits, enforced by the broker, that can be used to ensure sufficient resources are available for other lanes or for interactive queries (with no lane), or to limit overall throughput for queries within the lane. Requests in excess of the capacity are discarded with an HTTP 429 status code.
+
+|Property|Description|Default|
+|--------|-----------|-------|
+|`druid.query.scheduler.numThreads`|Maximum number of HTTP threads to dedicate to query processing. To save HTTP thread capacity, this should be lower than `druid.server.http.numThreads`.|Unbounded|
 
 Review comment:
   in what use case would I ever want to set it something other  than `druid.server.http.numThreads` ? If the recommendation is to set it lower than `druid.server.http.numThreads` then why the default value is not set to `druid.server.http.numThreads - 1` ?
   I guess, as a user I don't quite understand the importance of setting this higher/same/lower compared to `druid.server.http.numThreads` and when I should choose one vs the other.
   

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] clintropolis commented on a change in pull request #9407: query laning and load shedding

Posted by GitBox <gi...@apache.org>.
clintropolis commented on a change in pull request #9407: query laning and load shedding
URL: https://github.com/apache/druid/pull/9407#discussion_r388761228
 
 

 ##########
 File path: sql/src/test/java/org/apache/druid/sql/calcite/util/SpecificSegmentsQuerySegmentWalker.java
 ##########
 @@ -388,13 +410,33 @@ public SegmentDescriptor getDescriptor()
           toolChest
       );
 
+
       // Wrap baseRunner in a runner that rewrites the QuerySegmentSpec to mention the specific segments.
       // This mimics what CachingClusteredClient on the Broker does, and is required for certain queries (like Scan)
       // to function properly.
-      return (theQuery, responseContext) -> baseRunner.run(
-          theQuery.withQuery(Queries.withSpecificSegments(theQuery.getQuery(), ImmutableList.copyOf(specs))),
-          responseContext
-      );
+      return (theQuery, responseContext) -> {
+        if (scheduler != null) {
+          Set<SegmentServerSelector> segments = new HashSet<>();
+          specs.forEach(spec -> segments.add(new SegmentServerSelector(null, spec)));
+          return scheduler.run(
+              scheduler.laneQuery(theQuery, segments),
+              new LazySequence<>(
+                  () -> baseRunner.run(
+                      theQuery.withQuery(Queries.withSpecificSegments(
+                          theQuery.getQuery(),
+                          ImmutableList.copyOf(specs)
+                      )),
+                      responseContext
+                  )
+              )
+          );
 
 Review comment:
   If you mean which tests use this block, then it is the [SqlResourceTest](https://github.com/apache/druid/pull/9407/files#diff-5a3523f6924e529221fa60db2bb56f05R125) which create this thing with a scheduler.
   
   No laning strategy is currently using the set of `SegmentServerSelector` to make decisions at this time, I just went ahead and wired it up because we do have the set of `SegmentDescriptors` available here, in anticipation of some future usage. With the `ServerSelector` part of that set to `null` though, we would probably want to either have a mock or artifical set of servers to pretend to serve the test segments, so it might still need some future work?

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] clintropolis commented on a change in pull request #9407: query laning and load shedding

Posted by GitBox <gi...@apache.org>.
clintropolis commented on a change in pull request #9407: query laning and load shedding
URL: https://github.com/apache/druid/pull/9407#discussion_r389175395
 
 

 ##########
 File path: server/src/main/java/org/apache/druid/server/scheduling/HiLoQueryLaningStrategy.java
 ##########
 @@ -0,0 +1,79 @@
+/*
+ * 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.scheduling;
+
+import com.fasterxml.jackson.annotation.JsonCreator;
+import com.fasterxml.jackson.annotation.JsonProperty;
+import com.google.common.base.Preconditions;
+import com.google.common.primitives.Ints;
+import it.unimi.dsi.fastutil.objects.Object2IntArrayMap;
+import it.unimi.dsi.fastutil.objects.Object2IntMap;
+import org.apache.druid.client.SegmentServerSelector;
+import org.apache.druid.query.Query;
+import org.apache.druid.query.QueryContexts;
+import org.apache.druid.query.QueryPlus;
+import org.apache.druid.server.QueryLaningStrategy;
+
+import java.util.Optional;
+import java.util.Set;
+
+/**
+ * Query laning strategy which associates all {@link Query} with priority lower than 0 into a 'low' lane
+ */
+public class HiLoQueryLaningStrategy implements QueryLaningStrategy
+{
+  public static final String LOW = "low";
+
+  @JsonProperty
+  private final int maxLowPercent;
 
 Review comment:
   Ah, so do you mean the lane limit should be the min or (or max?) of `maxCapacity` and `maxLowPercent` of `totalCapacity`?

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] himanshug commented on issue #9407: query laning and load shedding

Posted by GitBox <gi...@apache.org>.
himanshug commented on issue #9407: query laning and load shedding
URL: https://github.com/apache/druid/pull/9407#issuecomment-592645426
 
 
   /subscribe

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org


[GitHub] [druid] himanshug commented on a change in pull request #9407: query laning and load shedding

Posted by GitBox <gi...@apache.org>.
himanshug commented on a change in pull request #9407: query laning and load shedding
URL: https://github.com/apache/druid/pull/9407#discussion_r390098011
 
 

 ##########
 File path: server/src/main/java/org/apache/druid/server/QuerySchedulerConfig.java
 ##########
 @@ -0,0 +1,42 @@
+/*
+ * 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;
+
+import com.fasterxml.jackson.annotation.JsonProperty;
+import org.apache.druid.server.scheduling.NoQueryLaningStrategy;
+
+public class QuerySchedulerConfig
+{
+  @JsonProperty
+  private Integer numThreads = 0;
 
 Review comment:
   why is this  Integer and not int considering it has a default non-null value?

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org