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/11/04 00:23:49 UTC

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

GitHub user kkhatua opened a pull request:

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

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

    Allow for queries to be cancelled if they don't complete within the stipulated time.
    This is done by having `Drill[Prepared]StatementImpl` create a `Stopwatch` timer to track elapsed time. 
      * `DrillCursor` uses this to detect timeouts. 
      * `DrillResultSetImpl` uses this to detech timeout from the client side (e.g. a slow client, when all batches have been processed by DrillCursor)
    
    Tests added to test these and other query timeout scenarios for both, `Statement` and `PreparedStatement`.

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

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

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

    https://github.com/apache/drill/pull/1024.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 #1024
    
----
commit 0fe2cfa71a60e7ed17ea12c1ba15fa28be66508f
Author: Kunal Khatua <kk...@maprtech.com>
Date:   2017-11-04T00:20:59Z

    DRILL-3640: Support JDBC Statement.setQueryTimeout(int)
    
    Allow for queries to be cancelled if they don't complete within the stipulated time.
    This is done by having Drill[Prepared]StatementImpl create a Stopwatch timer to track elapsed time. 
      * DrillCursor uses this to detect timeouts. 
      * DrillResultSetImpl uses this to detech timeout from the client side (e.g. a slow client, when all batches have been processed by DrillCursor)
    Tests added to test these and other query timeout scenarios.

----


---

[GitHub] drill pull request #1024: 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/1024#discussion_r149276207
  
    --- Diff: exec/jdbc/src/main/java/org/apache/drill/jdbc/impl/DrillCursor.java ---
    @@ -376,6 +415,19 @@ synchronized void cleanup() {
         currentBatchHolder.clear();
       }
     
    +  //Set the cursor's timeout in seconds
    --- End diff --
    
    why not simply getting the current timeout value from the statement itself?


---

