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

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

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

    https://github.com/apache/drill/pull/858#discussion_r123327453
  
    --- Diff: 
exec/jdbc/src/main/java/org/apache/drill/jdbc/impl/DrillStatementImpl.java ---
    @@ -38,8 +44,12 @@
     // methods for compatibility.)
     class DrillStatementImpl extends AvaticaStatement implements 
DrillStatement,
                                                                  
DrillRemoteStatement {
    +  //Not using the DrillbitContext's ExecutorService as this is threadPool 
is light-weight (threads wake up to cancel tasks) but needs a low response time
    +  private static ExecutorService queryTimeoutTaskPool = 
Executors.newCachedThreadPool(new NamedThreadFactory("q-timeout-"));
    --- End diff --
    
    DrillCursor is handling all the logic of executing queries, and waiting for 
results. It has access to the connection and the statement, so you would know 
the timeout (if set). In the cursor, we are using a lock for the first message, 
and a blocking queue for the batches, but when waiting on those, there's no 
timeout set. Instead we could use query timeout (or the remaining time left 
since the beginning of the execution) and throws SqlTimeoutException when the 
locks throws TimeoutException themselves.
    
    In that scenario, no thread pool involved (except the one for I/O but it 
was already existing)


> Drill JDBC driver support Statement.setQueryTimeout(int)
> --------------------------------------------------------
>
>                 Key: DRILL-3640
>                 URL: https://issues.apache.org/jira/browse/DRILL-3640
>             Project: Apache Drill
>          Issue Type: New Feature
>          Components: Client - JDBC
>    Affects Versions: 1.2.0
>            Reporter: Chun Chang
>            Assignee: Kunal Khatua
>             Fix For: 1.11.0
>
>
> It would be nice if we have this implemented. Run away queries can be 
> automatically canceled by setting the timeout. 
> java.sql.SQLFeatureNotSupportedException: Setting network timeout is not 
> supported.
>       at 
> org.apache.drill.jdbc.impl.DrillStatementImpl.setQueryTimeout(DrillStatementImpl.java:152)



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

Reply via email to