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

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

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

    https://github.com/apache/drill/pull/1051#discussion_r153290872
  
    --- Diff: 
exec/java-exec/src/main/java/org/apache/drill/exec/work/foreman/Foreman.java ---
    @@ -426,48 +413,25 @@ private void runPhysicalPlan(final PhysicalPlan plan) 
throws ExecutionSetupExcep
         queryManager.setTotalCost(plan.totalCost());
         work.applyPlan(drillbitContext.getPlanReader());
         logWorkUnit(work);
    -    admit(work);
    -    queryManager.setQueueName(queryRM.queueName());
    -
    -    final List<PlanFragment> planFragments = work.getFragments();
    -    final PlanFragment rootPlanFragment = work.getRootFragment();
    -    assert queryId == rootPlanFragment.getHandle().getQueryId();
     
    -    drillbitContext.getWorkBus().addFragmentStatusListener(queryId, 
queryManager.getFragmentStatusListener());
    -    
drillbitContext.getClusterCoordinator().addDrillbitStatusListener(queryManager.getDrillbitStatusListener());
    +    fragmentsRunner.setPlanFragments(work.getFragments());
    +    fragmentsRunner.setRootPlanFragment(work.getRootFragment());
    +    fragmentsRunner.setRootOperator(work.getRootOperator());
     
    -    logger.debug("Submitting fragments to run.");
    -
    -    // set up the root fragment first so we'll have incoming buffers 
available.
    -    setupRootFragment(rootPlanFragment, work.getRootOperator());
    -
    -    setupNonRootFragments(planFragments);
    -
    -    moveToState(QueryState.RUNNING, null);
    -    logger.debug("Fragments running.");
    +    admit();
       }
     
    -  private void admit(QueryWorkUnit work) throws ForemanSetupException {
    +  private void admit() throws ForemanSetupException {
         queryManager.markPlanningEndTime();
    -    try {
    -      queryRM.admit();
    -    } catch (QueueTimeoutException e) {
    -      throw UserException
    -          .resourceError()
    -          .message(e.getMessage())
    -          .build(logger);
    -    } catch (QueryQueueException e) {
    -      throw new ForemanSetupException(e.getMessage(), e);
    -    } finally {
    -      queryManager.markQueueWaitEndTime();
    -    }
    -    moveToState(QueryState.STARTING, null);
    +    planningQueries.dec();
    --- End diff --
    
    Better to handle increment and decrement of counters in moveToState. 


> Canceling a query hung in planning state, leaves the query in ENQUEUED state 
> for ever.
> --------------------------------------------------------------------------------------
>
>                 Key: DRILL-5963
>                 URL: https://issues.apache.org/jira/browse/DRILL-5963
>             Project: Apache Drill
>          Issue Type: Bug
>          Components: Execution - Flow
>    Affects Versions: 1.12.0
>         Environment: Drill 1.12.0-SNAPSHOT, commit: 
> 4a718a0bd728ae02b502ac93620d132f0f6e1b6c
>            Reporter: Khurram Faraaz
>            Assignee: Arina Ielchiieva
>            Priority: Critical
>             Fix For: 1.13.0
>
>         Attachments: enqueued-2.png
>
>
> Canceling the below query that is hung in planning state, leaves the query in 
> ENQUEUED state for ever.
> Here is the query that is hung in planning state
> {noformat}
> 0: jdbc:drill:schema=dfs.tmp> select 1 || ',' || 2 || ',' || 3 || ',' || 4 || 
> ',' || 5 || ',' || 6 || ',' || 7 || ',' || 8 || ',' || 9 || ',' || 0 || ',' 
> AS CSV_DATA from (values(1));
> +--+
> |  |
> +--+
> +--+
> No rows selected (304.291 seconds)
> {noformat}
> Explain plan for that query also just hangs.
> {noformat}
> explain plan for select 1 || ',' || 2 || ',' || 3 || ',' || 4 || ',' || 5 || 
> ',' || 6 || ',' || 7 || ',' || 8 || ',' || 9 || ',' || 0 || ',' AS CSV_DATA 
> from (values(1));
> ...
> {noformat}
> The above issues show the following problems:
> *1. Simple query with reasonable number of concat functions hangs.*
> In reality query does not hang it just take lots of time to execute. The root 
> cause is that during planning time DrillFuncHolderExpr return type is 
> extensively used to determine matching function, matching type etc. Though 
> this type is retrieved via 
> [getter|https://github.com/apache/drill/blob/master/exec/java-exec/src/main/java/org/apache/drill/exec/expr/DrillFuncHolderExpr.java#L41]
>  in reality complex logic is executed beaneath it. For example for [concat 
> function|https://github.com/apache/drill/blob/master/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/output/ConcatReturnTypeInference.java#L47].
>  Since function return type can not be changes during DrillFuncHolderExpr 
> life time, it is safe to cache it.
> *2. No mechanism to cancel query during ENQUEUED state.*
> Currently Drill does not have mechanism to cancel query before STARTING / 
> RUNNING states. Plus ENQUEUED state includes two PLANNING and ENQUEUED.
> Also submitting mechanism for submitting query to the queue is blocking, 
> making foreman wait till enqueueing is done Making it non-blocking will 
> prevent consuming threads that just sit idle in a busy system and also is 
> important when we move to a real admission control solution.
> The following changes were made to address above issues:
> a. two new states were added: PREPARING (when foreman is initialized) and 
> PLANNING (includes logical and / or physical planning).
> b. process of query enqueuing was made non-blocking. Once query was enqueued, 
> fragments runner is called to submit fragments locally and remotely.
> c. ability to cancel query during planning and enqueued states was added.



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)

Reply via email to