You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@drill.apache.org by kkhatua <gi...@git.apache.org> on 2017/06/21 01:50:30 UTC

[GitHub] drill pull request #858: DRILL-3640: Support JDBC Statement.setQueryTimeout(...

GitHub user kkhatua opened a pull request:

    https://github.com/apache/drill/pull/858

    DRILL-3640: Support JDBC Statement.setQueryTimeout(int)

    Allow for queries to be cancelled if they don't complete within the stipulated time.
    Tests added to test different query timeout scenarios.
    _Note:_
    We submit a timeout-managing task that sleeps for the stipulated period before trying to cancel the query (JDBC Statement).
    It might be worth having a similar feature as a System/Session variable so that the same can be achieved via SQLLine or REST APIs, but is beyond the scope of this JIRA's PR.

You can merge this pull request into a Git repository by running:

    $ git pull https://github.com/kkhatua/drill DRILL-3640

Alternatively you can review and apply these changes as the patch at:

    https://github.com/apache/drill/pull/858.patch

To close this pull request, make a commit to your master/trunk branch
with (at least) the following in the commit message:

    This closes #858
    
----
commit e7a1d36891342e6c418f3dc85253cf39eceb928d
Author: Kunal Khatua <kk...@maprtech.com>
Date:   2017-06-21T01:45:08Z

    DRILL-3640: Support JDBC Statement.setQueryTimeout(int)
    
    Allow for queries to be cancelled if they don't complete within the stipulated time.
    Tests added to test different query timeout scenarios.

----


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #858: DRILL-3640: Support JDBC Statement.setQueryTimeout(...

