[ 
https://issues.apache.org/jira/browse/DRILL-4187?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15068845#comment-15068845
 ] 

ASF GitHub Bot commented on DRILL-4187:
---------------------------------------

Github user sudheeshkatkam commented on a diff in the pull request:

    https://github.com/apache/drill/pull/310#discussion_r48306314
  
    --- Diff: 
exec/java-exec/src/main/java/org/apache/drill/exec/work/foreman/Foreman.java ---
    @@ -458,49 +467,46 @@ private void setupSortMemoryAllocations(final 
PhysicalPlan plan) {
        */
       private void acquireQuerySemaphore(final PhysicalPlan plan) throws 
ForemanSetupException {
         final OptionManager optionManager = queryContext.getOptions();
    -    final boolean queuingEnabled = 
optionManager.getOption(ExecConstants.ENABLE_QUEUE);
    -    if (queuingEnabled) {
    -      final long queueThreshold = 
optionManager.getOption(ExecConstants.QUEUE_THRESHOLD_SIZE);
    -      double totalCost = 0;
    -      for (final PhysicalOperator ops : plan.getSortedOperators()) {
    -        totalCost += ops.getCost();
    -      }
    -
    -      final long queueTimeout = 
optionManager.getOption(ExecConstants.QUEUE_TIMEOUT);
    -      final String queueName;
    -
    -      try {
    -        @SuppressWarnings("resource")
    -        final ClusterCoordinator clusterCoordinator = 
drillbitContext.getClusterCoordinator();
    -        final DistributedSemaphore distributedSemaphore;
    -
    -        // get the appropriate semaphore
    -        if (totalCost > queueThreshold) {
    -          final int largeQueue = (int) 
optionManager.getOption(ExecConstants.LARGE_QUEUE_SIZE);
    -          distributedSemaphore = 
clusterCoordinator.getSemaphore("query.large", largeQueue);
    -          queueName = "large";
    -        } else {
    -          final int smallQueue = (int) 
optionManager.getOption(ExecConstants.SMALL_QUEUE_SIZE);
    -          distributedSemaphore = 
clusterCoordinator.getSemaphore("query.small", smallQueue);
    -          queueName = "small";
    -        }
    +    final long queueThreshold = 
optionManager.getOption(ExecConstants.QUEUE_THRESHOLD_SIZE);
    +    double totalCost = 0;
    +    for (final PhysicalOperator ops : plan.getSortedOperators()) {
    +      totalCost += ops.getCost();
    +    }
     
    +    final long queueTimeout = 
optionManager.getOption(ExecConstants.QUEUE_TIMEOUT);
    +    final String queueName;
     
    -        lease = distributedSemaphore.acquire(queueTimeout, 
TimeUnit.MILLISECONDS);
    -      } catch (final Exception e) {
    -        throw new ForemanSetupException("Unable to acquire slot for 
query.", e);
    +    try {
    +      @SuppressWarnings("resource")
    +      final ClusterCoordinator clusterCoordinator = 
drillbitContext.getClusterCoordinator();
    +      final DistributedSemaphore distributedSemaphore;
    +
    +      // get the appropriate semaphore
    +      if (totalCost > queueThreshold) {
    +        final int largeQueue = (int) 
optionManager.getOption(ExecConstants.LARGE_QUEUE_SIZE);
    +        distributedSemaphore = 
clusterCoordinator.getSemaphore("query.large", largeQueue);
    +        queueName = "large";
    +      } else {
    +        final int smallQueue = (int) 
optionManager.getOption(ExecConstants.SMALL_QUEUE_SIZE);
    +        distributedSemaphore = 
clusterCoordinator.getSemaphore("query.small", smallQueue);
    +        queueName = "small";
           }
     
    -      if (lease == null) {
    -        throw UserException
    -            .resourceError()
    -            .message(
    -                "Unable to acquire queue resources for query within 
timeout.  Timeout for %s queue was set at %d seconds.",
    -                queueName, queueTimeout / 1000)
    -            .build(logger);
    -      }
    +      recordNewState(QueryState.ENQUEUED);
    --- End diff --
    
    Since the current "state" is ENQUEUED, is this required?


> Introduce a state to separate queries pending execution from those pending in 
> the queue.
> ----------------------------------------------------------------------------------------
>
>                 Key: DRILL-4187
>                 URL: https://issues.apache.org/jira/browse/DRILL-4187
>             Project: Apache Drill
>          Issue Type: Sub-task
>            Reporter: Hanifi Gunes
>            Assignee: Mehant Baid
>             Fix For: 1.5.0
>
>
> Currently queries pending in the queue are not listed in the web UI besides 
> we use the state PENDING to mean pending executions. This issue proposes i) 
> to list enqueued queries in the web UI ii) to introduce a new state for 
> queries sitting at the queue, differentiating then from those pending 
> execution.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)

Reply via email to