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 2018/11/19 05:39:12 UTC

[1/2] calcite git commit: [CALCITE-2678] RelBuilderTest#testRelBuilderToString fails on Windows (Stamatis Zampetakis)

Repository: calcite
Updated Branches:
  refs/heads/master bd956458f -> 53e15af6c


[CALCITE-2678] RelBuilderTest#testRelBuilderToString fails on Windows (Stamatis Zampetakis)

Close apache/calcite#924


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

Branch: refs/heads/master
Commit: 02ca9bc995cac5b4b97855a4d06df46e632d7c22
Parents: bd95645
Author: Stamatis Zampetakis <za...@gmail.com>
Authored: Fri Nov 16 16:11:16 2018 +0100
Committer: Julian Hyde <jh...@apache.org>
Committed: Fri Nov 16 20:13:26 2018 -0800

----------------------------------------------------------------------
 core/src/test/java/org/apache/calcite/test/RelBuilderTest.java | 6 +++---
 1 file changed, 3 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/calcite/blob/02ca9bc9/core/src/test/java/org/apache/calcite/test/RelBuilderTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/calcite/test/RelBuilderTest.java b/core/src/test/java/org/apache/calcite/test/RelBuilderTest.java
index e6cfe62..ff3f861 100644
--- a/core/src/test/java/org/apache/calcite/test/RelBuilderTest.java
+++ b/core/src/test/java/org/apache/calcite/test/RelBuilderTest.java
@@ -2196,20 +2196,20 @@ public class RelBuilderTest {
 
     // One entry on the stack, a single-node tree
     final String expected1 = "LogicalTableScan(table=[[scott, EMP]])\n";
-    assertThat(builder.toString(), is(expected1));
+    assertThat(Util.toLinux(builder.toString()), is(expected1));
 
     // One entry on the stack, a two-node tree
     builder.filter(builder.equals(builder.field(2), builder.literal(3)));
     final String expected2 = "LogicalFilter(condition=[=($2, 3)])\n"
         + "  LogicalTableScan(table=[[scott, EMP]])\n";
-    assertThat(builder.toString(), is(expected2));
+    assertThat(Util.toLinux(builder.toString()), is(expected2));
 
     // Two entries on the stack
     builder.scan("DEPT");
     final String expected3 = "LogicalTableScan(table=[[scott, DEPT]])\n"
         + "LogicalFilter(condition=[=($2, 3)])\n"
         + "  LogicalTableScan(table=[[scott, EMP]])\n";
-    assertThat(builder.toString(), is(expected3));
+    assertThat(Util.toLinux(builder.toString()), is(expected3));
   }
 
   /**


[2/2] calcite git commit: [CALCITE-563] In JDBC adapter, push bindable parameters down to the underlying JDBC data source (Vladimir Sitnikov, Piotr Bojko)

Posted by jh...@apache.org.
[CALCITE-563] In JDBC adapter, push bindable parameters down to the underlying JDBC data source (Vladimir Sitnikov, Piotr Bojko)

Fix up, moving JdbcPreparedStatementUtils methods into
ResultSetEnumerable, and fixing some comments and typos. (Julian Hyde)

Close apache/calcite#907
Close apache/calcite#902


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

Branch: refs/heads/master
Commit: 53e15af6c5e8e782b2edcd7f5bf4f5f32225d110
Parents: 02ca9bc
Author: Vladimir Sitnikov <si...@gmail.com>
Authored: Mon Nov 5 18:57:50 2018 +0300
Committer: Julian Hyde <jh...@apache.org>
Committed: Sun Nov 18 16:15:42 2018 -0800

----------------------------------------------------------------------
 .../adapter/jdbc/JdbcToEnumerableConverter.java |  63 ++++--
 .../calcite/prepare/CalcitePrepareImpl.java     |   2 +-
 .../apache/calcite/runtime/CalciteResource.java |   3 +
 .../calcite/runtime/ResultSetEnumerable.java    | 193 +++++++++++++++++--
 .../org/apache/calcite/sql/SqlDynamicParam.java |   3 +-
 .../java/org/apache/calcite/sql/SqlNode.java    |   3 +-
 .../java/org/apache/calcite/sql/SqlWriter.java  |   5 +
 .../calcite/sql/pretty/SqlPrettyWriter.java     |  17 +-
 .../org/apache/calcite/sql/util/SqlString.java  |  42 +++-
 .../org/apache/calcite/util/BuiltInMethod.java  |   5 +
 .../calcite/runtime/CalciteResource.properties  |   1 +
 .../org/apache/calcite/test/CalciteAssert.java  | 113 ++++++++++-
 .../apache/calcite/test/JdbcAdapterTest.java    |  11 ++
 .../java/org/apache/calcite/test/JdbcTest.java  |  53 +++++
 plus/pom.xml                                    |   9 +
 .../chinook/CalciteConnectionProvider.java      |   4 +-
 .../calcite/chinook/ChinookAvaticaServer.java   | 108 +++++++++++
 .../calcite/chinook/ConnectionFactory.java      |   4 +-
 .../RemotePreparedStatementParametersTest.java  |  76 ++++++++
 .../java/org/apache/calcite/test/PlusSuite.java |   4 +-
 20 files changed, 658 insertions(+), 61 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/calcite/blob/53e15af6/core/src/main/java/org/apache/calcite/adapter/jdbc/JdbcToEnumerableConverter.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/adapter/jdbc/JdbcToEnumerableConverter.java b/core/src/main/java/org/apache/calcite/adapter/jdbc/JdbcToEnumerableConverter.java
index b018132..a43d2b1 100644
--- a/core/src/main/java/org/apache/calcite/adapter/jdbc/JdbcToEnumerableConverter.java
+++ b/core/src/main/java/org/apache/calcite/adapter/jdbc/JdbcToEnumerableConverter.java
@@ -16,6 +16,7 @@
  */
 package org.apache.calcite.adapter.jdbc;
 
+import org.apache.calcite.DataContext;
 import org.apache.calcite.adapter.enumerable.EnumerableRel;
 import org.apache.calcite.adapter.enumerable.EnumerableRelImplementor;
 import org.apache.calcite.adapter.enumerable.JavaRowFormat;
@@ -23,6 +24,7 @@ import org.apache.calcite.adapter.enumerable.PhysType;
 import org.apache.calcite.adapter.enumerable.PhysTypeImpl;
 import org.apache.calcite.adapter.java.JavaTypeFactory;
 import org.apache.calcite.linq4j.tree.BlockBuilder;
+import org.apache.calcite.linq4j.tree.ConstantExpression;
 import org.apache.calcite.linq4j.tree.Expression;
 import org.apache.calcite.linq4j.tree.Expressions;
 import org.apache.calcite.linq4j.tree.ParameterExpression;
@@ -43,6 +45,7 @@ import org.apache.calcite.runtime.SqlFunctions;
 import org.apache.calcite.schema.Schemas;
 import org.apache.calcite.sql.SqlDialect;
 import org.apache.calcite.sql.type.SqlTypeName;
+import org.apache.calcite.sql.util.SqlString;
 import org.apache.calcite.util.BuiltInMethod;
 
 import java.lang.reflect.Method;