Posted by kkhatua <gi...@git.apache.org>.
Github user kkhatua commented on a diff in the pull request:

    https://github.com/apache/drill/pull/858#discussion_r123872136
  
    --- Diff: exec/jdbc/src/main/java/org/apache/drill/jdbc/impl/TimeoutTrigger.java ---
    @@ -0,0 +1,105 @@
    +/**
    + * 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.jdbc.impl;
    +
    +import java.util.concurrent.Callable;
    +import java.util.concurrent.ExecutorService;
    +import java.util.concurrent.Executors;
    +import java.util.concurrent.Future;
    +import java.util.concurrent.TimeUnit;
    +
    +import org.apache.calcite.avatica.AvaticaStatement;
    +import org.apache.drill.exec.rpc.NamedThreadFactory;
    +import org.apache.drill.jdbc.SqlTimeoutException;
    +
    +/**
    + * Timeout Trigger required for canceling of running queries
    + */
    +class TimeoutTrigger implements Callable<Boolean> {
    +  //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
    --- End diff --
    
    Good point.. but they are primarily sleeping, and compared to the number of fragment threads, this extra thread should not cause much heartache on the system.  


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #858: DRILL-3640: Support JDBC Statement.setQueryTimeout(...

Posted by kkhatua <gi...@git.apache.org>.
Github user kkhatua commented on a diff in the pull request:

    https://github.com/apache/drill/pull/858#discussion_r123865562
  
    --- Diff: exec/jdbc/src/main/java/org/apache/drill/jdbc/impl/DrillStatementImpl.java ---
    @@ -204,7 +276,7 @@ public boolean isClosed() {
     
       @Override
       public int getMaxFieldSize() throws SQLException {
    -    throwIfClosed();
    +    throwIfTimedOutOrClosed();
    --- End diff --
    
    TimeoutException is now thrown only for execute and data fetch operations.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill issue #858: DRILL-3640: Support JDBC Statement.setQueryTimeout(int)

Posted by kkhatua <gi...@git.apache.org>.
Github user kkhatua commented on the issue:

    https://github.com/apache/drill/pull/858
  
    Wouldn't a query cancellation automatically interrupt those potentially blocking operations? I'm simply looking up whether the trigger was a timeout DrillStatement.isTimedOut() to decide if the returning exception is a SqlTimeoutExcp.
    
    The extra thread (though mostly sleeping) didn't seem like a huge cost considering that the timeout is at the granularity of seconds.
    
    I ran a small custom JDBC client application to test this theory. Running longevity and concurrency testing worked well when testing for timeouts. For timeouts as low as 1 second for queries that needed to do lot of back end processing before returning rows much later (like a massive join), I noticed that we caught underlying exceptions as the timeouts during the 'executeQuery' call itself. If I were to not go the route of issuing a Statement.cancel() but relying primarily on the DrillCursor, unless I put in some mechanism of constantly polling for the out-of-time state of the clock. I think then I also need to cancel then cancel all operations from the level of the DrillCursor in both directions (up to the Statement and down to the fragments). All this versus issuing a Statement.cancel() that effectively cancels all the operations down to the fragment level. 
    
    I am now wondering whether I missied a corner case which will be addressed by doing the timeout in the DrillCursor?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #858: DRILL-3640: Support JDBC Statement.setQueryTimeout(...

Posted by kkhatua <gi...@git.apache.org>.
Github user kkhatua commented on a diff in the pull request:

    https://github.com/apache/drill/pull/858#discussion_r123327737
  
    --- Diff: exec/jdbc/src/main/java/org/apache/drill/jdbc/impl/DrillStatementImpl.java ---
    @@ -159,24 +230,25 @@ public void cleanUp() {
       public int getQueryTimeout() throws AlreadyClosedSqlException
       {
         throwIfClosed();
    -    return 0;  // (No no timeout.)
    --- End diff --
    
    Interestingly, AvaticaStatement returns the timeout value that was set... but does not honour it! :) 
    Originally the setter would trigger a NotSupported exception and the explicit return was the default 0
    Now that we're able to support it, I can read Avatica's value directly. 



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #858: DRILL-3640: Support JDBC Statement.setQueryTimeout(...

Posted by laurentgo <gi...@git.apache.org>.
Github user laurentgo commented on a diff in the pull request:

    https://github.com/apache/drill/pull/858#discussion_r123866367
  
    --- Diff: exec/jdbc/src/main/java/org/apache/drill/jdbc/impl/TimeoutTrigger.java ---
    @@ -0,0 +1,105 @@
    +/**
    + * 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.jdbc.impl;
    +
    +import java.util.concurrent.Callable;
    +import java.util.concurrent.ExecutorService;
    +import java.util.concurrent.Executors;
    +import java.util.concurrent.Future;
    +import java.util.concurrent.TimeUnit;
    +
    +import org.apache.calcite.avatica.AvaticaStatement;
    +import org.apache.drill.exec.rpc.NamedThreadFactory;
    +import org.apache.drill.jdbc.SqlTimeoutException;
    +
    +/**
    + * Timeout Trigger required for canceling of running queries
    + */
    +class TimeoutTrigger implements Callable<Boolean> {
    +  //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 timeoutService;
    --- End diff --
    
    this executor is never shut down. Maybe it should be at the connection, or the driver level and properly closed?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #858: DRILL-3640: Support JDBC Statement.setQueryTimeout(...

Posted by laurentgo <gi...@git.apache.org>.
Github user laurentgo commented on a diff in the pull request:

    https://github.com/apache/drill/pull/858#discussion_r149228776
  
    --- Diff: exec/jdbc/src/main/java/org/apache/drill/jdbc/DrillStatement.java ---
    @@ -34,11 +35,11 @@
        *            if connection is closed
        */
       @Override
    -  int getQueryTimeout() throws AlreadyClosedSqlException;
    +  int getQueryTimeout() throws AlreadyClosedSqlException, SQLException;
     
       /**
        * <strong>Drill</strong>:
    -   * Not supported (for non-zero timeout value).
    +   * Supported (for non-zero timeout value).
        * <p>
        *   Normally, just throws {@link SQLFeatureNotSupportedException} unless
    --- End diff --
    
    need to update javadoc description


---

[GitHub] drill pull request #858: DRILL-3640: Support JDBC Statement.setQueryTimeout(...

Posted by laurentgo <gi...@git.apache.org>.
Github user laurentgo commented on a diff in the pull request:

    https://github.com/apache/drill/pull/858#discussion_r123295947
  
    --- Diff: exec/jdbc/src/main/java/org/apache/drill/jdbc/impl/DrillResultSetImpl.java ---
    @@ -125,7 +154,7 @@ protected void cancel() {
       // (Not delegated.)
       @Override
       public boolean next() throws SQLException {
    -    throwIfClosed();
    +    throwIfTimedOutOrClosed();
    --- End diff --
    
    if we chose to support `queryTimeout` for `ResultSet`, shouldn't we interrupt `next()` too if the operation is taking too long? As per your code, it seems the exception would be thrown after facts...


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #858: DRILL-3640: Support JDBC Statement.setQueryTimeout(...

Posted by kkhatua <gi...@git.apache.org>.
Github user kkhatua commented on a diff in the pull request:

    https://github.com/apache/drill/pull/858#discussion_r123913767
  
    --- Diff: exec/jdbc/src/main/java/org/apache/drill/jdbc/impl/DrillResultSetImpl.java ---
    @@ -98,6 +102,32 @@ private void throwIfClosed() throws AlreadyClosedSqlException,
         }
       }
     
    +  /**
    +   * Throws SqlTimeoutException if the statement has already timed out
    +   * @throws AlreadyClosedSqlException   if Connection is closed
    +   * @throws SqlTimeoutException         if Statement is already timed out
    +   */
    +  private void throwIfTimedOut() throws AlreadyClosedSqlException,
    +                                        SqlTimeoutException,
    +                                        SQLException {
    +    if ( (statement instanceof DrillStatementImpl && ((DrillStatementImpl) statement).isTimedOut()) ||
    --- End diff --
    
    Good idea.  The original DrillStatement and DrillPreparedStatement are derived from the JDBC interfaces directly. I'll add the isTimedOut to DrillStatement and tie up the loose ends.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #858: DRILL-3640: Support JDBC Statement.setQueryTimeout(...

Posted by laurentgo <gi...@git.apache.org>.
Github user laurentgo commented on a diff in the pull request:

    https://github.com/apache/drill/pull/858#discussion_r123866148
  
    --- Diff: exec/jdbc/src/main/java/org/apache/drill/jdbc/impl/DrillResultSetImpl.java ---
    @@ -98,6 +102,32 @@ private void throwIfClosed() throws AlreadyClosedSqlException,
         }
       }
     
    +  /**
    +   * Throws SqlTimeoutException if the statement has already timed out
    +   * @throws AlreadyClosedSqlException   if Connection is closed
    +   * @throws SqlTimeoutException         if Statement is already timed out
    +   */
    +  private void throwIfTimedOut() throws AlreadyClosedSqlException,
    +                                        SqlTimeoutException,
    +                                        SQLException {
    +    if ( (statement instanceof DrillStatementImpl && ((DrillStatementImpl) statement).isTimedOut()) ||
    --- End diff --
    
    since PreparedStatement extends from Statement, maybe DrillPreparedStatement should extends DrillStatement? this way, you could add isTimedOut to the interface? (but you would not need to if done in DrillCursor ;-))


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill issue #858: DRILL-3640: Support JDBC Statement.setQueryTimeout(int)

Posted by kkhatua <gi...@git.apache.org>.
Github user kkhatua commented on the issue:

    https://github.com/apache/drill/pull/858
  
    Closing this in favor of #1024 


---

[GitHub] drill pull request #858: DRILL-3640: Support JDBC Statement.setQueryTimeout(...

Posted by kkhatua <gi...@git.apache.org>.
Github user kkhatua commented on a diff in the pull request:

    https://github.com/apache/drill/pull/858#discussion_r123329155
  
    --- Diff: exec/jdbc/src/main/java/org/apache/drill/jdbc/impl/DrillStatementImpl.java ---
    @@ -497,14 +594,64 @@ public boolean isPoolable() throws SQLException {
     
       @Override
       public void closeOnCompletion() throws SQLException {
    -    throwIfClosed();
    +    throwIfTimedOutOrClosed();
         super.closeOnCompletion();
       }
     
       @Override
       public boolean isCloseOnCompletion() throws SQLException {
    -    throwIfClosed();
    +    throwIfTimedOutOrClosed();
         return super.isCloseOnCompletion();
       }
     
     }
    +
    +/**
    + * Timeout Trigger required for canceling of running queries
    + */
    +class TimeoutTrigger implements Callable<Boolean> {
    +  private int timeoutInSeconds;
    +
    +  /**
    +   * Get Timeout period in seconds
    +   */
    +  public int getTimeoutInSeconds() {
    +    return timeoutInSeconds;
    +  }
    +
    +  private DrillStatementImpl statementHandle;
    +
    +  //Default Constructor is Invalid
    +  @SuppressWarnings("unused")
    +  private TimeoutTrigger() {}
    +
    +  /**
    +   * Timeout Constructor
    +   * @param stmtContext   Statement Handle
    +   * @param timeoutInSec  Timeout defined in seconds
    +   */
    +  TimeoutTrigger(DrillStatementImpl stmtContext, int timeoutInSec) {
    +    timeoutInSeconds = timeoutInSec;
    +    statementHandle = stmtContext;
    +  }
    +
    +  @Override
    +  public Boolean call() throws Exception {
    +    try {
    +      Thread.sleep(timeoutInSeconds*1000L);
    --- End diff --
    
    +1 Will make the change.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #858: DRILL-3640: Support JDBC Statement.setQueryTimeout(...

Posted by kkhatua <gi...@git.apache.org>.
Github user kkhatua commented on a diff in the pull request:

    https://github.com/apache/drill/pull/858#discussion_r123865572
  
    --- Diff: exec/jdbc/src/main/java/org/apache/drill/jdbc/impl/DrillResultSetImpl.java ---
    @@ -64,13 +65,17 @@
           org.slf4j.LoggerFactory.getLogger(DrillResultSetImpl.class);
     
       private final DrillConnectionImpl connection;
    +  private DrillStatementImpl drillStatement = null;
       private volatile boolean hasPendingCancelationNotification = false;
     
       DrillResultSetImpl(AvaticaStatement statement, Meta.Signature signature,
                          ResultSetMetaData resultSetMetaData, TimeZone timeZone,
                          Meta.Frame firstFrame) {
         super(statement, signature, resultSetMetaData, timeZone, firstFrame);
         connection = (DrillConnectionImpl) statement.getConnection();
    +    if (statement instanceof DrillStatementImpl) {
    --- End diff --
    
    Support for PreparedStatement added, including JUnit tests


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #858: DRILL-3640: Support JDBC Statement.setQueryTimeout(...

Posted by kkhatua <gi...@git.apache.org>.
Github user kkhatua commented on a diff in the pull request:

    https://github.com/apache/drill/pull/858#discussion_r123329050
  
    --- Diff: exec/jdbc/src/main/java/org/apache/drill/jdbc/impl/DrillStatementImpl.java ---
    @@ -204,7 +276,7 @@ public boolean isClosed() {
     
       @Override
       public int getMaxFieldSize() throws SQLException {
    -    throwIfClosed();
    +    throwIfTimedOutOrClosed();
    --- End diff --
    
    The throwIfTimedOutOrClosed() call is basically a wrapper around a sequential check for, first the timed-out state, followed by the check for the closed state. A timed-out query (i.e. statement/resultset) is already in a closed state, but we need to throw the correct exception (in this case, the timeout), which is why it was done like that. 
    My understanding was that any execute and data fetch operations can throw timeoutExceptions. Are you suggesting that for such '_getter_' methods, only an AlreadyClosed exception needs to be thrown and not time out?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #858: DRILL-3640: Support JDBC Statement.setQueryTimeout(...

Posted by laurentgo <gi...@git.apache.org>.
Github user laurentgo commented on a diff in the pull request:

    https://github.com/apache/drill/pull/858#discussion_r123294241
  
    --- 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 --
    
    I believe this is unnecessary: DrillClient provides an asynchronous API, which is used by the JDBC driver, so all the timeout logic could be done without the use of thread pool. You might want to look at DrillCursor which is where all the magic happens I believe.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #858: DRILL-3640: Support JDBC Statement.setQueryTimeout(...

Posted by laurentgo <gi...@git.apache.org>.
Github user laurentgo commented on a diff in the pull request:

    https://github.com/apache/drill/pull/858#discussion_r123329682
  
    --- Diff: exec/jdbc/src/main/java/org/apache/drill/jdbc/impl/DrillStatementImpl.java ---
    @@ -204,7 +276,7 @@ public boolean isClosed() {
     
       @Override
       public int getMaxFieldSize() throws SQLException {
    -    throwIfClosed();
    +    throwIfTimedOutOrClosed();
    --- End diff --
    
    yes, execute and (optionally) data fetch operations. Other methods are not impacted.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #858: DRILL-3640: Support JDBC Statement.setQueryTimeout(...

Posted by laurentgo <gi...@git.apache.org>.
Github user laurentgo commented on a diff in the pull request:

    https://github.com/apache/drill/pull/858#discussion_r149229998
  
    --- Diff: exec/jdbc/src/main/java/org/apache/drill/jdbc/DrillStatement.java ---
    @@ -54,7 +55,8 @@
       void setQueryTimeout( int milliseconds )
    --- End diff --
    
    java.sql.Statement says this is seconds...


---

[GitHub] drill pull request #858: DRILL-3640: Support JDBC Statement.setQueryTimeout(...

Posted by laurentgo <gi...@git.apache.org>.
Github user laurentgo commented on a diff in the pull request:

    https://github.com/apache/drill/pull/858#discussion_r123295256
  
    --- Diff: exec/jdbc/src/main/java/org/apache/drill/jdbc/impl/DrillResultSetImpl.java ---
    @@ -64,13 +65,17 @@
           org.slf4j.LoggerFactory.getLogger(DrillResultSetImpl.class);
     
       private final DrillConnectionImpl connection;
    +  private DrillStatementImpl drillStatement = null;
       private volatile boolean hasPendingCancelationNotification = false;
     
       DrillResultSetImpl(AvaticaStatement statement, Meta.Signature signature,
                          ResultSetMetaData resultSetMetaData, TimeZone timeZone,
                          Meta.Frame firstFrame) {
         super(statement, signature, resultSetMetaData, timeZone, firstFrame);
         connection = (DrillConnectionImpl) statement.getConnection();
    +    if (statement instanceof DrillStatementImpl) {
    --- End diff --
    
    what about prepared statement?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #858: DRILL-3640: Support JDBC Statement.setQueryTimeout(...

Posted by laurentgo <gi...@git.apache.org>.
Github user laurentgo commented on a diff in the pull request:

    https://github.com/apache/drill/pull/858#discussion_r123330075
  
    --- Diff: exec/jdbc/src/main/java/org/apache/drill/jdbc/SqlTimeoutException.java ---
    @@ -23,12 +23,17 @@
      * Indicates that an operation timed out. This is not an error; you can
      * retry the operation.
      */
    -public class SqlTimeoutException
    -    extends SQLException
    -{
    +public class SqlTimeoutException extends SQLException {
    +
    +  private static final long serialVersionUID = 2017_06_20L;
    --- End diff --
    
    missed this one. Honestly not super important (feel free to ignore my initial comment)


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #858: DRILL-3640: Support JDBC Statement.setQueryTimeout(...

Posted by laurentgo <gi...@git.apache.org>.
Github user laurentgo commented on a diff in the pull request:

    https://github.com/apache/drill/pull/858#discussion_r123866213
  
    --- Diff: exec/jdbc/src/main/java/org/apache/drill/jdbc/impl/TimeoutTrigger.java ---
    @@ -0,0 +1,105 @@
    +/**
    + * 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.jdbc.impl;
    +
    +import java.util.concurrent.Callable;
    +import java.util.concurrent.ExecutorService;
    +import java.util.concurrent.Executors;
    +import java.util.concurrent.Future;
    +import java.util.concurrent.TimeUnit;
    +
    +import org.apache.calcite.avatica.AvaticaStatement;
    +import org.apache.drill.exec.rpc.NamedThreadFactory;
    +import org.apache.drill.jdbc.SqlTimeoutException;
    +
    +/**
    + * Timeout Trigger required for canceling of running queries
    + */
    +class TimeoutTrigger implements Callable<Boolean> {
    +  //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
    --- End diff --
    
    this is still one extra thread per statement overhead. Threads are reused, but if lots of queries are run concurrently, that may be a lot.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #858: DRILL-3640: Support JDBC Statement.setQueryTimeout(...

Posted by kkhatua <gi...@git.apache.org>.
Github user kkhatua commented on a diff in the pull request:

    https://github.com/apache/drill/pull/858#discussion_r123865524
  
    --- Diff: exec/jdbc/src/main/java/org/apache/drill/jdbc/impl/DrillStatementImpl.java ---
    @@ -159,24 +230,25 @@ public void cleanUp() {
       public int getQueryTimeout() throws AlreadyClosedSqlException
       {
         throwIfClosed();
    -    return 0;  // (No no timeout.)
    --- End diff --
    
    Using Avatica's stored value and returning that now..


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill issue #858: DRILL-3640: Support JDBC Statement.setQueryTimeout(int)

Posted by kkhatua <gi...@git.apache.org>.
Github user kkhatua commented on the issue:

    https://github.com/apache/drill/pull/858
  
    @laurentgo  Updated based on the review comments. The Timeout Executor Service is now maintained for the lifetime of a connection and closed during shutdown. Also refactored by introducing `isTimedOut()` and `cancelDueToTimeout()` .



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #858: DRILL-3640: Support JDBC Statement.setQueryTimeout(...

Posted by kkhatua <gi...@git.apache.org>.
Github user kkhatua commented on a diff in the pull request:

    https://github.com/apache/drill/pull/858#discussion_r123325052
  
    --- Diff: exec/jdbc/src/main/java/org/apache/drill/jdbc/SqlTimeoutException.java ---
    @@ -23,12 +23,17 @@
      * Indicates that an operation timed out. This is not an error; you can
      * retry the operation.
      */
    -public class SqlTimeoutException
    -    extends SQLException
    -{
    +public class SqlTimeoutException extends SQLException {
    +
    +  private static final long serialVersionUID = 2017_06_20L;
    --- End diff --
    
    Fair enough... will set it to a -1. 
    I saw a similar assignment for _InvalidParameterSqlException_ , which was set as a timestamp, so I just tried to follow the convention.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #858: DRILL-3640: Support JDBC Statement.setQueryTimeout(...

Posted by kkhatua <gi...@git.apache.org>.
Github user kkhatua commented on a diff in the pull request:

    https://github.com/apache/drill/pull/858#discussion_r123331635
  
    --- Diff: exec/jdbc/src/main/java/org/apache/drill/jdbc/impl/DrillResultSetImpl.java ---
    @@ -125,7 +154,7 @@ protected void cancel() {
       // (Not delegated.)
       @Override
       public boolean next() throws SQLException {
    -    throwIfClosed();
    +    throwIfTimedOutOrClosed();
    --- End diff --
    
    The query cancellation should take care of it. It'll be hard to have a unit test specifically for this, but I'll try.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #858: DRILL-3640: Support JDBC Statement.setQueryTimeout(...

Posted by laurentgo <gi...@git.apache.org>.
Github user laurentgo commented on a diff in the pull request:

    https://github.com/apache/drill/pull/858#discussion_r123293197
  
    --- Diff: exec/jdbc/src/main/java/org/apache/drill/jdbc/impl/DrillStatementImpl.java ---
    @@ -159,24 +230,25 @@ public void cleanUp() {
       public int getQueryTimeout() throws AlreadyClosedSqlException
       {
         throwIfClosed();
    -    return 0;  // (No no timeout.)
    --- End diff --
    
    might want to check what was Avatica's behavior before we overriden it...


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #858: DRILL-3640: Support JDBC Statement.setQueryTimeout(...

Posted by laurentgo <gi...@git.apache.org>.
Github user laurentgo commented on a diff in the pull request:

    https://github.com/apache/drill/pull/858#discussion_r123303854
  
    --- Diff: exec/jdbc/src/main/java/org/apache/drill/jdbc/impl/DrillStatementImpl.java ---
    @@ -422,6 +507,9 @@ public ResultSet getGeneratedKeys() throws SQLException {
       public int executeUpdate(String sql, int autoGeneratedKeys) throws SQLException {
         throwIfClosed();
         try {
    +      if (timeoutTrigger != null) {
    --- End diff --
    
    so the trigger is created, but if super.executeUpdate takes more than queryTimeout seconds, the method is not interrupted. I don't believe this is conform to JDBC spec, or somehow useful to the end-user (if query takes 2min but timeout is 60s, exception should be thrown after 60s, not 120s...)


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #858: DRILL-3640: Support JDBC Statement.setQueryTimeout(...

Posted by laurentgo <gi...@git.apache.org>.
Github user laurentgo commented on a diff in the pull request:

    https://github.com/apache/drill/pull/858#discussion_r123296355
  
    --- Diff: exec/jdbc/src/main/java/org/apache/drill/jdbc/impl/DrillResultSetImpl.java ---
    @@ -1384,7 +1402,7 @@ public void updateRowId( String columnLabel, RowId x ) throws SQLException {
     
       @Override
       public int getHoldability() throws SQLException {
    -    throwIfClosed();
    +    throwIfTimedOutOrClosed();
    --- End diff --
    
    not sure if it makes sense (and correct per spec) to throw SqlTimeoutException here... (and similar methods)


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill issue #858: DRILL-3640: Support JDBC Statement.setQueryTimeout(int)

Posted by laurentgo <gi...@git.apache.org>.
Github user laurentgo commented on the issue:

    https://github.com/apache/drill/pull/858
  
    Thanks for the changes. That said, I still believe that it should be done using DrillCursor and the async framework already in place in Drill, and so I'm not comfortable giving my approval. Maybe @parthchandra can review it too?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #858: DRILL-3640: Support JDBC Statement.setQueryTimeout(...

Posted by kkhatua <gi...@git.apache.org>.
Github user kkhatua commented on a diff in the pull request:

    https://github.com/apache/drill/pull/858#discussion_r123325595
  
    --- 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 --
    
    I'm not sure if there is a way for me to reference back to the Statement object... since the objective is to simply have a sleeping thread in this pool timeout and **cancel** the query. Let me see look around.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill issue #858: DRILL-3640: Support JDBC Statement.setQueryTimeout(int)

Posted by kkhatua <gi...@git.apache.org>.
Github user kkhatua commented on the issue:

    https://github.com/apache/drill/pull/858
  
    @laurentgo I'm done with the refactoring. Please review. 
    @parthchandra , since this will be the basis of exploring the cause for DRILL-5420 , perhaps you could do a review as well. 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #858: DRILL-3640: Support JDBC Statement.setQueryTimeout(...

Posted by laurentgo <gi...@git.apache.org>.
Github user laurentgo commented on a diff in the pull request:

    https://github.com/apache/drill/pull/858#discussion_r149274508
  
    --- Diff: exec/jdbc/src/main/java/org/apache/drill/jdbc/impl/DrillPreparedStatementImpl.java ---
    @@ -552,16 +701,22 @@ public void clearParameters() throws SQLException {
       public boolean execute() throws SQLException {
         throwIfClosed();
         try {
    +      if (timeoutTrigger != null) {
    +        timeoutTriggerHandle = timeoutTrigger.startCountdown();
    +      }
           return super.execute();
         }
         catch (UnsupportedOperationException e) {
    +      if ( isTimedOut() ) {
    --- End diff --
    
    i'm not sure it makes sense to check it query timed out if operation is unsupported


---

[GitHub] drill pull request #858: DRILL-3640: Support JDBC Statement.setQueryTimeout(...

Posted by kkhatua <gi...@git.apache.org>.
Github user kkhatua commented on a diff in the pull request:

    https://github.com/apache/drill/pull/858#discussion_r123865295
  
    --- Diff: exec/jdbc/src/main/java/org/apache/drill/jdbc/impl/DrillResultSetImpl.java ---
    @@ -1384,7 +1402,7 @@ public void updateRowId( String columnLabel, RowId x ) throws SQLException {
     
       @Override
       public int getHoldability() throws SQLException {
    -    throwIfClosed();
    +    throwIfTimedOutOrClosed();
    --- End diff --
    
    Removed the timeout check for most getter methods, exceptions being the getResultSet() related APIs.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #858: DRILL-3640: Support JDBC Statement.setQueryTimeout(...

Posted by laurentgo <gi...@git.apache.org>.
Github user laurentgo commented on a diff in the pull request:

    https://github.com/apache/drill/pull/858#discussion_r123329889
  
    --- Diff: exec/jdbc/src/main/java/org/apache/drill/jdbc/impl/DrillStatementImpl.java ---
    @@ -159,24 +230,25 @@ public void cleanUp() {
       public int getQueryTimeout() throws AlreadyClosedSqlException
       {
         throwIfClosed();
    -    return 0;  // (No no timeout.)
    --- End diff --
    
    it's an optional thing, but the spec doesn't say what happens if you set it but it's actually not used...


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #858: DRILL-3640: Support JDBC Statement.setQueryTimeout(...

Posted by kkhatua <gi...@git.apache.org>.
Github user kkhatua commented on a diff in the pull request:

    https://github.com/apache/drill/pull/858#discussion_r123330684
  
    --- Diff: exec/jdbc/src/main/java/org/apache/drill/jdbc/impl/DrillStatementImpl.java ---
    @@ -422,6 +507,9 @@ public ResultSet getGeneratedKeys() throws SQLException {
       public int executeUpdate(String sql, int autoGeneratedKeys) throws SQLException {
         throwIfClosed();
         try {
    +      if (timeoutTrigger != null) {
    --- End diff --
    
    I'm submit the timeout trigger to the pool and counting on that trigger doing a query cancellation to do that. I don't think Drill supports executeUpdate, but as long as a query cancellation for updates does the rollback of the transaction, this should suffice. This worked well for large queries where the execute###() call was longer than the timeout period and allowed for the cancellation to do the interrupt.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #858: DRILL-3640: Support JDBC Statement.setQueryTimeout(...

Posted by laurentgo <gi...@git.apache.org>.
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)


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill issue #858: DRILL-3640: Support JDBC Statement.setQueryTimeout(int)

Posted by kkhatua <gi...@git.apache.org>.
Github user kkhatua commented on the issue:

    https://github.com/apache/drill/pull/858
  
    @laurentgo Within the DrillCursor, the only place I could do such a check was 
    https://github.com/kkhatua/drill/blob/c51473859d1dd81cf70e857f729c3a8491b2834a/exec/jdbc/src/main/java/org/apache/drill/jdbc/impl/DrillCursor.java#L582
    (Hosting a fork of this PR in this repo: https://github.com/kkhatua/drill/commits/altDrill3640 )
    
    My JDBC client for performance testing has a similar mechanism for JDBC drivers that don't support timeout, but allow for query cancellation by using a cancelling-trigger thread to sleep until the timeout, before waking up explicitly cancelling the query. I've simply replicated behaviour that in the Drill JDBC package. Having a constant check on the time remaining using a system call like {{System.currentTimeMillis()}} is actually expensive, which is why I didn't want to have the DrillCursor contantly do that check before throwing an exception. Can you point to me on which async framework should I be looking at as well ?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #858: DRILL-3640: Support JDBC Statement.setQueryTimeout(...

Posted by laurentgo <gi...@git.apache.org>.
Github user laurentgo commented on a diff in the pull request:

    https://github.com/apache/drill/pull/858#discussion_r123292200
  
    --- Diff: exec/jdbc/src/main/java/org/apache/drill/jdbc/SqlTimeoutException.java ---
    @@ -23,12 +23,17 @@
      * Indicates that an operation timed out. This is not an error; you can
      * retry the operation.
      */
    -public class SqlTimeoutException
    -    extends SQLException
    -{
    +public class SqlTimeoutException extends SQLException {
    +
    +  private static final long serialVersionUID = 2017_06_20L;
    --- End diff --
    
    maybe use -1  or use your IDE serialVersionUID generator (I know there's an algorithm for it...), but I don't think Drill is making any effort to make this class serializable so it could be a `@SuppressedWarning` annotation as well.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #858: DRILL-3640: Support JDBC Statement.setQueryTimeout(...

Posted by kkhatua <gi...@git.apache.org>.
Github user kkhatua commented on a diff in the pull request:

    https://github.com/apache/drill/pull/858#discussion_r123865494
  
    --- 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 --
    
    Using a threadPool model is actually making the solution simpler, since I'm triggering the query cancellation. The TimeoutTrigger class uses an executorService for this cleanly, without exposing anything else. Checking constantly for the remaining time seemed like an overhead that would be cumulatively add to the runtime. 
    The one benefit of doing this in the Cursor would be that I could, potentially, have a new QueryState defined as TIMED_OUT, but I'm reluctant to modify the Protobuf as of now.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #858: DRILL-3640: Support JDBC Statement.setQueryTimeout(...

Posted by laurentgo <gi...@git.apache.org>.
Github user laurentgo commented on a diff in the pull request:

    https://github.com/apache/drill/pull/858#discussion_r149230118
  
    --- Diff: exec/jdbc/src/main/java/org/apache/drill/jdbc/impl/DrillResultSetImpl.java ---
    @@ -98,6 +103,34 @@ private void throwIfClosed() throws AlreadyClosedSqlException,
         }
       }
     
    +  /**
    +   * Throws SqlTimeoutException if the statement has already timed out
    +   * @throws AlreadyClosedSqlException   if Connection is closed
    +   * @throws SqlTimeoutException         if Statement is already timed out
    +   */
    +  private void throwIfTimedOut() throws AlreadyClosedSqlException,
    +                                        SqlTimeoutException,
    +                                        SQLException {
    +    if ( ((DrillStatement) statement).isTimedOut() ) {
    +      //TODO
    +//    if ( (statement instanceof DrillStatementImpl && ((DrillStatementImpl) statement).isTimedOut()) ||
    --- End diff --
    
    to be removed?


---

[GitHub] drill pull request #858: DRILL-3640: Support JDBC Statement.setQueryTimeout(...

Posted by kkhatua <gi...@git.apache.org>.
Github user kkhatua closed the pull request at:

    https://github.com/apache/drill/pull/858


---

[GitHub] drill issue #858: DRILL-3640: Support JDBC Statement.setQueryTimeout(int)

Posted by kkhatua <gi...@git.apache.org>.
Github user kkhatua commented on the issue:

    https://github.com/apache/drill/pull/858
  
    Got it... traced it to a change in the DrillResultSetImpl. Thanks!


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #858: DRILL-3640: Support JDBC Statement.setQueryTimeout(...

Posted by laurentgo <gi...@git.apache.org>.
Github user laurentgo commented on a diff in the pull request:

    https://github.com/apache/drill/pull/858#discussion_r149273958
  
    --- Diff: exec/jdbc/src/main/java/org/apache/drill/jdbc/impl/DrillPreparedStatementImpl.java ---
    @@ -201,31 +252,57 @@ public int getQueryTimeout() throws SQLException {
     
       @Override
       public void setQueryTimeout(int seconds) throws SQLException {
    -    throwIfClosed();
    -    super.setQueryTimeout(seconds);
    +    throwIfTimedOutOrClosed();
    +    if ( seconds < 0 ) {
    +      throw new InvalidParameterSqlException(
    +          "Invalid (negative) \"seconds\" parameter to setQueryTimeout(...)"
    +              + " (" + seconds + ")" );
    +    }
    +    else {
    +      if ( 0 < seconds ) {
    +        timeoutTrigger = new TimeoutTrigger(this, seconds);
    --- End diff --
    
    it might be nice to cancel the previous trigger too, assuming that changing the timeout after the query was started should have any effect (I would say no, but JDBC API is not clear here).


---

[GitHub] drill pull request #858: DRILL-3640: Support JDBC Statement.setQueryTimeout(...

Posted by kkhatua <gi...@git.apache.org>.
Github user kkhatua commented on a diff in the pull request:

    https://github.com/apache/drill/pull/858#discussion_r123324366
  
    --- Diff: exec/jdbc/src/main/java/org/apache/drill/jdbc/impl/DrillResultSetImpl.java ---
    @@ -64,13 +65,17 @@
           org.slf4j.LoggerFactory.getLogger(DrillResultSetImpl.class);
     
       private final DrillConnectionImpl connection;
    +  private DrillStatementImpl drillStatement = null;
       private volatile boolean hasPendingCancelationNotification = false;
     
       DrillResultSetImpl(AvaticaStatement statement, Meta.Signature signature,
                          ResultSetMetaData resultSetMetaData, TimeZone timeZone,
                          Meta.Frame firstFrame) {
         super(statement, signature, resultSetMetaData, timeZone, firstFrame);
         connection = (DrillConnectionImpl) statement.getConnection();
    +    if (statement instanceof DrillStatementImpl) {
    --- End diff --
    
    The original DrillStatement threw an exception for setting the query time out, but I don't see the same for the DrillPreparedStatement, which tries to call Avatica's implementation. My testing indicates that this is effectively a No-Op. I can extend it to that as well, but I was hoping to hear back from you first. 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #858: DRILL-3640: Support JDBC Statement.setQueryTimeout(...

Posted by laurentgo <gi...@git.apache.org>.
Github user laurentgo commented on a diff in the pull request:

    https://github.com/apache/drill/pull/858#discussion_r123865940
  
    --- 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 --
    
    Maybe simpler, but I'm doubtful this is less overhead: there's a new executor and a new thread created for each statement (to compare with just telling the system how long to wait in DrillCursor, knowing that the i/o system is using a fixed number of threads vs your proposal which is unbounded). 
    I'm also not following why you would need a new state, as this would be a pure client thing.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #858: DRILL-3640: Support JDBC Statement.setQueryTimeout(...

Posted by laurentgo <gi...@git.apache.org>.
Github user laurentgo commented on a diff in the pull request:

    https://github.com/apache/drill/pull/858#discussion_r149274342
  
    --- Diff: exec/jdbc/src/main/java/org/apache/drill/jdbc/impl/TimeoutTrigger.java ---
    @@ -0,0 +1,87 @@
    +/**
    + * 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.jdbc.impl;
    +
    +import java.sql.SQLException;
    +import java.util.concurrent.Callable;
    +import java.util.concurrent.Future;
    +import java.util.concurrent.TimeUnit;
    +
    +import org.apache.calcite.avatica.AvaticaStatement;
    +import org.apache.drill.jdbc.DrillStatement;
    +import org.apache.drill.jdbc.SqlTimeoutException;
    +
    +/**
    + * Timeout Trigger required for canceling of running queries
    + */
    +class TimeoutTrigger implements Callable<Boolean> {
    +  private int timeoutInSeconds;
    +  private AvaticaStatement statementHandle;
    +  private Future<Boolean> triggerFuture;
    +  private DrillConnectionImpl connectionHandle;
    +
    +  public boolean isTriggered() {
    +    return (triggerFuture != null);
    +  }
    +
    +
    +  //Default Constructor is Invalid
    +  @SuppressWarnings("unused")
    +  private TimeoutTrigger() {}
    +
    +  /**
    +   * Timeout Constructor
    +   * @param stmtContext   Statement Handle
    +   * @param timeoutInSec  Timeout defined in seconds
    +   * @throws SQLException
    +   */
    +  TimeoutTrigger(AvaticaStatement stmtContext, int timeoutInSec) throws SQLException {
    +    timeoutInSeconds = timeoutInSec;
    +    statementHandle = stmtContext;
    +    connectionHandle = (DrillConnectionImpl) ((DrillStatement) stmtContext).getConnection();
    +  }
    +
    +  @Override
    +  public Boolean call() throws Exception {
    +    try {
    +      TimeUnit.SECONDS.sleep(timeoutInSeconds);
    +    } catch (InterruptedException e) {
    +      //Skip interruption that occur due due to query completion
    +    }
    +    try {
    +      if (!statementHandle.isClosed()) {
    +        // Cancel Statement
    +        ((DrillStatement) statementHandle).cancelDueToTimeout();
    +      }
    +    } catch (SqlTimeoutException toe) {
    +      throw toe;
    +    }
    +    return false;
    +  }
    +
    +  /**
    +   * Start the timer
    +   * @return
    +   */
    +  public Future<Boolean> startCountdown() {
    +    if (!isTriggered()) {
    --- End diff --
    
    I believe a new future needs to be created each time (because it is a new statement).


---

[GitHub] drill pull request #858: DRILL-3640: Support JDBC Statement.setQueryTimeout(...

Posted by laurentgo <gi...@git.apache.org>.
Github user laurentgo commented on a diff in the pull request:

    https://github.com/apache/drill/pull/858#discussion_r149229229
  
    --- Diff: exec/jdbc/src/main/java/org/apache/drill/jdbc/DrillStatement.java ---
    @@ -34,11 +35,11 @@
        *            if connection is closed
        */
       @Override
    -  int getQueryTimeout() throws AlreadyClosedSqlException;
    +  int getQueryTimeout() throws AlreadyClosedSqlException, SQLException;
    --- End diff --
    
    javadoc needs to be updated too


---

[GitHub] drill issue #858: DRILL-3640: Support JDBC Statement.setQueryTimeout(int)

Posted by kkhatua <gi...@git.apache.org>.
Github user kkhatua commented on the issue:

    https://github.com/apache/drill/pull/858
  
    @laurentgo I'm seeing JUnit test failures with this commit.... 
    
    >   ConnectionTest.testPrepareStatementBasicCaseWorks:94 » ClassCast org.apache.dr...
    >   PreparedStatementTest.testQueryMetadataInPreparedStatement:135 » ClassCast org...
    >   PreparedStatementTest.testExecuteQueryBasicCaseWorks:100 » ClassCast org.apach...
    >   LegacyPreparedStatementTest.testExecuteQueryBasicCaseWorks:84 » ClassCast org....
    
    They all point to a common ClassCaseException
    
    >   Drill2769UnsupportedReportsUseSqlExceptionTest.testPreparedStatementMethodsThrowRight:430 Non-SQLException exception error(s): 
    > Failures:
    > - PreparedStatement.execute() threw <java.lang.ClassCastException: org.apache.drill.jdbc.impl.DrillJdbc41Factory$DrillJdbc41PreparedStatement cannot be cast to org.apache.drill.jdbc.impl.DrillStatementImpl> instead of a SQLException
    > - PreparedStatement.executeQuery() threw <java.lang.ClassCastException: org.apache.drill.jdbc.impl.DrillJdbc41Factory$DrillJdbc41PreparedStatement cannot be cast to org.apache.drill.jdbc.impl.DrillStatementImpl> instead of a SQLException
    > - PreparedStatement.executeUpdate() threw <java.lang.ClassCastException: org.apache.drill.jdbc.impl.DrillJdbc41Factory$DrillJdbc41PreparedStatement cannot be cast to org.apache.drill.jdbc.impl.DrillStatementImpl> instead of a SQLException
    
    The regular Statement.execute###() calls work fine. Any idea why I'm getting this?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill issue #858: DRILL-3640: Support JDBC Statement.setQueryTimeout(int)

Posted by kkhatua <gi...@git.apache.org>.
Github user kkhatua commented on the issue:

    https://github.com/apache/drill/pull/858
  
    @laurentgo I'm hoping to commit and close the PR before the long weekend... just waiting on your review of the changes :)


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill issue #858: DRILL-3640: Support JDBC Statement.setQueryTimeout(int)

Posted by laurentgo <gi...@git.apache.org>.
Github user laurentgo commented on the issue:

    https://github.com/apache/drill/pull/858
  
    I think you can actually do 2 and 3 with the same approach, depending on how you compute the remaining time.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #858: DRILL-3640: Support JDBC Statement.setQueryTimeout(...

Posted by kkhatua <gi...@git.apache.org>.
Github user kkhatua commented on a diff in the pull request:

    https://github.com/apache/drill/pull/858#discussion_r124068854
  
    --- Diff: exec/jdbc/src/main/java/org/apache/drill/jdbc/impl/TimeoutTrigger.java ---
    @@ -0,0 +1,105 @@
    +/**
    + * 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.jdbc.impl;
    +
    +import java.util.concurrent.Callable;
    +import java.util.concurrent.ExecutorService;
    +import java.util.concurrent.Executors;
    +import java.util.concurrent.Future;
    +import java.util.concurrent.TimeUnit;
    +
    +import org.apache.calcite.avatica.AvaticaStatement;
    +import org.apache.drill.exec.rpc.NamedThreadFactory;
    +import org.apache.drill.jdbc.SqlTimeoutException;
    +
    +/**
    + * Timeout Trigger required for canceling of running queries
    + */
    +class TimeoutTrigger implements Callable<Boolean> {
    +  //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 timeoutService;
    --- End diff --
    
    The Executor has only sleepign threads. If the Drillbit is shutdown, I'm assuming that running queries are cancelled (which implicitly terminates any timeout-related threads in this executor), so the Executor will automatically be void of any active threads. However, I see your point about having a clean way to shut it down. I'll make this as a part of the DrillbitContext, since the Executor actually services a running query and, IMO, shouldn't be bound to a connection. At the DrillbitContext, I should be able to do a clean shutdown without the need for any complex checks. 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill issue #858: DRILL-3640: Support JDBC Statement.setQueryTimeout(int)

Posted by laurentgo <gi...@git.apache.org>.
Github user laurentgo commented on the issue:

    https://github.com/apache/drill/pull/858
  
    As for System.currentTimeMillis(), maybe not as expensive as you think: if I remember correctly, it's a JVM intrinsic (no JNI cost), and if mapped to gettimeofday (which I believe is the case), it's a VSDO function on Linux, which means it's pretty cheap (no user/kernel context change).  Compared to creating threads, and waiting on conditions, should be way cheaper.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill issue #858: DRILL-3640: Support JDBC Statement.setQueryTimeout(int)

Posted by parthchandra <gi...@git.apache.org>.
Github user parthchandra commented on the issue:

    https://github.com/apache/drill/pull/858
  
    Been looking at this and the first thing that occurs to me is that we are not too clear about what the timeout means in the context of ResultSet. The API specification is rather silent on that topic. 
    The only reference I could find to this question is this one: http://mail-archives.apache.org/mod_mbox/db-derby-dev/200504.mbox/%3C426909BA.1060801@sun.com%3E
    We have the same choices:
     
    
    > 1. setQueryTimeout() only affects Statement.execute()
    > 2.  setQueryTimeout() affects Statement.execute() and ResultSet.next(),starting from zero for each invocation
    > 3.  setQueryTimeout() affects Statement.execute() and ResultSet.next(),accumulating time spent in each invocation
    
    My own inclination was to select #2 as the appropriate behavior. In fact that is what I assumed before I looked at the code. Laurent's suggestion to implement the timeout in DrillCursor provides this behavior and is a little bit easier to implement.
    
    OTOH, Kunal has chosen #3 as the right behavior. MySQL implements this behavior, BTW, so it is not going to be a surprise to end users. And he has already done the work. 
    
    I'm +0 on this so far. Let me see if I can get a quick prototype to test things out.  



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #858: DRILL-3640: Support JDBC Statement.setQueryTimeout(...

Posted by laurentgo <gi...@git.apache.org>.
Github user laurentgo commented on a diff in the pull request:

    https://github.com/apache/drill/pull/858#discussion_r123293533
  
    --- Diff: exec/jdbc/src/main/java/org/apache/drill/jdbc/impl/DrillStatementImpl.java ---
    @@ -204,7 +276,7 @@ public boolean isClosed() {
     
       @Override
       public int getMaxFieldSize() throws SQLException {
    -    throwIfClosed();
    +    throwIfTimedOutOrClosed();
    --- End diff --
    
    this methods cannot throw SqlTimeoutException... only `execute*` methods can, and eventually some ResultSet methods (but this part is optional per JDBC standard)


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #858: DRILL-3640: Support JDBC Statement.setQueryTimeout(...

Posted by laurentgo <gi...@git.apache.org>.
Github user laurentgo commented on a diff in the pull request:

    https://github.com/apache/drill/pull/858#discussion_r123294533
  
    --- Diff: exec/jdbc/src/main/java/org/apache/drill/jdbc/impl/DrillStatementImpl.java ---
    @@ -497,14 +594,64 @@ public boolean isPoolable() throws SQLException {
     
       @Override
       public void closeOnCompletion() throws SQLException {
    -    throwIfClosed();
    +    throwIfTimedOutOrClosed();
         super.closeOnCompletion();
       }
     
       @Override
       public boolean isCloseOnCompletion() throws SQLException {
    -    throwIfClosed();
    +    throwIfTimedOutOrClosed();
         return super.isCloseOnCompletion();
       }
     
     }
    +
    +/**
    + * Timeout Trigger required for canceling of running queries
    + */
    +class TimeoutTrigger implements Callable<Boolean> {
    +  private int timeoutInSeconds;
    +
    +  /**
    +   * Get Timeout period in seconds
    +   */
    +  public int getTimeoutInSeconds() {
    +    return timeoutInSeconds;
    +  }
    +
    +  private DrillStatementImpl statementHandle;
    +
    +  //Default Constructor is Invalid
    +  @SuppressWarnings("unused")
    +  private TimeoutTrigger() {}
    +
    +  /**
    +   * Timeout Constructor
    +   * @param stmtContext   Statement Handle
    +   * @param timeoutInSec  Timeout defined in seconds
    +   */
    +  TimeoutTrigger(DrillStatementImpl stmtContext, int timeoutInSec) {
    +    timeoutInSeconds = timeoutInSec;
    +    statementHandle = stmtContext;
    +  }
    +
    +  @Override
    +  public Boolean call() throws Exception {
    +    try {
    +      Thread.sleep(timeoutInSeconds*1000L);
    --- End diff --
    
    `TimeUnit.SECONDS.sleep(timeoutInSeconds)`


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill pull request #858: DRILL-3640: Support JDBC Statement.setQueryTimeout(...

Posted by kkhatua <gi...@git.apache.org>.
Github user kkhatua commented on a diff in the pull request:

    https://github.com/apache/drill/pull/858#discussion_r123865256
  
    --- Diff: exec/jdbc/src/main/java/org/apache/drill/jdbc/impl/DrillResultSetImpl.java ---
    @@ -125,7 +154,7 @@ protected void cancel() {
       // (Not delegated.)
       @Override
       public boolean next() throws SQLException {
    -    throwIfClosed();
    +    throwIfTimedOutOrClosed();
    --- End diff --
    
    Tested this out with the latest commit and I'm able to successfully interrupt the next() as well. Occasionally, the exceptions get missed, so we do a check down to the DrillResultSet and not the DrillCursor as I'm trying to restrict the timeout check only to the JDBC layers and not too close to the DrillClient. Testcases verify by executing and then sleeping till the duration of the timeout before invoking the next() method.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] drill issue #858: DRILL-3640: Support JDBC Statement.setQueryTimeout(int)

Posted by laurentgo <gi...@git.apache.org>.
Github user laurentgo commented on the issue:

    https://github.com/apache/drill/pull/858
  
    you should look at:
    - https://github.com/kkhatua/drill/blob/c51473859d1dd81cf70e857f729c3a8491b2834a/exec/jdbc/src/main/java/org/apache/drill/jdbc/impl/DrillCursor.java#L144
    - https://github.com/kkhatua/drill/blob/c51473859d1dd81cf70e857f729c3a8491b2834a/exec/jdbc/src/main/java/org/apache/drill/jdbc/impl/DrillCursor.java#L226
    - https://github.com/kkhatua/drill/blob/c51473859d1dd81cf70e857f729c3a8491b2834a/exec/jdbc/src/main/java/org/apache/drill/jdbc/impl/DrillCursor.java#L255
    
    I believe those are the places where blocking operations are done, most of them don't have a timeout configured, but could be configured to have one (getNext is a bit special since it's waking up every 50ms to manage throttling)
    - https://github.com/kkhatua/drill/blob/c51473859d1dd81cf70e857f729c3a8491b2834a/exec/jdbc/src/main/java/org/apache/drill/jdbc/impl/DrillCursor.java#L548
    - 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---