You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@calcite.apache.org by jh...@apache.org on 2015/09/13 06:30:39 UTC

[2/6] incubator-calcite git commit: [CALCITE-687] Synchronize HSQLDB at a coarse level using a Lock (Josh Elser)

[CALCITE-687] Synchronize HSQLDB at a coarse level using a Lock (Josh Elser)

For whatever reason, HSQLDB doesn't seem to be as threadsafe
as it claims.


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

Branch: refs/heads/master
Commit: 8f2fa3a50d43bc91b54a42d70ce5fd471fa25a9d
Parents: b3d35c0
Author: Josh Elser <el...@apache.org>
Authored: Thu Sep 10 15:11:14 2015 -0400
Committer: Julian Hyde <jh...@apache.org>
Committed: Fri Sep 11 13:34:00 2015 -0700

----------------------------------------------------------------------
 .../apache/calcite/avatica/ConnectionSpec.java  |  55 +++
 .../calcite/avatica/RemoteDriverTest.java       | 420 +++++++++++--------
 .../calcite/avatica/remote/RemoteMetaTest.java  |  28 +-
 3 files changed, 322 insertions(+), 181 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/8f2fa3a5/avatica-server/src/test/java/org/apache/calcite/avatica/ConnectionSpec.java
----------------------------------------------------------------------
diff --git a/avatica-server/src/test/java/org/apache/calcite/avatica/ConnectionSpec.java b/avatica-server/src/test/java/org/apache/calcite/avatica/ConnectionSpec.java
new file mode 100644
index 0000000..ba4c5b8
--- /dev/null
+++ b/avatica-server/src/test/java/org/apache/calcite/avatica/ConnectionSpec.java
@@ -0,0 +1,55 @@
+/*
+ * 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.calcite.avatica;
+
+import net.hydromatic.scott.data.hsqldb.ScottHsqldb;
+
+import java.util.concurrent.locks.ReentrantLock;
+
+/** Information necessary to create a JDBC connection. Specify one to run
+ * tests against a different database. (hsqldb is the default.) */
+public class ConnectionSpec {
+  public final String url;
+  public final String username;
+  public final String password;
+  public final String driver;
+
+  // CALCITE-687 HSQLDB seems to fail oddly when multiple tests are run concurrently
+  private static final ReentrantLock HSQLDB_LOCK = new ReentrantLock();
+
+  public ConnectionSpec(String url, String username, String password,
+      String driver) {
+    this.url = url;
+    this.username = username;
+    this.password = password;
+    this.driver = driver;
+  }
+
+  public static final ConnectionSpec HSQLDB =
+      new ConnectionSpec(ScottHsqldb.URI, ScottHsqldb.USER,
+          ScottHsqldb.PASSWORD, "org.hsqldb.jdbcDriver");
+
+  /**
+   * Return a lock used for controlling concurrent access to the database as it has been observed
+   * that concurrent access is causing problems with HSQLDB.
+   */
+  public static ReentrantLock getDatabaseLock() {
+    return HSQLDB_LOCK;
+  }
+}
+
+// End ConnectionSpec.java

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/8f2fa3a5/avatica-server/src/test/java/org/apache/calcite/avatica/RemoteDriverTest.java
----------------------------------------------------------------------
diff --git a/avatica-server/src/test/java/org/apache/calcite/avatica/RemoteDriverTest.java b/avatica-server/src/test/java/org/apache/calcite/avatica/RemoteDriverTest.java
index 0097309..f2fa8e3 100644
--- a/avatica-server/src/test/java/org/apache/calcite/avatica/RemoteDriverTest.java
+++ b/avatica-server/src/test/java/org/apache/calcite/avatica/RemoteDriverTest.java
@@ -24,8 +24,6 @@ import org.apache.calcite.avatica.remote.Service;
 
 import com.google.common.cache.Cache;
 
-import net.hydromatic.scott.data.hsqldb.ScottHsqldb;
-
 import org.junit.Before;
 import org.junit.Ignore;
 import org.junit.Test;
