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/03/07 19:28:02 UTC

[24/59] [partial] calcite git commit: [CALCITE-1078] Detach avatica from the core calcite Maven project

http://git-wip-us.apache.org/repos/asf/calcite/blob/5cee486f/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
new file mode 100644
index 0000000..ff95a45
--- /dev/null
+++ b/avatica/server/src/test/java/org/apache/calcite/avatica/remote/RemoteMetaTest.java
@@ -0,0 +1,626 @@
+/*
+ * 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.remote;
+
+import org.apache.calcite.avatica.AvaticaConnection;
+import org.apache.calcite.avatica.AvaticaSqlException;
+import org.apache.calcite.avatica.AvaticaStatement;
+import org.apache.calcite.avatica.AvaticaUtils;
+import org.apache.calcite.avatica.ConnectionPropertiesImpl;
+import org.apache.calcite.avatica.ConnectionSpec;
+import org.apache.calcite.avatica.Meta;
+import org.apache.calcite.avatica.jdbc.JdbcMeta;
+import org.apache.calcite.avatica.remote.Service.ErrorResponse;
+import org.apache.calcite.avatica.remote.Service.Response;
+import org.apache.calcite.avatica.server.AvaticaJsonHandler;
+import org.apache.calcite.avatica.server.AvaticaProtobufHandler;
+import org.apache.calcite.avatica.server.HttpServer;
+import org.apache.calcite.avatica.server.Main;
+import org.apache.calcite.avatica.server.Main.HandlerFactory;
+import org.apache.calcite.avatica.util.ArrayImpl;
+
+import com.google.common.base.Throwables;
+import com.google.common.cache.Cache;
+
+import org.junit.AfterClass;
+import org.junit.Ignore;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+import org.junit.runners.Parameterized.Parameters;
+
+import java.lang.reflect.Field;
+import java.lang.reflect.Method;
+import java.net.InetAddress;
+import java.net.URL;
+import java.nio.charset.StandardCharsets;
+import java.sql.Array;
+import java.sql.Connection;
+import java.sql.DriverManager;
+import java.sql.PreparedStatement;
+import java.sql.ResultSet;
+import java.sql.SQLException;
+import java.sql.Statement;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.Random;
+import java.util.UUID;
+
+import static org.hamcrest.CoreMatchers.containsString;
+import static org.hamcrest.CoreMatchers.instanceOf;
+import static org.hamcrest.CoreMatchers.is;
+import static org.junit.Assert.assertArrayEquals;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertThat;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
+/** Tests covering {@link RemoteMeta}. */
+@RunWith(Parameterized.class)
+public class RemoteMetaTest {
+  private static final Random RANDOM = new Random();
+  private static final ConnectionSpec CONNECTION_SPEC = ConnectionSpec.HSQLDB;
+
+  // Keep a reference to the servers we start to clean them up after
+  private static final List<HttpServer> ACTIVE_SERVERS = new ArrayList<>();
+
+  private final HttpServer server;
+  private final String url;
+  private final int port;
+  private final Driver.Serialization serialization;
+
+  @Parameters
+  public static List<Object[]> parameters() throws Exception {
+    List<Object[]> params = new ArrayList<>();
+
+    final String[] mainArgs = { FullyRemoteJdbcMetaFactory.class.getName() };
+
+    // Bind to '0' to pluck an ephemeral port instead of expecting a certain one to be free
+
+    final HttpServer jsonServer = Main.start(mainArgs, 0, new HandlerFactory() {
+      @Override public AvaticaJsonHandler createHandler(Service service) {
+        return new AvaticaJsonHandler(service);
+      }
+    });
+    params.add(new Object[] {jsonServer, Driver.Serialization.JSON});
+    ACTIVE_SERVERS.add(jsonServer);
+
+    final HttpServer protobufServer = Main.start(mainArgs, 0, new HandlerFactory() {
+      @Override public AvaticaProtobufHandler createHandler(Service service) {
+        return new AvaticaProtobufHandler(service);
+      }
+    });
+    params.add(new Object[] {protobufServer, Driver.Serialization.PROTOBUF});
+
+    ACTIVE_SERVERS.add(protobufServer);
+
+    return params;
+  }
+
+  public RemoteMetaTest(HttpServer server, Driver.Serialization serialization) {
+    this.server = server;
+    this.port = this.server.getPort();
+    this.serialization = serialization;
+    url = "jdbc:avatica:remote:url=http://localhost:" + port + ";serialization="
+        + serialization.name();
+  }
+
+  @AfterClass public static void afterClass() throws Exception {
+    for (HttpServer server : ACTIVE_SERVERS) {
+      if (server != null) {
+        server.stop();
+      }
+    }
+  }
+
+  private static Meta getMeta(AvaticaConnection conn) throws Exception {
+    Field f = AvaticaConnection.class.getDeclaredField("meta");
+    f.setAccessible(true);
+    return (Meta) f.get(conn);
+  }
+
+  private static Meta.ExecuteResult prepareAndExecuteInternal(AvaticaConnection conn,
+    final AvaticaStatement statement, String sql, int maxRowCount) throws Exception {
+    Method m =
+        AvaticaConnection.class.getDeclaredMethod("prepareAndExecuteInternal",
+            AvaticaStatement.class, String.class, long.class);
+    m.setAccessible(true);
+    return (Meta.ExecuteResult) m.invoke(conn, statement, sql, maxRowCount);
+  }
+
+  private static Connection getConnection(JdbcMeta m, String id) throws Exception {
+    Field f = JdbcMeta.class.getDeclaredField("connectionCache");
+    f.setAccessible(true);
+    //noinspection unchecked
+    Cache<String, Connection> connectionCache = (Cache<String, Connection>) f.get(m);
+    return connectionCache.getIfPresent(id);
+  }
+
+  @Test public void testRemoteExecuteMaxRowCount() throws Exception {
+    ConnectionSpec.getDatabaseLock().lock();
+    try (AvaticaConnection conn = (AvaticaConnection) DriverManager.getConnection(url)) {
+      final AvaticaStatement statement = conn.createStatement();
+      prepareAndExecuteInternal(conn, statement,
+        "select * from (values ('a', 1), ('b', 2))", 0);
+      ResultSet rs = statement.getResultSet();
+      int count = 0;
+      while (rs.next()) {
+        count++;
+      }
+      assertEquals("Check maxRowCount=0 and ResultSets is 0 row", count, 0);
+      assertEquals("Check result set meta is still there",
+        rs.getMetaData().getColumnCount(), 2);
+      rs.close();
+      statement.close();
+      conn.close();
+    } finally {
+      ConnectionSpec.getDatabaseLock().unlock();
+    }
+  }
+
+  /** Test case for
+   * <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 {
+    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 {
+    try (AvaticaConnection conn = (AvaticaConnection) DriverManager.getConnection(url)) {
+      final AvaticaStatement statement = conn.createStatement();
+      final String frenchDisko = "It said human existence is pointless\n"
+          + "As acts of rebellious solidarity\n"
+          + "Can bring sense in this world\n"
+          + "La resistance!\n";
+      final String sql = "select '"
+          + longString(frenchDisko, n)
+          + "' as s from (values 'x')";
+      prepareAndExecuteInternal(conn, statement, sql, -1);
+      ResultSet rs = statement.getResultSet();
+      int count = 0;
+      while (rs.next()) {
+        count++;
+      }
+      assertThat(count, is(1));
+      rs.close();
+      statement.close();
+      conn.close();
+    }
+  }
+
+  /** Creates a string of exactly {@code length} characters by concatenating
+   * {@code fragment}. */
+  private static String longString(String fragment, int length) {
+    assert fragment.length() > 0;
+    final StringBuilder buf = new StringBuilder();
+    while (buf.length() < length) {
+      buf.append(fragment);
+    }
+    buf.setLength(length);
+    return buf.toString();
+  }
+
+  @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;
+      assertFalse("remote connection map should start ignorant", m.containsKey(id));
+      // force creating a connection object on the remote side.
+      try (final Statement stmt = conn.createStatement()) {
+        assertTrue("creating a statement starts a local object.", m.containsKey(id));
+        assertTrue(stmt.execute("select count(1) from EMP"));
+      }
+      Connection remoteConn = getConnection(FullyRemoteJdbcMetaFactory.getInstance(), id);
+      final boolean defaultRO = remoteConn.isReadOnly();
+      final boolean defaultAutoCommit = remoteConn.getAutoCommit();
+      final String defaultCatalog = remoteConn.getCatalog();
+      final String defaultSchema = remoteConn.getSchema();
+      conn.setReadOnly(!defaultRO);
+      assertTrue("local changes dirty local state", m.get(id).isDirty());
+      assertEquals("remote connection has not been touched", defaultRO, remoteConn.isReadOnly());
+      conn.setAutoCommit(!defaultAutoCommit);
+      assertEquals("remote connection has not been touched",
+          defaultAutoCommit, remoteConn.getAutoCommit());
+
+      // further interaction with the connection will force a sync
+      try (final Statement stmt = conn.createStatement()) {
+        assertEquals(!defaultAutoCommit, remoteConn.getAutoCommit());
+        assertFalse("local values should be clean", m.get(id).isDirty());
+      }
+    } finally {
+      ConnectionSpec.getDatabaseLock().unlock();
+    }
+  }
+
+  @Test public void testRemoteStatementInsert() throws Exception {
+    final String t = AvaticaUtils.unique("TEST_TABLE2");
+    AvaticaConnection conn = (AvaticaConnection) DriverManager.getConnection(url);
+    Statement statement = conn.createStatement();
+    final String create =
+        String.format("create table if not exists %s ("
+            + "  id int not null, msg varchar(255) not null)", t);
+    int status = statement.executeUpdate(create);
+    assertEquals(status, 0);
+
+    statement = conn.createStatement();
+    final String update = String.format("insert into %s values ('%d', '%s')",
+        t, RANDOM.nextInt(Integer.MAX_VALUE), UUID.randomUUID());
+    status = statement.executeUpdate(update);
+    assertEquals(status, 1);
+  }
+
+  @Test public void testBigints() throws Exception {
+    final String table = "TESTBIGINTS";
+    ConnectionSpec.getDatabaseLock().lock();
+    try (AvaticaConnection conn = (AvaticaConnection) DriverManager.getConnection(url);
+        Statement stmt = conn.createStatement()) {
+      assertFalse(stmt.execute("DROP TABLE IF EXISTS " + table));
+      assertFalse(stmt.execute("CREATE TABLE " + table + " (id BIGINT)"));
+      assertFalse(stmt.execute("INSERT INTO " + table + " values(10)"));
+      ResultSet results = conn.getMetaData().getColumns(null, null, table, null);
+      assertTrue(results.next());
+      assertEquals(table, results.getString(3));
+      // ordinal position
+      assertEquals(1L, results.getLong(17));
+    } finally {
+      ConnectionSpec.getDatabaseLock().unlock();
+    }
+  }
+
+  @Test public void testOpenConnectionWithProperties() throws Exception {
+    // This tests that username and password are used for creating a connection on the
+    // server. If this was not the case, it would succeed.
+    try {
+      DriverManager.getConnection(url, "john", "doe");
+      fail("expected exception");
+    } catch (RuntimeException e) {
+      assertEquals("Remote driver error: RuntimeException: "
+          + "java.sql.SQLInvalidAuthorizationSpecException: invalid authorization specification"
+          + " - not found: john"
+          + " -> SQLInvalidAuthorizationSpecException: invalid authorization specification - "
+          + "not found: john"
+          + " -> HsqlException: invalid authorization specification - not found: john",
+          e.getMessage());
+    }
+  }
+
+  @Test public void testRemoteConnectionsAreDifferent() throws SQLException {
+    Connection conn1 = DriverManager.getConnection(url);
+    Statement stmt = conn1.createStatement();
+    stmt.execute("DECLARE LOCAL TEMPORARY TABLE"
+        + " buffer (id INTEGER PRIMARY KEY, textdata VARCHAR(100))");
+    stmt.execute("insert into buffer(id, textdata) values(1, 'abc')");
+    stmt.executeQuery("select * from buffer");
+
+    // The local temporary table is local to the connection above, and should
+    // not be visible on another connection
+    Connection conn2 = DriverManager.getConnection(url);
+    Statement stmt2 = conn2.createStatement();
+    try {
+      stmt2.executeQuery("select * from buffer");
+      fail("expected exception");
+    } catch (Exception e) {
+      assertEquals("Error -1 (00000) : Error while executing SQL \"select * from buffer\": "
+          + "Remote driver error: RuntimeException: java.sql.SQLSyntaxErrorException: "
+          + "user lacks privilege or object not found: BUFFER -> "
+          + "SQLSyntaxErrorException: user lacks privilege or object not found: BUFFER -> "
+          + "HsqlException: user lacks privilege or object not found: BUFFER",
+          e.getMessage());
+    }
+  }
+
+  @Ignore("[CALCITE-942] AvaticaConnection should fail-fast when closed.")
+  @Test public void testRemoteConnectionClosing() throws Exception {
+    AvaticaConnection conn = (AvaticaConnection) DriverManager.getConnection(url);
+    // Verify connection is usable
+    conn.createStatement();
+    conn.close();
+
+    // After closing the connection, it should not be usable anymore
+    try {
+      conn.createStatement();
+      fail("expected exception");
+    } catch (SQLException e) {
+      assertThat(e.getMessage(),
+          containsString("Connection is closed"));
+    }
+  }
+
+  @Test public void testExceptionPropagation() throws Exception {
+    final String sql = "SELECT * from EMP LIMIT FOOBARBAZ";
+    ConnectionSpec.getDatabaseLock().lock();
+    try (final AvaticaConnection conn = (AvaticaConnection) DriverManager.getConnection(url);
+        final Statement stmt = conn.createStatement()) {
+      try {
+        // invalid SQL
+        stmt.execute(sql);
+        fail("Expected an AvaticaSqlException");
+      } catch (AvaticaSqlException e) {
+        assertEquals(ErrorResponse.UNKNOWN_ERROR_CODE, e.getErrorCode());
+        assertEquals(ErrorResponse.UNKNOWN_SQL_STATE, e.getSQLState());
+        assertTrue("Message should contain original SQL, was '" + e.getMessage() + "'",
+            e.getMessage().contains(sql));
+        assertEquals(1, e.getStackTraces().size());
+        final String stacktrace = e.getStackTraces().get(0);
+        final String substring = "unexpected token: FOOBARBAZ";
+        assertTrue("Message should contain '" + substring + "', was '" + e.getMessage() + ",",
+            stacktrace.contains(substring));
+      }
+    } finally {
+      ConnectionSpec.getDatabaseLock().unlock();
+    }
+  }
+
+  @Test public void testRemoteColumnsMeta() throws Exception {
+    // Verify all columns are retrieved, thus that frame-based fetching works correctly for columns
+    int rowCount = 0;
+    try (AvaticaConnection conn = (AvaticaConnection) DriverManager.getConnection(url)) {
+      ResultSet rs = conn.getMetaData().getColumns(null, null, null, null);
+      while (rs.next()) {
+        rowCount++;
+      }
+      rs.close();
+
+      // The implicitly created statement should have been closed
+      assertTrue(rs.getStatement().isClosed());
+    }
+    // default fetch size is 100, we are well beyond it
+    assertTrue(rowCount > 900);
+  }
+
+  @Test public void testArrays() throws SQLException {
+    ConnectionSpec.getDatabaseLock().lock();
+    try (AvaticaConnection conn = (AvaticaConnection) DriverManager.getConnection(url);
+         Statement stmt = conn.createStatement()) {
+      ResultSet resultSet =
+          stmt.executeQuery("select * from (values ('a', array['b', 'c']));");
+
+      assertTrue(resultSet.next());
+      assertEquals("a", resultSet.getString(1));
+      Array arr = resultSet.getArray(2);
+      assertTrue(arr instanceof ArrayImpl);
+      Object[] values = (Object[]) ((ArrayImpl) arr).getArray();
+      assertArrayEquals(new String[]{"b", "c"}, values);
+    } finally {
+      ConnectionSpec.getDatabaseLock().unlock();
+    }
+  }
+
+  @Test public void testBinaryAndStrings() throws Exception {
+    final String tableName = "testbinaryandstrs";
+    final byte[] data = "asdf".getBytes(StandardCharsets.UTF_8);
+    ConnectionSpec.getDatabaseLock().lock();
+    try (final Connection conn = DriverManager.getConnection(url);
+        final Statement stmt = conn.createStatement()) {
+      assertFalse(stmt.execute("DROP TABLE IF EXISTS " + tableName));
+      assertFalse(stmt.execute("CREATE TABLE " + tableName + "(id int, bin BINARY(4))"));
+      try (final PreparedStatement prepStmt = conn.prepareStatement(
+          "INSERT INTO " + tableName + " values(1, ?)")) {
+        prepStmt.setBytes(1, data);
+        assertFalse(prepStmt.execute());
+      }
+      try (ResultSet results = stmt.executeQuery("SELECT id, bin from " + tableName)) {
+        assertTrue(results.next());
+        assertEquals(1, results.getInt(1));
+        // byte comparison should work
+        assertArrayEquals("Bytes were " + Arrays.toString(results.getBytes(2)),
+            data, results.getBytes(2));
+        // as should string
+        assertEquals(new String(data, StandardCharsets.UTF_8), results.getString(2));
+        assertFalse(results.next());
+      }
+    } finally {
+      ConnectionSpec.getDatabaseLock().unlock();
+    }
+  }
+
+  @Test public void testLocalStackTraceHasServerStackTrace() {
+    ConnectionSpec.getDatabaseLock().lock();
+    try {
+      Statement statement = DriverManager.getConnection(url).createStatement();
+      statement.executeQuery("SELECT * FROM BOGUS_TABLE_DEF_DOESNT_EXIST");
+    } catch (SQLException e) {
+      // Verify that we got the expected exception
+      assertThat(e, instanceOf(AvaticaSqlException.class));
+
+      // Attempt to verify that we got a "server-side" class in the stack.
+      assertThat(Throwables.getStackTraceAsString(e),
+          containsString(JdbcMeta.class.getName()));
+    } finally {
+      ConnectionSpec.getDatabaseLock().unlock();
+    }
+  }
+
+  @Test public void testServerAddressInResponse() throws Exception {
+    ConnectionSpec.getDatabaseLock().lock();
+    try {
+      URL url = new URL("http://localhost:" + this.port);
+      AvaticaHttpClient httpClient = new AvaticaHttpClientImpl(url);
+      byte[] request;
+
+      Service.OpenConnectionRequest jsonReq = new Service.OpenConnectionRequest(
+          UUID.randomUUID().toString(), Collections.<String, String>emptyMap());
+      switch (this.serialization) {
+      case JSON:
+        request = JsonService.MAPPER.writeValueAsBytes(jsonReq);
+        break;
+      case PROTOBUF:
+        ProtobufTranslation pbTranslation = new ProtobufTranslationImpl();
+        request = pbTranslation.serializeRequest(jsonReq);
+        break;
+      default:
+        throw new IllegalStateException("Should not reach here");
+      }
+
+      byte[] response = httpClient.send(request);
+      Service.OpenConnectionResponse openCnxnResp;
+      switch (this.serialization) {
+      case JSON:
+        openCnxnResp = JsonService.MAPPER.readValue(response,
+            Service.OpenConnectionResponse.class);
+        break;
+      case PROTOBUF:
+        ProtobufTranslation pbTranslation = new ProtobufTranslationImpl();
+        Response genericResp = pbTranslation.parseResponse(response);
+        assertTrue("Expected an OpenConnnectionResponse, but got " + genericResp.getClass(),
+            genericResp instanceof Service.OpenConnectionResponse);
+        openCnxnResp = (Service.OpenConnectionResponse) genericResp;
+        break;
+      default:
+        throw new IllegalStateException("Should not reach here");
+      }
+
+      String hostname = InetAddress.getLocalHost().getHostName();
+
+      assertNotNull(openCnxnResp.rpcMetadata);
+      assertEquals(hostname + ":" + this.port, openCnxnResp.rpcMetadata.serverAddress);
+    } finally {
+      ConnectionSpec.getDatabaseLock().unlock();
+    }
+  }
+
+  @Test public void testCommitRollback() throws Exception {
+    final String productTable = "commitrollback_products";
+    final String salesTable = "commitrollback_sales";
+    ConnectionSpec.getDatabaseLock().lock();
+    try (final Connection conn = DriverManager.getConnection(url);
+        final Statement stmt = conn.createStatement()) {
+      assertFalse(stmt.execute("DROP TABLE IF EXISTS " + productTable));
+      assertFalse(
+          stmt.execute(
+              String.format("CREATE TABLE %s(id integer, stock integer)", productTable)));
+      assertFalse(stmt.execute("DROP TABLE IF EXISTS " + salesTable));
+      assertFalse(
+          stmt.execute(
+              String.format("CREATE TABLE %s(id integer, units_sold integer)", salesTable)));
+
+      final int productId = 1;
+      // No products and no sales
+      assertFalse(
+          stmt.execute(
+              String.format("INSERT INTO %s VALUES(%d, 0)", productTable, productId)));
+      assertFalse(
+          stmt.execute(
+              String.format("INSERT INTO %s VALUES(%d, 0)", salesTable, productId)));
+
+      conn.setAutoCommit(false);
+      PreparedStatement productStmt = conn.prepareStatement(
+          String.format("UPDATE %s SET stock = stock + ? WHERE id = ?", productTable));
+      PreparedStatement salesStmt = conn.prepareStatement(
+          String.format("UPDATE %s SET units_sold = units_sold + ? WHERE id = ?", salesTable));
+
+      // No stock
+      assertEquals(0, getInventory(conn, productTable, productId));
+
+      // Set a stock of 10 for product 1
+      productStmt.setInt(1, 10);
+      productStmt.setInt(2, productId);
+      productStmt.executeUpdate();
+
+      conn.commit();
+      assertEquals(10, getInventory(conn, productTable, productId));
+
+      // Sold 5 items (5 in stock, 5 sold)
+      productStmt.setInt(1, -5);
+      productStmt.setInt(2, productId);
+      productStmt.executeUpdate();
+      salesStmt.setInt(1, 5);
+      salesStmt.setInt(2, productId);
+      salesStmt.executeUpdate();
+
+      conn.commit();
+      // We will definitely see the updated values
+      assertEquals(5, getInventory(conn, productTable, productId));
+      assertEquals(5, getSales(conn, salesTable, productId));
+
+      // Update some "bad" values
+      productStmt.setInt(1, -10);
+      productStmt.setInt(2, productId);
+      productStmt.executeUpdate();
+      salesStmt.setInt(1, 10);
+      salesStmt.setInt(2, productId);
+      salesStmt.executeUpdate();
+
+      // We just went negative, nonsense. Better rollback.
+      conn.rollback();
+
+      // Should still have 5 and 5
+      assertEquals(5, getInventory(conn, productTable, productId));
+      assertEquals(5, getSales(conn, salesTable, productId));
+    } finally {
+      ConnectionSpec.getDatabaseLock().unlock();
+    }
+  }
+
+  private int getInventory(Connection conn, String productTable, int productId) throws Exception {
+    try (Statement stmt = conn.createStatement()) {
+      ResultSet results = stmt.executeQuery(
+          String.format("SELECT stock FROM %s WHERE id = %d", productTable, productId));
+      assertTrue(results.next());
+      return results.getInt(1);
+    }
+  }
+
+  private int getSales(Connection conn, String salesTable, int productId) throws Exception {
+    try (Statement stmt = conn.createStatement()) {
+      ResultSet results = stmt.executeQuery(
+          String.format("SELECT units_sold FROM %s WHERE id = %d", salesTable, productId));
+      assertTrue(results.next());
+      return results.getInt(1);
+    }
+  }
+
+  /** Factory that provides a {@link JdbcMeta}. */
+  public static class FullyRemoteJdbcMetaFactory implements Meta.Factory {
+
+    private static JdbcMeta instance = null;
+
+    private static JdbcMeta getInstance() {
+      if (instance == null) {
+        try {
+          instance = new JdbcMeta(CONNECTION_SPEC.url, CONNECTION_SPEC.username,
+              CONNECTION_SPEC.password);
+        } catch (SQLException e) {
+          throw new RuntimeException(e);
+        }
+      }
+      return instance;
+    }
+
+    @Override public Meta create(List<String> args) {
+      return getInstance();
+    }
+  }
+}
+
+// End RemoteMetaTest.java

