gortiz commented on code in PR #14574:
URL: https://github.com/apache/pinot/pull/14574#discussion_r1886491028
##########
pinot-broker/src/main/java/org/apache/pinot/broker/requesthandler/MultiStageBrokerRequestHandler.java:
##########
@@ -224,67 +227,89 @@ protected BrokerResponse handleRequest(long requestId,
String query, SqlNodeAndO
return new
BrokerResponseNative(QueryException.getException(QueryException.QUOTA_EXCEEDED_ERROR,
errorMessage));
}
- Tracing.ThreadAccountantOps.setupRunner(String.valueOf(requestId),
ThreadExecutionContext.TaskType.MSE);
-
- long executionStartTimeNs = System.nanoTime();
- QueryDispatcher.QueryResult queryResults;
+ Timer queryTimer = new Timer(queryTimeoutMs);
try {
- queryResults =
- _queryDispatcher.submitAndReduce(requestContext,
dispatchableSubPlan, queryTimeoutMs, queryOptions);
- } catch (TimeoutException e) {
- for (String table : tableNames) {
- _brokerMetrics.addMeteredTableValue(table,
BrokerMeter.BROKER_RESPONSES_WITH_TIMEOUTS, 1);
+ // It's fine to block in this thread because we use a separate thread
pool from the main Jersey server to process
+ // these requests.
+ if (!_querySemaphore.tryAcquire(queryTimeoutMs, TimeUnit.MILLISECONDS)) {
+ LOGGER.warn("Timed out waiting to execute request {}: {}", requestId,
query);
+
requestContext.setErrorCode(QueryException.EXECUTION_TIMEOUT_ERROR_CODE);
+ return new
BrokerResponseNative(QueryException.EXECUTION_TIMEOUT_ERROR);
}
- LOGGER.warn("Timed out executing request {}: {}", requestId, query);
+ } catch (InterruptedException e) {
+ LOGGER.warn("Interrupt received while waiting to execute request {}:
{}", requestId, query);
requestContext.setErrorCode(QueryException.EXECUTION_TIMEOUT_ERROR_CODE);
return new BrokerResponseNative(QueryException.EXECUTION_TIMEOUT_ERROR);
Review Comment:
Maybe not in this PR, but we will probably need to have a way to tell the
user where we were waiting most of the time. It could be useful for example to
tell users that their queries are timing out because other queries are
consuming all threads.
##########
pinot-broker/src/main/java/org/apache/pinot/broker/requesthandler/MultiStageQuerySemaphore.java:
##########
@@ -0,0 +1,174 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pinot.broker.requesthandler;
+
+import com.google.common.base.Preconditions;
+import java.util.Collections;
+import java.util.concurrent.Semaphore;
+import java.util.concurrent.TimeUnit;
+import org.apache.helix.HelixAdmin;
+import org.apache.helix.HelixConstants;
+import org.apache.helix.HelixManager;
+import org.apache.helix.model.HelixConfigScope;
+import org.apache.helix.model.builder.HelixConfigScopeBuilder;
+import org.apache.pinot.broker.broker.helix.ClusterChangeHandler;
+import org.apache.pinot.spi.utils.CommonConstants;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/**
+ * This class helps limit the number of multi-stage queries being executed
concurrently. Currently, this limit is
+ * applied at the broker level, but could be moved to the server level in the
future.
+ */
+public class MultiStageQuerySemaphore extends Semaphore implements
ClusterChangeHandler {
Review Comment:
IMHO this class does too many things. I would suggest to split it in two
different classes:
1. The semaphore itself. It doesn't need to know the number of brokers, max
concurrent queries, etc. It has a `setTotalPermits` for a new number of
permits, which calls `super.reducePermits` and `super.release` if needed.
2. The ClusterChangeHandler, which has a reference to the semaphore. It
knows the number of brokers and max concurrent queries and calls
`setTotalPermits` each time there are cluster changes.
By doing so the semaphore has no knowledge related to this feature and it is
just a semaphore with the ability to change permits. It is easier to understand
and we can use it in other developments. And different handlers can be used as
needed.
##########
pinot-broker/src/main/java/org/apache/pinot/broker/requesthandler/MultiStageQuerySemaphore.java:
##########
@@ -0,0 +1,174 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pinot.broker.requesthandler;
+
+import com.google.common.base.Preconditions;
+import java.util.Collections;
+import java.util.concurrent.Semaphore;
+import java.util.concurrent.TimeUnit;
+import org.apache.helix.HelixAdmin;
+import org.apache.helix.HelixConstants;
+import org.apache.helix.HelixManager;
+import org.apache.helix.model.HelixConfigScope;
+import org.apache.helix.model.builder.HelixConfigScopeBuilder;
+import org.apache.pinot.broker.broker.helix.ClusterChangeHandler;
+import org.apache.pinot.spi.utils.CommonConstants;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/**
+ * This class helps limit the number of multi-stage queries being executed
concurrently. Currently, this limit is
+ * applied at the broker level, but could be moved to the server level in the
future.
+ */
+public class MultiStageQuerySemaphore extends Semaphore implements
ClusterChangeHandler {
+
+ private static final Logger LOGGER =
LoggerFactory.getLogger(MultiStageQuerySemaphore.class);
+
+ private HelixManager _helixManager;
+ private HelixAdmin _helixAdmin;
+ private HelixConfigScope _helixConfigScope;
+ private int _numBrokers;
+ private int _maxConcurrentQueries;
+ private int _totalPermits;
+
+ public MultiStageQuerySemaphore(int numBrokers, int maxConcurrentQueries) {
+ super(Math.max(1, maxConcurrentQueries / Math.max(numBrokers, 1)), true);
+ _maxConcurrentQueries = maxConcurrentQueries;
+ _numBrokers = Math.max(1, numBrokers);
+ _totalPermits =
+ maxConcurrentQueries > 0 ? Math.max(1, maxConcurrentQueries /
Math.max(numBrokers, 1)) : maxConcurrentQueries;
+ }
Review Comment:
I think we are using too many _max_es here. IMO it would be better to create
a static factory method that does all the max logic and then call a simple
constructor.
##########
pinot-broker/src/main/java/org/apache/pinot/broker/requesthandler/MultiStageQuerySemaphore.java:
##########
@@ -0,0 +1,174 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pinot.broker.requesthandler;
+
+import com.google.common.base.Preconditions;
+import java.util.Collections;
+import java.util.concurrent.Semaphore;
+import java.util.concurrent.TimeUnit;
+import org.apache.helix.HelixAdmin;
+import org.apache.helix.HelixConstants;
+import org.apache.helix.HelixManager;
+import org.apache.helix.model.HelixConfigScope;
+import org.apache.helix.model.builder.HelixConfigScopeBuilder;
+import org.apache.pinot.broker.broker.helix.ClusterChangeHandler;
+import org.apache.pinot.spi.utils.CommonConstants;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/**
+ * This class helps limit the number of multi-stage queries being executed
concurrently. Currently, this limit is
+ * applied at the broker level, but could be moved to the server level in the
future.
+ */
+public class MultiStageQuerySemaphore extends Semaphore implements
ClusterChangeHandler {
+
+ private static final Logger LOGGER =
LoggerFactory.getLogger(MultiStageQuerySemaphore.class);
+
+ private HelixManager _helixManager;
+ private HelixAdmin _helixAdmin;
+ private HelixConfigScope _helixConfigScope;
+ private int _numBrokers;
+ private int _maxConcurrentQueries;
+ private int _totalPermits;
+
+ public MultiStageQuerySemaphore(int numBrokers, int maxConcurrentQueries) {
+ super(Math.max(1, maxConcurrentQueries / Math.max(numBrokers, 1)), true);
+ _maxConcurrentQueries = maxConcurrentQueries;
+ _numBrokers = Math.max(1, numBrokers);
+ _totalPermits =
+ maxConcurrentQueries > 0 ? Math.max(1, maxConcurrentQueries /
Math.max(numBrokers, 1)) : maxConcurrentQueries;
+ }
+
+ @Override
+ public void acquire()
+ throws InterruptedException {
+ // If _totalPermits is <= 0, it means that the cluster is not configured
to limit the number of multi-stage queries
Review Comment:
It is fine to keep it here, but given this is an important attribute
invariant, I think it is worth to add it as javadoc in the attribute itself.
##########
pinot-broker/src/main/java/org/apache/pinot/broker/broker/helix/BaseBrokerStarter.java:
##########
@@ -335,13 +339,26 @@ public void start()
MultiStageBrokerRequestHandler multiStageBrokerRequestHandler = null;
QueryDispatcher queryDispatcher = null;
if (_brokerConf.getProperty(Helix.CONFIG_OF_MULTI_STAGE_ENGINE_ENABLED,
Helix.DEFAULT_MULTI_STAGE_ENGINE_ENABLED)) {
+ int numBrokers = Math.max(1, (int) _helixAdmin
+ .getInstancesInCluster(_spectatorHelixManager.getClusterName())
+ .stream()
+ .filter(instance ->
instance.startsWith(CommonConstants.Helix.PREFIX_OF_BROKER_INSTANCE))
+ .count());
+ int maxConcurrentQueries = Integer.parseInt(
+ _helixAdmin.getConfig(new
HelixConfigScopeBuilder(HelixConfigScope.ConfigScopeProperty.CLUSTER).forCluster(
+ _spectatorHelixManager.getClusterName()).build(),
Review Comment:
I don't think this code belongs here. It should be done automatically by
`MultiStageQuerySemaphore`
##########
pinot-broker/src/main/java/org/apache/pinot/broker/requesthandler/MultiStageQuerySemaphore.java:
##########
@@ -0,0 +1,174 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pinot.broker.requesthandler;
+
+import com.google.common.base.Preconditions;
+import java.util.Collections;
+import java.util.concurrent.Semaphore;
+import java.util.concurrent.TimeUnit;
+import org.apache.helix.HelixAdmin;
+import org.apache.helix.HelixConstants;
+import org.apache.helix.HelixManager;
+import org.apache.helix.model.HelixConfigScope;
+import org.apache.helix.model.builder.HelixConfigScopeBuilder;
+import org.apache.pinot.broker.broker.helix.ClusterChangeHandler;
+import org.apache.pinot.spi.utils.CommonConstants;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/**
+ * This class helps limit the number of multi-stage queries being executed
concurrently. Currently, this limit is
+ * applied at the broker level, but could be moved to the server level in the
future.
+ */
+public class MultiStageQuerySemaphore extends Semaphore implements
ClusterChangeHandler {
Review Comment:
Probably you can even generalize the handler to depend on some function
callbacks or abstract methods so you don't need to mock all the helix
attributes. I would prefer that (the less mocking the better) but I understand
it can also be seen as overengineering.
--
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
To unsubscribe, e-mail: [email protected]
For queries about this service, please contact Infrastructure at:
[email protected]
---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]