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/11 09:09:05 UTC

[1/2] drill git commit: DRILL-2961: Part 2--Core: Have set...Timeout methods throw rather than ignore.

Repository: drill
Updated Branches:
  refs/heads/master 84371eaee -> a3ec52a72


DRILL-2961: Part 2--Core: Have set...Timeout methods throw rather than ignore.

Changed implementation of Statement.setQueryTimeout(...) and
Connection.setNetworkTimeout(...) to throw SQLFeatureNotSupportedException
rather than silently not setting timeout.

Main:
- Added implementations of Statement.{set,get}QueryTimeout(...) and
  Connection.{set,get}NetworkTimeout(...) methods (in classes
  DrillStatementImpl, DrillConnectionImpl).
- Added Drill-specific narrowed declarations and documentation (in
  interfaces DrillStatement and DrillConnection).
- Added unit tests (in new StatementTest, ConnectionTest).

Narrowed other thrown exceptions to allow narrowing in above methods:
- Narrowed exceptions on isClosed() and checkNotClosed() (in
  DrillConnectionImpl, DrillStatementImpl).
- Added InvalidParameterSqlException.


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

Branch: refs/heads/master
Commit: a3ec52a721860a966dfa351f719458a200b27cbf
Parents: 1dcf6cf
Author: dbarclay <db...@maprtech.com>
Authored: Tue May 5 23:12:32 2015 -0700
Committer: Parth Chandra <pa...@apache.org>
Committed: Sun May 10 23:47:17 2015 -0700

----------------------------------------------------------------------
 .../org/apache/drill/jdbc/DrillConnection.java  |  40 ++++++
 .../apache/drill/jdbc/DrillConnectionImpl.java  |  59 ++++++++-
 .../org/apache/drill/jdbc/DrillStatement.java   |  38 ++++++
 .../jdbc/InvalidParameterSqlException.java      |  86 ++++++++++++
 .../drill/jdbc/impl/DrillStatementImpl.java     |  51 ++++++-
 .../org/apache/drill/jdbc/ConnectionTest.java   | 132 +++++++++++++++++++
 .../org/apache/drill/jdbc/StatementTest.java    | 115 ++++++++++++++++
 7 files changed, 512 insertions(+), 9 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/drill/blob/a3ec52a7/exec/jdbc/src/main/java/org/apache/drill/jdbc/DrillConnection.java
----------------------------------------------------------------------
diff --git a/exec/jdbc/src/main/java/org/apache/drill/jdbc/DrillConnection.java b/exec/jdbc/src/main/java/org/apache/drill/jdbc/DrillConnection.java
index 33acb42..d64f980 100644
--- a/exec/jdbc/src/main/java/org/apache/drill/jdbc/DrillConnection.java
+++ b/exec/jdbc/src/main/java/org/apache/drill/jdbc/DrillConnection.java
@@ -22,6 +22,7 @@ import java.sql.Connection;
 import java.sql.SQLException;
 import java.sql.SQLFeatureNotSupportedException;
 import java.sql.Savepoint;
+import java.util.concurrent.Executor;
 
 import org.apache.drill.exec.client.DrillClient;
 
@@ -95,6 +96,16 @@ public interface DrillConnection extends Connection {
   /**
    * {@inheritDoc}
    * <p>
+   *   <strong>Drill</strong>: Does not throw SQLException.
+   * </p>
+   */
+  @Override
+  boolean isClosed();
+
+
+  /**
+   * {@inheritDoc}
+   * <p>
    *   <strong>Drill</strong>:
    *   Accepts only {@link Connection.TRANSACTION_NONE}.
    * </p>
@@ -153,6 +164,35 @@ public interface DrillConnection extends Connection {
   String getSchema() throws SQLException;
 
 
+  /**
+   * Not supported (for non-zero timeout value).
+   * <p>
+   *   Normally, just throws {@link SQLFeatureNotSupportedException} unless
+   *   request is trivially for no timeout (zero {@code milliseconds} value).
+   * </p>
+   * @throws  AlreadyClosedSqlException
+   *            if connection is closed
+   * @throws  JdbcApiSqlException
+   *            if an invalid parameter value is detected (and not above case)
+   * @throws  SQLFeatureNotSupportedException
+   *            if timeout is non-zero (and not above case)
+   */
+  @Override
+  void setNetworkTimeout( Executor executor, int milliseconds )
+      throws AlreadyClosedSqlException,
+             JdbcApiSqlException,
+             SQLFeatureNotSupportedException;
+
+  /**
+   * Returns zero.
+   * {@inheritDoc}
+   * @throws  AlreadyClosedSqlException
+   *            if connection is closed
+   */
+  @Override
+  int getNetworkTimeout() throws AlreadyClosedSqlException;
+
+
   //////////////////////////////////////////////////////////////////////
   // Drill extensions.
 

http://git-wip-us.apache.org/repos/asf/drill/blob/a3ec52a7/exec/jdbc/src/main/java/org/apache/drill/jdbc/DrillConnectionImpl.java
----------------------------------------------------------------------
diff --git a/exec/jdbc/src/main/java/org/apache/drill/jdbc/DrillConnectionImpl.java b/exec/jdbc/src/main/java/org/apache/drill/jdbc/DrillConnectionImpl.java
index f8c6c8b..30279e6 100644
--- a/exec/jdbc/src/main/java/org/apache/drill/jdbc/DrillConnectionImpl.java
+++ b/exec/jdbc/src/main/java/org/apache/drill/jdbc/DrillConnectionImpl.java
@@ -24,6 +24,7 @@ import java.sql.SQLFeatureNotSupportedException;
 import java.sql.Savepoint;
 import java.util.Properties;
 import java.util.TimeZone;
+import java.util.concurrent.Executor;
 
 import net.hydromatic.avatica.AvaticaConnection;
 import net.hydromatic.avatica.AvaticaFactory;
@@ -32,6 +33,7 @@ import net.hydromatic.avatica.Meta;
 import net.hydromatic.avatica.UnregisteredDriver;
 
 import org.apache.drill.common.config.DrillConfig;
+import org.apache.drill.common.exceptions.DrillRuntimeException;
 import org.apache.drill.exec.client.DrillClient;
 import org.apache.drill.exec.memory.BufferAllocator;
 import org.apache.drill.exec.memory.TopLevelAllocator;
@@ -123,12 +125,10 @@ public abstract class DrillConnectionImpl extends AvaticaConnection
   }
 
   /**
-   * Throws AlreadyClosedSqlException if this Connection is closed.
+   * Throws AlreadyClosedSqlException <i>iff</i> this Connection is closed.
    *
-   * @throws AlreadyClosedSqlException if Connection is closed
-   * @throws SQLException if error in calling {@link #isClosed()}
-   */
-  private void checkNotClosed() throws SQLException {
+   * @throws  AlreadyClosedSqlException  if Connection is closed   */
+  private void checkNotClosed() throws AlreadyClosedSqlException {
     if ( isClosed() ) {
       throw new AlreadyClosedSqlException( "Connection is already closed." );
     }
@@ -194,6 +194,23 @@ public abstract class DrillConnectionImpl extends AvaticaConnection
     }
   }
 
