You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@calcite.apache.org by el...@apache.org on 2016/05/19 19:29:00 UTC

calcite git commit: [CALCITE-1247] Ensure setMaxRows is called for all prepare paths

Repository: calcite
Updated Branches:
  refs/heads/master 987a257ae -> 25bba0697


[CALCITE-1247] Ensure setMaxRows is called for all prepare paths

Noticed that prepareAndExecute calls setMaxRows, but prepare did
not. Wrote some tests to verify that they do this in the future.


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

Branch: refs/heads/master
Commit: 25bba06974e07f89a36f3ddd2596fe1f2d4d916e
Parents: 987a257
Author: Josh Elser <el...@apache.org>
Authored: Thu May 19 15:22:14 2016 -0400
Committer: Josh Elser <el...@apache.org>
Committed: Thu May 19 15:22:14 2016 -0400

----------------------------------------------------------------------
 .../java/org/apache/calcite/avatica/Meta.java   |  5 +-
 .../apache/calcite/avatica/jdbc/JdbcMeta.java   | 41 ++++++++---
 .../calcite/avatica/jdbc/JdbcMetaTest.java      | 77 ++++++++++++++++++++
 3 files changed, 112 insertions(+), 11 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/calcite/blob/25bba069/avatica/core/src/main/java/org/apache/calcite/avatica/Meta.java
----------------------------------------------------------------------
diff --git a/avatica/core/src/main/java/org/apache/calcite/avatica/Meta.java b/avatica/core/src/main/java/org/apache/calcite/avatica/Meta.java
index 8afeda9..829ab5a 100644
--- a/avatica/core/src/main/java/org/apache/calcite/avatica/Meta.java
+++ b/avatica/core/src/main/java/org/apache/calcite/avatica/Meta.java
@@ -1141,7 +1141,7 @@ public interface Meta {
   }
 
   /** Connection handle. */