http://git-wip-us.apache.org/repos/asf/calcite/blob/5cee486f/avatica/server/src/test/java/org/apache/calcite/avatica/server/HandlerFactoryTest.java
----------------------------------------------------------------------
diff --git a/avatica/server/src/test/java/org/apache/calcite/avatica/server/HandlerFactoryTest.java b/avatica/server/src/test/java/org/apache/calcite/avatica/server/HandlerFactoryTest.java
new file mode 100644
index 0000000..3504e02
--- /dev/null
+++ b/avatica/server/src/test/java/org/apache/calcite/avatica/server/HandlerFactoryTest.java
@@ -0,0 +1,58 @@
+/*
+ * 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.server;
+
+import org.apache.calcite.avatica.remote.Driver.Serialization;
+import org.apache.calcite.avatica.remote.Service;
+
+import org.eclipse.jetty.server.Handler;
+import org.junit.Before;
+import org.junit.Test;
+import org.mockito.Mockito;
+
+import static org.junit.Assert.assertTrue;
+
+/**
+ * Tests the {@link HandlerFactory} implementation.
+ */
+public class HandlerFactoryTest {
+
+  private HandlerFactory factory;
+  private Service service;
+
+  @Before
+  public void setup() {
+    this.factory = new HandlerFactory();
+    this.service = Mockito.mock(Service.class);
+  }
+
+  @Test
+  public void testJson() {
+    Handler handler = factory.getHandler(service, Serialization.JSON);
+    assertTrue("Expected an implementation of the AvaticaHandler, "
+        + "but got " + handler.getClass(), handler instanceof AvaticaJsonHandler);
+  }
+
+  @Test
+  public void testProtobuf() {
+    Handler handler = factory.getHandler(service, Serialization.PROTOBUF);
+    assertTrue("Expected an implementation of the AvaticaProtobufHandler, "
+        + "but got " + handler.getClass(), handler instanceof AvaticaProtobufHandler);
+  }
+}
+
+// End HandlerFactoryTest.java

