You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@drill.apache.org by pa...@apache.org on 2015/05/06 07:26:18 UTC

[2/9] drill git commit: DRILL-2884: 1-Core: Throw "canceled" (not "ResultSet closed") after cancel().

DRILL-2884: 1-Core: Throw "canceled" (not "ResultSet closed") after cancel().

Changed DrillResultSetImpl to throw new ExecutionCanceledSqlException, rather
than (previously existing) AlreadyClosedSqlException, at the first call to the
result set after the associated SQL statement's execution is canceled (except
that currently only some methods check and throw those exceptions).

Also fixed two little documentation errors in DrillResultSet.


Project: http://git-wip-us.apache.org/repos/asf/drill/repo
Commit: http://git-wip-us.apache.org/repos/asf/drill/commit/f9efc3bc
Tree: http://git-wip-us.apache.org/repos/asf/drill/tree/f9efc3bc
Diff: http://git-wip-us.apache.org/repos/asf/drill/diff/f9efc3bc

Branch: refs/heads/master
Commit: f9efc3bc21a4aadb28a9033e142b37d57684125d
Parents: 5769278
Author: dbarclay <db...@maprtech.com>
Authored: Mon Apr 27 10:03:32 2015 -0700
Committer: Parth Chandra <pc...@maprtech.com>
Committed: Tue May 5 17:40:13 2015 -0700

----------------------------------------------------------------------
 .../org/apache/drill/jdbc/DrillResultSet.java   |  4 +-
 .../org/apache/drill/jdbc/DrillStatement.java   |  2 +-
 .../jdbc/ExecutionCanceledSqlException.java     | 44 ++++++++++++++++++++
 .../drill/jdbc/impl/DrillResultSetImpl.java     | 22 +++++++---
 4 files changed, 64 insertions(+), 8 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/drill/blob/f9efc3bc/exec/jdbc/src/main/java/org/apache/drill/jdbc/DrillResultSet.java