@@ -53,6 +56,7 @@ import java.util.ArrayList;
 import java.util.Calendar;
 import java.util.List;
 import java.util.TimeZone;
+import java.util.stream.Collectors;
 
 /**
  * Relational expression representing a scan of a table in a JDBC data source.
@@ -88,7 +92,8 @@ public class JdbcToEnumerableConverter
             pref.prefer(JavaRowFormat.CUSTOM));
     final JdbcConvention jdbcConvention =
         (JdbcConvention) child.getConvention();
-    String sql = generateSql(jdbcConvention.dialect);
+    SqlString sqlString = generateSql(jdbcConvention.dialect);
+    String sql = sqlString.getSql();
     if (CalcitePrepareImpl.DEBUG) {
       System.out.println("[" + sql + "]");
     }
@@ -151,22 +156,52 @@ public class JdbcToEnumerableConverter
                                                 RuntimeException.class,
                                                 e_)))))))),
                 resultSet_));
-    final Expression enumerable =
-        builder0.append(
-            "enumerable",
-            Expressions.call(
-                BuiltInMethod.RESULT_SET_ENUMERABLE_OF.method,
-                Expressions.call(
-                    Schemas.unwrap(jdbcConvention.expression,
-                        JdbcSchema.class),
-                    BuiltInMethod.JDBC_SCHEMA_DATA_SOURCE.method),
-                sql_,
-                rowBuilderFactory_));
+
+    final Expression enumerable;
+
+    if (sqlString.getDynamicParameters() != null
+        && !sqlString.getDynamicParameters().isEmpty()) {
+      final Expression preparedStatementConsumer_ =
+          builder0.append("preparedStatementConsumer",
+              Expressions.call(BuiltInMethod.CREATE_ENRICHER.method,
+                  Expressions.newArrayInit(Integer.class, 1,
+                      toIndexesTableExpression(sqlString)),
+                  DataContext.ROOT));
+
+      enumerable = builder0.append("enumerable",
+          Expressions.call(
+              BuiltInMethod.RESULT_SET_ENUMERABLE_OF_PREPARED.method,
+              Expressions.call(
+                  Schemas.unwrap(jdbcConvention.expression,
+                      JdbcSchema.class),
+                  BuiltInMethod.JDBC_SCHEMA_DATA_SOURCE.method),
+              sql_,
+              rowBuilderFactory_,
+              preparedStatementConsumer_));
+    } else {
+      enumerable = builder0.append(
+          "enumerable",
+          Expressions.call(
+              BuiltInMethod.RESULT_SET_ENUMERABLE_OF.method,
+              Expressions.call(
+                  Schemas.unwrap(jdbcConvention.expression,
+                      JdbcSchema.class),
+                  BuiltInMethod.JDBC_SCHEMA_DATA_SOURCE.method),
+              sql_,
+              rowBuilderFactory_));
+    }
+
     builder0.add(
         Expressions.return_(null, enumerable));
     return implementor.result(physType, builder0.toBlock());
   }
 
+  private List<ConstantExpression> toIndexesTableExpression(SqlString sqlString) {
+    return sqlString.getDynamicParameters().stream()
+        .map(Expressions::constant)
+        .collect(Collectors.toList());
+  }
+
   private UnaryExpression getTimeZoneExpression(
       EnumerableRelImplementor implementor) {
     return Expressions.convert_(
@@ -292,13 +327,13 @@ public class JdbcToEnumerableConverter
         : "get" + SqlFunctions.initcap(primitive.primitiveName);
   }
 
-  private String generateSql(SqlDialect dialect) {
+  private SqlString generateSql(SqlDialect dialect) {
     final JdbcImplementor jdbcImplementor =
         new JdbcImplementor(dialect,
             (JavaTypeFactory) getCluster().getTypeFactory());
     final JdbcImplementor.Result result =
         jdbcImplementor.visitChild(0, getInput());
-    return result.asStatement().toSqlString(dialect).getSql();
+    return result.asStatement().toSqlString(dialect);
   }
 }
 

http://git-wip-us.apache.org/repos/asf/calcite/blob/53e15af6/core/src/main/java/org/apache/calcite/prepare/CalcitePrepareImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/prepare/CalcitePrepareImpl.java b/core/src/main/java/org/apache/calcite/prepare/CalcitePrepareImpl.java
index 55db902..90979c4 100644
--- a/core/src/main/java/org/apache/calcite/prepare/CalcitePrepareImpl.java
+++ b/core/src/main/java/org/apache/calcite/prepare/CalcitePrepareImpl.java
@@ -943,7 +943,7 @@ public class CalcitePrepareImpl implements CalcitePrepare {
   }
 
   private static String getClassName(RelDataType type) {
-    return null;
+    return Object.class.getName(); // CALCITE-2613
   }
 
   private static int getScale(RelDataType type) {

http://git-wip-us.apache.org/repos/asf/calcite/blob/53e15af6/core/src/main/java/org/apache/calcite/runtime/CalciteResource.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/runtime/CalciteResource.java b/core/src/main/java/org/apache/calcite/runtime/CalciteResource.java
index 14089c5..ee0953d 100644
--- a/core/src/main/java/org/apache/calcite/runtime/CalciteResource.java
+++ b/core/src/main/java/org/apache/calcite/runtime/CalciteResource.java
@@ -846,6 +846,9 @@ public interface CalciteResource {
 
   @BaseMessage("Null key of JSON object is not allowed")
   ExInst<CalciteException> nullKeyOfJsonObjectNotAllowed();
+
+  @BaseMessage("While executing SQL [{0}] on JDBC sub-schema")
+  ExInst<RuntimeException> exceptionWhilePerformingQueryOnJdbcSubSchema(String sql);
 }
 
 // End CalciteResource.java

http://git-wip-us.apache.org/repos/asf/calcite/blob/53e15af6/core/src/main/java/org/apache/calcite/runtime/ResultSetEnumerable.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/runtime/ResultSetEnumerable.java b/core/src/main/java/org/apache/calcite/runtime/ResultSetEnumerable.java
index 771772f..52c11f9 100644
--- a/core/src/main/java/org/apache/calcite/runtime/ResultSetEnumerable.java
+++ b/core/src/main/java/org/apache/calcite/runtime/ResultSetEnumerable.java
@@ -16,6 +16,8 @@
  */
 package org.apache.calcite.runtime;
 
+import org.apache.calcite.DataContext;
+import org.apache.calcite.avatica.SqlType;
 import org.apache.calcite.linq4j.AbstractEnumerable;
 import org.apache.calcite.linq4j.Enumerable;
 import org.apache.calcite.linq4j.Enumerator;
@@ -23,16 +25,29 @@ import org.apache.calcite.linq4j.Linq4j;
 import org.apache.calcite.linq4j.function.Function0;
 import org.apache.calcite.linq4j.function.Function1;
 import org.apache.calcite.linq4j.tree.Primitive;
+import org.apache.calcite.util.Static;
 
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import java.math.BigDecimal;
+import java.net.URL;
+import java.sql.Blob;
+import java.sql.Clob;
 import java.sql.Connection;