http://git-wip-us.apache.org/repos/asf/calcite/blob/5cee486f/avatica/server/src/test/java/org/apache/calcite/avatica/test/AvaticaSuite.java
----------------------------------------------------------------------
diff --git a/avatica/server/src/test/java/org/apache/calcite/avatica/test/AvaticaSuite.java b/avatica/server/src/test/java/org/apache/calcite/avatica/test/AvaticaSuite.java
new file mode 100644
index 0000000..4a0c26c
--- /dev/null
+++ b/avatica/server/src/test/java/org/apache/calcite/avatica/test/AvaticaSuite.java
@@ -0,0 +1,37 @@
+/*
+ * 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.test;
+
+import org.apache.calcite.avatica.RemoteDriverTest;
+
+import org.junit.runner.RunWith;
+
+import org.junit.runners.Suite;
+
+/**
+ * Avatica test suite.
+ */
+@RunWith(Suite.class)
+@Suite.SuiteClasses({
+    AvaticaUtilsTest.class,
+    ConnectStringParserTest.class,
+    RemoteDriverTest.class
+})
+public class AvaticaSuite {
+}
+
+// End AvaticaSuite.java

http://git-wip-us.apache.org/repos/asf/calcite/blob/5cee486f/avatica/server/src/test/resources/log4j.properties
----------------------------------------------------------------------
diff --git a/avatica/server/src/test/resources/log4j.properties b/avatica/server/src/test/resources/log4j.properties
new file mode 100644
index 0000000..834e2db
--- /dev/null
+++ b/avatica/server/src/test/resources/log4j.properties
@@ -0,0 +1,24 @@
+# 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.
+
+# Root logger is configured at INFO and is sent to A1
+log4j.rootLogger=INFO, A1
+
+# A1 goes to the console
+log4j.appender.A1=org.apache.log4j.ConsoleAppender
+
+# Set the pattern for each log message
+log4j.appender.A1.layout=org.apache.log4j.PatternLayout
+log4j.appender.A1.layout.ConversionPattern=%d [%t] %-5p - %m%n