+
+  @Override
+  public boolean isClosed() {
+    try {
+      return super.isClosed();
+    }
+    catch ( SQLException e ) {
+      // Currently can't happen, since AvaticaConnection.isClosed() never throws
+      // SQLException.
+      throw new DrillRuntimeException(
+          "Unexpected exception from " + getClass().getSuperclass()
+          + ".isClosed(): " + e,
+          e );
+    }
+  }
+
+
   @Override
   public Savepoint setSavepoint() throws SQLException {
     checkNotClosed();
@@ -260,6 +277,38 @@ public abstract class DrillConnectionImpl extends AvaticaConnection
   }
 
   @Override
+  public void setNetworkTimeout( Executor executor, int milliseconds )
+      throws AlreadyClosedSqlException,
+             JdbcApiSqlException,
+             SQLFeatureNotSupportedException {
+    checkNotClosed();
+    if ( null == executor ) {
+      throw new InvalidParameterSqlException(
+          "Invalid (null) \"executor\" parameter to setNetworkTimeout(...)" );
+    }
+    else if ( milliseconds < 0 ) {
+      throw new InvalidParameterSqlException(
+          "Invalid (negative) \"milliseconds\" parameter to setNetworkTimeout(...)"
+          + " (" + milliseconds + ")" );
+    }
+    else {
+      if ( 0 != milliseconds ) {
+        throw new SQLFeatureNotSupportedException(
+            "Setting network timeout is not supported." );
+      }
+    }
+  }
+
+
+  @Override
+  public int getNetworkTimeout() throws AlreadyClosedSqlException
+  {
+    checkNotClosed();
+    return 0;  // (No no timeout.)
+  }
+
+
+  @Override
   public DrillStatementImpl createStatement(int resultSetType, int resultSetConcurrency,
                                         int resultSetHoldability) throws SQLException {
     checkNotClosed();

http://git-wip-us.apache.org/repos/asf/drill/blob/a3ec52a7/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 79ae9dd..56a8b7f 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
@@ -16,6 +16,7 @@
  */
 package org.apache.drill.jdbc;
 
+import java.sql.SQLFeatureNotSupportedException;
 import java.sql.Statement;
 
 
@@ -24,4 +25,41 @@ import java.sql.Statement;
  */
 public interface DrillStatement extends Statement {
 
+  /**
+   * Returns zero.
+   * {@inheritDoc}
+   * @throws  AlreadyClosedSqlException
+   *            if connection is closed
+   */
+  @Override
+  int getQueryTimeout() throws AlreadyClosedSqlException;
+
+  /**
+   * Not supported (for non-zero timeout value).
+   * <p>
+   *   Normally, just throws {@link SQLFeatureNotSupportedException} unless
+   *   request is trivially for no timeout (zero {@code milliseconds} value).
+   * </p>
+   * @throws  AlreadyClosedSqlException
+   *            if connection is closed
+   * @throws  JdbcApiSqlException
+   *            if an invalid parameter value is detected (and not above case)
+   * @throws  SQLFeatureNotSupportedException
+   *            if timeout is non-zero (and not above case)
+   */
+  @Override
+  void setQueryTimeout( int milliseconds )
+      throws AlreadyClosedSqlException,
+             JdbcApiSqlException,
+             SQLFeatureNotSupportedException;
+
+  /**
+   * {@inheritDoc}
+   * <p>
+   *   <strong>Drill</strong>: Does not throw SQLException.
+   * </p>
+   */
+  @Override
+  boolean isClosed();
+
 }

http://git-wip-us.apache.org/repos/asf/drill/blob/a3ec52a7/exec/jdbc/src/main/java/org/apache/drill/jdbc/InvalidParameterSqlException.java
----------------------------------------------------------------------
diff --git a/exec/jdbc/src/main/java/org/apache/drill/jdbc/InvalidParameterSqlException.java b/exec/jdbc/src/main/java/org/apache/drill/jdbc/InvalidParameterSqlException.java
new file mode 100644
index 0000000..7d51a38
--- /dev/null
+++ b/exec/jdbc/src/main/java/org/apache/drill/jdbc/InvalidParameterSqlException.java
@@ -0,0 +1,86 @@
+/*
+ * 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;
+
+
+/**
+ * {@link JdbcApiSqlException} for invalid-parameter-value conditions.
+ */
+public class InvalidParameterSqlException extends JdbcApiSqlException {
+
+  private static final long serialVersionUID = 2015_05_05L;
+
+
+  /**
+   * See {@link JdbcApiSqlException#JdbcApiSqlException(String, String, int)}.
+   */
+  public InvalidParameterSqlException( String reason,
+                                       String SQLState,
+                                       int vendorCode ) {
+    super( reason, SQLState, vendorCode );
+  }
+
+  /**
+   * See {@link JdbcApiSqlException#JdbcApiSqlException(String, String)}.
+   */
+  public InvalidParameterSqlException( String reason, String SQLState ) {
+    super( reason, SQLState );
+  }
+
+  /**
+   * See {@link JdbcApiSqlException#JdbcApiSqlException(String)}.
+   */
+  public InvalidParameterSqlException( String reason ) {
+    super( reason );
+  }
+
+  /**
+   * See {@link JdbcApiSqlException#JdbcApiSqlException(Throwable cause)}.
+   */
+  public InvalidParameterSqlException( Throwable cause ) {
+    super( cause );
+  }
+
+  /**
+   * See {@link JdbcApiSqlException#JdbcApiSqlException(String, Throwable)}.
+   */
+  public InvalidParameterSqlException( String reason, Throwable cause ) {
+    super( reason, cause );
+  }
+
+  /**
+   * See
+   * {@link JdbcApiSqlException#JdbcApiSqlException(String, String, Throwable)}.
+   */
+  public InvalidParameterSqlException( String reason, String sqlState,
+                                       Throwable cause ) {
+    super( reason, sqlState, cause );
+  }
+
+  /**
+   * See
+   * {@link JdbcApiSqlException#JdbcApiSqlException(String, String, int, Throwable)}.
+   */
+  public InvalidParameterSqlException( String reason,
+                                       String sqlState,
+                                       int vendorCode,
+                                       Throwable cause ) {
+    super( reason, sqlState, vendorCode, cause );
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/a3ec52a7/exec/jdbc/src/main/java/org/apache/drill/jdbc/impl/DrillStatementImpl.java
----------------------------------------------------------------------
diff --git a/exec/jdbc/src/main/java/org/apache/drill/jdbc/impl/DrillStatementImpl.java b/exec/jdbc/src/main/java/org/apache/drill/jdbc/impl/DrillStatementImpl.java
index 0f0d174..5160c31 100644
--- a/exec/jdbc/src/main/java/org/apache/drill/jdbc/impl/DrillStatementImpl.java
+++ b/exec/jdbc/src/main/java/org/apache/drill/jdbc/impl/DrillStatementImpl.java
@@ -19,11 +19,14 @@ package org.apache.drill.jdbc.impl;
 
 import java.sql.ResultSet;
 import java.sql.SQLException;
+import java.sql.SQLFeatureNotSupportedException;
 
+import org.apache.drill.common.exceptions.DrillRuntimeException;
 import org.apache.drill.jdbc.AlreadyClosedSqlException;
 import org.apache.drill.jdbc.DrillConnectionImpl;
 import org.apache.drill.jdbc.DrillRemoteStatement;
 import org.apache.drill.jdbc.DrillStatement;
+import org.apache.drill.jdbc.InvalidParameterSqlException;
 
 import net.hydromatic.avatica.AvaticaStatement;
 
@@ -37,12 +40,11 @@ public abstract class DrillStatementImpl extends AvaticaStatement
   }
 
   /**
-   * Throws AlreadyClosedSqlException if this Statement is closed.
+   * Throws AlreadyClosedSqlException <i>iff</i> this Statement is closed.
    *
-   * @throws AlreadyClosedSqlException if Statement is closed
-   * @throws SQLException if error in calling {@link #isClosed()}
+   * @throws  AlreadyClosedSqlException  if Statement is closed
    */
-  private void checkNotClosed() throws SQLException {
+  private void checkNotClosed() throws AlreadyClosedSqlException {
     if ( isClosed() ) {
       throw new AlreadyClosedSqlException( "Statement is already closed." );
     }
@@ -120,4 +122,45 @@ public abstract class DrillStatementImpl extends AvaticaStatement
     connection1.openStatementsRegistry.removeStatement(this);
   }
 
+  @Override
+  public int getQueryTimeout() throws AlreadyClosedSqlException
+  {
+    checkNotClosed();
+    return 0;  // (No no timeout.)
+  }
+
+  @Override
+  public void setQueryTimeout( int milliseconds )
+      throws AlreadyClosedSqlException,
+             InvalidParameterSqlException,
+             SQLFeatureNotSupportedException {
+    checkNotClosed();
+    if ( milliseconds < 0 ) {
+      throw new InvalidParameterSqlException(
+          "Invalid (negative) \"milliseconds\" parameter to setQueryTimeout(...)"
+          + " (" + milliseconds + ")" );
+    }
+    else {
+      if ( 0 != milliseconds ) {
+        throw new SQLFeatureNotSupportedException(
+            "Setting network timeout is not supported." );
+      }
+    }
+  }
+
+  @Override
+  public boolean isClosed() {
+    try {
+      return super.isClosed();
+    }
+    catch ( SQLException e ) {
+      // Currently can't happen, since AvaticaStatement.isClosed() never throws
+      // SQLException.
+      throw new DrillRuntimeException(
+          "Unexpected exception from " + getClass().getSuperclass()
+          + ".isClosed(): " + e,
+          e );
+    }
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/drill/blob/a3ec52a7/exec/jdbc/src/test/java/org/apache/drill/jdbc/ConnectionTest.java
----------------------------------------------------------------------
diff --git a/exec/jdbc/src/test/java/org/apache/drill/jdbc/ConnectionTest.java b/exec/jdbc/src/test/java/org/apache/drill/jdbc/ConnectionTest.java
new file mode 100644
index 0000000..8a469c3
--- /dev/null
+++ b/exec/jdbc/src/test/java/org/apache/drill/jdbc/ConnectionTest.java
@@ -0,0 +1,132 @@
+/**
+ * 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 org.apache.drill.jdbc.Driver;
+
+import static org.hamcrest.CoreMatchers.*;
+import static org.junit.Assert.assertThat;
+import static org.junit.Assert.fail;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import java.sql.Connection;
+import java.sql.SQLFeatureNotSupportedException;
+import java.sql.SQLException;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+
+
+/**
+ * Test for Drill's implementation of Connection's methods (other than
+ * main transaction-related methods in {@link ConnectionTransactionMethodsTest}).
+ */
+public class ConnectionTest extends JdbcTestBase {
+
+  private static Connection connection;
+
+  private static ExecutorService executor;
+
+  @BeforeClass
+  public static void setUpConnection() throws SQLException {
+    // (Note: Can't use JdbcTest's connect(...) because JdbcTest closes
+    // Connection--and other JDBC objects--on test method failure, but this test
+    // class uses some objects across methods.)
+    connection = new Driver().connect( "jdbc:drill:zk=local", null );
+    executor = Executors.newSingleThreadExecutor();
+  }
+
+  @AfterClass
+  public static void tearDownConnection() throws SQLException {
+    connection.close();
+    executor.shutdown();
+  }
+
+
+  ////////////////////////////////////////
+  // Network timeout methods:
+
+  //////////
+  // getNetworkTimeout():
+
+  /** Tests that getNetworkTimeout() indicates no timeout set. */
+  @Test
+  public void testGetNetworkTimeoutSaysNoTimeout() throws SQLException {
+    assertThat( connection.getNetworkTimeout(), equalTo( 0 ) );
+  }
+
+  //////////
+  // setNetworkTimeout(...):
+
+  /** Tests that setNetworkTimeout(...) accepts (redundantly) setting to
+   *  no-timeout mode. */
+  @Test
+  public void testSetNetworkTimeoutAcceptsNotimeoutRequest() throws SQLException {
+    connection.setNetworkTimeout( executor, 0 );
+  }
+
+  /** Tests that setNetworkTimeout(...) rejects setting a timeout. */
+  @Test( expected = SQLFeatureNotSupportedException.class )
+  public void testSetNetworkTimeoutRejectsTimeoutRequest() throws SQLException {
+    try {
+      connection.setNetworkTimeout( executor, 1_000 );
+    }
+    catch ( SQLFeatureNotSupportedException e ) {
+      // Check exception for some mention of network timeout:
+      assertThat( e.getMessage(), anyOf( containsString( "Timeout" ),
+                                         containsString( "timeout" ) ) );
+      throw e;
+    }
+  }
+
+  /** Tests that setNetworkTimeout(...) rejects setting a timeout (different
+   *  value). */
+  @Test( expected = SQLFeatureNotSupportedException.class )
+  public void testSetNetworkTimeoutRejectsTimeoutRequest2() throws SQLException {
+    connection.setNetworkTimeout( executor, Integer.MAX_VALUE );
+  }
+
+  @Test( expected = InvalidParameterSqlException.class )
+  public void testSetNetworkTimeoutRejectsBadTimeoutValue() throws SQLException {
+    try {
+      connection.setNetworkTimeout( executor, -1 );
+    }
+    catch ( InvalidParameterSqlException e ) {
+      // Check exception for some mention of parameter name or semantics:
+      assertThat( e.getMessage(), anyOf( containsString( "milliseconds" ),
+                                         containsString( "timeout" ),
+                                         containsString( "Timeout" ) ) );
+      throw e;
+    }
+  }
+
+  @Test( expected = InvalidParameterSqlException.class )
+  public void testSetNetworkTimeoutRejectsBadExecutorValue() throws SQLException {
+    try {
+      connection.setNetworkTimeout( null, 1 );
+    }
+    catch ( InvalidParameterSqlException e ) {
+      // Check exception for some mention of parameter name or semantics:
+      assertThat( e.getMessage(), anyOf( containsString( "executor" ),
+                                         containsString( "Executor" ) ) );
+      throw e;
+    }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/a3ec52a7/exec/jdbc/src/test/java/org/apache/drill/jdbc/StatementTest.java
----------------------------------------------------------------------
diff --git a/exec/jdbc/src/test/java/org/apache/drill/jdbc/StatementTest.java b/exec/jdbc/src/test/java/org/apache/drill/jdbc/StatementTest.java
new file mode 100644
index 0000000..3e64fcb
--- /dev/null
+++ b/exec/jdbc/src/test/java/org/apache/drill/jdbc/StatementTest.java
@@ -0,0 +1,115 @@
+/**
+ * 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 org.apache.drill.jdbc.Driver;
+
+import static org.hamcrest.CoreMatchers.*;
+import static org.junit.Assert.assertThat;
+
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import java.sql.Connection;
+import java.sql.Statement;
+import java.sql.SQLFeatureNotSupportedException;
+import java.sql.SQLException;
+
+
+/**
+ * Test for Drill's implementation of Statement's methods (most).
+ */
+public class StatementTest extends JdbcTestBase {
+
+  private static Connection connection;
+  private static Statement statement;
+
+  @BeforeClass
+  public static void setUpStatement() throws SQLException {
+    // (Note: Can't use JdbcTest's connect(...) because JdbcTest closes
+    // Connection--and other JDBC objects--on test method failure, but this test
+    // class uses some objects across methods.)
+    connection = new Driver().connect( "jdbc:drill:zk=local", null );
+    statement = connection.createStatement();
+  }
+
+  @AfterClass
+  public static void tearDownStatement() throws SQLException {
+    connection.close();
+  }
+
+
+  ////////////////////////////////////////
+  // Query timeout methods:
+
+  //////////
+  // getQueryTimeout():
+
+  /** Tests that getQueryTimeout() indicates no timeout set. */
+  @Test
+  public void testGetQueryTimeoutSaysNoTimeout() throws SQLException {
+    assertThat( statement.getQueryTimeout(), equalTo( 0 ) );
+  }
+
+  //////////
+  // setQueryTimeout(...):
+
+  /** Tests that setQueryTimeout(...) accepts (redundantly) setting to
+   *  no-timeout mode. */
+  @Test
+  public void testSetQueryTimeoutAcceptsNotimeoutRequest() throws SQLException {
+    statement.setQueryTimeout( 0 );
+  }
+
+  /** Tests that setQueryTimeout(...) rejects setting a timeout. */
+  @Test( expected = SQLFeatureNotSupportedException.class )
+  public void testSetQueryTimeoutRejectsTimeoutRequest() throws SQLException {
+    try {
+      statement.setQueryTimeout( 1_000 );
+    }
+    catch ( SQLFeatureNotSupportedException e ) {
+      // Check exception for some mention of query timeout:
+      assertThat( e.getMessage(), anyOf( containsString( "Timeout" ),
+                                         containsString( "timeout" ) ) );
+      throw e;
+    }
+  }
+
+  /** Tests that setQueryTimeout(...) rejects setting a timeout (different
+   *  value). */
+  @Test( expected = SQLFeatureNotSupportedException.class )
+  public void testSetQueryTimeoutRejectsTimeoutRequest2() throws SQLException {
+    statement.setQueryTimeout( Integer.MAX_VALUE / 2 );
+  }
+
+  @Test( expected = InvalidParameterSqlException.class )
+  public void testSetQueryTimeoutRejectsBadTimeoutValue() throws SQLException {
+    try {
+      statement.setQueryTimeout( -2 );
+    }
+    catch ( InvalidParameterSqlException e ) {
+      // Check exception for some mention of parameter name or semantics:
+      assertThat( e.getMessage(), anyOf( containsString( "milliseconds" ),
+                                         containsString( "timeout" ),
+                                         containsString( "Timeout" ) ) );
+      throw e;
+    }
+  }
+
+}


[2/2] drill git commit: DRILL-2961: Part 1--Prep., Hyg.: Split DrillStatement -> DrillStatement{, Impl}.

Posted by pa...@apache.org.
DRILL-2961: Part 1--Prep., Hyg.: Split DrillStatement -> DrillStatement{,Impl}.

Preparation for query/network timeout methods:
- Moved class ...jdbc.DrillStatement to ...jdbc.impl.DrillStatementImpl.
- Created interface ...jdbc.DrillStatement interface, extending
  java.sql.Statement (for place for Drill-specific narrowed declarations and
  documentation).  Declared DrillStatementImpl to implement new DrillStatement.
- Updated references.
- Adjusted visibility (temporarily make somethings publich until DRILL-2089 is
  done more).

Misc. code hygiene:
- Purged extraneous imports of ResultSet.
- Purged TODO made obsolete by UserException.
- Misc. whitespace.


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

Branch: refs/heads/master
Commit: 1dcf6cfb0d52435bcc7d1f4eada48fc0ce366dc4
Parents: 84371ea
Author: dbarclay <db...@maprtech.com>
Authored: Tue May 5 22:39:25 2015 -0700
Committer: Parth Chandra <pa...@apache.org>
Committed: Sun May 10 23:47:17 2015 -0700

----------------------------------------------------------------------
 .../drill/jdbc/AlreadyClosedSqlException.java   |   2 -
 .../apache/drill/jdbc/DrillConnectionImpl.java  |  14 ++-
 .../apache/drill/jdbc/DrillJdbc41Factory.java   |   3 +-
 .../org/apache/drill/jdbc/DrillStatement.java   | 114 ++---------------
 .../drill/jdbc/DrillStatementRegistry.java      |   3 +-
 .../jdbc/InvalidCursorStateSqlException.java    |   1 -
 .../apache/drill/jdbc/JdbcApiSqlException.java  |   4 -
 .../drill/jdbc/impl/DrillStatementImpl.java     | 123 +++++++++++++++++++
 8 files changed, 147 insertions(+), 117 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/drill/blob/1dcf6cfb/exec/jdbc/src/main/java/org/apache/drill/jdbc/AlreadyClosedSqlException.java
----------------------------------------------------------------------
diff --git a/exec/jdbc/src/main/java/org/apache/drill/jdbc/AlreadyClosedSqlException.java b/exec/jdbc/src/main/java/org/apache/drill/jdbc/AlreadyClosedSqlException.java
index 6e41bb4..a2a06ab 100644
--- a/exec/jdbc/src/main/java/org/apache/drill/jdbc/AlreadyClosedSqlException.java
+++ b/exec/jdbc/src/main/java/org/apache/drill/jdbc/AlreadyClosedSqlException.java
@@ -17,8 +17,6 @@
  */
 package org.apache.drill.jdbc;
 
-import java.sql.ResultSet;
-
 
 /**
  * SQLException for object-already-closed conditions, e.g., calling a method

http://git-wip-us.apache.org/repos/asf/drill/blob/1dcf6cfb/exec/jdbc/src/main/java/org/apache/drill/jdbc/DrillConnectionImpl.java
----------------------------------------------------------------------
diff --git a/exec/jdbc/src/main/java/org/apache/drill/jdbc/DrillConnectionImpl.java b/exec/jdbc/src/main/java/org/apache/drill/jdbc/DrillConnectionImpl.java
index 74c6655..f8c6c8b 100644
--- a/exec/jdbc/src/main/java/org/apache/drill/jdbc/DrillConnectionImpl.java
+++ b/exec/jdbc/src/main/java/org/apache/drill/jdbc/DrillConnectionImpl.java
@@ -40,6 +40,7 @@ import org.apache.drill.exec.server.Drillbit;
 import org.apache.drill.exec.server.RemoteServiceSet;
 import org.apache.drill.exec.store.StoragePluginRegistry;
 import org.apache.drill.exec.util.TestUtilities;
+import org.apache.drill.jdbc.impl.DrillStatementImpl;
 
 // (Public until JDBC impl. classes moved out of published-intf. package. (DRILL-2089).)
 /**
@@ -53,7 +54,8 @@ public abstract class DrillConnectionImpl extends AvaticaConnection
                                           implements DrillConnection {
   private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(DrillConnection.class);
 
-  final DrillStatementRegistry openStatementsRegistry = new DrillStatementRegistry();
+  // (Public until JDBC impl. classes moved out of published-intf. package. (DRILL-2089).)
+  public final DrillStatementRegistry openStatementsRegistry = new DrillStatementRegistry();
   final DrillConnectionConfig config;
 
   private final DrillClient client;
@@ -167,9 +169,9 @@ public abstract class DrillConnectionImpl extends AvaticaConnection
   }
 
   @Override
-  public void commit() throws SQLException  {
+  public void commit() throws SQLException {
     checkNotClosed();
-    if ( getAutoCommit()  ) {
+    if ( getAutoCommit() ) {
       throw new JdbcApiSqlException( "Can't call commit() in auto-commit mode." );
     }
     else {
@@ -258,11 +260,11 @@ public abstract class DrillConnectionImpl extends AvaticaConnection
   }
 
   @Override
-  public DrillStatement createStatement(int resultSetType, int resultSetConcurrency,
+  public DrillStatementImpl createStatement(int resultSetType, int resultSetConcurrency,
                                         int resultSetHoldability) throws SQLException {
     checkNotClosed();
-    DrillStatement statement =
-        (DrillStatement) super.createStatement(resultSetType, resultSetConcurrency,
+    DrillStatementImpl statement =
+        (DrillStatementImpl) super.createStatement(resultSetType, resultSetConcurrency,
                                                resultSetHoldability);
     return statement;
   }

http://git-wip-us.apache.org/repos/asf/drill/blob/1dcf6cfb/exec/jdbc/src/main/java/org/apache/drill/jdbc/DrillJdbc41Factory.java
----------------------------------------------------------------------
diff --git a/exec/jdbc/src/main/java/org/apache/drill/jdbc/DrillJdbc41Factory.java b/exec/jdbc/src/main/java/org/apache/drill/jdbc/DrillJdbc41Factory.java
index a4a97fd..6240b62 100644
--- a/exec/jdbc/src/main/java/org/apache/drill/jdbc/DrillJdbc41Factory.java
+++ b/exec/jdbc/src/main/java/org/apache/drill/jdbc/DrillJdbc41Factory.java
@@ -30,6 +30,7 @@ import java.util.Properties;
 import java.util.TimeZone;
 
 import org.apache.drill.jdbc.impl.DrillResultSetImpl;
+import org.apache.drill.jdbc.impl.DrillStatementImpl;
 
 import net.hydromatic.avatica.AvaticaConnection;
 import net.hydromatic.avatica.AvaticaDatabaseMetaData;
@@ -98,7 +99,7 @@ public class DrillJdbc41Factory extends DrillFactory {
 
   }
 
-  private static class DrillJdbc41Statement extends DrillStatement {
+  private static class DrillJdbc41Statement extends DrillStatementImpl {
     public DrillJdbc41Statement(DrillConnectionImpl connection, int resultSetType, int resultSetConcurrency,
         int resultSetHoldability) {
       super(connection, resultSetType, resultSetConcurrency, resultSetHoldability);

http://git-wip-us.apache.org/repos/asf/drill/blob/1dcf6cfb/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 a609bb1..79ae9dd 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
@@ -1,11 +1,10 @@
 /**
- * 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
+ * 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
  *
@@ -17,101 +16,12 @@
  */
 package org.apache.drill.jdbc;
 
-import java.sql.ResultSet;
-import java.sql.SQLException;
+import java.sql.Statement;
 
-import net.hydromatic.avatica.AvaticaStatement;
 
-public abstract class DrillStatement extends AvaticaStatement
-   implements DrillRemoteStatement {
-
-  DrillStatement(DrillConnectionImpl connection, int resultSetType, int resultSetConcurrency, int resultSetHoldability) {
-    super(connection, resultSetType, resultSetConcurrency, resultSetHoldability);
-    connection.openStatementsRegistry.addStatement(this);
-  }
-
-  /**
-   * Throws AlreadyClosedSqlException if this Statement is closed.
-   *
-   * @throws AlreadyClosedSqlException if Statement is closed
-   * @throws SQLException if error in calling {@link #isClosed()}
-   */
-  private void checkNotClosed() throws SQLException {
-    if ( isClosed() ) {
-      throw new AlreadyClosedSqlException( "Statement is already closed." );
-    }
-  }
-
-  @Override
-  public DrillConnectionImpl getConnection() {
-    return (DrillConnectionImpl) connection;
-  }
-
-  // WORKAROUND:  Work around AvaticaStatement's code that wraps _any_ exception,
-  // even if SQLException, by unwrapping to get cause exception so caller can
-  // throw it directly if it's a SQLException:
-  // TODO:  Any ideas for a better name?
-  private SQLException unwrapIfExtra( final SQLException superMethodException ) {
-    final SQLException result;
-    final Throwable cause = superMethodException.getCause();
-    if ( null != cause && cause instanceof SQLException ) {
-      result = (SQLException) cause;
-    }
-    else {
-      result = superMethodException;
-    }
-    return result;
-  }
-
-  @Override
-  public boolean execute( String sql ) throws SQLException {
-    checkNotClosed();
-    try {
-      return super.execute( sql );
-    }
-    catch ( final SQLException possiblyExtraWrapperException ) {
-      throw unwrapIfExtra( possiblyExtraWrapperException );
-    }
-  }
-
-  @Override
-  public ResultSet executeQuery( String sql ) throws SQLException {
-    try {
-       checkNotClosed();
-       return super.executeQuery( sql );
-    }
-    catch ( final SQLException possiblyExtraWrapperException ) {
-      throw unwrapIfExtra( possiblyExtraWrapperException );
-    }
-  }
-
-  @Override
-  public int executeUpdate( String sql ) throws SQLException {
-    checkNotClosed();
-    try {
-      return super.executeUpdate( sql );
-    }
-    catch ( final SQLException possiblyExtraWrapperException ) {
-      throw unwrapIfExtra( possiblyExtraWrapperException );
-    }
-  }
-
-  @Override
-  public int executeUpdate( String sql, int[] columnIndexes ) throws SQLException {
-    checkNotClosed();
-    return super.executeUpdate( sql, columnIndexes );
-  }
-
-  @Override
-  public int executeUpdate( String sql, String[] columnNames ) throws SQLException {
-    checkNotClosed();
-    return super.executeUpdate( sql, columnNames );
-  }
-
-  @Override
-  public void cleanup() {
-    final DrillConnectionImpl connection1 = (DrillConnectionImpl) connection;
-    connection1.openStatementsRegistry.removeStatement(this);
-  }
+/**
+ * Drill-specific {@link Statement}.
+ */
+public interface DrillStatement extends Statement {
 
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/1dcf6cfb/exec/jdbc/src/main/java/org/apache/drill/jdbc/DrillStatementRegistry.java
----------------------------------------------------------------------
diff --git a/exec/jdbc/src/main/java/org/apache/drill/jdbc/DrillStatementRegistry.java b/exec/jdbc/src/main/java/org/apache/drill/jdbc/DrillStatementRegistry.java
index adbbb64..fb38e31 100644
--- a/exec/jdbc/src/main/java/org/apache/drill/jdbc/DrillStatementRegistry.java
+++ b/exec/jdbc/src/main/java/org/apache/drill/jdbc/DrillStatementRegistry.java
@@ -37,7 +37,8 @@ import static org.slf4j.LoggerFactory.getLogger;
  *   coordinated elsewhere).)
  * </p>
  */
-class DrillStatementRegistry {
+// (Public until JDBC impl. classes moved out of published-intf. package. (DRILL-2089).)
+public class DrillStatementRegistry {
 
   private static final Logger logger = getLogger( DrillStatementRegistry.class );
 

http://git-wip-us.apache.org/repos/asf/drill/blob/1dcf6cfb/exec/jdbc/src/main/java/org/apache/drill/jdbc/InvalidCursorStateSqlException.java
----------------------------------------------------------------------
diff --git a/exec/jdbc/src/main/java/org/apache/drill/jdbc/InvalidCursorStateSqlException.java b/exec/jdbc/src/main/java/org/apache/drill/jdbc/InvalidCursorStateSqlException.java
index 8d882e9..d71248c 100644
--- a/exec/jdbc/src/main/java/org/apache/drill/jdbc/InvalidCursorStateSqlException.java
+++ b/exec/jdbc/src/main/java/org/apache/drill/jdbc/InvalidCursorStateSqlException.java
@@ -17,7 +17,6 @@
  */
 package org.apache.drill.jdbc;
 
-import java.sql.ResultSet;
 
 /**
  * SQLException for invalid-cursor-state conditions, e.g., calling a column

http://git-wip-us.apache.org/repos/asf/drill/blob/1dcf6cfb/exec/jdbc/src/main/java/org/apache/drill/jdbc/JdbcApiSqlException.java
----------------------------------------------------------------------
diff --git a/exec/jdbc/src/main/java/org/apache/drill/jdbc/JdbcApiSqlException.java b/exec/jdbc/src/main/java/org/apache/drill/jdbc/JdbcApiSqlException.java
index a7e6d98..ce83e01 100644
--- a/exec/jdbc/src/main/java/org/apache/drill/jdbc/JdbcApiSqlException.java
+++ b/exec/jdbc/src/main/java/org/apache/drill/jdbc/JdbcApiSqlException.java
@@ -37,10 +37,6 @@ import java.sql.SQLNonTransientException;
  *   {@link SQLTransientException}), or internal Drill errors.)
  * </p>
  * <p>
- *  TODO:  Consider having a DrillSqlException (in part for reviewing,
- *  coordinating, and revising the many uses of SQLException in the code).
- * </p>
- * <p>
  *  TODO:  Consider using ANSI-/XOPEN-standard SQL State values.  (See:
  * </p>
  * <ul>

http://git-wip-us.apache.org/repos/asf/drill/blob/1dcf6cfb/exec/jdbc/src/main/java/org/apache/drill/jdbc/impl/DrillStatementImpl.java
----------------------------------------------------------------------
diff --git a/exec/jdbc/src/main/java/org/apache/drill/jdbc/impl/DrillStatementImpl.java b/exec/jdbc/src/main/java/org/apache/drill/jdbc/impl/DrillStatementImpl.java
new file mode 100644
index 0000000..0f0d174
--- /dev/null
+++ b/exec/jdbc/src/main/java/org/apache/drill/jdbc/impl/DrillStatementImpl.java
@@ -0,0 +1,123 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.drill.jdbc.impl;
+
+import java.sql.ResultSet;
+import java.sql.SQLException;
+
+import org.apache.drill.jdbc.AlreadyClosedSqlException;
+import org.apache.drill.jdbc.DrillConnectionImpl;
+import org.apache.drill.jdbc.DrillRemoteStatement;
+import org.apache.drill.jdbc.DrillStatement;
+
+import net.hydromatic.avatica.AvaticaStatement;
+
+public abstract class DrillStatementImpl extends AvaticaStatement
+   implements DrillStatement, DrillRemoteStatement {
+
+  // (Public until JDBC impl. classes moved out of published-intf. package. (DRILL-2089).)
+  public DrillStatementImpl(DrillConnectionImpl connection, int resultSetType, int resultSetConcurrency, int resultSetHoldability) {
+    super(connection, resultSetType, resultSetConcurrency, resultSetHoldability);
+    connection.openStatementsRegistry.addStatement(this);
+  }
+
+  /**
+   * Throws AlreadyClosedSqlException if this Statement is closed.
+   *
+   * @throws AlreadyClosedSqlException if Statement is closed
+   * @throws SQLException if error in calling {@link #isClosed()}
+   */
+  private void checkNotClosed() throws SQLException {
+    if ( isClosed() ) {
+      throw new AlreadyClosedSqlException( "Statement is already closed." );
+    }
+  }
+
+  @Override
+  public DrillConnectionImpl getConnection() {
+    return (DrillConnectionImpl) connection;
+  }
+
+  // WORKAROUND:  Work around AvaticaStatement's code that wraps _any_ exception,
+  // even if SQLException, by unwrapping to get cause exception so caller can
+  // throw it directly if it's a SQLException:
+  // TODO:  Any ideas for a better name?
+  private SQLException unwrapIfExtra( final SQLException superMethodException ) {
+    final SQLException result;
+    final Throwable cause = superMethodException.getCause();
+    if ( null != cause && cause instanceof SQLException ) {
+      result = (SQLException) cause;
+    }
+    else {
+      result = superMethodException;
+    }
+    return result;
+  }
+
+  @Override
+  public boolean execute( String sql ) throws SQLException {
+    checkNotClosed();
+    try {
+      return super.execute( sql );
+    }
+    catch ( final SQLException possiblyExtraWrapperException ) {
+      throw unwrapIfExtra( possiblyExtraWrapperException );
+    }
+  }
+
+  @Override
+  public ResultSet executeQuery( String sql ) throws SQLException {
+    try {
+       checkNotClosed();
+       return super.executeQuery( sql );
+    }
+    catch ( final SQLException possiblyExtraWrapperException ) {
+      throw unwrapIfExtra( possiblyExtraWrapperException );
+    }
+  }
+
+  @Override
+  public int executeUpdate( String sql ) throws SQLException {
+    checkNotClosed();
+    try {
+      return super.executeUpdate( sql );
+    }
+    catch ( final SQLException possiblyExtraWrapperException ) {
+      throw unwrapIfExtra( possiblyExtraWrapperException );
+    }
+  }
+
+  @Override
+  public int executeUpdate( String sql, int[] columnIndexes ) throws SQLException {
+    checkNotClosed();
+    return super.executeUpdate( sql, columnIndexes );
+  }
+
+  @Override
+  public int executeUpdate( String sql, String[] columnNames ) throws SQLException {
+    checkNotClosed();
+    return super.executeUpdate( sql, columnNames );
+  }
+
+  @Override
+  public void cleanup() {
+    final DrillConnectionImpl connection1 = (DrillConnectionImpl) connection;
+    connection1.openStatementsRegistry.removeStatement(this);
+  }
+
+}