----------------------------------------------------------------------
diff --git a/exec/jdbc/src/main/java/org/apache/drill/jdbc/DrillResultSet.java b/exec/jdbc/src/main/java/org/apache/drill/jdbc/DrillResultSet.java
index 7a4f426..2e3d49f 100644
--- a/exec/jdbc/src/main/java/org/apache/drill/jdbc/DrillResultSet.java
+++ b/exec/jdbc/src/main/java/org/apache/drill/jdbc/DrillResultSet.java
@@ -108,7 +108,7 @@ public interface DrillResultSet extends ResultSet  {
    * </ul>
    * <p>
    *   Conversion throws {@link SQLConversionOverflowException} for a source
-   *   value whose magnitude is outside the range of {@code short} values.
+   *   value whose magnitude is outside the range of {@code byte} values.
    * </p>
    * @throws  SQLConversionOverflowException  if a source value was too large
    *   to convert
@@ -135,7 +135,7 @@ public interface DrillResultSet extends ResultSet  {
    * </ul>
    * <p>
    *   Conversion throws {@link SQLConversionOverflowException} for a source
-   *   value whose magnitude is outside the range of {@code int} values.
+   *   value whose magnitude is outside the range of {@code short} values.
    * </p>
    * @throws  SQLConversionOverflowException  if a source value was too large
    *   to convert

http://git-wip-us.apache.org/repos/asf/drill/blob/f9efc3bc/exec/jdbc/src/main/java/org/apache/drill/jdbc/DrillStatement.java
----------------------------------------------------------------------
diff --git a/exec/jdbc/src/main/java/org/apache/drill/jdbc/DrillStatement.java b/exec/jdbc/src/main/java/org/apache/drill/jdbc/DrillStatement.java
index ba265e6..7fc79be 100644
--- a/exec/jdbc/src/main/java/org/apache/drill/jdbc/DrillStatement.java
+++ b/exec/jdbc/src/main/java/org/apache/drill/jdbc/DrillStatement.java
@@ -55,7 +55,7 @@ public abstract class DrillStatement extends AvaticaStatement
   }
 
   @Override
-  public ResultSet  executeQuery( String sql ) throws SQLException {
+  public ResultSet executeQuery( String sql ) throws SQLException {
     checkNotClosed();
     return super.executeQuery( sql );
   }

http://git-wip-us.apache.org/repos/asf/drill/blob/f9efc3bc/exec/jdbc/src/main/java/org/apache/drill/jdbc/ExecutionCanceledSqlException.java
----------------------------------------------------------------------
diff --git a/exec/jdbc/src/main/java/org/apache/drill/jdbc/ExecutionCanceledSqlException.java b/exec/jdbc/src/main/java/org/apache/drill/jdbc/ExecutionCanceledSqlException.java
new file mode 100644
index 0000000..edd1bde
--- /dev/null
+++ b/exec/jdbc/src/main/java/org/apache/drill/jdbc/ExecutionCanceledSqlException.java
@@ -0,0 +1,44 @@
+/*
+ * 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;
+
+import java.beans.Statement;
+import java.sql.ResultSet;
+import java.sql.SQLNonTransientException;
+
+
+/**
+ * SQLException for execution-canceled condition.
+ * <p>
+ *   Intended for throwing by methods that start SQL statement execution (e.g.,
+ *   {@link Statement#execute}) or return data from execution (e.g.,
+ *   {@link ResultSet} methods).
+ * </p>
+ */
+public class ExecutionCanceledSqlException extends SQLNonTransientException {
+
+  private static final long serialVersionUID = 2015_04_23L;
+
+  /**
+   * See {@link SQLNonTransientException#SQLNonTransientException(String)}.
+   */
+  public ExecutionCanceledSqlException( String reason ) {
+    super( reason );
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/f9efc3bc/exec/jdbc/src/main/java/org/apache/drill/jdbc/impl/DrillResultSetImpl.java
----------------------------------------------------------------------
diff --git a/exec/jdbc/src/main/java/org/apache/drill/jdbc/impl/DrillResultSetImpl.java b/exec/jdbc/src/main/java/org/apache/drill/jdbc/impl/DrillResultSetImpl.java
index 484a5e5..35674bf 100644
--- a/exec/jdbc/src/main/java/org/apache/drill/jdbc/impl/DrillResultSetImpl.java
+++ b/exec/jdbc/src/main/java/org/apache/drill/jdbc/impl/DrillResultSetImpl.java
@@ -44,6 +44,7 @@ import org.apache.drill.jdbc.DrillConnection;
 import org.apache.drill.jdbc.DrillConnectionImpl;
 import org.apache.drill.jdbc.DrillCursor;
 import org.apache.drill.jdbc.DrillResultSet;
+import org.apache.drill.jdbc.ExecutionCanceledSqlException;
 import org.apache.drill.jdbc.SchemaChangeListener;
 
 import com.google.common.collect.Queues;
@@ -67,6 +68,7 @@ public class DrillResultSetImpl extends AvaticaResultSet implements DrillResultS
   public final RecordBatchLoader currentBatch;
   // (Public until JDBC impl. classes moved out of published-intf. package. (DRILL-2089).)
   public final DrillCursor cursor;
+  public boolean hasPendingCancelationNotification;
 
   public DrillResultSetImpl(AvaticaStatement statement, AvaticaPrepareResult prepareResult,
       ResultSetMetaData resultSetMetaData, TimeZone timeZone) {
@@ -80,14 +82,25 @@ public class DrillResultSetImpl extends AvaticaResultSet implements DrillResultS
   }
 
   /**
-   * Throws AlreadyClosedSqlException if this ResultSet is closed.
+   * Throws AlreadyClosedSqlException or QueryCanceledSqlException if this
+   * ResultSet is closed.
    *
-   * @throws AlreadyClosedSqlException if ResultSet is closed
+   * @throws  ExecutionCanceledSqlException  if ResultSet is closed because of
+   *          cancelation and no QueryCanceledSqlException had been thrown yet
+   *          for this ResultSet
+   * @throws  AlreadyClosedSqlException  if ResultSet is closed
    * @throws SQLException if error in calling {@link #isClosed()}
    */
   private void checkNotClosed() throws SQLException {
     if ( isClosed() ) {
-      throw new AlreadyClosedSqlException( "ResultSet is already closed." );
+      if ( hasPendingCancelationNotification ) {
+        hasPendingCancelationNotification = false;
+        throw new ExecutionCanceledSqlException(
+            "SQL statement execution canceled; resultSet closed." );
+      }
+      else {
+        throw new AlreadyClosedSqlException( "ResultSet is already closed." );
+      }
     }
   }
 
@@ -100,6 +113,7 @@ public class DrillResultSetImpl extends AvaticaResultSet implements DrillResultS
 
   @Override
   protected void cancel() {
+    hasPendingCancelationNotification = true;
     cleanup();
     close();
   }
@@ -128,8 +142,6 @@ public class DrillResultSetImpl extends AvaticaResultSet implements DrillResultS
 
   @Override
   protected DrillResultSetImpl execute() throws SQLException{
-    checkNotClosed();
-    // Call driver's callback. It is permitted to throw a RuntimeException.
     DrillConnectionImpl connection = (DrillConnectionImpl) statement.getConnection();
 
     connection.getClient().runQuery(QueryType.SQL, this.prepareResult.getSql(),