You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@drill.apache.org by "ASF GitHub Bot (JIRA)" <ji...@apache.org> on 2017/12/04 16:07:01 UTC

[jira] [Commented] (DRILL-5963) Canceling a query hung in planning state, leaves the query in ENQUEUED state for ever.

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

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

Github user arina-ielchiieva commented on a diff in the pull request:

    https://github.com/apache/drill/pull/1051#discussion_r153460494
  
    --- Diff: exec/java-exec/src/main/java/org/apache/drill/exec/work/foreman/FragmentsRunner.java ---
    @@ -0,0 +1,439 @@
    +/*
    + * 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.drill.exec.work.foreman;
    +
    +import com.google.common.base.Preconditions;
    +import com.google.common.collect.ArrayListMultimap;
    +import com.google.common.collect.Multimap;
    +import com.google.common.collect.Sets;
    +import io.netty.buffer.ByteBuf;
    +import org.apache.drill.common.concurrent.ExtendedLatch;
    +import org.apache.drill.common.exceptions.ExecutionSetupException;
    +import org.apache.drill.common.exceptions.UserException;
    +import org.apache.drill.exec.ops.FragmentContext;
    +import org.apache.drill.exec.physical.base.FragmentRoot;
    +import org.apache.drill.exec.proto.BitControl;
    +import org.apache.drill.exec.proto.BitControl.PlanFragment;
    +import org.apache.drill.exec.proto.CoordinationProtos;
    +import org.apache.drill.exec.proto.GeneralRPCProtos;
    +import org.apache.drill.exec.proto.UserBitShared.QueryId;
    +import org.apache.drill.exec.proto.UserBitShared.QueryResult.QueryState;
    +import org.apache.drill.exec.rpc.RpcException;
    +import org.apache.drill.exec.rpc.UserClientConnection;
    +import org.apache.drill.exec.rpc.control.Controller;
    +import org.apache.drill.exec.server.DrillbitContext;
    +import org.apache.drill.exec.testing.ControlsInjector;
    +import org.apache.drill.exec.testing.ControlsInjectorFactory;
    +import org.apache.drill.exec.work.EndpointListener;
    +import org.apache.drill.exec.work.WorkManager.WorkerBee;
    +import org.apache.drill.exec.work.fragment.FragmentExecutor;
    +import org.apache.drill.exec.work.fragment.FragmentStatusReporter;
    +import org.apache.drill.exec.work.fragment.NonRootFragmentManager;
    +import org.apache.drill.exec.work.fragment.RootFragmentManager;
    +
    +
    +import java.util.ArrayList;
    +import java.util.Collection;
    +import java.util.LinkedList;
    +import java.util.List;
    +import java.util.Set;
    +import java.util.concurrent.CountDownLatch;
    +
    +/**
    + * Is responsible for submitting query fragments for running (locally and remotely).
    + */
    +public class FragmentsRunner {
    +
    +  private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(FragmentsRunner.class);
    +  private static final ControlsInjector injector = ControlsInjectorFactory.getInjector(FragmentsRunner.class);
    +
    +  private static final long RPC_WAIT_IN_MSECS_PER_FRAGMENT = 5000;
    +
    +  private final WorkerBee bee;
    +  private final UserClientConnection initiatingClient;
    +  private final DrillbitContext drillbitContext;
    +  private final Foreman foreman;
    +
    +  private List<PlanFragment> planFragments;
    +  private PlanFragment rootPlanFragment;
    +  private FragmentRoot rootOperator;
    +
    +  public FragmentsRunner(WorkerBee bee, UserClientConnection initiatingClient, DrillbitContext drillbitContext, Foreman foreman) {
    +    this.bee = bee;
    +    this.initiatingClient = initiatingClient;
    +    this.drillbitContext = drillbitContext;
    +    this.foreman = foreman;
    +  }
    +
    +  public WorkerBee getBee() {
    +    return bee;
    +  }
    +
    +  public void setPlanFragments(List<PlanFragment> planFragments) {
    +    this.planFragments = planFragments;
    +  }
    +
    +  public void setRootPlanFragment(PlanFragment rootPlanFragment) {
    +    this.rootPlanFragment = rootPlanFragment;
    +  }
    +
    +  public void setRootOperator(FragmentRoot rootOperator) {
    +    this.rootOperator = rootOperator;
    +  }
    +
    +  /**
    +   * Submits root and non-root fragments fragments for running.
    +   * In case of success move query to the running state.
    +   */
    +  public void submit() {
    +    try {
    +      assert planFragments != null;
    +      assert rootPlanFragment != null;
    +      assert rootOperator != null;
    +
    +      QueryId queryId = foreman.getQueryId();
    +      assert queryId == rootPlanFragment.getHandle().getQueryId();
    +
    +      QueryManager queryManager = foreman.getQueryManager();
    +
    +      try {
    +        drillbitContext.getWorkBus().addFragmentStatusListener(queryId, queryManager.getFragmentStatusListener());
    +        drillbitContext.getClusterCoordinator().addDrillbitStatusListener(queryManager.getDrillbitStatusListener());
    +
    +        logger.debug("Submitting fragments to run.");
    +        // set up the root fragment first so we'll have incoming buffers available.
    +        setupRootFragment(rootPlanFragment, rootOperator);
    +        setupNonRootFragments(planFragments);
    +
    +      } catch (ExecutionSetupException e) {
    +        foreman.moveToState(QueryState.FAILED, e);
    --- End diff --
    
    Fixed. Moved into try block.


> 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)