@@ -124,41 +122,46 @@ public class RemoteDriverTest {
   }
 
   @Test public void testDatabaseProperties() throws Exception {
-    final Connection connection = ljs();
-    for (Meta.DatabaseProperty p : Meta.DatabaseProperty.values()) {
-      switch (p) {
-      case GET_NUMERIC_FUNCTIONS:
-        assertThat(connection.getMetaData().getNumericFunctions(),
-            equalTo("ABS,ACOS,ASIN,ATAN,ATAN2,BITAND,BITOR,BITXOR,"
-                + "CEILING,COS,COT,DEGREES,EXP,FLOOR,LOG,LOG10,MOD,"
-                + "PI,POWER,RADIANS,RAND,ROUND,ROUNDMAGIC,SIGN,SIN,"
-                + "SQRT,TAN,TRUNCATE"));
-        break;
-      case GET_SYSTEM_FUNCTIONS:
-        assertThat(connection.getMetaData().getSystemFunctions(),
-            equalTo("DATABASE,IFNULL,USER"));
-        break;
-      case GET_TIME_DATE_FUNCTIONS:
-        assertThat(connection.getMetaData().getTimeDateFunctions(),
-            equalTo("CURDATE,CURTIME,DATEDIFF,DAYNAME,DAYOFMONTH,DAYOFWEEK,"
-                + "DAYOFYEAR,HOUR,MINUTE,MONTH,MONTHNAME,NOW,QUARTER,SECOND,"
-                + "SECONDS_SINCE_MIDNIGHT,TIMESTAMPADD,TIMESTAMPDIFF,"
-                + "TO_CHAR,WEEK,YEAR"));
-        break;
-      case GET_S_Q_L_KEYWORDS:
-        assertThat(connection.getMetaData().getSQLKeywords(),
-            equalTo("")); // No SQL keywords return for HSQLDB
-        break;
-      case GET_STRING_FUNCTIONS:
-        assertThat(connection.getMetaData().getStringFunctions(),
-            equalTo("ASCII,CHAR,CONCAT,DIFFERENCE,HEXTORAW,INSERT,LCASE,"
-                + "LEFT,LENGTH,LOCATE,LTRIM,RAWTOHEX,REPEAT,REPLACE,"
-                + "RIGHT,RTRIM,SOUNDEX,SPACE,SUBSTR,UCASE"));
-        break;
-      default:
+    ConnectionSpec.getDatabaseLock().lock();
+    try {
+      final Connection connection = ljs();
+      for (Meta.DatabaseProperty p : Meta.DatabaseProperty.values()) {
+        switch (p) {
+        case GET_NUMERIC_FUNCTIONS:
+          assertThat(connection.getMetaData().getNumericFunctions(),
+              equalTo("ABS,ACOS,ASIN,ATAN,ATAN2,BITAND,BITOR,BITXOR,"
+                  + "CEILING,COS,COT,DEGREES,EXP,FLOOR,LOG,LOG10,MOD,"
+                  + "PI,POWER,RADIANS,RAND,ROUND,ROUNDMAGIC,SIGN,SIN,"
+                  + "SQRT,TAN,TRUNCATE"));
+          break;
+        case GET_SYSTEM_FUNCTIONS:
+          assertThat(connection.getMetaData().getSystemFunctions(),
+              equalTo("DATABASE,IFNULL,USER"));
+          break;
+        case GET_TIME_DATE_FUNCTIONS:
+          assertThat(connection.getMetaData().getTimeDateFunctions(),
+              equalTo("CURDATE,CURTIME,DATEDIFF,DAYNAME,DAYOFMONTH,DAYOFWEEK,"
+                  + "DAYOFYEAR,HOUR,MINUTE,MONTH,MONTHNAME,NOW,QUARTER,SECOND,"
+                  + "SECONDS_SINCE_MIDNIGHT,TIMESTAMPADD,TIMESTAMPDIFF,"
+                  + "TO_CHAR,WEEK,YEAR"));
+          break;
+        case GET_S_Q_L_KEYWORDS:
+          assertThat(connection.getMetaData().getSQLKeywords(),
+              equalTo("")); // No SQL keywords return for HSQLDB
+          break;
+        case GET_STRING_FUNCTIONS:
+          assertThat(connection.getMetaData().getStringFunctions(),
+              equalTo("ASCII,CHAR,CONCAT,DIFFERENCE,HEXTORAW,INSERT,LCASE,"
+                  + "LEFT,LENGTH,LOCATE,LTRIM,RAWTOHEX,REPEAT,REPLACE,"
+                  + "RIGHT,RTRIM,SOUNDEX,SPACE,SUBSTR,UCASE"));
+          break;
+        default:
+        }
       }
+      connection.close();
+    } finally {
+      ConnectionSpec.getDatabaseLock().unlock();
     }
-    connection.close();
   }
 
   @Test public void testTables() throws Exception {
@@ -176,20 +179,24 @@ public class RemoteDriverTest {
   }
 
   @Test public void testTypeInfo() throws Exception {
-    final Connection connection = ljs();
-    final ResultSet resultSet =
-        connection.getMetaData().getTypeInfo();
-    assertTrue(resultSet.next());
-    final ResultSetMetaData metaData = resultSet.getMetaData();
-    assertTrue(metaData.getColumnCount() >= 18);
-    assertEquals("TYPE_NAME", metaData.getColumnName(1));
-    assertEquals("DATA_TYPE", metaData.getColumnName(2));
-    assertEquals("PRECISION", metaData.getColumnName(3));
-    assertEquals("SQL_DATA_TYPE", metaData.getColumnName(16));
-    assertEquals("SQL_DATETIME_SUB", metaData.getColumnName(17));
-    assertEquals("NUM_PREC_RADIX", metaData.getColumnName(18));
-    resultSet.close();
-    connection.close();
+    ConnectionSpec.getDatabaseLock().lock();
+    try {
+      final Connection connection = ljs();
+      final ResultSet resultSet =
+          connection.getMetaData().getTypeInfo();
+      assertTrue(resultSet.next());
+      final ResultSetMetaData metaData = resultSet.getMetaData();
+      assertTrue(metaData.getColumnCount() >= 18);
+      assertEquals("TYPE_NAME", metaData.getColumnName(1));
+      assertEquals("DATA_TYPE", metaData.getColumnName(2));
+      assertEquals("PRECISION", metaData.getColumnName(3));
+      assertEquals("SQL_DATA_TYPE", metaData.getColumnName(16));
+      assertEquals("SQL_DATETIME_SUB", metaData.getColumnName(17));
+      assertEquals("NUM_PREC_RADIX", metaData.getColumnName(18));
+      resultSet.close();
+    } finally {
+      ConnectionSpec.getDatabaseLock().unlock();
+    }
   }
 
   @Ignore
@@ -224,21 +231,41 @@ public class RemoteDriverTest {
   }
 
   @Test public void testStatementExecuteQueryLocal() throws Exception {
-    checkStatementExecuteQuery(ljs(), false);
+    ConnectionSpec.getDatabaseLock().lock();
+    try {
+      checkStatementExecuteQuery(ljs(), false);
+    } finally {
+      ConnectionSpec.getDatabaseLock().unlock();
+    }
   }
 
   @Ignore
   @Test public void testStatementExecuteQueryMock() throws Exception {
-    checkStatementExecuteQuery(mjs(), false);
+    ConnectionSpec.getDatabaseLock().lock();
+    try {
+      checkStatementExecuteQuery(mjs(), false);
+    } finally {
+      ConnectionSpec.getDatabaseLock().unlock();
+    }
   }
 
   @Test public void testPrepareExecuteQueryLocal() throws Exception {
-    checkStatementExecuteQuery(ljs(), true);
+    ConnectionSpec.getDatabaseLock().lock();
+    try {
+      checkStatementExecuteQuery(ljs(), true);
+    } finally {
+      ConnectionSpec.getDatabaseLock().unlock();
+    }
   }
 
   @Ignore
   @Test public void testPrepareExecuteQueryMock() throws Exception {
-    checkStatementExecuteQuery(mjs(), true);
+    ConnectionSpec.getDatabaseLock().lock();
+    try {
+      checkStatementExecuteQuery(mjs(), true);
+    } finally {
+      ConnectionSpec.getDatabaseLock().unlock();
+    }
   }
 
   private void checkStatementExecuteQuery(Connection connection,
@@ -275,17 +302,27 @@ public class RemoteDriverTest {
   }
 
   @Test public void testStatementExecuteLocal() throws Exception {
-    checkStatementExecute(ljs(), false);
+    ConnectionSpec.getDatabaseLock().lock();
+    try {
+      checkStatementExecute(ljs(), false);
+    } finally {
+      ConnectionSpec.getDatabaseLock().unlock();
+    }
   }
 
   @Test public void testStatementExecuteFetch() throws Exception {
-    // Creating a > 100 rows queries to enable fetch request
-    String sql = "select * from emp cross join emp";
-    checkExecuteFetch(ljs(), sql, false, 1);
-    // PreparedStatement needed an extra fetch, as the execute will
-    // trigger the 1st fetch. Where statement execute will execute direct
-    // with results back.
-    checkExecuteFetch(ljs(), sql, true, 2);
+    ConnectionSpec.getDatabaseLock().lock();
+    try {
+      // Creating a > 100 rows queries to enable fetch request
+      String sql = "select * from emp cross join emp";
+      checkExecuteFetch(ljs(), sql, false, 1);
+      // PreparedStatement needed an extra fetch, as the execute will
+      // trigger the 1st fetch. Where statement execute will execute direct
+      // with results back.
+      checkExecuteFetch(ljs(), sql, true, 2);
+    } finally {
+      ConnectionSpec.getDatabaseLock().unlock();
+    }
   }
 
   private void checkExecuteFetch(Connection conn, String sql, boolean isPrepare,
@@ -320,16 +357,31 @@ public class RemoteDriverTest {
   }
 
   @Test public void testStatementExecuteLocalMaxRow() throws Exception {
-    checkStatementExecute(ljs(), false, 2);
+    ConnectionSpec.getDatabaseLock().lock();
+    try {
+      checkStatementExecute(ljs(), false, 2);
+    } finally {
+      ConnectionSpec.getDatabaseLock().unlock();
+    }
   }
 
   @Ignore("CALCITE-719: Refactor PreparedStatement to support setMaxRows")
   @Test public void testStatementPrepareExecuteLocalMaxRow() throws Exception {
-    checkStatementExecute(ljs(), true, 2);
+    ConnectionSpec.getDatabaseLock().lock();
+    try {
+      checkStatementExecute(ljs(), true, 2);
+    } finally {
+      ConnectionSpec.getDatabaseLock().unlock();
+    }
   }
 
   @Test public void testPrepareExecuteLocal() throws Exception {
-    checkStatementExecute(ljs(), true);
+    ConnectionSpec.getDatabaseLock().lock();
+    try {
+      checkStatementExecute(ljs(), true);
+    } finally {
+      ConnectionSpec.getDatabaseLock().unlock();
+    }
   }
 
   private void checkStatementExecute(Connection connection,
@@ -374,6 +426,7 @@ public class RemoteDriverTest {
   }
 
   @Test public void testCreateInsertUpdateDrop() throws Exception {
+    ConnectionSpec.getDatabaseLock().lock();
     final String drop = "drop table TEST_TABLE if exists";
     final String create = "create table TEST_TABLE("
         + "id int not null, "
@@ -470,18 +523,25 @@ public class RemoteDriverTest {
           }
         }
       }
+    } finally {
+      ConnectionSpec.getDatabaseLock().unlock();
     }
   }
 
   @Test public void testTypeHandling() throws Exception {
-    final String query = "select * from EMP";
-    try (Connection cannon = canon();
-        Connection underTest = ljs();
-        Statement s1 = cannon.createStatement();
-        Statement s2 = underTest.createStatement()) {
-      assertTrue(s1.execute(query));
-      assertTrue(s2.execute(query));
-      assertResultSetsEqual(s1, s2);
+    ConnectionSpec.getDatabaseLock().lock();
+    try {
+      final String query = "select * from EMP";
+      try (Connection cannon = canon();
+          Connection underTest = ljs();
+          Statement s1 = cannon.createStatement();
+          Statement s2 = underTest.createStatement()) {
+        assertTrue(s1.execute(query));
+        assertTrue(s2.execute(query));
+        assertResultSetsEqual(s1, s2);
+      }
+    } finally {
+      ConnectionSpec.getDatabaseLock().unlock();
     }
   }
 
@@ -556,24 +616,29 @@ public class RemoteDriverTest {
   }
 
   @Test public void testSetParameter() throws Exception {
-    checkSetParameter("select ? from (values 1)",
-        new PreparedStatementFunction() {
-          public void apply(PreparedStatement s1, PreparedStatement s2)
-              throws SQLException {
-            final Date d = new Date(1234567890);
-            s1.setDate(1, d);
-            s2.setDate(1, d);
-          }
-        });
-    checkSetParameter("select ? from (values 1)",
-        new PreparedStatementFunction() {
-          public void apply(PreparedStatement s1, PreparedStatement s2)
-              throws SQLException {
-            final Timestamp ts = new Timestamp(123456789012L);
-            s1.setTimestamp(1, ts);
-            s2.setTimestamp(1, ts);
-          }
-        });
+    ConnectionSpec.getDatabaseLock().lock();
+    try {
+      checkSetParameter("select ? from (values 1)",
+          new PreparedStatementFunction() {
+            public void apply(PreparedStatement s1, PreparedStatement s2)
+                throws SQLException {
+              final Date d = new Date(1234567890);
+              s1.setDate(1, d);
+              s2.setDate(1, d);
+            }
+          });
+      checkSetParameter("select ? from (values 1)",
+          new PreparedStatementFunction() {
+            public void apply(PreparedStatement s1, PreparedStatement s2)
+                throws SQLException {
+              final Timestamp ts = new Timestamp(123456789012L);
+              s1.setTimestamp(1, ts);
+              s2.setTimestamp(1, ts);
+            }
+          });
+    } finally {
+      ConnectionSpec.getDatabaseLock().unlock();
+    }
   }
 
   void checkSetParameter(String query, PreparedStatementFunction fn)
@@ -589,12 +654,15 @@ public class RemoteDriverTest {
     }
   }
 
-  @Ignore("[CALCITE-687] Make RemoteDriverTest.testStatementLifecycle thread-safe")
   @Test public void testStatementLifecycle() throws Exception {
+    ConnectionSpec.getDatabaseLock().lock();
     try (AvaticaConnection connection = (AvaticaConnection) ljs()) {
       Map<Integer, AvaticaStatement> clientMap = connection.statementMap;
       Cache<Integer, Object> serverMap =
           QuasiRemoteJdbcServiceFactory.getRemoteStatementMap(connection);
+      // Other tests being run might leave statements in the cache.
+      // The lock guards against more statements being cached during the test.
+      serverMap.invalidateAll();
       assertEquals(0, clientMap.size());
       assertEquals(0, serverMap.size());
       Statement stmt = connection.createStatement();
@@ -603,44 +671,56 @@ public class RemoteDriverTest {
       stmt.close();
       assertEquals(0, clientMap.size());
       assertEquals(0, serverMap.size());
+    } finally {
+      ConnectionSpec.getDatabaseLock().unlock();
     }
   }
 
   @Test public void testConnectionIsolation() throws Exception {
-    final String sql = "select * from (values (1, 'a'))";
-    Connection conn1 = ljs();
-    Connection conn2 = ljs();
-    Cache<String, Connection> connectionMap =
-        QuasiRemoteJdbcServiceFactory.getRemoteConnectionMap(
-            (AvaticaConnection) conn1);
-    assertEquals("connection cache should start empty",
-        0, connectionMap.size());
-    PreparedStatement conn1stmt1 = conn1.prepareStatement(sql);
-    assertEquals(
-        "statement creation implicitly creates a connection server-side",
-        1, connectionMap.size());
-    PreparedStatement conn2stmt1 = conn2.prepareStatement(sql);
-    assertEquals(
-        "statement creation implicitly creates a connection server-side",
-        2, connectionMap.size());
-    AvaticaPreparedStatement s1 = (AvaticaPreparedStatement) conn1stmt1;
-    AvaticaPreparedStatement s2 = (AvaticaPreparedStatement) conn2stmt1;
-    assertFalse("connection id's should be unique",
-        s1.handle.connectionId.equalsIgnoreCase(s2.handle.connectionId));
-    conn2.close();
-    assertEquals("closing a connection closes the server-side connection",
-        1, connectionMap.size());
-    conn1.close();
-    assertEquals("closing a connection closes the server-side connection",
-        0, connectionMap.size());
+    ConnectionSpec.getDatabaseLock().lock();
+    try {
+      final String sql = "select * from (values (1, 'a'))";
+      Connection conn1 = ljs();
+      Connection conn2 = ljs();
+      Cache<String, Connection> connectionMap =
+          QuasiRemoteJdbcServiceFactory.getRemoteConnectionMap(
+              (AvaticaConnection) conn1);
+      assertEquals("connection cache should start empty",
+          0, connectionMap.size());
+      PreparedStatement conn1stmt1 = conn1.prepareStatement(sql);
+      assertEquals(
+          "statement creation implicitly creates a connection server-side",
+          1, connectionMap.size());
+      PreparedStatement conn2stmt1 = conn2.prepareStatement(sql);
+      assertEquals(
+          "statement creation implicitly creates a connection server-side",
+          2, connectionMap.size());
+      AvaticaPreparedStatement s1 = (AvaticaPreparedStatement) conn1stmt1;
+      AvaticaPreparedStatement s2 = (AvaticaPreparedStatement) conn2stmt1;
+      assertFalse("connection id's should be unique",
+          s1.handle.connectionId.equalsIgnoreCase(s2.handle.connectionId));
+      conn2.close();
+      assertEquals("closing a connection closes the server-side connection",
+          1, connectionMap.size());
+      conn1.close();
+      assertEquals("closing a connection closes the server-side connection",
+          0, connectionMap.size());
+    } finally {
+      ConnectionSpec.getDatabaseLock().unlock();
+    }
   }
 
   @Test public void testPrepareBindExecuteFetch() throws Exception {
-    LoggingLocalJsonService.THREAD_LOG.get().enableAndClear();
-    checkPrepareBindExecuteFetch(ljs());
-    List<String[]> x = LoggingLocalJsonService.THREAD_LOG.get().getAndDisable();
-    for (String[] pair : x) {
-      System.out.println(pair[0] + "=" + pair[1]);
+    ConnectionSpec.getDatabaseLock().lock();
+    try {
+      LoggingLocalJsonService.THREAD_LOG.get().enableAndClear();
+      checkPrepareBindExecuteFetch(ljs());
+      List<String[]> x = LoggingLocalJsonService.THREAD_LOG.get().getAndDisable();
+      for (String[] pair : x) {
+        System.out.println(pair[0] + "=" + pair[1]);
+      }
+    } finally {
+      ConnectionSpec.getDatabaseLock().unlock();
     }
   }
 
@@ -694,30 +774,40 @@ public class RemoteDriverTest {
   }
 
   @Test public void testPrepareBindExecuteFetchVarbinary() throws Exception {
-    final Connection connection = ljs();
-    final String sql = "select x'de' || ? as c from (values (1, 'a'))";
-    final PreparedStatement ps =
-        connection.prepareStatement(sql);
-    final ParameterMetaData parameterMetaData = ps.getParameterMetaData();
-    assertThat(parameterMetaData.getParameterCount(), equalTo(1));
-
-    ps.setBytes(1, new byte[]{65, 0, 66});
-    final ResultSet resultSet = ps.executeQuery();
-    assertTrue(resultSet.next());
-    assertThat(resultSet.getBytes(1),
-        equalTo(new byte[]{(byte) 0xDE, 65, 0, 66}));
-    resultSet.close();
-    ps.close();
-    connection.close();
+    ConnectionSpec.getDatabaseLock().lock();
+    try {
+      final Connection connection = ljs();
+      final String sql = "select x'de' || ? as c from (values (1, 'a'))";
+      final PreparedStatement ps =
+          connection.prepareStatement(sql);
+      final ParameterMetaData parameterMetaData = ps.getParameterMetaData();
+      assertThat(parameterMetaData.getParameterCount(), equalTo(1));
+
+      ps.setBytes(1, new byte[]{65, 0, 66});
+      final ResultSet resultSet = ps.executeQuery();
+      assertTrue(resultSet.next());
+      assertThat(resultSet.getBytes(1),
+          equalTo(new byte[]{(byte) 0xDE, 65, 0, 66}));
+      resultSet.close();
+      ps.close();
+      connection.close();
+    } finally {
+      ConnectionSpec.getDatabaseLock().unlock();
+    }
   }
 
   @Test public void testPrepareBindExecuteFetchDate() throws Exception {
-    eachConnection(
-        new ConnectionFunction() {
-          public void apply(Connection c1) throws Exception {
-            checkPrepareBindExecuteFetchDate(c1);
-          }
-        });
+    ConnectionSpec.getDatabaseLock().lock();
+    try {
+      eachConnection(
+          new ConnectionFunction() {
+            public void apply(Connection c1) throws Exception {
+              checkPrepareBindExecuteFetchDate(c1);
+            }
+          });
+    } finally {
+      ConnectionSpec.getDatabaseLock().unlock();
+    }
   }
 
   private void checkPrepareBindExecuteFetchDate(Connection connection) throws Exception {
@@ -786,12 +876,17 @@ public class RemoteDriverTest {
   }
 
   @Test public void testDatabaseProperty() throws Exception {
-    eachConnection(
-        new ConnectionFunction() {
-          public void apply(Connection c1) throws Exception {
-            checkDatabaseProperty(c1);
-          }
-        });
+    ConnectionSpec.getDatabaseLock().lock();
+    try {
+      eachConnection(
+          new ConnectionFunction() {
+            public void apply(Connection c1) throws Exception {
+              checkDatabaseProperty(c1);
+            }
+          });
+    } finally {
+      ConnectionSpec.getDatabaseLock().unlock();
+    }
   }
 
   private void checkDatabaseProperty(Connection connection)
@@ -858,7 +953,8 @@ public class RemoteDriverTest {
       Field remoteMetaServiceF = clientMeta.getClass().getDeclaredField("service");
       remoteMetaServiceF.setAccessible(true);
       LocalJsonService remoteMetaService = (LocalJsonService) remoteMetaServiceF.get(clientMeta);
-      Field remoteMetaServiceServiceF = remoteMetaService.getClass().getDeclaredField("service");
+      // Use the explicitly class to avoid issues with LoggingLocalJsonService
+      Field remoteMetaServiceServiceF = LocalJsonService.class.getDeclaredField("service");
       remoteMetaServiceServiceF.setAccessible(true);
       LocalService remoteMetaServiceService =
           (LocalService) remoteMetaServiceServiceF.get(remoteMetaService);
@@ -885,7 +981,8 @@ public class RemoteDriverTest {
       Field remoteMetaServiceF = clientMeta.getClass().getDeclaredField("service");
       remoteMetaServiceF.setAccessible(true);
       LocalJsonService remoteMetaService = (LocalJsonService) remoteMetaServiceF.get(clientMeta);
-      Field remoteMetaServiceServiceF = remoteMetaService.getClass().getDeclaredField("service");
+      // Get the field explicitly off the correct class to avoid LocalLoggingJsonService.class
+      Field remoteMetaServiceServiceF = LocalJsonService.class.getDeclaredField("service");
       remoteMetaServiceServiceF.setAccessible(true);
       LocalService remoteMetaServiceService =
           (LocalService) remoteMetaServiceServiceF.get(remoteMetaService);
@@ -954,27 +1051,6 @@ public class RemoteDriverTest {
       return new ArrayList<>(requestResponses);
     }
   }
-
-  /** Information necessary to create a JDBC connection. Specify one to run
-   * tests against a different database. (hsqldb is the default.) */
-  public static class ConnectionSpec {
-    public final String url;
-    public final String username;
-    public final String password;
-    public final String driver;
-
-    public ConnectionSpec(String url, String username, String password,
-        String driver) {
-      this.url = url;
-      this.username = username;
-      this.password = password;
-      this.driver = driver;
-    }
-
-    public static final ConnectionSpec HSQLDB =
-        new ConnectionSpec(ScottHsqldb.URI, ScottHsqldb.USER,
-            ScottHsqldb.PASSWORD, "org.hsqldb.jdbcDriver");
-  }
 }
 
 // End RemoteDriverTest.java

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/8f2fa3a5/avatica-server/src/test/java/org/apache/calcite/avatica/remote/RemoteMetaTest.java
----------------------------------------------------------------------
diff --git a/avatica-server/src/test/java/org/apache/calcite/avatica/remote/RemoteMetaTest.java b/avatica-server/src/test/java/org/apache/calcite/avatica/remote/RemoteMetaTest.java
index 2a9e846..4cef99e 100644
--- a/avatica-server/src/test/java/org/apache/calcite/avatica/remote/RemoteMetaTest.java
+++ b/avatica-server/src/test/java/org/apache/calcite/avatica/remote/RemoteMetaTest.java
@@ -19,8 +19,8 @@ package org.apache.calcite.avatica.remote;
 import org.apache.calcite.avatica.AvaticaConnection;
 import org.apache.calcite.avatica.AvaticaStatement;
 import org.apache.calcite.avatica.ConnectionPropertiesImpl;
+import org.apache.calcite.avatica.ConnectionSpec;
 import org.apache.calcite.avatica.Meta;
-import org.apache.calcite.avatica.RemoteDriverTest;
 import org.apache.calcite.avatica.jdbc.JdbcMeta;
 import org.apache.calcite.avatica.server.HttpServer;
 import org.apache.calcite.avatica.server.Main;
@@ -49,8 +49,7 @@ import static org.junit.Assert.assertTrue;
 
 /** Tests covering {@link RemoteMeta}. */
 public class RemoteMetaTest {
-  private static final RemoteDriverTest.ConnectionSpec CONNECTION_SPEC =
-      RemoteDriverTest.ConnectionSpec.HSQLDB;
+  private static final ConnectionSpec CONNECTION_SPEC = ConnectionSpec.HSQLDB;
 
   private static HttpServer start;
   private static String url;
@@ -113,6 +112,7 @@ public class RemoteMetaTest {
   }
 
   @Test public void testRemoteExecuteMaxRowCount() throws Exception {
+    ConnectionSpec.getDatabaseLock().lock();
     try (AvaticaConnection conn = (AvaticaConnection) DriverManager.getConnection(url)) {
       final AvaticaStatement statement = conn.createStatement();
       prepareAndExecuteInternal(conn, statement,
@@ -128,6 +128,8 @@ public class RemoteMetaTest {
       rs.close();
       statement.close();
       conn.close();
+    } finally {
+      ConnectionSpec.getDatabaseLock().unlock();
     }
   }
 
@@ -135,12 +137,17 @@ public class RemoteMetaTest {
    * <a href="https://issues.apache.org/jira/browse/CALCITE-780">[CALCITE-780]
    * HTTP error 413 when sending a long string to the Avatica server</a>. */
   @Test public void testRemoteExecuteVeryLargeQuery() throws Exception {
-    // Before the bug was fixed, a value over 7998 caused an HTTP 413.
-    // 16K bytes, I guess.
-    checkLargeQuery(8);
-    checkLargeQuery(240);
-    checkLargeQuery(8000);
-    checkLargeQuery(240000);
+    ConnectionSpec.getDatabaseLock().lock();
+    try {
+      // Before the bug was fixed, a value over 7998 caused an HTTP 413.
+      // 16K bytes, I guess.
+      checkLargeQuery(8);
+      checkLargeQuery(240);
+      checkLargeQuery(8000);
+      checkLargeQuery(240000);
+    } finally {
+      ConnectionSpec.getDatabaseLock().unlock();
+    }
   }
 
   private void checkLargeQuery(int n) throws Exception {
@@ -179,6 +186,7 @@ public class RemoteMetaTest {
   }
 
   @Test public void testRemoteConnectionProperties() throws Exception {
+    ConnectionSpec.getDatabaseLock().lock();
     try (AvaticaConnection conn = (AvaticaConnection) DriverManager.getConnection(url)) {
       String id = conn.id;
       final Map<String, ConnectionPropertiesImpl> m = ((RemoteMeta) getMeta(conn)).propsMap;
@@ -205,6 +213,8 @@ public class RemoteMetaTest {
         assertEquals(!defaultAutoCommit, remoteConn.getAutoCommit());
         assertFalse("local values should be clean", m.get(id).isDirty());
       }
+    } finally {
+      ConnectionSpec.getDatabaseLock().unlock();
     }
   }
 }