[
https://issues.apache.org/jira/browse/DRILL-5963?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16277009#comment-16277009
]
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_r154119336
--- 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);
+ }
+
+ foreman.moveToState(QueryState.RUNNING, null);
+ logger.debug("Fragments running.");
+ } finally {
+ foreman.startProcessingEvents();
--- End diff --
Added explanation why we need to process events in any case, Logic is moved
into Foreman class.
> 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)