-  class ConnectionHandle {
+  public class ConnectionHandle {
     public final String id;
 
     @Override public String toString() {
@@ -1155,7 +1155,8 @@ public interface Meta {
   }
 
   /** Statement handle. */
-  class StatementHandle {
+  // Visible for testing
+  public class StatementHandle {
     private static final FieldDescriptor SIGNATURE_DESCRIPTOR = Common.StatementHandle
         .getDescriptor().findFieldByNumber(Common.StatementHandle.SIGNATURE_FIELD_NUMBER);
     public final String connectionId;

http://git-wip-us.apache.org/repos/asf/calcite/blob/25bba069/avatica/server/src/main/java/org/apache/calcite/avatica/jdbc/JdbcMeta.java
----------------------------------------------------------------------
diff --git a/avatica/server/src/main/java/org/apache/calcite/avatica/jdbc/JdbcMeta.java b/avatica/server/src/main/java/org/apache/calcite/avatica/jdbc/JdbcMeta.java
index 9dac3fb..c68041f 100644
--- a/avatica/server/src/main/java/org/apache/calcite/avatica/jdbc/JdbcMeta.java
+++ b/avatica/server/src/main/java/org/apache/calcite/avatica/jdbc/JdbcMeta.java
@@ -208,6 +208,16 @@ public class JdbcMeta implements ProtobufMeta {
     });
   }
 
+  // For testing purposes
+  protected AtomicInteger getStatementIdGenerator() {
+    return statementIdGenerator;
+  }
+
+  // For testing purposes
+  protected Cache<Integer, StatementInfo> getStatementCache() {
+    return statementCache;
+  }
+
   /**
    * Converts from JDBC metadata to Avatica columns.
    */
@@ -677,7 +687,7 @@ public class JdbcMeta implements ProtobufMeta {
     try {
       final Connection conn = getConnection(ch.id);
       final PreparedStatement statement = conn.prepareStatement(sql);
-      final int id = statementIdGenerator.getAndIncrement();
+      final int id = getStatementIdGenerator().getAndIncrement();
       Meta.StatementType statementType = null;
       if (statement.isWrapperFor(AvaticaPreparedStatement.class)) {
         final AvaticaPreparedStatement avaticaPreparedStatement;
@@ -685,7 +695,9 @@ public class JdbcMeta implements ProtobufMeta {
             statement.unwrap(AvaticaPreparedStatement.class);
         statementType = avaticaPreparedStatement.getStatementType();
       }
-      statementCache.put(id, new StatementInfo(statement));
+      // Set the maximum number of rows
+      setMaxRows(statement, maxRowCount);
+      getStatementCache().put(id, new StatementInfo(statement));
       StatementHandle h = new StatementHandle(ch.id, id,
           signature(statement.getMetaData(), statement.getParameterMetaData(),
               sql, statementType));
@@ -704,17 +716,13 @@ public class JdbcMeta implements ProtobufMeta {
   public ExecuteResult prepareAndExecute(StatementHandle h, String sql, long maxRowCount,
       int maxRowsInFirstFrame, PrepareCallback callback) throws NoSuchStatementException {
     try {
-      final StatementInfo info = statementCache.getIfPresent(h.id);
+      final StatementInfo info = getStatementCache().getIfPresent(h.id);
       if (info == null) {
         throw new NoSuchStatementException(h);
       }
       final Statement statement = info.statement;
-      // Special handling of maxRowCount as JDBC 0 is unlimited, our meta 0 row
-      if (maxRowCount > 0) {
-        AvaticaUtils.setLargeMaxRows(statement, maxRowCount);
-      } else if (maxRowCount < 0) {
-        statement.setMaxRows(0);
-      }
+      // Make sure that we limit the number of rows for the query
+      setMaxRows(statement, maxRowCount);
       boolean ret = statement.execute(sql);
       info.setResultSet(statement.getResultSet());
       // Either execute(sql) returned true or the resultSet was null
@@ -737,6 +745,21 @@ public class JdbcMeta implements ProtobufMeta {
     }
   }
 
+  /**
+   * Sets the provided maximum number of rows on the given statement.
+   *
+   * @param statement The JDBC Statement to operate on
+   * @param maxRowCount The maximum number of rows which should be returned for the query
+   */
+  void setMaxRows(Statement statement, long maxRowCount) throws SQLException {
+    // Special handling of maxRowCount as JDBC 0 is unlimited, our meta 0 row
+    if (maxRowCount > 0) {
+      AvaticaUtils.setLargeMaxRows(statement, maxRowCount);
+    } else if (maxRowCount < 0) {
+      statement.setMaxRows(0);
+    }
+  }
+
   public boolean syncResults(StatementHandle sh, QueryState state, long offset)
       throws NoSuchStatementException {
     try {

http://git-wip-us.apache.org/repos/asf/calcite/blob/25bba069/avatica/server/src/test/java/org/apache/calcite/avatica/jdbc/JdbcMetaTest.java
----------------------------------------------------------------------
diff --git a/avatica/server/src/test/java/org/apache/calcite/avatica/jdbc/JdbcMetaTest.java b/avatica/server/src/test/java/org/apache/calcite/avatica/jdbc/JdbcMetaTest.java
index 48887b4..d84fd29 100644
--- a/avatica/server/src/test/java/org/apache/calcite/avatica/jdbc/JdbcMetaTest.java
+++ b/avatica/server/src/test/java/org/apache/calcite/avatica/jdbc/JdbcMetaTest.java
@@ -16,9 +16,23 @@
  */
 package org.apache.calcite.avatica.jdbc;
 
+import org.apache.calcite.avatica.AvaticaPreparedStatement;
+import org.apache.calcite.avatica.Meta.ConnectionHandle;
+import org.apache.calcite.avatica.Meta.Signature;
+import org.apache.calcite.avatica.Meta.StatementHandle;
+
+import com.google.common.cache.Cache;
+
 import org.junit.Test;
+import org.mockito.Mockito;
 
+import java.sql.Connection;
+import java.sql.ParameterMetaData;
+import java.sql.PreparedStatement;
+import java.sql.ResultSetMetaData;
 import java.sql.SQLException;
+import java.util.UUID;
+import java.util.concurrent.atomic.AtomicInteger;
 
 import static org.hamcrest.CoreMatchers.is;
 import static org.junit.Assert.assertThat;
@@ -41,6 +55,69 @@ public class JdbcMetaTest {
       assertThat(rte.getCause(), is(e));
     }
   }
+
+  @Test public void testPrepareSetsMaxRows() throws Exception {
+    final String id = UUID.randomUUID().toString();
+    final String sql = "SELECT * FROM FOO";
+    final int maxRows = 500;
+    final ConnectionHandle ch = new ConnectionHandle(id);
+    final AtomicInteger statementIdGenerator = new AtomicInteger(0);
+
+    JdbcMeta meta = Mockito.mock(JdbcMeta.class);
+    Connection connection = Mockito.mock(Connection.class);
+    PreparedStatement statement = Mockito.mock(PreparedStatement.class);
+    ResultSetMetaData resultSetMetaData = Mockito.mock(ResultSetMetaData.class);
+    ParameterMetaData parameterMetaData = Mockito.mock(ParameterMetaData.class);
+    @SuppressWarnings("unchecked")
+    Cache<Integer, StatementInfo> statementCache =
+        (Cache<Integer, StatementInfo>) Mockito.mock(Cache.class);
+
+    Mockito.when(meta.getStatementIdGenerator()).thenReturn(statementIdGenerator);
+    Mockito.when(meta.getStatementCache()).thenReturn(statementCache);
+    Mockito.when(meta.getConnection(id)).thenReturn(connection);
+    Mockito.when(connection.prepareStatement(sql)).thenReturn(statement);
+    Mockito.when(statement.isWrapperFor(AvaticaPreparedStatement.class)).thenReturn(false);
+    Mockito.when(statement.getMetaData()).thenReturn(resultSetMetaData);
+    Mockito.when(statement.getParameterMetaData()).thenReturn(parameterMetaData);
+    // Call the real methods
+    Mockito.doCallRealMethod().when(meta).setMaxRows(statement, maxRows);
+    Mockito.doCallRealMethod().when(meta).prepare(ch, sql, maxRows);
+
+    meta.prepare(ch, sql, maxRows);
+
+    Mockito.verify(statement).setMaxRows(maxRows);
+  }
+
+  @Test public void testPrepareAndExecuteSetsMaxRows() throws Exception {
+    final String id = UUID.randomUUID().toString();
+    final int statementId = 12345;
+    final String sql = "SELECT * FROM FOO";
+    final int maxRows = 500;
+
+    JdbcMeta meta = Mockito.mock(JdbcMeta.class);
+    PreparedStatement statement = Mockito.mock(PreparedStatement.class);
+    @SuppressWarnings("unchecked")
+    Cache<Integer, StatementInfo> statementCache =
+        (Cache<Integer, StatementInfo>) Mockito.mock(Cache.class);
+    Signature signature = Mockito.mock(Signature.class);
+
+    final StatementInfo statementInfo = new StatementInfo(statement);
+    final StatementHandle statementHandle = new StatementHandle(id, statementId, signature);
+
+    Mockito.when(meta.getStatementCache()).thenReturn(statementCache);
+    Mockito.when(statementCache.getIfPresent(statementId)).thenReturn(statementInfo);
+    Mockito.when(statement.getResultSet()).thenReturn(null);
+    // The real methods
+    Mockito.when(meta.prepareAndExecute(statementHandle, sql, maxRows, 50, null)).
+        thenCallRealMethod();
+    Mockito.doCallRealMethod().when(meta).setMaxRows(statement, maxRows);
+
+    // Call our method
+    meta.prepareAndExecute(statementHandle, sql, maxRows, 50, null);
+
+    // Verify we called setMaxRows with the right value
+    Mockito.verify(statement).setMaxRows(maxRows);
+  }
 }
 
 // End JdbcMetaTest.java