+import java.sql.Date;
+import java.sql.NClob;
+import java.sql.PreparedStatement;
+import java.sql.Ref;
 import java.sql.ResultSet;
 import java.sql.ResultSetMetaData;
+import java.sql.RowId;
 import java.sql.SQLException;
 import java.sql.SQLFeatureNotSupportedException;
+import java.sql.SQLXML;
 import java.sql.Statement;
+import java.sql.Time;
+import java.sql.Timestamp;
 import java.sql.Types;
 import java.util.ArrayList;
 import java.util.List;
@@ -47,6 +62,8 @@ public class ResultSetEnumerable<T> extends AbstractEnumerable<T> {
   private final DataSource dataSource;
   private final String sql;
   private final Function1<ResultSet, Function0<T>> rowBuilderFactory;
+  private final PreparedStatementEnricher preparedStatementEnricher;
+
   private static final Logger LOGGER = LoggerFactory.getLogger(
       ResultSetEnumerable.class);
 
@@ -96,10 +113,19 @@ public class ResultSetEnumerable<T> extends AbstractEnumerable<T> {
   private ResultSetEnumerable(
       DataSource dataSource,
       String sql,
-      Function1<ResultSet, Function0<T>> rowBuilderFactory) {
+      Function1<ResultSet, Function0<T>> rowBuilderFactory,
+      PreparedStatementEnricher preparedStatementEnricher) {
     this.dataSource = dataSource;
     this.sql = sql;
     this.rowBuilderFactory = rowBuilderFactory;
+    this.preparedStatementEnricher = preparedStatementEnricher;
+  }
+
+  private ResultSetEnumerable(
+      DataSource dataSource,
+      String sql,
+      Function1<ResultSet, Function0<T>> rowBuilderFactory) {
+    this(dataSource, sql, rowBuilderFactory, null);
   }
 
   /** Creates an ResultSetEnumerable. */
@@ -123,17 +149,100 @@ public class ResultSetEnumerable<T> extends AbstractEnumerable<T> {
     return new ResultSetEnumerable<>(dataSource, sql, rowBuilderFactory);
   }
 
+  /** Executes a SQL query and returns the results as an enumerator, using a
+   * row builder to convert JDBC column values into rows.
+   *
+   * <p>It uses a {@link PreparedStatement} for computing the query result,
+   * and that means that it can bind parameters. */
+  public static <T> Enumerable<T> of(
+      DataSource dataSource,
+      String sql,
+      Function1<ResultSet, Function0<T>> rowBuilderFactory,
+      PreparedStatementEnricher consumer) {
+    return new ResultSetEnumerable<>(dataSource, sql, rowBuilderFactory, consumer);
+  }
+
+  /** Called from generated code that proposes to create a
+   * {@code ResultSetEnumerable} over a prepared statement. */
+  public static PreparedStatementEnricher createEnricher(Integer[] indexes,
+      DataContext context) {
+    return preparedStatement -> {
+      for (int i = 0; i < indexes.length; i++) {
+        final int index = indexes[i];
+        setDynamicParam(preparedStatement, i + 1,
+            context.get("?" + index));
+      }
+    };
+  }
+
+  /** Assigns a value to a dynamic parameter in a prepared statement, calling
+   * the appropriate {@code setXxx} method based on the type of the value. */
+  private static void setDynamicParam(PreparedStatement preparedStatement,
+      int i, Object value) throws SQLException {
+    if (value == null) {
+      preparedStatement.setObject(i, null, SqlType.ANY.id);
+    } else if (value instanceof Timestamp) {
+      preparedStatement.setTimestamp(i, (Timestamp) value);
+    } else if (value instanceof Time) {
+      preparedStatement.setTime(i, (Time) value);
+    } else if (value instanceof String) {
+      preparedStatement.setString(i, (String) value);
+    } else if (value instanceof Integer) {
+      preparedStatement.setInt(i, (Integer) value);
+    } else if (value instanceof Double) {
+      preparedStatement.setDouble(i, (Double) value);
+    } else if (value instanceof java.sql.Array) {
+      preparedStatement.setArray(i, (java.sql.Array) value);
+    } else if (value instanceof BigDecimal) {
+      preparedStatement.setBigDecimal(i, (BigDecimal) value);
+    } else if (value instanceof Boolean) {
+      preparedStatement.setBoolean(i, (Boolean) value);
+    } else if (value instanceof Blob) {
+      preparedStatement.setBlob(i, (Blob) value);
+    } else if (value instanceof Byte) {
+      preparedStatement.setByte(i, (Byte) value);
+    } else if (value instanceof NClob) {
+      preparedStatement.setNClob(i, (NClob) value);
+    } else if (value instanceof Clob) {
+      preparedStatement.setClob(i, (Clob) value);
+    } else if (value instanceof byte[]) {
+      preparedStatement.setBytes(i, (byte[]) value);
+    } else if (value instanceof Date) {
+      preparedStatement.setDate(i, (Date) value);
+    } else if (value instanceof Float) {
+      preparedStatement.setFloat(i, (Float) value);
+    } else if (value instanceof Long) {
+      preparedStatement.setLong(i, (Long) value);
+    } else if (value instanceof Ref) {
+      preparedStatement.setRef(i, (Ref) value);
+    } else if (value instanceof RowId) {
+      preparedStatement.setRowId(i, (RowId) value);
+    } else if (value instanceof Short) {
+      preparedStatement.setShort(i, (Short) value);
+    } else if (value instanceof URL) {
+      preparedStatement.setURL(i, (URL) value);
+    } else if (value instanceof SQLXML) {
+      preparedStatement.setSQLXML(i, (SQLXML) value);
+    } else {
+      preparedStatement.setObject(i, value);
+    }
+  }
+
   public Enumerator<T> enumerator() {
+    if (preparedStatementEnricher == null) {
+      return enumeratorBasedOnStatement();
+    } else {
+      return enumeratorBasedOnPreparedStatement();
+    }
+  }
+
+  private Enumerator<T> enumeratorBasedOnStatement() {
     Connection connection = null;
     Statement statement = null;
     try {
       connection = dataSource.getConnection();
       statement = connection.createStatement();
-      try {
-        statement.setQueryTimeout(10);
-      } catch (SQLFeatureNotSupportedException e) {
-        LOGGER.debug("Failed to set query timeout.");
-      }
+      setTimeoutIfPossible(statement);
       if (statement.execute(sql)) {
         final ResultSet resultSet = statement.getResultSet();
         statement = null;
@@ -144,21 +253,59 @@ public class ResultSetEnumerable<T> extends AbstractEnumerable<T> {
         return Linq4j.singletonEnumerator((T) updateCount);
       }
     } catch (SQLException e) {
-      throw new RuntimeException("while executing SQL [" + sql + "]", e);
+      throw Static.RESOURCE.exceptionWhilePerformingQueryOnJdbcSubSchema(sql)
+          .ex(e);
     } finally {
-      if (statement != null) {
-        try {
-          statement.close();
-        } catch (SQLException e) {
-          // ignore
-        }
+      closeIfPossible(connection, statement);
+    }
+  }
+
+  private Enumerator<T> enumeratorBasedOnPreparedStatement() {
+    Connection connection = null;
+    PreparedStatement preparedStatement = null;
+    try {
+      connection = dataSource.getConnection();
+      preparedStatement = connection.prepareStatement(sql);
+      setTimeoutIfPossible(preparedStatement);
+      preparedStatementEnricher.enrich(preparedStatement);
+      if (preparedStatement.execute()) {
+        final ResultSet resultSet = preparedStatement.getResultSet();
+        preparedStatement = null;
+        connection = null;
+        return new ResultSetEnumerator<>(resultSet, rowBuilderFactory);
+      } else {
+        Integer updateCount = preparedStatement.getUpdateCount();
+        return Linq4j.singletonEnumerator((T) updateCount);
       }
-      if (connection != null) {
-        try {
-          connection.close();
-        } catch (SQLException e) {
-          // ignore
-        }
+    } catch (SQLException e) {
+      throw Static.RESOURCE.exceptionWhilePerformingQueryOnJdbcSubSchema(sql)
+          .ex(e);
+    } finally {
+      closeIfPossible(connection, preparedStatement);
+    }
+  }
+
+  private void setTimeoutIfPossible(Statement statement) throws SQLException {
+    try {
+      statement.setQueryTimeout(10);
+    } catch (SQLFeatureNotSupportedException e) {
+      LOGGER.debug("Failed to set query timeout.");
+    }
+  }
+
+  private void closeIfPossible(Connection connection, Statement statement) {
+    if (statement != null) {
+      try {
+        statement.close();
+      } catch (SQLException e) {
+        // ignore
+      }
+    }
+    if (connection != null) {
+      try {
+        connection.close();
+      } catch (SQLException e) {
+        // ignore
       }
     }
   }
@@ -254,6 +401,14 @@ public class ResultSetEnumerable<T> extends AbstractEnumerable<T> {
       };
     };
   }
+
+  /**
+   * Consumer for decorating a {@link PreparedStatement}, that is, setting
+   * its parameters.
+   */
+  public interface PreparedStatementEnricher {
+    void enrich(PreparedStatement statement) throws SQLException;
+  }
 }
 
 // End ResultSetEnumerable.java

http://git-wip-us.apache.org/repos/asf/calcite/blob/53e15af6/core/src/main/java/org/apache/calcite/sql/SqlDynamicParam.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/SqlDynamicParam.java b/core/src/main/java/org/apache/calcite/sql/SqlDynamicParam.java
index db2a6b0..e000ca1 100644
--- a/core/src/main/java/org/apache/calcite/sql/SqlDynamicParam.java
+++ b/core/src/main/java/org/apache/calcite/sql/SqlDynamicParam.java
@@ -61,8 +61,7 @@ public class SqlDynamicParam extends SqlNode {
       SqlWriter writer,
       int leftPrec,
       int rightPrec) {
-    writer.print("?");
-    writer.setNeedWhitespace(true);
+    writer.dynamicParam(index);
   }
 
   public void validate(SqlValidator validator, SqlValidatorScope scope) {

http://git-wip-us.apache.org/repos/asf/calcite/blob/53e15af6/core/src/main/java/org/apache/calcite/sql/SqlNode.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/SqlNode.java b/core/src/main/java/org/apache/calcite/sql/SqlNode.java
index c6d7d0b..1a5a9f9 100644
--- a/core/src/main/java/org/apache/calcite/sql/SqlNode.java
+++ b/core/src/main/java/org/apache/calcite/sql/SqlNode.java
@@ -149,8 +149,7 @@ public abstract class SqlNode implements Cloneable {
     writer.setSelectListItemsOnSeparateLines(false);
     writer.setIndentation(0);
     unparse(writer, 0, 0);
-    final String sql = writer.toString();
-    return new SqlString(dialect, sql);
+    return writer.toSqlString();
   }
 
   public SqlString toSqlString(SqlDialect dialect) {

http://git-wip-us.apache.org/repos/asf/calcite/blob/53e15af6/core/src/main/java/org/apache/calcite/sql/SqlWriter.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/SqlWriter.java b/core/src/main/java/org/apache/calcite/sql/SqlWriter.java
index c5d4dab..51882f7 100644
--- a/core/src/main/java/org/apache/calcite/sql/SqlWriter.java
+++ b/core/src/main/java/org/apache/calcite/sql/SqlWriter.java
@@ -325,6 +325,11 @@ public interface SqlWriter {
   void identifier(String name);
 
   /**
+   * Prints a dynamic parameter (e.g. {@code ?} for default JDBC)
+   */
+  void dynamicParam(int index);
+
+  /**
    * Prints the OFFSET/FETCH clause.
    */
   void fetchOffset(SqlNode fetch, SqlNode offset);

http://git-wip-us.apache.org/repos/asf/calcite/blob/53e15af6/core/src/main/java/org/apache/calcite/sql/pretty/SqlPrettyWriter.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/pretty/SqlPrettyWriter.java b/core/src/main/java/org/apache/calcite/sql/pretty/SqlPrettyWriter.java
index 06358a1..f8186b2 100644
--- a/core/src/main/java/org/apache/calcite/sql/pretty/SqlPrettyWriter.java
+++ b/core/src/main/java/org/apache/calcite/sql/pretty/SqlPrettyWriter.java
@@ -21,13 +21,13 @@ import org.apache.calcite.sql.SqlDialect;
 import org.apache.calcite.sql.SqlNode;
 import org.apache.calcite.sql.SqlWriter;
 import org.apache.calcite.sql.dialect.AnsiSqlDialect;
-import org.apache.calcite.sql.util.SqlBuilder;
 import org.apache.calcite.sql.util.SqlString;
 import org.apache.calcite.util.Unsafe;
 import org.apache.calcite.util.Util;
 import org.apache.calcite.util.trace.CalciteLogger;
 
 import com.google.common.base.Preconditions;
+import com.google.common.collect.ImmutableList;
 
 import org.slf4j.LoggerFactory;
 
@@ -141,6 +141,7 @@ public class SqlPrettyWriter implements SqlWriter {
   private final StringWriter sw = new StringWriter();
   protected final PrintWriter pw;
   private final Deque<FrameImpl> listStack = new ArrayDeque<>();
+  private ImmutableList.Builder<Integer> dynamicParameters;
   protected FrameImpl frame;
   private boolean needWhitespace;
   protected String nextWhitespace;
@@ -279,6 +280,7 @@ public class SqlPrettyWriter implements SqlWriter {
   public void reset() {
     pw.flush();
     Unsafe.clear(sw);
+    dynamicParameters = null;
     setNeedWhitespace(false);
     nextWhitespace = " ";
   }
@@ -810,7 +812,9 @@ public class SqlPrettyWriter implements SqlWriter {
   }
 
   public SqlString toSqlString() {
-    return new SqlBuilder(dialect, toString()).toSqlString();
+    ImmutableList<Integer> dynamicParameters =
+        this.dynamicParameters == null ? null : this.dynamicParameters.build();
+    return new SqlString(dialect, toString(), dynamicParameters);
   }
 
   public SqlDialect getDialect() {
@@ -904,6 +908,15 @@ public class SqlPrettyWriter implements SqlWriter {
     setNeedWhitespace(true);
   }
 
+  @Override public void dynamicParam(int index) {
+    if (dynamicParameters == null) {
+      dynamicParameters = ImmutableList.builder();
+    }
+    dynamicParameters.add(index);
+    print("?");
+    setNeedWhitespace(true);
+  }
+
   public void fetchOffset(SqlNode fetch, SqlNode offset) {
     if (fetch == null && offset == null) {
       return;

http://git-wip-us.apache.org/repos/asf/calcite/blob/53e15af6/core/src/main/java/org/apache/calcite/sql/util/SqlString.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/util/SqlString.java b/core/src/main/java/org/apache/calcite/sql/util/SqlString.java
index 92b295c..ef9c686 100644
--- a/core/src/main/java/org/apache/calcite/sql/util/SqlString.java
+++ b/core/src/main/java/org/apache/calcite/sql/util/SqlString.java
@@ -18,6 +18,8 @@ package org.apache.calcite.sql.util;
 
 import org.apache.calcite.sql.SqlDialect;
 
+import com.google.common.collect.ImmutableList;
+
 /**
  * String that represents a kocher SQL statement, expression, or fragment.
  *
@@ -28,29 +30,40 @@ import org.apache.calcite.sql.SqlDialect;
  * <p>The easiest way to do build a SqlString is to use a {@link SqlBuilder}.
  */
 public class SqlString {
-  private final String s;
+  private final String sql;
   private SqlDialect dialect;
+  private ImmutableList<Integer> dynamicParameters;
 
   /**
    * Creates a SqlString.
+   */
+  public SqlString(SqlDialect dialect, String sql) {
+    this(dialect, sql, ImmutableList.of());
+  }
+
+  /**
+   * Creates a SqlString. The SQL might contain dynamic parameters, dynamicParameters
+   * designate the order of the parameters.
    *
-   * @param s Contents of string
+   * @param sql text
+   * @param dynamicParameters indices
    */
-  public SqlString(SqlDialect dialect, String s) {
+  public SqlString(SqlDialect dialect, String sql, ImmutableList<Integer> dynamicParameters) {
     this.dialect = dialect;
-    this.s = s;
-    assert s != null;
-    assert dialect != null;
+    this.sql = sql;
+    this.dynamicParameters = dynamicParameters;
+    assert sql != null : "sql must be NOT null";
+    assert dialect != null : "dialect must be NOT null";
   }
 
   @Override public int hashCode() {
-    return s.hashCode();
+    return sql.hashCode();
   }
 
   @Override public boolean equals(Object obj) {
     return obj == this
         || obj instanceof SqlString
-        && s.equals(((SqlString) obj).s);
+        && sql.equals(((SqlString) obj).sql);
   }
 
   /**
@@ -62,7 +75,7 @@ public class SqlString {
    * @see #getSql()
    */
   @Override public String toString() {
-    return s;
+    return sql;
   }
 
   /**
@@ -71,7 +84,16 @@ public class SqlString {
    * @return SQL string
    */
   public String getSql() {
-    return s;
+    return sql;
+  }
+
+  /**
+   * Returns indices of dynamic parameters.
+   *
+   * @return indices of dynamic parameters
+   */
+  public ImmutableList<Integer> getDynamicParameters() {
+    return dynamicParameters;
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/calcite/blob/53e15af6/core/src/main/java/org/apache/calcite/util/BuiltInMethod.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/util/BuiltInMethod.java b/core/src/main/java/org/apache/calcite/util/BuiltInMethod.java
index b3803dd..8160ead 100644
--- a/core/src/main/java/org/apache/calcite/util/BuiltInMethod.java
+++ b/core/src/main/java/org/apache/calcite/util/BuiltInMethod.java
@@ -148,6 +148,11 @@ public enum BuiltInMethod {
   ROW_AS_COPY(Row.class, "asCopy", Object[].class),
   RESULT_SET_ENUMERABLE_OF(ResultSetEnumerable.class, "of", DataSource.class,
       String.class, Function1.class),
+  RESULT_SET_ENUMERABLE_OF_PREPARED(ResultSetEnumerable.class, "of",
+      DataSource.class, String.class, Function1.class,
+      ResultSetEnumerable.PreparedStatementEnricher.class),
+  CREATE_ENRICHER(ResultSetEnumerable.class, "createEnricher", Integer[].class,
+      DataContext.class),
   JOIN(ExtendedEnumerable.class, "join", Enumerable.class, Function1.class,
       Function1.class, Function2.class),
   MERGE_JOIN(EnumerableDefaults.class, "mergeJoin", Enumerable.class,

http://git-wip-us.apache.org/repos/asf/calcite/blob/53e15af6/core/src/main/resources/org/apache/calcite/runtime/CalciteResource.properties
----------------------------------------------------------------------
diff --git a/core/src/main/resources/org/apache/calcite/runtime/CalciteResource.properties b/core/src/main/resources/org/apache/calcite/runtime/CalciteResource.properties
index da48650..1ec7c95 100644
--- a/core/src/main/resources/org/apache/calcite/runtime/CalciteResource.properties
+++ b/core/src/main/resources/org/apache/calcite/runtime/CalciteResource.properties
@@ -275,4 +275,5 @@ IllegalEmptyBehaviorInJsonQueryFunc=Illegal empty behavior ''{0}'' specified in
 ArrayOrObjectValueRequiredInStrictModeOfJsonQueryFunc=Strict jsonpath mode requires array or object value, and the actual value is: ''{0}''
 IllegalErrorBehaviorInJsonQueryFunc=Illegal error behavior ''{0}'' specified in JSON_VALUE function
 NullKeyOfJsonObjectNotAllowed=Null key of JSON object is not allowed
+ExceptionWhilePerformingQueryOnJdbcSubSchema = While executing SQL [{0}] on JDBC sub-schema
 # End CalciteResource.properties

http://git-wip-us.apache.org/repos/asf/calcite/blob/53e15af6/core/src/test/java/org/apache/calcite/test/CalciteAssert.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/calcite/test/CalciteAssert.java b/core/src/test/java/org/apache/calcite/test/CalciteAssert.java
index 4519d04..da7518b 100644
--- a/core/src/test/java/org/apache/calcite/test/CalciteAssert.java
+++ b/core/src/test/java/org/apache/calcite/test/CalciteAssert.java
@@ -574,14 +574,92 @@ public class CalciteAssert {
     }
   }
 
+  private static void assertPrepare(
+      Connection connection,
+      String sql,
+      int limit,
+      boolean materializationsEnabled,
+      List<Pair<Hook, Consumer>> hooks,
+      Consumer<ResultSet> resultChecker,
+      Consumer<Integer> updateChecker,
+      Consumer<Throwable> exceptionChecker,
+      PreparedStatementConsumer consumer) {
+    final String message = "With materializationsEnabled="
+        + materializationsEnabled + ", limit=" + limit;
+    try (Closer closer = new Closer()) {
+      if (connection.isWrapperFor(CalciteConnection.class)) {
+        final CalciteConnection calciteConnection =
+            connection.unwrap(CalciteConnection.class);
+        final Properties properties = calciteConnection.getProperties();
+        properties.setProperty(
+            CalciteConnectionProperty.MATERIALIZATIONS_ENABLED.camelName(),
+            Boolean.toString(materializationsEnabled));
+        properties.setProperty(
+            CalciteConnectionProperty.CREATE_MATERIALIZATIONS.camelName(),
+            Boolean.toString(materializationsEnabled));
+        if (!properties
+            .containsKey(CalciteConnectionProperty.TIME_ZONE.camelName())) {
+          // Do not override id some test has already set this property.
+          properties.setProperty(
+              CalciteConnectionProperty.TIME_ZONE.camelName(),
+              DateTimeUtils.UTC_ZONE.getID());
+        }
+      }
+      for (Pair<Hook, Consumer> hook : hooks) {
+        //noinspection unchecked
+        closer.add(hook.left.addThread(hook.right));
+      }
+      PreparedStatement statement = connection.prepareStatement(sql);
+      statement.setMaxRows(limit <= 0 ? limit : Math.max(limit, 1));
+      ResultSet resultSet = null;
+      Integer updateCount = null;
+      try {
+        consumer.accept(statement);
+        if (updateChecker == null) {
+          resultSet = statement.executeQuery();
+        } else {
+          updateCount = statement.executeUpdate(sql);
+        }
+        if (exceptionChecker != null) {
+          exceptionChecker.accept(null);
+          return;
+        }
+      } catch (Exception | Error e) {
+        if (exceptionChecker != null) {
+          exceptionChecker.accept(e);
+          return;
+        }
+        throw e;
+      }
+      if (resultChecker != null) {
+        resultChecker.accept(resultSet);
+      }
+      if (updateChecker != null) {
+        updateChecker.accept(updateCount);
+      }
+      if (resultSet != null) {
+        resultSet.close();
+      }
+      statement.close();
+      connection.close();
+    } catch (Error | RuntimeException e) {
+      // We ignore extended message for non-runtime exception, however
+      // it does not matter much since it is better to have AssertionError
+      // at the very top level of the exception stack.
+      throw e;
+    } catch (Throwable e) {
+      throw new RuntimeException(message, e);
+    }
+  }
+
   static void assertPrepare(
       Connection connection,
       String sql,
       boolean materializationsEnabled,
       final Function<RelNode, Void> convertChecker,
       final Function<RelNode, Void> substitutionChecker) throws Exception {
-    final String message =
-        "With materializationsEnabled=" + materializationsEnabled;
+    final String message = "With materializationsEnabled="
+        + materializationsEnabled;
     try (Closer closer = new Closer()) {
       if (convertChecker != null) {
         closer.add(
@@ -1282,6 +1360,7 @@ public class CalciteAssert {
     private int limit;
     private boolean materializationsEnabled = false;
     private final List<Pair<Hook, Consumer>> hooks = new ArrayList<>();
+    private PreparedStatementConsumer consumer;
 
     private AssertQuery(ConnectionFactory connectionFactory, String sql) {
       this.sql = sql;
@@ -1367,8 +1446,13 @@ public class CalciteAssert {
 
     protected AssertQuery returns(String sql, Consumer<ResultSet> checker) {
       try (Connection connection = createConnection()) {
-        assertQuery(connection, sql, limit, materializationsEnabled,
-            hooks, checker, null, null);
+        if (consumer == null) {
+          assertQuery(connection, sql, limit, materializationsEnabled,
+              hooks, checker, null, null);
+        } else {
+          assertPrepare(connection, sql, limit, materializationsEnabled,
+              hooks, checker, null, null, consumer);
+        }
         return this;
       } catch (Exception e) {
         throw new RuntimeException(
@@ -1426,8 +1510,13 @@ public class CalciteAssert {
 
     public AssertQuery runs() {
       try (Connection connection = createConnection()) {
-        assertQuery(connection, sql, limit, materializationsEnabled,
-            hooks, null, null, null);
+        if (consumer == null) {
+          assertQuery(connection, sql, limit, materializationsEnabled,
+              hooks, null, null, null);
+        } else {
+          assertPrepare(connection, sql, limit, materializationsEnabled,
+              hooks, null, null, null, consumer);
+        }
         return this;
       } catch (Exception e) {
         throw new RuntimeException(
@@ -1453,6 +1542,11 @@ public class CalciteAssert {
       return convertMatches(checkRel(expected, null));
     }
 
+    public final AssertQuery consumesPreparedStatement(PreparedStatementConsumer consumer) {
+      this.consumer =  consumer;
+      return this;
+    }
+
     public AssertQuery convertMatches(final Function<RelNode, Void> checker) {
       try (Connection connection = createConnection()) {
         assertPrepare(connection, sql, this.materializationsEnabled,
@@ -1906,6 +2000,13 @@ public class CalciteAssert {
       return properties;
     }
   }
+
+  /**
+   * We want a consumer which can throw SqlException
+   */
+  public interface PreparedStatementConsumer {
+    void accept(PreparedStatement statement) throws SQLException;
+  }
 }
 
 // End CalciteAssert.java

http://git-wip-us.apache.org/repos/asf/calcite/blob/53e15af6/core/src/test/java/org/apache/calcite/test/JdbcAdapterTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/calcite/test/JdbcAdapterTest.java b/core/src/test/java/org/apache/calcite/test/JdbcAdapterTest.java
index ed547c0..eae066b 100644
--- a/core/src/test/java/org/apache/calcite/test/JdbcAdapterTest.java
+++ b/core/src/test/java/org/apache/calcite/test/JdbcAdapterTest.java
@@ -857,6 +857,17 @@ public class JdbcAdapterTest {
         .typeIs("[employee_id INTEGER NOT NULL, position_id INTEGER]");
   }
 
+  @Test public void pushBindParameters() throws Exception {
+    final String sql = "select empno, ename from emp where empno = ?";
+    CalciteAssert.model(JdbcTest.SCOTT_MODEL)
+        .query(sql)
+        .consumesPreparedStatement(p -> {
+          p.setInt(1, 7566);
+        })
+        .returnsCount(1)
+        .planHasSql("SELECT \"EMPNO\", \"ENAME\"\nFROM \"SCOTT\".\"EMP\"\nWHERE \"EMPNO\" = ?");
+  }
+
   /** Acquires a lock, and releases it when closed. */
   static class LockWrapper implements AutoCloseable {
     private final Lock lock;

http://git-wip-us.apache.org/repos/asf/calcite/blob/53e15af6/core/src/test/java/org/apache/calcite/test/JdbcTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/calcite/test/JdbcTest.java b/core/src/test/java/org/apache/calcite/test/JdbcTest.java
index 888ed6c..e1d2917 100644
--- a/core/src/test/java/org/apache/calcite/test/JdbcTest.java
+++ b/core/src/test/java/org/apache/calcite/test/JdbcTest.java
@@ -6732,6 +6732,59 @@ public class JdbcTest {
         .returns("EXPR$0=[250, 500, 1000]\n");
   }
 
+  /**
+   * Test case for
+   * <a href="https://issues.apache.org/jira/browse/CALCITE-2609">[CALCITE-2609]
+   * Dynamic parameters ("?") pushed to underlying JDBC schema, causing
+   * error</a>.
+   */
+  @Test public void testQueryWithParameter() throws Exception {
+    String hsqldbMemUrl = "jdbc:hsqldb:mem:.";
+    try (Connection baseConnection = DriverManager.getConnection(hsqldbMemUrl);
+         Statement baseStmt = baseConnection.createStatement()) {
+      baseStmt.execute("CREATE TABLE T3 (\n"
+          + "ID INTEGER,\n"
+          + "VALS DOUBLE)");
+      baseStmt.execute("INSERT INTO T3 VALUES (1, 1.0)");
+      baseStmt.execute("INSERT INTO T3 VALUES (2, null)");
+      baseStmt.execute("INSERT INTO T3 VALUES (null, 2.0)");
+      baseStmt.close();
+      baseConnection.commit();
+
+      Properties info = new Properties();
+      final String model = "inline:"
+          + "{\n"
+          + "  version: '1.0',\n"
+          + "  defaultSchema: 'BASEJDBC',\n"
+          + "  schemas: [\n"
+          + "     {\n"
+          + "       type: 'jdbc',\n"
+          + "       name: 'BASEJDBC',\n"
+          + "       jdbcDriver: '" + jdbcDriver.class.getName() + "',\n"
+          + "       jdbcUrl: '" + hsqldbMemUrl + "',\n"
+          + "       jdbcCatalog: null,\n"
+          + "       jdbcSchema: null\n"
+          + "     }\n"
+          + "  ]\n"
+          + "}";
+      info.put("model", model);
+
+      Connection calciteConnection =
+          DriverManager.getConnection("jdbc:calcite:", info);
+
+      final String sql = "select * from t3 where vals = ?";
+      try (PreparedStatement ps =
+               calciteConnection.prepareStatement(sql)) {
+        ParameterMetaData pmd = ps.getParameterMetaData();
+        assertThat(pmd.getParameterCount(), is(1));
+        assertThat(pmd.getParameterType(1), is(Types.DOUBLE));
+        ps.setDouble(1, 1.0);
+        ps.executeQuery();
+      }
+      calciteConnection.close();
+    }
+  }
+
   private static String sums(int n, boolean c) {
     final StringBuilder b = new StringBuilder();
     for (int i = 0; i < n; i++) {

http://git-wip-us.apache.org/repos/asf/calcite/blob/53e15af6/plus/pom.xml
----------------------------------------------------------------------
diff --git a/plus/pom.xml b/plus/pom.xml
index 7bbafcc..e9f950c 100644
--- a/plus/pom.xml
+++ b/plus/pom.xml
@@ -56,6 +56,14 @@ limitations under the License.
       <artifactId>avatica-core</artifactId>
     </dependency>
     <dependency>
+      <groupId>org.apache.calcite.avatica</groupId>
+      <artifactId>avatica-server</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>org.eclipse.jetty</groupId>
+      <artifactId>jetty-server</artifactId>
+    </dependency>
+    <dependency>
       <groupId>com.google.guava</groupId>
       <artifactId>guava</artifactId>
     </dependency>
@@ -147,6 +155,7 @@ limitations under the License.
               <!-- ignore "unused but declared" warnings -->
               <ignoredUnusedDeclaredDependencies>
                 <ignoredUnusedDeclaredDependency>net.hydromatic:chinook-data-hsqldb</ignoredUnusedDeclaredDependency>
+                <ignoredUnusedDeclaredDependency>org.eclipse.jetty:jetty-server</ignoredUnusedDeclaredDependency>
                 <ignoredUnusedDeclaredDependency>org.hsqldb:hsqldb</ignoredUnusedDeclaredDependency>
                 <ignoredUnusedDeclaredDependency>org.incava:java-diff</ignoredUnusedDeclaredDependency>
               </ignoredUnusedDeclaredDependencies>

http://git-wip-us.apache.org/repos/asf/calcite/blob/53e15af6/plus/src/main/java/org/apache/calcite/chinook/CalciteConnectionProvider.java
----------------------------------------------------------------------
diff --git a/plus/src/main/java/org/apache/calcite/chinook/CalciteConnectionProvider.java b/plus/src/main/java/org/apache/calcite/chinook/CalciteConnectionProvider.java
index c18e560..ad9aab8 100644
--- a/plus/src/main/java/org/apache/calcite/chinook/CalciteConnectionProvider.java
+++ b/plus/src/main/java/org/apache/calcite/chinook/CalciteConnectionProvider.java
@@ -32,13 +32,13 @@ import java.util.Properties;
  */
 public class CalciteConnectionProvider {
 
-  private static final String DRIVER_URL = "jdbc:calcite:";
+  public static final String DRIVER_URL = "jdbc:calcite:";
 
   public Connection connection() throws IOException, SQLException {
     return DriverManager.getConnection(DRIVER_URL, provideConnectionInfo());
   }
 
-  private Properties provideConnectionInfo() throws IOException {
+  public Properties provideConnectionInfo() throws IOException {
     Properties info = new Properties();
     info.setProperty("lex", "MYSQL");
     info.setProperty("model", "inline:" + provideSchema());

http://git-wip-us.apache.org/repos/asf/calcite/blob/53e15af6/plus/src/main/java/org/apache/calcite/chinook/ChinookAvaticaServer.java
----------------------------------------------------------------------
diff --git a/plus/src/main/java/org/apache/calcite/chinook/ChinookAvaticaServer.java b/plus/src/main/java/org/apache/calcite/chinook/ChinookAvaticaServer.java
new file mode 100644
index 0000000..6b24ae7
--- /dev/null
+++ b/plus/src/main/java/org/apache/calcite/chinook/ChinookAvaticaServer.java
@@ -0,0 +1,108 @@
+/*
+ * 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.chinook;
+
+import org.apache.calcite.avatica.Meta;
+import org.apache.calcite.avatica.jdbc.JdbcMeta;
+import org.apache.calcite.avatica.remote.Driver;
+import org.apache.calcite.avatica.server.AvaticaProtobufHandler;
+import org.apache.calcite.avatica.server.HttpServer;
+import org.apache.calcite.avatica.server.Main;
+
+import net.hydromatic.chinook.data.hsqldb.ChinookHsqldb;
+
+import java.io.IOException;
+import java.sql.SQLException;
+import java.util.List;
+
+/**
+ * Wrapping Calcite engine with Avatica tansport for testing JDBC capabilities
+ * between Avatica JDBC transport and Calcite.
+ */
+public class ChinookAvaticaServer {
+  private HttpServer server;
+
+  public void startWithCalcite() throws Exception {
+    final String[] args = {CalciteChinookMetaFactory.class.getName()};
+    this.server = Main.start(args, 0, AvaticaProtobufHandler::new);
+  }
+
+  public void startWithRaw() throws Exception {
+    final String[] args = {RawChinookMetaFactory.class.getName()};
+    this.server = Main.start(args, 0, AvaticaProtobufHandler::new);
+  }
+
+  public String getURL() {
+    return "jdbc:avatica:remote:url=http://localhost:" + server.getPort()
+        + ";serialization=" + Driver.Serialization.PROTOBUF.name();
+  }
+
+  public void stop() {
+    server.stop();
+  }
+
+  /**
+   * Factory for Chinook Calcite database wrapped in meta for Avatica.
+   */
+  public static class CalciteChinookMetaFactory implements Meta.Factory {
+    private static final CalciteConnectionProvider CONNECTION_PROVIDER =
+        new CalciteConnectionProvider();
+
+    private static JdbcMeta instance = null;
+
+    private static JdbcMeta getInstance() {
+      if (instance == null) {
+        try {
+          instance = new JdbcMeta(CalciteConnectionProvider.DRIVER_URL,
+              CONNECTION_PROVIDER.provideConnectionInfo());
+        } catch (SQLException | IOException e) {
+          throw new RuntimeException(e);
+        }
+      }
+      return instance;
+    }
+
+    @Override public Meta create(List<String> args) {
+      return getInstance();
+    }
+  }
+
+  /**
+   * Factory for Chinook Calcite database wrapped in meta for Avatica.
+   */
+  public static class RawChinookMetaFactory implements Meta.Factory {
+    private static JdbcMeta instance = null;
+
+    private static JdbcMeta getInstance() {
+      if (instance == null) {
+        try {
+          instance = new JdbcMeta(ChinookHsqldb.URI,
+              ChinookHsqldb.USER, ChinookHsqldb.PASSWORD);
+        } catch (SQLException e) {
+          throw new RuntimeException(e);
+        }
+      }
+      return instance;
+    }
+
+    @Override public Meta create(List<String> args) {
+      return getInstance();
+    }
+  }
+}
+
+// End ChinookAvaticaServer.java

http://git-wip-us.apache.org/repos/asf/calcite/blob/53e15af6/plus/src/main/java/org/apache/calcite/chinook/ConnectionFactory.java
----------------------------------------------------------------------
diff --git a/plus/src/main/java/org/apache/calcite/chinook/ConnectionFactory.java b/plus/src/main/java/org/apache/calcite/chinook/ConnectionFactory.java
index 376a793..cd0ef01 100644
--- a/plus/src/main/java/org/apache/calcite/chinook/ConnectionFactory.java
+++ b/plus/src/main/java/org/apache/calcite/chinook/ConnectionFactory.java
@@ -30,13 +30,13 @@ public class ConnectionFactory implements Quidem.ConnectionFactory {
   private static final CalciteConnectionProvider CALCITE = new CalciteConnectionProvider();
 
   public Connection connect(String db, boolean bln) throws Exception {
-    return DBWrapper.valueOf(db).connection();
+    return DatabaseWrapper.valueOf(db).connection();
   }
 
   /**
    * Wrapping with Fairy environmental decoration
    */
-  private enum DBWrapper {
+  public enum DatabaseWrapper {
     CALCITE_AS_ADMIN {
       @Override public Connection connection() throws Exception {
         EnvironmentFairy.login(EnvironmentFairy.User.ADMIN);

http://git-wip-us.apache.org/repos/asf/calcite/blob/53e15af6/plus/src/test/java/org/apache/calcite/chinook/RemotePreparedStatementParametersTest.java
----------------------------------------------------------------------
diff --git a/plus/src/test/java/org/apache/calcite/chinook/RemotePreparedStatementParametersTest.java b/plus/src/test/java/org/apache/calcite/chinook/RemotePreparedStatementParametersTest.java
new file mode 100644
index 0000000..ac39149
--- /dev/null
+++ b/plus/src/test/java/org/apache/calcite/chinook/RemotePreparedStatementParametersTest.java
@@ -0,0 +1,76 @@
+/*
+ * 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.chinook;
+
+import org.junit.Test;
+
+import java.sql.Connection;
+import java.sql.DriverManager;
+import java.sql.PreparedStatement;
+import java.sql.ResultSet;
+
+/**
+ * Tests against parameters in prepared statement when using underlying jdbc subschema
+ */
+public class RemotePreparedStatementParametersTest {
+
+  @Test public void testSimpleStringParameterShouldWorkWithCalcite() throws Exception {
+    // given
+    ChinookAvaticaServer server = new ChinookAvaticaServer();
+    server.startWithCalcite();
+    Connection connection = DriverManager.getConnection(server.getURL());
+    // when
+    PreparedStatement pS =
+        connection.prepareStatement("select * from chinook.artist where name = ?");
+    pS.setString(1, "AC/DC");
+    // then
+    ResultSet resultSet = pS.executeQuery();
+    server.stop();
+  }
+
+  @Test public void testSeveralParametersShouldWorkWithCalcite() throws Exception {
+    // given
+    ChinookAvaticaServer server = new ChinookAvaticaServer();
+    server.startWithCalcite();
+    Connection connection = DriverManager.getConnection(server.getURL());
+    // when
+    PreparedStatement pS =
+        connection.prepareStatement(
+            "select * from chinook.track where name = ? or milliseconds > ?");
+    pS.setString(1, "AC/DC");
+    pS.setInt(2, 10);
+    // then
+    ResultSet resultSet = pS.executeQuery();
+    server.stop();
+  }
+
+  @Test public void testParametersShouldWorkWithRaw() throws Exception {
+    // given
+    ChinookAvaticaServer server = new ChinookAvaticaServer();
+    server.startWithRaw();
+    Connection connection = DriverManager.getConnection(server.getURL());
+    // when
+    PreparedStatement pS =
+        connection.prepareStatement("select * from artist where name = ?");
+    pS.setString(1, "AC/DC");
+    // then
+    ResultSet resultSet = pS.executeQuery();
+    server.stop();
+  }
+}
+
+// End RemotePreparedStatementParametersTest.java

http://git-wip-us.apache.org/repos/asf/calcite/blob/53e15af6/plus/src/test/java/org/apache/calcite/test/PlusSuite.java
----------------------------------------------------------------------
diff --git a/plus/src/test/java/org/apache/calcite/test/PlusSuite.java b/plus/src/test/java/org/apache/calcite/test/PlusSuite.java
index 5f8a67a..94387bb 100644
--- a/plus/src/test/java/org/apache/calcite/test/PlusSuite.java
+++ b/plus/src/test/java/org/apache/calcite/test/PlusSuite.java
@@ -20,6 +20,7 @@ import org.apache.calcite.adapter.os.OsAdapterTest;
 import org.apache.calcite.adapter.tpcds.TpcdsTest;
 import org.apache.calcite.adapter.tpch.TpchTest;
 import org.apache.calcite.chinook.EndToEndTest;
+import org.apache.calcite.chinook.RemotePreparedStatementParametersTest;
 
 import org.junit.runner.RunWith;
 import org.junit.runners.Suite;
@@ -32,7 +33,8 @@ import org.junit.runners.Suite;
     OsAdapterTest.class,
     TpcdsTest.class,
     TpchTest.class,
-    EndToEndTest.class
+    EndToEndTest.class,
+    RemotePreparedStatementParametersTest.class
     })
 public class PlusSuite {
 }