[GitHub] drill pull request #1024: 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/1024#discussion_r149548759
  
    --- Diff: exec/jdbc/src/main/java/org/apache/drill/jdbc/impl/DrillCursor.java ---
    @@ -333,8 +368,14 @@ void close() {
         final int batchQueueThrottlingThreshold =
             client.getConfig().getInt(
                 ExecConstants.JDBC_BATCH_QUEUE_THROTTLING_THRESHOLD );
    -    resultsListener = new ResultsListener(batchQueueThrottlingThreshold);
    +    resultsListener = new ResultsListener(this, batchQueueThrottlingThreshold);
         currentBatchHolder = new RecordBatchLoader(client.getAllocator());
    +    try {
    +      setTimeout(this.statement.getQueryTimeout());
    +    } catch (SQLException e) {
    +      // Printing any unexpected SQLException stack trace
    +      e.printStackTrace();
    --- End diff --
    
    I agree. Thankfully, the _caller_ does handle any thrown `SQLException`s, so I'm going to pass this off to that. IMO, I don't think we'll have an issue because the `Statement.setQueryTimeout()` would have handled any corner cases before this is invoked via `execute()`


---

[GitHub] drill pull request #1024: 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/1024#discussion_r149476740
  
    --- Diff: exec/jdbc/src/main/java/org/apache/drill/jdbc/impl/DrillResultSetImpl.java ---
    @@ -66,11 +70,27 @@
       private final DrillConnectionImpl connection;
       private volatile boolean hasPendingCancelationNotification = false;
     
    +  private Stopwatch elapsedTimer;
    +
    +  private int queryTimeoutInSeconds;
    +
       DrillResultSetImpl(AvaticaStatement statement, Meta.Signature signature,
                          ResultSetMetaData resultSetMetaData, TimeZone timeZone,
                          Meta.Frame firstFrame) {
         super(statement, signature, resultSetMetaData, timeZone, firstFrame);
         connection = (DrillConnectionImpl) statement.getConnection();
    +    try {
    +      if (statement.getQueryTimeout() > 0) {
    +        queryTimeoutInSeconds = statement.getQueryTimeout();
    +      }
    +    } catch (Exception e) {
    +      e.printStackTrace();
    --- End diff --
    
    I think so


---

[GitHub] drill pull request #1024: 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/1024#discussion_r149550602
  
    --- Diff: exec/jdbc/src/main/java/org/apache/drill/jdbc/impl/DrillCursor.java ---
    @@ -260,6 +288,10 @@ void close() {
           // when the main thread is blocked waiting for the result.  In that case
           // we want to unblock the main thread.
           firstMessageReceived.countDown(); // TODO:  Why not call releaseIfFirst as used elsewhere?
    +      //Stopping timeout clock
    --- End diff --
    
    Ok. Guess we'll do away with it. +1


---

[GitHub] drill pull request #1024: 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/1024#discussion_r149471937
  
    --- Diff: exec/jdbc/src/main/java/org/apache/drill/jdbc/impl/DrillPreparedStatementImpl.java ---
    @@ -46,6 +48,8 @@
                    DrillRemoteStatement {
     
       private final PreparedStatement preparedStatementHandle;
    +  int queryTimeoutInSeconds = 0;
    --- End diff --
    
    Ah.. i missed this during the clean up. Thanks! +1


---

[GitHub] drill pull request #1024: 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/1024#discussion_r149547712
  
    --- Diff: exec/jdbc/src/main/java/org/apache/drill/jdbc/impl/DrillResultSetImpl.java ---
    @@ -96,6 +105,14 @@ private void throwIfClosed() throws AlreadyClosedSqlException,
             throw new AlreadyClosedSqlException( "ResultSet is already closed." );
           }
         }
    +
    +    //Implicit check for whether timeout is set
    +    if (elapsedTimer != null) {
    --- End diff --
    
    I was originally wondering as to when should we trigger the countdown on the timer. 
    Creating a `[Prepared]Statement` object should not be the basis for the starting the clock, but only when you actually call execute(). The `DrillCursor` is initialized in this method and is what starts the clock. 
    I could create a clone of the `testTriggeredQueryTimeout` method and simply have the client pause after `execute()` but before fetching the `ResultSet` instance or invoking `ResultSet.next()` . Would that work ?


---

[GitHub] drill pull request #1024: 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/1024#discussion_r149465309
  
    --- Diff: exec/jdbc/src/main/java/org/apache/drill/jdbc/impl/DrillCursor.java ---
    @@ -239,6 +259,11 @@ QueryDataBatch getNext() throws UserException, InterruptedException {
                 }
                 return qdb;
               }
    +
    +          // Check and throw SQLTimeoutException
    +          if ( parent.timeoutInSeconds > 0 && parent.elapsedTimer.elapsed(TimeUnit.SECONDS) >= parent.timeoutInSeconds ) {
    --- End diff --
    
    you don't really need a check after the pool: if it's not null, it means it completed before timeout and you can proceed forward. If it is null, then you would loop and redo the check based on the current time and might be able to throw a timeout exception


---

[GitHub] drill pull request #1024: 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/1024#discussion_r149543078
  
    --- Diff: exec/jdbc/src/main/java/org/apache/drill/jdbc/impl/DrillCursor.java ---
    @@ -333,8 +368,14 @@ void close() {
         final int batchQueueThrottlingThreshold =
             client.getConfig().getInt(
                 ExecConstants.JDBC_BATCH_QUEUE_THROTTLING_THRESHOLD );
    -    resultsListener = new ResultsListener(batchQueueThrottlingThreshold);
    +    resultsListener = new ResultsListener(this, batchQueueThrottlingThreshold);
         currentBatchHolder = new RecordBatchLoader(client.getAllocator());
    +    try {
    +      setTimeout(this.statement.getQueryTimeout());
    +    } catch (SQLException e) {
    +      // Printing any unexpected SQLException stack trace
    +      e.printStackTrace();
    --- End diff --
    
    two choices here:
    - we don't think it's important if we cannot get the value, so we should log it properly and not simply dump the exception
    - we think this is important, and we propagate the exception to the caller
    
    (I think it is important: the most likely reason why we could not get the value if that the statement was closed, and we should probably notify the user about it).


---

[GitHub] drill pull request #1024: 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/1024#discussion_r150131105
  
    --- Diff: exec/jdbc/src/main/java/org/apache/drill/jdbc/impl/DrillResultSetImpl.java ---
    @@ -96,6 +105,14 @@ private void throwIfClosed() throws AlreadyClosedSqlException,
             throw new AlreadyClosedSqlException( "ResultSet is already closed." );
           }
         }
    +
    +    //Implicit check for whether timeout is set
    +    if (elapsedTimer != null) {
    --- End diff --
    
    Yes, I'm wrong? (asking because the rest of the sentence suggest I was right in my interpretation of the test). Maybe we can/should test both? I would have like to test for the first batch, but it's not possible to access the query id until `statement.execute()`, and I'd need it to unpause the request.


---

[GitHub] drill issue #1024: 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/1024
  
    @laurentgo Done the changes... ready for review. 


---

[GitHub] drill issue #1024: 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/1024
  
    Ready for a final review. 
    All unit tests, with the exception of `PreparedStatementTest.testServerTriggeredQueryTimeout` . The test is being ignored because the timed pause injection is not being honoured for a PreparedStatement, though it is honoured for a regular Statement. In actual dev/functional testing, however, the timeout works, which makes me believe there is a limitation with the test framework injecting pauses for Prepared Statement.


---

[GitHub] drill pull request #1024: 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/1024#discussion_r149278066
  
    --- Diff: exec/jdbc/src/test/java/org/apache/drill/jdbc/PreparedStatementTest.java ---
    @@ -237,6 +245,127 @@ public String toString() {
         }
       }
     
    +  /**
    +   * Test for reading of default query timeout
    +   */
    +  @Test
    +  public void testDefaultGetQueryTimeout() throws SQLException {
    +    PreparedStatement stmt = connection.prepareStatement(SYS_VERSION_SQL);
    +    int timeoutValue = stmt.getQueryTimeout();
    +    assert( 0 == timeoutValue );
    +  }
    +
    +  /**
    +   * Test Invalid parameter by giving negative timeout
    +   */
    +  @Test ( expected = InvalidParameterSqlException.class )
    +  public void testInvalidSetQueryTimeout() throws SQLException {
    +    PreparedStatement stmt = connection.prepareStatement(SYS_VERSION_SQL);
    +    //Setting negative value
    +    int valueToSet = -10;
    +    if (0L == valueToSet) {
    +      valueToSet--;
    +    }
    +    try {
    +      stmt.setQueryTimeout(valueToSet);
    +    } catch ( final Exception e) {
    +      // TODO: handle exception
    +      assertThat( e.getMessage(), containsString( "illegal timeout value") );
    +      //Converting this to match expected Exception
    +      throw new InvalidParameterSqlException(e.getMessage());
    --- End diff --
    
    there's no point I guess? you already verified the exception...


---

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

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

    https://github.com/apache/drill/pull/1024#discussion_r155409333
  
    --- Diff: exec/jdbc/src/main/java/org/apache/drill/jdbc/impl/DrillResultSetImpl.java ---
    @@ -96,6 +105,14 @@ private void throwIfClosed() throws AlreadyClosedSqlException,
             throw new AlreadyClosedSqlException( "ResultSet is already closed." );
           }
         }
    +
    +    //Implicit check for whether timeout is set
    +    if (elapsedTimer != null) {
    --- End diff --
    
    Do you want to try statement.cancel() to release the memory ?


---

[GitHub] drill issue #1024: 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/1024
  
    Did a rebase on the latest master to resolve the merge conflict from DRILL-3993 [commit](https://github.com/apache/drill/commit/9fabb612f16f6f541b3bde68ad7d734cad26df33#diff-f5de5223afdaaec6d009c4e06015e34d) that upgraded to Calcite 1.13.


---

[GitHub] drill pull request #1024: 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/1024#discussion_r149277205
  
    --- Diff: exec/jdbc/src/main/java/org/apache/drill/jdbc/impl/DrillCursor.java ---
    @@ -100,13 +103,17 @@
         final LinkedBlockingDeque<QueryDataBatch> batchQueue =
             Queues.newLinkedBlockingDeque();
     
    +    private final DrillCursor parent;
    --- End diff --
    
    you only need to know the moment after which the query times out, basically StopWatch (but you could use a long as an epoch too with Long.MAX meaning infinite wait).


---

[GitHub] drill pull request #1024: 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/1024#discussion_r149278905
  
    --- Diff: exec/jdbc/src/test/java/org/apache/drill/jdbc/StatementTest.java ---
    @@ -61,55 +71,129 @@ public static void tearDownStatement() throws SQLException {
       //////////
       // getQueryTimeout():
     
    -  /** Tests that getQueryTimeout() indicates no timeout set. */
    +  /**
    +   * Test for reading of default query timeout
    +   */
       @Test
    -  public void testGetQueryTimeoutSaysNoTimeout() throws SQLException {
    -    assertThat( statement.getQueryTimeout(), equalTo( 0 ) );
    +  public void testDefaultGetQueryTimeout() throws SQLException {
    +    Statement stmt = connection.createStatement();
    +    int timeoutValue = stmt.getQueryTimeout();
    +    assert( 0 == timeoutValue );
    --- End diff --
    
    I personally prefer the old construct using hamcrest. This `assert` is actually less useful for two reasons:
    - there's no error message (hamcrest prints a nice error message based on the arguments)
    - if the assert fails, you don't know `timeoutValue`'s value


---

[GitHub] drill pull request #1024: 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/1024#discussion_r150157923
  
    --- Diff: exec/jdbc/src/main/java/org/apache/drill/jdbc/impl/DrillResultSetImpl.java ---
    @@ -96,6 +105,14 @@ private void throwIfClosed() throws AlreadyClosedSqlException,
             throw new AlreadyClosedSqlException( "ResultSet is already closed." );
           }
         }
    +
    +    //Implicit check for whether timeout is set
    +    if (elapsedTimer != null) {
    --- End diff --
    
    I don't think you are wrong, but I think the interpretation of the timeout is ambiguous. My understanding based on what drivers like Oracle do is to start the timeout only when the execute call is made. So, for a regular Statement object, just initialization (or even setting the timeout) should not be the basis of starting the timer. 
    With regards to whether we are testing for the time when only the DrillCursor is in operation, we'd need a query that is running sufficiently long to timeout before the server can send back anything for the very first time. The `awaitFirstMessage()` already has the timeout applied there and worked in some of my longer running sample queries. If you're hinting towards this, then yes.. it is certainly doesn't hurt to have the test, although the timeout does guarantee exactly that.
    
    I'm not familiar with the Drillbit Injection feature, so let me tinker a bit to confirm it before I update the PR.


---

[GitHub] drill pull request #1024: 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/1024#discussion_r149463638
  
    --- Diff: exec/jdbc/src/main/java/org/apache/drill/jdbc/impl/DrillCursor.java ---
    @@ -239,6 +259,11 @@ QueryDataBatch getNext() throws UserException, InterruptedException {
                 }
                 return qdb;
               }
    +
    +          // Check and throw SQLTimeoutException
    +          if ( parent.timeoutInSeconds > 0 && parent.elapsedTimer.elapsed(TimeUnit.SECONDS) >= parent.timeoutInSeconds ) {
    --- End diff --
    
    Good point, and I thought it might help in avoiding going into polling all together.  
    However, the granularity of the timeout is in seconds, so 50ms is insignificant. If I do a check before the poll, I'd need to do after the poll as well.. over a 50ms window. So, a post-poll check works fine, because we'll, at most, exceed the timeout by 50ms. So a timeout of 1sec would occur in 1.05sec. For any larger timeout values, the 50ms is of diminishing significance.


---

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

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

    https://github.com/apache/drill/pull/1024#discussion_r155402523
  
    --- Diff: exec/jdbc/src/main/java/org/apache/drill/jdbc/impl/DrillStatementImpl.java ---
    @@ -156,29 +156,19 @@ public void cleanUp() {
       }
     
       @Override
    -  public int getQueryTimeout() throws AlreadyClosedSqlException
    +  public int getQueryTimeout() throws AlreadyClosedSqlException, SQLException
       {
         throwIfClosed();
    -    return 0;  // (No no timeout.)
    +    return super.getQueryTimeout();
       }
     
       @Override
    -  public void setQueryTimeout( int milliseconds )
    +  public void setQueryTimeout( int seconds )
           throws AlreadyClosedSqlException,
                  InvalidParameterSqlException,
    -             SQLFeatureNotSupportedException {
    +             SQLException {
    --- End diff --
    
    the parent setQueryTimeout will throw a SQLException if the parameter is invalid, so this method now no longer throws an InvalidParameterSqlException


---

[GitHub] drill pull request #1024: 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/1024#discussion_r149542720
  
    --- Diff: exec/jdbc/src/main/java/org/apache/drill/jdbc/impl/DrillCursor.java ---
    @@ -260,6 +288,10 @@ void close() {
           // when the main thread is blocked waiting for the result.  In that case
           // we want to unblock the main thread.
           firstMessageReceived.countDown(); // TODO:  Why not call releaseIfFirst as used elsewhere?
    +      //Stopping timeout clock
    --- End diff --
    
    since we are closing, do we need to care about the stopwatch?


---

[GitHub] drill pull request #1024: 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/1024#discussion_r149476820
  
    --- Diff: exec/jdbc/src/test/java/org/apache/drill/jdbc/PreparedStatementTest.java ---
    @@ -237,6 +245,127 @@ public String toString() {
         }
       }
     
    +  /**
    +   * Test for reading of default query timeout
    +   */
    +  @Test
    +  public void testDefaultGetQueryTimeout() throws SQLException {
    +    PreparedStatement stmt = connection.prepareStatement(SYS_VERSION_SQL);
    +    int timeoutValue = stmt.getQueryTimeout();
    +    assert( 0 == timeoutValue );
    +  }
    +
    +  /**
    +   * Test Invalid parameter by giving negative timeout
    +   */
    +  @Test ( expected = InvalidParameterSqlException.class )
    +  public void testInvalidSetQueryTimeout() throws SQLException {
    +    PreparedStatement stmt = connection.prepareStatement(SYS_VERSION_SQL);
    +    //Setting negative value
    +    int valueToSet = -10;
    +    if (0L == valueToSet) {
    +      valueToSet--;
    +    }
    +    try {
    +      stmt.setQueryTimeout(valueToSet);
    +    } catch ( final Exception e) {
    +      // TODO: handle exception
    +      assertThat( e.getMessage(), containsString( "illegal timeout value") );
    +      //Converting this to match expected Exception
    +      throw new InvalidParameterSqlException(e.getMessage());
    +    }
    +  }
    +
    +  /**
    +   * Test setting a valid timeout
    +   */
    +  @Test
    +  public void testValidSetQueryTimeout() throws SQLException {
    +    PreparedStatement stmt = connection.prepareStatement(SYS_VERSION_SQL);
    +    //Setting positive value
    +    int valueToSet = new Random(System.currentTimeMillis()).nextInt(60);
    +    if (0L == valueToSet) {
    +      valueToSet++;
    +    }
    +    stmt.setQueryTimeout(valueToSet);
    +    assert( valueToSet == stmt.getQueryTimeout() );
    +  }
    +
    +  /**
    +   * Test setting timeout as zero and executing
    +   */
    +  @Test
    +  public void testSetQueryTimeoutAsZero() throws SQLException {
    +    PreparedStatement stmt = connection.prepareStatement(SYS_RANDOM_SQL);
    +    stmt.setQueryTimeout(0);
    +    stmt.executeQuery();
    +    ResultSet rs = stmt.getResultSet();
    +    int rowCount = 0;
    +    while (rs.next()) {
    +      rs.getBytes(1);
    +      rowCount++;
    +    }
    +    stmt.close();
    +    assert( 3 == rowCount );
    +  }
    +
    +  /**
    +   * Test setting timeout for a query that actually times out
    +   */
    +  @Test ( expected = SQLTimeoutException.class )
    +  public void testTriggeredQueryTimeout() throws SQLException {
    +    PreparedStatement stmt = null;
    +    //Setting to a very low value (3sec)
    +    int timeoutDuration = 3;
    +    int rowsCounted = 0;
    +    try {
    +      stmt = connection.prepareStatement(SYS_RANDOM_SQL);
    +      stmt.setQueryTimeout(timeoutDuration);
    +      System.out.println("Set a timeout of "+ stmt.getQueryTimeout() +" seconds");
    --- End diff --
    
    I think I previously came across some unit tests that are using System.out instead of logger, so i figured there wasn't any preference. Logger is probably the cleaner way of doing things. +1


---

[GitHub] drill pull request #1024: 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/1024#discussion_r149277524
  
    --- Diff: exec/jdbc/src/main/java/org/apache/drill/jdbc/impl/DrillPreparedStatementImpl.java ---
    @@ -61,8 +65,14 @@ protected DrillPreparedStatementImpl(DrillConnectionImpl connection,
         if (preparedStatementHandle != null) {
           ((DrillColumnMetaDataList) signature.columns).updateColumnMetaData(preparedStatementHandle.getColumnsList());
         }
    +    //Implicit query timeout
    +    this.queryTimeoutInSeconds = 0;
    +    this.elapsedTimer = Stopwatch.createUnstarted();
    --- End diff --
    
    you want a new timer for each execution: Best way is to let cursor manage it


---

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

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

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


---

[GitHub] drill pull request #1024: 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/1024#discussion_r149476190
  
    --- Diff: exec/jdbc/src/test/java/org/apache/drill/jdbc/PreparedStatementTest.java ---
    @@ -237,6 +245,127 @@ public String toString() {
         }
       }
     
    +  /**
    +   * Test for reading of default query timeout
    +   */
    +  @Test
    +  public void testDefaultGetQueryTimeout() throws SQLException {
    +    PreparedStatement stmt = connection.prepareStatement(SYS_VERSION_SQL);
    +    int timeoutValue = stmt.getQueryTimeout();
    +    assert( 0 == timeoutValue );
    +  }
    +
    +  /**
    +   * Test Invalid parameter by giving negative timeout
    +   */
    +  @Test ( expected = InvalidParameterSqlException.class )
    +  public void testInvalidSetQueryTimeout() throws SQLException {
    +    PreparedStatement stmt = connection.prepareStatement(SYS_VERSION_SQL);
    +    //Setting negative value
    +    int valueToSet = -10;
    +    if (0L == valueToSet) {
    +      valueToSet--;
    +    }
    +    try {
    +      stmt.setQueryTimeout(valueToSet);
    +    } catch ( final Exception e) {
    +      // TODO: handle exception
    +      assertThat( e.getMessage(), containsString( "illegal timeout value") );
    +      //Converting this to match expected Exception
    +      throw new InvalidParameterSqlException(e.getMessage());
    +    }
    +  }
    +
    +  /**
    +   * Test setting a valid timeout
    +   */
    +  @Test
    +  public void testValidSetQueryTimeout() throws SQLException {
    +    PreparedStatement stmt = connection.prepareStatement(SYS_VERSION_SQL);
    +    //Setting positive value
    +    int valueToSet = new Random(System.currentTimeMillis()).nextInt(60);
    --- End diff --
    
    I am trying to add some randomness to the test parameters, since the expected behaviour should be the same. I'll fix this up and get rid of that check.  +1


---

[GitHub] drill pull request #1024: 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/1024#discussion_r149543581
  
    --- Diff: exec/jdbc/src/main/java/org/apache/drill/jdbc/impl/DrillResultSetImpl.java ---
    @@ -96,6 +105,14 @@ private void throwIfClosed() throws AlreadyClosedSqlException,
             throw new AlreadyClosedSqlException( "ResultSet is already closed." );
           }
         }
    +
    +    //Implicit check for whether timeout is set
    +    if (elapsedTimer != null) {
    --- End diff --
    
    maybe an helper method from the cursor to see if we timed out instead of exposing elapsedTimer?
    
    I'm not sure if this is really necessary (posted another comment about it previously), except maybe because of unit tests where it's hard to time out inside the cursor?
    
    I did a prototype too and used control injection to pause the screen operator: the test would look like this:
    ```
      /**
       * Test setting timeout for a query that actually times out
       */
      @Test ( expected = SqlTimeoutException.class )
      public void testTriggeredQueryTimeout() throws SQLException {
        // Prevent the server to complete the query to trigger a timeout
        final String controls = Controls.newBuilder()
          .addPause(ScreenCreator.class, "send-complete", 0)
          .build();
    
        try(Statement statement = connection.createStatement()) {
          assertThat(
              statement.execute(String.format(
                  "ALTER session SET `%s` = '%s'",
                  ExecConstants.DRILLBIT_CONTROL_INJECTIONS,
                  controls)),
              equalTo(true));
        }
        String queryId = null;
        try(Statement statement = connection.createStatement()) {
          int timeoutDuration = 3;
          //Setting to a very low value (3sec)
          statement.setQueryTimeout(timeoutDuration);
          ResultSet rs = statement.executeQuery(SYS_VERSION_SQL);
          queryId = ((DrillResultSet) rs).getQueryId();
          //Fetch rows
          while (rs.next()) {
            rs.getBytes(1);
          }
        } catch (SQLException sqlEx) {
          if (sqlEx instanceof SqlTimeoutException) {
            throw (SqlTimeoutException) sqlEx;
          }
        } finally {
          // Do not forget to unpause to avoid memory leak.
          if (queryId != null) {
            DrillClient drillClient = ((DrillConnection) connection).getClient();
            drillClient.resumeQuery(QueryIdHelper.getQueryIdFromString(queryId));
          }
      }
    ```
    
    Works for PreparedStatementTest too, need to make sure you pause after prepared statement is created but before it is executed.


---

[GitHub] drill pull request #1024: 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/1024#discussion_r149472806
  
    --- Diff: exec/jdbc/src/main/java/org/apache/drill/jdbc/impl/DrillPreparedStatementImpl.java ---
    @@ -61,8 +65,14 @@ protected DrillPreparedStatementImpl(DrillConnectionImpl connection,
         if (preparedStatementHandle != null) {
           ((DrillColumnMetaDataList) signature.columns).updateColumnMetaData(preparedStatementHandle.getColumnsList());
         }
    +    //Implicit query timeout
    +    this.queryTimeoutInSeconds = 0;
    +    this.elapsedTimer = Stopwatch.createUnstarted();
    --- End diff --
    
    I thought the Statement and Cursor had a 1:1 relationship, so they can share the timer. I guess for a PreparedStatement I cannot make that assumption. Will fix this. +1


---

[GitHub] drill pull request #1024: 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/1024#discussion_r149277896
  
    --- Diff: exec/jdbc/src/test/java/org/apache/drill/jdbc/PreparedStatementTest.java ---
    @@ -237,6 +245,127 @@ public String toString() {
         }
       }
     
    +  /**
    +   * Test for reading of default query timeout
    +   */
    +  @Test
    +  public void testDefaultGetQueryTimeout() throws SQLException {
    +    PreparedStatement stmt = connection.prepareStatement(SYS_VERSION_SQL);
    +    int timeoutValue = stmt.getQueryTimeout();
    +    assert( 0 == timeoutValue );
    +  }
    +
    +  /**
    +   * Test Invalid parameter by giving negative timeout
    +   */
    +  @Test ( expected = InvalidParameterSqlException.class )
    +  public void testInvalidSetQueryTimeout() throws SQLException {
    +    PreparedStatement stmt = connection.prepareStatement(SYS_VERSION_SQL);
    +    //Setting negative value
    +    int valueToSet = -10;
    +    if (0L == valueToSet) {
    --- End diff --
    
    not possible!


---

[GitHub] drill pull request #1024: 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/1024#discussion_r149278294
  
    --- Diff: exec/jdbc/src/test/java/org/apache/drill/jdbc/PreparedStatementTest.java ---
    @@ -237,6 +245,127 @@ public String toString() {
         }
       }
     
    +  /**
    +   * Test for reading of default query timeout
    +   */
    +  @Test
    +  public void testDefaultGetQueryTimeout() throws SQLException {
    +    PreparedStatement stmt = connection.prepareStatement(SYS_VERSION_SQL);
    +    int timeoutValue = stmt.getQueryTimeout();
    +    assert( 0 == timeoutValue );
    +  }
    +
    +  /**
    +   * Test Invalid parameter by giving negative timeout
    +   */
    +  @Test ( expected = InvalidParameterSqlException.class )
    +  public void testInvalidSetQueryTimeout() throws SQLException {
    +    PreparedStatement stmt = connection.prepareStatement(SYS_VERSION_SQL);
    +    //Setting negative value
    +    int valueToSet = -10;
    +    if (0L == valueToSet) {
    +      valueToSet--;
    +    }
    +    try {
    +      stmt.setQueryTimeout(valueToSet);
    +    } catch ( final Exception e) {
    +      // TODO: handle exception
    +      assertThat( e.getMessage(), containsString( "illegal timeout value") );
    +      //Converting this to match expected Exception
    +      throw new InvalidParameterSqlException(e.getMessage());
    +    }
    +  }
    +
    +  /**
    +   * Test setting a valid timeout
    +   */
    +  @Test
    +  public void testValidSetQueryTimeout() throws SQLException {
    +    PreparedStatement stmt = connection.prepareStatement(SYS_VERSION_SQL);
    +    //Setting positive value
    +    int valueToSet = new Random(System.currentTimeMillis()).nextInt(60);
    +    if (0L == valueToSet) {
    +      valueToSet++;
    +    }
    +    stmt.setQueryTimeout(valueToSet);
    +    assert( valueToSet == stmt.getQueryTimeout() );
    +  }
    +
    +  /**
    +   * Test setting timeout as zero and executing
    +   */
    +  @Test
    +  public void testSetQueryTimeoutAsZero() throws SQLException {
    +    PreparedStatement stmt = connection.prepareStatement(SYS_RANDOM_SQL);
    +    stmt.setQueryTimeout(0);
    +    stmt.executeQuery();
    +    ResultSet rs = stmt.getResultSet();
    +    int rowCount = 0;
    +    while (rs.next()) {
    +      rs.getBytes(1);
    +      rowCount++;
    +    }
    +    stmt.close();
    +    assert( 3 == rowCount );
    +  }
    +
    +  /**
    +   * Test setting timeout for a query that actually times out
    +   */
    +  @Test ( expected = SQLTimeoutException.class )
    +  public void testTriggeredQueryTimeout() throws SQLException {
    +    PreparedStatement stmt = null;
    +    //Setting to a very low value (3sec)
    +    int timeoutDuration = 3;
    +    int rowsCounted = 0;
    +    try {
    +      stmt = connection.prepareStatement(SYS_RANDOM_SQL);
    +      stmt.setQueryTimeout(timeoutDuration);
    +      System.out.println("Set a timeout of "+ stmt.getQueryTimeout() +" seconds");
    --- End diff --
    
    debug code? maybe use a logger instead?


---

[GitHub] drill issue #1024: 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/1024
  
    @parthchandra / @laurentgo 
    I've implemented changes based on the conversation in PR https://github.com/apache/drill/pull/858 and waiting for your review of this commit.


---

[GitHub] drill pull request #1024: 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/1024#discussion_r149545640
  
    --- Diff: exec/jdbc/src/main/java/org/apache/drill/jdbc/impl/DrillCursor.java ---
    @@ -260,6 +288,10 @@ void close() {
           // when the main thread is blocked waiting for the result.  In that case
           // we want to unblock the main thread.
           firstMessageReceived.countDown(); // TODO:  Why not call releaseIfFirst as used elsewhere?
    +      //Stopping timeout clock
    --- End diff --
    
    Just wrapping up any 'running' resources. 


---

[GitHub] drill issue #1024: 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/1024
  
    +1. Look like there are no more review comments to be addressed.


---

[GitHub] drill pull request #1024: 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/1024#discussion_r149546258
  
    --- Diff: exec/jdbc/src/main/java/org/apache/drill/jdbc/impl/DrillCursor.java ---
    @@ -139,8 +147,22 @@ private boolean stopThrottlingIfSo() {
           return stopped;
         }
     
    -    public void awaitFirstMessage() throws InterruptedException {
    -      firstMessageReceived.await();
    +    public void awaitFirstMessage() throws InterruptedException, SQLTimeoutException {
    +      //Check if a non-zero timeout has been set
    +      if ( parent.timeoutInMilliseconds > 0 ) {
    +        //Identifying remaining in milliseconds to maintain a granularity close to integer value of timeout
    +        long timeToTimeout = (parent.timeoutInMilliseconds) - parent.elapsedTimer.elapsed(TimeUnit.MILLISECONDS);
    +        if ( timeToTimeout > 0 ) {
    --- End diff --
    
    Affects readability, but I think comments can convey the intent. +1


---

[GitHub] drill pull request #1024: 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/1024#discussion_r149477222
  
    --- Diff: exec/jdbc/src/test/java/org/apache/drill/jdbc/PreparedStatementTest.java ---
    @@ -237,6 +245,127 @@ public String toString() {
         }
       }
     
    +  /**
    +   * Test for reading of default query timeout
    +   */
    +  @Test
    +  public void testDefaultGetQueryTimeout() throws SQLException {
    +    PreparedStatement stmt = connection.prepareStatement(SYS_VERSION_SQL);
    +    int timeoutValue = stmt.getQueryTimeout();
    +    assert( 0 == timeoutValue );
    +  }
    +
    +  /**
    +   * Test Invalid parameter by giving negative timeout
    +   */
    +  @Test ( expected = InvalidParameterSqlException.class )
    +  public void testInvalidSetQueryTimeout() throws SQLException {
    +    PreparedStatement stmt = connection.prepareStatement(SYS_VERSION_SQL);
    +    //Setting negative value
    +    int valueToSet = -10;
    +    if (0L == valueToSet) {
    +      valueToSet--;
    +    }
    +    try {
    +      stmt.setQueryTimeout(valueToSet);
    +    } catch ( final Exception e) {
    +      // TODO: handle exception
    +      assertThat( e.getMessage(), containsString( "illegal timeout value") );
    +      //Converting this to match expected Exception
    +      throw new InvalidParameterSqlException(e.getMessage());
    --- End diff --
    
    but you did since you catch the exception and do a check on the message. Rewrapping it so that the test framework can check the new type has no value.


---

[GitHub] drill pull request #1024: 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/1024#discussion_r149548337
  
    --- Diff: exec/jdbc/src/main/java/org/apache/drill/jdbc/impl/DrillResultSetImpl.java ---
    @@ -96,6 +105,14 @@ private void throwIfClosed() throws AlreadyClosedSqlException,
             throw new AlreadyClosedSqlException( "ResultSet is already closed." );
           }
         }
    +
    +    //Implicit check for whether timeout is set
    +    if (elapsedTimer != null) {
    --- End diff --
    
    yes, pausing before execute would totally work! After execute, likely not since injection is done when query is executed on the server side.


---

[GitHub] drill pull request #1024: 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/1024#discussion_r149277808
  
    --- Diff: exec/jdbc/src/test/java/org/apache/drill/jdbc/PreparedStatementTest.java ---
    @@ -237,6 +245,127 @@ public String toString() {
         }
       }
     
    +  /**
    +   * Test for reading of default query timeout
    +   */
    +  @Test
    +  public void testDefaultGetQueryTimeout() throws SQLException {
    +    PreparedStatement stmt = connection.prepareStatement(SYS_VERSION_SQL);
    --- End diff --
    
    statements should be closed: easy to do with a `try(resource) { }` construct (valid for all test cases)


---

[GitHub] drill pull request #1024: 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/1024#discussion_r149542622
  
    --- Diff: exec/jdbc/src/main/java/org/apache/drill/jdbc/impl/DrillCursor.java ---
    @@ -239,6 +261,11 @@ QueryDataBatch getNext() throws UserException, InterruptedException {
                 }
                 return qdb;
               }
    +
    +          // Check and throw SQLTimeoutException
    +          if ( parent.timeoutInMilliseconds > 0 && parent.elapsedTimer.elapsed(TimeUnit.SECONDS) >= parent.timeoutInMilliseconds ) {
    --- End diff --
    
    wrong unit for the comparison (should be millis)


---

[GitHub] drill pull request #1024: 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/1024#discussion_r149550418
  
    --- Diff: exec/jdbc/src/main/java/org/apache/drill/jdbc/impl/DrillResultSetImpl.java ---
    @@ -96,6 +105,14 @@ private void throwIfClosed() throws AlreadyClosedSqlException,
             throw new AlreadyClosedSqlException( "ResultSet is already closed." );
           }
         }
    +
    +    //Implicit check for whether timeout is set
    +    if (elapsedTimer != null) {
    --- End diff --
    
    ```yes, pausing before execute would totally work!```
    Current here is what the test does (_italics indicate what we're doing under the covers_):
    1. Init Statement
    2. Set timeout on statement (_validating the timeout value_)
    3. Calling `execute()` and fetching ResultSet instance (_starting the clock_) 
    4. Fetching a row using ResultSet.next()
    5. Pausing briefly
    6. Repeat step 4 onwards (_enough pause to trigger timeout_)
    
    I was intending to pause between step 3 and 4 as an additional step. 
    You believe that we are not exercising any tests for timeout within the `execute()` call? 
    (Ref: https://github.com/kkhatua/drill/blob/9c4e3f3f727e70ca058facd4767556087a1876e1/exec/jdbc/src/main/java/org/apache/drill/jdbc/impl/DrillResultSetImpl.java#L1908 )



---

[GitHub] drill pull request #1024: 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/1024#discussion_r149277286
  
    --- Diff: exec/jdbc/src/main/java/org/apache/drill/jdbc/impl/DrillResultSetImpl.java ---
    @@ -66,11 +70,27 @@
       private final DrillConnectionImpl connection;
       private volatile boolean hasPendingCancelationNotification = false;
     
    +  private Stopwatch elapsedTimer;
    +
    +  private int queryTimeoutInSeconds;
    +
       DrillResultSetImpl(AvaticaStatement statement, Meta.Signature signature,
                          ResultSetMetaData resultSetMetaData, TimeZone timeZone,
                          Meta.Frame firstFrame) {
         super(statement, signature, resultSetMetaData, timeZone, firstFrame);
         connection = (DrillConnectionImpl) statement.getConnection();
    +    try {
    +      if (statement.getQueryTimeout() > 0) {
    +        queryTimeoutInSeconds = statement.getQueryTimeout();
    +      }
    +    } catch (Exception e) {
    +      e.printStackTrace();
    --- End diff --
    
    debug code?


---

[GitHub] drill pull request #1024: 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/1024#discussion_r149467572
  
    --- Diff: exec/jdbc/src/main/java/org/apache/drill/jdbc/impl/DrillCursor.java ---
    @@ -376,6 +415,19 @@ synchronized void cleanup() {
         currentBatchHolder.clear();
       }
     
    +  //Set the cursor's timeout in seconds
    --- End diff --
    
    We do get the timeout value from the Statement (Ref: https://github.com/kkhatua/drill/blob/a008707c7b97ea95700ab0f2eb5182d779a9bcb3/exec/jdbc/src/main/java/org/apache/drill/jdbc/impl/DrillCursor.java#L372 )
    However, the Statement is referred to by the ResultSet object as well, to get a handle of the timer object. During testing, I found that there is a possibility that the DrillCursor completes fetching all batches, but a slow client would call ResultSet.next() slowly and time out. The ResultSet object has no reference to the timer, except via the Statement object.
    There is a bigger problem that this block of code fixes. During iteration, we don't want to be able to change the timeout period. Hence, the DrillCursor (invoked by the _first_ `ResultSet.next()` call) will be initialized and set the timer to start ticking.Thereafter, any attempt to change the timeout can be ignored.


---

[GitHub] drill pull request #1024: 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/1024#discussion_r150127658
  
    --- Diff: exec/jdbc/src/main/java/org/apache/drill/jdbc/impl/DrillResultSetImpl.java ---
    @@ -96,6 +105,14 @@ private void throwIfClosed() throws AlreadyClosedSqlException,
             throw new AlreadyClosedSqlException( "ResultSet is already closed." );
           }
         }
    +
    +    //Implicit check for whether timeout is set
    +    if (elapsedTimer != null) {
    --- End diff --
    
    Yes. So I'm testing for the part when the batch has been fetched byt DrillCursor but not consumed via the DrillResultSetImpl. That's why I found the need for pausing the Screen operator odd and, hence, the question.


---

[GitHub] drill pull request #1024: 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/1024#discussion_r150160362
  
    --- Diff: exec/jdbc/src/main/java/org/apache/drill/jdbc/impl/DrillResultSetImpl.java ---
    @@ -96,6 +105,14 @@ private void throwIfClosed() throws AlreadyClosedSqlException,
             throw new AlreadyClosedSqlException( "ResultSet is already closed." );
           }
         }
    +
    +    //Implicit check for whether timeout is set
    +    if (elapsedTimer != null) {
    --- End diff --
    
    Ok, so I think I see how you've been trying to help me test the server side timeout.
    
    You are hoping to have a unit test force the awaiteFirstMessage() throw the exception by preventing the server from sending back any batch of data, since the sample test data doesn't allow for any query to run sufficiently long. All the current tests I've added essentially have already delivered the data from the 'Drill Server' to the 'DrillClient', but the application downstream has not consumed it.
    
    Your suggestion of putting a `pause` before the `execute()` call got me thinking that the timer had already begun after Statement initialization. My understanding now is that you're simply asking to block any SCREEN operator from sending back any batches. So, the DrillCursor should time out waiting for the first batch. In fact, I'm thinking that I don't even need a pause. The DrillCursor awaits all the time for something from the SCREEN operator that never comes and eventually times out.
    
    However, since the control injection is essentially applying to the Connection (`alter session ...`, any other unit tests in parallel execution on the same connection, would be affected by this. So, I would need to also undo this at the end of the test, if the connection is reused. Or fork off a connection exclusively for this.
    
    Was that what you've been suggesting all along?


---

[GitHub] drill pull request #1024: 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/1024#discussion_r159790956
  
    --- Diff: exec/jdbc/src/main/java/org/apache/drill/jdbc/impl/DrillStatementImpl.java ---
    @@ -156,29 +156,19 @@ public void cleanUp() {
       }
     
       @Override
    -  public int getQueryTimeout() throws AlreadyClosedSqlException
    +  public int getQueryTimeout() throws AlreadyClosedSqlException, SQLException
       {
         throwIfClosed();
    -    return 0;  // (No no timeout.)
    +    return super.getQueryTimeout();
       }
     
       @Override
    -  public void setQueryTimeout( int milliseconds )
    +  public void setQueryTimeout( int seconds )
           throws AlreadyClosedSqlException,
                  InvalidParameterSqlException,
    -             SQLFeatureNotSupportedException {
    +             SQLException {
    --- End diff --
    
    +1


---

[GitHub] drill pull request #1024: 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/1024#discussion_r149546431
  
    --- Diff: exec/jdbc/src/main/java/org/apache/drill/jdbc/impl/DrillCursor.java ---
    @@ -376,6 +417,19 @@ synchronized void cleanup() {
         currentBatchHolder.clear();
       }
     
    +  long getTimeoutInMilliseconds() {
    +    return timeoutInMilliseconds;
    +  }
    +
    +  //Set the cursor's timeout in seconds
    +  void setTimeout(int timeoutDurationInSeconds){
    +    this.timeoutInMilliseconds = timeoutDurationInSeconds*1000L;
    --- End diff --
    
    +1 


---

[GitHub] drill pull request #1024: 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/1024#discussion_r149277720
  
    --- Diff: exec/jdbc/src/main/java/org/apache/drill/jdbc/impl/DrillResultSetImpl.java ---
    @@ -66,11 +70,27 @@
       private final DrillConnectionImpl connection;
       private volatile boolean hasPendingCancelationNotification = false;
     
    +  private Stopwatch elapsedTimer;
    --- End diff --
    
    same as for `DrillStatementImpl`: you need to reset between executions, so better to let `DrillCursor` handle it.


---

[GitHub] drill pull request #1024: 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/1024#discussion_r149277578
  
    --- Diff: exec/jdbc/src/main/java/org/apache/drill/jdbc/impl/DrillPreparedStatementImpl.java ---
    @@ -46,6 +48,8 @@
                    DrillRemoteStatement {
     
       private final PreparedStatement preparedStatementHandle;
    +  int queryTimeoutInSeconds = 0;
    --- End diff --
    
    unnecessary I believe: you can simply access the parent class' value


---

[GitHub] drill pull request #1024: 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/1024#discussion_r149542468
  
    --- Diff: exec/jdbc/src/main/java/org/apache/drill/jdbc/impl/DrillCursor.java ---
    @@ -139,8 +147,22 @@ private boolean stopThrottlingIfSo() {
           return stopped;
         }
     
    -    public void awaitFirstMessage() throws InterruptedException {
    -      firstMessageReceived.await();
    +    public void awaitFirstMessage() throws InterruptedException, SQLTimeoutException {
    +      //Check if a non-zero timeout has been set
    +      if ( parent.timeoutInMilliseconds > 0 ) {
    +        //Identifying remaining in milliseconds to maintain a granularity close to integer value of timeout
    +        long timeToTimeout = (parent.timeoutInMilliseconds) - parent.elapsedTimer.elapsed(TimeUnit.MILLISECONDS);
    +        if ( timeToTimeout > 0 ) {
    --- End diff --
    
    maybe a style issue, but to avoid code duplication both conditions could be checked together?
    ```
    if ( timeToTimeout <= 0 || !firstMessageReceived.await(timeToTimeout, TimeUnit.MILLISECONDS) ) {
      throw new SqlTimeoutException(TimeUnit.MILLISECONDS.toSeconds(parent.timeoutInMilliseconds));
    }
    ```


---

[GitHub] drill pull request #1024: 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/1024#discussion_r149276882
  
    --- Diff: exec/jdbc/src/main/java/org/apache/drill/jdbc/impl/DrillCursor.java ---
    @@ -239,6 +259,11 @@ QueryDataBatch getNext() throws UserException, InterruptedException {
                 }
                 return qdb;
               }
    +
    +          // Check and throw SQLTimeoutException
    +          if ( parent.timeoutInSeconds > 0 && parent.elapsedTimer.elapsed(TimeUnit.SECONDS) >= parent.timeoutInSeconds ) {
    --- End diff --
    
    maybe do it sooner (before pooling)? you may also want to limit the amount of time you spent pooling for batchQueue (if less than 50ms left)


---

[GitHub] drill pull request #1024: 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/1024#discussion_r149278215
  
    --- Diff: exec/jdbc/src/test/java/org/apache/drill/jdbc/PreparedStatementTest.java ---
    @@ -237,6 +245,127 @@ public String toString() {
         }
       }
     
    +  /**
    +   * Test for reading of default query timeout
    +   */
    +  @Test
    +  public void testDefaultGetQueryTimeout() throws SQLException {
    +    PreparedStatement stmt = connection.prepareStatement(SYS_VERSION_SQL);
    +    int timeoutValue = stmt.getQueryTimeout();
    +    assert( 0 == timeoutValue );
    +  }
    +
    +  /**
    +   * Test Invalid parameter by giving negative timeout
    +   */
    +  @Test ( expected = InvalidParameterSqlException.class )
    +  public void testInvalidSetQueryTimeout() throws SQLException {
    +    PreparedStatement stmt = connection.prepareStatement(SYS_VERSION_SQL);
    +    //Setting negative value
    +    int valueToSet = -10;
    +    if (0L == valueToSet) {
    +      valueToSet--;
    +    }
    +    try {
    +      stmt.setQueryTimeout(valueToSet);
    +    } catch ( final Exception e) {
    +      // TODO: handle exception
    +      assertThat( e.getMessage(), containsString( "illegal timeout value") );
    +      //Converting this to match expected Exception
    +      throw new InvalidParameterSqlException(e.getMessage());
    +    }
    +  }
    +
    +  /**
    +   * Test setting a valid timeout
    +   */
    +  @Test
    +  public void testValidSetQueryTimeout() throws SQLException {
    +    PreparedStatement stmt = connection.prepareStatement(SYS_VERSION_SQL);
    +    //Setting positive value
    +    int valueToSet = new Random(System.currentTimeMillis()).nextInt(60);
    --- End diff --
    
    or you can do `new Random().nextInt(59) +1`. You should probably always use the same seed, so it would always be the same random number, and the test would be reproducible (but I guess you could simply choose one value by dice and hardcode it too...)


---

[GitHub] drill pull request #1024: 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/1024#discussion_r149476642
  
    --- Diff: exec/jdbc/src/main/java/org/apache/drill/jdbc/impl/DrillPreparedStatementImpl.java ---
    @@ -61,8 +65,14 @@ protected DrillPreparedStatementImpl(DrillConnectionImpl connection,
         if (preparedStatementHandle != null) {
           ((DrillColumnMetaDataList) signature.columns).updateColumnMetaData(preparedStatementHandle.getColumnsList());
         }
    +    //Implicit query timeout
    +    this.queryTimeoutInSeconds = 0;
    +    this.elapsedTimer = Stopwatch.createUnstarted();
    --- End diff --
    
    not even true for a statement: you can execute multiple queries, but the previous resultset will be closed and a new cursor created...


---

[GitHub] drill pull request #1024: 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/1024#discussion_r149473521
  
    --- Diff: exec/jdbc/src/main/java/org/apache/drill/jdbc/impl/DrillResultSetImpl.java ---
    @@ -66,11 +70,27 @@
       private final DrillConnectionImpl connection;
       private volatile boolean hasPendingCancelationNotification = false;
     
    +  private Stopwatch elapsedTimer;
    +
    +  private int queryTimeoutInSeconds;
    +
       DrillResultSetImpl(AvaticaStatement statement, Meta.Signature signature,
                          ResultSetMetaData resultSetMetaData, TimeZone timeZone,
                          Meta.Frame firstFrame) {
         super(statement, signature, resultSetMetaData, timeZone, firstFrame);
         connection = (DrillConnectionImpl) statement.getConnection();
    +    try {
    +      if (statement.getQueryTimeout() > 0) {
    +        queryTimeoutInSeconds = statement.getQueryTimeout();
    +      }
    +    } catch (Exception e) {
    +      e.printStackTrace();
    --- End diff --
    
    Guess I was not sure what am I to do if `getQueryTImeout()` threw an Exception. Didn't want to lose the stack trace. Should I just ignore it?


---

[GitHub] drill pull request #1024: 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/1024#discussion_r149546105
  
    --- Diff: exec/jdbc/src/main/java/org/apache/drill/jdbc/impl/DrillCursor.java ---
    @@ -260,6 +288,10 @@ void close() {
           // when the main thread is blocked waiting for the result.  In that case
           // we want to unblock the main thread.
           firstMessageReceived.countDown(); // TODO:  Why not call releaseIfFirst as used elsewhere?
    +      //Stopping timeout clock
    --- End diff --
    
    Nothing is actually running in Stopwatch (it's just a state to indicate if elapsed time should use the current time or the time when Stopwatch was stopped...)


---

[GitHub] drill issue #1024: 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/1024
  
    @laurentgo 
    I've added server-triggered timeout tests and made other changes as well, but they require support for [DRILL-5973](https://issues.apache.org/jira/browse/DRILL-5973) . I tested this commit (#1024 ) as a cherry pick on top of that PR's commit (#1055) and I was able to simulate the server-induced timeout.
    Will need a +1 for that PR before I can enable the tests here.
    For now, I've marked these tests as `@ignore` to ensure that the remaining tests pass and the feature works as intended. 
    
    Can you review them both (this and #1055 )?


---

[GitHub] drill pull request #1024: 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/1024#discussion_r149475115
  
    --- Diff: exec/jdbc/src/test/java/org/apache/drill/jdbc/PreparedStatementTest.java ---
    @@ -237,6 +245,127 @@ public String toString() {
         }
       }
     
    +  /**
    +   * Test for reading of default query timeout
    +   */
    +  @Test
    +  public void testDefaultGetQueryTimeout() throws SQLException {
    +    PreparedStatement stmt = connection.prepareStatement(SYS_VERSION_SQL);
    +    int timeoutValue = stmt.getQueryTimeout();
    +    assert( 0 == timeoutValue );
    +  }
    +
    +  /**
    +   * Test Invalid parameter by giving negative timeout
    +   */
    +  @Test ( expected = InvalidParameterSqlException.class )
    +  public void testInvalidSetQueryTimeout() throws SQLException {
    +    PreparedStatement stmt = connection.prepareStatement(SYS_VERSION_SQL);
    +    //Setting negative value
    +    int valueToSet = -10;
    +    if (0L == valueToSet) {
    +      valueToSet--;
    +    }
    +    try {
    +      stmt.setQueryTimeout(valueToSet);
    +    } catch ( final Exception e) {
    --- End diff --
    
    Yes, it should be. Might be a legacy code. Will fix it. +1


---

[GitHub] drill pull request #1024: 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/1024#discussion_r149471249
  
    --- Diff: exec/jdbc/src/main/java/org/apache/drill/jdbc/impl/DrillCursor.java ---
    @@ -100,13 +103,17 @@
         final LinkedBlockingDeque<QueryDataBatch> batchQueue =
             Queues.newLinkedBlockingDeque();
     
    +    private final DrillCursor parent;
    --- End diff --
    
    Stopwatch seemed a convenient way of visualizing a timer object that is passed between different JDBC entities, and also provides a clean way of specifying elapsed time, etc. 


---

[GitHub] drill pull request #1024: 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/1024#discussion_r149477233
  
    --- Diff: exec/jdbc/src/test/java/org/apache/drill/jdbc/StatementTest.java ---
    @@ -61,55 +71,129 @@ public static void tearDownStatement() throws SQLException {
       //////////
       // getQueryTimeout():
     
    -  /** Tests that getQueryTimeout() indicates no timeout set. */
    +  /**
    +   * Test for reading of default query timeout
    +   */
       @Test
    -  public void testGetQueryTimeoutSaysNoTimeout() throws SQLException {
    -    assertThat( statement.getQueryTimeout(), equalTo( 0 ) );
    +  public void testDefaultGetQueryTimeout() throws SQLException {
    +    Statement stmt = connection.createStatement();
    +    int timeoutValue = stmt.getQueryTimeout();
    +    assert( 0 == timeoutValue );
    --- End diff --
    
    +1 


---

[GitHub] drill pull request #1024: 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/1024#discussion_r149278003
  
    --- Diff: exec/jdbc/src/test/java/org/apache/drill/jdbc/PreparedStatementTest.java ---
    @@ -237,6 +245,127 @@ public String toString() {
         }
       }
     
    +  /**
    +   * Test for reading of default query timeout
    +   */
    +  @Test
    +  public void testDefaultGetQueryTimeout() throws SQLException {
    +    PreparedStatement stmt = connection.prepareStatement(SYS_VERSION_SQL);
    +    int timeoutValue = stmt.getQueryTimeout();
    +    assert( 0 == timeoutValue );
    +  }
    +
    +  /**
    +   * Test Invalid parameter by giving negative timeout
    +   */
    +  @Test ( expected = InvalidParameterSqlException.class )
    +  public void testInvalidSetQueryTimeout() throws SQLException {
    +    PreparedStatement stmt = connection.prepareStatement(SYS_VERSION_SQL);
    +    //Setting negative value
    +    int valueToSet = -10;
    +    if (0L == valueToSet) {
    +      valueToSet--;
    +    }
    +    try {
    +      stmt.setQueryTimeout(valueToSet);
    +    } catch ( final Exception e) {
    --- End diff --
    
    shouldn't be a `SQLException`?


---

[GitHub] drill pull request #1024: 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/1024#discussion_r149472887
  
    --- Diff: exec/jdbc/src/main/java/org/apache/drill/jdbc/impl/DrillResultSetImpl.java ---
    @@ -66,11 +70,27 @@
       private final DrillConnectionImpl connection;
       private volatile boolean hasPendingCancelationNotification = false;
     
    +  private Stopwatch elapsedTimer;
    --- End diff --
    
    +1


---

[GitHub] drill pull request #1024: 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/1024#discussion_r149474455
  
    --- Diff: exec/jdbc/src/test/java/org/apache/drill/jdbc/PreparedStatementTest.java ---
    @@ -237,6 +245,127 @@ public String toString() {
         }
       }
     
    +  /**
    +   * Test for reading of default query timeout
    +   */
    +  @Test
    +  public void testDefaultGetQueryTimeout() throws SQLException {
    +    PreparedStatement stmt = connection.prepareStatement(SYS_VERSION_SQL);
    --- End diff --
    
    +1


---

[GitHub] drill pull request #1024: 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/1024#discussion_r151580948
  
    --- Diff: exec/jdbc/src/main/java/org/apache/drill/jdbc/impl/DrillResultSetImpl.java ---
    @@ -96,6 +105,14 @@ private void throwIfClosed() throws AlreadyClosedSqlException,
             throw new AlreadyClosedSqlException( "ResultSet is already closed." );
           }
         }
    +
    +    //Implicit check for whether timeout is set
    +    if (elapsedTimer != null) {
    --- End diff --
    
    I've logged https://issues.apache.org/jira/browse/DRILL-5973 to support a time-bound pause on the server. That should help resolve this test, which otherwise passes... but just doesn't clean up gracefully.
    I'll make the required changes and update this PR.


---

[GitHub] drill pull request #1024: 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/1024#discussion_r149506412
  
    --- Diff: exec/jdbc/src/test/java/org/apache/drill/jdbc/PreparedStatementTest.java ---
    @@ -237,6 +245,127 @@ public String toString() {
         }
       }
     
    +  /**
    +   * Test for reading of default query timeout
    +   */
    +  @Test
    +  public void testDefaultGetQueryTimeout() throws SQLException {
    +    PreparedStatement stmt = connection.prepareStatement(SYS_VERSION_SQL);
    +    int timeoutValue = stmt.getQueryTimeout();
    +    assert( 0 == timeoutValue );
    +  }
    +
    +  /**
    +   * Test Invalid parameter by giving negative timeout
    +   */
    +  @Test ( expected = InvalidParameterSqlException.class )
    +  public void testInvalidSetQueryTimeout() throws SQLException {
    +    PreparedStatement stmt = connection.prepareStatement(SYS_VERSION_SQL);
    +    //Setting negative value
    +    int valueToSet = -10;
    +    if (0L == valueToSet) {
    +      valueToSet--;
    +    }
    +    try {
    +      stmt.setQueryTimeout(valueToSet);
    +    } catch ( final Exception e) {
    +      // TODO: handle exception
    +      assertThat( e.getMessage(), containsString( "illegal timeout value") );
    +      //Converting this to match expected Exception
    +      throw new InvalidParameterSqlException(e.getMessage());
    --- End diff --
    
    Yep. Agree. Was trying to make use of the large number of `###SqlException`s defined within the Drill JDBC package. Will fix this. +1


---

[GitHub] drill pull request #1024: 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/1024#discussion_r151286559
  
    --- Diff: exec/jdbc/src/main/java/org/apache/drill/jdbc/impl/DrillResultSetImpl.java ---
    @@ -96,6 +105,14 @@ private void throwIfClosed() throws AlreadyClosedSqlException,
             throw new AlreadyClosedSqlException( "ResultSet is already closed." );
           }
         }
    +
    +    //Implicit check for whether timeout is set
    +    if (elapsedTimer != null) {
    --- End diff --
    
    So, I used your approach and made a modification to injecting the pause for `sending-data` () in the `ScreenCreator` class.
    ```
    org.apache.drill.jdbc.SqlTimeoutException: Query timed out in 2 seconds
            at org.apache.drill.jdbc.impl.DrillCursor$ResultsListener.awaitFirstMessage(DrillCursor.java:156)
            at org.apache.drill.jdbc.impl.DrillCursor.loadInitialSchema(DrillCursor.java:588)
            at org.apache.drill.jdbc.impl.DrillResultSetImpl.execute(DrillResultSetImpl.java:1917)
            at org.apache.drill.jdbc.impl.DrillResultSetImpl.execute(DrillResultSetImpl.java:65)
            at org.apache.calcite.avatica.AvaticaConnection$1.execute(AvaticaConnection.java:473)
            at org.apache.drill.jdbc.impl.DrillMetaImpl.prepareAndExecute(DrillMetaImpl.java:1100)
            at org.apache.calcite.avatica.AvaticaConnection.prepareAndExecuteInternal(AvaticaConnection.java:477)
            at org.apache.drill.jdbc.impl.DrillConnectionImpl.prepareAndExecuteInternal(DrillConnectionImpl.java:191)
            at org.apache.calcite.avatica.AvaticaStatement.executeInternal(AvaticaStatement.java:110)
            at org.apache.calcite.avatica.AvaticaStatement.executeQuery(AvaticaStatement.java:130)
            at org.apache.drill.jdbc.impl.DrillStatementImpl.executeQuery(DrillStatementImpl.java:112)
            at org.apache.drill.jdbc.StatementTest.testServerTriggeredQueryTimeout(StatementTest.java:206)
    ```
    
    However, since the executeQuery() call failed, I don't have a queryId to unpause at the injection.
    Closing the (test-specific) connection did not help release back memory.
    ```
    15:43:41.818 [Thread-8] ERROR o.a.d.exec.server.BootStrapContext - Pool did not terminate
    15:43:41.836 [Thread-8] ERROR o.a.d.exec.server.BootStrapContext - Error while closing
    java.lang.IllegalStateException: Allocator[ROOT] closed with outstanding child allocators.
    Allocator(ROOT) 0/30180480/33376320/3221225472 (res/actual/peak/limit)
      child allocators: 2
        Allocator(query:25f32edb-db10-4f8a-6a9c-58b0b455c0ab) 1048576/0/0/268435456 (res/actual/peak/limit)
    ....
            at org.apache.drill.exec.memory.BaseAllocator.close(BaseAllocator.java:496) ~[drill-memory-base-1.12.0-SNAPSHOT.jar:1.12.0-SNAPSHOT]
            at org.apache.drill.common.AutoCloseables.close(AutoCloseables.java:76) [drill-common-1.12.0-SNAPSHOT.jar:1.12.0-SNAPSHOT]
            at org.apache.drill.common.AutoCloseables.close(AutoCloseables.java:64) [drill-common-1.12.0-SNAPSHOT.jar:1.12.0-SNAPSHOT]
            at org.apache.drill.exec.server.BootStrapContext.close(BootStrapContext.java:256) ~[drill-java-exec-1.12.0-SNAPSHOT.jar:1.12.0-SNAPSHOT]
            at org.apache.drill.common.AutoCloseables.close(AutoCloseables.java:76) [drill-common-1.12.0-SNAPSHOT.jar:1.12.0-SNAPSHOT]
            at org.apache.drill.common.AutoCloseables.close(AutoCloseables.java:64) [drill-common-1.12.0-SNAPSHOT.jar:1.12.0-SNAPSHOT]
            at org.apache.drill.exec.server.Drillbit.close(Drillbit.java:205) [drill-java-exec-1.12.0-SNAPSHOT.jar:1.12.0-SNAPSHOT]
            at org.apache.drill.jdbc.impl.DrillConnectionImpl.cleanup(DrillConnectionImpl.java:817) [classes/:na]
            at org.apache.drill.jdbc.impl.DrillHandler.onConnectionClose(DrillHandler.java:36) [classes/:na]
            at org.apache.calcite.avatica.AvaticaConnection.close(AvaticaConnection.java:156) [calcite-avatica-1.4.0-drill-r23.jar:1.4.0-drill-r23]
            at org.apache.drill.jdbc.StatementTest.testServerTriggeredQueryTimeout(StatementTest.java:225) [test-classes/:na]
    ```
    
    Any idea on how I can _resume_ such a query without the queryId?


---

[GitHub] drill pull request #1024: 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/1024#discussion_r149475535
  
    --- Diff: exec/jdbc/src/test/java/org/apache/drill/jdbc/PreparedStatementTest.java ---
    @@ -237,6 +245,127 @@ public String toString() {
         }
       }
     
    +  /**
    +   * Test for reading of default query timeout
    +   */
    +  @Test
    +  public void testDefaultGetQueryTimeout() throws SQLException {
    +    PreparedStatement stmt = connection.prepareStatement(SYS_VERSION_SQL);
    +    int timeoutValue = stmt.getQueryTimeout();
    +    assert( 0 == timeoutValue );
    +  }
    +
    +  /**
    +   * Test Invalid parameter by giving negative timeout
    +   */
    +  @Test ( expected = InvalidParameterSqlException.class )
    +  public void testInvalidSetQueryTimeout() throws SQLException {
    +    PreparedStatement stmt = connection.prepareStatement(SYS_VERSION_SQL);
    +    //Setting negative value
    +    int valueToSet = -10;
    +    if (0L == valueToSet) {
    +      valueToSet--;
    +    }
    +    try {
    +      stmt.setQueryTimeout(valueToSet);
    +    } catch ( final Exception e) {
    +      // TODO: handle exception
    +      assertThat( e.getMessage(), containsString( "illegal timeout value") );
    +      //Converting this to match expected Exception
    +      throw new InvalidParameterSqlException(e.getMessage());
    --- End diff --
    
    Wanted to make sure that the unit test also reports the correct exception. 
    This only rewraps the thrown SQLException to an InvalidParameterSqlException for JUnit to confirm.


---

[GitHub] drill pull request #1024: 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/1024#discussion_r149277380
  
    --- Diff: exec/jdbc/src/main/java/org/apache/drill/jdbc/impl/DrillResultSetImpl.java ---
    @@ -96,6 +117,13 @@ private void throwIfClosed() throws AlreadyClosedSqlException,
             throw new AlreadyClosedSqlException( "ResultSet is already closed." );
           }
         }
    +
    +    //Query Timeout Check. The timer has already been started by the DrillCursor at this point
    --- End diff --
    
    not sure if necessary: cursor would throw the timeout anyway, no?


---

[GitHub] drill pull request #1024: 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/1024#discussion_r149474798
  
    --- Diff: exec/jdbc/src/test/java/org/apache/drill/jdbc/PreparedStatementTest.java ---
    @@ -237,6 +245,127 @@ public String toString() {
         }
       }
     
    +  /**
    +   * Test for reading of default query timeout
    +   */
    +  @Test
    +  public void testDefaultGetQueryTimeout() throws SQLException {
    +    PreparedStatement stmt = connection.prepareStatement(SYS_VERSION_SQL);
    +    int timeoutValue = stmt.getQueryTimeout();
    +    assert( 0 == timeoutValue );
    +  }
    +
    +  /**
    +   * Test Invalid parameter by giving negative timeout
    +   */
    +  @Test ( expected = InvalidParameterSqlException.class )
    +  public void testInvalidSetQueryTimeout() throws SQLException {
    +    PreparedStatement stmt = connection.prepareStatement(SYS_VERSION_SQL);
    +    //Setting negative value
    +    int valueToSet = -10;
    +    if (0L == valueToSet) {
    --- End diff --
    
    My bad. The original code would assign the negation of a random integer.,.. hence the check for 0L and followed by a decrement. +1


---

[GitHub] drill pull request #1024: 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/1024#discussion_r149543163
  
    --- Diff: exec/jdbc/src/main/java/org/apache/drill/jdbc/impl/DrillCursor.java ---
    @@ -376,6 +417,19 @@ synchronized void cleanup() {
         currentBatchHolder.clear();
       }
     
    +  long getTimeoutInMilliseconds() {
    +    return timeoutInMilliseconds;
    +  }
    +
    +  //Set the cursor's timeout in seconds
    +  void setTimeout(int timeoutDurationInSeconds){
    +    this.timeoutInMilliseconds = timeoutDurationInSeconds*1000L;
    --- End diff --
    
    Preferably use `TimeUnit.SECONDS.toMillis(timeoutDurationInSeconds)` to avoid magic constants


---

[GitHub] drill pull request #1024: 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/1024#discussion_r149473999
  
    --- Diff: exec/jdbc/src/main/java/org/apache/drill/jdbc/impl/DrillResultSetImpl.java ---
    @@ -96,6 +117,13 @@ private void throwIfClosed() throws AlreadyClosedSqlException,
             throw new AlreadyClosedSqlException( "ResultSet is already closed." );
           }
         }
    +
    +    //Query Timeout Check. The timer has already been started by the DrillCursor at this point
    --- End diff --
    
    This code block gets touched even if there is no timeout set, hence the check to implicitly confirm if there is a timeout set.


---

[GitHub] drill pull request #1024: 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/1024#discussion_r150119338
  
    --- Diff: exec/jdbc/src/main/java/org/apache/drill/jdbc/impl/DrillResultSetImpl.java ---
    @@ -96,6 +105,14 @@ private void throwIfClosed() throws AlreadyClosedSqlException,
             throw new AlreadyClosedSqlException( "ResultSet is already closed." );
           }
         }
    +
    +    //Implicit check for whether timeout is set
    +    if (elapsedTimer != null) {
    --- End diff --
    
    I wonder if we actually test timeout during DrillCursor operations. It seems your test relies on the user being slow to read data from the result set although the data has already been fetched by the client. Am I wrong?


---

[GitHub] drill pull request #1024: 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/1024#discussion_r149278719
  
    --- Diff: exec/jdbc/src/test/java/org/apache/drill/jdbc/StatementTest.java ---
    @@ -61,55 +71,129 @@ public static void tearDownStatement() throws SQLException {
       //////////
       // getQueryTimeout():
     
    -  /** Tests that getQueryTimeout() indicates no timeout set. */
    +  /**
    +   * Test for reading of default query timeout
    +   */
       @Test
    -  public void testGetQueryTimeoutSaysNoTimeout() throws SQLException {
    -    assertThat( statement.getQueryTimeout(), equalTo( 0 ) );
    +  public void testDefaultGetQueryTimeout() throws SQLException {
    +    Statement stmt = connection.createStatement();
    --- End diff --
    
    same comment as for `PreparedStatementTest` regarding closing statement


---

[GitHub] drill pull request #1024: 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/1024#discussion_r149545534
  
    --- Diff: exec/jdbc/src/main/java/org/apache/drill/jdbc/impl/DrillCursor.java ---
    @@ -239,6 +261,11 @@ QueryDataBatch getNext() throws UserException, InterruptedException {
                 }
                 return qdb;
               }
    +
    +          // Check and throw SQLTimeoutException
    +          if ( parent.timeoutInMilliseconds > 0 && parent.elapsedTimer.elapsed(TimeUnit.SECONDS) >= parent.timeoutInMilliseconds ) {
    --- End diff --
    
    Darn! +1


---

[GitHub] drill pull request #1024: 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/1024#discussion_r149477955
  
    --- Diff: exec/jdbc/src/main/java/org/apache/drill/jdbc/impl/DrillCursor.java ---
    @@ -376,6 +415,19 @@ synchronized void cleanup() {
         currentBatchHolder.clear();
       }
     
    +  //Set the cursor's timeout in seconds
    --- End diff --
    
    you just need to get the value when the query is executed (in DrillCursor) once to make sure the timeout doesn't change (that and StopWatch being managed by DrillCursor too.
    
    Also, it is subject to interpretation but it seems the intent of the API is to time bound how much time it takes the query to complete. I'm not sure it is necessary to make the extra work of having a slow client reading the result set data although all data has already been read by the driver from the server (and from the server point of view, the query is completed).


---