http://git-wip-us.apache.org/repos/asf/calcite/blob/5cee486f/avatica/site/.gitignore
----------------------------------------------------------------------
diff --git a/avatica/site/.gitignore b/avatica/site/.gitignore
new file mode 100644
index 0000000..09c86a2
--- /dev/null
+++ b/avatica/site/.gitignore
@@ -0,0 +1,2 @@
+.sass-cache
+Gemfile.lock

http://git-wip-us.apache.org/repos/asf/calcite/blob/5cee486f/avatica/site/Gemfile
----------------------------------------------------------------------
diff --git a/avatica/site/Gemfile b/avatica/site/Gemfile
new file mode 100644
index 0000000..c13c470
--- /dev/null
+++ b/avatica/site/Gemfile
@@ -0,0 +1,20 @@
+# 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.
+#
+source 'https://rubygems.org'
+gem 'github-pages'
+gem 'rouge'
+gem 'jekyll-oembed', :require => 'jekyll_oembed'
+# End Gemfile

http://git-wip-us.apache.org/repos/asf/calcite/blob/5cee486f/avatica/site/README.md
----------------------------------------------------------------------
diff --git a/avatica/site/README.md b/avatica/site/README.md
new file mode 100644
index 0000000..ea3f212
--- /dev/null
+++ b/avatica/site/README.md
@@ -0,0 +1,57 @@
+<!--
+{% comment %}
+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.
+{% endcomment %}
+-->
+
+# Apache Calcite docs site
+
+This directory contains the code for the Apache Calcite web site,
+[calcite.apache.org](https://calcite.apache.org/).
+
+## Setup
+
+1. `cd site`
+2. `svn co https://svn.apache.org/repos/asf/calcite/site target`
+3. `sudo apt-get install rubygems ruby2.1-dev zlib1g-dev` (linux)
+4. `sudo gem install bundler github-pages jekyll jekyll-oembed`
+5. `bundle install`
+
+## Add javadoc
+
+1. `cd ..`
+2. `mvn -DskipTests site`
+3. `rm -rf site/target/apidocs site/target/testapidocs`
+4. `mv target/site/apidocs target/site/testapidocs site/target`
+
+## Running locally
+
+Before opening a pull request, you can preview your contributions by
+running from within the directory:
+
+1. `bundle exec jekyll serve`
+2. Open [http://localhost:4000](http://localhost:4000)
+
+## Pushing to site
+
+1. `cd site/target`
+2. `svn status`
+3. You'll need to `svn add` any new files
+4. `svn ci`
+
+Within a few minutes, svnpubsub should kick in and you'll be able to
+see the results at
+[calcite.apache.org](https://calcite.apache.org/).

http://git-wip-us.apache.org/repos/asf/calcite/blob/5cee486f/avatica/site/_config.yml
----------------------------------------------------------------------
diff --git a/avatica/site/_config.yml b/avatica/site/_config.yml
new file mode 100644
index 0000000..9b8de85
--- /dev/null
+++ b/avatica/site/_config.yml
@@ -0,0 +1,43 @@
+# 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.
+#
+markdown: kramdown
+permalink: /news/:year/:month/:day/:title/
+excerpt_separator: ""
+
+repository: https://github.com/apache/calcite
+destination: target
+exclude: [README.md,Gemfile*]
+keep_files: [".git", ".svn", "apidocs", "testapidocs"]
+
+collections:
+  docs:
+    output: true
+
+# The URL where the code can be found
+sourceRoot: https://github.com/apache/calcite/blob/master
+
+# The URL where Javadocs are located
+apiRoot: /apidocs
+# apiRoot: http://calcite.apache.org/apidocs
+
+# The URL where Test Javadocs are located
+testApiRoot: /testapidocs
+# testApiRoot: http://calcite.apache.org/testapidocs
+
+# The base path where the website is deployed
+baseurl:
+
+# End _config.yml

http://git-wip-us.apache.org/repos/asf/calcite/blob/5cee486f/avatica/site/_data/contributors.yml
----------------------------------------------------------------------
diff --git a/avatica/site/_data/contributors.yml b/avatica/site/_data/contributors.yml
new file mode 100644
index 0000000..98f8bf9
--- /dev/null
+++ b/avatica/site/_data/contributors.yml
@@ -0,0 +1,96 @@
+# 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.
+#
+# Database of contributors to Apache Calcite.
+# Pages such as developer.md use this data.
+#
+- name: Alan Gates
+  apacheId: gates
+  githubId: alanfgates
+  org: Hortonworks
+  role: PMC
+- name: Aman Sinha
+  apacheId: amansinha
+  githubId: amansinha100
+  org: MapR
+  role: PMC
+- name: Ashutosh Chauhan
+  apacheId: hashutosh
+  githubId: ashutoshc
+  org: Hortonworks
+  role: PMC
+- name: James R. Taylor
+  apacheId: jamestaylor
+  githubId: JamesRTaylor
+  org: Salesforce
+  role: PMC
+- name: Jacques Nadeau
+  apacheId: jacques
+  githubId: jacques-n
+  org: Dremio
+  role: PMC
+- name: Jesús Camacho Rodríguez
+  apacheId: jcamacho
+  githubId: jcamachor
+  org: Hortonworks
+  role: PMC
+- name: Jinfeng Ni
+  apacheId: jni
+  githubId: jinfengni
+  org: MapR
+  role: PMC
+- name: John Pullokkaran
+  apacheId: jpullokk
+  githubId: jpullokkaran
+  org: Hortonworks
+  role: PMC
+- name: Josh Elser
+  apacheId: elserj
+  githubId: joshelser
+  org: Hortonworks
+  role: PMC
+- name: Julian Hyde
+  apacheId: jhyde
+  githubId: julianhyde
+  org: Hortonworks
+  role: PMC Chair
+  homepage: http://people.apache.org/~jhyde
+- name: Maryann Xue
+  apacheId: maryannxue
+  githubId: maryannxue
+  org: Intel
+  role: Committer
+- name: Nick Dimiduk
+  apacheId: ndimiduk
+  githubId: ndimiduk
+  org: Hortonworks
+  role: PMC
+- name: Steven Noels
+  apacheId: stevenn
+  githubId: stevenn
+  org: NGData
+  role: PMC
+- name: Ted Dunning
+  apacheId: tdunning
+  githubId: tdunning
+  org: MapR
+  role: PMC
+  avatar: https://www.mapr.com/sites/default/files/otherpageimages/ted-circle-80.png
+- name: Vladimir Sitnikov
+  apacheId: vladimirsitnikov
+  githubId: vlsi
+  org: NetCracker
+  role: PMC
+# End contributors.yml

http://git-wip-us.apache.org/repos/asf/calcite/blob/5cee486f/avatica/site/_data/docs.yml
----------------------------------------------------------------------
diff --git a/avatica/site/_data/docs.yml b/avatica/site/_data/docs.yml
new file mode 100644
index 0000000..a996097
--- /dev/null
+++ b/avatica/site/_data/docs.yml
@@ -0,0 +1,48 @@
+# 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.
+#
+# Data that defines menu structure
+#
+- title: Overview
+  docs:
+  - index
+  - tutorial
+  - algebra
+
+- title: Advanced
+  docs:
+  - adapter
+  - stream
+  - lattice
+
+- title: Avatica
+  docs:
+  - avatica_overview
+  - avatica_roadmap
+  - avatica_json_reference
+  - avatica_protobuf_reference
+
+- title: Reference
+  docs:
+  - reference
+  - model
+  - howto
+
+- title: Meta
+  docs:
+  - history
+  - api
+  - testapi
+# End docs.yml

http://git-wip-us.apache.org/repos/asf/calcite/blob/5cee486f/avatica/site/_docs/adapter.md
----------------------------------------------------------------------
diff --git a/avatica/site/_docs/adapter.md b/avatica/site/_docs/adapter.md
new file mode 100644
index 0000000..fa8eb21
--- /dev/null
+++ b/avatica/site/_docs/adapter.md
@@ -0,0 +1,38 @@
+---
+layout: docs
+title: Adapters
+permalink: /docs/adapter.html
+---
+<!--
+{% comment %}
+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.
+{% endcomment %}
+-->
+
+* <a href="https://github.com/apache/drill">Apache Drill adapter</a>
+* Cascading adapter (<a href="https://github.com/Cascading/lingual">Lingual</a>)
+* Cassandra adapter (<a href="{{ site.apiRoot }}/org/apache/calcite/adapter/cassandra/package-summary.html">calcite-cassandra</a>)
+* CSV adapter (<a href="{{ site.apiRoot }}/org/apache/calcite/adapter/csv/package-summary.html">example/csv</a>)
+* JDBC adapter (part of <a href="{{ site.apiRoot }}/org/apache/calcite/adapter/jdbc/package-summary.html">calcite-core</a>)
+* MongoDB adapter (<a href="{{ site.apiRoot }}/org/apache/calcite/adapter/mongodb/package-summary.html">calcite-mongodb</a>)
+* Spark adapter (<a href="{{ site.apiRoot }}/org/apache/calcite/adapter/spark/package-summary.html">calcite-spark</a>)
+* Splunk adapter (<a href="{{ site.apiRoot }}/org/apache/calcite/adapter/splunk/package-summary.html">calcite-splunk</a>)
+* Eclipse Memory Analyzer (MAT) adapter (<a href="https://github.com/vlsi/mat-calcite-plugin">mat-calcite-plugin</a>)
+
+## Drivers
+
+* <a href="{{ site.apiRoot }}/org/apache/calcite/jdbc/package-summary.html">JDBC driver</a>
+

http://git-wip-us.apache.org/repos/asf/calcite/blob/5cee486f/avatica/site/_docs/algebra.md
----------------------------------------------------------------------
diff --git a/avatica/site/_docs/algebra.md b/avatica/site/_docs/algebra.md
new file mode 100644
index 0000000..e651dd0
--- /dev/null
+++ b/avatica/site/_docs/algebra.md
@@ -0,0 +1,369 @@
+---
+layout: docs
+title: Algebra
+permalink: /docs/algebra.html
+---
+<!--
+{% comment %}
+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.
+{% endcomment %}
+-->
+
+Relational algebra is at the heart of Calcite. Every query is
+represented as a tree of relational operators. You can translate from
+SQL to relational algebra, or you can build the tree directly.
+
+Planner rules transform expression trees using mathematical identities
+that preserve semantics. For example, it is valid to push a filter
+into an input of an inner join if the filter does not reference
+columns from the other input.
+
+Calcite optimizes queries by repeatedly applying planner rules to a
+relational expression. A cost model guides the process, and the
+planner engine generates an alternative expression that has the same
+semantics as the original but a lower cost.
+
+The planning process is extensible. You can add your own relational
+operators, planner rules, cost model, and statistics.
+
+## Algebra builder
+
+The simplest way to build a relational expression is to use the algebra builder,
+[RelBuilder]({{ site.apiRoot }}/org/apache/calcite/tools/RelBuilder.html).
+Here is an example:
+
+### TableScan
+
+{% highlight java %}
+final FrameworkConfig config;
+final RelBuilder builder = RelBuilder.create(config);
+final RelNode node = builder
+  .scan("EMP")
+  .build();
+System.out.println(RelOptUtil.toString(node));
+{% endhighlight %}
+
+(You can find the full code for this and other examples in
+[RelBuilderExample.java]({{ site.sourceRoot }}/core/src/test/java/org/apache/calcite/examples/RelBuilderExample.java).)
+
+The code prints
+
+{% highlight text %}
+LogicalTableScan(table=[[scott, EMP]])
+{% endhighlight %}
+
+It has created a scan of the `EMP` table; equivalent to the SQL
+
+{% highlight sql %}
+SELECT *
+FROM scott.EMP;
+{% endhighlight %}
+
+### Adding a Project
+
+Now, let's add a Project, the equivalent of
+
+{% highlight sql %}
+SELECT ename, deptno
+FROM scott.EMP;
+{% endhighlight %}
+
+We just add a call to the `project` method before calling
+`build`:
+
+{% highlight java %}
+final RelNode node = builder
+  .scan("EMP")
+  .project(builder.field("DEPTNO"), builder.field("ENAME"))
+  .build();
+System.out.println(RelOptUtil.toString(node));
+{% endhighlight %}
+
+and the output is
+
+{% highlight text %}
+LogicalProject(DEPTNO=[$7], ENAME=[$1])
+  LogicalTableScan(table=[[scott, EMP]])
+{% endhighlight %}
+
+The two calls to `builder.field` create simple expressions
+that return the fields from the input relational expression,
+namely the TableScan created by the `scan` call.
+
+Calcite has converted them to field references by ordinal,
+`$7` and `$1`.
+
+### Adding a Filter and Aggregate
+
+A query with an Aggregate, and a Filter:
+
+{% highlight java %}
+final RelNode node = builder
+  .scan("EMP")
+  .aggregate(builder.groupKey("DEPTNO"),
+      builder.count(false, "C"),
+      builder.sum(false, "S", builder.field("SAL")))
+  .filter(
+      builder.call(SqlStdOperatorTable.GREATER_THAN,
+          builder.field("C"),
+          builder.literal(10)))
+  .build();
+System.out.println(RelOptUtil.toString(node));
+{% endhighlight %}
+
+is equivalent to SQL
+
+{% highlight sql %}
+SELECT deptno, count(*) AS c, sum(sal) AS s
+FROM emp
+GROUP BY deptno
+HAVING count(*) > 10
+{% endhighlight %}
+
+and produces
+
+{% highlight text %}
+LogicalFilter(condition=[>($1, 10)])
+  LogicalAggregate(group=[{7}], C=[COUNT()], S=[SUM($5)])
+    LogicalTableScan(table=[[scott, EMP]])
+{% endhighlight %}
+
+### Push and pop
+
+The builder uses a stack to store the relational expression produced by
+one step and pass it as an input to the next step. This allows the
+methods that produce relational expressions to produce a builder.
+
+Most of the time, the only stack method you will use is `build()`, to get the
+last relational expression, namely the root of the tree.
+
+Sometimes the stack becomes so deeply nested it gets confusing. To keep things
+straight, you can remove expressions from the stack. For example, here we are
+building a bushy join:
+
+{% highlight text %}
+.
+               join
+             /      \
+        join          join
+      /      \      /      \
+CUSTOMERS ORDERS LINE_ITEMS PRODUCTS
+{% endhighlight %}
+
+We build it in three stages. Store the intermediate results in variables
+`left` and `right`, and use `push()` to put them back on the stack when it is
+time to create the final `Join`:
+
+{% highlight java %}
+final RelNode left = builder
+  .scan("CUSTOMERS")
+  .scan("ORDERS")
+  .join(JoinRelType.INNER, "ORDER_ID")
+  .build();
+
+final RelNode right = builder
+  .scan("LINE_ITEMS")
+  .scan("PRODUCTS")
+  .join(JoinRelType.INNER, "PRODUCT_ID")
+  .build();
+
+final RelNode result = builder
+  .push(left)
+  .push(right)
+  .join(JoinRelType.INNER, "ORDER_ID")
+  .build();
+{% endhighlight %}
+
+### Field names and ordinals
+
+You can reference a field by name or ordinal.
+
+Ordinals are zero-based. Each operator guarantees the order in which its output
+fields occur. For example, `Project` returns the fields in the generated by
+each of the scalar expressions.
+
+The field names of an operator are guaranteed to be unique, but sometimes that
+means that the names are not exactly what you expect. For example, when you
+join EMP to DEPT, one of the output fields will be called DEPTNO and another
+will be called something like DEPTNO_1.
+
+Some relational expression methods give you more control over field names:
+
+* `project` lets you wrap expressions using `alias(expr, fieldName)`. It
+  removes the wrapper but keeps the suggested name (as long as it is unique).
+* `values(String[] fieldNames, Object... values)` accepts an array of field
+  names. If any element of the array is null, the builder will generate a unique
+  name.
+
+If an expression projects an input field, or a cast of an input field, it will
+use the name of that input field.
+
+Once the unique field names have been assigned, the names are immutable.
+If you have a particular `RelNode` instance, you can rely on the field names not
+changing. In fact, the whole relational expression is immutable.
+
+But if a relational expression has passed through several rewrite rules (see
+([RelOptRule]({{ site.apiRoot }}/org/apache/calcite/plan/RelOptRule.html)), the field
+names of the resulting expression might not look much like the originals.
+At that point it is better to reference fields by ordinal.
+
+When you are building a relational expression that accepts multiple inputs,
+you need to build field references that take that into account. This occurs
+most often when building join conditions.
+
+Suppose you are building a join on EMP,
+which has 8 fields [EMPNO, ENAME, JOB, MGR, HIREDATE, SAL, COMM, DEPTNO]
+and DEPT,
+which has 3 fields [DEPTNO, DNAME, LOC].
+Internally, Calcite represents those fields as offsets into
+a combined input row with 11 fields: the first field of the left input is
+field #0 (0-based, remember), and the first field of the right input is
+field #8.
+
+But through the builder API, you specify which field of which input.
+To reference "SAL", internal field #5,
+write `builder.field(2, 0, "SAL")`
+or `builder.field(2, 0, 5)`.
+This means "the field #5 of input #0 of two inputs".
+(Why does it need to know that there are two inputs? Because they are stored on
+the stack; input #1 is at the top of the stack, and input #0 is below it.
+If we did not tell the builder that were two inputs, it would not know how deep
+to go for input #0.)
+
+Similarly, to reference "DNAME", internal field #9 (8 + 1),
+write `builder.field(2, 1, "DNAME")`
+or `builder.field(2, 1, 1)`.
+
+### API summary
+
+#### Relational operators
+
+The following methods create a relational expression
+([RelNode]({{ site.apiRoot }}/org/apache/calcite/rel/RelNode.html)),
+push it onto the stack, and
+return the `RelBuilder`.
+
+| Method              | Description
+|:------------------- |:-----------
+| `scan(tableName)` | Creates a [TableScan]({{ site.apiRoot }}/org/apache/calcite/rel/core/TableScan.html).
+| `values(fieldNames, value...)`<br/>`values(rowType, tupleList)` | Creates a [Values]({{ site.apiRoot }}/org/apache/calcite/rel/core/Values.html).
+| `filter(expr...)`<br/>`filter(exprList)` | Creates a [Filter]({{ site.apiRoot }}/org/apache/calcite/rel/core/Filter.html) over the AND of the given predicates.
+| `project(expr...)`<br/>`project(exprList [, fieldNames])` | Creates a [Project]({{ site.apiRoot }}/org/apache/calcite/rel/core/Project.html). To override the default name, wrap expressions using `alias`, or specify the `fieldNames` argument.
+| `permute(mapping)` | Creates a [Project]({{ site.apiRoot }}/org/apache/calcite/rel/core/Project.html) that permutes the fields using `mapping`.
+| `convert(rowType [, rename])` | Creates a [Project]({{ site.apiRoot }}/org/apache/calcite/rel/core/Project.html) that converts the fields to the given types, optionally also renaming them.
+| `aggregate(groupKey, aggCall...)`<br/>`aggregate(groupKey, aggCallList)` | Creates an [Aggregate]({{ site.apiRoot }}/org/apache/calcite/rel/core/Aggregate.html).
+| `distinct()` | Creates an [Aggregate]({{ site.apiRoot }}/org/apache/calcite/rel/core/Aggregate.html) that eliminates duplicate records.
+| `sort(fieldOrdinal...)`<br/>`sort(expr...)`<br/>`sort(exprList)` | Creates a [Sort]({{ site.apiRoot }}/org/apache/calcite/rel/core/Sort.html).<br/><br/>In the first form, field ordinals are 0-based, and a negative ordinal indicates descending; for example, -2 means field 1 descending.<br/><br/>In the other forms, you can wrap expressions in `as`, `nullsFirst` or `nullsLast`.
+| `sortLimit(offset, fetch, expr...)`<br/>`sortLimit(offset, fetch, exprList)` | Creates a [Sort]({{ site.apiRoot }}/org/apache/calcite/rel/core/Sort.html) with offset and limit.
+| `limit(offset, fetch)` | Creates a [Sort]({{ site.apiRoot }}/org/apache/calcite/rel/core/Sort.html) that does not sort, only applies with offset and limit.
+| `join(joinType, expr...)`<br/>`join(joinType, exprList)`<br/>`join(joinType, fieldName...)` | Creates a [Join]({{ site.apiRoot }}/org/apache/calcite/rel/core/Join.html) of the two most recent relational expressions.<br/><br/>The first form joins on a boolean expression (multiple conditions are combined using AND).<br/><br/>The last form joins on named fields; each side must have a field of each name.
+| `semiJoin(expr)` | Creates a [SemiJoin]({{ site.apiRoot }}/org/apache/calcite/rel/core/SemiJoin.html) of the two most recent relational expressions.
+| `union(all [, n])` | Creates a [Union]({{ site.apiRoot }}/org/apache/calcite/rel/core/Union.html) of the `n` (default two) most recent relational expressions.
+| `intersect(all [, n])` | Creates an [Intersect]({{ site.apiRoot }}/org/apache/calcite/rel/core/Intersect.html) of the `n` (default two) most recent relational expressions.
+| `minus(all)` | Creates a [Minus]({{ site.apiRoot }}/org/apache/calcite/rel/core/Minus.html) of the two most recent relational expressions.
+
+Argument types:
+
+* `expr`  [RexNode]({{ site.apiRoot }}/org/apache/calcite/rex/RexNode.html)
+* `expr...` Array of [RexNode]({{ site.apiRoot }}/org/apache/calcite/rex/RexNode.html)
+* `exprList` Iterable of [RexNode]({{ site.apiRoot }}/org/apache/calcite/rex/RexNode.html)
+* `fieldOrdinal` Ordinal of a field within its row (starting from 0)
+* `fieldName` Name of a field, unique within its row
+* `fieldName...` Array of String
+* `fieldNames` Iterable of String
+* `rowType` [RelDataType]({{ site.apiRoot }}/org/apache/calcite/rel/type/RelDataType.html)
+* `groupKey` [RelBuilder.GroupKey]({{ site.apiRoot }}/org/apache/calcite/tools/RelBuilder.GroupKey.html)
+* `aggCall...` Array of [RelBuilder.AggCall]({{ site.apiRoot }}/org/apache/calcite/tools/RelBuilder.AggCall.html)
+* `aggCallList` Iterable of [RelBuilder.AggCall]({{ site.apiRoot }}/org/apache/calcite/tools/RelBuilder.AggCall.html)
+* `value...` Array of Object
+* `value` Object
+* `tupleList` Iterable of List of [RexLiteral]({{ site.apiRoot }}/org/apache/calcite/rex/RexLiteral.html)
+* `all` boolean
+* `distinct` boolean
+* `alias` String
+
+The builder methods perform various optimizations, including:
+* `project` returns its input if asked to project all columns in order
+* `filter` flattens the condition (so an `AND` and `OR` may have more than 2 children),
+  simplifies (converting say `x = 1 AND TRUE` to `x = 1`)
+* If you apply `sort` then `limit`, the effect is as if you had called `sortLimit`
+
+### Stack methods
+
+
+| Method              | Description
+|:------------------- |:-----------
+| `build()`           | Pops the most recently created relational expression off the stack
+| `push(rel)`         | Pushes a relational expression onto the stack. Relational methods such as `scan`, above, call this method, but user code generally does not
+| `pushAll(collection)` | Pushes a collection of relational expressions onto the stack
+| `peek()`            | Returns the relational expression most recently put onto the stack, but does not remove it
+
+#### Scalar expression methods
+
+The following methods return a scalar expression
+([RexNode]({{ site.apiRoot }}/org/apache/calcite/rex/RexNode.html)).
+
+Many of them use the contents of the stack. For example, `field("DEPTNO")`
+returns a reference to the "DEPTNO" field of the relational expression just
+added to the stack.
+
+| Method              | Description
+|:------------------- |:-----------
+| `literal(value)` | Constant
+| `field(fieldName)` | Reference, by name, to a field of the top-most relational expression
+| `field(fieldOrdinal)` | Reference, by ordinal, to a field of the top-most relational expression
+| `field(inputCount, inputOrdinal, fieldName)` | Reference, by name, to a field of the (`inputCount` - `inputOrdinal`)th relational expression
+| `field(inputCount, inputOrdinal, fieldOrdinal)` | Reference, by ordinal, to a field of the (`inputCount` - `inputOrdinal`)th relational expression
+| `fields(fieldOrdinalList)` | List of expressions referencing input fields by ordinal
+| `fields(mapping)` | List of expressions referencing input fields by a given mapping
+| `fields(collation)` | List of expressions, `exprList`, such that `sort(exprList)` would replicate collation
+| `call(op, expr...)`<br/>`call(op, exprList)` | Call to a function or operator
+| `and(expr...)`<br/>`and(exprList)` | Logical AND. Flattens nested ANDs, and optimizes cases involving TRUE and FALSE.
+| `or(expr...)`<br/>`or(exprList)` | Logical OR. Flattens nested ORs, and optimizes cases involving TRUE and FALSE.
+| `not(expr)` | Logical NOT
+| `equals(expr, expr)` | Equals
+| `isNull(expr)` | Checks whether an expression is null
+| `isNotNull(expr)` | Checks whether an expression is not null
+| `alias(expr, fieldName)` | Renames an expression (only valid as an argument to `project`)
+| `cast(expr, typeName)`<br/>`cast(expr, typeName, precision)`<br/>`cast(expr, typeName, precision, scale)`<br/> | Converts an expression to a given type
+| `desc(expr)` | Changes sort direction to descending (only valid as an argument to `sort` or `sortLimit`)
+| `nullsFirst(expr)` | Changes sort order to nulls first (only valid as an argument to `sort` or `sortLimit`)
+| `nullsLast(expr)` | Changes sort order to nulls last (only valid as an argument to `sort` or `sortLimit`)
+
+### Group key methods
+
+The following methods return a
+[RelBuilder.GroupKey]({{ site.apiRoot }}/org/apache/calcite/tools/RelBuilder.GroupKey.html).
+
+| Method              | Description
+|:------------------- |:-----------
+| `groupKey(fieldName...)`<br/>`groupKey(fieldOrdinal...)`<br/>`groupKey(expr...)`<br/>`groupKey(exprList)` | Creates a group key of the given expressions
+| `groupKey(exprList, exprListList)` | Creates a group key of the given expressions with grouping sets
+| `groupKey(bitSet, bitSets)` | Creates a group key of the given input columns with grouping sets
+
+### Aggregate call methods
+
+The following methods return an
+[RelBuilder.AggCall]({{ site.apiRoot }}/org/apache/calcite/tools/RelBuilder.AggCall.html).
+
+| Method              | Description
+|:------------------- |:-----------
+| `aggregateCall(op, distinct, filter, alias, expr...)`<br/>`aggregateCall(op, distinct, filter, alias, exprList)` | Creates a call to a given aggregate function, with an optional filter expression
+| `count(distinct, alias, expr...)` | Creates a call to the COUNT aggregate function
+| `countStar(alias)` | Creates a call to the COUNT(*) aggregate function
+| `sum(distinct, alias, expr)` | Creates a call to the SUM aggregate function
+| `min(alias, expr)` | Creates a call to the MIN aggregate function
+| `max(alias, expr)` | Creates a call to the MAX aggregate function

http://git-wip-us.apache.org/repos/asf/calcite/blob/5cee486f/avatica/site/_docs/api.md
----------------------------------------------------------------------
diff --git a/avatica/site/_docs/api.md b/avatica/site/_docs/api.md
new file mode 100644
index 0000000..49f456b
--- /dev/null
+++ b/avatica/site/_docs/api.md
@@ -0,0 +1,28 @@
+---
+title: API
+layout: external
+external_url: http://calcite.apache.org/apidocs
+---
+{% comment %}
+Ideally, we want to use {{ site.apiRoot }} instead of hardcoding
+the above external_url value, but I don't believe there's a way to do that
+{% endcomment %}
+
+<!--
+{% comment %}
+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.
+{% endcomment %}
+-->